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.
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 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.
+
+
+
+### 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,
|