diff --git a/.gitignore b/.gitignore index 75c436dab..49d2e3bdf 100644 --- a/.gitignore +++ b/.gitignore @@ -30,3 +30,4 @@ config doc/* /.vscode/ +.DS_store diff --git a/Makefile b/Makefile index 1aeecf0ee..78cad2660 100644 --- a/Makefile +++ b/Makefile @@ -13,7 +13,7 @@ dep_aten = hex 0.6.0 dep_seshat = hex 1.0.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 942be0f8b..5dee12b35 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 000000000..aa5dbe8ea --- /dev/null +++ b/docs/internals/COMPACTION.md @@ -0,0 +1,271 @@ +# 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 +below 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 the write amplification +incurred by writing the snapshot. + +### 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 an empty list of live indexes. + +TODO: how to ensure segments containing overwritten entries only are cleaned +up. + +### 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: + +* 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. + +During compaction the target segment will have the naming format `001-002-003.compacting` +such that each segment (001, 002, 003) name is present in the compacting name. +An upper limit on the maximum number of source segments will have to be set to +ensure the compacting file name doesn't get ridiculously long. E.g. 8. + +Once the compacting segment has been synced the lowest numbered segment will +be hard linked to the compacting segment. Each of the compacted +higher numbered segments (003, 004) will then have a symlink created (e.g. 003.link) +pointing to the lowest numbered segment (002) +then the link is renamed to the source file: `003.link -> 003` (NB not atomic). + +`001-002-003.compacting` is then deleted (but 002 is still hard linked so the data +will remain). + +This naming format means it is easy to identify partially compacted segments +after an unclean exit. All `*.compacting` files with a link count of 1 will +be deleted as it is not clear at what stage the unclean exit occurred. + +If a compacting file has a link count of 2 (or more???) the compacting writes +completed and the lowest numbered segment was hard linked to the compacting +segment. We don't know if all symlinks were created correctly so we need to ensure +this during recovery. + +Once we've ensured there are hard or symlinks for all the source files the compacting +file can be deleted. + +The symlinks are there so that any pending read references to the old +segment name are still valid for some time after but the disk space for the +source segment will still be reclaimed when the links replace the files. + +Some time later the symbolic links can be removed. + +Single segment compaction would work the same as we can directly rename +e.g. the compacted segment `001.compacting` to `001.segment` without breaking +any references to the segment. Single segment compaction should only be triggered +when a certain limit has been reached, e.g. > 50% of indexes can be cleared up. + +TODO: how to handle compaction of segments that have indexes that never were +committed, i.e. overwritten? + + +#### 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. +* Explicitly through a new ra API. + +![segments](compaction2.jpg) + +### Phase 4 compaction (optional) + +At some point the number of live indexes could become completely sparse (no +adjacent indexes) and large which is sub optimal memory wise. + +At this point the state machine could implement a "rewrite" command (or we +provide one in Ra) to rewrite a subset or all of the indexes at the head of +the Ra log to "clump" their indexes better together. + +This is ofc optional and has replication costs but could be a manually triggered +maintenance option perhaps. + +### 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 process (currently transient) first sends all live +entries for the given snapshot, then performs normal chunk based +snapshot replication. + + +#### 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 log order). +* 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 + + +#### How to work out which live indexes the follower needs + +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 + +* 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 000000000..57ae3d2c0 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 000000000..738222cfb Binary files /dev/null and b/docs/internals/compaction2.jpg differ diff --git a/src/ra.erl b/src/ra.erl index 1a00d4bbe..fab51e936 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -33,9 +33,9 @@ leader_query/3, consistent_query/2, consistent_query/3, + consistent_aux/3, ping/2, % cluster operations - start_cluster/1, start_cluster/2, start_cluster/3, start_cluster/4, @@ -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,13 @@ 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 + key_metrics/2, + trigger_compaction/1 ]). -%% 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 +87,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()}. @@ -169,21 +153,20 @@ start(Params) when is_list(Params) -> {ok, [Started]} | {error, term()} when Started :: term(). start_in(DataDir) -> + ra_env:configure_logger(logger), + LogFile = filename:join(DataDir, "ra.log"), + SaslFile = filename:join(DataDir, "ra_sasl.log"), + logger:remove_handler(ra_handler), + ok = logger:set_primary_config(level, debug), + Config = #{config => #{file => LogFile}}, + ok = logger:add_handler(ra_handler, logger_std_h, Config), + application:load(sasl), + application:set_env(sasl, sasl_error_logger, {file, SaslFile}), + application:stop(sasl), + application:start(sasl), + _ = error_logger:tty(false), 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 +210,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 @@ -390,16 +363,6 @@ start_cluster(System, ClusterName, Machine, ServerIds, Timeout) end || Id <- ServerIds], start_cluster(System, Configs, Timeout). -%% @doc Same as `start_cluster/2' but uses the default Ra system. -%% @param ServerConfigs a list of initial server configurations -%% DEPRECATED: use start_cluster/2 -%% @end --spec start_cluster([ra_server:ra_server_config()]) -> - {ok, [ra_server_id()], [ra_server_id()]} | - {error, cluster_not_formed}. -start_cluster(ServerConfigs) -> - start_cluster(default, ServerConfigs). - %% @doc Starts a new distributed ra cluster. %% @param System the system name %% @param ServerConfigs a list of initial server configurations @@ -515,16 +478,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. @@ -748,14 +701,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 @@ -1066,6 +1011,20 @@ consistent_query(ServerId, QueryFun) -> consistent_query(ServerId, QueryFun, Timeout) -> ra_server_proc:query(ServerId, QueryFun, consistent, #{}, Timeout). + +%% @doc Similar to `consistent_query/3' but will process an aux command +%% after consensus has been achieved. +%% @param ServerId the ra server id(s) to send the query to +%% @param QueryFun the query function to run +%% @param Timeout the timeout to use +%% @see consistent_query/2 +%% @end +-spec consistent_aux(ServerId :: ra_server_id() | [ra_server_id()], + AuxCmd :: term(), + Timeout :: timeout()) -> + ra_server_proc:ra_leader_call_ret(Reply :: term()). +consistent_aux(ServerId, AuxCmd, Timeout) -> + ra_server_proc:query(ServerId, AuxCmd, consistent_aux, #{}, Timeout). %% @doc Returns a list of cluster members %% %% Except if `{local, ServerId}' is passed, the query is sent to the specified @@ -1197,18 +1156,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 @@ -1279,6 +1226,13 @@ key_metrics({Name, N} = ServerId, _Timeout) when N == node() -> key_metrics({_, N} = ServerId, Timeout) -> erpc:call(N, ?MODULE, ?FUNCTION_NAME, [ServerId], Timeout). +%% @doc Potentially triggers a major compaction for the provided member +%% @param ServerId the Ra server to send the request to +%% @end +-spec trigger_compaction(ra_server_id()) -> ok. +trigger_compaction(ServerRef) -> + gen_statem:cast(ServerRef, {ra_log_event, major_compaction}). + %% internal -spec usr(UserCommand, ReplyMode) -> Command when diff --git a/src/ra.hrl b/src/ra.hrl index 5d5c6142d..1e1f2a458 100644 --- a/src/ra.hrl +++ b/src/ra.hrl @@ -83,17 +83,26 @@ %% represent a unique entry in the ra log -type log_entry() :: {ra_index(), ra_term(), term()}. --type chunk_flag() :: next | last. +-type chunk_flag() :: init | pre | next | last. --type consistent_query_ref() :: {From :: term(), Query :: ra:query_fun(), ConmmitIndex :: ra_index()}. +-type consistent_query_ref() :: + {query, From :: from(), Query :: ra:query_fun(), CommitIndex :: ra_index()} | + {aux, From :: from(), AuxCmd :: term(), CommitIndex :: ra_index()}. -type safe_call_ret(T) :: timeout | {error, noproc | nodedown | shutdown} | T. --type states() :: leader | follower | candidate | await_condition. +-type states() :: + leader | + follower | + candidate | + pre_vote | + await_condition. %% 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 @@ -167,7 +176,7 @@ {term :: ra_term(), % the leader's term leader_id :: ra_server_id(), meta :: snapshot_meta(), - chunk_state :: {pos_integer(), chunk_flag()} | undefined, + chunk_state :: {non_neg_integer(), chunk_flag()}, data :: term() }). @@ -276,8 +285,17 @@ "Number of checkpoint bytes written"}, {checkpoints_promoted, ?C_RA_LOG_CHECKPOINTS_PROMOTED, counter, "Number of checkpoints promoted to snapshots"}, + {major_compactions, ?C_RA_LOG_COMPACTIONS_MAJOR_COUNT, counter, + "Number of requested major compactions"}, + {major_compaction_segments_written, + ?C_RA_LOG_COMPACTIONS_SEGMENTS_WRITTEN, counter, + "Number of segments written during major compactions"}, + {major_compaction_segments_compacted, + ?C_RA_LOG_COMPACTIONS_SEGMENTS_COMPACTED, counter, + "Number of segments compacted during major compactions"}, {reserved_1, ?C_RA_LOG_RESERVED, counter, "Reserved counter"} ]). + -define(C_RA_LOG_WRITE_OPS, 1). -define(C_RA_LOG_WRITE_RESENDS, 2). -define(C_RA_LOG_READ_OPS, 3). @@ -293,7 +311,10 @@ -define(C_RA_LOG_CHECKPOINTS_WRITTEN, 13). -define(C_RA_LOG_CHECKPOINT_BYTES_WRITTEN, 14). -define(C_RA_LOG_CHECKPOINTS_PROMOTED, 15). --define(C_RA_LOG_RESERVED, 16). +-define(C_RA_LOG_COMPACTIONS_MAJOR_COUNT, 16). +-define(C_RA_LOG_COMPACTIONS_SEGMENTS_WRITTEN, 17). +-define(C_RA_LOG_COMPACTIONS_SEGMENTS_COMPACTED, 18). +-define(C_RA_LOG_RESERVED, 19). -define(C_RA_SRV_AER_RECEIVED_FOLLOWER, ?C_RA_LOG_RESERVED + 1). -define(C_RA_SRV_AER_REPLIES_SUCCESS, ?C_RA_LOG_RESERVED + 2). diff --git a/src/ra_bench.erl b/src/ra_bench.erl index 651b158b3..25099bf3f 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 bd40551a4..6f14a7f30 100644 --- a/src/ra_directory.erl +++ b/src/ra_directory.erl @@ -145,7 +145,7 @@ where_is_parent(#{directory := Dir}, UId) when is_binary(UId) -> -spec name_of(atom() | ra_system:names(), ra_uid()) -> option(atom()). name_of(SystemOrNames, UId) -> - Tbl = get_name(SystemOrNames), + Tbl = get_tbl(SystemOrNames), case ets:lookup(Tbl, UId) of [{_, _, _, ServerName, _}] -> ServerName; [] -> @@ -155,7 +155,7 @@ name_of(SystemOrNames, UId) -> -spec cluster_name_of(ra_system:names() | atom(), ra_uid()) -> option(ra_cluster_name()). cluster_name_of(SystemOrNames, UId) -> - Tbl = get_name(SystemOrNames), + Tbl = get_tbl(SystemOrNames), case ets:lookup(Tbl, UId) of [{_, _, _, _, ClusterName}] when ClusterName /= undefined -> @@ -166,7 +166,7 @@ cluster_name_of(SystemOrNames, UId) -> -spec pid_of(atom() | ra_system:names(), ra_uid()) -> option(pid()). pid_of(SystemOrNames, UId) -> - case ets:lookup(get_name(SystemOrNames), UId) of + case ets:lookup(get_tbl(SystemOrNames), UId) of [{_, Pid, _, _, _}] when is_pid(Pid) -> Pid; _ -> undefined end. @@ -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 => @@ -219,9 +222,9 @@ is_registered_uid(SystemOrNames, UId) is_binary(UId) -> name_of(SystemOrNames, UId) =/= undefined. -get_name(#{directory := Tbl}) -> +get_tbl(#{directory := Tbl}) -> Tbl; -get_name(System) when is_atom(System) -> +get_tbl(System) when is_atom(System) -> {ok, Tbl} = ra_system:lookup_name(System, directory), Tbl. diff --git a/src/ra_flru.erl b/src/ra_flru.erl index 5be0b4c3e..f4ccba271 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_kv.erl b/src/ra_kv.erl new file mode 100644 index 000000000..7c033e514 --- /dev/null +++ b/src/ra_kv.erl @@ -0,0 +1,266 @@ +-module(ra_kv). +-feature(maybe_expr, enable). +-behaviour(ra_machine). + +-include("src/ra.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-export([ + init/1, + apply/3, + % state_enter/2, + % tick/2, + init_aux/1, + handle_aux/5, + live_indexes/1, + overview/1 + ]). + +-export([ + start_cluster/3, + add_member/3, + remove_member/3, + member_overview/1, + + put/4, + get/3, + take_snapshot/1 + ]). + +-export([read_entry/4]). + + +-define(STATE, ?MODULE). +-define(TUPLE(A, B), [A | B]). + +-type key() :: binary(). +-type value() :: term(). + +-record(?STATE, {keys = #{} :: + #{key() => ?TUPLE(non_neg_integer(), Hash :: integer())}}). + + +-record(put, {key :: key(), + value :: term(), + meta :: #{size := non_neg_integer(), + hash := integer()}}). + +-type command() :: #put{}. +-opaque state() :: #?STATE{}. + +-export_type([state/0, + command/0]). + +%% mgmt +-spec start_cluster(System :: atom(), + ClusterName :: atom(), + Config :: #{members := [ra_server_id()]}) -> + {ok, [ra_server_id()], [ra_server_id()]} | + {error, cluster_not_formed}. +start_cluster(System, ClusterName, #{members := ServerIds}) + when is_atom(ClusterName) andalso + is_atom(System) -> + Machine = {module, ?MODULE, #{}}, + Configs = [begin + UId = ra:new_uid(ra_lib:to_binary(ClusterName)), + #{id => Id, + uid => UId, + cluster_name => ClusterName, + log_init_args => #{uid => UId, + min_snapshot_interval => 0}, + initial_members => ServerIds, + machine => Machine} + end || Id <- ServerIds], + ra:start_cluster(System, Configs). + +-spec add_member(System :: atom(), + NewMemberId :: ra_server_id(), + LeaderId :: ra_server_id()) -> + ok | {error, term()} | {timeout, ra_server_id()}. +add_member(System, {Name, _} = Id, LeaderId0) -> + {ok, Members, LeaderId1} = ra:members(LeaderId0), + ?assert(not lists:member(Id, Members)), + UId = ra:new_uid(ra_lib:to_binary(Name)), + Machine = {module, ?MODULE, #{}}, + Config = #{id => Id, + uid => UId, + cluster_name => Name, + log_init_args => #{uid => UId, + min_snapshot_interval => 0}, + initial_members => Members, + machine => Machine}, + %% TODO: rollback handling + maybe + %% first start the server + ok ?= ra:start_server(System, Config), + %% then add the member + {ok, {_, _} = IdxTerm, LeaderId} ?= ra:add_member(LeaderId1, Id), + %% then wait for the cluster change command to become applied + {ok, _, _} ?= ra:local_query(LeaderId, {ra_lib, ignore, []}, + #{timeout => 30_000, + condition => {applied, IdxTerm}}), + ok + end. + +-spec remove_member(System :: atom(), + NewMemberId :: ra_server_id(), + LeaderId :: ra_server_id()) -> + ok | {error, term()} | {timeout, ra_server_id()}. +remove_member(System, Id, LeaderId0) -> + {ok, Members, _} = ra:members(LeaderId0), + RemainingMembers = lists:delete(Id, Members), + ?assert(lists:member(Id, Members)), + maybe + %% first remove the mem + {ok, {_, _} = IdxTerm, LeaderId} ?= ra:remove_member(RemainingMembers, Id), + %% first start the server + %% then wait for the cluster change command to become applied + {ok, _, _} ?= ra:local_query(LeaderId, {ra_lib, ignore, []}, + #{timeout => 30_000, + condition => {applied, IdxTerm}}), + % ok ?= ra:stop_server(System, Id), + ok ?= ra:force_delete_server(System, Id), + ok + end. + +member_overview(ServerId) -> + case ra:member_overview(ServerId) of + {ok, O, _} -> + maps:with([log, machine], O); + Err -> + Err + end. + +%% client +-spec put(ra:server_id(), key(), value(), non_neg_integer()) -> + {ok, map()} | {error, term()} | {timeout, ra:server_id()}. +put(ServerId, Key, Value, Timeout) -> + Hash = erlang:phash2(Value), + Put = #put{key = Key, + value = Value, + meta = #{size => erlang:external_size(Value), + hash => Hash}}, + case ra:process_command(ServerId, Put, Timeout) of + {ok, {ok, Meta}, LeaderId} -> + {ok, Meta#{leader => LeaderId}}; + Err -> + Err + end. + + +%% @doc get performs a consistent query that returns the index, hash and member set +%% then perform an aux query to actually get the data for a given index. +%% if addressing a follower (say there is a local one) then the read may need +%% to wait if the index isn't yet available locally (term also need to be checked) +%% or check that the machien state has the right index for a given key before +%% reading the value from the log +-spec get(ra:server_id(), key(), non_neg_integer()) -> + {ok, map(), value()} | {error, term()} | {timeout, ra:server_id()}. +get(ServerId, Key, Timeout) -> + case ra:consistent_aux(ServerId, {get, Key}, Timeout) of + {ok, {ok, Idx, Members}, LeaderId} -> + %% see if there is a local member in the list of members + %% else query leader + QueryServerId = + case lists:search(fun ({_, N}) -> + N == node() + end, Members) of + {value, {Name, _} = LocalMember} -> + IsAlive = is_pid(whereis(Name)), + case ra_counters:counters(LocalMember, [last_applied]) of + #{last_applied := LastApplied} + when IsAlive andalso + LastApplied >= Idx -> + %% the local member has applied sufficient indexes + LocalMember; + _ -> + %% fall back to leader for any other case + LeaderId + end; + false -> + LeaderId + end, + + read_entry(QueryServerId, Idx, Members, Timeout); + {ok, Err, _} -> + Err; + Err -> + Err + end. + +read_entry({_, Node} = ServerId, Idx, Members, Timeout) + when Node == node() -> + case ra_server_proc:read_entries(ServerId, [Idx], + undefined, Timeout) of + {ok, {#{Idx := {Idx, Term, + {'$usr', Meta, #put{value = Value}, _}}}, + Flru}} -> + _ = ra_flru:evict_all(Flru), + {ok, Meta#{index => Idx, + members => Members, + term => Term}, Value}; + Err -> + Err + end; +read_entry({_, Node} = ServerId, Idxs, Members, Timeout) -> + try erpc:call(Node, ?MODULE, ?FUNCTION_NAME, + [ServerId, Idxs, Members, Timeout]) of + Res -> + Res + catch T:E:_S -> + {error, {T, E}} + end. + +-spec take_snapshot(ra_server_id()) -> ok. +take_snapshot(ServerId) -> + ra:aux_command(ServerId, take_snapshot). + +%% state machine + +init(_) -> + #?MODULE{}. + +%% we use improper lists in this module +-dialyzer({no_improper_lists, [apply/3]}). + +apply(#{index := Idx} = Meta, + #put{key = Key, + meta = #{hash := Hash}}, + #?STATE{keys = Keys} = State0) -> + State = State0#?STATE{keys = Keys#{Key => ?TUPLE(Idx, Hash)}}, + {State, {ok, Meta}, []}. + +live_indexes(#?STATE{keys = Keys}) -> + maps:fold(fun (_K, [Idx | _], Acc) -> + [Idx | Acc] + end, [], Keys). + + +-record(aux, {}). + +init_aux(_) -> + #aux{}. + +handle_aux(_RaState, {call, _From}, take_snapshot, Aux, Internal) -> + MacState = ra_aux:machine_state(Internal), + LastAppliedIdx = ra_aux:last_applied(Internal), + %% TODO: replace release cursor with simpler snapshot effect that is always + %% attempted? + {reply, ok, Aux, Internal, + [{release_cursor, LastAppliedIdx, MacState}]}; +handle_aux(_RaState, {call, _}, {get, Key}, Aux, Internal) -> + #?STATE{keys = Keys} = ra_aux:machine_state(Internal), + Members = maps:keys(ra_aux:members_info(Internal)), + Reply = case Keys of + #{Key := [Idx | _]} -> + {ok, Idx, Members}; + _ -> + {error, not_found} + end, + {reply, Reply, Aux, Internal, []}; +handle_aux(_RaState, _, _, Aux, Internal) -> + {no_reply, Aux, Internal}. + +overview(#?STATE{keys = Keys} = State) -> + #{num_keys => maps:size(Keys), + live_indexes => live_indexes(State)}. diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl new file mode 100644 index 000000000..c0e018b7b --- /dev/null +++ b/src/ra_kv_harness.erl @@ -0,0 +1,943 @@ +-module(ra_kv_harness). + +-export([ + run/1, + run/2, + read_all_keys/0, + teardown_cluster/1, + timestamp/0, + log/2 + ]). + +-include_lib("eunit/include/eunit.hrl"). + +-define(SYS, default). +-define(CLUSTER_NAME, kv). +-define(TIMEOUT, 30000). +-define(MAX_KEY, 10000). % Limited key space for more conflicts +-define(MIN_VALUE_SIZE, 1). +-define(MAX_VALUE_SIZE, 10_000_000). % 10MB +-define(MAX_NODES, 7). % Maximum number of nodes in the cluster + +-type state() :: #{options => map(), + members := #{ra:server_id() => peer:server_ref()}, + reference_map := #{binary() => term()}, + operations_count := non_neg_integer(), + successful_ops := non_neg_integer(), + failed_ops := non_neg_integer(), + next_node_id := pos_integer(), + remaining_ops := non_neg_integer(), + consistency_failed := boolean(), + partition_state => #{partitioned_node => ra:server_id(), + heal_time => non_neg_integer(), + other_nodes => [node()]}}. + +-spec timestamp() -> string(). +timestamp() -> + {MegaSecs, Secs, MicroSecs} = os:timestamp(), + {{Year, Month, Day}, {Hour, Min, Sec}} = calendar:now_to_local_time({MegaSecs, Secs, MicroSecs}), + Millisecs = MicroSecs div 1000, + io_lib:format("[~4..0w-~2..0w-~2..0w ~2..0w:~2..0w:~2..0w.~3..0w]", + [Year, Month, Day, Hour, Min, Sec, Millisecs]). + +milliseconds() -> + erlang:system_time(millisecond). + +-spec log(string(), list()) -> ok. +log(Format, Args) -> + Message = io_lib:format(Format, Args), + io:format("~s", [Message]), + file:write_file("ra_kv_harness.log", Message, [append]). + +-spec new_state() -> state(). +new_state() -> + #{members => #{}, + reference_map => #{}, + operations_count => 0, + successful_ops => 0, + failed_ops => 0, + next_node_id => 1, + remaining_ops => 0, + consistency_failed => false}. + +-type operation() :: {put, Key :: binary(), Value :: term()} | + {get, Key :: binary()} | + {snapshot} | + {major_compaction} | + {update_almost_all_keys} | + {kill_wal} | + {kill_member} | + {add_member} | + {remove_member} | + {network_partition}. + +-spec run(NumOperations :: pos_integer()) -> + {ok, #{successful := non_neg_integer(), + failed := non_neg_integer(), + consistency_checks := non_neg_integer()}} | + {error, term()}. +run(NumOperations) -> + run(NumOperations, #{}). + +read_all_keys() -> + [_ = ra_kv:get({?CLUSTER_NAME, + node()}, + <<"key_",(integer_to_binary(N))/binary>>, 1000) + || N <- lists:seq(1, ?MAX_KEY)], + ok. + +read_all_keys_loop(Members0) when is_list(Members0) -> + receive + stop -> + log("~s Read all keys loop stopped~n", [timestamp()]), + ok + after 0 -> + %% resolve current members + {ok, Members, _} = ra:members(Members0), + Member = hd(Members), + NodeName = element(2, Member), + log("~s Begin reading all keys on member ~0p~n", [timestamp(), Member]), + T1 = erlang:monotonic_time(), + ok = erpc:call(NodeName, ra_kv_harness, read_all_keys, []), + T2 = erlang:monotonic_time(), + Diff = erlang:convert_time_unit(T2 - T1, native, millisecond), + log("~s Read all keys on member ~0p in ~bms~n", [timestamp(), Member, Diff]), + read_all_keys_loop(Members) + end. + +-spec run(NumOperations :: pos_integer(), + Options :: map()) -> + {ok, #{successful := non_neg_integer(), + failed := non_neg_integer(), + consistency_checks := non_neg_integer()}} | + {error, term()}. +run(NumOperations, Options) when NumOperations > 0 -> + % Start with a random number of nodes between 1 and 7 + NumNodes = rand:uniform(7), + logger:set_primary_config(level, warning), + application:set_env(sasl, sasl_error_logger, false), + application:stop(sasl), + log("~s Starting cluster with ~p nodes~n", [timestamp(), NumNodes]), + case setup_cluster(NumNodes, Options) of + {ok, Members, PeerNodes} -> + MembersMap = maps:from_list(lists:zip(Members, PeerNodes)), + InitialState = (new_state())#{members => MembersMap, + next_node_id => NumNodes + 1, + remaining_ops => NumOperations, + options => Options}, + %% keep reading all keys while the other operations are running + ReaderPid = spawn(fun() -> read_all_keys_loop(maps:keys(MembersMap)) end), + try + State = execute_operation(InitialState, {put, <<"never_updated">>, <<"never_updated">>}), + FinalState = run_operations(State, ?CLUSTER_NAME), + ReaderPid ! stop, + case maps:get(consistency_failed, FinalState, false) of + true -> + log("~s EMERGENCY STOP: Leaving cluster running for investigation~n", [timestamp()]), + {error, {consistency_failure, FinalState}}; + false -> + ConsistencyChecks = validate_final_consistency(FinalState), + teardown_cluster(FinalState), + {ok, #{successful => maps:get(successful_ops, FinalState), + failed => maps:get(failed_ops, FinalState), + consistency_checks => ConsistencyChecks}} + end + catch + Class:Reason:Stack -> + ReaderPid ! stop, + teardown_cluster(InitialState), + {error, {Class, Reason, Stack}} + end; + {error, Reason} -> + {error, Reason} + end. + +setup_cluster(NumNodes, Opts) when NumNodes > 0 -> + % Start peer nodes + case start_peer_nodes(NumNodes, Opts) of + {ok, PeerNodes, NodeNames} -> + Members = [{?CLUSTER_NAME, NodeName} || NodeName <- NodeNames], + case ra_kv:start_cluster(?SYS, ?CLUSTER_NAME, #{members => Members}) of + {ok, StartedMembers, _} -> + log("~s Started cluster with ~p members~n", [timestamp(), length(StartedMembers)]), + {ok, StartedMembers, PeerNodes}; + {error, Reason} -> + [peer:stop(PeerRef) || PeerRef <- PeerNodes], + {error, Reason} + end; + {error, Reason} -> + {error, Reason} + end. + +start_peer_nodes(NumNodes, Opts) -> + start_peer_nodes(NumNodes, [], [], Opts). + +start_peer_nodes(0, PeerRefs, NodeNames, _Opts) -> + {ok, lists:reverse(PeerRefs), lists:reverse(NodeNames)}; +start_peer_nodes(N, PeerRefs, NodeNames, Opts) when N > 0 -> + case start_single_peer_node(N, Opts) of + {ok, PeerRef, NodeName} -> + start_peer_nodes(N - 1, [PeerRef | PeerRefs], [NodeName | NodeNames], Opts); + {error, Reason} -> + % Clean up any already started peers + [peer:stop(PeerRef) || PeerRef <- PeerRefs], + {error, Reason} + end. + +start_single_peer_node(NodeId, Opts) -> + NodeName = list_to_atom("ra_test_" ++ integer_to_list(NodeId) ++ "@" ++ + inet_db:gethostname()), + + log("~s Starting node ~p nodes~n", [timestamp(), NodeName]), + % Get all code paths from current node + CodePaths = code:get_path(), + PaArgs = lists:flatmap(fun(Path) -> ["-pa", Path] end, CodePaths), + BufferSize = ["+zdbbl", "102400"], + + % Check if inet_tcp_proxy_dist is available + % If yes, add -proto_dist argument at the front with its path, like erlang_node_helpers does + ProtoDistArgs = case code:where_is_file("inet_tcp_proxy_dist.beam") of + non_existing -> + % inet_tcp_proxy not available, don't use it + log("~s WARNING: inet_tcp_proxy_dist not found, starting peer without it~n", [timestamp()]), + []; + BeamPath -> + DistModPath = filename:dirname(BeamPath), + ["-pa", DistModPath, "-proto_dist", "inet_tcp_proxy", + "-kernel", "prevent_overlapping_partitions", "false"] + end, + + case peer:start_link(#{name => NodeName, + args => ProtoDistArgs ++ PaArgs ++ BufferSize, + connection => standard_io}) of + {ok, PeerRef, NodeName} -> + BaseDir = maps:get(dir, Opts, ""), + erpc:call(NodeName, logger, set_primary_config, [level, warning]), + erpc:call(NodeName, application, set_env, [sasl, sasl_error_logger, false]), + erpc:call(NodeName, application, stop, [sasl]), + Dir = filename:join(BaseDir, NodeName), + {ok, _} = erpc:call(NodeName, ra, start_in, [Dir]), + % Set logger level to reduce verbosity on peer node + % Start ra application on the new peer node + {ok, PeerRef, NodeName}; + {error, Reason} -> + {error, Reason} + end. + +start_new_peer_node(NodeId, Opts) -> + start_single_peer_node(NodeId, Opts). + +-spec teardown_cluster(state()) -> ok. +teardown_cluster(#{members := Members}) -> + % Stop Ra servers on each node and stop peer nodes + maps:foreach(fun(Member, PeerRef) -> + NodeName = element(2, Member), + log("~s Stopping member ~p~n", [timestamp(), Member]), + catch erpc:call(NodeName, ra, stop_server, [?SYS, Member]), + catch peer:stop(PeerRef), + ok + end, Members), + ok. + +run_operations(State, _ClusterName) -> + RemainingOps = maps:get(remaining_ops, State), + case RemainingOps =< 0 of + true -> + State; + false -> + case RemainingOps rem 1000 of + 0 -> log("~s ~p operations remaining~n", [timestamp(), RemainingOps]); + _ -> ok + end, + Operation = generate_operation(), + NewState = execute_operation(State, Operation), + + % Update remaining operations count + UpdatedState = NewState#{remaining_ops => RemainingOps - 1}, + + % Validate consistency every 100 operations + ValidationState = case maps:get(operations_count, UpdatedState) rem 100 of + 0 -> validate_consistency(UpdatedState); + _ -> UpdatedState + end, + + run_operations(ValidationState, _ClusterName) + end. + +-spec generate_operation() -> operation(). +generate_operation() -> + case rand:uniform(100) of + 1 -> % 1% update almost all keys + {update_almost_all_keys}; + 2 -> % 1% add member + {add_member}; + 3 -> % 1% remove member + {remove_member}; + 4 -> % 1% kill WAL + {kill_wal}; + 5 -> % 1% kill member + {kill_member}; + 6 -> % 1% network partition + {network_partition}; + N when N =< 10 -> % 4% snapshot + {snapshot}; + N when N =< 12 -> % 2% major compactions + {major_compaction}; + N when N =< 80 -> + Key = generate_key(), + Value = generate_value(), + {put, Key, Value}; + _ -> + Key = generate_key(), + {get, Key} + end. + +key(N) when is_integer(N) -> + <<"key_", (integer_to_binary(N))/binary>>. +generate_key() -> + KeyNum = rand:uniform(?MAX_KEY), % Limited key space for more conflicts + key(KeyNum). + +-spec generate_value() -> binary(). +generate_value() -> + Size = rand:uniform(?MAX_VALUE_SIZE - ?MIN_VALUE_SIZE) + ?MIN_VALUE_SIZE, + rand:bytes(Size). + +-spec execute_operation(state(), operation()) -> state(). +execute_operation(State, {put, Key, Value}) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + + % Pick a random cluster member to send the operation to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + + case ra_kv:put(Member, Key, Value, ?TIMEOUT) of + {ok, _Meta} -> + NewRefMap = RefMap#{Key => Value}, + State#{reference_map => NewRefMap, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {error, _Reason} -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + {timeout, _ServerId} -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1} + end; + +execute_operation(State, {get, Key}) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + + % Pick a random cluster member to send the operation to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + RefValue = maps:get(Key, RefMap, not_found), + + log("~s ra_kv:get/3 from node ~w ~n", [timestamp(), NodeName]), + case erpc:call(NodeName, ra_kv, get, [Member, Key, ?TIMEOUT]) of + % case apply(ra_kv, get, [Member, Key, ?TIMEOUT]) of + {error, not_found} when RefValue =:= not_found -> + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {error, not_found} when RefValue =/= not_found -> + log("~s CONSISTENCY ERROR: Key ~p should exist but not found~n", [timestamp(), Key]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + {ok, _Meta, Value} when RefValue =:= Value -> + log("~s ra_kv:get/3 from node ~w ok! ~n", [timestamp(), NodeName]), + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {ok, _Meta, Value} when RefValue =/= Value -> + log("~s CONSISTENCY ERROR: Key ~p, Expected ~p, Got ~p~n", + [timestamp(), Key, RefValue, Value]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1} + % _ -> + % State#{operations_count => OpCount + 1, + % failed_ops => FailedOps + 1} + end; + +execute_operation(State, {update_almost_all_keys}) -> + RefMap = maps:get(reference_map, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + + Member = random_non_partitioned_member(State), + + % Update a random percentage (10-90%) of keys + PercentToUpdate = rand:uniform(81) + 9, % Random number between 10 and 90 + AllKeys = lists:seq(1, ?MAX_KEY), + NumKeysToUpdate = (?MAX_KEY * PercentToUpdate) div 100, + KeysToUpdate = lists:sublist([N || {_, N} <- lists:sort([{rand:uniform(), K} || K <- AllKeys])], + NumKeysToUpdate), + + {T, Results} = timer:tc(fun() -> + [begin + case ra_kv:put(Member, key(N), OpCount, ?TIMEOUT) of + {ok, _} -> {ok, N}; + {timeout, _} -> {timeout, N}; + {error, _Reason} -> {error, N} + end + end || N <- KeysToUpdate] + end), + + % Count successful and failed updates + SuccessfulKeys = [N || {ok, N} <- Results], + FailedResults = [R || R <- Results, element(1, R) =/= ok], + + case FailedResults of + [] -> + log("~s Updated ~p% of the ~p keys (~p successful) in ~bms~n", + [timestamp(), PercentToUpdate, ?MAX_KEY, length(SuccessfulKeys), T div 1000]); + _ -> + log("~s Updated ~p/~p keys (~p% target) in ~bms (aborted due to ~p failures/timeouts)~n", + [timestamp(), length(SuccessfulKeys), length(KeysToUpdate), PercentToUpdate, T div 1000, length(FailedResults)]) + end, + + % Only update reference map for successfully updated keys + NewRefMap = maps:merge(RefMap, maps:from_list([{key(N), OpCount} || N <- SuccessfulKeys])), + State#{reference_map => NewRefMap, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + +execute_operation(State, {snapshot}) -> + Members = maps:get(members, State), + + % Pick a random cluster member to send snapshot command to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + + case erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]) of + undefined -> + State; + _Pid -> + log("~s Rollover/snapshot on node ~p...~n", [timestamp(), NodeName]), + erpc:call(NodeName, ra_log_wal, force_roll_over, [ra_log_wal]), + erpc:call(NodeName, ra, aux_command, [Member, take_snapshot]), + State + end; + +execute_operation(State, {major_compaction}) -> + Members = maps:get(members, State), + + % Pick a random cluster member to send snapshot command to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + + case erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]) of + undefined -> + State; + _Pid -> + log("~s Triggering major compaction on node ~p...~n", [timestamp(), NodeName]), + erpc:call(NodeName, ra, trigger_compaction, [Member]), + State + end; + +execute_operation(#{options := Opts} = State, {add_member}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + NextNodeId = maps:get(next_node_id, State), + + % Don't add members if we already have 7 (maximum 7 nodes) + case maps:size(Members) >= ?MAX_NODES of + true -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + false -> + + log("~s Adding member on node ~p.~n", + [timestamp(), NextNodeId]), + case start_new_peer_node(NextNodeId, Opts) of + {ok, PeerRef, NodeName} -> + NewMember = {?CLUSTER_NAME, NodeName}, + ExistingMember = random_non_partitioned_member(State), + + case ra_kv:add_member(?SYS, NewMember, ExistingMember) of + ok -> + NewMembers = Members#{NewMember => PeerRef}, + NewMembersList = maps:keys(NewMembers), + log("~s Added member ~p. Cluster now has ~p members: ~0p; partitioned node: ~0p~n", + [timestamp(), NewMember, + length(NewMembersList), + NewMembersList, + partitioned_node_name(State)]), + State#{members => NewMembers, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1, + next_node_id => NextNodeId + 1}; + {timeout, _ServerId} -> + log("~s Timeout adding member ~p~n", + [timestamp(), NewMember]), + % Clean up the peer node since add failed + catch peer:stop(PeerRef), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1, + next_node_id => NextNodeId + 1}; + {error, Reason} -> + log("~s Failed to add member ~p: ~p~n", + [timestamp(), NewMember, Reason]), + % Clean up the peer node since add failed + catch peer:stop(PeerRef), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1, + next_node_id => NextNodeId + 1} + end; + {error, Reason} -> + log("~s Failed to start peer node: ~p~n", [timestamp(), Reason]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1, + next_node_id => NextNodeId + 1} + end + end; + +execute_operation(State, {remove_member}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + + % Don't remove members if we only have one left (minimum 1 node) + case maps:size(Members) =< 1 of + true -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + false -> + % Pick a random member to remove + MembersList = maps:keys(Members), + MemberToRemove = random_non_partitioned_member(State), + + % Pick a different member to send the remove command to + RemainingMembers = MembersList -- [MemberToRemove], + CommandTarget = random_non_partitioned_member(RemainingMembers, State), + + log("~s Removing member ~w... command target ~w~n", + [timestamp(), MemberToRemove, CommandTarget]), + + case ra_kv:remove_member(?SYS, MemberToRemove, CommandTarget) of + ok -> + % Stop the peer node for the removed member + case maps:get(MemberToRemove, Members, undefined) of + undefined -> + ok; + PeerRef -> + catch peer:stop(PeerRef) + end, + + NewMembers = maps:remove(MemberToRemove, Members), + NewMembersList = maps:keys(NewMembers), + log("~s Member ~w removed. Cluster now has ~p members: ~0p; paritioned node: ~p~n", + [timestamp(), + MemberToRemove, + length(NewMembersList), + NewMembersList, + partitioned_node_name(State)]), + + State#{members => NewMembers, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {timeout, _ServerId} -> + log("~s Timeout removing member ~p~n", + [timestamp(), MemberToRemove]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + {error, Reason} -> + log("~s Failed to remove member ~p: ~p~n", + [timestamp(), MemberToRemove, Reason]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1} + end + end; + +execute_operation(State, {kill_wal}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + + % Pick a node to kill WAL on + MembersList = maps:keys(Members), + Rnd = rand:uniform(?MAX_NODES), + case Rnd > length(MembersList) of + true -> + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + false -> + Member = lists:nth(Rnd, MembersList), + NodeName = element(2, Member), + + log("~s Killing WAL on member ~w...~n", [timestamp(), NodeName]), + + case erpc:call(NodeName, erlang, whereis, [ra_log_wal]) of + Pid when is_pid(Pid) -> + erpc:call(NodeName, erlang, exit, [Pid, kill]), + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + _ -> + State + end + + end; + +execute_operation(State, {kill_member}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + Kills = maps:get(kills, State, #{}), + + % Pick a random member to kill + MembersList = maps:keys(Members), + Rnd = rand:uniform(?MAX_NODES), + case Rnd > length(MembersList) of + true -> + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + false -> + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + + Now = milliseconds(), + LastKill = maps:get(Member, Kills, Now - 10000), + if(Now > LastKill + 5500) -> + case erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]) of + Pid when is_pid(Pid) -> + log("~s Killing member ~w...~n", [timestamp(), Member]), + erpc:call(NodeName, erlang, exit, [Pid, kill]), + %% give it a bit of time after a kill in case this member is chosen + %% for the next operation + timer:sleep(100), + State#{operations_count => OpCount + 1, + kills => Kills#{Member => Now}, + successful_ops => SuccessOps + 1}; + _ -> + State + end; + true -> + log("~s Not killing member ~w...~n", [timestamp(), Member]), + State + end + + end; + +execute_operation(State, {network_partition}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + PartitionState = maps:get(partition_state, State, #{}), + Now = milliseconds(), + + case PartitionState of + #{heal_time := HealTime} when Now < HealTime -> + % There's an active partition already + State; + #{partitioned_node := PartitionedNode, heal_time := HealTime, other_nodes := OtherNodes} when Now >= HealTime -> + % Time to heal the partition + PartitionedNodeName = element(2, PartitionedNode), + + log("~s Healing network partition; node ~w will rejoin the rest~n", [timestamp(), PartitionedNode]), + + % Allow communication between partitioned node and other nodes + % Call allow on each node to restore bidirectional communication + % We use rpc:call with infinity timeout since nodes might be reconnecting + [begin + spawn(fun() -> + case rpc:call(PartitionedNodeName, inet_tcp_proxy_dist, allow, [OtherNode], 5000) of + ok -> ok; + Err -> + log("~s rpc:call(~p, inet_tcp_proxy_dist, allow, [~p]) failed with ~p~n", + [timestamp(), + PartitionedNodeName, + OtherNode, + Err]), + ok + end + end), + spawn(fun() -> + case rpc:call(OtherNode, inet_tcp_proxy_dist, allow, [PartitionedNodeName], 5000) of + ok -> ok; + Err -> + log("~s rpc:call(~p, inet_tcp_proxy_dist, allow, [~p]) failed with ~p~n", + [timestamp(), + OtherNode, + PartitionedNodeName, + Err]), + ok + end + end) + end || OtherNode <- OtherNodes], + + % Give nodes time to reconnect + timer:sleep(5000), + + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1, + partition_state => #{}}; + _ -> + MembersList = maps:keys(Members), + case length(MembersList) < 3 of + true -> + % Need at least 3 nodes for partition + State; + false -> + % No active partition, create a new one + % Pick a random node to partition + NodeToPartition = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeToPartitionName = element(2, NodeToPartition), + + % Get the other nodes (excluding the one to partition) + OtherNodes = [element(2, M) || M <- MembersList, M =/= NodeToPartition], + + % Choose partition duration: 3s, 15s, 55s, or 90s + PartitionDurations = [3000, 15000, 55000, 90000], + Duration = lists:nth(rand:uniform(length(PartitionDurations)), PartitionDurations), + HealTime = Now + Duration, + + log("~s Creating network partition: isolating node ~w from ~w for ~wms~n", + [timestamp(), NodeToPartitionName, OtherNodes, Duration]), + + % Block communication between partitioned node and other nodes + % The harness node (current node) maintains access to all nodes + [begin + erpc:call(NodeToPartitionName, inet_tcp_proxy_dist, block, [OtherNode]), + erpc:call(OtherNode, inet_tcp_proxy_dist, block, [NodeToPartitionName]) + end || OtherNode <- OtherNodes], + + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1, + partition_state => #{partitioned_node => NodeToPartition, + heal_time => HealTime, + other_nodes => OtherNodes}} + end + end. + +-spec wait_for_applied_index_convergence([ra:server_id()], non_neg_integer()) -> ok. +wait_for_applied_index_convergence(Members, MaxRetries) when MaxRetries > 0 -> + IndicesMap = get_applied_indices(Members), + Indices = maps:values(IndicesMap), + case lists:uniq(Indices) of + [_SingleIndex] -> + ok; % All nodes have converged + _MultipleIndices -> + timer:sleep(100), % Wait 100ms before retry + wait_for_applied_index_convergence(Members, MaxRetries - 1) + end; +wait_for_applied_index_convergence(Members, 0) -> + IndicesMap = get_applied_indices(Members), + log("~s WARNING: Applied index convergence timeout. Reported values: ~0p~n", + [timestamp(), IndicesMap]), + ok. + +-spec get_applied_indices([ra:server_id()]) -> #{ra:server_id() => ra:index() | undefined}. +get_applied_indices(Members) -> + maps:from_list([{Member, case ra:member_overview(Member, 1000) of + {ok, #{last_applied := Index}, _} -> + Index; + _ -> + undefined + end} || Member <- Members]). + +-spec validate_consistency(state()) -> state(). +validate_consistency(State) -> + Members = maps:get(members, State), + PartitionState = maps:get(partition_state, State, #{}), + MembersList = maps:keys(Members), + + % Determine which members to validate based on active partition + MembersToValidate = case PartitionState of + #{partitioned_node := PartitionedNode} -> + % Active partition - exclude the partitioned node + log("~s Consistency check during partition; skipping validation of partitioned node ~0p~n", + [timestamp(), partitioned_node_name(State)]), + lists:delete(PartitionedNode, MembersList); + _ -> + % No active partition, validate all members + MembersList + end, + + % Perform consistency check on selected members + perform_consistency_check(State, MembersToValidate). + +-spec perform_consistency_check(state(), [ra:server_id()]) -> state(). +perform_consistency_check(State, MembersToValidate) -> + RefMap = maps:get(reference_map, State), + + % Wait for all nodes to converge to the same applied index + wait_for_applied_index_convergence(MembersToValidate, 300), % Wait up to 30 seconds + + % Check that all members have the same view + ValidationResults = [validate_member_consistency(Member, RefMap) + || Member <- MembersToValidate], + + Result1 = hd(ValidationResults), + case lists:all(fun(Result) -> + is_map(Result) andalso + is_map(Result1) andalso + lists:sort(maps:get(live_indexes, Result)) =:= + lists:sort(maps:get(live_indexes, Result1)) + end, ValidationResults) of + true -> + State; + false -> + % Brief console output with live_indexes summary + LiveIndexesSummary = [{Member, case Result of + #{live_indexes := LI, + log := #{last_index := LastIndex}} -> + {length(LI), LastIndex}; + _ -> error + end} || {Member, Result} <- + lists:zip(MembersToValidate, ValidationResults)], + log("~s Consistency check failed. Live indexes per node: ~p~n", + [timestamp(), LiveIndexesSummary]), + log("~s STOPPING: No more operations will be performed due to consistency failure~n", [timestamp()]), + + % Write full details to log file with difference analysis + LogEntry = format_consistency_failure(MembersToValidate, ValidationResults), + file:write_file("ra_kv_harness.log", LogEntry, [append]), + + FailedOps = maps:get(failed_ops, State), + State#{failed_ops => FailedOps + 1, remaining_ops => 0, consistency_failed => true} + end. + +-spec format_consistency_failure([ra:server_id()], [map() | error]) -> iolist(). +format_consistency_failure(Members, Results) -> + MemberResults = lists:zip(Members, Results), + + % Extract all unique results for comparison + UniqueResults = lists:usort([R || {_, R} <- MemberResults, R =/= error]), + + Header = io_lib:format("~s Consistency check failed:~n", [timestamp()]), + + % Log raw data + RawData = [io_lib:format(" Member ~p: ~p~n", [Member, Result]) || {Member, Result} <- MemberResults], + + % Analyze differences + DiffAnalysis = case UniqueResults of + [] -> + [" ANALYSIS: All members returned errors\n"]; + [_SingleResult] -> + [" ANALYSIS: All successful members have identical results (errors may exist)\n"]; + MultipleResults -> + [" ANALYSIS: Found ~p different result patterns:\n" | + [io_lib:format(" Pattern ~p: ~p\n", [I, Pattern]) || + {I, Pattern} <- lists:zip(lists:seq(1, length(MultipleResults)), MultipleResults)] ++ + [" DIFFERENCES:\n"] ++ + analyze_field_differences(MultipleResults)] + end, + + [Header, RawData, DiffAnalysis, "\n"]. + +-spec analyze_field_differences([map()]) -> iolist(). +analyze_field_differences(Results) -> + % Extract live_indexes and num_keys for comparison + LiveIndexes = [maps:get(live_indexes, R, undefined) || R <- Results, is_map(R)], + NumKeys = [maps:get(num_keys, R, undefined) || R <- Results, is_map(R)], + + LiveIndexDiff = case lists:usort(LiveIndexes) of + [_] -> []; + MultipleLI -> [io_lib:format(" live_indexes differ: ~p\n", [MultipleLI])] + end, + + NumKeysDiff = case lists:usort(NumKeys) of + [_] -> []; + MultipleNK -> [io_lib:format(" num_keys differ: ~p\n", [MultipleNK])] + end, + + [LiveIndexDiff, NumKeysDiff]. + +-spec validate_member_consistency(ra:server_id(), map()) -> map() | error. +validate_member_consistency(Member, _RefMap) -> + case ra_kv:member_overview(Member) of + #{log := Log, + machine := #{live_indexes := Live, num_keys := Num}} -> + %io:format("Member ~p overview: Live indexes ~p, Num keys ~p", [Member, Live, Num]), + #{log => Log, + live_indexes => Live, + num_keys => Num}; + Error -> + log("~s Member ~p failed overview check: ~p~n", + [timestamp(), Member, Error]), + error + end. + +-spec validate_final_consistency(state()) -> non_neg_integer(). +validate_final_consistency(State) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + + log("~s Performing final consistency validation...~n", [timestamp()]), + log("~s Reference map has ~p keys~n", [timestamp(), maps:size(RefMap)]), + + % Wait for all nodes to converge before final validation + MembersList = maps:keys(Members), + log("~s Waiting for applied index convergence...~n", [timestamp()]), + wait_for_applied_index_convergence(MembersList, 100), % Wait up to 10 seconds for final check + + % Validate all keys across all members + Keys = maps:keys(RefMap), + + MembersList = maps:keys(Members), + ValidationCount = lists:foldl( + fun(Key, Acc) -> + RefValue = maps:get(Key, RefMap), + case validate_key_across_members(Key, RefValue, MembersList) of + ok -> Acc + 1; + error -> Acc + end + end, 0, Keys), + + log("~s Final consistency check: ~p/~p keys validated successfully~n", + [timestamp(), ValidationCount, length(Keys)]), + ValidationCount. + +-spec validate_key_across_members(binary(), term(), [ra:server_id()]) -> ok | error. +validate_key_across_members(Key, ExpectedValue, Members) -> + Results = [begin + case erpc:call(Node, ra_kv, get, [Member, Key, ?TIMEOUT]) of + {ok, _Meta, Value} when Value =:= ExpectedValue -> ok; + {ok, _Meta, Value} -> + log("~s Key ~p mismatch on ~p: expected ~p, got ~p~n", + [timestamp(), Key, Member, ExpectedValue, Value]), + error; + {error, not_found} -> + log("~s Key ~p not found on ~p but should exist~n", [timestamp(), Key, Member]), + error; + Other -> + log("~s Key ~p query failed on ~p: ~p~n", [timestamp(), Key, Member, Other]), + error + end + end || {_, Node} = Member <- Members], + + case lists:all(fun(R) -> R =:= ok end, Results) of + true -> ok; + false -> error + end. + +partitioned_node_name(#{partition_state := #{partitioned_node := {_, Node}}}) -> + Node; +partitioned_node_name(_State) -> + none. + +random_non_partitioned_member(MemberList, State) -> + PartitionState = maps:get(partition_state, State, #{}), + PartitionedNode = maps:get(partitioned_node, PartitionState, none), + MajorityMembers = lists:delete(PartitionedNode, MemberList), + lists:nth(rand:uniform(length(MajorityMembers)), MajorityMembers). + +random_non_partitioned_member(State) -> + Members = maps:get(members, State, #{}), + PartitionState = maps:get(partition_state, State, #{}), + PartitionedNode = maps:get(partitioned_node, PartitionState, none), + MajorityMembers = lists:delete(PartitionedNode, maps:keys(Members)), + lists:nth(rand:uniform(length(MajorityMembers)), MajorityMembers). + diff --git a/src/ra_lib.erl b/src/ra_lib.erl index a8be1361a..e0e629a55 100644 --- a/src/ra_lib.erl +++ b/src/ra_lib.erl @@ -49,6 +49,7 @@ lists_shuffle/1, is_dir/1, is_file/1, + is_any_file/1, ensure_dir/1, consult/1, cons/2 @@ -344,9 +345,9 @@ retry(Func, Attempt, Sleep) -> ok; true -> ok; - _ -> + _Err -> timer:sleep(Sleep), - retry(Func, Attempt - 1) + retry(Func, Attempt - 1, Sleep) end. -spec write_file(file:name_all(), iodata()) -> @@ -466,6 +467,14 @@ is_file(File) -> false end. +is_any_file(File) -> + case prim_file:read_file_info(File) of + {ok, #file_info{}} -> + true; + _ -> + false + end. + -spec consult(file:filename()) -> {ok, term()} | {error, term()}. @@ -478,19 +487,11 @@ consult(Path) -> Err end. +-spec cons(term(), list()) -> list(). cons(Item, List) when is_list(List) -> [Item | List]. -tokens(Str) -> - case erl_scan:string(Str) of - {ok, Tokens, _EndLoc} -> - erl_parse:parse_term(Tokens); - {error, Err, _ErrLoc} -> - {error, Err} - end. - - %% raw copy of ensure_dir ensure_dir("/") -> ok; @@ -517,6 +518,15 @@ ensure_dir(F) -> end end. +tokens(Str) -> + case erl_scan:string(Str) of + {ok, Tokens, _EndLoc} -> + erl_parse:parse_term(Tokens); + {error, Err, _ErrLoc} -> + {error, Err} + end. + + -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/ra_log.erl b/src/ra_log.erl index c527fe974..9b3298154 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -7,6 +7,7 @@ %% @hidden -module(ra_log). +-include_lib("stdlib/include/assert.hrl"). -compile([inline_list_funcs]). -export([pre_init/1, @@ -16,13 +17,16 @@ commit_tx/1, append/2, write/2, + write_sparse/3, append_sync/2, write_sync/2, fold/5, + fold/6, sparse_read/2, partial_read/3, execute_read_plan/4, read_plan_info/1, + previous_wal_index/1, last_index_term/1, set_last_index/2, handle_event/2, @@ -32,7 +36,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,10 +53,8 @@ delete_everything/1, release_resources/3, - % external reader - register_reader/2, - readers/1, - tick/2 + tick/2, + assert/1 ]). -include("ra.hrl"). @@ -64,11 +66,15 @@ -define(WAL_RESEND_TIMEOUT, 5000). -type ra_meta_key() :: atom(). --type segment_ref() :: {ra_range:range(), File :: file:filename_all()}. --type event_body() :: {written, ra_term(), ra:range()} | +-type segment_ref() :: {File :: binary(), ra_range: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()} | + {compaction_result, term()} | + major_compaction | {down, pid(), term()}. -type event() :: {ra_log_event, event_body()}. @@ -100,22 +106,23 @@ -record(?MODULE, {cfg = #cfg{}, %% mutable data below - first_index = -1 :: ra_index(), - last_index = -1 :: -1 | ra_index(), + range :: ra:range(), last_term = 0 :: ra_term(), last_written_index_term = {0, 0} :: ra_idxterm(), snapshot_state :: ra_snapshot:state(), + current_snapshot :: option(ra_idxterm()), last_resend_time :: option({integer(), WalPid :: pid() | undefined}), - last_wal_write :: {pid(), Ms :: integer()}, - reader :: ra_log_reader:state(), - readers = [] :: [pid()], + last_wal_write :: {pid(), Ms :: integer(), ra:index() | -1}, + reader :: ra_log_segments:state(), mem_table :: ra_mt:state(), - tx = false :: boolean() + tx = false :: false | {true, ra:range()}, + pending = [] :: ra_seq:state(), + live_indexes = [] :: ra_seq:state() }). -record(read_plan, {dir :: file:filename_all(), read :: #{ra_index() := log_entry()}, - plan :: ra_log_reader:read_plan()}). + plan :: ra_log_segments:read_plan()}). -opaque read_plan() :: #read_plan{}. -opaque state() :: #?MODULE{}. @@ -139,9 +146,9 @@ -type overview() :: #{type := ra_log, - last_index := ra_index(), + range := ra:range(), + last_index := ra:index(), last_term := ra_term(), - first_index := ra_index(), last_written_index_term := ra_idxterm(), num_segments := non_neg_integer(), open_segments => non_neg_integer(), @@ -183,7 +190,7 @@ init(#{uid := UId, segment_writer := SegWriter} = Names} } = Conf) -> Dir = server_data_dir(DataDir, UId), - MaxOpen = maps:get(max_open_segments, Conf, 5), + MaxOpen = maps:get(max_open_segments, Conf, 1), SnapModule = maps:get(snapshot_module, Conf, ?DEFAULT_SNAPSHOT_MODULE), %% this has to be patched by ra_server LogId = maps:get(log_id, Conf, UId), @@ -206,9 +213,15 @@ init(#{uid := UId, SnapshotState = ra_snapshot:init(UId, SnapModule, SnapshotsDir, CheckpointsDir, Counter, MaxCheckpoints), {SnapIdx, SnapTerm} = case ra_snapshot:current(SnapshotState) of - undefined -> {-1, -1}; + undefined -> {-1, 0}; 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), @@ -216,26 +229,56 @@ init(#{uid := UId, % this queries the segment writer and thus blocks until any % segments it is currently processed have been finished MtRange = ra_mt:range(Mt0), - {{FirstIdx, LastIdx0}, SegRefs} = case recover_ranges(UId, MtRange, SegWriter) of - {undefined, SRs} -> - {{-1, -1}, SRs}; - R -> R - end, - %% TODO: don't think this is necessary given the range is calculated from this - %% but can't hurt as it may trigger some cleanup - {DeleteSpecs, Mt} = ra_mt:set_first(FirstIdx, Mt0), - - ok = exec_mem_table_delete(Names, UId, DeleteSpecs), - Reader = ra_log_reader:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, - Names, Counter), + ok = ra_log_segments:purge_dangling_symlinks(Dir), + SegRefs = my_segrefs(UId, SegWriter), + SegmentMaxCount = maps:get(segment_max_entries, Conf, ?SEGMENT_MAX_ENTRIES), + SegmentMaxSize = maps:get(segment_max_size_bytes, Conf, ?SEGMENT_MAX_SIZE_B), + CompConf = #{max_size => SegmentMaxSize, + max_count => SegmentMaxCount}, + Reader = ra_log_segments:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, + Counter, CompConf, LogId), + SegmentRange = ra_log_segments:range(Reader), + %% The ranges can be sparse at this point so ra_range:add/2 does + %% not do the right thing here as it requires a contiguous range + Range = ra_range:combine(MtRange, SegmentRange), + + [begin + ?DEBUG("~ts: deleting overwritten segment ~w", + [LogId, SR]), + catch prim_file:delete(filename:join(Dir, F)) + end + || {F, _} = SR <- SegRefs -- ra_log_segments:segment_refs(Reader)], + %% assert there is no gap between the snapshot %% and the first index in the log - case (FirstIdx - SnapIdx) > 1 of - true -> - exit({corrupt_log, gap_between_snapshot_and_first_index, - {SnapIdx, FirstIdx}}); - false -> ok - end, + Mt = case Range of + undefined -> + Mt0; + {FstIdx, LstIdx} -> + case FstIdx == SnapIdx + 1 orelse + ra_range:in(SnapIdx, Range) orelse + SnapIdx > LstIdx of + true -> + {DeleteSpecs, Mt1} = ra_mt:set_first(FstIdx, Mt0), + ok = exec_mem_table_delete(Names, UId, DeleteSpecs), + Mt1; + false -> + exit({corrupt_log, + gap_between_snapshot_and_log_range, + {SnapIdx, Range}}) + end + end, + LastWalIdx = case ra_log_wal:last_writer_seq(Wal, UId) of + {ok, undefined} -> + -1; + {ok, Idx} -> + Idx; + {error, wal_down} -> + ?ERROR("~ts: ra_log:init/1 cannot complete as wal" + " process is down.", + [LogId]), + exit(wal_down) + end, Cfg = #cfg{directory = Dir, uid = UId, log_id = LogId, @@ -248,18 +291,25 @@ init(#{uid := UId, counter = Counter, names = Names}, State0 = #?MODULE{cfg = Cfg, - first_index = max(SnapIdx + 1, FirstIdx), - last_index = max(SnapIdx, LastIdx0), + range = ra_range:truncate(SnapIdx, Range), reader = Reader, mem_table = Mt, snapshot_state = SnapshotState, - last_wal_write = {whereis(Wal), now_ms()} + current_snapshot = ra_snapshot:current(SnapshotState), + last_wal_write = {whereis(Wal), now_ms(), LastWalIdx}, + live_indexes = LiveIndexes }, put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx), - LastIdx = State0#?MODULE.last_index, + LastIdx = case Range of + undefined -> + SnapIdx; + {_, Lst} -> + Lst + end, 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_segments:segment_ref_count(Reader)), case ra_snapshot:latest_checkpoint(SnapshotState) of undefined -> ok; @@ -268,31 +318,19 @@ init(#{uid := UId, end, % recover the last term - {LastTerm0, State2} = case LastIdx of - SnapIdx -> - {SnapTerm, State0}; - -1 -> - {0, State0}; - LI -> - fetch_term(LI, State0) - end, - LastSegRefIdx = case SegRefs of - [] -> + {LastTerm0, State2} = case Range of + undefined -> + {SnapTerm, State0}; + {_, LI} -> + fetch_term(LI, State0) + end, + LastSegRefIdx = case SegmentRange of + undefined -> -1; - [{{_, L}, _} | _] -> + {_, L} -> L end, - LastWrittenIdx = case ra_log_wal:last_writer_seq(Wal, UId) of - {ok, undefined} -> - %% take last segref index - max(SnapIdx, LastSegRefIdx); - {ok, Idx} -> - max(Idx, LastSegRefIdx); - {error, wal_down} -> - ?ERROR("~ts: ra_log:init/1 cannot complete as wal process is down.", - [State2#?MODULE.cfg#cfg.log_id]), - exit(wal_down) - end, + LastWrittenIdx = lists:max([LastWalIdx, SnapIdx, LastSegRefIdx]), {LastWrittenTerm, State3} = case LastWrittenIdx of SnapIdx -> {SnapTerm, State2}; @@ -310,74 +348,88 @@ init(#{uid := UId, State = maybe_append_first_entry(State4), ?DEBUG("~ts: ra_log:init recovered last_index_term ~w" " snapshot_index_term ~w, last_written_index_term ~w", - [State#?MODULE.cfg#cfg.log_id, - last_index_term(State), - {SnapIdx, SnapTerm}, + [LogId, last_index_term(State), {SnapIdx, SnapTerm}, State#?MODULE.last_written_index_term ]), - element(1, delete_segments(SnapIdx, State)). + assert(State). -spec close(state()) -> ok. close(#?MODULE{cfg = #cfg{uid = _UId}, reader = Reader}) -> % deliberately ignoring return value % close all open segments - _ = ra_log_reader:close(Reader), + _ = ra_log_segments:close(Reader), ok. -spec begin_tx(state()) -> state(). begin_tx(State) -> - State#?MODULE{tx = true}. + State#?MODULE{tx = {true, undefined}}. -spec commit_tx(state()) -> {ok, state()} | {error, wal_down, state()}. commit_tx(#?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, - tx = true, + tx = {true, TxRange}, + range = Range, mem_table = Mt1} = State) -> {Entries, Mt} = ra_mt:commit(Mt1), Tid = ra_mt:tid(Mt), WriterId = {UId, self()}, - {WalCommands, Num} = - lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N}) -> + PrevIdx = previous_wal_index(State), + {WalCommands, Num, _} = + lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N, Prev}) -> Cmd = {ttb, term_to_iovec(Cmd0)}, - WalC = {append, WriterId, Tid, Idx, Term, Cmd}, - {[WalC | WC], N+1} - end, {[], 0}, Entries), + WalC = {append, WriterId, Tid, Prev, Idx, Term, Cmd}, + {[WalC | WC], N+1, Idx} + end, {[], 0, PrevIdx}, Entries), + {_, LastIdx} = TxRange, case ra_log_wal:write_batch(Wal, lists:reverse(WalCommands)) of {ok, Pid} -> ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, Num), {ok, State#?MODULE{tx = false, - last_wal_write = {Pid, now_ms()}, + range = ra_range:add(TxRange, Range), + last_wal_write = {Pid, now_ms(), LastIdx}, mem_table = Mt}}; {error, wal_down} -> - %% still need to return the state here + %% TODO: review this - still need to return the state here {error, wal_down, State#?MODULE{tx = false, mem_table = Mt}} end; commit_tx(#?MODULE{tx = false} = State) -> State. +-define(IS_NEXT_IDX(Idx, Range), + Range == undefined orelse + element(2, Range) + 1 =:= Idx). + +-define(IS_IN_RANGE(Idx, Range), + Range =/= undefined andalso + Idx >= element(1, Range) andalso + Idx =< element(2, Range)). + -spec append(Entry :: log_entry(), State :: state()) -> state() | no_return(). append({Idx, Term, Cmd0} = Entry, #?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, - last_index = LastIdx, + range = Range, tx = false, + pending = Pend0, mem_table = Mt0} = State) - when Idx =:= LastIdx + 1 -> + when ?IS_NEXT_IDX(Idx, Range) -> case ra_mt:insert(Entry, Mt0) of {ok, Mt} -> Cmd = {ttb, term_to_iovec(Cmd0)}, 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, + State#?MODULE{range = ra_range:extend(Idx, Range), last_term = Term, - last_wal_write = {Pid, now_ms()}, + last_wal_write = {Pid, now_ms(), Idx}, + pending = ra_seq:append(Idx, Pend), mem_table = Mt}; {error, wal_down} -> error(wal_down) @@ -393,15 +445,16 @@ append({Idx, Term, Cmd0} = Entry, end; append({Idx, Term, _Cmd} = Entry, #?MODULE{cfg = Cfg, - last_index = LastIdx, - tx = true, + tx = {true, TxRange}, + pending = Pend0, mem_table = Mt0} = State) - when Idx =:= LastIdx + 1 -> + when ?IS_NEXT_IDX(Idx, TxRange) -> case ra_mt:stage(Entry, Mt0) of {ok, Mt} -> put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), - State#?MODULE{last_index = Idx, + State#?MODULE{tx = {true, ra_range:extend(Idx, TxRange)}, 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 " @@ -412,9 +465,10 @@ append({Idx, Term, _Cmd} = Entry, Cfg#cfg.uid, Mt0), append(Entry, State#?MODULE{mem_table = M0}) end; -append({Idx, _, _}, #?MODULE{last_index = LastIdx}) -> - Msg = lists:flatten(io_lib:format("tried writing ~b - expected ~b", - [Idx, LastIdx+1])), +append({Idx, _, _}, #?MODULE{range = Range, + tx = Tx}) -> + Msg = lists:flatten(io_lib:format("tried writing ~b - current range ~w tx ~p", + [Idx, Range, Tx])), exit({integrity_error, Msg}). -spec write(Entries :: [log_entry()], State :: state()) -> @@ -422,58 +476,110 @@ append({Idx, _, _}, #?MODULE{last_index = LastIdx}) -> {error, {integrity_error, term()} | wal_down}. write([{FstIdx, _, _} | _Rest] = Entries, #?MODULE{cfg = Cfg, - last_index = LastIdx, + range = Range, + pending = Pend0, mem_table = Mt0} = State0) - when FstIdx =< LastIdx + 1 andalso - FstIdx >= 0 -> + when Range == undefined orelse + (FstIdx =< element(2, Range) + 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; write([], State) -> {ok, State}; write([{Idx, _, _} | _], #?MODULE{cfg = #cfg{uid = UId}, - last_index = LastIdx}) -> + range = Range}) -> Msg = lists:flatten(io_lib:format("~s: ra_log:write/2 " - "tried writing ~b - expected ~b", - [UId, Idx, LastIdx+1])), + "tried writing ~b - current range ~w", + [UId, Idx, Range])), {error, {integrity_error, Msg}}. +-spec write_sparse(log_entry(), option(ra:index()), state()) -> + {ok, state()} | {error, wal_down | gap_detected}. +write_sparse({Idx, Term, _} = Entry, PrevIdx0, + #?MODULE{cfg = #cfg{uid = UId, + wal = Wal} = Cfg, + range = Range, + mem_table = Mt0} = State0) + when PrevIdx0 == undefined orelse + Range == undefined orelse + (PrevIdx0 == element(2, Range)) -> + {ok, Mt} = ra_mt:insert_sparse(Entry, PrevIdx0, Mt0), + ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), + Tid = ra_mt:tid(Mt), + PrevIdx = previous_wal_index(State0), + 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), + NewRange = case Range of + undefined -> + ra_range:new(Idx); + {S, _} -> + ra_range:new(S, Idx) + end, + {ok, State0#?MODULE{range = NewRange, + last_term = Term, + mem_table = Mt, + last_wal_write = {Pid, now_ms(), Idx}}}; + {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(). +fold(From0, To0, Fun, Acc0, State) -> + fold(From0, To0, Fun, Acc0, State, error). + +-spec fold(FromIdx :: ra_index(), ToIdx :: ra_index(), + fun((log_entry(), Acc) -> Acc), Acc, state(), + MissingKeyStrategy :: error | return) -> + {Acc, state()} when Acc :: term(). fold(From0, To0, Fun, Acc0, #?MODULE{cfg = Cfg, mem_table = Mt, - first_index = FirstIdx, - last_index = LastIdx, - reader = Reader0} = State) + range = {StartIdx, EndIdx}, + reader = Reader0} = State, MissingKeyStrat) when To0 >= From0 andalso - To0 >= FirstIdx -> - From = max(From0, FirstIdx), - To = min(To0, LastIdx), - ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), + To0 >= StartIdx -> + %% TODO: move to ra_range function + From = max(From0, StartIdx), + To = min(To0, EndIdx), + + ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), 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_segments:fold(RemStart, RemEnd, Fun, + Acc0, Reader0, + MissingKeyStrat), {Acc, State#?MODULE{reader = Reader}}; {{MtStart, MtEnd}, {RemStart, RemEnd}} -> - {Reader, Acc1} = ra_log_reader:fold(RemStart, RemEnd, Fun, Acc0, Reader0), - Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc1, Mt), + {Reader, Acc1} = ra_log_segments:fold(RemStart, RemEnd, Fun, + Acc0, Reader0, + MissingKeyStrat), + Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc1, Mt, MissingKeyStrat), NumRead = MtEnd - MtStart + 1, ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, NumRead), {Acc, State#?MODULE{reader = Reader}}; {{MtStart, MtEnd}, undefined} -> - Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc0, Mt), + Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc0, Mt, MissingKeyStrat), + %% TODO: if fold is short circuited with MissingKeyStrat == return + %% this count isn't correct, it doesn't massively matter so leaving + %% for now NumRead = MtEnd - MtStart + 1, ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, NumRead), {Acc, State} end; -fold(_From, _To, _Fun, Acc, State) -> +fold(_From, _To, _Fun, Acc, State, _) -> {Acc, State}. %% @doc Reads a list of indexes. @@ -483,7 +589,8 @@ fold(_From, _To, _Fun, Acc, State) -> {[log_entry()], state()}. sparse_read(Indexes0, #?MODULE{cfg = Cfg, reader = Reader0, - last_index = LastIdx, + range = Range, + 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,10 +606,14 @@ 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) -> + ra_range:in(I, Range) orelse + ra_seq:in(I, LiveIndexes) + 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), + {Entries1, Reader} = ra_log_segments:sparse_read(Reader0, Indexes, Entries0), %% here we recover the original order of indexes Entries = case Sort of descending -> @@ -523,12 +634,14 @@ 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, - last_index = LastIdx, + range = Range, + snapshot_state = SnapState, mem_table = Mt}, TransformFun) -> ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), @@ -543,6 +656,17 @@ partial_read(Indexes0, #?MODULE{cfg = Cfg, % descending or undefined Indexes0 end, + LastIdx = case Range of + undefined -> + case ra_snapshot:current(SnapState) of + undefined -> + -1; + {SnapIdx, _} -> + SnapIdx + end; + {_, End} -> + End + end, %% drop any indexes that are larger than the last index available Indexes2 = lists:dropwhile(fun (I) -> I > LastIdx end, Indexes1), @@ -552,7 +676,7 @@ partial_read(Indexes0, #?MODULE{cfg = Cfg, maps:put(I, TransformFun(I, T, Cmd), Acc) end, #{}, Entries0), - Plan = ra_log_reader:read_plan(Reader0, Indexes), + Plan = ra_log_segments:read_plan(Reader0, Indexes), #read_plan{dir = Cfg#cfg.directory, read = Read, plan = Plan}. @@ -560,13 +684,13 @@ partial_read(Indexes0, #?MODULE{cfg = Cfg, -spec execute_read_plan(read_plan(), undefined | ra_flru:state(), TransformFun :: transform_fun(), - ra_log_reader:read_plan_options()) -> + ra_log_segments:read_plan_options()) -> {#{ra_index() => Command :: term()}, ra_flru:state()}. execute_read_plan(#read_plan{dir = Dir, read = Read, plan = Plan}, Flru0, TransformFun, Options) -> - ra_log_reader:exec_read_plan(Dir, Plan, Flru0, TransformFun, + ra_log_segments:exec_read_plan(Dir, Plan, Flru0, TransformFun, Options, Read). -spec read_plan_info(read_plan()) -> map(). @@ -581,9 +705,21 @@ read_plan_info(#read_plan{read = Read, num_segments => NumSegments}. --spec last_index_term(state()) -> ra_idxterm(). -last_index_term(#?MODULE{last_index = LastIdx, last_term = LastTerm}) -> - {LastIdx, LastTerm}. +-spec previous_wal_index(state()) -> ra_idxterm() | -1. +previous_wal_index(#?MODULE{range = Range}) -> + case Range of + undefined -> + -1; + {_, LastIdx} -> + LastIdx + end. + +-spec last_index_term(state()) -> option(ra_idxterm()). +last_index_term(#?MODULE{range = {_, LastIdx}, + last_term = LastTerm}) -> + {LastIdx, LastTerm}; +last_index_term(#?MODULE{current_snapshot = CurSnap}) -> + CurSnap. -spec last_written(state()) -> ra_idxterm(). last_written(#?MODULE{last_written_index_term = LWTI}) -> @@ -593,136 +729,185 @@ last_written(#?MODULE{last_written_index_term = LWTI}) -> -spec set_last_index(ra_index(), state()) -> {ok, state()} | {not_found, state()}. set_last_index(Idx, #?MODULE{cfg = Cfg, + range = Range, + snapshot_state = SnapState, + mem_table = Mt0, last_written_index_term = {LWIdx0, _}} = State0) -> + Cur = ra_snapshot:current(SnapState), + %% TODO: can a log recover to the right reset point? I doubt it case fetch_term(Idx, State0) of - {undefined, State} -> + {undefined, State} when element(1, Cur) =/= Idx -> + %% not found and Idx isn't equal to latest snapshot index {not_found, State}; + {_, State} when element(1, Cur) =:= Idx -> + {_, SnapTerm} = Cur, + %% Idx is equal to the current snapshot + {ok, Mt} = ra_log_ets:new_mem_table_please(Cfg#cfg.names, + Cfg#cfg.uid, Mt0), + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, Idx), + {ok, State#?MODULE{range = ra_range:limit(Idx + 1, Range), + last_term = SnapTerm, + mem_table = Mt, + last_written_index_term = Cur}}; {Term, State1} -> LWIdx = min(Idx, LWIdx0), - {LWTerm, State2} = fetch_term(LWIdx, State1), + {LWTerm, State2} = case Cur of + {LWIdx, SnapTerm} -> + {SnapTerm, State1}; + _ -> + fetch_term(LWIdx, State1) + end, + %% this should always be found but still assert just in case %% _if_ this ends up as a genuine reversal next time we try %% to write to the mem table it will detect this and open %% a new one true = LWTerm =/= undefined, + {ok, Mt} = ra_log_ets:new_mem_table_please(Cfg#cfg.names, + Cfg#cfg.uid, Mt0), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, LWIdx), - {ok, State2#?MODULE{last_index = Idx, + {ok, State2#?MODULE{range = ra_range:limit(Idx + 1, Range), last_term = Term, + mem_table = Mt, last_written_index_term = {LWIdx, LWTerm}}} end. -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 + snapshot_state = SnapState, + pending = Pend0} = State0) -> + CurSnap = ra_snapshot:current(SnapState), + %% 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 =< element(1, CurSnap) -> % 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 + % + % 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} when OtherTerm =/= Term -> + %% 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({segments, TidRanges, NewSegs}, - #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, + #?MODULE{cfg = #cfg{uid = UId, + log_id = LogId, + directory = Dir, + names = Names} = Cfg, reader = Reader0, - mem_table = Mt0, - readers = Readers - } = State0) -> - Reader = ra_log_reader:update_segments(NewSegs, Reader0), + pending = Pend0, + mem_table = Mt0} = State0) -> + {Reader, OverwrittenSegRefs} = ra_log_segments:update_segments(NewSegs, Reader0), + put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, - ra_log_reader:segment_ref_count(Reader)), + ra_log_segments: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), + MtRange = ra_mt:range(Mt), + Pend = case MtRange of + {Start, _End} when Start > FstPend -> + ra_seq:floor(Start, Pend0); + _ -> + Pend0 + end, + ?DEBUG("~ts: ~b segments tidranges received ", [LogId, length(TidRanges)]), State = State0#?MODULE{reader = Reader, + pending = Pend, mem_table = Mt}, - case Readers of - [] -> - {State, []}; + Fun = fun () -> + [begin + ?DEBUG("~ts: deleting overwritten segment ~w", + [LogId, SR]), + catch prim_file:delete(filename:join(Dir, F)) + end + || {F, _} = SR <- OverwrittenSegRefs], + ok + end, + {State, [{bg_work, Fun, fun (_Err) -> ok end}]}; +handle_event({compaction_result, Result}, + #?MODULE{cfg = #cfg{log_id = LogId}, + reader = Reader0} = State) -> + ?DEBUG("~ts: compaction result ~p", [LogId, Result]), + {Reader, Effs} = ra_log_segments:handle_compaction_result(Result, Reader0), + {State#?MODULE{reader = Reader}, Effs}; +handle_event(major_compaction, #?MODULE{cfg = #cfg{log_id = LogId}, + reader = Reader0, + live_indexes = LiveIndexes, + snapshot_state = SS} = State) -> + case ra_snapshot:current(SS) of + {SnapIdx, _} -> + ?DEBUG("~ts: ra_log: major_compaction requested at snapshot index ~b, " + "~b live indexes", + [LogId, SnapIdx, ra_seq:length(LiveIndexes)]), + {Reader, Effs} = ra_log_segments:schedule_compaction(major, SnapIdx, + LiveIndexes, Reader0), + {State#?MODULE{reader = Reader}, Effs}; _ -> - %% 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)} + {State, []} end; -handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind}, +handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, + log_id = LogId, names = Names} = Cfg, - first_index = FstIdx, - last_index = LstIdx, + range = {FstIdx, _} = Range, 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), + % ?assert(ra_snapshot:pending(SnapState0) =/= undefined), + SnapState1 = ra_snapshot:complete_snapshot(Snap, SnapKind, LiveIndexes, + SnapState0), + ?DEBUG("~ts: ra_log: ~s written at index ~b with ~b live indexes", + [LogId, SnapKind, SnapIdx, ra_seq:length(LiveIndexes)]), 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,26 +920,47 @@ 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, + + %% remove all pending below smallest live index as the wal + %% may not write them + %% TODO: test that a written event 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), - {State#?MODULE{first_index = SnapIdx + 1, - last_index = max(LstIdx, SnapIdx), - last_written_index_term = LWIdxTerm, - mem_table = Mt1, - snapshot_state = SnapState}, Effects}; + State = State0#?MODULE{range = ra_range:truncate(SnapIdx, Range), + last_written_index_term = LWIdxTerm, + mem_table = Mt1, + pending = Pend, + live_indexes = LiveIndexes, + current_snapshot = Snap, + snapshot_state = SnapState}, + {Reader, CompEffs} = ra_log_segments:schedule_compaction(minor, SnapIdx, + LiveIndexes, + State#?MODULE.reader), + Effects = CompEffs ++ Effects0, + {State#?MODULE{reader = Reader}, Effects}; checkpoint -> put_counter(Cfg, ?C_RA_SVR_METRIC_CHECKPOINT_INDEX, SnapIdx), %% If we already have the maximum allowed number of checkpoints, @@ -765,7 +971,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,45 +983,56 @@ 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}) -> - LastIdx + 1. +next_index(#?MODULE{tx = {true, {_, Last}}}) -> + Last + 1; +next_index(#?MODULE{range = {_, LastIdx}}) -> + LastIdx + 1; +next_index(#?MODULE{current_snapshot = {SnapIdx, _}}) -> + SnapIdx + 1; +next_index(#?MODULE{current_snapshot = undefined}) -> + 0. -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()) -> {option(ra_term()), state()}. -fetch_term(Idx, #?MODULE{last_index = LastIdx, - first_index = FirstIdx} = State0) - when Idx < FirstIdx orelse Idx > LastIdx -> - {undefined, State0}; -fetch_term(Idx, #?MODULE{mem_table = Mt, reader = Reader0} = State0) -> +fetch_term(Idx, #?MODULE{mem_table = Mt, + range = Range, + reader = Reader0} = State0) + when ?IS_IN_RANGE(Idx, Range) -> case ra_mt:lookup_term(Idx, Mt) of undefined -> - {Term, Reader} = ra_log_reader:fetch_term(Idx, Reader0), + {Term, Reader} = ra_log_segments:fetch_term(Idx, Reader0), {Term, State0#?MODULE{reader = Reader}}; Term when is_integer(Term) -> {Term, State0} - end. + end; +fetch_term(_Idx, #?MODULE{} = State0) -> + {undefined, State0}. -spec snapshot_state(State :: state()) -> ra_snapshot:state(). snapshot_state(State) -> @@ -825,32 +1042,51 @@ 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(), module(), ra_seq:state(), state()) -> + {ok, state(), effects()}. +install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, LiveIndexes, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, - mem_table = Mt0 - } = State0) -> + snapshot_state = SnapState0, + 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), + {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), + SmallestLiveIndex = case ra_seq:first(LiveIndexes) of + undefined -> + SnapIdx + 1; + I -> + I + end, + %% TODO: more mt entries could potentially be cleared up in the + %% mem table here + {Spec, Mt1} = ra_mt:set_first(SmallestLiveIndex, 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}. + %% always create a new mem table here as we could have written + %% sparese entries in the snapshot install + %% TODO: check an empty mt doesn't leak + {ok, Mt} = ra_log_ets:new_mem_table_please(Cfg#cfg.names, + Cfg#cfg.uid, Mt1), + State = State0#?MODULE{snapshot_state = SnapState, + current_snapshot = IdxTerm, + range = undefined, + last_term = SnapTerm, + live_indexes = LiveIndexes, + mem_table = Mt, + last_written_index_term = IdxTerm}, + {Reader, CompEffs} = ra_log_segments:schedule_compaction(minor, SnapIdx, + LiveIndexes, + State#?MODULE.reader), + {ok, State#?MODULE{reader = Reader}, CompEffs ++ CPEffects}. + -spec recover_snapshot(State :: state()) -> option({ra_snapshot:meta(), term()}). @@ -866,25 +1102,31 @@ 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(), + MacCtx :: {MacVer :: ra_machine:version(), 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, {MacVersion, MacModule} = MacCtx, + MacState, State) + when is_atom(MacModule) andalso + is_integer(MacVersion) -> + suggest_snapshot(snapshot, Idx, Cluster, MacCtx, MacState, State). -spec checkpoint(Idx :: ra_index(), Cluster :: ra_cluster(), - MacVersion :: ra_machine:version(), + MacCtx :: {MacVer :: ra_machine:version(), module()}, MacState :: term(), State :: state()) -> {state(), effects()}. -checkpoint(Idx, Cluster, MacVersion, MacState, State) -> - suggest_snapshot(checkpoint, Idx, Cluster, MacVersion, MacState, State). +checkpoint(Idx, Cluster, {MacVersion, MacModule} = MacCtx, MacState, State) + when is_atom(MacModule) andalso + is_integer(MacVersion) -> + suggest_snapshot(checkpoint, Idx, Cluster, MacCtx, MacState, State). -suggest_snapshot(SnapKind, Idx, Cluster, MacVersion, MacState, +suggest_snapshot(SnapKind, Idx, Cluster, MacCtx, 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, MacCtx, MacState, State); _ -> %% Only one snapshot or checkpoint may be written at a time to %% prevent excessive I/O usage. @@ -894,7 +1136,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, []}; @@ -914,7 +1156,7 @@ promote_checkpoint(Idx, #?MODULE{cfg = Cfg, tick(Now, #?MODULE{cfg = #cfg{wal = Wal}, mem_table = Mt, last_written_index_term = {LastWrittenIdx, _}, - last_wal_write = {WalPid, Ms}} = State) -> + last_wal_write = {WalPid, Ms, _}} = State) -> CurWalPid = whereis(Wal), MtRange = ra_mt:range(Mt), case Now > Ms + ?WAL_RESEND_TIMEOUT andalso @@ -930,13 +1172,27 @@ 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}, +assert(#?MODULE{cfg = #cfg{log_id = LogId}, + range = Range, + snapshot_state = SnapState, + current_snapshot = CurrSnap, + live_indexes = LiveIndexes + } = State) -> + %% TODO: remove this at some point? + ?DEBUG("~ts: ra_log: asserting Range ~p Snapshot ~p", + [LogId, Range, CurrSnap]), + %% perform assertions to ensure log state is correct + ?assert(CurrSnap =:= ra_snapshot:current(SnapState)), + ?assert(Range == undefined orelse + CurrSnap == undefined orelse + element(1, Range) - 1 == element(1, CurrSnap)), + ?assert(CurrSnap == undefined orelse + LiveIndexes == [] orelse + ra_seq:last(LiveIndexes) =< element(1, CurrSnap)), + State. + +suggest_snapshot0(SnapKind, Idx, Cluster, {MachineVersion, MacModule}, + MacState, State0) -> case should_snapshot(SnapKind, Idx, State0) of true -> % TODO: here we use the current cluster configuration in @@ -954,7 +1210,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), + Meta = #{index => Idx, + term => Term, + cluster => ClusterServerIds, + machine_version => MachineVersion}, + write_snapshot(Meta, MacModule, MacState, SnapKind, State) end; false -> @@ -974,7 +1238,7 @@ should_snapshot(snapshot, Idx, % We should take a snapshot if the new snapshot index would allow us % to discard any segments or if the we've handled enough commands % since the last snapshot. - CanFreeSegments = case ra_log_reader:range(Reader) of + CanFreeSegments = case ra_log_segments:range(Reader) of undefined -> false; {Start, _End} -> @@ -1025,32 +1289,33 @@ exists({Idx, Term}, Log0) -> end. -spec overview(state()) -> overview(). -overview(#?MODULE{last_index = LastIndex, +overview(#?MODULE{range = Range, last_term = LastTerm, - first_index = FirstIndex, last_written_index_term = LWIT, snapshot_state = SnapshotState, + current_snapshot = CurrSnap, reader = Reader, - last_wal_write = {_LastPid, LastMs}, - mem_table = Mt - }) -> - CurrSnap = ra_snapshot:current(SnapshotState), + last_wal_write = {_LastPid, LastMs, LastWalIdx}, + mem_table = Mt, + pending = Pend + } = State) -> + {LastIndex, _} = last_index_term(State), #{type => ?MODULE, + range => Range, last_index => LastIndex, last_term => LastTerm, - first_index => FirstIndex, last_written_index_term => LWIT, - num_segments => ra_log_reader:segment_ref_count(Reader), - segments_range => ra_log_reader:range(Reader), - open_segments => ra_log_reader:num_open_segments(Reader), + num_segments => ra_log_segments:segment_ref_count(Reader), + segments_range => ra_log_segments:range(Reader), + open_segments => ra_log_segments:num_open_segments(Reader), snapshot_index => case CurrSnap of undefined -> undefined; {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 +1323,9 @@ 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, + last_wal_index => LastWalIdx, + num_pending => ra_seq:length(Pend) }. -spec write_config(ra_server:config(), state()) -> ok. @@ -1089,24 +1356,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 @@ -1118,85 +1373,27 @@ delete_everything(#?MODULE{cfg = #cfg{uid = UId, -spec release_resources(non_neg_integer(), sequential | random, state()) -> state(). -release_resources(MaxOpenSegments, - AccessPattern, - #?MODULE{cfg = #cfg{uid = UId, - directory = Dir, - counter = Counter, - names = Names}, +release_resources(MaxOpenSegments, AccessPattern, + #?MODULE{cfg = #cfg{}, reader = Reader} = State) -> - ActiveSegs = ra_log_reader:segment_refs(Reader), - % close all open segments - % deliberately ignoring return value - _ = ra_log_reader:close(Reader), - %% open a new segment with the new max open segment value - State#?MODULE{reader = ra_log_reader:init(UId, Dir, 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. - + State#?MODULE{reader = ra_log_segments:update_conf(MaxOpenSegments, + AccessPattern, Reader)}. %%% 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]. - - -%% 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. -%% unly used by resend to wal functionality and doesn't update the mem table +%% only used by resend to wal functionality and doesn't update the mem table wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, - wal = Wal} = Cfg} = State, + wal = Wal} = Cfg, + last_wal_write = {_, _, _}} = State, Tid, {Idx, Term, Cmd}) -> case ra_log_wal:write(Wal, {UId, self()}, Tid, Idx, Term, Cmd) of {ok, Pid} -> 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, + State#?MODULE{%last_index = Idx, last_term = Term, - last_wal_write = {Pid, now_ms()} + last_wal_write = {Pid, now_ms(), Idx} }; {error, wal_down} -> error(wal_down) @@ -1204,28 +1401,38 @@ wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, + pending = Pend0, + range = Range, mem_table = Mt0} = State, - Entries) -> + [{FstIdx, _, _} | _] = Entries) -> WriterId = {UId, self()}, + PrevIdx = previous_wal_index(State), %% 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, LastIdx, Pend} = + lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N, Prev, 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, Prev, Idx, Term, Cmd}, + {[WalC | WC], N+1, Idx, ra_seq:append(Idx, P)} + end, {[], 0, PrevIdx, 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), + NewRange = case Range of + undefined -> + ra_range:new(FstIdx, LastIdx); + {Start, _} -> + ra_range:new(Start, LastIdx) + end, case ra_log_wal:write_batch(Wal, lists:reverse(WalCommands)) of {ok, Pid} -> - {ok, State#?MODULE{last_index = LastIdx, + {ok, State#?MODULE{range = NewRange, last_term = LastTerm, - last_wal_write = {Pid, now_ms()}, - mem_table = Mt}}; + last_wal_write = {Pid, now_ms(), LastIdx}, + 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 @@ -1233,16 +1440,17 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, Err end. -maybe_append_first_entry(State0 = #?MODULE{last_index = -1}) -> - State = append({0, 0, undefined}, State0), +maybe_append_first_entry(#?MODULE{range = undefined, + current_snapshot = 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{range = ra_range:new(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,8 +1464,25 @@ 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, + range = {_, LastIdx}, last_resend_time = undefined, mem_table = Mt} = State) -> ?DEBUG("~ts: ra_log: resending from ~b to ~b", @@ -1306,7 +1531,7 @@ stage_entries0(Cfg, [Entry | Rem], Mt0) -> stage_entries0(Cfg, Rem, Mt); {error, overwriting} -> Range = ra_mt:range(Mt0), - Msg = io_lib:format("ra_log:verify_entries/2 " + Msg = io_lib:format("ra_log:stage_entries/3 " "tried writing ~p - mem table range ~w", [Rem, Range]), {error, {integrity_error, lists:flatten(Msg)}} @@ -1314,7 +1539,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,66 +1547,41 @@ 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) -> - % 1. check mem_tables (this assumes wal has finished recovering - % which means it is essential that ra_servers are part of the same - % supervision tree - % 2. check segments +my_segrefs(UId, SegWriter) -> SegFiles = ra_log_segment_writer:my_segments(SegWriter, UId), - SegRefs = lists:foldl( - fun (File, Acc) -> + lists:foldl(fun (File, Acc) -> %% if a server recovered when a segment had been opened %% but never had any entries written the segref would be %% undefined - case ra_log_segment:segref(File) of - undefined -> - Acc; - SegRef -> - [SegRef | Acc] + case ra_log_segment:info(File) of + #{ref := SegRef, + file_type := regular} + when is_tuple(SegRef) -> + [SegRef | Acc]; + _ -> + Acc end - end, [], SegFiles), - SegRanges = [Range || {Range, _} <- SegRefs], - Ranges = [MtRange | SegRanges], - {pick_range(Ranges, undefined), SegRefs}. - -% picks the current range from a sorted (newest to oldest) list of ranges -pick_range([], Res) -> - Res; -pick_range([H | Tail], undefined) -> - pick_range(Tail, H); -pick_range([{Fst, _Lst} | Tail], {CurFst, CurLst}) -> - pick_range(Tail, {min(Fst, CurFst), CurLst}). - + end, [], SegFiles). %% 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. - incr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> counters:add(Cnt, Ix, N); incr_counter(#cfg{counter = undefined}, _Ix, _N) -> @@ -1420,14 +1620,14 @@ exec_mem_table_delete(#{} = Names, UId, Specs) -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). -pick_range_test() -> - Ranges1 = [{76, 90}, {50, 75}, {1, 100}], - {1, 90} = pick_range(Ranges1, undefined), +% pick_range_test() -> +% Ranges1 = [{76, 90}, {50, 75}, {1, 100}], +% {1, 90} = pick_range(Ranges1, undefined), - Ranges2 = [{76, 110}, {50, 75}, {1, 49}], - {1, 110} = pick_range(Ranges2, undefined), +% Ranges2 = [{76, 110}, {50, 75}, {1, 49}], +% {1, 110} = pick_range(Ranges2, undefined), - Ranges3 = [{25, 30}, {25, 35}, {1, 50}], - {1, 30} = pick_range(Ranges3, undefined), - ok. +% Ranges3 = [{25, 30}, {25, 35}, {1, 50}], +% {1, 30} = pick_range(Ranges3, undefined), +% ok. -endif. diff --git a/src/ra_log_pre_init.erl b/src/ra_log_pre_init.erl index 868f44acf..aad58242a 100644 --- a/src/ra_log_pre_init.erl +++ b/src/ra_log_pre_init.erl @@ -75,7 +75,7 @@ code_change(_OldVsn, State, _Extra) -> pre_init(System, UId) -> case ets:lookup(?ETSTBL, UId) of - [{_, _}] -> + [{_, _, _, _}] -> %% already initialised ok; [] -> diff --git a/src/ra_log_read_plan.erl b/src/ra_log_read_plan.erl index 1be245617..0e2e5282a 100644 --- a/src/ra_log_read_plan.erl +++ b/src/ra_log_read_plan.erl @@ -18,7 +18,7 @@ execute(Plan, Flru) -> file_advise => normal}). -spec execute(ra_log:read_plan(), undefined | ra_flru:state(), - ra_log_reader:read_plan_options()) -> + ra_log_segments:read_plan_options()) -> {#{ra:index() => Command :: term()}, ra_flru:state()}. execute(Plan, Flru, Options) -> ra_log:execute_read_plan(Plan, Flru, diff --git a/src/ra_log_reader.erl b/src/ra_log_reader.erl deleted file mode 100644 index e50c778c5..000000000 --- a/src/ra_log_reader.erl +++ /dev/null @@ -1,554 +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-2025 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. -%% --module(ra_log_reader). - --compile(inline_list_funcs). - --export([ - init/5, - init/7, - close/1, - update_segments/2, - handle_log_update/2, - segment_refs/1, - segment_ref_count/1, - range/1, - num_open_segments/1, - update_first_index/2, - fold/5, - sparse_read/3, - read_plan/2, - exec_read_plan/6, - fetch_term/2 - ]). - --include("ra.hrl"). - --define(STATE, ?MODULE). - --type access_pattern() :: sequential | random. -%% holds static or rarely changing fields --record(cfg, {uid :: ra_uid(), - counter :: undefined | counters:counters_ref(), - directory :: file:filename(), - access_pattern = random :: access_pattern() - }). - --type segment_ref() :: ra_log:segment_ref(). --record(?STATE, {cfg :: #cfg{}, - range :: ra_range:range(), - segment_refs :: ra_lol:state(), - open_segments :: ra_flru:state() - }). - --opaque state() :: #?STATE{}. --type read_plan() :: [{BaseName :: file:filename_all(), [ra:index()]}]. --type read_plan_options() :: #{access_pattern => random | sequential, - file_advise => ra_log_segment:posix_file_advise()}. - - --export_type([ - state/0, - read_plan/0, - read_plan_options/0 - ]). - -%% PUBLIC - --spec init(ra_uid(), file:filename(), non_neg_integer(), - [segment_ref()], ra_system:names()) -> state(). -init(UId, Dir, MaxOpen, SegRefs, Names) -> - init(UId, Dir, MaxOpen, random, SegRefs, Names, undefined). - --spec init(ra_uid(), file:filename(), non_neg_integer(), - access_pattern(), - [segment_ref()], ra_system:names(), - undefined | counters:counters_ref()) -> state(). -init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, #{}, Counter) - when is_binary(UId) -> - Cfg = #cfg{uid = UId, - counter = Counter, - directory = Dir, - access_pattern = AccessPattern}, - FlruHandler = fun ({_, Seg}) -> - _ = ra_log_segment:close(Seg), - decr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1) - end, - SegRefs = compact_segrefs(SegRefs0, []), - Range = case SegRefs of - [{{_, L}, _} | _] -> - {{F, _}, _} = lists:last(SegRefs), - ra_range:new(F, L); - _ -> - undefined - end, - SegRefsRev = lists:reverse(SegRefs), - reset_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS), - #?STATE{cfg = Cfg, - open_segments = ra_flru:new(MaxOpen, FlruHandler), - range = Range, - segment_refs = - ra_lol:from_list(fun seg_ref_gt/2, SegRefsRev)}. - -seg_ref_gt({{Start, _}, Fn1}, {{_, End}, Fn2}) -> - Start > End andalso Fn1 > Fn2. - --spec close(state()) -> ok. -close(#?STATE{open_segments = Open}) -> - _ = ra_flru:evict_all(Open), - ok. - --spec update_segments([segment_ref()], state()) -> state(). -update_segments(NewSegmentRefs, - #?STATE{open_segments = Open0, - segment_refs = SegRefs0} = State) -> - - SegmentRefs0 = ra_lol:to_list(SegRefs0), - SegmentRefsComp = compact_segrefs(NewSegmentRefs, SegmentRefs0), - SegmentRefsCompRev = lists:reverse(SegmentRefsComp ), - SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev), - Range = case SegmentRefsComp of - [{{_, L}, _} | _] -> - [{{F, _}, _} | _] = SegmentRefsCompRev, - ra_range:new(F, L); - _ -> - undefined - end, - %% check if any of the updated segrefs refer to open segments - %% we close these segments so that they can be re-opened with updated - %% indexes if needed - Open = lists:foldl(fun ({_, Fn}, Acc0) -> - case ra_flru:evict(Fn, Acc0) of - {_, Acc} -> Acc; - error -> Acc0 - end - end, Open0, NewSegmentRefs), - State#?MODULE{segment_refs = SegRefs, - 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) -> - 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)), - open_segments = Open}. - --spec update_first_index(ra_index(), state()) -> - {state(), [segment_ref()]}. -update_first_index(FstIdx, #?STATE{segment_refs = SegRefs0, - open_segments = OpenSegs0} = State) -> - %% TODO: refactor this so that ra_lol just returns plain lists on both sides? - case ra_lol:takewhile(fun({{_, To}, _}) -> - To >= FstIdx - end, SegRefs0) of - {Active, Obsolete0} -> - case ra_lol:len(Obsolete0) of - 0 -> - {State, []}; - _ -> - Obsolete = ra_lol:to_list(Obsolete0), - ObsoleteKeys = [K || {_, K} <- Obsolete], - % close any open segments - OpenSegs = lists:foldl(fun (K, OS0) -> - case ra_flru:evict(K, OS0) of - {_, OS} -> OS; - error -> OS0 - end - end, OpenSegs0, ObsoleteKeys), - {State#?STATE{open_segments = OpenSegs, - segment_refs = ra_lol:from_list(fun seg_ref_gt/2, - lists:reverse(Active))}, - Obsolete} - end - end. - --spec segment_refs(state()) -> [segment_ref()]. -segment_refs(#?STATE{segment_refs = SegmentRefs}) -> - ra_lol:to_list(SegmentRefs). - --spec segment_ref_count(state()) -> non_neg_integer(). -segment_ref_count(#?STATE{segment_refs = SegmentRefs}) -> - ra_lol:len(SegmentRefs). - --spec range(state()) -> ra_range:range(). -range(#?STATE{range = Range}) -> - Range. - --spec num_open_segments(state()) -> non_neg_integer(). -num_open_segments(#?STATE{open_segments = Open}) -> - ra_flru:size(Open). - --spec fold(ra_index(), ra_index(), fun(), term(), state()) -> - {state(), term()}. -fold(FromIdx, ToIdx, Fun, Acc, - #?STATE{cfg = #cfg{} = Cfg} = State0) - when ToIdx >= FromIdx -> - ok = incr_counter(Cfg, ?C_RA_LOG_READ_SEGMENT, ToIdx - FromIdx + 1), - segment_fold(State0, FromIdx, ToIdx, Fun, Acc); -fold(_FromIdx, _ToIdx, _Fun, Acc, #?STATE{} = State) -> - {State, Acc}. - --spec sparse_read(state(), [ra_index()], [log_entry()]) -> - {[log_entry()], state()}. -sparse_read(#?STATE{cfg = #cfg{} = Cfg} = State, Indexes, Entries0) -> - {Open, SegC, Entries} = (catch segment_sparse_read(State, Indexes, Entries0)), - ok = incr_counter(Cfg, ?C_RA_LOG_READ_SEGMENT, SegC), - {Entries, State#?MODULE{open_segments = Open}}. - --spec read_plan(state(), [ra_index()]) -> read_plan(). -read_plan(#?STATE{segment_refs = SegRefs}, Indexes) -> - %% TODO: add counter for number of read plans requested - segment_read_plan(SegRefs, Indexes, []). - --spec exec_read_plan(file:filename_all(), - read_plan(), - undefined | ra_flru:state(), - TransformFun :: fun((ra_index(), ra_term(), binary()) -> term()), - read_plan_options(), - #{ra_index() => Command :: term()}) -> - {#{ra_index() => Command :: term()}, ra_flru:state()}. -exec_read_plan(Dir, Plan, undefined, TransformFun, Options, Acc0) -> - Open = ra_flru:new(1, fun({_, Seg}) -> ra_log_segment:close(Seg) end), - exec_read_plan(Dir, Plan, Open, TransformFun, Options, Acc0); -exec_read_plan(Dir, Plan, Open0, TransformFun, Options, Acc0) - when is_list(Plan) -> - Fun = fun (I, T, B, Acc) -> - E = TransformFun(I, T, binary_to_term(B)), - Acc#{I => E} - end, - lists:foldl( - fun ({Idxs, BaseName}, {Acc1, Open1}) -> - {Seg, Open2} = get_segment_ext(Dir, Open1, BaseName, Options), - case ra_log_segment:read_sparse(Seg, Idxs, Fun, Acc1) of - {ok, _, Acc} -> - {Acc, Open2}; - {error, modified} -> - %% if the segment has been modified since it was opened - %% it is not safe to attempt the read as the read plan - %% may refer to indexes that weren't in the segment at - %% that time. In this case we evict all segments and - %% re-open what we need. - {_, Open3} = ra_flru:evict(BaseName, Open2), - {SegNew, Open} = get_segment_ext(Dir, Open3, BaseName, Options), - %% at this point we can read without checking for modification - %% as the read plan would have been created before we - %% read the index from the segment - {ok, _, Acc} = ra_log_segment:read_sparse_no_checks( - SegNew, Idxs, Fun, Acc1), - {Acc, Open} - end - end, {Acc0, Open0}, Plan). - --spec fetch_term(ra_index(), state()) -> {option(ra_index()), state()}. -fetch_term(Idx, #?STATE{cfg = #cfg{} = Cfg} = State0) -> - incr_counter(Cfg, ?C_RA_LOG_FETCH_TERM, 1), - segment_term_query(Idx, State0). - -%% LOCAL - -segment_read_plan(_SegRefs, [], Acc) -> - lists:reverse(Acc); -segment_read_plan(SegRefs, [Idx | _] = Indexes, Acc) -> - case ra_lol:search(seg_ref_search_fun(Idx), SegRefs) of - {{Range, Fn}, Cont} -> - case sparse_read_split(fun (I) -> - ra_range:in(I, Range) - end, Indexes, []) of - {[], _} -> - segment_read_plan(Cont, Indexes, Acc); - {Idxs, Rem} -> - segment_read_plan(Cont, Rem, [{Idxs, Fn} | Acc]) - end; - undefined -> - %% not found - lists:reverse(Acc) - end. - -seg_ref_search_fun(Idx) -> - fun({{Start, End}, _}) -> - if Idx > End -> higher; - Idx < Start -> lower; - true -> equal - end - end. - -segment_term_query(Idx, #?MODULE{segment_refs = SegRefs, - cfg = Cfg, - open_segments = OpenSegs} = State) -> - {Result, Open} = segment_term_query0(Idx, SegRefs, OpenSegs, Cfg), - {Result, State#?MODULE{open_segments = Open}}. - -segment_term_query0(Idx, SegRefs, Open0, - #cfg{directory = Dir, - access_pattern = AccessPattern} = Cfg) -> - case ra_lol:search(seg_ref_search_fun(Idx), SegRefs) of - {{_Range, Fn}, _Cont} -> - case ra_flru:fetch(Fn, Open0) of - {ok, Seg, Open} -> - Term = ra_log_segment:term_query(Seg, Idx), - {Term, Open}; - error -> - AbsFn = filename:join(Dir, Fn), - {ok, Seg} = ra_log_segment:open(AbsFn, - #{mode => read, - access_pattern => AccessPattern}), - - incr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1), - Term = ra_log_segment:term_query(Seg, Idx), - {Term, ra_flru:insert(Fn, Seg, Open0)} - end; - undefined -> - {undefined, Open0} - end. - -segment_fold_plan(_SegRefs, undefined, Acc) -> - Acc; -segment_fold_plan(SegRefs, {_ReqStart, ReqEnd} = ReqRange, Acc) -> - case ra_lol:search(seg_ref_search_fun(ReqEnd), SegRefs) of - {{Range, Fn}, Cont} -> - This = ra_range:overlap(ReqRange, Range), - ReqRem = case ra_range:subtract(This, ReqRange) of - [] -> - undefined; - [Rem] -> - Rem - end, - segment_fold_plan(Cont, ReqRem, [{This, Fn} | Acc]); - undefined -> - %% not found - Acc - end. - -segment_fold(#?STATE{segment_refs = SegRefs, - open_segments = OpenSegs, - cfg = Cfg} = State, - RStart, REnd, Fun, Acc) -> - Plan = segment_fold_plan(SegRefs, {RStart, REnd}, []), - {Op, A} = - lists:foldl( - fun ({{Start, End}, Fn}, {Open0, Ac0}) -> - {Seg, Open} = get_segment(Cfg, Open0, Fn), - {Open, ra_log_segment:fold(Seg, Start, End, - fun binary_to_term/1, - Fun, - Ac0)} - end, {OpenSegs, Acc}, Plan), - {State#?MODULE{open_segments = Op}, A}. - - -segment_sparse_read(#?STATE{open_segments = Open}, [], Entries0) -> - {Open, 0, Entries0}; -segment_sparse_read(#?STATE{segment_refs = SegRefs, - open_segments = OpenSegs, - cfg = Cfg}, Indexes, Entries0) -> - Plan = segment_read_plan(SegRefs, Indexes, []), - lists:foldl( - fun ({Idxs, Fn}, {Open0, C, En0}) -> - {Seg, Open} = get_segment(Cfg, Open0, Fn), - {ok, ReadSparseCount, Entries} = - ra_log_segment:read_sparse_no_checks( - Seg, Idxs, fun (I, T, B, Acc) -> - [{I, T, binary_to_term(B)} | Acc] - end, []), - {Open, C + ReadSparseCount, lists:reverse(Entries, En0)} - end, {OpenSegs, 0, Entries0}, Plan). - -%% like lists:splitwith but without reversing the accumulator -sparse_read_split(Fun, [E | Rem] = All, Acc) -> - case Fun(E) of - true -> - sparse_read_split(Fun, Rem, [E | Acc]); - false -> - {Acc, All} - end; -sparse_read_split(_Fun, [], Acc) -> - {Acc, []}. - - -get_segment(#cfg{directory = Dir, - access_pattern = AccessPattern} = Cfg, Open0, Fn) -> - case ra_flru:fetch(Fn, Open0) of - {ok, S, Open1} -> - {S, Open1}; - error -> - AbsFn = filename:join(Dir, Fn), - case ra_log_segment:open(AbsFn, - #{mode => read, - access_pattern => AccessPattern}) - of - {ok, S} -> - incr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1), - {S, ra_flru:insert(Fn, S, Open0)}; - {error, Err} -> - exit({ra_log_failed_to_open_segment, Err, - AbsFn}) - end - end. - -get_segment_ext(Dir, Open0, Fn, Options) -> - case ra_flru:fetch(Fn, Open0) of - {ok, S, Open1} -> - {S, Open1}; - error -> - AbsFn = filename:join(Dir, Fn), - case ra_log_segment:open(AbsFn, - Options#{mode => read}) - of - {ok, S} -> - {S, ra_flru:insert(Fn, S, Open0)}; - {error, Err} -> - exit({ra_log_failed_to_open_segment, Err, - AbsFn}) - end - end. - -compact_segrefs(New, Cur) -> - %% all are in descending order - lists:foldr( - fun - (S, []) -> - [S]; - ({{Start, _}, _} = SegRef, Prev) -> - [SegRef | limit(Start, Prev)] - end, Cur, New). - -limit(_LimitIdx, []) -> - []; -limit(LimitIdx, [{PrevRange, PrevFn} | PrevRem]) -> - case ra_range:limit(LimitIdx, PrevRange) of - undefined -> - limit(LimitIdx, PrevRem); - NewPrevRange -> - [{NewPrevRange, PrevFn} | PrevRem] - end. - -reset_counter(#cfg{counter = Cnt}, Ix) - when Cnt =/= undefined -> - counters:put(Cnt, Ix, 0); -reset_counter(#cfg{counter = undefined}, _) -> - ok. - -incr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> - counters:add(Cnt, Ix, N); -incr_counter(#cfg{counter = undefined}, _, _) -> - ok. - -decr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> - counters:sub(Cnt, Ix, N); -decr_counter(#cfg{counter = undefined}, _, _) -> - ok. - --ifdef(TEST). --include_lib("eunit/include/eunit.hrl"). - -compact_seg_refs_test() -> - NewRefs = [{{10, 100}, "2"}], - PrevRefs = [{{10, 75}, "2"}, {{1, 9}, "1"}], - ?assertEqual([{{10, 100}, "2"}, {{1, 9}, "1"}], - compact_segrefs(NewRefs, PrevRefs)). - -compact_segref_3_test() -> - Data = [ - {{2, 7}, "B"}, - %% this entry has overwritten the prior two - {{5, 10}, "B"}, - {{1, 4}, "A"} - ], - Res = compact_segrefs(Data, []), - ?assertMatch([{{2, 7}, "B"}, - {{1, 1}, "A"}], Res), - ok. - -compact_segref_2_test() -> - Data = [ - {{80, 89}, "80"}, - %% this entry has overwritten the prior two - {{56, 79}, "71"}, - {{70, 85}, "70"}, - {{60, 69}, "60"}, - {{50, 59}, "50"} - ], - Res = compact_segrefs(Data, []), - ?assertMatch([{{80, 89}, "80"}, - {{56, 79}, "71"}, - {{50, 55}, "50"} - ], Res), - ok. - -compact_segref_1_test() -> - Data = [ - {{80, 89}, "80"}, - %% this entry has overwritten the prior one - {{70, 79}, "71"}, - {{70, 85}, "70"}, - %% partial overwrite - {{65, 69}, "65"}, - {{60, 69}, "60"}, - {{50, 59}, "50"}, - {{40, 49}, "40"} - ], - - Res = compact_segrefs(Data, [ - {{30, 39}, "30"}, - {{20, 29}, "20"} - ]), - - %% overwritten entry is no longer there - %% and the segment prior to the partial overwrite has been limited - %% to provide a continuous range - ?assertMatch([{{80, 89}, "80"}, - {{70, 79}, "71"}, - {{65, 69}, "65"}, - {{60, 64}, "60"}, - {{50, 59}, "50"}, - {{40, 49}, "40"}, - {{30, 39}, "30"}, - {{20, 29}, "20"} - ], Res), - ok. - - -segrefs_to_read_test() -> - - SegRefs = ra_lol:from_list( - fun seg_ref_gt/2, - lists:reverse( - compact_segrefs( - [{{412,499},"00000006.segment"}, - {{284,411},"00000005.segment"}, - %% this segment got overwritten - {{284,500},"00000004.segment"}, - {{200,285},"00000003.segment"}, - {{128,255},"00000002.segment"}, - {{0,127},"00000001.segment"}], []))), - - - ?assertEqual([{{199, 199}, "00000002.segment"}, - {{200, 283}, "00000003.segment"}, - {{284, 411}, "00000005.segment"}, - {{412, 499}, "00000006.segment"}], - segment_fold_plan(SegRefs, {199, 499}, [])), - - %% out of range - ?assertEqual([], segment_fold_plan(SegRefs, {500, 500}, [])), - ?assertEqual([ - {{127,127},"00000001.segment"}, - {{128,128},"00000002.segment"} - ], - segment_fold_plan(SegRefs, {127, 128}, [])), - ok. - --endif. diff --git a/src/ra_log_segment.erl b/src/ra_log_segment.erl index 2f8e29844..0d802333a 100644 --- a/src/ra_log_segment.erl +++ b/src/ra_log_segment.erl @@ -12,6 +12,7 @@ append/4, sync/1, fold/6, + fold/7, is_modified/1, read_sparse/4, read_sparse_no_checks/4, @@ -22,13 +23,17 @@ max_count/1, filename/1, segref/1, - is_same_as/2]). + info/1, + info/2, + is_same_as/2, + copy/3]). -export([dump/1, dump_index/1]). -include("ra.hrl"). +-include_lib("stdlib/include/assert.hrl"). -include_lib("kernel/include/file.hrl"). -define(VERSION, 2). @@ -54,7 +59,7 @@ fd :: option(file:io_device()), index_size :: pos_integer(), access_pattern :: sequential | random, - file_advise = normal :: posix_file_advise(), + file_advise = normal :: posix_file_advise(), mode = append :: read | append, compute_checksums = true :: boolean()}). @@ -296,7 +301,22 @@ fold(#state{cfg = #cfg{mode = read} = Cfg, cache = Cache, index = Index}, FromIdx, ToIdx, Fun, AccFun, Acc) -> - fold0(Cfg, Cache, FromIdx, ToIdx, Index, Fun, AccFun, Acc). + fold0(Cfg, Cache, FromIdx, ToIdx, Index, Fun, AccFun, Acc, + error). + +-spec fold(state(), + FromIdx :: ra_index(), + ToIdx :: ra_index(), + fun((binary()) -> term()), + fun(({ra_index(), ra_term(), term()}, Acc) -> Acc), Acc, + MissingKeyStrat :: error | return) -> + Acc when Acc :: term(). +fold(#state{cfg = #cfg{mode = read} = Cfg, + cache = Cache, + index = Index}, + FromIdx, ToIdx, Fun, AccFun, Acc, MissingKeyStrat) -> + fold0(Cfg, Cache, FromIdx, ToIdx, Index, Fun, AccFun, Acc, + MissingKeyStrat). -spec is_modified(state()) -> boolean(). is_modified(#state{cfg = #cfg{fd = Fd}, @@ -399,10 +419,10 @@ term_query(#state{index = Index}, Idx) -> _ -> undefined end. -fold0(_Cfg, _Cache, Idx, FinalIdx, _, _Fun, _AccFun, Acc) +fold0(_Cfg, _Cache, Idx, FinalIdx, _, _Fun, _AccFun, Acc, _) when Idx > FinalIdx -> Acc; -fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0) -> +fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0, MissingKeyStrat) -> case Index of #{Idx := {Term, Offset, Length, Crc} = IdxRec} -> case pread(Cfg, Cache0, Offset, Length) of @@ -411,7 +431,8 @@ fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0) -> case validate_checksum(Crc, Data) of true -> Acc = AccFun({Idx, Term, Fun(Data)}, Acc0), - fold0(Cfg, Cache, Idx+1, FinalIdx, Index, Fun, AccFun, Acc); + fold0(Cfg, Cache, Idx+1, FinalIdx, + Index, Fun, AccFun, Acc, MissingKeyStrat); false -> %% CRC check failures are irrecoverable exit({ra_log_segment_crc_check_failure, Idx, IdxRec, @@ -422,8 +443,10 @@ fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0) -> exit({ra_log_segment_unexpected_eof, Idx, IdxRec, Cfg#cfg.filename}) end; - _ -> - exit({missing_key, Idx, Cfg#cfg.filename}) + _ when MissingKeyStrat == error -> + exit({missing_key, Idx, Cfg#cfg.filename}); + _ when MissingKeyStrat == return -> + Acc0 end. -spec range(state()) -> option({ra_index(), ra_index()}). @@ -444,13 +467,67 @@ segref(#state{range = undefined}) -> undefined; segref(#state{range = Range, cfg = #cfg{filename = Fn}}) -> - {Range, filename:basename(Fn)}; + {ra_lib:to_binary(filename:basename(Fn)), Range}; segref(Filename) -> {ok, Seg} = open(Filename, #{mode => read}), SegRef = segref(Seg), close(Seg), SegRef. +-type infos() :: #{size => non_neg_integer(), + index_size => non_neg_integer(), + max_count => non_neg_integer(), + file_type => regular | symlink, + ctime => integer(), + links => non_neg_integer(), + num_entries => non_neg_integer(), + ref => option(ra_log:segment_ref()), + indexes => ra_seq:state(), + live_size => non_neg_integer() + }. + +-spec info(file:filename_all()) -> infos(). +info(Filename) -> + info(Filename, undefined). + +-spec info(file:filename_all(), option(ra_seq:state())) -> infos(). +info(Filename, Live0) + when not is_tuple(Filename) -> + %% TODO: this can be much optimised by a specialised index parsing + %% function + {ok, #file_info{type = Type, + links = Links, + ctime = CTime}} = prim_file:read_link_info(Filename, + [raw, {time, posix}]), + + {ok, Seg} = open(Filename, #{mode => read}), + Index = Seg#state.index, + AllIndexesSeq = ra_seq:from_list(maps:keys(Index)), + Live = case Live0 of + undefined -> + AllIndexesSeq; + _ -> + Live0 + end, + LiveSize = ra_seq:fold(fun (I, Acc) -> + {_, _, Sz, _} = maps:get(I, Index), + Acc + Sz + end, 0, Live), + Info = #{size => Seg#state.data_write_offset, + index_size => Seg#state.data_start, + file_type => Type, + links => Links, + ctime => CTime, + max_count => max_count(Seg), + num_entries => maps:size(Index), + ref => segref(Seg), + live_size => LiveSize, + %% TODO: this is most likely just here for debugging + indexes => AllIndexesSeq + }, + close(Seg), + Info. + -spec is_same_as(state(), file:filename_all()) -> boolean(). is_same_as(#state{cfg = #cfg{filename = Fn0}}, Fn) -> is_same_filename_all(Fn0, Fn). @@ -474,6 +551,23 @@ close(#state{cfg = #cfg{fd = Fd}}) -> _ = file:close(Fd), ok. +-spec copy(state(), file:filename_all(), [ra:index()]) -> + {ok, state()} | {error, term()}. +copy(#state{} = State0, FromFile, Indexes) + when is_list(Indexes) -> + {ok, From} = open(FromFile, #{mode => read}), + %% TODO: the current approach recalculates the CRC and isn't completely + %% optimial. Also it does not allow for a future where copy_file_range may + %% be available + State = lists:foldl( + fun (I, S0) -> + {ok, Term, Data} = simple_read(From, I), + {ok, S} = append(S0, I, Term, Data), + S + end, State0, lists:sort(Indexes)), + close(From), + sync(State). + %%% Internal is_same_filename_all(Fn, Fn) -> @@ -693,6 +787,20 @@ is_full(#state{cfg = #cfg{max_size = MaxSize}, IndexOffset >= DataStart orelse (DataOffset - DataStart) > MaxSize. +simple_read(#state{cfg = #cfg{fd = Fd}, + index = SegIndex}, Idx) + when is_map_key(Idx, SegIndex) -> + {Term, Pos, Len, _} = map_get(Idx, SegIndex), + case file:pread(Fd, Pos, Len) of + {ok, Data} -> + ?assert(byte_size(Data) == Len), + {ok, Term, Data}; + Err -> + Err + end; +simple_read(_State, _) -> + {error, not_found}. + -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index 3d2d11848..9b1494878 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 {ok, _, Failures} = @@ -184,7 +184,7 @@ handle_cast({mem_tables, Ranges, WalFile}, #state{data_dir = Dir, "~s in ~bms", [System, length(RangesList), WalFile, Diff]), {noreply, State}; -handle_cast({truncate_segments, Who, {_Range, Name} = SegRef}, +handle_cast({truncate_segments, Who, {Name, _Range} = SegRef}, #state{segment_conf = SegConf, system = System} = State0) -> %% remove all segments below the provided SegRef @@ -193,7 +193,7 @@ handle_cast({truncate_segments, Who, {_Range, Name} = SegRef}, Files = segments_for(Who, State0), {_Keep, Discard} = lists:splitwith( fun (F) -> - ra_lib:to_string(filename:basename(F)) =/= Name + ra_lib:to_binary(filename:basename(F)) =/= Name end, lists:reverse(Files)), case Discard of [] -> @@ -218,7 +218,7 @@ handle_cast({truncate_segments, Who, {_Range, Name} = SegRef}, ?DEBUG("segment_writer in '~w': ~s for ~s took ~bms", [System, ?FUNCTION_NAME, Who, Diff]), case open_successor_segment(Seg, SegConf) of - undefined -> + enoent -> %% directory must have been deleted after the pivot %% segment was opened {noreply, State0}; @@ -262,40 +262,54 @@ 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), - %% TidRanges arrive here sorted new -> old. + SmallestIdx = smallest_live_idx(ServerUId), + LiveIndexes = live_indexes(ServerUId), + LastLive = ra_seq:last(LiveIndexes), + %% TidSeqs arrive here sorted new -> old. + + %% TODO: use live indexes from ra_log_snapshot_state table to only + %% write live entries below the snapshot index - %% truncate and limit all ranges to create a contiguous non-overlapping + %% truncate and limit all seqa 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 when LiveIndexes == []-> + [{T, Seq}]; + Seq -> + L = ra_seq:in_range(ra_seq:range(Seq), + LiveIndexes), + + [{T, ra_seq:add(ra_seq:floor(LastLive + 1, Seq), L)}] + 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 when LiveIndexes == [] -> + [{T, Seq} | Acc]; + Seq -> + L = ra_seq:in_range(ra_seq:range(Seq), + LiveIndexes), + [{T, ra_seq:add(ra_seq:floor(LastLive + 1, Seq), L)} + | 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 @@ -303,20 +317,19 @@ flush_mem_table_ranges({ServerUId, TidRanges0}, %% order they are kept by the ra_log SegRefs = lists:reverse( lists:foldl( - fun ({_, FILE}, [{_, FILE} | _] = Acc) -> + fun ({FILE, _}, [{FILE, _} | _] = Acc) -> Acc; (Seg, Acc) -> [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), case open_file(Dir, SegConf) of enoent -> ?DEBUG("segment_writer: skipping segment as directory ~ts does " @@ -325,8 +338,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", @@ -353,15 +365,13 @@ 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). + +live_indexes(ServerUId) -> + ra_log_snapshot_state:live_indexes(ra_log_snapshot_state, ServerUId). send_segments(System, ServerUId, TidRanges, SegRefs) -> case ra_directory:pid_of(System, ServerUId) of @@ -372,23 +382,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), @@ -396,7 +416,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. @@ -425,11 +445,11 @@ 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 - undefined -> + enoent -> %% a successor cannot be opened - this is most likely due %% to the directory having been deleted. undefined; @@ -438,8 +458,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), @@ -478,7 +504,7 @@ open_successor_segment(CurSeg, SegConf) -> {error, enoent} -> %% the directory has been deleted whilst segments were being %% written - undefined; + enoent; {ok, Seg} -> Seg end. @@ -554,3 +580,14 @@ maybe_upgrade_segment_file_names(System, DataDir) -> true -> 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_segments.erl b/src/ra_log_segments.erl new file mode 100644 index 000000000..f52fb3bab --- /dev/null +++ b/src/ra_log_segments.erl @@ -0,0 +1,1010 @@ +%% 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. +%% @hidden +-module(ra_log_segments). + +-compile(inline_list_funcs). + +-include_lib("kernel/include/file.hrl"). +-export([ + init/8, + update_conf/3, + close/1, + update_segments/2, + schedule_compaction/4, + handle_compaction_result/2, + segment_refs/1, + segment_ref_count/1, + range/1, + num_open_segments/1, + update_first_index/2, + fold/6, + sparse_read/3, + read_plan/2, + exec_read_plan/6, + fetch_term/2, + info/1, + purge_symlinks/2, + purge_dangling_symlinks/1, + compaction_conf/1 + ]). + +-include("ra.hrl"). + +-define(STATE, ?MODULE). + +-define(SYMLINK_KEEPFOR_S, 60). + +-type compaction_conf() :: #{max_count => non_neg_integer(), + max_size => non_neg_integer()}. +-type access_pattern() :: sequential | random. +%% holds static or rarely changing fields +-record(cfg, {uid :: ra_uid(), + log_id = "" :: unicode:chardata(), + counter :: undefined | counters:counters_ref(), + directory :: file:filename(), + access_pattern = random :: access_pattern(), + compaction_conf :: compaction_conf() + }). + +-type segment_ref() :: ra_log:segment_ref(). + +-record(?STATE, {cfg :: #cfg{}, + range :: ra_range:range(), + segment_refs :: ra_lol:state(), + open_segments :: ra_flru:state(), + compaction :: undefined | major | minor, + next_compaction :: undefined | major | minor + }). + +-record(compaction_result, + {unreferenced = [] :: [file:filename_all()], + linked = [] :: [file:filename_all()], + compacted = [] :: [segment_ref()]}). + +-opaque state() :: #?STATE{}. +-type read_plan() :: [{BaseName :: file:filename_all(), [ra:index()]}]. +-type read_plan_options() :: #{access_pattern => random | sequential, + file_advise => ra_log_segment:posix_file_advise()}. + +-export_type([ + state/0, + read_plan/0, + read_plan_options/0 + ]). + +%% PUBLIC + +-spec init(ra_uid(), file:filename_all(), non_neg_integer(), + access_pattern(), [segment_ref()], + undefined | counters:counters_ref(), + map(), + unicode:chardata()) -> state(). +init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, Counter, CompConf, LogId) + when is_binary(UId) -> + Cfg = #cfg{uid = UId, + log_id = LogId, + counter = Counter, + directory = Dir, + access_pattern = AccessPattern, + compaction_conf = CompConf}, + FlruHandler = fun ({_, Seg}) -> + _ = ra_log_segment:close(Seg), + decr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1) + end, + SegRefs = compact_segrefs(SegRefs0, []), + Range = case SegRefs of + [{_, {_, L}} | _] -> + {_, {F, _}} = lists:last(SegRefs), + ra_range:new(F, L); + _ -> + undefined + end, + SegRefsRev = lists:reverse(SegRefs), + reset_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS), + Result = recover_compaction(Dir), + %% handle_compaction_result/2 will never return an effect here + %% as no segments got deleted + State0 = #?STATE{cfg = Cfg, + open_segments = ra_flru:new(MaxOpen, FlruHandler), + range = Range, + segment_refs = ra_lol:from_list( + fun seg_ref_gt/2, SegRefsRev)}, + {State, _} = handle_compaction_result(Result, State0), + State. + +-spec close(state()) -> ok. +close(#?STATE{open_segments = Open}) -> + _ = ra_flru:evict_all(Open), + ok. + + +-spec update_conf(non_neg_integer(), sequential | random, state()) -> + state(). +update_conf(MaxOpen, AccessPattern, + #?STATE{cfg = Cfg, + open_segments = Open} = State) -> + FlruHandler = fun ({_, Seg}) -> + _ = ra_log_segment:close(Seg), + decr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1) + end, + _ = ra_flru:evict_all(Open), + State#?STATE{cfg = Cfg#cfg{access_pattern = AccessPattern}, + open_segments = ra_flru:new(MaxOpen, FlruHandler)}. + + +-spec update_segments([segment_ref()], state()) -> + {state(), OverwrittenSegments :: [segment_ref()]}. +update_segments(NewSegmentRefs, #?STATE{open_segments = Open0, + segment_refs = SegRefs0} = State) -> + + SegmentRefs0 = ra_lol:to_list(SegRefs0), + SegmentRefsComp = compact_segrefs(NewSegmentRefs, SegmentRefs0), + %% capture segrefs removed by compact_segrefs/2 and delete them + %% a major compaction will also remove these + OverwrittenSegments = NewSegmentRefs -- SegmentRefsComp, + SegmentRefsCompRev = lists:reverse(SegmentRefsComp), + SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev), + Range = case SegmentRefsComp of + [{_, {_, L}} | _] -> + [{_, {F, _}} | _] = SegmentRefsCompRev, + ra_range:new(F, L); + _ -> + undefined + end, + %% check if any of the updated segrefs refer to open segments + %% we close these segments so that they can be re-opened with updated + %% indexes if needed + Open = lists:foldl(fun ({Fn, _}, Acc0) -> + case ra_flru:evict(Fn, Acc0) of + {_, Acc} -> Acc; + error -> Acc0 + end + end, Open0, NewSegmentRefs), + {State#?MODULE{segment_refs = SegRefs, + range = Range, + open_segments = Open}, + OverwrittenSegments}. + +-spec schedule_compaction(minor | major, ra:index(), + ra_seq:state(), state()) -> + {state(), [ra_server:effect()]}. +schedule_compaction(Type, SnapIdx, LiveIndexes, + #?MODULE{cfg = #cfg{log_id = LogId, + compaction_conf = CompConf, + directory = Dir} = Cfg, + compaction = undefined} = State) -> + case compactable_segrefs(SnapIdx, State) of + [] -> + {State, []}; + SegRefs when LiveIndexes == [] -> + %% if LiveIndexes is [] we can just delete all compactable + %% segment refs + Unreferenced = [F || {F, _} <- SegRefs], + Result = #compaction_result{unreferenced = Unreferenced}, + {State#?MODULE{compaction = minor}, + [{next_event, + {ra_log_event, {compaction_result, Result}}}]}; + SegRefs when Type == minor -> + %% TODO evaluate if minor compactions are fast enough to run + %% in server process + Result = minor_compaction(SegRefs, LiveIndexes), + {State#?MODULE{compaction = minor}, + [{next_event, + {ra_log_event, {compaction_result, Result}}}]}; + SegRefs -> + Self = self(), + Fun = fun () -> + ok = incr_counter(Cfg, ?C_RA_LOG_COMPACTIONS_MAJOR_COUNT, 1), + MajConf = CompConf#{dir => Dir}, + Result = major_compaction(MajConf, SegRefs, + LiveIndexes), + %% need to update the ra_servers list of seg refs + %% _before_ the segments can actually be deleted + Self ! {ra_log_event, + {compaction_result, Result}}, + %% TODO: this could be done on a timer if more + %% timely symlink cleanup is needed + purge_symlinks(Dir, ?SYMLINK_KEEPFOR_S), + ok + end, + + {State#?MODULE{compaction = major}, + [{bg_work, Fun, + fun (Err) -> + %% send an empty compaction result to ensure the + %% a future compaction can be performed (TODO:) + Self ! {ra_log_event, + {compaction_result, #compaction_result{}}}, + ?WARN("~ts: Major compaction failed with ~p", + [LogId, Err]), ok + end}]} + end; +schedule_compaction(Type, SnapIdx, _LiveIndexes, + #?MODULE{cfg = #cfg{log_id = LogId}, + compaction = Comp} = State) -> + ?DEBUG("~ts: ~s compaction requested at ~b but ~s compaction already in progress", + [LogId, Type, SnapIdx, Comp]), + {State, []}. + + +-spec handle_compaction_result(#compaction_result{}, state()) -> + {state(), [ra_server:effect()]}. +handle_compaction_result(#compaction_result{unreferenced = [], + linked = [], + compacted = []}, + State) -> + {State#?MODULE{compaction = undefined}, []}; +handle_compaction_result(#compaction_result{unreferenced = Unreferenced, + linked = Linked, + compacted = Compacted}, + #?STATE{cfg = #cfg{directory = Dir} = Cfg, + open_segments = Open0, + segment_refs = SegRefs0} = State) -> + SegRefs1 = maps:from_list(ra_lol:to_list(SegRefs0)), + SegRefs2 = maps:without(Unreferenced, SegRefs1), + SegRefs = maps:without(Linked, SegRefs2), + SegmentRefs0 = maps:merge(SegRefs, maps:from_list(Compacted)), + SegmentRefs = maps:to_list(maps:iterator(SegmentRefs0, ordered)), + Open = ra_flru:evict_all(Open0), + Fun = fun () -> + [prim_file:delete(filename:join(Dir, F)) + || F <- Unreferenced], + ok + end, + ok = incr_counter(Cfg, ?C_RA_LOG_COMPACTIONS_SEGMENTS_WRITTEN, + length(Compacted)), + ok = incr_counter(Cfg, ?C_RA_LOG_COMPACTIONS_SEGMENTS_COMPACTED, + length(Linked) + length(Compacted)), + {State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, + SegmentRefs), + compaction = undefined, + open_segments = Open}, + [{bg_work, Fun, fun (_Err) -> ok end}]}. + + +-spec update_first_index(ra_index(), state()) -> + {state(), [segment_ref()]}. +update_first_index(FstIdx, #?STATE{segment_refs = SegRefs0, + open_segments = OpenSegs0} = State) -> + %% TODO: refactor this so that ra_lol just returns plain lists on both sides? + case ra_lol:takewhile(fun({_Fn, {_, To}}) -> + To >= FstIdx + end, SegRefs0) of + {Active, Obsolete0} -> + case ra_lol:len(Obsolete0) of + 0 -> + {State, []}; + _ -> + Obsolete = ra_lol:to_list(Obsolete0), + ObsoleteKeys = [K || {K, _} <- Obsolete], + % close any open segments + OpenSegs = lists:foldl(fun (K, OS0) -> + case ra_flru:evict(K, OS0) of + {_, OS} -> OS; + error -> OS0 + end + end, OpenSegs0, ObsoleteKeys), + {State#?STATE{open_segments = OpenSegs, + segment_refs = ra_lol:from_list(fun seg_ref_gt/2, + lists:reverse(Active))}, + Obsolete} + end + end. + +-spec segment_refs(state()) -> [segment_ref()]. +segment_refs(#?STATE{segment_refs = SegmentRefs}) -> + ra_lol:to_list(SegmentRefs). + +-spec segment_ref_count(state()) -> non_neg_integer(). +segment_ref_count(#?STATE{segment_refs = SegmentRefs}) -> + ra_lol:len(SegmentRefs). + +-spec range(state()) -> ra_range:range(). +range(#?STATE{range = Range}) -> + Range. + +-spec compaction_conf(state()) -> map(). +compaction_conf(#?STATE{cfg = #cfg{compaction_conf = Conf}}) -> + Conf. + +-spec num_open_segments(state()) -> non_neg_integer(). +num_open_segments(#?STATE{open_segments = Open}) -> + ra_flru:size(Open). + +-spec fold(ra_index(), ra_index(), fun(), term(), state(), + MissingKeyStrategy :: error | return) -> + {state(), term()}. +fold(FromIdx, ToIdx, Fun, Acc, + #?STATE{cfg = #cfg{} = Cfg} = State0, MissingKeyStrat) + when ToIdx >= FromIdx -> + ok = incr_counter(Cfg, ?C_RA_LOG_READ_SEGMENT, ToIdx - FromIdx + 1), + segment_fold(State0, FromIdx, ToIdx, Fun, Acc, MissingKeyStrat); +fold(_FromIdx, _ToIdx, _Fun, Acc, #?STATE{} = State, _Strat) -> + {State, Acc}. + +-spec sparse_read(state(), [ra_index()], [log_entry()]) -> + {[log_entry()], state()}. +sparse_read(#?STATE{cfg = #cfg{} = Cfg} = State, Indexes, Entries0) -> + {Open, SegC, Entries} = (catch segment_sparse_read(State, Indexes, Entries0)), + ok = incr_counter(Cfg, ?C_RA_LOG_READ_SEGMENT, SegC), + {Entries, State#?MODULE{open_segments = Open}}. + +-spec read_plan(state(), [ra_index()]) -> read_plan(). +read_plan(#?STATE{segment_refs = SegRefs}, Indexes) -> + %% TODO: add counter for number of read plans requested + segment_read_plan(SegRefs, Indexes, []). + +-spec exec_read_plan(file:filename_all(), + read_plan(), + undefined | ra_flru:state(), + TransformFun :: fun((ra_index(), ra_term(), binary()) -> term()), + read_plan_options(), + #{ra_index() => Command :: term()}) -> + {#{ra_index() => Command :: term()}, ra_flru:state()}. +exec_read_plan(Dir, Plan, undefined, TransformFun, Options, Acc0) -> + Open = ra_flru:new(1, fun({_, Seg}) -> ra_log_segment:close(Seg) end), + exec_read_plan(Dir, Plan, Open, TransformFun, Options, Acc0); +exec_read_plan(Dir, Plan, Open0, TransformFun, Options, Acc0) + when is_list(Plan) -> + Fun = fun (I, T, B, Acc) -> + E = TransformFun(I, T, binary_to_term(B)), + Acc#{I => E} + end, + lists:foldl( + fun ({BaseName, Idxs}, {Acc1, Open1}) -> + {Seg, Open2} = get_segment_ext(Dir, Open1, BaseName, Options), + case ra_log_segment:read_sparse(Seg, Idxs, Fun, Acc1) of + {ok, _, Acc} -> + {Acc, Open2}; + {error, modified} -> + %% if the segment has been modified since it was opened + %% it is not safe to attempt the read as the read plan + %% may refer to indexes that weren't in the segment at + %% that time. In this case we evict all segments and + %% re-open what we need. + {_, Open3} = ra_flru:evict(BaseName, Open2), + {SegNew, Open} = get_segment_ext(Dir, Open3, BaseName, Options), + %% at this point we can read without checking for modification + %% as the read plan would have been created before we + %% read the index from the segment + {ok, _, Acc} = ra_log_segment:read_sparse_no_checks( + SegNew, Idxs, Fun, Acc1), + {Acc, Open} + end + end, {Acc0, Open0}, Plan). + +-spec fetch_term(ra_index(), state()) -> {option(ra_index()), state()}. +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)}. + +-spec purge_symlinks(file:filename_all(), + OlderThanSec :: non_neg_integer()) -> ok. +purge_symlinks(Dir, OlderThanSec) -> + Now = erlang:system_time(second), + [begin + Fn = filename:join(Dir, F), + case prim_file:read_link_info(Fn, [raw, {time, posix}]) of + {ok, #file_info{type = symlink, + ctime = Time}} + when Now - Time > OlderThanSec -> + prim_file:delete(Fn), + ok; + _ -> + ok + end + end || F <- list_files(Dir, ".segment")], + ok. + +-spec purge_dangling_symlinks(file:filename_all()) -> ok. +purge_dangling_symlinks(Dir) -> + [begin + Fn = filename:join(Dir, F), + case file:read_link_info(Fn, [raw]) of + {ok, #file_info{type = symlink}} -> + case file:open(Fn, [raw, read, binary]) of + {ok, Fd} -> + ok = file:close(Fd); + {error, enoent} -> + %% dangling symlink + ok = prim_file:delete(Fn) + end; + _ -> + ok + end + end || F <- list_files(Dir, ".segment")], + ok. +%% LOCAL + +segment_read_plan(_SegRefs, [], Acc) -> + lists:reverse(Acc); +segment_read_plan(SegRefs, [Idx | _] = Indexes, Acc) -> + case ra_lol:search(seg_ref_search_fun(Idx), SegRefs) of + {{Fn, Range}, Cont} -> + case sparse_read_split(fun (I) -> + ra_range:in(I, Range) + end, Indexes, []) of + {[], _} -> + segment_read_plan(Cont, Indexes, Acc); + {Idxs, Rem} -> + segment_read_plan(Cont, Rem, [{Fn, Idxs} | Acc]) + end; + undefined -> + %% not found + lists:reverse(Acc) + end. + +seg_ref_search_fun(Idx) -> + fun({__Fn, {Start, End}}) -> + if Idx > End -> higher; + Idx < Start -> lower; + true -> equal + end + end. + +segment_term_query(Idx, #?MODULE{segment_refs = SegRefs, + cfg = Cfg, + open_segments = OpenSegs} = State) -> + {Result, Open} = segment_term_query0(Idx, SegRefs, OpenSegs, Cfg), + {Result, State#?MODULE{open_segments = Open}}. + +segment_term_query0(Idx, SegRefs, Open0, + #cfg{directory = Dir, + access_pattern = AccessPattern} = Cfg) -> + case ra_lol:search(seg_ref_search_fun(Idx), SegRefs) of + {{Fn, _Range}, _Cont} -> + case ra_flru:fetch(Fn, Open0) of + {ok, Seg, Open} -> + Term = ra_log_segment:term_query(Seg, Idx), + {Term, Open}; + error -> + AbsFn = filename:join(Dir, Fn), + {ok, Seg} = ra_log_segment:open(AbsFn, + #{mode => read, + access_pattern => AccessPattern}), + + incr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1), + Term = ra_log_segment:term_query(Seg, Idx), + {Term, ra_flru:insert(Fn, Seg, Open0)} + end; + undefined -> + {undefined, Open0} + end. + +segment_fold_plan(_SegRefs, undefined, Acc) -> + Acc; +segment_fold_plan(SegRefs, {_ReqStart, ReqEnd} = ReqRange, Acc) -> + case ra_lol:search(seg_ref_search_fun(ReqEnd), SegRefs) of + {{Fn, Range}, Cont} -> + This = ra_range:overlap(ReqRange, Range), + ReqRem = case ra_range:subtract(This, ReqRange) of + [] -> + undefined; + [Rem] -> + Rem + end, + segment_fold_plan(Cont, ReqRem, [{Fn, This} | Acc]); + undefined -> + %% not found + Acc + end. + +segment_fold(#?STATE{segment_refs = SegRefs, + open_segments = OpenSegs, + cfg = Cfg} = State, + RStart, REnd, Fun, Acc, MissingKeyStrat) -> + Plan = segment_fold_plan(SegRefs, {RStart, REnd}, []), + {Op, A} = + lists:foldl( + fun ({Fn, {Start, End}}, {Open0, Ac0}) -> + {Seg, Open} = get_segment(Cfg, Open0, Fn), + {Open, ra_log_segment:fold(Seg, Start, End, + fun binary_to_term/1, + Fun, Ac0, MissingKeyStrat)} + end, {OpenSegs, Acc}, Plan), + {State#?MODULE{open_segments = Op}, A}. + + +segment_sparse_read(#?STATE{open_segments = Open}, [], Entries0) -> + {Open, 0, Entries0}; +segment_sparse_read(#?STATE{segment_refs = SegRefs, + open_segments = OpenSegs, + cfg = Cfg}, Indexes, Entries0) -> + Plan = segment_read_plan(SegRefs, Indexes, []), + lists:foldl( + fun ({Fn, Idxs}, {Open0, C, En0}) -> + {Seg, Open} = get_segment(Cfg, Open0, Fn), + {ok, ReadSparseCount, Entries} = + ra_log_segment:read_sparse_no_checks( + Seg, Idxs, fun (I, T, B, Acc) -> + [{I, T, binary_to_term(B)} | Acc] + end, []), + {Open, C + ReadSparseCount, lists:reverse(Entries, En0)} + end, {OpenSegs, 0, Entries0}, Plan). + +%% like lists:splitwith but without reversing the accumulator +sparse_read_split(Fun, [E | Rem] = All, Acc) -> + case Fun(E) of + true -> + sparse_read_split(Fun, Rem, [E | Acc]); + false -> + {Acc, All} + end; +sparse_read_split(_Fun, [], Acc) -> + {Acc, []}. + + +get_segment(#cfg{directory = Dir, + access_pattern = AccessPattern} = Cfg, Open0, Fn) + when is_binary(Fn) -> + case ra_flru:fetch(Fn, Open0) of + {ok, S, Open1} -> + {S, Open1}; + error -> + AbsFn = filename:join(Dir, Fn), + case ra_log_segment:open(AbsFn, + #{mode => read, + access_pattern => AccessPattern}) + of + {ok, S} -> + incr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1), + {S, ra_flru:insert(Fn, S, Open0)}; + {error, Err} -> + exit({ra_log_failed_to_open_segment, Err, + AbsFn}) + end + end. + +get_segment_ext(Dir, Open0, Fn, Options) -> + case ra_flru:fetch(Fn, Open0) of + {ok, S, Open1} -> + {S, Open1}; + error -> + AbsFn = filename:join(Dir, Fn), + case ra_log_segment:open(AbsFn, + Options#{mode => read}) + of + {ok, S} -> + {S, ra_flru:insert(Fn, S, Open0)}; + {error, Err} -> + exit({ra_log_failed_to_open_segment, Err, + AbsFn}) + end + end. + +compact_segrefs(New, Cur) -> + %% all are in descending order + lists:foldr( + fun + (S, []) -> + [S]; + ({_, {Start, _}} = SegRef, Prev) -> + [SegRef | limit(Start, Prev)] + end, Cur, New). + +limit(_LimitIdx, []) -> + []; +limit(LimitIdx, [{PrevFn, PrevRange} | PrevRem]) -> + case ra_range:limit(LimitIdx, PrevRange) of + undefined -> + limit(LimitIdx, PrevRem); + NewPrevRange -> + [{PrevFn, NewPrevRange} | PrevRem] + end. + +reset_counter(#cfg{counter = Cnt}, Ix) + when Cnt =/= undefined -> + counters:put(Cnt, Ix, 0); +reset_counter(#cfg{counter = undefined}, _) -> + ok. + +incr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> + counters:add(Cnt, Ix, N); +incr_counter(#cfg{counter = undefined}, _, _) -> + ok. + +decr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> + counters:sub(Cnt, Ix, N); +decr_counter(#cfg{counter = undefined}, _, _) -> + ok. + +segment_files(Dir, Fun) -> + list_files(Dir, ".segment", Fun). + +% delete_files(Dir, Ext) -> +% case list_files(Dir, Ext, fun (_) -> true end) of +% [] -> +% ok; +% Files -> +% [prim_file:delete(filename:join(Dir, F)) || F <- Files] +% end. + +list_files(Dir, Ext) -> + list_files(Dir, Ext, fun (_) -> true end). + +list_files(Dir, Ext, Fun) -> + case prim_file:list_dir(Dir) of + {ok, Files0} -> + Files = [list_to_binary(F) + || F <- Files0, + filename:extension(F) =:= Ext, + Fun(F)], + lists:sort(Files); + {error, enoent} -> + [] + end. + +major_compaction(#{dir := Dir} = CompConf, SegRefs, LiveIndexes) -> + {Compactable, Delete} = + lists:foldl(fun({Fn0, Range} = S, + {Comps, Del}) -> + case ra_seq:in_range(Range, + LiveIndexes) of + [] -> + {Comps, [Fn0 | Del]}; + Seq -> + %% get the info map from each + %% potential segment + Fn = filename:join(Dir, Fn0), + Info = ra_log_segment:info(Fn, Seq), + {[{Info, Seq, S} | Comps], Del} + end + end, {[], []}, SegRefs), + + %% ensure there are no remaining fully overwritten (unused) segments in + %% the compacted range + Lookup = maps:from_list(SegRefs), + {LastFn, {_, _}} = lists:last(SegRefs), + UnusedFiles = segment_files(Dir, fun (F) -> + Key = list_to_binary(F), + Key =< LastFn andalso + not maps:is_key(Key, Lookup) + end), + [begin + ok = prim_file:delete(filename:join(Dir, F)) + end || F <- UnusedFiles], + %% group compactable + CompactionGroups = compaction_groups(lists:reverse(Compactable), [], + CompConf), + Compacted0 = + [begin + AllFns = [F || {_, _, {F, _}} <- All], + %% create a compaction marker with the compaction group i + CompactionMarker = filename:join(Dir, with_ext(CompGroupLeaderFn, + ".compaction_group")), + ok = ra_lib:write_file(CompactionMarker, term_to_binary(AllFns)), + %% create a new segment with .compacting extension + CompactingFn = filename:join(Dir, with_ext(CompGroupLeaderFn, + ".compacting")), + %% max_count is the sum of all live indexes for segments in the + %% compaction group + MaxCount = lists:sum([ra_seq:length(S) || {_, S, _} <- All]), + %% copy live indexes from all segments in compaction group to + %% the compacting segment + {ok, CompSeg0} = ra_log_segment:open(CompactingFn, + #{max_count => MaxCount}), + CompSeg = lists:foldl( + fun ({_, Live, {F, _}}, S0) -> + {ok, S} = ra_log_segment:copy(S0, filename:join(Dir, F), + ra_seq:expand(Live)), + S + end, CompSeg0, All), + ok = ra_log_segment:close(CompSeg), + + FirstSegmentFn = filename:join(Dir, CompGroupLeaderFn), + + %% perform sym linking of the additional segments in the compaction + %% group, the target is not yet updated which can be detected at + %% recovery by the presence of a sym link _and_ the .compacting + %% file + ok = make_symlinks(Dir, FirstSegmentFn, + [F || {_, _, {F, _}} <- Additional]), + + %% rename the .compacting segment on top of the group leader + ok = prim_file:rename(CompactingFn, FirstSegmentFn), + %% finally delete the .compaction_marker file to signal + %% compaction group is complete + ok = prim_file:delete(CompactionMarker), + %% return the new segref and additional segment keys + {ra_log_segment:segref(FirstSegmentFn), + [A || {_, _, {A, _}} <- Additional]} + end || [{_Info, _, {CompGroupLeaderFn, _}} | Additional] = All + <- CompactionGroups], + + {Compacted, AddDelete} = lists:unzip(Compacted0), + + #compaction_result{unreferenced = Delete, + linked = lists:append(AddDelete), + compacted = Compacted}. + +minor_compaction(SegRefs, LiveIndexes) -> + %% identifies unreferences / unused segments with no live indexes + %% in them + Delete = lists:foldl(fun({Fn, Range}, Del) -> + case ra_seq:in_range(Range, + LiveIndexes) of + [] -> + [Fn | Del]; + _ -> + Del + end + end, [], SegRefs), + #compaction_result{unreferenced = Delete}. + +compactable_segrefs(SnapIdx, State) -> + %% TODO: provide a ra_lol:foldr API to avoid creatinga segref list + %% then filtering that + case segment_refs(State) 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 + lists:foldl(fun ({_Fn, {_Start, End}} = SR, Acc) + when End =< SnapIdx -> + [SR | Acc]; + (_, Acc) -> + Acc + end, [], Compactable) + end. + +make_symlinks(Dir, To, From) + when is_list(From) -> + [begin + SymFn = filename:join(Dir, with_ext(FromFn, ".link")), + SegFn = filename:join(Dir, with_ext(FromFn, ".segment")), + %% just in case it already exists + _ = prim_file:delete(SymFn), + %% make a symlink from the compacted target segment to a new .link + %% where the compacted indexes now can be found + ok = prim_file:make_symlink(filename:basename(To), SymFn), + %% rename to link to replace original segment + ok = prim_file:rename(SymFn, SegFn) + end || FromFn <- From], + ok. + +with_ext(Fn, Ext) when is_binary(Fn) andalso is_list(Ext) -> + <<(filename:rootname(Fn))/binary, (ra_lib:to_binary(Ext))/binary>>. + +compaction_groups([], Groups, _Conf) -> + lists:reverse(Groups); +compaction_groups(Infos, Groups, Conf) -> + case take_group(Infos, Conf, []) of + {[], RemInfos} -> + compaction_groups(RemInfos, Groups, Conf); + {Group, RemInfos} -> + compaction_groups(RemInfos, [Group | Groups], Conf) + end. + +take_group([], _, Acc) -> + {lists:reverse(Acc), []}; +take_group([{#{num_entries := NumEnts, + index_size := IdxSz, + size := Sz, + live_size := LiveSz}, Live, {_, _}} = E | Rem] = All, + #{max_count := MaxCnt, + max_size := MaxSz}, Acc) -> + NumLive = ra_seq:length(Live), + AllDataSz = Sz - IdxSz, + %% group on either num relaimable entries or data saved + case NumLive / NumEnts < 0.5 orelse + LiveSz / AllDataSz < 0.5 of + %% there are fewer than half live entries in the segment + true -> + %% check that adding this segment to the current group will no + %% exceed entry or size limits + case MaxCnt - NumLive < 0 orelse + MaxSz - LiveSz < 0 of + true -> + %% adding this segment to the group will exceed limits + %% so returning current group + {lists:reverse(Acc), All}; + false -> + take_group(Rem, #{max_count => MaxCnt - NumLive, + max_size => MaxSz - LiveSz}, + [E | Acc]) + end; + %% skip this segment + false when Acc == [] -> + take_group(Rem, #{max_count => MaxCnt, + max_size => MaxSz}, Acc); + false -> + {lists:reverse(Acc), Rem} + end. + +recover_compaction(Dir) -> + case list_files(Dir, ".compaction_group") of + [] -> + %% no pending compactions + #compaction_result{}; + [CompactionGroupFn0] -> + %% compaction recovery is needed as there is a .compaction_group file + CompactionGroupFn = filename:join(Dir, CompactionGroupFn0), + %% if corrupt, just delete .compaction_group file + {ok, Bin} = prim_file:read_file(CompactionGroupFn), + CompactionGroup = try binary_to_term(Bin) of + Group -> + Group + catch _:_ -> + %% any error just return empty + _ = prim_file:delete(CompactionGroupFn), + [] + end, + + %% there _may_ be a .compacting file + CompactingFn = filename:join(Dir, with_ext(CompactionGroupFn0, + ".compacting")), + + case CompactionGroup of + [] -> + #compaction_result{}; + [_] -> + %% single segment compaction, we cannot know if the + %% compaction into the compacting segment completed or + %% not + %% ignore return value as CompactingFn may not exist + _ = prim_file:delete(CompactingFn), + ok = prim_file:delete(CompactionGroupFn), + #compaction_result{}; + [TargetShortFn | [FstLinkSeg | _] = LinkTargets] -> + %% multiple segments in group, + %% if any of the additional segments is a symlink + %% the writes to the .compacting segment completed and we + %% can complete the compaction work + FstLinkSegFn = filename:join(Dir, FstLinkSeg), + FstLinkSegLinkFn = filename:join(Dir, with_ext(FstLinkSeg, ".link")), + Target = filename:join(Dir, TargetShortFn), + AtLeastOneLink = ra_lib:is_any_file(FstLinkSegLinkFn), + CompactingExists = ra_lib:is_any_file(CompactingFn), + case file:read_link_info(FstLinkSegFn, [raw]) of + {ok, #file_info{type = Type}} + when Type == symlink orelse + AtLeastOneLink -> + %% it is a symlink, recreate all symlinks and delete + %% compaction marker + ok = make_symlinks(Dir, Target, LinkTargets), + %% if compacting file exists, rename it to target + if CompactingExists -> + ok = prim_file:rename(CompactingFn, Target); + true -> + ok + end, + ok = prim_file:delete(CompactionGroupFn), + Compacted = [ra_log_segment:segref(Target)], + #compaction_result{compacted = Compacted, + linked = LinkTargets}; + {error, enoent} -> + %% segment does not exist indicates what exactly? + _ = prim_file:delete(CompactingFn), + ok = prim_file:delete(CompactionGroupFn), + #compaction_result{}; + {ok, #file_info{type = regular}} -> + _ = prim_file:delete(CompactingFn), + ok = prim_file:delete(CompactionGroupFn), + #compaction_result{} + end + end + end. + +seg_ref_gt({Fn1, {Start, _}}, {Fn2, {_, End}}) -> + Start > End andalso Fn1 > Fn2. + +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). + +-define(SR(N, R), {<>, R}). + +compact_seg_refs_test() -> + NewRefs = [?SR("2", {10, 100})], + PrevRefs = [?SR("2", {10, 75}), + ?SR("1", {1, 9})], + ?assertEqual([?SR("2", {10, 100}), + ?SR("1", {1, 9})], + compact_segrefs(NewRefs, PrevRefs)). + +compact_segref_3_test() -> + Data = [ + {"C", {2, 7}}, + %% this entry has overwritten the prior two + {"B", {5, 10}}, + {"A", {1, 4}} + ], + Res = compact_segrefs(Data, []), + ?assertMatch([{"C", {2, 7}}, + {"A", {1, 1}}], Res), + ok. + +compact_segref_2_test() -> + Data = [ + {"80", {80, 89}}, + %% this entry has overwritten the prior two + {"71", {56, 79}}, + {"70", {70, 85}}, + {"60", {60, 69}}, + {"50", {50, 59}} + ], + Res = compact_segrefs(Data, []), + ?assertMatch([{"80", {80, 89}}, + {"71", {56, 79}}, + {"50", {50, 55}} + ], Res), + ok. + +compact_segref_1_test() -> + Data = [ + {"80", {80, 89}}, + %% this entry has overwritten the prior one + {"71", {70, 79}}, + {"70", {70, 85}}, + %% partial overwrite + {"65", {65, 69}}, + {"60", {60, 69}}, + {"50", {50, 59}}, + {"40", {40, 49}} + ], + + Res = compact_segrefs(Data, [ + {"30", {30, 39}}, + {"20", {20, 29}} + ]), + + %% overwritten entry is no longer there + %% and the segment prior to the partial overwrite has been limited + %% to provide a continuous range + ?assertMatch([{"80", {80, 89}}, + {"71", {70, 79}}, + {"65", {65, 69}}, + {"60", {60, 64}}, + {"50", {50, 59}}, + {"40", {40, 49}}, + {"30", {30, 39}}, + {"20", {20, 29}} + ], Res), + ok. + + +segrefs_to_read_test() -> + + SegRefs = ra_lol:from_list( + fun seg_ref_gt/2, + lists:reverse( + compact_segrefs( + [{"00000006.segment", {412, 499}}, + {"00000005.segment", {284, 411}}, + %% this segment got overwritten + {"00000004.segment",{284, 500}}, + {"00000003.segment",{200, 285}}, + {"00000002.segment",{128, 255}}, + {"00000001.segment", {0, 127}}], []))), + + + ?assertEqual([{"00000002.segment", {199, 199}}, + {"00000003.segment", {200, 283}}, + {"00000005.segment", {284, 411}}, + {"00000006.segment", {412, 499}}], + segment_fold_plan(SegRefs, {199, 499}, [])), + + %% out of range + ?assertEqual([], segment_fold_plan(SegRefs, {500, 500}, [])), + ?assertEqual([ + {"00000001.segment", {127,127}}, + {"00000002.segment", {128,128}} + ], + segment_fold_plan(SegRefs, {127, 128}, [])), + ok. + +-endif. diff --git a/src/ra_log_snapshot.erl b/src/ra_log_snapshot.erl index a6ed282d3..9b1b00cb2 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 000000000..ebc78786c --- /dev/null +++ b/src/ra_log_snapshot_state.erl @@ -0,0 +1,56 @@ +-module(ra_log_snapshot_state). + +-export([ + insert/5, + delete/2, + smallest/2, + live_indexes/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 live_indexes(ets:table(), ra:uid()) -> + ra_seq:state(). +live_indexes(Table, UId) when is_binary(UId) -> + ets:lookup_element(Table, UId, 4, []). + +-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 79f1ddf3f..fa5409371 100644 --- a/src/ra_log_sup.erl +++ b/src/ra_log_sup.erl @@ -57,8 +57,7 @@ init([#{data_dir := DataDir, make_wal_conf(#{data_dir := DataDir, name := System, - names := #{wal := WalName, - segment_writer := SegWriterName} = Names} = Cfg) -> + 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,13 +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, system => System, - names => Names, 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 21aa84a8f..d94db6afb 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,16 +122,13 @@ }). -type state() :: #state{}. --type wal_conf() :: #{name := atom(), %% the name to register the wal as +-type wal_conf() :: #{names := ra_system:names(), system := atom(), - names := ra_system:names(), 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(), @@ -150,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()}. @@ -163,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}. @@ -236,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), @@ -255,23 +258,22 @@ start_link(#{name := Name} = Config) -spec init(wal_conf()) -> {ok, state()} | {stop, wal_checksum_validation_failure} | {stop, term()}. -init(#{dir := Dir, system := System} = 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 @@ -283,12 +285,12 @@ init(#{dir := Dir, system := System} = Conf0) -> ?COUNTER_FIELDS, #{ra_system => System, module => ?MODULE}), 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), @@ -302,7 +304,9 @@ init(#{dir := Dir, system := System} = 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. @@ -320,21 +324,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), @@ -355,7 +358,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), @@ -379,27 +383,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, @@ -439,7 +448,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, @@ -452,7 +461,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} -> @@ -477,7 +486,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, @@ -487,39 +496,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 for `~s`, " + "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), @@ -531,29 +543,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, @@ -567,51 +578,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), @@ -640,9 +645,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 -> @@ -653,11 +657,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) -> @@ -669,26 +674,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) -> @@ -712,18 +702,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), @@ -768,8 +757,8 @@ dump_records(<<_:1/unsigned, 0:1/unsigned, _:22/unsigned, _EntryData:0/binary, _Rest/binary>>, Entries) -> Entries; -dump_records(<<_:1/unsigned, 0:1/unsigned, _:22/unsigned, - IdDataLen:16/unsigned, _:IdDataLen/binary, +dump_records(<<_:1/unsigned, 0:1/unsigned, _Id2:22/unsigned, + IdDataLen:16/unsigned, _Id:IdDataLen/binary, Crc:32/integer, EntryDataLen:32/unsigned, Idx:64/unsigned, Term:64/unsigned, @@ -782,7 +771,7 @@ dump_records(<<_:1/unsigned, 0:1/unsigned, _:22/unsigned, _ -> exit({crc_failed_for, Idx, EntryData}) end; -dump_records(<<_:1/unsigned, 1:1/unsigned, _:22/unsigned, +dump_records(<<_:1/unsigned, 1:1/unsigned, _Id:22/unsigned, Crc:32/integer, EntryDataLen:32/unsigned, Idx:64/unsigned, Term:64/unsigned, @@ -797,9 +786,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, @@ -819,13 +809,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} -> @@ -834,20 +824,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 -> @@ -863,13 +857,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} -> @@ -878,11 +872,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; _ -> @@ -903,28 +898,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. @@ -959,7 +954,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, @@ -986,29 +980,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, @@ -1019,19 +1011,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, @@ -1054,7 +1056,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}}} @@ -1063,15 +1065,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 4c3c1d1ec..e6b9945b5 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, @@ -224,6 +225,7 @@ handle_aux/5, handle_aux/6, overview/1, + live_indexes/1, snapshot_module/0, version/0, which_module/1 @@ -289,6 +291,8 @@ -callback overview(state()) -> map(). +-callback live_indexes(state()) -> [ra:index()]. + -callback snapshot_module() -> module(). -callback version() -> version(). @@ -346,11 +350,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 df25476e5..515e6dc93 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -13,13 +13,15 @@ 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, fold/5, + fold/6, get_items/2, record_flushed/3, set_first/2, @@ -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,9 @@ -type delete_spec() :: undefined | {'<', ets:tid(), ra:index()} | {delete, ets:tid()} | - {range, ets:tid(), ra:range()}. + {indexes, ets:tid(), ra_seq:state()} | + {multi, [delete_spec()]}. + -export_type([ state/0, delete_spec/0 @@ -72,20 +77,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 +102,88 @@ 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 = []} = State) -> + %% when the indexes is empty always accept the next entry + true = ets:insert(Tid, Entry), + {ok, State#?MODULE{indexes = ra_seq:append(Idx, [])}}; +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,23 +203,6 @@ 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 lookup(ra:index(), state()) -> log_entry() | undefined. lookup(Idx, #?MODULE{staged = {FstStagedIdx, Staged}}) @@ -189,10 +215,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 +242,15 @@ 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. -spec tid_for(ra:index(), ra_term(), state()) -> undefined | ets:tid(). @@ -235,15 +265,28 @@ tid_for(Idx, Term, State) -> tid_for(Idx, Term, State#?MODULE.prev) end. +-spec fold(ra:index(), ra:index(), + fun(), term(), state(), MissingKeyStrategy :: error | return) -> + term(). +fold(From, To, Fun, Acc, State, MissingKeyStrat) + when is_atom(MissingKeyStrat) andalso + To >= From -> + case lookup(From, State) of + undefined when MissingKeyStrat == error -> + error({missing_key, From, Acc}); + undefined when MissingKeyStrat == return -> + Acc; + E -> + fold(From + 1, To, Fun, Fun(E, Acc), + State, MissingKeyStrat) + end; +fold(_From, _To, _Fun, Acc, _State, _Strat) -> + Acc. + -spec fold(ra:index(), ra:index(), fun(), term(), state()) -> term(). -fold(To, To, Fun, Acc, State) -> - E = lookup(To, State), - Fun(E, Acc); -fold(From, To, Fun, Acc, State) - when To > From -> - E = lookup(From, State), - fold(From + 1, To, Fun, Fun(E, Acc), State). +fold(From, To, Fun, Acc, State) -> + fold(From, To, Fun, Acc, State, error). -spec get_items([ra:index()], state()) -> {[log_entry()], @@ -256,8 +299,13 @@ get_items(Indexes, #?MODULE{} = State) -> non_neg_integer(). delete(undefined) -> 0; -delete({range, Tid, {Start, End}}) -> - NumToDelete = End - Start + 1, +delete({indexes, _Tid, []}) -> + 0; +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,16 +316,27 @@ 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) -> DelSpec = [{{'$1', '_', '_'}, [{'<', '$1', Idx}], [true]}], ets:select_delete(Tid, DelSpec); delete({delete, Tid}) -> + Sz = ets:info(Tid, size), true = ets:delete(Tid), - 0. + Sz; +delete({multi, Specs}) -> + lists:foldl( + fun (Spec, Acc) -> + Acc + delete(Spec) + end, 0, Specs). + + -spec range_overlap(ra:range(), state()) -> {Overlap :: ra:range(), Remainder :: ra:range()}. @@ -297,13 +356,21 @@ 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 = [], + prev = Prev}) -> + range(Prev); +range(#?MODULE{indexes = Seq, prev = Prev}) -> - PrevRange = ra_range:limit(End, range(Prev)), - ra_range:add(Range, PrevRange); + {Start, End} = Range = ra_seq:range(Seq), + case ra_range:limit(End, range(Prev)) of + undefined -> + Range; + {PrevStart, _PrevEnd} -> + ra_range:new(min(Start, PrevStart), End) + end; range(_State) -> undefined. @@ -327,35 +394,53 @@ prev(#?MODULE{prev = Prev}) -> -spec info(state()) -> map(). info(#?MODULE{tid = Tid, + indexes = Seq, prev = Prev} = State) -> #{tid => Tid, size => ets:info(Tid, size), name => ets:info(Tid, name), range => range(State), + local_range => ra_seq:range(Seq), + previous => case Prev of + undefined -> + undefined; + _ -> + info(Prev) + end, 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)}}; + + prev = Prev0, + indexes = Seq} = State) -> + End = ra_seq:last(FlushedSeq), + case ra_seq:in(End, Seq) of true -> - {{range, Tid, {Start, End}}, - State#?MODULE{range = ra_range:truncate(End, Range)}}; + %% indexes are always written in order so we can delete + %% the entire sequence preceeding, this will handle the case + %% where a segments notifications is missed + Spec0 = {indexes, Tid, ra_seq:limit(End, Seq)}, + {Spec, Prev} = case prev_set_first(End + 1, Prev0, true) of + {[], P} -> + {Spec0, P}; + {PSpecs, P} -> + {{multi, [Spec0 | PSpecs]}, P} + end, + {Spec, + State#?MODULE{indexes = ra_seq:floor(End + 1, Seq), + prev = Prev}}; 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 +452,77 @@ 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 - undefined -> - {[], undefined}; - _ -> - 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 + indexes = Seq, + prev = Prev0} = State) -> + {PrevSpecs, Prev} = prev_set_first(Idx, Prev0, Idx >= ra_seq:first(Seq)), + % 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 + % %% also upgrade if the outer seq is truncated + % %% by the set_first operation + % % case range_shallow(P) of + % case Idx >= ra_seq:first(Seq) orelse + % range_shallow(P) == undefined of + % true -> + % {[{delete, tid(P)} | Rem], + % prev(P)}; + % _ -> + % 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}. + State#?MODULE{indexes = ra_seq:floor(Idx, Seq), + prev = Prev}}. + + +%% Internal + +prev_set_first(_Idx, undefined, _Force) -> + {[], undefined}; +prev_set_first(Idx, Prev0, Force) -> + 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 + %% also upgrade if the outer seq is truncated + %% by the set_first operation + case range_shallow(P) == undefined orelse + Force of + true -> + {[{delete, tid(P)} | Rem], prev(P)}; + false -> + Res + end; + Res -> + Res + end. -delete(End, End, Tid) -> - ets:delete(Tid, End); -delete(Start, End, Tid) -> - _ = ets:delete(Tid, Start), - delete(Start+1, End, Tid). +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). @@ -434,3 +537,5 @@ read_sparse([Next | Rem] = Indexes, State, Num, Acc) -> read_sparse(Rem, State, Num + 1, [Entry | Acc]) end. +range_shallow(#?MODULE{indexes = Seq}) -> + ra_seq:range(Seq). diff --git a/src/ra_range.erl b/src/ra_range.erl index ff2342a50..f7a695e67 100644 --- a/src/ra_range.erl +++ b/src/ra_range.erl @@ -11,13 +11,15 @@ new/1, new/2, add/2, + combine/2, in/2, extend/2, limit/2, truncate/2, size/1, overlap/2, - subtract/2 + subtract/2, + fold/3 ]). @@ -25,6 +27,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 +44,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(). @@ -49,6 +60,15 @@ add(AddRange, _Range) -> %% no overlap, return add range AddRange. +-spec combine(AddRange :: range(), CurRange :: range()) -> range(). +combine(undefined, Range) -> + Range; +combine({AddStart, AddEnd}, {Start, End}) -> + {min(AddStart, Start), max(AddEnd, End)}; +combine(AddRange, _Range) -> + %% no overlap, return combine range + AddRange. + -spec in(ra:index(), range()) -> boolean(). in(_Idx, undefined) -> false; @@ -71,14 +91,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) -> @@ -119,6 +142,19 @@ subtract({_SubStart, _SubEnd} = SubRange, {Start, End} = Range) -> new(OEnd + 1, End)]] end. +-spec fold(range(), fun((ra:index(), Acc) -> Acc), Acc) -> + Acc when Acc :: term(). +fold(undefined, _Fun, Acc) -> + Acc; +fold({S, E}, Fun, Acc) -> + fold0(S, E, Fun, Acc). + +%% internals + +fold0(S, S, Fun, Acc) -> + Fun(S, Acc); +fold0(S, E, Fun, Acc) -> + fold0(S+1, E, Fun, Fun(S, Acc)). -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). @@ -196,4 +232,9 @@ extend_test() -> ?assertError({cannot_extend, 12, {1, 10}}, extend(12, {1, 10})), ok. +fold_test() -> + ?assertEqual([4,3,2,1], fold({1, 4}, fun ra_lib:cons/2, [])), + ?assertEqual([], fold(undefined, fun ra_lib:cons/2, [])), + ok. + -endif. diff --git a/src/ra_seq.erl b/src/ra_seq.erl new file mode 100644 index 000000000..8c4ff2e2d --- /dev/null +++ b/src/ra_seq.erl @@ -0,0 +1,248 @@ +%% 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) + a list:reverse/1 +-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(CeilIdxIncl, [Last | Rem]) + when is_integer(Last) andalso + Last > CeilIdxIncl -> + limit(CeilIdxIncl, Rem); +limit(CeilIdxIncl, [{_, _} = T | Rem]) -> + case ra_range:limit(CeilIdxIncl + 1, T) of + undefined -> + limit(CeilIdxIncl, Rem); + {I, I} -> + [I | Rem]; + {I, I2} when I == I2 - 1 -> + [I2, I | Rem]; + NewRange -> + [NewRange | Rem] + end; +limit(_CeilIdxIncl, Seq) -> + Seq. + +%% @doc adds two sequences together where To is +%% the "lower" sequence +-spec add(Add :: state(), To :: state()) -> state(). +add([], To) -> + To; +add(Add, []) -> + Add; +add(Add, To) -> + Fst = first(Add), + fold(fun append/2, limit(Fst - 1, To), Add). + +-spec fold(fun ((ra:index(), Acc) -> Acc), Acc, state()) -> + Acc when Acc :: term(). +fold(Fun, Acc0, Seq) -> + lists:foldr( + fun ({_, _} = Range, Acc) -> + ra_range:fold(Range, Fun, Acc); + (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)). + + +-spec in_range(ra:range(), state()) -> + state(). +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 0cc2a55f6..d1010c0f3 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -1,4 +1,3 @@ -%% 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/. %% @@ -50,8 +49,9 @@ promote_checkpoint/2, checkpoint/3, persist_last_applied/1, + peers/1, + peer_status/2, update_peer/3, - register_external_log_reader/2, update_disconnected_peers/3, handle_down/5, handle_node_status/6, @@ -98,7 +98,8 @@ queries_waiting_heartbeats := queue:queue({non_neg_integer(), consistent_query_ref()}), pending_consistent_queries := [consistent_query_ref()], - commit_latency => option(non_neg_integer()) + commit_latency => option(non_neg_integer()), + snapshot_phase => chunk_flag() }. -type state() :: ra_server_state(). @@ -144,7 +145,8 @@ {command, command()} | {commands, [command()]} | ra_log:event() | - {consistent_query, term(), ra:query_fun()} | + {consistent_query, from(), ra:query_fun()} | + {consistent_aux, from(), AuxCmd :: term()} | #heartbeat_rpc{} | #info_rpc{} | #info_reply{} | @@ -168,7 +170,10 @@ {cast, ra_server_id(), term()} | {send_vote_requests, [{ra_server_id(), #request_vote_rpc{} | #pre_vote_rpc{}}]} | - {send_rpc, ra_server_id(), #append_entries_rpc{}} | + {send_rpc, ra_server_id(), + #append_entries_rpc{} | + #heartbeat_rpc{} | + #info_rpc{}} | {send_snapshot, To :: ra_server_id(), {Module :: module(), Ref :: term(), LeaderId :: ra_server_id(), Term :: ra_term()}} | @@ -177,7 +182,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(), fun()}. -type effects() :: [effect()]. @@ -232,7 +238,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(). @@ -401,6 +408,7 @@ init(#{id := Id, maps:get(membership, Config, voter)), #{cfg => Cfg, + leader_id => undefined, current_term => CurrentTerm, cluster => Cluster0, % There may be scenarios when a single server @@ -436,11 +444,11 @@ recover(#{cfg := #cfg{log_id = LogId, {#{log := Log0, cfg := #cfg{effective_machine_version = EffMacVerAfter}} = State1, _} = apply_to(CommitIndex, - fun({Idx, _, _} = E, S0) -> + fun({_Idx, _, _} = E, S0) -> %% Clear out the effects and notifies map %% to avoid memory explosion {Mod, LastAppl, S, MacSt, _E, _N, LastTs} = apply_with(E, S0), - put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_APPLIED, Idx), + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_APPLIED, LastAppl), {Mod, LastAppl, S, MacSt, [], #{}, LastTs} end, State0, []), @@ -453,12 +461,26 @@ recover(#{cfg := #cfg{log_id = LogId, FromScan = CommitIndex + 1, {ToScan, _} = ra_log:last_index_term(Log0), ?DEBUG("~ts: scanning for cluster changes ~b:~b ", [LogId, FromScan, ToScan]), - {State, Log} = ra_log:fold(FromScan, ToScan, - fun cluster_scan_fun/2, - State1, Log0), + %% if we're recovering after a partial sparse write phase this will fail + {{LastScannedIdx, State2}, Log1} = ra_log:fold(FromScan, ToScan, + fun cluster_scan_fun/2, + {CommitIndex, State1}, Log0, + return), + + State = case LastScannedIdx < ToScan of + true -> + ?DEBUG("~ts: scan detected sparse log last scanned ~b:~b " + "resetting log to last contiguous index ~b", + [LogId, LastScannedIdx, ToScan, LastScannedIdx]), + %% the end of the log is sparse and needs to be reset + {ok, Log} = ra_log:set_last_index(LastScannedIdx, Log1), + State2#{log => Log}; + false -> + State2 + end, put_counter(Cfg, ?C_RA_SVR_METRIC_COMMIT_LATENCY, 0), - State#{log => Log, + State#{ %% reset commit latency as recovery may calculate a very old value commit_latency => 0}. @@ -476,7 +498,7 @@ handle_leader({PeerId, #append_entries_reply{term = Term, success = true, ?WARN("~ts: saw append_entries_reply from unknown peer ~w", [LogId, PeerId]), {leader, State0, []}; - Peer0 = #{match_index := MI, next_index := NI} -> + #{match_index := MI, next_index := NI} = Peer0 -> Peer = Peer0#{match_index => max(MI, LastIdx), next_index => max(NI, NextIdx)}, State1 = put_peer(PeerId, Peer, State0), @@ -602,12 +624,12 @@ handle_leader({command, Cmd}, #{cfg := #cfg{id = Self, State = State0#{condition => #{predicate_fun => fun wal_down_condition/2, transition_to => leader, - %% TODO: make duration configurable? timeout => #{duration => 5000, effects => CondEffs, transition_to => leader}}}, - {await_condition, State, Effects0}; + Effects = append_error_reply(Cmd, wal_down, Effects0), + {await_condition, State, Effects}; {not_appended, Reason, State, Effects0} -> ?WARN("~ts command ~W NOT appended to log. Reason ~w", [LogId, Cmd, 10, Reason]), @@ -771,14 +793,26 @@ handle_leader(#append_entries_rpc{leader_id = LeaderId}, handle_leader({consistent_query, From, QueryFun}, #{commit_index := CommitIndex, cluster_change_permitted := true} = State0) -> - QueryRef = {From, QueryFun, CommitIndex}, + QueryRef = {query, From, QueryFun, CommitIndex}, {State1, Effects} = make_heartbeat_rpc_effects(QueryRef, State0), {leader, State1, Effects}; handle_leader({consistent_query, From, QueryFun}, #{commit_index := CommitIndex, cluster_change_permitted := false, pending_consistent_queries := PQ} = State0) -> - QueryRef = {From, QueryFun, CommitIndex}, + QueryRef = {query, From, QueryFun, CommitIndex}, + {leader, State0#{pending_consistent_queries => [QueryRef | PQ]}, []}; +handle_leader({consistent_aux, From, AuxCmd}, + #{commit_index := CommitIndex, + cluster_change_permitted := true} = State0) -> + QueryRef = {aux, From, AuxCmd, CommitIndex}, + {State1, Effects} = make_heartbeat_rpc_effects(QueryRef, State0), + {leader, State1, Effects}; +handle_leader({consistent_aux, From, AuxCmd}, + #{commit_index := CommitIndex, + cluster_change_permitted := false, + pending_consistent_queries := PQ} = State0) -> + QueryRef = {aux, From, AuxCmd, CommitIndex}, {leader, State0#{pending_consistent_queries => [QueryRef | PQ]}, []}; %% Lihtweight version of append_entries_rpc handle_leader(#heartbeat_rpc{term = Term} = Msg, @@ -929,9 +963,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) -> @@ -1040,9 +1071,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, @@ -1141,9 +1169,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, @@ -1343,8 +1368,8 @@ handle_follower(#heartbeat_rpc{leader_id = LeaderId, {follower, State, [cast_reply(Id, LeaderId, Reply)]}; handle_follower({ra_log_event, Evt}, #{log := Log0, cfg := #cfg{id = Id}, - leader_id := LeaderId, current_term := Term} = State0) -> + LeaderId = maps:get(leader_id, State0, undefined), % forward events to ra_log % if the last written changes then send an append entries reply LW = ra_log:last_written(Log0), @@ -1438,35 +1463,50 @@ handle_follower(#install_snapshot_rpc{term = Term, meta = #{index := SnapIdx, machine_version := SnapMacVer} = Meta, leader_id = LeaderId, - chunk_state = {1, _ChunkFlag}} = Rpc, + chunk_state = {Num, ChunkFlag}} = Rpc, #{cfg := #cfg{log_id = LogId, - machine_version = MacVer}, log := Log0, + machine_version = MacVer}, + log := Log0, last_applied := LastApplied, current_term := CurTerm} = State0) when Term >= CurTerm andalso SnapIdx > LastApplied andalso %% only install snapshot if the machine version is understood - MacVer >= SnapMacVer -> + MacVer >= SnapMacVer andalso + Num =< 1 andalso + ChunkFlag /= pre -> %% only begin snapshot procedure if Idx is higher than the last_applied %% index. - ?DEBUG("~ts: begin_accept snapshot at index ~b in term ~b", - [LogId, SnapIdx, Term]), + ?DEBUG("~ts: begin_accept snapshot at index ~b in term ~b, phase ~s", + [LogId, SnapIdx, Term, ChunkFlag]), SnapState0 = ra_log:snapshot_state(Log0), {ok, SS} = ra_snapshot:begin_accept(Meta, SnapState0), - Log = ra_log:set_snapshot_state(SS, Log0), + Log1 = ra_log:set_snapshot_state(SS, Log0), + + %% if the snaphost includes pre entries (live entries) then we need + %% to reset the log to the last applied index to avoid issues + Log = case ChunkFlag of + init -> + {ok, L} = ra_log:set_last_index(LastApplied, Log1), + L; + _ -> + Log1 + end, {receive_snapshot, update_term(Term, State0#{log => Log, + snapshot_phase => ChunkFlag, leader_id => LeaderId}), [{next_event, Rpc}, {record_leader_msg, LeaderId}]}; handle_follower(#install_snapshot_rpc{term = Term, - meta = #{index := LastIndex, + meta = #{index := SnapIdx, machine_version := SnapMacVer, term := _LastTerm}}, #{cfg := #cfg{log_id = LogId, machine_version = MacVer}, last_applied := LastApplied} = State0) when MacVer >= SnapMacVer -> - ?DEBUG("~ts: install_snapshot received with lower last index ~b in ~b", - [LogId, LastIndex, Term]), + ?DEBUG("~ts: install_snapshot received with snapshot index ~b, + in ~b, local last applied index ~b ", + [LogId, SnapIdx, Term, LastApplied]), %% follower receives a snapshot for an index lower than its last applied %% index, just reply with append_entries_reply to make the leader skip %% ahead @@ -1493,9 +1533,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, @@ -1529,33 +1566,82 @@ handle_follower(Msg, State) -> handle_receive_snapshot(#install_snapshot_rpc{term = Term, meta = #{index := SnapIndex, machine_version := SnapMacVer, + cluster := ClusterIds, term := SnapTerm} = SnapMeta, chunk_state = {Num, ChunkFlag}, - data = Data}, + data = ChunkOrEntries} = Rpc, #{cfg := #cfg{id = Id, log_id = LogId, effective_machine_version = CurEffMacVer, machine_versions = MachineVersions, machine = Machine} = Cfg0, - log := Log0, + log := Log00, cluster := Cluster, current_term := CurTerm, last_applied := LastApplied, - machine_state := OldMacState} = State0) + machine_state := OldMacState, + snapshot_phase := SnapPhase} = State0) when Term >= CurTerm -> - ?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), Reply = #install_snapshot_result{term = CurTerm, last_term = SnapTerm, last_index = SnapIndex}, + SnapState0 = ra_log:snapshot_state(Log00), + %% works as an assertion also + {AcceptingSnapIdx, _} = ra_snapshot:accepting(SnapState0), case ChunkFlag of + init when SnapPhase == init andalso + SnapIndex == AcceptingSnapIdx -> + %% this is ok, just reply + %% need to set snapshot_phase to pre here as else a new snapshot + %% init could be sent without detecting this + {receive_snapshot, State0, [{reply, Reply}]}; + init -> + ?DEBUG("~ts: receiving snapshot saw unexpected init phase at snapshot " + "index term {~b, ~b}, current phase ~s restarting " + "snapshot receive phase", + [LogId, SnapIndex, SnapTerm, SnapPhase]), + %% the snapshot sending must have been interrupted and restarted + %% during the init or pre-phase + %% abort the snapshot, and revert to follower + State = abort_receive(State0), + {follower, State, [{next_event, Rpc}]}; + pre when is_list(ChunkOrEntries) -> + ?assert(SnapIndex == AcceptingSnapIdx), + % ?DEBUG("~ts: receiving snapshot chunk pre first index ~b snap index ~b, term ~b", + % [LogId, FstIdx, SnapIndex, SnapTerm]), + %% reset last index to last applied + %% as we dont know for sure indexes after last applied + %% are of the right term + {LastIdx, _} = ra_log:last_index_term(Log00), + {Log, _} = lists:foldl( + fun ({I, _, _} = E, {L0, LstIdx}) + when I > LastApplied -> + {ok, L} = ra_log:write_sparse(E, LstIdx, L0), + {L, I}; + (_, Acc) -> + %% drop any entries that are lower than last applied + Acc + end, {Log00, LastIdx}, ChunkOrEntries), + State = update_term(Term, State0#{log => Log, + snapshot_phase => pre}), + {receive_snapshot, State, [{reply, Reply}]}; + next -> + ?assert(SnapIndex == AcceptingSnapIdx), + ?DEBUG("~ts: receiving snapshot chunk: ~b / ~w, index ~b, term ~b", + [LogId, Num, ChunkFlag, SnapIndex, SnapTerm]), + SnapState = ra_snapshot:accept_chunk(ChunkOrEntries, Num, SnapState0), + Log0 = ra_log:set_snapshot_state(SnapState, Log00), + State = update_term(Term, State0#{log => Log0, + snapshot_phase => next}), + {receive_snapshot, State, [{reply, Reply}]}; last -> - %% this is the last chunk so we can "install" it - {Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm}, - SnapState, Log0), + ?assert(SnapIndex == AcceptingSnapIdx), + ?DEBUG("~ts: receiving snapshot chunk: ~b / ~w, index ~b, term ~b", + [LogId, Num, ChunkFlag, SnapIndex, SnapTerm]), + {SnapState, MacState, LiveIndexes, Effs0} = + ra_snapshot:complete_accept(ChunkOrEntries, Num, Machine, + SnapState0), + Log0 = ra_log:set_snapshot_state(SnapState, Log00), %% 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), @@ -1571,9 +1657,10 @@ 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 + {ok, Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm}, + EffMacMod, + LiveIndexes, Log0), OldServerIds = maps:map(fun (_, V) -> maps:with([voter_status], V) end, Cluster), @@ -1587,88 +1674,73 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, MacState, OldMeta, OldMacState), - State = update_term(Term, - State0#{cfg => Cfg, - log => Log, - commit_index => SnapIndex, - last_applied => SnapIndex, - %% this may not be the actual cluster index - cluster_index_term => {SnapIndex, - SnapTerm}, - cluster => make_cluster(Id, ClusterIds), - membership => get_membership(ClusterIds, State0), - machine_state => MacState}), + State1 = update_term(Term, + State0#{cfg => Cfg, + log => Log, + commit_index => SnapIndex, + last_applied => SnapIndex, + %% this may not be the actual + %% cluster index + cluster_index_term => {SnapIndex, + SnapTerm}, + cluster => make_cluster(Id, ClusterIds), + membership => + get_membership(ClusterIds, State0), + machine_state => MacState}), + State = maps:remove(snapshot_phase, State1), + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_APPLIED, SnapIndex), %% it was the last snapshot chunk so we can revert back to %% follower status {follower, persist_last_applied(State), [{reply, Reply} | - Effs ++ SnapInstalledEffs]}; - next -> - Log = ra_log:set_snapshot_state(SnapState, Log0), - State = update_term(Term, State0#{log => Log}), - {receive_snapshot, State, [{reply, Reply}]} + Effs0 ++ Effs ++ + SnapInstalledEffs]} end; handle_receive_snapshot(#append_entries_rpc{term = Term} = Msg, #{current_term := CurTerm, - cfg := #cfg{log_id = LogId}, - log := Log0} = State) + cfg := #cfg{log_id = LogId}} = State0) when Term > CurTerm -> ?INFO("~ts: follower receiving snapshot saw append_entries_rpc from ~w for term ~b " "abdicates term: ~b!", [LogId, Msg#append_entries_rpc.leader_id, Term, CurTerm]), - SnapState0 = ra_log:snapshot_state(Log0), - SnapState = ra_snapshot:abort_accept(SnapState0), - Log = ra_log:set_snapshot_state(SnapState, Log0), - {follower, update_term(Term, clear_leader_id(State#{log => Log})), - [{next_event, Msg}]}; + State = abort_receive(State0), + {follower, update_term(Term, State), [{next_event, Msg}]}; handle_receive_snapshot({ra_log_event, Evt}, - State = #{cfg := #cfg{id = _Id, log_id = LogId}, - log := Log0}) -> - ?DEBUG("~ts: ~s ra_log_event received: ~w", - [LogId, ?FUNCTION_NAME, Evt]), + #{cfg := #cfg{log_id = _LogId}, + log := Log0} = State) -> % simply forward all other events to ra_log % whilst the snapshot is being received {Log, Effects} = ra_log:handle_event(Evt, Log0), {receive_snapshot, State#{log => Log}, Effects}; -handle_receive_snapshot(receive_snapshot_timeout, #{log := Log0} = State) -> - SnapState0 = ra_log:snapshot_state(Log0), - 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(receive_snapshot_timeout, + #{cfg := #cfg{log_id = LogId}} = State0) -> + ?INFO("~ts: ~s receive snapshot timed out.", + [LogId, ?FUNCTION_NAME]), + State = abort_receive(State0), + {follower, State, []}; handle_receive_snapshot(#info_rpc{term = Term} = Msg, #{current_term := CurTerm, - cfg := #cfg{log_id = LogId}, - log := Log0} = State) + cfg := #cfg{log_id = LogId}} = State0) when CurTerm < Term -> ?INFO("~ts: follower receiving snapshot saw info_rpc from ~w for term ~b " - "abdicates term: ~b!", + "current term: ~b!", [LogId, Msg#info_rpc.from, Term, CurTerm]), - SnapState0 = ra_log:snapshot_state(Log0), - SnapState = ra_snapshot:abort_accept(SnapState0), - Log = ra_log:set_snapshot_state(SnapState, Log0), - {follower, update_term(Term, clear_leader_id(State#{log => Log})), - [{next_event, Msg}]}; + State = abort_receive(State0), + {follower, update_term(Term, State), [{next_event, Msg}]}; handle_receive_snapshot(#info_rpc{} = InfoRpc, State) -> InfoReplyEffect = empty_info_reply_effect(State, InfoRpc), {receive_snapshot, State, [InfoReplyEffect]}; handle_receive_snapshot(#info_reply{term = Term} = Msg, #{current_term := CurTerm, - cfg := #cfg{log_id = LogId}, - log := Log0} = State) + cfg := #cfg{log_id = LogId}} = State0) when CurTerm < Term -> ?INFO("~ts: follower receiving snapshot saw info_reply from ~w for term ~b " "abdicates term: ~b!", [LogId, Msg#info_reply.from, Term, CurTerm]), - SnapState0 = ra_log:snapshot_state(Log0), - SnapState = ra_snapshot:abort_accept(SnapState0), - Log = ra_log:set_snapshot_state(SnapState, Log0), - {follower, update_term(Term, clear_leader_id(State#{log => Log})), - [{next_event, Msg}]}; + State = abort_receive(State0), + {follower, update_term(Term, State), [{next_event, Msg}]}; handle_receive_snapshot(#info_reply{}, State) -> {receive_snapshot, State, []}; handle_receive_snapshot(Msg, State) -> @@ -1677,6 +1749,21 @@ handle_receive_snapshot(Msg, State) -> %% TODO: work out what else to handle {receive_snapshot, State, [{reply, {error, {unsupported_call, Msg}}}]}. +abort_receive(#{snapshot_phase := Phase, + last_applied := LastApplied, + log := Log0} = State) -> + SnapState0 = ra_log:snapshot_state(Log0), + SnapState = ra_snapshot:abort_accept(SnapState0), + Log1 = ra_log:set_snapshot_state(SnapState, Log0), + Log = case Phase of + pre -> + {ok, Log2} = ra_log:set_last_index(LastApplied, Log1), + Log2; + _ -> + Log1 + end, + clear_leader_id(maps:remove(snapshot_phase, State#{log => Log})). + -spec handle_await_condition(ra_msg(), ra_server_state()) -> {ra_state(), ra_server_state(), effects()}. handle_await_condition(#request_vote_rpc{} = Msg, State) -> @@ -1709,9 +1796,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} -> @@ -1728,9 +1812,9 @@ handle_await_condition(Msg, #{condition := #{predicate_fun := Pred} = Cond} = St {ra_server_state(), [from()]}. process_new_leader_queries(#{pending_consistent_queries := Pending, queries_waiting_heartbeats := Waiting} = State0) -> - From0 = lists:map(fun({From, _, _}) -> From end, Pending), + From0 = lists:map(fun({_, From, _, _}) -> From end, Pending), - From1 = lists:map(fun({_, {From, _, _}}) -> From end, + From1 = lists:map(fun({_, {_, From, _, _}}) -> From end, queue:to_list(Waiting)), {State0#{pending_consistent_queries => [], @@ -1826,39 +1910,52 @@ handle_aux(RaftState, Type, _Cmd, [{reply, {error, aux_handler_not_implemented}}] end, {RaftState, State0, Effects}; -handle_aux(RaftState, Type, Cmd, +handle_aux(RaftState, Type, MaybeWrappedCmd, #{cfg := #cfg{effective_machine_module = MacMod, effective_handle_aux_fun = {handle_aux, 5}}, aux_state := Aux0} = State0) -> + {Wrap, Cmd} = case MaybeWrappedCmd of + {'$wrap_reply', C} -> + {true, C}; + C -> + {false, C} + end, + %% NEW API case ra_machine:handle_aux(MacMod, RaftState, Type, Cmd, Aux0, State0) of {reply, Reply, Aux, State} -> {RaftState, State#{aux_state => Aux}, - [{reply, Reply}]}; + [{reply, wrap_reply(Wrap, Reply)}]}; {reply, Reply, Aux, State, Effects} -> {RaftState, State#{aux_state => Aux}, - [{reply, Reply} | Effects]}; + [{reply, wrap_reply(Wrap, Reply)} | Effects]}; {no_reply, Aux, State} -> {RaftState, State#{aux_state => Aux}, []}; {no_reply, Aux, State, Effects} -> {RaftState, State#{aux_state => Aux}, Effects} end; -handle_aux(RaftState, Type, Cmd, +handle_aux(RaftState, Type, MaybeWrappedCmd , #{cfg := #cfg{effective_machine_module = MacMod, effective_handle_aux_fun = {handle_aux, 6}}, aux_state := Aux0, machine_state := MacState, log := Log0} = State0) -> + {Wrap, Cmd} = case MaybeWrappedCmd of + {'$wrap_reply', C} -> + {true, C}; + C -> + {false, C} + end, %% OLD API case ra_machine:handle_aux(MacMod, RaftState, Type, Cmd, Aux0, Log0, MacState) of {reply, Reply, Aux, Log} -> {RaftState, State0#{log => Log, aux_state => Aux}, - [{reply, Reply}]}; + [{reply, wrap_reply(Wrap, Reply)}]}; {reply, Reply, Aux, Log, Effects} -> {RaftState, State0#{log => Log, aux_state => Aux}, - [{reply, Reply} | Effects]}; + [{reply, wrap_reply(Wrap, Reply)} | Effects]}; {no_reply, Aux, Log} -> {RaftState, State0#{log => Log, aux_state => Aux}, []}; {no_reply, Aux, Log, Effects} -> @@ -1869,6 +1966,11 @@ handle_aux(RaftState, Type, Cmd, % property helpers +wrap_reply(true, Cmd) -> + {wrap_reply, Cmd}; +wrap_reply(false, Cmd) -> + Cmd. + -spec id(ra_server_state()) -> ra_server_id(). id(#{cfg := #cfg{id = Id}}) -> Id. @@ -1906,13 +2008,10 @@ machine(#{cfg := #cfg{machine = Machine}}) -> machine_query(QueryFun, #{cfg := #cfg{effective_machine_module = MacMod}, machine_state := MacState, last_applied := Last, - current_term := Term - }) -> + current_term := Term}) -> Res = ra_machine:query(MacMod, QueryFun, MacState), {{Last, Term}, Res}. - - % Internal become(leader, OldRaftState, #{cluster := Cluster, @@ -2181,25 +2280,30 @@ log_fold_cache(From, _To, _Cache, Acc) -> % stores the cluster config at an index such that we can later snapshot % at this index. --spec update_release_cursor(ra_index(), - term(), ra_server_state()) -> +-spec update_release_cursor(ra_index(), 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, + {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), + {MacVersion, MacMod}, + MacState, Log0), {State#{log => Log}, Effects}. -spec promote_checkpoint(ra_index(), ra_server_state()) -> @@ -2216,7 +2320,7 @@ promote_checkpoint(Index, #{log := Log0} = State) -> -spec persist_last_applied(ra_server_state()) -> ra_server_state(). persist_last_applied(#{persisted_last_applied := PLA, last_applied := LA} = State) when LA =< PLA -> - % if last applied is less than PL for some reason do nothing + % if last applied is less than or equal to PL do nothing State; persist_last_applied(#{last_applied := LastApplied, cfg := #cfg{uid = UId} = Cfg} = State) -> @@ -2235,12 +2339,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) -> @@ -2289,20 +2387,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}; @@ -2508,6 +2606,14 @@ new_peer_with(Map) -> peers(#{cfg := #cfg{id = Id}, cluster := Peers}) -> maps:remove(Id, Peers). +peer_status(PeerId, #{cluster := Peers}) -> + case Peers of + #{PeerId := #{status := Status}} -> + Status; + _ -> + undefined + end. + %% remove any peers that are currently receiving a snapshot peers_with_normal_status(State) -> maps:filter(fun (_, #{status := normal}) -> true; @@ -2775,7 +2881,6 @@ apply_to(ApplyTo, ApplyFun, Notifys0, Effects0, erlang:system_time(millisecond) - LastTs end, %% due to machine versioning all entries may not have been applied - %% FinalEffs = make_notify_effects(Notifys, lists:reverse(Effects)), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_APPLIED, AppliedTo), put_counter(Cfg, ?C_RA_SVR_METRIC_COMMIT_LATENCY, CommitLatency), @@ -2801,16 +2906,16 @@ append_machine_effects(AppEffs, Effs) -> [AppEffs | Effs]. cluster_scan_fun({Idx, Term, {'$ra_cluster_change', _Meta, NewCluster, _}}, - State0) -> + {_, State0}) -> ?DEBUG("~ts: ~ts: applying ra cluster change to ~w", [log_id(State0), ?FUNCTION_NAME, maps:keys(NewCluster)]), %% we are recovering and should apply the cluster change - State0#{cluster => NewCluster, - membership => get_membership(NewCluster, State0), - cluster_change_permitted => true, - cluster_index_term => {Idx, Term}}; -cluster_scan_fun(_Cmd, State) -> - State. + {Idx, State0#{cluster => NewCluster, + membership => get_membership(NewCluster, State0), + cluster_change_permitted => true, + cluster_index_term => {Idx, Term}}}; +cluster_scan_fun({Idx, _, _}, {_, State}) -> + {Idx, State}. apply_with(_Cmd, {Mod, LastAppliedIdx, @@ -3280,7 +3385,8 @@ heartbeat_rpc_effects(Peers, Id, Term, QueryIndex) -> end, maps:to_list(Peers)). -heartbeat_rpc_effect_for_peer(PeerId, Peer, Id, Term, QueryIndex) -> +heartbeat_rpc_effect_for_peer(PeerId, #{status := normal} = Peer, + Id, Term, QueryIndex) -> case maps:get(query_index, Peer, 0) < QueryIndex of true -> {true, @@ -3290,7 +3396,9 @@ heartbeat_rpc_effect_for_peer(PeerId, Peer, Id, Term, QueryIndex) -> leader_id = Id}}}; false -> false - end. + end; +heartbeat_rpc_effect_for_peer(_PeerId, _Peer, _Id, _Term, _QueryIndex) -> + false. heartbeat_rpc_quorum(NewQueryIndex, PeerId, #{queries_waiting_heartbeats := Waiting0} = State) -> @@ -3305,8 +3413,10 @@ heartbeat_rpc_quorum(NewQueryIndex, PeerId, end, Waiting0), case QueryRefs of - [] -> {[], State1}; - _ -> {QueryRefs, State1#{queries_waiting_heartbeats := Waiting1}} + [] -> + {[], State1}; + _ -> + {QueryRefs, State1#{queries_waiting_heartbeats := Waiting1}} end. update_peer_query_index(PeerId, QueryIndex, #{cluster := Cluster} = State0) -> @@ -3347,24 +3457,21 @@ take_from_queue_while(Fun, Queue, Result) -> {Result, Queue} end. --spec apply_consistent_queries_effects([consistent_query_ref()], - ra_server_state()) -> - effects(). apply_consistent_queries_effects(QueryRefs, #{last_applied := LastApplied} = State) -> - lists:map(fun({_, _, ReadCommitIndex} = QueryRef) -> + lists:map(fun({_, _, _, ReadCommitIndex} = QueryRef) -> true = LastApplied >= ReadCommitIndex, - consistent_query_reply(QueryRef, State) + process_consistent_query(QueryRef, State) end, QueryRefs). --spec consistent_query_reply(consistent_query_ref(), ra_server_state()) -> effect(). -consistent_query_reply({From, QueryFun, _ReadCommitIndex}, - #{cfg := #cfg{id = Id, - machine = {machine, MacMod, _}}, - machine_state := MacState - }) -> +process_consistent_query({query, From, QueryFun, _ReadCommitIndex}, + #{cfg := #cfg{id = Id, + machine = {machine, MacMod, _}}, + machine_state := MacState}) -> Result = ra_machine:query(MacMod, QueryFun, MacState), - {reply, From, {ok, Result, Id}}. + {reply, From, {ok, Result, Id}}; +process_consistent_query({aux, From, AuxCmd, _ReadCommitIndex}, _State0) -> + {next_event, {call, From}, {aux_command, {'$wrap_reply', AuxCmd}}}. process_pending_consistent_queries(#{cluster_change_permitted := false} = State0, Effects0) -> diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index fb414023d..425a21c7d 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -12,6 +12,7 @@ -compile({inline, [handle_raft_state/3]}). +-include_lib("stdlib/include/assert.hrl"). -include("ra.hrl"). -include("ra_server.hrl"). @@ -50,7 +51,7 @@ local_state_query/3, trigger_election/2, ping/2, - log_fold/4, + read_entries/4, transfer_leadership/3, force_shrink_members_to_current_member/1 ]). @@ -146,7 +147,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{}, @@ -186,8 +188,8 @@ cast_command(ServerId, Cmd) -> cast_command(ServerId, Priority, Cmd) -> gen_statem:cast(ServerId, {command, Priority, Cmd}). --spec query(server_loc(), query_fun(), - local | consistent | leader, +-spec query(server_loc(), AuxCmd :: term() | query_fun(), + local | consistent | consistent_aux | leader, query_options(), timeout()) -> ra_server_proc:ra_leader_call_ret({ra_idxterm(), Reply :: term()}) @@ -204,11 +206,29 @@ query(ServerLoc, QueryFun, leader, Options, Timeout) query(ServerLoc, QueryFun, leader, Options, Timeout) -> leader_call(ServerLoc, {local_query, QueryFun, Options}, Timeout); query(ServerLoc, QueryFun, consistent, _Options, Timeout) -> - leader_call(ServerLoc, {consistent_query, QueryFun}, Timeout). + leader_call(ServerLoc, {consistent_query, QueryFun}, Timeout); +query(ServerLoc, AuxCmd, consistent_aux, _Options, Timeout) -> + leader_call(ServerLoc, {consistent_aux, AuxCmd}, Timeout). + +-spec read_entries(ra_server_id(), [ra:index()], + undefined | ra_flru:state(), + non_neg_integer()) -> + {ok, {map(), ra_flru:state()}} | {error, term()}. +read_entries({_, Node} = ServerId, Indexes, Flru0, Timeout) + when Node == node() -> + case local_call(ServerId, {read_entries, Indexes}, Timeout) of + {ok, ReadPlan} -> + {Reads, Flru} = ra_log:execute_read_plan(ReadPlan, Flru0, + fun (Idx, Term, Cmd) -> + {Idx, Term, Cmd} + end, + #{access_pattern => random, + file_advise => random}), + {ok, {Reads, Flru}}; + Err -> + Err + end. --spec log_fold(ra_server_id(), fun(), term(), integer()) -> term(). -log_fold(ServerId, Fun, InitialState, Timeout) -> - gen_statem:call(ServerId, {log_fold, Fun, InitialState}, Timeout). %% used to query the raft state rather than the machine state -spec state_query(server_loc(), @@ -301,18 +321,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), @@ -362,6 +378,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, @@ -373,7 +399,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]), @@ -510,7 +537,7 @@ leader(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; leader(EventType, flush_commands, #state{conf = #conf{flush_commands_size = Size}, low_priority_commands = Delayed0} = State0) -> @@ -530,8 +557,7 @@ leader(EventType, flush_commands, leader({call, _From} = EventType, {local_query, QueryFun}, State) -> leader(EventType, {local_query, QueryFun, #{}}, State); leader({call, From} = EventType, {local_query, QueryFun, Options}, State) -> - perform_or_delay_local_query( - leader, EventType, From, QueryFun, Options, State); + evaluate_local_query(leader, EventType, From, QueryFun, Options, State); leader({call, From}, {state_query, Spec}, State) -> Reply = {ok, do_state_query(Spec, State), id(State)}, {keep_state, State, [{reply, From, Reply}]}; @@ -546,6 +572,17 @@ leader({call, From}, {consistent_query, QueryFun}, ?HANDLE_EFFECTS(Effects, {call, From}, State0#state{server_state = ServerState1}), {keep_state, State1, Actions}; +leader({call, From}, {consistent_aux, AuxCmd}, + #state{conf = Conf, + server_state = ServerState0} = State0) -> + {leader, ServerState1, Effects} = + ra_server:handle_leader({consistent_aux, From, AuxCmd}, + ServerState0), + incr_counter(Conf, ?C_RA_SRV_CONSISTENT_QUERIES, 1), + {State1, Actions} = + ?HANDLE_EFFECTS(Effects, {call, From}, + State0#state{server_state = ServerState1}), + {keep_state, State1, Actions}; leader({call, From}, ping, State) -> {keep_state, State, [{reply, From, {pong, leader}}]}; leader(info, {node_event, _Node, _Evt}, State) -> @@ -556,8 +593,13 @@ leader(info, {Status, Node, InfoList}, State0) when Status =:= nodedown orelse Status =:= nodeup -> handle_node_status_change(Node, Status, InfoList, ?FUNCTION_NAME, State0); -leader(info, {update_peer, PeerId, Update}, State0) -> - State = update_peer(PeerId, Update, State0), +leader(info, {unsuspend_peer, PeerId}, State0) -> + State = case ra_server:peer_status(PeerId, State0#state.server_state) of + suspended -> + update_peer(PeerId, #{status => normal}, State0); + _ -> + State0 + end, {keep_state, State, []}; leader(_, tick_timeout, State0) -> {State1, RpcEffs} = make_rpcs(State0), @@ -567,7 +609,7 @@ leader(_, tick_timeout, State0) -> {State2, Actions} = ?HANDLE_EFFECTS(RpcEffs ++ Effects ++ [{aux, tick}], cast, State1#state{server_state = ServerState}), %% try sending any pending applied notifications again - State = send_applied_notifications(State2, #{}), + State = send_applied_notifications(#{}, State2), {keep_state, State, set_tick_timer(State, Actions)}; leader({timeout, Name}, machine_timeout, State0) -> @@ -578,8 +620,8 @@ leader({timeout, Name}, machine_timeout, State0) -> {keep_state, State, Actions}; leader({call, From}, trigger_election, State) -> {keep_state, State, [{reply, From, ok}]}; -leader({call, From}, {log_fold, Fun, Term}, State) -> - fold_log(From, Fun, Term, State); +leader({call, From}, {read_entries, Indexes}, State) -> + read_entries0(From, Indexes, State); leader(EventType, Msg, State0) -> case handle_leader(Msg, State0) of {leader, State1, Effects1} -> @@ -627,8 +669,7 @@ candidate(cast, {command, _Priority, candidate({call, _From} = EventType, {local_query, QueryFun}, State) -> candidate(EventType, {local_query, QueryFun, #{}}, State); candidate({call, From} = EventType, {local_query, QueryFun, Options}, State) -> - perform_or_delay_local_query( - candidate, EventType, From, QueryFun, Options, State); + evaluate_local_query(candidate, EventType, From, QueryFun, Options, State); candidate({call, From}, {state_query, Spec}, State) -> Reply = {ok, do_state_query(Spec, State), id(State)}, {keep_state, State, [{reply, From, Reply}]}; @@ -638,7 +679,7 @@ candidate(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; candidate({call, From}, ping, State) -> {keep_state, State, [{reply, From, {pong, candidate}}]}; candidate(info, {node_event, _Node, _Evt}, State) -> @@ -648,6 +689,8 @@ candidate(_, tick_timeout, State0) -> {keep_state, State, set_tick_timer(State, [])}; candidate({call, From}, trigger_election, State) -> {keep_state, State, [{reply, From, ok}]}; +candidate({call, From}, {read_entries, Indexes}, State) -> + read_entries0(From, Indexes, State); candidate(EventType, Msg, State0) -> case handle_candidate(Msg, State0) of {candidate, State1, Effects} -> @@ -683,8 +726,7 @@ pre_vote(cast, {command, _Priority, pre_vote({call, _From} = EventType, {local_query, QueryFun}, State) -> pre_vote(EventType, {local_query, QueryFun, #{}}, State); pre_vote({call, From} = EventType, {local_query, QueryFun, Options}, State) -> - perform_or_delay_local_query( - pre_vote, EventType, From, QueryFun, Options, State); + evaluate_local_query(pre_vote, EventType, From, QueryFun, Options, State); pre_vote({call, From}, {state_query, Spec}, State) -> Reply = {ok, do_state_query(Spec, State), id(State)}, {keep_state, State, [{reply, From, Reply}]}; @@ -694,7 +736,7 @@ pre_vote(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; pre_vote({call, From}, ping, State) -> {keep_state, State, [{reply, From, {pong, pre_vote}}]}; pre_vote(info, {node_event, _Node, _Evt}, State) -> @@ -709,6 +751,8 @@ pre_vote(_, tick_timeout, State0) -> {keep_state, State, set_tick_timer(State, [])}; pre_vote({call, From}, trigger_election, State) -> {keep_state, State, [{reply, From, ok}]}; +pre_vote({call, From}, {read_entries, Indexes}, State) -> + read_entries0(From, Indexes, State); pre_vote(EventType, Msg, State0) -> case handle_pre_vote(Msg, State0) of {pre_vote, State1, Effects} -> @@ -773,8 +817,7 @@ follower(cast, {command, _Priority, follower({call, _From} = EventType, {local_query, QueryFun}, State) -> follower(EventType, {local_query, QueryFun, #{}}, State); follower({call, From} = EventType, {local_query, QueryFun, Options}, State) -> - perform_or_delay_local_query( - follower, EventType, From, QueryFun, Options, State); + evaluate_local_query(follower, EventType, From, QueryFun, Options, State); follower({call, From}, {state_query, Spec}, State) -> Reply = {ok, do_state_query(Spec, State), id(State)}, {keep_state, State, [{reply, From, Reply}]}; @@ -784,7 +827,7 @@ follower(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; follower({call, From}, trigger_election, State) -> ?DEBUG("~ts: election triggered by ~w", [log_id(State), element(1, From)]), {keep_state, State, [{reply, From, ok}, @@ -850,8 +893,8 @@ follower(_, tick_timeout, #state{server_state = ServerState0} = State0) -> {State, Actions} = ?HANDLE_EFFECTS([{aux, tick}], cast, State0#state{server_state = ServerState}), {keep_state, State, set_tick_timer(State, Actions)}; -follower({call, From}, {log_fold, Fun, Term}, State) -> - fold_log(From, Fun, Term, State); +follower({call, From}, {read_entries, Indexes}, State) -> + read_entries0(From, Indexes, State); follower(EventType, Msg, #state{conf = #conf{name = Name}, server_state = SS0} = State0) -> case handle_follower(Msg, State0) of @@ -898,8 +941,8 @@ receive_snapshot({call, _From} = EventType, {local_query, QueryFun}, State) -> receive_snapshot(EventType, {local_query, QueryFun, #{}}, State); receive_snapshot({call, From} = EventType, {local_query, QueryFun, Options}, State) -> - perform_or_delay_local_query( - receive_snapshot, EventType, From, QueryFun, Options, State); + evaluate_local_query(receive_snapshot, EventType, From, QueryFun, + Options, State); receive_snapshot({call, From}, {state_query, Spec}, State) -> Reply = {ok, do_state_query(Spec, State), id(State)}, {keep_state, State, [{reply, From, Reply}]}; @@ -909,7 +952,15 @@ receive_snapshot(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; +receive_snapshot(info, {'DOWN', MRef, process, _Pid, _Info}, + #state{leader_monitor = MRef} = State) -> + %% leader is down + ?INFO("~ts: receive_snapshot - Leader monitor down. Aborting snapshot receive. " + "Entering follower state.", + [log_id(State)]), + receive_snapshot(info, receive_snapshot_timeout, + State#state{leader_monitor = undefined}); receive_snapshot(EventType, Msg, State0) -> case handle_receive_snapshot(Msg, State0) of {receive_snapshot, State1, Effects} -> @@ -918,8 +969,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, @@ -999,8 +1051,8 @@ await_condition({call, _From} = EventType, {local_query, QueryFun}, State) -> await_condition(EventType, {local_query, QueryFun, #{}}, State); await_condition({call, From} = EventType, {local_query, QueryFun, Options}, State) -> - perform_or_delay_local_query( - await_condition, EventType, From, QueryFun, Options, State); + evaluate_local_query(await_condition, EventType, From, QueryFun, + Options, State); await_condition({call, From}, {state_query, Spec}, State) -> Reply = {ok, do_state_query(Spec, State), id(State)}, {keep_state, State, [{reply, From, Reply}]}; @@ -1010,7 +1062,7 @@ await_condition(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; await_condition({call, From}, ping, State) -> {keep_state, State, [{reply, From, {pong, await_condition}}]}; await_condition({call, From}, trigger_election, State) -> @@ -1039,6 +1091,8 @@ await_condition(info, {Status, Node, InfoList}, State0) await_condition(_, tick_timeout, State0) -> {State, Actions} = ?HANDLE_EFFECTS([{aux, tick}], cast, State0), {keep_state, State, set_tick_timer(State, Actions)}; +await_condition({call, From}, {read_entries, Indexes}, State) -> + read_entries0(From, Indexes, State); await_condition(EventType, Msg, State0) -> case handle_await_condition(Msg, State0) of {follower, State1, Effects} -> @@ -1234,19 +1288,12 @@ handle_receive_snapshot(Msg, State) -> handle_await_condition(Msg, State) -> handle_raft_state(?FUNCTION_NAME, Msg, State). -perform_or_delay_local_query( - RaftState, EventType, From, QueryFun, Options, State0) -> - {NextState, State1, Effects} = do_perform_or_delay_local_query( - RaftState, From, QueryFun, Options, - State0), - {State, Actions} = handle_effects(RaftState, Effects, EventType, State1), - {NextState, State, Actions}. - -do_perform_or_delay_local_query( - RaftState, From, QueryFun, Options, - #state{conf = Conf, - server_state = #{cfg := #cfg{id = ThisMember}} = ServerState, - pending_queries = PendingQueries} = State) -> +evaluate_local_query(RaftState, EventType, From, QueryFun, Options, + #state{conf = Conf, + server_state = + #{cfg := #cfg{id = ThisMember}} = ServerState, + pending_queries = PendingQueries} = State0) -> + %% The caller might decide it wants the query to be executed only after a %% specific index has been applied on the local node. It can specify that %% with the `condition' option. @@ -1257,21 +1304,25 @@ do_perform_or_delay_local_query( %% If the condition is set to `{applied, {Index, Term}}', the query is %% added to a list of pending queries. It will be evaluated once that %% index is applied locally. - case maps:get(condition, Options, undefined) of - undefined -> - Leader = determine_leader(RaftState, State), - Reply = perform_local_query(QueryFun, Leader, ServerState, Conf), - {keep_state, State, [{reply, From, Reply, {member, ThisMember}}]}; - Condition -> - PendingQuery = {Condition, From, QueryFun}, - PendingQueries1 = [PendingQuery | PendingQueries], - State1 = State#state{pending_queries = PendingQueries1}, - %% It's possible that the specified index was already applied. - %% That's why we evaluate pending queries just after adding the - %% query to the list. - {State2, Actions} = perform_pending_queries(RaftState, State1), - {keep_state, State2, Actions} - end. + {State3, Effects} = + case maps:get(condition, Options, undefined) of + undefined -> + Leader = determine_leader(RaftState, State0), + Reply = perform_local_query(QueryFun, Leader, + ServerState, Conf), + {State0, [{reply, From, Reply, {member, ThisMember}}]}; + Condition -> + PendingQuery = {Condition, From, QueryFun}, + PendingQueries1 = [PendingQuery | PendingQueries], + State1 = State0#state{pending_queries = PendingQueries1}, + %% It's possible that the specified index was already applied. + %% That's why we evaluate pending queries just after adding the + %% query to the list. + {State2, Effects0} = perform_pending_queries(RaftState, State1), + {State2, Effects0} + end, + {State, Actions} = handle_effects(RaftState, Effects, EventType, State3), + {keep_state, State, Actions}. perform_pending_queries(_RaftState, #state{pending_queries = []} = State) -> {State, []}; @@ -1285,25 +1336,26 @@ perform_pending_queries(RaftState, LastApplied, pending_queries = PendingQueries0} = State0, Actions0) -> Leader = determine_leader(RaftState, State0), - {PendingQueries, - Actions, - ServerState} = lists:foldr( - fun(PendingQuery, Acc) -> - perform_pending_queries1( - PendingQuery, Acc, - #{last_applied => LastApplied, - leader => Leader, - conf => Conf}) - end, {[], Actions0, ServerState0}, PendingQueries0), + {PendingQueries, Actions, ServerState} = + lists:foldr( + fun (PendingQuery, Acc) -> + perform_pending_queries1(PendingQuery, Acc, + #{last_applied => LastApplied, + leader => Leader, + conf => Conf}) + end, {[], Actions0, ServerState0}, PendingQueries0), State = State0#state{server_state = ServerState, pending_queries = PendingQueries}, {State, Actions}. -perform_pending_queries1( - {{applied, {TargetIndex, TargetTerm}}, From, QueryFun} = PendingQuery, - {PendingQueries0, Actions0, #{cfg := #cfg{id = ThisMember}} = ServerState0}, - #{last_applied := LastApplied, leader := Leader, conf := Conf}) +perform_pending_queries1({{applied, {TargetIndex, TargetTerm}}, From, QueryFun} + = PendingQuery, + {PendingQueries0, Actions0, #{cfg := #cfg{id = ThisMember}} = ServerState0}, + #{last_applied := LastApplied, + leader := Leader, + conf := Conf}) when TargetIndex =< LastApplied -> + %% NB: this could mean a disk seek + reed {Term, ServerState} = ra_server:fetch_term(TargetIndex, ServerState0), case Term of TargetTerm -> @@ -1321,9 +1373,8 @@ perform_pending_queries1( PendingQueries = [PendingQuery | PendingQueries0], {PendingQueries, Actions0, ServerState} end; -perform_pending_queries1( - PendingQuery, - {PendingQueries0, Actions, ServerState}, _Context) -> +perform_pending_queries1(PendingQuery, + {PendingQueries0, Actions, ServerState}, _Context) -> PendingQueries = [PendingQuery | PendingQueries0], {PendingQueries, Actions, ServerState}. @@ -1367,13 +1418,15 @@ handle_effects(RaftState, Effects0, EvtType, State0, Actions0) -> {State, lists:reverse(Actions)}. handle_effect(_RaftState, {send_rpc, To, Rpc}, _, - #state{conf = Conf} = State0, Actions) -> + #state{conf = Conf, + server_state = SS} = State0, Actions) -> % fully qualified use only so that we can mock it for testing % TODO: review / refactor to remove the mod call here + PeerStatus = ra_server:peer_status(To, SS), case ?MODULE:send_rpc(To, Rpc, State0) of ok -> {State0, Actions}; - nosuspend -> + nosuspend when PeerStatus == normal -> %% update peer status to suspended and spawn a process %% to send the rpc without nosuspend so that it will block until %% the data can get through @@ -1384,9 +1437,13 @@ handle_effect(_RaftState, {send_rpc, To, Rpc}, _, %% the peer status back to normal ok = gen_statem:cast(To, Rpc), incr_counter(Conf, ?C_RA_SRV_MSGS_SENT, 1), - Self ! {update_peer, To, #{status => normal}} + Self ! {unsuspend_peer, To} end), + % ?DEBUG("~ts: temporarily suspending peer ~w due to full distribution buffer ~W", + % [log_id(State0), To, Rpc, 5]), {update_peer(To, #{status => suspended}, State0), Actions}; + nosuspend -> + {State0, Actions}; noconnect -> %% for noconnects just allow it to pipeline and catch up later {State0, Actions} @@ -1462,7 +1519,7 @@ handle_effect(RaftState, {aux, Cmd}, EventType, State0, Actions0) -> {State, Actions0 ++ Actions}; handle_effect(leader, {notify, Nots}, _, #state{} = State0, Actions) -> %% should only be done by leader - State = send_applied_notifications(State0, Nots), + State = send_applied_notifications(Nots, State0), {State, Actions}; handle_effect(_AnyState, {cast, To, Msg}, _, State, Actions) -> %% TODO: handle send failure @@ -1505,22 +1562,26 @@ handle_effect(_RaftState, {reply, Reply}, {call, From}, State, Actions) -> {State, Actions}; handle_effect(_RaftState, {reply, _From, _Reply}, _EvtType, State, Actions) -> {State, Actions}; -handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, Id, Term}}, _, +handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, _Id, Term}}, _, #state{server_state = SS0, monitors = Monitors, conf = #conf{snapshot_chunk_size = ChunkSize, - install_snap_rpc_timeout = InstallSnapTimeout} = Conf} = State0, + log_id = LogId, + install_snap_rpc_timeout = InstallSnapTimeout} = Conf} + = State0, Actions) -> case lists:member(ToNode, [node() | nodes()]) of true -> %% node is connected %% leader effect only - Self = self(), Machine = ra_server:machine(SS0), + %% temporary assertion + #{To := #{status := normal}} = ra_server:peers(SS0), + Id = ra_server:id(SS0), Pid = spawn(fun () -> - try send_snapshots(Self, Id, Term, To, + try send_snapshots(Id, Term, To, ChunkSize, InstallSnapTimeout, - SnapState, Machine) of + SnapState, Machine, LogId) of _ -> ok catch C:timeout:S -> @@ -1548,7 +1609,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) @@ -1639,6 +1700,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}. @@ -1869,36 +1935,95 @@ send(To, Msg, Conf) -> Res end. -fold_log(From, Fun, Term, State) -> - case ra_server:log_fold(State#state.server_state, Fun, Term) of - {ok, Result, ServerState} -> - {keep_state, State#state{server_state = ServerState}, - [{reply, From, {ok, Result}}]} - end. +read_entries0(From, Idxs, #state{server_state = #{log := Log}} = State) -> + ReadState = ra_log:partial_read(Idxs, Log, fun (Idx, Term, Cmd) -> + {Idx, Term, Cmd} + end), + {keep_state, State, [{reply, From, {ok, ReadState}}]}. -send_snapshots(Me, Id, Term, {_, ToNode} = To, ChunkSize, - InstallTimeout, SnapState, Machine) -> +send_snapshots(Id, Term, {_, ToNode} = To, ChunkSize, + InstallTimeout, SnapState, Machine, LogId) -> Context = ra_snapshot:context(SnapState, ToNode), {ok, #{machine_version := SnapMacVer} = Meta, ReadState} = ra_snapshot:begin_read(SnapState, Context), - %% only send the snapshot if the target server can accept it + %% TODO: consolidate getting the context, machinve version and last + %% applied index in one rpc, and handle errors TheirMacVer = erpc:call(ToNode, ra_machine, version, [Machine]), + %% only send the snapshot if the target server can accept it case SnapMacVer > TheirMacVer of true -> + ?DEBUG("~ts: not sending snapshot to ~w as their machine version ~b " + "is lower than snapshot machine version ~b", + [LogId, To, TheirMacVer, SnapMacVer]), ok; false -> + %% TODO: this could be stale, replace with a call into the + %% process insted perhaps? + #{last_applied := LastApplied} = erpc:call(ToNode, + ra_counters, + counters, + [To, [last_applied]]), RPC = #install_snapshot_rpc{term = Term, leader_id = Id, + chunk_state = {0, init}, meta = Meta}, - Result = read_chunks_and_send_rpc(RPC, To, ReadState, 1, - ChunkSize, InstallTimeout, SnapState), - ok = gen_statem:cast(Me, {To, Result}) + case ra_snapshot:indexes( + ra_snapshot:current_snapshot_dir(SnapState)) of + {ok, [_|_] = Indexes0} -> + %% remove all indexes lower than the target's last applied + Indexes = ra_seq:floor(LastApplied + 1, Indexes0), + ?DEBUG("~ts: sending ~b live indexes in the range ~w to ~w ", + [LogId, ra_seq:length(Indexes), ra_seq:range(Indexes), To]), + %% first send the init phase + Res0 = gen_statem:call(To, RPC, + {dirty_timeout, InstallTimeout}), + %% this asserts we should continue, + %% anything else should crash the process + ?assert(is_record(Res0, install_snapshot_result)), + %% there are live indexes to send before the snapshot + %% TODO: write ra_seq:list_chunk function to avoid expansion + Idxs = lists:reverse(ra_seq:expand(Indexes)), + Flru = lists:foldl( + fun (Is, F0) -> + {ok, {Ents0, F}} = + ra_server_proc:read_entries(Id, Is, F0, 5000), + Ents = [map_get(I, Ents0) || I <- Is], + RPC1 = RPC#install_snapshot_rpc{chunk_state = {0, pre}, + data = Ents}, + Res1 = gen_statem:call(To, RPC1, + {dirty_timeout, InstallTimeout}), + %% this asserts we should continue, + %% anything else should crash the process + ?assert(is_record(Res1, install_snapshot_result)), + F + end, undefined, ra_lib:lists_chunk(16, Idxs)), + _ = ra_flru:evict_all(Flru), + ok; + _ -> + ok + end, + %% send install sntaphost RPC with entries here + %% Read indexes for snapshot, if non-empty + %% Find out what the follower's last applied index is + %% + %% Call into `Id' to do sparse read of some chunk of indexes + %% begin rpc with gen_statem:send_request/2 + %% while waiting for reply call into `Id' to get the next chunk + %% of entries + %% wait for response + %% send again, etc + Result = send_snapshot_chunks(RPC, To, ReadState, 1, + ChunkSize, InstallTimeout, + SnapState), + ?DEBUG("~ts: sending snapshot to ~w completed", + [LogId, To]), + ok = gen_statem:cast(Id, {To, Result}) end. -read_chunks_and_send_rpc(RPC0, - To, ReadState0, Num, ChunkSize, InstallTimeout, SnapState) -> +send_snapshot_chunks(RPC0, To, ReadState0, Num, ChunkSize, + InstallTimeout, SnapState) -> {ok, Data, ContState} = ra_snapshot:read_chunk(ReadState0, ChunkSize, SnapState), ChunkFlag = case ContState of @@ -1913,7 +2038,7 @@ read_chunks_and_send_rpc(RPC0, {dirty_timeout, InstallTimeout}), case ContState of {next, ReadState1} -> - read_chunks_and_send_rpc(RPC0, To, ReadState1, Num + 1, + send_snapshot_chunks(RPC0, To, ReadState1, Num + 1, ChunkSize, InstallTimeout, SnapState); last -> Res1 @@ -2048,6 +2173,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, @@ -2099,13 +2229,13 @@ update_peer(PeerId, Update, State0#state{server_state = ra_server:update_peer(PeerId, Update, ServerState)}. -send_applied_notifications(#state{pending_notifys = PendingNots} = State, - Nots0) when map_size(PendingNots) > 0 -> +send_applied_notifications(Nots0, #state{pending_notifys = PendingNots} = State) + when map_size(PendingNots) > 0 -> Nots = maps:merge_with(fun(_K, V1, V2) -> V1 ++ V2 end, PendingNots, Nots0), - send_applied_notifications(State#state{pending_notifys = #{}}, Nots); -send_applied_notifications(#state{} = State, Nots) -> + send_applied_notifications(Nots, State#state{pending_notifys = #{}}); +send_applied_notifications(Nots, #state{} = State) -> Id = id(State), %% any notifications that could not be sent %% will be kept and retried diff --git a/src/ra_server_sup.erl b/src/ra_server_sup.erl index d4d85d1d0..9346e416b 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 c3869c918..c9c6b52e8 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -8,6 +8,7 @@ -include("ra.hrl"). +-include_lib("stdlib/include/assert.hrl"). -type file_err() :: ra_lib:file_err(). %% alias @@ -28,26 +29,32 @@ 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, + accept_chunk/3, + complete_accept/4, abort_accept/1, 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. @@ -65,6 +72,7 @@ -record(accept, {%% the next expected chunk next = 1 :: non_neg_integer(), state :: term(), + machine_version :: non_neg_integer(), idxterm :: ra_idxterm()}). -record(?MODULE, @@ -79,7 +87,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 +200,22 @@ 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}), + {ok, #{index := Idx, + term := Term}} = Module:read_meta(Current), + %% recover live indexes and record that + {ok, Indexes} = indexes(Current), + SmallestLiveIdx = case ra_seq:first(Indexes) of + undefined -> + Idx+1; + First -> + First + end, + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, SmallestLiveIdx, + Indexes), ok = delete_snapshots(SnapshotsDir, lists:delete(Current0, Snaps)), %% delete old snapshots if any @@ -329,7 +347,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 +363,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(), + MacState :: 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,14 +390,33 @@ 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 %% 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), + PostPrepareEqualsMacState = Ref == MacState, + LiveIndexes0 = case PostPrepareEqualsMacState of + false -> + ra_seq:from_list( + ra_machine:live_indexes(MacMod, MacState)); + true -> + [] + end, %% write the snapshot in a separate process Self = self(), - Pid = spawn(fun () -> + IdxTerm = {Idx, Term}, + BgWorkFun = fun () -> ok = ra_lib:make_dir(SnapDir), + %% if the Ref returned by ra_snapshot:prepare/2 is + %% the same as the mac state then indexes can be + %% calculated here + LiveIndexes = + case PostPrepareEqualsMacState of + true -> + ra_seq:from_list( + ra_machine:live_indexes(MacMod, Ref)); + false -> + LiveIndexes0 + end, case Mod:write(SnapDir, Meta, Ref, Sync) of ok -> ok; {ok, BytesWritten} -> @@ -386,15 +424,24 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind, BytesWritten), ok end, + + 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 +458,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 = ra_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 = ra_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,63 +503,110 @@ 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, + %% 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]}. -spec begin_accept(meta(), state()) -> {ok, state()}. -begin_accept(#{index := Idx, term := Term} = Meta, +begin_accept(#{index := Idx, + machine_version := SnapMacVer, + term := Term} = Meta, #?MODULE{module = Mod, snapshot_directory = Dir} = State) -> SnapDir = make_snapshot_dir(Dir, Idx, Term), ok = ra_lib:make_dir(SnapDir), {ok, AcceptState} = Mod:begin_accept(SnapDir, Meta), {ok, State#?MODULE{accepting = #accept{idxterm = {Idx, Term}, + machine_version = SnapMacVer, state = AcceptState}}}. --spec accept_chunk(term(), non_neg_integer(), chunk_flag(), state()) -> - {ok, state()}. -accept_chunk(Chunk, Num, last, - #?MODULE{uid = UId, - module = Mod, - snapshot_directory = Dir, - current = Current, - accepting = #accept{next = Num, - idxterm = {Idx, _} = IdxTerm, - state = AccState}} = State) -> +-spec complete_accept(Chunk :: term(), Num :: non_neg_integer(), + Machine :: ra_machine:machine(), state()) -> + {state(), MacState :: term(), ra_seq:state(), [effect()]}. +complete_accept(Chunk, Num, Machine, + #?MODULE{uid = UId, + module = Mod, + snapshot_directory = Dir, + current = Current, + pending = Pending, + accepting = #accept{next = Num, + idxterm = {Idx, Term} = IdxTerm, + state = AccState}} = State0) -> %% last chunk ok = Mod:complete_accept(Chunk, AccState), %% run validate here? %% delete the current snapshot if any - _ = spawn(fun () -> delete(Dir, Current) end), - %% update ets table - true = ets:insert(?ETSTBL, {UId, Idx}), - {ok, State#?MODULE{accepting = undefined, - %% reset any pending snapshot writes - pending = undefined, - current = IdxTerm}}; -accept_chunk(Chunk, Num, next, - #?MODULE{module = Mod, - accepting = - #accept{state = AccState0, - next = Num} = Accept} = State) -> + Dels = case Pending of + undefined -> + [Current]; + {PendIdxTerm, _} -> + [Current, PendIdxTerm] + end, + Eff = {bg_work, + fun() -> [delete(Dir, Del) || Del <- Dels] end, + fun (_) -> ok end}, + State = State0#?MODULE{accepting = undefined, + %% reset any pending snapshot writes + pending = undefined, + current = IdxTerm}, + {ok, #{machine_version := SnapMacVer}, MacState} = recover(State), + SnapMacMod = ra_machine:which_module(Machine, SnapMacVer), + %% TODO: allow the ra machine to return a re_seq instead of a plain list + LiveIndexes = ra_seq:from_list( + ra_machine:live_indexes(SnapMacMod, MacState)), + SnapDir = make_snapshot_dir(Dir, Idx, Term), + ok = write_indexes(SnapDir, LiveIndexes), + %% delete accepting marker file + AcceptMarker = filename:join(SnapDir, <<"accepting">>), + _ = prim_file:delete(AcceptMarker), + %% assert accepting marker is no longer there + ?assertNot(filelib:is_file(AcceptMarker)), + SmallestIdx = case ra_seq:first(LiveIndexes) of + undefined -> + Idx + 1; + I -> + I + end, + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, SmallestIdx, + LiveIndexes), + {State, MacState, LiveIndexes, [Eff]}. + +-spec accept_chunk(Chunk :: term(), Num :: non_neg_integer(), state()) -> + state(). +accept_chunk(Chunk, Num, #?MODULE{module = Mod, + accepting = + #accept{state = AccState0, + next = Num} = Accept} = State) -> {ok, AccState} = Mod:accept_chunk(Chunk, AccState0), - {ok, State#?MODULE{accepting = Accept#accept{state = AccState, - next = Num + 1}}}; -accept_chunk(_Chunk, Num, _ChunkFlag, + State#?MODULE{accepting = Accept#accept{state = AccState, + next = Num + 1}}; +accept_chunk(_Chunk, Num, #?MODULE{accepting = #accept{next = Next}} = State) when Next > Num -> %% this must be a resend - we can just ignore it - {ok, State}. + State. -spec abort_accept(state()) -> state(). abort_accept(#?MODULE{accepting = undefined} = State) -> @@ -527,27 +629,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 +753,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 +801,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 303b84806..6b2080cab 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 000000000..313755d97 --- /dev/null +++ b/src/ra_worker.erl @@ -0,0 +1,89 @@ +%% 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). + +-include("ra.hrl"). +-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, {log_id = "" :: unicode:chardata() + }). + +%%% 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(#{id := Id} = Config) when is_map(Config) -> + process_flag(trap_exit, true), + LogId = maps:get(friendly_name, Config, + lists:flatten(io_lib:format("~w", [Id]))), + {ok, #state{log_id = LogId}}. + +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 -> + ?WARN("~ts: worker encounted error ~0p of type ~s, Stack:~n~p", + [State#state.log_id, Err, Type, Stack]), + ErrFun({Type, Err}), + ok + end; + _ when is_function(FunOrMfa) -> + try FunOrMfa() of + _ -> + ok + catch Type:Err:Stack -> + ?WARN("~ts: worker encounted error ~0p of type ~s, Stack:~n~p", + [State#state.log_id, Err, Type, Stack]), + 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 144e282b0..13e8c435a 100644 --- a/test/coordination_SUITE.erl +++ b/test/coordination_SUITE.erl @@ -55,7 +55,8 @@ all_tests() -> segment_writer_or_wal_crash_leader, server_recovery_strategy, stopped_wal_causes_leader_change_registered, - stopped_wal_causes_leader_change_mfa + stopped_wal_causes_leader_change_mfa, + ra_kv ]. groups() -> @@ -388,7 +389,7 @@ shrink_cluster_with_snapshot(Config) -> %% resume activity ok PrivDir = ?config(data_dir, Config), ClusterName = ?config(cluster_name, Config), - Peers = start_peers([s1,s2,s3], PrivDir), + Peers = start_peers([s1, s2, s3], PrivDir), ServerIds = server_ids(ClusterName, Peers), [_A, _B, _C] = ServerIds, @@ -410,7 +411,6 @@ shrink_cluster_with_snapshot(Config) -> exit(Pid, kill), {ok, _, _} = ra:remove_member(Leader1, Leader1), - timer:sleep(500), {ok, _, Leader2} = ra:members(ServerIds), @@ -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), @@ -1243,7 +1243,7 @@ stopped_wal_causes_leader_change(Config, RecoverStrat) -> #{term := T} = ra:key_metrics(Follower), T > Term andalso (begin - P = ct_rpc:call(LeaderNode, erlang, whereis, [LeaderName]),% [ra_log_wal]), + P = ct_rpc:call(LeaderNode, erlang, whereis, [LeaderName]), is_pid(P) andalso P =/= LeaderPid end) end, 200), @@ -1251,6 +1251,26 @@ stopped_wal_causes_leader_change(Config, RecoverStrat) -> stop_peers(Peers), ok. +ra_kv(Config) -> + PrivDir = ?config(data_dir, Config), + ClusterName = ?config(cluster_name, Config), + Peers = start_peers([s1,s2,s3], PrivDir), + ServerIds = server_ids(ClusterName, Peers), + {ok, Started, []} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, #{members => ServerIds}), + %% synchronously get leader + {ok, _, Leader} = ra:members(hd(Started)), + {ok, _} = ra_kv:put(Leader, <<"k1">>, <<"value1">>, 5000), + + %% roll wall on all nodeso + [ok = erpc:call(N, ra_log_wal, force_roll_over, [ra_log_wal]) + || {_, N} <- ServerIds], + timer:sleep(100), + [{ok, _, <<"value1">>} = ra_kv:get(ServerId, <<"k1">>, 5000) + || ServerId <- ServerIds], + + stop_peers(Peers), + ok. + %% Utility test_local_msg(Leader, ReceiverNode, ExpectedSenderNode, CmdTag, Opts0) -> @@ -1432,8 +1452,6 @@ snapshot_installed(#{machine_version := _, node_setup(DataDir) -> ok = ra_lib:make_dir(DataDir), - % NodeDir = filename:join(DataDir, atom_to_list(node())), - % ok = ra_lib:make_dir(DataDir), LogFile = filename:join(DataDir, "ra.log"), SaslFile = filename:join(DataDir, "ra_sasl.log"), logger:set_primary_config(level, debug), diff --git a/test/ra_2_SUITE.erl b/test/ra_2_SUITE.erl index 349f84f6c..cb01be157 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, @@ -648,33 +646,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 d16812352..fd89299c3 100644 --- a/test/ra_SUITE.erl +++ b/test/ra_SUITE.erl @@ -131,7 +131,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), @@ -698,6 +697,7 @@ wait_for_applied(Msg) -> false -> wait_for_applied(Msg) end after 10000 -> + flush(), error({timeout_waiting_for_applied, Msg}) end. @@ -777,8 +777,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)], @@ -791,15 +791,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), @@ -818,11 +818,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), @@ -840,12 +840,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 @@ -855,9 +855,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), @@ -1486,13 +1486,16 @@ overviews(Node) -> [ra:member_overview(P) || {_, _} = P <- Members]. voters({ok, #{cluster := Peers}, _} = _Overview) -> - [Id || {Id, Status} <- maps:to_list(Peers), maps:get(membership, Status, voter) == voter]. + [Id || {Id, Status} <- maps:to_list(Peers), + maps:get(membership, Status, voter) == voter]. %% machine impl init(_) -> 0. apply(_Meta, Num, State) -> {Num + State, Num + State}. +%% end machine impl + gather_applied([], Timeout) -> %% have a longer timeout first %% as we assume we expect to receive at least one ra_event diff --git a/test/ra_checkpoint_SUITE.erl b/test/ra_checkpoint_SUITE.erl index 8640d7f51..375be3cb3 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_kv_SUITE.erl b/test/ra_kv_SUITE.erl new file mode 100644 index 000000000..2e18778e1 --- /dev/null +++ b/test/ra_kv_SUITE.erl @@ -0,0 +1,263 @@ +-module(ra_kv_SUITE). + +-compile(nowarn_export_all). +-compile(export_all). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-define(SYS, default). + +%%%=================================================================== +%%% Common Test callbacks +%%%=================================================================== + +all() -> + [ + {group, tests} + ]. + + +all_tests() -> + [ + basics, + snapshot_replication, + snapshot_replication_interrupted + ]. + +groups() -> + [ + {tests, [], all_tests()} + ]. + +init_per_suite(Config) -> + %% as we're not starting the ra application and we want the logs + ra_env:configure_logger(logger), + {ok, _} = ra:start_in(?config(priv_dir, Config)), + Config. + +end_per_suite(_Config) -> + application:stop(ra), + ok. + +init_per_group(_Group, Config) -> + Config. + +end_per_group(_Group, _Config) -> + ok. + +init_per_testcase(TestCase, Config) -> + DataDir = filename:join(?config(priv_dir, Config), TestCase), + [{data_dir, DataDir}, {cluster_name, TestCase} | Config]. + +end_per_testcase(_TestCase, _Config) -> + ok. + +%%%=================================================================== +%%% Test cases +%%%=================================================================== + +-define(KV(N), + binary_to_atom(<<(atom_to_binary(?FUNCTION_NAME))/binary, + (integer_to_binary(N))/binary>>)). + +snapshot_replication_interrupted(_Config) -> + Kv1 = ?KV(1), Kv2 = ?KV(2), Kv3 = ?KV(3), + Members = [{Kv1, node()}, {Kv2, node()}], + KvId = hd(Members), + {ok, _, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, + #{members => Members}), + ra:transfer_leadership(KvId, KvId), + Data = crypto:strong_rand_bytes(100_000), + %% write 10k entries of the same key + [{ok, #{}} = ra_kv:put(KvId, term_to_binary(I), Data, 5000) + || I <- lists:seq(1, 10_000)], + ?assertMatch({ok, #{machine := #{num_keys := 10_000}}, KvId}, + ra:member_overview(KvId)), + + ra_log_wal:force_roll_over(ra_log_wal), + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + ra_log_segment_writer:await(ra_log_segment_writer), + ok = ra:aux_command(KvId, take_snapshot), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{snapshot_index := SnapIdx, + last_index := LastIdx}}, _} = + ra:member_overview(KvId), + SnapIdx == LastIdx + end, 100, 100), + KvId3 = {Kv3, node()}, + ok = ra_kv:add_member(?SYS, KvId3, KvId), + KvId3Pid = whereis(Kv3), + ?assert(is_pid(KvId3Pid)), + %% wait for the follower to enter snapshot state + ok = ra_lib:retry( + fun () -> + receive_snapshot == element(2, hd(ets:lookup(ra_state, Kv3))) + end, 100, 100), + + ct:pal("ra_state ~p", [ets:tab2list(ra_state)]), + ok = ra:stop_server(?SYS, KvId3), + [{ok, #{}} = ra_kv:put(KvId, term_to_binary(I), Data, 5000) + || I <- lists:seq(10_001, 10_010)], + ok = ra:restart_server(?SYS, KvId3), + {ok, #{log := #{last_index := Kv1LastIndex }}, _} = ra:member_overview(KvId), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{last_index := LastIdx}}, _} = + ra:member_overview(KvId3), + Kv1LastIndex == LastIdx + end, 100, 256), + ra:delete_cluster([KvId, {Kv2, node()}, KvId3]), + ok. + +snapshot_replication(_Config) -> + Kv1 = ?KV(1), Kv2 = ?KV(2), Kv3 = ?KV(3), + Members = [{Kv1, node()}, {Kv2, node()}], + KvId = hd(Members), + {ok, _, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, + #{members => Members}), + ra:transfer_leadership(KvId, KvId), + {ok, #{}} = ra_kv:put(KvId, <<"k1">>, <<"k1-value01">>, 5000), + %% write 10k entries of the same key + [{ok, #{}} = ra_kv:put(KvId, integer_to_binary(I), I, 5000) + || I <- lists:seq(1, 5000)], + + ?assertMatch({ok, #{machine := #{num_keys := _}}, KvId}, + ra:member_overview(KvId)), + ra_log_wal:force_roll_over(ra_log_wal), + %% wait for rollover processing + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + %% wait for segment writer to process + ra_log_segment_writer:await(ra_log_segment_writer), + %% promt ra_kv to take a snapshot + ok = ra:aux_command(KvId, take_snapshot), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{snapshot_index := SnapIdx, + last_index := LastIdx}}, _} = + ra:member_overview(KvId), + SnapIdx == LastIdx + end, 100, 100), + + KvId3 = {Kv3, node()}, + ok = ra_kv:add_member(?SYS, KvId3, KvId), + KvId3Pid = whereis(Kv3), + ?assert(is_pid(KvId3Pid)), + {ok, #{}} = ra_kv:put(KvId, <<"k3">>, <<"k3-value">>, 5000), + {ok, #{}} = ra_kv:put(KvId, <<"k4">>, <<"k4-value">>, 5000), + ok = ra:aux_command(KvId, take_snapshot), + % timer:sleep(1000), + {ok, #{log := #{last_index := Kv1LastIndex }}, _} = ra:member_overview(KvId), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{last_index := LastIdx}}, _} = + ra:member_overview(KvId3), + Kv1LastIndex == LastIdx + end, 100, 100), + ct:pal("counters ~p", [ra_counters:counters(KvId3, [last_applied])]), + %% ensure Kv3 did not crash during snapshot replication + ?assertEqual(KvId3Pid, whereis(Kv3)), + + ok = ra:stop_server(default, KvId3), + + {ok, #{}} = ra_kv:put(KvId, <<"k5">>, <<"k5-value">>, 5000), + {ok, #{}} = ra_kv:put(KvId, <<"k6">>, <<"k6-value">>, 5000), + ok = ra:aux_command(KvId, take_snapshot), + + ok = ra:restart_server(default, KvId3), + {ok, #{log := #{last_index := Kv1LastIndex2}}, _} = ra:member_overview(KvId), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{last_index := LastIdx}}, _} = + ra:member_overview(KvId3), + Kv1LastIndex2 == LastIdx + end, 100, 100), + + ra:delete_cluster([KvId, {kv2, node()}, KvId3]), + ok. + +basics(_Config) -> + Kv1 = ?KV(1), Kv2 = ?KV(2), _Kv3 = ?KV(3), + Members = [{Kv1, node()}], + KvId = hd(Members), + {ok, Members, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, + #{members => Members}), + {ok, #{}} = ra_kv:put(KvId, <<"k1">>, <<"k1-value01">>, 5000), + K2 = <<"k2">>, + %% write 10k entries of the same key + [{ok, #{}} = ra_kv:put(KvId, K2, I, 5000) + || I <- lists:seq(1, 10000)], + + ct:pal("kv get ~p", [ra_kv:get(KvId, <<"k1">>, 5000)]), + ct:pal("leaderboard ~p", [ets:tab2list(ra_leaderboard)]), + + ?assertMatch({ok, #{machine := #{num_keys := 2}}, KvId}, + ra:member_overview(KvId)), + ra_log_wal:force_roll_over(ra_log_wal), + %% wait for rollover processing + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + %% wait for segment writer to process + ra_log_segment_writer:await(ra_log_segment_writer), + %% promt ra_kv to take a snapshot + ok = ra:aux_command(KvId, take_snapshot), + %% wait for snapshot to complete + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{snapshot_index := SnapIdx, + num_segments := NumSegments, + last_index := LastIdx}}, _} = + ra:member_overview(KvId), + SnapIdx == LastIdx andalso NumSegments == 2 + end, 100, 100), + %% restart server to test recovery + ok = ra:stop_server(default, KvId), + ok = ra:restart_server(default, KvId), + {ok, #{index := LastIdx}} = ra_kv:put(KvId, <<"k3">>, <<"k3">>, 5000), + {ok, #{machine := #{live_indexes := Live}, + log := #{range := {_, KvIdLastIdx}}}, _} = ra:member_overview(KvId), + {ok, {Reads, _}} = ra_server_proc:read_entries(KvId, [LastIdx | Live], + undefined, 1000), + ?assertEqual(3, map_size(Reads)), + % ct:pal("ReadRes ~p", [Reads]), + KvId2 = {Kv2, node()}, + ok = ra_kv:add_member(?SYS, KvId2, KvId), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{range := {_, Last}}}, _} = + ra:member_overview(KvId2), + Last >= KvIdLastIdx + end, 100, 100), + {ok, {Reads2, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], + undefined, 1000), + ?assertEqual(3, map_size(Reads2)), + ra_log_wal:force_roll_over(ra_log_wal), + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + ra_log_segment_writer:await(ra_log_segment_writer), + {ok, {Reads3, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], + undefined, 1000), + ?assertEqual(3, map_size(Reads3)), + + %% TODO: test recovery of kv + ok = ra:stop_server(default, KvId2), + ok = ra:restart_server(default, KvId2), + {ok, {Reads4, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], + undefined, 1000), + + ?assertEqual(3, map_size(Reads4)), + ra:trigger_compaction(KvId), + %% wait for compaction by querying counters + ok = ra_lib:retry( + fun () -> + #{major_compactions := Maj} = + ra_counters:counters(KvId, [major_compactions]), + Maj == 1 + end, 10, 100), + {ok, {Reads5, _}} = ra_server_proc:read_entries(KvId, [LastIdx | Live], + undefined, 1000), + ?assertEqual(Reads4, Reads5), + ct:pal("counters ~p", [ra_counters:overview(KvId)]), + ok = ra_kv:remove_member(?SYS, KvId2, KvId), + ct:pal("members ~p", [ra:members(KvId)]), + ra:delete_cluster([KvId]), + ok. diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 6c5816c25..c02f0509c 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, @@ -32,6 +33,11 @@ all_tests() -> last_written_overwrite, last_written_overwrite_2, last_index_reset, + fold_after_sparse_mem_table, + fold_after_sparse_segments, + write_sparse_re_init, + write_sparse_after_index_reset, + write_sparse_after_index_reset_segments, last_index_reset_before_written, recovery, recover_many, @@ -49,6 +55,7 @@ all_tests() -> snapshot_written_after_installation, oldcheckpoints_deleted_after_snapshot_install, append_after_snapshot_installation, + release_cursor_after_snapshot_installation, written_event_after_snapshot_installation, update_release_cursor, update_release_cursor_with_machine_version, @@ -65,9 +72,11 @@ 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, + overwritten_segment_is_cleared, + overwritten_segment_is_cleared_on_init, + snapshot_installation_with_live_indexes ]. groups() -> @@ -130,7 +139,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 +149,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 +168,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 +183,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 +211,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), @@ -254,7 +263,6 @@ read_one(Config) -> ra_counters:new(?FUNCTION_NAME, ?RA_COUNTER_FIELDS), Log0 = ra_log_init(Config, #{counter => ra_counters:fetch(?FUNCTION_NAME)}), Log1 = append_n(1, 2, 1, Log0), - % Log1 = ra_log:append({1, 1, <<1:64/integer>>}, Log0), % ensure the written event is delivered Log2 = deliver_all_log_events(Log1, 200), {[_], Log} = ra_log_take(1, 1, Log2), @@ -309,9 +317,9 @@ validate_sequential_fold(Config) -> ct:pal("ra_log:overview/1 ~p", [ra_log:overview(FinLog)]), - #{?FUNCTION_NAME := #{read_mem_table := M1, - open_segments := 2, %% as this is the max - read_segment := M4} = O} = ra_counters:overview(), + #{read_mem_table := M1, + open_segments := 2, %% as this is the max + read_segment := M4} = O = ra_counters:overview(?FUNCTION_NAME), ct:pal("counters ~p", [O]), ?assertEqual(1000, M1 + M4), @@ -399,16 +407,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, #{}, macctx(), + <<"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 +544,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, #{}, macctx(), + <<"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, #{}, macctx(), + <<"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 +583,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, #{}, macctx(), + <<"one+two">>, Log2), + run_effs(Effs), Log4 = deliver_all_log_events(Log3, 100), ra_log:close(Log4), restart_wal(), @@ -578,7 +593,7 @@ recover_after_snapshot(Config) -> Log = ra_log_init(Config, #{min_snapshot_interval => 1}), Overview = ra_log:overview(Log), ra_log:close(Log), - ?assertMatch(#{last_index := 2, + ?assertMatch(#{range := undefined, last_term := 1, snapshot_index := 2, last_written_index_term := {2, 1}}, Overview), @@ -591,12 +606,13 @@ 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, #{}, macctx(), + <<"100">>, Log2), + run_effs(Effs0), Log4 = deliver_all_log_events(Log3, 500), Overview = ra_log:overview(Log4), - ?assertMatch(#{last_index := 499, - first_index := 101, + ?assertMatch(#{range := {101, 499}, mem_table_range := {101, 499}, last_written_index_term := {100, 1}}, Overview), @@ -605,7 +621,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( @@ -623,8 +639,7 @@ writes_lower_than_snapshot_index_are_dropped(Config) -> ct:fail("expected log event not received") end, OverviewAfter = ra_log:overview(Log5), - ?assertMatch(#{last_index := 499, - first_index := 101, + ?assertMatch(#{range := {101, 499}, snapshot_index := 100, mem_table_range := {101, 499}, last_written_index_term := {499, 1}}, OverviewAfter), @@ -658,11 +673,7 @@ updated_segment_can_be_read(Config) -> % this should return all entries {Entries1, _} = ra_log_take(1, 15, Log4), ?assertEqual(15, length(Entries1)), - ct:pal("Entries: ~p", [Entries]), - ct:pal("Entries1: ~p", [Entries1]), - ct:pal("Counters ~p", [ra_counters:overview(?FUNCTION_NAME)]), ?assertEqual(15, length(Entries1)), - % l18 = length(Entries1), ok. cache_overwrite_then_take(Config) -> @@ -716,19 +727,165 @@ last_written_overwrite_2(Config) -> last_index_reset(Config) -> Log0 = ra_log_init(Config), - Log1 = write_n(1, 5, 1, Log0), + Log1 = write_n(1, 6, 1, Log0), Pred = fun (L) -> - {4, 1} == ra_log:last_written(L) + {5, 1} == ra_log:last_written(L) end, Log2 = assert_log_events(Log1, Pred, 2000), - 5 = ra_log:next_index(Log2), - {4, 1} = ra_log:last_index_term(Log2), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), % reverts last index to a previous index % needs to be done if a new leader sends an empty AER {ok, Log3} = ra_log:set_last_index(3, Log2), {3, 1} = ra_log:last_written(Log3), 4 = ra_log:next_index(Log3), {3, 1} = ra_log:last_index_term(Log3), + O = ra_log:overview(Log3), + ct:pal("o ~p", [O]), + ?assertMatch(#{range := {0, 3}, + %% we have a new mem table but the mem table does not know + %% whatever the first index should be so reports the + %% full previous range, this will be corrected after the + %% next write at index 4 + mem_table_range := {0, 5}}, + O), + {ok, Log} = ra_log:write([{4, 2, hi}], Log3), + O2 = ra_log:overview(Log), + ct:pal("o ~p", [O2]), + ?assertMatch(#{range := {0, 4}, + mem_table_range := {0, 4}}, + O2), + ok. + +fold_after_sparse_mem_table(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 500), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log2b} = ra_log:set_last_index(5, Log2), + {ok, Log3} = ra_log:write_sparse({7, 1, hi}, undefined, Log2b), + {ok, Log4} = ra_log:write_sparse({17, 1, hi}, 7, Log3), + Log = deliver_all_log_events(Log4, 500), + {Res, _Log} = ra_log:fold(1, 17, + fun ({I, _, _}, Is) -> + [I | Is] + end, [], Log, return), + ct:pal("Res ~p", [Res]), + ?assertMatch([5,4,3,2,1], Res), + ok. + +fold_after_sparse_segments(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 500), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log2b} = ra_log:set_last_index(5, Log2), + {ok, Log3} = ra_log:write_sparse({7, 1, hi}, undefined, Log2b), + {ok, Log4} = ra_log:write_sparse({17, 1, hi}, 7, Log3), + ok = ra_log_wal:force_roll_over(ra_log_wal), + Log = deliver_all_log_events(Log4, 500), + ra_log:fold(1, 17, fun (_, _) -> ok end, undefined, Log, return), + ok. + +write_sparse_re_init(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_and_roll(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 500), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log2b} = ra_log:set_last_index(5, Log2), + {ok, Log3} = ra_log:write_sparse({7, 1, hi}, undefined, Log2b), + {ok, Log4} = ra_log:write_sparse({17, 1, hi}, 7, Log3), + Log = deliver_all_log_events(Log4, 500), + + O = ra_log:overview(Log), + ra_log:close(Log), + + ReInitLog = ra_log_init(Config), + O2 = ra_log:overview(ReInitLog ), + OO = maps:without([last_wal_write, open_segments], O), + OO2 = maps:without([last_wal_write, open_segments], O2), + ?assertEqual(OO, OO2), + ok. + +write_sparse_after_index_reset(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 500), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log3} = ra_log:set_last_index(3, Log2), + {3, 1} = ra_log:last_written(Log3), + 4 = ra_log:next_index(Log3), + {3, 1} = ra_log:last_index_term(Log3), + O = ra_log:overview(Log3), + ct:pal("o ~p", [O]), + {ok, Log4} = ra_log:write_sparse({7, 1, hi}, undefined, Log3), + {ok, Log} = ra_log:write_sparse({17, 1, hi}, 7, Log4), + O2 = ra_log:overview(Log), + ct:pal("o ~p", [O2]), + ?assertMatch(#{range := {0, 17}, + mem_table_range := {0, 17}}, + O2), + ok. + +write_sparse_after_index_reset_segments(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 2000), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + ra_log_wal:force_roll_over(ra_log_wal), + Log2b = deliver_all_log_events(Log2, 500), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log3} = ra_log:set_last_index(3, Log2b), + {3, 1} = ra_log:last_written(Log3), + 4 = ra_log:next_index(Log3), + {3, 1} = ra_log:last_index_term(Log3), + O = ra_log:overview(Log3), + ct:pal("o ~p", [O]), + {ok, Log4} = ra_log:write_sparse({7, 1, hi}, undefined, Log3), + {ok, Log5} = ra_log:write_sparse({17, 1, hi}, 7, Log4), + Log5b = deliver_all_log_events(Log5, 500), + O2 = ra_log:overview(Log5b), + ?assertMatch(#{range := {0, 17}, + mem_table_range := {7, 17}}, + O2), + + %% try overwrite again + {ok, Log6} = ra_log:set_last_index(3, Log5b), + {3, 1} = ra_log:last_index_term(Log6), + {ok, Log7} = ra_log:write_sparse({7, 1, hi}, undefined, Log6), + {ok, Log8} = ra_log:write_sparse({17, 1, hi}, 7, Log7), + Log = deliver_all_log_events(Log8, 500), + O5 = ra_log:overview(Log), + ct:pal("o ~p", [O5]), ok. last_index_reset_before_written(Config) -> @@ -752,7 +909,6 @@ last_index_reset_before_written(Config) -> end), 4 = ra_log:next_index(Log3), {3, 1} = ra_log:last_index_term(Log3), - % #{cache_size := 0} = ra_log:overview(Log3), ok. recovery(Config) -> @@ -892,6 +1048,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 +1065,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 +1247,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, #{}, macctx(), + <<"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 +1264,12 @@ 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), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log3, _} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, 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 +1297,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, #{}, macctx(), <<"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 +1314,14 @@ 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), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log4, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, 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), @@ -1148,6 +1348,61 @@ oldcheckpoints_deleted_after_snapshot_install(Config) -> ok. +snapshot_installation_with_live_indexes(Config) -> + Log0 = ra_log_init(Config), + {0, 0} = ra_log:last_index_term(Log0), + Log1 = assert_log_events(write_n(1, 10, 2, Log0), + fun (L) -> + LW = ra_log:last_written(L), + {9, 2} == LW + end), + Log2 = Log1, + + %% create snapshot chunk + Meta = meta(15, 2, [?N1]), + Chunk = create_snapshot_chunk(Config, Meta, [2, 9, 14], #{}), + SnapState0 = ra_log:snapshot_state(Log2), + {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), + Machine = {machine, ?MODULE, #{}}, + + %% write a sparse one + {ok, Log2b} = ra_log:write_sparse({14, 2, <<>>}, 9, Log2), + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log3, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log2b)), + + + run_effs(Effs4), + {15, _} = ra_log:last_index_term(Log3), + {15, _} = ra_log:last_written(Log3), + %% write the next index, bearning in mind the last index the WAL saw + %% was 14 + {ok, Log4} = ra_log:write([{16, 2, <<>>}], Log3), + Log5 = assert_log_events(Log4, + fun (L) -> + LW = ra_log:last_written(L), + {16, 2} == LW + end), + ra_log_wal:force_roll_over(ra_log_wal), + Log = assert_log_events(Log5, + fun (L) -> + #{mem_table_range := R} = ra_log:overview(L), + R == undefined + end), + ct:pal("o ~p", [ra_log:overview(Log)]), + UId = ?config(uid, Config), + ?assertEqual(LiveIndexes, ra_log_snapshot_state:live_indexes( + ra_log_snapshot_state, UId)), + ra_log:close(Log), + flush(), + _LogAfter = ra_log_init(Config), + %% validate recovery recovers the live indexes correctly + ?assertEqual(LiveIndexes, ra_log_snapshot_state:live_indexes( + ra_log_snapshot_state, UId)), + ok. + snapshot_installation(Config) -> Log0 = ra_log_init(Config), {0, 0} = ra_log:last_index_term(Log0), @@ -1164,9 +1419,14 @@ 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), - + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log3, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log2)), + + run_effs(Effs4), {15, _} = ra_log:last_index_term(Log3), {15, _} = ra_log:last_written(Log3), #{mem_table_range := undefined} = ra_log:overview(Log3), @@ -1179,9 +1439,12 @@ snapshot_installation(Config) -> % after a snapshot we need a "truncating write" that ignores missing % indexes Log5 = write_n(16, 20, 2, Log4), - {[], _} = ra_log_take(1, 9, Log5), - {[_, _], _} = ra_log_take(16, 17, Log5), - Log6 = assert_log_events(Log5, fun (L) -> + %% check that the log can be reset to a pending write index + {ok, Log5b} = ra_log:set_last_index(19, Log5), + + {[], _} = ra_log_take(1, 9, Log5b), + {[_, _], _} = ra_log_take(16, 17, Log5b), + Log6 = assert_log_events(Log5b, fun (L) -> {19, 2} == ra_log:last_written(L) end), {[], _} = ra_log_take(1, 9, Log6), @@ -1213,8 +1476,13 @@ 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), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log2, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log1)), + run_effs(Effs4), {15, _} = ra_log:last_index_term(Log2), {15, _} = ra_log:last_written(Log2), @@ -1228,6 +1496,57 @@ append_after_snapshot_installation(Config) -> {[_, _], _} = ra_log_take(16, 17, Log), ok. +release_cursor_after_snapshot_installation(Config) -> + Log0 = ra_log_init(Config, #{min_snapshot_interval => 0}), + {0, 0} = ra_log:last_index_term(Log0), + Log1 = assert_log_events(write_n(1, 16, 2, Log0), + fun (L) -> + LW = ra_log:last_written(L), + {15, 2} == LW + end), + + Log2 = Log1, + + %% create snapshot chunk + Meta = meta(15, 2, [?N1]), + Chunk = create_snapshot_chunk(Config, Meta, [1, 5, 10], #{}), + SnapState0 = ra_log:snapshot_state(Log2), + {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log3, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log2)), + + run_effs(Effs4), + {15, 2} = ra_snapshot:current(ra_log:snapshot_state(Log3)), + + %% Write some entries + Log4 = assert_log_events(write_n(16, 20, 2, Log3), + fun (L) -> + LW = ra_log:last_written(L), + {19, 2} == LW + end), + + %% then take a snapshot + {Log5, Effs5} = ra_log:update_release_cursor(19, #{?N1 => new_peer(), + ?N2 => new_peer()}, + macctx(), [1, 5, 10, 17], Log4), + + run_effs(Effs5), + %% ensure snapshot index has been updated and 1 segment deleted + Log = assert_log_events(Log5, + fun (L) -> + {19, 2} == ra_log:snapshot_index_term(L) + % andalso + % length(find_segments(Config)) == 1 + end), + + ct:pal("Log ~p", [Log]), + + ok. + written_event_after_snapshot_installation(Config) -> logger:set_primary_config(level, all), %% simulates scenario where a server receives a written event from the wal @@ -1244,8 +1563,13 @@ 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), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log2, Effs4} = ra_log:install_snapshot({SnapIdx, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log1)), + run_effs(Effs4), {SnapIdx, _} = ra_log:last_index_term(Log2), {SnapIdx, _} = ra_log:last_written(Log2), NextIdx = SnapIdx + 1, @@ -1281,33 +1605,35 @@ 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()}, + macctx(), 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), + ct:pal("Log3 ~p", [Log3]), 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()}, + macctx(), 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 +1643,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 +1663,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()}, + {0, ?MODULE}, + initial_state, Log1), + run_effs(Effs), Log = assert_log_events(Log2, fun (L) -> {127, 2} == ra_log:snapshot_index_term(L) @@ -1350,7 +1676,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 := 0}, Meta), ok. missed_mem_table_entries_are_deleted_at_next_opportunity(Config) -> @@ -1388,9 +1714,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), + macctx(), 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,71 +1781,7 @@ 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, id => {?MODULE, node()}, uid => <<"blah">>, @@ -1530,14 +1795,133 @@ 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), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexesSeq, AEffs} = ra_snapshot:complete_accept(Chunk, 1, + Machine, + SnapState1), + ?assertEqual(LiveIndexes, lists:reverse(ra_seq:expand(LiveIndexesSeq))), + run_effs(AEffs), + Log3 = ra_log:set_snapshot_state(SnapState, Log2), + {ok, Log4, _} = ra_log:install_snapshot({15, 2}, ?MODULE, + LiveIndexes, Log3), + + ct:pal("overview Log4 ~p", [ra_log:overview(Log4)]), + ?assertEqual(16, ra_log:next_index(Log4)), + + {ok, Log} = ra_log:write([{16, 1, <<>>}], Log4), + {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. + +overwritten_segment_is_cleared(Config) -> + Log0 = ra_log_init(Config, #{}), + % write a few entries + Log1 = write_and_roll(1, 256, 1, Log0), + Log2 = assert_log_events(Log1, + fun(L) -> + #{num_segments := N} = ra_log:overview(L), + N == 2 + end), + Log3 = write_and_roll(128, 256 + 128, 2, Log2), + UId = ?config(uid, Config), + Log = assert_log_events(Log3, + fun(L) -> + #{num_segments := N} = ra_log:overview(L), + N == 3 andalso + 3 == length(ra_log_segment_writer:my_segments(ra_log_segment_writer, UId)) + end), + + ct:pal("Log overview ~p", [ra_log:overview(Log)]), + ok. + +overwritten_segment_is_cleared_on_init(Config) -> + Log0 = ra_log_init(Config, #{}), + % write a few entries + Log1 = write_and_roll(1, 256, 1, Log0), + Log2 = assert_log_events(Log1, + fun(L) -> + #{num_segments := N} = ra_log:overview(L), + N == 2 + end), + Log3 = write_n(128, 256 + 128, 2, Log2), + ok = ra_log_wal:force_roll_over(ra_log_wal), + ra_log:close(Log3), + % _Log3 = write_and_roll(128, 256 + 128, 2, Log2), + UId = ?config(uid, Config), + timer:sleep(1000), + flush(), + Log = ra_log_init(Config, #{}), + + ct:pal("my segments ~p", + [ra_log_segment_writer:my_segments(ra_log_segment_writer, UId)]), + ct:pal("Log overview ~p", [ra_log:overview(Log)]), + ?assertEqual(3, length( + ra_log_segment_writer:my_segments(ra_log_segment_writer, UId))), + + ok. + validate_fold(From, To, Term, Log0) -> - {Entries0, Log} = ra_log:fold(From, To, - fun (E, A) -> [E | A] end, - [], Log0), + {Entries0, Log} = ra_log:fold(From, To, fun ra_lib:cons/2, [], Log0), ?assertEqual(To - From + 1, length(Entries0)), % validate entries are correctly read - Expected = [ {I, Term, <>} || - I <- lists:seq(To, From, -1) ], + Expected = [{I, Term, <>} || + I <- lists:seq(To, From, -1)], ?assertEqual(Expected, Entries0), Log. @@ -1583,6 +1967,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,17 +1990,20 @@ 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, Effs1} = ra_log:handle_event(E, Acc0), + run_effs(Effs1), + 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); false -> - deliver_log_events_cond(Log2, CondFun, N-1); + deliver_log_events_cond(Log1, CondFun, N-1); {true, Log} -> ct:pal("condition was true!!"), Log; @@ -1642,12 +2036,15 @@ deliver_all_log_events(Log0, Timeout) -> 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], % ct:pal("log evt effs: ~p", [Effs]), deliver_all_log_events(Log, Timeout) after Timeout -> @@ -1658,28 +2055,30 @@ assert_log_events(Log0, AssertPred) -> assert_log_events(Log0, AssertPred, 2000). assert_log_events(Log0, AssertPred, Timeout) -> - receive - {ra_log_event, Evt} -> - ct:pal("log evt: ~p", [Evt]), - {Log1, Effs} = ra_log:handle_event(Evt, Log0), - %% 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; - (_, Acc) -> - Acc - end, Log1, Effs), - case AssertPred(Log) of - true -> - Log; - false -> + case AssertPred(Log0) of + true -> + Log0; + false -> + receive + {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) -> + {Acc, Effs1} = ra_log:handle_event(E, Acc0), + run_effs(Effs1), + Acc; + (_, Acc) -> + Acc + end, Log1, Effs), assert_log_events(Log, AssertPred, Timeout) + + after Timeout -> + flush(), + exit({assert_log_events_timeout, Log0}) end - after Timeout -> - flush(), - exit({assert_log_events_timeout, Log0}) end. wait_for_segments(Log0, Timeout) -> @@ -1770,19 +2169,26 @@ meta(Idx, Term, Cluster) -> cluster => Cluster, machine_version => 1}. -create_snapshot_chunk(Config, #{index := Idx} = Meta, Context) -> +create_snapshot_chunk(Config, Meta, Context) -> + create_snapshot_chunk(Config, 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 +2227,17 @@ 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. +which_module(_) -> ?MODULE. +live_indexes(MacState) when is_list(MacState) -> + %% fake returning live indexes + MacState; +live_indexes(_) -> + []. + +macctx() -> + {version(), ?MODULE}. diff --git a/test/ra_log_SUITE.erl b/test/ra_log_SUITE.erl index 46c1a0cc3..648d8fe08 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 b5bd6c6b5..5eed78b9c 100644 --- a/test/ra_log_memory.erl +++ b/test/ra_log_memory.erl @@ -10,6 +10,7 @@ close/1, append/2, write/2, + write_sparse/3, take/3, fold/5, last_index_term/1, @@ -22,7 +23,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 +42,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 +53,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]). @@ -92,7 +95,7 @@ write([{FirstIdx, _, _} | _] = Entries, {Acc#{Idx => {Term, Data}}, Idx} end, {Log1, FirstIdx}, Entries), {ok, State#state{last_index = LastInIdx, - entries = Log}}; + entries = Log}}; write([{FirstIdx, _, _} | _] = Entries, #state{snapshot = {#{index := SnapIdx}, _}, entries = Log0} = State) when SnapIdx + 1 =:= FirstIdx -> @@ -104,6 +107,12 @@ write([{FirstIdx, _, _} | _] = Entries, write(_Entries, _State) -> {error, {integrity_error, undefined}}. +write_sparse({Idx, Term, Data}, PrevIdx, #state{last_index = PrevIdx, + entries = Log0} = State) -> + {ok, State#state{last_index = Idx, + entries = Log0#{Idx => {Term, Data}}}}; +write_sparse(_, _, _) -> + {error, gap_detected}. take(Start, Num, #state{last_index = LastIdx, entries = Log} = State) -> Entries = sparse_take(Start, Log, Num, LastIdx, []), @@ -170,17 +179,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 +220,17 @@ 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 +install_snapshot({Index, Term}, _MacMod, _LiveIndexes, + #state{entries = Log0} = State0) -> + % 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, + {ok, State, []}. -spec read_snapshot(State :: ra_log_memory_state()) -> {ok, ra_snapshot:meta(), term()}. @@ -274,7 +284,7 @@ write_meta_f(_Key, _Value, State) -> can_write(_Log) -> true. -overview(Log) -> +overview(#state{} = Log) -> #{type => ?MODULE, last_index => Log#state.last_index, last_written => Log#state.last_written, diff --git a/test/ra_log_meta_SUITE.erl b/test/ra_log_meta_SUITE.erl index ff78db02c..5ec194ee8 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 c2e7c9634..000000000 --- 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-2025 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_SUITE.erl b/test/ra_log_segment_SUITE.erl index 8632ca152..c0ae620bd 100644 --- a/test/ra_log_segment_SUITE.erl +++ b/test/ra_log_segment_SUITE.erl @@ -38,7 +38,10 @@ all_tests() -> corrupted_segment, large_segment, segref, - versions_v1 + info, + info_2, + versions_v1, + copy ]. groups() -> @@ -50,10 +53,6 @@ init_per_testcase(TestCase, Config) -> PrivDir = ?config(priv_dir, Config), Dir = filename:join(PrivDir, TestCase), ok = ra_lib:make_dir(Dir), - _ = ets:new(ra_open_file_metrics, - [named_table, public, {write_concurrency, true}]), - _ = ets:new(ra_io_metrics, - [named_table, public, {write_concurrency, true}]), [{test_case, TestCase}, {data_dir, Dir} | Config]. end_per_testcase(_, Config) -> @@ -207,9 +206,42 @@ segref(Config) -> {ok, Seg0} = ra_log_segment:open(Fn, #{max_count => 128}), undefined = ra_log_segment:segref(Seg0), {ok, Seg1} = ra_log_segment:append(Seg0, 1, 2, <<"Adsf">>), - {{1, 1}, "seg1.seg"} = ra_log_segment:segref(Seg1), + {<<"seg1.seg">>, {1, 1}} = ra_log_segment:segref(Seg1), ok. +info(Config) -> + Dir = ?config(data_dir, Config), + Fn = filename:join(Dir, "seg1.seg"), + {ok, Seg0} = ra_log_segment:open(Fn, #{max_count => 128}), + Info1 = ra_log_segment:info(Fn), + ?assertMatch(#{ref := undefined}, Info1), + {ok, Seg1} = ra_log_segment:append(Seg0, 1, 2, <<"Adsf">>), + _ = ra_log_segment:flush(Seg1), + Info2 = ra_log_segment:info(Fn), + ?assertMatch(#{ref := {<<"seg1.seg">>, {1, 1}}}, Info2), + ok. + +info_2(Config) -> + %% passes live indexes which will result in additional info keys + Dir = ?config(data_dir, Config), + Fn = filename:join(Dir, "seg1.seg"), + {ok, Seg0} = ra_log_segment:open(Fn, #{max_count => 128}), + Info1 = ra_log_segment:info(Fn, []), + ?assertMatch(#{ref := undefined, + live_size := 0}, Info1), + {ok, Seg1} = ra_log_segment:append(Seg0, 1, 2, <<"Adsf">>), + {ok, Seg2} = ra_log_segment:append(Seg1, 2, 2, <<"Adsf">>), + _ = ra_log_segment:flush(Seg2), + Info2 = ra_log_segment:info(Fn, [1]), + ?assertMatch(#{ref := {<<"seg1.seg">>, {1, 2}}, + num_entries := 2, + live_size := 4}, Info2), + Info3 = ra_log_segment:info(Fn), + %% info/1 assumes all indexes are "live" + ?assertMatch(#{ref := {<<"seg1.seg">>, {1, 2}}, + num_entries := 2, + live_size := 8}, Info3), + ok. full_file(Config) -> Dir = ?config(data_dir, Config), @@ -460,6 +492,37 @@ read_sparse_append_read(Config) -> ra_log_segment:close(R0), ok. +copy(Config) -> + Dir = ?config(data_dir, Config), + Indexes = lists:seq(1, 100), + SrcFn = filename:join(Dir, <<"SOURCE1.segment">>), + {ok, SrcSeg0} = ra_log_segment:open(SrcFn), + SrcSeg1 = lists:foldl( + fun (I, S0) -> + {ok, S} = ra_log_segment:append(S0, I, 1, term_to_binary(I)), + S + end, SrcSeg0, Indexes), + _ = ra_log_segment:close(SrcSeg1), + + Fn = filename:join(Dir, <<"TARGET.segment">>), + {ok, Seg0} = ra_log_segment:open(Fn), + CopyIndexes = lists:seq(1, 100, 2), + {ok, Seg} = ra_log_segment:copy(Seg0, SrcFn, CopyIndexes), + ra_log_segment:close(Seg), + {ok, R} = ra_log_segment:open(Fn, #{mode => read, + access_pattern => random}), + %%TODO: consider makeing read_sparse tolerant to missing indexes somehow + %% perhaps detecting if the segment is "sparse" + {ok, 2, [_, _]} = ra_log_segment:read_sparse(R, [1, 3], + fun (I, T, B, Acc) -> + [{I, T, binary_to_term(B)} | Acc] + end, []), + ra_log_segment:close(R), + + ok. + + +%%% Internal write_until_full(Idx, Term, Data, Seg0) -> case ra_log_segment:append(Seg0, Idx, Term, Data) of {ok, Seg} -> @@ -467,9 +530,6 @@ write_until_full(Idx, Term, Data, Seg0) -> {error, full} -> Seg0 end. - - -%%% Internal make_data(Size) -> term_to_binary(crypto:strong_rand_bytes(Size)). diff --git a/test/ra_log_segment_writer_SUITE.erl b/test/ra_log_segment_writer_SUITE.erl index af0ff7c5f..98b2c213e 100644 --- a/test/ra_log_segment_writer_SUITE.erl +++ b/test/ra_log_segment_writer_SUITE.erl @@ -41,7 +41,8 @@ all_tests() -> my_segments, upgrade_segment_name_format, skip_entries_lower_than_snapshot_index, - skip_all_entries_lower_than_snapshot_index + skip_all_entries_lower_than_snapshot_index, + live_indexes_1 ]. groups() -> @@ -91,16 +92,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, [{SegFile, {1, 3}}]}} -> 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 +127,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}]}], [{Fn, {1, 5}}]}} -> 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 +148,7 @@ accept_mem_tables_append(Config) -> flush(), throw(ra_log_event_timeout) end, + flush(), ok = gen_server:stop(TblWriterPid), ok. @@ -161,14 +160,14 @@ 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}]}], [{Fn, {3, 5}}]}} -> 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)), + ?assertMatch({_, {3, 5}}, ra_log_segment:segref(Seg)), ra_log_segment:close(Seg), ok after 3000 -> @@ -178,14 +177,14 @@ 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}]}], [{Fn2, {1, 3}}]}} -> 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)), + ?assertMatch({_, {1, 3}}, ra_log_segment:segref(Seg2)), C2 = term_to_binary(c2), [{1, 43, _}, {2, 43, _}] = read_sparse(Seg2, [1, 2]), [{3, 43, C2}] = read_sparse(Seg2, [3]), @@ -210,18 +209,15 @@ 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}]}], [{Fn, {2, 6}}]}} -> 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)), + ?assertMatch({_, {2, 6}}, ra_log_segment:segref(Seg)), [{2, 42, _}, {3, 42, _}, {4, 43, _}, @@ -249,12 +245,13 @@ 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")), receive - {ra_log_event, {segments, TidRanges, [{{9, 10}, _Seg2}, {{1, 8}, _Seg1}]}} -> + {ra_log_event, {segments, TidRanges, [{_, {9, 10}}, {_, {1, 8}}]}} -> ok after 3000 -> flush(), @@ -276,12 +273,12 @@ 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")), receive - {ra_log_event, {segments, TidRanges, [{{9, 10}, _Seg2}, {{1, 8}, _Seg1}]}} -> + {ra_log_event, {segments, TidRanges, [{_, {9, 10}}, {_, {1, 8}}]}} -> ok after 3000 -> flush(), @@ -303,13 +300,13 @@ 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")), LastFile = receive - {ra_log_event, {segments, TidRanges, [{{9, 10}, Seg2}, {{1, 8}, _Seg1}]}} -> + {ra_log_event, {segments, TidRanges, [{Seg2, {9, 10}}, {_Seg1, {1, 8}}]}} -> Seg2 % ok after 3000 -> @@ -320,13 +317,13 @@ 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")), receive {ra_log_event, {segments, TidRanges2, - [{{13, 15}, _}, {{7, 12}, LastFile}]}} -> + [{_, {13, 15}}, {LastFile, {7, 12}}]}} -> ok after 3000 -> flush(), @@ -360,13 +357,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}]}], [{Fn, {1, 3}}]}} -> 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 +373,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 +387,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 +420,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}]}], [{Fn, {1, 3}}]}} -> 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 +467,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 @@ -477,7 +476,7 @@ accept_mem_tables_with_corrupt_segment(Config) -> file:write_file(filename:join(?config(server_dir, Config), "0000001.segment"), <<>>), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, TidRanges, [{{1, 3}, SegFile}]}} -> + {ra_log_event, {segments, TidRanges, [{SegFile, {1, 3}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), SegFile), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -508,19 +507,20 @@ 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}, _}, - {{17, 32}, _}, - {{1, 16}, _} + {_, {49, 64}}, + {_, {33, 48}}, + {_, {17, 32}}, + {_, {1, 16}} ], SegRefs), ok after 3000 -> @@ -542,16 +542,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,19 +573,19 @@ 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), receive - {ra_log_event, {segments, TidRanges, [{{25, 32}, S} = Cur | Rem]}} -> + {ra_log_event, {segments, TidRanges, [{S, {25, 32}} = Cur | Rem]}} -> % test a lower index _does not_ delete the file SegmentFile = filename:join(?config(server_dir, Config), S), ?assert(filelib:is_file(SegmentFile)), ok = ra_log_segment_writer:truncate_segments(TblWriterPid, UId, Cur), ra_log_segment_writer:await(?SEGWR), - [{_, S1}, {_, S2}] = Rem, + [{S1, _}, {S2, _}] = Rem, SegmentFile1 = filename:join(?config(server_dir, Config), S1), ?assertNot(filelib:is_file(SegmentFile1)), SegmentFile2 = filename:join(?config(server_dir, Config), S2), @@ -607,18 +608,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, [{S, {25, 32}} = 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 @@ -629,7 +630,7 @@ truncate_segments_with_pending_update(Config) -> UId, Cur), ra_log_segment_writer:await(?SEGWR), ?assert(filelib:is_file(SegmentFile)), - [{_, S1}, {_, S2}] = Rem, + [{S1, _}, {S2, _}] = Rem, SegmentFile1 = filename:join(?config(server_dir, Config), S1), ?assertNot(filelib:is_file(SegmentFile1)), SegmentFile2 = filename:join(?config(server_dir, Config), S2), @@ -654,17 +655,17 @@ 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 - {ra_log_event, {segments, _Tid, [{{25, 32}, S} = Cur | Rem]}} -> + {ra_log_event, {segments, _Tid, [{S, {25, 32}} = Cur | Rem]}} -> % test a lower index _does not_ delete the file SegmentFile = filename:join(?config(server_dir, Config), S), ?assert(filelib:is_file(SegmentFile)), @@ -673,7 +674,7 @@ truncate_segments_with_pending_overwrite(Config) -> _ = ra_log_segment_writer:await(?SEGWR), SegmentFile = filename:join(?config(server_dir, Config), S), ?assert(filelib:is_file(SegmentFile)), - [{_, S1}, {_, S2}] = Rem, + [{S1, _}, {S2, _}] = Rem, SegmentFile1 = filename:join(?config(server_dir, Config), S1), ?assertNot(filelib:is_file(SegmentFile1)), SegmentFile2 = filename:join(?config(server_dir, Config), S2), @@ -685,7 +686,7 @@ truncate_segments_with_pending_overwrite(Config) -> throw(ra_log_event_timeout) end, receive - {ra_log_event, {segments, _, [{{16, 25}, F} = Cur2, {{12, 15}, F2}]}} -> + {ra_log_event, {segments, _, [{F, {16, 25}} = Cur2, {F2, {12, 15}}]}} -> ?assertMatch([_, _], segments_for(UId, Dir)), ok = ra_log_segment_writer:truncate_segments(TblWriterPid, UId, Cur2), @@ -715,12 +716,12 @@ 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), receive - {ra_log_event, {segments, TidRanges, [{{1, 3}, Fn}]}} -> + {ra_log_event, {segments, TidRanges, [{Fn, {1, 3}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), [MyFile] = ra_log_segment_writer:my_segments(?SEGWR,UId), ?assertEqual(SegmentFile, unicode:characters_to_binary(MyFile)), @@ -741,13 +742,13 @@ 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), File = receive - {ra_log_event, {segments, TidRanges, [{{1, 3}, _Fn}]}} -> + {ra_log_event, {segments, TidRanges, [{_, {1, 3}}]}} -> [MyFile] = ra_log_segment_writer:my_segments(?SEGWR,UId), MyFile after 2000 -> @@ -788,13 +789,13 @@ 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 - {ra_log_event, {segments, _Tid, [{{4, 5}, Fn}]}} -> + {ra_log_event, {segments, _Tid, [{Fn, {4, 5}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert only entries with a higher index than the snapshot @@ -821,9 +822,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), @@ -839,6 +840,47 @@ skip_all_entries_lower_than_snapshot_index(Config) -> ok = gen_server:stop(TblWriterPid), ok. +live_indexes_1(Config) -> + Dir = ?config(wal_dir, Config), + UId = ?config(uid, Config), + {ok, TblWriterPid} = ra_log_segment_writer:start_link(#{system => default, + name => ?SEGWR, + data_dir => Dir}), + % first batch + Entries = [{1, 42, a}, + {2, 42, b}, + {3, 43, c}, + {4, 43, d}, + {5, 43, e}, + {6, 43, f} + ], + Mt = make_mem_table(UId, Entries), + Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]}, + %% update snapshot state table + ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 4, 2, [4, 2]), + ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, + make_wal(Config, "w1.wal")), + receive + {ra_log_event, {segments, _Tid, [{Fn, {2, 6}}]}} -> + SegmentFile = filename:join(?config(server_dir, Config), Fn), + {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), + % assert only entries with a higher index than the snapshot + % have been written + ok = gen_server:stop(TblWriterPid), + ?assertExit({missing_key, 3}, read_sparse(Seg, [2, 3, 4])), + [ + {2, _, _}, + {4, _, _}, + {5, _, _}, + {6, _, _} + ] = read_sparse(Seg, [2, 4, 5, 6]) + after 3000 -> + flush(), + ok = gen_server:stop(TblWriterPid), + throw(ra_log_event_timeout) + end, + ok. + %%% Internal fake_mem_table(UId, Dir, Entries) -> diff --git a/test/ra_log_segments_SUITE.erl b/test/ra_log_segments_SUITE.erl new file mode 100644 index 000000000..4d0be398a --- /dev/null +++ b/test/ra_log_segments_SUITE.erl @@ -0,0 +1,787 @@ +%% 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_log_segments_SUITE). + +-compile(nowarn_export_all). +-compile(export_all). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("kernel/include/file.hrl"). + +%% common ra_log tests to ensure behaviour is equivalent across +%% ra_log backends + +all() -> + [ + {group, tests} + ]. + +all_tests() -> + [ + recover1, + recover2, + recover3, + recover4, + compactions_are_not_triggered_concurrently, + basics, + major, + major_max_size, + major_max_size_2, + minor, + overwrite, + result_after_segments, + result_after_segments_overwrite + ]. + +groups() -> + [ + {tests, [], all_tests()} + ]. + +init_per_testcase(TestCase, Config) -> + PrivDir = ?config(priv_dir, Config), + Dir = filename:join(PrivDir, TestCase), + ok = ra_lib:make_dir(Dir), + CompConf = #{max_count => 128, + max_size => 128_000}, + [{uid, atom_to_binary(TestCase, utf8)}, + {comp_conf, CompConf}, + {test_case, TestCase}, + {dir, Dir} | Config]. + +end_per_testcase(_, Config) -> + Config. + +%% TESTS + +result_after_segments(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {major, 128 * 3, Live}, + {entries, 1, lists:seq(128 * 3, 128 * 4)}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, 1, lists:seq(128 * 3, 128 * 4)}, + {assert, fun (S) -> + ct:pal("seg refs ~p", [ra_log_segments:segment_refs(S)]), + true + end} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ct:pal("infos ~p", [infos(Dir)]), + ok. + +result_after_segments_overwrite(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 2, 8), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {major, 128 * 3, Live}, + {entries, 2, lists:seq(128 * 2, 128 * 4)}, + handle_compaction_result, + {print, "1"}, + {assert, 1, ra_seq:expand(ra_seq:limit(128 * 2, Live))}, + {assert, 2, lists:seq(128 * 2, 128 * 3)}, + {print, "2"}, + {assert, fun (S) -> + ct:pal("seg refs ~p", [ra_log_segments:segment_refs(S)]), + true + end}, + {assert, 2, lists:seq(128 * 3, 128 * 4)} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ct:pal("infos ~p", [infos(Dir)]), + ok. + +recover1(Config) -> + %% major compactions can be interrupted at a variety of points and each + %% needs to be handled carefully to ensure the log isn't incorrectly + %% recovered + + %% There is a .compacting file in the segments directory + %% 1. Compaction stopped before the compacting segment got renamed to + %% the lowest numbered segments. + %% 2. Compaction stopped before or during additional segments in the group + %% where linked (.compacting link count > 1) + + Dir = ?config(dir, Config), + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + SegmentFn = <<"0000000000000001.segment">>, + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + Group = term_to_binary([SegmentFn]), + ok = ra_lib:write_file(CompactionGroupFn, Group), + ok = ra_lib:write_file(CompactingFn, <<>>), + true + end}, + reinit, + {assert, fun (_) -> + %% compaction groups and compacting files should be removed + not filelib:is_file(CompactionGroupFn) andalso + not filelib:is_file(CompactingFn) + end}, + {assert, fun (_) -> + %% only compaction_group case + Group = term_to_binary([SegmentFn]), + ok = ra_lib:write_file(CompactionGroupFn, Group), + true + end}, + reinit, + {assert, fun (_) -> + %% compaction groups and compacting files should be removed + not filelib:is_file(CompactionGroupFn) andalso + not filelib:is_file(CompactingFn) + end}, + {assert, fun (_) -> + %% partially written compaction_group file case + ok = ra_lib:write_file(CompactionGroupFn, <<"banana">>), + true + end}, + reinit, + {assert, fun (_) -> + %% compaction groups and compacting files should be removed + not filelib:is_file(CompactionGroupFn) andalso + not filelib:is_file(CompactingFn) + end} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. + +recover2(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + %% compactable segrefs + SegRefs = lists:reverse(tl(seg_refs(Dir))), + Segments = [F || {F, _} <- SegRefs], + %% write compaction_group file + ok = ra_lib:write_file(CompactionGroupFn, + term_to_binary(Segments)), + do_compaction(Dir, CompactingFn, Live, SegRefs), + + [{FstFn, _}, {SndFn, _}, {_ThrFn, _}] = SegRefs, + + %% create a .link for for the second compacted segment + %% and symlink it to the first segment in the compaction + %% group + FirstSegmentFn = filename:join(Dir, FstFn), + SndLinkFn = filename:join(Dir, with_ext(SndFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, SndLinkFn), + + %% rename the .link file on top of the .segment file + SndSegFn = filename:join(Dir, SndFn), + ok = prim_file:rename(SndLinkFn, SndSegFn), + %% this simulates a case where it stopped after only + %% creating 1 of the two symlinks + true + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (_) -> + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + NumLinks = length([a || #{file_type := symlink} <- Infos]), + %% a compacting file with 1 link only should just be deleted + %% during init + not filelib:is_file(CompactingFn) andalso + not filelib:is_file(CompactionGroupFn) andalso + NumLinks == 2 + end}, + {assert, 1, LiveList}, + {assert, fun(S) -> + SegRefs = ra_log_segments:segment_refs(S), + SegRefs == seg_refs(Dir) + end} + ], + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), + ok. + +recover3(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + %% compactable segrefs + SegRefs = lists:reverse(tl(seg_refs(Dir))), + Segments = [F || {F, _} <- SegRefs], + %% write compaction_group file + ok = ra_lib:write_file(CompactionGroupFn, + term_to_binary(Segments)), + do_compaction(Dir, CompactingFn, Live, SegRefs), + + [{FstFn, _}, {SndFn, _}, {_ThrFn, _}] = SegRefs, + + %% create a .link for for the second compacted segment + %% and symlink it to the first segment in the compaction + %% group + FirstSegmentFn = filename:join(Dir, FstFn), + SndLinkFn = filename:join(Dir, with_ext(SndFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, SndLinkFn), + true + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (_) -> + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + NumLinks = length([a || #{file_type := symlink} <- Infos]), + %% a compacting file with 1 link only should just be deleted + %% during init + not filelib:is_file(CompactingFn) andalso + not filelib:is_file(CompactionGroupFn) andalso + NumLinks == 2 + end}, + {assert, 1, LiveList}, + {assert, fun(S) -> + SegRefs = ra_log_segments:segment_refs(S), + SegRefs == seg_refs(Dir) + end} + ], + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), + ok. + +recover4(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + %% compactable segrefs + SegRefs = lists:reverse(tl(seg_refs(Dir))), + Segments = [F || {F, _} <- SegRefs], + %% write compaction_group file + ok = ra_lib:write_file(CompactionGroupFn, + term_to_binary(Segments)), + do_compaction(Dir, CompactingFn, Live, SegRefs), + + [{FstFn, _}, {SndFn, _}, {ThrFn, _}] = SegRefs, + + %% create a .link for for the second compacted segment + %% and symlink it to the first segment in the compaction + %% group + FirstSegmentFn = filename:join(Dir, FstFn), + SndLinkFn = filename:join(Dir, with_ext(SndFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, SndLinkFn), + ThrLinkFn = filename:join(Dir, with_ext(ThrFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, ThrLinkFn), + %% all symlinks completed but .compacting file was not + %% renamed + true + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (_) -> + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + NumLinks = length([a || #{file_type := symlink} <- Infos]), + %% a compacting file with 1 link only should just be deleted + %% during init + not filelib:is_file(CompactingFn) andalso + not filelib:is_file(CompactionGroupFn) andalso + NumLinks == 2 + end}, + {assert, 1, LiveList}, + {assert, fun(S) -> + SegRefs = ra_log_segments:segment_refs(S), + ct:pal("SegRefs ~p, ~p", [SegRefs, seg_refs(Dir)]), + SegRefs == seg_refs(Dir) + end} + ], + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), + ok. + +compactions_are_not_triggered_concurrently(Config) -> + %% creates 3 segments then a snapshot at the first index of the last segment + %% with live indexes only in the first segment. + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128, 5), + Live = ra_seq:from_list(LiveList), + ct:pal("Live ~p", [Live]), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {assert, 1, lists:seq(1, 128 * 3)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + {major, 128 * 2, Live}, + %% this compaction should be dropped + {major, 128 * 3, Live}, + {assert, + fun (_S) -> + receive + {ra_log_event, {compaction_result, _}} = Evt -> + receive + {ra_log_event, {compaction_result, Res}} -> + ct:pal("unexpected second compaction result ~p", [Res]), + false + after 100 -> + self() ! Evt, + true + end + after 1000 -> + flush(), + false + end + end}, + handle_compaction_result, + {major, 128 * 3, Live}, + handle_compaction_result + ], + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), + ok. + +basics(Config) -> + %% creates 3 segments then a snapshot at the first index of the last segment + %% with live indexes only in the first segment. + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128, 5), + Live = ra_seq:from_list(LiveList), + ct:pal("Live ~p", [Live]), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {assert, 1, lists:seq(1, 128 * 3)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + %% this compaction will delete 1 segment (segment 2) + {minor, 128 * 2, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 + end}, + %% this compaction will compact segment 1 + {major, 128 * 2, Live}, + handle_compaction_result, + reinit, + {assert, 1, LiveList}, + {assert, fun (S) -> + [#{num_entries := NumEntries} |_ ] = infos(Dir), + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 andalso + ra_seq:length(Live) == NumEntries + + end} + ], + + + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + + ct:pal("infos ~p", [infos(Dir)]), + ok. + +minor(Config) -> + LiveList = [1 | lists:seq(257, 500, 10)], + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, lists:seq(1, 500)}, + {assert, 1, lists:seq(1, 500)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 4 + end}, + {minor, 500, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + + %% simulate a purge command + {entries, 1, [501]}, + {minor, 501, []}, + handle_compaction_result, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 1 + end} + ], + + SegConf = #{max_count => 128}, + Dir = ?config(dir, Config), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + + ok. + +overwrite(Config) -> + Live = ra_seq:from_list(lists:seq(1, 600, 10)), + Scen = + [ + {entries, 1, lists:seq(1, 500)}, + {entries, 2, lists:seq(200, 700)}, + reinit, + {assert, 1, lists:seq(1, 200, 10)}, + {assert, 2, lists:seq(201, 700, 10)}, + {minor, 600, Live}, + handle_compaction_result, + {major, 600, Live}, + {assert, 1, lists:seq(1, 200, 10)}, + {assert, 2, lists:seq(201, 700, 10)}, + handle_compaction_result, + {assert, 1, lists:seq(1, 200, 10)}, + {assert, 2, lists:seq(201, 700, 10)} + ], + + SegConf = #{max_count => 128}, + Dir = ?config(dir, Config), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. + +major_max_size_2(Config) -> + %% this test could compact 3 segemtns into one just based on entry counts + %% however the max_size configuration needs to be taken into account + %% with the compaction grouping and not create an oversized taget segment + Dir = ?config(dir, Config), + Data1 = crypto:strong_rand_bytes(1000), + Data2 = crypto:strong_rand_bytes(2000), + Entries1 = [{I, 1, term_to_binary(Data1)} + || I <- lists:seq(1, 64)], + Entries2 = [{I, 1, term_to_binary(Data2)} + || I <- lists:seq(65, 130)], + %% only the smaller entries are still live + LiveList = lists:seq(1, 65), + Live = ra_seq:from_list(LiveList), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, Entries1}, + {entries, 1, Entries2}, + {assert, 1, lists:seq(1, 130)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 + end}, + {major, 130, Live}, + handle_compaction_result, + {assert, fun (S) -> + %% infos contain one symlink + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + %% assert this segment was compacted due to potential + %% data reclamation and not just for entries + ?assertMatch(#{num_entries := 65}, hd(Infos)), + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 + end} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. + +major_max_size(Config) -> + %% this test could compact 3 segemtns into one just based on entry counts + %% however the max_size configuration needs to be taken into account + %% with the compaction grouping and not create an oversized taget segment + Dir = ?config(dir, Config), + Data = crypto:strong_rand_bytes(2000), + Entries = [{I, 1, term_to_binary(Data)} + || I <- lists:seq(1, 128 * 4)], + LiveList = lists:seq(1, 30) ++ + lists:seq(128, 128 + 30) ++ + lists:seq(256, 256 + 30), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, Entries}, + {assert, 1, lists:seq(1, 128 * 4)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 4 + end}, + {major, 128 * 4, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + %% infos contain one symlink + Infos = infos(Dir), + Symlinks = [I || #{file_type := symlink} = I <- Infos], + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 andalso + length(Infos) == 4 andalso + length(Symlinks) == 1 + end} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. + +major(Config) -> + Dir = ?config(dir, Config), + Entries1 = lists:seq(1, 641), + LiveList = lists:seq(1, 383, 3) ++ + lists:seq(384, 511) ++ %% the 4th segment is still full + lists:seq(512, 640, 3), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, Entries1}, + {assert, 1, lists:seq(1, 641)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 6 + end}, + {major, 640, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + %% infos contain one symlink + Infos = infos(Dir), + Symlinks = [I || #{file_type := symlink} = I <- Infos], + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 5 andalso + length(Infos) == 6 andalso + length(Symlinks) == 1 + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 5 + end}, + {assert, + fun (_S) -> + ok = ra_log_segments:purge_symlinks(Dir, 1), + [_] = [I || #{file_type := symlink} = I <- infos(Dir)], + timer:sleep(2000), + ok = ra_log_segments:purge_symlinks(Dir, 1), + Infos = infos(Dir), + Symlinks = [I || #{file_type := symlink} = I <- Infos], + Files = [I || #{file_type := regular} = I <- Infos], + length(Symlinks) == 0 andalso + length(Files) == 5 + end} + ], + + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + + ok. + +%% Helpers + +open_last_segment(Config, SegConf) -> + Dir = ?config(dir, Config), + case seg_refs(Dir) of + [] -> + Fn = ra_lib:zpad_filename("", "segment", 1), + SegFn = filename:join(Dir, Fn), + {ok, Seg} = ra_log_segment:open(SegFn, SegConf), + Seg; + [{Fn, _} | _] -> + SegFn = filename:join(Dir, Fn), + {ok, Seg} = ra_log_segment:open(SegFn, SegConf), + Seg + end. + + + +append_to_segment(Seg0, [], Refs, _Conf) -> + {Seg0, [ra_log_segment:segref(Seg0) | Refs]}; +append_to_segment(Seg0, [{Idx, Term, Data} | Rem] = All, Refs, Conf) -> + DataSize = iolist_size(Data), + case ra_log_segment:append(Seg0, Idx, Term, {DataSize, Data}) of + {ok, Seg} -> + append_to_segment(Seg, Rem, Refs, Conf); + {error, full} -> + Ref = ra_log_segment:segref(Seg0), + % close and open a new segment + append_to_segment(open_successor_segment(Seg0, Conf), All, + [Ref | Refs], Conf) + end. + +open_successor_segment(CurSeg, SegConf) -> + Fn0 = ra_log_segment:filename(CurSeg), + Fn = ra_lib:zpad_filename_incr(Fn0), + ok = ra_log_segment:close(CurSeg), + {ok, Seg} = ra_log_segment:open(Fn, SegConf), + Seg. + + +seg_refs(Dir) -> + lists:reverse( + [ra_log_segment:segref(F) || F <- segment_files(Dir), + is_regular(F) andalso + lists:member(filename:extension(F), [".segment", <<".segment">>])]). + +infos(Dir) -> + [ra_log_segment:info(F) || F <- segment_files(Dir)]. + +segment_files(Dir) -> + case prim_file:list_dir(Dir) of + {ok, Files0} -> + Files = [filename:join(Dir, F) + || F <- Files0, + begin + Ext = filename:extension(F), + lists:member(Ext, [".segment", + % ".compacting", + % ".compacted", + ".link"]) + end], + lists:sort(Files); + {error, enoent} -> + [] + end. + +flush() -> + receive + Any -> + ct:pal("flush ~p", [Any]), + flush() + after 0 -> + ok + end. + +is_regular(Filename) -> + {ok, #file_info{type = T}} = file:read_link_info(Filename, [raw, {time, posix}]), + T == regular. + +run_scenario(_, Segs, []) -> + Segs; +run_scenario(Config, Segs0, [reinit | Rem]) -> + Dir = ?config(dir, Config), + CompConf = ?config(comp_conf, Config), + ra_log_segments:close(Segs0), + Segs = ra_log_segments:init(?config(uid, Config), Dir, 1, random, + seg_refs(Dir), undefined, CompConf, ""), + ?FUNCTION_NAME(Config, Segs, Rem); +run_scenario(Config, Segs0, [{entries, Term, IndexesOrEntries} | Rem]) -> + SegConf = ?config(seg_conf, Config), + Seg0 = open_last_segment(Config, SegConf), + Entries = case is_tuple(hd(IndexesOrEntries)) of + true -> + IndexesOrEntries; + false -> + [{I, Term, term_to_binary(<<"data1">>)} + || I <- IndexesOrEntries] + end, + {Seg, Refs} = append_to_segment(Seg0, Entries, [], SegConf), + _ = ra_log_segment:close(Seg), + {Segs, _Overwritten} = ra_log_segments:update_segments(Refs, Segs0), + %% TODO: what to do about overwritten + ?FUNCTION_NAME(Config, Segs, Rem); +run_scenario(Config, Segs0, [{Type, SnapIdx, Live} | Rem]) + when Type == major orelse Type == minor -> + {Segs1, Effs} = ra_log_segments:schedule_compaction(Type, SnapIdx, Live, Segs0), + Segs = lists:foldl(fun ({bg_work, Fun, _}, S0) -> + Fun(), + S0; + ({next_event, {ra_log_event, + {compaction_result, _Res}} = E}, S0) -> + self() ! E, + S0 + end, Segs1, Effs), + + ?FUNCTION_NAME(Config, Segs, Rem); +run_scenario(Config, Segs0, [handle_compaction_result = Step | Rem]) -> + CompRes3 = receive + {ra_log_event, {compaction_result, Res3}} -> + ct:pal("compaction result ~p", [Res3]), + Res3 + after 5000 -> + flush(), + exit({ra_log_event_timeout, Step}) + end, + + {Segs1, Effs} = + ra_log_segments:handle_compaction_result(CompRes3, Segs0), + [Fun1() || {bg_work, Fun1, _} <- Effs], + ?FUNCTION_NAME(Config, Segs1, Rem); +run_scenario(Config, Segs0, [{assert, Term, Indexes} | Rem]) -> + {Read, Segs2} = ra_log_segments:sparse_read(Segs0, + lists:reverse(Indexes), []), + %% assert we can read + ?assertEqual(length(Indexes), length(Read)), + ?assert(lists:all(fun ({_, T, _}) -> T == Term end, Read)), + ?FUNCTION_NAME(Config, Segs2, Rem); +run_scenario(Config, Segs0, [{assert, Fun} | Rem]) + when is_function(Fun) -> + ?assert(Fun(Segs0)), + ?FUNCTION_NAME(Config, Segs0, Rem); +run_scenario(Config, Segs0, [{print, What} | Rem]) -> + ct:pal(What), + ?FUNCTION_NAME(Config, Segs0, Rem). + + +with_ext(Fn, Ext) when is_binary(Fn) andalso is_list(Ext) -> + <<(filename:rootname(Fn))/binary, (ra_lib:to_binary(Ext))/binary>>. + +ra_log_segments_init(Config, Dir, SegRefs) -> + UId = ?config(uid, Config), + CompConf = ?config(comp_conf, Config), + ra_log_segments:init(UId, Dir, 1, random, + SegRefs, undefined, + CompConf, ""). + +do_compaction(Dir, CompactingFn, Live, All) -> + {ok, CompSeg0} = ra_log_segment:open(CompactingFn, + #{max_count => 128}), + CompSeg = lists:foldl( + fun ({F, R}, S0) -> + L = ra_seq:in_range(R, Live), + {ok, S} = ra_log_segment:copy( + S0, filename:join(Dir, F), + ra_seq:expand(L)), + S + end, CompSeg0, All), + ok = ra_log_segment:close(CompSeg), + ok. diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index c9d1a45b4..bc52fb0d0 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,10 +113,9 @@ init_per_testcase(TestCase, Config) -> TestCase, TestCase), Names = maps:get(names, Sys), WalConf = #{dir => Dir, - name => ra_log_wal, system => ?SYS, - 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}]), @@ -137,24 +137,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), @@ -162,23 +285,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(), @@ -194,25 +314,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(), @@ -227,23 +348,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(), @@ -258,21 +379,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(), @@ -288,24 +409,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(), @@ -321,7 +442,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), @@ -330,21 +451,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(), @@ -359,13 +483,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), @@ -379,13 +503,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 -> @@ -401,22 +525,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(), @@ -431,22 +555,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(), @@ -486,7 +610,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), @@ -509,7 +634,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), @@ -540,7 +666,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), @@ -560,7 +687,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 @@ -603,7 +730,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), @@ -611,7 +739,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 @@ -626,9 +754,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), @@ -645,13 +773,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. @@ -663,8 +791,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), @@ -674,11 +801,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 @@ -698,19 +825,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(), @@ -730,19 +856,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 -> @@ -756,19 +881,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), @@ -778,14 +911,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), @@ -798,7 +931,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(), @@ -806,7 +939,7 @@ recover(Config) -> end, receive {'$gen_cast', - {mem_tables, #{UId := [{MtTid, {101, 200}}]}, _}} -> + {mem_tables, #{UId := [{MtTid, [{101, 200}]}]}, _}} -> ok after 2000 -> flush(), @@ -829,9 +962,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), @@ -840,11 +972,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), @@ -852,7 +985,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(), @@ -865,9 +998,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), @@ -877,15 +1009,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), @@ -894,8 +1026,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 -> @@ -908,9 +1040,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), @@ -920,14 +1051,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), @@ -938,7 +1069,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(), @@ -946,7 +1077,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(), @@ -959,9 +1090,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), @@ -975,7 +1105,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), @@ -984,7 +1114,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(), @@ -1007,9 +1137,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), @@ -1023,7 +1152,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)), @@ -1034,7 +1163,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(), @@ -1046,9 +1175,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), @@ -1062,7 +1190,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), @@ -1073,7 +1201,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), @@ -1085,8 +1213,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(), @@ -1102,36 +1230,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(), @@ -1147,23 +1272,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)), @@ -1171,7 +1296,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 -> @@ -1186,8 +1311,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), @@ -1200,9 +1324,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), @@ -1210,7 +1333,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), @@ -1229,7 +1352,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(), @@ -1244,8 +1367,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), @@ -1253,7 +1375,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), @@ -1275,8 +1397,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), @@ -1284,7 +1405,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), @@ -1308,8 +1429,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), @@ -1317,7 +1437,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), @@ -1342,25 +1462,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,6 @@ suspend_process(Pid) -> erlang:raise(error, internal_error, Stack) 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 612c827a4..b78520257 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 2d8ec804b..39f23ae7b 100644 --- a/test/ra_machine_int_SUITE.erl +++ b/test/ra_machine_int_SUITE.erl @@ -48,6 +48,7 @@ all_tests() -> aux_eval, aux_tick, aux_handler_not_impl, + consistent_aux, aux_command, aux_command_v2, aux_command_v1_and_v2, @@ -610,6 +611,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 @@ -665,6 +667,50 @@ aux_handler_not_impl(Config) -> {error, aux_handler_not_implemented} = ra:aux_command(Leader, emit), ok. +consistent_aux(Config) -> + ClusterName = ?config(cluster_name, Config), + ServerId1 = ?config(server_id, Config), + ServerId2 = ?config(server_id2, Config), + ServerId3 = ?config(server_id3, Config), + Cluster = [ServerId1, ServerId2, ServerId3], + Mod = ?config(modname, Config), + meck:new(Mod, [non_strict]), + meck:expect(Mod, init, fun (_) -> [] end), + meck:expect(Mod, init_aux, fun (_) -> undefined end), + meck:expect(Mod, apply, + fun + (_, Cmd, State) -> + ct:pal("handling ~p", [Cmd]), + %% handle all + {State, ok} + end), + meck:expect(Mod, handle_aux, + fun + (RaftState, {call, _From}, emit, AuxState, Opaque) -> + %% emits aux state + {reply, {RaftState, AuxState}, AuxState, Opaque}; + (_RaftState, {call, _From}, banana, AuxState, Opaque) -> + %% emits aux state + {reply, banana, AuxState, Opaque}; + (_RaftState, cast, eval, AuxState, Opaque) -> + %% replaces aux state + {no_reply, AuxState, Opaque}; + (_RaftState, cast, NewState, _AuxState, Opaque) -> + %% replaces aux state + {no_reply, NewState, Opaque} + end), + ok = start_cluster(ClusterName, {module, Mod, #{}}, Cluster), + timer:sleep(100), + {ok, banana, _} = ra:consistent_aux(ServerId1, banana, 500), + ra:stop_server(?SYS, ServerId2), + {ok, banana, _} = ra:consistent_aux(ServerId1, banana, 500), + ra:stop_server(?SYS, ServerId3), + {timeout, ServerId1} = ra:consistent_aux(ServerId1, banana, 500), + ra:restart_server(?SYS, ServerId2), + ra:restart_server(?SYS, ServerId3), + ra:delete_cluster(Cluster), + ok. + aux_command(Config) -> ClusterName = ?config(cluster_name, Config), ServerId1 = ?config(server_id, Config), diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl index 4aff51ef9..d513b6852 100644 --- a/test/ra_mt_SUITE.erl +++ b/test/ra_mt_SUITE.erl @@ -21,7 +21,10 @@ all() -> all_tests() -> [ basics, + fold, record_flushed, + record_flushed_missed, + record_flushed_missed_prev, record_flushed_after_set_first, record_flushed_prev, set_first, @@ -33,31 +36,15 @@ all_tests() -> successor_below, stage_commit, range_overlap, - stage_commit_abort, - perf + stage_commit_2, + perf, + sparse, + sparse_after_non_sparse ]. 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 %%%=================================================================== @@ -73,11 +60,42 @@ 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), + ?assertEqual(lists:seq(510, 505, -1), + ra_mt:fold(505, 510, fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], 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. +fold(_Config) -> + Tid = ets:new(t1, [set, public]), + Mt0 = ra_mt:init(Tid), + Mt1 = lists:foldl( + fun (I, Acc) -> + element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc)) + end, Mt0, lists:seq(1, 1000)), + {[Spec], Mt2} = ra_mt:set_first(500, Mt1), + 499 = ra_mt:delete(Spec), + ?assertEqual({500, 1000}, ra_mt:range(Mt2)), + ?assertEqual(501, ets:info(Tid, size)), + ?assertEqual(lists:seq(510, 505, -1), + ra_mt:fold(505, 510, fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Mt2)), + ?assertError({missing_key, 1001, _}, + ra_mt:fold(999, 1010, + fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Mt2)), + ?assertEqual([1000, 999], + ra_mt:fold(999, 1010, + fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Mt2, return)), + ok. + record_flushed(_Config) -> %%TODO: test that deletes the same spec twice Tid = ets:new(t1, [set, public]), @@ -86,20 +104,60 @@ 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)), ok. +record_flushed_missed(_Config) -> + Tid = ets:new(t1, [set, public]), + Mt0 = ra_mt:init(Tid), + Mt1 = lists:foldl( + fun (I, Acc) -> + element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc)) + end, Mt0, lists:seq(1, 105)), + {Spec3, Mt4} = ra_mt:record_flushed(Tid, [{50, 100}], Mt1), + ?assertMatch({indexes, _, [{1, 100}]}, Spec3), + ?assertEqual({101, 105}, ra_mt:range(Mt4)), + _ = ra_mt:delete(Spec3), + ?assertMatch(#{size := 5}, ra_mt:info(Mt4)), + ok. + +record_flushed_missed_prev(_Config) -> + %% test that a prior mem table is cleared up when the current one is + %% recorded flushed + Tid = ets:new(t1, [set, public]), + Mt0 = ra_mt:init(Tid), + Mt1 = lists:foldl( + fun (I, Acc) -> + element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc)) + end, Mt0, lists:seq(1, 49)), + + Tid2 = ets:new(t2, [set, public]), + Mt2 = ra_mt:init_successor(Tid2, read_write, Mt1), + Mt3 = lists:foldl( + fun (I, Acc) -> + element(2, ra_mt:insert({I, 2, <<"apple">>}, Acc)) + end, Mt2, lists:seq(25, 105)), + + {Spec3, Mt4} = ra_mt:record_flushed(Tid2, [{25, 100}], Mt3), + ?assertMatch({multi, [{indexes, Tid2, [{25, 100}]}, + {delete, Tid}]}, Spec3), + ct:pal("Mt4 ~p", [Mt4]), + ?assertEqual({101, 105}, ra_mt:range(Mt4)), + _ = ra_mt:delete(Spec3), + ?assertMatch(#{size := 5}, ra_mt:info(Mt4)), + ok. + record_flushed_after_set_first(_Config) -> Tid = ets:new(t1, [set, public]), Mt0 = ra_mt:init(Tid), @@ -107,14 +165,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. @@ -128,20 +186,22 @@ record_flushed_prev(_Config) -> Tid2 = ets:new(t2, [set, public]), Mt2 = ra_mt:init_successor(Tid2, read_write, Mt1), + ?assertMatch({1, 100}, ra_mt:range(Mt2)), Mt3 = lists:foldl( fun (I, Acc) -> element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc)) end, Mt2, lists:seq(50, 80)), ?assertMatch({1, 100}, ra_mt:range(ra_mt:prev(Mt3))), + ?assertMatch({1, 80}, ra_mt:range(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,13 +223,14 @@ 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({delete, Tid}, Spec2), ?assertMatch({75, 120}, ra_mt:range(Mt4)), + ?assertMatch(undefined, ra_mt:prev(Mt4)), - {[Spec3, Spec4], Mt5} = ra_mt:set_first(105, Mt4), - ?assertMatch({range, Tid2, {75, 104}}, Spec3), - ?assertMatch({delete, Tid}, Spec4), + {[Spec3], Mt5} = ra_mt:set_first(105, Mt4), + ?assertMatch({indexes, Tid2, [{75, 104}]}, Spec3), + % ?assertMatch({delete, Tid}, Spec4), ?assertMatch({105, 120}, ra_mt:range(Mt5)), ?assertMatch(undefined, ra_mt:prev(Mt5)), ok. @@ -198,13 +259,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), - - {[{range, Tid3, {80, 159}}, + {[{indexes, Tid3, [{75, 79}]}, {delete, Tid2}, - {delete, Tid1}], _Mt5} = ra_mt:set_first(160, Mt4), + {delete, Tid1}], Mt4} = ra_mt:set_first(80, Mt3), + + {[{indexes, Tid3, [{80, 159}]}], _Mt5} = ra_mt:set_first(160, Mt4), ok. set_first_with_middle_small_range(_Config) -> @@ -233,13 +292,13 @@ 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), + {delete, Tid1}], Mt4} = ra_mt:set_first(85, Mt3), ?assertEqual({85, 200}, ra_mt:range(Mt4)), - {[{range, Tid3, {85, 100}}, - {delete, Tid1}], Mt5} = ra_mt:set_first(101, Mt4), + {[{indexes, Tid3, [{85, 100}]} + ], Mt5} = ra_mt:set_first(101, Mt4), ?assertEqual({101, 200}, ra_mt:range(Mt5)), ?assertEqual(undefined, ra_mt:prev(Mt5)), @@ -261,13 +320,11 @@ 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}]}, + {delete, Tid1}], 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 - {[{delete, Tid1}], Mt4} = ra_mt:set_first(101, Mt3), - ?assertEqual(undefined, ra_mt:prev(Mt4)), + ?assertEqual(undefined, ra_mt:prev(Mt3)), ok. set_first_with_old_smaller_range(_Config) -> @@ -286,7 +343,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. @@ -301,6 +358,7 @@ successor(_Config) -> ?assertMatch({1, 100}, ra_mt:range(Mt1)), Tid2 = ets:new(t2, [set, public]), Mt2 = ra_mt:init_successor(Tid2, read_write, Mt1), + ?assertMatch({1, 100}, ra_mt:range(Mt2)), Mt3 = lists:foldl( fun (I, Acc) -> element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc)) @@ -312,10 +370,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 +394,20 @@ 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)), + {{multi, [{indexes, Tid2, [{50, 60}]}, + {delete, Tid}]}, 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 +448,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 +458,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 +528,46 @@ 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">>}, 1, 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. + +sparse_after_non_sparse(_Config) -> + Tid = ets:new(t1, [set, public]), + Mt0 = ra_mt:init(Tid), + Mt1 = lists:foldl( + fun (I, M0) -> + {ok, M} = ra_mt:insert({I, 1, <<"banana">>}, M0), + M + end, Mt0, lists:seq(1, 10)), + Tid2 = ets:new(t2, [set, public]), + Mt2 = ra_mt:init_successor(Tid2, read_write, Mt1), + + {ok, Mt3} = ra_mt:insert_sparse({12, 1, <<"banana">>}, undefined, Mt2), + {ok, Mt4} = ra_mt:insert_sparse({15, 1, <<"banana">>}, 12, Mt3), + ?assertEqual({1, 15}, ra_mt:range(Mt4)), + ?assertMatch(#{size := 2, + range := {1, 15}}, ra_mt:info(Mt4)), + + {Spec, Mt5} = ra_mt:record_flushed(Tid, ra_seq:from_list(lists:seq(1, 10)), Mt4), + %% full table delete + 10 = ra_mt:delete(Spec), + {Spec2, Mt6} = ra_mt:record_flushed(Tid2, ra_seq:from_list([12, 15]), Mt5), + 2 = ra_mt:delete(Spec2), + ?assertMatch(#{size := 0, + range := undefined}, ra_mt:info(Mt6)), + ?assertEqual(0, ets:info(Tid2, size)), + ok. %%% Util diff --git a/test/ra_seq_SUITE.erl b/test/ra_seq_SUITE.erl new file mode 100644 index 000000000..1a1ca0f1e --- /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 23a4e402c..325a11d77 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, @@ -58,6 +64,8 @@ all() -> leader_appends_cluster_change_then_steps_before_applying_it, leader_receives_install_snapshot_rpc, follower_installs_snapshot, + follower_installs_snapshot_with_pre, + follower_aborts_snapshot_with_pre, follower_ignores_installs_snapshot_with_higher_machine_version, follower_receives_stale_snapshot, follower_receives_snapshot_lower_than_last_applied, @@ -152,32 +160,46 @@ 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, {Meta, _}) -> + {ok, {Meta, Data}, []} + end), + meck:expect(ra_snapshot, complete_accept, + fun(Data, _Num, _Machine, {_Meta, MacSt} = State) -> + {State, Data ++ MacSt, [], []} end), - meck:expect(ra_snapshot, abort_accept, fun(SS) -> SS end), - meck:expect(ra_snapshot, accepting, fun(_SS) -> undefined end), + meck:expect(ra_snapshot, abort_accept, fun(_SS) -> undefined end), + meck:expect(ra_snapshot, accepting, fun({Meta, _}) -> + {maps:get(index, Meta), + maps:get(term, Meta)}; + (_) -> + 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, fold, fun (A, B, C, D, E, _) -> + ra_log_memory:fold(A, B, C, D, E) + end), meck:expect(ra_log, release_resources, fun ra_log_memory:release_resources/3), + meck:expect(ra_log, overview, fun ra_log_memory:overview/1), 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), meck:expect(ra_log, append, fun ra_log_memory:append/2), meck:expect(ra_log, write, fun ra_log_memory:write/2), + meck:expect(ra_log, write_sparse, fun ra_log_memory:write_sparse/3), meck:expect(ra_log, handle_event, fun ra_log_memory:handle_event/2), meck:expect(ra_log, last_written, fun ra_log_memory:last_written/1), meck:expect(ra_log, last_index_term, fun ra_log_memory:last_index_term/1), @@ -222,8 +244,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), + {ok, Log1, _} = ra_log_memory:install_snapshot({3, 5}, + ?MODULE, [], Log0), + LogS = ra_log:set_snapshot_state({SnapshotMeta, SnapshotData}, Log1), meck:expect(ra_log, init, fun (_) -> LogS end), #{current_term := 5, commit_index := 3, @@ -259,7 +282,7 @@ recover_restores_cluster_changes(_Config) -> % n2 joins {leader, #{cluster := Cluster, - log := Log0}, _} = + log := Log0}, _Effs} = ra_server:handle_leader({command, {'$ra_join', meta(), N2, await_consensus}}, State), {LIdx, _} = ra_log:last_index_term(Log0), @@ -722,7 +745,9 @@ follower_aer_term_mismatch_snapshot(_Config) -> cluster => #{}, machine_version => 1}, Data = <<"hi3">>, - {Log,_} = ra_log_memory:install_snapshot({3, 5}, {Meta, Data}, Log0), + + {ok, Log1, _} = ra_log_memory:install_snapshot({3, 5}, ?MODULE, [], Log0), + Log = ra_log_memory:set_snapshot_state({Meta, Data}, Log1), State = maps:put(log, Log, State0), AE = #append_entries_rpc{term = 6, @@ -1690,6 +1715,10 @@ follower_install_snapshot_machine_version(_Config) -> _Effects0} = ra_server:handle_follower(ISR, State00), meck:expect(ra_log, recover_snapshot, fun (_) -> {SnapMeta, SnapData} end), + meck:expect(ra_snapshot, complete_accept, + fun (_, _, _, S) -> + {S, SnapData, [], []} + end), {follower, #{cfg := #cfg{machine_version = _,%% this gets populated on init only machine_versions = [{4, 1}, {0,0}], effective_machine_module = MacMod1, @@ -1867,8 +1896,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, @@ -2234,31 +2263,28 @@ leader_receives_install_snapshot_rpc(_Config) -> % leader ignores lower term {leader, State, _} = ra_server:handle_leader(ISRpc#install_snapshot_rpc{term = Term - 1}, - State), + State), ok. follower_installs_snapshot(_Config) -> N1 = ?N1, N2 = ?N2, N3 = ?N3, - #{N3 := {_, FState = #{cluster := Config}, _}} - = init_servers([N1, N2, N3], {module, ra_queue, #{}}), + #{N3 := {_, FState = #{cluster := Config}, _}} = + init_servers([N1, N2, N3], {module, ra_queue, #{}}), LastTerm = 1, % snapshot term Term = 2, % leader term Idx = 3, + ct:pal("FState ~p", [FState]), ISRpc = #install_snapshot_rpc{term = Term, leader_id = N1, meta = snap_meta(Idx, LastTerm, Config), chunk_state = {1, last}, data = []}, {receive_snapshot, FState1, [{next_event, ISRpc}, {record_leader_msg, _}]} = - ra_server:handle_follower(ISRpc, FState), + ra_server:handle_follower(ISRpc, FState#{current_term => Term}), - meck:expect(ra_log, recover_snapshot, - fun (_) -> - {#{index => Idx, - term => Term, - cluster => dehydrate_cluster(Config), - machine_version => 0}, - []} + meck:expect(ra_snapshot, complete_accept, + fun (_, _, _, S) -> + {S, [], [], []} end), {follower, #{current_term := Term, commit_index := Idx, @@ -2271,6 +2297,96 @@ follower_installs_snapshot(_Config) -> ok. +follower_installs_snapshot_with_pre(_Config) -> + N1 = ?N1, N2 = ?N2, N3 = ?N3, + #{N3 := {_, State = #{cluster := Config}, _}} = + init_servers([N1, N2, N3], {module, ra_queue, #{}}), + LastTerm = 1, % snapshot term + Term = 2, % leader term + Idx = 3, + ISRpcInit = #install_snapshot_rpc{term = Term, leader_id = N1, + meta = snap_meta(Idx, LastTerm, Config), + chunk_state = {0, init}, + data = []}, + %% the init message starts the process + {receive_snapshot, State1, + [{next_event, ISRpc}, {record_leader_msg, _}]} = + ra_server:handle_follower(ISRpcInit, State#{current_term => Term}), + + %% actually process init message in the correct state + {receive_snapshot, State2, [{reply, _}]} = + ra_server:handle_receive_snapshot(ISRpc, State1), + + %% check a higher snapshot index reverts to follower + {follower, _, [{next_event, _}]} = + ra_server:handle_receive_snapshot( + ISRpcInit#install_snapshot_rpc{meta = snap_meta(Idx + 1, LastTerm, Config)}, + State1), + %% now send a pre message + ISRpcPre = ISRpcInit#install_snapshot_rpc{chunk_state = {0, pre}, + data = [{2, 1, <<"e1">>}]}, + {receive_snapshot, State3, [{reply, _}]} = + ra_server:handle_receive_snapshot(ISRpcPre, State2), + + %% test that init returns to follower and retries + {follower, State3b, [{next_event, ISRpcInit}]} = + ra_server:handle_receive_snapshot(ISRpcInit, State3), + ?assertNot(maps:is_key(snapshot_phase, State3b)), + + meck:expect(ra_snapshot, complete_accept, + fun (Mac, _, _, S) -> + {S, Mac, [], []} + end), + + %% finally process the actual snapshot + ISRpc1 = ISRpc#install_snapshot_rpc{chunk_state = {1, last}, + data = [2]}, + {follower, #{current_term := Term, + commit_index := Idx, + last_applied := Idx, + cluster := Config, + machine_state := [2], + leader_id := N1} = _State, + [{reply, #install_snapshot_result{}}]} = + ra_server:handle_receive_snapshot(ISRpc1, State3), + ok. + +follower_aborts_snapshot_with_pre(_Config) -> + N1 = ?N1, N2 = ?N2, N3 = ?N3, + #{N3 := {_, State = #{cluster := Config}, _}} = + init_servers([N1, N2, N3], {module, ra_queue, #{}}), + LastTerm = 1, % snapshot term + Term = 2, % leader term + Idx = 3, + ISRpcInit = #install_snapshot_rpc{term = Term, leader_id = N1, + meta = snap_meta(Idx, LastTerm, Config), + chunk_state = {0, init}, + data = []}, + %% the init message starts the process + {receive_snapshot, State1, + [{next_event, ISRpc}, {record_leader_msg, _}]} = + ra_server:handle_follower(ISRpcInit, State#{current_term => Term}), + + %% actually process init message in the correct state + {receive_snapshot, State2, [{reply, _}]} = + ra_server:handle_receive_snapshot(ISRpc, State1), + + %% now send a pre message + ISRpcPre = ISRpcInit#install_snapshot_rpc{chunk_state = {0, pre}, + data = [{2, 1, <<"e1">>}]}, + {receive_snapshot, State3, [{reply, _}]} = + ra_server:handle_receive_snapshot(ISRpcPre, State2), + ?assertMatch(#{log := #{last_index := 2}}, + ra_server:overview(State3)), + + {follower, State3b, []} = + ra_server:handle_receive_snapshot(receive_snapshot_timeout, State3), + ?assertNot(maps:is_key(snapshot_phase, State3b)), + %% assert the aborted install reset the log + ?assertMatch(#{log := #{last_index := 0}}, + ra_server:overview(State3b)), + ok. + follower_ignores_installs_snapshot_with_higher_machine_version(_Config) -> %% currently followers cannot correctly handle snapshots with a higher %% machine version so have to ignore them @@ -2779,12 +2895,13 @@ leader_heartbeat(_Config) -> leader_heartbeat_reply_node_size_5(_Config) -> N3 = ?N3, + From1 = {self(), make_ref()}, BaseState = base_state(5, ?FUNCTION_NAME), #{current_term := Term, cfg := #cfg{id = Id}, commit_index := CommitIndex} = BaseState, QueryIndex = 2, - QueryRef1 = {from1, fun(_) -> query_result1 end, CommitIndex}, + QueryRef1 = {query, From1, fun(_) -> query_result1 end, CommitIndex}, %% Increase self query index to cover more cases ReplyingPeerId = ?N2, HeartbeatReply = #heartbeat_reply{term = Term, @@ -2797,18 +2914,20 @@ leader_heartbeat_reply_node_size_5(_Config) -> {leader, State, []} = ra_server:handle_leader({ReplyingPeerId, HeartbeatReply}, State0), - {leader, _, [{reply, from1, {ok, query_result1, Id}}]} + {leader, _, [{reply, From1, {ok, query_result1, Id}}]} = ra_server:handle_leader({N3, HeartbeatReply}, State), ok. leader_heartbeat_reply_same_term(_Config) -> + From1 = {self(), make_ref()}, + From2 = {self(), make_ref()}, BaseState = base_state(3, ?FUNCTION_NAME), #{current_term := Term, cfg := #cfg{id = Id}, commit_index := CommitIndex} = BaseState, QueryIndex = 2, - QueryRef1 = {from1, fun(_) -> query_result1 end, CommitIndex}, - QueryRef2 = {from2, fun(_) -> query_result2 end, CommitIndex - 1}, + QueryRef1 = {query, From1, fun(_) -> query_result1 end, CommitIndex}, + QueryRef2 = {query, From2, fun(_) -> query_result2 end, CommitIndex - 1}, %% Increase self query index to cover more cases State = set_peer_query_index(BaseState#{query_index => QueryIndex + 1}, Id, QueryIndex + 1), @@ -2844,7 +2963,7 @@ leader_heartbeat_reply_same_term(_Config) -> %% Reply applies a query if there is a consensus %% A single reply in 3 node cluster is a consensus {leader, StateWithQueryIndexForPeer, - [{reply, from1, {ok, query_result1, Id}}]} + [{reply, From1, {ok, query_result1, Id}}]} = ra_server:handle_leader({ReplyingPeerId, HeartbeatReply}, StateWithQuery), @@ -2873,14 +2992,14 @@ leader_heartbeat_reply_same_term(_Config) -> StateWithSecondQuery = StateWithQueryIndexForPeer#{queries_waiting_heartbeats => WaitingQuery2}, %% Apply single query out of 2 if there is a consensus for some - {leader, StateWithSecondQuery, [{reply, from1, {ok, query_result1, Id}}]} + {leader, StateWithSecondQuery, [{reply, From1, {ok, query_result1, Id}}]} = ra_server:handle_leader({ReplyingPeerId, HeartbeatReply}, StateWithTwoQueries), %% Apply multiple queries if there is consensus for all HighIndexReply = HeartbeatReply#heartbeat_reply{query_index = HighQueryIndex}, {leader, StateWithHighQueryIndexForPeer, - [{reply, from2, {ok, query_result2, Id}}, - {reply, from1, {ok, query_result1, Id}}]} + [{reply, From2, {ok, query_result2, Id}}, + {reply, From1, {ok, query_result1, Id}}]} = ra_server:handle_leader({ReplyingPeerId, HighIndexReply}, StateWithTwoQueries), ok. @@ -2895,17 +3014,20 @@ leader_consistent_query_delay(_Config) -> %% If cluster changes are not permitted - delay the heartbeats Fun = fun(_) -> query_result end, - StateWithPending = State#{pending_consistent_queries => [{from, Fun, CommitIndex}]}, + From = {self(), make_ref()}, + From2 = {self(), make_ref()}, + StateWithPending = State#{pending_consistent_queries => + [{query, From, Fun, CommitIndex}]}, {leader, StateWithPending, []} - = ra_server:handle_leader({consistent_query, from, Fun}, State), + = ra_server:handle_leader({consistent_query, From, Fun}, State), %% Pending stack together %% Order does not matter here, btw. StateWithMorePending = - State#{pending_consistent_queries => [{from1, Fun, CommitIndex}, - {from, Fun, CommitIndex}]}, + State#{pending_consistent_queries => [{query, From2, Fun, CommitIndex}, + {query, From, Fun, CommitIndex}]}, {leader, StateWithMorePending, []} - = ra_server:handle_leader({consistent_query, from1, Fun}, StateWithPending), + = ra_server:handle_leader({consistent_query, From2, Fun}, StateWithPending), QueryIndex1 = QueryIndex + 1, @@ -2918,8 +3040,9 @@ leader_consistent_query_delay(_Config) -> leader_id = Id}, %% Technically, order should not matter here. %% In theory these queries may have the same query index - WaitingQueries = queue:in({QueryIndex2, {from, Fun, CommitIndex}}, - queue:in({QueryIndex1, {from1, Fun, CommitIndex}}, queue:new())), + WaitingQueries = queue:in({QueryIndex2, {query, From, Fun, CommitIndex}}, + queue:in({QueryIndex1, {query, From2, Fun, CommitIndex}}, + queue:new())), %% Send heartbeats as soon as cluster changes permitted {leader, #{cluster_change_permitted := true, @@ -2941,10 +3064,12 @@ leader_consistent_query(_Config) -> query_index := QueryIndex, current_term := Term, cfg := #cfg{id = Id}} = State, + From1 = {self(), make_ref()}, + From2 = {self(), make_ref()}, Fun = fun(_) -> query_result end, - Query1 = {from1, Fun, CommitIndex}, - Query2 = {from2, Fun, CommitIndex}, + Query1 = {query, From1, Fun, CommitIndex}, + Query2 = {query, From2, Fun, CommitIndex}, QueryIndex1 = QueryIndex + 1, QueryIndex2 = QueryIndex1 + 1, WaitingQuery = queue:in({QueryIndex1, Query1}, queue:new()), @@ -2962,13 +3087,13 @@ leader_consistent_query(_Config) -> queries_waiting_heartbeats := WaitingQuery} = StateWithQuery, [{send_rpc, N2, HeartBeatRpc1}, {send_rpc, N3, HeartBeatRpc1}]} = - ra_server:handle_leader({consistent_query, from1, Fun}, State), + ra_server:handle_leader({consistent_query, From1, Fun}, State), {leader, #{query_index := QueryIndex2, queries_waiting_heartbeats := WaitingQueries}, [{send_rpc, N2, HeartBeatRpc2}, {send_rpc, N3, HeartBeatRpc2}]} = - ra_server:handle_leader({consistent_query, from2, Fun}, StateWithQuery). + ra_server:handle_leader({consistent_query, From2, Fun}, StateWithQuery). enable_cluster_change(State0) -> {leader, #{cluster_change_permitted := false} = State1, _Effects} = @@ -3119,15 +3244,16 @@ leader_heartbeat_reply_higher_term(_Config) -> % %%% helpers init_servers(ServerIds, Machine) -> - lists:foldl(fun (ServerId, Acc) -> - Args = #{cluster_name => some_id, - id => ServerId, - uid => atom_to_binary(element(1, ServerId), utf8), - initial_members => ServerIds, - log_init_args => #{uid => <<>>}, - machine => Machine}, - Acc#{ServerId => {follower, ra_server_init(Args), []}} - end, #{}, ServerIds). + lists:foldl( + fun (ServerId, Acc) -> + Args = #{cluster_name => some_id, + id => ServerId, + uid => atom_to_binary(element(1, ServerId), utf8), + initial_members => ServerIds, + log_init_args => #{uid => <<>>}, + machine => Machine}, + Acc#{ServerId => {follower, ra_server_init(Args), []}} + end, #{}, ServerIds). list(L) when is_list(L) -> L; list(L) -> [L]. @@ -3155,7 +3281,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 a931ca9b0..c120eaf53 100644 --- a/test/ra_snapshot_SUITE.erl +++ b/test/ra_snapshot_SUITE.erl @@ -16,6 +16,10 @@ -include_lib("eunit/include/eunit.hrl"). -include("src/ra.hrl"). +-define(MACMOD, ?MODULE). + +-define(MAGIC, "RASN"). +-define(VERSION, 1). %%%=================================================================== %%% Common Test callbacks %%%=================================================================== @@ -38,7 +42,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 +97,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 +122,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 +151,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 +179,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 +207,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 +251,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 +300,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 +332,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 +378,12 @@ 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), + S2 = ra_snapshot:accept_chunk(A, 1, S1), + S3 = ra_snapshot:accept_chunk(B, 2, S2), + S4 = ra_snapshot:accept_chunk(C, 3, S3), + S5 = ra_snapshot:accept_chunk(D, 4, S4), + Machine = {machine, ?MODULE, #{}}, + {S,_, _, _} = ra_snapshot:complete_accept(E, 5, Machine, S5), undefined = ra_snapshot:accepting(S), undefined = ra_snapshot:pending(S), @@ -383,8 +407,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), + S2 = ra_snapshot:accept_chunk(A, 1, S1), + S3 = ra_snapshot:accept_chunk(B, 2, S2), S = ra_snapshot:abort_accept(S3), undefined = ra_snapshot:accepting(S), undefined = ra_snapshot:pending(S), @@ -392,15 +416,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 +436,20 @@ 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), + State3 = ra_snapshot:accept_chunk(A, 1, 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), + Machine = {machine, ?MODULE, #{}}, + {State, _, _, _} = ra_snapshot:complete_accept(B, 2, Machine, State4), undefined = ra_snapshot:accepting(State), {165, 2} = ra_snapshot:current(State), ok @@ -432,37 +458,48 @@ 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), - MacRef = crypto:strong_rand_bytes(1024), - MacBin = term_to_binary(MacRef), + {State1, [{bg_work, Fun, _}]} = + ra_snapshot:begin_snapshot(MetaLow, ?MACMOD, ?FUNCTION_NAME, + snapshot, State0), + Fun(), + MacState = crypto:strong_rand_bytes(1024), + MetaBin = term_to_binary(MetaHigh), + IOVec = term_to_iovec(MacState), + Data = [<<(size(MetaBin)):32/unsigned>>, MetaBin | IOVec], + Checksum = erlang:crc32(Data), + MacBin = iolist_to_binary([<>,Data]), %% split into 1024 max byte chunks <> = 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), + State3 = ra_snapshot:accept_chunk(A, 1, State2), + {165, 2} = ra_snapshot:accepting(State3), + + {State4, _, _, _} = ra_snapshot:complete_accept(B, 2, {machine, ?MODULE, #{}}, + 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 +517,7 @@ meta(Idx, Term, Cluster) -> term => Term, cluster => Cluster, machine_version => 1}. + +%% ra_machine fakes +version() -> 1. +live_indexes(_) -> []. diff --git a/test/unit_SUITE.erl b/test/unit_SUITE.erl index bf4897200..7a32888a5 100644 --- a/test/unit_SUITE.erl +++ b/test/unit_SUITE.erl @@ -21,7 +21,7 @@ mods() -> ra_flru, ra_lib, ra_log, - ra_log_reader, + ra_log_segments, ra_log_segment, ra_monitors, ra_server,