From 21db40cbaf0b8b9d46681742acb1883bb950f9ab Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 2 Jan 2025 10:00:36 +0000 Subject: [PATCH 01/22] Remove deprected APIs in ra module. --- src/ra.erl | 69 ------------------------------------- test/coordination_SUITE.erl | 16 ++++----- test/ra_2_SUITE.erl | 29 ---------------- test/ra_SUITE.erl | 30 ++++++++-------- 4 files changed, 23 insertions(+), 121 deletions(-) diff --git a/src/ra.erl b/src/ra.erl index 92a594a0..0e6d2de3 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -44,16 +44,10 @@ delete_cluster/1, delete_cluster/2, % server management - % deprecated - start_server/1, start_server/2, start_server/5, - % deprecated - restart_server/1, restart_server/2, restart_server/3, - % deprecated - stop_server/1, stop_server/2, force_delete_server/2, trigger_election/1, @@ -68,8 +62,6 @@ leave_and_delete_server/3, leave_and_delete_server/4, %% troubleshooting - % deprecated - overview/0, overview/1, %% helpers new_uid/1, @@ -79,20 +71,12 @@ aux_command/2, aux_command/3, cast_aux_command/2, - register_external_log_reader/1, member_overview/1, member_overview/2, key_metrics/1, key_metrics/2 ]). -%% xref should pick these up --deprecated({start_server, 1}). --deprecated({restart_server, 1}). --deprecated({stop_server, 1}). --deprecated({overview, 0}). --deprecated({register_external_log_reader, 1}). - -define(START_TIMEOUT, ?DEFAULT_TIMEOUT). -type ra_cmd_ret() :: ra_server_proc:ra_cmd_ret(). @@ -171,19 +155,6 @@ start(Params) when is_list(Params) -> start_in(DataDir) -> start([{data_dir, DataDir}]). -%% @doc Restarts a previously successfully started ra server in the default system -%% @param ServerId the ra_server_id() of the server -%% @returns `{ok | error, Error}' where error can be -%% `not_found', `system_not_started' or `name_not_registered' when the -%% ra server has never before been started on the Erlang node. -%% DEPRECATED: use restart_server/2 -%% @end --spec restart_server(ra_server_id()) -> - ok | {error, term()}. -restart_server(ServerId) -> - %% TODO: this is a bad overload - restart_server(default, ServerId). - %% @doc Restarts a previously successfully started ra server %% @param System the system identifier %% @param ServerId the ra_server_id() of the server @@ -227,16 +198,6 @@ restart_server(System, ServerId, AddConfig) {'EXIT', Err} -> {error, Err} end. -%% @doc Stops a ra server in the default system -%% @param ServerId the ra_server_id() of the server -%% @returns `{ok | error, nodedown}' -%% DEPRECATED: use stop_server/2 -%% @end --spec stop_server(ra_server_id()) -> - ok | {error, nodedown | system_not_started}. -stop_server(ServerId) -> - stop_server(default, ServerId). - %% @doc Stops a ra server %% @param System the system name %% @param ServerId the ra_server_id() of the server @@ -509,16 +470,6 @@ start_server(System, ClusterName, #{id := {_, _}} = Conf0, Machine, ServerIds) machine => Machine}, start_server(System, maps:merge(Conf0, Conf)). -%% @doc Starts a ra server in the default system -%% @param Conf a ra_server_config() configuration map. -%% @returns `{ok | error, Error}' -%% DEPRECATED: use start_server/2 -%% @end --spec start_server(ra_server:ra_server_config()) -> - ok | {error, term()}. -start_server(Conf) -> - start_server(default, Conf). - %% @doc Starts a ra server %% @param System the system name %% @param Conf a ra_server_config() configuration map. @@ -742,14 +693,6 @@ new_uid(Source) when is_binary(Source) -> Prefix = ra_lib:derive_safe_string(Source, 6), ra_lib:make_uid(string:uppercase(Prefix)). -%% @doc Returns a map of overview data of the default Ra system on the current Erlang -%% node. -%% DEPRECATED: use overview/1 -%% @end --spec overview() -> map() | system_not_started. -overview() -> - overview(default). - %% @doc Returns a map of overview data of the Ra system on the current Erlang %% node. %% @end @@ -1191,18 +1134,6 @@ aux_command(ServerRef, Cmd, Timeout) -> cast_aux_command(ServerRef, Cmd) -> gen_statem:cast(ServerRef, {aux_command, Cmd}). -%% @doc Registers an external log reader. ServerId needs to be local to the node. -%% Returns an initiated ra_log_reader:state() state. -%% Deprecated. Now only reads log data stored in segments, not log data -%% in mem tables. -%% @end --spec register_external_log_reader(ra_server_id()) -> - ra_log_reader:state(). -register_external_log_reader({_, Node} = ServerId) - when Node =:= node() -> - {ok, Reader} = gen_statem:call(ServerId, {register_external_log_reader, self()}), - Reader. - %% @doc Returns a overview map of the internal server state %% %% The keys and values will typically remain stable but may diff --git a/test/coordination_SUITE.erl b/test/coordination_SUITE.erl index e338b52d..24220914 100644 --- a/test/coordination_SUITE.erl +++ b/test/coordination_SUITE.erl @@ -1004,15 +1004,15 @@ segment_writer_or_wal_crash_follower(Config) -> end || I <- lists:seq(1, 10)], %% stop and restart the follower - ok = ra:stop_server(Follower), - ok = ra:restart_server(Follower), + ok = ra:stop_server(?SYS, Follower), + ok = ra:restart_server(?SYS, Follower), await_condition(AwaitReplicated, 100), _ = ct_rpc:call(FollowerNode, ra_log_wal, force_rollover, [ra_log_wal]), - ok = ra:stop_server(Follower), - ok = ra:restart_server(Follower), + ok = ra:stop_server(?SYS, Follower), + ok = ra:restart_server(?SYS, Follower), await_condition(AwaitReplicated, 100), @@ -1114,15 +1114,15 @@ segment_writer_or_wal_crash_leader(Config) -> end || I <- lists:seq(1, 10)], %% stop and restart the leader - ok = ra:stop_server(Leader), - ok = ra:restart_server(Leader), + ok = ra:stop_server(?SYS, Leader), + ok = ra:restart_server(?SYS, Leader), await_condition(AwaitReplicated, 100), _ = ct_rpc:call(LeaderNode, ra_log_wal, force_rollover, [ra_log_wal]), - ok = ra:stop_server(Leader), - ok = ra:restart_server(Leader), + ok = ra:stop_server(?SYS, Leader), + ok = ra:restart_server(?SYS, Leader), await_condition(AwaitReplicated, 100), diff --git a/test/ra_2_SUITE.erl b/test/ra_2_SUITE.erl index 5d6a5fef..ac6740fa 100644 --- a/test/ra_2_SUITE.erl +++ b/test/ra_2_SUITE.erl @@ -43,8 +43,6 @@ all_tests() -> custom_ra_event_formatter, config_modification_at_restart, segment_writer_handles_server_deletion, - %% TODO: mt decide on whether to support this - % external_reader, add_member_without_quorum, force_start_follower_as_single_member, force_start_follower_as_single_member_nonvoter, @@ -649,33 +647,6 @@ segment_writer_handles_server_deletion(Config) -> end, ok. -external_reader(Config) -> - ok = logger:set_primary_config(level, all), - ServerId = ?config(server_id, Config), - ClusterName = ?config(cluster_name, Config), - ok = start_cluster(ClusterName, [ServerId]), - ra:members(ServerId), - ok = enqueue(ServerId, msg1), - [begin - _ = ra:pipeline_command(ServerId, {enq, N}, no_correlation, normal) - end || N <- lists:seq(1, 1023)], - _ = enqueue(ServerId, final), - R0 = ra:register_external_log_reader(ServerId), - ok = force_roll_over(), - receive - {ra_event, _, {machine, {ra_log_update, _, _, _} = E}} -> - R1 = ra_log_reader:handle_log_update(E, R0), - {Entries, _R2} = ra_log_reader:sparse_read(R1, lists:seq(0, 1026), []), - ct:pal("read ~w ~w", [length(Entries), lists:last(Entries)]), - %% read all entries - ok - after 3000 -> - flush(), - exit(ra_log_update_timeout) - end, - ra:delete_cluster([ServerId]), - ok. - add_member_without_quorum(Config) -> ok = logger:set_primary_config(level, all), %% ra:start_server should fail if the node already exists diff --git a/test/ra_SUITE.erl b/test/ra_SUITE.erl index fa3e0bf8..fddf7321 100644 --- a/test/ra_SUITE.erl +++ b/test/ra_SUITE.erl @@ -789,8 +789,8 @@ consistent_query_after_restart(Config) -> {ok, _, _} = ra:process_command(A, N, ?PROCESS_COMMAND_TIMEOUT), application:stop(ra), restart_ra(DataDir), - ok = ra:restart_server(A), - ok = ra:restart_server(B), + ok = ra:restart_server(?SYS, A), + ok = ra:restart_server(?SYS, B), ?assertMatch({ok, N, _}, ra:consistent_query(A, fun(S) -> S end)) end || N <- lists:seq(1, 30)], @@ -803,15 +803,15 @@ consistent_query_minority(Config) -> {ok, _, Leader} = ra:process_command(A, 9, ?PROCESS_COMMAND_TIMEOUT), [F1, F2] = Cluster -- [Leader], - ra:stop_server(F1), - ra:stop_server(F2), + ra:stop_server(?SYS, F1), + ra:stop_server(?SYS, F2), {timeout, _} = ra:consistent_query(Leader, fun(S) -> S end), %% restart after a short sleep so that quorum is restored whilst the next %% query is executing _ = spawn(fun() -> timer:sleep(1000), - ra:restart_server(F1), + ra:restart_server(?SYS, F1), ok end), {ok, 9, _} = ra:consistent_query(Leader, fun(S) -> S end, 10000), @@ -830,11 +830,11 @@ consistent_query_leader_change(Config) -> {ok, _, A} = ra:process_command(A, 9, ?PROCESS_COMMAND_TIMEOUT), %% do two consistent queries, this will put query_index == 2 everywhere {ok, 9, A} = ra:consistent_query(A, fun(S) -> S end), - ok = ra:stop_server(E), + ok = ra:stop_server(?SYS, E), {ok, 9, A} = ra:consistent_query(A, fun(S) -> S end), %% restart B - ok = ra:stop_server(B), - ok = ra:restart_server(B), + ok = ra:stop_server(?SYS, B), + ok = ra:restart_server(?SYS, B), %% Wait for B to recover and catch up. {ok, #{log := #{last_written_index_term := CurrentIdxTerm}}, _} = ra:member_overview(A), @@ -852,12 +852,12 @@ consistent_query_leader_change(Config) -> ra_leaderboard:lookup_leader(ClusterName) =:= B end, 20), %% restart E - ok = ra:restart_server(E), + ok = ra:restart_server(?SYS, E), {ok, 9, B} = ra:consistent_query(B, fun(S) -> S end), - ok = ra:stop_server(A), - ok = ra:stop_server(C), - ok = ra:stop_server(D), + ok = ra:stop_server(?SYS, A), + ok = ra:stop_server(?SYS, C), + ok = ra:stop_server(?SYS, D), %% there is no quorum now so this should time out case ra:consistent_query(B, fun(S) -> S end, 500) of @@ -867,9 +867,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), From f5ce68c37043c4c76f5fef4962088781ebcf8d9d Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 2 Jan 2025 16:21:40 +0000 Subject: [PATCH 02/22] Design doc wip --- .gitignore | 1 + docs/internals/COMPACTION.md | 188 +++++++++++++++++++++++++++++++++ docs/internals/compaction1.jpg | Bin 0 -> 36665 bytes docs/internals/compaction2.jpg | Bin 0 -> 53207 bytes 4 files changed, 189 insertions(+) create mode 100644 docs/internals/COMPACTION.md create mode 100644 docs/internals/compaction1.jpg create mode 100644 docs/internals/compaction2.jpg diff --git a/.gitignore b/.gitignore index 75c436da..49d2e3bd 100644 --- a/.gitignore +++ b/.gitignore @@ -30,3 +30,4 @@ config doc/* /.vscode/ +.DS_store diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md new file mode 100644 index 00000000..6d3eeed6 --- /dev/null +++ b/docs/internals/COMPACTION.md @@ -0,0 +1,188 @@ +# Ra log compaction + +This is a living document capturing current work on log compaction. + +## Overview + + +Compaction in Ra is intrinsically linked to the snapshotting +feature. Standard Raft snapshotting removes all entries in the Ra log +that precedes the snapshot index where the snapshot is a full representation of +the state machine state. + +The high level idea of compacting in Ra is that instead of deleting all +segment data that precedes the snapshot index the snapshot data can emit a list +of live raft indexes which will be kept, either in their original segments +or written to new compacted segments. the data for these indexes can then +be omitted from the snapshot to reduce its size and write amplification. + + +### Log sections + +Two named sections of the log then emerge. + +#### Normal log section + +The normal log section is the contiguous log that follows the last snapshot. + +#### Compacting log section + +The compacting log section consists of all live raft indexes that are lower +than or equal to the last snapshot taken. + +![compaction](compaction1.jpg) + + +## Compaction phases + +### Phase 1 + +Delete whole segments. This is the easiest and most efficient form of "compaction" +and will run immediately after each snapshot is taken. + +The run will start with the oldest segment and move towards the newest segment +in the compacting log section. Every segment that has no entries in the live +indexes list returned by the snapshot state will be deleted. Standard Raft +log truncation is achieved by returning and empty list of live indexes. + +### Compacted segments: naming (phase 3 compaction) + +Segment files in a Ra log have numeric names incremented as they are written. +This is essential as the order is required to ensure log integrity. + +Desired Properties of phase 3 compaction: + +* Retain immutability, entries will never be deleted from a segment. Instead they +will be written to a new segment. +* lexicographic sorting of file names needs to be consistent with order of writes +* Compaction walks from the old segment to new +* Easy to recover after unclean shutdown + +Segments will be compacted when 2 or more adjacent segments fit into a single +segment. + +The new segment will have the naming format `OLD-NEW.segment` + +This means that a single segment can only be compacted once e.g +`001.segment -> 001-001.segment` as after this there is no new name available +and it has to wait until it can be compacted with the adjacent segment. Single +segment compaction could be optional and only triggered when a substantial, +say 75% or more entries / data can be deleted. + +This naming format means it is easy to identify dead segments after an unclean +exit. + +During compaction a different extension will be used: `002-004.compacting` and +after an unclean shutdown any such files will be removed. Once synced it will be +renamed to `.segment` and some time after the source files will be deleted (Once +the Ra server has updated its list of segments). + +#### When does phase 3 compaction run? + +Options: + +* On a timer +* After phase 1 if needed based on a ratio of live to dead indexes in the compacting section +* After phase 1 if needed based on disk use / ratio of live data to dead. + +![segments](compaction2.jpg) + +### Ra Server log worker responsibilities + +* Write checkpoints and snapshots +* Perform compaction runs +* Report segments to be deleted back to the ra server (NB: the worker does +not perform the segment deletion itself, it needs to report changes back to the +ra server first). The ra server log worker maintains its own list of segments +to avoid double processing + + +```mermaid +sequenceDiagram + participant segment-writer + participant ra-server + participant ra-server-log + + segment-writer--)ra-server: new segments + ra-server-)+ra-server-log: new segments + ra-server-log->>ra-server-log: phase 1 compaction + ra-server-log-)-ra-server: segment changes (new, to be deleted) + ra-server-)+ra-server-log: new snapshot + ra-server-log->>ra-server-log: write snapshot + ra-server-log->>ra-server-log: phase 1 compaction + ra-server-log-)-ra-server: snapshot written, segment changes +``` + +#### Impact on segment writer process + +The segment writer process as well as the WAL relies heavily on the +`ra_log_snapshot_state` table to avoid writing data that is no longer +needed. This table contains the latest snapshot index for every +ra server in the system. In order to do the same for a compacting state machine +the segment writer would need access to the list of live indexes when flushing +the WAL to segments. + +Options: + +* It could read the latest snapshot to find out the live indexes +* Live indexes are stored in the `ra_log_snapshot_state` table for easy access. + +Snapshots can be taken ahead of the segment part of the log meaning that the +segment writer and log worker may modify the log at different times. To allow +this there needs to be an invariant that the log worker never marks the last +segment for deletion as it may have been appended to after or concurrently +to when the log worker evaluated it's state. + +The segment writer can query the `ra_log_snapshot_table` to see if the server +is using compaction (i.e. have preceding live entries) and if so read the +live indexes from the snapshot directory (however it is stored). Then it +can proceed writing any live indexes in the compacting section as well as +contiguous entries in the normal log section. + + +Segment range: (1, 1000) + +Memtable range: (1001, 2000) + +Snapshot: 1500, live indexes [1501, 1999], + + +Alt: if the log worker / Ra server is alive the segment writer could call into +the log worker and ask it to do the log flush and thus make easy use of the +live indexes list. If the Ra server is not running but is still registered +the segment writer will flush all entries (if compacting), even those preceding +last snapshot index. This option minimises changes to segment writer but could +delay flush _if_ log worker is busy (doing compaction perhaps) when +the flush request comes in. + + + +### Snapshot replication + +With the snapshot now defined as the snapshot state + live preceding raft indexes +the default snapshot replication approach will need to change. + +The snapshot sender (Ra log worker??) needs to negotiate with the follower to +discover which preceding raft indexes the follower does not yet have. Then it would +go on and replicate these before or after (??) sending the snapshot itself. + +T: probably before as if a new snapshot has been taken locally we'd most likely +skip some raft index replication on the second attempt. + +#### How to store live indexes with snapshot + +* New section in snapshot file format. +* Separate file (that can be rebuilt if needed from the snapshot). + + +### WAL impact + +The WAL will use the `ra_log_snapshot_state` to avoid writing entries that are +lower than a server's last snapshot index. This is an important optimisation +that can help the WAL catch up in cases where it is running a longer mailbox +backlog. + +`ra_log_snapshot_state` is going to have to be extended to not just store +the snapshot index but also the machine's smallest live raft index (at time of +snapshot) such that the WAL can use that to reduce write workload instead of +the snapshot index. diff --git a/docs/internals/compaction1.jpg b/docs/internals/compaction1.jpg new file mode 100644 index 0000000000000000000000000000000000000000..57ae3d2c0cd3c2fec04c19721ff3141ccefd1800 GIT binary patch literal 36665 zcmeEv1yq#Xy8j?u5&|NfA|N2$45)NTH;5=TboU4dDAEWhNP~2DOP6#?cX#K^|MlE2 z)N}6loqN~+-gWP}->_cRteO4p_t{VFXaAmO?}3lQ=RmkJ64DYNgc~3b0`LcdPl6tS z&`?lNQIOG4QBl#+(J-(Gu(2>PvF_mC!XY3hp`;)uAtR%rV`ZSCxlc<*c8{0w{sVR{ zE-p$2eqlZiAy!TVPX+uV-s^wlTma0hhOl|AUw1i{fM6t5oka+@DLF3 z5a2B!N`OvegulH&zq}A`AR-~7prWB;U;+)wa6vZ^5D{-6AtECqApxyDf$u>`c*ywF z>|!Xllnha6Yza8L!;{fy9~LzdDi7?@aXz#2LB}AvO?-!h{w~8kMkX$99$r3vfk%(U zB_yS!WmHtv)HO7Fg7tYGq-nebaHlab@P4Y=O6I;O<+XiyQt`x_a9X$*Yy_<{?dWJbl@)?_)7==(t*Ep;4dBc*VO@L@3Rrw*1b?Y)9I+98_NoOoNZ`V zVl1Rh|j|@sw5v5occY%YhI75Bmpf&n@ z=T5*&oJI0pw^ zphFT*VU#d#QHsX@;oW&GFR%r~YBU$TPC&3EIeZc}YaUn}_km#Ueqj)HcHN^w{vJ8zs4aka;Xk7Hhq!W;Yq?;HU?0BX)hB^<~-pR3*CELGWBvmZ2$h2mM zc8pTnr#i%13MPRFw589T?6deM89kU14e#MDW#akmSGMo8@-bMSp1u~LhEyZR_olmMo*B<*(u4P@&4ju?1@Z7ZMj~{q`hDIN zrN|Sv<#pSAQ-(vTw?@KtVp&&8^kkZQdYFxt(QeUd#Ie^Y#E#u}jP?$11D>Ox0XBMu zPggGtTdy?!;l^WE7R$h(^GYOkbz~cKVwTRuFdV4_7t5!v`Sc<8!d@guU}Ot-W-)hO zspV`4R5Te7mMzajX6m$KU_!1r6eFI>MBL$Jmq4BUIC`bM8|YAgXR1E2U}kNSE7(L^ zDyhZt94`h`!hTak8NKPSuMFhO_1xsnO6?9MflG~M|E`c?YHM14B&M8YZa!^Tk0XJc z+WQV-B%{(fOW_hVOKxFF-Lsrzb0=P+EJNq{L%VQU%ot5=skix$yc_r$+d#xr$4NK} zDS`J8+hJ^QQ0JpFBRI%V1rAEJ+#F~A2cO*o0geBaF2p_<4=`}*G3fRz93(@V@B2CN zKv9Sr4tknna7|FwcV!8f!=rpSXtDwJPErrm8@!a@0)aKEH(W59BHlH?IV+JEEhBBf z+8}v8svR_I7BQr%1-54du}do0*xW>0EK^CGKS_0;zkb2yNNUEN_v~O|V?}e35d%#R zBZZ9tLFJB6gloaWl$ZS1M$L!j`b_=YrmLC@L+VRXwL8{44^pj%?&OnsaP4E9v2HdM z>8?G|=?KYCk|LX$x-l8>Uf_lTf<*;6vRf+w!itE->(sNntauVVaBrw&sn&cg&T>P~ zU9=ZVxMJHK4CxS^vqyS^N4o11tu^L{=J@8fXB_Bs^h$lP?;zgKM(IcEJa@C#D-2|3 z45N8N4ALPB6Euz&r7&V^*14Q!XuY8^n9Jr^N%K-?T%y~WqB(y1^Zj;)k2r1B*Y=`u zYNH;)?qmZgX3myEfuENfgCh}VS#*w`pe9Pr;}Dp|itqX#dCJPk5*2l| z6laGsHPUFd!_c;4ws2zU$-SkUEz&!!!IzK1(t0gQa6 z&VM9pD#V>CF{b^|wDmrXa#!X-A)o#$2{H0kkR%+0SuwwQjRiiZw}pe+*5Dw&$-P6E zKjl?KB@o9r_CuS{zXdFWT;QwQz<62){(ijwlen}*!M9%#Q2SyZXMXksCHbVUcq3=k zYenj0CmOrKQ^LNi+D)Tjl%>G;irvtWd?Yb|>#h6i-nC`_HZBK|B)^Wl?0*@<_(ulL zoHzpYTm<9jz(IZYb^!mq8zlt?os>Bc_GRqCsJr1Hg;-@T3Wq#E7w#b@{fkX&e>g?w zLnW?H18ZdgGZ0#?DDAr}d;|U)W6l&$%P-1iudd81Bt`0V&cT;5F9rTDp~W{uks@C* zC+>+dG`guSJ)LSkNj-k|0es?CvEW{kai*eKi~9clr!dcl_)McQrd-!FT^(4MF8FMh z%c8J%2v&jjl5YGTxo$$+8o+)l6>v~u)%LdS2K@(Ga4N|fbK*YluGN*W85|^h{Fu&N zk}Lua;%jvv04a(8pSckFHVo?b>wQs8^LBPZizsxB;2?>_(Q1sxG&wV~+Ggc;db87F5K9W)9wBc$I~$I6?@q-19Kyb{ZkVXk>@FY-P^))cO$86~^<{GR>zws9jXAUH=(xm4 z#JXzI*-Coe;w<6=fxgluj8MLUx*BHc0WoTCmebc`OlAq3W#ZH}g9F5JIH)i`4h~8$ z>_-(7CwPMJUwP@mK|u{mr~uSqsK*Nj$r|SS0Ql}5XvfV-SW~q%EV@~g{67PHlBs=) zrwjyuLfm-ug~RT`S{*@%snwGAGbJqR6+_||@2mLvKO>T_pIm&(IeE=RPIS?y96Q>f z$dl~W`YNeoKO44tRzm7;HNGaDXDQnh>y!?qKaF!Yvu8qEiRjv%f!2&xdPa=3>$6ygqSP>^S`%{OaVn(p$j#b`bsnb+QAZfSV6q*0!B5}LS*x<3Hh0DjaiCP}A@$0-7rEFs^%4xhA z1yC_LYb3LNP2G0$JX`jXh{3PichV~T2@He1WuG%P`TBkYajMm_%tNMm^%B+V(Qn0x zmxfSh%DdPQ7tt?+y!(+6ha)3J(KB7jVa=@#UrU-HX2w=D{tVS2YL_yVy2Wy+(Zobd zxCb)FcL*ir&b(bSH`I4x?aLEwolfSJyB0m9?!1#3+&<`_mk<>~*^r{C7A!zFbB+y) ziMYoA7e&`pDoYQq9JkE7i;WFp>pS~J-g6VkSIASF`0UBNW5i+XX;jm#`dkUmJiSD zwucXg(m~q)5fC8FS#uyhgiu2Ih@h;^Lup^G9)C~wwCHPDUqah(lxMyA(NoIgUgCT0 z3_p6@JaSxwr1i$9(IA%88;qhi6YrgC-Vmit zUMZ=40S8eDT3=$UH(cmogB%Ey#6ka+7oqo}pzDqY&nASo%*L#(C-7HhXwE4`7bB<>8?AwV&?vB=Gbq^ppv|1%YvkMY<Hb|TWK+8E`egsSY+|OFp#kJwJOxT}OqI7)Gc+B)zzTj*G zMSjP780Q>SergfsKGg6N`y#9>!LtU}XsS^L`gvXN>oa6yn}23;=uZRB7*ILBXugP| zPW*{rDaTzBpVaMo%4Huos6)VNmGW>VEe-4jzF14Vz|HX&C{2sJ>mne-l48A031Gtj zCK~|V(*pw>Pra0E2Lo92{Or|jML>9)4+Cd6)v)UG0D$FQfJ;39 zL>LSs|E2OoBjm6ee|28Ka^mC`7Ka-e;!RTq81p7x6J+CeZp?R|e*~P&P)Lc|Zthlbo6Yx2N z30Z+RSE$KppUyS%nSfYKKl7qQzgmKY}Apg4-p&ne1Qz%1+_vC8I6Jkyk<{ke!ltG z_%~7#5=XPO7cOd9P2iyaJTd=(&)Sa2t5+^l&@mpE>SAE##6uSsI`zNf{XY<0`f;}J zMHwgfCk0shyL*AY}bij*DP9}U+=*|Fw9Q? zyN*+T7qU|B#v)Ve-M|dLjJrF3>Ldt5$2Lj+BQ_=t*w}!s{u;4ic)S2$wF*DE9VzhQ z4BIK{K^9YbXl`16s&CrD`#(3}f6tYu80_byRMz!BXr>kT=Kj(%0Y4Jz(lABYm9u$V zrE`0D-Con+CxiW`J=MTZY&^QScX9pnB8rA??8{QKUSyc#s2fA6{1vwq^K@~{oGz1Cm{ zXIbs|Vk+Y*Ffg%T zlyW!=>nHi-xqB|;$=ow+s2zx%V9sRKds-AlUYn)-1o6&$tapgXb0b|Ai5rt^X1Sim zjdNL=!=bHiuflUM@7UK)ipFp|4oakmmD*CAM8Ork zK)}_x7~tz;!Sjs6;MFdi%dJs00$2fp=QHC}mz*aUisMOhP>&&_ef?xQ^QJDu@aqF+ zR>fn>BX07HE4TKetP|XoBK;ZN3Se^htqUNJgszYb2Z=m^&KehIW?$5zU$pcXoXI%i zoFDkYF1et;)0KViq84%U7T2@=7gvE&B`*&oXKB&^1TNzV;Ct?YcS|(Yp&L5ahw4m5 z53%p6=5z{@H0-_Gpgdegy*@GhzM+~ib)P3*t>D5YZ;)X}@9tqa}f z1mPwKOQH4R(%+A|Fc%IQuDD#uRc(7kBA2BR;lvI7r}@K8rhmU7T$;#+Lh5o!-BF_i z4pN@ojg5JDTPj^&-2$jf`JFwI-wClRC1+!gVe{6%|EScBJ)B&*%_DgUJw?UK#{(%m zC%O!B7=yWBpva};Hf*BdFt2Og#QaZFAl>`7uU@XeL0;f%;TYAqyQAeQTDTgq%Tp!0Y#8nebRXo!7Ny3xaQ`!=HX&d92J4R!~MvMse1W?_Zi@Whuzk5$L=;)^|wz zdpsn`J4Lx$0DHwGqc~>(?uRTu$Lyl>9TC!|;Gh?1!RNY0EYLb_n?p&^hjokcv{-cC1umJ3px$Jkp8l7G$)xQi3{K<43c$%-Qg_ z^NTc9K{2Q9yS=Px7%Iy;e*1-PEONY9l6bSo{uI}%P97pJBC(xue12yb@`1K0bLGPE2ntkj<&>W;hArKmwaIE%8mD^u-~X zZN_dx{3@YmQrta6o`T|a#nt82x2C6cwck5Q`jBw6U@yNx3<^1(Rxhh|=13naBAc~= z);C<>W-qb%+^{6&S3{cWK=!gRGMr`7PF5VW~#NKB~MSl*Rwde=pcyft=e zR+NPj-V*_@%@CA1fn(x@vb5x`o#Wx&)Toi@0B0$$Bk1t}_=sBew|82DXNw!bOddG> zqnXyM$mvn7-41~~VK78)ZjE~UIP?}gNv?XWNwP%8UI9uYJqcXt&b#&`*ay>Pq49wB zu%?MSRj_BLete!O=z4|ic-*bm!&*TUZ*nuv(fYmyUN@~Q`s0@j3`l#5hJhPqkGI>k zI%P(!F3d8f!heez)P%);pR*O!VFa$%G84gbLr0TwZKSX8xZWRs?q!ZEtdEA&h$c{l zKvthxu^8N-X=usvyKLT^j9qht@CMv##MxYHHt!g{8KAPr~c`H{{QB@Ir zi<Unf#U2)9ITwwjouyGpkv?!MxJ5?wjR-lbb_>)7;~m8<>4(`#|bG-|{Nk?e_?7 z5R5NWWzo4FNh^)@x|qwNOVqJVHGJ4OQCwR3>(sK<)8W)nORh~ls{?6G*P$5AeKOI- z;$sHyvoSTz36@$QBvG%gd^0GWFK1S3#7>1Ebs`k|H7z7pcb6WUOAlYeTBE|U{_*X^ z{Y4;usylhEdi)(^FRKjyPL+E2S@^Q7v7(Z0&xyNM z#)M9{glB3!immb>kJCqx0v~F+w9MG>lJ?<~lD#bvniv&VtygxPR%#w8L6Tjy&v9{d zj|s=dPEZx?O2sQey}( z9p86%B46hYR=!}lEnZ%l*i|>Y?2l7Qh-HKo`xYk4EEa95JZ& zSSKww%(3Ay4J8f7*(`?>&)2^Nb0vd}piz_wfK%Oh)I^!6F zzA)hhILHRdB%97+!jVb$ejFsYFI}T*{RO`kuXn9FA&1wSt9zWKH#xHp!^FcI!D!_D z1&)=uBGSz4>>7t`mcrd?ovKAACBJjsARH^ctSfSB?hN13IQQ2HjH%^nF4Hz$o@m^+ zeJ!n6xZb|sF^sqkxs1W2iK3Sa5?d`IOgp!vi!nH)xm~NX4U>GN)Cer6_Mw&FDCytd zKCeeTxR$4EX~=J*d~1FDmgTNd>Lji7k~HnK%-nqh4gPj@qh3@6{!eMKJ*U17pN3Y| z>Z<&wy6vfuQi^?c5aud$aoh&So<%@$%xxbB9SEmoXQ$G?c?h~uxMO`RjQN?wWZYr$ zZLs84zbK8S)))E07at_~h=ttd)Wb#@=?d)V)8edU(al(PA{Pbq$jmDQqUu_6BfBaD zWhbMkn5~}hS%mwp>si`S+=`f(nO}hQjmE+@SF48Ea=&;%5T=fl#_!E(7^vth6s}uH zqF%YT!scLVp$qQQ`h9ZHEqBcsBeI||{7Q6?(#PA>8-3 zt!c6|JL$4M6YxaIyZdNlWEfF&=@}{Q-Tgox`~{LZrTw|j_CfD-SIsQ(a!~}rBDc!p z`UEhMY`P8vf0Ibs;(29+a**W;7D#Umv}Cd@6r}c+kz1bOv-TtH@jE+t+j%XcxoDB5 zYBvdN54}GkuB%!;F|X$uDr4c4tuCr4_g8Y|I%%8jodRj@iL&^gmjq!s<|*r+T0#pB zI4v*hj$jR6w$1r|Tc7;;F8B6lm_b&_SCUmhQ_3EA?%LGKmZ(bs1@o{%eoZ0QF?xDQ-RQDy; zqfipFD-_MMRdVVD-y$EMNR;ieP05fl{g-sde4jHDtr2cT!VLpUZ0uBRapAliEaHI= zFIiv%hzvt~+j<_~VsJ>%AeX1pWc!$ghTF%;#uF*uE{hQ&D}yB@2POrVjm%Se~qn_URV=tDd0q;HE@{G?iLzSLD$w0?5K;&H144 zF66${TdFpW2CDlAt5id!HJ%T<=c9s{)9HJbc-HH?mf35tBOXBCk_DaKh~qkNiED8q z8;!L*E_L%L5IVi>w2l7cNkMc?)idc;GFzqmM4F?q6(DJkY?lWzjYx~bm#k%IVB;9$g$CUi0(4wkupsleFrJGDX%1I=V%+PTO zyx5rOB!V1rh*zv!hGS>nR8(1GHuyYRL8O1Pu%g;u%J|XUwI$_TPFlP$&)1(^mHJThR7bBokCG`~ey!x&f$(r&nmO~`bew=pfe9?VVV3*ZugeydvvB75=OlG) z$=1d2;$~&ADx=ZRVOth>!CGQ-QU1DIpE^b};#hi^ zUs`QE-ypddXMav`AT&WN*V)i1orP*q7B$O@tv0 zDevWeA^T>YE!6TwjePNKXSqf^64Aga*j*9d$=KMwYguT$c`NQ2ZpJ?6>md)ZEbS}4 zhEdjOtAg!`)sG(O(Gp7Sr;-N*>z{^e`Kq~(d#iYoD#ny_bXz%+9zG}Iau;Rxon)ZN zJ#*^anQV+Hl+HMhEFF7cX@$~w&y4J)oxlTO&&|a00TyoNI+|I=v%z`2 zUN-fVQK)O1EJ#FudS59o+)XdxgWA?Fi#0WO@G%SQJdd*FwE-q0#Kyb=0bL3 zbIr7Dqelz+NJSMC>76jrOK!f~TpdzldOEn_R8C@3i(_-!L)rfIhj&;@AlEys=4Z`n zBV3!r)Poi~=?TjxTzfhn>U({2EXj2e5Ec(|b4W>L;UH}DSi+FF>`8fhYg*yaRI3i- zcR^Mr!U|H0gP#=jFhV8*2RBIw%foZ@HV&uXR?P>V%sdt&Ij=Pq0%@4hR(0ghR7&r# z35lDe0xRLK8Ye@|wn!CyLddjlp7B%h&lig#-r0=ZS^xYYp6wM-4{v*oOz-oC{>7K?he@27k2n2fX zTe~}lo{g{zpg8$qS`NqBM6wvnn_ap-(Pi0~i~gwB z>0f65n1A0iwNN=9Ok`@xfI9|dy7n*pgfC)w-sIgFnchd(SZ@|C7( zr=(p{02q$wdBZNSJW!go3E$8BK zM;ZMnWMMyr#_gm=&dn1sVc67NFxIVI2hIX6!0=X z5R#qYm%B8}_@G5uN%nn)xb6=YFZrYHCuWbrlQ(j>l(D=c-Dt+(3GB(N@-WB8y!k)$dwnq5T!`%$dMx&*C8tF&b z=38+&B8#aLm2a@AtWOie;n8aThhkLy=EkFrV-3EjeCmYd)%zDfjW!Yw=NYcAzWISq zov9lMI?yC(V0!}zbYFGRFAw@k@s`XiUgE#Sv`pqyL&sd+w=a3@z1NVXPgoRT-X*Z8 z6TBQ`f_#tMEcuG$ZzfmX$yP4dKIb%o&*lb)tMgIyHkL7AcH-!m-xq<=@TDDm01x!@NTZfPf8Bbr>_1HPeX z^9%`Sf*v5lLFENkn|4LD>_{gUk}dbf3iN`1N8a^4jhCB0PdVX&;mWfg`NdArGyrY;$$iF0^oie%mWBCKKlX% z&q_clKyLPMo)U&c38N{@Ox-;8-B1ml{S(6eEuZzjf-DY2!Y|o4Yfto-nJs#(KW`Td;kvbtE*dQ1De~@?oKQ&E-ZeO?aR$*bNAH%gP_>$$> zusu+qP}{IWx%_+a##g`%|6l_^uIqQjH&pt!ii!UfdiVd6cmEAmIuY7xFC^90B%(@r zyae4U{nh(Tvyn2-_=vA|lC= zIf@zM-9K`E1fKuO{Ww+YEp8BD+U@7Q`0!HAVvjgf(Y#8_3n+>TBLnYt0yT6bWz2qS zy{*}QI`0=>i38Pu{0+0f1T4W~u5R!esN3ef+y~ant(jSS9Lf8%5h;PpD>x~?C9UTj zEfstB{YF1zLk`!XZN*8oUdKPXlr}tM5BFYkX(yc4pv^YtoY-)Vj@|L&Nl%wj+orE5 zIcy5@=-TW=6~4*eV=3u;?b)X0^oZ4w#iV(LaD>&Q=B*NYb$+~q?FYmtwVB&KYH^tx z$U^b5&9QA^6#@VzXPGHunQ9#GXUzZHq#i==hS^U zgTr&A4JIa>^QmSit!(f9g|28fowkuql32u&c0nV zZ64Q;wZ+;kJMB5_qSAR7z5O&1I5*A0x`HC;(`43~k)=e@Xi6cm2Db!givbCx{cYpv zKg@|&LrG|$m~<(DOqjoka-mi>o5SVPtbF~b@qj}`*`e|RO{PhV6UGN(+)84;T_D{Z zGg^k(O|C9=j7_1Z_$Yz?>xsKHe@d8%jl+f*@hX{msTVC%13JO&SbyKN4FIlV&M}KV zS#jr|9`2Arz1J#=#Ti8DmN~d7K>jH__Z!f5o$d0ht~?x9B1NB*0>oSzKDCmi<>Ijs z0a=t8V&w{EI#qmdpBL}^p{(ymDOwmV3aVIaiE!)gYYQ$<(n}`Tj0=!vJ70a7W_$lJ zQF8(*9hQ->y&jRq)%ATNXY1}=me%Wn1;IMwg&~c~LRr$7($YY}hq-GgE0H#W-bO*T zFPUTo^9)+u6y7>oIyhVkzQ&4)XsqaAeQ@LMoDFB?YQc@1$r(Kt?Mth{ym8R6^tL@qPz;g{C*G53R1K{6lhLym{kas1MX zWKoI>2Y0uqD;nV-xeC)SekIYJm+c{nm3bShIs3xc>sQp`1nS1#-?nZj^#N1c0UT5b z1(I*Szt;l?<{eC@J#*X5_2z}`YR}Fs|qeDFFlJx3K3yO1)rF;wShfe;J z{Q9bCFF5GQ&@r0cQ71P!k!GMGVJ(d`Z6rUI##s#6Q-?e=4+%#vd~&K8%_&78x3QCY zzY}|^SZRD|qcA(s7a)%C&B3BzScT<0pNA1^7H5q`RdlntE5)M+=eL6!eLn8=DJcvd zRdVCp5*`groi-zZeC@`bkH(?4x>I=D!fPsioFYoEE!A=Ey$6{?&G4{O`p1a(ieW`L zSe+}LgFGSe_HE3`^w8={3;Xiz)5+mU=Z1ZpH{7yO00`w5pjHe--s+`aZ^z))<8fJ>nu(qK~Lmjp-%}< z7IQ*$$Z&v%1D79Z4p))vIMbmq$Hl$*q(uG`chj_O1W&Wos9GuCr z_P4ub5EccRKKu@)pZD(4_Nkro3DW$<|%(U+<_0HGE-fyF5KB;T?ot8Xn|>vglnI!km@M zhNo}tgs(NFJzfS&z)at+m7-8i^TPpxK9<7TzSsL6_UA+j**iaE^46yJKb~ zx54XmcQkJ+cQ>Ewl748hE^7PaJY>+bwHxszh#`a9K$ntC6X8&Ub8B&0P_5L!hVhbWpcD zz0*Fc>_Y7RjYp;TG@G;dIwor<(%e*P3s#5>fI{?Fj`BKB#vGSNIB^wAYIzzC8}&5( zz}rT5PJ0|j+Fb0VMz@7b@^tk#@Z}NjgKD(zL+qSzA4tmA@omo32vq7%gbNyo0)F=O zuYQ*H_c%xrX9~Pq0efXGqx6;VI}xsWm}Tjkx#9PjF^lHelG#t+ckiivd^Kpf8@j1S zXmsYks#a3!vnf}8sTuOr%JSWf199IPW9Vxe+nIOeOoHzxlyylbW#e&Lv2(wcY}fj) zxGYhlU2dq}NF%eIG@P(`>dRo&CCpo$?wi0qP=t_{^z1LzR!dogovu! zDn2PDyVzYS)cbGKCmhIfTIAqfZcOt=@U?R%Te~{Wy9Zkhl-68kRM$l8bZ#ec(D{qr z9BlTZF8R0^0-?Ly0tkxt8i!yKYC)P(E6?X-6Pk|b#i#bDFGA$h5HlRbBgxyYnq>}a16AhoQSiyIU zZxY{I_Ma1LRm2hqZaNFq50E^RpZB)RwN`=1l0Ax0Jz}^oD&iCD1>HkbjJd*t-Kq20 zRPkX4GPRd}q3#p1_%zaO=qiOopu;dlnQcf=WrE@@$OTz6kdH0alpDJ>r8ikBh|*NA zIeq$u&Vf8C}=ZXEz!LdCf}fUdRW$#n1SUcHmr7x}6TkBK2`?_cF=E882_ODyChPI1U4;R{>3ZTSu;z02@WkKRap7pYD!#5<-?N8dKe#0z9095wPS|*Q+FU*fz13+Ew z1FNQMQXyca^e`j`KyiqmS6-n2e*5I2hi!!+CF$z|&IN&^!4(>iw_QB4*hjs_5Ga5l zX?-il7s=5%f3(Tf@IE2X*W@*LO=$$WM*~bop#vo?lt=eA8qV-czHP_&MkCJu77{~# z*%zVm6OSbCdr{p3FNBYH0n=``I~Z8t%5Nn5<)+2G;cQX1g}kp27yEJ8Kd|ojx6IY8 zfhRw*@uvrU1M!o;p}&fM*x}J{NjzaflpnPs^+#d+3ih`gss1GB;2)Lix1`zs1`>bA zV&r~gG}*w}%SOOxUP-DgD_&KU2sdojsXi!s7ioLfB~XFIp6Jii|95czFO&LjwfX=c z@cjj^kqCvjXRSrwJ{jeB&zhGS%h5+Hs3mrXv<~~n%#*QIH*kVhxDHex1nX*y278aO>v^RvhFYD{__$i z$N4_!=AaHS_1`ovI&q-|HdU!#Du6h`i$3&tdkO z#s5EWEK|>j$D>qJW9|`dSy`Si{I^`mvIxU3-Ep)B^Y6Al87d7WKl@M?R|AoK8%iUq z#hRwsJc;%WH@(`MDw4{9C35toAfIvI3Q_{PvS!+oPMQZVp1R>mDk&E51Sk*gSRHK& zLN0?5n?p^6d$P2UD-x;R`#hqY2km0EJ@S!2Vdm$Sw^tux%{Ljf&YKx)_PO9hp*?(o zuYj*k45JsPcOyGm7o{i@cU36LHjm{_pC>t9cvD&<*j^bGRr`TtUg;~!gH$sxD|@?4 zk!JxU8WrZX*sZcu#xS6;d2rQxqPNbi~bddZJ!W zKjcWikG#A@RPB^qq+lcae&$w@rcVs(ii<{|WK7{cN6oxAqloL3yH759i`ZE((V36y73D;Y|TXq!PtGGpe#__7hiahcBgF*L^7 zNR*;Z+%-DOHb$arbhX-P;$?cfOUd^VhvLqU|1|Bixooa&=V$ zDZ?i2#K-9skhr)$#!;S9=~u>f8KxZUjcw3Zc;UdYta2P}#kul-lO#hl;vFjY=U&fh z`()}yj@|2M6%)1J0aEbrmt~IMrnAi^RAFtMQhSTt#gCZ9pY3$&jFY(*GZy=0Kuel=l5xzf@56kW&UNyd`bQw5|b5;xO0a6kJ$5kqoF z!hELz22MS+iUIcQYOlg@tf6PelzaZ`(8J?==-kOEuu^yeoJIm9)qeaC2(5p-3D~9n zS5N;Gp9ryA)5|8ZrzTq;s6^pYSx_p)wgRCrFHjGemL&N#S7Tv~{p_G^=*%J4`lrYL9epo(yxPqTO}%<> z-hF1i+gR$|kGHNsR6hSq2}$$qwdS~p!jzjj!xeS4!PPY+@l3c5Segz)n@H1vjpd`j zT<1GE3lkP4#T(aZrRdnEdc3-<`I+o^dlr#vskA(p9Djyh!*VP8C5pAmhUeyO816J1 z%nLZ0mtqE3)NX+1u?OLx)lFTXVtRWQ2uPE#8;U36!tIyDPhRqz*Pv{q#f!zA@ezNjd@`V}%W*CJ^-p#z+7D*Lm#?jgf5||$DvCo>#OK*CW z(4&QAmGtzbmMoQFGt!H`vAgj<+rrNVgtG2WTfbfOtUgl_e>PPMSJ+ z(BL?*1I=sd;)w9gpLVmL(@Y#hr<56Fcq5ZkGU+Jo68Qb69iwyNVh`VChAfJJNX^7Q z(*a8{h_yMfP9p(z?+RX$ZlX%aG525A{Xfum{d_9BPtN#Po%L$ZGQTu0wX#xKEXD|@ zl$Qk04{Si2>i<$FbpNyS?_cfywN7ULPM(L96x!7Les#02k31#S_o}~Iga~q(DPsmN zm2{JQ46guDhywAPLP~9_{P2={SR*FhB{=FBBdM#&#K$4|}BwcJm5O6OxY8IX)ffUxmS_gOgQx*%D}^|}%SHE26zrM&^4Nje zPLVkKiObH#)LCGk+K}b#Cj&=^nFuCMF9ID9LU;+06*P!x>rD}>Qz}WV$u~vUG$uj= zlFJ8TEb=bp--j- z6`Gd?#TjCVf(vNZG@r57nJ1Mf05za=q-NOB9b9qWnbTEi@==?%1VZS{(EX$Yc6XP1B1q-R_C;F{TZPH?wQUdG0qY$=~5p8lN_ut(Ig!G zl*;a1(jMFF#!oJGqk}spC9>-bb=40Z5h$LbJS-uyxb>2cFmK)fpY6?kRyud<;u)*4hG##-juI)-?NFX=MoneclNU93fmZT9h}yetw9&U7u9-URm|@Mz~Nk0ykFn zR--NZ*Mm78FTJTfPGI#t!Q8ie7VP?BzE0ub^f9W+5I6A_L#fZAU5DG(_@ta=8qcXW1MSTA7F9-P~}PxrDO4!Zw36$xcW7{^wW ze4DU$Q@g(AW@b4FYcSTF*s83%r%k_31|J$mDv=duKZw1L`=!_2#iq{8@^~cYp6qz) zl!!e{hxaC|#*?rDME-Ug;Wfm5(_A(j^Xk^CmZtFYWs4Cn z;Z3P~RXLN12n|rZRgevq-m<0`HK{Tk!FbgbhJeIQ`w^f0Gr;HpUCE}nYaQi1;kvfG zqOa+3-ErI4%$=93y<3@WIEay-$pzF^ohP;%S`~>JQ_P}Ii-qgn3Qp_3H+qc}8AABR zxR1=pH&T@1gShRe7gxhbElb9Rwz~KIQ($3haZ;`kne@is_cSp^f?>+;!V(64VGmTVupsu+60_YnE$jL1Q0dZevx zOWH)Cd1fck(QbNQcyDWjPos;B8eV$e0%aP5;-`f-2N>`8WEoR|fP)`I$uP|lnD(|; z>MWDZ$}2{z%erTDrlcKcF6oNz;nYdd$Jk8q9Ci}Yhc@+4Oti~M)(ofCloZKw3de6n zUk>B9`_cNe#BN(!az9DBX)-_d>h4~2xFg~47>=#*mwdIISXJ~D{)Y&BNdCyLH{6EG^4PQr_dj|u z4CJN^dvsG4PHWXJ5Ph~=TWc}OdMeg#zI87NhjUCwc;VV=NU>@psbbhr_S~eZKeY;v z+Aq?=Y~?9B23gT`BLg|Idf6UNZj2XQEcPwAK}$y#Iteoy?}ac%(wH$ML{YNxjQKo?&1Mn2qBll| zM>Uh^J7ON7A|o#xw~}Pj+Y@eQg$kMA(qnWhmc6fCTvDkb0ZfpbOuE|gz#UR)hG46^ zSObqWt!x@xyobHR69J4*aPW$80S8-bK3=(PaOMEkECc&eUu@Rm-e351po+M|a5j@` za~EZ^`XqAUvU_bi&r#TFR1g9Yk8%4P?pw;a82wDa<^f;+soaP~*;|v!!jJO!GJ7S+ zv^Rs)0;n9Q0;@y`99*qG4YDLE7wa(hR7N1bm%;V*J_K=wAzVTF?w2Z(YMvFW#fj8HI@QwYmiZzsEun(>%MiZ8dW-*<9`Xddoi^`R&Ba zJ1rVhiIH$jpGxLNG6&SyjoMo&x~A1~3!4fOA`N%0UVw|d@2>8vUQ+bCdvcq=8fm`cJ_D9I_vEJ|F`-B?~CU0I-LfP zsA@j|Aes#bNyb?k=NZtX6Fe|0-2wKVRWlur3@>{2O#*>cMK@>Wo4}<_X?==f6L+$J57nd`dMCxF zZKN8r&@CRUtJMob&PEeAQNkL#O%ASPJVuF9ZXP3-~aqFf5HQlSv-ZZxTcFw z{)ZoLEPJJULM9U})*L3=aT_=axmsu)g)0Tqk*9V9PVv5!_W1)YzN;kaXl`>7t6<`F511Hxm7v|SGk`Z5O8VfmL5Mt zufK2y87JRd4!|pjtER z33d@_^H>{IsN<{ru3oE`<-bUJnBbjc_(Q8nr^#Lu!7arz#p+VGF{g7f6dzbPcGyCc1>Va zRGel$ONz&eaJvajWIMj=WCIagdsd_LXreiGA&ZAMpcKnSymzy%xv_FLu8SEgVRyBy z9q8KDHbd~~E91pBU*@6yzR*BBx3XDt47BU?mNQ|?@mvwqX5|mtQZ1v$!2V;>Rb|)B zGM&%L&Cx6{s?WJ0os%>0%8z5tmjZdks#_pv* ztlqKf+yontor0;$gk&fOdbxXheFpO>V;7Db zq~*Snc9ld9)o`jJ60|pLahk!L>6{T9l%eh!oK$DOlPEtN=rZV_WLd6ch?|Jp4l7j# z*_pQxo$(+eBU`o{AV31I#!~(q-O10;G3lR)sA(NpjsMw&E3F6fbQe-)T@pCh6FMOe zB;c8BBsnkjF>DYoO8t2){o-{;9pghFez8rM!Wh`gC5DKmxUbx<-|Ug0>SN!Xu4X+($BTXs(TSN>Em%?6MiIOlj^f4iH`vvxg8Uiu*$r z6U3Xx?>-PvDwNbZOB>(4AC8vRxn9kmUaV*3CH3{^a4C<9k-wdsSp?xayG*CzZXf(! zoEudIx_o&QKeYM;KJ=QWogR}_n=Lr!CJL+@J2O{OU@e__Zh4z9Dg26%j7V&pT;OLQ z3(7=q)FKlGrY**scp;hqJoajpDI~Blx)=q{H|xSXy|vA1eMK;1$jRmb}D)u_mJ{PgM0D|Me7>S!>!AKe$fJ16TK8B%-iairMKtzTCdL#k`1{i;Obj z!I0x6k(#&{lG<*Txt(hHioGBMm1G{#_XbFG0^PZd^2=cbs^)#GlKorJ6YIG~qt2-!KlcS?7jb zN?f`!7@bozqt>Hi@maXLNj@wW<=kzGFbE8ANw3>Rb4SXRH}cqfH4M)|`y_Wzxppiy zdhpNKm?Glc!XbO9+^)4mnN)3QNs~5tT$SYBnk*?I^_b%Cr&~Gx6zT)IzL~Fb;XqoT zGFs&faq0nTRe;g$q*v|On5Du$8VTp(bW>WZl55JaZLHH>8+DAYZTC0r8pa;c`z5eq z$R-*%yKXZZ+OCF0Y=swJX>0$u(ok{t^IjjeOkx}M2AUVWdP5j;y^4Hm`~Xi)%Hh(iqqTl0bh*5fWNU{SD%OAS$<=eM z`~W7pB{2sa;MI)PG)0EaZHQi{J z%+A*0*$X43C$J!oh1JCRNnji}_xy|_v~~Ag&?QX9=r|Fmv2nd*4L3;9S^gx|GCT=? z^vYC&&EmIIKj3RVz$>FXy#gFZWDzYZS8Zjc3E8TC0(!(|9X(QYh70_uoscdXdzlmKK_MPx_g$I|bi9U768G*uMm5&AuU$wVsQ1qtUs40i39ZzV%IJ-woww zc`dTukunepJ+?9hhMy(BaD7`DtMKUFAGG9iBb7|g>H-2?Cu#L+Oq{x0#xQ`hL15{T zW!J&+t)6O3u{G5ftTf&4lN4398gWMlU`#Rq5xPPF)sTsr7Dn*6DmNlVw+3zLlehGBDP7_sv-cszA zYh|lTZ>xKF9-JHrVq$QnfWfKMC18Pu3uDpW`rv>2w>|AcobJmh1E&m}GH}YkDFder QoHB6A!2g5+#=oZi2}5|=hyVZp literal 0 HcmV?d00001 diff --git a/docs/internals/compaction2.jpg b/docs/internals/compaction2.jpg new file mode 100644 index 0000000000000000000000000000000000000000..738222cfbe65a16891543e0e93c0520753f2f9a4 GIT binary patch literal 53207 zcmeFa1yr2Nwl3NPf=eK{Cp5v`9g^VgE(s37-7QEOhX4t|-Mw)Q?!g*p+=IIWeZ9_o z_wJp2&q>z4@4j(tb&u-NjIO`_n)=VGIltmz=3yD|L`Fhd0s!|20DyzN0S|M4R{&&0 zL?pz=$Vf;?C@9FN=(rf@XlUp}I8U)~$%rY)$%siwDS^y1l+PKdNJ(jV=oneqI5{~f zX!wM9*#((7INARg1P%oS1sx5Y5Cemd{Tb;q_J8{I&(3u0FU6{;U6KuKYomW0PF1o`yGIQ{TSyN zn;7C#)3qM+gv5E2nnzo4O|qvzz}=HcbzfAv~i zLQ+avMpaE+LsLsz$JoTw%-q7#%E{Tq)y>_*Gw6MANNCsxP+a`Sgv6vzpOdq)b8_?Y z3kr*>s%yZtb@dI6Upu?HdwTo&2gWBRry$cav(S~*we^k7t?%1A$0w&}=NFe(Kd%3v z3l0GPKgfc8{tv?bfi7$qU5^kD;1Q7ipbPGi8?3=&BRqb_hKM7kjP%Cg>2vk~WZajr znUx(VR2(WtcyArYQ1OABE7Zq-koGT>{W-z{|Bop9UxfXQE+_yU9u6ivcx-?O;PRR_ z%OCZx_7@NS(t*Ep;4dBcO9%cV>Hx#%W8Wr^EcunLnv>%K>nF4FElxMJnl$908>C}v zEv~1Xi)6C96f7e=iAU*dZW!gyi+$gJLy{MFg62ngb7d_mn)?+2HI({0*@i?}O>;0|0A>*k&;s zu@D(ty}Ge6BtiZF;IDt|Q^l7v)@+>N9=KiVydZ8`uXZY!AGzk3z6jUO*t+)IpNF8%HB?_QUtS-ZzE5|IcJYI6iDo2(gH-+)35;ML3$5> zFkk9zd~(cINX?h9gUz}G$|uc^blQYRWcEbv{aZvm9K!%NTn&uFh+C0-9qsuAD-!%8 z{5mo8NT9#VbJpA=f#YrkLFLSJtWf>4;`2t$yBnX%Lc8gJBP_{szI0@#ekGG_Q z7mO=WXm?g}9`{nbY1dQoW53)j(fO%J`l z(q4fb(OThBXF^baZ4{*~l-an|8s$aW!OqghK>V<>9^39u(MjcG3}gPNu=rO1^-;;4MCpx-;i!)SVG||*p@Zj z_8AE^p^;jteY#VTr?On){}EmIqMHb4f<9dm8yh4AP=YO2W|WX(tr|Nonh3(3BA$(q)cT=jF|FQzmG?yfV@S$GcVk--R1aEI=2R9XSZ2T8 zh~b9Nzzaz48$i^xT@xPrAg?C+3qq##$G{JOh-A~AXhAn?@@V@w{g>>KH+_Ag3n+SV zk~Uvk=woP=g8^7#vZ9Z1mH+EB)hQMV#R}_>ktXy5sMW@S?Ves!1B1r|xBia#=;QDMKp6W0uq^ffSfzacyw9`yaTNUkc!c!;z}8aJ{BVBVTY63} zad=D3;Wzr;@9K%)6-Z=-?#sP5)dQeC=mF5BfrsnkTj3)U+w=fHky|=lp}0q&xPKmf zC1dsgNJoa@MQ)KrZpGgy zOHS`1O%{^0{cdK;=hq$oS>|Pu#_(B}L=`cqX26I%Buc8ko9soc2dQBm$d6C1q`}bBdGVh6z3@pF=)={foH^a{j{YE^| z#qC1P!%M$$fVBTThT3;->YZf7-&oTc_uR+?yI zso`q4%Hq$0e-*o!UWMO_I{CFx+(+n4mb78}^!Z2I|G*`0G-q{5>`m2!ii&%88FEb|gQbDtgkgv4RrFK)D-|%fqTbK(Lw$7F72wF(x7o{jmmKX0a`ko7D zy%2+Uy4-pM=Cr=q6;jn2E6Yn#vZfthg-W6VV@WAu)&MbT5yvSI>cnWDl`n=EhJ)5z z9W@~L^6Xkgwuupe6I{m*-Jpo`(sHa`q02C^jjOEoe(9D}>3-Y_s?@+|#%@JQoO$y! z2JM2`IXlP$VB?BNY+&aG7OFe|*eNirbf%U-=hi;@#{QCa2(danH(k%d&u#a6Y4UTO z`O6AzvbRqR7LrcB1*3*HeEtBSCF=*G4pe~a70Gr9SkS`}5)*gg@C391e*i9fB)8Qa%Tv4MxHDpQZ-6KcUevd~HZ}N_2eA`?gAPgnJ0DOT zjfCOjRI?$KZeOwE*bhdfRXI_l!QD~3q28>8+3uC$mj@X7Ol#vBBM$jV+8Oq*Fb3M# zRQq_zL^DHeTgoiE4KsD&XMHu+Ei!K$P_?Bh^*VuGroo^5*&DV`;bWwFWtarD ztz8PJ!)JE5tK~_g?JhoXqxomkm=$B76(Sje)NpE|POZvjWOC8J8XwG+U%p)aBT~Wc=ys2% z1G*1sGInE{e=2dTF9jMURHrBNR}Sh27~Nb?OY5}-c3ph4((xHioy6(%;SAUaE0QXw z!lPXviI&-6ATFe9dS@wJ_t?H*^Csx-`~mPSSXz1b66t350g#ERcF)Oo%NPfuEns9w zsQ0>a>*f#nf|&1+54Efmub+Fo1gv`0NgeP*6h7)lSy2OBUQ%Z()8t`pWlKZ#WL9vZ ztQ4_IoCIouQFBCO4tYW;bL}FONNw8(gtE${aVl3i4p|p3k}n+0S9A{J#kOVM$@lLT zz3Au4w00`8)VW&>VP2ndX@U~+*N#UFAc|2dIK&Q{bY|~p==g**UT|=YTTNI! zo$cX~s+4b#f^T|E?{HpW0jO1loqP#qpjgu5)b1LTt6@`%lMsoU!+n8U0R+Y!K$sDY8t^AJt8>z2f-A7?Mq|T1aA2$(RmXHyXbd>1KS)5q+=LtG8 zt@G!+!-1>Yz^;zia^*z;_0^68YmZL8CuxvVrzrG8xvfOB=th&dCC6>Wi7dDY`Ljz zyv8GOxWf=fUa&97KFmyRJ&F~j$5TZu)B4(1oIk#$t#stGx1GhQyj7z{ncYf<*Rywm z3-5g%04;TPYZ}qv&Yjt_R%{-pOsDT^18MscC5Q%s&k6ZP;mNER1f(Z=p`{aIl!3Ce z*wpl{mFPbJ*Abw#iI;8R;z#@I7mdl(k=;A*ru_yjRF0n=m8_dalt%SzVi1* zTgrY07W>A>`r+DE9K(+nY8t%_Y`SZXgA-H{0uiMsmrpRnt=AaPisz&-a&NH>Qfuot zI%?^n*a3W_RC%JKR-3wV*25xQZrrZKL&0agwIMz)B`wWqXsmiVUEvOQ6dv2|;`6X? z-{VS^<{H}-=n7^kr8B*Lha&867OIduaXo{(d469%1+J`JdkkRjTpJ)q&V(pWyuXTO2gUiJwg^xy?JdtZJ$I{8|;KE7smmaDEn8~vD-T13zr;)m7i(C zz`8=rH?lko%*B}b_^FR4diZemY4@@tJESlA@NyB7uCYLzckDJYYf9kB%%nFBXsvw4 za8alMCozhIYk1Kwq6-bSFRyoIhUyfHUloX%Q_fL|V@?60hSb=4$7{y5ZzWfU*{>IS zPM{NVA!h6bqGNW6;;6U`Mk(lVo*F00wTZiax_+MnVbQir3(LJY#>-vpvB;ch;>+~1 zc|A&EIbC!`5@BR@C@G*pp#T19^1)`ZfIBc*6NS`Df94E;9&g}RI=^rg^wNiWt+S49 z^@UM5{ah?ABgqGR+W<)eOfSthLX*My6TTfnLxDs6n^KuZpQAJfmqrLL373gMv5lHb z*SBbSdrNNwWBb+=TSq6QW;Q+8ccSMYe5k zu&@+3zFK3k?+0@q&Om=poP3}yd_hUr_@U;6%XGBjXQ1B`(M^n|&uSj86X6Bd;`BvbxW9_Ns^niv7B0 zX0AqZV$CmK{nKs{$z)otmYUmWg5drIGvPTdzjsSgjzsN7=UE$w)VWV?IJ(qT%L4( z=aM+MHB)5P7T-k_UnMso-hxCzEyAgzWNsLiY3Zt6q@H{ZPM%-rM$llJ&rdvDPyIpf zp)}k+LVrFy)e8d4;}bg1-AI*`3D3GvPc=kH3`F1|`W_Fuu05)rzHL2~7bzD@V>-q- zbF01Zl>73$uDMQkySKRh(X#`w{xib!GHCOX=21w)n=COK>rz?wynDLO_}#Ke?2k;$ z`e~5j(-mb7aCWS%!-a$`%52TtrMjG>sas(ix5&qItG^EQW(iUKil5pA?U+L_@bhuX15$mI>SmfXMyl7I2mPcHGco8*e0+2Lby{$uiRzH z4WtRu8!ed1y~&qXTced>a<^+FjG~`kK8azDThy+{8X66**R(!WF zL7IREF^PlIW@8}(`CGsr(P+cIlh~x)#<6rD)KPvvx9geoEB>DFdMeK&k8@l;X+MLP zGA!4p{KZ&Lbr%hLjkyxegS+|+5Sri9BqQnPJnbWmvG*w!xyMILx-fQSJbzIo$u{iF z)aHhQQMbu=FM3`w%sx)Cgl$>XVYy0|IMZCtUC0?0LzZL25Sc>$34~@^*2Y zdVTdVY|e?AigpgU)D_9R;m@UjjIUxZ-G$^`zz(f-FZ3UQXp1PLDC+%|?Cy{!6G%nu z9{{aO*Vu)0k|t_(Z(vEf4~GY<@v;+)G~AcO38Np)QBk6}R7TXuIRXOGb-pvC9Ls|T z<}Ha^z%_L=3XLp`Y#1YBjks^K8;Fo_b1NjBguq^H=vMrxkqhh4;AVbfEAB&wx*-xw zA`~T#_?n=o%}mSlR08>N@LY?5S9#4K>LRp}@`Hv7(m^|*P^GhBgfGhr@s{pWs+}IAU ze(E4V2sEWw{NcU@ZP<2caBf;qfSz_TlH49-5_u$C0$w*CQ6m8Vj8P{s|3QOkfN_&YZ4W7IN7-rNat!>L(OU? znk4ZuENz&z(tvLz+Obq8NTtqJzva<9L`Kz_AH@JM1Wg>*HIo zda?W;1hsR^q7ic!W zBtoGN#SmfoiY9`4Z%uA2qd?iPLn`Ef_c$bhVx+{q^%G^Ip|?*_p)v-Low0n;P`1ax z0gM7h%TDUIk{L@o*XbXg)dlxLo$HN5BnUNKTy2^f%`S zEnOXKxF$((j3fLpr4gb;jX2b|j=SHpcv4kP9^mw7M(s#3T`~%I#yL&25%F%}1=yG+ z2OlKh%v>dFXVphpCMl)84w=apsA>3Awl#s5YvY6QzKS8i%`OvyFQoI+ghzIRGw>#O3;4~ho>p5%paAuI=m>(d;{JL-dgU6K#M}~tKw{dquA2jWpBa1oI{asvF;C< ze;@Jul8OIIZTKQzb#hNb-zOa#7|zTOfZ~Nx?hTWX;iqqgXfF{*P_k2|Eynd_ZC#P@-s(jejxIw5dBC(4%7l-1|2iB=8Kbz9Tbf6tY%w}*ERf& zL>u1iDb{AB9)O(xgL7&0ZwsE1&TOkd>k(dz45`ipIa8{F^3 z>AxUf5!zYb^6_`QB*CHf7pi-|)?{)BS?8=`r2?qoColf9+IeZYVVMf|iWFODK${a! zo9nnQHRhVXgj;g+Bi?k@Q!d*}TKUFiLBEd;QCj9EyPeXcd7$+(7y~MWgZ%kX7k;wF ze|WMOjHezt-0&xYe#)RLq)tL=W+zu`JSj)lSYO!> z4XUR2xm_KutwE=jcfZXS{wkOF*KhxIYQ+8<$&z#{LQq;HbgyK!qE@z4Q@KG`me%C$ zQ!kT5M?PoDZby~El9a)%%FMq&$G0{%|V;$UAKr*YsI_i2-@Q^*e7w!&4YjT5|u z@blW*#BFEYW1%kAvMpBjkFLt|Wn#UM%(koaoCV!Zn-xA|2>46BS|EhW9v6p)GoRvl zK%A2($@OSNv&hN~7(5Ba11;x&-T>NB(ffG>K2rA?c`SU|y+~0}QQg`wRT?*b_SA-Z zFQ;Cm7r_hud>hh0f?+3j6dmQQ;Vtb#7Ljz)>0s@|`^-G}EgG1lK?9xpd@I+fQ7Am8 z72R^598EVJdr5zGZs3cYQkB&4ZGQfx|0OQmQu+sjC-{+$E+3&~%GoC-r)vHvLtbwF zaHO$Oo>RL^MDA`1HIJ>4eu<}ZuCK+`qed~JZ%pgywIezQwl(NCU2T+Us_(rmfh3#+qM_0ql?bWh0x<}nKV7N1umMN_7C zCtn0s^}OQd4YW=k6BNt5a{8*%6R=8{Xk)maWZ;{89M7CF zE;STPK*8%m54V*MmUi&5FlO9v16tSFN*D$yQ77#B4-}2X~VK4 zXlyNom-ewEkp!~w-N@o1)HwL-l5XV#BBnmw8m|0srYg-Vgquhddpg>Oyb&1uCZcc< z6Pa+7Ys-MVwdUv|e+9!_C)Vby|CYwa($0wSl8_Cc`5h%gGxyH`X{qYmWKxpIeS-Ix7TzX^z65g*O% zK}gRs;XrhOSSg710C?6$`~aA-X*-isOUHH$U$G8(04O7c*$^AgiY;cO+YHsf6??3f z(sZ_&O`NxAD`XjIK6~@_)DOk<8AR(!AS4SfFIBk4aGf}|(%S)4}P304b9^d^jM>rx8h4UdROY;0&YtI}u ziai_Et_ut+GGuLkwwySCXECebWjDYxF>;Lzwi6kry`2nFbRzZ$c^(y^$~&LwKi_b` z&R5iSRMAX#u~B_L6Fy|w92`IBN86ubbLzS`;=ZeGKxK^7jfE*sil~kFxxjbdvp;|c z4Ys?$e@bR&c>nFnsIT!Z=XKc3m6zPa)Z+|o^-od?+Ao*meD%`HDaogbuckYpc)Qv= z{Ge`r%DC5BX^KpWFE`ZQQyji6u`-|?;|wR!9UPXR4m_kVqx+2hgNOuAjHy=;-oR^B zcR@tqI0Woqy&8J7aXa>U8rzDypDQmbOOQbVizfQxKHpk?nMsn{u~jS1F=8S6l@3H7 zuQc&>jYbIM?eaeTa##>0;b@+a1^l#gq@Db9iSKOgLhcX=h=W;7gG@c3Lv>G->vNgN zhXa>zFAQAh%#z>sh8a zLK^3nXySE;_M$WZoNNS0D5zO1OJ@lKsT+R~GuF$l#g3WciS+Y2Hnw~Gc8vZ_eDn9*uns8cgymMufh@J< zF1Tt&f3(XzqU|%Gt^azlaSOnCnw<6Mye!le5SKp83xCYQAo%pM9V7n{$Q_q}4$w6({-p?oG z2`)D3Ez}wYDlj?f)`4#oq$rrJhRC^&Hby=;l=K#fU!YmO<`@_lcrsYogzDO`!u?D&ISBykhyy=E;% zV8pi;-|bWfT})bSBXXDb=3`&z&mU&$9*QY^1bzCHPI7I7Q6@4!jgM3lsrV}3oFyc= z#Bh$g(Yq|(yp!KGElbLNy&@*10zS#?YgF?$B=D{Q!yf`=vvM26ouIQ=eQgOeCDXNf zwED#()WmGoS&zylh=@dI}GP_Ic7AolFRk%B? z424?h$bXtRG~N(Xn{5U&mG|A&~yn)qOOBIE$KnG1Fv( zQ{aa+_j#7f+Ioa&@3+lgSGje4RPLn=tT9A-&31k$qwNP9i|Q4u2frv$^_2mBS-W(wvr(R#^B;K>oo6>y)`)|Dzd!zY# zk(wyhl_;TP>~#PDqN(I^|cTz+2Z-6q`KPV8ac5_RPu{ zOiT$$PaPOA;-@lK<)4uZrONk4|8j*)@1!&GITnruv{Rw)+9-ul%n^91M%TX|p)T^v z+STk`HNQ{7rJ6q!B4B7ABTRp#q9WD|xbKgh(tizfkc1RG09bu=%1=^>)ENl_V!y5B zGV={P`ThCJl-ff8K{MmiJUAHjU9u6Vq?bx;U6I*koPo<{sTXLCLs3@szr*`?^SbP z@*{i{o1pJnfk57+$@gcOs={83&WhlPt+(}Ocq=7J8OV;MWbwz%;4{kX{+=v~e-Qm2 z3XSzIQ!DYqwZOhg@0=!nU0s91A<-kQ-}NrmFu!%)mrL_C->a-~!Qezcp;rEl3qFF1vQYu#{(qi-8ZzH4yp^u^PKiu( z-`LpT&m-fej);v?hv}v9DhWAB{weZuvKrccDd)f0X&mTJI|YU>zj8RjV96grJ5zoi zw`u>Z9RHf--!N!D;cot(mv2SzhK{L7kbfxoO~XAq*?wOkEZg6 z0a5&Ou21}*aKO}ff7U28|F65~fB$)Q{T5Q9lmDVDc3?+bfZO@wZ#Z!G@dgdr+3=eu z6l@O`wx^x?8@|J~SA+iBjHPz=uRuzwIbnJAUmwfx6KoWgoPYjDa(?L$w(XLQpOxP} z5ZVRO)*F@eXn-#Q6>hh($KgqlKmiD6fd4bG}n10 zPC`^GXvOCCRe?Kby0g|^k1GQD-bvbnlhRfU8`{B+Op@Nli&vO!xTL+4+!N`-*%+iP zk%T|as|*ZVz6~Wr2+T2$*eo2gv~Ig4!70kGJI@_S2CX;q6$?VVP>GBTY+7__ouSTN|C<@8xPZ_!5_%dD9FC$Z!M3(3{Hl=;caS_I_#*H`a2G zgP!boY46+aF+U3(wK$|g`E(lC9w-5~JFH%JoEqGfxMyRxQr+M{N>D9UON4S$&(}Bl zRunP4I^F0Z_Q2U02I&uH+=rbjpaQwx-OC^;w8>bEF)`od>*w7Ap76;abdP ztq*3*(JFfrCsg%yNR&wL>_JQZTf)t})DqR!%qgAjyP78Ok|zy#Og0snm+SP#*wh9+0x9D!yrBTb{nJ|18xnDERp!b+v94N|RR6wMX zb36kY;JKa`#)DUus;oo`JC3r17c&6C=q<8HZnPSG=+&nKxfhh{44wbs`8S*BfF zcD<4ur-~G>7VRTir;O8UOu3=m)e**TX_;*1c;ivfTXmpy@&I67n2#YLt73IHR9Ue& z)dLF*ew@x~xH%-yp4D6Es?`aebd@AfoqI{ltHAR_L5vsSslS8jl6&IFxUANjkj?AW zO%{GyCaU=!k;9Z2z*YlpOG*<`O|+94sFR#=`~gszJ2{*88f5rv*&;fnY8{KkTz{4- z+E2zN3AAr}?p{;KSia9<8kq~zE3^}4zCNH++F~eg_Uvedm((RsL79w5nCI)n#LK)n zWqDyUWESGNOPSs)X(YZ;(nlOU*0Ku6Lb-Rh5EWemt$A@QDEFfH#$h-wGoK**G%x1@ zUFo+3DWT7G;!ZK{6N;(JPS$>S30;A_cDX_Cn5Dm-ruZwq6vf?bBh=B|AxL~OW2KqI zZQ`+QjdfC z#w3E3ie^Uhjf|3m%$+1rp+Sc@t{OfpMF#l$CW&(|=jATPhTO zc&D@Or9O8(v-`b_rn~idYt+T!sC2xE#%1EJ5AV4f;I&X6pKuU4cuq|Qla6PyU|71Q z@iE?T$C@n4->PTVkPh@eKv|t)P}YEOs*6`e5XiNc)VuP-oLayTn zKYuUZOAB_oLqn7JTfM7%g^-b+)uoXZo;B_YXx`RGxHwVXcCr%*`Wj#E3G!*R)RpG2 zZHq`3+=_kYg%ou2_}+=wKnu0J>pxTr7I_D)r3%?n1?;(SdQ5`zc2(~aBK(SpM(sU% zNvDea@VBeBlcp!a!?`MblK!EJHB`nrBSA_yZy)c&n^kMCxvM?1s@977&uHyQG4D@B zcWVMY;Nvi{z~->UhhOg9r-g7K>5|w1`M%_ z->XZDObJNz3i@R(U_L7+i`N%cnXMx~-=f7f|`15p8jgiVl5EU3B#Cvumd9y2RpUek#33pmAvTrr!sYAk zH}Qas5%h3SKKglWtfgC7NmugLgmY^0i9|9|#<||S$fQnf?m;Ox4r0v?3W`tdg72Wj z)&OJS=xQz91(L`P-hd-yff^1-Y=z&o;s7IkvVr<5CJ!+4=I;hf;H<;mE*&$hsm zw}HI<3G4ljxn17`rJQw(+pVDy9k4LvfLm)Ds=?NUnQ2c-3Peb8yo}ZsqfAIkzFU;3 z{~wl3>Kz}DmRB~28);wFj_hng=ort@IGcEDlXoyau=R){0Rtr|S&52HuU5Yjv|!Kd zxS<#`^tu4?ZxYE`r1`d&B2t_{e5<4-+uzGH4!d!p@hlQuV@A1x_nU@HqF7%A)mcAM zu!lFflhoNM%u8#Z5EI8$1@3zPguc}<4E0+_!n!%}``%M3bF$GS#Jd*=qRnB=0?`8g zt@acARwp58mE{NA@cpgdHJcT^H+q4$)OtY{=tt1^z~AC*%LYL8vWeTJL^zRqV_Og% zq00~(Z!SNkl4-(5z@=W=TqQWCiB|5)aP}zvhV4r z=uN8e1pQ?l(%j3^Kn)s>3cH&3ZZ9}L>eMbBF*-r;>rmEO-fpbK8pVw-sJzU#equ;T ze${uooqV>Y_6c zunYSG8QYa^(i!vir3if}?s^A&znQxcxQ><7qzAW&wsgbVb6LA-K;E<@vlSbMG+5aNl(n8=??qQ3eHOA|_^EK66f;A! zKv|V|-_JmUZcOp|v|%&p{|kt;k)Do!_+A85K`-q~5!5hv#i(e5n43R8yCBBMGa*H; z1Q73>>qS zgF4vVc`e;a3vH}9KZ$w=5xqrdHD7?Y+@2PE+d>W+$K&%P8X;X!;6)K+Sk)a&s%mbi zY^@E@kkx{LNMEDwn;J-n5em`!r|W%dX5+kQG##6C@p-8mqUJwjgFU)vN~T3i>X~kJ zhDx7cyL)+O9<|4X%N)Z&5VSLJBG~JRIz|;(#R|Mx>pYP{ncg-|9E?Kk zw8U+24!L7tX6@^)PNw*7S{r5?dd;;A@rx~^0@F-GIz9P<_&OJn-nsi7$v!b7JZG7@QfAZ92 ztW4n(mw-Hv5*B_r-uI9T$4de!kXi~icP+{@>CpjLNv-m^VVEZe2LAzscJIJ`TG`{> z`rhb(NBdg*^ozz)O=&RXUF)DLg@sEjIS>7DV@fd{88Vly+|hp77thOK2R9G*u^%tJ z8r-V#oWSk?^waV5+Hm=nbAl?%S)0uM&Rj#5uM5OovkJ`FX$v_)g94K%0l=v2C~H-N zaockiT)9|@UEFKD;evF_nX#cKrF+=FSRTQ&hgb#03n2hB3uGC}pi?-iYSOHgrBhl|0_ z%%wwbxz&scI4+}f>j?2?$8>J&SQHz3d~;&vGgOZ6j{1Py!-872yQIWT$QZ#p;X6WBj?|yvAA|x*WCv zLadMPzqdy?MX?#giW@>G-ueU{c-$qXx~yz-Vn{oG$wt?x?hr~*@yW6f?8PLhv~}m* zPCHNaoc5eK-iptg@8yI*!g0~lTizjc&~pTOu9dOrACzY5TAk>QoXpQH=-QsxGUd#C zCppAi4hrZXE4-Enx6YPi4aiHhTpCLASZ4v2Mm*}e#sqw)PjIodZ9B4+%{m;MxPKfm ztTN&usJY^?Ru`LDM<8rn6j!-PiKzH4jUj!SlT6~?5cd;PK6UxKgJ-BRL=aB>4wC7h z(Sq{VS=Eu|B)a)O=Ev1Omiz6>lHz~1#5fGAnKi08#vOcTpqZ|g{91~{k`Q%m%Q%J9 zA?hqwM7#ejc)6^ro@IJE9C9V1qD4YF;TP*@6n01U~1rZ*jB>hOJqYU<+# zw)YP>b84STVw*@K1cx~aA0ic&)xvzTY;t#xU>z*H%&7Vk;pLW~s*PhEu|&Cz4O9O_ zS@6G_(!t!t9oOkJ ziiVi}XBOl=lYMnEYDRpdh&z~j=xS+%AzPiTWMiDgO|lS4IqC#q z0A!GNr_Yr?Tc6GOL_W1g9)y@R48vLysV{T*D)0iH^4LP(`JdGW%bNeu~O?~ zNX@f05TdqoNKApVd`U#%s50ynh8Hb3pma0M8o`}8nZ}9IGaoZD`ZJ%j`_)xzKC{y@ zGU}IUW|+dh5t{M+n~AFjKIYw?{m}}I{&k^gym9Uq{V((@T+d0@0UoPOLWbgC$eUnh zVvF@aF-8utXVO&fuw#7RN6VPSXF#_(6z!|#=NCo*iJLp}C#+-yf&fHC0IFWJZP1qP zIp0kr|K62f2Wx3g?!Ij)u`{yM6D=*A&RR?)F2gQQjQjG!t2h#OJ)eBp5MGlv!ETyn zb4_`kSUpVx^eIojC2&Xc)mF>Dyp;Ly&Rd>wOiz+?ut4$ot9PoYvqrCCJmID%HChwi zxZrmR?D2|GcB#?qvLXhmz8)!>CaY*)TLcZi)jj@No zBm!gEF`IJzqT8)TS)Fi3#SNxSpJ{YgmVUr>-ruHQvlXlSl0(l ztz(8Aa{AG+Ipa9;@TB&ZFRwKsag!S>CaN2{f2xc987Sadqe(GIn`)?E%$3JZTdmhw63vvX)Z-oIU{tqvWNoTLq%wo1|b1zh z185cv*A$iSr`&K=wTS5!P}bt6i`H!%wyN6hD>U z7lX5+3zV_Srg+PLQ4(DS0SRZm7H(2oZTmOzQB69u*TGNzn$%yM`X_a2Wn~O3hG4vh zW1oHX%@u|eO8lBnH1sx{56=)D@1!s%n1x?QW zZwr)vT(JE0_s@e!yZ$#@rY*SUxw5HmXq~^y{yg`W4!vMQoVfLd3Kp!oypc)(#Or* zj^2jtWj|H#Nm}|M{Jav9uP?Q`s18>@MAfQ6BDz0*SOq}Cw!a8KaFpQN)hzcmU0L8C z@7dW^51?#|=(k#9d;w4423Wrx>{U(fIy%Fq0Nr!P^yCh=rtui3;Koy(QN)zNi%!<; zQ^{RvdKEoMtQ0RW(@@c%YDf10obRW6L#nX6ryb*x=ih=Ig7k(Pk`o>p&-Tfl-7N%A zPI~fKef-vGrNvws^tD%2v+=WRH%T|p*Mc-I_#@(48b`OseR;9C6Hnx=Vvgm}PNU)s zlTGwzY(YBq%);LSQBv2hCTq|i!JVXAWT$DSyuc}pArw0iuBl} zi3fJD)PVtMp@{xMijv3qq2S$%$_Xystp>GI1TP}AJckB1xO%TLad}wbyTPce;+_Do zu`SIx0KMU41>Ga|xL=)g*-HZ}EJ1TMI$)pxX>gK&()A=B^zC>l!p0kWrDt+W3^*=T zq5E3b!h?jje=p;a^cgOktRjSKpQR@;jSNz@2gYryiCZNJw|0W>8cUcjYx;yIP|vr6 zTw0F18`^n|J{>FZqRCZ?=?x9}yAxJ5!@O0FJq(9M07Q_>t8~zD5m*cHR`;VgG~hJV zS~Ku22%wb>6_|Xw-;n6fWhbbfbLF-N^{p$Wo0LOc zXJj<6TVX8l-vW{&iS28{QsFJ&w)guk3;5R%UGuiUMd|c|EY1Lvc}&z8l-=>F^A>BV zZ>=0qeXEmRyQda%vGW7MQZu6qjm)A{G~0^}k@8fNXQg&(AyjvfMeb=GrLQv4SP?nn zM6uM(kkYO)uvGrXKVUeK72;5qJ}_2f?NS~CnUj2EB}o$Mk>t|6$0rCDp8tQ^I}50| z);!x67J>%?2@VMa2oAwrf(6$ixLZ*W+=B%OZUGWJxVyVskYEKAUN{7I&8yoz^SbYS z-MQU!XWqPd9oAy8U>&MX{XaYBe0%R-jrT0BTR3-Ag`4df(TngFoB2;8WFUqvYlDl= z%l(-?Uns{papp^3vr;|r7T(Y^E{4YwLMbqb9_v@Vk0znOu}pMNufm|OL__;Rr-P^X zCH8a`30mzNb2pBc8GzxmydYkP;?Or90BrM7l&6ljICd)r6Wl13jx>q|y;MPGc#SLmP)S+)-D0}~{olip&0`(AYMR>NuJEJtHiy}zBL zX&-%OK^YId*ZX^woW1aUw{ly`&S%o= z2${mAVWNQ+k20|gk(371C)s0#VAzut#hTNx{J3?QOnaSJIHmDcVYJa~PtM~UJS5Il zXt?Yh!q*K?U}vVPtZgl;(Z;C1h{R1ero#G48fWd2j-9yDLrz{j3a$=z()KRXeNgu5 zU2CFuLqU4H06CzqJ!7#)h@xTC=JdT25430G>DoYBAG2^12g#PaiU+j-qWU+=F~b9mR7mG(l~i=4!)UiAH*xrWiJ zG#E6C*#c=42xEDtWtnj|SBr(7z^j4K<9zcjOr{@cZRoXsPYY@NC^P5BQ|Xp=FkPVA z>(_D$nsgc}NYB4uXEr-rtra(1>@QCk;mMbiovjj#L#P|D1KvD>B6*pPG`FHAN!O9hUhdTDU36XQ5>$THWUD6IYe7(_LD~ zQ(%NXjnvUxS7b8S67d_VfW2#p^R@Qa4RD18tu%;6yPo4RyRJTlfKpq`$dvBJGdozh z>bRFGWa-rVfl&6rxO(Wd*_2}3nDvFZ!uW&?HfcQG!+3!Gw`t|6M8FxmJ>B8$8OWq? zX_RQnL@FIl9hO-L7Eus94C$DBOSPrpUK+SuRTr}UMZ=P)gi7j7`8#U@DDSfgM7{`n z{;(uLV}*l_NLwdC_icV{d0ClCVy|z#xN#y2CWcptvJA$}$}JGbB#-Zad!QY|+sKS? zm6OXDsd7yLg~I2AmoI~&)3gA4j5iV>Tl3;YtHkc;s`o9t&htgJA$K-P__keF}vHtHdc!UM6lnY%^!oe*%ASpOm|cl zlH2Rh1uPp>k9hUq8ljFs_5ytpaf-bdoa)y}nssG|)irUB0}%I=M!_Uw6q?Xa53c;O zSbs?rEF2~#T)F*pavNQ+%w$4HMmYb3cBj8&q{rY3Y!O+z^E-fo0;(e4YcYnpc;Pu9 zj4MKx{XfAHODUo9Di8UT(kVN`=Ruye2@+hvz7P-Jb1w3d5{XngqzNzL_!gOi*3tF& zsUoLXa_V%qwX(>4xQd(=RfJAmolpsIf*2nH5G&ARL!$cJ`%1?lkuDvXp72520}o=Y ze+$>i1G!6-v}Vk2x%f7PejLEd#mvn!@OU6XHHY3R0Wi?I0B0UPYpHi}w_zTaCHC|m zK(8QM9{kwfBfYh+eF7(GP^u$G0pN;zEiF0(^xp{W;qMOrx(CQI1?mc6cP z{`=wWw+2^nR4zY0G5C22{f{{uPM1$uUOp*fF8z z;y?0;z_}=jQ#&~Sng#roxc_H&`!j_HK0NTj3|Kz1@Y`tuSV_O?uVkb6=5>~tp7UgL55J=h>CU2uJz zLc(;ZEsRXOOQVIX(r>xQY=SCLw*NDd{rw0CT9nA#?PXiBJn`Mth#G_gl%G98Jit*` z6WtL^Erq(u(?(4|C1B1OYzlvTWj56bJc16r0Y6P;*xZdBbUco@2_P-gSXvPTfjSRI zykJb)x-k$BklPr8he@4S4110&9Nd6hOSBOJvEp)`J#D>wdr!%}0Ur2&AUqh~+8hJ!7n;I?~kkQhd#HToxqolyfoDOKeY``-aoRYQCmuf1qi6yl#cxz5Nruy(v-M8frR znDCh7_u}7$qY)CA(_ogiNH1e&b#$cTN5ND~O@0BA?0{j{Vy`g1PuH(gQvRbJlge); zwgSA~#j{+o9`4NhdV5f96U^{tslMH`C_lDXXhqX=OFPb7UuKngD4}5^J74}R`Eo%2 zZt6ty2PT)xOYX}^sKVJ@t@I)}f4M~orn^ls^;uSEp;OK6S{)MaTRT5qLRub9_nmI8 z2inszh&p=x-V{k-lp>ET$2R^lX#i$~41rkBz#6CR<37j{{an>sS$v+@2M9-3$K4KW zo3+k%t`16#{y4Q%D(m}N&Y*ZyWwXND)=Q`dA*{eNgCfF@I6Y;*v zM;)S1szVM=P$uaHQ94x;jy8f9F9S47Z zos{8d&XGU?QO=yCg80V7Nn_9qvCQ5h-p7neVR?YmW|7^HZKovd;3oa$r1#6w%f5S_ zGXyefgPKHw6|Py!@|HUn40pz=NjJj$SO?CHP9@XxUtEw!n}8}$D|-lD1sUKQ9ElJL z-BphPS(eHLtBB*%9L$}N%NFG+-HhmC82Y+jaZ9d@HH7pOyt0`kRspx`{7wH%oNzey z-}BG>E#uab#=h2sC$AT*(M!uhPM)KtM)X04QSf`{3ehvaRC~7Qkp03DDvw)+j)XgW zdHO5Rn0ak=+!GoiJV~Pxlj{P&TTxzEZRW++xf5&Qowv8R()Q!GG?3+r%GSEF3V9)h ze!WH8kGnL^%$lSC1VP>*uAX!K3CE;nzJ+b!T@t~UGehou>6cVk>j7JhmPlkT5>1@P zE@X8A*NSaQicH51tK66jEnicIQknvjAGlwI6x8xT!qcs^vP)teuL6tnIHE;Y6F{zg z=*3SXmYD6fMT)J{Izg7UD$IA2S94?KiRZG1$yzLD**)1{&LkaaX6H{#eFg|~>2GGJ zP{*n~fdx92e3&sayN_dK$eK_B#Tv_$q|b)E(fba@pzrD`WCfl1XD7_+CrXOt6n4YR z?UwD*J~-lo$O#&PMC-1}Kj&Kkm4~nfmGu-TXt?JCmJFh!FV(6kEb5zDOlg^=Y%h#V z;EzslTq9*G>n3eDG32FXNJyfRGUI@*^lCUCd?ewIv1PrB=Uo+S3A+pAA{zIM%Al!I zQAKb=6Rca5zj6W+nZ^>_q?kE^o#th0p6gaAcm9Th*I7O!ge0GcI=Yt(^hCId@rFRy zKS8#|OUgRl?$r|^amhiuC$W3JRYy3ygxxwCQ?IV{$Mr9extnVsi|_>D%)>{mjC^v+ zTRdRNGbCz~1_?r)s^V`5D-%)l=K&>VO$Rx#4t7*B0yWW15%aG@{NxQ5T%}ZfOG~~v z^@=kUv_0hlJ(&*n36beTUOk+fEX7RPMQzuzTI_anW7CWh#VW%S$)bZwN?{DktXF<8 z{I>hJ_)BPl;A&{X9YZx78dEJxa@|v~tvmxm~y`9|5$mLOCl;c>;H z)cR~YhwcqN>NZuo zI{Fq@!!?Pi>*H^SoT1vRG0(YeP~vzgBkGoL)$&ENz5|L|OZybE#w~VKyWI`pT2r({ zkW-a6GmI$nspewXbG_A7Veg51zz8%~feGYCs^cyIH?%LFBqfr5Bi%IMi#eI&e719p6S6cr@t&Xgbv-b~EoV_N!Oj#<1r+Q;4wGY{3IGrmk!s|{Q{SLWKSGfva7 zM<5Mp&ArHg8?03CMnJMVGKo|i`keOV>bV$HM&!H5E;-K=10RZwpfK zF41SrGqY2B$0xjI6 zJy6hY(zmHjMK^)!2|2=AS{_tqhgt!f9>Hzx#Yb9NRtbF88hin3YYl}m1VtnVMeNx2 zsHZ1GZ@yYG3kYKeJLF!(dx@%pSF5gH23xd|K00PqqT#(|(=7=%7imcR%6&NsqeV&CciBDDn0E8XFHJzn)RwS|3| z3~;!kZEXyZXOSyQo}IG-3Ms>3SUi)myh^Ua_wIRt(n}n=xuzKGvaHS^V{<8dn@we1 z-29T8${`V}CZy_LjHv2Uq0NhdoMZ|;dq-fg1v0j35|_H!b*v!FcX>qpP|;oorK+pp z(3~x;-Gx%YDX3|HEm-%#%fjbK=LP_^e8ag7(ow;8cTAI<+Fdnu)m7kUCl|!qRPgNT zly1LNu`s$-ham}$^M-)rlZS$=Qh$-g}R@KqyR*XqVySjEuH zbR@ex?25$iF!Cy|k=>+<;tyDg7{VqD>T4S&sK<(`cVB2(&S!>v<)}$uRK#PWGCWFOA?F>sm9oq28oeeq?;y4TcQWOg!T<2pAsR5*;_jN0%~}f$~ZP9<*y={Xj&KB zA98<%1{h0?KQj}E?J!P2&>F^QUe{C;1G$2(4^Nil1iFeZRBf zHQTYW%csNENrlPK(8x7E|KjUg4hLyoBb%B3xDTrBKP*pG%BI0)ds+LI?t^O~JZvP; zt%wD@U`%c2M@rVHiRa)$Lc7g3rLOaP>eXr`@WN{UGZ6e1NJA8I;c_Pe$27Fk-i2Ag zX@$SORa*Pv>~1&nPv0x?>jZ+rX`~H;v!YX23#80ZTgP&&4lVNdWnM)7F!|c^J75yz z{e1V^JqiSzZ;Hba9oB6HU2_vqQi_`LRk^h|!u6nAqH~@rLdb9mFR@n<4 zHXiiP9G021hjTs!e?md<`LD%)pP0i7@F_O&yw>hGPAPWP#W7yXaO0W3&w@>{Y5geJ zSzQ>OW1jur*+jon$^Z5)^z;0Ft+4)Qhcw|;&L>cj zuB#2C<3_*V$-~1r&p1N6zp0Zr^|K^^sI9-pbw`)w2l91wMxiGR)csC*+|Hvv9AQ8E z_h0A}f2f7+<=4DzWkuo-X0R2k{x~M-GG%9Bw_B|qJCaQ~m3ESV)4sX4+dF zKN&9&bpYXpj=Xz>nT^2(!2~QV1>z&jjp}(ke($NC)%kgcc(uW?!MRXCNgjmKNHs&$ zIvKSVae#(UfvQK}8+K!vZvhRD3rxr!9GHT9onIXM7|0nt0hYTxr#xGJTZJ&MEaA`) z6mi3rZN;vsw|yvKhpV2jW~LduN!CW6cJP!ADI=q-+M95bm|$Smir7asFmmW9BY5pi z+QJhAN{`g%=7$ME;Ho4+4PCt%YZjSD=Ohumlz5UJqzrF7!kl0}+;l}`{5@~UC$0DQ zicYCrt4`Q;?}d-TbRP9fB9!~M_44FdH+e0CrBs6ys~}He)*Wgd5cu`AnH1EM$a)|p z0^rGl9_oi9fhV%(Y1X9|E^YO8;G#pF%Eo$Nn8HVA! zJw)*Bj7qwHedDo@;IRmq7s0UWxpw2hhMyCYR#y)*coLoHF~eX;F4d;Ie75UICv+j* znqyaix5PO?4f79|R&?aTwJT;d*{X z4U`Z;AsD70cQvqTooR(0+~y8n3(woK4Bc&f3QuY@=FPQg;XPOgetX6jm`@7C&9hWi zUK~LEY@hz_JHXe-hmV-yvFf3XZzfZv{`Q-$TIEmRzAu*pTt`Ww^XF3ldxvgw;jUKH z>@c^rd5kFkna>#`~7)b4H|j zx9l1pdwzQAnw%@&#upZ8rHX@kHHfqEq+I%~(R-V%*DVT8AMO(wsPsWPxJ!F(ggZh`bF zWZhPX@Z@Q=NWc;MZZl6iEixQRnRAF;i4)Ysp9gyQ+4b9!We5^H zxc>1W4@Ee@QdtMzeIvwCHXMDY>!IcN4CpZ~v>&CqQS!{b4=wB-qf3e8coeoYM?WW= z3mrKh4#0G{FJezX zn5L0DAN9<{T7APvjS49|>%o2ND7LU8c)-3*NavV+=+OxKd{0HyJ7<_#^CH8Hx*9%RKB z6nq?(h#4!HhNAr|{<<%Po{pLSGA%ZX=ZaNu#72pTybJ21qV&rRQ@WQz;G6FX=MyN- z1gvM^?ioJpCGOf=e{o}~E=zwM#h}-7HKkG!cs2mz54HRtVBa$v7nH|=U=-2;5io~i zA&$Q#Mr4%hSRWH}gs++??z4(GM>_Z_P6%kHTWZuK02_*%G|u1aL3Q?CivVu2%mr{G zbq*5&(2RP9{r6f=FJv{RIKEUh@;S#wNV@YUS>{jSYEXHwAQx7G3DOg~2w;up3gT@>su z49Ggy$^DCEGA~9tuTD2+IOZix>z=POf=POGg_|0E<^+2r0Jh3Hu#4kPtODj_PK%bW z<6bEWy+?!fJ?%7Hk46ZG9dyZll_>IGVMwzR;xo(vy?t~@z1-5CqxUMora3z)KRt{FSqIGhw>PyH5lhhly5c8-V=k+Qf(Q0%EA^>N6 zgf+f|_b1;0gcf>4UpeaK8nP1NBVDZUgPHIiVLgxX#ztRB%?dMX5DsoVLWJ*Led9ES z=YPgGvcg5TfQJ(+#Y_N#)hprm9Kx8JcCK)MCiF|vhi#(n6$YXmx z(JL{&k|8d?yRz&^BT0c%DfyD|dyjsB$a9gLh8b_)4fBE%YcyJ)b$i8gcN3<7+Fo zYaff!h{>qAS}6ib-C;NRa4-A-qNLxoygc>4Tm*&swmnI z?+$tdZgKnFw?0hGB-nM=EvmUAs;B~flqXAkNxp>MwB+HxlwqK$cHy zVWePA7x@z!_Zyi=mM<$Cmv415d8i1sW`WM%u%>(LhUw_D{6wf7sU6dC$PeNZ91IIR zxoRegp$s1#ZzUx{l#gE$MWyp@im}4!CAp#xf?%Qd$64&?%X^j2)OmzMDF&xItRL4c&ro^1lPjkST}2$z5$j0I<_728^XEpNK` z&vhdCO`T555Pk3gsD_f0^NmQ(Z14a%z+sA~tm|7^*SAAS*Ls_s4k#7=EfyMdwQ;9? zD6Xg7@>q9irQ4~5HU8bTT1{mO-F5^iIceyec_Q|vJ(w2v)FPS=usm=)$NrpLAI0|- zHKM!Nk{HL=6SzEY=Lb^cJmxI?%&tcy%VAI1J9wXA#M&VYnB~fS_tF*zPR`+i%l9Oh zz+hy5I1*s&E2@uG3#FG-8}X{}?4t&qVP>@pvra{Q-uq@F9~HE3Yu?~HtpdgK^(@a4 zo5N4YY12CS!Wsn2AG@&jGNqgt-~r~`Oh*L#0||%R_a-ql5n?A`f$D~VN>X*{*v89i zyFH(ml1M_6Fg$2zM9OEy48te*jU>be0x7rzj~D8j8M+$GeM<#^hQ%XipoEF_2_`B7 z2Pe|cuU~pE@K@B4VY!<=QK!a3p!!Ue4RAg4HM>K!MX}P_1B1OQ%*q=QMFXniWgeMg zdj|=WxSOb7dlO7sa=+y~JVs5DG6aFlw%N{d!!^l1ZD-G6&GXP!yGJAVA|v7`S&9G* zFcCz^RhL@vTwOE|srgC@Vf_5x0S|Sbhd{<2EU3qj%h|2k_fot_Rbb`W@$Mpjc<8n9 zj`RALV|t4&&#=q1dUKxDv-=S#EcFUImIy=7N~JP<1X|S(nUhw$FC;jciM^}z!WE!r zdnl?6V?6dF>)vlxQ2?j3A(8TjG$~6%!rgq##oV!Z#kZ5)+2tyFyEOcnN4M7BaEG~21F-4f9$H;zZQYkRXer_gyGdGtmb z)q;1)98xWW?QP}Vft_Y^O)=9lYIo*M<$QB_w7-}tbS~$-qvtOHa7zm-VPq7rVNU2u zASaw8ZdbNtei9+eU)Nba6DaH@BiP=FX;z-5HM1Ov0@A9kE=e&5mk3=J?+&t7d6;6b z%9)DCrg|iwmztPr5Sk*pH)z-pJFrv-T(KB(#V2(WxSdE?xy_hdL-yfVlpqyjk& zV0Sv|i!{$z=#Ud;3toz;-y9CR>9G%eQ{+cKk&XOP33!#vJFohJG!BO1qf6Ui+FYNE zb%l1zui(kN@+9aA(-~m3OBuQ@FK}G@d%YL@bGsaaQ>`@C+4H5f~utj zSU%N!DoohKwaz)0s5_$?k;Y*h*_Jjq^~5Qkj8@^0qY!M?x4?!z1$;wfR6b~zRz*N~ zorP)7YdGW~h>XZ8DGHyf$EtDN4uJ|$7m?R?+OHbIZ8B@5)929fqT>jSP-)k#O&s$u z*?O%|L)u2`Kdx*kRR*Wrv{3i|_A z%`Dox`o`~oMS@MJ%rrTSVJyf^ylH{deX@Uf@5219S37M(@HNDm+4#_F(U&S1ekblJ>isBB6DQ7q z2=rmZy2-Nlma|kJUe<`a|&$5o7J%=O2QdI?wVTi?PADrfs zfv*oBP&xq`lUk3d+WN*+7)_I}p&91to|@{>XI=xIrIV88I88$zsyP+-)Z0Ak>d~{n za5s4fJVVl*D;{KG%Ccvg5Yh^m9>#vPR5E($Ae&)%puKSOB0%L}HYr@vHU5quLgF5` z6F+`bCM`WbremoAeQ|gjDQVcBE=LkqnV1_7`RJ&as9VTG(fyyz4kY8`Z$593I7=yZ zEnHrkOdg%<#g23SJas{>7Od%k8^bVZ(L;ySBcN^UmZx|~PeQjE4e?@rO%=>t*;Ag$ zgR$tNz`sJ^K5*6BMb3P%F>|BsP%;8$(0K~g>SMxlbJLJaqq8I2KV5EYekZz2ikyqo#U$|n0? z%(zq+R5?7kzE>{|5i3eB~AsaBBJrHY#fakl;)kkj+)=Vy=M+v&d#o0?H9g9GRJUj2BohHto|{_DM5 z{+3H*o+i_mp7MAz1og@`BSFD}wf{4&_qPw%2>vQo9R$z7l=u#y_w2$Awg-7^->*(sQUhT%#nr2oF7%S zpUDGm;kR!XR3%$mDknL^iMK7C?p1^{GiANWkil)g#AZc9kscu$Qq35f^1k4Drmkqa zBb&lW(C_M?0x7ux7HV(rkRtNCQinje7oK417rNN+3JNlOo^+B0Qz*uqeQS+>N2jJc zI4F%EvL{2SHjorn|0--elFdG(r|j@q_1eT)C3JpTM;s@0bV2eV+Xjj+av0K7VxjE{ zKSnUqG<31k+#bfmV&-qn{t%9sj85(H!4ftXYTu=r+uST7S$y;ws;{G!U3@wgWdP?8{rKdx8;(y3_|r?2Xu9ro zXVkH6HB`kyW0;E}v*+vU3km~_XsAL73Lvw7&h z?Pq`fT?c)BM=LUjr3q+TD32DL6A$ToN+_qyMxtX(U8XuL$4R2=^duqm3|1dr+@3c?e;nxwL4=B#Ro19Gyc z!^8XJM+XX<2in3cFdrh?5VMdcQQo84CnqWAdM??6@ik(q<%FMP^+@q_l zTkmM*0$m(Y_^OEuI_k{oCXWrAur**8ZOl>*`0{F@ja*O$vD8&-9W;V&$<2+&5&OlG zq;GffV;=PcA6V*Bd`@z`aW zK$8X&+*9_&iz3PPe|yT%dC8(DV}J3!?l@*wWuHi&6c>0xX3yC%j*K_s;6obpb_3WX4To zSn4(Xfm{*YS8J!`9MluwF980$czwXLwidp$d7ka))P0-cA4 zmv>G?01trxqy(QY!{HB4pHwxqHfnZNNSHrxAWjVkro>!YGN&C&W5?F89@VsWt(4Uz z$C4qojE5=G&{2hb(vyo)H8&&V9g$PhxMM9(g&jdC{2C33M9b&{{)g?fHQ0ii z>!Jh0fgx+aUg+ybV}d!^WYtrT&ateW<)-^2+WU>a+WAi>HXFkVG0MF+x*ZJ8?o-#n zEu>+2qEG6maoe*;$d?DMNj_+xq=j1Vm4J<6MwBXLD?=?y(x+odO#t@N)b>aSiKW63 zj88m9$s%&!#IrZKsEFFu8_rI|rlwrLO;?$zPX>llL)L6=l<5vNg3T>4k2UpXn|h>I zka_r>%cwU1T#6G;B?XC=->^8~a~QiqZD;EdQqb+C@QJZTw&|}F_@&I+mE#e8a3HWa zT#fekL2>^$B#t56Z_gXUDsfK5GT~fVZ!YNzu-Z2ZdoY9ujlyxwULyLLJYwbkFF5&n=?OM_8)vWdb1x>hns*|O0 z{^EClrDdYiaK)8;PYUD9?Zb+X>xKdMHSbX&Z{(d9IA6IytFEz_b%nde7VZVp*3{KS zGi5*}wcSwA!%Ps85A`x7EKh@`N2ZxKh)f;amcQKE%v2@Kj=H#?JCDs$nt->3q^xh= z;fAG7#x2Xr1{VvU*~jmF?Zldio>Z7V$7EKtJ}b#N?%)EtxNxJn?d4`0#F-sR)Rn%;aOZYqeV($zEl#M1w4pJg#={W-m6pF=&>!0EXVaj(=$3b#js1_ z1qto=$bO?BJR^XRzMgA|nc$;Cv4YJ-`Dmo{PQ?z<_h$VjO1oa=krZ$Fdl{fAAZ z?t=J}AWQJ$)2raImf9&ezmF=k=^!E(Nndw*Cn8W=-RGq>M7;bV0)A{1TK`IYp&k)8 z6vwY>vNh!B$f%sskHmo5S6aC$EaGu;?2wb1TEnnxytrD7ixg7nY)=biOC6W{Zon+P z8+U`krN$0Mn-d_9@-T3Bpb%8->s?FpKDr^mNKlg1zbGH%PPCZ z{0;zF&NM|b?o;i0RMwWxZ)|D1?v(kC6nZV~AY(7Dtu^_8hB^@7z&~Ezz1<^QdoB+BBahSdM(xuwO z&dAH(8mfB}Wi{{O%Q!w-#; zKq(1}$LET{{0AL` z43(27+PY?hD$6+$($sjJ9C-LKT+Q>$uP-FV3e+zO5keWeH49?caAVnYe=W#l(la!|jKN?+W*N(+8c8_)}N ztTPUjBm_D+gBu&n({Dgc8TsvkXsQ;+Ng$317RS8!-mbABf-VQShIdEQT;06$@yNWW z>119Kpe>EZ$`+{%;k((ux|Sx_51;Qzm?B7@qETzCVzbu;$}ae zn(PecuBj0I$lt5sT#q?LEEm-dH_GxEsznUrcxPLT$`lr2@$>&46vFOSEO_!?4lt4T zD{IUjKym!WhX=Hi;PlN%`I^dJsPTK;2z{jt;h%JgpB#>#a1cPqPmBrx_RqpWV58L{ zLAQvCO^ZFLxwi%0zczyHT9m%x%qGk$2}+}9i?#RqJ=RU(&cnxLzQ4yJ3J&_$J|^XJ zCud76>5Hz>zLHA~s>aya9W4%JO+op1-+#Y_`uS4{;b#qg*5GF!_}K@3_JN;$;AbEB LztRU#zt8+Ht+S9Y literal 0 HcmV?d00001 From 81f5f4d1d5a06ce6fccd95c78268d7ac7a1d2a02 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 31 Jan 2025 13:00:18 +0000 Subject: [PATCH 03/22] WIP: Introduce Ra worker process - first draft Responsibilities: * Writing snapshots * Writing and promoting checkpoints * Compaction --- src/ra.hrl | 2 + src/ra_bench.erl | 1 + src/ra_log.erl | 34 ++++--- src/ra_server.erl | 43 +++++---- src/ra_server_proc.erl | 40 +++++--- src/ra_server_sup.erl | 32 +++++-- src/ra_snapshot.erl | 101 ++++++++++++-------- src/ra_worker.erl | 83 +++++++++++++++++ test/ra_SUITE.erl | 1 - test/ra_checkpoint_SUITE.erl | 55 +++++++---- test/ra_log_2_SUITE.erl | 82 +++++++++------- test/ra_machine_int_SUITE.erl | 1 + test/ra_server_SUITE.erl | 2 +- test/ra_snapshot_SUITE.erl | 170 +++++++++++++++++++--------------- 14 files changed, 420 insertions(+), 227 deletions(-) create mode 100644 src/ra_worker.erl diff --git a/src/ra.hrl b/src/ra.hrl index 539f1931..89b6948d 100644 --- a/src/ra.hrl +++ b/src/ra.hrl @@ -94,6 +94,8 @@ %% A member of the cluster from which replies should be sent. -type ra_reply_from() :: leader | local | {member, ra_server_id()}. +-type mfargs() :: {M :: module(), F :: atom(), A :: [term()]}. + -define(RA_PROTO_VERSION, 1). %% the protocol version should be incremented whenever extensions need to be %% done to the core protocol records (below). It is only ever exchanged by the diff --git a/src/ra_bench.erl b/src/ra_bench.erl index 56fff922..85ca9e95 100644 --- a/src/ra_bench.erl +++ b/src/ra_bench.erl @@ -24,6 +24,7 @@ % profile/0, % stop_profile/0 + start/2, prepare/0, run/3, diff --git a/src/ra_log.erl b/src/ra_log.erl index 53129fe7..7be55406 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -777,6 +777,12 @@ 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) -> % resend missing entries from mem tables. @@ -894,7 +900,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, []}; @@ -1089,24 +1095,24 @@ 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, + % case ra_snapshot:pending(SnapState) of + % {Pid, _, _} -> + % case is_process_alive(Pid) of + % true -> + % exit(Pid, kill), + % ok; + % false -> + % ok + % end; + % _ -> + % ok + % end, try ra_lib:recursive_delete(Dir) of ok -> ok catch diff --git a/src/ra_server.erl b/src/ra_server.erl index 5d820d5b..deea5997 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -178,7 +178,8 @@ {notify, #{pid() => [term()]}} | %% used for tracking valid leader messages {record_leader_msg, ra_server_id()} | - start_election_timeout. + start_election_timeout | + {bg_work, fun(() -> ok) | mfargs()}. -type effects() :: [effect()]. @@ -232,7 +233,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(). @@ -1543,8 +1545,8 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, ?DEBUG("~ts: receiving snapshot chunk: ~b / ~w, index ~b, term ~b", [LogId, Num, ChunkFlag, SnapIndex, SnapTerm]), SnapState0 = ra_log:snapshot_state(Log0), - {ok, SnapState} = ra_snapshot:accept_chunk(Data, Num, ChunkFlag, - SnapState0), + {ok, SnapState, Effs0} = ra_snapshot:accept_chunk(Data, Num, ChunkFlag, + SnapState0), Reply = #install_snapshot_result{term = CurTerm, last_term = SnapTerm, last_index = SnapIndex}, @@ -1598,11 +1600,12 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, %% it was the last snapshot chunk so we can revert back to %% follower status {follower, persist_last_applied(State), [{reply, Reply} | - Effs ++ SnapInstalledEffs]}; + Effs0 ++ Effs ++ + SnapInstalledEffs]}; next -> Log = ra_log:set_snapshot_state(SnapState, Log0), State = update_term(Term, State0#{log => Log}), - {receive_snapshot, State, [{reply, Reply}]} + {receive_snapshot, State, [{reply, Reply} | Effs0]} end; handle_receive_snapshot(#append_entries_rpc{term = Term} = Msg, #{current_term := CurTerm, @@ -2308,20 +2311,20 @@ handle_down(RaftState, snapshot_sender, Pid, Info, "~ts: Snapshot sender process ~w exited with ~W", [LogId, Pid, Info, 10]), {leader, peer_snapshot_process_exited(Pid, State), []}; -handle_down(RaftState, snapshot_writer, Pid, Info, - #{cfg := #cfg{log_id = LogId}, log := Log0} = State) - when is_pid(Pid) -> - case Info of - noproc -> ok; - normal -> ok; - _ -> - ?WARN("~ts: Snapshot write process ~w exited with ~w", - [LogId, Pid, Info]) - end, - SnapState0 = ra_log:snapshot_state(Log0), - SnapState = ra_snapshot:handle_down(Pid, Info, SnapState0), - Log = ra_log:set_snapshot_state(SnapState, Log0), - {RaftState, State#{log => Log}, []}; +% handle_down(RaftState, snapshot_writer, Pid, Info, +% #{cfg := #cfg{log_id = LogId}, log := Log0} = State) +% when is_pid(Pid) -> +% case Info of +% noproc -> ok; +% normal -> ok; +% _ -> +% ?WARN("~ts: Snapshot write process ~w exited with ~w", +% [LogId, Pid, Info]) +% end, +% SnapState0 = ra_log:snapshot_state(Log0), +% SnapState = ra_snapshot:handle_error(Pid, Info, SnapState0), +% Log = ra_log:set_snapshot_state(SnapState, Log0), +% {RaftState, State#{log => Log}, []}; handle_down(RaftState, log, Pid, Info, #{log := Log0} = State) -> {Log, Effects} = ra_log:handle_event({down, Pid, Info}, Log0), {RaftState, State#{log => Log}, Effects}; diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index 12eefe74..c27f5038 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -146,7 +146,8 @@ receive_snapshot_timeout = ?DEFAULT_RECEIVE_SNAPSHOT_TIMEOUT :: non_neg_integer(), install_snap_rpc_timeout :: non_neg_integer(), aten_poll_interval = 1000 :: non_neg_integer(), - counter :: undefined | counters:counters_ref() + counter :: undefined | counters:counters_ref(), + worker_pid :: pid() }). -record(state, {conf :: #conf{}, @@ -301,18 +302,14 @@ multi_statem_call([ServerId | ServerIds], Msg, Errs, Timeout) -> %%%=================================================================== init(#{reply_to := ReplyTo} = Config) -> - %% we have a reply to key, perform init async {ok, post_init, maps:remove(reply_to, Config), - [{next_event, internal, {go, ReplyTo}}]}; -init(Config) -> - %% no reply_to key, must have been started by an older node run synchronous - %% init - State = do_init(Config), - {ok, recover, State, [{next_event, cast, go}]}. + [{next_event, internal, {go, ReplyTo}}]}. do_init(#{id := Id, uid := UId, - cluster_name := ClusterName} = Config0) -> + parent := ParentPid, + cluster_name := ClusterName} = Config0) + when is_pid(ParentPid) -> Key = ra_lib:ra_server_id_to_local_name(Id), true = ets:insert(ra_state, {Key, init, unknown}), process_flag(trap_exit, true), @@ -361,6 +358,16 @@ do_init(#{id := Id, ?DEFAULT_RECEIVE_SNAPSHOT_TIMEOUT), AtenPollInt = application:get_env(aten, poll_interval, 1000), LogId = ra_server:log_id(ServerState), + %% TODO: full error handling + WorkerPid = case ra_server_sup:start_ra_worker(ParentPid, Config) of + {ok, P} -> P; + {error, {already_started, P}} -> + P + end, + ra_env:configure_logger(logger), + %% monitor worker process, it is easier to handle than linking as we're + %% already processing all downs + _ = monitor(process, WorkerPid), State = #state{conf = #conf{log_id = LogId, cluster_name = ClusterName, name = Key, @@ -372,7 +379,8 @@ do_init(#{id := Id, install_snap_rpc_timeout = InstallSnapRpcTimeout, receive_snapshot_timeout = ReceiveSnapshotTimeout, aten_poll_interval = AtenPollInt, - counter = Counter}, + counter = Counter, + worker_pid = WorkerPid}, low_priority_commands = ra_ets_queue:new(), server_state = ServerState}, ok = net_kernel:monitor_nodes(true, [nodedown_reason]), @@ -1550,7 +1558,7 @@ handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, Id, Term}}, SS = ra_server:update_peer(To, #{status => disconnected}, SS0), {State0#state{server_state = SS}, Actions} end; -handle_effect(_, {delete_snapshot, Dir, SnapshotRef}, _, State0, Actions) -> +handle_effect(_, {delete_snapshot, Dir, SnapshotRef}, _, State0, Actions) -> %% delete snapshots in separate process _ = spawn(fun() -> ra_snapshot:delete(Dir, SnapshotRef) @@ -1641,6 +1649,11 @@ handle_effect(follower, {record_leader_msg, _LeaderId}, _, State0, Actions) -> handle_effect(_, {record_leader_msg, _LeaderId}, _, State0, Actions) -> %% non follower states don't need to reset state timeout after an effect {State0, Actions}; +handle_effect(_, {bg_work, FunOrMfa, ErrFun}, _, + #state{conf = #conf{worker_pid = WorkerPid}} = State0, Actions) -> + %% non follower states don't need to reset state timeout after an effect + ra_worker:queue_work(WorkerPid, FunOrMfa, ErrFun), + {State0, Actions}; handle_effect(_, _, _, State0, Actions) -> {State0, Actions}. @@ -2055,6 +2068,11 @@ handle_node_status_change(Node, Status, InfoList, RaftState, monitors = Monitors}), {keep_state, State, Actions}. +handle_process_down(Pid, Info, _RaftState, + #state{conf = #conf{worker_pid = Pid}} = State) -> + ?WARN("~ts: worker exited with ~w", + [log_id(State), Info]), + {stop, Info, State}; handle_process_down(Pid, Info, RaftState, #state{monitors = Monitors0, pending_notifys = Nots, diff --git a/src/ra_server_sup.erl b/src/ra_server_sup.erl index 3857996c..11c39516 100644 --- a/src/ra_server_sup.erl +++ b/src/ra_server_sup.erl @@ -11,6 +11,7 @@ %% API functions -export([start_link/1]). +-export([start_ra_worker/2]). %% Supervisor callbacks -export([init/1]). @@ -20,7 +21,18 @@ %%%=================================================================== start_link(Config) -> - supervisor:start_link(?MODULE, [Config]). + supervisor:start_link(?MODULE, Config). + +-spec start_ra_worker(pid(), ra_server:config()) -> + supervisor:startchild_ret(). +start_ra_worker(SupPid, Config) + when is_pid(SupPid) andalso + is_map(Config) -> + RaWorker = #{id => ra_worker, + type => worker, + restart => transient, + start => {ra_worker, start_link, [Config]}}, + supervisor:start_child(SupPid, RaWorker). %%%=================================================================== %%% Supervisor callbacks @@ -28,20 +40,20 @@ start_link(Config) -> %%-------------------------------------------------------------------- -init([Config0]) -> +init(Config0) -> Id = maps:get(id, Config0), Config = Config0#{parent => self()}, Name = ra_lib:ra_server_id_to_local_name(Id), - SupFlags = #{strategy => one_for_one, + SupFlags = #{strategy => one_for_all, intensity => 2, period => 5}, - ChildSpec = #{id => Name, - type => worker, - % needs to be transient as may shut itself down by returning - % {stop, normal, State} - restart => transient, - start => {ra_server_proc, start_link, [Config]}}, - {ok, {SupFlags, [ChildSpec]}}. + RaServer = #{id => Name, + type => worker, + % needs to be transient as may shut itself down by returning + % {stop, normal, State} + restart => transient, + start => {ra_server_proc, start_link, [Config]}}, + {ok, {SupFlags, [RaServer]}}. %%%=================================================================== %%% Internal functions diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index be6ce2af..c734188b 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -38,7 +38,7 @@ context/2, - handle_down/3, + handle_error/3, current_snapshot_dir/1, latest_checkpoint/1, @@ -47,7 +47,8 @@ take_extra_checkpoints/1 ]). --type effect() :: {monitor, process, snapshot_writer, pid()}. +-type effect() :: {monitor, process, snapshot_writer, pid()} | + {bg_work, fun(), fun()}. -type kind() :: snapshot | checkpoint. @@ -79,7 +80,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()), @@ -329,7 +330,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. @@ -371,13 +372,14 @@ 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 + %% call prepare then write_snapshotS %% This needs to be called in the current process to "lock" potentially %% mutable machine state Ref = Mod:prepare(Meta, MacRef), %% write the snapshot in a separate process Self = self(), - Pid = spawn(fun () -> + IdxTerm = {Idx, Term}, + BgWorkFun = fun () -> ok = ra_lib:make_dir(SnapDir), case Mod:write(SnapDir, Meta, Ref, Sync) of ok -> ok; @@ -387,14 +389,14 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind, ok end, Self ! {ra_log_event, - {snapshot_written, {Idx, Term}, SnapKind}}, + {snapshot_written, IdxTerm, 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 +413,21 @@ promote_checkpoint(PromotionIdx, Checkpoint = make_snapshot_dir(CheckpointDir, Idx, Term), Snapshot = make_snapshot_dir(SnapDir, Idx, Term), Self = self(), - Pid = spawn(fun() -> - %% Checkpoints are created without calling - %% fsync. Snapshots must be fsync'd though, so - %% sync the checkpoint before promoting it - %% into a snapshot. - ok = Mod:sync(Checkpoint), - ok = prim_file:rename(Checkpoint, Snapshot), - Self ! {ra_log_event, - {snapshot_written, - {Idx, Term}, snapshot}} - end), - State = State0#?MODULE{pending = {Pid, {Idx, Term}, snapshot}, + Fun = fun() -> + %% Checkpoints are created without calling + %% fsync. Snapshots must be fsync'd though, so + %% sync the checkpoint before promoting it + %% into a snapshot. + ok = Mod:sync(Checkpoint), + ok = prim_file:rename(Checkpoint, Snapshot), + Self ! {ra_log_event, + {snapshot_written, + {Idx, Term}, 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. @@ -450,6 +453,11 @@ find_promotable_checkpoint(_Idx, [], _Acc) -> -spec complete_snapshot(ra_idxterm(), kind(), state()) -> state(). +complete_snapshot(_IdxTerm, snapshot, + #?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, #?MODULE{uid = UId} = State) -> true = ets:insert(?ETSTBL, {UId, Idx}), @@ -472,12 +480,13 @@ begin_accept(#{index := Idx, term := Term} = Meta, state = AcceptState}}}. -spec accept_chunk(term(), non_neg_integer(), chunk_flag(), state()) -> - {ok, state()}. + {ok, state(), [effect()]}. accept_chunk(Chunk, Num, last, #?MODULE{uid = UId, module = Mod, snapshot_directory = Dir, current = Current, + pending = Pending, accepting = #accept{next = Num, idxterm = {Idx, _} = IdxTerm, state = AccState}} = State) -> @@ -485,13 +494,22 @@ accept_chunk(Chunk, Num, last, ok = Mod:complete_accept(Chunk, AccState), %% run validate here? %% delete the current snapshot if any - _ = spawn(fun () -> delete(Dir, Current) end), + Dels = case Pending of + undefined -> + [Current]; + {PendIdxTerm, _} -> + [Current, PendIdxTerm] + end, + Eff = {bg_work, + fun() -> [delete(Dir, Del) || Del <- Dels] end, + fun (_) -> ok end}, + %% update ets table true = ets:insert(?ETSTBL, {UId, Idx}), {ok, State#?MODULE{accepting = undefined, %% reset any pending snapshot writes pending = undefined, - current = IdxTerm}}; + current = IdxTerm}, [Eff]}; accept_chunk(Chunk, Num, next, #?MODULE{module = Mod, accepting = @@ -499,12 +517,13 @@ accept_chunk(Chunk, Num, next, next = Num} = Accept} = State) -> {ok, AccState} = Mod:accept_chunk(Chunk, AccState0), {ok, State#?MODULE{accepting = Accept#accept{state = AccState, - next = Num + 1}}}; + next = Num + 1}}, + []}; accept_chunk(_Chunk, Num, _ChunkFlag, #?MODULE{accepting = #accept{next = Next}} = State) when Next > Num -> %% this must be a resend - we can just ignore it - {ok, State}. + {ok, State, []}. -spec abort_accept(state()) -> state(). abort_accept(#?MODULE{accepting = undefined} = State) -> @@ -527,27 +546,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; @@ -683,7 +697,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_worker.erl b/src/ra_worker.erl new file mode 100644 index 00000000..cddd2e03 --- /dev/null +++ b/src/ra_worker.erl @@ -0,0 +1,83 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2017-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. +%% +%% @hidden +-module(ra_worker). +-behaviour(gen_server). + +-export([start_link/1, + queue_work/3]). + +-export([init/1, + handle_call/3, + handle_cast/2, + handle_info/2, + terminate/2, + code_change/3]). + +-record(state, {}). + +%%% ra worker responsible for doing background work for a ra server. +%%% +%%% this could include, writing snapshots or checkpoints or log +%%% compaction + +%%%=================================================================== +%%% API functions +%%%=================================================================== + +start_link(Config) -> + gen_server:start_link(?MODULE, Config, []). + +queue_work(Pid, FunOrMfa, ErrFun) when is_pid(Pid) -> + gen_server:cast(Pid, {work, FunOrMfa, ErrFun}). + +%%%=================================================================== +%%% gen_server callbacks +%%%=================================================================== + +init(Config) when is_map(Config) -> + process_flag(trap_exit, true), + {ok, #state{}}. + +handle_call(_, _From, State) -> + {reply, ok, State}. + +handle_cast({work, FunOrMfa, ErrFun}, State) -> + case FunOrMfa of + {M, F, Args} -> + try erlang:apply(M, F, Args) of + _ -> + ok + catch Type:Err:_Stack -> + %% TODO: log + ErrFun({Type, Err}), + ok + end; + _ when is_function(FunOrMfa) -> + try FunOrMfa() of + _ -> + ok + catch Type:Err:_Stack -> + %% TODO: log + ErrFun({Type, Err}) + end, + ok + end, + {noreply, State}. + +handle_info(_Info, State) -> + {noreply, State}. + +terminate(_Reason, _State) -> + ok. + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%%=================================================================== +%%% Internal functions +%%%=================================================================== diff --git a/test/ra_SUITE.erl b/test/ra_SUITE.erl index fddf7321..c8b441f6 100644 --- a/test/ra_SUITE.erl +++ b/test/ra_SUITE.erl @@ -132,7 +132,6 @@ single_server_processes_command(Config) -> {_RaName, _} = N1 = nth_server_name(Config, 1), ok = ra:start_server(default, Name, N1, add_machine(), []), ok = ra:trigger_election(N1), - monitor(process, element(1, N1)), % index is 2 as leaders commit a no-op entry on becoming leaders {ok, 5, _} = ra:process_command(N1, 5, 2000), {ok, 10, _} = ra:process_command(N1, 5, 2000), diff --git a/test/ra_checkpoint_SUITE.erl b/test/ra_checkpoint_SUITE.erl index 8640d7f5..a4101426 100644 --- a/test/ra_checkpoint_SUITE.erl +++ b/test/ra_checkpoint_SUITE.erl @@ -83,10 +83,12 @@ take_checkpoint(Config) -> Meta = meta(55, 2, [node()]), MacRef = ?FUNCTION_NAME, - {State1, [{monitor, process, snapshot_writer, Pid}]} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(Meta, MacRef, 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), @@ -103,19 +105,19 @@ take_checkpoint_crash(Config) -> State0 = init_state(Config), Meta = meta(55, 2, [node()]), MacRef = ?FUNCTION_NAME, - {State1, [{monitor, process, snapshot_writer, Pid}]} = + {State1, [{bg_work, _Fun, ErrFun}]} = ra_snapshot:begin_snapshot(Meta, MacRef, 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,8 +133,9 @@ recover_from_checkpoint_only(Config) -> {error, no_current_snapshot} = ra_snapshot:recover(State0), Meta = meta(55, 2, [node()]), - {State1, [{monitor, process, snapshot_writer, _}]} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, checkpoint, State0), + Fun(), receive {ra_log_event, {snapshot_written, IdxTerm, checkpoint}} -> _ = ra_snapshot:complete_snapshot(IdxTerm, checkpoint, State1), @@ -157,8 +160,9 @@ recover_from_checkpoint_and_snapshot(Config) -> %% Snapshot. SnapMeta = meta(55, 2, [node()]), - {State1, [{monitor, process, snapshot_writer, _}]} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(SnapMeta, ?FUNCTION_NAME, snapshot, State0), + Fun(), State2 = receive {ra_log_event, {snapshot_written, IdxTerm1, snapshot}} -> ra_snapshot:complete_snapshot(IdxTerm1, snapshot, State1) @@ -168,8 +172,9 @@ recover_from_checkpoint_and_snapshot(Config) -> %% Checkpoint at a later index. CPMeta = meta(105, 3, [node()]), - {State3, [{monitor, process, snapshot_writer, _}]} = + {State3, [{bg_work, Fun2, _}]} = ra_snapshot:begin_snapshot(CPMeta, ?FUNCTION_NAME, checkpoint, State2), + Fun2(), receive {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} -> _ = ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3), @@ -195,8 +200,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Checkpoint at 25. CP1Meta = meta(25, 2, [node()]), - {State1, [{monitor, process, snapshot_writer, _}]} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(CP1Meta, ?FUNCTION_NAME, checkpoint, State0), + Fun(), State2 = receive {ra_log_event, {snapshot_written, IdxTerm1, checkpoint}} -> ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, State1) @@ -206,8 +212,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Checkpoint at 35. CP2Meta = meta(35, 3, [node()]), - {State3, [{monitor, process, snapshot_writer, _}]} = + {State3, [{bg_work, Fun2, _}]} = ra_snapshot:begin_snapshot(CP2Meta, ?FUNCTION_NAME, checkpoint, State2), + Fun2(), State4 = receive {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} -> ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3) @@ -217,8 +224,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Checkpoint at 55. CP3Meta = meta(55, 5, [node()]), - {State5, [{monitor, process, snapshot_writer, _}]} = + {State5, [{bg_work, Fun3, _}]} = ra_snapshot:begin_snapshot(CP3Meta, ?FUNCTION_NAME, checkpoint, State4), + Fun3(), State6 = receive {ra_log_event, {snapshot_written, IdxTerm3, checkpoint}} -> ra_snapshot:complete_snapshot(IdxTerm3, checkpoint, State5) @@ -228,8 +236,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Snapshot at 45. SnapMeta = meta(45, 4, [node()]), - {State7, [{monitor, process, snapshot_writer, _}]} = + {State7, [{bg_work, Fun4, _}]} = ra_snapshot:begin_snapshot(SnapMeta, ?FUNCTION_NAME, snapshot, State6), + Fun4(), State8 = receive {ra_log_event, {snapshot_written, IdxTerm4, snapshot}} -> ra_snapshot:complete_snapshot(IdxTerm4, snapshot, State7) @@ -262,7 +271,9 @@ 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, ?FUNCTION_NAME, checkpoint, State0), + Fun(), State2 = receive {ra_log_event, {snapshot_written, {55, 2} = IdxTerm1, checkpoint}} -> ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, State1) @@ -272,7 +283,9 @@ init_recover_corrupt(Config) -> %% 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, ?FUNCTION_NAME, checkpoint, State2), + Fun2(), receive {ra_log_event, {snapshot_written, {165, 2} = IdxTerm2, checkpoint}} -> _ = ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3), @@ -303,8 +316,9 @@ init_recover_multi_corrupt(Config) -> %% Checkpoint at 55. CP1Meta = meta(55, 2, [node()]), - {State1, _} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(CP1Meta, ?FUNCTION_NAME, checkpoint, State0), + Fun(), State2 = receive {ra_log_event, {snapshot_written, IdxTerm1, checkpoint}} -> ra_snapshot:complete_snapshot(IdxTerm1, checkpoint, State1) @@ -314,8 +328,9 @@ init_recover_multi_corrupt(Config) -> %% Checkpoint at 165. CP2Meta = meta(165, 2, [node()]), - {State3, _} = + {State3, [{bg_work, Fun2, _}]} = ra_snapshot:begin_snapshot(CP2Meta, ?FUNCTION_NAME, checkpoint, State2), + Fun2(), State4 = receive {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} -> ra_snapshot:complete_snapshot(IdxTerm2, checkpoint, State3) diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 6c5816c2..4044ef43 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -65,8 +65,6 @@ 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 ]. @@ -399,8 +397,9 @@ 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), + {Log2, Effs} = ra_log:update_release_cursor(SnapIdx, #{}, 2, + <<"snap@10">>, Log1), + run_effs(Effs), {Log3, _} = receive {ra_log_event, {snapshot_written, {10, 2}, snapshot} = Evt} -> @@ -534,8 +533,9 @@ 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, #{}, 1, + <<"one+two">>, Log1b), + run_effs(Effs), {Log3, _} = receive {ra_log_event, {snapshot_written, {2, 1}, snapshot} = Evt} -> @@ -549,9 +549,10 @@ written_event_after_snapshot(Config) -> 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, #{}, 1, + <<"one+two+three+four">>, + Log6b), + run_effs(Effs2), _ = receive {ra_log_event, {snapshot_written, {4, 1}, snapshot} = E} -> ra_log:handle_event(E, Log7) @@ -569,8 +570,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, #{}, 1, + <<"one+two">>, Log2), + run_effs(Effs), Log4 = deliver_all_log_events(Log3, 100), ra_log:close(Log4), restart_wal(), @@ -591,7 +593,8 @@ 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, #{}, 1, <<"100">>, Log2), + run_effs(Effs0), Log4 = deliver_all_log_events(Log3, 500), Overview = ra_log:overview(Log4), @@ -1059,8 +1062,9 @@ 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, #{}, 1, + <<"one-five">>, Log1), + run_effs(Effs), DelayedSnapWritten = receive {ra_log_event, {snapshot_written, {5, 1}, snapshot} = Evt} -> @@ -1075,7 +1079,8 @@ 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), + {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + run_effs(AEffs), {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2), %% write some more to create another segment Log4 = write_and_roll(16, 20, 2, Log3), @@ -1104,7 +1109,8 @@ 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, #{}, 1, <<"one-five">>, Log1), + run_effs(Effs), DelayedSnapWritten = receive {ra_log_event, {snapshot_written, {5, 1}, checkpoint} = Evt} -> @@ -1120,7 +1126,9 @@ 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), + {ok, SnapState, AcceptEffs} = + ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + run_effs(AcceptEffs), {Log4, Effs4} = ra_log:install_snapshot({15, 2}, SnapState, Log3), ?assert(lists:any(fun (E) -> element(1, E) == delete_snapshot end, Effs4)), %% write some more to create another segment @@ -1164,7 +1172,8 @@ 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), + {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + run_effs(AEffs), {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2), {15, _} = ra_log:last_index_term(Log3), @@ -1213,7 +1222,8 @@ 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), + {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + run_effs(AEffs), {Log2, _} = ra_log:install_snapshot({15, 2}, SnapState, Log1), {15, _} = ra_log:last_index_term(Log2), {15, _} = ra_log:last_written(Log2), @@ -1244,7 +1254,8 @@ 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), + {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + run_effs(AEffs), {Log2, _} = ra_log:install_snapshot({SnapIdx, 2}, SnapState, Log1), {SnapIdx, _} = ra_log:last_index_term(Log2), {SnapIdx, _} = ra_log:last_written(Log2), @@ -1281,10 +1292,11 @@ 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()}, + 1, initial_state, Log1), + run_effs(Effs), Log3 = assert_log_events(Log2, fun (L) -> {127, 2} == ra_log:snapshot_index_term(L) @@ -1299,9 +1311,10 @@ update_release_cursor(Config) -> end, 10, 100), Log3b = validate_fold(128, 149, 2, Log3), % update the release cursor all the way - {Log4, _} = ra_log:update_release_cursor(149, #{?N1 => new_peer(), - ?N2 => new_peer()}, - 1, initial_state, Log3b), + {Log4, Effs2} = ra_log:update_release_cursor(149, #{?N1 => new_peer(), + ?N2 => new_peer()}, + 1, initial_state, Log3b), + run_effs(Effs2), Log5 = assert_log_events(Log4, fun (L) -> {149, 2} == ra_log:snapshot_index_term(L) @@ -1338,10 +1351,11 @@ update_release_cursor_with_machine_version(Config) -> [_, _] = 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()}, + MacVer, + initial_state, Log1), + run_effs(Effs), Log = assert_log_events(Log2, fun (L) -> {127, 2} == ra_log:snapshot_index_term(L) @@ -1388,9 +1402,10 @@ 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), + run_effs(Effs2), Log7 = deliver_log_events_cond(Log6, fun (_) -> case find_segments(Config) of @@ -1778,7 +1793,8 @@ create_snapshot_chunk(Config, #{index := Idx} = Meta, Context) -> 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), + {Sn1, [{bg_work, Fun, _ErrFun}]} = ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, Sn0), + Fun(), Sn2 = receive {ra_log_event, {snapshot_written, {Idx, 2} = IdxTerm, snapshot}} -> @@ -1821,3 +1837,5 @@ 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]. diff --git a/test/ra_machine_int_SUITE.erl b/test/ra_machine_int_SUITE.erl index 0d74b0f8..09f752db 100644 --- a/test/ra_machine_int_SUITE.erl +++ b/test/ra_machine_int_SUITE.erl @@ -610,6 +610,7 @@ log_effect(Config) -> {[], ok, {log, lists:reverse(Idxs), fun (Cmds) -> + ct:pal("LOG!!"), Datas = [D || {_, D} <- Cmds], %% using a plain send here to %% ensure this effect is only diff --git a/test/ra_server_SUITE.erl b/test/ra_server_SUITE.erl index f811fb38..759f2170 100644 --- a/test/ra_server_SUITE.erl +++ b/test/ra_server_SUITE.erl @@ -157,7 +157,7 @@ setup_log() -> end), meck:expect(ra_snapshot, accept_chunk, fun(_Data, _OutOf, _Flag, SS) -> - {ok, SS} + {ok, SS, []} end), meck:expect(ra_snapshot, abort_accept, fun(SS) -> SS end), meck:expect(ra_snapshot, accepting, fun(_SS) -> undefined end), diff --git a/test/ra_snapshot_SUITE.erl b/test/ra_snapshot_SUITE.erl index 18d16403..0d696d31 100644 --- a/test/ra_snapshot_SUITE.erl +++ b/test/ra_snapshot_SUITE.erl @@ -38,7 +38,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() -> @@ -93,10 +94,11 @@ take_snapshot(Config) -> State0 = init_state(Config), Meta = meta(55, 2, [node()]), MacRef = ?FUNCTION_NAME, - {State1, [{monitor, process, snapshot_writer, Pid}]} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(Meta, MacRef, 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), @@ -115,27 +117,27 @@ take_snapshot_crash(Config) -> State0 = init_state(Config), Meta = meta(55, 2, [node()]), MacRef = ?FUNCTION_NAME, - {State1, [{monitor, process, snapshot_writer, Pid}]} = + {State1, [{bg_work, _Fun, ErrFun}]} = ra_snapshot:begin_snapshot(Meta, MacRef, 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,8 +145,9 @@ init_recover(Config) -> UId = ?config(uid, Config), State0 = init_state(Config), Meta = meta(55, 2, [node()]), - {State1, [{monitor, process, snapshot_writer, _}]} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot, State0), + Fun(), receive {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> _ = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1), @@ -169,8 +172,9 @@ 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, _}]} = + {State1, [{bg_work, Fun, _}]} = ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot, State0), + Fun(), receive {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> _ = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1), @@ -196,16 +200,20 @@ 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, ?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), + {State3, [{bg_work, Fun2, _}]} = + ra_snapshot:begin_snapshot(Meta2, ?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 +243,18 @@ 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, ?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), + {State3, [{bg_work, Fun2, _}]} = + ra_snapshot:begin_snapshot(Meta2, ?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,8 +290,9 @@ 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, ?FUNCTION_NAME, snapshot, State0), + Fun(), _ = receive {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1) @@ -310,22 +321,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, MacRef, snapshot, State0), + Fun(), + State = receive + {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> + ra_snapshot:complete_snapshot(IdxTerm, snapshot, 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 +367,11 @@ accept_snapshot(Config) -> undefined = ra_snapshot:accepting(State0), {ok, S1} = ra_snapshot:begin_accept(Meta, State0), {55, 2} = ra_snapshot:accepting(S1), - {ok, S2} = ra_snapshot:accept_chunk(A, 1, next, S1), - {ok, S3} = ra_snapshot:accept_chunk(B, 2, next, S2), - {ok, S4} = ra_snapshot:accept_chunk(C, 3, next, S3), - {ok, S5} = ra_snapshot:accept_chunk(D, 4, next, S4), - {ok, S} = ra_snapshot:accept_chunk(E, 5, last, S5), + {ok, S2, _} = ra_snapshot:accept_chunk(A, 1, next, S1), + {ok, S3, _} = ra_snapshot:accept_chunk(B, 2, next, S2), + {ok, S4, _} = ra_snapshot:accept_chunk(C, 3, next, S3), + {ok, S5, _} = ra_snapshot:accept_chunk(D, 4, next, S4), + {ok, S, _} = ra_snapshot:accept_chunk(E, 5, last, S5), undefined = ra_snapshot:accepting(S), undefined = ra_snapshot:pending(S), @@ -383,8 +395,8 @@ abort_accept(Config) -> undefined = ra_snapshot:accepting(State0), {ok, S1} = ra_snapshot:begin_accept(Meta, State0), {55, 2} = ra_snapshot:accepting(S1), - {ok, S2} = ra_snapshot:accept_chunk(A, 1, next, S1), - {ok, S3} = ra_snapshot:accept_chunk(B, 2, next, S2), + {ok, S2, _} = ra_snapshot:accept_chunk(A, 1, next, S1), + {ok, S3, _} = ra_snapshot:accept_chunk(B, 2, next, S2), S = ra_snapshot:abort_accept(S3), undefined = ra_snapshot:accepting(S), undefined = ra_snapshot:pending(S), @@ -392,15 +404,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, ?FUNCTION_NAME, snapshot, State0), + Fun(), MacRef = crypto:strong_rand_bytes(1024), MacBin = term_to_binary(MacRef), Crc = erlang:crc32([<<(size(MetaRemoteBin)):32/unsigned>>, @@ -411,9 +424,9 @@ accept_receives_snapshot_written_with_lower_index(Config) -> B/binary>> = <>, %% then begin an accept for a higher index - {ok, State2} = ra_snapshot:begin_accept(MetaRemote, State1), + {ok, State2} = ra_snapshot:begin_accept(MetaHigh, State1), {165, 2} = ra_snapshot:accepting(State2), - {ok, State3} = ra_snapshot:accept_chunk(A, 1, next, State2), + {ok, State3, _} = ra_snapshot:accept_chunk(A, 1, next, State2), %% then the snapshot written event is received receive @@ -423,7 +436,7 @@ accept_receives_snapshot_written_with_lower_index(Config) -> {55, 2} = ra_snapshot:current(State4), 55 = ra_snapshot:last_index_for(UId), %% then accept the last chunk - {ok, State} = ra_snapshot:accept_chunk(B, 2, last, State4), + {ok, State, _} = ra_snapshot:accept_chunk(B, 2, last, State4), undefined = ra_snapshot:accepting(State), {165, 2} = ra_snapshot:current(State), ok @@ -432,14 +445,15 @@ accept_receives_snapshot_written_with_lower_index(Config) -> end, ok. -accept_receives_snapshot_written_with_higher_index(Config) -> +accept_receives_snapshot_written_with_higher_index_2(Config) -> UId = ?config(uid, Config), State0 = init_state(Config), - MetaRemote = meta(55, 2, [node()]), - MetaLocal = meta(165, 2, [node()]), + MetaLow = meta(55, 2, [node()]), + MetaHigh = meta(165, 2, [node()]), %% begin a local snapshot - {State1, _} = ra_snapshot:begin_snapshot(MetaLocal, ?FUNCTION_NAME, - snapshot, State0), + {State1, [{bg_work, Fun, _}]} = + ra_snapshot:begin_snapshot(MetaLow, ?FUNCTION_NAME, snapshot, State0), + Fun(), MacRef = crypto:strong_rand_bytes(1024), MacBin = term_to_binary(MacRef), %% split into 1024 max byte chunks @@ -447,22 +461,24 @@ accept_receives_snapshot_written_with_higher_index(Config) -> B/binary>> = MacBin, %% then begin an accept for a higher index - {ok, State2} = ra_snapshot:begin_accept(MetaRemote, State1), - undefined = ra_snapshot:accepting(State2), - {ok, State3} = ra_snapshot:accept_chunk(A, 1, next, State2), - undefined = ra_snapshot:accepting(State3), - - %% then the snapshot written event is received + {ok, State2} = ra_snapshot:begin_accept(MetaHigh, State1), + {165, 2} = ra_snapshot:accepting(State2), + {ok, State3, _} = ra_snapshot:accept_chunk(A, 1, next, State2), + {165, 2} = ra_snapshot:accepting(State3), + + {ok, State4, _} = ra_snapshot:accept_chunk(B, 2, last, State3), + undefined = ra_snapshot:accepting(State4), + {165, 2} = ra_snapshot:current(State4), + undefined = ra_snapshot:pending(State4), + %% then the snapshot written event is received after the higher index + %% has been received receive {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, snapshot}} -> - State4 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State3), - undefined = ra_snapshot:pending(State4), - {55, 2} = ra_snapshot:current(State4), - 55 = ra_snapshot:last_index_for(UId), + State5 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, 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) From 6dddde9fa49b30388e74a09886dc0460887af046 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 17 Feb 2025 12:48:25 +0000 Subject: [PATCH 04/22] remove external readers --- src/ra_log.erl | 82 ++++++++++------------------------------- src/ra_server.erl | 25 ------------- test/ra_log_2_SUITE.erl | 65 -------------------------------- 3 files changed, 19 insertions(+), 153 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 7be55406..ccbef40c 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -49,9 +49,6 @@ delete_everything/1, release_resources/3, - % external reader - register_reader/2, - readers/1, tick/2 ]). @@ -108,7 +105,6 @@ last_resend_time :: option({integer(), WalPid :: pid() | undefined}), last_wal_write :: {pid(), Ms :: integer()}, reader :: ra_log_reader:state(), - readers = [] :: [pid()], mem_table :: ra_mt:state(), tx = false :: boolean() }). @@ -682,9 +678,7 @@ handle_event({written, _Term, {FromIdx, _}} = Evt, handle_event({segments, TidRanges, NewSegs}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, reader = Reader0, - mem_table = Mt0, - readers = Readers - } = State0) -> + mem_table = Mt0} = State0) -> Reader = ra_log_reader:update_segments(NewSegs, Reader0), put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, ra_log_reader:segment_ref_count(Reader)), @@ -698,15 +692,7 @@ handle_event({segments, TidRanges, NewSegs}, end, Mt0, TidRanges), State = State0#?MODULE{reader = Reader, mem_table = Mt}, - case Readers of - [] -> - {State, []}; - _ -> - %% HACK: but this feature is deprecated anyway - %% Dummy pid to swallow update notifications - Pid = spawn(fun () -> ok end), - {State, log_update_effects(Readers, Pid, State)} - end; + {State, []}; handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, @@ -784,15 +770,13 @@ handle_event({snapshot_error, Snap, SnapKind, Error}, 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}) -> @@ -1140,56 +1124,28 @@ release_resources(MaxOpenSegments, AccessPattern, ActiveSegs, Names, Counter)}. --spec register_reader(pid(), state()) -> - {state(), effects()}. -register_reader(Pid, #?MODULE{cfg = #cfg{uid = UId, - directory = Dir, - names = Names}, - reader = Reader, - readers = Readers} = State) -> - SegRefs = ra_log_reader:segment_refs(Reader), - NewReader = ra_log_reader:init(UId, Dir, 1, SegRefs, Names), - {State#?MODULE{readers = [Pid | Readers]}, - [{reply, {ok, NewReader}}, - {monitor, process, log, Pid}]}. - -readers(#?MODULE{readers = Readers}) -> - Readers. - %%% Local functions -log_update_effects(Pids, ReplyPid, #?MODULE{first_index = Idx, - reader = Reader}) -> - SegRefs = ra_log_reader:segment_refs(Reader), - [{send_msg, P, {ra_log_update, ReplyPid, Idx, SegRefs}, - [ra_event, local]} || P <- Pids]. - - %% 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)}; + {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), + ok = ra_log_segment_writer:truncate_segments(SegWriter, + UId, Pivot), NumActive = ra_log_reader:segment_ref_count(Reader), ?DEBUG("~ts: ~b obsolete segments at ~b - remaining: ~b, pivot ~0p", [LogId, length(Obsolete), SnapIdx, NumActive, Pivot]), put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, NumActive), State = State0#?MODULE{reader = Reader}, - {State, log_update_effects(Readers, Pid, State)} + {State, []} end. %% unly used by resend to wal functionality and doesn't update the mem table @@ -1377,16 +1333,16 @@ await_written_idx(Idx, Term, Log0) -> throw(ra_log_append_timeout) end. -log_update_wait_n(0) -> - ok; -log_update_wait_n(N) -> - receive - ra_log_update_processed -> - log_update_wait_n(N - 1) - after 1500 -> - %% just go ahead anyway - ok - end. +% log_update_wait_n(0) -> +% ok; +% log_update_wait_n(N) -> +% receive +% ra_log_update_processed -> +% log_update_wait_n(N - 1) +% after 1500 -> +% %% just go ahead anyway +% ok +% end. incr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> counters:add(Cnt, Ix, N); diff --git a/src/ra_server.erl b/src/ra_server.erl index deea5997..9deee4f2 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -52,7 +52,6 @@ checkpoint/3, persist_last_applied/1, update_peer/3, - register_external_log_reader/2, update_disconnected_peers/3, handle_down/5, handle_node_status/6, @@ -929,9 +928,6 @@ handle_leader({transfer_leadership, ServerId}, transition_to => leader}}}, [{reply, ok}, {send_msg, ServerId, election_timeout, cast}]} end; -handle_leader({register_external_log_reader, Pid}, #{log := Log0} = State) -> - {Log, Effs} = ra_log:register_reader(Pid, Log0), - {leader, State#{log => Log}, Effs}; handle_leader(force_member_change, State0) -> {follower, State0#{votes => 0}, [{next_event, force_member_change}]}; handle_leader(Msg, State) -> @@ -1040,9 +1036,6 @@ handle_candidate({ra_log_event, Evt}, State = #{log := Log0}) -> {candidate, State#{log => Log}, Effects}; handle_candidate(election_timeout, State) -> call_for_election(candidate, State); -handle_candidate({register_external_log_reader, Pid}, #{log := Log0} = State) -> - {Log, Effs} = ra_log:register_reader(Pid, Log0), - {candidate, State#{log => Log}, Effs}; handle_candidate(force_member_change, State0) -> {follower, State0#{votes => 0}, [{next_event, force_member_change}]}; handle_candidate(#info_rpc{term = Term} = Msg, @@ -1141,9 +1134,6 @@ handle_pre_vote({ra_log_event, Evt}, State = #{log := Log0}) -> % simply forward all other events to ra_log {Log, Effects} = ra_log:handle_event(Evt, Log0), {pre_vote, State#{log => Log}, Effects}; -handle_pre_vote({register_external_log_reader, Pid}, #{log := Log0} = State) -> - {Log, Effs} = ra_log:register_reader(Pid, Log0), - {pre_vote, State#{log => Log}, Effs}; handle_pre_vote(force_member_change, State0) -> {follower, State0#{votes => 0}, [{next_event, force_member_change}]}; handle_pre_vote(#info_rpc{term = Term} = Msg, @@ -1492,9 +1482,6 @@ handle_follower(election_timeout, State) -> call_for_election(pre_vote, State); handle_follower(try_become_leader, State) -> handle_follower(election_timeout, State); -handle_follower({register_external_log_reader, Pid}, #{log := Log0} = State) -> - {Log, Effs} = ra_log:register_reader(Pid, Log0), - {follower, State#{log => Log}, Effs}; handle_follower(force_member_change, #{cfg := #cfg{id = Id, uid = Uid, @@ -1635,9 +1622,6 @@ handle_receive_snapshot(receive_snapshot_timeout, #{log := Log0} = State) -> SnapState = ra_snapshot:abort_accept(SnapState0), Log = ra_log:set_snapshot_state(SnapState, Log0), {follower, State#{log => Log}, []}; -handle_receive_snapshot({register_external_log_reader, Pid}, #{log := Log0} = State) -> - {Log, Effs} = ra_log:register_reader(Pid, Log0), - {receive_snapshot, State#{log => Log}, Effs}; handle_receive_snapshot(#info_rpc{term = Term} = Msg, #{current_term := CurTerm, cfg := #cfg{log_id = LogId}, @@ -1709,9 +1693,6 @@ handle_await_condition({ra_log_event, Evt}, State = #{log := Log0}) -> % simply forward all other events to ra_log {Log, Effects} = ra_log:handle_event(Evt, Log0), {await_condition, State#{log => Log}, Effects}; -handle_await_condition({register_external_log_reader, Pid}, #{log := Log0} = State) -> - {Log, Effs} = ra_log:register_reader(Pid, Log0), - {await_condition, State#{log => Log}, Effs}; handle_await_condition(Msg, #{condition := #{predicate_fun := Pred} = Cond} = State0) -> case Pred(Msg, State0) of {true, State1} -> @@ -2257,12 +2238,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) -> diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 4044ef43..fb73bd31 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -1467,69 +1467,6 @@ open_segments_limit(Config) -> ?assert(Open =< Max), ok. -external_reader(Config) -> - %% external readers should be notified of all new segments - %% and the lower bound of the log - %% The upper bound needs to be discovered by querying - logger:set_primary_config(level, all), - Log0 = ra_log_init(Config), - {0, 0} = ra_log:last_index_term(Log0), - - Log1 = write_n(200, 220, 2, - write_and_roll(1, 200, 2, Log0)), - - Self = self(), - Pid = spawn( - fun () -> - receive - {ra_log_reader_state, R1} = Evt -> - {Es, R2} = ra_log_reader:sparse_read( - R1, lists:seq(0, 220), []), - Len1 = length(Es), - Self ! {got, Evt, Es}, - receive - {ra_log_update, _, F, _} = Evt2 -> - %% reader before update has been processed - %% should work - Indexes = lists:seq(F, 220), - {Stale, _} = ra_log_reader:sparse_read(R2, Indexes, []), - ?assertEqual(Len1, length(Stale)), - R3 = ra_log_reader:handle_log_update(Evt2, R2), - {Es2, _R4} = ra_log_reader:sparse_read(R3, Indexes, []), - ct:pal("Es2 ~w", [length(Es2)]), - ?assertEqual(Len1, length(Es2)), - Self ! {got, Evt2, Es2} - end - end - end), - {Log2, [{reply, {ok, Reader}} | _]} = - ra_log:register_reader(Pid, Log1), - Pid ! {ra_log_reader_state, Reader}, - %% TODO: validate there is monitor effect - receive - {got, Evt, Entries} -> - ct:pal("got segs: ~w ~w", [Evt, length(Entries)]), - ok - after 2000 -> - flush(), - exit(got_timeout) - end, - ra_log_wal:force_roll_over(ra_log_wal), - - deliver_log_events_cond( - Log2, fun (_L) -> - %% this should result in a segment update - receive - {got, Evt2, Entries1} -> - ct:pal("got segs: ~p ~p", [Evt2, length(Entries1)]), - true - after 10 -> - false - end - end, 100), - flush(), - ok. - write_config(Config) -> C = #{cluster_name => ?MODULE, @@ -1619,7 +1556,6 @@ deliver_log_events_cond(Log0, CondFun, N) -> (_, 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); @@ -1662,7 +1598,6 @@ deliver_all_log_events(Log0, Timeout) -> (_, Acc) -> Acc end, Log1, Effs), - [P ! E || {send_msg, P, E, _} <- Effs], % ct:pal("log evt effs: ~p", [Effs]), deliver_all_log_events(Log, Timeout) after Timeout -> From af5d9ff5e187eb13a598f6bfaa23aabeb038d5ba Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 11:19:11 +0000 Subject: [PATCH 05/22] Add ra_machine:live_indexes callback/API To allow machines to report which indexes should be kept in addition to the snapshot. These are written to a new file in the snapshots directory for easy recovery without having to recover the entire snapshot. Other API changes and refactoring. --- docs/internals/COMPACTION.md | 104 ++++++++++++++++++++++++++++++++++- src/ra_log.erl | 80 ++++++++++++++------------- src/ra_log_reader.erl | 27 ++++----- src/ra_log_wal.erl | 11 ++-- src/ra_machine.erl | 14 ++++- src/ra_server.erl | 13 +++-- src/ra_snapshot.erl | 21 +++++-- 7 files changed, 201 insertions(+), 69 deletions(-) diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md index 6d3eeed6..04a840b1 100644 --- a/docs/internals/COMPACTION.md +++ b/docs/internals/COMPACTION.md @@ -144,7 +144,7 @@ Segment range: (1, 1000) Memtable range: (1001, 2000) -Snapshot: 1500, live indexes [1501, 1999], +Snapshot: 1500, live indexes `[1501, 1999]`, Alt: if the log worker / Ra server is alive the segment writer could call into @@ -169,6 +169,98 @@ go on and replicate these before or after (??) sending the snapshot itself. T: probably before as if a new snapshot has been taken locally we'd most likely skip some raft index replication on the second attempt. +Q: How will the follower write the live indexes preceding the snapshot? +If the main Ra process does it this introduces a 3rd modifier of the Ra log +and there may be concurrent Ra log writes from the snapshot writer at this point. + +It can't write them to the WAL as they are not contiguous unless we allow +such writes. + +The Ra process can write them to a set of temporary segment files then call into +the segment writer to rename into the set of segments. +No this can't work with the +live indexes logic the segment writer applies as it relies on the mem tables +ranges to decide which indexes to flush. + +having pending segment flushes when receiving + +the ra process truncates the mem table on snapshot installations +so that the segment writer avoids +writing any of the live index preceding the snapshot. + +If this is done before live indexes are replicated if the Ra process then waits +for the mt delete to complete then + +Idea: the ra server could truncate the mt as soon as a snapshot installation +starts to minimise subsequent mem table flushes. Typically this means emptying +the memtable completely (the ra server could execute the delete perhaps to ensure). + +Scenario: pending mem table flushes when snapshot installation comes in. + +Need to ensure the old pending data in the WAL / memtable isn't flushed _after_ +the received live indexes are written to a segment and that segment is renamed +into the list of segments. + +Options: + +1. Call into segment writer to rename the temporary named segment(s) into the +main sequence of segments. This command will return the full list of new segments. +If the memtable has been truncated before this is done by the time the rename +returns we know there wont be any more segment records being written. +We can't update the `ra_log_snapshot_state` table until _after_ this as the +segment writer may assume the live indexes should be in the memtable. +Also need to handle concurrent compaction? +Downside: BLOCKING, segwriter can at at times run a backlog. + +2. + + +##### Steps when receiving a new valid `install_snapshot_rpc` + +* Once the `last` part of the snapshot has been installed and recovered as the +mem table state we can calculated which live indexes are needed to complete +the snapshot. The Ra follower remains in `receiving_snapshot` state and replies +back with a `ra_seq` of the required indexes. + * these are received and written to the log as normal +process itself + + +#### Alternative snapshot install procedure + +* Sender begins with sending negotiating which live indexes are needed. +* Then it proceeds to send the live indexes _before_ the snapshot (so in it's +natural order if you like). +* The receiving ra process then writes these commands to the WAL as normal but +using a special command / flag to avoid the WAL triggering its' gap detection. +Ideally the specialised command would include the previous idx so that we can +still do gap detection in the sparse sequence (should all sends include prior +sequence so this is the only mode?). +* Alt the live indexes replication could be done after the snapshot is complete +as it is easy for the follower to work out which live indexes it needs. +when it receives the `last` snapshot chunk it then replies with a special +continuation command instead of `install_snapshot_result{}` which will initiate +the live index replication. NB the snapshot sender process may need to call +into the leader process to get read plans as entries _could_ be in the memtable. + +#### How to work out which live indexes the follower needs + +Gnarly example: + +Follower term indexes: + +`{{100, 1000}, 1}, {{1001, 1500}, 2}` + +Incoming snapshot at 2000 in term 3 + +live indexes: `[100, 600, 1200, 1777]` + +Follower needs all live indexes greater than it's `last_applied` index. +This is the only safe way to ensure that the live indexes are not stale entries. + +If follower `last applied` is: 1500 then follower needs: `[1777]` only. +If follower `last_applied` is: 1100 then follower needs `[1200, 1777]` + + #### How to store live indexes with snapshot * New section in snapshot file format. @@ -186,3 +278,13 @@ backlog. 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 + diff --git a/src/ra_log.erl b/src/ra_log.erl index ccbef40c..191525fe 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -699,6 +699,7 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind}, first_index = FstIdx, last_index = LstIdx, mem_table = Mt0, + % reader = Reader, last_written_index_term = {LastWrittenIdx, _} = LWIdxTerm0, snapshot_state = SnapState0} = State0) %% only update snapshot if it is newer than the last snapshot @@ -707,8 +708,6 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind}, 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. @@ -721,21 +720,31 @@ 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, + % delete any segments outside of first_index + {State, Effects1} = delete_segments(SnapIdx, State0), + + %% 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), ok = exec_mem_table_delete(Names, UId, Spec), + %% TODO schedule compaction, + %% NB the mt cannot be truncated here with set_first as we need + %% to keep any live indexes + % SegRefs = ra_log_reader:segment_refs(Reader), + + Effects = Effects0 ++ Effects1, {State#?MODULE{first_index = SnapIdx + 1, last_index = max(LstIdx, SnapIdx), last_written_index_term = LWIdxTerm, @@ -832,7 +841,7 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, CPEffects = [{delete_snapshot, ra_snapshot:directory(SnapState, checkpoint), Checkpoint} || Checkpoint <- Checkpoints], - {Spec, Mt} = ra_mt:set_first(SnapIdx, Mt0), + {Spec, Mt} = ra_mt:set_first(SnapIdx + 1, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), {State#?MODULE{snapshot_state = SnapState, first_index = SnapIdx + 1, @@ -856,25 +865,28 @@ recover_snapshot(#?MODULE{snapshot_state = SnapState}) -> snapshot_index_term(#?MODULE{snapshot_state = SS}) -> ra_snapshot:current(SS). --spec update_release_cursor(Idx :: ra_index(), Cluster :: ra_cluster(), - MacVersion :: ra_machine:version(), +-spec update_release_cursor(Idx :: ra_index(), + Cluster :: ra_cluster(), + MacModule :: module(), MacState :: term(), State :: state()) -> {state(), effects()}. -update_release_cursor(Idx, Cluster, MacVersion, MacState, State) -> - suggest_snapshot(snapshot, Idx, Cluster, MacVersion, MacState, State). +update_release_cursor(Idx, Cluster, MacModule, MacState, State) + when is_atom(MacModule) -> + suggest_snapshot(snapshot, Idx, Cluster, MacModule, MacState, State). -spec checkpoint(Idx :: ra_index(), Cluster :: ra_cluster(), - MacVersion :: ra_machine:version(), + MacModule :: module(), MacState :: term(), State :: state()) -> {state(), effects()}. -checkpoint(Idx, Cluster, MacVersion, MacState, State) -> - suggest_snapshot(checkpoint, Idx, Cluster, MacVersion, MacState, State). +checkpoint(Idx, Cluster, MacModule, MacState, State) + when is_atom(MacModule) -> + suggest_snapshot(checkpoint, Idx, Cluster, MacModule, MacState, State). -suggest_snapshot(SnapKind, Idx, Cluster, MacVersion, MacState, +suggest_snapshot(SnapKind, Idx, Cluster, MacModule, MacState, #?MODULE{snapshot_state = SnapshotState} = State) -> case ra_snapshot:pending(SnapshotState) of undefined -> - suggest_snapshot0(SnapKind, Idx, Cluster, MacVersion, MacState, State); + suggest_snapshot0(SnapKind, Idx, Cluster, MacModule, MacState, State); _ -> %% Only one snapshot or checkpoint may be written at a time to %% prevent excessive I/O usage. @@ -920,13 +932,7 @@ tick(Now, #?MODULE{cfg = #cfg{wal = Wal}, State end. -suggest_snapshot0(SnapKind, Idx, Cluster, MacVersion, MacState, State0) -> - ClusterServerIds = maps:map(fun (_, V) -> - maps:with([voter_status], V) - end, Cluster), - Meta = #{index => Idx, - cluster => ClusterServerIds, - machine_version => MacVersion}, +suggest_snapshot0(SnapKind, Idx, Cluster, MacModule, MacState, State0) -> case should_snapshot(SnapKind, Idx, State0) of true -> % TODO: here we use the current cluster configuration in @@ -944,7 +950,15 @@ suggest_snapshot0(SnapKind, Idx, Cluster, MacVersion, MacState, State0) -> {undefined, _} -> {State0, []}; {Term, State} -> - write_snapshot(Meta#{term => Term}, MacState, + ClusterServerIds = + maps:map(fun (_, V) -> + maps:with([voter_status], V) + end, Cluster), + MachineVersion = ra_machine:version(MacModule), + Meta = #{index => Idx, + cluster => ClusterServerIds, + machine_version => MachineVersion}, + write_snapshot(Meta#{term => Term}, MacModule, MacState, SnapKind, State) end; false -> @@ -1085,18 +1099,6 @@ delete_everything(#?MODULE{cfg = #cfg{uid = UId, %% 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, try ra_lib:recursive_delete(Dir) of ok -> ok catch @@ -1276,7 +1278,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 @@ -1284,8 +1286,8 @@ write_snapshot(Meta, MacRef, SnapKind, checkpoint -> ?C_RA_LOG_CHECKPOINTS_WRITTEN end, ok = incr_counter(Cfg, Counter, 1), - {SnapState, Effects} = ra_snapshot:begin_snapshot(Meta, MacRef, SnapKind, - SnapState0), + {SnapState, Effects} = ra_snapshot:begin_snapshot(Meta, MacModule, MacState, + SnapKind, SnapState0), {State#?MODULE{snapshot_state = SnapState}, Effects}. recover_ranges(UId, MtRange, SegWriter) -> diff --git a/src/ra_log_reader.erl b/src/ra_log_reader.erl index ea171e69..d99a14a9 100644 --- a/src/ra_log_reader.erl +++ b/src/ra_log_reader.erl @@ -13,7 +13,7 @@ init/7, close/1, update_segments/2, - handle_log_update/2, + handle_compaction/2, segment_refs/1, segment_ref_count/1, range/1, @@ -109,7 +109,7 @@ update_segments(NewSegmentRefs, SegmentRefs0 = ra_lol:to_list(SegRefs0), SegmentRefsComp = compact_segrefs(NewSegmentRefs, SegmentRefs0), - SegmentRefsCompRev = lists:reverse(SegmentRefsComp ), + SegmentRefsCompRev = lists:reverse(SegmentRefsComp), SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev), Range = case SegmentRefsComp of [{{_, L}, _} | _] -> @@ -131,19 +131,20 @@ update_segments(NewSegmentRefs, range = Range, open_segments = Open}. --spec handle_log_update({ra_log_update, undefined | pid(), ra_index(), - [segment_ref()]}, state()) -> state(). -handle_log_update({ra_log_update, From, _FstIdx, SegRefs}, - #?STATE{open_segments = Open0} = State) -> +-record(log_compaction_result, + {%range :: ra:range(), + deleted :: [segment_ref()], + new :: [segment_ref()]}). +-spec handle_compaction(#log_compaction_result{}, state()) -> state(). +handle_compaction(#log_compaction_result{deleted = Deleted, + new = New}, + #?STATE{open_segments = Open0, + segment_refs = SegRefs0} = State) -> + SegmentRefs0 = ra_lol:to_list(SegRefs0), + SegmentRefs = lists:sort((SegmentRefs0 -- Deleted) ++ New), Open = ra_flru:evict_all(Open0), - case From of - undefined -> ok; - _ -> - %% reply to the updater process - From ! ra_log_update_processed - end, State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, - lists:reverse(SegRefs)), + lists:reverse(SegmentRefs)), open_segments = Open}. -spec update_first_index(ra_index(), state()) -> diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index 87ca98d3..e5235866 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -713,12 +713,11 @@ complete_batch(#state{batch = #batch{waiting = Waiting, 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) -> + 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{old = #batch_writer{} = OldBw} = Bw, diff --git a/src/ra_machine.erl b/src/ra_machine.erl index 33533c0e..8178a8db 100644 --- a/src/ra_machine.erl +++ b/src/ra_machine.erl @@ -72,6 +72,7 @@ snapshot_installed/5, state_enter/3, overview/2, + live_indexes/2, query/3, module/1, init_aux/2, @@ -222,6 +223,7 @@ handle_aux/5, handle_aux/6, overview/1, + live_indexes/1, snapshot_module/0, version/0, which_module/1 @@ -287,6 +289,8 @@ -callback overview(state()) -> map(). +-callback live_indexes(state()) -> [ra:index()]. + -callback snapshot_module() -> module(). -callback version() -> version(). @@ -344,11 +348,17 @@ state_enter(Mod, RaftState, State) -> overview(Mod, State) -> ?OPT_CALL(Mod:overview(State), State). +-spec live_indexes(module(), state()) -> [ra:index()]. +live_indexes(Mod, State) -> + ?OPT_CALL(Mod:live_indexes(State), []). + %% @doc used to discover the latest machine version supported by the current %% code --spec version(machine()) -> version(). +-spec version(machine() | module()) -> version(). +version(Mod) when is_atom(Mod) -> + ?OPT_CALL(assert_version(Mod:version()), ?DEFAULT_VERSION); version({machine, Mod, _}) -> - ?OPT_CALL(assert_version(Mod:version()), ?DEFAULT_VERSION). + version(Mod). -spec is_versioned(machine()) -> boolean(). is_versioned({machine, Mod, _}) -> diff --git a/src/ra_server.erl b/src/ra_server.erl index 9deee4f2..515c1863 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -2188,21 +2188,26 @@ log_fold_cache(From, _To, _Cache, Acc) -> term(), ra_server_state()) -> {ra_server_state(), effects()}. update_release_cursor(Index, MacState, - State = #{log := Log0, cluster := Cluster}) -> + #{cfg := #cfg{machine = Machine}, + log := Log0, + cluster := Cluster} = State) -> MacVersion = index_machine_version(Index, State), + MacMod = ra_machine:which_module(Machine, MacVersion), % simply pass on release cursor index to log {Log, Effects} = ra_log:update_release_cursor(Index, Cluster, - MacVersion, + MacMod, MacState, Log0), {State#{log => Log}, Effects}. -spec checkpoint(ra_index(), term(), ra_server_state()) -> {ra_server_state(), effects()}. checkpoint(Index, MacState, - State = #{log := Log0, cluster := Cluster}) -> + #{cfg := #cfg{machine = Machine}, + log := Log0, cluster := Cluster} = State) -> MacVersion = index_machine_version(Index, State), + MacMod = ra_machine:which_module(Machine, MacVersion), {Log, Effects} = ra_log:checkpoint(Index, Cluster, - MacVersion, MacState, Log0), + MacMod, MacState, Log0), {State#{log => Log}, Effects}. -spec promote_checkpoint(ra_index(), ra_server_state()) -> diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index c734188b..2ff3ef43 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -28,7 +28,7 @@ directory/2, last_index_for/1, - begin_snapshot/4, + begin_snapshot/5, promote_checkpoint/2, complete_snapshot/3, @@ -353,9 +353,10 @@ last_index_for(UId) -> undefined end. --spec begin_snapshot(meta(), ReleaseCursorRef :: term(), kind(), state()) -> +-spec begin_snapshot(meta(), MacModule :: module(), + MacStateb :: term(), kind(), state()) -> {state(), [effect()]}. -begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind, +begin_snapshot(#{index := Idx, term := Term} = Meta, MacMod, MacState, SnapKind, #?MODULE{module = Mod, counter = Counter, snapshot_directory = SnapshotDir, @@ -372,10 +373,14 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind, Sync = SnapKind =:= snapshot, %% create directory for this snapshot SnapDir = make_snapshot_dir(Dir, Idx, Term), + %% TODO: really we'd like to run this in the ra worker as good potentially + %% be quite large and a touch expensive to compute but also we don't want + %% to close over both the MacState and the MacRef + Indexes = ra_machine:live_indexes(MacMod, MacState), %% call prepare then write_snapshotS %% This needs to be called in the current process to "lock" potentially %% mutable machine state - Ref = Mod:prepare(Meta, MacRef), + Ref = Mod:prepare(Meta, MacState), %% write the snapshot in a separate process Self = self(), IdxTerm = {Idx, Term}, @@ -388,6 +393,14 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacRef, SnapKind, BytesWritten), ok end, + %% write the live indexes, if any + case Indexes of + [] -> ok; + _ -> + F = filename:join(SnapDir, "indexes"), + ok = ra_lib:write_file(F, term_to_binary(Indexes), true), + ok + end, Self ! {ra_log_event, {snapshot_written, IdxTerm, SnapKind}}, ok From ade685ebe8fe82b984e732172ea42c30c5231b9c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 11:33:38 +0000 Subject: [PATCH 06/22] fix tests --- test/ra_checkpoint_SUITE.erl | 47 ++++++++++++++++++++++++------------ test/ra_log_2_SUITE.erl | 35 +++++++++++++++------------ test/ra_snapshot_SUITE.erl | 41 +++++++++++++++++++------------ 3 files changed, 78 insertions(+), 45 deletions(-) diff --git a/test/ra_checkpoint_SUITE.erl b/test/ra_checkpoint_SUITE.erl index a4101426..915ae1f1 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,9 +84,9 @@ take_checkpoint(Config) -> State0 = init_state(Config), Meta = meta(55, 2, [node()]), - MacRef = ?FUNCTION_NAME, + MacState = ?FUNCTION_NAME, {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta, MacRef, checkpoint, State0), + ra_snapshot:begin_snapshot(Meta, ?MACMOD, MacState, checkpoint, State0), undefined = ra_snapshot:latest_checkpoint(State1), {{55, 2}, checkpoint} = ra_snapshot:pending(State1), @@ -104,9 +106,9 @@ take_checkpoint(Config) -> take_checkpoint_crash(Config) -> State0 = init_state(Config), Meta = meta(55, 2, [node()]), - MacRef = ?FUNCTION_NAME, + MacState = ?FUNCTION_NAME, {State1, [{bg_work, _Fun, ErrFun}]} = - ra_snapshot:begin_snapshot(Meta, MacRef, checkpoint, State0), + ra_snapshot:begin_snapshot(Meta, ?MODULE, MacState, checkpoint, State0), undefined = ra_snapshot:latest_checkpoint(State1), {{55, 2}, checkpoint} = ra_snapshot:pending(State1), ErrFun(it_failed), @@ -134,7 +136,8 @@ recover_from_checkpoint_only(Config) -> Meta = meta(55, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, checkpoint, State0), + ra_snapshot:begin_snapshot(Meta, ?MODULE, ?FUNCTION_NAME, + checkpoint, State0), Fun(), receive {ra_log_event, {snapshot_written, IdxTerm, checkpoint}} -> @@ -161,7 +164,8 @@ recover_from_checkpoint_and_snapshot(Config) -> %% Snapshot. SnapMeta = meta(55, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(SnapMeta, ?FUNCTION_NAME, snapshot, State0), + ra_snapshot:begin_snapshot(SnapMeta, ?MODULE, ?FUNCTION_NAME, + snapshot, State0), Fun(), State2 = receive {ra_log_event, {snapshot_written, IdxTerm1, snapshot}} -> @@ -173,7 +177,8 @@ recover_from_checkpoint_and_snapshot(Config) -> %% Checkpoint at a later index. CPMeta = meta(105, 3, [node()]), {State3, [{bg_work, Fun2, _}]} = - ra_snapshot:begin_snapshot(CPMeta, ?FUNCTION_NAME, checkpoint, State2), + ra_snapshot:begin_snapshot(CPMeta, ?MODULE, ?FUNCTION_NAME, + checkpoint, State2), Fun2(), receive {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} -> @@ -201,7 +206,8 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Checkpoint at 25. CP1Meta = meta(25, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(CP1Meta, ?FUNCTION_NAME, checkpoint, State0), + ra_snapshot:begin_snapshot(CP1Meta, ?MODULE, ?FUNCTION_NAME, + checkpoint, State0), Fun(), State2 = receive {ra_log_event, {snapshot_written, IdxTerm1, checkpoint}} -> @@ -213,7 +219,8 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Checkpoint at 35. CP2Meta = meta(35, 3, [node()]), {State3, [{bg_work, Fun2, _}]} = - ra_snapshot:begin_snapshot(CP2Meta, ?FUNCTION_NAME, checkpoint, State2), + ra_snapshot:begin_snapshot(CP2Meta, ?MODULE, ?FUNCTION_NAME, + checkpoint, State2), Fun2(), State4 = receive {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} -> @@ -225,7 +232,8 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Checkpoint at 55. CP3Meta = meta(55, 5, [node()]), {State5, [{bg_work, Fun3, _}]} = - ra_snapshot:begin_snapshot(CP3Meta, ?FUNCTION_NAME, checkpoint, State4), + ra_snapshot:begin_snapshot(CP3Meta, ?MODULE, ?FUNCTION_NAME, + checkpoint, State4), Fun3(), State6 = receive {ra_log_event, {snapshot_written, IdxTerm3, checkpoint}} -> @@ -237,7 +245,8 @@ newer_snapshot_deletes_older_checkpoints(Config) -> %% Snapshot at 45. SnapMeta = meta(45, 4, [node()]), {State7, [{bg_work, Fun4, _}]} = - ra_snapshot:begin_snapshot(SnapMeta, ?FUNCTION_NAME, snapshot, State6), + ra_snapshot:begin_snapshot(SnapMeta, ?MODULE, ?FUNCTION_NAME, + snapshot, State6), Fun4(), State8 = receive {ra_log_event, {snapshot_written, IdxTerm4, snapshot}} -> @@ -272,7 +281,8 @@ init_recover_corrupt(Config) -> %% Take a checkpoint. Meta1 = meta(55, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta1, ?FUNCTION_NAME, checkpoint, State0), + ra_snapshot:begin_snapshot(Meta1, ?MODULE, ?FUNCTION_NAME, + checkpoint, State0), Fun(), State2 = receive {ra_log_event, {snapshot_written, {55, 2} = IdxTerm1, checkpoint}} -> @@ -284,7 +294,8 @@ init_recover_corrupt(Config) -> %% Take another checkpoint. Meta2 = meta(165, 2, [node()]), {State3, [{bg_work, Fun2, _}]} = - ra_snapshot:begin_snapshot(Meta2, ?FUNCTION_NAME, checkpoint, State2), + ra_snapshot:begin_snapshot(Meta2, ?MODULE, ?FUNCTION_NAME, + checkpoint, State2), Fun2(), receive {ra_log_event, {snapshot_written, {165, 2} = IdxTerm2, checkpoint}} -> @@ -317,7 +328,8 @@ init_recover_multi_corrupt(Config) -> %% Checkpoint at 55. CP1Meta = meta(55, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(CP1Meta, ?FUNCTION_NAME, checkpoint, State0), + ra_snapshot:begin_snapshot(CP1Meta, ?MODULE, ?FUNCTION_NAME, + checkpoint, State0), Fun(), State2 = receive {ra_log_event, {snapshot_written, IdxTerm1, checkpoint}} -> @@ -329,7 +341,8 @@ init_recover_multi_corrupt(Config) -> %% Checkpoint at 165. CP2Meta = meta(165, 2, [node()]), {State3, [{bg_work, Fun2, _}]} = - ra_snapshot:begin_snapshot(CP2Meta, ?FUNCTION_NAME, checkpoint, State2), + ra_snapshot:begin_snapshot(CP2Meta, ?MODULE, ?FUNCTION_NAME, + checkpoint, State2), Fun2(), State4 = receive {ra_log_event, {snapshot_written, IdxTerm2, checkpoint}} -> @@ -380,3 +393,7 @@ list_checkpoint_dirs(Config) -> list_snap_dirs(Config) -> SnapDir = ?config(snap_dir, Config), filelib:wildcard(filename:join(SnapDir, "*")). + +%% ra_machine fakes +version() -> 1. +live_indexes(_) -> []. diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index fb73bd31..7a7b1145 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -397,7 +397,7 @@ sparse_read_out_of_range_2(Config) -> write_and_roll(1, 10, 2, Log0)), 50), SnapIdx = 10, %% do snapshot in - {Log2, Effs} = ra_log:update_release_cursor(SnapIdx, #{}, 2, + {Log2, Effs} = ra_log:update_release_cursor(SnapIdx, #{}, ?MODULE, <<"snap@10">>, Log1), run_effs(Effs), {Log3, _} = receive @@ -533,7 +533,7 @@ 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, Effs} = ra_log:update_release_cursor(2, #{}, 1, + {Log2, Effs} = ra_log:update_release_cursor(2, #{}, ?MODULE, <<"one+two">>, Log1b), run_effs(Effs), {Log3, _} = receive @@ -549,7 +549,7 @@ written_event_after_snapshot(Config) -> Log5 = ra_log:append({3, 1, <<"three">>}, Log4), Log6 = ra_log:append({4, 1, <<"four">>}, Log5), Log6b = deliver_all_log_events(Log6, 100), - {Log7, Effs2} = ra_log:update_release_cursor(4, #{}, 1, + {Log7, Effs2} = ra_log:update_release_cursor(4, #{}, ?MODULE, <<"one+two+three+four">>, Log6b), run_effs(Effs2), @@ -570,7 +570,7 @@ 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, Effs} = ra_log:update_release_cursor(2, #{}, 1, + {Log3, Effs} = ra_log:update_release_cursor(2, #{}, ?MODULE, <<"one+two">>, Log2), run_effs(Effs), Log4 = deliver_all_log_events(Log3, 100), @@ -593,7 +593,8 @@ 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, Effs0} = ra_log:update_release_cursor(100, #{}, 1, <<"100">>, Log2), + {Log3, Effs0} = ra_log:update_release_cursor(100, #{}, ?MODULE, + <<"100">>, Log2), run_effs(Effs0), Log4 = deliver_all_log_events(Log3, 500), @@ -1062,7 +1063,7 @@ 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, Effs} = ra_log:update_release_cursor(5, #{}, 1, + {Log2, Effs} = ra_log:update_release_cursor(5, #{}, ?MODULE, <<"one-five">>, Log1), run_effs(Effs), DelayedSnapWritten = receive @@ -1109,7 +1110,7 @@ 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, Effs} = ra_log:checkpoint(5, #{}, 1, <<"one-five">>, Log1), + {Log2, Effs} = ra_log:checkpoint(5, #{}, ?MODULE, <<"one-five">>, Log1), run_effs(Effs), DelayedSnapWritten = receive {ra_log_event, {snapshot_written, {5, 1}, @@ -1294,7 +1295,7 @@ update_release_cursor(Config) -> % update release cursor to the last entry of the first segment {Log2, Effs} = ra_log:update_release_cursor(127, #{?N1 => new_peer(), ?N2 => new_peer()}, - 1, initial_state, Log1), + ?MODULE, initial_state, Log1), run_effs(Effs), Log3 = assert_log_events(Log2, @@ -1313,7 +1314,7 @@ update_release_cursor(Config) -> % update the release cursor all the way {Log4, Effs2} = ra_log:update_release_cursor(149, #{?N1 => new_peer(), ?N2 => new_peer()}, - 1, initial_state, Log3b), + ?MODULE, initial_state, Log3b), run_effs(Effs2), Log5 = assert_log_events(Log4, fun (L) -> @@ -1350,10 +1351,9 @@ 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, Effs} = ra_log:update_release_cursor(127, #{?N1 => new_peer(), ?N2 => new_peer()}, - MacVer, + ?MODULE, initial_state, Log1), run_effs(Effs), Log = assert_log_events(Log2, @@ -1364,7 +1364,7 @@ update_release_cursor_with_machine_version(Config) -> %% assert the version is in the snapshot state meta data CurrentDir = ra_snapshot:current_snapshot_dir(SnapState), {ok, Meta} = ra_snapshot:read_meta(ra_log_snapshot, CurrentDir), - ?assertMatch(#{index := 127, machine_version := MacVer}, Meta), + ?assertMatch(#{index := 127, machine_version := 1}, Meta), ok. missed_mem_table_entries_are_deleted_at_next_opportunity(Config) -> @@ -1404,7 +1404,7 @@ missed_mem_table_entries_are_deleted_at_next_opportunity(Config) -> % then update the release cursor {Log6, Effs2} = ra_log:update_release_cursor(154, #{?N1 => new_peer(), ?N2 => new_peer()}, - 1, initial_state, Log5), + ?MODULE, initial_state, Log5), run_effs(Effs2), Log7 = deliver_log_events_cond(Log6, fun (_) -> @@ -1727,8 +1727,9 @@ create_snapshot_chunk(Config, #{index := Idx} = Meta, Context) -> ok = ra_lib:make_dir(CPDir), Sn0 = ra_snapshot:init(<<"someotheruid_adsfasdf">>, ra_log_snapshot, OthDir, CPDir, undefined, ?DEFAULT_MAX_CHECKPOINTS), - MacRef = <<"9">>, - {Sn1, [{bg_work, Fun, _ErrFun}]} = ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, Sn0), + MacState = <<"9">>, + {Sn1, [{bg_work, Fun, _ErrFun}]} = + ra_snapshot:begin_snapshot(Meta, ?MODULE, MacState, snapshot, Sn0), Fun(), Sn2 = receive @@ -1774,3 +1775,7 @@ wait_for_wal(OldPid) -> end, 100, 100). run_effs(Effs) -> [Fun() || {bg_work, Fun, _} <- Effs]. + +%% ra_machine fakes +version() -> 1. +live_indexes(_) -> []. diff --git a/test/ra_snapshot_SUITE.erl b/test/ra_snapshot_SUITE.erl index 0d696d31..8889a974 100644 --- a/test/ra_snapshot_SUITE.erl +++ b/test/ra_snapshot_SUITE.erl @@ -16,6 +16,8 @@ -include_lib("eunit/include/eunit.hrl"). -include("src/ra.hrl"). +-define(MACMOD, ?MODULE). + %%%=================================================================== %%% Common Test callbacks %%%=================================================================== @@ -93,9 +95,9 @@ take_snapshot(Config) -> UId = ?config(uid, Config), State0 = init_state(Config), Meta = meta(55, 2, [node()]), - MacRef = ?FUNCTION_NAME, + MacState = ?FUNCTION_NAME, {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, State0), + ra_snapshot:begin_snapshot(Meta, ?MACMOD,MacState, snapshot, State0), undefined = ra_snapshot:current(State1), Fun(), {{55, 2}, snapshot} = ra_snapshot:pending(State1), @@ -116,9 +118,9 @@ take_snapshot_crash(Config) -> SnapDir = ?config(snap_dir, Config), State0 = init_state(Config), Meta = meta(55, 2, [node()]), - MacRef = ?FUNCTION_NAME, + MacState = ?FUNCTION_NAME, {State1, [{bg_work, _Fun, ErrFun}]} = - ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, State0), + ra_snapshot:begin_snapshot(Meta, ?MACMOD, MacState, snapshot, State0), ErrFun({error, blah}), undefined = ra_snapshot:current(State1), {{55, 2}, snapshot} = ra_snapshot:pending(State1), @@ -146,7 +148,7 @@ init_recover(Config) -> State0 = init_state(Config), Meta = meta(55, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot, State0), + ra_snapshot:begin_snapshot(Meta, ?MACMOD, ?FUNCTION_NAME, snapshot, State0), Fun(), receive {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> @@ -173,7 +175,7 @@ init_recover_voter_status(Config) -> State0 = init_state(Config), Meta = meta(55, 2, #{node() => #{voter_status => test}}), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot, State0), + ra_snapshot:begin_snapshot(Meta, ?MACMOD, ?FUNCTION_NAME, snapshot, State0), Fun(), receive {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> @@ -201,15 +203,16 @@ init_multi(Config) -> Meta1 = meta(55, 2, [node()]), Meta2 = meta(165, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta1, ?FUNCTION_NAME, snapshot, State0), + 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, [{bg_work, Fun2, _}]} = - ra_snapshot:begin_snapshot(Meta2, ?FUNCTION_NAME, snapshot, - State2), + 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), @@ -244,13 +247,15 @@ init_recover_multi_corrupt(Config) -> Meta1 = meta(55, 2, [node()]), Meta2 = meta(165, 2, [node()]), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta1, ?FUNCTION_NAME, snapshot, State0), + 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, [{bg_work, Fun2, _}]} = - ra_snapshot:begin_snapshot(Meta2, ?FUNCTION_NAME, snapshot, State2), + 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), @@ -291,7 +296,8 @@ init_recover_corrupt(Config) -> SnapsDir = ?config(snap_dir, Config), State0 = init_state(Config), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta, ?FUNCTION_NAME, snapshot, State0), + ra_snapshot:begin_snapshot(Meta, ?MACMOD, ?FUNCTION_NAME, + snapshot, State0), Fun(), _ = receive {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> @@ -322,7 +328,7 @@ read_snapshot(Config) -> Meta = meta(55, 2, [node()]), MacRef = crypto:strong_rand_bytes(1024 * 4), {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(Meta, MacRef, snapshot, State0), + ra_snapshot:begin_snapshot(Meta, ?MACMOD, MacRef, snapshot, State0), Fun(), State = receive {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> @@ -412,7 +418,7 @@ accept_receives_snapshot_written_with_higher_index(Config) -> MetaRemoteBin = term_to_binary(MetaHigh), %% begin a local snapshot {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(MetaLow, ?FUNCTION_NAME, snapshot, State0), + ra_snapshot:begin_snapshot(MetaLow, ?MACMOD, ?FUNCTION_NAME, snapshot, State0), Fun(), MacRef = crypto:strong_rand_bytes(1024), MacBin = term_to_binary(MacRef), @@ -452,7 +458,8 @@ accept_receives_snapshot_written_with_higher_index_2(Config) -> MetaHigh = meta(165, 2, [node()]), %% begin a local snapshot {State1, [{bg_work, Fun, _}]} = - ra_snapshot:begin_snapshot(MetaLow, ?FUNCTION_NAME, snapshot, State0), + ra_snapshot:begin_snapshot(MetaLow, ?MACMOD, ?FUNCTION_NAME, + snapshot, State0), Fun(), MacRef = crypto:strong_rand_bytes(1024), MacBin = term_to_binary(MacRef), @@ -496,3 +503,7 @@ meta(Idx, Term, Cluster) -> term => Term, cluster => Cluster, machine_version => 1}. + +%% ra_machine fakes +version() -> 1. +live_indexes(_) -> []. From 2ae3c5df9f2d0602474865b0bd780ae53bc7f5d0 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 11:57:22 +0000 Subject: [PATCH 07/22] WAL: remove write strategies. These have not proved useful o_sync perform very badly and sync_after_notify only provides benefit when there is hardly any activity. It is better to settle for a more predictable mode like the default write + sync provides. --- README.md | 14 -------- src/ra.erl | 3 +- src/ra_log_sup.erl | 2 -- src/ra_log_wal.erl | 70 ++++++--------------------------------- src/ra_system.erl | 3 -- test/ra_log_wal_SUITE.erl | 30 +++++++++-------- 6 files changed, 28 insertions(+), 94 deletions(-) diff --git a/README.md b/README.md index 5b747a26..5ecbdff2 100644 --- a/README.md +++ b/README.md @@ -348,20 +348,6 @@ is available in a separate repository. Indicate whether the wal should compute and validate checksums. Default: `true` Boolean - - wal_write_strategy - -
    -
  • - default: used by default. write(2) system calls are delayed until a buffer is due to be flushed. Then it writes all the data in a single call then fsyncs. Fastest option but incurs some additional memory use. -
  • -
  • - o_sync: Like default but will try to open the file with O_SYNC and thus won't need the additional fsync(2) system call. If it fails to open the file with this flag this mode falls back to default. -
  • -
- - Enumeration: default | o_sync - wal_sync_method diff --git a/src/ra.erl b/src/ra.erl index 0e6d2de3..ebf3d50f 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -86,8 +86,7 @@ {wal_data_dir, file:filename()} | {segment_max_entries, non_neg_integer()} | {wal_max_size_bytes, non_neg_integer()} | - {wal_compute_checksums, boolean()} | - {wal_write_strategy, default | o_sync}. + {wal_compute_checksums, boolean()}. -type query_fun() :: fun((term()) -> term()) | {M :: module(), F :: atom(), A :: list()}. diff --git a/src/ra_log_sup.erl b/src/ra_log_sup.erl index b2850a2f..720f423c 100644 --- a/src/ra_log_sup.erl +++ b/src/ra_log_sup.erl @@ -69,7 +69,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), @@ -82,7 +81,6 @@ make_wal_conf(#{data_dir := DataDir, 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 e5235866..cd2d7163 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -67,16 +67,6 @@ 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()}}. @@ -86,7 +76,6 @@ 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(), @@ -139,7 +128,6 @@ segment_writer => atom() | pid(), compute_checksums => boolean(), pre_allocate => boolean(), - write_strategy => wal_write_strategy(), sync_method => sync | datasync, recovery_chunk_size => non_neg_integer(), hibernate_after => non_neg_integer(), @@ -149,8 +137,7 @@ 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()}. @@ -261,7 +248,6 @@ init(#{dir := Dir} = Conf0) -> segment_writer := SegWriter, compute_checksums := ComputeChecksums, pre_allocate := PreAllocate, - write_strategy := WriteStrategy, sync_method := SyncMethod, garbage_collect := Gc, min_heap_size := MinHeapSize, @@ -285,7 +271,6 @@ init(#{dir := Dir} = Conf0) -> max_size_bytes = max(?WAL_MIN_SIZE, MaxWalSize), max_entries = MaxEntries, recovery_chunk_size = RecoveryChunkSize, - write_strategy = WriteStrategy, sync_method = SyncMethod, counter = CRef, mem_tables_tid = ets:whereis(MemTablesName), @@ -299,7 +284,9 @@ init(#{dir := Dir} = Conf0) -> % generated during recovery ok = ra_log_segment_writer:await(SegWriter), {ok, Result} - catch _:Err:_Stack -> + catch _:Err:Stack -> + ?ERROR("WAL: failed to initialise with ~p, stack ~p", + [Err, Stack]), {stop, Err} end. @@ -322,16 +309,14 @@ terminate(Reason, State) -> _ = 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), @@ -595,20 +580,6 @@ roll_over(#state{wal = Wal0, file_num = Num0, wal = Wal, file_num = Num}. -open_wal(File, Max, #conf{write_strategy = o_sync} = Conf) -> - Modes = [sync | ?FILE_MODES], - case prepare_file(File, Modes) of - {ok, Fd} -> - % many platforms implement O_SYNC a bit like O_DSYNC - % perform a manual sync here to ensure metadata is flushed - {Conf, #wal{fd = Fd, - max_size = Max, - filename = File}}; - {error, enotsup} -> - ?WARN("wal: o_sync write strategy not supported. " - "Reverting back to default strategy.", []), - open_wal(File, Max, Conf#conf{write_strategy = default}) - end; open_wal(File, Max, #conf{} = Conf0) -> {ok, Fd} = prepare_file(File, ?FILE_MODES), Conf = maybe_pre_allocate(Conf0, Fd, Max), @@ -637,9 +608,7 @@ 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{pre_allocate = true} = Conf, Fd, Max0) -> Max = Max0 - ?HEADER_SIZE, case file:allocate(Fd, ?HEADER_SIZE, Max) of ok -> @@ -654,7 +623,7 @@ maybe_pre_allocate(#conf{pre_allocate = true, " falling back to fsync instead of fdatasync", []), Conf#conf{pre_allocate = false} end; -maybe_pre_allocate(Conf, _Fd, _Max) -> +maybe_pre_allocate(Conf, _Fd, _Max0) -> Conf. close_file(undefined) -> @@ -666,26 +635,11 @@ start_batch(#state{conf = #conf{counter = CRef}} = State) -> ok = counters:add(CRef, ?C_BATCHES, 1), State#state{batch = #batch{}}. - -post_notify_flush(#state{wal = #wal{fd = Fd}, - conf = #conf{write_strategy = sync_after_notify, - sync_method = SyncMeth}}) -> - sync(Fd, SyncMeth); -post_notify_flush(_State) -> - ok. - flush_pending(#state{wal = #wal{fd = Fd}, batch = #batch{pending = Pend}, - conf = #conf{write_strategy = WriteStrategy, - sync_method = SyncMeth}} = State0) -> - - case WriteStrategy of - default -> - ok = file:write(Fd, Pend), - sync(Fd, SyncMeth); - _ -> - ok = file:write(Fd, Pend) - end, + conf = #conf{sync_method = SyncMeth}} = State0) -> + ok = file:write(Fd, Pend), + sync(Fd, SyncMeth), State0#state{batch = undefined}. sync(_Fd, none) -> @@ -709,7 +663,6 @@ 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, @@ -955,7 +908,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, diff --git a/src/ra_system.erl b/src/ra_system.erl index 303b8480..6b2080ca 100644 --- a/src/ra_system.erl +++ b/src/ra_system.erl @@ -36,7 +36,6 @@ wal_compute_checksums => boolean(), wal_max_batch_size => non_neg_integer(), wal_max_entries => undefined | non_neg_integer(), - wal_write_strategy => default | o_sync | sync_after_notify, wal_sync_method => datasync | sync | none, wal_hibernate_after => non_neg_integer(), wal_garbage_collect => boolean(), @@ -92,7 +91,6 @@ default_config() -> WalMaxBatchSize = application:get_env(ra, wal_max_batch_size, ?WAL_DEFAULT_MAX_BATCH_SIZE), WalMaxEntries = application:get_env(ra, wal_max_entries, undefined), - WalWriteStrategy = application:get_env(ra, wal_write_strategy, default), WalSyncMethod = application:get_env(ra, wal_sync_method, datasync), DataDir = ra_env:data_dir(), WalDataDir = application:get_env(ra, wal_data_dir, DataDir), @@ -127,7 +125,6 @@ default_config() -> wal_compute_checksums => WalComputeChecksums, wal_max_batch_size => WalMaxBatchSize, wal_max_entries => WalMaxEntries, - wal_write_strategy => WalWriteStrategy, wal_garbage_collect => WalGarbageCollect, wal_pre_allocate => WalPreAllocate, wal_sync_method => WalSyncMethod, diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index a538a52a..93448acb 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} ]. @@ -64,8 +62,6 @@ groups() -> {default, [], all_tests()}, %% uses fsync instead of the default fdatasync {fsync, [], all_tests()}, - {o_sync, [], all_tests()}, - {sync_after_notify, [], all_tests()}, {no_sync, [], all_tests()} ]. @@ -83,16 +79,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 +97,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), @@ -114,7 +109,6 @@ init_per_testcase(TestCase, Config) -> WalConf = #{dir => Dir, name => ra_log_wal, names => Names, - write_strategy => G, max_size_bytes => ?MAX_SIZE_BYTES}, _ = ets:new(ra_log_snapshot_state, [named_table, public, {write_concurrency, true}]), @@ -758,8 +752,16 @@ sys_get_status(Config) -> Conf = ?config(wal_conf, Config), {_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. From c511f4d49bcd392f9294f0f669c95f70decaf655 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 12:59:59 +0000 Subject: [PATCH 08/22] WAL: refactor config to reduce duplication. In particular the segment writer and the wal name was provided twice, once in the wal conf and once in the system names map. Also improve WAL log messages to include the system name. --- src/ra_log_sup.erl | 10 ++--- src/ra_log_wal.erl | 82 ++++++++++++++++++--------------- test/ra_log_wal_SUITE.erl | 95 ++++++++++++++++++--------------------- 3 files changed, 94 insertions(+), 93 deletions(-) diff --git a/src/ra_log_sup.erl b/src/ra_log_sup.erl index 720f423c..117febdb 100644 --- a/src/ra_log_sup.erl +++ b/src/ra_log_sup.erl @@ -56,9 +56,8 @@ init([#{data_dir := DataDir, make_wal_conf(#{data_dir := DataDir, - name := _System, - names := #{wal := WalName, - segment_writer := SegWriterName} = Names} = Cfg) -> + name := System, + names := #{} = Names} = Cfg) -> WalDir = case Cfg of #{wal_data_dir := D} -> D; _ -> DataDir @@ -76,10 +75,9 @@ make_wal_conf(#{data_dir := DataDir, MinBinVheapSize = maps:get(wal_min_bin_vheap_size, Cfg, ?MIN_BIN_VHEAP_SIZE), MinHeapSize = maps:get(wal_min_heap_size, Cfg, ?MIN_HEAP_SIZE), - #{name => WalName, - names => Names, + #{names => Names, + system => System, dir => WalDir, - segment_writer => SegWriterName, compute_checksums => ComputeChecksums, max_size_bytes => MaxSizeBytes, max_entries => MaxEntries, diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index cd2d7163..19e9f425 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -71,6 +71,7 @@ #{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(), @@ -120,12 +121,11 @@ }). -type state() :: #state{}. --type wal_conf() :: #{name := atom(), %% the name to register the wal as - names := ra_system:names(), +-type wal_conf() :: #{names := ra_system:names(), + system := atom(), dir := file:filename_all(), max_size_bytes => non_neg_integer(), max_entries => non_neg_integer(), - segment_writer => atom() | pid(), compute_checksums => boolean(), pre_allocate => boolean(), sync_method => sync | datasync, @@ -222,7 +222,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), @@ -241,11 +243,11 @@ start_link(#{name := Name} = Config) -spec init(wal_conf()) -> {ok, state()} | {stop, wal_checksum_validation_failure} | {stop, term()}. -init(#{dir := Dir} = Conf0) -> +init(#{system := System, + dir := Dir} = Conf0) -> #{max_size_bytes := MaxWalSize, max_entries := MaxEntries, recovery_chunk_size := RecoveryChunkSize, - segment_writer := SegWriter, compute_checksums := ComputeChecksums, pre_allocate := PreAllocate, sync_method := SyncMethod, @@ -253,10 +255,10 @@ init(#{dir := Dir} = Conf0) -> 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 @@ -266,6 +268,7 @@ init(#{dir := Dir} = Conf0) -> process_flag(min_heap_size, MinHeapSize), CRef = ra_counters:new(WalName, ?COUNTER_FIELDS), Conf = #conf{dir = Dir, + system = System, segment_writer = SegWriter, compute_checksums = ComputeChecksums, max_size_bytes = max(?WAL_MIN_SIZE, MaxWalSize), @@ -285,8 +288,8 @@ init(#{dir := Dir} = Conf0) -> ok = ra_log_segment_writer:await(SegWriter), {ok, Result} catch _:Err:Stack -> - ?ERROR("WAL: failed to initialise with ~p, stack ~p", - [Err, Stack]), + ?ERROR("WAL in ~ts failed to initialise with ~p, stack ~p", + [System, Err, Stack]), {stop, Err} end. @@ -304,8 +307,9 @@ 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. @@ -337,7 +341,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), @@ -363,17 +368,18 @@ recover_wal(Dir, #conf{segment_writer = SegWriter, WalFiles = lists:sort(Files), AllWriters = [begin - ?DEBUG("wal: recovering ~ts, Mode ~s", [F, Mode]), + ?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, Mode) end), + 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)]), + ?DEBUG("WAL in ~ts: recovered ~ts time taken ~bms - recovered ~b writers", + [System, F, Time div 1000, map_size(Writers)]), Writers end || F <- WalFiles], @@ -381,7 +387,8 @@ recover_wal(Dir, #conf{segment_writer = SegWriter, maps:merge(Acc, New) end, #{}, AllWriters), - ?DEBUG("wal: recovered ~b writers", [map_size(FinalWriters)]), + ?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, @@ -492,9 +499,9 @@ handle_msg({append, {UId, Pid} = Id, MtTid, Idx, Term, Entry}, {ok, {in_seq, PrevIdx}} -> % writer was in seq but has sent an out of seq entry % notify writer - ?DEBUG("WAL: requesting resend from `~w`, " + ?DEBUG("WAL in ~ts: requesting resend from `~w`, " "last idx ~b idx received ~b", - [UId, PrevIdx, Idx]), + [Conf#conf.system, UId, PrevIdx, Idx]), Pid ! {ra_log_event, {resend_write, PrevIdx + 1}}, State0#state{writers = Writers#{UId => {out_of_seq, PrevIdx}}} end; @@ -549,13 +556,14 @@ 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 @@ -608,7 +616,8 @@ make_tmp(File) -> ok = file:close(Fd), Tmp. -maybe_pre_allocate(#conf{pre_allocate = true} = Conf, Fd, Max0) -> +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 -> @@ -619,8 +628,9 @@ maybe_pre_allocate(#conf{pre_allocate = true} = Conf, Fd, Max0) -> {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, _Max0) -> @@ -792,11 +802,12 @@ recover_records(#conf{names = Names} = Conf, Fd, recover_records(Conf, Fd, Rest, Cache, State0#recovery{writers = 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 -> @@ -827,11 +838,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; _ -> @@ -860,20 +872,20 @@ recover_snap_idx(Conf, UId, Trunc, CurIdx) -> snap_idx(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. diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index 93448acb..4f08c3c9 100644 --- a/test/ra_log_wal_SUITE.erl +++ b/test/ra_log_wal_SUITE.erl @@ -107,8 +107,8 @@ init_per_testcase(TestCase, Config) -> TestCase, TestCase), Names = maps:get(names, Sys), WalConf = #{dir => Dir, - name => ra_log_wal, - names => Names, + 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}]), @@ -130,7 +130,7 @@ 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, _} = ra_log_wal:write(Pid, WriterId, Tid, 13, 1, "value2"), @@ -142,7 +142,7 @@ basic_log_writes(Config) -> 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(), @@ -155,7 +155,7 @@ 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, _} = ra_log_wal:write(Pid, WriterId, Tid, 13, 1, "value2"), @@ -168,7 +168,7 @@ wal_filename_upgrade(Config) -> % 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"}} -> @@ -187,7 +187,7 @@ 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}), Self = self(), @@ -220,7 +220,7 @@ 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) @@ -251,7 +251,7 @@ 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) @@ -281,7 +281,7 @@ 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) @@ -314,7 +314,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), @@ -352,7 +352,7 @@ 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, []), @@ -394,7 +394,7 @@ 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, []), @@ -424,7 +424,7 @@ 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, []), @@ -479,7 +479,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), @@ -533,7 +534,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), @@ -596,7 +598,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), @@ -656,8 +659,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), @@ -691,8 +693,7 @@ 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, []), @@ -723,8 +724,7 @@ 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, []), @@ -749,7 +749,8 @@ 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), @@ -767,9 +768,8 @@ sys_get_status(Config) -> 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), @@ -830,9 +830,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), @@ -866,9 +865,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), @@ -909,9 +907,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), @@ -960,9 +957,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), @@ -1008,9 +1004,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), @@ -1047,9 +1042,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), @@ -1103,7 +1097,7 @@ 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) @@ -1125,7 +1119,7 @@ recover_implicit_truncate(Config) -> % debugger:start(), % int:i(ra_log_wal), % int:break(ra_log_wal, 900), - {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 := 2}, ra_mt:info(Mt)), @@ -1148,7 +1142,7 @@ 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, []), @@ -1164,7 +1158,7 @@ recover_delete_uid(Config) -> 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)), @@ -1187,8 +1181,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), @@ -1201,9 +1194,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), @@ -1245,8 +1237,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), @@ -1276,8 +1267,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), @@ -1309,8 +1299,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), @@ -1435,3 +1424,5 @@ suspend_process(Pid) -> end end. +set_segment_writer(#{names := Names} = Conf, Writer) -> + Conf#{names => maps:put(segment_writer, Writer, Names)}. From e21635c31e1ffeabfdf016a2c5dd0b574042858d Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 3 Mar 2025 08:10:38 +0000 Subject: [PATCH 09/22] Extend ra_log_snapshot_state to support live indexes. Instead of working off the snapshot index to reduce writes to WAL and segments there is a new "smallest index" that needs to be populated. For normal snapshotting machines this will be snapshot index + 1 but when a machine reportst live indexes it will be the smallest of those. --- src/ra_directory.erl | 5 +- src/ra_log.erl | 2 +- src/ra_log_segment_writer.erl | 17 ++--- src/ra_log_snapshot_state.erl | 50 +++++++++++++++ src/ra_log_wal.erl | 96 ++++++++++++++-------------- src/ra_seq.erl | 9 +++ src/ra_snapshot.erl | 18 ++++-- test/ra_log_2_SUITE.erl | 5 +- test/ra_log_segment_writer_SUITE.erl | 6 +- test/ra_log_wal_SUITE.erl | 22 ++++--- 10 files changed, 150 insertions(+), 80 deletions(-) create mode 100644 src/ra_log_snapshot_state.erl create mode 100644 src/ra_seq.erl diff --git a/src/ra_directory.erl b/src/ra_directory.erl index caf81ecf..bdb47c89 100644 --- a/src/ra_directory.erl +++ b/src/ra_directory.erl @@ -191,7 +191,10 @@ overview(System) when is_atom(System) -> end, ets:tab2list(ra_state)), States = maps:from_list(Rows), - Snaps = maps:from_list(ets:tab2list(ra_log_snapshot_state)), + Snaps = lists:foldl( + fun (T, Acc) -> + Acc#{element(1, T) => erlang:delete_element(1, T)} + end, #{}, ets:tab2list(ra_log_snapshot_state)), lists:foldl(fun ({UId, Pid, Parent, ServerName, ClusterName}, Acc) -> {S, V} = maps:get(ServerName, States, {undefined, undefined}), Acc#{ServerName => diff --git a/src/ra_log.erl b/src/ra_log.erl index 191525fe..2d9fba73 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -1098,7 +1098,7 @@ delete_everything(#?MODULE{cfg = #cfg{uid = UId, %% 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), + catch ra_log_snapshot_state:delete(ra_log_snapshot_state, UId), try ra_lib:recursive_delete(Dir) of ok -> ok catch diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index 7e94a778..2db94670 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -263,7 +263,7 @@ get_overview(#state{data_dir = Dir, flush_mem_table_ranges({ServerUId, TidRanges0}, #state{system = System} = State) -> - SnapIdx = snap_idx(ServerUId), + SmallestIdx = smallest_live_idx(ServerUId), %% TidRanges arrive here sorted new -> old. %% truncate and limit all ranges to create a contiguous non-overlapping @@ -272,14 +272,14 @@ flush_mem_table_ranges({ServerUId, TidRanges0}, %% processing TidRanges = lists:foldl( fun ({T, Range0}, []) -> - case ra_range:truncate(SnapIdx, Range0) of + case ra_range:truncate(SmallestIdx - 1, Range0) of undefined -> []; Range -> [{T, Range}] end; ({T, Range0}, [{_T, {Start, _}} | _] = Acc) -> - Range1 = ra_range:truncate(SnapIdx, Range0), + Range1 = ra_range:truncate(SmallestIdx - 1, Range0), case ra_range:limit(Start, Range1) of undefined -> Acc; @@ -352,15 +352,10 @@ flush_mem_table_range(ServerUId, {Tid, {StartIdx0, EndIdx}}, end. start_index(ServerUId, StartIdx0) -> - max(snap_idx(ServerUId) + 1, StartIdx0). + max(smallest_live_idx(ServerUId), StartIdx0). -snap_idx(ServerUId) -> - case ets:lookup(ra_log_snapshot_state, ServerUId) of - [{_, SnapIdx}] -> - SnapIdx; - _ -> - -1 - end. +smallest_live_idx(ServerUId) -> + ra_log_snapshot_state:smallest(ra_log_snapshot_state, ServerUId). send_segments(System, ServerUId, TidRanges, SegRefs) -> case ra_directory:pid_of(System, ServerUId) of diff --git a/src/ra_log_snapshot_state.erl b/src/ra_log_snapshot_state.erl new file mode 100644 index 00000000..2f3e5f58 --- /dev/null +++ b/src/ra_log_snapshot_state.erl @@ -0,0 +1,50 @@ +-module(ra_log_snapshot_state). + +-export([ + insert/5, + delete/2, + smallest/2, + snapshot/2 + ]). + +-spec insert(ets:table(), ra:uid(), -1 | ra:index(), ra:index(), ra_seq:state()) -> + ok. +insert(Table, UId, SnapIdx, SmallestIdx, LiveIndexes) + when is_binary(UId) andalso + is_integer(SnapIdx) andalso + is_integer(SmallestIdx) andalso + is_list(LiveIndexes) -> + true = ets:insert(Table, {UId, SnapIdx, SmallestIdx, LiveIndexes}), + ok. + +delete(Table, UId) -> + true = ets:delete(Table, UId), + ok. + +-spec smallest(ets:table(), ra:uid()) -> + ra:index(). +smallest(Table, UId) when is_binary(UId) -> + ets:lookup_element(Table, UId, 3, 0). + +-spec snapshot(ets:table(), ra:uid()) -> + ra:index() | -1. +snapshot(Table, UId) when is_binary(UId) -> + ets:lookup_element(Table, UId, 2, -1). + +%%% =================== +%%% Internal unit tests +%%% =================== + +-ifdef(TEST). +-include_lib("eunit/include/eunit.hrl"). + +basics_test() -> + + UId = atom_to_binary(?FUNCTION_NAME, utf8), + T = ets:new(?FUNCTION_NAME, [set]), + ok = insert(T, UId, 50, 51, []), + ?assertEqual(51, smallest(T, UId)), + ?assertEqual(50, snapshot(T, UId)), + ok. + +-endif. diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index 19e9f425..1471bb76 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -54,7 +54,7 @@ % 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(), @@ -140,7 +140,7 @@ -export_type([wal_conf/0]). -type wal_command() :: - {append | truncate, writer_id(), ra_index(), ra_term(), term()}. + {append, writer_id(), ra_index(), ra_term(), term()}. -type wal_op() :: {cast, wal_command()} | {call, from(), wal_command()}. @@ -428,7 +428,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, @@ -441,7 +441,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} -> @@ -466,7 +466,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, @@ -479,19 +479,22 @@ write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SnapIdx, handle_msg({append, {UId, Pid} = Id, MtTid, 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, + Trunc = Idx == SmallestIdx, + 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}}}; + _ when Idx < SmallestIdx -> + %% the smallest live index for the last snapshot is higher than + %% this index, just drop it + PrevIdx = SmallestIdx - 1, + State0#state{writers = Writers#{UId => {in_seq, PrevIdx}}}; {ok, {_, PrevIdx}} when Idx =< PrevIdx + 1 orelse Trunc -> - write_data(Id, MtTid, Idx, Term, Entry, Trunc, SnapIdx, State0); + write_data(Id, MtTid, Idx, Term, Entry, Trunc, SmallestIdx, State0); error -> - write_data(Id, MtTid, Idx, Term, Entry, false, SnapIdx, State0); + write_data(Id, MtTid, Idx, Term, Entry, false, SmallestIdx, State0); {ok, {out_of_seq, _}} -> % writer is out of seq simply ignore drop the write % TODO: capture metric for dropped writes? @@ -505,10 +508,6 @@ handle_msg({append, {UId, Pid} = Id, MtTid, Idx, Term, Entry}, 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), @@ -520,23 +519,24 @@ 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, SmallestIdx) -> Waiting = case Waiting0 of #{Pid := #batch_writer{term = TERM, tid = MT_TID, range = Range0 } = W} -> %% The Tid and term is the same so add to current batch_writer - Range = ra_range:extend(Idx, ra_range:truncate(SnapIdx, Range0)), + Range = ra_range:extend(Idx, ra_range:truncate(SmallestIdx - 1, + Range0)), Waiting0#{Pid => W#batch_writer{range = Range, - snap_idx = SnapIdx, + smallest_live_idx = SmallestIdx, 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 = SmallestIdx, tid = MtTid, range = ra_range:new(Idx), uid = UId, @@ -675,14 +675,14 @@ complete_batch(#state{batch = #batch{waiting = Waiting, end, Wal#wal.ranges, Waiting), State#state{wal = Wal#wal{ranges = Ranges}}. -complete_batch_writer(Pid, #batch_writer{snap_idx = SnapIdx, +complete_batch_writer(Pid, #batch_writer{smallest_live_idx = SmallestIdx, 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); + update_ranges(Ranges, UId, MtTid, SmallestIdx, Range); complete_batch_writer(Pid, #batch_writer{old = #batch_writer{} = OldBw} = Bw, Ranges0) -> Ranges = complete_batch_writer(Pid, OldBw, Ranges0), @@ -778,13 +778,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} -> @@ -793,12 +793,14 @@ 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, + W = State0#recovery.writers, + Writers = W#{UId => {in_seq, SmallestIdx - 1}}, recover_records(Conf, Fd, Rest, Cache, State0#recovery{writers = Writers}); error -> @@ -823,13 +825,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} -> @@ -864,12 +866,12 @@ 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>>, _) -> @@ -946,15 +948,15 @@ 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, {Start, _} = AddRange) -> case Ranges of #{UId := [{MtTid, Range0} | Rem]} -> - %% SnapIdx might have moved to we truncate the old range first + %% SmallestIdx might have moved to we truncate the old range first %% before extending - Range1 = ra_range:truncate(SnapIdx, Range0), + Range1 = ra_range:truncate(SmallestIdx - 1, Range0), %% 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)), @@ -962,13 +964,13 @@ update_ranges(Ranges, UId, MtTid, SnapIdx, {Start, _} = AddRange) -> #{UId := [{OldMtTid, OldMtRange} | Rem]} -> %% new Tid, need to add a new range record for this Ranges#{UId => [{MtTid, AddRange}, - ra_range:truncate(SnapIdx, {OldMtTid, OldMtRange}) + ra_range:truncate(SmallestIdx - 1, {OldMtTid, OldMtRange}) | Rem]}; _ -> Ranges#{UId => [{MtTid, AddRange}]} end. -recover_entry(Names, UId, {Idx, _, _} = Entry, SnapIdx, +recover_entry(Names, UId, {Idx, _, _} = Entry, SmallestIdx, #recovery{mode = initial, ranges = Ranges0, writers = Writers, @@ -982,7 +984,7 @@ recover_entry(Names, UId, {Idx, _, _} = Entry, SnapIdx, case ra_mt:insert(Entry, Mt0) of {ok, Mt1} -> Ranges = update_ranges(Ranges0, UId, ra_mt:tid(Mt1), - SnapIdx, ra_range:new(Idx)), + SmallestIdx, ra_range:new(Idx)), {ok, State#recovery{ranges = Ranges, writers = Writers#{UId => {in_seq, Idx}}, tables = Tables#{UId => Mt1}}}; @@ -991,7 +993,7 @@ recover_entry(Names, UId, {Idx, _, _} = Entry, SnapIdx, {ok, Mt1} = ra_log_ets:new_mem_table_please(Names, UId, Mt0), {retry, State#recovery{tables = Tables#{UId => Mt1}}} end; -recover_entry(Names, UId, {Idx, Term, _}, SnapIdx, +recover_entry(Names, UId, {Idx, Term, _}, SmallestIdx, #recovery{mode = post_boot, ranges = Ranges0, writers = Writers, @@ -1014,7 +1016,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, ra_range:new(Idx)), {ok, State#recovery{ranges = Ranges, writers = Writers#{UId => {in_seq, Idx}}, tables = Tables#{UId => Mt0}}} diff --git a/src/ra_seq.erl b/src/ra_seq.erl new file mode 100644 index 00000000..f7007fba --- /dev/null +++ b/src/ra_seq.erl @@ -0,0 +1,9 @@ +-module(ra_seq). + +%% open type +-type state() :: [ra:index() | ra:range()]. + +-export_type([state/0]). + + + diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index 2ff3ef43..613d239c 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -193,12 +193,13 @@ find_snapshots(#?MODULE{uid = UId, ok = delete_snapshots(SnapshotsDir, Snaps), %% initialise snapshots table even if no snapshots have been taken %% this ensure these is an entry when the WAL queries it - true = ets:insert(?ETSTBL, {UId, -1}), + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, -1, 0, []), State; Current0 -> Current = filename:join(SnapshotsDir, Current0), {ok, #{index := Idx, term := Term}} = Module:read_meta(Current), - true = ets:insert(?ETSTBL, {UId, Idx}), + %% TODO: recover live indexes and record that + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, Idx+1, []), ok = delete_snapshots(SnapshotsDir, lists:delete(Current0, Snaps)), %% delete old snapshots if any @@ -346,8 +347,8 @@ 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 @@ -473,7 +474,8 @@ complete_snapshot(_IdxTerm, snapshot, State; complete_snapshot({Idx, _} = IdxTerm, snapshot, #?MODULE{uid = UId} = State) -> - true = ets:insert(?ETSTBL, {UId, Idx}), + %% TODO live indexes + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, Idx+1, []), State#?MODULE{pending = undefined, current = IdxTerm}; complete_snapshot(IdxTerm, checkpoint, @@ -518,7 +520,11 @@ accept_chunk(Chunk, Num, last, fun (_) -> ok end}, %% update ets table - true = ets:insert(?ETSTBL, {UId, Idx}), + % true = ets:insert(?ETSTBL, {UId, Idx}), + + %% TODO: move this to install_snapshot so we can work out the + %% live indexes + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, Idx+1, []), {ok, State#?MODULE{accepting = undefined, %% reset any pending snapshot writes pending = undefined, diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 7a7b1145..ac2c3b60 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -1304,7 +1304,8 @@ update_release_cursor(Config) -> 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), @@ -1321,7 +1322,7 @@ update_release_cursor(Config) -> {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 diff --git a/test/ra_log_segment_writer_SUITE.erl b/test/ra_log_segment_writer_SUITE.erl index 5aed1caa..3a93317c 100644 --- a/test/ra_log_segment_writer_SUITE.erl +++ b/test/ra_log_segment_writer_SUITE.erl @@ -546,7 +546,7 @@ accept_mem_tables_multiple_ranges_snapshot(Config)-> {ra_mt:tid(Mt2), ra_mt:range(Mt2)}, {ra_mt:tid(Mt), ra_mt:range(Mt)} ]}, - ets:insert(ra_log_snapshot_state, {UId, 64}), + 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")), @@ -790,7 +790,7 @@ skip_entries_lower_than_snapshot_index(Config) -> Mt = make_mem_table(UId, Entries), 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 @@ -823,7 +823,7 @@ skip_all_entries_lower_than_snapshot_index(Config) -> Mt = make_mem_table(UId, Entries), Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]}, %% update snapshot state table - ets:insert(ra_log_snapshot_state, {UId, 3}), + ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 3, 4, []), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), TIDRANGES = maps:get(UId, Ranges), diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index 4f08c3c9..bc9d07c5 100644 --- a/test/ra_log_wal_SUITE.erl +++ b/test/ra_log_wal_SUITE.erl @@ -258,7 +258,7 @@ writes_snapshot_idx_overtakes(Config) -> || I <- lists:seq(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}), @@ -291,7 +291,7 @@ writes_implicit_truncate_write(Config) -> % 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}), @@ -323,10 +323,13 @@ 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), @@ -844,7 +847,8 @@ recover_with_snapshot_index(Config) -> 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), @@ -1107,7 +1111,7 @@ recover_implicit_truncate(Config) -> % 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}), @@ -1115,7 +1119,7 @@ recover_implicit_truncate(Config) -> ok = proc_lib:stop(Pid), %% this could happen potentially in some edge cases?? - ets:delete(ra_log_snapshot_state, UId), + ra_log_snapshot_state:delete(ra_log_snapshot_state, UId), % debugger:start(), % int:i(ra_log_wal), % int:break(ra_log_wal, 900), From 4c766b2892747b743f9bb071c4c9886a9e74f479 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 5 Mar 2025 10:56:58 +0000 Subject: [PATCH 10/22] WIP: WAL sparse writes --- docs/internals/COMPACTION.md | 6 +++-- src/ra_log.erl | 6 ++--- src/ra_log_segment_writer.erl | 18 +++++++-------- src/ra_log_wal.erl | 37 +++++++++++++++++++++++------- test/ra_log_wal_SUITE.erl | 42 ++++++++++++++++++++++++++++++++++- 5 files changed, 86 insertions(+), 23 deletions(-) diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md index 04a840b1..6a8af47c 100644 --- a/docs/internals/COMPACTION.md +++ b/docs/internals/COMPACTION.md @@ -243,7 +243,7 @@ the live index replication. NB the snapshot sender process may need to call into the leader process to get read plans as entries _could_ be in the memtable. #### How to work out which live indexes the follower needs - +WA Gnarly example: Follower term indexes: @@ -287,4 +287,6 @@ 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/src/ra_log.erl b/src/ra_log.erl index 2d9fba73..77f2b6ae 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -336,7 +336,7 @@ commit_tx(#?MODULE{cfg = #cfg{uid = UId, {WalCommands, Num} = lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N}) -> Cmd = {ttb, term_to_iovec(Cmd0)}, - WalC = {append, WriterId, Tid, Idx, Term, Cmd}, + WalC = {append, WriterId, Tid, Idx-1, Idx, Term, Cmd}, {[WalC | WC], N+1} end, {[], 0}, Entries), @@ -1176,11 +1176,11 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, {WalCommands, Num} = lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N}) -> Cmd = {ttb, term_to_iovec(Cmd0)}, - WalC = {append, WriterId, Tid, Idx, Term, Cmd}, + WalC = {append, WriterId, Tid, Idx-1, Idx, Term, Cmd}, {[WalC | WC], N+1} end, {[], 0}, Entries), - [{_, _, _, 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), diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index 2db94670..6d03fd58 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -66,14 +66,13 @@ start_link(#{name := Name} = Config) -> -spec accept_mem_tables(atom() | pid(), #{ra_uid() => [{ets:tid(), ra:range()}]}, - string()) -> - ok. + string()) -> ok. accept_mem_tables(_SegmentWriter, Tables, undefined) when map_size(Tables) == 0 -> ok; -accept_mem_tables(SegmentWriter, Tables, WalFile) - when is_map(Tables) -> - gen_server:cast(SegmentWriter, {mem_tables, Tables, WalFile}). +accept_mem_tables(SegmentWriter, UIdTidRanges, WalFile) + when is_map(UIdTidRanges) -> + gen_server:cast(SegmentWriter, {mem_tables, UIdTidRanges, WalFile}). -spec truncate_segments(atom() | pid(), ra_uid(), ra_log:segment_ref()) -> ok. truncate_segments(SegWriter, Who, SegRef) -> @@ -135,10 +134,11 @@ segments_for(UId, #state{data_dir = DataDir}) -> Dir = filename:join(DataDir, ra_lib:to_list(UId)), segment_files(Dir). -handle_cast({mem_tables, Ranges, WalFile}, #state{data_dir = Dir, - system = System} = State) -> +handle_cast({mem_tables, UIdTidRanges, WalFile}, + #state{data_dir = Dir, + system = System} = State) -> T1 = erlang:monotonic_time(), - ok = counters:add(State#state.counter, ?C_MEM_TABLES, map_size(Ranges)), + ok = counters:add(State#state.counter, ?C_MEM_TABLES, map_size(UIdTidRanges)), #{names := Names} = ra_system:fetch(System), Degree = erlang:system_info(schedulers), %% TODO: refactor to make better use of time where each uid has an @@ -157,7 +157,7 @@ handle_cast({mem_tables, Ranges, WalFile}, #state{data_dir = Dir, ok = ra_log_ets:delete_mem_tables(Names, UId), Acc end - end, [], Ranges), + end, [], UIdTidRanges), _ = [begin {_, Failures} = ra_lib:partition_parallel( diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index 1471bb76..d1b83b84 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]). @@ -140,7 +141,8 @@ -export_type([wal_conf/0]). -type wal_command() :: - {append, 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()}. @@ -149,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}. @@ -476,7 +491,7 @@ write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SmallestIndex, end. -handle_msg({append, {UId, Pid} = Id, MtTid, Idx, Term, Entry}, +handle_msg({append, {UId, Pid} = Id, MtTid, PrevIdx0, Idx, Term, Entry}, #state{conf = Conf, writers = Writers} = State0) -> SmallestIdx = smallest_live_index(Conf, UId), @@ -487,13 +502,17 @@ handle_msg({append, {UId, Pid} = Id, MtTid, Idx, Term, Entry}, _ when Idx < SmallestIdx -> %% the smallest live index for the last snapshot is higher than %% this index, just drop it - PrevIdx = SmallestIdx - 1, - State0#state{writers = Writers#{UId => {in_seq, PrevIdx}}}; + LastIdx = SmallestIdx - 1, + State0#state{writers = Writers#{UId => {in_seq, LastIdx}}}; {ok, {_, PrevIdx}} - when Idx =< PrevIdx + 1 orelse + when PrevIdx0 =< PrevIdx orelse Trunc -> + %% 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); error -> + %% no state for the UId is known so go ahead and write write_data(Id, MtTid, Idx, Term, Entry, false, SmallestIdx, State0); {ok, {out_of_seq, _}} -> % writer is out of seq simply ignore drop the write @@ -528,6 +547,8 @@ incr_batch(#batch{num_writes = Writes, %% The Tid and term is the same so add to current batch_writer Range = ra_range:extend(Idx, ra_range:truncate(SmallestIdx - 1, Range0)), + %% TODO: range nees to become a ra_seq so that we can + %% capture sparse writes correctly Waiting0#{Pid => W#batch_writer{range = Range, smallest_live_idx = SmallestIdx, term = Term diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index bc9d07c5..8c59d356 100644 --- a/test/ra_log_wal_SUITE.erl +++ b/test/ra_log_wal_SUITE.erl @@ -24,6 +24,7 @@ all() -> all_tests() -> [ basic_log_writes, + sparse_writes, wal_filename_upgrade, same_uid_different_process, consecutive_terms_in_batch_should_result_in_two_written_events, @@ -61,7 +62,8 @@ groups() -> [ {default, [], all_tests()}, %% uses fsync instead of the default fdatasync - {fsync, [], all_tests()}, + %% just testing that the configuration and dispatch works + {fsync, [], [basic_log_writes]}, {no_sync, [], all_tests()} ]. @@ -148,6 +150,44 @@ basic_log_writes(Config) -> 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, 12}), + debugger:start(), + int:i(ra_log_wal), + int:break(ra_log_wal, 975), + timer:sleep(1000), + {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 15, 1, "value2"), + timer:sleep(200000), + ok = await_written(WriterId, 1, {15, 15}), + ra_log_wal:force_roll_over(Pid), + receive + {'$gen_cast', + {mem_tables, #{UId := [{Tid, {12, 15}}]}, _}} -> + 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) -> + ct:fail("~s", [?FUNCTION_NAME]). + +sparse_write_recover(_Config) -> + ct:fail("~s", [?FUNCTION_NAME]). + +sparse_write_overwrite(_Config) -> + ct:fail("~s", [?FUNCTION_NAME]). + wal_filename_upgrade(Config) -> meck:new(ra_log_segment_writer, [passthrough]), meck:expect(ra_log_segment_writer, await, fun(_) -> ok end), From fff07a6dc4f3d64e7edc7ae7624701658a3bf814 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 17 Mar 2025 18:09:42 +0000 Subject: [PATCH 11/22] wip --- src/ra_log_wal.erl | 67 +++++++++++----------- src/ra_range.erl | 12 +++- src/ra_seq.erl | 117 ++++++++++++++++++++++++++++++++++++++ test/ra_log_wal_SUITE.erl | 10 ++-- test/ra_seq_SUITE.erl | 105 ++++++++++++++++++++++++++++++++++ 5 files changed, 271 insertions(+), 40 deletions(-) create mode 100644 test/ra_seq_SUITE.erl diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index d1b83b84..8dc64311 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -58,7 +58,7 @@ -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{} }). @@ -491,39 +491,39 @@ write_data({UId, Pid} = Id, MtTid, Idx, Term, Data0, Trunc, SmallestIndex, end. -handle_msg({append, {UId, Pid} = Id, MtTid, PrevIdx0, Idx, Term, Entry}, +handle_msg({append, {UId, Pid} = Id, MtTid, ExpectedPrevIdx, Idx, Term, Entry}, #state{conf = Conf, writers = Writers} = State0) -> SmallestIdx = smallest_live_index(Conf, UId), %% detect if truncating flag should be set Trunc = Idx == SmallestIdx, - case maps:find(UId, Writers) of + 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}}}; - {ok, {_, PrevIdx}} - when PrevIdx0 =< PrevIdx orelse + {_, PrevIdx} + when ExpectedPrevIdx =< PrevIdx orelse Trunc -> %% 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); - error -> + undefined -> %% no state for the UId is known so go ahead and write write_data(Id, MtTid, Idx, Term, Entry, false, SmallestIdx, State0); - {ok, {out_of_seq, _}} -> + {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 in ~ts: requesting resend from `~w`, " - "last idx ~b idx received ~b", - [Conf#conf.system, UId, PrevIdx, Idx]), + "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; @@ -538,32 +538,28 @@ incr_batch(#batch{num_writes = Writes, waiting = Waiting0, pending = Pend} = Batch, UId, Pid, MT_TID = MtTid, - Idx, TERM = Term, Data, SmallestIdx) -> + 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(SmallestIdx - 1, - Range0)), + 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{range = Range, - smallest_live_idx = SmallestIdx, - term = Term - }}; + 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{smallest_live_idx = SmallestIdx, + 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, @@ -699,7 +695,7 @@ complete_batch(#state{batch = #batch{waiting = Waiting, complete_batch_writer(Pid, #batch_writer{smallest_live_idx = SmallestIdx, tid = MtTid, uid = UId, - range = Range, + seq = Range, term = Term, old = undefined}, Ranges) -> Pid ! {ra_log_event, {written, Term, Range}}, @@ -972,23 +968,24 @@ should_roll_wal(#state{conf = #conf{max_entries = MaxEntries}, smallest_live_index(#conf{ra_log_snapshot_state_tid = Tid}, ServerUId) -> ra_log_snapshot_state:smallest(Tid, ServerUId). -update_ranges(Ranges, UId, MtTid, SmallestIdx, {Start, _} = AddRange) -> +update_ranges(Ranges, UId, MtTid, SmallestIdx, AddSeq) -> case Ranges of - #{UId := [{MtTid, Range0} | Rem]} -> + #{UId := [{MtTid, Seq0} | Rem]} -> %% SmallestIdx might have moved to we truncate the old range first %% before extending - Range1 = ra_range:truncate(SmallestIdx - 1, 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, Seq1), + Ranges#{UId => [{MtTid, Seq} | Rem]}; + #{UId := [{OldMtTid, OldMtSeq} | Rem]} -> %% new Tid, need to add a new range record for this - Ranges#{UId => [{MtTid, AddRange}, - ra_range:truncate(SmallestIdx - 1, {OldMtTid, OldMtRange}) + Ranges#{UId => [{MtTid, AddSeq}, + {OldMtTid, + ra_seq:floor(SmallestIdx, OldMtSeq)} | Rem]}; _ -> - Ranges#{UId => [{MtTid, AddRange}]} + Ranges#{UId => [{MtTid, AddSeq}]} end. recover_entry(Names, UId, {Idx, _, _} = Entry, SmallestIdx, @@ -1005,7 +1002,7 @@ recover_entry(Names, UId, {Idx, _, _} = Entry, SmallestIdx, case ra_mt:insert(Entry, Mt0) of {ok, Mt1} -> Ranges = update_ranges(Ranges0, UId, ra_mt:tid(Mt1), - SmallestIdx, ra_range:new(Idx)), + SmallestIdx, [Idx]), {ok, State#recovery{ranges = Ranges, writers = Writers#{UId => {in_seq, Idx}}, tables = Tables#{UId => Mt1}}}; @@ -1037,7 +1034,7 @@ recover_entry(Names, UId, {Idx, Term, _}, SmallestIdx, tables = Tables#{UId => Mt0}}}; Tid -> Ranges = update_ranges(Ranges0, UId, Tid, - SmallestIdx, ra_range:new(Idx)), + SmallestIdx, [Idx]), {ok, State#recovery{ranges = Ranges, writers = Writers#{UId => {in_seq, Idx}}, tables = Tables#{UId => Mt0}}} diff --git a/src/ra_range.erl b/src/ra_range.erl index 7bb53965..cb7f0d88 100644 --- a/src/ra_range.erl +++ b/src/ra_range.erl @@ -25,6 +25,13 @@ -export_type([range/0]). +-define(IS_RANGE(R), ((is_tuple(R) andalso + tuple_size(R) == 2 andalso + is_integer(element(1, R)) andalso + is_integer(element(2, R))) orelse + R == undefined)). + + -spec new(ra:index()) -> range(). new(Start) when is_integer(Start) -> {Start, Start}. @@ -71,14 +78,17 @@ limit(CeilExcl, Range) -spec truncate(ra:index(), range()) -> range(). truncate(UpToIncl, {_Start, End}) when is_integer(UpToIncl) andalso + is_integer(End) andalso UpToIncl >= End -> undefined; truncate(UpToIncl, {Start, End}) when is_integer(UpToIncl) andalso + is_integer(Start) andalso UpToIncl >= Start -> {UpToIncl + 1, End}; truncate(UpToIncl, Range) - when is_integer(UpToIncl) -> + when is_integer(UpToIncl) andalso + ?IS_RANGE(Range) -> Range. size(undefined) -> diff --git a/src/ra_seq.erl b/src/ra_seq.erl index f7007fba..ac243c0e 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -1,9 +1,126 @@ +%% 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()]. -export_type([state/0]). +-export([ + append/2, + from_list/1, + floor/2, + limit/2, + add/2, + fold/3 + ]). +-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)). + +-spec floor(ra:index(), state()) -> state(). +floor(FloorIdxIncl, Seq) -> + %% TODO: assert appendable + %% for now assume appendable + floor0(FloorIdxIncl, Seq, []). + + +limit(CeilIdx, [Last | Rem]) + when is_integer(Last) andalso + Last > CeilIdx -> + limit(CeilIdx, Rem); +limit(CeilIdx, [{_, _} = T | Rem]) -> + case ra_range:limit(CeilIdx + 1, T) of + undefined -> + limit(CeilIdx, Rem); + {I, I} -> + [I | Rem]; + {I, I2} when I == I2 - 1 -> + [I2, I | Rem]; + NewRange -> + [NewRange | Rem] + end; +limit(_CeilIdx, Seq) -> + Seq. + +-spec add(state(), state()) -> state(). +add([], Seq2) -> + Seq2; +add(Seq1, Seq2) -> + Fst = case lists:last(Seq1) of + {I, _} -> I; + I -> I + end, + fold(fun append/2, limit(Fst - 1, Seq2), Seq1). + + +-spec fold(fun ((ra:index(), Acc) -> Acc), Acc, state()) +-> Acc when Acc :: term(). +fold(Fun, Acc0, Seq) -> + %% TODO: factor out the lists:seq/2 + lists:foldr( + fun ({S, E}, Acc) -> + lists:foldl(Fun, Acc, lists:seq(S, E)); + (Idx, Acc) -> + Fun(Idx, Acc) + end, Acc0, Seq). + +%% internal functions + +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). + +% first_index(Seq) -> +% last_index(lists:reverse(Seq)). + +% last_index([{_, I} | _]) -> +% I; +% last_index([I | _]) +% when is_integer(I) -> +% I; +% last_index([]) -> +% undefined. diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index 8c59d356..fff0c8a1 100644 --- a/test/ra_log_wal_SUITE.erl +++ b/test/ra_log_wal_SUITE.erl @@ -159,12 +159,14 @@ sparse_writes(Config) -> {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, 12}), - debugger:start(), - int:i(ra_log_wal), - int:break(ra_log_wal, 975), + % debugger:start(), + % int:i(ra_log_wal), + % int:break(ra_log_wal, 975), timer:sleep(1000), + %% 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"), - timer:sleep(200000), + % timer:sleep(200000), ok = await_written(WriterId, 1, {15, 15}), ra_log_wal:force_roll_over(Pid), receive diff --git a/test/ra_seq_SUITE.erl b/test/ra_seq_SUITE.erl new file mode 100644 index 00000000..f4437fba --- /dev/null +++ b/test/ra_seq_SUITE.erl @@ -0,0 +1,105 @@ +-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 + ]. + +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. From cda00e9ab2dfdf1e2fbbd58f64201742abdb462b Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 18 Mar 2025 17:09:28 +0000 Subject: [PATCH 12/22] Sparse writes Most things work now. Instead of passing ranges around we now pass ra_seq instances which is a compacted sequence of items. --- src/ra_log.erl | 49 +++-- src/ra_log_segment_writer.erl | 98 +++++---- src/ra_log_wal.erl | 80 +++++--- src/ra_range.erl | 4 +- src/ra_seq.erl | 89 ++++++++- test/ra_log_2_SUITE.erl | 16 +- test/ra_log_SUITE.erl | 15 +- test/ra_log_props_SUITE.erl | 2 +- test/ra_log_segment_writer_SUITE.erl | 98 ++++----- test/ra_log_wal_SUITE.erl | 288 ++++++++++++++++----------- test/ra_seq_SUITE.erl | 26 ++- 11 files changed, 493 insertions(+), 272 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 77f2b6ae..e3a6211f 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -62,7 +62,7 @@ -type ra_meta_key() :: atom(). -type segment_ref() :: {ra_range:range(), File :: file:filename_all()}. --type event_body() :: {written, ra_term(), ra:range()} | +-type event_body() :: {written, ra_term(), ra_seq:state()} | {segments, [{ets:tid(), ra:range()}], [segment_ref()]} | {resend_write, ra_index()} | {snapshot_written, ra_idxterm(), ra_snapshot:kind()} | @@ -610,14 +610,14 @@ set_last_index(Idx, #?MODULE{cfg = Cfg, -spec handle_event(event_body(), state()) -> {state(), [effect()]}. -handle_event({written, _Term, {FromIdx, _ToIdx}}, +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, [{FromIdx, ToIdx}]}, #?MODULE{cfg = Cfg, last_written_index_term = {LastWrittenIdx0, _LastWrittenTerm0}, @@ -648,10 +648,10 @@ handle_event({written, Term, {FromIdx, ToIdx}}, {State, []}; NextWrittenRange -> %% retry with a reduced range - handle_event({written, Term, NextWrittenRange}, State0) + handle_event({written, Term, [NextWrittenRange]}, State0) end end; -handle_event({written, _Term, {FromIdx, _}} = Evt, +handle_event({written, _Term, [{FromIdx, _}]} = Evt, #?MODULE{cfg = #cfg{log_id = LogId}, mem_table = Mt, last_written_index_term = {LastWrittenIdx, _}} = State0) @@ -675,6 +675,17 @@ handle_event({written, _Term, {FromIdx, _}} = Evt, handle_event(Evt, State#?MODULE{last_written_index_term = {Expected, Term}}) end; +handle_event({written, Term, Written}, State) -> + %% simple handling of ra_seqs for now + case Written of + [I] when is_integer(I) -> + handle_event({written, Term, [{I, I}]}, State); + [I2, I] when is_integer(I) andalso + I + 1 == I2 -> + handle_event({written, Term, [{I, I2}]}, State); + _ -> + exit({sparse_written_events_not_implemented, Written}) + end; handle_event({segments, TidRanges, NewSegs}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, reader = Reader0, @@ -685,7 +696,18 @@ handle_event({segments, TidRanges, NewSegs}, %% 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) -> + fun ({Tid, Seq}, Acc0) -> + %% TODO: HACK: only handles single range ra_seqs + Range = case Seq of + [] -> + undefined; + [{_, _} = R] -> + R; + [I] -> ra_range:new(I, I); + [I2, I] when I+1 == I2 -> + ra_range:new(I, I2) + end, + ct:pal("Range ~p Seq ~p", [Range, Seq]), {Spec, Acc} = ra_mt:record_flushed(Tid, Range, Acc0), ok = ra_log_ets:execute_delete(Names, UId, Spec), Acc @@ -1200,7 +1222,7 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, maybe_append_first_entry(State0 = #?MODULE{last_index = -1}) -> State = append({0, 0, undefined}, State0), receive - {ra_log_event, {written, 0, {0, 0}}} -> + {ra_log_event, {written, 0, [0]}} -> ok after 60000 -> exit({?FUNCTION_NAME, timeout}) @@ -1323,14 +1345,15 @@ pick_range([{Fst, _Lst} | Tail], {CurFst, CurLst}) -> %% TODO: implement synchronous writes using gen_batch_server:call/3 await_written_idx(Idx, Term, Log0) -> - IDX = Idx, receive - {ra_log_event, {written, Term, {_, IDX}} = Evt} -> + {ra_log_event, {written, Term, _Seq} = Evt} -> {Log, _} = handle_event(Evt, Log0), - Log; - {ra_log_event, {written, _, _} = Evt} -> - {Log, _} = handle_event(Evt, Log0), - await_written_idx(Idx, Term, Log) + case last_written(Log) of + {Idx, Term} -> + Log; + _ -> + await_written_idx(Idx, Term, Log) + end after ?LOG_APPEND_TIMEOUT -> throw(ra_log_append_timeout) end. diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index 6d03fd58..d84ee0ea 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -65,7 +65,7 @@ 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()}]}, + #{ra_uid() => [{ets:tid(), ra_seq:state()}]}, string()) -> ok. accept_mem_tables(_SegmentWriter, Tables, undefined) when map_size(Tables) == 0 -> @@ -261,7 +261,7 @@ 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) -> SmallestIdx = smallest_live_idx(ServerUId), %% TidRanges arrive here sorted new -> old. @@ -270,31 +270,32 @@ flush_mem_table_ranges({ServerUId, TidRanges0}, %% 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(SmallestIdx - 1, Range0) of - undefined -> - []; - Range -> - [{T, Range}] - end; - ({T, Range0}, [{_T, {Start, _}} | _] = Acc) -> - Range1 = ra_range:truncate(SmallestIdx - 1, 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 + undefined -> + []; + Seq -> + [{T, Seq}] + end; + ({T, Seq0}, [{_T, PrevSeq} | _] = Acc) -> + Start = ra_seq:first(PrevSeq), + Seq1 = ra_seq:floor(SmallestIdx, Seq0), + case ra_seq:limit(Start, Seq1) of + undefined -> + Acc; + Seq -> + [{T, Seq} | Acc] + end + end, [], TidSeqs0), SegRefs0 = lists:append( lists:reverse( %% segrefs are returned in appended order so new -> old %% so we need to reverse them so that the final appended list %% of segrefs is in the old -> new order - [flush_mem_table_range(ServerUId, TidRange, State) - || TidRange <- TidRanges])), + [flush_mem_table_range(ServerUId, TidSeq, State) + || TidSeq <- TidSeqs])), %% compact cases where a segment was appended in a subsequent call to %% flush_mem_table_range @@ -308,14 +309,14 @@ flush_mem_table_ranges({ServerUId, TidRanges0}, [Seg | Acc] end, [], SegRefs0)), - ok = send_segments(System, ServerUId, TidRanges0, SegRefs), + ok = send_segments(System, ServerUId, TidSeqs0, SegRefs), ok. -flush_mem_table_range(ServerUId, {Tid, {StartIdx0, EndIdx}}, +flush_mem_table_range(ServerUId, {Tid, Seq}, #state{data_dir = DataDir, segment_conf = SegConf} = State) -> Dir = filename:join(DataDir, binary_to_list(ServerUId)), - StartIdx = start_index(ServerUId, StartIdx0), + % StartIdx = start_index(ServerUId, StartIdx0), case open_file(Dir, SegConf) of enoent -> ?DEBUG("segment_writer: skipping segment as directory ~ts does " @@ -324,8 +325,7 @@ flush_mem_table_range(ServerUId, {Tid, {StartIdx0, EndIdx}}, %% clean up the tables for this process []; Segment0 -> - case append_to_segment(ServerUId, Tid, StartIdx, EndIdx, - Segment0, State) of + case append_to_segment(ServerUId, Tid, Seq, Segment0, State) of undefined -> ?WARN("segment_writer: skipping segments for ~w as directory ~ts disappeared whilst writing", @@ -366,23 +366,29 @@ send_segments(System, ServerUId, TidRanges, SegRefs) -> [ServerUId, "No Pid"]), %% delete from the memtable on the non-running server's behalf [begin + %% 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 + Range = {ra_seq:first(Seq), ra_seq:last(Seq)}, _ = catch ra_mt:delete({range, Tid, Range}) - end || {Tid, Range} <- TidRanges], + 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(UId, Tid, Seq0, Seg, State) -> + FirstIdx = ra_seq:first(Seq0), + 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(_, _, StartIdx, EndIdx, Seg, Closed, _State) - when StartIdx >= EndIdx -> +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), @@ -390,7 +396,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. @@ -419,7 +425,7 @@ append_to_segment(UId, Tid, Idx, EndIdx, Seg0, Closed, State) -> %% the segment index but is probably good enough to get comparative %% data rates for different Ra components ok = counters:add(State#state.counter, ?C_BYTES_WRITTEN, DataSize), - append_to_segment(UId, Tid, Idx+1, EndIdx, Seg, Closed, State); + append_to_segment(UId, Tid, ra_seq:next(SeqIter), Seg, Closed, State); {error, full} -> % close and open a new segment case open_successor_segment(Seg0, State#state.segment_conf) of @@ -432,8 +438,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), @@ -549,3 +561,13 @@ maybe_upgrade_segment_file_names(System, DataDir) -> ok end. +next_ra_seq(Idx, Iter0) -> + case ra_seq:next(Iter0) of + end_of_seq -> + end_of_seq; + {I, _} = Next + when I >= Idx -> + Next; + {_, Iter} -> + next_ra_seq(Idx, Iter) + end. diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index 8dc64311..0126e2bb 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -99,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()}} }). @@ -305,7 +305,7 @@ init(#{system := System, catch _:Err:Stack -> ?ERROR("WAL in ~ts failed to initialise with ~p, stack ~p", [System, Err, Stack]), - {stop, Err} + {stop, Err, Stack} end. -spec handle_batch([wal_op()], state()) -> @@ -584,22 +584,29 @@ roll_over(#state{wal = Wal0, file_num = Num0, %% 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, @@ -695,11 +702,12 @@ complete_batch(#state{batch = #batch{waiting = Waiting, complete_batch_writer(Pid, #batch_writer{smallest_live_idx = SmallestIdx, tid = MtTid, uid = UId, - seq = Range, + seq = Seq0, term = Term, old = undefined}, Ranges) -> - Pid ! {ra_log_event, {written, Term, Range}}, - update_ranges(Ranges, UId, MtTid, SmallestIdx, Range); + 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), @@ -968,22 +976,19 @@ should_roll_wal(#state{conf = #conf{max_entries = MaxEntries}, smallest_live_index(#conf{ra_log_snapshot_state_tid = Tid}, ServerUId) -> ra_log_snapshot_state:smallest(Tid, ServerUId). -update_ranges(Ranges, UId, MtTid, SmallestIdx, AddSeq) -> +update_ranges(Ranges, UId, MtTid, _SmallestIdx, AddSeq) -> case Ranges of - #{UId := [{MtTid, Seq0} | Rem]} -> + #{UId := [{MtTid, Seq0} | Seqs]} -> %% SmallestIdx might have moved to we truncate the old range first %% before extending - Seq1 = ra_seq:floor(SmallestIdx, Seq0), + % 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. - Seq = ra_seq:add(AddSeq, Seq1), - Ranges#{UId => [{MtTid, Seq} | Rem]}; - #{UId := [{OldMtTid, OldMtSeq} | 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, AddSeq}, - {OldMtTid, - ra_seq:floor(SmallestIdx, OldMtSeq)} - | Rem]}; + Ranges#{UId => [{MtTid, AddSeq} | Seqs]}; _ -> Ranges#{UId => [{MtTid, AddSeq}]} end. @@ -1043,13 +1048,24 @@ recover_entry(Names, UId, {Idx, Term, _}, SmallestIdx, handle_trunc(false, _UId, _Idx, State) -> State; handle_trunc(true, UId, Idx, #recovery{mode = Mode, + ranges = Ranges0, 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}, + ranges = Ranges}; _ -> State end. diff --git a/src/ra_range.erl b/src/ra_range.erl index cb7f0d88..7e995b81 100644 --- a/src/ra_range.erl +++ b/src/ra_range.erl @@ -42,7 +42,9 @@ new(Start, End) is_integer(End) andalso Start =< End -> {Start, End}; -new(_Start, _End) -> +new(Start, End) + when is_integer(Start) andalso + is_integer(End) -> undefined. -spec add(AddRange :: range(), CurRange :: range()) -> range(). diff --git a/src/ra_seq.erl b/src/ra_seq.erl index ac243c0e..22357439 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -11,7 +11,11 @@ %% [55, {20, 52}, 3] -type state() :: [ra:index() | ra:range()]. --export_type([state/0]). +-record(i, {seq :: state()}). +-opaque iter() :: #i{}. + +-export_type([state/0, + iter/0]). -export([ @@ -20,7 +24,13 @@ floor/2, limit/2, add/2, - fold/3 + fold/3, + expand/1, + subtract/2, + first/1, + last/1, + iterator/1, + next/1 ]). -spec append(ra:index(), state()) -> state(). @@ -94,6 +104,57 @@ fold(Fun, Acc0, Seq) -> Fun(Idx, Acc) end, Acc0, Seq). +-spec expand(state()) -> [ra:index()]. +expand(Seq) -> + fold(fun (I, Acc) -> [I | Acc] end, [], Seq). + +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 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. + %% internal functions floor0(FloorIdx, [Last | Rem], Acc) @@ -114,13 +175,19 @@ floor0(FloorIdx, [{_, _} = T | Rem], Acc) -> floor0(_FloorIdx, _Seq, Acc) -> lists:reverse(Acc). -% first_index(Seq) -> -% last_index(lists:reverse(Seq)). -% last_index([{_, I} | _]) -> -% I; -% last_index([I | _]) -% when is_integer(I) -> -% I; -% last_index([]) -> -% undefined. +last_index([{_, I} | _]) -> + I; +last_index([I | _]) + when is_integer(I) -> + I; +last_index([]) -> + undefined. + +first_index([{_, I} | _]) -> + I; +first_index([I | _]) + when is_integer(I) -> + I; +first_index([]) -> + undefined. diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index ac2c3b60..e009074d 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -128,7 +128,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, @@ -138,11 +138,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), @@ -157,7 +157,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) @@ -172,11 +172,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), @@ -200,7 +200,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), @@ -609,7 +609,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( diff --git a/test/ra_log_SUITE.erl b/test/ra_log_SUITE.erl index 46c1a0cc..5e7b32d9 100644 --- a/test/ra_log_SUITE.erl +++ b/test/ra_log_SUITE.erl @@ -115,10 +115,12 @@ 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 + after 1000 -> + flush(), + ct:pal("fail written event not received") end, ok. @@ -277,3 +279,12 @@ 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. diff --git a/test/ra_log_props_SUITE.erl b/test/ra_log_props_SUITE.erl index 38551e3d..a50456fd 100644 --- a/test/ra_log_props_SUITE.erl +++ b/test/ra_log_props_SUITE.erl @@ -833,7 +833,7 @@ run_proper_noshrink(Fun, Args, NumTests) -> basic_reset(Log) -> ra_log:write([{0, 0, empty}], Log), receive - {ra_log_event, {written, _, {0, 0}}} -> + {ra_log_event, {written, _, [0]}} -> ok end, ra_log:close(Log). diff --git a/test/ra_log_segment_writer_SUITE.erl b/test/ra_log_segment_writer_SUITE.erl index 3a93317c..42acf701 100644 --- a/test/ra_log_segment_writer_SUITE.erl +++ b/test/ra_log_segment_writer_SUITE.erl @@ -91,16 +91,13 @@ accept_mem_tables(Config) -> Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}], Mt = make_mem_table(UId, Entries), Tid = ra_mt:tid(Mt), - TidRanges = [{Tid, ra_mt:range(Mt)}], - Ranges = #{UId => TidRanges}, - % WalFile = "0000001.wal", - % FullWalFile = filename:join(Dir, WalFile), - % ok = file:write_file(FullWalFile, <<"waldata">>), + TidSeqs = [{Tid, [ra_mt:range(Mt)]}], + Ranges = #{UId => TidSeqs}, make_wal(Config, "w1.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), receive - {ra_log_event, {segments, TidRanges, [{{1, 3}, SegFile}]}} -> + {ra_log_event, {segments, TidSeqs, [{{1, 3}, SegFile}]}} -> SegmentFile = filename:join(?config(server_dir, Config), SegFile), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -129,18 +126,18 @@ accept_mem_tables_append(Config) -> Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}], Tid = ets:new(?FUNCTION_NAME, []), _ = make_mem_table(UId, Tid, Entries), - Ranges = #{UId => [{Tid, {1, 3}}]}, - ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, + FlushSpec = #{UId => [{Tid, [{1, 3}]}]}, + ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, FlushSpec, make_wal(Config, "w1.wal")), % second batch Entries2 = [{4, 43, d}, {5, 43, e}], _ = make_mem_table(UId, Tid, Entries2), - Ranges2 = #{UId => [{Tid, {4, 5}}]}, - ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, + FlushSpec2 = #{UId => [{Tid, [{4, 5}]}]}, + ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, FlushSpec2, make_wal(Config, "w2.wal")), AllEntries = Entries ++ Entries2, receive - {ra_log_event, {segments, [{Tid, {4, 5}}], [{{1, 5}, Fn}]}} -> + {ra_log_event, {segments, [{Tid, [{4, 5}]}], [{{1, 5}, Fn}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -150,6 +147,7 @@ accept_mem_tables_append(Config) -> flush(), throw(ra_log_event_timeout) end, + flush(), ok = gen_server:stop(TblWriterPid), ok. @@ -161,11 +159,11 @@ accept_mem_tables_overwrite(Config) -> UId = ?config(uid, Config), Entries = [{3, 42, c}, {4, 42, d}, {5, 42, e}], Tid = ra_mt:tid(make_mem_table(UId, Entries)), - Ranges = #{UId => [{Tid, {3, 5}}]}, + Ranges = #{UId => [{Tid, [{3, 5}]}]}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), receive - {ra_log_event, {segments, [{Tid, {3, 5}}], [{{3, 5}, Fn}]}} -> + {ra_log_event, {segments, [{Tid, [{3, 5}]}], [{{3, 5}, Fn}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), ?assertMatch({{3, 5}, _}, ra_log_segment:segref(Seg)), @@ -178,11 +176,11 @@ accept_mem_tables_overwrite(Config) -> % second batch Entries2 = [{1, 43, a}, {2, 43, b}, {3, 43, c2}], Tid2 = ra_mt:tid(make_mem_table(UId, Entries2)), - Ranges2 = #{UId => [{Tid2, {1, 3}}]}, + Ranges2 = #{UId => [{Tid2, [{1, 3}]}]}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.wal")), receive - {ra_log_event, {segments, [{Tid2, {1, 3}}], [{{1, 3}, Fn2}]}} -> + {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn2}]}} -> SegmentFile2 = filename:join(?config(server_dir, Config), Fn2), {ok, Seg2} = ra_log_segment:open(SegmentFile2, #{mode => read}), ?assertMatch({{1, 3}, _}, ra_log_segment:segref(Seg2)), @@ -210,7 +208,7 @@ 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}}]}, + Ranges2 = #{UId => [{Tid2, [{4, 6}]}, {Tid, [{2, 5}]}]}, % debugger:start(), % int:i(ra_log_segment_writer), % int:break(ra_log_segment_writer, 240), @@ -218,7 +216,7 @@ accept_mem_tables_overwrite_same_wal(Config) -> make_wal(Config, "w2.wal")), receive {ra_log_event, - {segments, [{Tid2, {4, 6}}, {Tid, {2, 5}}], [{{2, 6}, Fn}]}} -> + {segments, [{Tid2, [{4, 6}]}, {Tid, [{2, 5}]}], [{{2, 6}, Fn}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), ?assertMatch({{2, 6}, _}, ra_log_segment:segref(Seg)), @@ -249,8 +247,10 @@ 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}, + ct:pal("TidRanges ~p", [TidRanges]), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w.wal")), receive @@ -303,7 +303,7 @@ accept_mem_tables_multi_segment_overwrite(Config) -> Entries = [{I, 2, x} || I <- lists:seq(1, 10)], Mt = make_mem_table(UId, Entries), Tid = ra_mt:tid(Mt), - TidRanges = [{Tid, ra_mt:range(Mt)}], + TidRanges = [{Tid, [ra_mt:range(Mt)]}], Ranges = #{UId => TidRanges}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w.wal")), @@ -320,7 +320,7 @@ accept_mem_tables_multi_segment_overwrite(Config) -> Entries2 = [{I, 3, x} || I <- lists:seq(7, 15)], Mt2 = make_mem_table(UId, Entries2), Tid2 = ra_mt:tid(Mt2), - TidRanges2 = [{Tid2, ra_mt:range(Mt2)}], + TidRanges2 = [{Tid2, [ra_mt:range(Mt2)]}], Ranges2 = #{UId => TidRanges2}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.wal")), @@ -360,13 +360,13 @@ accept_mem_tables_for_down_server(Config) -> Mt2 = make_mem_table(UId, Entries), Tid = ra_mt:tid(Mt), Tid2 = ra_mt:tid(Mt2), - Ranges = #{DownUId => [{Tid, {1, 3}}], - UId => [{Tid2, {1, 3}}]}, + Ranges = #{DownUId => [{Tid, [{1, 3}]}], + UId => [{Tid2, [{1, 3}]}]}, WalFile = filename:join(Dir, "00001.wal"), ok = file:write_file(WalFile, <<"waldata">>), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, [{Tid2, {1, 3}}], [{{1, 3}, Fn}]}} -> + {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -376,6 +376,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 +390,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 +423,12 @@ accept_mem_tables_with_deleted_server(Config) -> Mt2 = make_mem_table(UId, Entries), Tid = ra_mt:tid(Mt), Tid2 = ra_mt:tid(Mt2), - Ranges = #{DeletedUId => [{Tid, {1, 3}}], - UId => [{Tid2, {1, 3}}]}, + Ranges = #{DeletedUId => [{Tid, [{1, 3}]}], + UId => [{Tid2, [{1, 3}]}]}, WalFile = make_wal(Config, "00001.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, [{Tid2, {1, 3}}], [{{1, 3}, Fn}]}} -> + {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -468,7 +470,7 @@ accept_mem_tables_with_corrupt_segment(Config) -> Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}], Mt = make_mem_table(UId, Entries), Tid = ra_mt:tid(Mt), - TidRanges = [{Tid, ra_mt:range(Mt)}], + TidRanges = [{Tid, [ra_mt:range(Mt)]}], Ranges = #{UId => TidRanges}, WalFile = make_wal(Config, "0000001.wal"), %% write an empty file to simulate corrupt segment @@ -508,14 +510,15 @@ accept_mem_tables_multiple_ranges(Config)-> Mt = make_mem_table(UId, Entries), Entries2 = [{N, 42, N} || N <- lists:seq(33, 64)], Mt2 = make_mem_table(UId, Entries2), - Ranges = #{UId => [ - {ra_mt:tid(Mt2), ra_mt:range(Mt2)}, - {ra_mt:tid(Mt), ra_mt:range(Mt)} - ]}, + TidRanges = [ + {ra_mt:tid(Mt2), [ra_mt:range(Mt2)]}, + {ra_mt:tid(Mt), [ra_mt:range(Mt)]} + ], + Ranges = #{UId => TidRanges}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), receive - {ra_log_event, {segments, _TidRanges, SegRefs}} -> + {ra_log_event, {segments, TidRanges, SegRefs}} -> ?assertMatch([ {{49, 64}, _}, {{33, 48}, _}, @@ -542,16 +545,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)} - ]}, + TidRanges = [ + {ra_mt:tid(Mt2), [ra_mt:range(Mt2)]}, + {ra_mt:tid(Mt), [ra_mt:range(Mt)]} + ], + Ranges = #{UId => TidRanges}, ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 64, 65, []), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), receive - {ra_log_event, {segments, _TidRanges, SegRefs}} -> + {ra_log_event, {segments, TidRanges, SegRefs}} -> ?assertMatch([], SegRefs), ok after 3000 -> @@ -572,7 +576,7 @@ truncate_segments(Config) -> Entries = [{N, 42, N} || N <- lists:seq(1, 32)], Mt = make_mem_table(UId, Entries), Tid = ra_mt:tid(Mt), - TidRanges = [{Tid, ra_mt:range(Mt)}], + TidRanges = [{Tid, [ra_mt:range(Mt)]}], Ranges = #{UId => TidRanges}, WalFile = make_wal(Config, "0000001.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), @@ -607,18 +611,18 @@ truncate_segments_with_pending_update(Config) -> UId = ?config(uid, Config), Entries = [{N, 42, N} || N <- lists:seq(1, 32)], Mt = make_mem_table(UId, Entries), - Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]}, + Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), ra_log_segment_writer:await(?SEGWR), %% write another range Entries2 = [{N, 42, N} || N <- lists:seq(33, 40)], Mt2 = make_mem_table(UId, Entries2), - Ranges2 = #{UId => [{ra_mt:tid(Mt2), ra_mt:range(Mt2)}]}, + Ranges2 = #{UId => [{ra_mt:tid(Mt2), [ra_mt:range(Mt2)]}]}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.erl")), receive - {ra_log_event, {segments, _Tid, [{{25, 32}, S} = Cur | Rem]}} -> + {ra_log_event, {segments, _TidRanges, [{{25, 32}, S} = Cur | Rem]}} -> % this is the event from the first call to accept_mem_tables, % the Cur segments has been appended to since so should _not_ % be deleted when it is provided as the cutoff segref for @@ -654,13 +658,13 @@ truncate_segments_with_pending_overwrite(Config) -> % fake up a mem segment for Self Entries = [{N, 42, N} || N <- lists:seq(1, 32)], Mt = make_mem_table(UId, Entries), - Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]}, + Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), %% write one more entry separately Entries2 = [{N, 43, N} || N <- lists:seq(12, 25)], Mt2 = make_mem_table(UId, Entries2), - Ranges2 = #{UId => [{ra_mt:tid(Mt2), ra_mt:range(Mt2)}]}, + Ranges2 = #{UId => [{ra_mt:tid(Mt2), [ra_mt:range(Mt2)]}]}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.wal")), receive @@ -715,7 +719,7 @@ my_segments(Config) -> % fake up a mem segment for Self Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}], Mt = make_mem_table(UId, Entries), - Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]}, + Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]}, TidRanges = maps:get(UId, Ranges), WalFile = make_wal(Config, "00001.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), @@ -741,7 +745,7 @@ upgrade_segment_name_format(Config) -> % fake up a mem segment for Self Entries = [{1, 42, a}, {2, 42, b}, {3, 43, c}], Mt = make_mem_table(UId, Entries), - Ranges = #{UId => [{ra_mt:tid(Mt), ra_mt:range(Mt)}]}, + Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]}, TidRanges = maps:get(UId, Ranges), WalFile = make_wal(Config, "00001.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), @@ -788,7 +792,7 @@ 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 ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 3, 4, []), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, @@ -821,7 +825,7 @@ 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 ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 3, 4, []), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index fff0c8a1..4b571f43 100644 --- a/test/ra_log_wal_SUITE.erl +++ b/test/ra_log_wal_SUITE.erl @@ -25,6 +25,9 @@ all_tests() -> [ basic_log_writes, sparse_writes, + sparse_write_same_batch, + sparse_write_overwrite, + sparse_write_recover, wal_filename_upgrade, same_uid_different_process, consecutive_terms_in_batch_should_result_in_two_written_events, @@ -134,13 +137,13 @@ basic_log_writes(Config) -> Tid = ets:new(?FUNCTION_NAME, []), {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(), @@ -158,20 +161,15 @@ sparse_writes(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, 12}), - % debugger:start(), - % int:i(ra_log_wal), - % int:break(ra_log_wal, 975), - timer:sleep(1000), + 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"), - % timer:sleep(200000), - ok = await_written(WriterId, 1, {15, 15}), + ok = await_written(WriterId, 1, [15]), ra_log_wal:force_roll_over(Pid), receive {'$gen_cast', - {mem_tables, #{UId := [{Tid, {12, 15}}]}, _}} -> + {mem_tables, #{UId := [{Tid, [15, 12]}]}, _}} -> ok after 5000 -> flush(), @@ -181,14 +179,64 @@ sparse_writes(Config) -> meck:unload(), ok. -sparse_write_same_batch(_Config) -> - ct:fail("~s", [?FUNCTION_NAME]). +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), -sparse_write_recover(_Config) -> - ct:fail("~s", [?FUNCTION_NAME]). + 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) -> + 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]), + {ok, _} = ra_log_wal:write(Pid, WriterId, Tid, 12, 15, 1, "value2"), + ok = await_written(WriterId, 1, [15]), + + ok = proc_lib:stop(ra_log_wal), + {ok, _Pid2} = ra_log_wal:start_link(Conf), + % {ok, Mt} = ra_log_ets:mem_table_please(?config(names, Config), UId), + 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. + +%% TODO: as sparse writes are pre committed I dont +%% think we'll ever overwrite anything. sparse_write_overwrite(_Config) -> - ct:fail("~s", [?FUNCTION_NAME]). + ok. wal_filename_upgrade(Config) -> meck:new(ra_log_segment_writer, [passthrough]), @@ -199,21 +247,18 @@ wal_filename_upgrade(Config) -> Tid = ets:new(?FUNCTION_NAME, []), {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), receive {'$gen_cast', - {mem_tables, #{UId := [{_Tid, {12, 13}}]}, "0000000000000001.wal"}} -> + {mem_tables, #{UId := [{_Tid, [13, 12]}]}, "0000000000000001.wal"}} -> ok after 5000 -> flush(), @@ -231,23 +276,24 @@ same_uid_different_process(Config) -> Tid = ets:new(?FUNCTION_NAME, []), {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(), @@ -267,18 +313,18 @@ consecutive_terms_in_batch_should_result_in_two_written_events(Config) -> 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(), @@ -298,16 +344,16 @@ writes_snapshot_idx_overtakes(Config) -> 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 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(), @@ -328,7 +374,7 @@ writes_implicit_truncate_write(Config) -> 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 @@ -336,11 +382,11 @@ writes_implicit_truncate_write(Config) -> 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(), @@ -365,24 +411,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 (_) -> + gen_batch_server:cast( + Pid, {query, + fun (_) -> - ok = ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, - 5, 6, []) - end}), + 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(), @@ -403,7 +449,7 @@ overwrite_in_same_batch(Config) -> 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), @@ -417,13 +463,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 -> @@ -445,16 +491,16 @@ overwrite_completely(Config) -> 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(), @@ -475,16 +521,16 @@ overwrite_inside(Config) -> 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(), @@ -548,7 +594,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), @@ -600,7 +647,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 @@ -652,7 +699,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 @@ -667,9 +714,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), @@ -686,13 +733,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. @@ -714,11 +761,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 @@ -745,11 +792,11 @@ roll_over_with_data_larger_than_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, 1}}]}, _Wal}} -> + {mem_tables, #{UId := [{Tid, [1]}]}, _Wal}} -> ok after 2000 -> flush(), @@ -776,11 +823,11 @@ roll_over_entry_limit(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, 1000}}]}, _Wal}} -> + {mem_tables, #{UId := [{Tid, [{1, 1000}]}]}, _Wal}} -> %% 1000 is the last entry before the limit was reached ok after 2000 -> @@ -824,14 +871,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), @@ -844,7 +891,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(), @@ -852,7 +899,7 @@ recover(Config) -> end, receive {'$gen_cast', - {mem_tables, #{UId := [{MtTid, {101, 200}}]}, _}} -> + {mem_tables, #{UId := [{MtTid, [{101, 200}]}]}, _}} -> ok after 2000 -> flush(), @@ -885,7 +932,7 @@ 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), @@ -898,7 +945,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(), @@ -922,12 +969,12 @@ 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), @@ -939,8 +986,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 -> @@ -964,14 +1011,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), @@ -982,7 +1029,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(), @@ -990,7 +1037,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(), @@ -1018,7 +1065,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), @@ -1027,7 +1074,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(), @@ -1065,7 +1112,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)), @@ -1076,7 +1123,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(), @@ -1103,7 +1150,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), @@ -1114,7 +1161,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), @@ -1126,8 +1173,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(), @@ -1148,7 +1195,7 @@ recover_implicit_truncate(Config) -> 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 @@ -1156,15 +1203,12 @@ recover_implicit_truncate(Config) -> 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?? ra_log_snapshot_state: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), {ok, Mt} = ra_log_ets:mem_table_please(?config(names, Config), UId), @@ -1172,7 +1216,7 @@ recover_implicit_truncate(Config) -> 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(), @@ -1194,12 +1238,12 @@ recover_delete_uid(Config) -> 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), @@ -1212,7 +1256,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 -> @@ -1249,7 +1293,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), @@ -1268,7 +1312,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(), @@ -1291,7 +1335,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), @@ -1321,7 +1365,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), @@ -1353,7 +1397,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), @@ -1378,25 +1422,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 diff --git a/test/ra_seq_SUITE.erl b/test/ra_seq_SUITE.erl index f4437fba..d6722f99 100644 --- a/test/ra_seq_SUITE.erl +++ b/test/ra_seq_SUITE.erl @@ -23,7 +23,9 @@ all_tests() -> append, floor, limit, - add + add, + subtract, + iter ]. groups() -> @@ -103,3 +105,25 @@ add(_Config) -> 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. + + + From 024a013c9c5df4261995174894f1e74f41807185 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 24 Mar 2025 12:14:18 +0000 Subject: [PATCH 13/22] docs --- docs/internals/COMPACTION.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md index 6a8af47c..0704a4be 100644 --- a/docs/internals/COMPACTION.md +++ b/docs/internals/COMPACTION.md @@ -227,7 +227,9 @@ process itself #### Alternative snapshot install procedure -* Sender begins with sending negotiating which live indexes are needed. +* Sender begins with sending negotiating which live indexes are needed. It is +probably sufficient for the receiver to return it's `last_applied` index and the +sender will send all sparse entries after that index * Then it proceeds to send the live indexes _before_ the snapshot (so in it's natural order if you like). * The receiving ra process then writes these commands to the WAL as normal but @@ -235,6 +237,8 @@ using a special command / flag to avoid the WAL triggering its' gap detection. Ideally the specialised command would include the previous idx so that we can still do gap detection in the sparse sequence (should all sends include prior sequence so this is the only mode?). +* The sparse writes are written to a new memtable using a new `ra_mt:sparse_write/2` +API that bypasses gap validation and stores a sparse sequence instead of range * Alt the live indexes replication could be done after the snapshot is complete as it is easy for the follower to work out which live indexes it needs. when it receives the `last` snapshot chunk it then replies with a special From c6d59b1b7721225128667855905c46bbbfe7617b Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 8 Apr 2025 11:30:09 +0100 Subject: [PATCH 14/22] wip --- src/ra_seq.erl | 30 +++++++++++++++--------------- test/ra_mt_SUITE.erl | 15 +++++++++++++++ 2 files changed, 30 insertions(+), 15 deletions(-) diff --git a/src/ra_seq.erl b/src/ra_seq.erl index 22357439..839dae00 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -176,18 +176,18 @@ floor0(_FloorIdx, _Seq, Acc) -> lists:reverse(Acc). -last_index([{_, I} | _]) -> - I; -last_index([I | _]) - when is_integer(I) -> - I; -last_index([]) -> - undefined. - -first_index([{_, I} | _]) -> - I; -first_index([I | _]) - when is_integer(I) -> - I; -first_index([]) -> - undefined. +% last_index([{_, I} | _]) -> +% I; +% last_index([I | _]) +% when is_integer(I) -> +% I; +% last_index([]) -> +% undefined. + +% first_index([{_, I} | _]) -> +% I; +% first_index([I | _]) +% when is_integer(I) -> +% I; +% first_index([]) -> +% undefined. diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl index 4aff51ef..d4616f66 100644 --- a/test/ra_mt_SUITE.erl +++ b/test/ra_mt_SUITE.erl @@ -468,6 +468,21 @@ perf(_Config) -> ok. +sparse(_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)), + {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 999}, Mt2), + 500 = ra_mt:delete(Spec2), + ?assertEqual(1, ra_mt:lookup_term(1000, Mt3)), + ok. %%% Util From e4790dbedaafde4c247df83d1747c9d1bdae090c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 17 Apr 2025 14:15:42 +0100 Subject: [PATCH 15/22] wip --- src/ra_log.erl | 57 ++++++++++++++++++++++++++++++++++--------- src/ra_mt.erl | 36 +++++++++++++++++++++++---- src/ra_seq.erl | 32 ++++++++++++++++++++---- test/ra_log_SUITE.erl | 31 +++++++++++++++++++++++ test/ra_mt_SUITE.erl | 25 ++++++++++--------- test/ra_seq_SUITE.erl | 19 ++++++++++++--- 6 files changed, 164 insertions(+), 36 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index e3a6211f..1f6b3c33 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -16,6 +16,7 @@ commit_tx/1, append/2, write/2, + write_sparse/3, append_sync/2, write_sync/2, fold/5, @@ -437,6 +438,38 @@ write([{Idx, _, _} | _], #?MODULE{cfg = #cfg{uid = UId}, [UId, Idx, LastIdx+1])), {error, {integrity_error, Msg}}. +write_sparse({Idx, Term, _} = Entry, PrevIdx0, + #?MODULE{cfg = #cfg{uid = UId, + wal = Wal} = Cfg, + last_index = LastIdx, + mem_table = Mt0} = State0) + when PrevIdx0 == undefined orelse + (PrevIdx0 == LastIdx) -> + {ok, Mt} = ra_mt:insert_sparse(Entry, PrevIdx0, Mt0), + % al_write_batch(State0#?MODULE{mem_table = Mt}, [Entry]). + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, LastIdx), + ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), + Tid = ra_mt:tid(Mt), + PrevIdx = case PrevIdx0 of + undefined -> + Idx - 1; + _ -> + PrevIdx0 + end, + case ra_log_wal:write(Wal, {UId, self()}, Tid, PrevIdx, Idx, + Term, Entry) of + {ok, Pid} -> + ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), + State0#?MODULE{last_index = Idx, + last_term = Term, + mem_table = Mt, + last_wal_write = {Pid, now_ms()} + }; + {error, wal_down} -> + error(wal_down) + end. + -spec fold(FromIdx :: ra_index(), ToIdx :: ra_index(), fun((log_entry(), Acc) -> Acc), Acc, state()) -> {Acc, state()} when Acc :: term(). @@ -453,12 +486,14 @@ fold(From0, To0, Fun, Acc0, ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), MtOverlap = ra_mt:range_overlap({From, To}, Mt), - case MtOverlap of + case MtOverlap oe {undefined, {RemStart, RemEnd}} -> - {Reader, Acc} = ra_log_reader:fold(RemStart, RemEnd, Fun, Acc0, Reader0), + {Reader, Acc} = ra_log_reader:fold(RemStart, RemEnd, Fun, + Acc0, Reader0), {Acc, State#?MODULE{reader = Reader}}; {{MtStart, MtEnd}, {RemStart, RemEnd}} -> - {Reader, Acc1} = ra_log_reader:fold(RemStart, RemEnd, Fun, Acc0, Reader0), + {Reader, Acc1} = ra_log_reader:fold(RemStart, RemEnd, Fun, + Acc0, Reader0), Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc1, Mt), NumRead = MtEnd - MtStart + 1, ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, NumRead), @@ -644,7 +679,7 @@ handle_event({written, Term, [{FromIdx, ToIdx}]}, undefined -> ?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, ToIdx, OtherTerm]), {State, []}; NextWrittenRange -> %% retry with a reduced range @@ -697,20 +732,20 @@ handle_event({segments, TidRanges, NewSegs}, %% (new -> old) so we need to foldr here to process the oldest first Mt = lists:foldr( fun ({Tid, Seq}, Acc0) -> - %% TODO: HACK: only handles single range ra_seqs + %% TODO: should mt take ra_seqs in case of sparse writes? Range = case Seq of [] -> undefined; [{_, _} = R] -> R; - [I] -> ra_range:new(I, I); - [I2, I] when I+1 == I2 -> - ra_range:new(I, I2) + _ -> + ra_range:new(ra_seq:first(Seq), + ra_seq:last(Seq)) end, - ct:pal("Range ~p Seq ~p", [Range, Seq]), + % ct:pal("Range ~p Seq ~p", [Range, Seq]), {Spec, Acc} = ra_mt:record_flushed(Tid, Range, Acc0), - ok = ra_log_ets:execute_delete(Names, UId, Spec), - Acc + ok = ra_log_ets:execute_delete(Names, UId, Spec), + Acc end, Mt0, TidRanges), State = State0#?MODULE{reader = Reader, mem_table = Mt}, diff --git a/src/ra_mt.erl b/src/ra_mt.erl index df25476e..a8c58619 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -13,6 +13,7 @@ init/2, init_successor/3, insert/2, + insert_sparse/3, stage/2, commit/1, abort/1, @@ -44,9 +45,9 @@ (is_tuple(Range) andalso Idx < element(1, Range))). -% -define(IS_AFTER_RANGE(Idx, Range), -% (is_tuple(Range) andalso -% Idx > element(2, Range))). +-define(IS_AFTER_RANGE(Idx, Range), + (is_tuple(Range) andalso + Idx > element(2, Range))). -define(IS_NEXT_IDX(Idx, Range), (Range == undefined orelse @@ -84,8 +85,7 @@ init(Tid, Mode) -> end, undefined, Tid) end, #?MODULE{tid = Tid, - range = Range - }. + range = Range}. -spec init(ets:tid()) -> state(). init(Tid) -> @@ -115,6 +115,32 @@ insert({Idx, _, _} = _Entry, ?IS_BEFORE_RANGE(Idx, Range) -> {error, overwriting}. +-spec insert_sparse(log_entry(), undefined | ra:index(), state()) -> + {ok, state()} | {error, gap_detected | limit_reached}. +insert_sparse({Idx, _, _} = Entry, LastIdx, + #?MODULE{tid = Tid, + range = Range} = State) + when Range == undefined orelse + LastIdx == undefined orelse + (element(2, Range) == LastIdx) -> + case ra_range:size(Range) > ?MAX_MEMTBL_ENTRIES of + true -> + {error, limit_reached}; + false -> + true = ets:insert(Tid, Entry), + NewRange = case Range of + undefined -> + ra_range:new(Idx); + {S, _} -> + {S, Idx} + end, + %% TODO: consider using ra_seq instead + {ok, State#?MODULE{range = NewRange}} + end; +insert_sparse({_Idx, _, _} = _Entry, _LastIdx, + #?MODULE{range = _Range} = _State0) -> + {error, gap_detected}. + -spec stage(log_entry(), state()) -> {ok, state()} | {error, overwriting | limit_reached}. stage({Idx, _, _} = Entry, diff --git a/src/ra_seq.erl b/src/ra_seq.erl index 839dae00..b180e747 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -27,6 +27,7 @@ fold/3, expand/1, subtract/2, + remove_prefix/2, first/1, last/1, iterator/1, @@ -64,6 +65,7 @@ floor(FloorIdxIncl, Seq) -> floor0(FloorIdxIncl, Seq, []). +-spec limit(ra:index(), state()) -> state(). limit(CeilIdx, [Last | Rem]) when is_integer(Last) andalso Last > CeilIdx -> @@ -92,9 +94,8 @@ add(Seq1, Seq2) -> end, fold(fun append/2, limit(Fst - 1, Seq2), Seq1). - --spec fold(fun ((ra:index(), Acc) -> Acc), Acc, state()) --> Acc when Acc :: term(). +-spec fold(fun ((ra:index(), Acc) -> Acc), Acc, state()) -> + Acc when Acc :: term(). fold(Fun, Acc0, Seq) -> %% TODO: factor out the lists:seq/2 lists:foldr( @@ -108,6 +109,7 @@ fold(Fun, Acc0, Seq) -> 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 @@ -137,11 +139,18 @@ last(Seq) -> 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}. +-spec next(iter()) -> {ra:index(), iter()} | end_of_seq. next(#i{seq = []}) -> end_of_seq; next(#i{seq = [Next | Rem]}) @@ -155,7 +164,20 @@ next(#i{seq = [{Next, End} | Rem]}) -> {Next, #i{seq = [NextRange | Rem]}} end. -%% internal functions +%% Internal functions + +drop_prefix({IDX, PI}, {IDX, SI}) -> + drop_prefix(next(PI), next(SI)); +drop_prefix(end_of_seq, {Idx, #i{seq = RevSeq}}) -> + {ok, lists:reverse([Idx | RevSeq])}; +drop_prefix({PrefIdx, PI}, {Idx, _SI} = I) + when PrefIdx < Idx -> + drop_prefix(next(PI), I); +drop_prefix({PrefIdx, PI}, {Idx, _SI} = I) + when Idx < PrefIdx -> + {error, not_prefix}. + + floor0(FloorIdx, [Last | Rem], Acc) when is_integer(Last) andalso diff --git a/test/ra_log_SUITE.erl b/test/ra_log_SUITE.erl index 5e7b32d9..994ef491 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,6 +99,22 @@ 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), + LastIdx = Idx + 5, + Entry1 = {Idx, Term, "entry"}, + %% sparse + Entry2 = {Idx+5, Term, "entry+5"}, + Log1 = ra_log:write_sparse(Entry1, undefined, Log0), + {{Idx, Term, "entry"}, Log2} = ra_log:fetch(Idx, Log1), + Log3 = ra_log:write_sparse(Entry2, Idx, Log2), + Log = await_written_idx(Idx, Term, Log3), + {LastIdx, Term} = ra_log:last_written(Log), + {{LastIdx, Term, "entry+5"}, Log2} = ra_log:fetch(Idx+5, Log1), + ok. + append_then_fetch_no_wait(Config) -> Log0 = ?config(ra_log, Config), Term = 1, @@ -288,3 +305,17 @@ flush() -> after 0 -> ok end. + +await_written_idx(Idx, Term, Log0) -> + receive + {ra_log_event, {written, Term, _Seq} = 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 -> + throw(ra_log_append_timeout) + end. diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl index d4616f66..31e359a1 100644 --- a/test/ra_mt_SUITE.erl +++ b/test/ra_mt_SUITE.erl @@ -34,7 +34,8 @@ all_tests() -> stage_commit, range_overlap, stage_commit_abort, - perf + perf, + sparse ]. groups() -> @@ -471,17 +472,17 @@ perf(_Config) -> sparse(_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)), - {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 999}, Mt2), - 500 = ra_mt:delete(Spec2), - ?assertEqual(1, ra_mt:lookup_term(1000, Mt3)), + {ok, Mt1} = ra_mt:insert_sparse({2, 1, <<"banana">>}, 0, 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, {2, 5}, Mt2), + ct:pal("Spec ~p", [Spec]), + % 499 = ra_mt:delete(Spec)2 + % ?assertEqual({500, 1000}, ra_mt:range(Mt2)), + % ?assertEqual(501, ets:info(Tid, size)), + % {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 999}, Mt2), + % 500 = ra_mt:delete(Spec2), + % ?assertEqual(1, ra_mt:lookup_term(1000, Mt3)), ok. %%% Util diff --git a/test/ra_seq_SUITE.erl b/test/ra_seq_SUITE.erl index d6722f99..fe1ef8fd 100644 --- a/test/ra_seq_SUITE.erl +++ b/test/ra_seq_SUITE.erl @@ -25,7 +25,8 @@ all_tests() -> limit, add, subtract, - iter + iter, + remove_prefix ]. groups() -> @@ -125,5 +126,17 @@ iter(_Config) -> 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. From 106f3db16e49cb0e613989d4e004a94338102c7c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 17 Apr 2025 17:15:55 +0100 Subject: [PATCH 16/22] broken wip --- src/ra_log.erl | 2 +- src/ra_mt.erl | 203 +++++++++++++++++++++++-------------------- src/ra_seq.erl | 31 ++++++- test/ra_mt_SUITE.erl | 6 +- 4 files changed, 143 insertions(+), 99 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 1f6b3c33..46ed9943 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -486,7 +486,7 @@ fold(From0, To0, Fun, Acc0, ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), MtOverlap = ra_mt:range_overlap({From, To}, Mt), - case MtOverlap oe + case MtOverlap of {undefined, {RemStart, RemEnd}} -> {Reader, Acc} = ra_log_reader:fold(RemStart, RemEnd, Fun, Acc0, Reader0), diff --git a/src/ra_mt.erl b/src/ra_mt.erl index a8c58619..294fdc75 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -6,6 +6,7 @@ %% @hidden -module(ra_mt). +-include_lib("stdlib/include/assert.hrl"). -include("ra.hrl"). -export([ @@ -16,7 +17,7 @@ insert_sparse/3, stage/2, commit/1, - abort/1, + % abort/1, lookup/2, lookup_term/2, tid_for/3, @@ -41,21 +42,22 @@ 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_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{} }). @@ -73,19 +75,17 @@ -spec init(ets:tid(), read | read_write) -> state(). init(Tid, Mode) -> - Range = case Mode of + Seq = case Mode of read -> - undefined; + []; read_write -> %% TODO: can this be optimised further? - ets:foldl(fun ({I, _, _}, undefined) -> - {I, I}; - ({I, _, _}, {S, E}) -> - {min(I, S), max(I, E)} - end, undefined, Tid) + ra_seq:from_list(ets:foldl(fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Tid)) end, #?MODULE{tid = Tid, - range = Range}. + indexes = Seq}. -spec init(ets:tid()) -> state(). init(Tid) -> @@ -100,73 +100,74 @@ 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, gap_detected | limit_reached}. insert_sparse({Idx, _, _} = Entry, LastIdx, #?MODULE{tid = Tid, - range = Range} = State) - when Range == undefined orelse + indexes = Seq} = State) + when Seq == undefined orelse LastIdx == undefined orelse - (element(2, Range) == LastIdx) -> - case ra_range:size(Range) > ?MAX_MEMTBL_ENTRIES of + ((is_integer(hd(Seq)) andalso hd(Seq) == LastIdx) orelse + (LastIdx == element(2, hd(Seq)))) -> + case ra_range:size(Seq) > ?MAX_MEMTBL_ENTRIES of true -> {error, limit_reached}; false -> true = ets:insert(Tid, Entry), - NewRange = case Range of - undefined -> - ra_range:new(Idx); - {S, _} -> - {S, Idx} - end, - %% TODO: consider using ra_seq instead - {ok, State#?MODULE{range = NewRange}} + {ok, State#?MODULE{indexes = ra_seq:append(Idx, Seq)}} end; insert_sparse({_Idx, _, _} = _Entry, _LastIdx, - #?MODULE{range = _Range} = _State0) -> + #?MODULE{indexes = _Seq} = _State0) -> {error, gap_detected}. -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) -> @@ -186,22 +187,22 @@ commit(#?MODULE{tid = Tid, {PrevStaged ++ Staged, State#?MODULE{staged = undefined, prev = Prev}}. --spec abort(state()) -> state(). -abort(#?MODULE{staged = undefined} = State) -> - State; -abort(#?MODULE{staged = {_, Staged}, - range = Range, - prev = Prev0} = State) -> - Prev = case Prev0 of - undefined -> - Prev0; - _ -> - abort(Prev0) - end, - {Idx, _, _} = lists:last(Staged), - State#?MODULE{staged = undefined, - range = ra_range:limit(Idx, Range), - prev = Prev}. +% -spec abort(state()) -> state(). +% abort(#?MODULE{staged = undefined} = State) -> +% State; +% abort(#?MODULE{staged = {_, Staged}, +% indexes = Range, +% prev = Prev0} = State) -> +% Prev = case Prev0 of +% undefined -> +% Prev0; +% _ -> +% abort(Prev0) +% end, +% {Idx, _, _} = lists:last(Staged), +% State#?MODULE{staged = undefined, +% indexes = ra_range:limit(Idx, Range), +% prev = Prev}. -spec lookup(ra:index(), state()) -> log_entry() | undefined. @@ -215,10 +216,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; @@ -240,13 +243,21 @@ lookup_term(Idx, #?MODULE{staged = {FstStagedIdx, Staged}}) undefined end; lookup_term(Idx, #?MODULE{tid = Tid, - range = Range}) - when ?IN_RANGE(Idx, Range) -> - ets:lookup_element(Tid, Idx, 2); -lookup_term(Idx, #?MODULE{prev = #?MODULE{} = Prev}) -> - lookup_term(Idx, Prev); -lookup_term(_Idx, _State) -> - undefined. + prev = Prev, + indexes = _Seq}) -> + %% TODO: implement properly, checking Seq + case ets:lookup_element(Tid, Idx, 2, undefined) of + undefined when Prev =/= undefined -> + lookup_term(Idx, Prev); + Term -> + Term + end. + % when ?IN_RANGE(Idx, Seq) -> + % ets:lookup_element(Tid, Idx, 2); +% lookup_term(Idx, #?MODULE{prev = #?MODULE{} = Prev}) -> + % lookup_term(Idx, Prev); +% lookup_term(_Idx, _State) -> + % undefined. -spec tid_for(ra:index(), ra_term(), state()) -> undefined | ets:tid(). @@ -269,6 +280,7 @@ fold(To, To, Fun, Acc, State) -> fold(From, To, Fun, Acc, State) when To > From -> E = lookup(From, State), + ?assert(E =/= undefined), fold(From + 1, To, Fun, Fun(E, Acc), State). -spec get_items([ra:index()], state()) -> @@ -323,10 +335,10 @@ range_overlap(ReqRange, #?MODULE{} = State) -> -spec range(state()) -> undefined | {ra:index(), ra:index()}. -range(#?MODULE{range = Range, +range(#?MODULE{indexes = Range, prev = undefined}) -> Range; -range(#?MODULE{range = {_, End} = Range, +range(#?MODULE{indexes = {_, End} = Range, prev = Prev}) -> PrevRange = ra_range:limit(End, range(Prev)), ra_range:add(Range, PrevRange); @@ -361,19 +373,19 @@ info(#?MODULE{tid = Tid, has_previous => Prev =/= undefined }. --spec record_flushed(ets:tid(), ra:range(), state()) -> +-spec record_flushed(ets:tid(), ra_seq:state(), state()) -> {delete_spec(), state()}. -record_flushed(TID = Tid, {Start, End}, +record_flushed(TID = Tid, FlushedSeq, #?MODULE{tid = TID, - range = Range} = State) -> - HasExtraEntries = ets:info(Tid, size) > ra_range:size(Range), - case ?IN_RANGE(End, Range) of + indexes = Seq} = State) -> + HasExtraEntries = ets:info(Tid, size) > ra_seq:length(Seq), + case ?IN_RANGE(End, Seq) of true when HasExtraEntries -> {{'<', Tid, End + 1}, - State#?MODULE{range = ra_range:truncate(End, Range)}}; + State#?MODULE{indexes = ra_range:truncate(End, Seq)}}; true -> {{range, Tid, {Start, End}}, - State#?MODULE{range = ra_range:truncate(End, Range)}}; + State#?MODULE{indexes = ra_range:truncate(End, Seq)}}; false -> {undefined, State} end; @@ -393,7 +405,7 @@ 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, + indexes = Range, prev = Prev0} = State) when (is_tuple(Range) andalso Idx > element(1, Range)) orelse @@ -426,7 +438,7 @@ set_first(Idx, #?MODULE{tid = Tid, PrevSpecs end, {Specs, - State#?MODULE{range = ra_range:truncate(Idx - 1, Range), + State#?MODULE{indexes = ra_range:truncate(Idx - 1, Range), prev = Prev}}; set_first(_Idx, State) -> {[], State}. @@ -434,12 +446,19 @@ set_first(_Idx, State) -> %% internal -update_range_end(Idx, {Start, End}) - when Idx =< End orelse - Idx == End + 1 -> - {Start, Idx}; -update_range_end(Idx, undefined) -> - {Idx, Idx}. +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. + + % when Idx =< End orelse + % Idx == End + 1 -> + % {Start, Idx}; +% update_ra_seq(Idx, undefined) -> + % {Idx, Idx}. delete(End, End, Tid) -> ets:delete(Tid, End); diff --git a/src/ra_seq.erl b/src/ra_seq.erl index b180e747..715926e7 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -31,7 +31,9 @@ first/1, last/1, iterator/1, - next/1 + next/1, + length/1, + in/2 ]). -spec append(ra:index(), state()) -> state(). @@ -164,8 +166,33 @@ next(#i{seq = [{Next, End} | Rem]}) -> {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. + + %% Internal functions + drop_prefix({IDX, PI}, {IDX, SI}) -> drop_prefix(next(PI), next(SI)); drop_prefix(end_of_seq, {Idx, #i{seq = RevSeq}}) -> @@ -173,7 +200,7 @@ drop_prefix(end_of_seq, {Idx, #i{seq = RevSeq}}) -> drop_prefix({PrefIdx, PI}, {Idx, _SI} = I) when PrefIdx < Idx -> drop_prefix(next(PI), I); -drop_prefix({PrefIdx, PI}, {Idx, _SI} = I) +drop_prefix({PrefIdx, _PI}, {Idx, _SI}) when Idx < PrefIdx -> {error, not_prefix}. diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl index 31e359a1..4a14f4ac 100644 --- a/test/ra_mt_SUITE.erl +++ b/test/ra_mt_SUITE.erl @@ -33,7 +33,7 @@ all_tests() -> successor_below, stage_commit, range_overlap, - stage_commit_abort, + stage_commit_2, perf, sparse ]. @@ -387,7 +387,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( @@ -397,8 +397,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)), From 5a3ba92b067bb538227938c48744bc312fb7e3c9 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 23 Apr 2025 09:59:27 +0100 Subject: [PATCH 17/22] ra mt sparse compat --- src/ra_mt.erl | 156 +++++++++++++++++++++---------------------- src/ra_seq.erl | 10 ++- test/ra_mt_SUITE.erl | 87 ++++++++++++------------ 3 files changed, 129 insertions(+), 124 deletions(-) diff --git a/src/ra_mt.erl b/src/ra_mt.erl index 294fdc75..bb24a188 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -37,10 +37,10 @@ -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 @@ -67,7 +67,7 @@ -type delete_spec() :: undefined | {'<', ets:tid(), ra:index()} | {delete, ets:tid()} | - {range, ets:tid(), ra:range()}. + {indexes, ets:tid(), ra_seq:state()}. -export_type([ state/0, delete_spec/0 @@ -123,21 +123,19 @@ insert({Idx, _, _} = _Entry, {ok, state()} | {error, gap_detected | limit_reached}. insert_sparse({Idx, _, _} = Entry, LastIdx, #?MODULE{tid = Tid, - indexes = Seq} = State) - when Seq == undefined orelse - LastIdx == undefined orelse - ((is_integer(hd(Seq)) andalso hd(Seq) == LastIdx) orelse - (LastIdx == element(2, hd(Seq)))) -> - case ra_range:size(Seq) > ?MAX_MEMTBL_ENTRIES of + indexes = Seq} = State) -> + case ra_seq:last(Seq) == LastIdx of true -> - {error, limit_reached}; + 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 -> - true = ets:insert(Tid, Entry), - {ok, State#?MODULE{indexes = ra_seq:append(Idx, Seq)}} - end; -insert_sparse({_Idx, _, _} = _Entry, _LastIdx, - #?MODULE{indexes = _Seq} = _State0) -> - {error, gap_detected}. + {error, gap_detected} + end. -spec stage(log_entry(), state()) -> {ok, state()} | {error, overwriting | limit_reached}. @@ -294,8 +292,11 @@ get_items(Indexes, #?MODULE{} = State) -> non_neg_integer(). delete(undefined) -> 0; -delete({range, Tid, {Start, End}}) -> - NumToDelete = End - Start + 1, +delete({indexes, Tid, Seq}) -> + NumToDelete = ra_seq:length(Seq), + Start = ra_seq:first(Seq), + End = ra_seq:last(Seq), + % NumToDelete = End - Start + 1, Limit = ets:info(Tid, size) div 2, %% check if there is an entry below the start of the deletion range, %% if there is we've missed a segment event at some point and need @@ -306,8 +307,11 @@ delete({range, Tid, {Start, End}}) -> %% more than half the table is to be deleted delete({'<', Tid, End + 1}); false -> - delete(Start, End, Tid), - End - Start + 1 + _ = ra_seq:fold(fun (I, Acc) -> + _ = ets:delete(Tid, I), + Acc + end, undefined, Seq), + NumToDelete end; delete({Op, Tid, Idx}) when is_integer(Idx) and is_atom(Op) -> @@ -335,11 +339,15 @@ range_overlap(ReqRange, #?MODULE{} = State) -> -spec range(state()) -> undefined | {ra:index(), ra:index()}. -range(#?MODULE{indexes = Range, +range(#?MODULE{indexes = Seq, prev = undefined}) -> - Range; -range(#?MODULE{indexes = {_, End} = Range, + ra_seq:range(Seq); +range(#?MODULE{indexes = []}) -> + undefined; +range(#?MODULE{indexes = Seq, prev = Prev}) -> + End = ra_seq:last(Seq), + Range = ra_seq:range(Seq), PrevRange = ra_range:limit(End, range(Prev)), ra_range:add(Range, PrevRange); range(_State) -> @@ -378,22 +386,23 @@ info(#?MODULE{tid = Tid, record_flushed(TID = Tid, FlushedSeq, #?MODULE{tid = TID, indexes = Seq} = State) -> - HasExtraEntries = ets:info(Tid, size) > ra_seq:length(Seq), - case ?IN_RANGE(End, Seq) of - true when HasExtraEntries -> - {{'<', Tid, End + 1}, - State#?MODULE{indexes = ra_range:truncate(End, Seq)}}; + % HasExtraEntries = ets:info(Tid, size) > ra_seq:length(Seq), + End = ra_seq:last(FlushedSeq), + case ra_seq:in(End, Seq) of + % true when HasExtraEntries -> + % {{'<', Tid, End + 1}, + % State#?MODULE{indexes = ra_range:truncate(End, Seq)}}; true -> - {{range, Tid, {Start, End}}, - State#?MODULE{indexes = ra_range:truncate(End, Seq)}}; + {{indexes, Tid, FlushedSeq}, + State#?MODULE{indexes = ra_seq:floor(End + 1, Seq)}}; false -> {undefined, State} end; -record_flushed(_Tid, _Range, #?MODULE{prev = undefined} = State) -> +record_flushed(_Tid, _FlushedSeq, #?MODULE{prev = undefined} = State) -> {undefined, State}; -record_flushed(Tid, Range, #?MODULE{prev = Prev0} = State) -> +record_flushed(Tid, FlushedSeq, #?MODULE{prev = Prev0} = State) -> %% TODO: test many levels deep flushes - {Spec, Prev} = record_flushed(Tid, Range, Prev0), + {Spec, Prev} = record_flushed(Tid, FlushedSeq, Prev0), case range(Prev) of undefined -> %% the prev table is now empty and can be deleted, @@ -405,46 +414,43 @@ record_flushed(Tid, Range, #?MODULE{prev = Prev0} = State) -> -spec set_first(ra:index(), state()) -> {[delete_spec()], state()}. set_first(Idx, #?MODULE{tid = Tid, - indexes = Range, - prev = Prev0} = State) - when (is_tuple(Range) andalso - Idx > element(1, Range)) orelse - Range == undefined -> - {PrevSpecs, Prev} = case Prev0 of + indexes = Seq, + prev = Prev0} = State) -> + {PrevSpecs, Prev} = + case Prev0 of + undefined -> + {[], undefined}; + _ -> + case set_first(Idx, Prev0) of + {[{indexes, PTID, _} | Rem], + #?MODULE{tid = PTID} = P} = Res -> + %% set_first/2 returned a range spec for + %% prev and prev is now empty, + %% upgrade to delete spec of whole tid + case range(P) of undefined -> - {[], undefined}; + {[{delete, tid(P)} | Rem], + prev(P)}; _ -> - case set_first(Idx, Prev0) of - {[{range, PTID, _} | Rem], - #?MODULE{tid = PTID} = P} = Res -> - %% set_first/2 returned a range spec for - %% prev and prev is now empty, - %% upgrade to delete spec of whole tid - case range(P) of - undefined -> - {[{delete, tid(P)} | Rem], - prev(P)}; - _ -> - Res - end; - Res -> - Res - end - end, - Specs = case Range of - {Start, End} -> - [{range, Tid, {Start, min(Idx - 1, End)}} | PrevSpecs]; - undefined -> - PrevSpecs + Res + end; + Res -> + Res + end + end, + Specs = case Seq of + [] -> + PrevSpecs; + _ -> + DeleteSeq = ra_seq:limit(Idx - 1, Seq), + [{indexes, Tid, DeleteSeq} | PrevSpecs] end, {Specs, - State#?MODULE{indexes = ra_range:truncate(Idx - 1, Range), - prev = Prev}}; -set_first(_Idx, State) -> - {[], State}. + State#?MODULE{indexes = ra_seq:floor(Idx, Seq), + prev = Prev}}. -%% internal +%% Internal update_ra_seq(Idx, Seq) -> case ra_seq:last(Seq) of @@ -454,18 +460,6 @@ update_ra_seq(Idx, Seq) -> ra_seq:append(Idx, Seq) end. - % when Idx =< End orelse - % Idx == End + 1 -> - % {Start, Idx}; -% update_ra_seq(Idx, undefined) -> - % {Idx, Idx}. - -delete(End, End, Tid) -> - ets:delete(Tid, End); -delete(Start, End, Tid) -> - _ = ets:delete(Tid, Start), - delete(Start+1, End, Tid). - read_sparse(Indexes, State, Acc) -> read_sparse(Indexes, State, 0, Acc). diff --git a/src/ra_seq.erl b/src/ra_seq.erl index 715926e7..17bd4719 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -33,7 +33,8 @@ iterator/1, next/1, length/1, - in/2 + in/2, + range/1 ]). -spec append(ra:index(), state()) -> state(). @@ -189,6 +190,13 @@ in(Idx, [Range | Rem]) -> in(Idx, Rem) end. +-spec range(state()) -> ra:range(). +range([]) -> + undefined; +range(Seq) -> + ra_range:new(first(Seq), last(Seq)). + + %% Internal functions diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl index 4a14f4ac..6d8df7af 100644 --- a/test/ra_mt_SUITE.erl +++ b/test/ra_mt_SUITE.erl @@ -74,7 +74,7 @@ basics(_Config) -> 499 = ra_mt:delete(Spec), ?assertEqual({500, 1000}, ra_mt:range(Mt2)), ?assertEqual(501, ets:info(Tid, size)), - {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 999}, Mt2), + {Spec2, Mt3} = ra_mt:record_flushed(Tid, [{1, 999}], Mt2), 500 = ra_mt:delete(Spec2), ?assertEqual(1, ra_mt:lookup_term(1000, Mt3)), ok. @@ -87,15 +87,15 @@ record_flushed(_Config) -> fun (I, Acc) -> element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc)) end, Mt0, lists:seq(1, 100)), - {Spec, Mt2} = ra_mt:record_flushed(Tid, {1, 49}, Mt1), - ?assertMatch({range, _, {1, 49}}, Spec), + {Spec, Mt2} = ra_mt:record_flushed(Tid, [{1, 49}], Mt1), + ?assertMatch({indexes, _, [{1, 49}]}, Spec), ?assertMatch({50, 100}, ra_mt:range(Mt2)), _ = ra_mt:delete(Spec), - {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 49}, Mt2), + {Spec2, Mt3} = ra_mt:record_flushed(Tid, [{1, 49}], Mt2), ?assertMatch(undefined, Spec2), _ = ra_mt:delete(Spec2), - {Spec3, Mt4} = ra_mt:record_flushed(Tid, {50, 100}, Mt3), - ?assertMatch({range, _, {50, 100}}, Spec3), + {Spec3, Mt4} = ra_mt:record_flushed(Tid, [{50, 100}], Mt3), + ?assertMatch({indexes, _, [{50, 100}]}, Spec3), ?assertEqual(undefined, ra_mt:range(Mt4)), _ = ra_mt:delete(Spec3), ?assertMatch(#{size := 0}, ra_mt:info(Mt4)), @@ -108,14 +108,14 @@ record_flushed_after_set_first(_Config) -> fun (I, Acc) -> element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc)) end, Mt0, lists:seq(1, 100)), - {Spec, Mt2} = ra_mt:record_flushed(Tid, {1, 49}, Mt1), - ?assertMatch({range, _, {1, 49}}, Spec), + {Spec, Mt2} = ra_mt:record_flushed(Tid, [{1, 49}], Mt1), + ?assertMatch({indexes, _, [{1, 49}]}, Spec), ?assertMatch({50, 100}, ra_mt:range(Mt2)), _ = ra_mt:delete(Spec), {[Spec2], Mt3} = ra_mt:set_first(150, Mt2), - ?assertMatch({range, Tid, {50, 100}}, Spec2), + ?assertMatch({indexes, Tid, [{50, 100}]}, Spec2), ?assertMatch(undefined, ra_mt:range(Mt3)), - {undefined, Mt4} = ra_mt:record_flushed(Tid, {1, 49}, Mt3), + {undefined, Mt4} = ra_mt:record_flushed(Tid, [{1, 49}], Mt3), ?assertMatch(undefined, ra_mt:range(Mt4)), ok. @@ -135,14 +135,14 @@ record_flushed_prev(_Config) -> end, Mt2, lists:seq(50, 80)), ?assertMatch({1, 100}, ra_mt:range(ra_mt:prev(Mt3))), %% - {Spec, Mt4} = ra_mt:record_flushed(Tid, {1, 49}, Mt3), - ?assertMatch({range, Tid, {1, 49}}, Spec), + {Spec, Mt4} = ra_mt:record_flushed(Tid, [{1, 49}], Mt3), + ?assertMatch({indexes, Tid, [{1, 49}]}, Spec), ?assertMatch({50, 80}, ra_mt:range(Mt4)), ?assertMatch({50, 100}, ra_mt:range(ra_mt:prev(Mt4))), _ = ra_mt:delete(Spec), %% delete the remainder of the old mt - {Spec2, Mt5} = ra_mt:record_flushed(Tid, {50, 100}, Mt4), + {Spec2, Mt5} = ra_mt:record_flushed(Tid, [{50, 100}], Mt4), ?assertMatch({delete, Tid}, Spec2), ?assertEqual(undefined, ra_mt:prev(Mt5)), ?assertMatch({50, 80}, ra_mt:range(Mt5)), @@ -164,12 +164,12 @@ set_first(_Config) -> element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc)) end, Mt2, lists:seq(50, 120)), {[Spec1, Spec2], Mt4} = ra_mt:set_first(75, Mt3), - ?assertMatch({range, Tid2, {50, 74}}, Spec1), - ?assertMatch({range, Tid, {1, 74}}, Spec2), + ?assertMatch({indexes, Tid2, [{50, 74}]}, Spec1), + ?assertMatch({indexes, Tid, [{1, 74}]}, Spec2), ?assertMatch({75, 120}, ra_mt:range(Mt4)), {[Spec3, Spec4], Mt5} = ra_mt:set_first(105, Mt4), - ?assertMatch({range, Tid2, {75, 104}}, Spec3), + ?assertMatch({indexes, Tid2, [{75, 104}]}, Spec3), ?assertMatch({delete, Tid}, Spec4), ?assertMatch({105, 120}, ra_mt:range(Mt5)), ?assertMatch(undefined, ra_mt:prev(Mt5)), @@ -199,11 +199,11 @@ set_first_with_multi_prev(_Config) -> ?assertEqual({1, 200}, ra_mt:range(Mt3)), - {[{range, Tid3, {75, 79}}, - {range, Tid2, {50, 79}}, - {range, Tid1, {1, 79}}], Mt4} = ra_mt:set_first(80, Mt3), + {[{indexes, Tid3, [{75, 79}]}, + {indexes, Tid2, [{50, 79}]}, + {indexes, Tid1, [{1, 79}]}], Mt4} = ra_mt:set_first(80, Mt3), - {[{range, Tid3, {80, 159}}, + {[{indexes, Tid3, [{80, 159}]}, {delete, Tid2}, {delete, Tid1}], _Mt5} = ra_mt:set_first(160, Mt4), ok. @@ -234,12 +234,12 @@ set_first_with_middle_small_range(_Config) -> ?assertEqual({1, 200}, ra_mt:range(Mt3)), - {[{range, Tid3, {75, 84}}, + {[{indexes, Tid3, [{75, 84}]}, {delete, Tid2}, - {range, Tid1, {1, 84}}], Mt4} = ra_mt:set_first(85, Mt3), + {indexes, Tid1, [{1, 84}]}], Mt4} = ra_mt:set_first(85, Mt3), ?assertEqual({85, 200}, ra_mt:range(Mt4)), - {[{range, Tid3, {85, 100}}, + {[{indexes, Tid3, [{85, 100}]}, {delete, Tid1}], Mt5} = ra_mt:set_first(101, Mt4), ?assertEqual({101, 200}, ra_mt:range(Mt5)), ?assertEqual(undefined, ra_mt:prev(Mt5)), @@ -262,11 +262,16 @@ set_first_with_old_larger_range(_Config) -> element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc)) end, ra_mt:init_successor(Tid2, read_write, Mt1), lists:seq(50, 75)), - {[{range, Tid2, {50, 75}}, - {range, Tid1, {1, 84}}], Mt3} = ra_mt:set_first(85, Mt2), + {[{indexes, Tid2, [{50, 75}]}, + {indexes, Tid1, [{1, 84}]}], Mt3} = ra_mt:set_first(85, Mt2), + ct:pal("Mt3 ~p", [Mt3]), ?assertEqual(undefined, ra_mt:range(Mt3)), %% eventually when set_first passes the end of the old range it gets %% deleted + % debugger:start(), + % int:i(ra_mt), + % int:break(ra_mt, 419), + {[{delete, Tid1}], Mt4} = ra_mt:set_first(101, Mt3), ?assertEqual(undefined, ra_mt:prev(Mt4)), ok. @@ -287,7 +292,7 @@ set_first_with_old_smaller_range(_Config) -> lists:seq(1, 100)), ?assertEqual({1, 100}, ra_mt:range(Mt2)), - {[{range, Tid2, {1, 84}}, + {[{indexes, Tid2, [{1, 84}]}, {delete, Tid1}], Mt3} = ra_mt:set_first(85, Mt2), ?assertEqual({85, 100}, ra_mt:range(Mt3)), ok. @@ -313,10 +318,10 @@ successor(_Config) -> ?assertMatch({I, T, _}, ra_mt:lookup(I, Mt3)) end, lists:seq(1, 100)), - {{range, Tid, {1, 20}}, Mt4a} = ra_mt:record_flushed(Tid, {1, 20}, Mt3), + {{indexes, Tid, [{1, 20}]}, Mt4a} = ra_mt:record_flushed(Tid, [{1, 20}], Mt3), ?assertMatch({21, 120}, ra_mt:range(Mt4a)), - {{range, Tid, {1, 60}}, Mt4b} = ra_mt:record_flushed(Tid, {1, 60}, Mt3), + {{indexes, Tid, [{1, 60}]}, Mt4b} = ra_mt:record_flushed(Tid, [{1, 60}], Mt3), ?assertMatch({50, 120}, ra_mt:range(Mt4b)), ok. @@ -337,16 +342,16 @@ successor_below(_Config) -> end, Mt2, lists:seq(50, 75)), ?assertMatch({50, 75}, ra_mt:range(Mt3)), - {{range, Tid, {100, 150}}, Mt4a} = - ra_mt:record_flushed(Tid, {100, 150}, Mt3), + {{indexes, Tid, [{100, 150}]}, Mt4a} = + ra_mt:record_flushed(Tid, [{100, 150}], Mt3), ?assertMatch({50, 75}, ra_mt:range(Mt4a)), - {{range, Tid2, {50, 60}}, Mt4b} = - ra_mt:record_flushed(Tid2, {50, 60}, Mt3), + {{indexes, Tid2, [{50, 60}]}, Mt4b} = + ra_mt:record_flushed(Tid2, [{50, 60}], Mt3), ?assertMatch({61, 75}, ra_mt:range(Mt4b)), {{delete, Tid}, Mt4c} = - ra_mt:record_flushed(Tid, {100, 200}, Mt3), + ra_mt:record_flushed(Tid, [{100, 200}], Mt3), ?assertMatch({50, 75}, ra_mt:range(Mt4c)), ?assertMatch(#{has_previous := false}, ra_mt:info(Mt4c)), ok. @@ -467,20 +472,18 @@ 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">>}, 0, Mt0), + {ok, Mt1} = ra_mt:insert_sparse({2, 1, <<"banana">>}, undefined, Mt0), {ok, Mt2} = ra_mt:insert_sparse({5, 1, <<"banana">>}, 2, Mt1), ?assertEqual({2, 5}, ra_mt:range(Mt2)), - {Spec, _Mt3} = ra_mt:record_flushed(Tid, {2, 5}, Mt2), - ct:pal("Spec ~p", [Spec]), - % 499 = ra_mt:delete(Spec)2 - % ?assertEqual({500, 1000}, ra_mt:range(Mt2)), - % ?assertEqual(501, ets:info(Tid, size)), - % {Spec2, Mt3} = ra_mt:record_flushed(Tid, {1, 999}, Mt2), - % 500 = ra_mt:delete(Spec2), - % ?assertEqual(1, ra_mt:lookup_term(1000, Mt3)), + {Spec, Mt3} = ra_mt:record_flushed(Tid, ra_seq:from_list([2, 5]), Mt2), + 2 = ra_mt:delete(Spec), + ?assertMatch(#{size := 0, + range := undefined}, ra_mt:info(Mt3)), + ?assertEqual(0, ets:info(Tid, size)), ok. %%% Util From 7a217495c803851d775cc4a8ab076c66d3548d22 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 28 Apr 2025 14:47:48 +0100 Subject: [PATCH 18/22] wip --- src/ra_log.erl | 148 +++++++++++++++++++-------- src/ra_log_segment_writer.erl | 10 +- src/ra_seq.erl | 25 ++--- src/ra_server_proc.erl | 5 +- src/ra_snapshot.erl | 42 +++++--- test/ra_checkpoint_SUITE.erl | 57 ++++++----- test/ra_log_2_SUITE.erl | 19 ++-- test/ra_log_SUITE.erl | 27 +++-- test/ra_log_segment_writer_SUITE.erl | 6 +- test/ra_seq_SUITE.erl | 11 +- test/ra_snapshot_SUITE.erl | 39 +++---- 11 files changed, 248 insertions(+), 141 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 46ed9943..844781f7 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -66,7 +66,9 @@ -type event_body() :: {written, ra_term(), ra_seq:state()} | {segments, [{ets:tid(), ra:range()}], [segment_ref()]} | {resend_write, ra_index()} | - {snapshot_written, ra_idxterm(), ra_snapshot:kind()} | + {snapshot_written, ra_idxterm(), + LiveIndexes :: ra_seq:state(), + ra_snapshot:kind()} | {down, pid(), term()}. -type event() :: {ra_log_event, event_body()}. @@ -107,7 +109,8 @@ last_wal_write :: {pid(), Ms :: integer()}, reader :: ra_log_reader:state(), mem_table :: ra_mt:state(), - tx = false :: boolean() + tx = false :: boolean(), + pending = [] :: ra_seq:state() }). -record(read_plan, {dir :: file:filename_all(), @@ -362,6 +365,7 @@ append({Idx, Term, Cmd0} = Entry, wal = Wal} = Cfg, last_index = LastIdx, tx = false, + pending = Pend0, mem_table = Mt0} = State) when Idx =:= LastIdx + 1 -> case ra_mt:insert(Entry, Mt0) of @@ -370,11 +374,13 @@ append({Idx, Term, Cmd0} = Entry, case ra_log_wal:write(Wal, {UId, self()}, ra_mt:tid(Mt), Idx, Term, Cmd) of {ok, Pid} -> + Pend = ra_seq:limit(Idx - 1, Pend0), ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), State#?MODULE{last_index = Idx, last_term = Term, last_wal_write = {Pid, now_ms()}, + pending = ra_seq:append(Idx, Pend), mem_table = Mt}; {error, wal_down} -> error(wal_down) @@ -392,6 +398,7 @@ append({Idx, Term, _Cmd} = Entry, #?MODULE{cfg = Cfg, last_index = LastIdx, tx = true, + pending = Pend0, mem_table = Mt0} = State) when Idx =:= LastIdx + 1 -> case ra_mt:stage(Entry, Mt0) of @@ -399,6 +406,7 @@ append({Idx, Term, _Cmd} = Entry, put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), State#?MODULE{last_index = Idx, last_term = Term, + pending = ra_seq:append(Idx, Pend0), mem_table = Mt}; {error, Reason} -> ?DEBUG("~ts: mem table ~s detected appending index ~b, tx=true " @@ -420,12 +428,15 @@ append({Idx, _, _}, #?MODULE{last_index = LastIdx}) -> write([{FstIdx, _, _} | _Rest] = Entries, #?MODULE{cfg = Cfg, last_index = LastIdx, + pending = Pend0, mem_table = Mt0} = State0) when FstIdx =< LastIdx + 1 andalso FstIdx >= 0 -> case stage_entries(Cfg, Entries, Mt0) of {ok, Mt} -> - wal_write_batch(State0#?MODULE{mem_table = Mt}, Entries); + Pend = ra_seq:limit(FstIdx - 1, Pend0), + wal_write_batch(State0#?MODULE{mem_table = Mt, + pending = Pend}, Entries); Error -> Error end; @@ -554,8 +565,9 @@ sparse_read(Indexes0, #?MODULE{cfg = Cfg, %% read a list of indexes, %% found indexes be returned in the same order as the input list of indexes -spec partial_read([ra_index()], state(), - fun ((ra_index(), ra_term(), ra_server:command()) -> term()) - ) -> + fun ((ra_index(), + ra_term(), + ra_server:command()) -> term())) -> read_plan(). partial_read(Indexes0, #?MODULE{cfg = Cfg, reader = Reader0, @@ -645,6 +657,52 @@ set_last_index(Idx, #?MODULE{cfg = Cfg, -spec handle_event(event_body(), state()) -> {state(), [effect()]}. +handle_event({written, Term, WrittenSeq}, + #?MODULE{cfg = Cfg, + first_index = FirstIdx, + pending = Pend0} = State0) -> + %% gap detection + %% 1. pending has lower indexes than the ra_seq:first index in WrittenSeq + %% 2. + LastWrittenIdx = ra_seq:last(WrittenSeq), + case fetch_term(LastWrittenIdx, State0) of + {Term, State} when is_integer(Term) -> + ok = put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, + 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]), + exit({not_prefix, WrittenSeq, Pend0}) + end; + {undefined, State} when LastWrittenIdx < FirstIdx -> + % A snapshot happened before the written event came in + % This can only happen on a leader when consensus is achieved by + % followers returning appending the entry and the leader committing + % and processing a snapshot before the written event comes in. + % + % At this point the items may still be in pending so need to + % remove them + {ok, Pend} = ra_seq:remove_prefix(WrittenSeq, Pend0), + {State#?MODULE{pending = Pend}, []}; + {OtherTerm, State} -> + %% term mismatch, let's reduce the seq and try again to see + %% if any entries in the range are valid + case ra_seq:limit(LastWrittenIdx - 1, WrittenSeq) of + [] -> + ?DEBUG("~ts: written event did not find term ~b for index ~b " + "found ~w", + [Cfg#cfg.log_id, Term, LastWrittenIdx, OtherTerm]), + {State, []}; + NewWrittenSeq -> + %% retry with a reduced range + handle_event({written, Term, NewWrittenSeq}, State0) + end + end; handle_event({written, _Term, [{FromIdx, _ToIdx}]}, #?MODULE{last_index = LastIdx} = State) when FromIdx > LastIdx -> @@ -732,36 +790,26 @@ handle_event({segments, TidRanges, NewSegs}, %% (new -> old) so we need to foldr here to process the oldest first Mt = lists:foldr( fun ({Tid, Seq}, Acc0) -> - %% TODO: should mt take ra_seqs in case of sparse writes? - Range = case Seq of - [] -> - undefined; - [{_, _} = R] -> - R; - _ -> - ra_range:new(ra_seq:first(Seq), - ra_seq:last(Seq)) - end, - % ct:pal("Range ~p Seq ~p", [Range, Seq]), - {Spec, Acc} = ra_mt:record_flushed(Tid, Range, Acc0), + {Spec, Acc} = ra_mt:record_flushed(Tid, Seq, Acc0), ok = ra_log_ets:execute_delete(Names, UId, Spec), Acc end, Mt0, TidRanges), State = State0#?MODULE{reader = Reader, mem_table = Mt}, {State, []}; -handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind}, +handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, first_index = FstIdx, last_index = LstIdx, mem_table = Mt0, - % reader = Reader, + pending = Pend0, last_written_index_term = {LastWrittenIdx, _} = LWIdxTerm0, snapshot_state = SnapState0} = State0) %% only update snapshot if it is newer than the last snapshot when SnapIdx >= FstIdx -> - SnapState1 = ra_snapshot:complete_snapshot(Snap, SnapKind, SnapState0), + SnapState1 = ra_snapshot:complete_snapshot(Snap, SnapKind, LiveIndexes, + SnapState0), case SnapKind of snapshot -> put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx), @@ -800,12 +848,26 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, SnapKind}, %% NB the mt cannot be truncated here with set_first as we need %% to keep any live indexes % SegRefs = ra_log_reader:segment_refs(Reader), + % + + %% remove all pending below smallest live index as the wal + %% may not write them + %% TODO: test that a written even can still be processed if it + %% contains lower indexes than pending + SmallestLiveIdx = case ra_seq:first(LiveIndexes) of + undefined -> + SnapIdx + 1; + I -> + I + end, + Pend = ra_seq:floor(SmallestLiveIdx, Pend0), Effects = Effects0 ++ Effects1, {State#?MODULE{first_index = SnapIdx + 1, last_index = max(LstIdx, SnapIdx), last_written_index_term = LWIdxTerm, mem_table = Mt1, + pending = Pend, snapshot_state = SnapState}, Effects}; checkpoint -> put_counter(Cfg, ?C_RA_SVR_METRIC_CHECKPOINT_INDEX, SnapIdx), @@ -817,7 +879,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 @@ -851,11 +913,11 @@ next_index(#?MODULE{last_index = LastIdx}) -> -spec fetch(ra_index(), state()) -> {option(log_entry()), state()}. fetch(Idx, State0) -> - case fold(Idx, Idx, fun(E, Acc) -> [E | Acc] end, [], State0) of - {[], State} -> - {undefined, State}; + case sparse_read([Idx], State0) of {[Entry], State} -> - {Entry, State} + {Entry, State}; + {[], State} -> + {undefined, State} end. -spec fetch_term(ra_index(), state()) -> @@ -1093,7 +1155,8 @@ overview(#?MODULE{last_index = LastIndex, snapshot_state = SnapshotState, reader = Reader, last_wal_write = {_LastPid, LastMs}, - mem_table = Mt + mem_table = Mt, + pending = Pend }) -> CurrSnap = ra_snapshot:current(SnapshotState), #{type => ?MODULE, @@ -1109,9 +1172,9 @@ overview(#?MODULE{last_index = LastIndex, {I, _} -> I end, snapshot_term => case CurrSnap of - undefined -> undefined; - {_, T} -> T - end, + undefined -> undefined; + {_, T} -> T + end, latest_checkpoint_index => case ra_snapshot:latest_checkpoint(SnapshotState) of undefined -> undefined; @@ -1119,7 +1182,8 @@ overview(#?MODULE{last_index = LastIndex, end, mem_table_range => ra_mt:range(Mt), mem_table_info => ra_mt:info(Mt), - last_wal_write => LastMs + last_wal_write => LastMs, + num_pending => ra_seq:length(Pend) }. -spec write_config(ra_server:config(), state()) -> ok. @@ -1225,17 +1289,18 @@ wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, + pending = Pend0, mem_table = Mt0} = State, Entries) -> WriterId = {UId, self()}, %% all entries in a transaction are written to the same tid Tid = ra_mt:tid(Mt0), - {WalCommands, Num} = - lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N}) -> + {WalCommands, Num, Pend} = + lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N, P}) -> Cmd = {ttb, term_to_iovec(Cmd0)}, WalC = {append, WriterId, Tid, Idx-1, Idx, Term, Cmd}, - {[WalC | WC], N+1} - end, {[], 0}, Entries), + {[WalC | WC], N+1, ra_seq:append(Idx, P)} + end, {[], 0, Pend0}, Entries), [{_, _, _, _PrevIdx, LastIdx, LastTerm, _} | _] = WalCommands, {_, Mt} = ra_mt:commit(Mt0), @@ -1246,7 +1311,8 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, {ok, State#?MODULE{last_index = LastIdx, last_term = LastTerm, last_wal_write = {Pid, now_ms()}, - mem_table = Mt}}; + mem_table = Mt, + pending = Pend}}; {error, wal_down} = Err -> %% if we get there the entry has already been inserted %% into the mem table but never reached the wal @@ -1255,15 +1321,15 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, end. maybe_append_first_entry(State0 = #?MODULE{last_index = -1}) -> - State = append({0, 0, undefined}, State0), + State1 = append({0, 0, undefined}, State0), receive - {ra_log_event, {written, 0, [0]}} -> - ok + {ra_log_event, {written, 0, [0]} = Evt} -> + State2 = State1#?MODULE{first_index = 0}, + {State, _Effs} = handle_event(Evt, State2), + State after 60000 -> exit({?FUNCTION_NAME, timeout}) - end, - State#?MODULE{first_index = 0, - last_written_index_term = {0, 0}}; + end; maybe_append_first_entry(State) -> State. diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index d84ee0ea..4c31790d 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -369,8 +369,7 @@ send_segments(System, ServerUId, TidRanges, SegRefs) -> %% 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 - Range = {ra_seq:first(Seq), ra_seq:last(Seq)}, - _ = catch ra_mt:delete({range, Tid, Range}) + _ = catch ra_mt:delete({indexes, Tid, Seq}) end || {Tid, Seq} <- TidRanges], ok; Pid -> @@ -379,7 +378,12 @@ send_segments(System, ServerUId, TidRanges, SegRefs) -> end. append_to_segment(UId, Tid, Seq0, Seg, State) -> - FirstIdx = ra_seq:first(Seq0), + 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), diff --git a/src/ra_seq.erl b/src/ra_seq.erl index 17bd4719..f23659be 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -62,7 +62,7 @@ from_list(L) -> lists:foldl(fun append/2, [], lists:sort(L)). -spec floor(ra:index(), state()) -> state(). -floor(FloorIdxIncl, Seq) -> +floor(FloorIdxIncl, Seq) when is_list(Seq) -> %% TODO: assert appendable %% for now assume appendable floor0(FloorIdxIncl, Seq, []). @@ -203,8 +203,12 @@ range(Seq) -> 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, lists:reverse([Idx | RevSeq])}; + {ok, add(lists:reverse(RevSeq), [Idx])}; drop_prefix({PrefIdx, PI}, {Idx, _SI} = I) when PrefIdx < Idx -> drop_prefix(next(PI), I); @@ -231,20 +235,3 @@ floor0(FloorIdx, [{_, _} = T | Rem], Acc) -> end; floor0(_FloorIdx, _Seq, Acc) -> lists:reverse(Acc). - - -% last_index([{_, I} | _]) -> -% I; -% last_index([I | _]) -% when is_integer(I) -> -% I; -% last_index([]) -> -% undefined. - -% first_index([{_, I} | _]) -> -% I; -% first_index([I | _]) -% when is_integer(I) -> -% I; -% first_index([]) -> -% undefined. diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index c27f5038..f98e495a 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -927,8 +927,9 @@ receive_snapshot(EventType, Msg, State0) -> TimeoutActions = case Msg of #install_snapshot_rpc{} -> %% Reset timeout only on receive snapshot progress. - [{state_timeout, Conf#conf.receive_snapshot_timeout, - receive_snapshot_timeout}]; + [{state_timeout, + Conf#conf.receive_snapshot_timeout, + receive_snapshot_timeout}]; _ -> [] end, diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index 613d239c..d80bd710 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -30,7 +30,7 @@ begin_snapshot/5, promote_checkpoint/2, - complete_snapshot/3, + complete_snapshot/4, begin_accept/2, accept_chunk/4, @@ -377,7 +377,7 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacMod, MacState, SnapKind, %% TODO: really we'd like to run this in the ra worker as good potentially %% be quite large and a touch expensive to compute but also we don't want %% to close over both the MacState and the MacRef - Indexes = ra_machine:live_indexes(MacMod, MacState), + LiveIndexes = ra_seq:from_list(ra_machine:live_indexes(MacMod, MacState)), %% call prepare then write_snapshotS %% This needs to be called in the current process to "lock" potentially %% mutable machine state @@ -395,15 +395,17 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacMod, MacState, SnapKind, ok end, %% write the live indexes, if any - case Indexes of + case LiveIndexes of [] -> ok; _ -> - F = filename:join(SnapDir, "indexes"), - ok = ra_lib:write_file(F, term_to_binary(Indexes), true), + Data = term_to_binary(LiveIndexes), + F = filename:join(SnapDir, <<"indexes">>), + ok = ra_lib:write_file(F, Data, true), ok end, Self ! {ra_log_event, - {snapshot_written, IdxTerm, SnapKind}}, + {snapshot_written, IdxTerm, + LiveIndexes, SnapKind}}, ok end, @@ -434,9 +436,16 @@ promote_checkpoint(PromotionIdx, %% into a snapshot. ok = Mod:sync(Checkpoint), ok = prim_file:rename(Checkpoint, Snapshot), + F = filename:join(SnapDir, <<"indexes">>), + Indexes = case file:read_file(F) of + {ok, Bin} -> + binary_to_term(Bin); + _ -> + [] + end, Self ! {ra_log_event, - {snapshot_written, - {Idx, Term}, snapshot}} + {snapshot_written, {Idx, Term}, + Indexes, snapshot}} end, State = State0#?MODULE{pending = {{Idx, Term}, snapshot}, @@ -465,20 +474,27 @@ find_promotable_checkpoint(Idx, [CP | Rest], Acc) -> find_promotable_checkpoint(_Idx, [], _Acc) -> undefined. --spec complete_snapshot(ra_idxterm(), kind(), state()) -> +-spec complete_snapshot(ra_idxterm(), kind(), ra_seq:state(), state()) -> state(). -complete_snapshot(_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, +complete_snapshot({Idx, _} = IdxTerm, snapshot, LiveIndexes, #?MODULE{uid = UId} = State) -> + SmallestIdx = case ra_seq:first(LiveIndexes) of + undefined -> + Idx + 1; + I -> + I + end, %% TODO live indexes - ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, Idx+1, []), + 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]}. diff --git a/test/ra_checkpoint_SUITE.erl b/test/ra_checkpoint_SUITE.erl index 915ae1f1..375be3cb 100644 --- a/test/ra_checkpoint_SUITE.erl +++ b/test/ra_checkpoint_SUITE.erl @@ -92,8 +92,9 @@ take_checkpoint(Config) -> {{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 @@ -140,8 +141,9 @@ recover_from_checkpoint_only(Config) -> 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) @@ -168,8 +170,9 @@ recover_from_checkpoint_and_snapshot(Config) -> 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, @@ -181,8 +184,9 @@ recover_from_checkpoint_and_snapshot(Config) -> 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) @@ -210,8 +214,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> 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, @@ -223,8 +228,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> 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, @@ -236,8 +242,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> 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, @@ -249,8 +256,9 @@ newer_snapshot_deletes_older_checkpoints(Config) -> 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, @@ -285,8 +293,8 @@ init_recover_corrupt(Config) -> 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, @@ -298,8 +306,8 @@ init_recover_corrupt(Config) -> 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) @@ -332,8 +340,8 @@ init_recover_multi_corrupt(Config) -> 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, @@ -345,8 +353,9 @@ init_recover_multi_corrupt(Config) -> 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, diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index e009074d..53f3b4b1 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -401,7 +401,7 @@ sparse_read_out_of_range_2(Config) -> <<"snap@10">>, Log1), run_effs(Effs), {Log3, _} = receive - {ra_log_event, {snapshot_written, {10, 2}, + {ra_log_event, {snapshot_written, {10, 2}, _, snapshot} = Evt} -> ra_log:handle_event(Evt, Log2) after 5000 -> @@ -537,14 +537,16 @@ written_event_after_snapshot(Config) -> <<"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), @@ -554,7 +556,7 @@ written_event_after_snapshot(Config) -> 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) @@ -896,6 +898,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!!") @@ -1067,7 +1070,7 @@ snapshot_written_after_installation(Config) -> <<"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 -> @@ -1113,7 +1116,7 @@ oldcheckpoints_deleted_after_snapshot_install(Config) -> {Log2, Effs} = ra_log:checkpoint(5, #{}, ?MODULE, <<"one-five">>, Log1), run_effs(Effs), DelayedSnapWritten = receive - {ra_log_event, {snapshot_written, {5, 1}, + {ra_log_event, {snapshot_written, {5, 1}, _, checkpoint} = Evt} -> Evt after 1000 -> @@ -1729,13 +1732,15 @@ create_snapshot_chunk(Config, #{index := Idx} = Meta, Context) -> Sn0 = ra_snapshot:init(<<"someotheruid_adsfasdf">>, ra_log_snapshot, OthDir, CPDir, undefined, ?DEFAULT_MAX_CHECKPOINTS), MacState = <<"9">>, + 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, diff --git a/test/ra_log_SUITE.erl b/test/ra_log_SUITE.erl index 994ef491..e6dd5984 100644 --- a/test/ra_log_SUITE.erl +++ b/test/ra_log_SUITE.erl @@ -103,16 +103,22 @@ write_sparse_then_fetch(Config) -> Log0 = ?config(ra_log, Config), Term = 1, Idx = ra_log:next_index(Log0), - LastIdx = Idx + 5, + Idx5 = Idx + 5, Entry1 = {Idx, Term, "entry"}, %% sparse - Entry2 = {Idx+5, Term, "entry+5"}, - Log1 = ra_log:write_sparse(Entry1, undefined, Log0), + Entry2 = {Idx5, Term, "entry+5"}, + + {LastIdx0, _} = ra_log:last_index_term(Log0), + Log1 = ra_log:write_sparse(Entry1, LastIdx0, Log0), {{Idx, Term, "entry"}, Log2} = ra_log:fetch(Idx, Log1), Log3 = ra_log:write_sparse(Entry2, Idx, Log2), - Log = await_written_idx(Idx, Term, Log3), - {LastIdx, Term} = ra_log:last_written(Log), - {{LastIdx, Term, "entry+5"}, Log2} = ra_log:fetch(Idx+5, Log1), + Log = await_written_idx(Idx5, Term, Log3), + {Idx5, Term} = ra_log:last_written(Log), + {Idx5, _} = ra_log:last_index_term(Log), + % debugger:start(), + % int:i(ra_log), + % int:break(ra_log, 524), + {{Idx5, Term, "entry+5"}, _Log} = ra_log:fetch(Idx5, Log), ok. append_then_fetch_no_wait(Config) -> @@ -120,7 +126,9 @@ append_then_fetch_no_wait(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 @@ -134,7 +142,8 @@ append_then_fetch_no_wait(Config) -> {ra_log_event, {written, _, _} = Evt} -> ct:pal("written ~p", [Evt]), {Log, _} = ra_log:handle_event(Evt, Log3), - {Idx, Term} = ra_log:last_written(Log) + {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") @@ -309,6 +318,7 @@ flush() -> 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} -> @@ -316,6 +326,7 @@ await_written_idx(Idx, Term, Log0) -> _ -> await_written_idx(Idx, Term, Log) end - after 1000 -> + after 1000_000 -> + flush(), throw(ra_log_append_timeout) end. diff --git a/test/ra_log_segment_writer_SUITE.erl b/test/ra_log_segment_writer_SUITE.erl index 42acf701..9cddcd87 100644 --- a/test/ra_log_segment_writer_SUITE.erl +++ b/test/ra_log_segment_writer_SUITE.erl @@ -209,9 +209,6 @@ accept_mem_tables_overwrite_same_wal(Config) -> 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), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.wal")), receive @@ -250,7 +247,6 @@ accept_mem_tables_multi_segment(Config) -> TidSeq = {Tid, [ra_mt:range(Mt)]}, TidRanges = [TidSeq], Ranges = #{UId => TidRanges}, - ct:pal("TidRanges ~p", [TidRanges]), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w.wal")), receive @@ -276,7 +272,7 @@ accept_mem_tables_multi_segment_max_size(Config) -> Entries = [{I, 2, crypto:strong_rand_bytes(120)} || I <- lists:seq(1, 10)], Mt = make_mem_table(UId, Entries), Tid = ra_mt:tid(Mt), - TidRanges = [{Tid, ra_mt:range(Mt)}], + TidRanges = [{Tid, [ra_mt:range(Mt)]}], Ranges = #{UId => TidRanges}, ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w.wal")), diff --git a/test/ra_seq_SUITE.erl b/test/ra_seq_SUITE.erl index fe1ef8fd..1a1ca0f1 100644 --- a/test/ra_seq_SUITE.erl +++ b/test/ra_seq_SUITE.erl @@ -26,7 +26,8 @@ all_tests() -> add, subtract, iter, - remove_prefix + remove_prefix, + remove_prefix_2 ]. groups() -> @@ -139,4 +140,12 @@ remove_prefix(_Config) -> %% 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_snapshot_SUITE.erl b/test/ra_snapshot_SUITE.erl index 8889a974..a0bac05d 100644 --- a/test/ra_snapshot_SUITE.erl +++ b/test/ra_snapshot_SUITE.erl @@ -102,8 +102,10 @@ take_snapshot(Config) -> 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), @@ -151,8 +153,9 @@ init_recover(Config) -> 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) @@ -178,8 +181,8 @@ init_recover_voter_status(Config) -> 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) @@ -208,8 +211,8 @@ init_multi(Config) -> %% simulate ra worker execution Fun(), receive - {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> - State2 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1), + {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), @@ -251,8 +254,8 @@ init_recover_multi_corrupt(Config) -> snapshot, State0), Fun(), receive - {ra_log_event, {snapshot_written, IdxTerm, snapshot}} -> - State2 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State1), + {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), @@ -300,8 +303,8 @@ init_recover_corrupt(Config) -> 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, @@ -331,8 +334,8 @@ read_snapshot(Config) -> ra_snapshot:begin_snapshot(Meta, ?MACMOD, MacRef, snapshot, State0), Fun(), State = 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, @@ -436,8 +439,8 @@ accept_receives_snapshot_written_with_higher_index(Config) -> %% 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), @@ -480,8 +483,8 @@ accept_receives_snapshot_written_with_higher_index_2(Config) -> %% then the snapshot written event is received after the higher index %% has been received receive - {ra_log_event, {snapshot_written, {55, 2} = IdxTerm, snapshot}} -> - State5 = ra_snapshot:complete_snapshot(IdxTerm, snapshot, State4), + {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), From 1471b37812a62b01e274267af0d6460aee9e68cc Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 1 May 2025 09:17:31 +0100 Subject: [PATCH 19/22] finished ra_seq refactoring --- Makefile | 2 +- src/ra_log.erl | 183 +++++++++++++++++++++++----------------- src/ra_seq.erl | 1 + test/ra_log_2_SUITE.erl | 32 +++++++ 4 files changed, 140 insertions(+), 78 deletions(-) diff --git a/Makefile b/Makefile index 07a8dfc2..1ca9e08f 100644 --- a/Makefile +++ b/Makefile @@ -13,7 +13,7 @@ dep_aten = hex 0.6.0 dep_seshat = hex 0.6.0 DEPS = aten gen_batch_server seshat -TEST_DEPS = proper meck eunit_formatters inet_tcp_proxy +TEST_DEPS = proper meck inet_tcp_proxy BUILD_DEPS = elvis_mk diff --git a/src/ra_log.erl b/src/ra_log.erl index 844781f7..89ec22df 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -677,7 +677,7 @@ handle_event({written, Term, WrittenSeq}, {error, not_prefix} -> ?DEBUG("~ts: ~p not prefix of ~p", [Cfg#cfg.log_id, WrittenSeq, Pend0]), - exit({not_prefix, WrittenSeq, Pend0}) + {resend_pending(State0), []} end; {undefined, State} when LastWrittenIdx < FirstIdx -> % A snapshot happened before the written event came in @@ -703,85 +703,86 @@ handle_event({written, Term, WrittenSeq}, handle_event({written, Term, NewWrittenSeq}, State0) end end; -handle_event({written, _Term, [{FromIdx, _ToIdx}]}, - #?MODULE{last_index = LastIdx} = State) - when FromIdx > LastIdx -> - %% we must have reverted back, either by explicit reset or by a snapshot - %% installation taking place whilst the WAL was processing the write - %% Just drop the event in this case as it is stale - {State, []}; -handle_event({written, Term, [{FromIdx, ToIdx}]}, - #?MODULE{cfg = Cfg, - last_written_index_term = {LastWrittenIdx0, - _LastWrittenTerm0}, - first_index = FirstIdx} = State0) - when FromIdx =< LastWrittenIdx0 + 1 -> - % We need to ignore any written events for the same index - % but in a prior term if we do not we may end up confirming - % to a leader writes that have not yet - % been fully flushed - case fetch_term(ToIdx, State0) of - {Term, State} when is_integer(Term) -> - ok = put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, ToIdx), - {State#?MODULE{last_written_index_term = {ToIdx, Term}}, []}; - {undefined, State} when ToIdx < FirstIdx -> - % A snapshot happened before the written event came in - % This can only happen on a leader when consensus is achieved by - % followers returning appending the entry and the leader committing - % and processing a snapshot before the written event comes in. - {State, []}; - {OtherTerm, State} -> - %% term mismatch, let's reduce the range and try again to see - %% if any entries in the range are valid - case ra_range:new(FromIdx, ToIdx-1) of - undefined -> - ?DEBUG("~ts: written event did not find term ~b for index ~b " - "found ~w", - [Cfg#cfg.log_id, Term, ToIdx, OtherTerm]), - {State, []}; - NextWrittenRange -> - %% retry with a reduced range - handle_event({written, Term, [NextWrittenRange]}, State0) - end - end; -handle_event({written, _Term, [{FromIdx, _}]} = Evt, - #?MODULE{cfg = #cfg{log_id = LogId}, - mem_table = Mt, - last_written_index_term = {LastWrittenIdx, _}} = State0) - when FromIdx > LastWrittenIdx + 1 -> - % leaving a gap is not ok - may need to resend from mem table - Expected = LastWrittenIdx + 1, - MtRange = ra_mt:range(Mt), - case ra_range:in(Expected, MtRange) of - true -> - ?INFO("~ts: ra_log: written gap detected at ~b expected ~b!", - [LogId, FromIdx, Expected]), - {resend_from(Expected, State0), []}; - false -> - ?DEBUG("~ts: ra_log: written gap detected at ~b but is outside - of mem table range ~w. Updating last written index to ~b!", - [LogId, FromIdx, MtRange, Expected]), - %% if the entry is not in the mem table we may have missed a - %% written event due to wal crash. Accept written event by updating - %% last written index term and recursing - {Term, State} = fetch_term(Expected, State0), - handle_event(Evt, - State#?MODULE{last_written_index_term = {Expected, Term}}) - end; -handle_event({written, Term, Written}, State) -> - %% simple handling of ra_seqs for now - case Written of - [I] when is_integer(I) -> - handle_event({written, Term, [{I, I}]}, State); - [I2, I] when is_integer(I) andalso - I + 1 == I2 -> - handle_event({written, Term, [{I, I2}]}, State); - _ -> - exit({sparse_written_events_not_implemented, Written}) - end; +% handle_event({written, _Term, [{FromIdx, _ToIdx}]}, +% #?MODULE{last_index = LastIdx} = State) +% when FromIdx > LastIdx -> +% %% we must have reverted back, either by explicit reset or by a snapshot +% %% installation taking place whilst the WAL was processing the write +% %% Just drop the event in this case as it is stale +% {State, []}; +% handle_event({written, Term, [{FromIdx, ToIdx}]}, +% #?MODULE{cfg = Cfg, +% last_written_index_term = {LastWrittenIdx0, +% _LastWrittenTerm0}, +% first_index = FirstIdx} = State0) +% when FromIdx =< LastWrittenIdx0 + 1 -> +% % We need to ignore any written events for the same index +% % but in a prior term if we do not we may end up confirming +% % to a leader writes that have not yet +% % been fully flushed +% case fetch_term(ToIdx, State0) of +% {Term, State} when is_integer(Term) -> +% ok = put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, ToIdx), +% {State#?MODULE{last_written_index_term = {ToIdx, Term}}, []}; +% {undefined, State} when ToIdx < FirstIdx -> +% % A snapshot happened before the written event came in +% % This can only happen on a leader when consensus is achieved by +% % followers returning appending the entry and the leader committing +% % and processing a snapshot before the written event comes in. +% {State, []}; +% {OtherTerm, State} -> +% %% term mismatch, let's reduce the range and try again to see +% %% if any entries in the range are valid +% case ra_range:new(FromIdx, ToIdx-1) of +% undefined -> +% ?DEBUG("~ts: written event did not find term ~b for index ~b " +% "found ~w", +% [Cfg#cfg.log_id, Term, ToIdx, OtherTerm]), +% {State, []}; +% NextWrittenRange -> +% %% retry with a reduced range +% handle_event({written, Term, [NextWrittenRange]}, State0) +% end +% end; +% handle_event({written, _Term, [{FromIdx, _}]} = Evt, +% #?MODULE{cfg = #cfg{log_id = LogId}, +% mem_table = Mt, +% last_written_index_term = {LastWrittenIdx, _}} = State0) +% when FromIdx > LastWrittenIdx + 1 -> +% % leaving a gap is not ok - may need to resend from mem table +% Expected = LastWrittenIdx + 1, +% MtRange = ra_mt:range(Mt), +% case ra_range:in(Expected, MtRange) of +% true -> +% ?INFO("~ts: ra_log: written gap detected at ~b expected ~b!", +% [LogId, FromIdx, Expected]), +% {resend_from(Expected, State0), []}; +% false -> +% ?DEBUG("~ts: ra_log: written gap detected at ~b but is outside +% of mem table range ~w. Updating last written index to ~b!", +% [LogId, FromIdx, MtRange, Expected]), +% %% if the entry is not in the mem table we may have missed a +% %% written event due to wal crash. Accept written event by updating +% %% last written index term and recursing +% {Term, State} = fetch_term(Expected, State0), +% handle_event(Evt, +% State#?MODULE{last_written_index_term = {Expected, Term}}) +% end; +% handle_event({written, Term, Written}, State) -> +% %% simple handling of ra_seqs for now +% case Written of +% [I] when is_integer(I) -> +% handle_event({written, Term, [{I, I}]}, State); +% [I2, I] when is_integer(I) andalso +% I + 1 == I2 -> +% handle_event({written, Term, [{I, I2}]}, State); +% _ -> +% exit({sparse_written_events_not_implemented, Written}) +% end; handle_event({segments, TidRanges, NewSegs}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, reader = Reader0, + pending = Pend0, mem_table = Mt0} = State0) -> Reader = ra_log_reader:update_segments(NewSegs, Reader0), put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, @@ -794,7 +795,18 @@ handle_event({segments, TidRanges, NewSegs}, ok = ra_log_ets:execute_delete(Names, UId, Spec), Acc end, Mt0, TidRanges), + %% it is theoretically possible that the segment writer flush _could_ + %% over take WAL notifications + %% + FstPend = ra_seq:first(Pend0), + Pend = case ra_mt:range(Mt) of + {Start, _End} when Start > FstPend -> + ra_seq:floor(Start, Pend0); + _ -> + Pend0 + end, State = State0#?MODULE{reader = Reader, + pending = Pend, mem_table = Mt}, {State, []}; handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, @@ -1343,6 +1355,23 @@ resend_from(Idx, #?MODULE{cfg = #cfg{uid = UId}} = State0) -> State0 end. +resend_pending(#?MODULE{cfg = Cfg, + last_resend_time = undefined, + pending = Pend, + mem_table = Mt} = State) -> + ct:pal("~ts: ra_log: resending from ~b to ~b mt ~p", + [State#?MODULE.cfg#cfg.log_id, ra_seq:first(Pend), + ra_seq:last(Pend), ra_mt:range(Mt)]), + ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_RESENDS, ra_seq:length(Pend)), + ra_seq:fold(fun (I, Acc) -> + {I, T, C} = ra_mt:lookup(I, Mt), + Tid = ra_mt:tid_for(I, T, Mt), + wal_rewrite(Acc, Tid, {I, T, C}) + end, + State#?MODULE{last_resend_time = {erlang:system_time(seconds), + whereis(Cfg#cfg.wal)}}, + Pend). + resend_from0(Idx, #?MODULE{cfg = Cfg, last_index = LastIdx, last_resend_time = undefined, diff --git a/src/ra_seq.erl b/src/ra_seq.erl index f23659be..3a98236d 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -61,6 +61,7 @@ append(Idx, [Prev | _] = Seq) from_list(L) -> lists:foldl(fun append/2, [], lists:sort(L)). +%% @doc This operation is O(n) -spec floor(ra:index(), state()) -> state(). floor(FloorIdxIncl, Seq) when is_list(Seq) -> %% TODO: assert appendable diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 53f3b4b1..1e5a1ace 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, @@ -915,6 +916,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), From ee09415ae193a02679840f1f3df0c134327680a2 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 1 May 2025 15:26:21 +0100 Subject: [PATCH 20/22] wip --- src/ra_log.erl | 85 ++++++++++++++++++++++------------------- test/ra_log_2_SUITE.erl | 84 ++++++++++++++++++++++++++++++++++++++-- test/ra_log_SUITE.erl | 5 +-- 3 files changed, 127 insertions(+), 47 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 89ec22df..ea8f1e2e 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -315,7 +315,8 @@ init(#{uid := UId, {SnapIdx, SnapTerm}, State#?MODULE.last_written_index_term ]), - element(1, delete_segments(SnapIdx, State)). + State. + % element(1, delete_segments(SnapIdx, State)). -spec close(state()) -> ok. close(#?MODULE{cfg = #cfg{uid = _UId}, @@ -449,6 +450,8 @@ write([{Idx, _, _} | _], #?MODULE{cfg = #cfg{uid = UId}, [UId, Idx, LastIdx+1])), {error, {integrity_error, Msg}}. +-spec write_sparse(log_entry(), ra:index(), state()) -> + {ok, state()} | {error, wal_down | gap_detected}. write_sparse({Idx, Term, _} = Entry, PrevIdx0, #?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, @@ -472,13 +475,12 @@ write_sparse({Idx, Term, _} = Entry, PrevIdx0, {ok, Pid} -> ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), - State0#?MODULE{last_index = Idx, - last_term = Term, - mem_table = Mt, - last_wal_write = {Pid, now_ms()} - }; - {error, wal_down} -> - error(wal_down) + {ok, State0#?MODULE{last_index = Idx, + last_term = Term, + mem_table = Mt, + last_wal_write = {Pid, now_ms()}}}; + {error, wal_down} = Err-> + Err end. -spec fold(FromIdx :: ra_index(), ToIdx :: ra_index(), @@ -847,18 +849,11 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, Snap end, % delete any segments outside of first_index - {State, Effects1} = delete_segments(SnapIdx, State0), + % {State, Effects1} = delete_segments(SnapIdx, State0), + State = State0, - %% 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), - ok = exec_mem_table_delete(Names, UId, Spec), %% TODO schedule compaction, - %% NB the mt cannot be truncated here with set_first as we need - %% to keep any live indexes % SegRefs = ra_log_reader:segment_refs(Reader), % @@ -872,9 +867,16 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, 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(SmallestLiveIdx, Mt0), + ok = exec_mem_table_delete(Names, UId, Spec), - Effects = Effects0 ++ Effects1, + Effects = Effects0, % ++ Effects1, {State#?MODULE{first_index = SnapIdx + 1, last_index = max(LstIdx, SnapIdx), last_written_index_term = LWIdxTerm, @@ -960,18 +962,21 @@ set_snapshot_state(SnapState, State) -> install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, - mem_table = Mt0 - } = State0) -> + mem_table = Mt0} = State0) -> 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), + % {State, Effs} = delete_segments(SnapIdx, State0), + % TODO: schedule compaction + State = State0, {SnapState, Checkpoints} = ra_snapshot:take_older_checkpoints(SnapIdx, SnapState0), CPEffects = [{delete_snapshot, ra_snapshot:directory(SnapState, checkpoint), Checkpoint} || Checkpoint <- Checkpoints], + %% TODO: can't really do this as the mem table may contain live indexes + %% below the snap idx {Spec, Mt} = ra_mt:set_first(SnapIdx + 1, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), {State#?MODULE{snapshot_state = SnapState, @@ -980,7 +985,7 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, last_term = SnapTerm, mem_table = Mt, last_written_index_term = IdxTerm}, - Effs ++ CPEffects}. + CPEffects}. -spec recover_snapshot(State :: state()) -> option({ra_snapshot:meta(), term()}). @@ -1264,24 +1269,24 @@ release_resources(MaxOpenSegments, %% 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, - reader = Reader0} = State0) -> - case ra_log_reader:update_first_index(SnapIdx + 1, Reader0) of - {Reader, []} -> - State = State0#?MODULE{reader = Reader}, - {State, []}; - {Reader, [Pivot | _] = Obsolete} -> - ok = ra_log_segment_writer:truncate_segments(SegWriter, - UId, Pivot), - NumActive = ra_log_reader:segment_ref_count(Reader), - ?DEBUG("~ts: ~b obsolete segments at ~b - remaining: ~b, pivot ~0p", - [LogId, length(Obsolete), SnapIdx, NumActive, Pivot]), - put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, NumActive), - State = State0#?MODULE{reader = Reader}, - {State, []} - end. +% delete_segments(SnapIdx, #?MODULE{cfg = #cfg{log_id = LogId, +% segment_writer = SegWriter, +% uid = UId} = Cfg, +% reader = Reader0} = State0) -> +% case ra_log_reader:update_first_index(SnapIdx + 1, Reader0) of +% {Reader, []} -> +% State = State0#?MODULE{reader = Reader}, +% {State, []}; +% {Reader, [Pivot | _] = Obsolete} -> +% ok = ra_log_segment_writer:truncate_segments(SegWriter, +% UId, Pivot), +% NumActive = ra_log_reader:segment_ref_count(Reader), +% ?DEBUG("~ts: ~b obsolete segments at ~b - remaining: ~b, pivot ~0p", +% [LogId, length(Obsolete), SnapIdx, NumActive, Pivot]), +% put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, NumActive), +% State = State0#?MODULE{reader = Reader}, +% {State, []} +% end. %% unly used by resend to wal functionality and doesn't update the mem table wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 1e5a1ace..9888b373 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -66,7 +66,8 @@ all_tests() -> recover_after_snapshot, updated_segment_can_be_read, open_segments_limit, - write_config + write_config, + sparse_write ]. groups() -> @@ -1518,6 +1519,70 @@ write_config(Config) -> ok. +sparse_write(Config) -> + Log00 = ra_log_init(Config), + % assert there are no segments at this point + [] = find_segments(Config), + + % create a segment + + Indexes = lists:seq(1, 10, 2), + Log0 = write_sparse(Indexes, 0, Log00), + Log0b = assert_log_events(Log0, + fun (L) -> + #{num_pending := Num} = ra_log:overview(L), + Num == 0 + end), + {Res0, _Log} = ra_log:sparse_read(Indexes, Log0b), + ?assertMatch([{1, _, _}, + {3, _, _}, + {5, _, _}, + {7, _, _}, + {9, _, _}], Res0), + + %% roll wal and assert we can read sparsely from segments + ra_log_wal:force_roll_over(ra_log_wal), + Log1 = assert_log_events(Log0b, + fun (L) -> + #{num_segments := Segs} = ra_log:overview(L), + Segs > 0 + end), + + {Res, Log2} = ra_log:sparse_read(Indexes, Log1), + ?assertMatch([{1, _, _}, + {3, _, _}, + {5, _, _}, + {7, _, _}, + {9, _, _}], Res), + + ct:pal("ov: ~p", [ra_log:overview(Log2)]), + + %% the snapshot is written after live index replication + Meta = meta(15, 2, [?N1]), + Context = #{}, + %% passing all Indexes but first one as snapshot state + LiveIndexes = tl(Indexes), + Chunk = create_snapshot_chunk(Config, Meta, LiveIndexes, Context), + SnapState0 = ra_log:snapshot_state(Log2), + {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), + {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + run_effs(AEffs), + {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2), + {ok, Log} = ra_log:write([{16, 1, <<>>}], Log3), + {ResFinal, _} = ra_log:sparse_read(LiveIndexes, Log), + ?assertMatch([{3, _, _}, + {5, _, _}, + {7, _, _}, + {9, _, _}], ResFinal), + + ReInitLog= ra_log_init(Config), + {ResReInit, _} = ra_log:sparse_read(LiveIndexes, ReInitLog), + ?assertMatch([{3, _, _}, + {5, _, _}, + {7, _, _}, + {9, _, _}], ResReInit), + ok. + validate_fold(From, To, Term, Log0) -> {Entries0, Log} = ra_log:fold(From, To, fun (E, A) -> [E | A] end, @@ -1571,6 +1636,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) -> @@ -1757,13 +1829,15 @@ meta(Idx, Term, Cluster) -> machine_version => 1}. create_snapshot_chunk(Config, #{index := Idx} = Meta, Context) -> + create_snapshot_chunk(Config, #{index := Idx} = Meta, <<"9">>, Context). + +create_snapshot_chunk(Config, #{index := Idx} = Meta, MacState, Context) -> OthDir = filename:join(?config(work_dir, Config), "snapshot_installation"), CPDir = filename:join(?config(work_dir, Config), "checkpoints"), ok = ra_lib:make_dir(OthDir), ok = ra_lib:make_dir(CPDir), Sn0 = ra_snapshot:init(<<"someotheruid_adsfasdf">>, ra_log_snapshot, OthDir, CPDir, undefined, ?DEFAULT_MAX_CHECKPOINTS), - MacState = <<"9">>, LiveIndexes = [], {Sn1, [{bg_work, Fun, _ErrFun}]} = ra_snapshot:begin_snapshot(Meta, ?MODULE, MacState, snapshot, Sn0), @@ -1816,4 +1890,8 @@ run_effs(Effs) -> %% ra_machine fakes version() -> 1. -live_indexes(_) -> []. +live_indexes(MacState) when is_list(MacState) -> + %% fake returning live indexes + MacState; +live_indexes(_) -> + []. diff --git a/test/ra_log_SUITE.erl b/test/ra_log_SUITE.erl index e6dd5984..0cb5083e 100644 --- a/test/ra_log_SUITE.erl +++ b/test/ra_log_SUITE.erl @@ -115,10 +115,7 @@ write_sparse_then_fetch(Config) -> Log = await_written_idx(Idx5, Term, Log3), {Idx5, Term} = ra_log:last_written(Log), {Idx5, _} = ra_log:last_index_term(Log), - % debugger:start(), - % int:i(ra_log), - % int:break(ra_log, 524), - {{Idx5, Term, "entry+5"}, _Log} = ra_log:fetch(Idx5, Log), + {{Idx5, Term, "entry+5"}, Log} = ra_log:fetch(Idx5, Log), ok. append_then_fetch_no_wait(Config) -> From d70433879c7fb1685b4e7e0e132766b8243d668c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 9 May 2025 07:33:30 +0100 Subject: [PATCH 21/22] wip --- src/ra_flru.erl | 7 +- src/ra_log.erl | 148 +++++++++++++++++++++++++++++++------- src/ra_log_reader.erl | 9 ++- src/ra_log_snapshot.erl | 4 +- src/ra_log_wal.erl | 2 +- src/ra_seq.erl | 12 +++- src/ra_server.erl | 15 ++-- src/ra_snapshot.erl | 31 ++++++-- test/ra_log_2_SUITE.erl | 57 ++++++++------- test/ra_log_SUITE.erl | 4 +- test/ra_log_memory.erl | 22 +++--- test/ra_log_wal_SUITE.erl | 1 - test/ra_server_SUITE.erl | 25 +++---- 13 files changed, 237 insertions(+), 100 deletions(-) diff --git a/src/ra_flru.erl b/src/ra_flru.erl index 386d92b6..b9305a9a 100644 --- a/src/ra_flru.erl +++ b/src/ra_flru.erl @@ -16,7 +16,8 @@ insert/3, evict/2, evict_all/1, - size/1 + size/1, + max_size/1 ]). -define(MAX_SIZE, 5). @@ -91,6 +92,10 @@ evict_all(#?MODULE{items = Items, size(#?MODULE{items = Items}) -> length(Items). +-spec max_size(state()) -> non_neg_integer(). +max_size(#?MODULE{max_size = MaxSize}) -> + MaxSize. + -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/ra_log.erl b/src/ra_log.erl index ea8f1e2e..df0ca410 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -33,7 +33,7 @@ next_index/1, snapshot_state/1, set_snapshot_state/2, - install_snapshot/3, + install_snapshot/4, recover_snapshot/1, snapshot_index_term/1, update_release_cursor/5, @@ -110,7 +110,8 @@ reader :: ra_log_reader:state(), mem_table :: ra_mt:state(), tx = false :: boolean(), - pending = [] :: ra_seq:state() + pending = [] :: ra_seq:state(), + live_indexes = [] :: ra_seq:state() }). -record(read_plan, {dir :: file:filename_all(), @@ -209,6 +210,13 @@ init(#{uid := UId, undefined -> {-1, -1}; Curr -> Curr end, + %% TODO: error handling + %% TODO: the "indexes" file isn't authoritative when it comes to live + %% indexes, we need to recover the snapshot and query it for live indexes + %% to get the actual valua + {ok, LiveIndexes} = ra_snapshot:indexes( + ra_snapshot:current_snapshot_dir(SnapshotState)), + ct:pal("log init live indexes ~p", [LiveIndexes]), AccessPattern = maps:get(initial_access_pattern, Conf, sequential), {ok, Mt0} = ra_log_ets:mem_table_please(Names, UId), @@ -253,13 +261,15 @@ init(#{uid := UId, reader = Reader, mem_table = Mt, snapshot_state = SnapshotState, - last_wal_write = {whereis(Wal), now_ms()} + last_wal_write = {whereis(Wal), now_ms()}, + live_indexes = LiveIndexes }, put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx), LastIdx = State0#?MODULE.last_index, put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, LastIdx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, LastIdx), - put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, ra_log_reader:segment_ref_count(Reader)), + put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, + ra_log_reader:segment_ref_count(Reader)), case ra_snapshot:latest_checkpoint(SnapshotState) of undefined -> ok; @@ -527,7 +537,9 @@ fold(_From, _To, _Fun, Acc, State) -> {[log_entry()], state()}. sparse_read(Indexes0, #?MODULE{cfg = Cfg, reader = Reader0, + first_index = FstIdx, last_index = LastIdx, + live_indexes = LiveIndexes, mem_table = Mt} = State) -> ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), %% indexes need to be sorted high -> low for correct and efficient reading @@ -543,7 +555,20 @@ sparse_read(Indexes0, #?MODULE{cfg = Cfg, end, %% drop any indexes that are larger than the last index available - Indexes2 = lists:dropwhile(fun (I) -> I > LastIdx end, Indexes1), + %% or smaller than first index and not in live indexes + Indexes2 = lists:filter(fun (I) -> + I =< LastIdx andalso + (case I < FstIdx of + true -> + %% TODO: this can be slow if + %% live indexes are mostly + %% non-contiguous. Create + %% BST for live index lookups + ra_seq:in(I, LiveIndexes); + false -> + true + end) + end, Indexes1), {Entries0, MemTblNumRead, Indexes} = ra_mt:get_items(Indexes2, Mt), ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, MemTblNumRead), {Entries1, Reader} = ra_log_reader:sparse_read(Reader0, Indexes, Entries0), @@ -811,6 +836,30 @@ handle_event({segments, TidRanges, NewSegs}, pending = Pend, mem_table = Mt}, {State, []}; +handle_event({segments_to_be_deleted, SegRefs}, + #?MODULE{cfg = #cfg{uid = UId, + log_id = LogId, + directory = Dir, + counter = Counter, + names = Names}, + reader = Reader} = State) -> + Fun = fun () -> + [prim_file:delete(filename:join(Dir, F)) + || {_, F} <- SegRefs], + ok + end, + ActiveSegs = ra_log_reader:segment_refs(Reader) -- SegRefs, + #{max_size := MaxOpenSegments} = ra_log_reader:info(Reader), + % close all open segments + ok = ra_log_reader:close(Reader), + ?DEBUG("~ts: ~b obsolete segments - remaining: ~b", + [LogId, length(SegRefs), length(ActiveSegs)]), + %% open a new segment with the new max open segment value + {State#?MODULE{reader = ra_log_reader:init(UId, Dir, MaxOpenSegments, + random, + ActiveSegs, Names, Counter)}, + + [{bg_work, Fun, fun (_Err) -> ok end}]}; handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, @@ -848,15 +897,11 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, false -> Snap end, - % delete any segments outside of first_index - % {State, Effects1} = delete_segments(SnapIdx, State0), + + CompEffs = schedule_compaction(SnapIdx, State0), State = State0, - %% TODO schedule compaction, - % SegRefs = ra_log_reader:segment_refs(Reader), - % - %% remove all pending below smallest live index as the wal %% may not write them %% TODO: test that a written even can still be processed if it @@ -876,12 +921,13 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, {Spec, Mt1} = ra_mt:set_first(SmallestLiveIdx, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), - Effects = Effects0, % ++ Effects1, + Effects = CompEffs ++ Effects0, % ++ Effects1, {State#?MODULE{first_index = SnapIdx + 1, last_index = max(LstIdx, SnapIdx), last_written_index_term = LWIdxTerm, mem_table = Mt1, pending = Pend, + live_indexes = LiveIndexes, snapshot_state = SnapState}, Effects}; checkpoint -> put_counter(Cfg, ?C_RA_SVR_METRIC_CHECKPOINT_INDEX, SnapIdx), @@ -957,19 +1003,20 @@ snapshot_state(State) -> set_snapshot_state(SnapState, State) -> State#?MODULE{snapshot_state = SnapState}. --spec install_snapshot(ra_idxterm(), ra_snapshot:state(), state()) -> - {state(), effects()}. -install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, +-spec install_snapshot(ra_idxterm(), ra_snapshot:state(), module(), state()) -> + {ra_snapshot:meta(), MacState :: term(), state(), effects()}. +install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, MacMod, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, - mem_table = Mt0} = State0) -> + mem_table = Mt0} = State0) + when is_atom(MacMod) -> ok = incr_counter(Cfg, ?C_RA_LOG_SNAPSHOTS_INSTALLED, 1), ok = put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, SnapIdx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, SnapIdx), - % {State, Effs} = delete_segments(SnapIdx, State0), - % TODO: schedule compaction - State = State0, + + CompEffs = schedule_compaction(SnapIdx, State0), + {SnapState, Checkpoints} = ra_snapshot:take_older_checkpoints(SnapIdx, SnapState0), CPEffects = [{delete_snapshot, @@ -979,13 +1026,22 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, %% below the snap idx {Spec, Mt} = ra_mt:set_first(SnapIdx + 1, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), - {State#?MODULE{snapshot_state = SnapState, - first_index = SnapIdx + 1, - last_index = SnapIdx, - last_term = SnapTerm, - mem_table = Mt, - last_written_index_term = IdxTerm}, - CPEffects}. + State = State0#?MODULE{snapshot_state = SnapState, + first_index = SnapIdx + 1, + last_index = SnapIdx, + last_term = SnapTerm, + mem_table = Mt, + last_written_index_term = IdxTerm}, + {Meta, MacState} = recover_snapshot(State), + LiveIndexes = ra_machine:live_indexes(MacMod, MacState), + %% TODO: it is not safe to write the indexes _after_ if we then treat + %% the persisted indexes as authoritative as if we crash in between + %% it may compact segments that still contain live indexes + SnapDir = ra_snapshot:current_snapshot_dir(SnapState), + ok = ra_snapshot:write_indexes(SnapDir, LiveIndexes), + {Meta, MacState, State#?MODULE{live_indexes = LiveIndexes}, + CompEffs ++ CPEffects}. + -spec recover_snapshot(State :: state()) -> option({ra_snapshot:meta(), term()}). @@ -1267,6 +1323,44 @@ release_resources(MaxOpenSegments, %%% Local functions + +schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{}, + snapshot_state = SnapState, + reader = Reader0}) -> + case ra_log_reader:segment_refs(Reader0) of + [] -> + []; + [_ | Compactable] -> + %% never compact the current segment + %% only take those who have a range lower than the snapshot index as + %% we never want to compact more than that + SegRefs = lists:takewhile(fun ({{_Start, End}, _}) -> + End =< SnapIdx + end, lists:reverse(Compactable)), + SnapDir = ra_snapshot:current_snapshot_dir(SnapState), + + Self = self(), + Fun = fun () -> + {ok, Indexes} = ra_snapshot:indexes(SnapDir), + {Delete, _} = lists:foldl( + fun ({Range, _} = S, {Del, Keep}) -> + case ra_seq:in_range(Range, Indexes) of + [] -> + {[S | Del], Keep}; + _ -> + {Del, [S | Keep]} + end + end, {[], []}, SegRefs), + %% need to update the ra_servers list of seg refs _before_ + %% the segments can actually be deleted + Self ! {ra_log_event, + {segments_to_be_deleted, Delete}}, + ok + end, + + [{bg_work, Fun, fun (_Err) -> ok end}] + end. + %% deletes all segments where the last index is lower than %% the Idx argument % delete_segments(SnapIdx, #?MODULE{cfg = #cfg{log_id = LogId, @@ -1364,7 +1458,7 @@ resend_pending(#?MODULE{cfg = Cfg, last_resend_time = undefined, pending = Pend, mem_table = Mt} = State) -> - ct:pal("~ts: ra_log: resending from ~b to ~b mt ~p", + ?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)), diff --git a/src/ra_log_reader.erl b/src/ra_log_reader.erl index d99a14a9..f87e7f36 100644 --- a/src/ra_log_reader.erl +++ b/src/ra_log_reader.erl @@ -23,7 +23,8 @@ sparse_read/3, read_plan/2, exec_read_plan/6, - fetch_term/2 + fetch_term/2, + info/1 ]). -include("ra.hrl"). @@ -258,6 +259,12 @@ fetch_term(Idx, #?STATE{cfg = #cfg{} = Cfg} = State0) -> incr_counter(Cfg, ?C_RA_LOG_FETCH_TERM, 1), segment_term_query(Idx, State0). +-spec info(state()) -> map(). +info(#?STATE{cfg = #cfg{} = _Cfg, + open_segments = Open} = State) -> + #{max_size => ra_flru:max_size(Open), + num_segments => segment_ref_count(State) + }. %% LOCAL segment_read_plan(_SegRefs, [], Acc) -> diff --git a/src/ra_log_snapshot.erl b/src/ra_log_snapshot.erl index 21acee4c..0d6ab070 100644 --- a/src/ra_log_snapshot.erl +++ b/src/ra_log_snapshot.erl @@ -159,7 +159,7 @@ read_chunk({Pos, Eof, Fd}, Size, _Dir) -> {error, unexpected_eof} end. --spec recover(file:filename()) -> +-spec recover(file:filename_all()) -> {ok, meta(), term()} | {error, invalid_format | {invalid_version, integer()} | @@ -178,6 +178,7 @@ recover(Dir) -> Err end. + validate(Dir) -> case recover(Dir) of {ok, _, _} -> ok; @@ -209,7 +210,6 @@ read_meta(Dir) -> context() -> #{can_accept_full_file => true}. - %% Internal read_meta_internal(Fd) -> diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index 0126e2bb..bf25e2d1 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -305,7 +305,7 @@ init(#{system := System, catch _:Err:Stack -> ?ERROR("WAL in ~ts failed to initialise with ~p, stack ~p", [System, Err, Stack]), - {stop, Err, Stack} + {stop, Err} end. -spec handle_batch([wal_op()], state()) -> diff --git a/src/ra_seq.erl b/src/ra_seq.erl index 3a98236d..18703590 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -34,7 +34,8 @@ next/1, length/1, in/2, - range/1 + range/1, + in_range/2 ]). -spec append(ra:index(), state()) -> state(). @@ -198,9 +199,16 @@ range(Seq) -> ra_range:new(first(Seq), last(Seq)). +in_range(_Range, []) -> + []; +in_range(undefined, _) -> + []; +in_range({Start, End}, Seq0) -> + %% TODO: optimise + floor(Start, limit(End, Seq0)). -%% Internal functions +%% Internal functions drop_prefix({IDX, PI}, {IDX, SI}) -> drop_prefix(next(PI), next(SI)); diff --git a/src/ra_server.erl b/src/ra_server.erl index 515c1863..5caeaa5f 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -1532,6 +1532,7 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, ?DEBUG("~ts: receiving snapshot chunk: ~b / ~w, index ~b, term ~b", [LogId, Num, ChunkFlag, SnapIndex, SnapTerm]), SnapState0 = ra_log:snapshot_state(Log0), + ct:pal("SnapStat0 ~p", [SnapState0]), {ok, SnapState, Effs0} = ra_snapshot:accept_chunk(Data, Num, ChunkFlag, SnapState0), Reply = #install_snapshot_result{term = CurTerm, @@ -1539,9 +1540,6 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, last_index = SnapIndex}, case ChunkFlag of last -> - %% this is the last chunk so we can "install" it - {Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm}, - SnapState, Log0), %% if the machine version of the snapshot is higher %% we also need to update the current effective machine configuration EffMacMod = ra_machine:which_module(Machine, SnapMacVer), @@ -1557,9 +1555,11 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, false -> Cfg0 end, - - {#{cluster := ClusterIds}, MacState} = ra_log:recover_snapshot(Log), - + %% this is the last chunk so we can "install" it + {#{cluster := ClusterIds}, + MacState, Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm}, + SnapState, + EffMacMod, Log0), OldServerIds = maps:map(fun (_, V) -> maps:with([voter_status], V) end, Cluster), @@ -1582,7 +1582,8 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, cluster_index_term => {SnapIndex, SnapTerm}, cluster => make_cluster(Id, ClusterIds), - membership => get_membership(ClusterIds, State0), + membership => + get_membership(ClusterIds, State0), machine_state => MacState}), %% it was the last snapshot chunk so we can revert back to %% follower status diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index d80bd710..f4bd0456 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -44,7 +44,11 @@ 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()} | @@ -398,9 +402,7 @@ begin_snapshot(#{index := Idx, term := Term} = Meta, MacMod, MacState, SnapKind, case LiveIndexes of [] -> ok; _ -> - Data = term_to_binary(LiveIndexes), - F = filename:join(SnapDir, <<"indexes">>), - ok = ra_lib:write_file(F, Data, true), + ok = write_indexes(SnapDir, LiveIndexes), ok end, Self ! {ra_log_event, @@ -705,6 +707,27 @@ take_extra_checkpoints(#?MODULE{checkpoints = Checkpoints0, {State0, Checks} end. +-spec write_indexes(file:filename_all(), ra_seq:state()) -> + ok | {error, file:posix()}. +write_indexes(Dir, Indexes) -> + File = filename:join(Dir, <<"indexes">>), + ra_lib:write_file(File, term_to_binary(Indexes)). + +-spec indexes(file:filename_all()) -> + {ok, ra_seq:state()} | {error, file:posix()}. +indexes(Dir) -> + File = filename:join(Dir, <<"indexes">>), + case prim_file:read_file(File) of + {ok, Bin} -> + {ok, binary_to_term(Bin)}; + {error, enoent} -> + %% no indexes + {ok, []}; + Err -> + Err + end. + + %% Utility -define(MAX_DIFF, 65_536). diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 9888b373..78e88e35 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -402,14 +402,15 @@ sparse_read_out_of_range_2(Config) -> {Log2, Effs} = ra_log:update_release_cursor(SnapIdx, #{}, ?MODULE, <<"snap@10">>, Log1), run_effs(Effs), - {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, + {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), @@ -1118,7 +1119,7 @@ snapshot_written_after_installation(Config) -> {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), run_effs(AEffs), - {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2), + {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2), %% write some more to create another segment Log4 = write_and_roll(16, 20, 2, Log3), {Log5, Efx4} = ra_log:handle_event(DelayedSnapWritten, Log4), @@ -1166,7 +1167,7 @@ oldcheckpoints_deleted_after_snapshot_install(Config) -> {ok, SnapState, AcceptEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), run_effs(AcceptEffs), - {Log4, Effs4} = ra_log:install_snapshot({15, 2}, SnapState, Log3), + {_, _, Log4, Effs4} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log3), ?assert(lists:any(fun (E) -> element(1, E) == delete_snapshot end, Effs4)), %% write some more to create another segment Log5 = write_and_roll(16, 20, 2, Log4), @@ -1211,7 +1212,7 @@ snapshot_installation(Config) -> {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), run_effs(AEffs), - {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2), + {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2), {15, _} = ra_log:last_index_term(Log3), {15, _} = ra_log:last_written(Log3), @@ -1261,7 +1262,7 @@ append_after_snapshot_installation(Config) -> {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), run_effs(AEffs), - {Log2, _} = ra_log:install_snapshot({15, 2}, SnapState, Log1), + {_, _, Log2, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log1), {15, _} = ra_log:last_index_term(Log2), {15, _} = ra_log:last_written(Log2), @@ -1293,7 +1294,7 @@ written_event_after_snapshot_installation(Config) -> {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), run_effs(AEffs), - {Log2, _} = ra_log:install_snapshot({SnapIdx, 2}, SnapState, Log1), + {_, _, Log2, _} = ra_log:install_snapshot({SnapIdx, 2}, SnapState, ?MODULE, Log1), {SnapIdx, _} = ra_log:last_index_term(Log2), {SnapIdx, _} = ra_log:last_written(Log2), NextIdx = SnapIdx + 1, @@ -1334,19 +1335,17 @@ update_release_cursor(Config) -> ?MODULE, initial_state, Log1), run_effs(Effs), + %% ensure snapshot index has been updated and 1 segment deleted Log3 = assert_log_events(Log2, fun (L) -> - {127, 2} == ra_log:snapshot_index_term(L) + {127, 2} == ra_log:snapshot_index_term(L) andalso + length(find_segments(Config)) == 1 end), %% now the snapshot_written should have been delivered and the %% snapshot state table updated UId = ?config(uid, Config), 127 = ra_log_snapshot_state:snapshot(ra_log_snapshot_state, UId), % this should delete a single segment - ra_lib:retry(fun () -> - Segments = find_segments(Config), - 1 == length(Segments) - end, 10, 100), Log3b = validate_fold(128, 149, 2, Log3), % update the release cursor all the way {Log4, Effs2} = ra_log:update_release_cursor(149, #{?N1 => new_peer(), @@ -1368,9 +1367,9 @@ update_release_cursor(Config) -> % append a few more items Log = assert_log_events(append_and_roll_no_deliver(150, 155, 2, Log5), - fun (L) -> - {154, 2} == ra_log:last_written(L) - end), + fun (L) -> + {154, 2} == ra_log:last_written(L) + end), validate_fold(150, 154, 2, Log), % assert there is only one segment - the current % snapshot has been confirmed. @@ -1443,6 +1442,8 @@ missed_mem_table_entries_are_deleted_at_next_opportunity(Config) -> ?N2 => new_peer()}, ?MODULE, initial_state, Log5), run_effs(Effs2), + ct:pal("Effs2 ~p", [Effs2]), + ct:pal("find segments ~p", [find_segments(Config)]), Log7 = deliver_log_events_cond(Log6, fun (_) -> case find_segments(Config) of @@ -1567,7 +1568,7 @@ sparse_write(Config) -> {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), run_effs(AEffs), - {Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, Log2), + {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2), {ok, Log} = ra_log:write([{16, 1, <<>>}], Log3), {ResFinal, _} = ra_log:sparse_read(LiveIndexes, Log), ?assertMatch([{3, _, _}, @@ -1659,7 +1660,11 @@ deliver_log_events_cond(Log0, CondFun, N) -> P ! E, Acc; ({next_event, {ra_log_event, E}}, Acc0) -> - {Acc, _} = ra_log:handle_event(E, Acc0), + {Acc, Effs} = ra_log:handle_event(E, Acc0), + run_effs(Effs), + Acc; + ({bg_work, Fun, _}, Acc) -> + Fun(), Acc; (_, Acc) -> Acc @@ -1720,11 +1725,12 @@ assert_log_events(Log0, AssertPred, Timeout) -> {ra_log_event, Evt} -> ct:pal("log evt: ~p", [Evt]), {Log1, Effs} = ra_log:handle_event(Evt, Log0), + run_effs(Effs), %% handle any next events Log = lists:foldl( fun ({next_event, {ra_log_event, E}}, Acc0) -> - ct:pal("eff log evt: ~p", [E]), - {Acc, _Effs} = ra_log:handle_event(E, Acc0), + {Acc, Effs} = ra_log:handle_event(E, Acc0), + run_effs(Effs), Acc; (_, Acc) -> Acc @@ -1846,6 +1852,7 @@ create_snapshot_chunk(Config, #{index := Idx} = Meta, MacState, Context) -> receive {ra_log_event, {snapshot_written, {Idx, 2} = IdxTerm, _, snapshot}} -> ra_snapshot:complete_snapshot(IdxTerm, snapshot, + LiveIndexes, Sn1) after 1000 -> exit(snapshot_timeout) diff --git a/test/ra_log_SUITE.erl b/test/ra_log_SUITE.erl index 0cb5083e..648d8fe0 100644 --- a/test/ra_log_SUITE.erl +++ b/test/ra_log_SUITE.erl @@ -109,9 +109,9 @@ write_sparse_then_fetch(Config) -> Entry2 = {Idx5, Term, "entry+5"}, {LastIdx0, _} = ra_log:last_index_term(Log0), - Log1 = ra_log:write_sparse(Entry1, LastIdx0, Log0), + {ok, Log1} = ra_log:write_sparse(Entry1, LastIdx0, Log0), {{Idx, Term, "entry"}, Log2} = ra_log:fetch(Idx, Log1), - Log3 = ra_log:write_sparse(Entry2, Idx, Log2), + {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), diff --git a/test/ra_log_memory.erl b/test/ra_log_memory.erl index a090b6ac..54755b0a 100644 --- a/test/ra_log_memory.erl +++ b/test/ra_log_memory.erl @@ -22,7 +22,7 @@ next_index/1, snapshot_state/1, set_snapshot_state/2, - install_snapshot/3, + install_snapshot/4, read_snapshot/1, recover_snapshot/1, snapshot_index_term/1, @@ -210,17 +210,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}, Data, _MacMod, + #state{entries = Log0} = State0) + when is_tuple(Data) -> + % discard log entries below snapshot index Log = maps:filter(fun (K, _) -> K > Index end, Log0), - {State#state{entries = Log, - last_index = Index, - last_written = {Index, Term}, - snapshot = Data}, []}; -install_snapshot(_Meta, _Data, State) -> - {State, []}. + State = State0#state{entries = Log, + last_index = Index, + last_written = {Index, Term}, + snapshot = Data}, + {Meta, MacState} = Data, + {Meta, MacState, State, []}. -spec read_snapshot(State :: ra_log_memory_state()) -> {ok, ra_snapshot:meta(), term()}. diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index 4b571f43..26f630e4 100644 --- a/test/ra_log_wal_SUITE.erl +++ b/test/ra_log_wal_SUITE.erl @@ -220,7 +220,6 @@ sparse_write_recover(Config) -> ok = proc_lib:stop(ra_log_wal), {ok, _Pid2} = ra_log_wal:start_link(Conf), - % {ok, Mt} = ra_log_ets:mem_table_please(?config(names, Config), UId), receive {'$gen_cast', {mem_tables, #{UId := [{Tid, [15, 12]}]}, _}} -> diff --git a/test/ra_server_SUITE.erl b/test/ra_server_SUITE.erl index 759f2170..ca52d3bd 100644 --- a/test/ra_server_SUITE.erl +++ b/test/ra_server_SUITE.erl @@ -152,18 +152,18 @@ setup_log() -> ra_lib:default(get({U, K}), D) end), meck:expect(ra_snapshot, begin_accept, - fun(_Meta, SS) -> - {ok, SS} + fun(Meta, undefined) -> + {ok, {Meta, undefined}} end), meck:expect(ra_snapshot, accept_chunk, - fun(_Data, _OutOf, _Flag, SS) -> - {ok, SS, []} + fun(Data, _OutOf, _Flag, {Meta, _}) -> + {ok, {Meta, Data}, []} end), meck:expect(ra_snapshot, abort_accept, fun(SS) -> SS end), meck:expect(ra_snapshot, accepting, fun(_SS) -> undefined end), meck:expect(ra_log, snapshot_state, fun ra_log_memory:snapshot_state/1), meck:expect(ra_log, set_snapshot_state, fun ra_log_memory:set_snapshot_state/2), - meck:expect(ra_log, install_snapshot, fun ra_log_memory:install_snapshot/3), + meck:expect(ra_log, install_snapshot, fun ra_log_memory:install_snapshot/4), meck:expect(ra_log, recover_snapshot, fun ra_log_memory:recover_snapshot/1), meck:expect(ra_log, snapshot_index_term, fun ra_log_memory:snapshot_index_term/1), meck:expect(ra_log, fold, fun ra_log_memory:fold/5), @@ -222,8 +222,9 @@ init_test(_Config) -> cluster => dehydrate_cluster(Cluster), machine_version => 1}, SnapshotData = "hi1+2+3", - {LogS, _} = ra_log_memory:install_snapshot({3, 5}, {SnapshotMeta, - SnapshotData}, Log0), + {_, _, LogS, _} = ra_log_memory:install_snapshot({3, 5}, {SnapshotMeta, + SnapshotData}, + ?MODULE, Log0), meck:expect(ra_log, init, fun (_) -> LogS end), #{current_term := 5, commit_index := 3, @@ -722,7 +723,7 @@ follower_aer_term_mismatch_snapshot(_Config) -> cluster => #{}, machine_version => 1}, Data = <<"hi3">>, - {Log,_} = ra_log_memory:install_snapshot({3, 5}, {Meta, Data}, Log0), + {_, _, Log,_} = ra_log_memory:install_snapshot({3, 5}, {Meta, Data}, ?MODULE, Log0), State = maps:put(log, Log, State0), AE = #append_entries_rpc{term = 6, @@ -2252,14 +2253,6 @@ follower_installs_snapshot(_Config) -> [{next_event, ISRpc}, {record_leader_msg, _}]} = ra_server:handle_follower(ISRpc, FState), - meck:expect(ra_log, recover_snapshot, - fun (_) -> - {#{index => Idx, - term => Term, - cluster => dehydrate_cluster(Config), - machine_version => 0}, - []} - end), {follower, #{current_term := Term, commit_index := Idx, last_applied := Idx, From ae8cbf2de8325d9665d3789b1d4817f5ddee60cb Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 9 May 2025 13:39:12 +0100 Subject: [PATCH 22/22] first cut phase 1 compaction impl --- src/ra_log.erl | 1 - src/ra_log_segment_writer.erl | 4 +- src/ra_log_wal.erl | 73 +-- src/ra_mt.erl | 17 +- src/ra_server.erl | 1 - test/ra_log_memory.erl | 17 +- test/ra_log_meta_SUITE.erl | 1 + test/ra_log_props_SUITE.erl | 858 ---------------------------------- test/ra_log_wal_SUITE.erl | 50 +- test/ra_machine_ets_SUITE.erl | 1 - test/ra_server_SUITE.erl | 20 +- 11 files changed, 129 insertions(+), 914 deletions(-) delete mode 100644 test/ra_log_props_SUITE.erl diff --git a/src/ra_log.erl b/src/ra_log.erl index df0ca410..3443fc71 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -216,7 +216,6 @@ init(#{uid := UId, %% to get the actual valua {ok, LiveIndexes} = ra_snapshot:indexes( ra_snapshot:current_snapshot_dir(SnapshotState)), - ct:pal("log init live indexes ~p", [LiveIndexes]), AccessPattern = maps:get(initial_access_pattern, Conf, sequential), {ok, Mt0} = ra_log_ets:mem_table_please(Names, UId), diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index 4c31790d..d2e64e18 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -273,7 +273,7 @@ flush_mem_table_ranges({ServerUId, TidSeqs0}, TidSeqs = lists:foldl( fun ({T, Seq0}, []) -> case ra_seq:floor(SmallestIdx, Seq0) of - undefined -> + [] -> []; Seq -> [{T, Seq}] @@ -282,7 +282,7 @@ flush_mem_table_ranges({ServerUId, TidSeqs0}, Start = ra_seq:first(PrevSeq), Seq1 = ra_seq:floor(SmallestIdx, Seq0), case ra_seq:limit(Start, Seq1) of - undefined -> + [] -> Acc; Seq -> [{T, Seq} | Acc] diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index bf25e2d1..bd8b8d26 100644 --- a/src/ra_log_wal.erl +++ b/src/ra_log_wal.erl @@ -381,26 +381,29 @@ recover_wal(Dir, #conf{system = System, end || File <- Files0, filename:extension(File) == ".wal"], WalFiles = lists:sort(Files), - AllWriters = - [begin - ?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, 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 || F <- WalFiles], - - FinalWriters = lists:foldl(fun (New, Acc) -> - maps:merge(Acc, New) - end, #{}, AllWriters), + 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)]), @@ -781,9 +784,10 @@ dump_records(<<_:1/unsigned, 1:1/unsigned, _:22/unsigned, dump_records(<<>>, Entries) -> Entries. -recover_wal_chunks(#conf{} = Conf, Fd, Mode) -> +recover_wal_chunks(#conf{} = Conf, Fd, Writers, Mode) -> Chunk = read_wal_chunk(Fd, Conf#conf.recovery_chunk_size), - recover_records(Conf, Fd, Chunk, #{}, #recovery{mode = Mode}). + recover_records(Conf, Fd, Chunk, #{}, #recovery{mode = Mode, + writers = Writers}). % All zeros indicates end of a pre-allocated wal file recover_records(_, _Fd, <<0:1/unsigned, 0:1/unsigned, 0:22/unsigned, IdDataLen:16/unsigned, _:IdDataLen/binary, @@ -824,10 +828,11 @@ recover_records(#conf{names = Names} = Conf, Fd, % W -> % W#{UId => {in_seq, SmallestIdx}} % end, - W = State0#recovery.writers, - Writers = W#{UId => {in_seq, SmallestIdx - 1}}, + 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 -> System = Conf#conf.system, ?DEBUG("WAL in ~ts: record failed CRC check. If this is the last record" @@ -1004,7 +1009,16 @@ recover_entry(Names, UId, {Idx, _, _} = Entry, SmallestIdx, {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), SmallestIdx, [Idx]), @@ -1014,7 +1028,8 @@ recover_entry(Names, UId, {Idx, _, _} = Entry, SmallestIdx, {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, _}, SmallestIdx, #recovery{mode = post_boot, @@ -1049,6 +1064,7 @@ 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 -> @@ -1065,9 +1081,10 @@ handle_trunc(true, UId, Idx, #recovery{mode = Mode, 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_mt.erl b/src/ra_mt.erl index bb24a188..55d3d011 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -120,11 +120,13 @@ insert({Idx, _, _} = _Entry, end. -spec insert_sparse(log_entry(), undefined | ra:index(), state()) -> - {ok, state()} | {error, gap_detected | limit_reached}. + {ok, state()} | {error, overwriting | gap_detected | limit_reached}. insert_sparse({Idx, _, _} = Entry, LastIdx, - #?MODULE{tid = Tid, - indexes = Seq} = State) -> - case ra_seq:last(Seq) == LastIdx of + #?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 -> @@ -134,7 +136,12 @@ insert_sparse({Idx, _, _} = Entry, LastIdx, {ok, State#?MODULE{indexes = ra_seq:append(Idx, Seq)}} end; false -> - {error, gap_detected} + case IsOverwriting of + true -> + {error, overwriting}; + false -> + {error, gap_detected} + end end. -spec stage(log_entry(), state()) -> diff --git a/src/ra_server.erl b/src/ra_server.erl index 5caeaa5f..9860306d 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -1532,7 +1532,6 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, ?DEBUG("~ts: receiving snapshot chunk: ~b / ~w, index ~b, term ~b", [LogId, Num, ChunkFlag, SnapIndex, SnapTerm]), SnapState0 = ra_log:snapshot_state(Log0), - ct:pal("SnapStat0 ~p", [SnapState0]), {ok, SnapState, Effs0} = ra_snapshot:accept_chunk(Data, Num, ChunkFlag, SnapState0), Reply = #install_snapshot_result{term = CurTerm, diff --git a/test/ra_log_memory.erl b/test/ra_log_memory.erl index 54755b0a..cad46764 100644 --- a/test/ra_log_memory.erl +++ b/test/ra_log_memory.erl @@ -41,6 +41,8 @@ -include("src/ra.hrl"). +-dialyzer({nowarn_function, install_snapshot/4}). + -type ra_log_memory_meta() :: #{atom() => term()}. -record(state, {last_index = 0 :: ra_index(), @@ -50,7 +52,7 @@ meta = #{} :: ra_log_memory_meta(), snapshot :: option({ra_snapshot:meta(), term()})}). --type ra_log_memory_state() :: #state{} | ra_log:state(). +-opaque ra_log_memory_state() :: #state{} | ra_log:state(). -export_type([ra_log_memory_state/0]). @@ -170,17 +172,18 @@ last_written(#state{last_written = LastWritten}) -> -spec handle_event(ra_log:event_body(), ra_log_memory_state()) -> {ra_log_memory_state(), list()}. -handle_event({written, Term, {_From, Idx} = Range0}, State0) -> +handle_event({written, Term, Seq0}, State0) when is_list(Seq0) -> + Idx = ra_seq:last(Seq0), case fetch_term(Idx, State0) of {Term, State} -> {State#state{last_written = {Idx, Term}}, []}; _ -> - case ra_range:limit(Idx, Range0) of - undefined -> + case ra_seq:limit(Idx - 1, Seq0) of + [] -> % if the term doesn't match we just ignore it {State0, []}; - Range -> - handle_event({written, Term, Range}, State0) + Seq -> + handle_event({written, Term, Seq}, State0) end end; handle_event(_Evt, State0) -> @@ -210,6 +213,8 @@ fetch_term(Idx, #state{entries = Log} = State) -> flush(_Idx, Log) -> Log. +-spec install_snapshot(ra_idxterm(), term(), module(), ra_log_memory_state()) -> + {ra_snapshot:meta(), term(), ra_log_memory_state(), list()}. install_snapshot({Index, Term}, Data, _MacMod, #state{entries = Log0} = State0) when is_tuple(Data) -> diff --git a/test/ra_log_meta_SUITE.erl b/test/ra_log_meta_SUITE.erl index ae2c7bee..8ce74607 100644 --- a/test/ra_log_meta_SUITE.erl +++ b/test/ra_log_meta_SUITE.erl @@ -38,6 +38,7 @@ init_per_group(_, Config) -> Config. end_per_group(_, Config) -> + application:stop(ra), Config. init_per_testcase(TestCase, Config) -> diff --git a/test/ra_log_props_SUITE.erl b/test/ra_log_props_SUITE.erl deleted file mode 100644 index a50456fd..00000000 --- a/test/ra_log_props_SUITE.erl +++ /dev/null @@ -1,858 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2017-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. -%% --module(ra_log_props_SUITE). --compile(nowarn_export_all). --compile(export_all). - --include_lib("proper/include/proper.hrl"). --include_lib("common_test/include/ct.hrl"). --include_lib("eunit/include/eunit.hrl"). - -all() -> - [ - {group, tests} - ]. - -%% these tests were useful during the early days but it isn't clear how -%% much they now contribute -%% TODO: consider refactoring using a more condensed set of properties -%% that only test clear log invariants (e.g. overwritten entries are never read). -all_tests() -> - [ - write, - % write_missing_entry, - % multi_write_missing_entry, - write_overwrite_entry, - % write_index_starts_zero, - append - % append_missing_entry, - % append_overwrite_entry, - % append_index_starts_one, - % take, - % take_out_of_range, - % fetch, - % fetch_out_of_range, - % last_index_term, - % fetch_term, - % fetch_out_of_range_term, - % next_index_term, - % last_written, - % last_written_with_wal, - % last_written_with_segment_writer, - % last_written_with_crashing_segment_writer - ]. - -groups() -> - [ - {tests, [], all_tests()} - ]. - -init_per_suite(Config) -> - {ok, _} = ra:start([{data_dir, ?config(priv_dir, Config)}, - {segment_max_entries, 128}]), - Config. - -end_per_suite(Config) -> - application:stop(ra), - Config. - -init_per_testcase(TestCase, Config) -> - application:stop(ra), - PrivDir = ?config(priv_dir, Config), - Dir = filename:join(PrivDir, TestCase), - ra:start_in(Dir), - % register(TestCase, self()), - UId = atom_to_binary(TestCase, utf8), - ok = ra_directory:register_name(default, UId, self(), undefined, - TestCase, TestCase), - [{test_case, UId}, {wal_dir, Dir} | Config]. - -%%------------------ -%% Generators -%%------------------ - -log_entries_gen(N) -> - ?LET(Length, choose(N, 100), - ?LET(Terms, term_sequence_gen(Length), - [{Idx, Term, <>} - || {Idx, Term} <- lists:zip(lists:seq(1, Length), - Terms)])). - -term_sequence_gen(N) -> - ?LET(List, vector(N, non_neg_integer()), - lists:sort(List)). - -wait_sequence_gen(N) -> - ?LET(List, vector(N, wait_gen()), List). - -wait_gen() -> - frequency([{8, 0}, {5, choose(0, 20)}, {1, choose(25, 150)}]). - -consume_gen(N) -> - ?LET(List, vector(N, boolean()), List). - -log_entry_but_one_gen() -> - ?LET(Idx, ?SUCHTHAT(Int, integer(), Int =/= 1), - {Idx, 1, <>}). - -log_entry_but_one_zero_gen() -> - ?LET(Idx, ?SUCHTHAT(Int, integer(), (Int =/= 1) and (Int =/= 0)), - {Idx, 1, <>}). - -slice_gen(Entries) -> - %% Head might be an empty list - ?LET(N, choose(1, max_length(Entries)), - begin - {Head, [NEntry | Tail]} = lists:split(N - 1, Entries), - {Head, NEntry, Tail} - end). - -sorted_subset_gen(Entries) -> - ?LET(Subset, list(elements(Entries)), lists:sort(Subset)). - -max_length(Entries) when length(Entries) > 1 -> - length(Entries) - 1; -max_length(_) -> - 1. - -range_gen(Entries) -> - %% Range can finish anywhere after total number of entries - ?LET(Start, between_gen(1, length(Entries)), - ?LET(Num, greater_than_gen(1), - {Start, Num})). - -out_of_range_begin_gen() -> - %% The range starts before the initial index - ?LET(Start, less_than_gen(0), - ?LET(Num, greater_than_gen(0), - {Start, Num})). - -out_of_range_end_gen(Entries) -> - %% The range starts after the last index - ?LET(Start, greater_than_gen(length(Entries)), - ?LET(Num, non_neg_integer(), - {Start, Num})). - -between_gen(N, M) -> - choose(N, M). - -greater_than_gen(N) -> - integer(N + 1, inf). - -less_than_gen(N) -> - integer(inf, N - 1). - -out_of_range_gen(Entries) -> - oneof([out_of_range_begin_gen(), - out_of_range_end_gen(Entries)]). - -%%------------------ -%% Properties -%%------------------ - -write(Config) -> - %% There is no way to create a log file from a list of entries without the write - %% API. We have to prove first that writing a consecutive log file succeeds, - %% so we can use it as a base for our tests - TestCase = ?config(test_case, Config), - run_proper(fun write_prop/1, [TestCase], 100). - -write_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {LogEntries, Log} = ra_log_take(1, length(Entries), Log0), - ?WHENFAIL(io:format("Entries taken from the log: ~p~nRa log state: ~p", - [LogEntries, Log]), - Entries == LogEntries) - end). - -append_all([], Log) -> - Log; -append_all([Entry | Entries], Log0) -> - Log = ra_log:append(Entry, Log0), - append_all(Entries, Log). - -write_missing_entry(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun write_missing_entry_prop/1, [TestCase], 100). - -write_missing_entry_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(3), - ?FORALL( - {Head, _Entry, Tail}, slice_gen(Entries), - begin - Log = ra_log_init(#{uid => TestCase}), - Reply = ra_log:write(Head ++ Tail, Log), - reset(Log), - ?WHENFAIL(ct:pal("Reply: ~p", [Reply]), - case Reply of - {error, {integrity_error, _}} -> true; - _ -> false - end) - end)). - -write_overwrite_entry(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun write_overwrite_entry_prop/1, [TestCase], 250). - -write_overwrite_entry_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(3), - ?FORALL( - {Head, {Idx, Term, _Value} = _Entry, _Tail}, slice_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - NewEntry = [{Idx, Term, <<"overwrite">>}], - {ok, Log} = ra_log:write(NewEntry, Log0), - {LogEntries, Log1} = ra_log_take(1, length(Entries), Log), - reset(Log1), - ?WHENFAIL(io:format("Head: ~p~n New entry: ~p~n" - "Entries taken from the log: ~p~n" - "Ra log state: ~p", - [Head, NewEntry, LogEntries, Log1]), - ((Head ++ NewEntry) == LogEntries)) - end)). - -multi_write_missing_entry(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun multi_write_missing_entry_prop/1, [TestCase], 100). - -multi_write_missing_entry_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(3), - ?FORALL( - {Head, _Entry, Tail}, slice_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Head, - ra_log_init(#{uid => TestCase})), - Reply = ra_log:write(Tail, Log0), - reset(Log0), - ?WHENFAIL(io:format("Reply: ~p", [Reply]), - case Reply of - {error, {integrity_error, _}} -> true; - _ -> false - end) - end)). - -append_missing_entry(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun append_missing_entry_prop/1, [TestCase], 100). - -append_missing_entry_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(3), - ?FORALL( - {Head, _Entry, Tail}, slice_gen(Entries), - begin - Log0 = append_all(Head, - ra_log_init(#{uid => TestCase})), - Failed = try - ra_log:append(hd(Tail), Log0), - false - catch - exit:{integrity_error, _} -> - true - end, - {LogEntries, Log} = ra_log_take(1, length(Head), Log0), - reset(Log), - ?WHENFAIL(io:format("Failed: ~p~nHead: ~p~n Tail: ~p~n" - "Entries taken from the log: ~p~n" - "Ra log state: ~p", - [Failed, Head, Tail, LogEntries, Log]), - (Head == LogEntries) and Failed) - end)). - -write_index_starts_zero(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun write_index_starts_zero_prop/1, [TestCase], 100). - -write_index_starts_zero_prop(TestCase) -> - ?FORALL( - Entry, log_entry_but_one_zero_gen(), - begin - Log = ra_log_init(#{uid => TestCase}), - Reply = ra_log:write([Entry], Log), - reset(Log), - ?WHENFAIL(io:format("Reply: ~p", [Reply]), - case Reply of - {error, {integrity_error, _}} -> true; - _ -> false - end) - end). - -append(Config) -> - %% There is no way to create a log file from a list of entries without the - %% write - %% API. We have to prove first that writing a consecutive log file succeeds, - %% so we can use it as a base for our tests - TestCase = ?config(test_case, Config), - run_proper(fun append_prop/1, [TestCase], 100). - -append_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - begin - Log0 = append_all( - Entries, - ra_log_init(#{uid => TestCase})), - {LogEntries, Log} = ra_log_take(1, length(Entries), Log0), - reset(Log), - ?WHENFAIL(io:format("Entries taken from the log: ~p~nRa log state: ~p", - [LogEntries, Log]), - Entries == LogEntries) - end). - -append_overwrite_entry(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun append_overwrite_entry_prop/1, [TestCase], 100). - -append_overwrite_entry_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(3), - ?FORALL( - {_Head, {Idx, Term, _Value} = _Entry, _Tail}, slice_gen(Entries), - begin - {ok, Log} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - Failed = try - ra_log:append({Idx, Term, <<"overwrite">>}, Log), - false - catch - exit:{integrity_error, _} -> - true - end, - reset(Log), - ?WHENFAIL(io:format("Failed: ~p", [Failed]), - Failed) - end)). - -append_index_starts_one(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun append_index_starts_one_prop/1, [TestCase], 100). - -append_index_starts_one_prop(TestCase) -> - ?FORALL( - Entry, log_entry_but_one_gen(), - begin - Log = ra_log_init(#{uid => TestCase}), - Failed = try - ra_log:append(Entry, Log), - false - catch - exit:{integrity_error, _} -> - true - end, - reset(Log), - ?WHENFAIL(io:format("Failed: ~p Entry: ~p", [Failed, Entry]), Failed) - end). - -take(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun take_prop/1, [TestCase], 100). - -take_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - {Start, Num}, range_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {Selected, Log} = ra_log_take(Start, Start + Num - 1, Log0), - Expected = lists:sublist(Entries, Start, Num), - reset(Log), - ?WHENFAIL(io:format("Selected: ~p~nExpected: ~p", - [Selected, Expected]), - Selected == Expected) - end)). - -take_out_of_range(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun take_out_of_range_prop/1, [TestCase], 100). - -take_out_of_range_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - {Start, Num}, out_of_range_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {Reply, Log} = ra_log_take(Start, Start + Num - 1, Log0), - reset(Log), - ?WHENFAIL(io:format("Start: ~p Num: ~p~nReply: ~p", [Start, Num, Reply]), - Reply == []) - end)). - -fetch(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun fetch_prop/1, [TestCase], 100). - -fetch_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - {_Head, {Idx, _Term, _Value} = Entry, _Tail}, slice_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {Got, Log} = ra_log:fetch(Idx, Log0), - reset(Log), - ?WHENFAIL(io:format("Got: ~p Expected: ~p", [Got, Entry]), - Entry == Got) - end)). - -fetch_out_of_range(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun fetch_out_of_range_prop/1, [TestCase], 100). - -fetch_out_of_range_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - {Start, _Num}, out_of_range_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {Reply, Log} = ra_log:fetch(Start, Log0), - reset(Log), - ?WHENFAIL(io:format("Got: ~p Expected: undefined", [Reply]), - Reply == undefined) - end)). - -last_index_term(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun last_index_term_prop/1, [TestCase], 100). - -last_index_term_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(0), - begin - {ok, Log} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {LastIdx, LastTerm} = case Entries of - [] -> - {0, 0}; - _ -> - {LI, LT, _} = lists:last(Entries), - {LI, LT} - end, - {Idx, Term} = ra_log:last_index_term(Log), - reset(Log), - ?WHENFAIL(io:format("Got: ~p Expected: ~p", [{Idx, Term}, {LastIdx, LastTerm}]), - (LastIdx == Idx) and (LastTerm == Term)) - end). - -fetch_term(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun fetch_term_prop/1, [TestCase], 100). - -fetch_term_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - {_Head, {Idx, ExpectedTerm, _}, _Tail}, slice_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {Term, Log} = ra_log:fetch_term(Idx, Log0), - reset(Log), - ?WHENFAIL(io:format("Got: ~p Expected: ~p", [Term, ExpectedTerm]), - (ExpectedTerm == Term)) - end)). - -fetch_out_of_range_term(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun fetch_out_of_range_term_prop/1, [TestCase], 100). - -fetch_out_of_range_term_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - {Start, _}, out_of_range_gen(Entries), - begin - {ok, Log0} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {Term, Log} = ra_log:fetch_term(Start, Log0), - reset(Log), - ?WHENFAIL(io:format("Got: ~p for index: ~p", [Term, Start]), - (undefined == Term) orelse ((0 == Term) and (Start == 0))) - end)). - -next_index_term(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun last_index_term_prop/1, [TestCase], 100). - -next_index_term_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - begin - {ok, Log} = ra_log:write( - Entries, - ra_log_init(#{uid => TestCase})), - {LastIdx, _LastTerm, _} = lists:last(Entries), - Idx = ra_log:next_index(Log), - reset(Log), - ?WHENFAIL(io:format("Got: ~p Expected: ~p", [Idx, LastIdx + 1]), - LastIdx + 1 == Idx) - end). - - -last_written_with_wal(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun last_written_with_wal_prop/1, [TestCase], 15). - -build_action_list(Entries, Actions) -> - lists:flatten(lists:map(fun(Index) -> - E = lists:nth(Index, Entries), - A = lists:foldl(fun({A0, I}, Acc) when I == Index -> - [A0 | Acc]; - (_, Acc) -> - Acc - end, [], Actions), - [E | A] - end, lists:seq(1, length(Entries)))). - -position(Entries) -> - choose(1, length(Entries)). - -last_written_with_wal_prop(TestCase) -> - ok = logger:set_primary_config(level, all), - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - Actions, list(frequency([{5, {{wait, wait_gen()}, position(Entries)}}, - {3, {consume, position(Entries)}}, - {2, {roll_wal, position(Entries)}}, - {2, {stop_wal, position(Entries)}}, - {2, {start_wal, position(Entries)}}])), - begin - flush(), - All = build_action_list(Entries, Actions), - Log0 = ra_log_init(#{uid => TestCase}), - {Log, Last, LastIdx, _Status} = - lists:foldl(fun({wait, Wait}, Acc) -> - timer:sleep(Wait), - Acc; - (consume, {Acc0, Last0, LastIdx, St}) -> - {Acc1, Last1} = consume_events(Acc0, Last0), - {Acc1, Last1, LastIdx, St}; - (roll_wal, {_, _, _, wal_down} = Acc) -> - Acc; - (roll_wal, Acc) -> - ra_log_wal:force_roll_over(ra_log_wal), - Acc; - (stop_wal, {Acc0, Last0, LastIdx, wal_up}) -> - ok = supervisor:terminate_child(wal_sup(), ra_log_wal), - {Acc0, Last0, LastIdx, wal_down}; - (stop_wal, {_, _, _, wal_down} = Acc) -> - Acc; - (start_wal, {Acc0, Last0, LastIdx, wal_down}) -> - supervisor:restart_child(wal_sup(), ra_log_wal), - {Acc0, Last0, LastIdx, wal_up}; - (start_wal, {_, _, _, wal_up} = Acc) -> - Acc; - ({Idx, _, _} = Entry, {Acc0, _, LastIdx, _} = Acc) when Idx > LastIdx + 1 -> - {error, {integrity_error, _}} = ra_log:write([Entry], Acc0), - Acc; - (Entry, {Acc0, _, _, wal_down} = Acc) -> - {error, wal_down} = ra_log:write([Entry], Acc0), - Acc; - ({Idx, _, _} = Entry, {Acc0, Last0, _LastIdx, St}) -> - {ok, Acc} = ra_log:write([Entry], Acc0), - {Acc, Last0, Idx, St} - end, {Log0, {0, 0}, 0, wal_up}, All), - Got = ra_log:last_written(Log), - {Written, Log1} = ra_log_take(1, LastIdx, Log), - reset(Log1), - ?WHENFAIL(io:format("Got: ~p, Expected: ~p Written: ~p~n Actions: ~p", - [Got, Last, Written, All]), - (Got == Last) and (Written == lists:sublist(Entries, 1, LastIdx))) - end)). - -last_written_with_segment_writer(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun last_written_with_segment_writer_prop/1, [TestCase], 25). - -last_written_with_segment_writer_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - Actions, list(frequency([{5, {{wait, wait_gen()}, position(Entries)}}, - {3, {consume, position(Entries)}}, - {2, {stop_segment_writer, position(Entries)}}, - {2, {start_segment_writer, position(Entries)}}])), - begin - flush(), - All = build_action_list(Entries, Actions), - _ = supervisor:restart_child(ra_log_sup, ra_log_segment_writer), - Log0 = ra_log_init(#{uid => TestCase}), - {Log, Last, LastIdx, _Status} = - lists:foldl(fun({wait, Wait}, Acc) -> - timer:sleep(Wait), - Acc; - (consume, {Acc0, Last0, LastIdx, St}) -> - {Acc1, Last1} = consume_events(Acc0, Last0), - {Acc1, Last1, LastIdx, St}; - (stop_segment_writer, {Acc0, Last0, LastIdx, sw_up}) -> - ok = supervisor:terminate_child(ra_log_sup, ra_log_segment_writer), - {Acc0, Last0, LastIdx, sw_down}; - (stop_segment_writer, {_, _, _, sw_down} = Acc) -> - Acc; - (start_segment_writer, {Acc0, Last0, LastIdx, sw_down}) -> - {ok, _} = supervisor:restart_child(ra_log_sup, ra_log_segment_writer), - {Acc0, Last0, LastIdx, sw_up}; - (start_segment_writer, {_, _, _, sw_up} = Acc) -> - Acc; - ({Idx, _, _} = Entry, {Acc0, _, LastIdx, _} = Acc) when Idx > LastIdx + 1 -> - {error, {integrity_error, _}} = ra_log:write([Entry], Acc0), - Acc; - ({Idx, _, _} = Entry, {Acc0, Last0, _LastIdx, St}) -> - {ok, Acc} = ra_log:write([Entry], Acc0), - {Acc, Last0, Idx, St} - end, {Log0, {0, 0}, 0, sw_up}, All), - Got = ra_log:last_written(Log), - {Written, Log1} = ra_log_take(1, LastIdx, Log), - reset(Log1), - ?WHENFAIL(ct:pal("Got: ~p, Expected: ~p Written: ~p~n Actions: ~p", - [Got, Last, Written, All]), - (Got == Last) and (Written == lists:sublist(Entries, 1, LastIdx))) - end)). - -last_written_with_crashing_segment_writer(Config) -> - TestCase = ?config(test_case, Config), - run_proper_noshrink(fun last_written_with_crashing_segment_writer_prop/1, - [TestCase], 1). - -last_written_with_crashing_segment_writer_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - Actions, list(frequency([{5, {{wait, wait_gen()}, position(Entries)}}, - {3, {consume, position(Entries)}}, - {2, {crash_segment_writer, position(Entries)}}])), - begin - flush(), - All = build_action_list(Entries, Actions), - _ = supervisor:restart_child(ra_log_sup, ra_log_segment_writer), - Log0 = ra_log_init(#{uid => TestCase, - resend_window => 2}), - ra_log_take(1, 10, Log0), - {Log, _Last, Ts} = - lists:foldl(fun({wait, Wait}, Acc) -> - timer:sleep(Wait), - Acc; - (consume, {Acc0, Last0, Ts}) -> - Acc1 = deliver_log_events(Acc0, 500), - {Acc1, Last0, Ts}; - (crash_segment_writer, {Acc0, Last0, _Ts}) -> - Acc = case whereis(ra_log_segment_writer) of - undefined -> - Acc0; - P -> - Acc1 = deliver_log_events(Acc0, 500), - exit(P, kill), - Acc1 - end, - {Acc, Last0, get_timestamp()}; - (Entry, {Acc0, Last0, Ts}) -> - case ra_log:write([Entry], Acc0) of - {ok, Acc} -> - {Acc, Last0, Ts}; - {error, wal_down} -> - wait_for_wal(50, 0), - {ok, Acc} = ra_log:write([Entry], Acc0), - {Acc, Last0, Ts} - end - end, {Log0, {0, 0}, get_timestamp()}, All), - %% We want to check that eventually we get the last written as the last entry, - %% despite the segment writer crash. The log file might have to resend - %% some entries after it, so it needs time to recover. - timer:sleep(time_diff_to(Ts, 3000)), - % write an entry to trigger resend protocol if required - {LastIdx, LastTerm} = ra_log:last_index_term(Log), - E = {LastIdx+1, LastTerm, <<>>}, - ActuallyLastIdxTerm = {LastIdx+1, LastTerm}, - {ok, Log1a} = ra_log:write([E], Log), - Log1 = deliver_log_events(Log1a, 500), - % Log1c = deliver_log_events(Log1b, 500), - %% Consume all events - % {Log1, Last1} = consume_events(Log1b, Last), - %% Request last written - LastWritten = ra_log:last_written(Log1), - %% Request entries available, which should be all generated by this test - {EIdx, ETerm, _} = lists:last(Entries), - LastEntry = {EIdx, ETerm}, - ct:pal("Log1 ~p~nopen ~p~nclosed~p", [Log1, - ets:tab2list(ra_log_open_mem_tables), - ets:tab2list(ra_log_closed_mem_tables) - ]), - {Written, Log2} = ra_log_take(1, EIdx, Log1), - %% We got all the data, can reset now - basic_reset(Log2), - ?WHENFAIL(ct:pal("Last written entry: ~p; actually last idx term: ~p;" - " last entry written: ~p~nEntries taken: ~p~n Actions: ~p", - [LastWritten, ActuallyLastIdxTerm, LastEntry, Written, Entries]), - (LastWritten == ActuallyLastIdxTerm) - and (Written == Entries)) - end)). - -get_timestamp() -> - {Mm, S, Mi} = os:timestamp(), - (Mm * 1000000 + S) * 1000 + round(Mi / 1000). - -time_diff_to(Ts, To) -> - Tnow = get_timestamp(), - case To - (Tnow - Ts) of - T when T < 0 -> - 0; - T -> - T - end. - -wait_for_wal(N, N) -> - exit(wait_for_wal_timeout); -wait_for_wal(M, N) -> - timer:sleep(100), - case whereis(ra_log_wal) of - undefined -> - wait_for_wal(M, N+1); - _ -> ok - end. - - -last_written(Config) -> - TestCase = ?config(test_case, Config), - run_proper(fun last_written_prop/1, [TestCase], 10). - -last_written_prop(TestCase) -> - ?FORALL( - Entries, log_entries_gen(1), - ?FORALL( - {Waits, Consumes}, {wait_sequence_gen(length(Entries)), consume_gen(length(Entries))}, - begin - flush(), - Actions = lists:zip3(Entries, Waits, Consumes), - Log0 = ra_log_init(#{uid => TestCase}), - {Log, Last} = lists:foldl(fun({Entry, Wait, Consume}, {Acc0, Last0}) -> - {ok, Acc} = ra_log:write([Entry], Acc0), - timer:sleep(Wait), - case Consume of - true -> - consume_events(Acc, Last0); - false -> - {Acc, Last0} - end - end, {Log0, {0, 0}}, Actions), - Got = ra_log:last_written(Log), - reset(Log), - ?WHENFAIL(io:format("Got: ~p, Expected: ~p~n Actions: ~p", - [Got, Last, Actions]), - Got == Last) - end)). - -flush() -> - receive - {ra_log_event, _} -> - flush() - after 100 -> - ok - end. - -deliver_log_events(Log0, Timeout) -> - receive - {ra_log_event, Evt} -> - ct:pal("ra_log_evt: ~w", [Evt]), - {Log, _} = ra_log:handle_event(Evt, Log0), - deliver_log_events(Log, Timeout) - after Timeout -> - Log0 - end. - -consume_events(Log0, Last) -> - receive - {ra_log_event, {written, Term, {_, To}} = Evt} -> - {Log, _} = ra_log:handle_event(Evt, Log0), - consume_events(Log, {To, Term}) - after 0 -> - {Log0, Last} - end. - -consume_all_events(Log0, Last) -> - receive - {ra_log_event, {written, Term, {_, To}} = Evt} -> - {Log, _} = ra_log:handle_event(Evt, Log0), - consume_events(Log, {To, Term}) - after 15000 -> - {Log0, Last} - end. - -last_idx_term([]) -> - {0,0}; -last_idx_term(List) -> - {Idx, Term, _} = lists:last(lists:sort(List)), - {Idx, Term}. - -%% TODO refactor -run_proper(Fun, Args, NumTests) -> - ?assertEqual( - true, - proper:counterexample(erlang:apply(Fun, Args), - [{numtests, NumTests}, - {on_output, fun(".", _) -> ok; % don't print the '.'s on new lines - (F, A) -> ct:pal(?LOW_IMPORTANCE, F, A) end}])). - -run_proper_noshrink(Fun, Args, NumTests) -> - ?assertEqual( - true, - proper:counterexample(erlang:apply(Fun, Args), - [{numtests, NumTests}, - noshrink, - {on_output, fun(".", _) -> ok; % don't print the '.'s on new lines - (F, A) -> ct:pal(?LOW_IMPORTANCE, F, A) end}])). -basic_reset(Log) -> - ra_log:write([{0, 0, empty}], Log), - receive - {ra_log_event, {written, _, [0]}} -> - 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_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index 26f630e4..31ee6be0 100644 --- a/test/ra_log_wal_SUITE.erl +++ b/test/ra_log_wal_SUITE.erl @@ -28,6 +28,7 @@ all_tests() -> 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, @@ -207,19 +208,57 @@ sparse_write_same_batch(Config) -> 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), - Tid = ets:new(?FUNCTION_NAME, []), + 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. - ok = proc_lib:stop(ra_log_wal), - {ok, _Pid2} = ra_log_wal:start_link(Conf), +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]}]}, _}} -> @@ -228,7 +267,8 @@ sparse_write_recover(Config) -> flush(), ct:fail("receiving mem table ranges timed out") end, - proc_lib:stop(Pid), + flush(), + proc_lib:stop(Pid2), meck:unload(), ok. @@ -976,7 +1016,7 @@ recover_overwrite(Config) -> _ = 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), diff --git a/test/ra_machine_ets_SUITE.erl b/test/ra_machine_ets_SUITE.erl index ceb7e53d..249fecf3 100644 --- a/test/ra_machine_ets_SUITE.erl +++ b/test/ra_machine_ets_SUITE.erl @@ -12,7 +12,6 @@ -export([ ]). --include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). %%%=================================================================== diff --git a/test/ra_server_SUITE.erl b/test/ra_server_SUITE.erl index ca52d3bd..044d30ab 100644 --- a/test/ra_server_SUITE.erl +++ b/test/ra_server_SUITE.erl @@ -11,6 +11,12 @@ -include("src/ra_server.hrl"). -include_lib("eunit/include/eunit.hrl"). +%% TODO: make so this is not needed +-dialyzer({nowarn_function, + [init_test/1, + higher_term_detected/1, + follower_aer_term_mismatch_snapshot/1]}). + all() -> [ init_test, @@ -169,10 +175,10 @@ setup_log() -> meck:expect(ra_log, fold, fun ra_log_memory:fold/5), meck:expect(ra_log, release_resources, fun ra_log_memory:release_resources/3), meck:expect(ra_log, append_sync, - fun({Idx, Term, _} = E, L) -> - L1 = ra_log_memory:append(E, L), - {LX, _} = ra_log_memory:handle_event({written, Term, {Idx, Idx}}, L1), - LX + fun({Idx, Term, _} = E, L0) -> + L1 = ra_log_memory:append(E, L0), + {L, _} = ra_log_memory:handle_event({written, Term, [Idx]}, L1), + L end), meck:expect(ra_log, write_config, fun ra_log_memory:write_config/2), meck:expect(ra_log, next_index, fun ra_log_memory:next_index/1), @@ -1868,8 +1874,8 @@ follower_cluster_change(_Config) -> ok. -written_evt(Term, Range) -> - {ra_log_event, {written, Term, Range}}. +written_evt(Term, Range) when is_tuple(Range) -> + {ra_log_event, {written, Term, [Range]}}. leader_applies_new_cluster(_Config) -> N1 = ?N1, N2 = ?N2, N3 = ?N3, N4 = ?N4, @@ -3146,7 +3152,7 @@ base_state(NumServers, MacMod) -> [{1, 1, usr(<<"hi1">>)}, {2, 3, usr(<<"hi2">>)}, {3, 5, usr(<<"hi3">>)}]), - {Log, _} = ra_log:handle_event({written, 5, {1, 3}}, Log0), + {Log, _} = ra_log:handle_event({written, 5, [{1, 3}]}, Log0), Servers = lists:foldl(fun(N, Acc) -> Name = {list_to_atom("n" ++ integer_to_list(N)), node()},