From 6fe10810619225da678af3a07cc3c29e3afd34f3 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 2 Jan 2025 10:00:36 +0000 Subject: [PATCH 01/60] 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 41788fade..5fd0419bd 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 @@ -515,16 +476,6 @@ start_server(System, ClusterName, #{id := {_, _}} = Conf0, Machine, ServerIds) machine => Machine}, start_server(System, maps:merge(Conf0, Conf)). -%% @doc Starts a ra server in the default system -%% @param Conf a ra_server_config() configuration map. -%% @returns `{ok | error, Error}' -%% DEPRECATED: use start_server/2 -%% @end --spec start_server(ra_server:ra_server_config()) -> - ok | {error, term()}. -start_server(Conf) -> - start_server(default, Conf). - %% @doc Starts a ra server %% @param System the system name %% @param Conf a ra_server_config() configuration map. @@ -748,14 +699,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 @@ -1197,18 +1140,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 12f2ebbfc..a21ad24fc 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 5d6a5fef1..ac6740fa7 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 fa3e0bf80..fddf73210 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 41581f05e288e057e4c8c43f6e616eefe03a2c44 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 2 Jan 2025 16:21:40 +0000 Subject: [PATCH 02/60] 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 75c436dab..49d2e3bdf 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 000000000..6d3eeed6b --- /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 fa1c3daa96ee403046493c98d905d039909a343a Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 31 Jan 2025 13:00:18 +0000 Subject: [PATCH 03/60] 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 986412ef4..843f5ec0c 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 56fff9223..85ca9e957 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 53129fe7c..7be554067 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 5d820d5b9..deea5997f 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 12eefe74a..c27f5038f 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 3857996c6..11c395164 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 8b18879f6..a4c1f06c3 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 = ra_file:rename(Checkpoint, Snapshot), - Self ! {ra_log_event, - {snapshot_written, - {Idx, Term}, snapshot}} - end), - State = State0#?MODULE{pending = {Pid, {Idx, Term}, snapshot}, + Fun = fun() -> + %% Checkpoints are created without calling + %% fsync. Snapshots must be fsync'd though, so + %% sync the checkpoint before promoting it + %% into a snapshot. + ok = Mod:sync(Checkpoint), + ok = ra_file:rename(Checkpoint, Snapshot), + 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 000000000..cddd2e030 --- /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 fddf73210..c8b441f64 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 8640d7f51..a41014261 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 6c5816c25..4044ef43e 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 0d74b0f8d..09f752db1 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 f811fb384..759f2170b 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 18d164034..0d696d316 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 e231f85ef2988d670dd4ea6cb4241e15357f57b5 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 17 Feb 2025 12:48:25 +0000 Subject: [PATCH 04/60] 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 7be554067..ccbef40cd 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 deea5997f..9deee4f27 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 4044ef43e..fb73bd314 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 b69a91ea9c12aaf558ea363b70c6a688b9037d38 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 11:19:11 +0000 Subject: [PATCH 05/60] 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 6d3eeed6b..04a840b18 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 ccbef40cd..191525fef 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 ea171e69f..d99a14a93 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 87ca98d3c..e5235866f 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 133ed5b54..943f13683 100644 --- a/src/ra_machine.erl +++ b/src/ra_machine.erl @@ -72,6 +72,7 @@ snapshot_installed/5, state_enter/3, overview/2, + live_indexes/2, query/3, module/1, init_aux/2, @@ -224,6 +225,7 @@ handle_aux/5, handle_aux/6, overview/1, + live_indexes/1, snapshot_module/0, version/0, which_module/1 @@ -289,6 +291,8 @@ -callback overview(state()) -> map(). +-callback live_indexes(state()) -> [ra:index()]. + -callback snapshot_module() -> module(). -callback version() -> version(). @@ -346,11 +350,17 @@ state_enter(Mod, RaftState, State) -> overview(Mod, State) -> ?OPT_CALL(Mod:overview(State), State). +-spec live_indexes(module(), state()) -> [ra:index()]. +live_indexes(Mod, State) -> + ?OPT_CALL(Mod:live_indexes(State), []). + %% @doc used to discover the latest machine version supported by the current %% code --spec version(machine()) -> version(). +-spec version(machine() | module()) -> version(). +version(Mod) when is_atom(Mod) -> + ?OPT_CALL(assert_version(Mod:version()), ?DEFAULT_VERSION); version({machine, Mod, _}) -> - ?OPT_CALL(assert_version(Mod:version()), ?DEFAULT_VERSION). + version(Mod). -spec is_versioned(machine()) -> boolean(). is_versioned({machine, Mod, _}) -> diff --git a/src/ra_server.erl b/src/ra_server.erl index 9deee4f27..515c1863b 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 a4c1f06c3..124f0df38 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 d643090e5993d50bbdb7241d762642f95715070b Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 11:33:38 +0000 Subject: [PATCH 06/60] 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 a41014261..915ae1f19 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 fb73bd314..7a7b11453 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 0d696d316..8889a974f 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 85df230c1134b889d5e063cb23b08c50a1faba03 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 11:57:22 +0000 Subject: [PATCH 07/60] 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 5b747a262..5ecbdff29 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 5fd0419bd..70f47af46 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 b2850a2f1..720f423c0 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 e5235866f..cd2d7163a 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 303b84806..6b2080cab 100644 --- a/src/ra_system.erl +++ b/src/ra_system.erl @@ -36,7 +36,6 @@ wal_compute_checksums => boolean(), wal_max_batch_size => non_neg_integer(), wal_max_entries => undefined | non_neg_integer(), - wal_write_strategy => default | o_sync | sync_after_notify, wal_sync_method => datasync | sync | none, wal_hibernate_after => non_neg_integer(), wal_garbage_collect => boolean(), @@ -92,7 +91,6 @@ default_config() -> WalMaxBatchSize = application:get_env(ra, wal_max_batch_size, ?WAL_DEFAULT_MAX_BATCH_SIZE), WalMaxEntries = application:get_env(ra, wal_max_entries, undefined), - WalWriteStrategy = application:get_env(ra, wal_write_strategy, default), WalSyncMethod = application:get_env(ra, wal_sync_method, datasync), DataDir = ra_env:data_dir(), WalDataDir = application:get_env(ra, wal_data_dir, DataDir), @@ -127,7 +125,6 @@ default_config() -> wal_compute_checksums => WalComputeChecksums, wal_max_batch_size => WalMaxBatchSize, wal_max_entries => WalMaxEntries, - wal_write_strategy => WalWriteStrategy, wal_garbage_collect => WalGarbageCollect, wal_pre_allocate => WalPreAllocate, wal_sync_method => WalSyncMethod, diff --git a/test/ra_log_wal_SUITE.erl b/test/ra_log_wal_SUITE.erl index a538a52a6..93448acb8 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 4c4e90ece16784f37d125da759a199168b71c2cb Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 25 Feb 2025 12:59:59 +0000 Subject: [PATCH 08/60] 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 720f423c0..117febdb0 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 cd2d7163a..19e9f4253 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 93448acb8..4f08c3c9d 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 bfaf36bcb6c8cd29cfd4fea19cff38a3a3d3c867 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 3 Mar 2025 08:10:38 +0000 Subject: [PATCH 09/60] 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 caf81ecf9..bdb47c89f 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 191525fef..2d9fba73c 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 71df7b947..16d4d3387 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -264,7 +264,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 @@ -273,14 +273,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; @@ -353,15 +353,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 000000000..2f3e5f58e --- /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 19e9f4253..1471bb768 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 000000000..f7007fba6 --- /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 124f0df38..e6cc37972 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 7a7b11453..ac2c3b608 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 5aed1caaf..3a93317c5 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 4f08c3c9d..bc9d07c5f 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 5511ce04a81fc6198ac79da981c7e06f7e9ab993 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 5 Mar 2025 10:56:58 +0000 Subject: [PATCH 10/60] 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 04a840b18..6a8af47c8 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 2d9fba73c..77f2b6ae8 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 16d4d3387..05d2a1836 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 {ok, _, Failures} = diff --git a/src/ra_log_wal.erl b/src/ra_log_wal.erl index 1471bb768..d1b83b841 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 bc9d07c5f..8c59d3568 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 b038de9b2bdb1e2eb95385557a3e3d5bdad8da5a Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 17 Mar 2025 18:09:42 +0000 Subject: [PATCH 11/60] 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 d1b83b841..8dc643118 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 7bb539653..cb7f0d88d 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 f7007fba6..ac243c0e1 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 8c59d3568..fff0c8a1e 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 000000000..f4437fbad --- /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 97f86c0870a47495705b18e1e32c8ef94a07e5d0 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 18 Mar 2025 17:09:28 +0000 Subject: [PATCH 12/60] 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 77f2b6ae8..e3a6211fe 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 05d2a1836..cdae7b4a7 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 -> @@ -262,7 +262,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. @@ -271,31 +271,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 @@ -309,14 +310,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 " @@ -325,8 +326,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", @@ -367,23 +367,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), @@ -391,7 +397,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. @@ -420,7 +426,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 @@ -433,8 +439,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), @@ -550,3 +562,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 8dc643118..0126e2bb7 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 cb7f0d88d..7e995b816 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 ac243c0e1..22357439d 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 ac2c3b608..e009074df 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 46c1a0cc3..5e7b32d9b 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 38551e3dd..a50456fd0 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 3a93317c5..42acf701c 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 fff0c8a1e..4b571f438 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 f4437fbad..d6722f997 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 678cb416a3b14055034717a06f9c184375e41d9c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 24 Mar 2025 12:14:18 +0000 Subject: [PATCH 13/60] 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 6a8af47c8..0704a4be2 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 1ad9f0a46b1dbc43fb27e12059a586000ba75b9a Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 8 Apr 2025 11:30:09 +0100 Subject: [PATCH 14/60] 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 22357439d..839dae004 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 4aff51ef9..d4616f66f 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 c88bd3866bc8364f503fce369d234b88d73aa425 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 17 Apr 2025 14:15:42 +0100 Subject: [PATCH 15/60] 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 e3a6211fe..1f6b3c33b 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 df25476e5..a8c58619e 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 839dae004..b180e747b 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 5e7b32d9b..994ef491e 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 d4616f66f..31e359a14 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 d6722f997..fe1ef8fdb 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 697558f32d4b52d3176e2deae8d88829869777f3 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 17 Apr 2025 17:15:55 +0100 Subject: [PATCH 16/60] 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 1f6b3c33b..46ed99432 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 a8c58619e..294fdc75a 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 b180e747b..715926e75 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 31e359a14..4a14f4acd 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 737f14f66d43fdfffeecb2118bbd2501b6eb5be4 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 23 Apr 2025 09:59:27 +0100 Subject: [PATCH 17/60] 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 294fdc75a..bb24a1883 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 715926e75..17bd47194 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 4a14f4acd..6d8df7af5 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 5fd398c0ac5c04c92baf4ca30098b929c00e306c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 28 Apr 2025 14:47:48 +0100 Subject: [PATCH 18/60] 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 46ed99432..844781f73 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 cdae7b4a7..49d0751e5 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -370,8 +370,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 -> @@ -380,7 +379,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 17bd47194..f23659be3 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 c27f5038f..f98e495a7 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 e6cc37972..fe9105aaf 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 = ra_file:rename(Checkpoint, Snapshot), + F = filename:join(SnapDir, <<"indexes">>), + Indexes = case file:read_file(F) of + {ok, Bin} -> + binary_to_term(Bin); + _ -> + [] + end, Self ! {ra_log_event, - {snapshot_written, - {Idx, Term}, 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 915ae1f19..375be3cb3 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 e009074df..53f3b4b11 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 994ef491e..e6dd59843 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 42acf701c..9cddcd874 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 fe1ef8fdb..1a1ca0f1e 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 8889a974f..a0bac05d0 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 1947919ef629cfbe85d5750794e8ff32533dd473 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 1 May 2025 09:17:31 +0100 Subject: [PATCH 19/60] 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 296731257..4acbbddf1 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 844781f73..89ec22df2 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 f23659be3..3a98236da 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 53f3b4b11..1e5a1acea 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 0db26df1b7fba80e55ea5f490cc2574e5eb1960c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 1 May 2025 15:26:21 +0100 Subject: [PATCH 20/60] 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 89ec22df2..ea8f1e2e9 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 1e5a1acea..9888b3735 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 e6dd59843..0cb5083ea 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 d71d71f3afa2d60823b8a83570fdf69fd29e1d82 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 9 May 2025 07:33:30 +0100 Subject: [PATCH 21/60] 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 386d92b6e..b9305a9a8 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 ea8f1e2e9..df0ca4102 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 d99a14a93..f87e7f36e 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 21acee4c2..0d6ab0703 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 0126e2bb7..bf25e2d14 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 3a98236da..187035903 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 515c1863b..5caeaa5f8 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 fe9105aaf..ead08fadd 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 9888b3735..78e88e35e 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 0cb5083ea..648d8fe08 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 0c6b34ad0..f86224cb1 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 4b571f438..26f630e42 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 759f2170b..ca52d3bdc 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 83d0fd6c8d914bd04811feefcc2d6e0be8d787d2 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 9 May 2025 13:39:12 +0100 Subject: [PATCH 22/60] 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 df0ca4102..3443fc71b 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 49d0751e5..0447801df 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -274,7 +274,7 @@ flush_mem_table_ranges({ServerUId, TidSeqs0}, TidSeqs = lists:foldl( fun ({T, Seq0}, []) -> case ra_seq:floor(SmallestIdx, Seq0) of - undefined -> + [] -> []; Seq -> [{T, Seq}] @@ -283,7 +283,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 bf25e2d14..bd8b8d260 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 bb24a1883..55d3d0116 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 5caeaa5f8..9860306de 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 f86224cb1..76301cc73 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 8ec8385d3..2d550fc20 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 a50456fd0..000000000 --- a/test/ra_log_props_SUITE.erl +++ /dev/null @@ -1,858 +0,0 @@ -%% This Source Code Form is subject to the terms of the Mozilla Public -%% License, v. 2.0. If a copy of the MPL was not distributed with this -%% file, You can obtain one at https://mozilla.org/MPL/2.0/. -%% -%% Copyright (c) 2017-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 26f630e42..31ee6be0f 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 ceb7e53d9..249fecf37 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 ca52d3bdc..044d30ab5 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()}, From 1f6cbdc86ea7ca3370614deea98689069f55639b Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 20 May 2025 10:37:20 +0100 Subject: [PATCH 23/60] wip ra_kv --- src/ra.erl | 11 --- src/ra.hrl | 4 +- src/ra_kv.erl | 138 +++++++++++++++++++++++++++++++++++++ src/ra_log.erl | 152 ++++++++++------------------------------- src/ra_server.erl | 43 ++++++++---- src/ra_server_proc.erl | 93 ++++++++++++++++++------- src/ra_snapshot.erl | 10 +-- test/ra_kv_SUITE.erl | 112 ++++++++++++++++++++++++++++++ 8 files changed, 390 insertions(+), 173 deletions(-) create mode 100644 src/ra_kv.erl create mode 100644 test/ra_kv_SUITE.erl diff --git a/src/ra.erl b/src/ra.erl index 70f47af46..7476cf6b7 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -35,7 +35,6 @@ consistent_query/3, ping/2, % cluster operations - start_cluster/1, start_cluster/2, start_cluster/3, start_cluster/4, @@ -350,16 +349,6 @@ start_cluster(System, ClusterName, Machine, ServerIds, Timeout) end || Id <- ServerIds], start_cluster(System, Configs, Timeout). -%% @doc Same as `start_cluster/2' but uses the default Ra system. -%% @param ServerConfigs a list of initial server configurations -%% DEPRECATED: use start_cluster/2 -%% @end --spec start_cluster([ra_server:ra_server_config()]) -> - {ok, [ra_server_id()], [ra_server_id()]} | - {error, cluster_not_formed}. -start_cluster(ServerConfigs) -> - start_cluster(default, ServerConfigs). - %% @doc Starts a new distributed ra cluster. %% @param System the system name %% @param ServerConfigs a list of initial server configurations diff --git a/src/ra.hrl b/src/ra.hrl index 843f5ec0c..a0a06f3bf 100644 --- a/src/ra.hrl +++ b/src/ra.hrl @@ -83,7 +83,7 @@ %% represent a unique entry in the ra log -type log_entry() :: {ra_index(), ra_term(), term()}. --type chunk_flag() :: next | last. +-type chunk_flag() :: pre | next | last. -type consistent_query_ref() :: {From :: term(), Query :: ra:query_fun(), ConmmitIndex :: ra_index()}. @@ -169,7 +169,7 @@ {term :: ra_term(), % the leader's term leader_id :: ra_server_id(), meta :: snapshot_meta(), - chunk_state :: {pos_integer(), chunk_flag()} | undefined, + chunk_state = {0, pre} :: {pos_integer(), chunk_flag()}, data :: term() }). diff --git a/src/ra_kv.erl b/src/ra_kv.erl new file mode 100644 index 000000000..7d31b724e --- /dev/null +++ b/src/ra_kv.erl @@ -0,0 +1,138 @@ +-module(ra_kv). +-behaviour(ra_machine). +-include("src/ra.hrl"). + +-include_lib("eunit/include/eunit.hrl"). + +-export([ + init/1, + apply/3, + % state_enter/2, + % tick/2, + init_aux/1, + handle_aux/5, + live_indexes/1, + overview/1 + ]). + +-export([ + start_cluster/3, + add_member/3, + + put/4 + ]). + + +-define(STATE, ?MODULE). + +-type key() :: binary(). +-type value() :: term(). + +-record(?STATE, {keys = #{} :: + #{key() := [ra:index() | Hash :: non_neg_integer()]}}). + + +-record(put, {key :: key(), + value :: term(), + meta :: #{size := non_neg_integer(), + hash := integer()}}). + +-type command() :: #put{}. +-opaque state() :: #?STATE{}. + +-export_type([state/0, + command/0]). + +%% mgmt +-spec start_cluster(atom(), atom(), map()) -> + {ok, [ra_server_id()], [ra_server_id()]} | + {error, cluster_not_formed}. +start_cluster(System, Name, #{members := ServerIds}) + when is_atom(Name) andalso + is_atom(System) -> + Machine = {module, ?MODULE, #{}}, + Configs = [begin + UId = ra:new_uid(ra_lib:to_binary(Name)), + #{id => Id, + uid => UId, + cluster_name => Name, + log_init_args => #{uid => UId}, + initial_members => ServerIds, + machine => Machine} + end || Id <- ServerIds], + ra:start_cluster(System, Configs). + +add_member(System, {Name, _} = Id, LeaderId) -> + {ok, Members, _} = ra:members(LeaderId), + UId = ra:new_uid(ra_lib:to_binary(Name)), + Machine = {module, ?MODULE, #{}}, + Config = #{id => Id, + uid => UId, + cluster_name => Name, + log_init_args => #{uid => UId}, + initial_members => Members, + machine => Machine}, + ok = ra:start_server(System, Config), + {ok, _, _} = ra:add_member(LeaderId, Id), + ok. + + +%% client +-spec put(ra:server_id(), key(), value(), non_neg_integer()) -> + {ok, map()} | {error, term()} | {timeout, ra:server_id()}. +put(ServerId, Key, Value, Timeout) -> + Hash = erlang:phash2(Value), + Put = #put{key = Key, + value = Value, + meta = #{size => erlang:external_size(Value), + hash => Hash}}, + case ra:process_command(ServerId, Put, Timeout) of + {ok, {ok, Meta}, LeaderId} -> + {ok, Meta#{leader => LeaderId}}; + Err -> + Err + end. + + +%% get performs a consistent query that returns the index, hash and member set +%% then perform an aux query to actually get the data for a given index. +%% if addressing a follower (say there is a local one) then the read may need +%% to wait if the index isn't yet available locally (term also need to be checked) +%% or check that the machien state has the right index for a given key before +%% reading the value from the log + + +%% state machine + +init(_) -> + #?MODULE{}. + +apply(#{index := Idx} = Meta, + #put{key = Key, + meta = #{hash := Hash}}, + #?STATE{keys = Keys} = State0) -> + State = State0#?STATE{keys = maps:put(Key, [Idx | Hash], Keys)}, + {State, {ok, Meta}, []}. + +live_indexes(#?STATE{keys = Keys}) -> + maps:fold(fun (_K, [Idx | _], Acc) -> + [Idx | Acc] + end, [], Keys). + +-record(aux, {}). +init_aux(_) -> + #aux{}. + +handle_aux(_RaState, {call, _From}, take_snapshot, Aux, Internal) -> + MacState = ra_aux:machine_state(Internal), + LastAppliedIdx = ra_aux:last_applied(Internal), + %% TODO: replace release cursor with simpler snapshot effect that is always + %% attempted? + {reply, ok, Aux, Internal, + [{release_cursor, LastAppliedIdx, MacState}]}; +handle_aux(_RaState, _, _, Aux, Internal) -> + {no_reply, Aux, Internal}. + +overview(#?STATE{keys = Keys} = State) -> + #{num_keys => maps:size(Keys), + live_indexes => live_indexes(State)}. diff --git a/src/ra_log.erl b/src/ra_log.erl index 3443fc71b..e3842f0c8 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/4, + install_snapshot/3, recover_snapshot/1, snapshot_index_term/1, update_release_cursor/5, @@ -60,6 +60,7 @@ -define(MIN_CHECKPOINT_INTERVAL, 16384). -define(LOG_APPEND_TIMEOUT, 5000). -define(WAL_RESEND_TIMEOUT, 5000). +-define(ETSTBL, ra_log_snapshot_state). -type ra_meta_key() :: atom(). -type segment_ref() :: {ra_range:range(), File :: file:filename_all()}. @@ -729,82 +730,6 @@ 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({segments, TidRanges, NewSegs}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, reader = Reader0, @@ -897,9 +822,6 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, Snap end, - CompEffs = schedule_compaction(SnapIdx, State0), - State = State0, - %% remove all pending below smallest live index as the wal %% may not write them @@ -920,14 +842,16 @@ 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 = 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}; + State = State0#?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}, + CompEffs = schedule_compaction(SnapIdx, State), + Effects = CompEffs ++ Effects0, + {State, Effects}; checkpoint -> put_counter(Cfg, ?C_RA_SVR_METRIC_CHECKPOINT_INDEX, SnapIdx), %% If we already have the maximum allowed number of checkpoints, @@ -1002,11 +926,12 @@ snapshot_state(State) -> set_snapshot_state(SnapState, State) -> State#?MODULE{snapshot_state = SnapState}. --spec install_snapshot(ra_idxterm(), ra_snapshot:state(), module(), state()) -> +-spec install_snapshot(ra_idxterm(), module(), state()) -> {ra_snapshot:meta(), MacState :: term(), state(), effects()}. -install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, MacMod, +install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, + snapshot_state = SnapState0, mem_table = Mt0} = State0) when is_atom(MacMod) -> ok = incr_counter(Cfg, ?C_RA_LOG_SNAPSHOTS_INSTALLED, 1), @@ -1021,24 +946,35 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, SnapState0, MacMod, 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 = 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 + SmallestLiveIndex = ra_seq:first(LiveIndexes), SnapDir = ra_snapshot:current_snapshot_dir(SnapState), ok = ra_snapshot:write_indexes(SnapDir, LiveIndexes), - {Meta, MacState, State#?MODULE{live_indexes = LiveIndexes}, + %% TODO: more mt entries could potentially be cleared up here + {Spec, Mt} = ra_mt:set_first(SmallestLiveIndex, Mt0), + ok = exec_mem_table_delete(Names, UId, Spec), + %% TODO: move this to install_snapshot so we can work out the + %% live indexes + SmallestIdx = case LiveIndexes of + [] -> + SnapIdx + 1; + _ -> + ra_seq:first(LiveIndexes) + end, + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, SnapIdx, SmallestIdx, + LiveIndexes), + {Meta, MacState, State#?MODULE{live_indexes = LiveIndexes, + mem_table = Mt + }, CompEffs ++ CPEffects}. @@ -1357,29 +1293,11 @@ schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{}, ok end, - [{bg_work, Fun, fun (_Err) -> ok end}] + [{bg_work, Fun, fun (_Err) -> + ?WARN("bgwork err ~p", [_Err]), ok + end}] end. -%% deletes all segments where the last index is lower than -%% the Idx argument -% delete_segments(SnapIdx, #?MODULE{cfg = #cfg{log_id = LogId, -% segment_writer = SegWriter, -% uid = UId} = Cfg, -% 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/src/ra_server.erl b/src/ra_server.erl index 9860306de..a412b0376 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -1427,7 +1427,7 @@ handle_follower(#install_snapshot_rpc{term = Term, meta = #{index := SnapIdx, machine_version := SnapMacVer} = Meta, leader_id = LeaderId, - chunk_state = {1, _ChunkFlag}} = Rpc, + chunk_state = {Num, _ChunkFlag}} = Rpc, #{cfg := #cfg{log_id = LogId, machine_version = MacVer}, log := Log0, last_applied := LastApplied, @@ -1435,7 +1435,8 @@ handle_follower(#install_snapshot_rpc{term = Term, when Term >= CurTerm andalso SnapIdx > LastApplied andalso %% only install snapshot if the machine version is understood - MacVer >= SnapMacVer -> + MacVer >= SnapMacVer andalso + Num =< 1 -> %% only begin snapshot procedure if Idx is higher than the last_applied %% index. ?DEBUG("~ts: begin_accept snapshot at index ~b in term ~b", @@ -1523,7 +1524,7 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, effective_machine_version = CurEffMacVer, machine_versions = MachineVersions, machine = Machine} = Cfg0, - log := Log0, + log := Log00, cluster := Cluster, current_term := CurTerm, last_applied := LastApplied, @@ -1531,14 +1532,37 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, when Term >= CurTerm -> ?DEBUG("~ts: receiving snapshot chunk: ~b / ~w, index ~b, term ~b", [LogId, Num, ChunkFlag, SnapIndex, SnapTerm]), - SnapState0 = ra_log:snapshot_state(Log0), - {ok, SnapState, Effs0} = ra_snapshot:accept_chunk(Data, Num, ChunkFlag, - SnapState0), Reply = #install_snapshot_result{term = CurTerm, last_term = SnapTerm, last_index = SnapIndex}, case ChunkFlag of + pre -> + %% TODO: we may need to reset the log here to + %% the last applied index as we + %% dont know for sure indexes after last applied + %% are of the right term + {LastIndex, _} = ra_log:last_index_term(Log00), + {Log0, _} = lists:foldl( + fun ({I, _, _} = E, {L0, LstIdx}) -> + {ok, L} = ra_log:write_sparse(E, LstIdx, L0), + {L, I} + end, {Log00, LastIndex}, Data), + State = update_term(Term, State0#{log => Log0}), + {receive_snapshot, State, [{reply, Reply}]}; + next -> + SnapState0 = ra_log:snapshot_state(Log00), + {ok, SnapState, Effs0} = + ra_snapshot:accept_chunk(Data, Num, ChunkFlag, + SnapState0), + Log0 = ra_log:set_snapshot_state(SnapState, Log00), + State = update_term(Term, State0#{log => Log0}), + {receive_snapshot, State, [{reply, Reply} | Effs0]}; last -> + SnapState0 = ra_log:snapshot_state(Log00), + {ok, SnapState, Effs0} = + ra_snapshot:accept_chunk(Data, Num, ChunkFlag, + SnapState0), + Log0 = ra_log:set_snapshot_state(SnapState, Log00), %% if the machine version of the snapshot is higher %% we also need to update the current effective machine configuration EffMacMod = ra_machine:which_module(Machine, SnapMacVer), @@ -1557,7 +1581,6 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, %% 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) @@ -1588,11 +1611,7 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, %% follower status {follower, persist_last_applied(State), [{reply, Reply} | Effs0 ++ Effs ++ - SnapInstalledEffs]}; - next -> - Log = ra_log:set_snapshot_state(SnapState, Log0), - State = update_term(Term, State0#{log => Log}), - {receive_snapshot, State, [{reply, Reply} | Effs0]} + SnapInstalledEffs]} end; handle_receive_snapshot(#append_entries_rpc{term = Term} = Msg, #{current_term := CurTerm, diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index f98e495a7..f0aadbcdb 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -50,7 +50,7 @@ local_state_query/3, trigger_election/2, ping/2, - log_fold/4, + read_entries/4, transfer_leadership/3, force_shrink_members_to_current_member/1 ]). @@ -207,9 +207,24 @@ query(ServerLoc, QueryFun, leader, Options, Timeout) -> query(ServerLoc, QueryFun, consistent, _Options, Timeout) -> leader_call(ServerLoc, {consistent_query, QueryFun}, Timeout). --spec log_fold(ra_server_id(), fun(), term(), integer()) -> term(). -log_fold(ServerId, Fun, InitialState, Timeout) -> - gen_statem:call(ServerId, {log_fold, Fun, InitialState}, Timeout). +-spec read_entries(ra_server_id(), [ra:index()], + undefined | ra_flru:state(), + non_neg_integer()) -> + {ok, {map(), ra_flru:state()}} | {error, term()}. +read_entries(ServerId, Indexes, Flru0, Timeout) -> + case local_call(ServerId, {read_entries, Indexes}, Timeout) of + {ok, ReadPlan} -> + {Reads, Flru} = ra_log:execute_read_plan(ReadPlan, Flru0, + fun (Idx, Term, Cmd) -> + {Idx, Term, Cmd} + end, + #{access_pattern => random, + file_advise => random}), + {ok, {Reads, Flru}}; + Err -> + Err + end. + %% used to query the raft state rather than the machine state -spec state_query(server_loc(), @@ -518,7 +533,7 @@ leader(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; leader(EventType, flush_commands, #state{conf = #conf{flush_commands_size = Size}, low_priority_commands = Delayed0} = State0) -> @@ -586,8 +601,8 @@ leader({timeout, Name}, machine_timeout, State0) -> {keep_state, State, Actions}; leader({call, From}, trigger_election, State) -> {keep_state, State, [{reply, From, ok}]}; -leader({call, From}, {log_fold, Fun, Term}, State) -> - fold_log(From, Fun, Term, State); +leader({call, From}, {read_entries, Indexes}, State) -> + read_entries0(From, Indexes, State); leader(EventType, Msg, State0) -> case handle_leader(Msg, State0) of {leader, State1, Effects1} -> @@ -859,8 +874,8 @@ follower(_, tick_timeout, #state{server_state = ServerState0} = State0) -> State0#state{server_state = ServerState}), {keep_state, handle_tick_metrics(State), set_tick_timer(State, Actions)}; -follower({call, From}, {log_fold, Fun, Term}, State) -> - fold_log(From, Fun, Term, State); +follower({call, From}, {read_entries, Indexes}, State) -> + read_entries0(From, Indexes, State); follower(EventType, Msg, #state{conf = #conf{name = Name}, server_state = SS0} = State0) -> case handle_follower(Msg, State0) of @@ -1516,7 +1531,7 @@ handle_effect(_RaftState, {reply, Reply}, {call, From}, State, Actions) -> {State, Actions}; handle_effect(_RaftState, {reply, _From, _Reply}, _EvtType, State, Actions) -> {State, Actions}; -handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, Id, Term}}, _, +handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, _Id, Term}}, _, #state{server_state = SS0, monitors = Monitors, conf = #conf{snapshot_chunk_size = ChunkSize, @@ -1526,10 +1541,10 @@ handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, Id, Term}}, true -> %% node is connected %% leader effect only - Self = self(), Machine = ra_server:machine(SS0), + Id = ra_server:id(SS0), Pid = spawn(fun () -> - try send_snapshots(Self, Id, Term, To, + try send_snapshots(Id, Term, To, ChunkSize, InstallSnapTimeout, SnapState, Machine) of _ -> ok @@ -1884,14 +1899,13 @@ send(To, Msg, Conf) -> Res end. -fold_log(From, Fun, Term, State) -> - case ra_server:log_fold(State#state.server_state, Fun, Term) of - {ok, Result, ServerState} -> - {keep_state, State#state{server_state = ServerState}, - [{reply, From, {ok, Result}}]} - end. +read_entries0(From, Idxs, #state{server_state = #{log := Log}} = State) -> + ReadState = ra_log:partial_read(Idxs, Log, fun (Idx, Term, Cmd) -> + {Idx, Term, Cmd} + end), + {keep_state, State, [{reply, From, {ok, ReadState}}]}. -send_snapshots(Me, Id, Term, {_, ToNode} = To, ChunkSize, +send_snapshots(Id, Term, {_, ToNode} = To, ChunkSize, InstallTimeout, SnapState, Machine) -> Context = ra_snapshot:context(SnapState, ToNode), {ok, #{machine_version := SnapMacVer} = Meta, ReadState} = @@ -1900,6 +1914,7 @@ send_snapshots(Me, Id, Term, {_, ToNode} = To, ChunkSize, %% only send the snapshot if the target server can accept it TheirMacVer = erpc:call(ToNode, ra_machine, version, [Machine]), + %% rpc the check what their case SnapMacVer > TheirMacVer of true -> ok; @@ -1907,13 +1922,45 @@ send_snapshots(Me, Id, Term, {_, ToNode} = To, ChunkSize, RPC = #install_snapshot_rpc{term = Term, leader_id = Id, meta = Meta}, + case ra_snapshot:indexes(ra_snapshot:current_snapshot_dir(SnapState)) of + {ok, [_|_] = Indexes} -> + %% there are live indexes to send before the snapshot + Idxs = lists:reverse(ra_seq:expand(Indexes)), + Flru = lists:foldl( + fun (Is, F0) -> + {ok, {Ents0, F}} = + ra_server_proc:read_entries(Id, Is, F0, 5000), + Ents = [map_get(I, Ents0) || I <- Is], + RPC1 = RPC#install_snapshot_rpc{chunk_state = {0, pre}, + data = Ents}, + _Res1 = gen_statem:call(To, RPC1, + {dirty_timeout, InstallTimeout}), + %% TODO: assert REs1 is successful + F + end, undefined, ra_lib:lists_chunk(16, Idxs)), + _ = ra_flru:evict_all(Flru), + ok; + _ -> + ok + end, + %% send install sntaphost RPC with entries here + %% Read indexes for snapshot, if non-empty + %% Find out what the follower's last applied index is + %% + %% Call into `Id' to do sparse read of some chunk of indexes + %% begin rpc with gen_statem:send_request/2 + %% while waiting for reply call into `Id' to get the next chunk + %% of entries + %% wait for response + %% send again, etc Result = read_chunks_and_send_rpc(RPC, To, ReadState, 1, - ChunkSize, InstallTimeout, SnapState), - ok = gen_statem:cast(Me, {To, Result}) + ChunkSize, InstallTimeout, + SnapState), + ok = gen_statem:cast(Id, {To, Result}) end. -read_chunks_and_send_rpc(RPC0, - To, ReadState0, Num, ChunkSize, InstallTimeout, SnapState) -> +read_chunks_and_send_rpc(RPC0, To, ReadState0, Num, ChunkSize, + InstallTimeout, SnapState) -> {ok, Data, ContState} = ra_snapshot:read_chunk(ReadState0, ChunkSize, SnapState), ChunkFlag = case ContState of diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index ead08fadd..e9d271254 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -515,13 +515,13 @@ begin_accept(#{index := Idx, term := Term} = Meta, -spec accept_chunk(term(), non_neg_integer(), chunk_flag(), state()) -> {ok, state(), [effect()]}. accept_chunk(Chunk, Num, last, - #?MODULE{uid = UId, + #?MODULE{uid = _UId, module = Mod, snapshot_directory = Dir, current = Current, pending = Pending, accepting = #accept{next = Num, - idxterm = {Idx, _} = IdxTerm, + idxterm = {_Idx, _} = IdxTerm, state = AccState}} = State) -> %% last chunk ok = Mod:complete_accept(Chunk, AccState), @@ -537,12 +537,6 @@ accept_chunk(Chunk, Num, last, fun() -> [delete(Dir, Del) || Del <- Dels] end, fun (_) -> ok end}, - %% update ets table - % 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_kv_SUITE.erl b/test/ra_kv_SUITE.erl new file mode 100644 index 000000000..f0f14e305 --- /dev/null +++ b/test/ra_kv_SUITE.erl @@ -0,0 +1,112 @@ +-module(ra_kv_SUITE). + +-compile(nowarn_export_all). +-compile(export_all). + +-export([ + ]). + +-include_lib("src/ra.hrl"). +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). + +-define(SYS, default). +%%%=================================================================== +%%% Common Test callbacks +%%%=================================================================== + +all() -> + [ + {group, tests} + ]. + + +all_tests() -> + [ + basics + ]. + +groups() -> + [ + {tests, [], all_tests()} + ]. + +init_per_suite(Config) -> + %% as we're not starting the ra application and we want the logs + ra_env:configure_logger(logger), + {ok, _} = ra:start_in(?config(priv_dir, Config)), + Config. + +end_per_suite(_Config) -> + application:stop(ra), + ok. + +init_per_group(_Group, Config) -> + Config. + +end_per_group(_Group, _Config) -> + ok. + +init_per_testcase(TestCase, Config) -> + DataDir = filename:join(?config(priv_dir, Config), TestCase), + [{data_dir, DataDir}, {cluster_name, TestCase} | Config]. + +end_per_testcase(_TestCase, _Config) -> + ok. + +%%%=================================================================== +%%% Test cases +%%%=================================================================== + + +basics(_Config) -> + Members = [{kv1, node()}], + KvId = hd(Members), + {ok, Members, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, + #{members => Members}), + {ok, #{}} = ra_kv:put(KvId, <<"k1">>, <<"k1-value01">>, 5000), + K2 = <<"k2">>, + %% write 10k entries of the same key + [{ok, #{}} = ra_kv:put(KvId, K2, I, 5000) + || I <- lists:seq(1, 10000)], + + ?assertMatch({ok, #{machine := #{num_keys := 2}}, KvId}, + ra:member_overview(KvId)), + ra_log_wal:force_roll_over(ra_log_wal), + timer:sleep(1000), + ok = ra:aux_command(KvId, take_snapshot), + timer:sleep(1000), + ok = ra:stop_server(default, KvId), + ok = ra:restart_server(default, KvId), + {ok, #{index := LastIdx}} = ra_kv:put(KvId, <<"k3">>, <<"k3">>, 5000), + ct:pal("overview after ~p", [ra:member_overview(KvId)]), + {ok, #{machine := #{live_indexes := Live}}, _} = ra:member_overview(KvId), + {ok, {Reads, _}} = ra_server_proc:read_entries(KvId, [LastIdx | Live], + undefined, 1000), + ct:pal("ReadRes ~p", [Reads]), + + % debugger:start(), + % int:i(ra_log), + % int:i(ra_snapshot), + % int:i(ra_server_proc), + % int:break(ra_server_proc, 1922), + % int:break(ra_log, 873), + % int:break(ra_log, 1002), + % int:break(ra_log, 1328), + KvId2 = {kv2, node()}, + ok = ra_kv:add_member(?SYS, KvId2, KvId), + timer:sleep(1000), + {ok, {Reads2, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], + undefined, 1000), + ct:pal("ReadRes2 ~p", [Reads2]), + ct:pal("overview ~p", [ra:member_overview(KvId2)]), + ?assertEqual(3, map_size(Reads2)), + ra_log_wal:force_roll_over(ra_log_wal), + timer:sleep(1000), + {ok, {Reads3, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], + undefined, 1000), + ct:pal("ReadRes3 ~p", [Reads3]), + ct:pal("overview3 ~p", [ra:member_overview(KvId2)]), + ?assertEqual(3, map_size(Reads3)), + + ok. From f0fe087f011c16c86da5762217fc15d1d297ba71 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 21 May 2025 16:02:38 +0100 Subject: [PATCH 24/60] snapshot api changes --- src/ra.hrl | 2 +- src/ra_log.erl | 41 +++++++------------ src/ra_server.erl | 25 ++++++------ src/ra_server_proc.erl | 2 + src/ra_snapshot.erl | 80 +++++++++++++++++++++++++------------- test/ra_kv_SUITE.erl | 54 ++++++++++++++++--------- test/ra_log_2_SUITE.erl | 54 ++++++++++++++++++------- test/ra_log_memory.erl | 16 ++++---- test/ra_server_SUITE.erl | 52 ++++++++++++++++--------- test/ra_snapshot_SUITE.erl | 37 +++++++++++------- 10 files changed, 221 insertions(+), 142 deletions(-) diff --git a/src/ra.hrl b/src/ra.hrl index a0a06f3bf..80379616f 100644 --- a/src/ra.hrl +++ b/src/ra.hrl @@ -169,7 +169,7 @@ {term :: ra_term(), % the leader's term leader_id :: ra_server_id(), meta :: snapshot_meta(), - chunk_state = {0, pre} :: {pos_integer(), chunk_flag()}, + chunk_state = {0, pre} :: {non_neg_integer(), chunk_flag()}, data :: term() }). diff --git a/src/ra_log.erl b/src/ra_log.erl index e3842f0c8..c0baa0abf 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, @@ -60,7 +60,6 @@ -define(MIN_CHECKPOINT_INTERVAL, 16384). -define(LOG_APPEND_TIMEOUT, 5000). -define(WAL_RESEND_TIMEOUT, 5000). --define(ETSTBL, ra_log_snapshot_state). -type ra_meta_key() :: atom(). -type segment_ref() :: {ra_range:range(), File :: file:filename_all()}. @@ -926,9 +925,9 @@ snapshot_state(State) -> set_snapshot_state(SnapState, State) -> State#?MODULE{snapshot_state = SnapState}. --spec install_snapshot(ra_idxterm(), module(), state()) -> - {ra_snapshot:meta(), MacState :: term(), state(), effects()}. -install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, +-spec install_snapshot(ra_idxterm(), module(), ra_seq:state(), state()) -> + {ok, state(), effects()}. +install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, LiveIndexes, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, snapshot_state = SnapState0, @@ -951,30 +950,18 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, last_index = SnapIdx, last_term = SnapTerm, 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 - SmallestLiveIndex = ra_seq:first(LiveIndexes), - SnapDir = ra_snapshot:current_snapshot_dir(SnapState), - ok = ra_snapshot:write_indexes(SnapDir, LiveIndexes), - %% TODO: more mt entries could potentially be cleared up here + %% TODO: more mt entries could potentially be cleared up in the + %% mem table here + SmallestLiveIndex = case ra_seq:first(LiveIndexes) of + undefined -> + SnapIdx + 1; + I -> + I + end, {Spec, Mt} = ra_mt:set_first(SmallestLiveIndex, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), - %% TODO: move this to install_snapshot so we can work out the - %% live indexes - SmallestIdx = case LiveIndexes of - [] -> - SnapIdx + 1; - _ -> - ra_seq:first(LiveIndexes) - end, - ok = ra_log_snapshot_state:insert(?ETSTBL, UId, SnapIdx, SmallestIdx, - LiveIndexes), - {Meta, MacState, State#?MODULE{live_indexes = LiveIndexes, - mem_table = Mt - }, + {ok, State#?MODULE{live_indexes = LiveIndexes, + mem_table = Mt}, CompEffs ++ CPEffects}. diff --git a/src/ra_server.erl b/src/ra_server.erl index a412b0376..dfc1f0ed6 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -1516,9 +1516,10 @@ handle_follower(Msg, State) -> handle_receive_snapshot(#install_snapshot_rpc{term = Term, meta = #{index := SnapIndex, machine_version := SnapMacVer, + cluster := ClusterIds, term := SnapTerm} = SnapMeta, chunk_state = {Num, ChunkFlag}, - data = Data}, + data = ChunkOrEntries}, #{cfg := #cfg{id = Id, log_id = LogId, effective_machine_version = CurEffMacVer, @@ -1536,7 +1537,7 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, last_term = SnapTerm, last_index = SnapIndex}, case ChunkFlag of - pre -> + pre when is_list(ChunkOrEntries) -> %% TODO: we may need to reset the log here to %% the last applied index as we %% dont know for sure indexes after last applied @@ -1546,22 +1547,20 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, fun ({I, _, _} = E, {L0, LstIdx}) -> {ok, L} = ra_log:write_sparse(E, LstIdx, L0), {L, I} - end, {Log00, LastIndex}, Data), + end, {Log00, LastIndex}, ChunkOrEntries), State = update_term(Term, State0#{log => Log0}), {receive_snapshot, State, [{reply, Reply}]}; next -> SnapState0 = ra_log:snapshot_state(Log00), - {ok, SnapState, Effs0} = - ra_snapshot:accept_chunk(Data, Num, ChunkFlag, - SnapState0), + SnapState = ra_snapshot:accept_chunk(ChunkOrEntries, Num, SnapState0), Log0 = ra_log:set_snapshot_state(SnapState, Log00), State = update_term(Term, State0#{log => Log0}), - {receive_snapshot, State, [{reply, Reply} | Effs0]}; + {receive_snapshot, State, [{reply, Reply}]}; last -> SnapState0 = ra_log:snapshot_state(Log00), - {ok, SnapState, Effs0} = - ra_snapshot:accept_chunk(Data, Num, ChunkFlag, - SnapState0), + {SnapState, MacState, LiveIndexes, Effs0} = + ra_snapshot:complete_accept(ChunkOrEntries, Num, Machine, + SnapState0), Log0 = ra_log:set_snapshot_state(SnapState, Log00), %% if the machine version of the snapshot is higher %% we also need to update the current effective machine configuration @@ -1579,9 +1578,9 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, Cfg0 end, %% this is the last chunk so we can "install" it - {#{cluster := ClusterIds}, - MacState, Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm}, - EffMacMod, Log0), + {ok, Log, Effs} = ra_log:install_snapshot({SnapIndex, SnapTerm}, + EffMacMod, + LiveIndexes, Log0), OldServerIds = maps:map(fun (_, V) -> maps:with([voter_status], V) end, Cluster), diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index f0aadbcdb..e3b5a3235 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -1912,6 +1912,8 @@ send_snapshots(Id, Term, {_, ToNode} = To, ChunkSize, ra_snapshot:begin_read(SnapState, Context), %% only send the snapshot if the target server can accept it + %% TODO: grab the last_applied index also and use this to floor + %% the live indexes TheirMacVer = erpc:call(ToNode, ra_machine, version, [Machine]), %% rpc the check what their diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index e9d271254..c50274c79 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -8,6 +8,7 @@ -include("ra.hrl"). +-include_lib("stdlib/include/assert.hrl"). -type file_err() :: ra_lib:file_err(). %% alias @@ -33,7 +34,8 @@ complete_snapshot/4, begin_accept/2, - accept_chunk/4, + accept_chunk/3, + complete_accept/4, abort_accept/1, context/2, @@ -70,6 +72,7 @@ -record(accept, {%% the next expected chunk next = 1 :: non_neg_integer(), state :: term(), + machine_version :: non_neg_integer(), idxterm :: ra_idxterm()}). -record(?MODULE, @@ -503,26 +506,30 @@ complete_snapshot(IdxTerm, checkpoint, _LiveIndexes, -spec begin_accept(meta(), state()) -> {ok, state()}. -begin_accept(#{index := Idx, term := Term} = Meta, +begin_accept(#{index := Idx, + machine_version := SnapMacVer, + term := Term} = Meta, #?MODULE{module = Mod, snapshot_directory = Dir} = State) -> SnapDir = make_snapshot_dir(Dir, Idx, Term), ok = ra_lib:make_dir(SnapDir), {ok, AcceptState} = Mod:begin_accept(SnapDir, Meta), {ok, State#?MODULE{accepting = #accept{idxterm = {Idx, Term}, + machine_version = SnapMacVer, state = AcceptState}}}. --spec accept_chunk(term(), non_neg_integer(), chunk_flag(), state()) -> - {ok, state(), [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) -> +-spec complete_accept(Chunk :: term(), Num :: non_neg_integer(), + Machine :: ra_machine:machine(), state()) -> + {state(), MacState :: term(), ra_seq:state(), [effect()]}. +complete_accept(Chunk, Num, Machine, + #?MODULE{uid = UId, + module = Mod, + snapshot_directory = Dir, + current = Current, + pending = Pending, + accepting = #accept{next = Num, + idxterm = {Idx, Term} = IdxTerm, + state = AccState}} = State0) -> %% last chunk ok = Mod:complete_accept(Chunk, AccState), %% run validate here? @@ -536,25 +543,44 @@ accept_chunk(Chunk, Num, last, Eff = {bg_work, fun() -> [delete(Dir, Del) || Del <- Dels] end, fun (_) -> ok end}, + State = State0#?MODULE{accepting = undefined, + %% reset any pending snapshot writes + pending = undefined, + current = IdxTerm}, + {ok, #{machine_version := SnapMacVer}, MacState} = recover(State), + SnapMacMod = ra_machine:which_module(Machine, SnapMacVer), + LiveIndexes = ra_machine:live_indexes(SnapMacMod, MacState), + SnapDir = make_snapshot_dir(Dir, Idx, Term), + ok = write_indexes(SnapDir, LiveIndexes), + %% delete accepting marker file + AcceptMarker = filename:join(SnapDir, <<"accepting">>), + _ = prim_file:delete(AcceptMarker), + %% assert accepting marker is no longer there + ?assertNot(filelib:is_file(AcceptMarker)), + SmallestIdx = case ra_seq:first(LiveIndexes) of + undefined -> + Idx + 1; + I -> + I + end, + ok = ra_log_snapshot_state:insert(?ETSTBL, UId, Idx, SmallestIdx, + LiveIndexes), + {State, MacState, LiveIndexes, [Eff]}. - {ok, State#?MODULE{accepting = undefined, - %% reset any pending snapshot writes - pending = undefined, - current = IdxTerm}, [Eff]}; -accept_chunk(Chunk, Num, next, - #?MODULE{module = Mod, - accepting = - #accept{state = AccState0, - next = Num} = Accept} = State) -> +-spec accept_chunk(Chunk :: term(), Num :: non_neg_integer(), state()) -> + state(). +accept_chunk(Chunk, Num, #?MODULE{module = Mod, + accepting = + #accept{state = AccState0, + next = Num} = Accept} = State) -> {ok, AccState} = Mod:accept_chunk(Chunk, AccState0), - {ok, State#?MODULE{accepting = Accept#accept{state = AccState, - next = Num + 1}}, - []}; -accept_chunk(_Chunk, Num, _ChunkFlag, + State#?MODULE{accepting = Accept#accept{state = AccState, + next = Num + 1}}; +accept_chunk(_Chunk, Num, #?MODULE{accepting = #accept{next = Next}} = State) when Next > Num -> %% this must be a resend - we can just ignore it - {ok, State, []}. + State. -spec abort_accept(state()) -> state(). abort_accept(#?MODULE{accepting = undefined} = State) -> diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index f0f14e305..b1cc0ec02 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -73,40 +73,56 @@ basics(_Config) -> ?assertMatch({ok, #{machine := #{num_keys := 2}}, KvId}, ra:member_overview(KvId)), ra_log_wal:force_roll_over(ra_log_wal), - timer:sleep(1000), + %% wait for rollover processing + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + %% wait for segment writer to process + ra_log_segment_writer:await(ra_log_segment_writer), + %% promt ra_kv to take a snapshot ok = ra:aux_command(KvId, take_snapshot), - timer:sleep(1000), + %% wait for snapshot to complete + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{snapshot_index := SnapIdx, + num_segments := NumSegments, + last_index := LastIdx}}, _} = + ra:member_overview(KvId), + SnapIdx == LastIdx andalso NumSegments == 2 + end, 100, 100), + %% restart server to test recovery ok = ra:stop_server(default, KvId), ok = ra:restart_server(default, KvId), {ok, #{index := LastIdx}} = ra_kv:put(KvId, <<"k3">>, <<"k3">>, 5000), - ct:pal("overview after ~p", [ra:member_overview(KvId)]), - {ok, #{machine := #{live_indexes := Live}}, _} = ra:member_overview(KvId), + {ok, #{machine := #{live_indexes := Live}, + log := #{last_index := KvIdLastIdx}}, _} = ra:member_overview(KvId), {ok, {Reads, _}} = ra_server_proc:read_entries(KvId, [LastIdx | Live], undefined, 1000), - ct:pal("ReadRes ~p", [Reads]), - - % debugger:start(), - % int:i(ra_log), - % int:i(ra_snapshot), - % int:i(ra_server_proc), - % int:break(ra_server_proc, 1922), - % int:break(ra_log, 873), - % int:break(ra_log, 1002), - % int:break(ra_log, 1328), + ?assertEqual(3, map_size(Reads)), + % ct:pal("ReadRes ~p", [Reads]), KvId2 = {kv2, node()}, ok = ra_kv:add_member(?SYS, KvId2, KvId), - timer:sleep(1000), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{last_index := Last}}, _} = + ra:member_overview(KvId2), + Last >= KvIdLastIdx + end, 100, 100), {ok, {Reads2, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], undefined, 1000), - ct:pal("ReadRes2 ~p", [Reads2]), - ct:pal("overview ~p", [ra:member_overview(KvId2)]), ?assertEqual(3, map_size(Reads2)), ra_log_wal:force_roll_over(ra_log_wal), - timer:sleep(1000), + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + ra_log_segment_writer:await(ra_log_segment_writer), {ok, {Reads3, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], undefined, 1000), ct:pal("ReadRes3 ~p", [Reads3]), - ct:pal("overview3 ~p", [ra:member_overview(KvId2)]), + % ct:pal("overview3 ~p", [ra:member_overview(KvId2)]), ?assertEqual(3, map_size(Reads3)), + %% TODO: test recovery of kv + ok = ra:stop_server(default, KvId2), + ok = ra:restart_server(default, KvId2), + {ok, {Reads4, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], + undefined, 1000), + + ?assertEqual(3, map_size(Reads4)), ok. diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 78e88e35e..496671abf 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -1117,9 +1117,12 @@ snapshot_written_after_installation(Config) -> Chunk = create_snapshot_chunk(Config, Meta, Context), SnapState0 = ra_log:snapshot_state(Log2), {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), - {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), run_effs(AEffs), - {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2), + {ok, Log3, _} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log2)), %% write some more to create another segment Log4 = write_and_roll(16, 20, 2, Log3), {Log5, Efx4} = ra_log:handle_event(DelayedSnapWritten, Log4), @@ -1164,10 +1167,14 @@ oldcheckpoints_deleted_after_snapshot_install(Config) -> Chunk = create_snapshot_chunk(Config, Meta, Context), SnapState0 = ra_log:snapshot_state(Log3), {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), - {ok, SnapState, AcceptEffs} = - ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), - run_effs(AcceptEffs), - {_, _, Log4, Effs4} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log3), + % {ok, SnapState, AcceptEffs} = + % ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log4, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log3)), ?assert(lists:any(fun (E) -> element(1, E) == delete_snapshot end, Effs4)), %% write some more to create another segment Log5 = write_and_roll(16, 20, 2, Log4), @@ -1210,10 +1217,14 @@ snapshot_installation(Config) -> Chunk = create_snapshot_chunk(Config, Meta, #{}), SnapState0 = ra_log:snapshot_state(Log2), {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), - {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), run_effs(AEffs), - {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2), + {ok, Log3, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log2)), + run_effs(Effs4), {15, _} = ra_log:last_index_term(Log3), {15, _} = ra_log:last_written(Log3), #{mem_table_range := undefined} = ra_log:overview(Log3), @@ -1260,9 +1271,13 @@ append_after_snapshot_installation(Config) -> Chunk = create_snapshot_chunk(Config, Meta, #{}), SnapState0 = ra_log:snapshot_state(Log1), {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), - {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), run_effs(AEffs), - {_, _, Log2, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log1), + {ok, Log2, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log1)), + run_effs(Effs4), {15, _} = ra_log:last_index_term(Log2), {15, _} = ra_log:last_written(Log2), @@ -1292,9 +1307,13 @@ written_event_after_snapshot_installation(Config) -> Chunk = create_snapshot_chunk(Config, Meta, #{}), SnapState0 = ra_log:snapshot_state(Log1), {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), - {ok, SnapState, AEffs} = ra_snapshot:accept_chunk(Chunk, 1, last, SnapState1), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), run_effs(AEffs), - {_, _, Log2, _} = ra_log:install_snapshot({SnapIdx, 2}, SnapState, ?MODULE, Log1), + {ok, Log2, Effs4} = ra_log:install_snapshot({SnapIdx, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log1)), + run_effs(Effs4), {SnapIdx, _} = ra_log:last_index_term(Log2), {SnapIdx, _} = ra_log:last_written(Log2), NextIdx = SnapIdx + 1, @@ -1566,10 +1585,14 @@ sparse_write(Config) -> 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), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, + Machine, + SnapState1), run_effs(AEffs), - {_, _, Log3, _} = ra_log:install_snapshot({15, 2}, SnapState, ?MODULE, Log2), - {ok, Log} = ra_log:write([{16, 1, <<>>}], Log3), + Log3 = ra_log:set_snapshot_state(SnapState, Log2), + {ok, Log4, _} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, Log3), + {ok, Log} = ra_log:write([{16, 1, <<>>}], Log4), {ResFinal, _} = ra_log:sparse_read(LiveIndexes, Log), ?assertMatch([{3, _, _}, {5, _, _}, @@ -1897,6 +1920,7 @@ run_effs(Effs) -> %% ra_machine fakes version() -> 1. +which_module(_) -> ?MODULE. live_indexes(MacState) when is_list(MacState) -> %% fake returning live indexes MacState; diff --git a/test/ra_log_memory.erl b/test/ra_log_memory.erl index 76301cc73..62df1a730 100644 --- a/test/ra_log_memory.erl +++ b/test/ra_log_memory.erl @@ -213,19 +213,17 @@ 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) -> +install_snapshot({Index, Term}, _MacMod, _LiveIndexes, + #state{entries = Log0} = State0) -> % discard log entries below snapshot index Log = maps:filter(fun (K, _) -> K > Index end, Log0), State = State0#state{entries = Log, last_index = Index, - last_written = {Index, Term}, - snapshot = Data}, - {Meta, MacState} = Data, - {Meta, MacState, State, []}. + last_written = {Index, Term} + % snapshot = Data + }, + % {Meta, MacState} = Data, + {ok, State, []}. -spec read_snapshot(State :: ra_log_memory_state()) -> {ok, ra_snapshot:meta(), term()}. diff --git a/test/ra_server_SUITE.erl b/test/ra_server_SUITE.erl index 044d30ab5..bf79cdade 100644 --- a/test/ra_server_SUITE.erl +++ b/test/ra_server_SUITE.erl @@ -162,9 +162,13 @@ setup_log() -> {ok, {Meta, undefined}} end), meck:expect(ra_snapshot, accept_chunk, - fun(Data, _OutOf, _Flag, {Meta, _}) -> + fun(Data, _OutOf, {Meta, _}) -> {ok, {Meta, Data}, []} end), + meck:expect(ra_snapshot, complete_accept, + fun(_Data, _Num, _Machine, {_Meta, MacSt} = State) -> + {State, MacSt, [], []} + 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), @@ -228,9 +232,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}, - ?MODULE, Log0), + {ok, Log1, _} = ra_log_memory:install_snapshot({3, 5}, + ?MODULE, [], Log0), + LogS = ra_log:set_snapshot_state({SnapshotMeta, SnapshotData}, Log1), meck:expect(ra_log, init, fun (_) -> LogS end), #{current_term := 5, commit_index := 3, @@ -729,7 +733,9 @@ follower_aer_term_mismatch_snapshot(_Config) -> cluster => #{}, machine_version => 1}, Data = <<"hi3">>, - {_, _, Log,_} = ra_log_memory:install_snapshot({3, 5}, {Meta, Data}, ?MODULE, Log0), + + {ok, Log1, _} = ra_log_memory:install_snapshot({3, 5}, ?MODULE, [], Log0), + Log = ra_log_memory:set_snapshot_state({Meta, Data}, Log1), State = maps:put(log, Log, State0), AE = #append_entries_rpc{term = 6, @@ -1697,6 +1703,10 @@ follower_install_snapshot_machine_version(_Config) -> _Effects0} = ra_server:handle_follower(ISR, State00), meck:expect(ra_log, recover_snapshot, fun (_) -> {SnapMeta, SnapData} end), + meck:expect(ra_snapshot, complete_accept, + fun (_, _, _, S) -> + {S, SnapData, [], []} + end), {follower, #{cfg := #cfg{machine_version = _,%% this gets populated on init only machine_versions = [{4, 1}, {0,0}], effective_machine_module = MacMod1, @@ -2241,24 +2251,29 @@ leader_receives_install_snapshot_rpc(_Config) -> % leader ignores lower term {leader, State, _} = ra_server:handle_leader(ISRpc#install_snapshot_rpc{term = Term - 1}, - State), + State), ok. follower_installs_snapshot(_Config) -> N1 = ?N1, N2 = ?N2, N3 = ?N3, - #{N3 := {_, FState = #{cluster := Config}, _}} - = init_servers([N1, N2, N3], {module, ra_queue, #{}}), + #{N3 := {_, FState = #{cluster := Config}, _}} = + init_servers([N1, N2, N3], {module, ra_queue, #{}}), LastTerm = 1, % snapshot term Term = 2, % leader term Idx = 3, + ct:pal("FState ~p", [FState]), ISRpc = #install_snapshot_rpc{term = Term, leader_id = N1, meta = snap_meta(Idx, LastTerm, Config), chunk_state = {1, last}, data = []}, {receive_snapshot, FState1, [{next_event, ISRpc}, {record_leader_msg, _}]} = - ra_server:handle_follower(ISRpc, FState), + ra_server:handle_follower(ISRpc, FState#{current_term => Term}), + meck:expect(ra_snapshot, complete_accept, + fun (_, _, _, S) -> + {S, [], [], []} + end), {follower, #{current_term := Term, commit_index := Idx, last_applied := Idx, @@ -3116,15 +3131,16 @@ leader_heartbeat_reply_higher_term(_Config) -> % %%% helpers init_servers(ServerIds, Machine) -> - lists:foldl(fun (ServerId, Acc) -> - Args = #{cluster_name => some_id, - id => ServerId, - uid => atom_to_binary(element(1, ServerId), utf8), - initial_members => ServerIds, - log_init_args => #{uid => <<>>}, - machine => Machine}, - Acc#{ServerId => {follower, ra_server_init(Args), []}} - end, #{}, ServerIds). + lists:foldl( + fun (ServerId, Acc) -> + Args = #{cluster_name => some_id, + id => ServerId, + uid => atom_to_binary(element(1, ServerId), utf8), + initial_members => ServerIds, + log_init_args => #{uid => <<>>}, + machine => Machine}, + Acc#{ServerId => {follower, ra_server_init(Args), []}} + end, #{}, ServerIds). list(L) when is_list(L) -> L; list(L) -> [L]. diff --git a/test/ra_snapshot_SUITE.erl b/test/ra_snapshot_SUITE.erl index a0bac05d0..08099bc0b 100644 --- a/test/ra_snapshot_SUITE.erl +++ b/test/ra_snapshot_SUITE.erl @@ -18,6 +18,8 @@ -define(MACMOD, ?MODULE). +-define(MAGIC, "RASN"). +-define(VERSION, 1). %%%=================================================================== %%% Common Test callbacks %%%=================================================================== @@ -376,11 +378,12 @@ accept_snapshot(Config) -> undefined = ra_snapshot:accepting(State0), {ok, S1} = ra_snapshot:begin_accept(Meta, State0), {55, 2} = ra_snapshot:accepting(S1), - {ok, S2, _} = ra_snapshot:accept_chunk(A, 1, next, S1), - {ok, S3, _} = ra_snapshot:accept_chunk(B, 2, next, S2), - {ok, S4, _} = ra_snapshot:accept_chunk(C, 3, next, S3), - {ok, S5, _} = ra_snapshot:accept_chunk(D, 4, next, S4), - {ok, S, _} = ra_snapshot:accept_chunk(E, 5, last, S5), + S2 = ra_snapshot:accept_chunk(A, 1, S1), + S3 = ra_snapshot:accept_chunk(B, 2, S2), + S4 = ra_snapshot:accept_chunk(C, 3, S3), + S5 = ra_snapshot:accept_chunk(D, 4, S4), + Machine = {machine, ?MODULE, #{}}, + {S,_, _, _} = ra_snapshot:complete_accept(E, 5, Machine, S5), undefined = ra_snapshot:accepting(S), undefined = ra_snapshot:pending(S), @@ -404,8 +407,8 @@ abort_accept(Config) -> undefined = ra_snapshot:accepting(State0), {ok, S1} = ra_snapshot:begin_accept(Meta, State0), {55, 2} = ra_snapshot:accepting(S1), - {ok, S2, _} = ra_snapshot:accept_chunk(A, 1, next, S1), - {ok, S3, _} = ra_snapshot:accept_chunk(B, 2, next, S2), + S2 = ra_snapshot:accept_chunk(A, 1, S1), + S3 = ra_snapshot:accept_chunk(B, 2, S2), S = ra_snapshot:abort_accept(S3), undefined = ra_snapshot:accepting(S), undefined = ra_snapshot:pending(S), @@ -435,7 +438,7 @@ accept_receives_snapshot_written_with_higher_index(Config) -> %% then begin an accept for a higher index {ok, State2} = ra_snapshot:begin_accept(MetaHigh, State1), {165, 2} = ra_snapshot:accepting(State2), - {ok, State3, _} = ra_snapshot:accept_chunk(A, 1, next, State2), + State3 = ra_snapshot:accept_chunk(A, 1, State2), %% then the snapshot written event is received receive @@ -445,7 +448,8 @@ accept_receives_snapshot_written_with_higher_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), + Machine = {machine, ?MODULE, #{}}, + {State, _, _, _} = ra_snapshot:complete_accept(B, 2, Machine, State4), undefined = ra_snapshot:accepting(State), {165, 2} = ra_snapshot:current(State), ok @@ -464,8 +468,14 @@ accept_receives_snapshot_written_with_higher_index_2(Config) -> ra_snapshot:begin_snapshot(MetaLow, ?MACMOD, ?FUNCTION_NAME, snapshot, State0), Fun(), - MacRef = crypto:strong_rand_bytes(1024), - MacBin = term_to_binary(MacRef), + MacState = crypto:strong_rand_bytes(1024), + MetaBin = term_to_binary(MetaHigh), + IOVec = term_to_iovec(MacState), + Data = [<<(size(MetaBin)):32/unsigned>>, MetaBin | IOVec], + Checksum = erlang:crc32(Data), + MacBin = iolist_to_binary([<>,Data]), %% split into 1024 max byte chunks <> = MacBin, @@ -473,10 +483,11 @@ accept_receives_snapshot_written_with_higher_index_2(Config) -> %% then begin an accept for a higher index {ok, State2} = ra_snapshot:begin_accept(MetaHigh, State1), {165, 2} = ra_snapshot:accepting(State2), - {ok, State3, _} = ra_snapshot:accept_chunk(A, 1, next, State2), + State3 = ra_snapshot:accept_chunk(A, 1, State2), {165, 2} = ra_snapshot:accepting(State3), - {ok, State4, _} = ra_snapshot:accept_chunk(B, 2, last, State3), + {State4, _, _, _} = ra_snapshot:complete_accept(B, 2, {machine, ?MODULE, #{}}, + State3), undefined = ra_snapshot:accepting(State4), {165, 2} = ra_snapshot:current(State4), undefined = ra_snapshot:pending(State4), From f3104b7884e28826cfb3f20ff6ed3ef0e8b397a8 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 22 May 2025 10:00:00 +0100 Subject: [PATCH 25/60] darned dialyzer --- src/ra_kv.erl | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/ra_kv.erl b/src/ra_kv.erl index 7d31b724e..9f138b410 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -24,12 +24,13 @@ -define(STATE, ?MODULE). +-define(TUPLE(A, B), [A | B]). -type key() :: binary(). -type value() :: term(). -record(?STATE, {keys = #{} :: - #{key() := [ra:index() | Hash :: non_neg_integer()]}}). + #{key() => ?TUPLE(non_neg_integer(), Hash :: integer())}}). -record(put, {key :: key(), @@ -107,11 +108,14 @@ put(ServerId, Key, Value, Timeout) -> init(_) -> #?MODULE{}. +%% we use improper lists in this module +-dialyzer({no_improper_lists, [apply/3]}). + apply(#{index := Idx} = Meta, #put{key = Key, meta = #{hash := Hash}}, #?STATE{keys = Keys} = State0) -> - State = State0#?STATE{keys = maps:put(Key, [Idx | Hash], Keys)}, + State = State0#?STATE{keys = Keys#{Key => ?TUPLE(Idx, Hash)}}, {State, {ok, Meta}, []}. live_indexes(#?STATE{keys = Keys}) -> From 7fbb2b32bd297281839422b7a2956680f9961b3f Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 27 May 2025 12:13:08 +0100 Subject: [PATCH 26/60] refine compaction design --- docs/internals/COMPACTION.md | 155 +++++++++++++++-------------------- src/ra_kv.erl | 44 ++++++++-- test/ra_kv_SUITE.erl | 3 + 3 files changed, 105 insertions(+), 97 deletions(-) diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md index 0704a4be2..985854bbd 100644 --- a/docs/internals/COMPACTION.md +++ b/docs/internals/COMPACTION.md @@ -7,19 +7,19 @@ This is a living document capturing current work on log compaction. 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 +below the snapshot index where the snapshot is a full representation of the state machine state. The high level idea of compacting in Ra is that instead of deleting all segment data that precedes the snapshot index the snapshot data can emit a list of live raft indexes which will be kept, either in their original segments -or written to new compacted segments. the data for these indexes can then -be omitted from the snapshot to reduce its size and write amplification. - +or written to new compacted segments. The data for these indexes can then +be omitted from the snapshot to reduce its size and the write amplification +incurred by writing the snapshot. ### Log sections -Two named sections of the log then emerge. +Two named sections of the log then emerge: #### Normal log section @@ -43,14 +43,17 @@ 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. +log truncation is achieved by returning an empty list of live indexes. + +TODO: how to ensure segments containing overwritten entries only are cleaned +up. ### Compacted segments: naming (phase 3 compaction) Segment files in a Ra log have numeric names incremented as they are written. This is essential as the order is required to ensure log integrity. -Desired Properties of phase 3 compaction: +Desired Properties of phase 3: * Retain immutability, entries will never be deleted from a segment. Instead they will be written to a new segment. @@ -61,21 +64,44 @@ will be written to a new segment. 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` +During compaction the target segment will have the naming format `001-002-003.compacting` +such that each segment (001, 002, 003) name is present in the compacting name. +An upper limit on the maximum number of source segments will have to be set to +ensure the compacting file name doesn't get ridiculously long. E.g. 8. + +Once the compacting segment has been synced the lowest numbered segment will +be hard linked to the compacting segment. Each of the compacted +higher numbered segments (003, 004) will then have a symlink created (e.g. 003.link) +pointing to the lowest numbered segment (002) +then the link is renamed to the source file: `003.link -> 003` (NB not atomic). + +`002-003-004.compacting` is then deleted (but 002 is still hard linked so the data +will remain). + +This naming format means it is easy to identify partially compacted segments +after an unclean exit. All `*.compacting` files with a link count of 1 will +be deleted as it is not clear at what stage the unclean exit occurred. + +If a compacting file has a link count of 2 (or more???) the compacting writes +completed and the lowest numbered segment was hard linked to the compacting +segment. We don't know if all symlinks were created correctly so we need to ensure +this during recovery. + +Once we've ensured there are hard or symlinks for all the source files the compacting +file can be deleted. + +The symlinks are there so that any pending read references to the old +segment name are still valid for some time after but the disk space for the +source segment will still be reclaimed when the links replace the files. -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. +Some time later the symbolic links can be removed. + +Single segment compaction would work the same as we can directly rename +e.g. the compacted segment `001.compacting` to `001.segment` without breaking +any references to the segment. Single segment compaction should only be triggered +when a certain limit has been reached, e.g. > 50% of indexes can be cleared up. -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? @@ -84,9 +110,22 @@ Options: * On a timer * After phase 1 if needed based on a ratio of live to dead indexes in the compacting section * After phase 1 if needed based on disk use / ratio of live data to dead. +* Explicitly through a new ra API. ![segments](compaction2.jpg) +### Phase 4 compaction (optional) + +At some point the number of live indexes could become completely sparse (no +adjacent indexes) and large which is sub optimal memory wise. + +At this point the state machine could implement a "rewrite" command (or we +provide one in Ra) to rewrite a subset or all of the indexes at the head of +the Ra log to "clump" their indexes better together. + +This is ofc optional and has replication costs but could be a manually triggered +maintenance option perhaps. + ### Ra Server log worker responsibilities * Write checkpoints and snapshots @@ -162,76 +201,18 @@ the flush request comes in. With the snapshot now defined as the snapshot state + live preceding raft indexes the default snapshot replication approach will need to change. -The snapshot sender (Ra log worker??) needs to negotiate with the follower to -discover which preceding raft indexes the follower does not yet have. Then it would -go on and replicate these before or after (??) sending the snapshot itself. - -T: probably before as if a new snapshot has been taken locally we'd most likely -skip some raft index replication on the second attempt. - -Q: How will the follower write the live indexes preceding the snapshot? -If the main Ra process does it this introduces a 3rd modifier of the Ra log -and there may be concurrent Ra log writes from the snapshot writer at this point. - -It can't write them to the WAL as they are not contiguous unless we allow -such writes. +The snapshot sender process (currently transient) first sends all live +entries for the given snapshot, then performs normal chunk based +snapshot replication. -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 +#### Snapshot install procedure * Sender begins with sending negotiating which live indexes are needed. It is probably sufficient for the receiver to return it's `last_applied` index and the sender will send all sparse entries after that index * Then it proceeds to send the live indexes _before_ the snapshot (so in it's -natural order if you like). +natural log order). * The receiving ra process then writes these commands to the WAL as normal but using a special command / flag to avoid the WAL triggering its' gap detection. Ideally the specialised command would include the previous idx so that we can @@ -239,16 +220,9 @@ still do gap detection in the sparse sequence (should all sends include prior sequence so this is the only mode?). * The sparse writes are written to a new memtable using a new `ra_mt:sparse_write/2` API that bypasses gap validation and stores a sparse sequence instead of range -* Alt the live indexes replication could be done after the snapshot is complete -as it is easy for the follower to work out which live indexes it needs. -when it receives the `last` snapshot chunk it then replies with a special -continuation command instead of `install_snapshot_result{}` which will initiate -the live index replication. NB the snapshot sender process may need to call -into the leader process to get read plans as entries _could_ be in the memtable. + #### How to work out which live indexes the follower needs -WA -Gnarly example: Follower term indexes: @@ -267,7 +241,6 @@ If follower `last_applied` is: 1100 then follower needs `[1200, 1777]` #### How to store live indexes with snapshot -* New section in snapshot file format. * Separate file (that can be rebuilt if needed from the snapshot). diff --git a/src/ra_kv.erl b/src/ra_kv.erl index 9f138b410..1b165cf43 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -19,7 +19,9 @@ start_cluster/3, add_member/3, - put/4 + put/4, + get/3, + query_get/3 ]). @@ -48,15 +50,15 @@ -spec start_cluster(atom(), atom(), map()) -> {ok, [ra_server_id()], [ra_server_id()]} | {error, cluster_not_formed}. -start_cluster(System, Name, #{members := ServerIds}) - when is_atom(Name) andalso +start_cluster(System, ClusterName, #{members := ServerIds}) + when is_atom(ClusterName) andalso is_atom(System) -> Machine = {module, ?MODULE, #{}}, Configs = [begin - UId = ra:new_uid(ra_lib:to_binary(Name)), + UId = ra:new_uid(ra_lib:to_binary(ClusterName)), #{id => Id, uid => UId, - cluster_name => Name, + cluster_name => ClusterName, log_init_args => #{uid => UId}, initial_members => ServerIds, machine => Machine} @@ -95,14 +97,43 @@ put(ServerId, Key, Value, Timeout) -> end. -%% get performs a consistent query that returns the index, hash and member set +%% @doc get performs a consistent query that returns the index, hash and member set %% then perform an aux query to actually get the data for a given index. %% if addressing a follower (say there is a local one) then the read may need %% to wait if the index isn't yet available locally (term also need to be checked) %% or check that the machien state has the right index for a given key before %% reading the value from the log +-spec get(ra:server_id(), key(), non_neg_integer()) -> + {ok, map(), value()} | {error, term()} | {timeout, ra:server_id()}. +get(ServerId, Key, Timeout) -> + case ra:consistent_query(ServerId, {?MODULE, query_get, + [element(1, ServerId), Key]}, Timeout) of + {ok, {ok, Idx, Members}, LeaderId} -> + case ra_server_proc:read_entries(LeaderId, [Idx], + undefined, Timeout) of + {ok, {#{Idx := {Idx, Term, + {'$usr', Meta, #put{value = Value}, _}}}, Flru}} -> + _ = ra_flru:evict_all(Flru), + {ok, Meta#{index => Idx, + members => Members, + term => Term}, Value}; + Err -> + Err + end; + Err -> + Err + end. +query_get(ClusterName, Key, #?STATE{keys = Keys}) -> + Members = ra_leaderboard:lookup_members(ClusterName), + case Keys of + #{Key := [Idx |_]} -> + {ok, Idx, Members}; + _ -> + {error, not_found} + end. + %% state machine init(_) -> @@ -124,6 +155,7 @@ live_indexes(#?STATE{keys = Keys}) -> end, [], Keys). -record(aux, {}). + init_aux(_) -> #aux{}. diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index b1cc0ec02..bc39c118b 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -70,6 +70,9 @@ basics(_Config) -> [{ok, #{}} = ra_kv:put(KvId, K2, I, 5000) || I <- lists:seq(1, 10000)], + ct:pal("kv get ~p", [ra_kv:get(KvId, <<"k1">>, 5000)]), + ct:pal("leaderboard ~p", [ets:tab2list(ra_leaderboard)]), + ?assertMatch({ok, #{machine := #{num_keys := 2}}, KvId}, ra:member_overview(KvId)), ra_log_wal:force_roll_over(ra_log_wal), From c043152d3e50f2992004639c423df31ebd7203d9 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 28 May 2025 08:29:54 +0100 Subject: [PATCH 27/60] add segment copy op --- docs/internals/COMPACTION.md | 2 ++ src/ra_log.erl | 28 ++++++++++++++------------ src/ra_log_reader.erl | 5 +++-- src/ra_log_segment.erl | 35 ++++++++++++++++++++++++++++++++- test/ra_log_segment_SUITE.erl | 37 +++++++++++++++++++++++++++++++---- 5 files changed, 88 insertions(+), 19 deletions(-) diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md index 985854bbd..cd3f9e150 100644 --- a/docs/internals/COMPACTION.md +++ b/docs/internals/COMPACTION.md @@ -101,6 +101,8 @@ e.g. the compacted segment `001.compacting` to `001.segment` without breaking any references to the segment. Single segment compaction should only be triggered when a certain limit has been reached, e.g. > 50% of indexes can be cleared up. +TODO: how to handle compaction of segments that have indexes that never were +committed, i.e. overwritten? #### When does phase 3 compaction run? diff --git a/src/ra_log.erl b/src/ra_log.erl index c0baa0abf..d9c4a9d3e 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -223,6 +223,8 @@ init(#{uid := UId, % this queries the segment writer and thus blocks until any % segments it is currently processed have been finished MtRange = ra_mt:range(Mt0), + %% TODO: init ra_log_reader here instead and let it take care of range + %% calulation and segref compaction {{FirstIdx, LastIdx0}, SegRefs} = case recover_ranges(UId, MtRange, SegWriter) of {undefined, SRs} -> {{-1, -1}, SRs}; @@ -938,31 +940,30 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, LiveIndexes, put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, SnapIdx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, SnapIdx), - CompEffs = schedule_compaction(SnapIdx, State0), - {SnapState, Checkpoints} = ra_snapshot:take_older_checkpoints(SnapIdx, SnapState0), CPEffects = [{delete_snapshot, ra_snapshot:directory(SnapState, checkpoint), Checkpoint} || Checkpoint <- Checkpoints], - State = State0#?MODULE{snapshot_state = SnapState, - first_index = SnapIdx + 1, - last_index = SnapIdx, - last_term = SnapTerm, - last_written_index_term = IdxTerm}, - %% TODO: more mt entries could potentially be cleared up in the - %% mem table here SmallestLiveIndex = case ra_seq:first(LiveIndexes) of undefined -> SnapIdx + 1; I -> I end, + %% TODO: more mt entries could potentially be cleared up in the + %% mem table here {Spec, Mt} = ra_mt:set_first(SmallestLiveIndex, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), - {ok, State#?MODULE{live_indexes = LiveIndexes, - mem_table = Mt}, - CompEffs ++ CPEffects}. + State = State0#?MODULE{snapshot_state = SnapState, + first_index = SnapIdx + 1, + last_index = SnapIdx, + last_term = SnapTerm, + live_indexes = LiveIndexes, + mem_table = Mt, + last_written_index_term = IdxTerm}, + CompEffs = schedule_compaction(SnapIdx, State), + {ok, State, CompEffs ++ CPEffects}. -spec recover_snapshot(State :: state()) -> @@ -1261,6 +1262,9 @@ schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{}, end, lists:reverse(Compactable)), SnapDir = ra_snapshot:current_snapshot_dir(SnapState), + %% TODO: minor compactions should also delete / truncate + %% segments with completely overwritten indexes + Self = self(), Fun = fun () -> {ok, Indexes} = ra_snapshot:indexes(SnapDir), diff --git a/src/ra_log_reader.erl b/src/ra_log_reader.erl index f87e7f36e..d632fa4a0 100644 --- a/src/ra_log_reader.erl +++ b/src/ra_log_reader.erl @@ -109,6 +109,7 @@ update_segments(NewSegmentRefs, segment_refs = SegRefs0} = State) -> SegmentRefs0 = ra_lol:to_list(SegRefs0), + %% TODO: capture segrefs removed by compact_segrefs/2 and delete them SegmentRefsComp = compact_segrefs(NewSegmentRefs, SegmentRefs0), SegmentRefsCompRev = lists:reverse(SegmentRefsComp), SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev), @@ -470,13 +471,13 @@ compact_seg_refs_test() -> compact_segref_3_test() -> Data = [ - {{2, 7}, "B"}, + {{2, 7}, "C"}, %% this entry has overwritten the prior two {{5, 10}, "B"}, {{1, 4}, "A"} ], Res = compact_segrefs(Data, []), - ?assertMatch([{{2, 7}, "B"}, + ?assertMatch([{{2, 7}, "C"}, {{1, 1}, "A"}], Res), ok. diff --git a/src/ra_log_segment.erl b/src/ra_log_segment.erl index 3ba8c2381..218e01dbd 100644 --- a/src/ra_log_segment.erl +++ b/src/ra_log_segment.erl @@ -22,13 +22,15 @@ max_count/1, filename/1, segref/1, - is_same_as/2]). + is_same_as/2, + copy/3]). -export([dump/1, dump_index/1]). -include("ra.hrl"). +-include_lib("stdlib/include/assert.hrl"). -include_lib("kernel/include/file.hrl"). -define(VERSION, 2). @@ -474,6 +476,23 @@ close(#state{cfg = #cfg{fd = Fd}}) -> _ = file:close(Fd), ok. +-spec copy(state(), file:filename_all(), [ra:index()]) -> + {ok, state()} | {error, term()}. +copy(#state{} = State0, FromFile, Indexes) + when is_list(Indexes) -> + {ok, From} = open(FromFile, #{mode => read}), + %% TODO: the current approach recalculates the CRC and isn't completely + %% optimial. Also it does not allow for a future where copy_file_range may + %% be available + State = lists:foldl( + fun (I, S0) -> + {ok, Term, Data} = simple_read(From, I), + {ok, S} = append(S0, I, Term, Data), + S + end, State0, lists:sort(Indexes)), + close(From), + sync(State). + %%% Internal is_same_filename_all(Fn, Fn) -> @@ -693,6 +712,20 @@ is_full(#state{cfg = #cfg{max_size = MaxSize}, IndexOffset >= DataStart orelse (DataOffset - DataStart) > MaxSize. +simple_read(#state{cfg = #cfg{fd = Fd}, + index = SegIndex}, Idx) + when is_map_key(Idx, SegIndex) -> + {Term, Pos, Len, _} = map_get(Idx, SegIndex), + case file:pread(Fd, Pos, Len) of + {ok, Data} -> + ?assert(byte_size(Data) == Len), + {ok, Term, Data}; + Err -> + Err + end; +simple_read(_State, _) -> + {error, not_found}. + -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/test/ra_log_segment_SUITE.erl b/test/ra_log_segment_SUITE.erl index a3a8b7fbf..270c7d2ce 100644 --- a/test/ra_log_segment_SUITE.erl +++ b/test/ra_log_segment_SUITE.erl @@ -38,7 +38,8 @@ all_tests() -> corrupted_segment, large_segment, segref, - versions_v1 + versions_v1, + copy ]. groups() -> @@ -460,6 +461,37 @@ read_sparse_append_read(Config) -> ra_log_segment:close(R0), ok. +copy(Config) -> + Dir = ?config(data_dir, Config), + Indexes = lists:seq(1, 100), + SrcFn = filename:join(Dir, <<"SOURCE1.segment">>), + {ok, SrcSeg0} = ra_log_segment:open(SrcFn), + SrcSeg1 = lists:foldl( + fun (I, S0) -> + {ok, S} = ra_log_segment:append(S0, I, 1, term_to_binary(I)), + S + end, SrcSeg0, Indexes), + _ = ra_log_segment:close(SrcSeg1), + + Fn = filename:join(Dir, <<"TARGET.segment">>), + {ok, Seg0} = ra_log_segment:open(Fn), + CopyIndexes = lists:seq(1, 100, 2), + {ok, Seg} = ra_log_segment:copy(Seg0, SrcFn, CopyIndexes), + ra_log_segment:close(Seg), + {ok, R} = ra_log_segment:open(Fn, #{mode => read, + access_pattern => random}), + %%TODO: consider makeing read_sparse tolerant to missing indexes somehow + %% perhaps detecting if the segment is "sparse" + {ok, 2, [_, _]} = ra_log_segment:read_sparse(R, [1, 3], + fun (I, T, B, Acc) -> + [{I, T, binary_to_term(B)} | Acc] + end, []), + ra_log_segment:close(R), + + ok. + + +%%% Internal write_until_full(Idx, Term, Data, Seg0) -> case ra_log_segment:append(Seg0, Idx, Term, Data) of {ok, Seg} -> @@ -467,9 +499,6 @@ write_until_full(Idx, Term, Data, Seg0) -> {error, full} -> Seg0 end. - - -%%% Internal make_data(Size) -> term_to_binary(crypto:strong_rand_bytes(Size)). From 71cb49878648fd9f30aac43499836fbe36e8a9be Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 28 May 2025 20:02:31 +0100 Subject: [PATCH 28/60] refactor log to use range --- src/ra_log.erl | 328 +++++++++++++++++++++--------------- src/ra_log_pre_init.erl | 2 +- test/coordination_SUITE.erl | 2 +- test/ra_kv_SUITE.erl | 4 +- test/ra_log_2_SUITE.erl | 9 +- 5 files changed, 203 insertions(+), 142 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index d9c4a9d3e..19158f8b3 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -100,11 +100,13 @@ -record(?MODULE, {cfg = #cfg{}, %% mutable data below - first_index = -1 :: ra_index(), - last_index = -1 :: -1 | ra_index(), + range :: ra:range(), + % first_index = -1 :: ra_index(), + % last_index = -1 :: -1 | ra_index(), last_term = 0 :: ra_term(), last_written_index_term = {0, 0} :: ra_idxterm(), snapshot_state :: ra_snapshot:state(), + current_snapshot :: option(ra_idxterm()), last_resend_time :: option({integer(), WalPid :: pid() | undefined}), last_wal_write :: {pid(), Ms :: integer()}, reader :: ra_log_reader:state(), @@ -140,9 +142,9 @@ -type overview() :: #{type := ra_log, - last_index := ra_index(), + range := ra:range(), + last_index := ra:index(), last_term := ra_term(), - first_index := ra_index(), last_written_index_term := ra_idxterm(), num_segments := non_neg_integer(), open_segments => non_neg_integer(), @@ -207,7 +209,7 @@ init(#{uid := UId, SnapshotState = ra_snapshot:init(UId, SnapModule, SnapshotsDir, CheckpointsDir, Counter, MaxCheckpoints), {SnapIdx, SnapTerm} = case ra_snapshot:current(SnapshotState) of - undefined -> {-1, -1}; + undefined -> {-1, 0}; Curr -> Curr end, %% TODO: error handling @@ -223,28 +225,32 @@ init(#{uid := UId, % this queries the segment writer and thus blocks until any % segments it is currently processed have been finished MtRange = ra_mt:range(Mt0), - %% TODO: init ra_log_reader here instead and let it take care of range - %% calulation and segref compaction - {{FirstIdx, LastIdx0}, SegRefs} = case recover_ranges(UId, MtRange, SegWriter) of - {undefined, SRs} -> - {{-1, -1}, SRs}; - R -> R - end, - %% TODO: don't think this is necessary given the range is calculated from this - %% but can't hurt as it may trigger some cleanup - {DeleteSpecs, Mt} = ra_mt:set_first(FirstIdx, Mt0), - - ok = exec_mem_table_delete(Names, UId, DeleteSpecs), + SegRefs = my_segrefs(UId, SegWriter), Reader = ra_log_reader:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, Names, Counter), + SegRange = ra_log_reader:range(Reader), + %% TODO: check ra_range:add/2 actually performas the correct logic we expect + Range = ra_range:add(MtRange, SegRange), + %% assert there is no gap between the snapshot %% and the first index in the log - case (FirstIdx - SnapIdx) > 1 of - true -> - exit({corrupt_log, gap_between_snapshot_and_first_index, - {SnapIdx, FirstIdx}}); - false -> ok - end, + Mt = case Range of + undefined -> + Mt0; + {FstIdx, LstIdx} -> + case FstIdx == SnapIdx + 1 orelse + ra_range:in(SnapIdx, Range) orelse + SnapIdx > LstIdx of + true -> + {DeleteSpecs, Mt1} = ra_mt:set_first(FstIdx, Mt0), + ok = exec_mem_table_delete(Names, UId, DeleteSpecs), + Mt1; + false -> + exit({corrupt_log, + gap_between_snapshot_and_log_range, + {SnapIdx, Range}}) + end + end, Cfg = #cfg{directory = Dir, uid = UId, log_id = LogId, @@ -257,16 +263,21 @@ init(#{uid := UId, counter = Counter, names = Names}, State0 = #?MODULE{cfg = Cfg, - first_index = max(SnapIdx + 1, FirstIdx), - last_index = max(SnapIdx, LastIdx0), + range = ra_range:truncate(SnapIdx, Range), reader = Reader, mem_table = Mt, snapshot_state = SnapshotState, + current_snapshot = ra_snapshot:current(SnapshotState), 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, + LastIdx = case Range of + undefined -> + SnapIdx; + {_, Lst} -> + Lst + end, put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, LastIdx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, LastIdx), put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, @@ -279,14 +290,12 @@ init(#{uid := UId, end, % recover the last term - {LastTerm0, State2} = case LastIdx of - SnapIdx -> - {SnapTerm, State0}; - -1 -> - {0, State0}; - LI -> - fetch_term(LI, State0) - end, + {LastTerm0, State2} = case Range of + undefined -> + {SnapTerm, State0}; + {_, LI} -> + fetch_term(LI, State0) + end, LastSegRefIdx = case SegRefs of [] -> -1; @@ -370,16 +379,25 @@ commit_tx(#?MODULE{cfg = #cfg{uid = UId, commit_tx(#?MODULE{tx = false} = State) -> State. +-define(IS_NEXT_IDX(Idx, Range), + Range == undefined orelse + element(2, Range) + 1 =:= Idx). + +-define(IS_IN_RANGE(Idx, Range), + Range =/= undefined andalso + Idx >= element(1, Range) andalso + Idx =< element(2, Range)). + -spec append(Entry :: log_entry(), State :: state()) -> state() | no_return(). append({Idx, Term, Cmd0} = Entry, #?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, - last_index = LastIdx, + range = Range, tx = false, pending = Pend0, mem_table = Mt0} = State) - when Idx =:= LastIdx + 1 -> + when ?IS_NEXT_IDX(Idx, Range) -> case ra_mt:insert(Entry, Mt0) of {ok, Mt} -> Cmd = {ttb, term_to_iovec(Cmd0)}, @@ -389,7 +407,7 @@ append({Idx, Term, Cmd0} = Entry, Pend = ra_seq:limit(Idx - 1, Pend0), ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), - State#?MODULE{last_index = Idx, + State#?MODULE{range = ra_range:extend(Idx, Range), last_term = Term, last_wal_write = {Pid, now_ms()}, pending = ra_seq:append(Idx, Pend), @@ -408,15 +426,15 @@ append({Idx, Term, Cmd0} = Entry, end; append({Idx, Term, _Cmd} = Entry, #?MODULE{cfg = Cfg, - last_index = LastIdx, + range = Range, tx = true, pending = Pend0, mem_table = Mt0} = State) - when Idx =:= LastIdx + 1 -> + when ?IS_NEXT_IDX(Idx, Range) -> case ra_mt:stage(Entry, Mt0) of {ok, Mt} -> put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), - State#?MODULE{last_index = Idx, + State#?MODULE{range = ra_range:extend(Idx, Range), last_term = Term, pending = ra_seq:append(Idx, Pend0), mem_table = Mt}; @@ -429,9 +447,9 @@ append({Idx, Term, _Cmd} = Entry, Cfg#cfg.uid, Mt0), append(Entry, State#?MODULE{mem_table = M0}) end; -append({Idx, _, _}, #?MODULE{last_index = LastIdx}) -> - Msg = lists:flatten(io_lib:format("tried writing ~b - expected ~b", - [Idx, LastIdx+1])), +append({Idx, _, _}, #?MODULE{range = Range}) -> + Msg = lists:flatten(io_lib:format("tried writing ~b - current range ~w", + [Idx, Range])), exit({integrity_error, Msg}). -spec write(Entries :: [log_entry()], State :: state()) -> @@ -439,11 +457,12 @@ append({Idx, _, _}, #?MODULE{last_index = LastIdx}) -> {error, {integrity_error, term()} | wal_down}. write([{FstIdx, _, _} | _Rest] = Entries, #?MODULE{cfg = Cfg, - last_index = LastIdx, + range = Range, pending = Pend0, mem_table = Mt0} = State0) - when FstIdx =< LastIdx + 1 andalso - FstIdx >= 0 -> + when Range == undefined orelse + (FstIdx =< element(2, Range) + 1 andalso + FstIdx >= 0) -> case stage_entries(Cfg, Entries, Mt0) of {ok, Mt} -> Pend = ra_seq:limit(FstIdx - 1, Pend0), @@ -455,10 +474,10 @@ write([{FstIdx, _, _} | _Rest] = Entries, write([], State) -> {ok, State}; write([{Idx, _, _} | _], #?MODULE{cfg = #cfg{uid = UId}, - last_index = LastIdx}) -> + range = Range}) -> Msg = lists:flatten(io_lib:format("~s: ra_log:write/2 " - "tried writing ~b - expected ~b", - [UId, Idx, LastIdx+1])), + "tried writing ~b - current range ~w", + [UId, Idx, Range])), {error, {integrity_error, Msg}}. -spec write_sparse(log_entry(), ra:index(), state()) -> @@ -466,13 +485,12 @@ write([{Idx, _, _} | _], #?MODULE{cfg = #cfg{uid = UId}, write_sparse({Idx, Term, _} = Entry, PrevIdx0, #?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, - last_index = LastIdx, + range = Range, mem_table = Mt0} = State0) when PrevIdx0 == undefined orelse - (PrevIdx0 == LastIdx) -> + Range == undefined orelse + (PrevIdx0 == element(2, Range)) -> {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 @@ -486,7 +504,13 @@ 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), - {ok, State0#?MODULE{last_index = Idx, + NewRange = case Range of + undefined -> + ra_range:new(Idx); + {S, _} -> + ra_range:new(S, Idx) + end, + {ok, State0#?MODULE{range = NewRange, last_term = Term, mem_table = Mt, last_wal_write = {Pid, now_ms()}}}; @@ -500,15 +524,16 @@ write_sparse({Idx, Term, _} = Entry, PrevIdx0, fold(From0, To0, Fun, Acc0, #?MODULE{cfg = Cfg, mem_table = Mt, - first_index = FirstIdx, - last_index = LastIdx, + range = {StartIdx, EndIdx}, reader = Reader0} = State) when To0 >= From0 andalso - To0 >= FirstIdx -> - From = max(From0, FirstIdx), - To = min(To0, LastIdx), - ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), + To0 >= StartIdx -> + %% TODO: move to ra_range function + From = max(From0, StartIdx), + To = min(To0, EndIdx), + + ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), MtOverlap = ra_mt:range_overlap({From, To}, Mt), case MtOverlap of {undefined, {RemStart, RemEnd}} -> @@ -538,8 +563,7 @@ fold(_From, _To, _Fun, Acc, State) -> {[log_entry()], state()}. sparse_read(Indexes0, #?MODULE{cfg = Cfg, reader = Reader0, - first_index = FstIdx, - last_index = LastIdx, + range = Range, live_indexes = LiveIndexes, mem_table = Mt} = State) -> ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), @@ -558,17 +582,8 @@ sparse_read(Indexes0, #?MODULE{cfg = Cfg, %% drop any indexes that are larger than the last index available %% 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) + ra_range:in(I, Range) orelse + ra_seq:in(I, LiveIndexes) end, Indexes1), {Entries0, MemTblNumRead, Indexes} = ra_mt:get_items(Indexes2, Mt), ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, MemTblNumRead), @@ -599,7 +614,8 @@ sparse_read(Indexes0, #?MODULE{cfg = Cfg, read_plan(). partial_read(Indexes0, #?MODULE{cfg = Cfg, reader = Reader0, - last_index = LastIdx, + range = Range, + snapshot_state = SnapState, mem_table = Mt}, TransformFun) -> ok = incr_counter(Cfg, ?C_RA_LOG_READ_OPS, 1), @@ -614,6 +630,17 @@ partial_read(Indexes0, #?MODULE{cfg = Cfg, % descending or undefined Indexes0 end, + LastIdx = case Range of + undefined -> + case ra_snapshot:current(SnapState) of + undefined -> + -1; + {SnapIdx, _} -> + SnapIdx + end; + {_, End} -> + End + end, %% drop any indexes that are larger than the last index available Indexes2 = lists:dropwhile(fun (I) -> I > LastIdx end, Indexes1), @@ -652,9 +679,12 @@ read_plan_info(#read_plan{read = Read, num_segments => NumSegments}. --spec last_index_term(state()) -> ra_idxterm(). -last_index_term(#?MODULE{last_index = LastIdx, last_term = LastTerm}) -> - {LastIdx, LastTerm}. +-spec last_index_term(state()) -> option(ra_idxterm()). +last_index_term(#?MODULE{range = {_, LastIdx}, + last_term = LastTerm}) -> + {LastIdx, LastTerm}; +last_index_term(#?MODULE{current_snapshot = CurSnap}) -> + CurSnap. -spec last_written(state()) -> ra_idxterm(). last_written(#?MODULE{last_written_index_term = LWTI}) -> @@ -664,6 +694,7 @@ last_written(#?MODULE{last_written_index_term = LWTI}) -> -spec set_last_index(ra_index(), state()) -> {ok, state()} | {not_found, state()}. set_last_index(Idx, #?MODULE{cfg = Cfg, + range = Range, last_written_index_term = {LWIdx0, _}} = State0) -> case fetch_term(Idx, State0) of {undefined, State} -> @@ -678,7 +709,7 @@ set_last_index(Idx, #?MODULE{cfg = Cfg, true = LWTerm =/= undefined, put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, LWIdx), - {ok, State2#?MODULE{last_index = Idx, + {ok, State2#?MODULE{range = ra_range:limit(Idx + 1, Range), last_term = Term, last_written_index_term = {LWIdx, LWTerm}}} end. @@ -687,8 +718,11 @@ set_last_index(Idx, #?MODULE{cfg = Cfg, {state(), [effect()]}. handle_event({written, Term, WrittenSeq}, #?MODULE{cfg = Cfg, - first_index = FirstIdx, + % range = Range, + snapshot_state = SnapState, + % first_index = FirstIdx, pending = Pend0} = State0) -> + CurSnap = ra_snapshot:current(SnapState), %% gap detection %% 1. pending has lower indexes than the ra_seq:first index in WrittenSeq %% 2. @@ -707,7 +741,7 @@ handle_event({written, Term, WrittenSeq}, [Cfg#cfg.log_id, WrittenSeq, Pend0]), {resend_pending(State0), []} end; - {undefined, State} when LastWrittenIdx < FirstIdx -> + {undefined, State} when LastWrittenIdx =< element(1, CurSnap) -> % A snapshot happened before the written event came in % This can only happen on a leader when consensus is achieved by % followers returning appending the entry and the leader committing @@ -717,7 +751,7 @@ handle_event({written, Term, WrittenSeq}, % remove them {ok, Pend} = ra_seq:remove_prefix(WrittenSeq, Pend0), {State#?MODULE{pending = Pend}, []}; - {OtherTerm, State} -> + {OtherTerm, State} when OtherTerm =/= Term -> %% term mismatch, let's reduce the seq and try again to see %% if any entries in the range are valid case ra_seq:limit(LastWrittenIdx - 1, WrittenSeq) of @@ -788,8 +822,9 @@ handle_event({segments_to_be_deleted, SegRefs}, handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, - first_index = FstIdx, - last_index = LstIdx, + range = {FstIdx, _} = Range, + % first_index = FstIdx, + % last_index = LstIdx, mem_table = Mt0, pending = Pend0, last_written_index_term = {LastWrittenIdx, _} = LWIdxTerm0, @@ -823,7 +858,6 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, Snap end, - %% 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 @@ -843,12 +877,12 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, {Spec, Mt1} = ra_mt:set_first(SmallestLiveIdx, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), - State = State0#?MODULE{first_index = SnapIdx + 1, - last_index = max(LstIdx, SnapIdx), + State = State0#?MODULE{range = ra_range:truncate(SnapIdx, Range), last_written_index_term = LWIdxTerm, mem_table = Mt1, pending = Pend, live_indexes = LiveIndexes, + current_snapshot = Snap, snapshot_state = SnapState}, CompEffs = schedule_compaction(SnapIdx, State), Effects = CompEffs ++ Effects0, @@ -891,8 +925,12 @@ handle_event({down, _Pid, _Info}, #?MODULE{} = State) -> {State, []}. -spec next_index(state()) -> ra_index(). -next_index(#?MODULE{last_index = LastIdx}) -> - LastIdx + 1. +next_index(#?MODULE{range = {_, LastIdx}}) -> + LastIdx + 1; +next_index(#?MODULE{current_snapshot = {SnapIdx, _}}) -> + SnapIdx + 1; +next_index(#?MODULE{current_snapshot = undefined}) -> + 0. -spec fetch(ra_index(), state()) -> {option(log_entry()), state()}. @@ -906,18 +944,19 @@ fetch(Idx, State0) -> -spec fetch_term(ra_index(), state()) -> {option(ra_term()), state()}. -fetch_term(Idx, #?MODULE{last_index = LastIdx, - first_index = FirstIdx} = State0) - when Idx < FirstIdx orelse Idx > LastIdx -> - {undefined, State0}; -fetch_term(Idx, #?MODULE{mem_table = Mt, reader = Reader0} = State0) -> +fetch_term(Idx, #?MODULE{mem_table = Mt, + range = Range, + reader = Reader0} = State0) + when ?IS_IN_RANGE(Idx, Range) -> case ra_mt:lookup_term(Idx, Mt) of undefined -> {Term, Reader} = ra_log_reader:fetch_term(Idx, Reader0), {Term, State0#?MODULE{reader = Reader}}; Term when is_integer(Term) -> {Term, State0} - end. + end; +fetch_term(_Idx, #?MODULE{} = State0) -> + {undefined, State0}. -spec snapshot_state(State :: state()) -> ra_snapshot:state(). snapshot_state(State) -> @@ -956,8 +995,8 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, LiveIndexes, {Spec, Mt} = ra_mt:set_first(SmallestLiveIndex, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), State = State0#?MODULE{snapshot_state = SnapState, - first_index = SnapIdx + 1, - last_index = SnapIdx, + current_snapshot = IdxTerm, + range = undefined, last_term = SnapTerm, live_indexes = LiveIndexes, mem_table = Mt, @@ -1144,21 +1183,21 @@ exists({Idx, Term}, Log0) -> end. -spec overview(state()) -> overview(). -overview(#?MODULE{last_index = LastIndex, +overview(#?MODULE{range = Range, last_term = LastTerm, - first_index = FirstIndex, last_written_index_term = LWIT, snapshot_state = SnapshotState, + current_snapshot = CurrSnap, reader = Reader, last_wal_write = {_LastPid, LastMs}, mem_table = Mt, pending = Pend - }) -> - CurrSnap = ra_snapshot:current(SnapshotState), + } = State) -> + {LastIndex, _} = last_index_term(State), #{type => ?MODULE, + range => Range, last_index => LastIndex, last_term => LastTerm, - first_index => FirstIndex, last_written_index_term => LWIT, num_segments => ra_log_reader:segment_ref_count(Reader), segments_range => ra_log_reader:range(Reader), @@ -1292,13 +1331,14 @@ schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{}, %% unly used by resend to wal functionality and doesn't update the mem table wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, - wal = Wal} = Cfg} = State, + wal = Wal} = Cfg, + range = _Range} = State, Tid, {Idx, Term, Cmd}) -> case ra_log_wal:write(Wal, {UId, self()}, Tid, Idx, Term, Cmd) of {ok, Pid} -> ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), - State#?MODULE{last_index = Idx, + State#?MODULE{%last_index = Idx, last_term = Term, last_wal_write = {Pid, now_ms()} }; @@ -1309,8 +1349,9 @@ wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, pending = Pend0, + range = Range, mem_table = Mt0} = State, - Entries) -> + [{FstIdx, _, _} | _] = Entries) -> WriterId = {UId, self()}, %% all entries in a transaction are written to the same tid Tid = ra_mt:tid(Mt0), @@ -1325,9 +1366,15 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, {_, Mt} = ra_mt:commit(Mt0), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, LastIdx), ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, Num), + NewRange = case Range of + undefined -> + ra_range:new(FstIdx, LastIdx); + {Start, _} -> + ra_range:new(Start, LastIdx) + end, case ra_log_wal:write_batch(Wal, lists:reverse(WalCommands)) of {ok, Pid} -> - {ok, State#?MODULE{last_index = LastIdx, + {ok, State#?MODULE{range = NewRange, last_term = LastTerm, last_wal_write = {Pid, now_ms()}, mem_table = Mt, @@ -1339,11 +1386,12 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, Err end. -maybe_append_first_entry(State0 = #?MODULE{last_index = -1}) -> +maybe_append_first_entry(#?MODULE{range = undefined, + current_snapshot = undefined} = State0) -> State1 = append({0, 0, undefined}, State0), receive {ra_log_event, {written, 0, [0]} = Evt} -> - State2 = State1#?MODULE{first_index = 0}, + State2 = State1#?MODULE{range = ra_range:new(0)}, {State, _Effs} = handle_event(Evt, State2), State after 60000 -> @@ -1380,7 +1428,7 @@ resend_pending(#?MODULE{cfg = Cfg, Pend). resend_from0(Idx, #?MODULE{cfg = Cfg, - last_index = LastIdx, + range = {_, LastIdx}, last_resend_time = undefined, mem_table = Mt} = State) -> ?DEBUG("~ts: ra_log: resending from ~b to ~b", @@ -1449,14 +1497,9 @@ write_snapshot(Meta, MacModule, MacState, SnapKind, SnapKind, SnapState0), {State#?MODULE{snapshot_state = SnapState}, Effects}. -recover_ranges(UId, MtRange, SegWriter) -> - % 1. check mem_tables (this assumes wal has finished recovering - % which means it is essential that ra_servers are part of the same - % supervision tree - % 2. check segments +my_segrefs(UId, SegWriter) -> SegFiles = ra_log_segment_writer:my_segments(SegWriter, UId), - SegRefs = lists:foldl( - fun (File, Acc) -> + lists:foldl(fun (File, Acc) -> %% if a server recovered when a segment had been opened %% but never had any entries written the segref would be %% undefined @@ -1466,18 +1509,37 @@ recover_ranges(UId, MtRange, SegWriter) -> SegRef -> [SegRef | Acc] end - end, [], SegFiles), - SegRanges = [Range || {Range, _} <- SegRefs], - Ranges = [MtRange | SegRanges], - {pick_range(Ranges, undefined), SegRefs}. + end, [], SegFiles). + +% recover_ranges(UId, MtRange, SegWriter) -> +% % 1. check mem_tables (this assumes wal has finished recovering +% % which means it is essential that ra_servers are part of the same +% % supervision tree +% % 2. check segments +% SegFiles = ra_log_segment_writer:my_segments(SegWriter, UId), +% SegRefs = lists:foldl( +% fun (File, Acc) -> +% %% if a server recovered when a segment had been opened +% %% but never had any entries written the segref would be +% %% undefined +% case ra_log_segment:segref(File) of +% undefined -> +% Acc; +% SegRef -> +% [SegRef | Acc] +% end +% end, [], SegFiles), +% SegRanges = [Range || {Range, _} <- SegRefs], +% Ranges = [MtRange | SegRanges], +% {pick_range(Ranges, undefined), SegRefs}. % picks the current range from a sorted (newest to oldest) list of ranges -pick_range([], Res) -> - Res; -pick_range([H | Tail], undefined) -> - pick_range(Tail, H); -pick_range([{Fst, _Lst} | Tail], {CurFst, CurLst}) -> - pick_range(Tail, {min(Fst, CurFst), CurLst}). +% pick_range([], Res) -> +% Res; +% pick_range([H | Tail], undefined) -> +% pick_range(Tail, H); +% pick_range([{Fst, _Lst} | Tail], {CurFst, CurLst}) -> +% pick_range(Tail, {min(Fst, CurFst), CurLst}). %% TODO: implement synchronous writes using gen_batch_server:call/3 @@ -1544,14 +1606,14 @@ exec_mem_table_delete(#{} = Names, UId, Specs) -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). -pick_range_test() -> - Ranges1 = [{76, 90}, {50, 75}, {1, 100}], - {1, 90} = pick_range(Ranges1, undefined), +% pick_range_test() -> +% Ranges1 = [{76, 90}, {50, 75}, {1, 100}], +% {1, 90} = pick_range(Ranges1, undefined), - Ranges2 = [{76, 110}, {50, 75}, {1, 49}], - {1, 110} = pick_range(Ranges2, undefined), +% Ranges2 = [{76, 110}, {50, 75}, {1, 49}], +% {1, 110} = pick_range(Ranges2, undefined), - Ranges3 = [{25, 30}, {25, 35}, {1, 50}], - {1, 30} = pick_range(Ranges3, undefined), - ok. +% Ranges3 = [{25, 30}, {25, 35}, {1, 50}], +% {1, 30} = pick_range(Ranges3, undefined), +% ok. -endif. diff --git a/src/ra_log_pre_init.erl b/src/ra_log_pre_init.erl index ecf07c24c..3929d51ba 100644 --- a/src/ra_log_pre_init.erl +++ b/src/ra_log_pre_init.erl @@ -75,7 +75,7 @@ code_change(_OldVsn, State, _Extra) -> pre_init(System, UId) -> case ets:lookup(?ETSTBL, UId) of - [{_, _}] -> + [{_, _, _, _}] -> %% already initialised ok; [] -> diff --git a/test/coordination_SUITE.erl b/test/coordination_SUITE.erl index a21ad24fc..546425f5e 100644 --- a/test/coordination_SUITE.erl +++ b/test/coordination_SUITE.erl @@ -1243,7 +1243,7 @@ stopped_wal_causes_leader_change(Config, RecoverStrat) -> #{term := T} = ra:key_metrics(Follower), T > Term andalso (begin - P = ct_rpc:call(LeaderNode, erlang, whereis, [LeaderName]),% [ra_log_wal]), + P = ct_rpc:call(LeaderNode, erlang, whereis, [LeaderName]), is_pid(P) andalso P =/= LeaderPid end) end, 200), diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index bc39c118b..1e10494eb 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -96,7 +96,7 @@ basics(_Config) -> ok = ra:restart_server(default, KvId), {ok, #{index := LastIdx}} = ra_kv:put(KvId, <<"k3">>, <<"k3">>, 5000), {ok, #{machine := #{live_indexes := Live}, - log := #{last_index := KvIdLastIdx}}, _} = ra:member_overview(KvId), + log := #{range := {_, KvIdLastIdx}}}, _} = ra:member_overview(KvId), {ok, {Reads, _}} = ra_server_proc:read_entries(KvId, [LastIdx | Live], undefined, 1000), ?assertEqual(3, map_size(Reads)), @@ -105,7 +105,7 @@ basics(_Config) -> ok = ra_kv:add_member(?SYS, KvId2, KvId), ok = ra_lib:retry( fun () -> - {ok, #{log := #{last_index := Last}}, _} = + {ok, #{log := #{range := {_, Last}}}, _} = ra:member_overview(KvId2), Last >= KvIdLastIdx end, 100, 100), diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 496671abf..06a306892 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -585,7 +585,7 @@ recover_after_snapshot(Config) -> Log = ra_log_init(Config, #{min_snapshot_interval => 1}), Overview = ra_log:overview(Log), ra_log:close(Log), - ?assertMatch(#{last_index := 2, + ?assertMatch(#{range := undefined, last_term := 1, snapshot_index := 2, last_written_index_term := {2, 1}}, Overview), @@ -604,8 +604,7 @@ writes_lower_than_snapshot_index_are_dropped(Config) -> Log4 = deliver_all_log_events(Log3, 500), Overview = ra_log:overview(Log4), - ?assertMatch(#{last_index := 499, - first_index := 101, + ?assertMatch(#{range := {101, 499}, mem_table_range := {101, 499}, last_written_index_term := {100, 1}}, Overview), @@ -632,8 +631,7 @@ writes_lower_than_snapshot_index_are_dropped(Config) -> ct:fail("expected log event not received") end, OverviewAfter = ra_log:overview(Log5), - ?assertMatch(#{last_index := 499, - first_index := 101, + ?assertMatch(#{range := {101, 499}, snapshot_index := 100, mem_table_range := {101, 499}, last_written_index_term := {499, 1}}, OverviewAfter), @@ -1365,6 +1363,7 @@ update_release_cursor(Config) -> UId = ?config(uid, Config), 127 = ra_log_snapshot_state:snapshot(ra_log_snapshot_state, UId), % this should delete a single segment + ct:pal("Log3 ~p", [ra_log:overview(Log3)]), 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(), From 645f2b14ab93270c7927d3040e7ecea2b17fdc73 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 29 May 2025 09:10:14 +0100 Subject: [PATCH 29/60] add assertion on log state --- src/ra_log.erl | 27 ++++++++++++++++++++++++--- test/coordination_SUITE.erl | 3 +-- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 19158f8b3..bc30c485a 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -7,6 +7,7 @@ %% @hidden -module(ra_log). +-include_lib("stdlib/include/assert.hrl"). -compile([inline_list_funcs]). -export([pre_init/1, @@ -50,7 +51,8 @@ delete_everything/1, release_resources/3, - tick/2 + tick/2, + assert/1 ]). -include("ra.hrl"). @@ -335,8 +337,7 @@ init(#{uid := UId, {SnapIdx, SnapTerm}, State#?MODULE.last_written_index_term ]), - State. - % element(1, delete_segments(SnapIdx, State)). + assert(State). -spec close(state()) -> ok. close(#?MODULE{cfg = #cfg{uid = _UId}, @@ -1086,6 +1087,26 @@ tick(Now, #?MODULE{cfg = #cfg{wal = Wal}, State end. +assert(#?MODULE{cfg = #cfg{log_id = LogId}, + range = Range, + snapshot_state = SnapState, + current_snapshot = CurrSnap, + live_indexes = LiveIndexes, + mem_table = _Mt + } = State) -> + %% TODO: remove this at some point? + ?DEBUG("~ts: ra_log: asserting Range ~p Snapshot ~p LiveIndexes ~p", + [LogId, Range, CurrSnap, LiveIndexes]), + %% perform assertions to ensure log state is correct + ?assert(CurrSnap =:= ra_snapshot:current(SnapState)), + ?assert(Range == undefined orelse + CurrSnap == undefined orelse + element(1, Range) - 1 == element(1, CurrSnap)), + ?assert(CurrSnap == undefined orelse + LiveIndexes == [] orelse + ra_seq:last(LiveIndexes) =< element(1, CurrSnap)), + State. + suggest_snapshot0(SnapKind, Idx, Cluster, MacModule, MacState, State0) -> case should_snapshot(SnapKind, Idx, State0) of true -> diff --git a/test/coordination_SUITE.erl b/test/coordination_SUITE.erl index 546425f5e..034ed8c4c 100644 --- a/test/coordination_SUITE.erl +++ b/test/coordination_SUITE.erl @@ -388,7 +388,7 @@ shrink_cluster_with_snapshot(Config) -> %% resume activity ok PrivDir = ?config(data_dir, Config), ClusterName = ?config(cluster_name, Config), - Peers = start_peers([s1,s2,s3], PrivDir), + Peers = start_peers([s1, s2, s3], PrivDir), ServerIds = server_ids(ClusterName, Peers), [_A, _B, _C] = ServerIds, @@ -410,7 +410,6 @@ shrink_cluster_with_snapshot(Config) -> exit(Pid, kill), {ok, _, _} = ra:remove_member(Leader1, Leader1), - timer:sleep(500), {ok, _, Leader2} = ra:members(ServerIds), From 99fb32b0c2d1b682daecb84287a11c112aa9f9e0 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 2 Jun 2025 08:42:06 +0100 Subject: [PATCH 30/60] Delete overwritten segment files when detected. --- src/ra_lib.erl | 19 +++---- src/ra_log.erl | 103 ++++++++++++++++++++++-------------- src/ra_log_reader.erl | 12 +++-- test/ra_log_2_SUITE.erl | 113 ++++++++++++++++++++++++++++------------ 4 files changed, 162 insertions(+), 85 deletions(-) diff --git a/src/ra_lib.erl b/src/ra_lib.erl index 7571ea070..a9235832e 100644 --- a/src/ra_lib.erl +++ b/src/ra_lib.erl @@ -478,19 +478,11 @@ consult(Path) -> Err end. +-spec cons(term(), list()) -> list(). cons(Item, List) when is_list(List) -> [Item | List]. -tokens(Str) -> - case erl_scan:string(Str) of - {ok, Tokens, _EndLoc} -> - erl_parse:parse_term(Tokens); - {error, Err, _ErrLoc} -> - {error, Err} - end. - - %% raw copy of ensure_dir ensure_dir("/") -> ok; @@ -517,6 +509,15 @@ ensure_dir(F) -> end end. +tokens(Str) -> + case erl_scan:string(Str) of + {ok, Tokens, _EndLoc} -> + erl_parse:parse_term(Tokens); + {error, Err, _ErrLoc} -> + {error, Err} + end. + + -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/src/ra_log.erl b/src/ra_log.erl index bc30c485a..05d0d4019 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -230,9 +230,17 @@ init(#{uid := UId, SegRefs = my_segrefs(UId, SegWriter), Reader = ra_log_reader:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, Names, Counter), - SegRange = ra_log_reader:range(Reader), + SegmentRange = ra_log_reader:range(Reader), %% TODO: check ra_range:add/2 actually performas the correct logic we expect - Range = ra_range:add(MtRange, SegRange), + Range = ra_range:add(MtRange, SegmentRange), + + %% TODO: review this + [begin + ?DEBUG("~ts: deleting overwritten segment ~w", + [LogId, SR]), + catch prim_file:delete(filename:join(Dir, F)) + end + || {_, F} = SR <- SegRefs -- ra_log_reader:segment_refs(Reader)], %% assert there is no gap between the snapshot %% and the first index in the log @@ -298,10 +306,10 @@ init(#{uid := UId, {_, LI} -> fetch_term(LI, State0) end, - LastSegRefIdx = case SegRefs of - [] -> + LastSegRefIdx = case SegmentRange of + undefined -> -1; - [{{_, L}, _} | _] -> + {_, L} -> L end, LastWrittenIdx = case ra_log_wal:last_writer_seq(Wal, UId) of @@ -311,7 +319,8 @@ init(#{uid := UId, {ok, Idx} -> max(Idx, LastSegRefIdx); {error, wal_down} -> - ?ERROR("~ts: ra_log:init/1 cannot complete as wal process is down.", + ?ERROR("~ts: ra_log:init/1 cannot complete as wal" + " process is down.", [State2#?MODULE.cfg#cfg.log_id]), exit(wal_down) end, @@ -332,9 +341,7 @@ init(#{uid := UId, State = maybe_append_first_entry(State4), ?DEBUG("~ts: ra_log:init recovered last_index_term ~w" " snapshot_index_term ~w, last_written_index_term ~w", - [State#?MODULE.cfg#cfg.log_id, - last_index_term(State), - {SnapIdx, SnapTerm}, + [LogId, last_index_term(State), {SnapIdx, SnapTerm}, State#?MODULE.last_written_index_term ]), assert(State). @@ -767,11 +774,15 @@ handle_event({written, Term, WrittenSeq}, end end; handle_event({segments, TidRanges, NewSegs}, - #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, + #?MODULE{cfg = #cfg{uid = UId, + log_id = LogId, + directory = Dir, + names = Names} = Cfg, reader = Reader0, pending = Pend0, mem_table = Mt0} = State0) -> - Reader = ra_log_reader:update_segments(NewSegs, Reader0), + {Reader, OverwrittenSegRefs} = ra_log_reader:update_segments(NewSegs, Reader0), + put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, ra_log_reader:segment_ref_count(Reader)), %% the tid ranges arrive in the reverse order they were written @@ -782,6 +793,7 @@ 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 %% @@ -795,7 +807,16 @@ handle_event({segments, TidRanges, NewSegs}, State = State0#?MODULE{reader = Reader, pending = Pend, mem_table = Mt}, - {State, []}; + Fun = fun () -> + [begin + ?DEBUG("~ts: deleting overwritten segment ~w", + [LogId, SR]), + catch prim_file:delete(filename:join(Dir, F)) + end + || {_, F} = SR <- OverwrittenSegRefs], + ok + end, + {State, [{bg_work, Fun, fun (_Err) -> ok end}]}; handle_event({segments_to_be_deleted, SegRefs}, #?MODULE{cfg = #cfg{uid = UId, log_id = LogId, @@ -803,11 +824,6 @@ handle_event({segments_to_be_deleted, SegRefs}, 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 @@ -815,6 +831,11 @@ handle_event({segments_to_be_deleted, SegRefs}, ?DEBUG("~ts: ~b obsolete segments - remaining: ~b", [LogId, length(SegRefs), length(ActiveSegs)]), %% open a new segment with the new max open segment value + Fun = fun () -> + [prim_file:delete(filename:join(Dir, F)) + || {_, F} <- SegRefs], + ok + end, {State#?MODULE{reader = ra_log_reader:init(UId, Dir, MaxOpenSegments, random, ActiveSegs, Names, Counter)}, @@ -824,8 +845,6 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, range = {FstIdx, _} = Range, - % first_index = FstIdx, - % last_index = LstIdx, mem_table = Mt0, pending = Pend0, last_written_index_term = {LastWrittenIdx, _} = LWIdxTerm0, @@ -1307,8 +1326,9 @@ release_resources(MaxOpenSegments, %%% Local functions -schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{}, - snapshot_state = SnapState, +schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{uid = _UId, + segment_writer = _SegWriter}, + live_indexes = LiveIndexes, reader = Reader0}) -> case ra_log_reader:segment_refs(Reader0) of [] -> @@ -1320,29 +1340,34 @@ schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{}, SegRefs = lists:takewhile(fun ({{_Start, End}, _}) -> End =< SnapIdx end, lists:reverse(Compactable)), - SnapDir = ra_snapshot:current_snapshot_dir(SnapState), + % SnapDir = ra_snapshot:current_snapshot_dir(SnapState), %% TODO: minor compactions should also delete / truncate %% segments with completely overwritten indexes 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, + Fun = + fun () -> + % {ok, Indexes} = ra_snapshot:indexes(SnapDir), + + %% get all current segrefs + % AllSegRefs = my_segrefs(UId, SegWriter), + Delete = lists:foldl( + fun({Range, _} = S, Del) -> + case ra_seq:in_range(Range, + LiveIndexes) of + [] -> + [S | Del]; + _ -> + Del + 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) -> ?WARN("bgwork err ~p", [_Err]), ok diff --git a/src/ra_log_reader.erl b/src/ra_log_reader.erl index d632fa4a0..254f5fbf3 100644 --- a/src/ra_log_reader.erl +++ b/src/ra_log_reader.erl @@ -103,14 +103,17 @@ close(#?STATE{open_segments = Open}) -> _ = ra_flru:evict_all(Open), ok. --spec update_segments([segment_ref()], state()) -> state(). +-spec update_segments([segment_ref()], state()) -> + {state(), OverwrittenSegments :: [segment_ref()]}. update_segments(NewSegmentRefs, - #?STATE{open_segments = Open0, + #?STATE{cfg = _Cfg, + open_segments = Open0, segment_refs = SegRefs0} = State) -> SegmentRefs0 = ra_lol:to_list(SegRefs0), %% TODO: capture segrefs removed by compact_segrefs/2 and delete them SegmentRefsComp = compact_segrefs(NewSegmentRefs, SegmentRefs0), + OverwrittenSegments = NewSegmentRefs -- SegmentRefsComp, SegmentRefsCompRev = lists:reverse(SegmentRefsComp), SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev), Range = case SegmentRefsComp of @@ -129,9 +132,10 @@ update_segments(NewSegmentRefs, error -> Acc0 end end, Open0, NewSegmentRefs), - State#?MODULE{segment_refs = SegRefs, + {State#?MODULE{segment_refs = SegRefs, range = Range, - open_segments = Open}. + open_segments = Open}, + OverwrittenSegments}. -record(log_compaction_result, {%range :: ra:range(), diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 06a306892..77e3053e3 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -67,7 +67,9 @@ all_tests() -> updated_segment_can_be_read, open_segments_limit, write_config, - sparse_write + sparse_write, + overwritten_segment_is_cleared, + overwritten_segment_is_cleared_on_init ]. groups() -> @@ -665,11 +667,7 @@ updated_segment_can_be_read(Config) -> % this should return all entries {Entries1, _} = ra_log_take(1, 15, Log4), ?assertEqual(15, length(Entries1)), - ct:pal("Entries: ~p", [Entries]), - ct:pal("Entries1: ~p", [Entries1]), - ct:pal("Counters ~p", [ra_counters:overview(?FUNCTION_NAME)]), ?assertEqual(15, length(Entries1)), - % l18 = length(Entries1), ok. cache_overwrite_then_take(Config) -> @@ -759,7 +757,6 @@ last_index_reset_before_written(Config) -> end), 4 = ra_log:next_index(Log3), {3, 1} = ra_log:last_index_term(Log3), - % #{cache_size := 0} = ra_log:overview(Log3), ok. recovery(Config) -> @@ -1606,14 +1603,59 @@ sparse_write(Config) -> {9, _, _}], ResReInit), ok. +overwritten_segment_is_cleared(Config) -> + Log0 = ra_log_init(Config, #{}), + % write a few entries + Log1 = write_and_roll(1, 256, 1, Log0), + Log2 = assert_log_events(Log1, + fun(L) -> + #{num_segments := N} = ra_log:overview(L), + N == 2 + end), + Log3 = write_and_roll(128, 256 + 128, 2, Log2), + UId = ?config(uid, Config), + Log = assert_log_events(Log3, + fun(L) -> + #{num_segments := N} = ra_log:overview(L), + N == 3 andalso + 3 == length(ra_log_segment_writer:my_segments(ra_log_segment_writer, UId)) + end), + + ct:pal("Log overview ~p", [ra_log:overview(Log)]), + ok. + +overwritten_segment_is_cleared_on_init(Config) -> + Log0 = ra_log_init(Config, #{}), + % write a few entries + Log1 = write_and_roll(1, 256, 1, Log0), + Log2 = assert_log_events(Log1, + fun(L) -> + #{num_segments := N} = ra_log:overview(L), + N == 2 + end), + Log3 = write_n(128, 256 + 128, 2, Log2), + ok = ra_log_wal:force_roll_over(ra_log_wal), + ra_log:close(Log3), + % _Log3 = write_and_roll(128, 256 + 128, 2, Log2), + UId = ?config(uid, Config), + timer:sleep(1000), + flush(), + Log = ra_log_init(Config, #{}), + + ct:pal("my segments ~p", + [ra_log_segment_writer:my_segments(ra_log_segment_writer, UId)]), + ct:pal("Log overview ~p", [ra_log:overview(Log)]), + ?assertEqual(3, length( + ra_log_segment_writer:my_segments(ra_log_segment_writer, UId))), + + ok. + validate_fold(From, To, Term, Log0) -> - {Entries0, Log} = ra_log:fold(From, To, - fun (E, A) -> [E | A] end, - [], Log0), + {Entries0, Log} = ra_log:fold(From, To, fun ra_lib:cons/2, [], Log0), ?assertEqual(To - From + 1, length(Entries0)), % validate entries are correctly read - Expected = [ {I, Term, <>} || - I <- lists:seq(To, From, -1) ], + Expected = [{I, Term, <>} || + I <- lists:seq(To, From, -1)], ?assertEqual(Expected, Entries0), Log. @@ -1728,7 +1770,11 @@ deliver_all_log_events(Log0, Timeout) -> P ! E, Acc; ({next_event, {ra_log_event, E}}, Acc0) -> - {Acc, _} = ra_log:handle_event(E, Acc0), + {Acc, Effs} = ra_log:handle_event(E, Acc0), + run_effs(Effs), + Acc; + ({bg_work, Fun, _}, Acc) -> + Fun(), Acc; (_, Acc) -> Acc @@ -1743,29 +1789,30 @@ assert_log_events(Log0, AssertPred) -> assert_log_events(Log0, AssertPred, 2000). assert_log_events(Log0, AssertPred, Timeout) -> - receive - {ra_log_event, Evt} -> - ct:pal("log evt: ~p", [Evt]), - {Log1, Effs} = ra_log:handle_event(Evt, Log0), - run_effs(Effs), - %% handle any next events - Log = lists:foldl( - fun ({next_event, {ra_log_event, E}}, Acc0) -> - {Acc, Effs} = ra_log:handle_event(E, Acc0), - run_effs(Effs), - Acc; - (_, Acc) -> - Acc - end, Log1, Effs), - case AssertPred(Log) of - true -> - Log; - false -> + case AssertPred(Log0) of + true -> + Log0; + false -> + receive + {ra_log_event, Evt} -> + ct:pal("log evt: ~p", [Evt]), + {Log1, Effs} = ra_log:handle_event(Evt, Log0), + run_effs(Effs), + %% handle any next events + Log = lists:foldl( + fun ({next_event, {ra_log_event, E}}, Acc0) -> + {Acc, Effs} = ra_log:handle_event(E, Acc0), + run_effs(Effs), + Acc; + (_, Acc) -> + Acc + end, Log1, Effs), assert_log_events(Log, AssertPred, Timeout) + + after Timeout -> + flush(), + exit({assert_log_events_timeout, Log0}) end - after Timeout -> - flush(), - exit({assert_log_events_timeout, Log0}) end. wait_for_segments(Log0, Timeout) -> From 98acc98080111001e906205dc540e7e23e38fc9d Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 2 Jun 2025 13:13:36 +0100 Subject: [PATCH 31/60] ra_log_reader -> ra_log_segments --- src/ra_log.erl | 56 +++++++++---------- src/ra_log_read_plan.erl | 2 +- ...{ra_log_reader.erl => ra_log_segments.erl} | 2 +- 3 files changed, 30 insertions(+), 30 deletions(-) rename src/{ra_log_reader.erl => ra_log_segments.erl} (99%) diff --git a/src/ra_log.erl b/src/ra_log.erl index 05d0d4019..073f3d36c 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -111,7 +111,7 @@ current_snapshot :: option(ra_idxterm()), last_resend_time :: option({integer(), WalPid :: pid() | undefined}), last_wal_write :: {pid(), Ms :: integer()}, - reader :: ra_log_reader:state(), + reader :: ra_log_segments:state(), mem_table :: ra_mt:state(), tx = false :: boolean(), pending = [] :: ra_seq:state(), @@ -120,7 +120,7 @@ -record(read_plan, {dir :: file:filename_all(), read :: #{ra_index() := log_entry()}, - plan :: ra_log_reader:read_plan()}). + plan :: ra_log_segments:read_plan()}). -opaque read_plan() :: #read_plan{}. -opaque state() :: #?MODULE{}. @@ -228,9 +228,9 @@ init(#{uid := UId, % segments it is currently processed have been finished MtRange = ra_mt:range(Mt0), SegRefs = my_segrefs(UId, SegWriter), - Reader = ra_log_reader:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, + Reader = ra_log_segments:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, Names, Counter), - SegmentRange = ra_log_reader:range(Reader), + SegmentRange = ra_log_segments:range(Reader), %% TODO: check ra_range:add/2 actually performas the correct logic we expect Range = ra_range:add(MtRange, SegmentRange), @@ -240,7 +240,7 @@ init(#{uid := UId, [LogId, SR]), catch prim_file:delete(filename:join(Dir, F)) end - || {_, F} = SR <- SegRefs -- ra_log_reader:segment_refs(Reader)], + || {_, F} = SR <- SegRefs -- ra_log_segments:segment_refs(Reader)], %% assert there is no gap between the snapshot %% and the first index in the log @@ -291,7 +291,7 @@ init(#{uid := UId, 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)), + ra_log_segments:segment_ref_count(Reader)), case ra_snapshot:latest_checkpoint(SnapshotState) of undefined -> ok; @@ -351,7 +351,7 @@ close(#?MODULE{cfg = #cfg{uid = _UId}, reader = Reader}) -> % deliberately ignoring return value % close all open segments - _ = ra_log_reader:close(Reader), + _ = ra_log_segments:close(Reader), ok. -spec begin_tx(state()) -> state(). @@ -545,11 +545,11 @@ fold(From0, To0, Fun, Acc0, MtOverlap = ra_mt:range_overlap({From, To}, Mt), case MtOverlap of {undefined, {RemStart, RemEnd}} -> - {Reader, Acc} = ra_log_reader:fold(RemStart, RemEnd, Fun, + {Reader, Acc} = ra_log_segments:fold(RemStart, RemEnd, Fun, Acc0, Reader0), {Acc, State#?MODULE{reader = Reader}}; {{MtStart, MtEnd}, {RemStart, RemEnd}} -> - {Reader, Acc1} = ra_log_reader:fold(RemStart, RemEnd, Fun, + {Reader, Acc1} = ra_log_segments:fold(RemStart, RemEnd, Fun, Acc0, Reader0), Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc1, Mt), NumRead = MtEnd - MtStart + 1, @@ -595,7 +595,7 @@ sparse_read(Indexes0, #?MODULE{cfg = Cfg, end, Indexes1), {Entries0, MemTblNumRead, Indexes} = ra_mt:get_items(Indexes2, Mt), ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, MemTblNumRead), - {Entries1, Reader} = ra_log_reader:sparse_read(Reader0, Indexes, Entries0), + {Entries1, Reader} = ra_log_segments:sparse_read(Reader0, Indexes, Entries0), %% here we recover the original order of indexes Entries = case Sort of descending -> @@ -658,7 +658,7 @@ partial_read(Indexes0, #?MODULE{cfg = Cfg, maps:put(I, TransformFun(I, T, Cmd), Acc) end, #{}, Entries0), - Plan = ra_log_reader:read_plan(Reader0, Indexes), + Plan = ra_log_segments:read_plan(Reader0, Indexes), #read_plan{dir = Cfg#cfg.directory, read = Read, plan = Plan}. @@ -666,13 +666,13 @@ partial_read(Indexes0, #?MODULE{cfg = Cfg, -spec execute_read_plan(read_plan(), undefined | ra_flru:state(), TransformFun :: transform_fun(), - ra_log_reader:read_plan_options()) -> + ra_log_segments:read_plan_options()) -> {#{ra_index() => Command :: term()}, ra_flru:state()}. execute_read_plan(#read_plan{dir = Dir, read = Read, plan = Plan}, Flru0, TransformFun, Options) -> - ra_log_reader:exec_read_plan(Dir, Plan, Flru0, TransformFun, + ra_log_segments:exec_read_plan(Dir, Plan, Flru0, TransformFun, Options, Read). -spec read_plan_info(read_plan()) -> map(). @@ -781,10 +781,10 @@ handle_event({segments, TidRanges, NewSegs}, reader = Reader0, pending = Pend0, mem_table = Mt0} = State0) -> - {Reader, OverwrittenSegRefs} = ra_log_reader:update_segments(NewSegs, Reader0), + {Reader, OverwrittenSegRefs} = ra_log_segments:update_segments(NewSegs, Reader0), put_counter(Cfg, ?C_RA_SVR_METRIC_NUM_SEGMENTS, - ra_log_reader:segment_ref_count(Reader)), + ra_log_segments:segment_ref_count(Reader)), %% the tid ranges arrive in the reverse order they were written %% (new -> old) so we need to foldr here to process the oldest first Mt = lists:foldr( @@ -824,10 +824,10 @@ handle_event({segments_to_be_deleted, SegRefs}, counter = Counter, names = Names}, reader = Reader} = State) -> - ActiveSegs = ra_log_reader:segment_refs(Reader) -- SegRefs, - #{max_size := MaxOpenSegments} = ra_log_reader:info(Reader), + ActiveSegs = ra_log_segments:segment_refs(Reader) -- SegRefs, + #{max_size := MaxOpenSegments} = ra_log_segments:info(Reader), % close all open segments - ok = ra_log_reader:close(Reader), + ok = ra_log_segments: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 @@ -836,7 +836,7 @@ handle_event({segments_to_be_deleted, SegRefs}, || {_, F} <- SegRefs], ok end, - {State#?MODULE{reader = ra_log_reader:init(UId, Dir, MaxOpenSegments, + {State#?MODULE{reader = ra_log_segments:init(UId, Dir, MaxOpenSegments, random, ActiveSegs, Names, Counter)}, @@ -970,7 +970,7 @@ fetch_term(Idx, #?MODULE{mem_table = Mt, when ?IS_IN_RANGE(Idx, Range) -> case ra_mt:lookup_term(Idx, Mt) of undefined -> - {Term, Reader} = ra_log_reader:fetch_term(Idx, Reader0), + {Term, Reader} = ra_log_segments:fetch_term(Idx, Reader0), {Term, State0#?MODULE{reader = Reader}}; Term when is_integer(Term) -> {Term, State0} @@ -1172,7 +1172,7 @@ should_snapshot(snapshot, Idx, % We should take a snapshot if the new snapshot index would allow us % to discard any segments or if the we've handled enough commands % since the last snapshot. - CanFreeSegments = case ra_log_reader:range(Reader) of + CanFreeSegments = case ra_log_segments:range(Reader) of undefined -> false; {Start, _End} -> @@ -1239,9 +1239,9 @@ overview(#?MODULE{range = Range, last_index => LastIndex, last_term => LastTerm, last_written_index_term => LWIT, - num_segments => ra_log_reader:segment_ref_count(Reader), - segments_range => ra_log_reader:range(Reader), - open_segments => ra_log_reader:num_open_segments(Reader), + num_segments => ra_log_segments:segment_ref_count(Reader), + segments_range => ra_log_segments:range(Reader), + open_segments => ra_log_segments:num_open_segments(Reader), snapshot_index => case CurrSnap of undefined -> undefined; {I, _} -> I @@ -1313,12 +1313,12 @@ release_resources(MaxOpenSegments, counter = Counter, names = Names}, reader = Reader} = State) -> - ActiveSegs = ra_log_reader:segment_refs(Reader), + ActiveSegs = ra_log_segments:segment_refs(Reader), % close all open segments % deliberately ignoring return value - _ = ra_log_reader:close(Reader), + _ = ra_log_segments:close(Reader), %% open a new segment with the new max open segment value - State#?MODULE{reader = ra_log_reader:init(UId, Dir, MaxOpenSegments, + State#?MODULE{reader = ra_log_segments:init(UId, Dir, MaxOpenSegments, AccessPattern, ActiveSegs, Names, Counter)}. @@ -1330,7 +1330,7 @@ schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{uid = _UId, segment_writer = _SegWriter}, live_indexes = LiveIndexes, reader = Reader0}) -> - case ra_log_reader:segment_refs(Reader0) of + case ra_log_segments:segment_refs(Reader0) of [] -> []; [_ | Compactable] -> diff --git a/src/ra_log_read_plan.erl b/src/ra_log_read_plan.erl index 16a7148f5..20a187618 100644 --- a/src/ra_log_read_plan.erl +++ b/src/ra_log_read_plan.erl @@ -18,7 +18,7 @@ execute(Plan, Flru) -> file_advise => normal}). -spec execute(ra_log:read_plan(), undefined | ra_flru:state(), - ra_log_reader:read_plan_options()) -> + ra_log_segments:read_plan_options()) -> {#{ra:index() => Command :: term()}, ra_flru:state()}. execute(Plan, Flru, Options) -> ra_log:execute_read_plan(Plan, Flru, diff --git a/src/ra_log_reader.erl b/src/ra_log_segments.erl similarity index 99% rename from src/ra_log_reader.erl rename to src/ra_log_segments.erl index 254f5fbf3..1e46a7917 100644 --- a/src/ra_log_reader.erl +++ b/src/ra_log_segments.erl @@ -4,7 +4,7 @@ %% %% Copyright (c) 2017-2023 Broadcom. All Rights Reserved. The term Broadcom refers to Broadcom Inc. and/or its subsidiaries. %% --module(ra_log_reader). +-module(ra_log_segments). -compile(inline_list_funcs). From 3214e42666013b6651fce015fe4144e712ff7ff2 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 2 Jun 2025 21:00:50 +0100 Subject: [PATCH 32/60] Change SegRef structure From {ra:range(), list()} -> {binary(), ra:range()} --- docs/internals/COMPACTION.md | 2 +- src/ra_log.erl | 22 ++-- src/ra_log_segment.erl | 2 +- src/ra_log_segment_writer.erl | 6 +- src/ra_log_segments.erl | 154 ++++++++++++++------------- test/ra_log_2_SUITE.erl | 2 +- test/ra_log_segment_SUITE.erl | 2 +- test/ra_log_segment_writer_SUITE.erl | 58 +++++----- 8 files changed, 126 insertions(+), 122 deletions(-) diff --git a/docs/internals/COMPACTION.md b/docs/internals/COMPACTION.md index cd3f9e150..aa5dbe8ea 100644 --- a/docs/internals/COMPACTION.md +++ b/docs/internals/COMPACTION.md @@ -75,7 +75,7 @@ higher numbered segments (003, 004) will then have a symlink created (e.g. 003.l pointing to the lowest numbered segment (002) then the link is renamed to the source file: `003.link -> 003` (NB not atomic). -`002-003-004.compacting` is then deleted (but 002 is still hard linked so the data +`001-002-003.compacting` is then deleted (but 002 is still hard linked so the data will remain). This naming format means it is easy to identify partially compacted segments diff --git a/src/ra_log.erl b/src/ra_log.erl index 073f3d36c..5f31ae4bb 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -64,7 +64,7 @@ -define(WAL_RESEND_TIMEOUT, 5000). -type ra_meta_key() :: atom(). --type segment_ref() :: {ra_range:range(), File :: file:filename_all()}. +-type segment_ref() :: {File :: binary(), ra_range:range()}. -type event_body() :: {written, ra_term(), ra_seq:state()} | {segments, [{ets:tid(), ra:range()}], [segment_ref()]} | {resend_write, ra_index()} | @@ -234,13 +234,13 @@ init(#{uid := UId, %% TODO: check ra_range:add/2 actually performas the correct logic we expect Range = ra_range:add(MtRange, SegmentRange), - %% TODO: review this + %% TODO: review thi [begin ?DEBUG("~ts: deleting overwritten segment ~w", [LogId, SR]), catch prim_file:delete(filename:join(Dir, F)) end - || {_, F} = SR <- SegRefs -- ra_log_segments:segment_refs(Reader)], + || {F, _} = SR <- SegRefs -- ra_log_segments:segment_refs(Reader)], %% assert there is no gap between the snapshot %% and the first index in the log @@ -813,7 +813,7 @@ handle_event({segments, TidRanges, NewSegs}, [LogId, SR]), catch prim_file:delete(filename:join(Dir, F)) end - || {_, F} = SR <- OverwrittenSegRefs], + || {F, _} = SR <- OverwrittenSegRefs], ok end, {State, [{bg_work, Fun, fun (_Err) -> ok end}]}; @@ -833,12 +833,12 @@ handle_event({segments_to_be_deleted, SegRefs}, %% open a new segment with the new max open segment value Fun = fun () -> [prim_file:delete(filename:join(Dir, F)) - || {_, F} <- SegRefs], + || {F, _} <- SegRefs], ok end, {State#?MODULE{reader = ra_log_segments:init(UId, Dir, MaxOpenSegments, - random, - ActiveSegs, Names, Counter)}, + random, + ActiveSegs, Names, Counter)}, [{bg_work, Fun, fun (_Err) -> ok end}]}; handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, @@ -1337,7 +1337,7 @@ schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{uid = _UId, %% 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}, _}) -> + SegRefs = lists:takewhile(fun ({_Fn, {_Start, End}}) -> End =< SnapIdx end, lists:reverse(Compactable)), % SnapDir = ra_snapshot:current_snapshot_dir(SnapState), @@ -1348,12 +1348,8 @@ schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{uid = _UId, Self = self(), Fun = fun () -> - % {ok, Indexes} = ra_snapshot:indexes(SnapDir), - - %% get all current segrefs - % AllSegRefs = my_segrefs(UId, SegWriter), Delete = lists:foldl( - fun({Range, _} = S, Del) -> + fun({_Fn, Range} = S, Del) -> case ra_seq:in_range(Range, LiveIndexes) of [] -> diff --git a/src/ra_log_segment.erl b/src/ra_log_segment.erl index 218e01dbd..ab47e8c0f 100644 --- a/src/ra_log_segment.erl +++ b/src/ra_log_segment.erl @@ -446,7 +446,7 @@ segref(#state{range = undefined}) -> undefined; segref(#state{range = Range, cfg = #cfg{filename = Fn}}) -> - {Range, filename:basename(Fn)}; + {ra_lib:to_binary(filename:basename(Fn)), Range}; segref(Filename) -> {ok, Seg} = open(Filename, #{mode => read}), SegRef = segref(Seg), diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index 0447801df..87e97d4b1 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -184,7 +184,7 @@ handle_cast({mem_tables, UIdTidRanges, WalFile}, "~s in ~bms", [System, length(RangesList), WalFile, Diff]), {noreply, State}; -handle_cast({truncate_segments, Who, {_Range, Name} = SegRef}, +handle_cast({truncate_segments, Who, {Name, _Range} = SegRef}, #state{segment_conf = SegConf, system = System} = State0) -> %% remove all segments below the provided SegRef @@ -193,7 +193,7 @@ handle_cast({truncate_segments, Who, {_Range, Name} = SegRef}, Files = segments_for(Who, State0), {_Keep, Discard} = lists:splitwith( fun (F) -> - ra_lib:to_string(filename:basename(F)) =/= Name + ra_lib:to_binary(filename:basename(F)) =/= Name end, lists:reverse(Files)), case Discard of [] -> @@ -304,7 +304,7 @@ flush_mem_table_ranges({ServerUId, TidSeqs0}, %% order they are kept by the ra_log SegRefs = lists:reverse( lists:foldl( - fun ({_, FILE}, [{_, FILE} | _] = Acc) -> + fun ({FILE, _}, [{FILE, _} | _] = Acc) -> Acc; (Seg, Acc) -> [Seg | Acc] diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index 1e46a7917..f1455a9ab 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -81,8 +81,8 @@ init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, #{}, Counter) end, SegRefs = compact_segrefs(SegRefs0, []), Range = case SegRefs of - [{{_, L}, _} | _] -> - {{F, _}, _} = lists:last(SegRefs), + [{_, {_, L}} | _] -> + {_, {F, _}} = lists:last(SegRefs), ra_range:new(F, L); _ -> undefined @@ -95,7 +95,7 @@ init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, #{}, Counter) segment_refs = ra_lol:from_list(fun seg_ref_gt/2, SegRefsRev)}. -seg_ref_gt({{Start, _}, Fn1}, {{_, End}, Fn2}) -> +seg_ref_gt({Fn1, {Start, _}}, {Fn2, {_, End}}) -> Start > End andalso Fn1 > Fn2. -spec close(state()) -> ok. @@ -117,8 +117,8 @@ update_segments(NewSegmentRefs, SegmentRefsCompRev = lists:reverse(SegmentRefsComp), SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev), Range = case SegmentRefsComp of - [{{_, L}, _} | _] -> - [{{F, _}, _} | _] = SegmentRefsCompRev, + [{_, {_, L}} | _] -> + [{_, {F, _}} | _] = SegmentRefsCompRev, ra_range:new(F, L); _ -> undefined @@ -126,7 +126,7 @@ update_segments(NewSegmentRefs, %% check if any of the updated segrefs refer to open segments %% we close these segments so that they can be re-opened with updated %% indexes if needed - Open = lists:foldl(fun ({_, Fn}, Acc0) -> + Open = lists:foldl(fun ({Fn, _}, Acc0) -> case ra_flru:evict(Fn, Acc0) of {_, Acc} -> Acc; error -> Acc0 @@ -139,26 +139,30 @@ update_segments(NewSegmentRefs, -record(log_compaction_result, {%range :: ra:range(), - deleted :: [segment_ref()], - new :: [segment_ref()]}). + unreferenced :: [segment_ref()], + linked :: [segment_ref()], + compacted :: [segment_ref()]}). + -spec handle_compaction(#log_compaction_result{}, state()) -> state(). -handle_compaction(#log_compaction_result{deleted = Deleted, - new = New}, +handle_compaction(#log_compaction_result{unreferenced = Deleted, + linked = Linked, + compacted = Compacted}, #?STATE{open_segments = Open0, segment_refs = SegRefs0} = State) -> SegmentRefs0 = ra_lol:to_list(SegRefs0), - SegmentRefs = lists:sort((SegmentRefs0 -- Deleted) ++ New), + SegmentRefs = lists:usort(((SegmentRefs0 -- Deleted) -- Linked) ++ Compacted), Open = ra_flru:evict_all(Open0), State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, lists:reverse(SegmentRefs)), open_segments = Open}. + -spec update_first_index(ra_index(), state()) -> {state(), [segment_ref()]}. update_first_index(FstIdx, #?STATE{segment_refs = SegRefs0, open_segments = OpenSegs0} = State) -> %% TODO: refactor this so that ra_lol just returns plain lists on both sides? - case ra_lol:takewhile(fun({{_, To}, _}) -> + case ra_lol:takewhile(fun({_Fn, {_, To}}) -> To >= FstIdx end, SegRefs0) of {Active, Obsolete0} -> @@ -167,7 +171,7 @@ update_first_index(FstIdx, #?STATE{segment_refs = SegRefs0, {State, []}; _ -> Obsolete = ra_lol:to_list(Obsolete0), - ObsoleteKeys = [K || {_, K} <- Obsolete], + ObsoleteKeys = [K || {K, _} <- Obsolete], % close any open segments OpenSegs = lists:foldl(fun (K, OS0) -> case ra_flru:evict(K, OS0) of @@ -237,7 +241,7 @@ exec_read_plan(Dir, Plan, Open0, TransformFun, Options, Acc0) Acc#{I => E} end, lists:foldl( - fun ({Idxs, BaseName}, {Acc1, Open1}) -> + fun ({BaseName, Idxs}, {Acc1, Open1}) -> {Seg, Open2} = get_segment_ext(Dir, Open1, BaseName, Options), case ra_log_segment:read_sparse(Seg, Idxs, Fun, Acc1) of {ok, _, Acc} -> @@ -276,14 +280,14 @@ segment_read_plan(_SegRefs, [], Acc) -> lists:reverse(Acc); segment_read_plan(SegRefs, [Idx | _] = Indexes, Acc) -> case ra_lol:search(seg_ref_search_fun(Idx), SegRefs) of - {{Range, Fn}, Cont} -> + {{Fn, Range}, Cont} -> case sparse_read_split(fun (I) -> ra_range:in(I, Range) end, Indexes, []) of {[], _} -> segment_read_plan(Cont, Indexes, Acc); {Idxs, Rem} -> - segment_read_plan(Cont, Rem, [{Idxs, Fn} | Acc]) + segment_read_plan(Cont, Rem, [{Fn, Idxs} | Acc]) end; undefined -> %% not found @@ -291,7 +295,7 @@ segment_read_plan(SegRefs, [Idx | _] = Indexes, Acc) -> end. seg_ref_search_fun(Idx) -> - fun({{Start, End}, _}) -> + fun({__Fn, {Start, End}}) -> if Idx > End -> higher; Idx < Start -> lower; true -> equal @@ -308,7 +312,7 @@ segment_term_query0(Idx, SegRefs, Open0, #cfg{directory = Dir, access_pattern = AccessPattern} = Cfg) -> case ra_lol:search(seg_ref_search_fun(Idx), SegRefs) of - {{_Range, Fn}, _Cont} -> + {{Fn, _Range}, _Cont} -> case ra_flru:fetch(Fn, Open0) of {ok, Seg, Open} -> Term = ra_log_segment:term_query(Seg, Idx), @@ -331,7 +335,7 @@ segment_fold_plan(_SegRefs, undefined, Acc) -> Acc; segment_fold_plan(SegRefs, {_ReqStart, ReqEnd} = ReqRange, Acc) -> case ra_lol:search(seg_ref_search_fun(ReqEnd), SegRefs) of - {{Range, Fn}, Cont} -> + {{Fn, Range}, Cont} -> This = ra_range:overlap(ReqRange, Range), ReqRem = case ra_range:subtract(This, ReqRange) of [] -> @@ -339,7 +343,7 @@ segment_fold_plan(SegRefs, {_ReqStart, ReqEnd} = ReqRange, Acc) -> [Rem] -> Rem end, - segment_fold_plan(Cont, ReqRem, [{This, Fn} | Acc]); + segment_fold_plan(Cont, ReqRem, [{Fn, This} | Acc]); undefined -> %% not found Acc @@ -352,12 +356,11 @@ segment_fold(#?STATE{segment_refs = SegRefs, Plan = segment_fold_plan(SegRefs, {RStart, REnd}, []), {Op, A} = lists:foldl( - fun ({{Start, End}, Fn}, {Open0, Ac0}) -> + fun ({Fn, {Start, End}}, {Open0, Ac0}) -> {Seg, Open} = get_segment(Cfg, Open0, Fn), {Open, ra_log_segment:fold(Seg, Start, End, fun binary_to_term/1, - Fun, - Ac0)} + Fun, Ac0)} end, {OpenSegs, Acc}, Plan), {State#?MODULE{open_segments = Op}, A}. @@ -369,7 +372,7 @@ segment_sparse_read(#?STATE{segment_refs = SegRefs, cfg = Cfg}, Indexes, Entries0) -> Plan = segment_read_plan(SegRefs, Indexes, []), lists:foldl( - fun ({Idxs, Fn}, {Open0, C, En0}) -> + fun ({Fn, Idxs}, {Open0, C, En0}) -> {Seg, Open} = get_segment(Cfg, Open0, Fn), {ok, ReadSparseCount, Entries} = ra_log_segment:read_sparse_no_checks( @@ -392,7 +395,8 @@ sparse_read_split(_Fun, [], Acc) -> get_segment(#cfg{directory = Dir, - access_pattern = AccessPattern} = Cfg, Open0, Fn) -> + access_pattern = AccessPattern} = Cfg, Open0, Fn) + when is_binary(Fn) -> case ra_flru:fetch(Fn, Open0) of {ok, S, Open1} -> {S, Open1}; @@ -434,18 +438,18 @@ compact_segrefs(New, Cur) -> fun (S, []) -> [S]; - ({{Start, _}, _} = SegRef, Prev) -> + ({_, {Start, _}} = SegRef, Prev) -> [SegRef | limit(Start, Prev)] end, Cur, New). limit(_LimitIdx, []) -> []; -limit(LimitIdx, [{PrevRange, PrevFn} | PrevRem]) -> +limit(LimitIdx, [{PrevFn, PrevRange} | PrevRem]) -> case ra_range:limit(LimitIdx, PrevRange) of undefined -> limit(LimitIdx, PrevRem); NewPrevRange -> - [{NewPrevRange, PrevFn} | PrevRem] + [{PrevFn, NewPrevRange} | PrevRem] end. reset_counter(#cfg{counter = Cnt}, Ix) @@ -467,69 +471,73 @@ decr_counter(#cfg{counter = undefined}, _, _) -> -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). +-define(SR(N, R), {<>, R}). + compact_seg_refs_test() -> - NewRefs = [{{10, 100}, "2"}], - PrevRefs = [{{10, 75}, "2"}, {{1, 9}, "1"}], - ?assertEqual([{{10, 100}, "2"}, {{1, 9}, "1"}], + NewRefs = [?SR("2", {10, 100})], + PrevRefs = [?SR("2", {10, 75}), + ?SR("1", {1, 9})], + ?assertEqual([?SR("2", {10, 100}), + ?SR("1", {1, 9})], compact_segrefs(NewRefs, PrevRefs)). compact_segref_3_test() -> Data = [ - {{2, 7}, "C"}, + {"C", {2, 7}}, %% this entry has overwritten the prior two - {{5, 10}, "B"}, - {{1, 4}, "A"} + {"B", {5, 10}}, + {"A", {1, 4}} ], Res = compact_segrefs(Data, []), - ?assertMatch([{{2, 7}, "C"}, - {{1, 1}, "A"}], Res), + ?assertMatch([{"C", {2, 7}}, + {"A", {1, 1}}], Res), ok. compact_segref_2_test() -> Data = [ - {{80, 89}, "80"}, + {"80", {80, 89}}, %% this entry has overwritten the prior two - {{56, 79}, "71"}, - {{70, 85}, "70"}, - {{60, 69}, "60"}, - {{50, 59}, "50"} + {"71", {56, 79}}, + {"70", {70, 85}}, + {"60", {60, 69}}, + {"50", {50, 59}} ], Res = compact_segrefs(Data, []), - ?assertMatch([{{80, 89}, "80"}, - {{56, 79}, "71"}, - {{50, 55}, "50"} + ?assertMatch([{"80", {80, 89}}, + {"71", {56, 79}}, + {"50", {50, 55}} ], Res), ok. compact_segref_1_test() -> Data = [ - {{80, 89}, "80"}, + {"80", {80, 89}}, %% this entry has overwritten the prior one - {{70, 79}, "71"}, - {{70, 85}, "70"}, + {"71", {70, 79}}, + {"70", {70, 85}}, %% partial overwrite - {{65, 69}, "65"}, - {{60, 69}, "60"}, - {{50, 59}, "50"}, - {{40, 49}, "40"} + {"65", {65, 69}}, + {"60", {60, 69}}, + {"50", {50, 59}}, + {"40", {40, 49}} ], Res = compact_segrefs(Data, [ - {{30, 39}, "30"}, - {{20, 29}, "20"} + {"30", {30, 39}}, + {"20", {20, 29}} ]), %% overwritten entry is no longer there %% and the segment prior to the partial overwrite has been limited %% to provide a continuous range - ?assertMatch([{{80, 89}, "80"}, - {{70, 79}, "71"}, - {{65, 69}, "65"}, - {{60, 64}, "60"}, - {{50, 59}, "50"}, - {{40, 49}, "40"}, - {{30, 39}, "30"}, - {{20, 29}, "20"} + ?assertMatch([{"80", {80, 89}}, + {"71", {70, 79}}, + {"65", {65, 69}}, + {"60", {60, 64}}, + {"50", {50, 59}}, + {"40", {40, 49}}, + {"30", {30, 39}}, + {"20", {20, 29}} ], Res), ok. @@ -540,26 +548,26 @@ segrefs_to_read_test() -> fun seg_ref_gt/2, lists:reverse( compact_segrefs( - [{{412,499},"00000006.segment"}, - {{284,411},"00000005.segment"}, + [{"00000006.segment", {412, 499}}, + {"00000005.segment", {284, 411}}, %% this segment got overwritten - {{284,500},"00000004.segment"}, - {{200,285},"00000003.segment"}, - {{128,255},"00000002.segment"}, - {{0,127},"00000001.segment"}], []))), + {"00000004.segment",{284, 500}}, + {"00000003.segment",{200, 285}}, + {"00000002.segment",{128, 255}}, + {"00000001.segment", {0, 127}}], []))), - ?assertEqual([{{199, 199}, "00000002.segment"}, - {{200, 283}, "00000003.segment"}, - {{284, 411}, "00000005.segment"}, - {{412, 499}, "00000006.segment"}], + ?assertEqual([{"00000002.segment", {199, 199}}, + {"00000003.segment", {200, 283}}, + {"00000005.segment", {284, 411}}, + {"00000006.segment", {412, 499}}], segment_fold_plan(SegRefs, {199, 499}, [])), %% out of range ?assertEqual([], segment_fold_plan(SegRefs, {500, 500}, [])), ?assertEqual([ - {{127,127},"00000001.segment"}, - {{128,128},"00000002.segment"} + {"00000001.segment", {127,127}}, + {"00000002.segment", {128,128}} ], segment_fold_plan(SegRefs, {127, 128}, [])), ok. diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 77e3053e3..c0ec94b27 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -1360,7 +1360,7 @@ update_release_cursor(Config) -> UId = ?config(uid, Config), 127 = ra_log_snapshot_state:snapshot(ra_log_snapshot_state, UId), % this should delete a single segment - ct:pal("Log3 ~p", [ra_log:overview(Log3)]), + ct:pal("Log3 ~p", [Log3]), 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(), diff --git a/test/ra_log_segment_SUITE.erl b/test/ra_log_segment_SUITE.erl index 270c7d2ce..6153360ac 100644 --- a/test/ra_log_segment_SUITE.erl +++ b/test/ra_log_segment_SUITE.erl @@ -208,7 +208,7 @@ segref(Config) -> {ok, Seg0} = ra_log_segment:open(Fn, #{max_count => 128}), undefined = ra_log_segment:segref(Seg0), {ok, Seg1} = ra_log_segment:append(Seg0, 1, 2, <<"Adsf">>), - {{1, 1}, "seg1.seg"} = ra_log_segment:segref(Seg1), + {<<"seg1.seg">>, {1, 1}} = ra_log_segment:segref(Seg1), ok. diff --git a/test/ra_log_segment_writer_SUITE.erl b/test/ra_log_segment_writer_SUITE.erl index 9cddcd874..93311f27c 100644 --- a/test/ra_log_segment_writer_SUITE.erl +++ b/test/ra_log_segment_writer_SUITE.erl @@ -97,7 +97,7 @@ accept_mem_tables(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), receive - {ra_log_event, {segments, TidSeqs, [{{1, 3}, SegFile}]}} -> + {ra_log_event, {segments, TidSeqs, [{SegFile, {1, 3}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), SegFile), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -137,7 +137,7 @@ accept_mem_tables_append(Config) -> make_wal(Config, "w2.wal")), AllEntries = Entries ++ Entries2, receive - {ra_log_event, {segments, [{Tid, [{4, 5}]}], [{{1, 5}, Fn}]}} -> + {ra_log_event, {segments, [{Tid, [{4, 5}]}], [{Fn, {1, 5}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -163,10 +163,10 @@ accept_mem_tables_overwrite(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), receive - {ra_log_event, {segments, [{Tid, [{3, 5}]}], [{{3, 5}, Fn}]}} -> + {ra_log_event, {segments, [{Tid, [{3, 5}]}], [{Fn, {3, 5}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), - ?assertMatch({{3, 5}, _}, ra_log_segment:segref(Seg)), + ?assertMatch({_, {3, 5}}, ra_log_segment:segref(Seg)), ra_log_segment:close(Seg), ok after 3000 -> @@ -180,10 +180,10 @@ accept_mem_tables_overwrite(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.wal")), receive - {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn2}]}} -> + {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{Fn2, {1, 3}}]}} -> SegmentFile2 = filename:join(?config(server_dir, Config), Fn2), {ok, Seg2} = ra_log_segment:open(SegmentFile2, #{mode => read}), - ?assertMatch({{1, 3}, _}, ra_log_segment:segref(Seg2)), + ?assertMatch({_, {1, 3}}, ra_log_segment:segref(Seg2)), C2 = term_to_binary(c2), [{1, 43, _}, {2, 43, _}] = read_sparse(Seg2, [1, 2]), [{3, 43, C2}] = read_sparse(Seg2, [3]), @@ -213,10 +213,10 @@ 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}]}], [{Fn, {2, 6}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), - ?assertMatch({{2, 6}, _}, ra_log_segment:segref(Seg)), + ?assertMatch({_, {2, 6}}, ra_log_segment:segref(Seg)), [{2, 42, _}, {3, 42, _}, {4, 43, _}, @@ -250,7 +250,7 @@ accept_mem_tables_multi_segment(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w.wal")), receive - {ra_log_event, {segments, TidRanges, [{{9, 10}, _Seg2}, {{1, 8}, _Seg1}]}} -> + {ra_log_event, {segments, TidRanges, [{_, {9, 10}}, {_, {1, 8}}]}} -> ok after 3000 -> flush(), @@ -277,7 +277,7 @@ accept_mem_tables_multi_segment_max_size(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w.wal")), receive - {ra_log_event, {segments, TidRanges, [{{9, 10}, _Seg2}, {{1, 8}, _Seg1}]}} -> + {ra_log_event, {segments, TidRanges, [{_, {9, 10}}, {_, {1, 8}}]}} -> ok after 3000 -> flush(), @@ -305,7 +305,7 @@ accept_mem_tables_multi_segment_overwrite(Config) -> make_wal(Config, "w.wal")), LastFile = receive - {ra_log_event, {segments, TidRanges, [{{9, 10}, Seg2}, {{1, 8}, _Seg1}]}} -> + {ra_log_event, {segments, TidRanges, [{Seg2, {9, 10}}, {_Seg1, {1, 8}}]}} -> Seg2 % ok after 3000 -> @@ -322,7 +322,7 @@ accept_mem_tables_multi_segment_overwrite(Config) -> make_wal(Config, "w2.wal")), receive {ra_log_event, {segments, TidRanges2, - [{{13, 15}, _}, {{7, 12}, LastFile}]}} -> + [{_, {13, 15}}, {LastFile, {7, 12}}]}} -> ok after 3000 -> flush(), @@ -362,7 +362,7 @@ accept_mem_tables_for_down_server(Config) -> ok = file:write_file(WalFile, <<"waldata">>), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn}]}} -> + {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{Fn, {1, 3}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -424,7 +424,7 @@ accept_mem_tables_with_deleted_server(Config) -> WalFile = make_wal(Config, "00001.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{{1, 3}, Fn}]}} -> + {ra_log_event, {segments, [{Tid2, [{1, 3}]}], [{Fn, {1, 3}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -475,7 +475,7 @@ accept_mem_tables_with_corrupt_segment(Config) -> file:write_file(filename:join(?config(server_dir, Config), "0000001.segment"), <<>>), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, TidRanges, [{{1, 3}, SegFile}]}} -> + {ra_log_event, {segments, TidRanges, [{SegFile, {1, 3}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), SegFile), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert Entries have been fully transferred @@ -516,10 +516,10 @@ accept_mem_tables_multiple_ranges(Config)-> receive {ra_log_event, {segments, TidRanges, SegRefs}} -> ?assertMatch([ - {{49, 64}, _}, - {{33, 48}, _}, - {{17, 32}, _}, - {{1, 16}, _} + {_, {49, 64}}, + {_, {33, 48}}, + {_, {17, 32}}, + {_, {1, 16}} ], SegRefs), ok after 3000 -> @@ -577,14 +577,14 @@ truncate_segments(Config) -> WalFile = make_wal(Config, "0000001.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, TidRanges, [{{25, 32}, S} = Cur | Rem]}} -> + {ra_log_event, {segments, TidRanges, [{S, {25, 32}} = Cur | Rem]}} -> % test a lower index _does not_ delete the file SegmentFile = filename:join(?config(server_dir, Config), S), ?assert(filelib:is_file(SegmentFile)), ok = ra_log_segment_writer:truncate_segments(TblWriterPid, UId, Cur), ra_log_segment_writer:await(?SEGWR), - [{_, S1}, {_, S2}] = Rem, + [{S1, _}, {S2, _}] = Rem, SegmentFile1 = filename:join(?config(server_dir, Config), S1), ?assertNot(filelib:is_file(SegmentFile1)), SegmentFile2 = filename:join(?config(server_dir, Config), S2), @@ -618,7 +618,7 @@ truncate_segments_with_pending_update(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.erl")), receive - {ra_log_event, {segments, _TidRanges, [{{25, 32}, S} = Cur | Rem]}} -> + {ra_log_event, {segments, _TidRanges, [{S, {25, 32}} = Cur | Rem]}} -> % this is the event from the first call to accept_mem_tables, % the Cur segments has been appended to since so should _not_ % be deleted when it is provided as the cutoff segref for @@ -629,7 +629,7 @@ truncate_segments_with_pending_update(Config) -> UId, Cur), ra_log_segment_writer:await(?SEGWR), ?assert(filelib:is_file(SegmentFile)), - [{_, S1}, {_, S2}] = Rem, + [{S1, _}, {S2, _}] = Rem, SegmentFile1 = filename:join(?config(server_dir, Config), S1), ?assertNot(filelib:is_file(SegmentFile1)), SegmentFile2 = filename:join(?config(server_dir, Config), S2), @@ -664,7 +664,7 @@ truncate_segments_with_pending_overwrite(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges2, make_wal(Config, "w2.wal")), receive - {ra_log_event, {segments, _Tid, [{{25, 32}, S} = Cur | Rem]}} -> + {ra_log_event, {segments, _Tid, [{S, {25, 32}} = Cur | Rem]}} -> % test a lower index _does not_ delete the file SegmentFile = filename:join(?config(server_dir, Config), S), ?assert(filelib:is_file(SegmentFile)), @@ -673,7 +673,7 @@ truncate_segments_with_pending_overwrite(Config) -> _ = ra_log_segment_writer:await(?SEGWR), SegmentFile = filename:join(?config(server_dir, Config), S), ?assert(filelib:is_file(SegmentFile)), - [{_, S1}, {_, S2}] = Rem, + [{S1, _}, {S2, _}] = Rem, SegmentFile1 = filename:join(?config(server_dir, Config), S1), ?assertNot(filelib:is_file(SegmentFile1)), SegmentFile2 = filename:join(?config(server_dir, Config), S2), @@ -685,7 +685,7 @@ truncate_segments_with_pending_overwrite(Config) -> throw(ra_log_event_timeout) end, receive - {ra_log_event, {segments, _, [{{16, 25}, F} = Cur2, {{12, 15}, F2}]}} -> + {ra_log_event, {segments, _, [{F, {16, 25}} = Cur2, {F2, {12, 15}}]}} -> ?assertMatch([_, _], segments_for(UId, Dir)), ok = ra_log_segment_writer:truncate_segments(TblWriterPid, UId, Cur2), @@ -720,7 +720,7 @@ my_segments(Config) -> WalFile = make_wal(Config, "00001.wal"), ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), receive - {ra_log_event, {segments, TidRanges, [{{1, 3}, Fn}]}} -> + {ra_log_event, {segments, TidRanges, [{Fn, {1, 3}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), [MyFile] = ra_log_segment_writer:my_segments(?SEGWR,UId), ?assertEqual(SegmentFile, unicode:characters_to_binary(MyFile)), @@ -747,7 +747,7 @@ upgrade_segment_name_format(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, WalFile), File = receive - {ra_log_event, {segments, TidRanges, [{{1, 3}, _Fn}]}} -> + {ra_log_event, {segments, TidRanges, [{_, {1, 3}}]}} -> [MyFile] = ra_log_segment_writer:my_segments(?SEGWR,UId), MyFile after 2000 -> @@ -794,7 +794,7 @@ skip_entries_lower_than_snapshot_index(Config) -> ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, make_wal(Config, "w1.wal")), receive - {ra_log_event, {segments, _Tid, [{{4, 5}, Fn}]}} -> + {ra_log_event, {segments, _Tid, [{Fn, {4, 5}}]}} -> SegmentFile = filename:join(?config(server_dir, Config), Fn), {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), % assert only entries with a higher index than the snapshot From 83d84bcaac598efcf0c3efc7738cbfdcf1de3f63 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 3 Jun 2025 09:57:02 +0100 Subject: [PATCH 33/60] wip --- src/ra_log.erl | 85 ++++++---------------------------------- src/ra_log_segments.erl | 86 +++++++++++++++++++++++++++++++++-------- src/ra_server.erl | 2 +- 3 files changed, 81 insertions(+), 92 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 5f31ae4bb..80ea658eb 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -103,8 +103,6 @@ {cfg = #cfg{}, %% mutable data below range :: ra:range(), - % first_index = -1 :: ra_index(), - % last_index = -1 :: -1 | ra_index(), last_term = 0 :: ra_term(), last_written_index_term = {0, 0} :: ra_idxterm(), snapshot_state :: ra_snapshot:state(), @@ -817,30 +815,10 @@ handle_event({segments, TidRanges, NewSegs}, ok end, {State, [{bg_work, Fun, fun (_Err) -> ok end}]}; -handle_event({segments_to_be_deleted, SegRefs}, - #?MODULE{cfg = #cfg{uid = UId, - log_id = LogId, - directory = Dir, - counter = Counter, - names = Names}, - reader = Reader} = State) -> - ActiveSegs = ra_log_segments:segment_refs(Reader) -- SegRefs, - #{max_size := MaxOpenSegments} = ra_log_segments:info(Reader), - % close all open segments - ok = ra_log_segments: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 - Fun = fun () -> - [prim_file:delete(filename:join(Dir, F)) - || {F, _} <- SegRefs], - ok - end, - {State#?MODULE{reader = ra_log_segments:init(UId, Dir, MaxOpenSegments, - random, - ActiveSegs, Names, Counter)}, - - [{bg_work, Fun, fun (_Err) -> ok end}]}; +handle_event({compaction_result, Result}, + #?MODULE{reader = Reader0} = State) -> + {Reader, Effs} = ra_log_segments:handle_compaction_result(Result, Reader0), + {State#?MODULE{reader = Reader}, Effs}; handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, @@ -904,7 +882,9 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, live_indexes = LiveIndexes, current_snapshot = Snap, snapshot_state = SnapState}, - CompEffs = schedule_compaction(SnapIdx, State), + CompEffs = ra_log_segments:schedule_compaction(minor, SnapIdx, + LiveIndexes, + State#?MODULE.reader), Effects = CompEffs ++ Effects0, {State, Effects}; checkpoint -> @@ -1021,7 +1001,9 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, LiveIndexes, live_indexes = LiveIndexes, mem_table = Mt, last_written_index_term = IdxTerm}, - CompEffs = schedule_compaction(SnapIdx, State), + CompEffs = ra_log_segments:schedule_compaction(minor, SnapIdx, + LiveIndexes, + State#?MODULE.reader), {ok, State, CompEffs ++ CPEffects}. @@ -1326,52 +1308,7 @@ release_resources(MaxOpenSegments, %%% Local functions -schedule_compaction(SnapIdx, #?MODULE{cfg = #cfg{uid = _UId, - segment_writer = _SegWriter}, - live_indexes = LiveIndexes, - reader = Reader0}) -> - case ra_log_segments: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 ({_Fn, {_Start, End}}) -> - End =< SnapIdx - end, lists:reverse(Compactable)), - % SnapDir = ra_snapshot:current_snapshot_dir(SnapState), - - %% TODO: minor compactions should also delete / truncate - %% segments with completely overwritten indexes - - Self = self(), - Fun = - fun () -> - Delete = lists:foldl( - fun({_Fn, Range} = S, Del) -> - case ra_seq:in_range(Range, - LiveIndexes) of - [] -> - [S | Del]; - _ -> - Del - 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) -> - ?WARN("bgwork err ~p", [_Err]), ok - end}] - end. - - -%% unly used by resend to wal functionality and doesn't update the mem table +%% only used by resend to wal functionality and doesn't update the mem table wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, range = _Range} = State, diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index f1455a9ab..f71817a33 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -13,7 +13,8 @@ init/7, close/1, update_segments/2, - handle_compaction/2, + schedule_compaction/4, + handle_compaction_result/2, segment_refs/1, segment_ref_count/1, range/1, @@ -137,24 +138,75 @@ update_segments(NewSegmentRefs, open_segments = Open}, OverwrittenSegments}. --record(log_compaction_result, - {%range :: ra:range(), - unreferenced :: [segment_ref()], - linked :: [segment_ref()], - compacted :: [segment_ref()]}). - --spec handle_compaction(#log_compaction_result{}, state()) -> state(). -handle_compaction(#log_compaction_result{unreferenced = Deleted, - linked = Linked, - compacted = Compacted}, - #?STATE{open_segments = Open0, - segment_refs = SegRefs0} = State) -> +-record(compaction_result, + {unreferenced = [] :: [segment_ref()], + linked = [] :: [segment_ref()], + compacted = [] :: [segment_ref()]}). + +-spec schedule_compaction(minor | major, ra:index(), + ra_seq:state(), state()) -> + [ra_server:effect()]. +schedule_compaction(minor, SnapIdx, LiveIndexes, State) -> + case ra_log_segments:segment_refs(State) of + [] -> + []; + [_ | Compactable] -> + %% never compact the current segment + %% only take those who have a range lower than the snapshot index as + %% we never want to compact more than that + SegRefs = lists:takewhile(fun ({_Fn, {_Start, End}}) -> + End =< SnapIdx + end, lists:reverse(Compactable)), + %% TODO: minor compactions should also delete / truncate + %% segments with completely overwritten indexes + + Self = self(), + Fun = fun () -> + Delete = lists:foldl( + fun({_Fn, Range} = S, Del) -> + case ra_seq:in_range(Range, + LiveIndexes) of + [] -> + [S | Del]; + _ -> + Del + end + end, [], SegRefs), + Result = #compaction_result{unreferenced = Delete}, + %% need to update the ra_servers list of seg refs _before_ + %% the segments can actually be deleted + Self ! {ra_log_event, + {compaction_result, Result}}, + ok + end, + + [{bg_work, Fun, fun (_Err) -> + ?WARN("bgwork err ~p", [_Err]), ok + end}] + end. + + +-spec handle_compaction_result(#compaction_result{}, state()) -> + {state(), [ra_server:effect()]}. +handle_compaction_result(#compaction_result{unreferenced = Unreferenced, + linked = Linked, + compacted = Compacted}, + #?STATE{cfg = #cfg{directory = Dir}, + open_segments = Open0, + segment_refs = SegRefs0} = State) -> SegmentRefs0 = ra_lol:to_list(SegRefs0), - SegmentRefs = lists:usort(((SegmentRefs0 -- Deleted) -- Linked) ++ Compacted), + SegmentRefs = lists:usort(((SegmentRefs0 -- Unreferenced) -- Linked) + ++ Compacted), Open = ra_flru:evict_all(Open0), - State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, - lists:reverse(SegmentRefs)), - open_segments = Open}. + Fun = fun () -> + [prim_file:delete(filename:join(Dir, F)) + || {F, _} <- Unreferenced], + ok + end, + {State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, + SegmentRefs), + open_segments = Open}, + [{bg_work, Fun, fun (_Err) -> ok end}]}. -spec update_first_index(ra_index(), state()) -> diff --git a/src/ra_server.erl b/src/ra_server.erl index dfc1f0ed6..631f43a70 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -178,7 +178,7 @@ %% used for tracking valid leader messages {record_leader_msg, ra_server_id()} | start_election_timeout | - {bg_work, fun(() -> ok) | mfargs()}. + {bg_work, fun(() -> ok) | mfargs(), fun()}. -type effects() :: [effect()]. From 62178674ff23796180ed772fbd92a097413bd643 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 6 Jun 2025 17:05:12 +0100 Subject: [PATCH 34/60] major compactions --- src/ra_log.erl | 68 +---- src/ra_log_segment.erl | 36 ++- src/ra_log_segments.erl | 405 +++++++++++++++++++++----- test/ra_log_2_SUITE.erl | 17 +- test/ra_log_segment_SUITE.erl | 4 - test/ra_log_segments_SUITE.erl | 507 +++++++++++++++++++++++++++++++++ 6 files changed, 898 insertions(+), 139 deletions(-) create mode 100644 test/ra_log_segments_SUITE.erl diff --git a/src/ra_log.erl b/src/ra_log.erl index 80ea658eb..f44a5a651 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -227,12 +227,11 @@ init(#{uid := UId, MtRange = ra_mt:range(Mt0), SegRefs = my_segrefs(UId, SegWriter), Reader = ra_log_segments:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, - Names, Counter), + Counter, LogId), SegmentRange = ra_log_segments:range(Reader), %% TODO: check ra_range:add/2 actually performas the correct logic we expect Range = ra_range:add(MtRange, SegmentRange), - %% TODO: review thi [begin ?DEBUG("~ts: deleting overwritten segment ~w", [LogId, SR]), @@ -1288,12 +1287,11 @@ delete_everything(#?MODULE{cfg = #cfg{uid = UId, -spec release_resources(non_neg_integer(), sequential | random, state()) -> state(). -release_resources(MaxOpenSegments, - AccessPattern, +release_resources(MaxOpenSegments, AccessPattern, #?MODULE{cfg = #cfg{uid = UId, + log_id = LogId, directory = Dir, - counter = Counter, - names = Names}, + counter = Counter}, reader = Reader} = State) -> ActiveSegs = ra_log_segments:segment_refs(Reader), % close all open segments @@ -1301,8 +1299,8 @@ release_resources(MaxOpenSegments, _ = ra_log_segments:close(Reader), %% open a new segment with the new max open segment value State#?MODULE{reader = ra_log_segments:init(UId, Dir, MaxOpenSegments, - AccessPattern, - ActiveSegs, Names, Counter)}. + AccessPattern, ActiveSegs, + Counter, LogId)}. %%% Local functions @@ -1482,45 +1480,16 @@ my_segrefs(UId, SegWriter) -> %% if a server recovered when a segment had been opened %% but never had any entries written the segref would be %% undefined - case ra_log_segment:segref(File) of - undefined -> - Acc; - SegRef -> - [SegRef | Acc] + case ra_log_segment:info(File) of + #{ref := SegRef, + file_type := regular} + when is_tuple(SegRef) -> + [SegRef | Acc]; + _ -> + Acc end end, [], SegFiles). -% recover_ranges(UId, MtRange, SegWriter) -> -% % 1. check mem_tables (this assumes wal has finished recovering -% % which means it is essential that ra_servers are part of the same -% % supervision tree -% % 2. check segments -% SegFiles = ra_log_segment_writer:my_segments(SegWriter, UId), -% SegRefs = lists:foldl( -% fun (File, Acc) -> -% %% if a server recovered when a segment had been opened -% %% but never had any entries written the segref would be -% %% undefined -% case ra_log_segment:segref(File) of -% undefined -> -% Acc; -% SegRef -> -% [SegRef | Acc] -% end -% end, [], SegFiles), -% SegRanges = [Range || {Range, _} <- SegRefs], -% Ranges = [MtRange | SegRanges], -% {pick_range(Ranges, undefined), SegRefs}. - -% picks the current range from a sorted (newest to oldest) list of ranges -% pick_range([], Res) -> -% Res; -% pick_range([H | Tail], undefined) -> -% pick_range(Tail, H); -% pick_range([{Fst, _Lst} | Tail], {CurFst, CurLst}) -> -% pick_range(Tail, {min(Fst, CurFst), CurLst}). - - %% TODO: implement synchronous writes using gen_batch_server:call/3 await_written_idx(Idx, Term, Log0) -> receive @@ -1536,17 +1505,6 @@ 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. - incr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> counters:add(Cnt, Ix, N); incr_counter(#cfg{counter = undefined}, _Ix, _N) -> diff --git a/src/ra_log_segment.erl b/src/ra_log_segment.erl index ab47e8c0f..5c42fbbc4 100644 --- a/src/ra_log_segment.erl +++ b/src/ra_log_segment.erl @@ -22,6 +22,7 @@ max_count/1, filename/1, segref/1, + info/1, is_same_as/2, copy/3]). @@ -56,7 +57,7 @@ fd :: option(file:io_device()), index_size :: pos_integer(), access_pattern :: sequential | random, - file_advise = normal :: posix_file_advise(), + file_advise = normal :: posix_file_advise(), mode = append :: read | append, compute_checksums = true :: boolean()}). @@ -453,6 +454,39 @@ segref(Filename) -> close(Seg), SegRef. +-spec info(file:filename_all()) -> + #{size => non_neg_integer(), + max_count => non_neg_integer(), + file_type => regular | symlink, + ctime => integer(), + links => non_neg_integer(), + num_entries => non_neg_integer(), + ref => option(ra_log:segment_ref()), + indexes => ra_seq:state() + }. +info(Filename) + when not is_tuple(Filename) -> + %% TODO: this can be much optimised by a specialised index parsing + %% function + {ok, Seg} = open(Filename, #{mode => read}), + Index = Seg#state.index, + {ok, #file_info{type = T, + links = Links, + ctime = CTime}} = file:read_link_info(Filename, + [raw, {time, posix}]), + + Info = #{size => Seg#state.data_write_offset, + file_type => T, + links => Links, + ctime => CTime, + max_count => max_count(Seg), + num_entries => maps:size(Index), + ref => segref(Seg), + indexes => ra_seq:from_list(maps:keys(Index)) + }, + close(Seg), + Info. + -spec is_same_as(state(), file:filename_all()) -> boolean(). is_same_as(#state{cfg = #cfg{filename = Fn0}}, Fn) -> is_same_filename_all(Fn0, Fn). diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index f71817a33..8e5923fe8 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -2,14 +2,15 @@ %% 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. -%% +%% Copyright (c) 2017-2025 Broadcom. All Rights Reserved. The term Broadcom +%% refers to Broadcom Inc. and/or its subsidiaries. +%% @hidden -module(ra_log_segments). -compile(inline_list_funcs). +-include_lib("kernel/include/file.hrl"). -export([ - init/5, init/7, close/1, update_segments/2, @@ -25,34 +26,43 @@ read_plan/2, exec_read_plan/6, fetch_term/2, - info/1 + info/1, + purge_symlinks/2 ]). -include("ra.hrl"). -define(STATE, ?MODULE). +-define(SYMLINK_KEEPFOR_S, 60). + -type access_pattern() :: sequential | random. %% holds static or rarely changing fields -record(cfg, {uid :: ra_uid(), + log_id = "" :: unicode:chardata(), counter :: undefined | counters:counters_ref(), directory :: file:filename(), access_pattern = random :: access_pattern() }). -type segment_ref() :: ra_log:segment_ref(). + -record(?STATE, {cfg :: #cfg{}, range :: ra_range:range(), segment_refs :: ra_lol:state(), open_segments :: ra_flru:state() }). +-record(compaction_result, + {unreferenced = [] :: [file:filename_all()], + linked = [] :: [file:filename_all()], + compacted = [] :: [segment_ref()]}). + -opaque state() :: #?STATE{}. -type read_plan() :: [{BaseName :: file:filename_all(), [ra:index()]}]. -type read_plan_options() :: #{access_pattern => random | sequential, file_advise => ra_log_segment:posix_file_advise()}. - -export_type([ state/0, read_plan/0, @@ -61,18 +71,14 @@ %% PUBLIC --spec init(ra_uid(), file:filename(), non_neg_integer(), - [segment_ref()], ra_system:names()) -> state(). -init(UId, Dir, MaxOpen, SegRefs, Names) -> - init(UId, Dir, MaxOpen, random, SegRefs, Names, undefined). - --spec init(ra_uid(), file:filename(), non_neg_integer(), - access_pattern(), - [segment_ref()], ra_system:names(), - undefined | counters:counters_ref()) -> state(). -init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, #{}, Counter) +-spec init(ra_uid(), file:filename_all(), non_neg_integer(), + access_pattern(), [segment_ref()], + undefined | counters:counters_ref(), + unicode:chardata()) -> state(). +init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, Counter, LogId) when is_binary(UId) -> Cfg = #cfg{uid = UId, + log_id = LogId, counter = Counter, directory = Dir, access_pattern = AccessPattern}, @@ -90,14 +96,16 @@ init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, #{}, Counter) end, SegRefsRev = lists:reverse(SegRefs), reset_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS), - #?STATE{cfg = Cfg, - open_segments = ra_flru:new(MaxOpen, FlruHandler), - range = Range, - segment_refs = - ra_lol:from_list(fun seg_ref_gt/2, SegRefsRev)}. - -seg_ref_gt({Fn1, {Start, _}}, {Fn2, {_, End}}) -> - Start > End andalso Fn1 > Fn2. + Result = recover_compaction(Dir), + %% handle_compaction_result/2 will never return an effect here + %% as no segments got deleted + State0 = #?STATE{cfg = Cfg, + open_segments = ra_flru:new(MaxOpen, FlruHandler), + range = Range, + segment_refs = ra_lol:from_list( + fun seg_ref_gt/2, SegRefsRev)}, + {State, _} = handle_compaction_result(Result, State0), + State. -spec close(state()) -> ok. close(#?STATE{open_segments = Open}) -> @@ -106,14 +114,13 @@ close(#?STATE{open_segments = Open}) -> -spec update_segments([segment_ref()], state()) -> {state(), OverwrittenSegments :: [segment_ref()]}. -update_segments(NewSegmentRefs, - #?STATE{cfg = _Cfg, - open_segments = Open0, - segment_refs = SegRefs0} = State) -> +update_segments(NewSegmentRefs, #?STATE{open_segments = Open0, + segment_refs = SegRefs0} = State) -> SegmentRefs0 = ra_lol:to_list(SegRefs0), - %% TODO: capture segrefs removed by compact_segrefs/2 and delete them SegmentRefsComp = compact_segrefs(NewSegmentRefs, SegmentRefs0), + %% capture segrefs removed by compact_segrefs/2 and delete them + %% a major compaction will also remove these OverwrittenSegments = NewSegmentRefs -- SegmentRefsComp, SegmentRefsCompRev = lists:reverse(SegmentRefsComp), SegRefs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefsCompRev), @@ -134,73 +141,77 @@ update_segments(NewSegmentRefs, end end, Open0, NewSegmentRefs), {State#?MODULE{segment_refs = SegRefs, - range = Range, - open_segments = Open}, + range = Range, + open_segments = Open}, OverwrittenSegments}. --record(compaction_result, - {unreferenced = [] :: [segment_ref()], - linked = [] :: [segment_ref()], - compacted = [] :: [segment_ref()]}). - -spec schedule_compaction(minor | major, ra:index(), ra_seq:state(), state()) -> [ra_server:effect()]. -schedule_compaction(minor, SnapIdx, LiveIndexes, State) -> - case ra_log_segments:segment_refs(State) of +schedule_compaction(Type, SnapIdx, LiveIndexes, + #?MODULE{cfg = #cfg{log_id = LogId, + directory = Dir}} = State) -> + case compactable_segrefs(SnapIdx, State) of [] -> []; - [_ | Compactable] -> - %% never compact the current segment - %% only take those who have a range lower than the snapshot index as - %% we never want to compact more than that - SegRefs = lists:takewhile(fun ({_Fn, {_Start, End}}) -> - End =< SnapIdx - end, lists:reverse(Compactable)), - %% TODO: minor compactions should also delete / truncate - %% segments with completely overwritten indexes - + SegRefs when LiveIndexes == [] -> + %% if LiveIndexes is [] we can just delete all compactable + %% segment refs + Unreferenced = [F || {F, _} <- SegRefs], + Result = #compaction_result{unreferenced = Unreferenced}, + [{next_event, + {ra_log_event, {compaction_result, Result}}}]; + SegRefs when Type == minor -> + %% TODO evaluate if minor compactions are fast enough to run + %% in server process + Result = minor_compaction(SegRefs, LiveIndexes), + [{next_event, + {ra_log_event, {compaction_result, Result}}}]; + SegRefs -> Self = self(), Fun = fun () -> - Delete = lists:foldl( - fun({_Fn, Range} = S, Del) -> - case ra_seq:in_range(Range, - LiveIndexes) of - [] -> - [S | Del]; - _ -> - Del - end - end, [], SegRefs), - Result = #compaction_result{unreferenced = Delete}, - %% need to update the ra_servers list of seg refs _before_ - %% the segments can actually be deleted + MajConf = #{dir => Dir}, + Result = major_compaction(MajConf, SegRefs, + LiveIndexes), + %% TODO: this could be done on a timer if more + %% timely symlink cleanup is needed + purge_symlinks(Dir, ?SYMLINK_KEEPFOR_S), + %% need to update the ra_servers list of seg refs + %% _before_ the segments can actually be deleted Self ! {ra_log_event, {compaction_result, Result}}, ok end, - [{bg_work, Fun, fun (_Err) -> - ?WARN("bgwork err ~p", [_Err]), ok - end}] + [{bg_work, Fun, + fun (Err) -> + ?WARN("~ts: Major compaction failed with ~p", + [LogId, Err]), ok + end}] end. - -spec handle_compaction_result(#compaction_result{}, state()) -> {state(), [ra_server:effect()]}. +handle_compaction_result(#compaction_result{unreferenced = [], + linked = [], + compacted = []}, + State) -> + {State, []}; handle_compaction_result(#compaction_result{unreferenced = Unreferenced, linked = Linked, compacted = Compacted}, #?STATE{cfg = #cfg{directory = Dir}, open_segments = Open0, segment_refs = SegRefs0} = State) -> - SegmentRefs0 = ra_lol:to_list(SegRefs0), - SegmentRefs = lists:usort(((SegmentRefs0 -- Unreferenced) -- Linked) - ++ Compacted), + SegRefs1 = maps:from_list(ra_lol:to_list(SegRefs0)), + SegRefs2 = maps:without(Unreferenced, SegRefs1), + SegRefs = maps:without(Linked, SegRefs2), + SegmentRefs0 = maps:merge(SegRefs, maps:from_list(Compacted)), + SegmentRefs = maps:to_list(maps:iterator(SegmentRefs0, ordered)), Open = ra_flru:evict_all(Open0), Fun = fun () -> [prim_file:delete(filename:join(Dir, F)) - || {F, _} <- Unreferenced], + || F <- Unreferenced], ok end, {State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, @@ -324,8 +335,26 @@ fetch_term(Idx, #?STATE{cfg = #cfg{} = Cfg} = State0) -> info(#?STATE{cfg = #cfg{} = _Cfg, open_segments = Open} = State) -> #{max_size => ra_flru:max_size(Open), - num_segments => segment_ref_count(State) - }. + num_segments => segment_ref_count(State)}. + +-spec purge_symlinks(file:filename_all(), + OlderThanSec :: non_neg_integer()) -> ok. +purge_symlinks(Dir, OlderThanSec) -> + Now = erlang:system_time(second), + [begin + Fn = filename:join(Dir, F), + case file:read_link_info(Fn, [raw, {time, posix}]) of + {ok, #file_info{type = symlink, + ctime = Time}} + when Now - Time > OlderThanSec -> + prim_file:delete(Fn), + ok; + _ -> + ok + end + end || F <- list_files(Dir, ".segment")], + ok. + %% LOCAL segment_read_plan(_SegRefs, [], Acc) -> @@ -520,6 +549,242 @@ decr_counter(#cfg{counter = Cnt}, Ix, N) when Cnt =/= undefined -> decr_counter(#cfg{counter = undefined}, _, _) -> ok. +segment_files(Dir, Fun) -> + list_files(Dir, ".segment", Fun). + +list_files(Dir, Ext) -> + list_files(Dir, Ext, fun (_) -> true end). + +list_files(Dir, Ext, Fun) -> + case prim_file:list_dir(Dir) of + {ok, Files0} -> + Files = [list_to_binary(F) + || F <- Files0, + filename:extension(F) =:= Ext, + Fun(F)], + lists:sort(Files); + {error, enoent} -> + [] + end. + +major_compaction(#{dir := Dir}, SegRefs, LiveIndexes) -> + {Compactable, Delete} = + lists:foldl(fun({Fn0, Range} = S, + {Comps, Del}) -> + case ra_seq:in_range(Range, + LiveIndexes) of + [] -> + {Comps, [Fn0 | Del]}; + Seq -> + %% get the info map from each + %% potential segment + Fn = filename:join(Dir, Fn0), + Info = ra_log_segment:info(Fn), + {[{Info, Seq, S} | Comps], Del} + end + end, {[], []}, SegRefs), + + %% ensure there are no remaining fully overwritten (unused) segments in + %% the compacted range + Lookup = maps:from_list(SegRefs), + {LastFn, {_, _}} = lists:last(SegRefs), + UnusedFiles = segment_files(Dir, fun (F) -> + Key = list_to_binary(F), + Key =< LastFn andalso + not maps:is_key(Key, Lookup) + end), + [begin + ok = prim_file:delete(filename:join(Dir, F)) + end || F <- UnusedFiles], + %% group compactable + CompactionGroups = compaction_groups(lists:reverse(Compactable), []), + Compacted0 = + [begin + %% create a new segment with .compacting extension + AllShortFns = [F || {_, _, {F, _}} <- All], + CompactingShortFn = make_compacting_file_name(AllShortFns), + CompactingFn = filename:join(Dir, CompactingShortFn), + %% max_count is the sum of all live indexes for segments in the + %% compaction group + MaxCount = lists:sum([ra_seq:length(S) || {_, S, _} <- All]), + %% copy live indexes from all segments in compaction group to + %% the compacting segment + {ok, CompSeg0} = ra_log_segment:open(CompactingFn, + #{max_count => MaxCount}), + CompSeg = lists:foldl( + fun ({_, Live, {F, _}}, S0) -> + {ok, S} = ra_log_segment:copy(S0, filename:join(Dir, F), + ra_seq:expand(Live)), + S + end, CompSeg0, All), + ok = ra_log_segment:close(CompSeg), + + %% link .compacting segment to the original .segment file + %% first we have to create a hard link to a new .compacted file + %% from the .compacting file (as we need to keep this as a marker + %% until the end + %% then we can rename this on top of the first segment file in the + %% group (the target) + FirstSegmentFn = filename:join(Dir, FstFn0), + CompactedFn = filename:join(Dir, with_ext(FstFn0, ".compacted")), + ok = prim_file:make_link(CompactingFn, CompactedFn), + ok = prim_file:rename(CompactedFn, FirstSegmentFn), + + %% perform sym linking of the additional segments in the compaction + %% group + ok = make_links(Dir, FirstSegmentFn, + [F || {_, _, {F, _}} <- Additional]), + %% finally deleted the .compacting file to signal compaction group + %% is complete + ok = prim_file:delete(CompactingFn), + %% return the new segref and additional segment keys + {ra_log_segment:segref(FirstSegmentFn), + [A || {_, _, {A, _}} <- Additional]} + end || [{_Info, _, {FstFn0, _}} | Additional] = All + <- CompactionGroups], + + {Compacted, AddDelete} = lists:unzip(Compacted0), + + #compaction_result{unreferenced = Delete, + linked = lists:append(AddDelete), + compacted = Compacted}. + +minor_compaction(SegRefs, LiveIndexes) -> + %% identifies unreferences / unused segments with no live indexes + %% in them + Delete = lists:foldl(fun({Fn, Range}, Del) -> + case ra_seq:in_range(Range, + LiveIndexes) of + [] -> + [Fn | Del]; + _ -> + Del + end + end, [], SegRefs), + #compaction_result{unreferenced = Delete}. + +compactable_segrefs(SnapIdx, State) -> + %% TODO: provide a ra_lol:foldr API to avoid creatinga segref list + %% then filtering that + case segment_refs(State) of + [] -> + []; + [_] -> + []; + [_ | Compactable] -> + %% never compact the current segment + %% only take those who have a range lower than the snapshot index as + %% we never want to compact more than that + lists:foldl(fun ({_Fn, {_Start, End}} = SR, Acc) + when End =< SnapIdx -> + [SR | Acc]; + (_, Acc) -> + Acc + end, [], Compactable) + end. + +make_links(Dir, To, From) + when is_list(From) -> + [begin + SymFn = filename:join(Dir, with_ext(FromFn, ".link")), + SegFn = filename:join(Dir, with_ext(FromFn, ".segment")), + %% just in case it already exists + _ = prim_file:delete(SymFn), + %% make a symlink from the compacted target segment to a new .link + %% where the compacted indexes now can be found + ok = prim_file:make_symlink(To, SymFn), + %% rename to link to replace original segment + ok = prim_file:rename(SymFn, SegFn) + end || FromFn <- From], + ok. + +with_ext(Fn, Ext) when is_binary(Fn) andalso is_list(Ext) -> + <<(filename:rootname(Fn))/binary, (ra_lib:to_binary(Ext))/binary>>. + +compaction_groups([], Groups) -> + lists:reverse(Groups); +compaction_groups(Infos, Groups) -> + case take_group(Infos, #{max_count => 128}, []) of + {Group, RemInfos} -> + compaction_groups(RemInfos, [Group | Groups]) + end. + +%% TODO: try to take potential size into account +take_group([], _, Acc) -> + {lists:reverse(Acc), []}; +take_group([{#{num_entries := NumEnts}, Live, {_, _}} = E | Rem] = All, + #{max_count := Mc}, Acc) -> + Num = ra_seq:length(Live), + case Num < NumEnts div 2 of + true -> + case Mc - Num < 0 of + true -> + {lists:reverse(Acc), All}; + false -> + take_group(Rem, #{max_count => Mc - Num}, [E | Acc]) + end; + %% skip this secment + false when Acc == [] -> + take_group(Rem, #{max_count => Mc}, Acc); + false -> + {lists:reverse(Acc), Rem} + end. + + +parse_compacting_filename(Fn) when is_binary(Fn) -> + binary:split(filename:rootname(Fn), <<"-">>, [global]). + +make_compacting_file_name([N1 | Names]) -> + Root = lists:foldl(fun (N, Acc) -> + [filename:rootname(N), <<"-">> | Acc] + end, [N1], Names), + iolist_to_binary(lists:reverse([<<".compacting">> | Root])). + +recover_compaction(Dir) -> + case list_files(Dir, ".compacting") of + [] -> + %% no pending compactions + #compaction_result{}; + [ShortFn] -> + %% compaction recovery is needed + CompactingFn = filename:join(Dir, ShortFn), + {ok, #file_info{links = Links}} = + file:read_link_info(CompactingFn, [raw, {time, posix}]), + case Links of + 1 -> + %% must have exited before the target file was renamed + %% just delete + ok = prim_file:delete(CompactingFn), + #compaction_result{}; + 2 -> + [FstFn | RemFns] = parse_compacting_filename(ShortFn), + %% there may be a .compacted file + Target = filename:join(Dir, with_ext(FstFn, ".segment")), + case list_files(Dir, ".compacted") of + [CompactedShortFn] -> + CompactedFn = filename:join(Dir, CompactedShortFn), + %% all entries were copied but it failed before + %% this hard link could be renamed over the target + ok = prim_file:rename(CompactedFn, Target), + ok; + [] -> + %% links may not have been fully created, + %% delete all .link files then relink + ok + end, + ok = make_links(Dir, Target, RemFns), + ok = prim_file:delete(CompactingFn), + + Linked = [with_ext(L, ".segment") || L <- RemFns], + Compacted = [ra_log_segment:segref(Target)], + #compaction_result{compacted = Compacted, + linked = Linked} + end + end. + +seg_ref_gt({Fn1, {Start, _}}, {Fn2, {_, End}}) -> + Start > End andalso Fn1 > Fn2. + -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index c0ec94b27..3c11d3843 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -311,9 +311,9 @@ validate_sequential_fold(Config) -> ct:pal("ra_log:overview/1 ~p", [ra_log:overview(FinLog)]), - #{?FUNCTION_NAME := #{read_mem_table := M1, - open_segments := 2, %% as this is the max - read_segment := M4} = O} = ra_counters:overview(), + #{read_mem_table := M1, + open_segments := 2, %% as this is the max + read_segment := M4} = O = ra_counters:overview(?FUNCTION_NAME), ct:pal("counters ~p", [O]), ?assertEqual(1000, M1 + M4), @@ -1521,7 +1521,6 @@ open_segments_limit(Config) -> ok. write_config(Config) -> - C = #{cluster_name => ?MODULE, id => {?MODULE, node()}, uid => <<"blah">>, @@ -1724,8 +1723,8 @@ deliver_log_events_cond(Log0, CondFun, N) -> P ! E, Acc; ({next_event, {ra_log_event, E}}, Acc0) -> - {Acc, Effs} = ra_log:handle_event(E, Acc0), - run_effs(Effs), + {Acc, Effs1} = ra_log:handle_event(E, Acc0), + run_effs(Effs1), Acc; ({bg_work, Fun, _}, Acc) -> Fun(), @@ -1737,7 +1736,7 @@ deliver_log_events_cond(Log0, CondFun, N) -> {false, Log} -> deliver_log_events_cond(Log, CondFun, N-1); false -> - deliver_log_events_cond(Log2, CondFun, N-1); + deliver_log_events_cond(Log1, CondFun, N-1); {true, Log} -> ct:pal("condition was true!!"), Log; @@ -1801,8 +1800,8 @@ assert_log_events(Log0, AssertPred, Timeout) -> %% handle any next events Log = lists:foldl( fun ({next_event, {ra_log_event, E}}, Acc0) -> - {Acc, Effs} = ra_log:handle_event(E, Acc0), - run_effs(Effs), + {Acc, Effs1} = ra_log:handle_event(E, Acc0), + run_effs(Effs1), Acc; (_, Acc) -> Acc diff --git a/test/ra_log_segment_SUITE.erl b/test/ra_log_segment_SUITE.erl index 6153360ac..990db1c89 100644 --- a/test/ra_log_segment_SUITE.erl +++ b/test/ra_log_segment_SUITE.erl @@ -51,10 +51,6 @@ init_per_testcase(TestCase, Config) -> PrivDir = ?config(priv_dir, Config), Dir = filename:join(PrivDir, TestCase), ok = ra_lib:make_dir(Dir), - _ = ets:new(ra_open_file_metrics, - [named_table, public, {write_concurrency, true}]), - _ = ets:new(ra_io_metrics, - [named_table, public, {write_concurrency, true}]), [{test_case, TestCase}, {data_dir, Dir} | Config]. end_per_testcase(_, Config) -> diff --git a/test/ra_log_segments_SUITE.erl b/test/ra_log_segments_SUITE.erl new file mode 100644 index 000000000..e7b6d233b --- /dev/null +++ b/test/ra_log_segments_SUITE.erl @@ -0,0 +1,507 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2017-2025 Broadcom. All Rights Reserved. The term Broadcom +%% refers to Broadcom Inc. and/or its subsidiaries. +-module(ra_log_segments_SUITE). + +-compile(nowarn_export_all). +-compile(export_all). + +-include_lib("common_test/include/ct.hrl"). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("kernel/include/file.hrl"). + +%% common ra_log tests to ensure behaviour is equivalent across +%% ra_log backends + +all() -> + [ + {group, tests} + ]. + +all_tests() -> + [ + recover1, + recover2, + basics, + major, + minor, + overwrite, + result_after_segments, + result_after_segments_overwrite + ]. + +groups() -> + [ + {tests, [], all_tests()} + ]. + +init_per_testcase(TestCase, Config) -> + PrivDir = ?config(priv_dir, Config), + Dir = filename:join(PrivDir, TestCase), + ok = ra_lib:make_dir(Dir), + [{uid, atom_to_binary(TestCase, utf8)}, + {test_case, TestCase}, + {dir, Dir} | Config]. + +end_per_testcase(_, Config) -> + Config. + +%% TESTS + +result_after_segments(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {major, 128 * 3, Live}, + {entries, 1, lists:seq(128 * 3, 128 * 4)}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, 1, lists:seq(128 * 3, 128 * 4)}, + {assert, fun (S) -> + ct:pal("seg refs ~p", [ra_log_segments:segment_refs(S)]), + true + end} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ct:pal("infos ~p", [infos(Dir)]), + ok. + +result_after_segments_overwrite(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 2, 8), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {major, 128 * 3, Live}, + {entries, 2, lists:seq(128 * 2, 128 * 4)}, + handle_compaction_result, + {print, "1"}, + {assert, 1, ra_seq:expand(ra_seq:limit(128 * 2, Live))}, + {assert, 2, lists:seq(128 * 2, 128 * 3)}, + {print, "2"}, + {assert, fun (S) -> + ct:pal("seg refs ~p", [ra_log_segments:segment_refs(S)]), + true + end}, + {assert, 2, lists:seq(128 * 3, 128 * 4)} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ct:pal("infos ~p", [infos(Dir)]), + ok. + +recover1(Config) -> + %% major compactions can be interrupted at a variety of points and each + %% needs to be handled carefully to ensure the log isn't incorrectly + %% recovered + + %% There is a .compacting file in the segments directory + %% 1. Compaction stopped before the compacting segment got renamed to + %% the lowest numbered segments. + %% 2. Compaction stopped before or during additional segments in the group + %% where linked (.compacting link count > 1) + + Dir = ?config(dir, Config), + CompactingFn = <<"0000000000000001-0000000000000002-0000000000000003.compacting">>, + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + ok = ra_lib:write_file(filename:join(Dir, CompactingFn), <<>>), + true + end}, + reinit, + {assert, fun (_) -> + %% a compacting file with 1 link only should just be deleted + %% during init + not filelib:is_file(filename:join(Dir, CompactingFn)) + end} + ], + ct:pal("infos ~p", [infos(Dir)]), + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. + +recover2(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + CompactingShortFn = <<"0000000000000001-0000000000000002-0000000000000003.compacting">>, + CompactingFn = filename:join(Dir, CompactingShortFn), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + %% fake a .compacting file, and perform a copy + %% (this code is copied from ra_log_segments + All = lists:reverse(tl(seg_refs(Dir))), + {ok, CompSeg0} = ra_log_segment:open(CompactingFn, + #{max_count => 128}), + CompSeg = lists:foldl( + fun ({F, R}, S0) -> + L = ra_seq:in_range(R, Live), + {ok, S} = ra_log_segment:copy( + S0, filename:join(Dir, F), + ra_seq:expand(L)), + S + end, CompSeg0, All), + ok = ra_log_segment:close(CompSeg), + [{FstFn, _}, {SndFn, _}, {_ThrFn, _}] = All, + + CompactedFn = filename:join(Dir, with_ext(FstFn, ".compacted")), + ok = prim_file:make_link(CompactingFn, CompactedFn), + FirstSegmentFn = filename:join(Dir, FstFn), + ok = prim_file:rename(CompactedFn, FirstSegmentFn), + % SecondFn = filename:join(Dir, SndFn), + SndLinkFn = filename:join(Dir, with_ext(SndFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, SndLinkFn), + %% the first segment has now been replaced with a link + %% to the compacting segment but not all symlinks may have been created + true + end}, + reinit, + {assert, fun (_) -> + Infos = infos(Dir), + NumLinks = length([a || #{file_type := symlink} <- Infos]), + %% a compacting file with 1 link only should just be deleted + %% during init + not filelib:is_file(CompactingFn) andalso + NumLinks == 2 + end}, + {assert, fun(S) -> + SegRefs = ra_log_segments:segment_refs(S), + ct:pal("SegRefs ~p, ~p", [SegRefs, seg_refs(Dir)]), + SegRefs == seg_refs(Dir) + end} + + + + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ct:pal("infos ~p", [infos(Dir)]), + ok. + +basics(Config) -> + %% creates 3 segments then a snapshot at the first index of the last segment + %% with live indexes only in the first segment. + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128, 5), + Live = ra_seq:from_list(LiveList), + ct:pal("Live ~p", [Live]), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {assert, 1, lists:seq(1, 128 * 3)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + %% this compaction will delete 1 segment (segment 2) + {minor, 128 * 2, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 + end}, + %% this compaction will compact segment 1 + {major, 128 * 2, Live}, + handle_compaction_result, + reinit, + {assert, 1, LiveList}, + {assert, fun (S) -> + [#{num_entries := NumEntries} |_ ] = infos(Dir), + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 andalso + ra_seq:length(Live) == NumEntries + + end} + ], + + + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + + ct:pal("infos ~p", [infos(Dir)]), + ok. + +minor(Config) -> + LiveList = [1 | lists:seq(257, 500, 10)], + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, lists:seq(1, 500)}, + {assert, 1, lists:seq(1, 500)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 4 + end}, + {minor, 500, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + + %% simulate a purge command + {entries, 1, [501]}, + {minor, 501, []}, + handle_compaction_result, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 1 + end} + ], + + SegConf = #{max_count => 128}, + Dir = ?config(dir, Config), + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + + ok. + +overwrite(Config) -> + Live = ra_seq:from_list(lists:seq(1, 600, 10)), + Scen = + [ + {entries, 1, lists:seq(1, 500)}, + {entries, 2, lists:seq(200, 700)}, + reinit, + {assert, 1, lists:seq(1, 200, 10)}, + {assert, 2, lists:seq(201, 700, 10)}, + {minor, 600, Live}, + handle_compaction_result, + {major, 600, Live}, + {assert, 1, lists:seq(1, 200, 10)}, + {assert, 2, lists:seq(201, 700, 10)}, + handle_compaction_result, + {assert, 1, lists:seq(1, 200, 10)}, + {assert, 2, lists:seq(201, 700, 10)} + ], + + SegConf = #{max_count => 128}, + Dir = ?config(dir, Config), + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. + + +major(Config) -> + Dir = ?config(dir, Config), + Entries1 = lists:seq(1, 641), + LiveList = lists:seq(1, 383, 3) ++ + lists:seq(384, 511) ++ %% the 4th segment is still full + lists:seq(512, 640, 3), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, Entries1}, + {assert, 1, lists:seq(1, 641)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 6 + end}, + {major, 640, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + %% infos contain one symlink + Infos = infos(Dir), + Symlinks = [I || #{file_type := symlink} = I <- Infos], + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 5 andalso + length(Infos) == 6 andalso + length(Symlinks) == 1 + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 5 + end}, + {assert, + fun (_S) -> + ok = ra_log_segments:purge_symlinks(Dir, 1), + [_] = [I || #{file_type := symlink} = I <- infos(Dir)], + timer:sleep(2000), + ok = ra_log_segments:purge_symlinks(Dir, 1), + Infos = infos(Dir), + Symlinks = [I || #{file_type := symlink} = I <- Infos], + Files = [I || #{file_type := regular} = I <- Infos], + length(Symlinks) == 0 andalso + length(Files) == 5 + end} + ], + + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + + ok. + +%% Helpers + +open_last_segment(Config, SegConf) -> + Dir = ?config(dir, Config), + case seg_refs(Dir) of + [] -> + Fn = ra_lib:zpad_filename("", "segment", 1), + SegFn = filename:join(Dir, Fn), + {ok, Seg} = ra_log_segment:open(SegFn, SegConf), + Seg; + [{Fn, _} | _] -> + SegFn = filename:join(Dir, Fn), + {ok, Seg} = ra_log_segment:open(SegFn, SegConf), + Seg + end. + + + +append_to_segment(Seg0, [], Refs, _Conf) -> + {Seg0, [ra_log_segment:segref(Seg0) | Refs]}; +append_to_segment(Seg0, [{Idx, Term, Data} | Rem] = All, Refs, Conf) -> + DataSize = iolist_size(Data), + case ra_log_segment:append(Seg0, Idx, Term, {DataSize, Data}) of + {ok, Seg} -> + append_to_segment(Seg, Rem, Refs, Conf); + {error, full} -> + Ref = ra_log_segment:segref(Seg0), + % close and open a new segment + append_to_segment(open_successor_segment(Seg0, Conf), All, + [Ref | Refs], Conf) + end. + +open_successor_segment(CurSeg, SegConf) -> + Fn0 = ra_log_segment:filename(CurSeg), + Fn = ra_lib:zpad_filename_incr(Fn0), + ok = ra_log_segment:close(CurSeg), + {ok, Seg} = ra_log_segment:open(Fn, SegConf), + Seg. + + +seg_refs(Dir) -> + lists:reverse( + [ra_log_segment:segref(F) || F <- segment_files(Dir), + is_regular(F) andalso + lists:member(filename:extension(F), [".segment", <<".segment">>])]). + +infos(Dir) -> + [ra_log_segment:info(F) || F <- segment_files(Dir)]. + +segment_files(Dir) -> + case prim_file:list_dir(Dir) of + {ok, Files0} -> + Files = [filename:join(Dir, F) + || F <- Files0, + begin + Ext = filename:extension(F), + lists:member(Ext, [".segment", + ".compacting", + ".compacted", + ".link"]) + end], + lists:sort(Files); + {error, enoent} -> + [] + end. + +flush() -> + receive + Any -> + ct:pal("flush ~p", [Any]), + flush() + after 0 -> + ok + end. + +is_regular(Filename) -> + {ok, #file_info{type = T}} = file:read_link_info(Filename, [raw, {time, posix}]), + T == regular. + +run_scenario(_, Segs, []) -> + Segs; +run_scenario(Config, Segs0, [reinit | Rem]) -> + Dir = ?config(dir, Config), + ra_log_segments:close(Segs0), + Segs = ra_log_segments:init(?config(uid, Config), Dir, 1, random, + seg_refs(Dir), undefined, ""), + ?FUNCTION_NAME(Config, Segs, Rem); +run_scenario(Config, Segs0, [{entries, Term, Indexes} | Rem]) -> + SegConf = ?config(seg_conf, Config), + Seg0 = open_last_segment(Config, SegConf), + Entries = [{I, Term, term_to_binary(<<"data1">>)} || I <- Indexes], + {Seg, Refs} = append_to_segment(Seg0, Entries, [], SegConf), + _ = ra_log_segment:close(Seg), + {Segs, _Overwritten} = ra_log_segments:update_segments(Refs, Segs0), + %% TODO: what to do about overwritten + ?FUNCTION_NAME(Config, Segs, Rem); +run_scenario(Config, Segs0, [{Type, SnapIdx, Live} | Rem]) + when Type == major orelse Type == minor -> + Effs = ra_log_segments:schedule_compaction(Type, SnapIdx, Live, Segs0), + Segs = lists:foldl(fun ({bg_work, Fun, _}, S0) -> + Fun(), + S0; + ({next_event, {ra_log_event, + {compaction_result, _Res}} = E}, S0) -> + self() ! E, + S0 + end, Segs0, Effs), + + ?FUNCTION_NAME(Config, Segs, Rem); +run_scenario(Config, Segs0, [handle_compaction_result = Step | Rem]) -> + CompRes3 = receive + {ra_log_event, {compaction_result, Res3}} -> + ct:pal("compaction result ~p", [Res3]), + Res3 + after 5000 -> + flush(), + exit({ra_log_event_timeout, Step}) + end, + + {Segs1, Effs} = + ra_log_segments:handle_compaction_result(CompRes3, Segs0), + [Fun1() || {bg_work, Fun1, _} <- Effs], + ?FUNCTION_NAME(Config, Segs1, Rem); +run_scenario(Config, Segs0, [{assert, Term, Indexes} | Rem]) -> + {Read, Segs2} = ra_log_segments:sparse_read(Segs0, + lists:reverse(Indexes), []), + %% assert we can read + ?assertEqual(length(Indexes), length(Read)), + ?assert(lists:all(fun ({_, T, _}) -> T == Term end, Read)), + ?FUNCTION_NAME(Config, Segs2, Rem); +run_scenario(Config, Segs0, [{assert, Fun} | Rem]) + when is_function(Fun) -> + ?assert(Fun(Segs0)), + ?FUNCTION_NAME(Config, Segs0, Rem); +run_scenario(Config, Segs0, [{print, What} | Rem]) -> + ct:pal(What), + ?FUNCTION_NAME(Config, Segs0, Rem). + + +with_ext(Fn, Ext) when is_binary(Fn) andalso is_list(Ext) -> + <<(filename:rootname(Fn))/binary, (ra_lib:to_binary(Ext))/binary>>. + +ra_log_segments_init(UId, Dir, SegRefs) -> + ra_log_segments:init(UId, Dir, 1, random, + SegRefs, undefined, ""). From 234ba96c6cf1fe2f96908c65e5b295ea2d88a028 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 16 Jun 2025 10:53:47 +0100 Subject: [PATCH 35/60] compaction counters --- src/ra.erl | 10 +++++++++- src/ra.hrl | 14 +++++++++++++- src/ra_lib.erl | 4 ++-- src/ra_log.erl | 13 +++++++++++++ src/ra_log_segments.erl | 9 +++++++-- test/ra_kv_SUITE.erl | 18 +++++++++++++----- test/ra_log_2_SUITE.erl | 1 - 7 files changed, 57 insertions(+), 12 deletions(-) diff --git a/src/ra.erl b/src/ra.erl index 7476cf6b7..51ce47500 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -73,7 +73,8 @@ member_overview/1, member_overview/2, key_metrics/1, - key_metrics/2 + key_metrics/2, + trigger_compaction/1 ]). -define(START_TIMEOUT, ?DEFAULT_TIMEOUT). @@ -1198,6 +1199,13 @@ key_metrics({Name, N} = ServerId, _Timeout) when N == node() -> key_metrics({_, N} = ServerId, Timeout) -> erpc:call(N, ?MODULE, ?FUNCTION_NAME, [ServerId], Timeout). +%% @doc Potentially triggers a major compaction for the provided member +%% @param ServerId the Ra server to send the request to +%% @end +-spec trigger_compaction(ra_server_id()) -> ok. +trigger_compaction(ServerRef) -> + gen_statem:cast(ServerRef, {ra_log_event, major_compaction}). + %% internal -spec usr(UserCommand, ReplyMode) -> Command when diff --git a/src/ra.hrl b/src/ra.hrl index 80379616f..1adf93713 100644 --- a/src/ra.hrl +++ b/src/ra.hrl @@ -278,8 +278,17 @@ "Number of checkpoint bytes written"}, {checkpoints_promoted, ?C_RA_LOG_CHECKPOINTS_PROMOTED, counter, "Number of checkpoints promoted to snapshots"}, + {major_compactions, ?C_RA_LOG_COMPACTIONS_MAJOR_COUNT, counter, + "Number of requested major compactions"}, + {major_compaction_segments_written, + ?C_RA_LOG_COMPACTIONS_SEGMENTS_WRITTEN, counter, + "Number of segments written during major compactions"}, + {major_compaction_segments_compacted, + ?C_RA_LOG_COMPACTIONS_SEGMENTS_COMPACTED, counter, + "Number of segments compacted during major compactions"}, {reserved_1, ?C_RA_LOG_RESERVED, counter, "Reserved counter"} ]). + -define(C_RA_LOG_WRITE_OPS, 1). -define(C_RA_LOG_WRITE_RESENDS, 2). -define(C_RA_LOG_READ_OPS, 3). @@ -295,7 +304,10 @@ -define(C_RA_LOG_CHECKPOINTS_WRITTEN, 13). -define(C_RA_LOG_CHECKPOINT_BYTES_WRITTEN, 14). -define(C_RA_LOG_CHECKPOINTS_PROMOTED, 15). --define(C_RA_LOG_RESERVED, 16). +-define(C_RA_LOG_COMPACTIONS_MAJOR_COUNT, 16). +-define(C_RA_LOG_COMPACTIONS_SEGMENTS_WRITTEN, 17). +-define(C_RA_LOG_COMPACTIONS_SEGMENTS_COMPACTED, 18). +-define(C_RA_LOG_RESERVED, 19). -define(C_RA_SRV_AER_RECEIVED_FOLLOWER, ?C_RA_LOG_RESERVED + 1). -define(C_RA_SRV_AER_REPLIES_SUCCESS, ?C_RA_LOG_RESERVED + 2). diff --git a/src/ra_lib.erl b/src/ra_lib.erl index a9235832e..0c0b80a81 100644 --- a/src/ra_lib.erl +++ b/src/ra_lib.erl @@ -344,9 +344,9 @@ retry(Func, Attempt, Sleep) -> ok; true -> ok; - _ -> + _Err -> timer:sleep(Sleep), - retry(Func, Attempt - 1) + retry(Func, Attempt - 1, Sleep) end. -spec write_file(file:name_all(), iodata()) -> diff --git a/src/ra_log.erl b/src/ra_log.erl index f44a5a651..4b50c17bf 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -71,6 +71,8 @@ {snapshot_written, ra_idxterm(), LiveIndexes :: ra_seq:state(), ra_snapshot:kind()} | + {compaction_result, term()} | + major_compaction | {down, pid(), term()}. -type event() :: {ra_log_event, event_body()}. @@ -818,6 +820,17 @@ handle_event({compaction_result, Result}, #?MODULE{reader = Reader0} = State) -> {Reader, Effs} = ra_log_segments:handle_compaction_result(Result, Reader0), {State#?MODULE{reader = Reader}, Effs}; +handle_event(major_compaction, #?MODULE{reader = Reader0, + live_indexes = LiveIndexes, + snapshot_state = SS} = State) -> + case ra_snapshot:current(SS) of + {SnapIdx, _} -> + Effs = ra_log_segments:schedule_compaction(major,SnapIdx, + LiveIndexes, Reader0), + {State, Effs}; + _ -> + {State, []} + end; handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, #?MODULE{cfg = #cfg{uid = UId, names = Names} = Cfg, diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index 8e5923fe8..f3bc5c941 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -150,7 +150,7 @@ update_segments(NewSegmentRefs, #?STATE{open_segments = Open0, [ra_server:effect()]. schedule_compaction(Type, SnapIdx, LiveIndexes, #?MODULE{cfg = #cfg{log_id = LogId, - directory = Dir}} = State) -> + directory = Dir} = Cfg} = State) -> case compactable_segrefs(SnapIdx, State) of [] -> []; @@ -170,6 +170,7 @@ schedule_compaction(Type, SnapIdx, LiveIndexes, SegRefs -> Self = self(), Fun = fun () -> + ok = incr_counter(Cfg, ?C_RA_LOG_COMPACTIONS_MAJOR_COUNT, 1), MajConf = #{dir => Dir}, Result = major_compaction(MajConf, SegRefs, LiveIndexes), @@ -200,7 +201,7 @@ handle_compaction_result(#compaction_result{unreferenced = [], handle_compaction_result(#compaction_result{unreferenced = Unreferenced, linked = Linked, compacted = Compacted}, - #?STATE{cfg = #cfg{directory = Dir}, + #?STATE{cfg = #cfg{directory = Dir} = Cfg, open_segments = Open0, segment_refs = SegRefs0} = State) -> SegRefs1 = maps:from_list(ra_lol:to_list(SegRefs0)), @@ -214,6 +215,10 @@ handle_compaction_result(#compaction_result{unreferenced = Unreferenced, || F <- Unreferenced], ok end, + ok = incr_counter(Cfg, ?C_RA_LOG_COMPACTIONS_SEGMENTS_WRITTEN, + length(Compacted)), + ok = incr_counter(Cfg, ?C_RA_LOG_COMPACTIONS_SEGMENTS_COMPACTED, + length(Linked) + length(Compacted)), {State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefs), open_segments = Open}, diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index 1e10494eb..4baf0e1d9 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -3,14 +3,11 @@ -compile(nowarn_export_all). -compile(export_all). --export([ - ]). - --include_lib("src/ra.hrl"). -include_lib("common_test/include/ct.hrl"). -include_lib("eunit/include/eunit.hrl"). -define(SYS, default). + %%%=================================================================== %%% Common Test callbacks %%%=================================================================== @@ -118,7 +115,6 @@ basics(_Config) -> {ok, {Reads3, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], undefined, 1000), ct:pal("ReadRes3 ~p", [Reads3]), - % ct:pal("overview3 ~p", [ra:member_overview(KvId2)]), ?assertEqual(3, map_size(Reads3)), %% TODO: test recovery of kv @@ -128,4 +124,16 @@ basics(_Config) -> undefined, 1000), ?assertEqual(3, map_size(Reads4)), + ra:trigger_compaction(KvId), + %% wait for compaction by querying counters + ok = ra_lib:retry( + fun () -> + #{major_compactions := Maj} = + ra_counters:counters(KvId, [major_compactions]), + Maj == 1 + end, 10, 100), + {ok, {Reads5, _}} = ra_server_proc:read_entries(KvId, [LastIdx | Live], + undefined, 1000), + ?assertEqual(Reads4, Reads5), + ct:pal("counters ~p", [ra_counters:overview(KvId)]), ok. diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 3c11d3843..a92af51ca 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -256,7 +256,6 @@ read_one(Config) -> ra_counters:new(?FUNCTION_NAME, ?RA_COUNTER_FIELDS), Log0 = ra_log_init(Config, #{counter => ra_counters:fetch(?FUNCTION_NAME)}), Log1 = append_n(1, 2, 1, Log0), - % Log1 = ra_log:append({1, 1, <<1:64/integer>>}, Log0), % ensure the written event is delivered Log2 = deliver_all_log_events(Log1, 200), {[_], Log} = ra_log_take(1, 1, Log2), From 1520b6bfd4342e60b3c819c02532b4f61c1b5ff8 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 16 Jun 2025 13:22:01 +0100 Subject: [PATCH 36/60] only flush live indexes in seg writer --- src/ra_kv.erl | 12 ++++++-- src/ra_log_segment_writer.erl | 28 ++++++++++++++---- src/ra_log_snapshot_state.erl | 6 ++++ src/ra_seq.erl | 24 +++++++++------ test/ra_log_segment_writer_SUITE.erl | 44 +++++++++++++++++++++++++++- 5 files changed, 96 insertions(+), 18 deletions(-) diff --git a/src/ra_kv.erl b/src/ra_kv.erl index 1b165cf43..aa5fcb792 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -21,7 +21,8 @@ put/4, get/3, - query_get/3 + query_get/3, + take_snapshot/1 ]). @@ -47,7 +48,9 @@ command/0]). %% mgmt --spec start_cluster(atom(), atom(), map()) -> +-spec start_cluster(System :: atom(), + ClusterName :: atom(), + Config :: #{members := [ra_server_id()]}) -> {ok, [ra_server_id()], [ra_server_id()]} | {error, cluster_not_formed}. start_cluster(System, ClusterName, #{members := ServerIds}) @@ -134,6 +137,10 @@ query_get(ClusterName, Key, #?STATE{keys = Keys}) -> {error, not_found} end. +-spec take_snapshot(ra_server_id()) -> ok. +take_snapshot(ServerId) -> + ra:aux_command(ServerId, take_snapshot). + %% state machine init(_) -> @@ -154,6 +161,7 @@ live_indexes(#?STATE{keys = Keys}) -> [Idx | Acc] end, [], Keys). + -record(aux, {}). init_aux(_) -> diff --git a/src/ra_log_segment_writer.erl b/src/ra_log_segment_writer.erl index 87e97d4b1..043e1fc77 100644 --- a/src/ra_log_segment_writer.erl +++ b/src/ra_log_segment_writer.erl @@ -265,19 +265,27 @@ get_overview(#state{data_dir = Dir, flush_mem_table_ranges({ServerUId, TidSeqs0}, #state{system = System} = State) -> SmallestIdx = smallest_live_idx(ServerUId), - %% TidRanges arrive here sorted new -> old. + LiveIndexes = live_indexes(ServerUId), + LastLive = ra_seq:last(LiveIndexes), + %% TidSeqs arrive here sorted new -> old. - %% truncate and limit all ranges to create a contiguous non-overlapping + %% TODO: use live indexes from ra_log_snapshot_state table to only + %% write live entries below the snapshot index + + %% truncate and limit all seqa to create a contiguous non-overlapping %% list of tid ranges to flush to disk - %% now TidRanges are sorted old -> new, i.e the correct order of - %% processing TidSeqs = lists:foldl( fun ({T, Seq0}, []) -> case ra_seq:floor(SmallestIdx, Seq0) of [] -> []; + Seq when LiveIndexes == []-> + [{T, Seq}]; Seq -> - [{T, Seq}] + L = ra_seq:in_range(ra_seq:range(Seq), + LiveIndexes), + + [{T, ra_seq:add(ra_seq:floor(LastLive + 1, Seq), L)}] end; ({T, Seq0}, [{_T, PrevSeq} | _] = Acc) -> Start = ra_seq:first(PrevSeq), @@ -285,8 +293,13 @@ flush_mem_table_ranges({ServerUId, TidSeqs0}, case ra_seq:limit(Start, Seq1) of [] -> Acc; + Seq when LiveIndexes == [] -> + [{T, Seq} | Acc]; Seq -> - [{T, Seq} | Acc] + L = ra_seq:in_range(ra_seq:range(Seq), + LiveIndexes), + [{T, ra_seq:add(ra_seq:floor(LastLive + 1, Seq), L)} + | Acc] end end, [], TidSeqs0), @@ -358,6 +371,9 @@ start_index(ServerUId, StartIdx0) -> smallest_live_idx(ServerUId) -> ra_log_snapshot_state:smallest(ra_log_snapshot_state, ServerUId). +live_indexes(ServerUId) -> + ra_log_snapshot_state:live_indexes(ra_log_snapshot_state, ServerUId). + send_segments(System, ServerUId, TidRanges, SegRefs) -> case ra_directory:pid_of(System, ServerUId) of undefined -> diff --git a/src/ra_log_snapshot_state.erl b/src/ra_log_snapshot_state.erl index 2f3e5f58e..621c86eba 100644 --- a/src/ra_log_snapshot_state.erl +++ b/src/ra_log_snapshot_state.erl @@ -4,6 +4,7 @@ insert/5, delete/2, smallest/2, + live_indexes/2, snapshot/2 ]). @@ -26,6 +27,11 @@ delete(Table, UId) -> smallest(Table, UId) when is_binary(UId) -> ets:lookup_element(Table, UId, 3, 0). +-spec live_indexes(ets:table(), ra:uid()) -> + ra:index(). +live_indexes(Table, UId) when is_binary(UId) -> + ets:lookup_element(Table, UId, 4, []). + -spec snapshot(ets:table(), ra:uid()) -> ra:index() | -1. snapshot(Table, UId) when is_binary(UId) -> diff --git a/src/ra_seq.erl b/src/ra_seq.erl index 187035903..da5161937 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -89,15 +89,19 @@ limit(CeilIdx, [{_, _} = T | Rem]) -> 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). +%% @doc adds two sequences together where To is +%% the "lower" sequence +-spec add(Add :: state(), To :: state()) -> state(). +add([], To) -> + To; +add(Add, []) -> + Add; +add(Add, To) -> + Fst = first(Add), + % {I, _} -> I; + % I -> I + % end, + fold(fun append/2, limit(Fst - 1, To), Add). -spec fold(fun ((ra:index(), Acc) -> Acc), Acc, state()) -> Acc when Acc :: term(). @@ -199,6 +203,8 @@ range(Seq) -> ra_range:new(first(Seq), last(Seq)). +-spec in_range(ra:range(), state()) -> + state(). in_range(_Range, []) -> []; in_range(undefined, _) -> diff --git a/test/ra_log_segment_writer_SUITE.erl b/test/ra_log_segment_writer_SUITE.erl index 93311f27c..2d3ffe6cb 100644 --- a/test/ra_log_segment_writer_SUITE.erl +++ b/test/ra_log_segment_writer_SUITE.erl @@ -41,7 +41,8 @@ all_tests() -> my_segments, upgrade_segment_name_format, skip_entries_lower_than_snapshot_index, - skip_all_entries_lower_than_snapshot_index + skip_all_entries_lower_than_snapshot_index, + live_indexes_1 ]. groups() -> @@ -839,6 +840,47 @@ skip_all_entries_lower_than_snapshot_index(Config) -> ok = gen_server:stop(TblWriterPid), ok. +live_indexes_1(Config) -> + Dir = ?config(wal_dir, Config), + UId = ?config(uid, Config), + {ok, TblWriterPid} = ra_log_segment_writer:start_link(#{system => default, + name => ?SEGWR, + data_dir => Dir}), + % first batch + Entries = [{1, 42, a}, + {2, 42, b}, + {3, 43, c}, + {4, 43, d}, + {5, 43, e}, + {6, 43, f} + ], + Mt = make_mem_table(UId, Entries), + Ranges = #{UId => [{ra_mt:tid(Mt), [ra_mt:range(Mt)]}]}, + %% update snapshot state table + ra_log_snapshot_state:insert(ra_log_snapshot_state, UId, 4, 2, [4, 2]), + ok = ra_log_segment_writer:accept_mem_tables(?SEGWR, Ranges, + make_wal(Config, "w1.wal")), + receive + {ra_log_event, {segments, _Tid, [{Fn, {2, 6}}]}} -> + SegmentFile = filename:join(?config(server_dir, Config), Fn), + {ok, Seg} = ra_log_segment:open(SegmentFile, #{mode => read}), + % assert only entries with a higher index than the snapshot + % have been written + ok = gen_server:stop(TblWriterPid), + ?assertExit({missing_key, 3}, read_sparse(Seg, [2, 3, 4])), + [ + {2, _, _}, + {4, _, _}, + {5, _, _}, + {6, _, _} + ] = read_sparse(Seg, [2, 4, 5, 6]) + after 3000 -> + flush(), + ok = gen_server:stop(TblWriterPid), + throw(ra_log_event_timeout) + end, + ok. + %%% Internal fake_mem_table(UId, Dir, Entries) -> From 41f91d2a3ac81a5e51fa0d91f4a985ccd158079e Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 18 Jun 2025 10:16:52 +0100 Subject: [PATCH 37/60] wip --- src/ra.erl | 1 + src/ra_kv.erl | 15 ++++++++-- src/ra_log.erl | 5 ++-- src/ra_log_segments.erl | 2 +- src/ra_log_snapshot_state.erl | 2 +- src/ra_snapshot.erl | 2 +- test/ra_log_2_SUITE.erl | 52 +++++++++++++++++++++++++++++++++++ 7 files changed, 71 insertions(+), 8 deletions(-) diff --git a/src/ra.erl b/src/ra.erl index 51ce47500..f33793c8f 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -140,6 +140,7 @@ start(Params) when is_list(Params) -> [ok = application:set_env(ra, Param, Value) || {Param, Value} <- Params], Res = application:ensure_all_started(ra), + ra_env:configure_logger(logger), _ = ra_system:start_default(), Res. diff --git a/src/ra_kv.erl b/src/ra_kv.erl index aa5fcb792..7a2a23d29 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -18,6 +18,7 @@ -export([ start_cluster/3, add_member/3, + member_overview/1, put/4, get/3, @@ -62,7 +63,8 @@ start_cluster(System, ClusterName, #{members := ServerIds}) #{id => Id, uid => UId, cluster_name => ClusterName, - log_init_args => #{uid => UId}, + log_init_args => #{uid => UId, + min_snapshot_interval => 0}, initial_members => ServerIds, machine => Machine} end || Id <- ServerIds], @@ -75,13 +77,22 @@ add_member(System, {Name, _} = Id, LeaderId) -> Config = #{id => Id, uid => UId, cluster_name => Name, - log_init_args => #{uid => UId}, + log_init_args => #{uid => UId, + min_snapshot_interval => 0}, initial_members => Members, machine => Machine}, ok = ra:start_server(System, Config), {ok, _, _} = ra:add_member(LeaderId, Id), ok. +member_overview(ServerId) -> + case ra:member_overview(ServerId) of + {ok, O, _} -> + maps:with([log, machine], O); + Err -> + Err + end. + %% client -spec put(ra:server_id(), key(), value(), non_neg_integer()) -> diff --git a/src/ra_log.erl b/src/ra_log.erl index 4b50c17bf..90c3a8a20 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -725,9 +725,7 @@ set_last_index(Idx, #?MODULE{cfg = Cfg, {state(), [effect()]}. handle_event({written, Term, WrittenSeq}, #?MODULE{cfg = Cfg, - % range = Range, snapshot_state = SnapState, - % first_index = FirstIdx, pending = Pend0} = State0) -> CurSnap = ra_snapshot:current(SnapState), %% gap detection @@ -825,7 +823,7 @@ handle_event(major_compaction, #?MODULE{reader = Reader0, snapshot_state = SS} = State) -> case ra_snapshot:current(SS) of {SnapIdx, _} -> - Effs = ra_log_segments:schedule_compaction(major,SnapIdx, + Effs = ra_log_segments:schedule_compaction(major, SnapIdx, LiveIndexes, Reader0), {State, Effs}; _ -> @@ -841,6 +839,7 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, snapshot_state = SnapState0} = State0) %% only update snapshot if it is newer than the last snapshot when SnapIdx >= FstIdx -> + % ?assert(ra_snapshot:pending(SnapState0) =/= undefined), SnapState1 = ra_snapshot:complete_snapshot(Snap, SnapKind, LiveIndexes, SnapState0), case SnapKind of diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index f3bc5c941..0cf8a6184 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -697,7 +697,7 @@ make_links(Dir, To, From) _ = prim_file:delete(SymFn), %% make a symlink from the compacted target segment to a new .link %% where the compacted indexes now can be found - ok = prim_file:make_symlink(To, SymFn), + ok = prim_file:make_symlink(filename:basename(To), SymFn), %% rename to link to replace original segment ok = prim_file:rename(SymFn, SegFn) end || FromFn <- From], diff --git a/src/ra_log_snapshot_state.erl b/src/ra_log_snapshot_state.erl index 621c86eba..ebc78786c 100644 --- a/src/ra_log_snapshot_state.erl +++ b/src/ra_log_snapshot_state.erl @@ -28,7 +28,7 @@ smallest(Table, UId) when is_binary(UId) -> ets:lookup_element(Table, UId, 3, 0). -spec live_indexes(ets:table(), ra:uid()) -> - ra:index(). + ra_seq:state(). live_indexes(Table, UId) when is_binary(UId) -> ets:lookup_element(Table, UId, 4, []). diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index c50274c79..25778b1b1 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -362,7 +362,7 @@ last_index_for(UId) -> end. -spec begin_snapshot(meta(), MacModule :: module(), - MacStateb :: term(), kind(), state()) -> + MacState :: term(), kind(), state()) -> {state(), [effect()]}. begin_snapshot(#{index := Idx, term := Term} = Meta, MacMod, MacState, SnapKind, #?MODULE{module = Mod, diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index a92af51ca..0046467b9 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -50,6 +50,7 @@ all_tests() -> snapshot_written_after_installation, oldcheckpoints_deleted_after_snapshot_install, append_after_snapshot_installation, + release_cursor_after_snapshot_installation, written_event_after_snapshot_installation, update_release_cursor, update_release_cursor_with_machine_version, @@ -1285,6 +1286,57 @@ append_after_snapshot_installation(Config) -> {[_, _], _} = ra_log_take(16, 17, Log), ok. +release_cursor_after_snapshot_installation(Config) -> + Log0 = ra_log_init(Config, #{min_snapshot_interval => 0}), + {0, 0} = ra_log:last_index_term(Log0), + Log1 = assert_log_events(write_n(1, 16, 2, Log0), + fun (L) -> + LW = ra_log:last_written(L), + {15, 2} == LW + end), + + Log2 = Log1, + + %% create snapshot chunk + Meta = meta(15, 2, [?N1]), + Chunk = create_snapshot_chunk(Config, Meta, [1, 5, 10], #{}), + SnapState0 = ra_log:snapshot_state(Log2), + {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log3, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log2)), + + run_effs(Effs4), + {15, 2} = ra_snapshot:current(ra_log:snapshot_state(Log3)), + + %% Write some entries + Log4 = assert_log_events(write_n(16, 20, 2, Log3), + fun (L) -> + LW = ra_log:last_written(L), + {19, 2} == LW + end), + + %% then take a snapshot + {Log5, Effs5} = ra_log:update_release_cursor(19, #{?N1 => new_peer(), + ?N2 => new_peer()}, + ?MODULE, [1, 5, 10, 17], Log4), + + run_effs(Effs5), + %% ensure snapshot index has been updated and 1 segment deleted + Log = assert_log_events(Log5, + fun (L) -> + {19, 2} == ra_log:snapshot_index_term(L) + % andalso + % length(find_segments(Config)) == 1 + end), + + ct:pal("Log ~p", [Log]), + + ok. + written_event_after_snapshot_installation(Config) -> logger:set_primary_config(level, all), %% simulates scenario where a server receives a written event from the wal From 9cdcfb54c6c367a3acefab80474f7a2ddaa8a986 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 18 Jun 2025 12:40:34 +0100 Subject: [PATCH 38/60] bug fix --- src/ra.erl | 12 +++++++++++- src/ra_server_proc.erl | 10 +++++----- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/ra.erl b/src/ra.erl index f33793c8f..0393fc929 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -140,7 +140,6 @@ start(Params) when is_list(Params) -> [ok = application:set_env(ra, Param, Value) || {Param, Value} <- Params], Res = application:ensure_all_started(ra), - ra_env:configure_logger(logger), _ = ra_system:start_default(), Res. @@ -153,6 +152,17 @@ start(Params) when is_list(Params) -> {ok, [Started]} | {error, term()} when Started :: term(). start_in(DataDir) -> + ra_env:configure_logger(logger), + LogFile = filename:join(DataDir, "ra.log"), + SaslFile = filename:join(DataDir, "ra_sasl.log"), + logger:set_primary_config(level, debug), + Config = #{config => #{file => LogFile}}, + logger:add_handler(ra_handler, logger_std_h, Config), + application:load(sasl), + application:set_env(sasl, sasl_error_logger, {file, SaslFile}), + application:stop(sasl), + application:start(sasl), + _ = error_logger:tty(false), start([{data_dir, DataDir}]). %% @doc Restarts a previously successfully started ra server diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index e3b5a3235..ce9e1a27d 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -661,7 +661,7 @@ candidate(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; candidate({call, From}, ping, State) -> {keep_state, State, [{reply, From, {pong, candidate}}]}; candidate(info, {node_event, _Node, _Evt}, State) -> @@ -717,7 +717,7 @@ pre_vote(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; pre_vote({call, From}, ping, State) -> {keep_state, State, [{reply, From, {pong, pre_vote}}]}; pre_vote(info, {node_event, _Node, _Evt}, State) -> @@ -807,7 +807,7 @@ follower(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; follower({call, From}, trigger_election, State) -> ?DEBUG("~ts: election triggered by ~w", [log_id(State), element(1, From)]), {keep_state, State, [{reply, From, ok}, @@ -933,7 +933,7 @@ receive_snapshot(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; receive_snapshot(EventType, Msg, State0) -> case handle_receive_snapshot(Msg, State0) of {receive_snapshot, State1, Effects} -> @@ -1035,7 +1035,7 @@ await_condition(EventType, {aux_command, Cmd}, State0) -> {State, Actions} = ?HANDLE_EFFECTS(Effects, EventType, State0#state{server_state = ServerState}), - {keep_state, State#state{server_state = ServerState}, Actions}; + {keep_state, State, Actions}; await_condition({call, From}, ping, State) -> {keep_state, State, [{reply, From, {pong, await_condition}}]}; await_condition({call, From}, trigger_election, State) -> From e2e27974f62b10a73f177b08d8e618fd914720b0 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 23 Jun 2025 16:45:14 +0100 Subject: [PATCH 39/60] Take live data sizes into account when grouping for compaction --- src/ra_log.erl | 11 +++-- src/ra_log_segment.erl | 48 +++++++++++++++------ src/ra_log_segments.erl | 66 ++++++++++++++++++---------- src/ra_server_proc.erl | 4 +- test/coordination_SUITE.erl | 2 - test/ra_log_segment_SUITE.erl | 35 +++++++++++++++ test/ra_log_segments_SUITE.erl | 78 ++++++++++++++++++++++++++++------ 7 files changed, 190 insertions(+), 54 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 90c3a8a20..416f47f16 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -188,7 +188,7 @@ init(#{uid := UId, segment_writer := SegWriter} = Names} } = Conf) -> Dir = server_data_dir(DataDir, UId), - MaxOpen = maps:get(max_open_segments, Conf, 5), + MaxOpen = maps:get(max_open_segments, Conf, 1), SnapModule = maps:get(snapshot_module, Conf, ?DEFAULT_SNAPSHOT_MODULE), %% this has to be patched by ra_server LogId = maps:get(log_id, Conf, UId), @@ -228,8 +228,12 @@ init(#{uid := UId, % segments it is currently processed have been finished MtRange = ra_mt:range(Mt0), SegRefs = my_segrefs(UId, SegWriter), + SegmentMaxCount = maps:get(segment_max_entries, Conf, ?SEGMENT_MAX_ENTRIES), + SegmentMaxSize = maps:get(segment_max_size_bytes, Conf, ?SEGMENT_MAX_SIZE_B), + CompConf = #{max_size => SegmentMaxSize, + max_count => SegmentMaxCount}, Reader = ra_log_segments:init(UId, Dir, MaxOpen, AccessPattern, SegRefs, - Counter, LogId), + Counter, CompConf, LogId), SegmentRange = ra_log_segments:range(Reader), %% TODO: check ra_range:add/2 actually performas the correct logic we expect Range = ra_range:add(MtRange, SegmentRange), @@ -1306,13 +1310,14 @@ release_resources(MaxOpenSegments, AccessPattern, counter = Counter}, reader = Reader} = State) -> ActiveSegs = ra_log_segments:segment_refs(Reader), + CompConf = ra_log_segments:compaction_conf(Reader), % close all open segments % deliberately ignoring return value _ = ra_log_segments:close(Reader), %% open a new segment with the new max open segment value State#?MODULE{reader = ra_log_segments:init(UId, Dir, MaxOpenSegments, AccessPattern, ActiveSegs, - Counter, LogId)}. + Counter, CompConf, LogId)}. %%% Local functions diff --git a/src/ra_log_segment.erl b/src/ra_log_segment.erl index 5c42fbbc4..ec06630e8 100644 --- a/src/ra_log_segment.erl +++ b/src/ra_log_segment.erl @@ -23,6 +23,7 @@ filename/1, segref/1, info/1, + info/2, is_same_as/2, copy/3]). @@ -454,17 +455,23 @@ segref(Filename) -> close(Seg), SegRef. --spec info(file:filename_all()) -> - #{size => non_neg_integer(), - max_count => non_neg_integer(), - file_type => regular | symlink, - ctime => integer(), - links => non_neg_integer(), - num_entries => non_neg_integer(), - ref => option(ra_log:segment_ref()), - indexes => ra_seq:state() - }. -info(Filename) +-type infos() :: #{size => non_neg_integer(), + max_count => non_neg_integer(), + file_type => regular | symlink, + ctime => integer(), + links => non_neg_integer(), + num_entries => non_neg_integer(), + ref => option(ra_log:segment_ref()), + indexes => ra_seq:state(), + live_size => non_neg_integer() + }. + +-spec info(file:filename_all()) -> infos(). +info(Filename) -> + info(Filename, undefined). + +-spec info(file:filename_all(), option(ra_seq:state())) -> infos(). +info(Filename, Live0) when not is_tuple(Filename) -> %% TODO: this can be much optimised by a specialised index parsing %% function @@ -475,14 +482,29 @@ info(Filename) ctime = CTime}} = file:read_link_info(Filename, [raw, {time, posix}]), - Info = #{size => Seg#state.data_write_offset, + AllIndexesSeq = ra_seq:from_list(maps:keys(Index)), + Live = case Live0 of + undefined -> + AllIndexesSeq; + _ -> + Live0 + end, + LiveSize = ra_seq:fold(fun (I, Acc) -> + {_, _, Sz, _} = maps:get(I, Index), + Acc + Sz + end, 0, Live), + Info = #{ + size => Seg#state.data_write_offset, + index_size => Seg#state.data_start, file_type => T, links => Links, ctime => CTime, max_count => max_count(Seg), num_entries => maps:size(Index), ref => segref(Seg), - indexes => ra_seq:from_list(maps:keys(Index)) + live_size => LiveSize, + %% TODO: this is most likely just here for debugging + indexes => AllIndexesSeq }, close(Seg), Info. diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index 0cf8a6184..abbfebef9 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -11,7 +11,7 @@ -include_lib("kernel/include/file.hrl"). -export([ - init/7, + init/8, close/1, update_segments/2, schedule_compaction/4, @@ -27,7 +27,8 @@ exec_read_plan/6, fetch_term/2, info/1, - purge_symlinks/2 + purge_symlinks/2, + compaction_conf/1 ]). -include("ra.hrl"). @@ -36,13 +37,16 @@ -define(SYMLINK_KEEPFOR_S, 60). +-type compaction_conf() :: #{max_count => non_neg_integer(), + max_size => non_neg_integer()}. -type access_pattern() :: sequential | random. %% holds static or rarely changing fields -record(cfg, {uid :: ra_uid(), log_id = "" :: unicode:chardata(), counter :: undefined | counters:counters_ref(), directory :: file:filename(), - access_pattern = random :: access_pattern() + access_pattern = random :: access_pattern(), + compaction_conf :: compaction_conf() }). -type segment_ref() :: ra_log:segment_ref(). @@ -74,14 +78,16 @@ -spec init(ra_uid(), file:filename_all(), non_neg_integer(), access_pattern(), [segment_ref()], undefined | counters:counters_ref(), + map(), unicode:chardata()) -> state(). -init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, Counter, LogId) +init(UId, Dir, MaxOpen, AccessPattern, SegRefs0, Counter, CompConf, LogId) when is_binary(UId) -> Cfg = #cfg{uid = UId, log_id = LogId, counter = Counter, directory = Dir, - access_pattern = AccessPattern}, + access_pattern = AccessPattern, + compaction_conf = CompConf}, FlruHandler = fun ({_, Seg}) -> _ = ra_log_segment:close(Seg), decr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1) @@ -150,6 +156,7 @@ update_segments(NewSegmentRefs, #?STATE{open_segments = Open0, [ra_server:effect()]. schedule_compaction(Type, SnapIdx, LiveIndexes, #?MODULE{cfg = #cfg{log_id = LogId, + compaction_conf = CompConf, directory = Dir} = Cfg} = State) -> case compactable_segrefs(SnapIdx, State) of [] -> @@ -171,21 +178,25 @@ schedule_compaction(Type, SnapIdx, LiveIndexes, Self = self(), Fun = fun () -> ok = incr_counter(Cfg, ?C_RA_LOG_COMPACTIONS_MAJOR_COUNT, 1), - MajConf = #{dir => Dir}, + MajConf = CompConf#{dir => Dir}, Result = major_compaction(MajConf, SegRefs, LiveIndexes), - %% TODO: this could be done on a timer if more - %% timely symlink cleanup is needed - purge_symlinks(Dir, ?SYMLINK_KEEPFOR_S), %% need to update the ra_servers list of seg refs %% _before_ the segments can actually be deleted Self ! {ra_log_event, {compaction_result, Result}}, + %% TODO: this could be done on a timer if more + %% timely symlink cleanup is needed + purge_symlinks(Dir, ?SYMLINK_KEEPFOR_S), ok end, [{bg_work, Fun, fun (Err) -> + %% send an empty compaction result to ensure the + %% a future compaction can be performed (TODO:) + Self ! {ra_log_event, + {compaction_result, #compaction_result{}}}, ?WARN("~ts: Major compaction failed with ~p", [LogId, Err]), ok end}] @@ -266,6 +277,10 @@ segment_ref_count(#?STATE{segment_refs = SegmentRefs}) -> range(#?STATE{range = Range}) -> Range. +-spec compaction_conf(state()) -> map(). +compaction_conf(#?STATE{cfg = #cfg{compaction_conf = Conf}}) -> + Conf. + -spec num_open_segments(state()) -> non_neg_integer(). num_open_segments(#?STATE{open_segments = Open}) -> ra_flru:size(Open). @@ -572,7 +587,7 @@ list_files(Dir, Ext, Fun) -> [] end. -major_compaction(#{dir := Dir}, SegRefs, LiveIndexes) -> +major_compaction(#{dir := Dir} = CompConf, SegRefs, LiveIndexes) -> {Compactable, Delete} = lists:foldl(fun({Fn0, Range} = S, {Comps, Del}) -> @@ -584,7 +599,7 @@ major_compaction(#{dir := Dir}, SegRefs, LiveIndexes) -> %% get the info map from each %% potential segment Fn = filename:join(Dir, Fn0), - Info = ra_log_segment:info(Fn), + Info = ra_log_segment:info(Fn, Seq), {[{Info, Seq, S} | Comps], Del} end end, {[], []}, SegRefs), @@ -602,7 +617,8 @@ major_compaction(#{dir := Dir}, SegRefs, LiveIndexes) -> ok = prim_file:delete(filename:join(Dir, F)) end || F <- UnusedFiles], %% group compactable - CompactionGroups = compaction_groups(lists:reverse(Compactable), []), + CompactionGroups = compaction_groups(lists:reverse(Compactable), [], + CompConf), Compacted0 = [begin %% create a new segment with .compacting extension @@ -706,31 +722,37 @@ make_links(Dir, To, From) with_ext(Fn, Ext) when is_binary(Fn) andalso is_list(Ext) -> <<(filename:rootname(Fn))/binary, (ra_lib:to_binary(Ext))/binary>>. -compaction_groups([], Groups) -> +compaction_groups([], Groups, _Conf) -> lists:reverse(Groups); -compaction_groups(Infos, Groups) -> - case take_group(Infos, #{max_count => 128}, []) of +compaction_groups(Infos, Groups, Conf) -> + case take_group(Infos, Conf, []) of {Group, RemInfos} -> - compaction_groups(RemInfos, [Group | Groups]) + compaction_groups(RemInfos, [Group | Groups], Conf) end. %% TODO: try to take potential size into account take_group([], _, Acc) -> {lists:reverse(Acc), []}; -take_group([{#{num_entries := NumEnts}, Live, {_, _}} = E | Rem] = All, - #{max_count := Mc}, Acc) -> +take_group([{#{num_entries := NumEnts, + live_size := LiveSize}, Live, {_, _}} = E | Rem] = All, + #{max_count := Mc, + max_size := MaxSz}, Acc) -> Num = ra_seq:length(Live), - case Num < NumEnts div 2 of + case Num / NumEnts < 0.5 of true -> - case Mc - Num < 0 of + case Mc - Num < 0 orelse + MaxSz - LiveSize < 0 of true -> {lists:reverse(Acc), All}; false -> - take_group(Rem, #{max_count => Mc - Num}, [E | Acc]) + take_group(Rem, #{max_count => Mc - Num, + max_size => MaxSz - LiveSize}, + [E | Acc]) end; %% skip this secment false when Acc == [] -> - take_group(Rem, #{max_count => Mc}, Acc); + take_group(Rem, #{max_count => Mc, + max_size => MaxSz}, Acc); false -> {lists:reverse(Acc), Rem} end. diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index ce9e1a27d..88305b7e0 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -1927,6 +1927,8 @@ send_snapshots(Id, Term, {_, ToNode} = To, ChunkSize, case ra_snapshot:indexes(ra_snapshot:current_snapshot_dir(SnapState)) of {ok, [_|_] = Indexes} -> %% there are live indexes to send before the snapshot + %% %% TODO: only send live indexes higher than the follower's + %% last_applied index Idxs = lists:reverse(ra_seq:expand(Indexes)), Flru = lists:foldl( fun (Is, F0) -> @@ -1937,7 +1939,7 @@ send_snapshots(Id, Term, {_, ToNode} = To, ChunkSize, data = Ents}, _Res1 = gen_statem:call(To, RPC1, {dirty_timeout, InstallTimeout}), - %% TODO: assert REs1 is successful + %% TODO: assert Res1 is successful F end, undefined, ra_lib:lists_chunk(16, Idxs)), _ = ra_flru:evict_all(Flru), diff --git a/test/coordination_SUITE.erl b/test/coordination_SUITE.erl index 034ed8c4c..d47e64f91 100644 --- a/test/coordination_SUITE.erl +++ b/test/coordination_SUITE.erl @@ -1431,8 +1431,6 @@ snapshot_installed(#{machine_version := _, node_setup(DataDir) -> ok = ra_lib:make_dir(DataDir), - % NodeDir = filename:join(DataDir, atom_to_list(node())), - % ok = ra_lib:make_dir(DataDir), LogFile = filename:join(DataDir, "ra.log"), SaslFile = filename:join(DataDir, "ra_sasl.log"), logger:set_primary_config(level, debug), diff --git a/test/ra_log_segment_SUITE.erl b/test/ra_log_segment_SUITE.erl index 990db1c89..5e2cc8922 100644 --- a/test/ra_log_segment_SUITE.erl +++ b/test/ra_log_segment_SUITE.erl @@ -38,6 +38,8 @@ all_tests() -> corrupted_segment, large_segment, segref, + info, + info_2, versions_v1, copy ]. @@ -207,6 +209,39 @@ segref(Config) -> {<<"seg1.seg">>, {1, 1}} = ra_log_segment:segref(Seg1), ok. +info(Config) -> + Dir = ?config(data_dir, Config), + Fn = filename:join(Dir, "seg1.seg"), + {ok, Seg0} = ra_log_segment:open(Fn, #{max_count => 128}), + Info1 = ra_log_segment:info(Fn), + ?assertMatch(#{ref := undefined}, Info1), + {ok, Seg1} = ra_log_segment:append(Seg0, 1, 2, <<"Adsf">>), + _ = ra_log_segment:flush(Seg1), + Info2 = ra_log_segment:info(Fn), + ?assertMatch(#{ref := {<<"seg1.seg">>, {1, 1}}}, Info2), + ok. + +info_2(Config) -> + %% passes live indexes which will result in additional info keys + Dir = ?config(data_dir, Config), + Fn = filename:join(Dir, "seg1.seg"), + {ok, Seg0} = ra_log_segment:open(Fn, #{max_count => 128}), + Info1 = ra_log_segment:info(Fn, []), + ?assertMatch(#{ref := undefined, + live_size := 0}, Info1), + {ok, Seg1} = ra_log_segment:append(Seg0, 1, 2, <<"Adsf">>), + {ok, Seg2} = ra_log_segment:append(Seg1, 2, 2, <<"Adsf">>), + _ = ra_log_segment:flush(Seg2), + Info2 = ra_log_segment:info(Fn, [1]), + ?assertMatch(#{ref := {<<"seg1.seg">>, {1, 2}}, + num_entries := 2, + live_size := 4}, Info2), + Info3 = ra_log_segment:info(Fn), + %% info/1 assumes all indexes are "live" + ?assertMatch(#{ref := {<<"seg1.seg">>, {1, 2}}, + num_entries := 2, + live_size := 8}, Info3), + ok. full_file(Config) -> Dir = ?config(data_dir, Config), diff --git a/test/ra_log_segments_SUITE.erl b/test/ra_log_segments_SUITE.erl index e7b6d233b..8da688050 100644 --- a/test/ra_log_segments_SUITE.erl +++ b/test/ra_log_segments_SUITE.erl @@ -27,6 +27,7 @@ all_tests() -> recover2, basics, major, + major_max_size, minor, overwrite, result_after_segments, @@ -42,7 +43,10 @@ init_per_testcase(TestCase, Config) -> PrivDir = ?config(priv_dir, Config), Dir = filename:join(PrivDir, TestCase), ok = ra_lib:make_dir(Dir), + CompConf = #{max_count => 128, + max_size => 128_000}, [{uid, atom_to_binary(TestCase, utf8)}, + {comp_conf, CompConf}, {test_case, TestCase}, {dir, Dir} | Config]. @@ -69,7 +73,7 @@ result_after_segments(Config) -> end} ], SegConf = #{max_count => 128}, - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ct:pal("infos ~p", [infos(Dir)]), ok. @@ -95,7 +99,7 @@ result_after_segments_overwrite(Config) -> {assert, 2, lists:seq(128 * 3, 128 * 4)} ], SegConf = #{max_count => 128}, - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ct:pal("infos ~p", [infos(Dir)]), ok. @@ -129,7 +133,7 @@ recover1(Config) -> ], ct:pal("infos ~p", [infos(Dir)]), SegConf = #{max_count => 128}, - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ok. @@ -189,7 +193,7 @@ recover2(Config) -> ], SegConf = #{max_count => 128}, - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ct:pal("infos ~p", [infos(Dir)]), ok. @@ -233,7 +237,7 @@ basics(Config) -> SegConf = #{max_count => 128}, - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ct:pal("infos ~p", [infos(Dir)]), @@ -276,7 +280,7 @@ minor(Config) -> SegConf = #{max_count => 128}, Dir = ?config(dir, Config), - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ok. @@ -302,10 +306,48 @@ overwrite(Config) -> SegConf = #{max_count => 128}, Dir = ?config(dir, Config), - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ok. +major_max_size(Config) -> + %% this test could compact 3 segemtns into one just based on entry counts + %% however the max_size configuration needs to be taken into account + %% with the compaction grouping and not create an oversized taget segment + Dir = ?config(dir, Config), + Data = crypto:strong_rand_bytes(2000), + Entries = [{I, 1, term_to_binary(Data)} + || I <- lists:seq(1, 128 * 4)], + LiveList = lists:seq(1, 30) ++ + lists:seq(128, 128 + 30) ++ + lists:seq(256, 256 + 30), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, Entries}, + {assert, 1, lists:seq(1, 128 * 4)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 4 + end}, + {major, 128 * 4, Live}, + handle_compaction_result, + {assert, 1, LiveList}, + {assert, fun (S) -> + %% infos contain one symlink + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + Symlinks = [I || #{file_type := symlink} = I <- Infos], + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 andalso + length(Infos) == 4 andalso + length(Symlinks) == 1 + end} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. major(Config) -> Dir = ?config(dir, Config), @@ -355,7 +397,7 @@ major(Config) -> ], SegConf = #{max_count => 128}, - Segs0 = ra_log_segments_init(?config(uid, Config), Dir, seg_refs(Dir)), + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ok. @@ -443,14 +485,21 @@ run_scenario(_, Segs, []) -> Segs; run_scenario(Config, Segs0, [reinit | Rem]) -> Dir = ?config(dir, Config), + CompConf = ?config(comp_conf, Config), ra_log_segments:close(Segs0), Segs = ra_log_segments:init(?config(uid, Config), Dir, 1, random, - seg_refs(Dir), undefined, ""), + seg_refs(Dir), undefined, CompConf, ""), ?FUNCTION_NAME(Config, Segs, Rem); -run_scenario(Config, Segs0, [{entries, Term, Indexes} | Rem]) -> +run_scenario(Config, Segs0, [{entries, Term, IndexesOrEntries} | Rem]) -> SegConf = ?config(seg_conf, Config), Seg0 = open_last_segment(Config, SegConf), - Entries = [{I, Term, term_to_binary(<<"data1">>)} || I <- Indexes], + Entries = case is_tuple(hd(IndexesOrEntries)) of + true -> + IndexesOrEntries; + false -> + [{I, Term, term_to_binary(<<"data1">>)} + || I <- IndexesOrEntries] + end, {Seg, Refs} = append_to_segment(Seg0, Entries, [], SegConf), _ = ra_log_segment:close(Seg), {Segs, _Overwritten} = ra_log_segments:update_segments(Refs, Segs0), @@ -502,6 +551,9 @@ run_scenario(Config, Segs0, [{print, What} | Rem]) -> with_ext(Fn, Ext) when is_binary(Fn) andalso is_list(Ext) -> <<(filename:rootname(Fn))/binary, (ra_lib:to_binary(Ext))/binary>>. -ra_log_segments_init(UId, Dir, SegRefs) -> +ra_log_segments_init(Config, Dir, SegRefs) -> + UId = ?config(uid, Config), + CompConf = ?config(comp_conf, Config), ra_log_segments:init(UId, Dir, 1, random, - SegRefs, undefined, ""). + SegRefs, undefined, + CompConf, ""). From 1e9f131cf1184c5ad987d65c03e2ccb202ec88c8 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 24 Jun 2025 12:48:15 +0100 Subject: [PATCH 40/60] Fix snapshot live indexes replication bug --- src/ra_log.erl | 20 +++++++--- src/ra_log_segment.erl | 4 +- src/ra_log_segments.erl | 33 ++++++++++------ src/ra_mt.erl | 7 +++- src/ra_server.erl | 26 +++++++++---- src/ra_server_proc.erl | 34 +++++++++++------ test/ra_kv_SUITE.erl | 69 +++++++++++++++++++++++++++++++++- test/ra_log_segments_SUITE.erl | 44 +++++++++++++++++++++- 8 files changed, 198 insertions(+), 39 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 416f47f16..ec975c794 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -706,10 +706,21 @@ last_written(#?MODULE{last_written_index_term = LWTI}) -> {ok, state()} | {not_found, state()}. set_last_index(Idx, #?MODULE{cfg = Cfg, range = Range, + snapshot_state = SnapState, last_written_index_term = {LWIdx0, _}} = State0) -> + Cur = ra_snapshot:current(SnapState), case fetch_term(Idx, State0) of - {undefined, State} -> + {undefined, State} when element(1, Cur) =/= Idx -> + %% not found and Idx isn't equal to latest snapshot index {not_found, State}; + {_, State} when element(1, Cur) =:= Idx -> + {_, SnapTerm} = Cur, + %% Idx is equal to the current snapshot + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, Idx), + {ok, State#?MODULE{range = ra_range:limit(Idx + 1, Range), + last_term = SnapTerm, + last_written_index_term = Cur}}; {Term, State1} -> LWIdx = min(Idx, LWIdx0), {LWTerm, State2} = fetch_term(LWIdx, State1), @@ -1107,12 +1118,11 @@ assert(#?MODULE{cfg = #cfg{log_id = LogId}, range = Range, snapshot_state = SnapState, current_snapshot = CurrSnap, - live_indexes = LiveIndexes, - mem_table = _Mt + live_indexes = LiveIndexes } = State) -> %% TODO: remove this at some point? - ?DEBUG("~ts: ra_log: asserting Range ~p Snapshot ~p LiveIndexes ~p", - [LogId, Range, CurrSnap, LiveIndexes]), + ?DEBUG("~ts: ra_log: asserting Range ~p Snapshot ~p", + [LogId, Range, CurrSnap]), %% perform assertions to ensure log state is correct ?assert(CurrSnap =:= ra_snapshot:current(SnapState)), ?assert(Range == undefined orelse diff --git a/src/ra_log_segment.erl b/src/ra_log_segment.erl index ec06630e8..cc42889df 100644 --- a/src/ra_log_segment.erl +++ b/src/ra_log_segment.erl @@ -456,6 +456,7 @@ segref(Filename) -> SegRef. -type infos() :: #{size => non_neg_integer(), + index_size => non_neg_integer(), max_count => non_neg_integer(), file_type => regular | symlink, ctime => integer(), @@ -493,8 +494,7 @@ info(Filename, Live0) {_, _, Sz, _} = maps:get(I, Index), Acc + Sz end, 0, Live), - Info = #{ - size => Seg#state.data_write_offset, + Info = #{size => Seg#state.data_write_offset, index_size => Seg#state.data_start, file_type => T, links => Links, diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index abbfebef9..6248842dd 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -726,32 +726,43 @@ compaction_groups([], Groups, _Conf) -> lists:reverse(Groups); compaction_groups(Infos, Groups, Conf) -> case take_group(Infos, Conf, []) of + {[], RemInfos} -> + compaction_groups(RemInfos, Groups, Conf); {Group, RemInfos} -> compaction_groups(RemInfos, [Group | Groups], Conf) end. -%% TODO: try to take potential size into account take_group([], _, Acc) -> {lists:reverse(Acc), []}; take_group([{#{num_entries := NumEnts, - live_size := LiveSize}, Live, {_, _}} = E | Rem] = All, - #{max_count := Mc, + index_size := IdxSz, + size := Sz, + live_size := LiveSz}, Live, {_, _}} = E | Rem] = All, + #{max_count := MaxCnt, max_size := MaxSz}, Acc) -> - Num = ra_seq:length(Live), - case Num / NumEnts < 0.5 of + NumLive = ra_seq:length(Live), + AllDataSz = Sz - IdxSz, + %% group on either num relaimable entries or data saved + case NumLive / NumEnts < 0.5 orelse + LiveSz / AllDataSz < 0.5 of + %% there are fewer than half live entries in the segment true -> - case Mc - Num < 0 orelse - MaxSz - LiveSize < 0 of + %% check that adding this segment to the current group will no + %% exceed entry or size limits + case MaxCnt - NumLive < 0 orelse + MaxSz - LiveSz < 0 of true -> + %% adding this segment to the group will exceed limits + %% so returning current group {lists:reverse(Acc), All}; false -> - take_group(Rem, #{max_count => Mc - Num, - max_size => MaxSz - LiveSize}, + take_group(Rem, #{max_count => MaxCnt - NumLive, + max_size => MaxSz - LiveSz}, [E | Acc]) end; - %% skip this secment + %% skip this segment false when Acc == [] -> - take_group(Rem, #{max_count => Mc, + take_group(Rem, #{max_count => MaxCnt, max_size => MaxSz}, Acc); false -> {lists:reverse(Acc), Rem} diff --git a/src/ra_mt.erl b/src/ra_mt.erl index 55d3d0116..4e57d74ce 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -120,7 +120,10 @@ insert({Idx, _, _} = _Entry, end. -spec insert_sparse(log_entry(), undefined | ra:index(), state()) -> - {ok, state()} | {error, overwriting | gap_detected | limit_reached}. + {ok, state()} | {error, + overwriting | + gap_detected | + limit_reached}. insert_sparse({Idx, _, _} = Entry, LastIdx, #?MODULE{tid = Tid, indexes = Seq} = State) -> @@ -299,6 +302,8 @@ get_items(Indexes, #?MODULE{} = State) -> non_neg_integer(). delete(undefined) -> 0; +delete({indexes, _Tid, []}) -> + 0; delete({indexes, Tid, Seq}) -> NumToDelete = ra_seq:length(Seq), Start = ra_seq:first(Seq), diff --git a/src/ra_server.erl b/src/ra_server.erl index 631f43a70..be88a1006 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -1427,7 +1427,7 @@ handle_follower(#install_snapshot_rpc{term = Term, meta = #{index := SnapIdx, machine_version := SnapMacVer} = Meta, leader_id = LeaderId, - chunk_state = {Num, _ChunkFlag}} = Rpc, + chunk_state = {Num, ChunkFlag}} = Rpc, #{cfg := #cfg{log_id = LogId, machine_version = MacVer}, log := Log0, last_applied := LastApplied, @@ -1443,7 +1443,17 @@ handle_follower(#install_snapshot_rpc{term = Term, [LogId, SnapIdx, Term]), SnapState0 = ra_log:snapshot_state(Log0), {ok, SS} = ra_snapshot:begin_accept(Meta, SnapState0), - Log = ra_log:set_snapshot_state(SS, Log0), + Log1 = ra_log:set_snapshot_state(SS, Log0), + + %% if the snaphost includes pre entries (live entries) then we need + %% to reset the log to the last applied index to avoid issues + Log = case ChunkFlag of + pre -> + {ok, L} = ra_log:set_last_index(LastApplied, Log1), + L; + _ -> + Log1 + end, {receive_snapshot, update_term(Term, State0#{log => Log, leader_id => LeaderId}), [{next_event, Rpc}, {record_leader_msg, LeaderId}]}; @@ -1538,17 +1548,18 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, last_index = SnapIndex}, case ChunkFlag of pre when is_list(ChunkOrEntries) -> - %% TODO: we may need to reset the log here to - %% the last applied index as we - %% dont know for sure indexes after last applied + %% reset last index to last applied + %% as we dont know for sure indexes after last applied %% are of the right term {LastIndex, _} = ra_log:last_index_term(Log00), - {Log0, _} = lists:foldl( + {Log, _} = lists:foldl( fun ({I, _, _} = E, {L0, LstIdx}) -> {ok, L} = ra_log:write_sparse(E, LstIdx, L0), {L, I} end, {Log00, LastIndex}, ChunkOrEntries), - State = update_term(Term, State0#{log => Log0}), + ?DEBUG("~ts: receiving snapshot log last index ~p", + [LogId, ra_log:last_index_term(Log)]), + State = update_term(Term, State0#{log => Log}), {receive_snapshot, State, [{reply, Reply}]}; next -> SnapState0 = ra_log:snapshot_state(Log00), @@ -1606,6 +1617,7 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, membership => get_membership(ClusterIds, State0), machine_state => MacState}), + put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_APPLIED, SnapIndex), %% it was the last snapshot chunk so we can revert back to %% follower status {follower, persist_last_applied(State), [{reply, Reply} | diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index 88305b7e0..b8fef5606 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -1535,7 +1535,9 @@ handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, _Id, Term}}, #state{server_state = SS0, monitors = Monitors, conf = #conf{snapshot_chunk_size = ChunkSize, - install_snap_rpc_timeout = InstallSnapTimeout} = Conf} = State0, + log_id = LogId, + install_snap_rpc_timeout = InstallSnapTimeout} = Conf} + = State0, Actions) -> case lists:member(ToNode, [node() | nodes()]) of true -> @@ -1546,7 +1548,7 @@ handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, _Id, Term}}, Pid = spawn(fun () -> try send_snapshots(Id, Term, To, ChunkSize, InstallSnapTimeout, - SnapState, Machine) of + SnapState, Machine, LogId) of _ -> ok catch C:timeout:S -> @@ -1906,29 +1908,39 @@ read_entries0(From, Idxs, #state{server_state = #{log := Log}} = State) -> {keep_state, State, [{reply, From, {ok, ReadState}}]}. send_snapshots(Id, Term, {_, ToNode} = To, ChunkSize, - InstallTimeout, SnapState, Machine) -> + InstallTimeout, SnapState, Machine, LogId) -> Context = ra_snapshot:context(SnapState, ToNode), {ok, #{machine_version := SnapMacVer} = Meta, ReadState} = ra_snapshot:begin_read(SnapState, Context), - %% only send the snapshot if the target server can accept it - %% TODO: grab the last_applied index also and use this to floor - %% the live indexes + %% TODO: consolidate getting the context, machinve version and last + %% applied index in one rpc, and handle errors TheirMacVer = erpc:call(ToNode, ra_machine, version, [Machine]), - %% rpc the check what their + %% only send the snapshot if the target server can accept it case SnapMacVer > TheirMacVer of true -> + ?DEBUG("~ts: not sending snapshot to ~w as their machine version ~b " + "is lower than snapshot machine version ~b", + [LogId, To, TheirMacVer, SnapMacVer]), ok; false -> + #{last_applied := LastApplied} = erpc:call(ToNode, + ra_counters, + counters, + [To, [last_applied]]), RPC = #install_snapshot_rpc{term = Term, leader_id = Id, meta = Meta}, - case ra_snapshot:indexes(ra_snapshot:current_snapshot_dir(SnapState)) of - {ok, [_|_] = Indexes} -> + case ra_snapshot:indexes( + ra_snapshot:current_snapshot_dir(SnapState)) of + {ok, [_|_] = Indexes0} -> + %% remove all indexes lower than the target's last applied + Indexes = ra_seq:floor(LastApplied + 1, Indexes0), + ?DEBUG("~ts: sending live indexes ~w to ~w ", + [LogId, ra_seq:range(Indexes), To]), %% there are live indexes to send before the snapshot - %% %% TODO: only send live indexes higher than the follower's - %% last_applied index + %% TODO: write ra_seq:list_chunk function to avoid expansion Idxs = lists:reverse(ra_seq:expand(Indexes)), Flru = lists:foldl( fun (Is, F0) -> diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index 4baf0e1d9..154da5ffc 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -20,7 +20,8 @@ all() -> all_tests() -> [ - basics + basics, + snapshot_replication ]. groups() -> @@ -56,6 +57,71 @@ end_per_testcase(_TestCase, _Config) -> %%%=================================================================== +snapshot_replication(_Config) -> + Members = [{kv1, node()}, {kv2, node()}], + KvId = hd(Members), + {ok, _, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, + #{members => Members}), + ra:transfer_leadership(KvId, KvId), + {ok, #{}} = ra_kv:put(KvId, <<"k1">>, <<"k1-value01">>, 5000), + %% write 10k entries of the same key + [{ok, #{}} = ra_kv:put(KvId, integer_to_binary(I), I, 5000) + || I <- lists:seq(1, 5000)], + + ?assertMatch({ok, #{machine := #{num_keys := _}}, KvId}, + ra:member_overview(KvId)), + ra_log_wal:force_roll_over(ra_log_wal), + %% wait for rollover processing + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + %% wait for segment writer to process + ra_log_segment_writer:await(ra_log_segment_writer), + %% promt ra_kv to take a snapshot + ok = ra:aux_command(KvId, take_snapshot), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{snapshot_index := SnapIdx, + last_index := LastIdx}}, _} = + ra:member_overview(KvId), + SnapIdx == LastIdx + end, 100, 100), + + KvId3 = {kv3, node()}, + ok = ra_kv:add_member(?SYS, KvId3, KvId), + KvId3Pid = whereis(kv3), + ?assert(is_pid(KvId3Pid)), + {ok, #{}} = ra_kv:put(KvId, <<"k3">>, <<"k3-value">>, 5000), + {ok, #{}} = ra_kv:put(KvId, <<"k4">>, <<"k4-value">>, 5000), + ok = ra:aux_command(KvId, take_snapshot), + % timer:sleep(1000), + {ok, #{log := #{last_index := Kv1LastIndex }}, _} = ra:member_overview(KvId), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{last_index := LastIdx}}, _} = + ra:member_overview(KvId3), + Kv1LastIndex == LastIdx + end, 100, 100), + ct:pal("counters ~p", [ra_counters:counters(KvId3, [last_applied])]), + %% ensure kv3 did not crash during snapshot replication + ?assertEqual(KvId3Pid, whereis(kv3)), + + ok = ra:stop_server(default, KvId3), + + {ok, #{}} = ra_kv:put(KvId, <<"k5">>, <<"k5-value">>, 5000), + {ok, #{}} = ra_kv:put(KvId, <<"k6">>, <<"k6-value">>, 5000), + ok = ra:aux_command(KvId, take_snapshot), + + ok = ra:restart_server(default, KvId3), + {ok, #{log := #{last_index := Kv1LastIndex2}}, _} = ra:member_overview(KvId), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{last_index := LastIdx}}, _} = + ra:member_overview(KvId3), + Kv1LastIndex2 == LastIdx + end, 100, 100), + + ra:delete_cluster([KvId, {kv2, node()}, KvId3]), + ok. + basics(_Config) -> Members = [{kv1, node()}], KvId = hd(Members), @@ -136,4 +202,5 @@ basics(_Config) -> undefined, 1000), ?assertEqual(Reads4, Reads5), ct:pal("counters ~p", [ra_counters:overview(KvId)]), + ra:delete_cluster([KvId, KvId2]), ok. diff --git a/test/ra_log_segments_SUITE.erl b/test/ra_log_segments_SUITE.erl index 8da688050..e517098ff 100644 --- a/test/ra_log_segments_SUITE.erl +++ b/test/ra_log_segments_SUITE.erl @@ -28,6 +28,7 @@ all_tests() -> basics, major, major_max_size, + major_max_size_2, minor, overwrite, result_after_segments, @@ -310,6 +311,48 @@ overwrite(Config) -> run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), ok. +major_max_size_2(Config) -> + %% this test could compact 3 segemtns into one just based on entry counts + %% however the max_size configuration needs to be taken into account + %% with the compaction grouping and not create an oversized taget segment + Dir = ?config(dir, Config), + Data1 = crypto:strong_rand_bytes(1000), + Data2 = crypto:strong_rand_bytes(2000), + Entries1 = [{I, 1, term_to_binary(Data1)} + || I <- lists:seq(1, 64)], + Entries2 = [{I, 1, term_to_binary(Data2)} + || I <- lists:seq(65, 130)], + %% only the smaller entries are still live + LiveList = lists:seq(1, 65), + Live = ra_seq:from_list(LiveList), + Live = ra_seq:from_list(LiveList), + Scen = + [ + {entries, 1, Entries1}, + {entries, 1, Entries2}, + {assert, 1, lists:seq(1, 130)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 + end}, + {major, 130, Live}, + handle_compaction_result, + {assert, fun (S) -> + %% infos contain one symlink + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + %% assert this segment was compacted due to potential + %% data reclamation and not just for entries + ?assertMatch(#{num_entries := 65}, hd(Infos)), + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 2 + end} + ], + SegConf = #{max_count => 128}, + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), + ok. + major_max_size(Config) -> %% this test could compact 3 segemtns into one just based on entry counts %% however the max_size configuration needs to be taken into account @@ -336,7 +379,6 @@ major_max_size(Config) -> {assert, fun (S) -> %% infos contain one symlink Infos = infos(Dir), - ct:pal("Infos ~p", [Infos]), Symlinks = [I || #{file_type := symlink} = I <- Infos], SegRefs = ra_log_segments:segment_refs(S), length(SegRefs) == 3 andalso From 5f09e811d4ec16eb9e1cf4ff503b3717a47825b1 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 27 Jun 2025 10:42:03 +0100 Subject: [PATCH 41/60] fix msg --- src/ra_log.erl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index ec975c794..93cb79e07 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -1481,7 +1481,7 @@ stage_entries0(Cfg, [Entry | Rem], Mt0) -> stage_entries0(Cfg, Rem, Mt); {error, overwriting} -> Range = ra_mt:range(Mt0), - Msg = io_lib:format("ra_log:verify_entries/2 " + Msg = io_lib:format("ra_log:stage_entries/3 " "tried writing ~p - mem table range ~w", [Rem, Range]), {error, {integrity_error, lists:flatten(Msg)}} From c9b2ccd4283750afc9974e2f4a481aa8149deac5 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 30 Jun 2025 14:44:42 +0100 Subject: [PATCH 42/60] Sparse write fixes --- src/ra_log.erl | 10 ++++- src/ra_mt.erl | 52 ++++++++++++------------ src/ra_server.erl | 4 +- test/ra_log_2_SUITE.erl | 88 +++++++++++++++++++++++++++++++++++++++-- test/ra_mt_SUITE.erl | 67 ++++++++++++++++++++----------- 5 files changed, 166 insertions(+), 55 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index 93cb79e07..92a2484ee 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -506,7 +506,8 @@ write_sparse({Idx, Term, _} = Entry, PrevIdx0, Tid = ra_mt:tid(Mt), PrevIdx = case PrevIdx0 of undefined -> - Idx - 1; + %% this is likely to always be accepted + 0; _ -> PrevIdx0 end, @@ -707,6 +708,7 @@ last_written(#?MODULE{last_written_index_term = LWTI}) -> set_last_index(Idx, #?MODULE{cfg = Cfg, range = Range, snapshot_state = SnapState, + mem_table = Mt0, last_written_index_term = {LWIdx0, _}} = State0) -> Cur = ra_snapshot:current(SnapState), case fetch_term(Idx, State0) of @@ -716,10 +718,13 @@ set_last_index(Idx, #?MODULE{cfg = Cfg, {_, State} when element(1, Cur) =:= Idx -> {_, SnapTerm} = Cur, %% Idx is equal to the current snapshot + {ok, Mt} = ra_log_ets:new_mem_table_please(Cfg#cfg.names, + Cfg#cfg.uid, Mt0), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, Idx), {ok, State#?MODULE{range = ra_range:limit(Idx + 1, Range), last_term = SnapTerm, + mem_table = Mt, last_written_index_term = Cur}}; {Term, State1} -> LWIdx = min(Idx, LWIdx0), @@ -729,10 +734,13 @@ set_last_index(Idx, #?MODULE{cfg = Cfg, %% to write to the mem table it will detect this and open %% a new one true = LWTerm =/= undefined, + {ok, Mt} = ra_log_ets:new_mem_table_please(Cfg#cfg.names, + Cfg#cfg.uid, Mt0), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_WRITTEN_INDEX, LWIdx), {ok, State2#?MODULE{range = ra_range:limit(Idx + 1, Range), last_term = Term, + mem_table = Mt, last_written_index_term = {LWIdx, LWTerm}}} end. diff --git a/src/ra_mt.erl b/src/ra_mt.erl index 4e57d74ce..678e18604 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -124,6 +124,12 @@ insert({Idx, _, _} = _Entry, overwriting | gap_detected | limit_reached}. +insert_sparse({Idx, _, _} = Entry, _LastIdx, + #?MODULE{tid = Tid, + indexes = []} = State) -> + %% when the indexes is empty always accept the next entry + true = ets:insert(Tid, Entry), + {ok, State#?MODULE{indexes = ra_seq:append(Idx, [])}}; insert_sparse({Idx, _, _} = Entry, LastIdx, #?MODULE{tid = Tid, indexes = Seq} = State) -> @@ -195,23 +201,6 @@ commit(#?MODULE{tid = Tid, {PrevStaged ++ Staged, State#?MODULE{staged = undefined, prev = Prev}}. -% -spec abort(state()) -> state(). -% abort(#?MODULE{staged = undefined} = State) -> -% State; -% abort(#?MODULE{staged = {_, Staged}, -% 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. lookup(Idx, #?MODULE{staged = {FstStagedIdx, Staged}}) @@ -330,8 +319,9 @@ delete({Op, Tid, Idx}) DelSpec = [{{'$1', '_', '_'}, [{'<', '$1', Idx}], [true]}], ets:select_delete(Tid, DelSpec); delete({delete, Tid}) -> + Sz= ets:info(Tid, size), true = ets:delete(Tid), - 0. + Sz. -spec range_overlap(ra:range(), state()) -> {Overlap :: ra:range(), Remainder :: ra:range()}. @@ -354,14 +344,18 @@ range_overlap(ReqRange, #?MODULE{} = State) -> range(#?MODULE{indexes = Seq, prev = undefined}) -> ra_seq:range(Seq); -range(#?MODULE{indexes = []}) -> - undefined; +range(#?MODULE{indexes = [], + prev = Prev}) -> + range(Prev); 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); + {Start, End} = Range = ra_seq:range(Seq), + case ra_range:limit(End, range(Prev)) of + undefined -> + Range; + {PrevStart, _PrevEnd} -> + ra_range:new(min(Start, PrevStart), End) + end; range(_State) -> undefined. @@ -439,8 +433,12 @@ set_first(Idx, #?MODULE{tid = Tid, %% 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 -> + %% also upgrade if the outer seq is truncated + %% by the set_first operation + % case range_shallow(P) of + case Idx >= ra_seq:first(Seq) orelse + range_shallow(P) == undefined of + true -> {[{delete, tid(P)} | Rem], prev(P)}; _ -> @@ -485,3 +483,5 @@ read_sparse([Next | Rem] = Indexes, State, Num, Acc) -> read_sparse(Rem, State, Num + 1, [Entry | Acc]) end. +range_shallow(#?MODULE{indexes = Seq}) -> + ra_seq:range(Seq). diff --git a/src/ra_server.erl b/src/ra_server.erl index be88a1006..e2fbc63ad 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -1551,12 +1551,12 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, %% reset last index to last applied %% as we dont know for sure indexes after last applied %% are of the right term - {LastIndex, _} = ra_log:last_index_term(Log00), + {LastIdx, _} = ra_log:last_index_term(Log00), {Log, _} = lists:foldl( fun ({I, _, _} = E, {L0, LstIdx}) -> {ok, L} = ra_log:write_sparse(E, LstIdx, L0), {L, I} - end, {Log00, LastIndex}, ChunkOrEntries), + end, {Log00, LastIdx}, ChunkOrEntries), ?DEBUG("~ts: receiving snapshot log last index ~p", [LogId, ra_log:last_index_term(Log)]), State = update_term(Term, State0#{log => Log}), diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 0046467b9..55d702c4f 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -33,6 +33,8 @@ all_tests() -> last_written_overwrite, last_written_overwrite_2, last_index_reset, + write_sparse_after_index_reset, + write_sparse_after_index_reset_segments, last_index_reset_before_written, recovery, recover_many, @@ -721,19 +723,97 @@ last_written_overwrite_2(Config) -> last_index_reset(Config) -> Log0 = ra_log_init(Config), - Log1 = write_n(1, 5, 1, Log0), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 2000), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log3} = ra_log:set_last_index(3, Log2), + {3, 1} = ra_log:last_written(Log3), + 4 = ra_log:next_index(Log3), + {3, 1} = ra_log:last_index_term(Log3), + O = ra_log:overview(Log3), + ct:pal("o ~p", [O]), + ?assertMatch(#{range := {0, 3}, + %% we have a new mem table but the mem table does not know + %% whatever the first index should be so reports the + %% full previous range, this will be corrected after the + %% next write at index 4 + mem_table_range := {0, 5}}, + O), + {ok, Log} = ra_log:write([{4, 2, hi}], Log3), + O2 = ra_log:overview(Log), + ct:pal("o ~p", [O2]), + ?assertMatch(#{range := {0, 4}, + mem_table_range := {0, 4}}, + O2), + ok. + +write_sparse_after_index_reset(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), Pred = fun (L) -> - {4, 1} == ra_log:last_written(L) + {5, 1} == ra_log:last_written(L) end, Log2 = assert_log_events(Log1, Pred, 2000), - 5 = ra_log:next_index(Log2), - {4, 1} = ra_log:last_index_term(Log2), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), % reverts last index to a previous index % needs to be done if a new leader sends an empty AER {ok, Log3} = ra_log:set_last_index(3, Log2), {3, 1} = ra_log:last_written(Log3), 4 = ra_log:next_index(Log3), {3, 1} = ra_log:last_index_term(Log3), + O = ra_log:overview(Log3), + ct:pal("o ~p", [O]), + {ok, Log4} = ra_log:write_sparse({7, 1, hi}, undefined, Log3), + {ok, Log} = ra_log:write_sparse({17, 1, hi}, 7, Log4), + O2 = ra_log:overview(Log), + ct:pal("o ~p", [O2]), + ?assertMatch(#{range := {0, 17}, + mem_table_range := {0, 17}}, + O2), + ok. + +write_sparse_after_index_reset_segments(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 2000), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + ra_log_wal:force_roll_over(ra_log_wal), + Log2b = deliver_all_log_events(Log2, 500), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log3} = ra_log:set_last_index(3, Log2b), + {3, 1} = ra_log:last_written(Log3), + 4 = ra_log:next_index(Log3), + {3, 1} = ra_log:last_index_term(Log3), + O = ra_log:overview(Log3), + ct:pal("o ~p", [O]), + {ok, Log4} = ra_log:write_sparse({7, 1, hi}, undefined, Log3), + {ok, Log5} = ra_log:write_sparse({17, 1, hi}, 7, Log4), + Log5b = deliver_all_log_events(Log5, 500), + O2 = ra_log:overview(Log5b), + ?assertMatch(#{range := {0, 17}, + mem_table_range := {7, 17}}, + O2), + + %% try overwrite again + {ok, Log6} = ra_log:set_last_index(3, Log5b), + {3, 1} = ra_log:last_index_term(Log6), + {ok, Log7} = ra_log:write_sparse({7, 1, hi}, undefined, Log6), + {ok, Log8} = ra_log:write_sparse({17, 1, hi}, 7, Log7), + Log = deliver_all_log_events(Log8, 500), + O5 = ra_log:overview(Log), + ct:pal("o ~p", [O5]), ok. last_index_reset_before_written(Config) -> diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl index 6d8df7af5..7eaad63cd 100644 --- a/test/ra_mt_SUITE.erl +++ b/test/ra_mt_SUITE.erl @@ -35,7 +35,8 @@ all_tests() -> range_overlap, stage_commit_2, perf, - sparse + sparse, + sparse_after_non_sparse ]. groups() -> @@ -129,11 +130,13 @@ record_flushed_prev(_Config) -> Tid2 = ets:new(t2, [set, public]), Mt2 = ra_mt:init_successor(Tid2, read_write, Mt1), + ?assertMatch({1, 100}, ra_mt:range(Mt2)), Mt3 = lists:foldl( fun (I, Acc) -> element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc)) end, Mt2, lists:seq(50, 80)), ?assertMatch({1, 100}, ra_mt:range(ra_mt:prev(Mt3))), + ?assertMatch({1, 80}, ra_mt:range(Mt3)), %% {Spec, Mt4} = ra_mt:record_flushed(Tid, [{1, 49}], Mt3), ?assertMatch({indexes, Tid, [{1, 49}]}, Spec), @@ -165,12 +168,13 @@ set_first(_Config) -> end, Mt2, lists:seq(50, 120)), {[Spec1, Spec2], Mt4} = ra_mt:set_first(75, Mt3), ?assertMatch({indexes, Tid2, [{50, 74}]}, Spec1), - ?assertMatch({indexes, Tid, [{1, 74}]}, Spec2), + ?assertMatch({delete, Tid}, Spec2), ?assertMatch({75, 120}, ra_mt:range(Mt4)), + ?assertMatch(undefined, ra_mt:prev(Mt4)), - {[Spec3, Spec4], Mt5} = ra_mt:set_first(105, Mt4), + {[Spec3], Mt5} = ra_mt:set_first(105, Mt4), ?assertMatch({indexes, Tid2, [{75, 104}]}, Spec3), - ?assertMatch({delete, Tid}, Spec4), + % ?assertMatch({delete, Tid}, Spec4), ?assertMatch({105, 120}, ra_mt:range(Mt5)), ?assertMatch(undefined, ra_mt:prev(Mt5)), ok. @@ -200,12 +204,10 @@ set_first_with_multi_prev(_Config) -> ?assertEqual({1, 200}, ra_mt:range(Mt3)), {[{indexes, Tid3, [{75, 79}]}, - {indexes, Tid2, [{50, 79}]}, - {indexes, Tid1, [{1, 79}]}], Mt4} = ra_mt:set_first(80, Mt3), - - {[{indexes, Tid3, [{80, 159}]}, {delete, Tid2}, - {delete, Tid1}], _Mt5} = ra_mt:set_first(160, Mt4), + {delete, Tid1}], Mt4} = ra_mt:set_first(80, Mt3), + + {[{indexes, Tid3, [{80, 159}]}], _Mt5} = ra_mt:set_first(160, Mt4), ok. set_first_with_middle_small_range(_Config) -> @@ -236,11 +238,11 @@ set_first_with_middle_small_range(_Config) -> {[{indexes, Tid3, [{75, 84}]}, {delete, Tid2}, - {indexes, Tid1, [{1, 84}]}], Mt4} = ra_mt:set_first(85, Mt3), + {delete, Tid1}], Mt4} = ra_mt:set_first(85, Mt3), ?assertEqual({85, 200}, ra_mt:range(Mt4)), - {[{indexes, Tid3, [{85, 100}]}, - {delete, Tid1}], Mt5} = ra_mt:set_first(101, Mt4), + {[{indexes, Tid3, [{85, 100}]} + ], Mt5} = ra_mt:set_first(101, Mt4), ?assertEqual({101, 200}, ra_mt:range(Mt5)), ?assertEqual(undefined, ra_mt:prev(Mt5)), @@ -263,17 +265,10 @@ set_first_with_old_larger_range(_Config) -> end, ra_mt:init_successor(Tid2, read_write, Mt1), lists:seq(50, 75)), {[{indexes, Tid2, [{50, 75}]}, - {indexes, Tid1, [{1, 84}]}], Mt3} = ra_mt:set_first(85, Mt2), + {delete, Tid1}], Mt3} = ra_mt:set_first(85, Mt2), ct:pal("Mt3 ~p", [Mt3]), ?assertEqual(undefined, ra_mt:range(Mt3)), - %% eventually when set_first passes the end of the old range it gets - %% deleted - % 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)), + ?assertEqual(undefined, ra_mt:prev(Mt3)), ok. set_first_with_old_smaller_range(_Config) -> @@ -307,6 +302,7 @@ successor(_Config) -> ?assertMatch({1, 100}, ra_mt:range(Mt1)), Tid2 = ets:new(t2, [set, public]), Mt2 = ra_mt:init_successor(Tid2, read_write, Mt1), + ?assertMatch({1, 100}, ra_mt:range(Mt2)), Mt3 = lists:foldl( fun (I, Acc) -> element(2, ra_mt:insert({I, 2, <<"banana">>}, Acc)) @@ -476,7 +472,7 @@ perf(_Config) -> sparse(_Config) -> Tid = ets:new(t1, [set, public]), Mt0 = ra_mt:init(Tid), - {ok, Mt1} = ra_mt:insert_sparse({2, 1, <<"banana">>}, undefined, Mt0), + {ok, Mt1} = ra_mt:insert_sparse({2, 1, <<"banana">>}, 1, Mt0), {ok, Mt2} = ra_mt:insert_sparse({5, 1, <<"banana">>}, 2, Mt1), ?assertEqual({2, 5}, ra_mt:range(Mt2)), {Spec, Mt3} = ra_mt:record_flushed(Tid, ra_seq:from_list([2, 5]), Mt2), @@ -486,6 +482,33 @@ sparse(_Config) -> ?assertEqual(0, ets:info(Tid, size)), ok. +sparse_after_non_sparse(_Config) -> + Tid = ets:new(t1, [set, public]), + Mt0 = ra_mt:init(Tid), + Mt1 = lists:foldl( + fun (I, M0) -> + {ok, M} = ra_mt:insert({I, 1, <<"banana">>}, M0), + M + end, Mt0, lists:seq(1, 10)), + Tid2 = ets:new(t2, [set, public]), + Mt2 = ra_mt:init_successor(Tid2, read_write, Mt1), + + {ok, Mt3} = ra_mt:insert_sparse({12, 1, <<"banana">>}, undefined, Mt2), + {ok, Mt4} = ra_mt:insert_sparse({15, 1, <<"banana">>}, 12, Mt3), + ?assertEqual({1, 15}, ra_mt:range(Mt4)), + ?assertMatch(#{size := 2, + range := {1, 15}}, ra_mt:info(Mt4)), + + {Spec, Mt5} = ra_mt:record_flushed(Tid, ra_seq:from_list(lists:seq(1, 10)), Mt4), + %% full table delete + 10 = ra_mt:delete(Spec), + {Spec2, Mt6} = ra_mt:record_flushed(Tid2, ra_seq:from_list([12, 15]), Mt5), + 2 = ra_mt:delete(Spec2), + ?assertMatch(#{size := 0, + range := undefined}, ra_mt:info(Mt6)), + ?assertEqual(0, ets:info(Tid2, size)), + ok. + %%% Util read_n(N, N, Acc, _Mt) -> From f81afc17dfcccf1c1bfcacb7adede05811b57bd9 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 1 Jul 2025 15:30:44 +0100 Subject: [PATCH 43/60] Fix multiple issues around interrupted snapshot replication with sparse entries --- src/ra.erl | 5 ++- src/ra_log.erl | 27 ++++++++++---- src/ra_log_segment.erl | 31 +++++++++++++--- src/ra_log_segments.erl | 15 ++++---- src/ra_mt.erl | 36 ++++++++++-------- src/ra_server.erl | 6 +-- test/ra_kv_SUITE.erl | 73 ++++++++++++++++++++++++++++++++----- test/ra_log_2_SUITE.erl | 79 +++++++++++++++++++++++++++++++++++++++- test/ra_mt_SUITE.erl | 50 ++++++++++++++++--------- test/ra_server_SUITE.erl | 3 ++ 10 files changed, 257 insertions(+), 68 deletions(-) diff --git a/src/ra.erl b/src/ra.erl index 0393fc929..8705ac26f 100644 --- a/src/ra.erl +++ b/src/ra.erl @@ -155,9 +155,10 @@ start_in(DataDir) -> ra_env:configure_logger(logger), LogFile = filename:join(DataDir, "ra.log"), SaslFile = filename:join(DataDir, "ra_sasl.log"), - logger:set_primary_config(level, debug), + logger:remove_handler(ra_handler), + ok = logger:set_primary_config(level, debug), Config = #{config => #{file => LogFile}}, - logger:add_handler(ra_handler, logger_std_h, Config), + ok = logger:add_handler(ra_handler, logger_std_h, Config), application:load(sasl), application:set_env(sasl, sasl_error_logger, {file, SaslFile}), application:stop(sasl), diff --git a/src/ra_log.erl b/src/ra_log.erl index 92a2484ee..18c36e96b 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -21,6 +21,7 @@ append_sync/2, write_sync/2, fold/5, + fold/6, sparse_read/2, partial_read/3, execute_read_plan/4, @@ -491,7 +492,7 @@ write([{Idx, _, _} | _], #?MODULE{cfg = #cfg{uid = UId}, [UId, Idx, Range])), {error, {integrity_error, Msg}}. --spec write_sparse(log_entry(), ra:index(), state()) -> +-spec write_sparse(log_entry(), option(ra:index()), state()) -> {ok, state()} | {error, wal_down | gap_detected}. write_sparse({Idx, Term, _} = Entry, PrevIdx0, #?MODULE{cfg = #cfg{uid = UId, @@ -533,11 +534,18 @@ write_sparse({Idx, Term, _} = Entry, PrevIdx0, -spec fold(FromIdx :: ra_index(), ToIdx :: ra_index(), fun((log_entry(), Acc) -> Acc), Acc, state()) -> {Acc, state()} when Acc :: term(). +fold(From0, To0, Fun, Acc0, State) -> + fold(From0, To0, Fun, Acc0, State, error). + +-spec fold(FromIdx :: ra_index(), ToIdx :: ra_index(), + fun((log_entry(), Acc) -> Acc), Acc, state(), + MissingKeyStrategy :: error | return) -> + {Acc, state()} when Acc :: term(). fold(From0, To0, Fun, Acc0, #?MODULE{cfg = Cfg, mem_table = Mt, range = {StartIdx, EndIdx}, - reader = Reader0} = State) + reader = Reader0} = State, MissingKeyStrat) when To0 >= From0 andalso To0 >= StartIdx -> @@ -550,22 +558,27 @@ fold(From0, To0, Fun, Acc0, case MtOverlap of {undefined, {RemStart, RemEnd}} -> {Reader, Acc} = ra_log_segments:fold(RemStart, RemEnd, Fun, - Acc0, Reader0), + Acc0, Reader0, + MissingKeyStrat), {Acc, State#?MODULE{reader = Reader}}; {{MtStart, MtEnd}, {RemStart, RemEnd}} -> {Reader, Acc1} = ra_log_segments:fold(RemStart, RemEnd, Fun, - Acc0, Reader0), - Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc1, Mt), + Acc0, Reader0, + MissingKeyStrat), + Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc1, Mt, MissingKeyStrat), NumRead = MtEnd - MtStart + 1, ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, NumRead), {Acc, State#?MODULE{reader = Reader}}; {{MtStart, MtEnd}, undefined} -> - Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc0, Mt), + Acc = ra_mt:fold(MtStart, MtEnd, Fun, Acc0, Mt, MissingKeyStrat), + %% TODO: if fold is short circuited with MissingKeyStrat == return + %% this count isn't correct, it doesn't massively matter so leaving + %% for now NumRead = MtEnd - MtStart + 1, ok = incr_counter(Cfg, ?C_RA_LOG_READ_MEM_TBL, NumRead), {Acc, State} end; -fold(_From, _To, _Fun, Acc, State) -> +fold(_From, _To, _Fun, Acc, State, _) -> {Acc, State}. %% @doc Reads a list of indexes. diff --git a/src/ra_log_segment.erl b/src/ra_log_segment.erl index cc42889df..c04bcca46 100644 --- a/src/ra_log_segment.erl +++ b/src/ra_log_segment.erl @@ -12,6 +12,7 @@ append/4, sync/1, fold/6, + fold/7, is_modified/1, read_sparse/4, read_sparse_no_checks/4, @@ -300,7 +301,22 @@ fold(#state{cfg = #cfg{mode = read} = Cfg, cache = Cache, index = Index}, FromIdx, ToIdx, Fun, AccFun, Acc) -> - fold0(Cfg, Cache, FromIdx, ToIdx, Index, Fun, AccFun, Acc). + fold0(Cfg, Cache, FromIdx, ToIdx, Index, Fun, AccFun, Acc, + error). + +-spec fold(state(), + FromIdx :: ra_index(), + ToIdx :: ra_index(), + fun((binary()) -> term()), + fun(({ra_index(), ra_term(), term()}, Acc) -> Acc), Acc, + MissingKeyStrat :: error | return) -> + Acc when Acc :: term(). +fold(#state{cfg = #cfg{mode = read} = Cfg, + cache = Cache, + index = Index}, + FromIdx, ToIdx, Fun, AccFun, Acc, MissingKeyStrat) -> + fold0(Cfg, Cache, FromIdx, ToIdx, Index, Fun, AccFun, Acc, + MissingKeyStrat). -spec is_modified(state()) -> boolean(). is_modified(#state{cfg = #cfg{fd = Fd}, @@ -403,10 +419,10 @@ term_query(#state{index = Index}, Idx) -> _ -> undefined end. -fold0(_Cfg, _Cache, Idx, FinalIdx, _, _Fun, _AccFun, Acc) +fold0(_Cfg, _Cache, Idx, FinalIdx, _, _Fun, _AccFun, Acc, _) when Idx > FinalIdx -> Acc; -fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0) -> +fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0, MissingKeyStrat) -> case Index of #{Idx := {Term, Offset, Length, Crc} = IdxRec} -> case pread(Cfg, Cache0, Offset, Length) of @@ -415,7 +431,8 @@ fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0) -> case validate_checksum(Crc, Data) of true -> Acc = AccFun({Idx, Term, Fun(Data)}, Acc0), - fold0(Cfg, Cache, Idx+1, FinalIdx, Index, Fun, AccFun, Acc); + fold0(Cfg, Cache, Idx+1, FinalIdx, + Index, Fun, AccFun, Acc, MissingKeyStrat); false -> %% CRC check failures are irrecoverable exit({ra_log_segment_crc_check_failure, Idx, IdxRec, @@ -426,8 +443,10 @@ fold0(Cfg, Cache0, Idx, FinalIdx, Index, Fun, AccFun, Acc0) -> exit({ra_log_segment_unexpected_eof, Idx, IdxRec, Cfg#cfg.filename}) end; - _ -> - exit({missing_key, Idx, Cfg#cfg.filename}) + _ when MissingKeyStrat == error -> + exit({missing_key, Idx, Cfg#cfg.filename}); + _ when MissingKeyStrat == return -> + Acc0 end. -spec range(state()) -> option({ra_index(), ra_index()}). diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index 6248842dd..f9156fb3c 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -21,7 +21,7 @@ range/1, num_open_segments/1, update_first_index/2, - fold/5, + fold/6, sparse_read/3, read_plan/2, exec_read_plan/6, @@ -285,14 +285,15 @@ compaction_conf(#?STATE{cfg = #cfg{compaction_conf = Conf}}) -> num_open_segments(#?STATE{open_segments = Open}) -> ra_flru:size(Open). --spec fold(ra_index(), ra_index(), fun(), term(), state()) -> +-spec fold(ra_index(), ra_index(), fun(), term(), state(), + MissingKeyStrategy :: error | return) -> {state(), term()}. fold(FromIdx, ToIdx, Fun, Acc, - #?STATE{cfg = #cfg{} = Cfg} = State0) + #?STATE{cfg = #cfg{} = Cfg} = State0, MissingKeyStrat) when ToIdx >= FromIdx -> ok = incr_counter(Cfg, ?C_RA_LOG_READ_SEGMENT, ToIdx - FromIdx + 1), - segment_fold(State0, FromIdx, ToIdx, Fun, Acc); -fold(_FromIdx, _ToIdx, _Fun, Acc, #?STATE{} = State) -> + segment_fold(State0, FromIdx, ToIdx, Fun, Acc, MissingKeyStrat); +fold(_FromIdx, _ToIdx, _Fun, Acc, #?STATE{} = State, _Strat) -> {State, Acc}. -spec sparse_read(state(), [ra_index()], [log_entry()]) -> @@ -453,7 +454,7 @@ segment_fold_plan(SegRefs, {_ReqStart, ReqEnd} = ReqRange, Acc) -> segment_fold(#?STATE{segment_refs = SegRefs, open_segments = OpenSegs, cfg = Cfg} = State, - RStart, REnd, Fun, Acc) -> + RStart, REnd, Fun, Acc, MissingKeyStrat) -> Plan = segment_fold_plan(SegRefs, {RStart, REnd}, []), {Op, A} = lists:foldl( @@ -461,7 +462,7 @@ segment_fold(#?STATE{segment_refs = SegRefs, {Seg, Open} = get_segment(Cfg, Open0, Fn), {Open, ra_log_segment:fold(Seg, Start, End, fun binary_to_term/1, - Fun, Ac0)} + Fun, Ac0, MissingKeyStrat)} end, {OpenSegs, Acc}, Plan), {State#?MODULE{open_segments = Op}, A}. diff --git a/src/ra_mt.erl b/src/ra_mt.erl index 678e18604..162878b37 100644 --- a/src/ra_mt.erl +++ b/src/ra_mt.erl @@ -6,7 +6,6 @@ %% @hidden -module(ra_mt). --include_lib("stdlib/include/assert.hrl"). -include("ra.hrl"). -export([ @@ -22,6 +21,7 @@ lookup_term/2, tid_for/3, fold/5, + fold/6, get_items/2, record_flushed/3, set_first/2, @@ -249,12 +249,6 @@ lookup_term(Idx, #?MODULE{tid = Tid, 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,16 +263,28 @@ tid_for(Idx, Term, State) -> tid_for(Idx, Term, State#?MODULE.prev) end. +-spec fold(ra:index(), ra:index(), + fun(), term(), state(), MissingKeyStrategy :: error | return) -> + term(). +fold(From, To, Fun, Acc, State, MissingKeyStrat) + when is_atom(MissingKeyStrat) andalso + To >= From -> + case lookup(From, State) of + undefined when MissingKeyStrat == error -> + error({missing_key, From, Acc}); + undefined when MissingKeyStrat == return -> + Acc; + E -> + fold(From + 1, To, Fun, Fun(E, Acc), + State, MissingKeyStrat) + end; +fold(_From, _To, _Fun, Acc, _State, _Strat) -> + Acc. + -spec fold(ra:index(), ra:index(), fun(), term(), state()) -> term(). -fold(To, To, Fun, Acc, State) -> - E = lookup(To, State), - Fun(E, Acc); -fold(From, To, Fun, Acc, State) - when To > From -> - E = lookup(From, State), - ?assert(E =/= undefined), - fold(From + 1, To, Fun, Fun(E, Acc), State). +fold(From, To, Fun, Acc, State) -> + fold(From, To, Fun, Acc, State, error). -spec get_items([ra:index()], state()) -> {[log_entry()], diff --git a/src/ra_server.erl b/src/ra_server.erl index e2fbc63ad..eea848ab0 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -452,9 +452,11 @@ recover(#{cfg := #cfg{log_id = LogId, FromScan = CommitIndex + 1, {ToScan, _} = ra_log:last_index_term(Log0), ?DEBUG("~ts: scanning for cluster changes ~b:~b ", [LogId, FromScan, ToScan]), + %% if we're recovering after a partial sparse write phase this will fail + %% {State, Log} = ra_log:fold(FromScan, ToScan, fun cluster_scan_fun/2, - State1, Log0), + State1, Log0, return), put_counter(Cfg, ?C_RA_SVR_METRIC_COMMIT_LATENCY, 0), State#{log => Log, @@ -1557,8 +1559,6 @@ handle_receive_snapshot(#install_snapshot_rpc{term = Term, {ok, L} = ra_log:write_sparse(E, LstIdx, L0), {L, I} end, {Log00, LastIdx}, ChunkOrEntries), - ?DEBUG("~ts: receiving snapshot log last index ~p", - [LogId, ra_log:last_index_term(Log)]), State = update_term(Term, State0#{log => Log}), {receive_snapshot, State, [{reply, Reply}]}; next -> diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index 154da5ffc..c3cb521f8 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -21,7 +21,8 @@ all() -> all_tests() -> [ basics, - snapshot_replication + snapshot_replication, + snapshot_replication_interrupted ]. groups() -> @@ -56,9 +57,63 @@ end_per_testcase(_TestCase, _Config) -> %%% Test cases %%%=================================================================== +-define(KV(N), + binary_to_atom(<<(atom_to_binary(?FUNCTION_NAME))/binary, + (integer_to_binary(N))/binary>>)). + +snapshot_replication_interrupted(_Config) -> + Kv1 = ?KV(1), Kv2 = ?KV(2), Kv3 = ?KV(3), + Members = [{Kv1, node()}, {Kv2, node()}], + KvId = hd(Members), + {ok, _, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, + #{members => Members}), + ra:transfer_leadership(KvId, KvId), + Data = crypto:strong_rand_bytes(100_000), + %% write 10k entries of the same key + [{ok, #{}} = ra_kv:put(KvId, term_to_binary(I), Data, 5000) + || I <- lists:seq(1, 50_000)], + ?assertMatch({ok, #{machine := #{num_keys := 50_000}}, KvId}, + ra:member_overview(KvId)), + + ra_log_wal:force_roll_over(ra_log_wal), + ra_log_wal:last_writer_seq(ra_log_wal, <<>>), + ra_log_segment_writer:await(ra_log_segment_writer), + ok = ra:aux_command(KvId, take_snapshot), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{snapshot_index := SnapIdx, + last_index := LastIdx}}, _} = + ra:member_overview(KvId), + SnapIdx == LastIdx + end, 100, 100), + KvId3 = {Kv3, node()}, + ok = ra_kv:add_member(?SYS, KvId3, KvId), + KvId3Pid = whereis(Kv3), + ?assert(is_pid(KvId3Pid)), + %% wait for the follower to enter snapshot state + ok = ra_lib:retry( + fun () -> + receive_snapshot == element(2, hd(ets:lookup(ra_state, Kv3))) + end, 100, 100), + + ct:pal("ra_state ~p", [ets:tab2list(ra_state)]), + ok = ra:stop_server(?SYS, KvId3), + [{ok, #{}} = ra_kv:put(KvId, term_to_binary(I), Data, 5000) + || I <- lists:seq(50_001, 50_010)], + ok = ra:restart_server(?SYS, KvId3), + {ok, #{log := #{last_index := Kv1LastIndex }}, _} = ra:member_overview(KvId), + ok = ra_lib:retry( + fun () -> + {ok, #{log := #{last_index := LastIdx}}, _} = + ra:member_overview(KvId3), + Kv1LastIndex == LastIdx + end, 100, 100), + ra:delete_cluster([KvId, {Kv2, node()}, KvId3]), + ok. snapshot_replication(_Config) -> - Members = [{kv1, node()}, {kv2, node()}], + Kv1 = ?KV(1), Kv2 = ?KV(2), Kv3 = ?KV(3), + Members = [{Kv1, node()}, {Kv2, node()}], KvId = hd(Members), {ok, _, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, #{members => Members}), @@ -85,9 +140,9 @@ snapshot_replication(_Config) -> SnapIdx == LastIdx end, 100, 100), - KvId3 = {kv3, node()}, + KvId3 = {Kv3, node()}, ok = ra_kv:add_member(?SYS, KvId3, KvId), - KvId3Pid = whereis(kv3), + KvId3Pid = whereis(Kv3), ?assert(is_pid(KvId3Pid)), {ok, #{}} = ra_kv:put(KvId, <<"k3">>, <<"k3-value">>, 5000), {ok, #{}} = ra_kv:put(KvId, <<"k4">>, <<"k4-value">>, 5000), @@ -101,8 +156,8 @@ snapshot_replication(_Config) -> Kv1LastIndex == LastIdx end, 100, 100), ct:pal("counters ~p", [ra_counters:counters(KvId3, [last_applied])]), - %% ensure kv3 did not crash during snapshot replication - ?assertEqual(KvId3Pid, whereis(kv3)), + %% ensure Kv3 did not crash during snapshot replication + ?assertEqual(KvId3Pid, whereis(Kv3)), ok = ra:stop_server(default, KvId3), @@ -123,7 +178,8 @@ snapshot_replication(_Config) -> ok. basics(_Config) -> - Members = [{kv1, node()}], + Kv1 = ?KV(1), Kv2 = ?KV(2), _Kv3 = ?KV(3), + Members = [{Kv1, node()}], KvId = hd(Members), {ok, Members, _} = ra_kv:start_cluster(?SYS, ?FUNCTION_NAME, #{members => Members}), @@ -164,7 +220,7 @@ basics(_Config) -> undefined, 1000), ?assertEqual(3, map_size(Reads)), % ct:pal("ReadRes ~p", [Reads]), - KvId2 = {kv2, node()}, + KvId2 = {Kv2, node()}, ok = ra_kv:add_member(?SYS, KvId2, KvId), ok = ra_lib:retry( fun () -> @@ -180,7 +236,6 @@ basics(_Config) -> ra_log_segment_writer:await(ra_log_segment_writer), {ok, {Reads3, _}} = ra_server_proc:read_entries(KvId2, [LastIdx | Live], undefined, 1000), - ct:pal("ReadRes3 ~p", [Reads3]), ?assertEqual(3, map_size(Reads3)), %% TODO: test recovery of kv diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 55d702c4f..8f9f78acb 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -33,6 +33,9 @@ all_tests() -> last_written_overwrite, last_written_overwrite_2, last_index_reset, + fold_after_sparse_mem_table, + fold_after_sparse_segments, + write_sparse_re_init, write_sparse_after_index_reset, write_sparse_after_index_reset_segments, last_index_reset_before_written, @@ -753,13 +756,87 @@ last_index_reset(Config) -> O2), ok. +fold_after_sparse_mem_table(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 500), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log2b} = ra_log:set_last_index(5, Log2), + {ok, Log3} = ra_log:write_sparse({7, 1, hi}, undefined, Log2b), + {ok, Log4} = ra_log:write_sparse({17, 1, hi}, 7, Log3), + Log = deliver_all_log_events(Log4, 500), + {Res, _Log} = ra_log:fold(1, 17, + fun ({I, _, _}, Is) -> + [I | Is] + end, [], Log, return), + ct:pal("Res ~p", [Res]), + ?assertMatch([5,4,3,2,1], Res), + ok. + +fold_after_sparse_segments(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 500), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log2b} = ra_log:set_last_index(5, Log2), + {ok, Log3} = ra_log:write_sparse({7, 1, hi}, undefined, Log2b), + {ok, Log4} = ra_log:write_sparse({17, 1, hi}, 7, Log3), + ok = ra_log_wal:force_roll_over(ra_log_wal), + Log = deliver_all_log_events(Log4, 500), + ra_log:fold(1, 17, fun (_, _) -> ok end, undefined, Log, return), + ok. + +write_sparse_re_init(Config) -> + Log0 = ra_log_init(Config), + Log1 = write_n(1, 6, 1, Log0), + Pred = fun (L) -> + {5, 1} == ra_log:last_written(L) + end, + Log2 = assert_log_events(Log1, Pred, 500), + 6 = ra_log:next_index(Log2), + {5, 1} = ra_log:last_index_term(Log2), + % reverts last index to a previous index + % needs to be done if a new leader sends an empty AER + {ok, Log2b} = ra_log:set_last_index(5, Log2), + {ok, Log3} = ra_log:write_sparse({7, 1, hi}, undefined, Log2b), + {ok, Log4} = ra_log:write_sparse({17, 1, hi}, 7, Log3), + Log = deliver_all_log_events(Log4, 500), + + O = ra_log:overview(Log), + ct:pal("o ~p", [O]), + ra_log:close(Log), + + ReInitLog = ra_log_init(Config), + O2 = ra_log:overview(ReInitLog ), + ct:pal("o2 ~p", [O2]), + ?assertEqual(maps:remove(last_wal_write, O), + maps:remove(last_wal_write, O2)), + + + %% server init does a fold from last applied + 1 to last index which + %% will fail if the log head is sparse + + ok. + write_sparse_after_index_reset(Config) -> Log0 = ra_log_init(Config), Log1 = write_n(1, 6, 1, Log0), Pred = fun (L) -> {5, 1} == ra_log:last_written(L) end, - Log2 = assert_log_events(Log1, Pred, 2000), + Log2 = assert_log_events(Log1, Pred, 500), 6 = ra_log:next_index(Log2), {5, 1} = ra_log:last_index_term(Log2), % reverts last index to a previous index diff --git a/test/ra_mt_SUITE.erl b/test/ra_mt_SUITE.erl index 7eaad63cd..60dda4347 100644 --- a/test/ra_mt_SUITE.erl +++ b/test/ra_mt_SUITE.erl @@ -21,6 +21,7 @@ all() -> all_tests() -> [ basics, + fold, record_flushed, record_flushed_after_set_first, record_flushed_prev, @@ -42,24 +43,6 @@ all_tests() -> 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 %%%=================================================================== @@ -75,11 +58,42 @@ basics(_Config) -> 499 = ra_mt:delete(Spec), ?assertEqual({500, 1000}, ra_mt:range(Mt2)), ?assertEqual(501, ets:info(Tid, size)), + ?assertEqual(lists:seq(510, 505, -1), + ra_mt:fold(505, 510, fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Mt2)), {Spec2, Mt3} = ra_mt:record_flushed(Tid, [{1, 999}], Mt2), 500 = ra_mt:delete(Spec2), ?assertEqual(1, ra_mt:lookup_term(1000, Mt3)), ok. +fold(_Config) -> + Tid = ets:new(t1, [set, public]), + Mt0 = ra_mt:init(Tid), + Mt1 = lists:foldl( + fun (I, Acc) -> + element(2, ra_mt:insert({I, 1, <<"banana">>}, Acc)) + end, Mt0, lists:seq(1, 1000)), + {[Spec], Mt2} = ra_mt:set_first(500, Mt1), + 499 = ra_mt:delete(Spec), + ?assertEqual({500, 1000}, ra_mt:range(Mt2)), + ?assertEqual(501, ets:info(Tid, size)), + ?assertEqual(lists:seq(510, 505, -1), + ra_mt:fold(505, 510, fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Mt2)), + ?assertError({missing_key, 1001, _}, + ra_mt:fold(999, 1010, + fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Mt2)), + ?assertEqual([1000, 999], + ra_mt:fold(999, 1010, + fun ({I, _, _}, Acc) -> + [I | Acc] + end, [], Mt2, return)), + ok. + record_flushed(_Config) -> %%TODO: test that deletes the same spec twice Tid = ets:new(t1, [set, public]), diff --git a/test/ra_server_SUITE.erl b/test/ra_server_SUITE.erl index bf79cdade..556864b53 100644 --- a/test/ra_server_SUITE.erl +++ b/test/ra_server_SUITE.erl @@ -177,6 +177,9 @@ setup_log() -> meck:expect(ra_log, recover_snapshot, fun ra_log_memory:recover_snapshot/1), meck:expect(ra_log, snapshot_index_term, fun ra_log_memory:snapshot_index_term/1), meck:expect(ra_log, fold, fun ra_log_memory:fold/5), + meck:expect(ra_log, fold, fun (A, B, C, D, E, _) -> + ra_log_memory:fold(A, B, C, D, E) + end), meck:expect(ra_log, release_resources, fun ra_log_memory:release_resources/3), meck:expect(ra_log, append_sync, fun({Idx, Term, _} = E, L0) -> From af74ad63ee3d580c27001c44d9a9fa6a147a579b Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 2 Jul 2025 13:16:55 +0100 Subject: [PATCH 44/60] fixes --- src/ra_server.erl | 25 ++++++++++++++++++------- src/ra_server_proc.erl | 4 ++++ test/ra_log_2_SUITE.erl | 35 ++++++++++++++++++++++++++++++++++- test/unit_SUITE.erl | 2 +- 4 files changed, 57 insertions(+), 9 deletions(-) diff --git a/src/ra_server.erl b/src/ra_server.erl index eea848ab0..289f2f8fc 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -51,6 +51,7 @@ promote_checkpoint/2, checkpoint/3, persist_last_applied/1, + peers/1, update_peer/3, update_disconnected_peers/3, handle_down/5, @@ -168,7 +169,10 @@ {cast, ra_server_id(), term()} | {send_vote_requests, [{ra_server_id(), #request_vote_rpc{} | #pre_vote_rpc{}}]} | - {send_rpc, ra_server_id(), #append_entries_rpc{}} | + {send_rpc, ra_server_id(), + #append_entries_rpc{} | + #heartbeat_rpc{} | + #info_rpc{}} | {send_snapshot, To :: ra_server_id(), {Module :: module(), Ref :: term(), LeaderId :: ra_server_id(), Term :: ra_term()}} | @@ -477,7 +481,7 @@ handle_leader({PeerId, #append_entries_reply{term = Term, success = true, ?WARN("~ts: saw append_entries_reply from unknown peer ~w", [LogId, PeerId]), {leader, State0, []}; - Peer0 = #{match_index := MI, next_index := NI} -> + #{match_index := MI, next_index := NI} = Peer0 -> Peer = Peer0#{match_index => max(MI, LastIdx), next_index => max(NI, NextIdx)}, State1 = put_peer(PeerId, Peer, State0), @@ -1639,15 +1643,19 @@ handle_receive_snapshot(#append_entries_rpc{term = Term} = Msg, {follower, update_term(Term, clear_leader_id(State#{log => Log})), [{next_event, Msg}]}; handle_receive_snapshot({ra_log_event, Evt}, - State = #{cfg := #cfg{id = _Id, log_id = LogId}, - log := Log0}) -> + #{cfg := #cfg{log_id = LogId}, + log := Log0} = State) -> ?DEBUG("~ts: ~s ra_log_event received: ~w", [LogId, ?FUNCTION_NAME, Evt]), % simply forward all other events to ra_log % whilst the snapshot is being received {Log, Effects} = ra_log:handle_event(Evt, Log0), {receive_snapshot, State#{log => Log}, Effects}; -handle_receive_snapshot(receive_snapshot_timeout, #{log := Log0} = State) -> +handle_receive_snapshot(receive_snapshot_timeout, + #{cfg := #cfg{log_id = LogId}, + log := Log0} = State) -> + ?INFO("~ts: ~s receive snapshot timed out.", + [LogId, ?FUNCTION_NAME]), SnapState0 = ra_log:snapshot_state(Log0), SnapState = ra_snapshot:abort_accept(SnapState0), Log = ra_log:set_snapshot_state(SnapState, Log0), @@ -3312,7 +3320,8 @@ heartbeat_rpc_effects(Peers, Id, Term, QueryIndex) -> end, maps:to_list(Peers)). -heartbeat_rpc_effect_for_peer(PeerId, Peer, Id, Term, QueryIndex) -> +heartbeat_rpc_effect_for_peer(PeerId, #{status := normal} = Peer, + Id, Term, QueryIndex) -> case maps:get(query_index, Peer, 0) < QueryIndex of true -> {true, @@ -3322,7 +3331,9 @@ heartbeat_rpc_effect_for_peer(PeerId, Peer, Id, Term, QueryIndex) -> leader_id = Id}}}; false -> false - end. + end; +heartbeat_rpc_effect_for_peer(_PeerId, _Peer, _Id, _Term, _QueryIndex) -> + false. heartbeat_rpc_quorum(NewQueryIndex, PeerId, #{queries_waiting_heartbeats := Waiting0} = State) -> diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index b8fef5606..9cb2c29b1 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -1412,6 +1412,8 @@ handle_effect(_RaftState, {send_rpc, To, Rpc}, _, incr_counter(Conf, ?C_RA_SRV_MSGS_SENT, 1), Self ! {update_peer, To, #{status => normal}} end), + ?DEBUG("~ts: temporarily suspending peer ~w due to full distribution buffer", + [log_id(State0), To]), {update_peer(To, #{status => suspended}, State0), Actions}; noconnect -> %% for noconnects just allow it to pipeline and catch up later @@ -1544,6 +1546,8 @@ handle_effect(leader, {send_snapshot, {_, ToNode} = To, {SnapState, _Id, Term}}, %% node is connected %% leader effect only Machine = ra_server:machine(SS0), + %% temporary assertion + #{To := #{status := normal}} = ra_server:peers(SS0), Id = ra_server:id(SS0), Pid = spawn(fun () -> try send_snapshots(Id, Term, To, diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 8f9f78acb..1a5c7ca84 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -1353,6 +1353,34 @@ oldcheckpoints_deleted_after_snapshot_install(Config) -> ok. +snapshot_installation_with_live_indexes(Config) -> + Log0 = ra_log_init(Config), + {0, 0} = ra_log:last_index_term(Log0), + Log1 = assert_log_events(write_n(1, 10, 2, Log0), + fun (L) -> + LW = ra_log:last_written(L), + {9, 2} == LW + end), + + Log2 = Log1, + + %% create snapshot chunk + Meta = meta(15, 2, [?N1]), + Chunk = create_snapshot_chunk(Config, Meta, #{}), + SnapState0 = ra_log:snapshot_state(Log2), + {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), + Machine = {machine, ?MODULE, #{}}, + {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, + SnapState1), + run_effs(AEffs), + {ok, Log3, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, + ra_log:set_snapshot_state(SnapState, Log2)), + + run_effs(Effs4), + {15, _} = ra_log:last_index_term(Log3), + {15, _} = ra_log:last_written(Log3), + ok. + snapshot_installation(Config) -> Log0 = ra_log_init(Config), {0, 0} = ra_log:last_index_term(Log0), @@ -1794,7 +1822,12 @@ sparse_write(Config) -> SnapState1), run_effs(AEffs), Log3 = ra_log:set_snapshot_state(SnapState, Log2), - {ok, Log4, _} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, Log3), + {ok, Log4, _} = ra_log:install_snapshot({15, 2}, ?MODULE, + LiveIndexes, Log3), + + ct:pal("overview Log4 ~p", [ra_log:overview(Log4)]), + ?assertEqual(16, ra_log:next_index(Log4)), + {ok, Log} = ra_log:write([{16, 1, <<>>}], Log4), {ResFinal, _} = ra_log:sparse_read(LiveIndexes, Log), ?assertMatch([{3, _, _}, diff --git a/test/unit_SUITE.erl b/test/unit_SUITE.erl index bf4897200..7a32888a5 100644 --- a/test/unit_SUITE.erl +++ b/test/unit_SUITE.erl @@ -21,7 +21,7 @@ mods() -> ra_flru, ra_lib, ra_log, - ra_log_reader, + ra_log_segments, ra_log_segment, ra_monitors, ra_server, From b726b16f2e5eefe1ed20cccd3d8484d14fdc2260 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Thu, 3 Jul 2025 12:19:41 +0100 Subject: [PATCH 45/60] suspension fixes --- src/ra_server.erl | 9 +++++++++ src/ra_server_proc.erl | 25 ++++++++++++++++++------- test/ra_kv_SUITE.erl | 6 +++--- test/ra_server_SUITE.erl | 2 +- 4 files changed, 31 insertions(+), 11 deletions(-) diff --git a/src/ra_server.erl b/src/ra_server.erl index 289f2f8fc..7a3a474e6 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -52,6 +52,7 @@ checkpoint/3, persist_last_applied/1, peers/1, + peer_status/2, update_peer/3, update_disconnected_peers/3, handle_down/5, @@ -2548,6 +2549,14 @@ new_peer_with(Map) -> peers(#{cfg := #cfg{id = Id}, cluster := Peers}) -> maps:remove(Id, Peers). +peer_status(PeerId, #{cluster := Peers}) -> + case Peers of + #{PeerId := #{status := Status}} -> + Status; + _ -> + undefined + end. + %% remove any peers that are currently receiving a snapshot peers_with_normal_status(State) -> maps:filter(fun (_, #{status := normal}) -> true; diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index 9cb2c29b1..67e9690c4 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -579,8 +579,13 @@ leader(info, {Status, Node, InfoList}, State0) when Status =:= nodedown orelse Status =:= nodeup -> handle_node_status_change(Node, Status, InfoList, ?FUNCTION_NAME, State0); -leader(info, {update_peer, PeerId, Update}, State0) -> - State = update_peer(PeerId, Update, State0), +leader(info, {unsuspend_peer, PeerId}, State0) -> + State = case ra_server:peer_status(PeerId, State0#state.server_state) of + suspended -> + update_peer(PeerId, #{status => normal}, State0); + _ -> + State0 + end, {keep_state, State, []}; leader(_, tick_timeout, State0) -> {State1, RpcEffs} = make_rpcs(State0), @@ -1393,13 +1398,15 @@ handle_effects(RaftState, Effects0, EvtType, State0, Actions0) -> {State, lists:reverse(Actions)}. handle_effect(_RaftState, {send_rpc, To, Rpc}, _, - #state{conf = Conf} = State0, Actions) -> + #state{conf = Conf, + server_state = SS} = State0, Actions) -> % fully qualified use only so that we can mock it for testing % TODO: review / refactor to remove the mod call here + PeerStatus = ra_server:peer_status(To, SS), case ?MODULE:send_rpc(To, Rpc, State0) of ok -> {State0, Actions}; - nosuspend -> + nosuspend when PeerStatus == normal -> %% update peer status to suspended and spawn a process %% to send the rpc without nosuspend so that it will block until %% the data can get through @@ -1410,11 +1417,13 @@ handle_effect(_RaftState, {send_rpc, To, Rpc}, _, %% the peer status back to normal ok = gen_statem:cast(To, Rpc), incr_counter(Conf, ?C_RA_SRV_MSGS_SENT, 1), - Self ! {update_peer, To, #{status => normal}} + Self ! {unsuspend_peer, To} end), - ?DEBUG("~ts: temporarily suspending peer ~w due to full distribution buffer", - [log_id(State0), To]), + % ?DEBUG("~ts: temporarily suspending peer ~w due to full distribution buffer ~W", + % [log_id(State0), To, Rpc, 5]), {update_peer(To, #{status => suspended}, State0), Actions}; + nosuspend -> + {State0, Actions}; noconnect -> %% for noconnects just allow it to pipeline and catch up later {State0, Actions} @@ -1976,6 +1985,8 @@ send_snapshots(Id, Term, {_, ToNode} = To, ChunkSize, Result = read_chunks_and_send_rpc(RPC, To, ReadState, 1, ChunkSize, InstallTimeout, SnapState), + ?DEBUG("~ts: sending snapshot to ~w completed", + [LogId, To]), ok = gen_statem:cast(Id, {To, Result}) end. diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index c3cb521f8..7c67c9ea4 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -71,8 +71,8 @@ snapshot_replication_interrupted(_Config) -> Data = crypto:strong_rand_bytes(100_000), %% write 10k entries of the same key [{ok, #{}} = ra_kv:put(KvId, term_to_binary(I), Data, 5000) - || I <- lists:seq(1, 50_000)], - ?assertMatch({ok, #{machine := #{num_keys := 50_000}}, KvId}, + || I <- lists:seq(1, 10_000)], + ?assertMatch({ok, #{machine := #{num_keys := 10_000}}, KvId}, ra:member_overview(KvId)), ra_log_wal:force_roll_over(ra_log_wal), @@ -99,7 +99,7 @@ snapshot_replication_interrupted(_Config) -> ct:pal("ra_state ~p", [ets:tab2list(ra_state)]), ok = ra:stop_server(?SYS, KvId3), [{ok, #{}} = ra_kv:put(KvId, term_to_binary(I), Data, 5000) - || I <- lists:seq(50_001, 50_010)], + || I <- lists:seq(10_001, 10_010)], ok = ra:restart_server(?SYS, KvId3), {ok, #{log := #{last_index := Kv1LastIndex }}, _} = ra:member_overview(KvId), ok = ra_lib:retry( diff --git a/test/ra_server_SUITE.erl b/test/ra_server_SUITE.erl index 556864b53..dd83beb2a 100644 --- a/test/ra_server_SUITE.erl +++ b/test/ra_server_SUITE.erl @@ -273,7 +273,7 @@ recover_restores_cluster_changes(_Config) -> % n2 joins {leader, #{cluster := Cluster, - log := Log0}, _} = + log := Log0}, _Effs} = ra_server:handle_leader({command, {'$ra_join', meta(), N2, await_consensus}}, State), {LIdx, _} = ra_log:last_index_term(Log0), From e368de3cfafd2a25ff393ac7b17db38eff86af5f Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Fri, 4 Jul 2025 14:34:38 +0100 Subject: [PATCH 46/60] compaction refactoring --- src/ra_lib.erl | 9 ++ src/ra_log_segments.erl | 159 ++++++++++++++---------- test/ra_kv_SUITE.erl | 2 +- test/ra_log_segments_SUITE.erl | 216 +++++++++++++++++++++++++++------ 4 files changed, 283 insertions(+), 103 deletions(-) diff --git a/src/ra_lib.erl b/src/ra_lib.erl index 0c0b80a81..581ee4d2b 100644 --- a/src/ra_lib.erl +++ b/src/ra_lib.erl @@ -49,6 +49,7 @@ lists_shuffle/1, is_dir/1, is_file/1, + is_any_file/1, ensure_dir/1, consult/1, cons/2 @@ -466,6 +467,14 @@ is_file(File) -> false end. +is_any_file(File) -> + case prim_file:read_file_info(File) of + {ok, #file_info{}} -> + true; + _ -> + false + end. + -spec consult(file:filename()) -> {ok, term()} | {error, term()}. diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index f9156fb3c..7d3352ae9 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -573,6 +573,14 @@ decr_counter(#cfg{counter = undefined}, _, _) -> segment_files(Dir, Fun) -> list_files(Dir, ".segment", Fun). +% delete_files(Dir, Ext) -> +% case list_files(Dir, Ext, fun (_) -> true end) of +% [] -> +% ok; +% Files -> +% [prim_file:delete(filename:join(Dir, F)) || F <- Files] +% end. + list_files(Dir, Ext) -> list_files(Dir, Ext, fun (_) -> true end). @@ -622,10 +630,13 @@ major_compaction(#{dir := Dir} = CompConf, SegRefs, LiveIndexes) -> CompConf), Compacted0 = [begin + %% create a compaction marker with the compaction group i + CompactionMarker = filename:join(Dir, with_ext(CompGroupLeaderFn, + ".compaction_group")), + ok = ra_lib:write_file(CompactionMarker, term_to_binary(All)), %% create a new segment with .compacting extension - AllShortFns = [F || {_, _, {F, _}} <- All], - CompactingShortFn = make_compacting_file_name(AllShortFns), - CompactingFn = filename:join(Dir, CompactingShortFn), + CompactingFn = filename:join(Dir, with_ext(CompGroupLeaderFn, + ".compacting")), %% max_count is the sum of all live indexes for segments in the %% compaction group MaxCount = lists:sum([ra_seq:length(S) || {_, S, _} <- All]), @@ -641,28 +652,24 @@ major_compaction(#{dir := Dir} = CompConf, SegRefs, LiveIndexes) -> end, CompSeg0, All), ok = ra_log_segment:close(CompSeg), - %% link .compacting segment to the original .segment file - %% first we have to create a hard link to a new .compacted file - %% from the .compacting file (as we need to keep this as a marker - %% until the end - %% then we can rename this on top of the first segment file in the - %% group (the target) - FirstSegmentFn = filename:join(Dir, FstFn0), - CompactedFn = filename:join(Dir, with_ext(FstFn0, ".compacted")), - ok = prim_file:make_link(CompactingFn, CompactedFn), - ok = prim_file:rename(CompactedFn, FirstSegmentFn), + FirstSegmentFn = filename:join(Dir, CompGroupLeaderFn), %% perform sym linking of the additional segments in the compaction - %% group - ok = make_links(Dir, FirstSegmentFn, - [F || {_, _, {F, _}} <- Additional]), - %% finally deleted the .compacting file to signal compaction group - %% is complete - ok = prim_file:delete(CompactingFn), + %% group, the target is not yet updated which can be detected at + %% recovery by the presence of a sym link _and_ the .compacting + %% file + ok = make_symlinks(Dir, FirstSegmentFn, + [F || {_, _, {F, _}} <- Additional]), + + %% rename the .compacting segment on top of the group leader + ok = prim_file:rename(CompactingFn, FirstSegmentFn), + %% finally delete the .compaction_marker file to signal + %% compaction group is complete + ok = prim_file:delete(CompactionMarker), %% return the new segref and additional segment keys {ra_log_segment:segref(FirstSegmentFn), [A || {_, _, {A, _}} <- Additional]} - end || [{_Info, _, {FstFn0, _}} | Additional] = All + end || [{_Info, _, {CompGroupLeaderFn, _}} | Additional] = All <- CompactionGroups], {Compacted, AddDelete} = lists:unzip(Compacted0), @@ -705,7 +712,7 @@ compactable_segrefs(SnapIdx, State) -> end, [], Compactable) end. -make_links(Dir, To, From) +make_symlinks(Dir, To, From) when is_list(From) -> [begin SymFn = filename:join(Dir, with_ext(FromFn, ".link")), @@ -769,55 +776,77 @@ take_group([{#{num_entries := NumEnts, {lists:reverse(Acc), Rem} end. - -parse_compacting_filename(Fn) when is_binary(Fn) -> - binary:split(filename:rootname(Fn), <<"-">>, [global]). - -make_compacting_file_name([N1 | Names]) -> - Root = lists:foldl(fun (N, Acc) -> - [filename:rootname(N), <<"-">> | Acc] - end, [N1], Names), - iolist_to_binary(lists:reverse([<<".compacting">> | Root])). - recover_compaction(Dir) -> - case list_files(Dir, ".compacting") of + case list_files(Dir, ".compaction_group") of [] -> %% no pending compactions #compaction_result{}; - [ShortFn] -> - %% compaction recovery is needed - CompactingFn = filename:join(Dir, ShortFn), - {ok, #file_info{links = Links}} = - file:read_link_info(CompactingFn, [raw, {time, posix}]), - case Links of - 1 -> - %% must have exited before the target file was renamed - %% just delete - ok = prim_file:delete(CompactingFn), + [CompactionGroupFn0] -> + %% compaction recovery is needed as there is a .compaction_group file + CompactionGroupFn = filename:join(Dir, CompactionGroupFn0), + %% if corrupt, just delete .compaction_group file + {ok, Bin} = prim_file:read_file(CompactionGroupFn), + CompactionGroup = try binary_to_term(Bin) of + Group -> + Group + catch _:_ -> + %% any error just return empty + _ = prim_file:delete(CompactionGroupFn), + [] + end, + + %% there _may_ be a .compacting file + CompactingFn = filename:join(Dir, with_ext(CompactionGroupFn0, + ".compacting")), + + case CompactionGroup of + [] -> + #compaction_result{}; + [_] -> + %% single segment compaction, we cannot know if the + %% compaction into the compacting segment completed or + %% not + %% ignore return value as CompactingFn may not exist + _ = prim_file:delete(CompactingFn), + ok = prim_file:delete(CompactionGroupFn), #compaction_result{}; - 2 -> - [FstFn | RemFns] = parse_compacting_filename(ShortFn), - %% there may be a .compacted file - Target = filename:join(Dir, with_ext(FstFn, ".segment")), - case list_files(Dir, ".compacted") of - [CompactedShortFn] -> - CompactedFn = filename:join(Dir, CompactedShortFn), - %% all entries were copied but it failed before - %% this hard link could be renamed over the target - ok = prim_file:rename(CompactedFn, Target), - ok; - [] -> - %% links may not have been fully created, - %% delete all .link files then relink - ok - end, - ok = make_links(Dir, Target, RemFns), - ok = prim_file:delete(CompactingFn), - - Linked = [with_ext(L, ".segment") || L <- RemFns], - Compacted = [ra_log_segment:segref(Target)], - #compaction_result{compacted = Compacted, - linked = Linked} + [TargetShortFn | [FstLinkSeg | _] = LinkTargets] -> + %% multiple segments in group, + %% if any of the additional segments is a symlink + %% the writes to the .compacting segment completed and we + %% can complete the compaction work + FstLinkSegFn = filename:join(Dir, FstLinkSeg), + FstLinkSegLinkFn = filename:join(Dir, with_ext(FstLinkSeg, ".link")), + Target = filename:join(Dir, TargetShortFn), + AtLeastOneLink = ra_lib:is_any_file(FstLinkSegLinkFn), + CompactingExists = ra_lib:is_any_file(CompactingFn), + case file:read_link_info(FstLinkSegFn, [raw]) of + {ok, #file_info{type = Type}} + when Type == symlink orelse + AtLeastOneLink -> + %% it is a symlink, recreate all symlinks and delete + %% compaction marker + ok = make_symlinks(Dir, Target, LinkTargets), + %% if compacting file exists, rename it to target + if CompactingExists -> + ok = prim_file:rename(CompactingFn, Target); + true -> + ok + end, + ok = prim_file:delete(CompactionGroupFn), + Compacted = [ra_log_segment:segref(Target)], + #compaction_result{compacted = Compacted, + linked = LinkTargets}; + {error, enoent} -> + %% segment does not exist indicates what exactly? + _ = prim_file:delete(CompactingFn), + ok = prim_file:delete(CompactionGroupFn), + #compaction_result{}; + {ok, #file_info{type = regular}} -> + _ = prim_file:delete(CompactingFn), + ok = prim_file:delete(CompactionGroupFn), + #compaction_result{} + end end end. diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index 7c67c9ea4..4c1744e3f 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -107,7 +107,7 @@ snapshot_replication_interrupted(_Config) -> {ok, #{log := #{last_index := LastIdx}}, _} = ra:member_overview(KvId3), Kv1LastIndex == LastIdx - end, 100, 100), + end, 100, 256), ra:delete_cluster([KvId, {Kv2, node()}, KvId3]), ok. diff --git a/test/ra_log_segments_SUITE.erl b/test/ra_log_segments_SUITE.erl index e517098ff..efff70e96 100644 --- a/test/ra_log_segments_SUITE.erl +++ b/test/ra_log_segments_SUITE.erl @@ -25,6 +25,8 @@ all_tests() -> [ recover1, recover2, + recover3, + recover4, basics, major, major_max_size, @@ -117,22 +119,48 @@ recover1(Config) -> %% where linked (.compacting link count > 1) Dir = ?config(dir, Config), - CompactingFn = <<"0000000000000001-0000000000000002-0000000000000003.compacting">>, + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + SegmentFn = <<"0000000000000001.segment">>, + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), Scen = [ {entries, 1, lists:seq(1, 128 * 4)}, {assert, fun (_) -> - ok = ra_lib:write_file(filename:join(Dir, CompactingFn), <<>>), + Group = term_to_binary([SegmentFn]), + ok = ra_lib:write_file(CompactionGroupFn, Group), + ok = ra_lib:write_file(CompactingFn, <<>>), true end}, reinit, {assert, fun (_) -> - %% a compacting file with 1 link only should just be deleted - %% during init - not filelib:is_file(filename:join(Dir, CompactingFn)) + %% compaction groups and compacting files should be removed + not filelib:is_file(CompactionGroupFn) andalso + not filelib:is_file(CompactingFn) + end}, + {assert, fun (_) -> + %% only compaction_group case + Group = term_to_binary([SegmentFn]), + ok = ra_lib:write_file(CompactionGroupFn, Group), + true + end}, + reinit, + {assert, fun (_) -> + %% compaction groups and compacting files should be removed + not filelib:is_file(CompactionGroupFn) andalso + not filelib:is_file(CompactingFn) + end}, + {assert, fun (_) -> + %% partially written compaction_group file case + ok = ra_lib:write_file(CompactionGroupFn, <<"banana">>), + true + end}, + reinit, + {assert, fun (_) -> + %% compaction groups and compacting files should be removed + not filelib:is_file(CompactionGroupFn) andalso + not filelib:is_file(CompactingFn) end} ], - ct:pal("infos ~p", [infos(Dir)]), SegConf = #{max_count => 128}, Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), @@ -142,61 +170,161 @@ recover2(Config) -> Dir = ?config(dir, Config), LiveList = lists:seq(1, 128 * 3, 8), Live = ra_seq:from_list(LiveList), - CompactingShortFn = <<"0000000000000001-0000000000000002-0000000000000003.compacting">>, - CompactingFn = filename:join(Dir, CompactingShortFn), + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), Scen = [ {entries, 1, lists:seq(1, 128 * 4)}, {assert, fun (_) -> - %% fake a .compacting file, and perform a copy - %% (this code is copied from ra_log_segments - All = lists:reverse(tl(seg_refs(Dir))), - {ok, CompSeg0} = ra_log_segment:open(CompactingFn, - #{max_count => 128}), - CompSeg = lists:foldl( - fun ({F, R}, S0) -> - L = ra_seq:in_range(R, Live), - {ok, S} = ra_log_segment:copy( - S0, filename:join(Dir, F), - ra_seq:expand(L)), - S - end, CompSeg0, All), - ok = ra_log_segment:close(CompSeg), - [{FstFn, _}, {SndFn, _}, {_ThrFn, _}] = All, - - CompactedFn = filename:join(Dir, with_ext(FstFn, ".compacted")), - ok = prim_file:make_link(CompactingFn, CompactedFn), + %% compactable segrefs + SegRefs = lists:reverse(tl(seg_refs(Dir))), + Segments = [F || {F, _} <- SegRefs], + %% write compaction_group file + ok = ra_lib:write_file(CompactionGroupFn, + term_to_binary(Segments)), + do_compaction(Dir, CompactingFn, Live, SegRefs), + + [{FstFn, _}, {SndFn, _}, {_ThrFn, _}] = SegRefs, + + %% create a .link for for the second compacted segment + %% and symlink it to the first segment in the compaction + %% group FirstSegmentFn = filename:join(Dir, FstFn), - ok = prim_file:rename(CompactedFn, FirstSegmentFn), - % SecondFn = filename:join(Dir, SndFn), SndLinkFn = filename:join(Dir, with_ext(SndFn, ".link")), ok = prim_file:make_symlink(FirstSegmentFn, SndLinkFn), - %% the first segment has now been replaced with a link - %% to the compacting segment but not all symlinks may have been created + + %% rename the .link file on top of the .segment file + SndSegFn = filename:join(Dir, SndFn), + ok = prim_file:rename(SndLinkFn, SndSegFn), + %% this simulates a case where it stopped after only + %% creating 1 of the two symlinks true end}, reinit, + {assert, 1, LiveList}, {assert, fun (_) -> Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), NumLinks = length([a || #{file_type := symlink} <- Infos]), %% a compacting file with 1 link only should just be deleted %% during init not filelib:is_file(CompactingFn) andalso + not filelib:is_file(CompactionGroupFn) andalso NumLinks == 2 end}, + {assert, 1, LiveList}, {assert, fun(S) -> SegRefs = ra_log_segments:segment_refs(S), - ct:pal("SegRefs ~p, ~p", [SegRefs, seg_refs(Dir)]), SegRefs == seg_refs(Dir) end} + ], + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), + ok. +recover3(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + %% compactable segrefs + SegRefs = lists:reverse(tl(seg_refs(Dir))), + Segments = [F || {F, _} <- SegRefs], + %% write compaction_group file + ok = ra_lib:write_file(CompactionGroupFn, + term_to_binary(Segments)), + do_compaction(Dir, CompactingFn, Live, SegRefs), + + [{FstFn, _}, {SndFn, _}, {_ThrFn, _}] = SegRefs, + + %% create a .link for for the second compacted segment + %% and symlink it to the first segment in the compaction + %% group + FirstSegmentFn = filename:join(Dir, FstFn), + SndLinkFn = filename:join(Dir, with_ext(SndFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, SndLinkFn), + true + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (_) -> + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + NumLinks = length([a || #{file_type := symlink} <- Infos]), + %% a compacting file with 1 link only should just be deleted + %% during init + not filelib:is_file(CompactingFn) andalso + not filelib:is_file(CompactionGroupFn) andalso + NumLinks == 2 + end}, + {assert, 1, LiveList}, + {assert, fun(S) -> + SegRefs = ra_log_segments:segment_refs(S), + SegRefs == seg_refs(Dir) + end} + ], + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), + ok. - +recover4(Config) -> + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128 * 3, 8), + Live = ra_seq:from_list(LiveList), + CompactionGroupFn = filename:join(Dir, <<"0000000000000001.compaction_group">>), + CompactingFn = filename:join(Dir, <<"0000000000000001.compacting">>), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 4)}, + {assert, fun (_) -> + %% compactable segrefs + SegRefs = lists:reverse(tl(seg_refs(Dir))), + Segments = [F || {F, _} <- SegRefs], + %% write compaction_group file + ok = ra_lib:write_file(CompactionGroupFn, + term_to_binary(Segments)), + do_compaction(Dir, CompactingFn, Live, SegRefs), + + [{FstFn, _}, {SndFn, _}, {ThrFn, _}] = SegRefs, + + %% create a .link for for the second compacted segment + %% and symlink it to the first segment in the compaction + %% group + FirstSegmentFn = filename:join(Dir, FstFn), + SndLinkFn = filename:join(Dir, with_ext(SndFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, SndLinkFn), + ThrLinkFn = filename:join(Dir, with_ext(ThrFn, ".link")), + ok = prim_file:make_symlink(FirstSegmentFn, ThrLinkFn), + %% all symlinks completed but .compacting file was not + %% renamed + true + end}, + reinit, + {assert, 1, LiveList}, + {assert, fun (_) -> + Infos = infos(Dir), + ct:pal("Infos ~p", [Infos]), + NumLinks = length([a || #{file_type := symlink} <- Infos]), + %% a compacting file with 1 link only should just be deleted + %% during init + not filelib:is_file(CompactingFn) andalso + not filelib:is_file(CompactionGroupFn) andalso + NumLinks == 2 + end}, + {assert, 1, LiveList}, + {assert, fun(S) -> + SegRefs = ra_log_segments:segment_refs(S), + ct:pal("SegRefs ~p, ~p", [SegRefs, seg_refs(Dir)]), + SegRefs == seg_refs(Dir) + end} ], - SegConf = #{max_count => 128}, Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), - run_scenario([{seg_conf, SegConf} | Config], Segs0, Scen), - ct:pal("infos ~p", [infos(Dir)]), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), ok. basics(Config) -> @@ -501,8 +629,8 @@ segment_files(Dir) -> begin Ext = filename:extension(F), lists:member(Ext, [".segment", - ".compacting", - ".compacted", + % ".compacting", + % ".compacted", ".link"]) end], lists:sort(Files); @@ -599,3 +727,17 @@ ra_log_segments_init(Config, Dir, SegRefs) -> ra_log_segments:init(UId, Dir, 1, random, SegRefs, undefined, CompConf, ""). + +do_compaction(Dir, CompactingFn, Live, All) -> + {ok, CompSeg0} = ra_log_segment:open(CompactingFn, + #{max_count => 128}), + CompSeg = lists:foldl( + fun ({F, R}, S0) -> + L = ra_seq:in_range(R, Live), + {ok, S} = ra_log_segment:copy( + S0, filename:join(Dir, F), + ra_seq:expand(L)), + S + end, CompSeg0, All), + ok = ra_log_segment:close(CompSeg), + ok. From a444057348676b60853d42880ccef0e39fb0736f Mon Sep 17 00:00:00 2001 From: Michal Kuratczyk Date: Mon, 7 Jul 2025 10:25:25 +0200 Subject: [PATCH 47/60] [skip ci] Tool for stress-testing Ra KV to test Ra v3 --- src/ra_kv_harness.erl | 625 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 625 insertions(+) create mode 100644 src/ra_kv_harness.erl diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl new file mode 100644 index 000000000..df6ed2182 --- /dev/null +++ b/src/ra_kv_harness.erl @@ -0,0 +1,625 @@ +-module(ra_kv_harness). + +-export([ + run/1, + run/2, + setup_cluster/1, + teardown_cluster/1, + timestamp/0, + log/2 + ]). + +-include_lib("eunit/include/eunit.hrl"). + +-define(SYS, default). +-define(CLUSTER_NAME, kv). +-define(TIMEOUT, 30000). +-define(MAX_KEY, 10000). % Limited key space for more conflicts +-define(MIN_VALUE_SIZE, 1). +-define(MAX_VALUE_SIZE, 10_000_000). % 10MB + +-type state() :: #{members := #{ra:server_id() => peer:server_ref()}, + reference_map := #{binary() => term()}, + operations_count := non_neg_integer(), + successful_ops := non_neg_integer(), + failed_ops := non_neg_integer(), + next_node_id := pos_integer(), + remaining_ops := non_neg_integer(), + consistency_failed := boolean()}. + +-spec timestamp() -> string(). +timestamp() -> + {MegaSecs, Secs, MicroSecs} = os:timestamp(), + {{Year, Month, Day}, {Hour, Min, Sec}} = calendar:now_to_local_time({MegaSecs, Secs, MicroSecs}), + Millisecs = MicroSecs div 1000, + io_lib:format("[~4..0w-~2..0w-~2..0w ~2..0w:~2..0w:~2..0w.~3..0w]", + [Year, Month, Day, Hour, Min, Sec, Millisecs]). + +-spec log(string(), list()) -> ok. +log(Format, Args) -> + Message = io_lib:format(Format, Args), + io:format("~s", [Message]), + file:write_file("ra_kv_harness.log", Message, [append]). + +-spec new_state() -> state(). +new_state() -> + #{members => #{}, + reference_map => #{}, + operations_count => 0, + successful_ops => 0, + failed_ops => 0, + next_node_id => 1, + remaining_ops => 0, + consistency_failed => false}. + +-type operation() :: {put, Key :: binary(), Value :: term()} | + {get, Key :: binary()} | + {snapshot} | + {update_almost_all_keys} | + {add_member} | + {remove_member}. + +-spec run(NumOperations :: pos_integer()) -> + {ok, #{successful := non_neg_integer(), + failed := non_neg_integer(), + consistency_checks := non_neg_integer()}} | + {error, term()}. +run(NumOperations) -> + run(NumOperations, #{}). + +read_all_keys_loop(Members) when is_list(Members) -> + Member = lists:nth(rand:uniform(length(Members)), Members), + T1 = erlang:monotonic_time(), + [{ok, _, _} = ra_kv:get(Member, <<"key_", (integer_to_binary(N))/binary>>, 1000) || N <- lists:seq(1, ?MAX_KEY)], + T2 = erlang:monotonic_time(), + Diff = erlang:convert_time_unit(T2 - T1, native, millisecond), + log("~s Read all keys from member ~p in ~bms~n", [timestamp(), Member, Diff]), + read_all_keys_loop(Members). + +-spec run(NumOperations :: pos_integer(), + Options :: map()) -> + {ok, #{successful := non_neg_integer(), + failed := non_neg_integer(), + consistency_checks := non_neg_integer()}} | + {error, term()}. +run(NumOperations, _Options) when NumOperations > 0 -> + % Start with a random number of nodes between 1 and 7 + NumNodes = rand:uniform(7), + logger:set_primary_config(level, warning), + application:set_env(sasl, sasl_error_logger, false), + application:stop(sasl), + log("~s Starting cluster with ~p nodes~n", [timestamp(), NumNodes]), + case setup_cluster(NumNodes) of + {ok, Members, PeerNodes} -> + MembersMap = maps:from_list(lists:zip(Members, PeerNodes)), + InitialState = (new_state())#{members => MembersMap, next_node_id => NumNodes + 1, remaining_ops => NumOperations}, + try + State = execute_operation(InitialState, {put, <<"never_updated">>, <<"never_updated">>}), + %% keep reading all keys while the other operations are running + %spawn(fun() -> read_all_keys_loop(maps:keys(MembersMap)) end), + FinalState = run_operations(State, ?CLUSTER_NAME), + case maps:get(consistency_failed, FinalState, false) of + true -> + log("~s EMERGENCY STOP: Leaving cluster running for investigation~n", [timestamp()]), + {error, {consistency_failure, FinalState}}; + false -> + ConsistencyChecks = validate_final_consistency(FinalState), + teardown_cluster(FinalState), + {ok, #{successful => maps:get(successful_ops, FinalState), + failed => maps:get(failed_ops, FinalState), + consistency_checks => ConsistencyChecks}} + end + catch + Class:Reason:Stack -> + teardown_cluster(InitialState), + {error, {Class, Reason, Stack}} + end; + {error, Reason} -> + {error, Reason} + end. + +-spec setup_cluster(NumNodes :: pos_integer()) -> + {ok, [ra:server_id()], [peer:server_ref()]} | {error, term()}. +setup_cluster(NumNodes) when NumNodes > 0 -> + % Start peer nodes + case start_peer_nodes(NumNodes) of + {ok, PeerNodes, NodeNames} -> + Members = [{?CLUSTER_NAME, NodeName} || NodeName <- NodeNames], + + % Start ra application on all peer nodes + [ + begin + % Set logger level to reduce verbosity on peer node + erpc:call(NodeName, logger, set_primary_config, [level, warning]), + erpc:call(NodeName, application, set_env, [sasl, sasl_error_logger, false]), + erpc:call(NodeName, application, stop, [sasl]), + {ok, _} = erpc:call(NodeName, ra, start_in, [NodeName]) + % {ok, _} = erpc:call(NodeName, ra_system, start, [#{name => default, data_dir => atom_to_list(NodeName), names => ra_system:derive_names(default)}]) + end + || NodeName <- NodeNames], + + case ra_kv:start_cluster(?SYS, ?CLUSTER_NAME, #{members => Members}) of + {ok, StartedMembers, _} -> + log("~s Started cluster with ~p members~n", [timestamp(), length(StartedMembers)]), + {ok, StartedMembers, PeerNodes}; + {error, Reason} -> + [peer:stop(PeerRef) || PeerRef <- PeerNodes], + {error, Reason} + end; + {error, Reason} -> + {error, Reason} + end. + +-spec start_peer_nodes(pos_integer()) -> + {ok, [peer:server_ref()], [node()]} | {error, term()}. +start_peer_nodes(NumNodes) -> + start_peer_nodes(NumNodes, [], []). + +start_peer_nodes(0, PeerRefs, NodeNames) -> + {ok, lists:reverse(PeerRefs), lists:reverse(NodeNames)}; +start_peer_nodes(N, PeerRefs, NodeNames) when N > 0 -> + case start_single_peer_node(N) of + {ok, PeerRef, NodeName} -> + start_peer_nodes(N - 1, [PeerRef | PeerRefs], [NodeName | NodeNames]); + {error, Reason} -> + % Clean up any already started peers + [peer:stop(PeerRef) || PeerRef <- PeerRefs], + {error, Reason} + end. + +-spec start_single_peer_node(pos_integer()) -> {ok, peer:server_ref(), node()} | {error, term()}. +start_single_peer_node(NodeId) -> + NodeName = list_to_atom("ra_test_" ++ integer_to_list(NodeId) ++ "@" ++ + inet_db:gethostname()), + + % Get all code paths from current node + CodePaths = code:get_path(), + PaArgs = lists:flatmap(fun(Path) -> ["-pa", Path] end, CodePaths), + BufferSize = ["+zdbbl", "102400"], + + case peer:start_link(#{name => NodeName, + args => PaArgs ++ BufferSize}) of + {ok, PeerRef, NodeName} -> + % Set logger level to reduce verbosity on peer node + erpc:call(NodeName, logger, set_primary_config, [level, warning]), + % Start ra application on the new peer node + {ok, _} = erpc:call(NodeName, ra, start_in, [NodeName]), + {ok, PeerRef, NodeName}; + {error, Reason} -> + {error, Reason} + end. + +-spec start_new_peer_node(pos_integer()) -> {ok, peer:server_ref(), node()} | {error, term()}. +start_new_peer_node(NodeId) -> + start_single_peer_node(NodeId). + +-spec teardown_cluster(state()) -> ok. +teardown_cluster(#{members := Members}) -> + % Stop Ra servers on each node and stop peer nodes + maps:foreach(fun(Member, PeerRef) -> + NodeName = element(2, Member), + log("~s Stopping member ~p~n", [timestamp(), Member]), + catch erpc:call(NodeName, ra, stop_server, [?SYS, Member]), + catch peer:stop(PeerRef), + ok + end, Members), + ok. + +run_operations(State, _ClusterName) -> + RemainingOps = maps:get(remaining_ops, State), + case RemainingOps =< 0 of + true -> + State; + false -> + case RemainingOps rem 100000 of + 0 -> log("~s ~p operations remaining~n", [timestamp(), RemainingOps]); + _ -> ok + end, + Operation = generate_operation(), + NewState = execute_operation(State, Operation), + + % Update remaining operations count + UpdatedState = NewState#{remaining_ops => RemainingOps - 1}, + + % Validate consistency every 100 operations + ValidationState = case maps:get(operations_count, UpdatedState) rem 100 of + 0 -> validate_consistency(UpdatedState); + _ -> UpdatedState + end, + + run_operations(ValidationState, _ClusterName) + end. + +-spec generate_operation() -> operation(). +generate_operation() -> + case rand:uniform(100) of + 1 -> % 1% update almost all keys + {update_almost_all_keys}; + 2 -> % 1% add member + {add_member}; + 3 -> % 1% remove member + {remove_member}; + N when N =< 7 -> % 1% snapshot + {snapshot}; + N when N =< 80 -> + Key = generate_key(), + Value = generate_value(), + {put, Key, Value}; + _ -> + Key = generate_key(), + {get, Key} + end. + +key(N) when is_integer(N) -> + <<"key_", (integer_to_binary(N))/binary>>. +generate_key() -> + KeyNum = rand:uniform(?MAX_KEY), % Limited key space for more conflicts + key(KeyNum). + +-spec generate_value() -> binary(). +generate_value() -> + Size = rand:uniform(?MAX_VALUE_SIZE - ?MIN_VALUE_SIZE) + ?MIN_VALUE_SIZE, + rand:bytes(Size). + +-spec execute_operation(state(), operation()) -> state(). +execute_operation(State, {put, Key, Value}) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + + % Pick a random cluster member to send the operation to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + + case ra_kv:put(Member, Key, Value, ?TIMEOUT) of + {ok, _Meta} -> + NewRefMap = RefMap#{Key => Value}, + State#{reference_map => NewRefMap, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {error, _Reason} -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + {timeout, _ServerId} -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1} + end; + +execute_operation(State, {get, Key}) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + + % Pick a random cluster member to send the operation to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + %NodeName = element(2, Member), + RefValue = maps:get(Key, RefMap, not_found), + + case ra_kv:get(Member, Key, ?TIMEOUT) of + {ok, {error, not_found}, _Value} when RefValue =:= not_found -> + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {ok, {error, not_found}} when RefValue =/= not_found -> + log("~s CONSISTENCY ERROR: Key ~p should exist but not found~n", [timestamp(), Key]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + {ok, _Meta, Value} when RefValue =:= Value -> + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {ok, _Meta, Value} when RefValue =/= Value -> + log("~s CONSISTENCY ERROR: Key ~p, Expected ~p, Got ~p~n", + [timestamp(), Key, RefValue, Value]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + _ -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1} + end; + +execute_operation(State, {update_almost_all_keys}) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + + % Pick a random cluster member to send the operations to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + + X = rand:uniform(100), + {T, _ } = timer:tc(fun() -> + [ {ok, _} = ra_kv:put(Member, key(N), 0, ?TIMEOUT) || N <- lists:seq(1, ?MAX_KEY), + N rem X =/= 0] end), % Update every 100th key + log("~s Updated roughly 99% of the ~p keys in ~bms...~n", [timestamp(), ?MAX_KEY, T div 1000]), + + NewRefMap = maps:merge(RefMap, #{ key(N) => 0 || N <- lists:seq(1, ?MAX_KEY), + N rem X =/= 0}), % Update every 100th key + State#{reference_map => NewRefMap, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + +execute_operation(State, {snapshot}) -> + Members = maps:get(members, State), + + % Pick a random cluster member to send snapshot command to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + + case erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]) of + undefined -> + State; + _Pid -> + log("~s Rollover/snapshot/compaction on node ~p...~n", [timestamp(), NodeName]), + erpc:call(NodeName, ra_log_wal, force_roll_over, [ra_log_wal]), + erpc:call(NodeName, ra, aux_command, [Member, take_snapshot]), + erpc:call(NodeName, ra, trigger_compaction, [Member]), + State + end; + +execute_operation(State, {add_member}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + NextNodeId = maps:get(next_node_id, State), + + % Don't add members if we already have 7 (maximum 7 nodes) + case maps:size(Members) >= 7 of + true -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + false -> + + case start_new_peer_node(NextNodeId) of + {ok, PeerRef, NodeName} -> + NewMember = {?CLUSTER_NAME, NodeName}, + + % Pick a random existing member to send the add_member command to + MembersList = maps:keys(Members), + ExistingMember = lists:nth(rand:uniform(length(MembersList)), MembersList), + + try ra_kv:add_member(?SYS, NewMember, ExistingMember) of + ok -> + NewMembers = Members#{NewMember => PeerRef}, + NewMembersList = maps:keys(NewMembers), + log("~s Added member ~p. Cluster now has ~p members: ~0p~n", [timestamp(), NewMember, length(NewMembersList), NewMembersList]), + State#{members => NewMembers, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1, + next_node_id => NextNodeId + 1} + catch + _:Reason -> + log("~s Failed to add member ~p: ~p~n", [timestamp(), NewMember, Reason]), + % Clean up the peer node since add failed + catch peer:stop(PeerRef), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1, + next_node_id => NextNodeId + 1} + end; + {error, Reason} -> + log("~s Failed to start peer node: ~p~n", [timestamp(), Reason]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1, + next_node_id => NextNodeId + 1} + end + end; + +execute_operation(State, {remove_member}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + FailedOps = maps:get(failed_ops, State), + + % Don't remove members if we only have one left (minimum 1 node) + case maps:size(Members) =< 1 of + true -> + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1}; + false -> + % Pick a random member to remove + MembersList = maps:keys(Members), + MemberToRemove = lists:nth(rand:uniform(length(MembersList)), MembersList), + log("~s Removing member ~p...", [timestamp(), MemberToRemove]), + + % Pick a different member to send the remove command to + RemainingMembers = MembersList -- [MemberToRemove], + CommandTarget = lists:nth(rand:uniform(length(RemainingMembers)), RemainingMembers), + + case ra:remove_member(CommandTarget, MemberToRemove, ?TIMEOUT) of + {ok, _Meta, _} -> + % Stop the peer node for the removed member + case maps:get(MemberToRemove, Members, undefined) of + undefined -> + ok; + PeerRef -> + catch peer:stop(PeerRef) + end, + + NewMembers = maps:remove(MemberToRemove, Members), + NewMembersList = maps:keys(NewMembers), + log("~s done. Cluster now has ~p members: ~0p~n", [timestamp(), length(NewMembersList), NewMembersList]), + + State#{members => NewMembers, + operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + {error, Reason} -> + log("~s Failed to remove member ~p: ~p~n", [timestamp(), MemberToRemove, Reason]), + State#{operations_count => OpCount + 1, + failed_ops => FailedOps + 1} + end + end. + +-spec wait_for_applied_index_convergence([ra:server_id()], non_neg_integer()) -> ok. +wait_for_applied_index_convergence(Members, MaxRetries) when MaxRetries > 0 -> + IndicesMap = get_applied_indices(Members), + Indices = maps:values(IndicesMap), + case lists:uniq(Indices) of + [_SingleIndex] -> + ok; % All nodes have converged + _MultipleIndices -> + timer:sleep(100), % Wait 100ms before retry + wait_for_applied_index_convergence(Members, MaxRetries - 1) + end; +wait_for_applied_index_convergence(Members, 0) -> + IndicesMap = get_applied_indices(Members), + log("~s WARNING: Applied index convergence timeout. Reported values: ~0p~n", [timestamp(), IndicesMap]), + ok. + +-spec get_applied_indices([ra:server_id()]) -> #{ra:server_id() => ra:index() | undefined}. +get_applied_indices(Members) -> + maps:from_list([{Member, case ra:member_overview(Member, 1000) of + #{log := #{last_applied_index := Index}} -> + Index; + _ -> + undefined + end} || Member <- Members]). + +-spec validate_consistency(state()) -> state(). +validate_consistency(State) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + + % Wait for all nodes to converge to the same applied index + MembersList = maps:keys(Members), + wait_for_applied_index_convergence(MembersList, 300), % Wait up to 30 seconds + + % Check that all members have the same view + ValidationResults = [validate_member_consistency(Member, RefMap) || Member <- MembersList], + + Result1 = hd(ValidationResults), + case lists:all(fun(Result) -> Result =:= Result1 end, ValidationResults) of + true -> + State; + false -> + % Brief console output with live_indexes summary + LiveIndexesSummary = [{Member, case Result of + #{live_indexes := LI} -> length(LI); + _ -> error + end} || {Member, Result} <- lists:zip(MembersList, ValidationResults)], + log("~s Consistency check failed. Live indexes per node: ~p~n", [timestamp(), LiveIndexesSummary]), + log("~s STOPPING: No more operations will be performed due to consistency failure~n", [timestamp()]), + + % Write full details to log file with difference analysis + LogEntry = format_consistency_failure(MembersList, ValidationResults), + file:write_file("ra_kv_harness.log", LogEntry, [append]), + + FailedOps = maps:get(failed_ops, State), + State#{failed_ops => FailedOps + 1, remaining_ops => 0, consistency_failed => true} + end. + +-spec format_consistency_failure([ra:server_id()], [map() | error]) -> iolist(). +format_consistency_failure(Members, Results) -> + MemberResults = lists:zip(Members, Results), + + % Extract all unique results for comparison + UniqueResults = lists:usort([R || {_, R} <- MemberResults, R =/= error]), + + Header = io_lib:format("~s Consistency check failed:~n", [timestamp()]), + + % Log raw data + RawData = [io_lib:format(" Member ~p: ~p~n", [Member, Result]) || {Member, Result} <- MemberResults], + + % Analyze differences + DiffAnalysis = case UniqueResults of + [] -> + [" ANALYSIS: All members returned errors\n"]; + [_SingleResult] -> + [" ANALYSIS: All successful members have identical results (errors may exist)\n"]; + MultipleResults -> + [" ANALYSIS: Found ~p different result patterns:\n" | + [io_lib:format(" Pattern ~p: ~p\n", [I, Pattern]) || + {I, Pattern} <- lists:zip(lists:seq(1, length(MultipleResults)), MultipleResults)] ++ + [" DIFFERENCES:\n"] ++ + analyze_field_differences(MultipleResults)] + end, + + [Header, RawData, DiffAnalysis, "\n"]. + +-spec analyze_field_differences([map()]) -> iolist(). +analyze_field_differences(Results) -> + % Extract live_indexes and num_keys for comparison + LiveIndexes = [maps:get(live_indexes, R, undefined) || R <- Results, is_map(R)], + NumKeys = [maps:get(num_keys, R, undefined) || R <- Results, is_map(R)], + + LiveIndexDiff = case lists:usort(LiveIndexes) of + [_] -> []; + MultipleLI -> [io_lib:format(" live_indexes differ: ~p\n", [MultipleLI])] + end, + + NumKeysDiff = case lists:usort(NumKeys) of + [_] -> []; + MultipleNK -> [io_lib:format(" num_keys differ: ~p\n", [MultipleNK])] + end, + + [LiveIndexDiff, NumKeysDiff]. + +-spec validate_member_consistency(ra:server_id(), map()) -> map() | error. +validate_member_consistency(Member, _RefMap) -> + NodeName = element(2, Member), + case erpc:call(NodeName, ra_kv, member_overview, [Member]) of + #{machine := #{live_indexes := Live, num_keys := Num}} -> + %io:format("Member ~p overview: Live indexes ~p, Num keys ~p", [Member, Live, Num]), + #{live_indexes => Live, num_keys => Num}; + Error -> + log("~s Member ~p failed overview check: ~p~n", [timestamp(), Member, Error]), + error + end. + +-spec validate_final_consistency(state()) -> non_neg_integer(). +validate_final_consistency(State) -> + Members = maps:get(members, State), + RefMap = maps:get(reference_map, State), + + log("~s Performing final consistency validation...~n", [timestamp()]), + log("~s Reference map has ~p keys~n", [timestamp(), maps:size(RefMap)]), + + % Wait for all nodes to converge before final validation + MembersList = maps:keys(Members), + log("~s Waiting for applied index convergence...~n", [timestamp()]), + wait_for_applied_index_convergence(MembersList, 100), % Wait up to 10 seconds for final check + + % Validate all keys across all members + Keys = maps:keys(RefMap), + + MembersList = maps:keys(Members), + ValidationCount = lists:foldl( + fun(Key, Acc) -> + RefValue = maps:get(Key, RefMap), + case validate_key_across_members(Key, RefValue, MembersList) of + ok -> Acc + 1; + error -> Acc + end + end, 0, Keys), + + log("~s Final consistency check: ~p/~p keys validated successfully~n", + [timestamp(), ValidationCount, length(Keys)]), + ValidationCount. + +-spec validate_key_across_members(binary(), term(), [ra:server_id()]) -> ok | error. +validate_key_across_members(Key, ExpectedValue, Members) -> + Results = [begin + case ra_kv:get(Member, Key, ?TIMEOUT) of + {ok, _Meta, Value} when Value =:= ExpectedValue -> ok; + {ok, _Meta, Value} -> + log("~s Key ~p mismatch on ~p: expected ~p, got ~p~n", + [timestamp(), Key, Member, ExpectedValue, Value]), + error; + {error, not_found} -> + log("~s Key ~p not found on ~p but should exist~n", [timestamp(), Key, Member]), + error; + Other -> + log("~s Key ~p query failed on ~p: ~p~n", [timestamp(), Key, Member, Other]), + error + end + end || Member <- Members], + + case lists:all(fun(R) -> R =:= ok end, Results) of + true -> ok; + false -> error + end. From 82fdbaae89cfe0f8d6544fcf2fae15bae6f1f0f0 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 7 Jul 2025 10:03:35 +0100 Subject: [PATCH 48/60] avoid concurrent compactions --- src/ra_log_segments.erl | 52 +++++++++++++++++++++------------- test/ra_log_segments_SUITE.erl | 48 +++++++++++++++++++++++++++++-- 2 files changed, 79 insertions(+), 21 deletions(-) diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index 7d3352ae9..e5b3de149 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -54,7 +54,9 @@ -record(?STATE, {cfg :: #cfg{}, range :: ra_range:range(), segment_refs :: ra_lol:state(), - open_segments :: ra_flru:state() + open_segments :: ra_flru:state(), + compaction :: undefined | major | minor, + next_compaction :: undefined | major | minor }). -record(compaction_result, @@ -153,27 +155,30 @@ update_segments(NewSegmentRefs, #?STATE{open_segments = Open0, -spec schedule_compaction(minor | major, ra:index(), ra_seq:state(), state()) -> - [ra_server:effect()]. + {state(), [ra_server:effect()]}. schedule_compaction(Type, SnapIdx, LiveIndexes, #?MODULE{cfg = #cfg{log_id = LogId, compaction_conf = CompConf, - directory = Dir} = Cfg} = State) -> + directory = Dir} = Cfg, + compaction = undefined} = State) -> case compactable_segrefs(SnapIdx, State) of [] -> - []; + {State, []}; SegRefs when LiveIndexes == [] -> %% if LiveIndexes is [] we can just delete all compactable %% segment refs Unreferenced = [F || {F, _} <- SegRefs], Result = #compaction_result{unreferenced = Unreferenced}, - [{next_event, - {ra_log_event, {compaction_result, Result}}}]; + {State#?MODULE{compaction = minor}, + [{next_event, + {ra_log_event, {compaction_result, Result}}}]}; SegRefs when Type == minor -> %% TODO evaluate if minor compactions are fast enough to run %% in server process Result = minor_compaction(SegRefs, LiveIndexes), - [{next_event, - {ra_log_event, {compaction_result, Result}}}]; + {State#?MODULE{compaction = minor}, + [{next_event, + {ra_log_event, {compaction_result, Result}}}]}; SegRefs -> Self = self(), Fun = fun () -> @@ -191,16 +196,24 @@ schedule_compaction(Type, SnapIdx, LiveIndexes, ok end, - [{bg_work, Fun, - fun (Err) -> - %% send an empty compaction result to ensure the - %% a future compaction can be performed (TODO:) - Self ! {ra_log_event, - {compaction_result, #compaction_result{}}}, - ?WARN("~ts: Major compaction failed with ~p", - [LogId, Err]), ok - end}] - end. + {State#?MODULE{compaction = major}, + [{bg_work, Fun, + fun (Err) -> + %% send an empty compaction result to ensure the + %% a future compaction can be performed (TODO:) + Self ! {ra_log_event, + {compaction_result, #compaction_result{}}}, + ?WARN("~ts: Major compaction failed with ~p", + [LogId, Err]), ok + end}]} + end; +schedule_compaction(Type, SnapIdx, _LiveIndexes, + #?MODULE{cfg = #cfg{log_id = LogId}, + compaction = Comp} = State) -> + ?DEBUG("~ts: ~s compaction requested at ~b but ~s compaction already in progress", + [LogId, Type, SnapIdx, Comp]), + {State, []}. + -spec handle_compaction_result(#compaction_result{}, state()) -> {state(), [ra_server:effect()]}. @@ -208,7 +221,7 @@ handle_compaction_result(#compaction_result{unreferenced = [], linked = [], compacted = []}, State) -> - {State, []}; + {State#?MODULE{compaction = undefined}, []}; handle_compaction_result(#compaction_result{unreferenced = Unreferenced, linked = Linked, compacted = Compacted}, @@ -232,6 +245,7 @@ handle_compaction_result(#compaction_result{unreferenced = Unreferenced, length(Linked) + length(Compacted)), {State#?MODULE{segment_refs = ra_lol:from_list(fun seg_ref_gt/2, SegmentRefs), + compaction = undefined, open_segments = Open}, [{bg_work, Fun, fun (_Err) -> ok end}]}. diff --git a/test/ra_log_segments_SUITE.erl b/test/ra_log_segments_SUITE.erl index efff70e96..4d0be398a 100644 --- a/test/ra_log_segments_SUITE.erl +++ b/test/ra_log_segments_SUITE.erl @@ -27,6 +27,7 @@ all_tests() -> recover2, recover3, recover4, + compactions_are_not_triggered_concurrently, basics, major, major_max_size, @@ -327,6 +328,49 @@ recover4(Config) -> run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), ok. +compactions_are_not_triggered_concurrently(Config) -> + %% creates 3 segments then a snapshot at the first index of the last segment + %% with live indexes only in the first segment. + Dir = ?config(dir, Config), + LiveList = lists:seq(1, 128, 5), + Live = ra_seq:from_list(LiveList), + ct:pal("Live ~p", [Live]), + Scen = + [ + {entries, 1, lists:seq(1, 128 * 3)}, + {assert, 1, lists:seq(1, 128 * 3)}, + {assert, fun (S) -> + SegRefs = ra_log_segments:segment_refs(S), + length(SegRefs) == 3 + end}, + {major, 128 * 2, Live}, + %% this compaction should be dropped + {major, 128 * 3, Live}, + {assert, + fun (_S) -> + receive + {ra_log_event, {compaction_result, _}} = Evt -> + receive + {ra_log_event, {compaction_result, Res}} -> + ct:pal("unexpected second compaction result ~p", [Res]), + false + after 100 -> + self() ! Evt, + true + end + after 1000 -> + flush(), + false + end + end}, + handle_compaction_result, + {major, 128 * 3, Live}, + handle_compaction_result + ], + Segs0 = ra_log_segments_init(Config, Dir, seg_refs(Dir)), + run_scenario([{seg_conf, #{max_count => 128}} | Config], Segs0, Scen), + ok. + basics(Config) -> %% creates 3 segments then a snapshot at the first index of the last segment %% with live indexes only in the first segment. @@ -677,7 +721,7 @@ run_scenario(Config, Segs0, [{entries, Term, IndexesOrEntries} | Rem]) -> ?FUNCTION_NAME(Config, Segs, Rem); run_scenario(Config, Segs0, [{Type, SnapIdx, Live} | Rem]) when Type == major orelse Type == minor -> - Effs = ra_log_segments:schedule_compaction(Type, SnapIdx, Live, Segs0), + {Segs1, Effs} = ra_log_segments:schedule_compaction(Type, SnapIdx, Live, Segs0), Segs = lists:foldl(fun ({bg_work, Fun, _}, S0) -> Fun(), S0; @@ -685,7 +729,7 @@ run_scenario(Config, Segs0, [{Type, SnapIdx, Live} | Rem]) {compaction_result, _Res}} = E}, S0) -> self() ! E, S0 - end, Segs0, Effs), + end, Segs1, Effs), ?FUNCTION_NAME(Config, Segs, Rem); run_scenario(Config, Segs0, [handle_compaction_result = Step | Rem]) -> From 7d6a4974466b0793e1fb8a8e0c30bbc835f26e38 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 7 Jul 2025 10:36:59 +0100 Subject: [PATCH 49/60] mosty dialyzer fixes --- src/ra_kv.erl | 2 ++ src/ra_kv_harness.erl | 22 +++++++++++----------- src/ra_log.erl | 22 +++++++++++----------- 3 files changed, 24 insertions(+), 22 deletions(-) diff --git a/src/ra_kv.erl b/src/ra_kv.erl index 7a2a23d29..981bd15d6 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -134,6 +134,8 @@ get(ServerId, Key, Timeout) -> Err -> Err end; + {ok, Err, _} -> + Err; Err -> Err end. diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl index df6ed2182..19a8af9df 100644 --- a/src/ra_kv_harness.erl +++ b/src/ra_kv_harness.erl @@ -67,14 +67,14 @@ new_state() -> run(NumOperations) -> run(NumOperations, #{}). -read_all_keys_loop(Members) when is_list(Members) -> - Member = lists:nth(rand:uniform(length(Members)), Members), - T1 = erlang:monotonic_time(), - [{ok, _, _} = ra_kv:get(Member, <<"key_", (integer_to_binary(N))/binary>>, 1000) || N <- lists:seq(1, ?MAX_KEY)], - T2 = erlang:monotonic_time(), - Diff = erlang:convert_time_unit(T2 - T1, native, millisecond), - log("~s Read all keys from member ~p in ~bms~n", [timestamp(), Member, Diff]), - read_all_keys_loop(Members). +% read_all_keys_loop(Members) when is_list(Members) -> +% Member = lists:nth(rand:uniform(length(Members)), Members), +% T1 = erlang:monotonic_time(), +% [{ok, _, _} = ra_kv:get(Member, <<"key_", (integer_to_binary(N))/binary>>, 1000) || N <- lists:seq(1, ?MAX_KEY)], +% T2 = erlang:monotonic_time(), +% Diff = erlang:convert_time_unit(T2 - T1, native, millisecond), +% log("~s Read all keys from member ~p in ~bms~n", [timestamp(), Member, Diff]), +% read_all_keys_loop(Members). -spec run(NumOperations :: pos_integer(), Options :: map()) -> @@ -301,10 +301,10 @@ execute_operation(State, {get, Key}) -> RefValue = maps:get(Key, RefMap, not_found), case ra_kv:get(Member, Key, ?TIMEOUT) of - {ok, {error, not_found}, _Value} when RefValue =:= not_found -> + {error, not_found} when RefValue =:= not_found -> State#{operations_count => OpCount + 1, successful_ops => SuccessOps + 1}; - {ok, {error, not_found}} when RefValue =/= not_found -> + {error, not_found} when RefValue =/= not_found -> log("~s CONSISTENCY ERROR: Key ~p should exist but not found~n", [timestamp(), Key]), State#{operations_count => OpCount + 1, failed_ops => FailedOps + 1}; @@ -474,7 +474,7 @@ wait_for_applied_index_convergence(Members, 0) -> -spec get_applied_indices([ra:server_id()]) -> #{ra:server_id() => ra:index() | undefined}. get_applied_indices(Members) -> maps:from_list([{Member, case ra:member_overview(Member, 1000) of - #{log := #{last_applied_index := Index}} -> + {ok, #{log := #{last_applied_index := Index}}, _} -> Index; _ -> undefined diff --git a/src/ra_log.erl b/src/ra_log.erl index 18c36e96b..5fd76c3a7 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -859,9 +859,9 @@ handle_event(major_compaction, #?MODULE{reader = Reader0, snapshot_state = SS} = State) -> case ra_snapshot:current(SS) of {SnapIdx, _} -> - Effs = ra_log_segments:schedule_compaction(major, SnapIdx, - LiveIndexes, Reader0), - {State, Effs}; + {Reader, Effs} = ra_log_segments:schedule_compaction(major, SnapIdx, + LiveIndexes, Reader0), + {State#?MODULE{reader = Reader}, Effs}; _ -> {State, []} end; @@ -929,11 +929,11 @@ handle_event({snapshot_written, {SnapIdx, _} = Snap, LiveIndexes, SnapKind}, live_indexes = LiveIndexes, current_snapshot = Snap, snapshot_state = SnapState}, - CompEffs = ra_log_segments:schedule_compaction(minor, SnapIdx, - LiveIndexes, - State#?MODULE.reader), + {Reader, CompEffs} = ra_log_segments:schedule_compaction(minor, SnapIdx, + LiveIndexes, + State#?MODULE.reader), Effects = CompEffs ++ Effects0, - {State, Effects}; + {State#?MODULE{reader = Reader}, Effects}; checkpoint -> put_counter(Cfg, ?C_RA_SVR_METRIC_CHECKPOINT_INDEX, SnapIdx), %% If we already have the maximum allowed number of checkpoints, @@ -1048,10 +1048,10 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, LiveIndexes, live_indexes = LiveIndexes, mem_table = Mt, last_written_index_term = IdxTerm}, - CompEffs = ra_log_segments:schedule_compaction(minor, SnapIdx, - LiveIndexes, - State#?MODULE.reader), - {ok, State, CompEffs ++ CPEffects}. + {Reader, CompEffs} = ra_log_segments:schedule_compaction(minor, SnapIdx, + LiveIndexes, + State#?MODULE.reader), + {ok, State#?MODULE{reader = Reader}, CompEffs ++ CPEffects}. -spec recover_snapshot(State :: state()) -> From c524cdb1f0ed67d68fb3a6daac558b75893e7fd4 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 7 Jul 2025 11:46:10 +0100 Subject: [PATCH 50/60] add ra_kv:remove_member/3 --- src/ra_kv.erl | 45 ++++++++++++++++++++++++++++++---- src/ra_kv_harness.erl | 56 +++++++++++++++++++++---------------------- test/ra_kv_SUITE.erl | 4 +++- 3 files changed, 71 insertions(+), 34 deletions(-) diff --git a/src/ra_kv.erl b/src/ra_kv.erl index 981bd15d6..44ece90bf 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -18,6 +18,7 @@ -export([ start_cluster/3, add_member/3, + remove_member/3, member_overview/1, put/4, @@ -70,8 +71,13 @@ start_cluster(System, ClusterName, #{members := ServerIds}) end || Id <- ServerIds], ra:start_cluster(System, Configs). -add_member(System, {Name, _} = Id, LeaderId) -> - {ok, Members, _} = ra:members(LeaderId), +-spec add_member(System :: atom(), + NewMemberId :: ra_server_id(), + LeaderId :: ra_server_id()) -> + ok | {error, term()}. +add_member(System, {Name, _} = Id, LeaderId0) -> + {ok, Members, LeaderId1} = ra:members(LeaderId0), + ?assert(not lists:member(Id, Members)), UId = ra:new_uid(ra_lib:to_binary(Name)), Machine = {module, ?MODULE, #{}}, Config = #{id => Id, @@ -81,9 +87,38 @@ add_member(System, {Name, _} = Id, LeaderId) -> min_snapshot_interval => 0}, initial_members => Members, machine => Machine}, - ok = ra:start_server(System, Config), - {ok, _, _} = ra:add_member(LeaderId, Id), - ok. + %% TODO: rollback handling + maybe + %% first start the server + ok ?= ra:start_server(System, Config), + %% then add the member + {ok, {_, _} = IdxTerm, LeaderId} ?= ra:add_member(LeaderId1, Id), + %% then wait for the cluster change command to become applied + {ok, _, _} ?= ra:local_query(LeaderId, {ra_lib, ignore, []}, + #{timeout => 30_000, + condition => {applied, IdxTerm}}), + ok + end. + +-spec remove_member(System :: atom(), + NewMemberId :: ra_server_id(), + LeaderId :: ra_server_id()) -> + ok | {error, term()}. +remove_member(System, Id, LeaderId0) -> + {ok, Members, LeaderId1} = ra:members(LeaderId0), + ?assert(lists:member(Id, Members)), + maybe + ok ?= ra:stop_server(System, Id), + %% first remove the mem + {ok, {_, _} = IdxTerm, LeaderId} ?= ra:remove_member(LeaderId1, Id), + %% first start the server + %% then wait for the cluster change command to become applied + {ok, _, _} ?= ra:local_query(LeaderId, {ra_lib, ignore, []}, + #{timeout => 30_000, + condition => {applied, IdxTerm}}), + ok ?= ra:force_delete_server(System, Id), + ok + end. member_overview(ServerId) -> case ra:member_overview(ServerId) of diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl index 19a8af9df..a5efa7624 100644 --- a/src/ra_kv_harness.erl +++ b/src/ra_kv_harness.erl @@ -171,7 +171,7 @@ start_peer_nodes(N, PeerRefs, NodeNames) when N > 0 -> start_single_peer_node(NodeId) -> NodeName = list_to_atom("ra_test_" ++ integer_to_list(NodeId) ++ "@" ++ inet_db:gethostname()), - + % Get all code paths from current node CodePaths = code:get_path(), PaArgs = lists:flatmap(fun(Path) -> ["-pa", Path] end, CodePaths), @@ -217,16 +217,16 @@ run_operations(State, _ClusterName) -> end, Operation = generate_operation(), NewState = execute_operation(State, Operation), - + % Update remaining operations count UpdatedState = NewState#{remaining_ops => RemainingOps - 1}, - + % Validate consistency every 100 operations ValidationState = case maps:get(operations_count, UpdatedState) rem 100 of 0 -> validate_consistency(UpdatedState); _ -> UpdatedState end, - + run_operations(ValidationState, _ClusterName) end. @@ -368,22 +368,22 @@ execute_operation(State, {add_member}) -> SuccessOps = maps:get(successful_ops, State), FailedOps = maps:get(failed_ops, State), NextNodeId = maps:get(next_node_id, State), - + % Don't add members if we already have 7 (maximum 7 nodes) case maps:size(Members) >= 7 of true -> State#{operations_count => OpCount + 1, failed_ops => FailedOps + 1}; false -> - + case start_new_peer_node(NextNodeId) of {ok, PeerRef, NodeName} -> NewMember = {?CLUSTER_NAME, NodeName}, - + % Pick a random existing member to send the add_member command to MembersList = maps:keys(Members), ExistingMember = lists:nth(rand:uniform(length(MembersList)), MembersList), - + try ra_kv:add_member(?SYS, NewMember, ExistingMember) of ok -> NewMembers = Members#{NewMember => PeerRef}, @@ -426,13 +426,13 @@ execute_operation(State, {remove_member}) -> MembersList = maps:keys(Members), MemberToRemove = lists:nth(rand:uniform(length(MembersList)), MembersList), log("~s Removing member ~p...", [timestamp(), MemberToRemove]), - + % Pick a different member to send the remove command to RemainingMembers = MembersList -- [MemberToRemove], CommandTarget = lists:nth(rand:uniform(length(RemainingMembers)), RemainingMembers), - - case ra:remove_member(CommandTarget, MemberToRemove, ?TIMEOUT) of - {ok, _Meta, _} -> + + case ra_kv:remove_member(?SYS, CommandTarget, MemberToRemove) of + ok -> % Stop the peer node for the removed member case maps:get(MemberToRemove, Members, undefined) of undefined -> @@ -440,11 +440,11 @@ execute_operation(State, {remove_member}) -> PeerRef -> catch peer:stop(PeerRef) end, - + NewMembers = maps:remove(MemberToRemove, Members), NewMembersList = maps:keys(NewMembers), log("~s done. Cluster now has ~p members: ~0p~n", [timestamp(), length(NewMembersList), NewMembersList]), - + State#{members => NewMembers, operations_count => OpCount + 1, successful_ops => SuccessOps + 1}; @@ -488,7 +488,7 @@ validate_consistency(State) -> % Wait for all nodes to converge to the same applied index MembersList = maps:keys(Members), wait_for_applied_index_convergence(MembersList, 300), % Wait up to 30 seconds - + % Check that all members have the same view ValidationResults = [validate_member_consistency(Member, RefMap) || Member <- MembersList], @@ -504,11 +504,11 @@ validate_consistency(State) -> end} || {Member, Result} <- lists:zip(MembersList, ValidationResults)], log("~s Consistency check failed. Live indexes per node: ~p~n", [timestamp(), LiveIndexesSummary]), log("~s STOPPING: No more operations will be performed due to consistency failure~n", [timestamp()]), - + % Write full details to log file with difference analysis LogEntry = format_consistency_failure(MembersList, ValidationResults), file:write_file("ra_kv_harness.log", LogEntry, [append]), - + FailedOps = maps:get(failed_ops, State), State#{failed_ops => FailedOps + 1, remaining_ops => 0, consistency_failed => true} end. @@ -516,15 +516,15 @@ validate_consistency(State) -> -spec format_consistency_failure([ra:server_id()], [map() | error]) -> iolist(). format_consistency_failure(Members, Results) -> MemberResults = lists:zip(Members, Results), - + % Extract all unique results for comparison UniqueResults = lists:usort([R || {_, R} <- MemberResults, R =/= error]), - + Header = io_lib:format("~s Consistency check failed:~n", [timestamp()]), - + % Log raw data RawData = [io_lib:format(" Member ~p: ~p~n", [Member, Result]) || {Member, Result} <- MemberResults], - + % Analyze differences DiffAnalysis = case UniqueResults of [] -> @@ -532,13 +532,13 @@ format_consistency_failure(Members, Results) -> [_SingleResult] -> [" ANALYSIS: All successful members have identical results (errors may exist)\n"]; MultipleResults -> - [" ANALYSIS: Found ~p different result patterns:\n" | - [io_lib:format(" Pattern ~p: ~p\n", [I, Pattern]) || + [" ANALYSIS: Found ~p different result patterns:\n" | + [io_lib:format(" Pattern ~p: ~p\n", [I, Pattern]) || {I, Pattern} <- lists:zip(lists:seq(1, length(MultipleResults)), MultipleResults)] ++ [" DIFFERENCES:\n"] ++ analyze_field_differences(MultipleResults)] end, - + [Header, RawData, DiffAnalysis, "\n"]. -spec analyze_field_differences([map()]) -> iolist(). @@ -546,17 +546,17 @@ analyze_field_differences(Results) -> % Extract live_indexes and num_keys for comparison LiveIndexes = [maps:get(live_indexes, R, undefined) || R <- Results, is_map(R)], NumKeys = [maps:get(num_keys, R, undefined) || R <- Results, is_map(R)], - + LiveIndexDiff = case lists:usort(LiveIndexes) of [_] -> []; MultipleLI -> [io_lib:format(" live_indexes differ: ~p\n", [MultipleLI])] end, - + NumKeysDiff = case lists:usort(NumKeys) of [_] -> []; MultipleNK -> [io_lib:format(" num_keys differ: ~p\n", [MultipleNK])] end, - + [LiveIndexDiff, NumKeysDiff]. -spec validate_member_consistency(ra:server_id(), map()) -> map() | error. @@ -578,7 +578,7 @@ validate_final_consistency(State) -> log("~s Performing final consistency validation...~n", [timestamp()]), log("~s Reference map has ~p keys~n", [timestamp(), maps:size(RefMap)]), - + % Wait for all nodes to converge before final validation MembersList = maps:keys(Members), log("~s Waiting for applied index convergence...~n", [timestamp()]), diff --git a/test/ra_kv_SUITE.erl b/test/ra_kv_SUITE.erl index 4c1744e3f..2e18778e1 100644 --- a/test/ra_kv_SUITE.erl +++ b/test/ra_kv_SUITE.erl @@ -257,5 +257,7 @@ basics(_Config) -> undefined, 1000), ?assertEqual(Reads4, Reads5), ct:pal("counters ~p", [ra_counters:overview(KvId)]), - ra:delete_cluster([KvId, KvId2]), + ok = ra_kv:remove_member(?SYS, KvId2, KvId), + ct:pal("members ~p", [ra:members(KvId)]), + ra:delete_cluster([KvId]), ok. From fe1cd9545e75da755b7bcb1d5c9c7e27dc612d87 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 7 Jul 2025 12:21:22 +0100 Subject: [PATCH 51/60] harness improvements --- src/ra_kv_harness.erl | 91 +++++++++++++++++++++++-------------------- 1 file changed, 49 insertions(+), 42 deletions(-) diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl index a5efa7624..c62e1417e 100644 --- a/src/ra_kv_harness.erl +++ b/src/ra_kv_harness.erl @@ -3,7 +3,6 @@ -export([ run/1, run/2, - setup_cluster/1, teardown_cluster/1, timestamp/0, log/2 @@ -18,7 +17,8 @@ -define(MIN_VALUE_SIZE, 1). -define(MAX_VALUE_SIZE, 10_000_000). % 10MB --type state() :: #{members := #{ra:server_id() => peer:server_ref()}, +-type state() :: #{options => map(), + members := #{ra:server_id() => peer:server_ref()}, reference_map := #{binary() => term()}, operations_count := non_neg_integer(), successful_ops := non_neg_integer(), @@ -55,6 +55,7 @@ new_state() -> -type operation() :: {put, Key :: binary(), Value :: term()} | {get, Key :: binary()} | {snapshot} | + {major_compaction} | {update_almost_all_keys} | {add_member} | {remove_member}. @@ -82,17 +83,20 @@ run(NumOperations) -> failed := non_neg_integer(), consistency_checks := non_neg_integer()}} | {error, term()}. -run(NumOperations, _Options) when NumOperations > 0 -> +run(NumOperations, Options) when NumOperations > 0 -> % Start with a random number of nodes between 1 and 7 NumNodes = rand:uniform(7), logger:set_primary_config(level, warning), application:set_env(sasl, sasl_error_logger, false), application:stop(sasl), log("~s Starting cluster with ~p nodes~n", [timestamp(), NumNodes]), - case setup_cluster(NumNodes) of + case setup_cluster(NumNodes, Options) of {ok, Members, PeerNodes} -> MembersMap = maps:from_list(lists:zip(Members, PeerNodes)), - InitialState = (new_state())#{members => MembersMap, next_node_id => NumNodes + 1, remaining_ops => NumOperations}, + InitialState = (new_state())#{members => MembersMap, + next_node_id => NumNodes + 1, + remaining_ops => NumOperations, + options => Options}, try State = execute_operation(InitialState, {put, <<"never_updated">>, <<"never_updated">>}), %% keep reading all keys while the other operations are running @@ -118,26 +122,11 @@ run(NumOperations, _Options) when NumOperations > 0 -> {error, Reason} end. --spec setup_cluster(NumNodes :: pos_integer()) -> - {ok, [ra:server_id()], [peer:server_ref()]} | {error, term()}. -setup_cluster(NumNodes) when NumNodes > 0 -> +setup_cluster(NumNodes, Opts) when NumNodes > 0 -> % Start peer nodes - case start_peer_nodes(NumNodes) of + case start_peer_nodes(NumNodes, Opts) of {ok, PeerNodes, NodeNames} -> Members = [{?CLUSTER_NAME, NodeName} || NodeName <- NodeNames], - - % Start ra application on all peer nodes - [ - begin - % Set logger level to reduce verbosity on peer node - erpc:call(NodeName, logger, set_primary_config, [level, warning]), - erpc:call(NodeName, application, set_env, [sasl, sasl_error_logger, false]), - erpc:call(NodeName, application, stop, [sasl]), - {ok, _} = erpc:call(NodeName, ra, start_in, [NodeName]) - % {ok, _} = erpc:call(NodeName, ra_system, start, [#{name => default, data_dir => atom_to_list(NodeName), names => ra_system:derive_names(default)}]) - end - || NodeName <- NodeNames], - case ra_kv:start_cluster(?SYS, ?CLUSTER_NAME, #{members => Members}) of {ok, StartedMembers, _} -> log("~s Started cluster with ~p members~n", [timestamp(), length(StartedMembers)]), @@ -150,25 +139,22 @@ setup_cluster(NumNodes) when NumNodes > 0 -> {error, Reason} end. --spec start_peer_nodes(pos_integer()) -> - {ok, [peer:server_ref()], [node()]} | {error, term()}. -start_peer_nodes(NumNodes) -> - start_peer_nodes(NumNodes, [], []). +start_peer_nodes(NumNodes, Opts) -> + start_peer_nodes(NumNodes, [], [], Opts). -start_peer_nodes(0, PeerRefs, NodeNames) -> +start_peer_nodes(0, PeerRefs, NodeNames, _Opts) -> {ok, lists:reverse(PeerRefs), lists:reverse(NodeNames)}; -start_peer_nodes(N, PeerRefs, NodeNames) when N > 0 -> - case start_single_peer_node(N) of +start_peer_nodes(N, PeerRefs, NodeNames, Opts) when N > 0 -> + case start_single_peer_node(N, Opts) of {ok, PeerRef, NodeName} -> - start_peer_nodes(N - 1, [PeerRef | PeerRefs], [NodeName | NodeNames]); + start_peer_nodes(N - 1, [PeerRef | PeerRefs], [NodeName | NodeNames], Opts); {error, Reason} -> % Clean up any already started peers [peer:stop(PeerRef) || PeerRef <- PeerRefs], {error, Reason} end. --spec start_single_peer_node(pos_integer()) -> {ok, peer:server_ref(), node()} | {error, term()}. -start_single_peer_node(NodeId) -> +start_single_peer_node(NodeId, Opts) -> NodeName = list_to_atom("ra_test_" ++ integer_to_list(NodeId) ++ "@" ++ inet_db:gethostname()), @@ -180,18 +166,21 @@ start_single_peer_node(NodeId) -> case peer:start_link(#{name => NodeName, args => PaArgs ++ BufferSize}) of {ok, PeerRef, NodeName} -> - % Set logger level to reduce verbosity on peer node + BaseDir = maps:get(dir, Opts, ""), erpc:call(NodeName, logger, set_primary_config, [level, warning]), + erpc:call(NodeName, application, set_env, [sasl, sasl_error_logger, false]), + erpc:call(NodeName, application, stop, [sasl]), + Dir = filename:join(BaseDir, NodeName), + {ok, _} = erpc:call(NodeName, ra, start_in, [Dir]), + % Set logger level to reduce verbosity on peer node % Start ra application on the new peer node - {ok, _} = erpc:call(NodeName, ra, start_in, [NodeName]), {ok, PeerRef, NodeName}; {error, Reason} -> {error, Reason} end. --spec start_new_peer_node(pos_integer()) -> {ok, peer:server_ref(), node()} | {error, term()}. -start_new_peer_node(NodeId) -> - start_single_peer_node(NodeId). +start_new_peer_node(NodeId, Opts) -> + start_single_peer_node(NodeId, Opts). -spec teardown_cluster(state()) -> ok. teardown_cluster(#{members := Members}) -> @@ -239,8 +228,10 @@ generate_operation() -> {add_member}; 3 -> % 1% remove member {remove_member}; - N when N =< 7 -> % 1% snapshot + N when N =< 7 -> % 4% snapshot {snapshot}; + N when N =< 9 -> % 2% major compactions + {major_compaction}; N when N =< 80 -> Key = generate_key(), Value = generate_value(), @@ -355,14 +346,30 @@ execute_operation(State, {snapshot}) -> undefined -> State; _Pid -> - log("~s Rollover/snapshot/compaction on node ~p...~n", [timestamp(), NodeName]), + log("~s Rollover/snapshot on node ~p...~n", [timestamp(), NodeName]), erpc:call(NodeName, ra_log_wal, force_roll_over, [ra_log_wal]), erpc:call(NodeName, ra, aux_command, [Member, take_snapshot]), + State + end; + +execute_operation(State, {major_compaction}) -> + Members = maps:get(members, State), + + % Pick a random cluster member to send snapshot command to + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + + case erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]) of + undefined -> + State; + _Pid -> + log("~s Triggering major compaction on node ~p...~n", [timestamp(), NodeName]), erpc:call(NodeName, ra, trigger_compaction, [Member]), State end; -execute_operation(State, {add_member}) -> +execute_operation(#{options := Opts} = State, {add_member}) -> Members = maps:get(members, State), OpCount = maps:get(operations_count, State), SuccessOps = maps:get(successful_ops, State), @@ -376,7 +383,7 @@ execute_operation(State, {add_member}) -> failed_ops => FailedOps + 1}; false -> - case start_new_peer_node(NextNodeId) of + case start_new_peer_node(NextNodeId, Opts) of {ok, PeerRef, NodeName} -> NewMember = {?CLUSTER_NAME, NodeName}, @@ -431,7 +438,7 @@ execute_operation(State, {remove_member}) -> RemainingMembers = MembersList -- [MemberToRemove], CommandTarget = lists:nth(rand:uniform(length(RemainingMembers)), RemainingMembers), - case ra_kv:remove_member(?SYS, CommandTarget, MemberToRemove) of + case ra_kv:remove_member(?SYS, MemberToRemove, CommandTarget) of ok -> % Stop the peer node for the removed member case maps:get(MemberToRemove, Members, undefined) of From 7ad8fe75a2c853e62b94a5184c82d700c08be631 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 7 Jul 2025 16:15:50 +0100 Subject: [PATCH 52/60] fixes --- src/ra_kv.erl | 8 +++---- src/ra_kv_harness.erl | 55 ++++++++++++++++++++++++++++++------------- 2 files changed, 42 insertions(+), 21 deletions(-) diff --git a/src/ra_kv.erl b/src/ra_kv.erl index 44ece90bf..3892bb23f 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -105,17 +105,18 @@ add_member(System, {Name, _} = Id, LeaderId0) -> LeaderId :: ra_server_id()) -> ok | {error, term()}. remove_member(System, Id, LeaderId0) -> - {ok, Members, LeaderId1} = ra:members(LeaderId0), + {ok, Members, _} = ra:members(LeaderId0), + RemainingMembers = lists:delete(Id, Members), ?assert(lists:member(Id, Members)), maybe - ok ?= ra:stop_server(System, Id), %% first remove the mem - {ok, {_, _} = IdxTerm, LeaderId} ?= ra:remove_member(LeaderId1, Id), + {ok, {_, _} = IdxTerm, LeaderId} ?= ra:remove_member(RemainingMembers, Id), %% first start the server %% then wait for the cluster change command to become applied {ok, _, _} ?= ra:local_query(LeaderId, {ra_lib, ignore, []}, #{timeout => 30_000, condition => {applied, IdxTerm}}), + % ok ?= ra:stop_server(System, Id), ok ?= ra:force_delete_server(System, Id), ok end. @@ -128,7 +129,6 @@ member_overview(ServerId) -> Err end. - %% client -spec put(ra:server_id(), key(), value(), non_neg_integer()) -> {ok, map()} | {error, term()} | {timeout, ra:server_id()}. diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl index c62e1417e..72f14f906 100644 --- a/src/ra_kv_harness.erl +++ b/src/ra_kv_harness.erl @@ -383,13 +383,16 @@ execute_operation(#{options := Opts} = State, {add_member}) -> failed_ops => FailedOps + 1}; false -> + log("~s Adding member on node ~p.~n", + [timestamp(), NextNodeId]), case start_new_peer_node(NextNodeId, Opts) of {ok, PeerRef, NodeName} -> NewMember = {?CLUSTER_NAME, NodeName}, % Pick a random existing member to send the add_member command to MembersList = maps:keys(Members), - ExistingMember = lists:nth(rand:uniform(length(MembersList)), MembersList), + ExistingMember = lists:nth(rand:uniform(length(MembersList)), + MembersList), try ra_kv:add_member(?SYS, NewMember, ExistingMember) of ok -> @@ -402,7 +405,8 @@ execute_operation(#{options := Opts} = State, {add_member}) -> next_node_id => NextNodeId + 1} catch _:Reason -> - log("~s Failed to add member ~p: ~p~n", [timestamp(), NewMember, Reason]), + log("~s Failed to add member ~p: ~p~n", + [timestamp(), NewMember, Reason]), % Clean up the peer node since add failed catch peer:stop(PeerRef), State#{operations_count => OpCount + 1, @@ -431,12 +435,15 @@ execute_operation(State, {remove_member}) -> false -> % Pick a random member to remove MembersList = maps:keys(Members), - MemberToRemove = lists:nth(rand:uniform(length(MembersList)), MembersList), - log("~s Removing member ~p...", [timestamp(), MemberToRemove]), + MemberToRemove = lists:nth(rand:uniform(length(MembersList)), + MembersList), % Pick a different member to send the remove command to RemainingMembers = MembersList -- [MemberToRemove], - CommandTarget = lists:nth(rand:uniform(length(RemainingMembers)), RemainingMembers), + CommandTarget = lists:nth(rand:uniform(length(RemainingMembers)), + RemainingMembers), + log("~s Removing member ~w... command target ~w~n", + [timestamp(), MemberToRemove, CommandTarget]), case ra_kv:remove_member(?SYS, MemberToRemove, CommandTarget) of ok -> @@ -456,7 +463,8 @@ execute_operation(State, {remove_member}) -> operations_count => OpCount + 1, successful_ops => SuccessOps + 1}; {error, Reason} -> - log("~s Failed to remove member ~p: ~p~n", [timestamp(), MemberToRemove, Reason]), + log("~s Failed to remove member ~p: ~p~n", + [timestamp(), MemberToRemove, Reason]), State#{operations_count => OpCount + 1, failed_ops => FailedOps + 1} end @@ -475,13 +483,14 @@ wait_for_applied_index_convergence(Members, MaxRetries) when MaxRetries > 0 -> end; wait_for_applied_index_convergence(Members, 0) -> IndicesMap = get_applied_indices(Members), - log("~s WARNING: Applied index convergence timeout. Reported values: ~0p~n", [timestamp(), IndicesMap]), + log("~s WARNING: Applied index convergence timeout. Reported values: ~0p~n", + [timestamp(), IndicesMap]), ok. -spec get_applied_indices([ra:server_id()]) -> #{ra:server_id() => ra:index() | undefined}. get_applied_indices(Members) -> maps:from_list([{Member, case ra:member_overview(Member, 1000) of - {ok, #{log := #{last_applied_index := Index}}, _} -> + {ok, #{last_applied := Index}, _} -> Index; _ -> undefined @@ -500,16 +509,25 @@ validate_consistency(State) -> ValidationResults = [validate_member_consistency(Member, RefMap) || Member <- MembersList], Result1 = hd(ValidationResults), - case lists:all(fun(Result) -> Result =:= Result1 end, ValidationResults) of + case lists:all(fun(Result) -> + is_map(Result) andalso + is_map(Result1) andalso + lists:sort(maps:get(live_indexes, Result)) =:= + lists:sort(maps:get(live_indexes, Result1)) + end, ValidationResults) of true -> State; false -> % Brief console output with live_indexes summary LiveIndexesSummary = [{Member, case Result of - #{live_indexes := LI} -> length(LI); + #{live_indexes := LI, + log := #{last_index := LastIndex}} -> + {length(LI), LastIndex}; _ -> error - end} || {Member, Result} <- lists:zip(MembersList, ValidationResults)], - log("~s Consistency check failed. Live indexes per node: ~p~n", [timestamp(), LiveIndexesSummary]), + end} || {Member, Result} <- + lists:zip(MembersList, ValidationResults)], + log("~s Consistency check failed. Live indexes per node: ~p~n", + [timestamp(), LiveIndexesSummary ]), log("~s STOPPING: No more operations will be performed due to consistency failure~n", [timestamp()]), % Write full details to log file with difference analysis @@ -568,13 +586,16 @@ analyze_field_differences(Results) -> -spec validate_member_consistency(ra:server_id(), map()) -> map() | error. validate_member_consistency(Member, _RefMap) -> - NodeName = element(2, Member), - case erpc:call(NodeName, ra_kv, member_overview, [Member]) of - #{machine := #{live_indexes := Live, num_keys := Num}} -> + case ra_kv:member_overview(Member) of + #{log := Log, + machine := #{live_indexes := Live, num_keys := Num}} -> %io:format("Member ~p overview: Live indexes ~p, Num keys ~p", [Member, Live, Num]), - #{live_indexes => Live, num_keys => Num}; + #{log => Log, + live_indexes => Live, + num_keys => Num}; Error -> - log("~s Member ~p failed overview check: ~p~n", [timestamp(), Member, Error]), + log("~s Member ~p failed overview check: ~p~n", + [timestamp(), Member, Error]), error end. From ffc209101a14f961f2f1aa0f238219bbefa6b2e5 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Mon, 7 Jul 2025 16:42:19 +0100 Subject: [PATCH 53/60] tweaks --- src/ra_kv_harness.erl | 2 +- src/ra_log.erl | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl index 72f14f906..7055a261e 100644 --- a/src/ra_kv_harness.erl +++ b/src/ra_kv_harness.erl @@ -200,7 +200,7 @@ run_operations(State, _ClusterName) -> true -> State; false -> - case RemainingOps rem 100000 of + case RemainingOps rem 1000 of 0 -> log("~s ~p operations remaining~n", [timestamp(), RemainingOps]); _ -> ok end, diff --git a/src/ra_log.erl b/src/ra_log.erl index 5fd76c3a7..ce983ca26 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -1039,8 +1039,13 @@ install_snapshot({SnapIdx, SnapTerm} = IdxTerm, MacMod, LiveIndexes, end, %% TODO: more mt entries could potentially be cleared up in the %% mem table here - {Spec, Mt} = ra_mt:set_first(SmallestLiveIndex, Mt0), + {Spec, Mt1} = ra_mt:set_first(SmallestLiveIndex, Mt0), ok = exec_mem_table_delete(Names, UId, Spec), + %% always create a new mem table here as we could have written + %% sparese entries in the snapshot install + %% TODO: check an empty mt doesn't leak + {ok, Mt} = ra_log_ets:new_mem_table_please(Cfg#cfg.names, + Cfg#cfg.uid, Mt1), State = State0#?MODULE{snapshot_state = SnapState, current_snapshot = IdxTerm, range = undefined, From c2ff8fb4b897567fd44d608c94dd2ba7315ad16c Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 8 Jul 2025 10:54:32 +0100 Subject: [PATCH 54/60] Avoid complete segments re-init when releasing resources. --- src/ra_log.erl | 17 +++-------------- src/ra_log_segments.erl | 19 ++++++++++++++++++- test/ra_log_2_SUITE.erl | 5 +++++ 3 files changed, 26 insertions(+), 15 deletions(-) diff --git a/src/ra_log.erl b/src/ra_log.erl index ce983ca26..55311ed31 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -1340,21 +1340,10 @@ delete_everything(#?MODULE{cfg = #cfg{uid = UId, -spec release_resources(non_neg_integer(), sequential | random, state()) -> state(). release_resources(MaxOpenSegments, AccessPattern, - #?MODULE{cfg = #cfg{uid = UId, - log_id = LogId, - directory = Dir, - counter = Counter}, + #?MODULE{cfg = #cfg{}, reader = Reader} = State) -> - ActiveSegs = ra_log_segments:segment_refs(Reader), - CompConf = ra_log_segments:compaction_conf(Reader), - % close all open segments - % deliberately ignoring return value - _ = ra_log_segments:close(Reader), - %% open a new segment with the new max open segment value - State#?MODULE{reader = ra_log_segments:init(UId, Dir, MaxOpenSegments, - AccessPattern, ActiveSegs, - Counter, CompConf, LogId)}. - + State#?MODULE{reader = ra_log_segments:update_conf(MaxOpenSegments, + AccessPattern, Reader)}. %%% Local functions diff --git a/src/ra_log_segments.erl b/src/ra_log_segments.erl index e5b3de149..f5296d51f 100644 --- a/src/ra_log_segments.erl +++ b/src/ra_log_segments.erl @@ -12,6 +12,7 @@ -include_lib("kernel/include/file.hrl"). -export([ init/8, + update_conf/3, close/1, update_segments/2, schedule_compaction/4, @@ -120,6 +121,21 @@ close(#?STATE{open_segments = Open}) -> _ = ra_flru:evict_all(Open), ok. + +-spec update_conf(non_neg_integer(), sequential | random, state()) -> + state(). +update_conf(MaxOpen, AccessPattern, + #?STATE{cfg = Cfg, + open_segments = Open} = State) -> + FlruHandler = fun ({_, Seg}) -> + _ = ra_log_segment:close(Seg), + decr_counter(Cfg, ?C_RA_LOG_OPEN_SEGMENTS, 1) + end, + _ = ra_flru:evict_all(Open), + State#?STATE{cfg = Cfg#cfg{access_pattern = AccessPattern}, + open_segments = ra_flru:new(MaxOpen, FlruHandler)}. + + -spec update_segments([segment_ref()], state()) -> {state(), OverwrittenSegments :: [segment_ref()]}. update_segments(NewSegmentRefs, #?STATE{open_segments = Open0, @@ -644,10 +660,11 @@ major_compaction(#{dir := Dir} = CompConf, SegRefs, LiveIndexes) -> CompConf), Compacted0 = [begin + AllFns = [F || {_, _, {F, _}} <- All], %% create a compaction marker with the compaction group i CompactionMarker = filename:join(Dir, with_ext(CompGroupLeaderFn, ".compaction_group")), - ok = ra_lib:write_file(CompactionMarker, term_to_binary(All)), + ok = ra_lib:write_file(CompactionMarker, term_to_binary(AllFns)), %% create a new segment with .compacting extension CompactingFn = filename:join(Dir, with_ext(CompGroupLeaderFn, ".compacting")), diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 1a5c7ca84..928adce37 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -830,6 +830,11 @@ write_sparse_re_init(Config) -> ok. +write_sparse_after_snapshot_install(Config) -> + + + ok. + write_sparse_after_index_reset(Config) -> Log0 = ra_log_init(Config), Log1 = write_n(1, 6, 1, Log0), From 4819290a3692f03b1229a9b83dd0be617cba460c Mon Sep 17 00:00:00 2001 From: Michal Kuratczyk Date: Tue, 8 Jul 2025 14:52:41 +0200 Subject: [PATCH 55/60] ra_kv_harness: add kill_wal and kill_member ops --- src/ra_kv_harness.erl | 46 ++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 43 insertions(+), 3 deletions(-) diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl index 7055a261e..eca1e2701 100644 --- a/src/ra_kv_harness.erl +++ b/src/ra_kv_harness.erl @@ -57,6 +57,8 @@ new_state() -> {snapshot} | {major_compaction} | {update_almost_all_keys} | + {kill_wal} | + {kill_member} | {add_member} | {remove_member}. @@ -228,9 +230,13 @@ generate_operation() -> {add_member}; 3 -> % 1% remove member {remove_member}; - N when N =< 7 -> % 4% snapshot + 4 -> % 1% kill WAL + {kill_wal}; + 5 -> % 1% kill member + {kill_member}; + N when N =< 9 -> % 4% snapshot {snapshot}; - N when N =< 9 -> % 2% major compactions + N when N =< 11 -> % 2% major compactions {major_compaction}; N when N =< 80 -> Key = generate_key(), @@ -468,7 +474,41 @@ execute_operation(State, {remove_member}) -> State#{operations_count => OpCount + 1, failed_ops => FailedOps + 1} end - end. + end; + +execute_operation(State, {kill_wal}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + + % Pick a node to kill WAL on + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + + log("~s Killing WAL on member ~w...~n", [timestamp(), NodeName]), + + Pid = erpc:call(NodeName, erlang, whereis, [ra_log_wal]), + erpc:call(NodeName, erlang, exit, [Pid, kill]), + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + +execute_operation(State, {kill_member}) -> + Members = maps:get(members, State), + OpCount = maps:get(operations_count, State), + SuccessOps = maps:get(successful_ops, State), + + % Pick a random member to kill + MembersList = maps:keys(Members), + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), + + log("~s Killing member ~w...~n", [timestamp(), Member]), + + Pid = erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]), + erpc:call(NodeName, erlang, exit, [Pid, kill]), + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}. -spec wait_for_applied_index_convergence([ra:server_id()], non_neg_integer()) -> ok. wait_for_applied_index_convergence(Members, MaxRetries) when MaxRetries > 0 -> From 14a6f0a9f497d71968f4ae8d999843c1fb8017ae Mon Sep 17 00:00:00 2001 From: Michal Kuratczyk Date: Tue, 8 Jul 2025 16:09:16 +0200 Subject: [PATCH 56/60] Harness tweaks 1. kill_wal/kill_member picks a random value between 1 and MAX_NODES, not the current number of members. These prevents very frequent kills if there's a small number of nodes 2. restore read_all_keys_loop and stop it when the tests stop --- src/ra_kv_harness.erl | 73 ++++++++++++++++++++++++++++--------------- 1 file changed, 48 insertions(+), 25 deletions(-) diff --git a/src/ra_kv_harness.erl b/src/ra_kv_harness.erl index eca1e2701..ae6412b5d 100644 --- a/src/ra_kv_harness.erl +++ b/src/ra_kv_harness.erl @@ -16,6 +16,7 @@ -define(MAX_KEY, 10000). % Limited key space for more conflicts -define(MIN_VALUE_SIZE, 1). -define(MAX_VALUE_SIZE, 10_000_000). % 10MB +-define(MAX_NODES, 7). % Maximum number of nodes in the cluster -type state() :: #{options => map(), members := #{ra:server_id() => peer:server_ref()}, @@ -70,14 +71,20 @@ new_state() -> run(NumOperations) -> run(NumOperations, #{}). -% read_all_keys_loop(Members) when is_list(Members) -> -% Member = lists:nth(rand:uniform(length(Members)), Members), -% T1 = erlang:monotonic_time(), -% [{ok, _, _} = ra_kv:get(Member, <<"key_", (integer_to_binary(N))/binary>>, 1000) || N <- lists:seq(1, ?MAX_KEY)], -% T2 = erlang:monotonic_time(), -% Diff = erlang:convert_time_unit(T2 - T1, native, millisecond), -% log("~s Read all keys from member ~p in ~bms~n", [timestamp(), Member, Diff]), -% read_all_keys_loop(Members). +read_all_keys_loop(Members) when is_list(Members) -> + receive + stop -> + log("~s Read all keys loop stopped~n", [timestamp()]), + ok + after 0 -> + Member = lists:nth(rand:uniform(length(Members)), Members), + T1 = erlang:monotonic_time(), + [_ = ra_kv:get(Member, <<"key_", (integer_to_binary(N))/binary>>, 1000) || N <- lists:seq(1, ?MAX_KEY)], + T2 = erlang:monotonic_time(), + Diff = erlang:convert_time_unit(T2 - T1, native, millisecond), + log("~s Read all keys from member ~p in ~bms~n", [timestamp(), Member, Diff]), + read_all_keys_loop(Members) + end. -spec run(NumOperations :: pos_integer(), Options :: map()) -> @@ -99,11 +106,12 @@ run(NumOperations, Options) when NumOperations > 0 -> next_node_id => NumNodes + 1, remaining_ops => NumOperations, options => Options}, + %% keep reading all keys while the other operations are running + ReaderPid = spawn(fun() -> read_all_keys_loop(maps:keys(MembersMap)) end), try State = execute_operation(InitialState, {put, <<"never_updated">>, <<"never_updated">>}), - %% keep reading all keys while the other operations are running - %spawn(fun() -> read_all_keys_loop(maps:keys(MembersMap)) end), FinalState = run_operations(State, ?CLUSTER_NAME), + ReaderPid ! stop, case maps:get(consistency_failed, FinalState, false) of true -> log("~s EMERGENCY STOP: Leaving cluster running for investigation~n", [timestamp()]), @@ -117,6 +125,7 @@ run(NumOperations, Options) when NumOperations > 0 -> end catch Class:Reason:Stack -> + ReaderPid ! stop, teardown_cluster(InitialState), {error, {Class, Reason, Stack}} end; @@ -383,7 +392,7 @@ execute_operation(#{options := Opts} = State, {add_member}) -> NextNodeId = maps:get(next_node_id, State), % Don't add members if we already have 7 (maximum 7 nodes) - case maps:size(Members) >= 7 of + case maps:size(Members) >= ?MAX_NODES of true -> State#{operations_count => OpCount + 1, failed_ops => FailedOps + 1}; @@ -483,15 +492,22 @@ execute_operation(State, {kill_wal}) -> % Pick a node to kill WAL on MembersList = maps:keys(Members), - Member = lists:nth(rand:uniform(length(MembersList)), MembersList), - NodeName = element(2, Member), + Rnd = rand:uniform(?MAX_NODES), + case Rnd > length(MembersList) of + true -> + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + false -> + Member = lists:nth(Rnd, MembersList), + NodeName = element(2, Member), - log("~s Killing WAL on member ~w...~n", [timestamp(), NodeName]), + log("~s Killing WAL on member ~w...~n", [timestamp(), NodeName]), - Pid = erpc:call(NodeName, erlang, whereis, [ra_log_wal]), - erpc:call(NodeName, erlang, exit, [Pid, kill]), - State#{operations_count => OpCount + 1, - successful_ops => SuccessOps + 1}; + Pid = erpc:call(NodeName, erlang, whereis, [ra_log_wal]), + erpc:call(NodeName, erlang, exit, [Pid, kill]), + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1} + end; execute_operation(State, {kill_member}) -> Members = maps:get(members, State), @@ -500,15 +516,22 @@ execute_operation(State, {kill_member}) -> % Pick a random member to kill MembersList = maps:keys(Members), - Member = lists:nth(rand:uniform(length(MembersList)), MembersList), - NodeName = element(2, Member), + Rnd = rand:uniform(?MAX_NODES), + case Rnd > length(MembersList) of + true -> + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1}; + false -> + Member = lists:nth(rand:uniform(length(MembersList)), MembersList), + NodeName = element(2, Member), - log("~s Killing member ~w...~n", [timestamp(), Member]), + log("~s Killing member ~w...~n", [timestamp(), Member]), - Pid = erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]), - erpc:call(NodeName, erlang, exit, [Pid, kill]), - State#{operations_count => OpCount + 1, - successful_ops => SuccessOps + 1}. + Pid = erpc:call(NodeName, erlang, whereis, [?CLUSTER_NAME]), + erpc:call(NodeName, erlang, exit, [Pid, kill]), + State#{operations_count => OpCount + 1, + successful_ops => SuccessOps + 1} + end. -spec wait_for_applied_index_convergence([ra:server_id()], non_neg_integer()) -> ok. wait_for_applied_index_convergence(Members, MaxRetries) when MaxRetries > 0 -> From fb90a82aa99b2fb34eae173a00acdd80a7c9d104 Mon Sep 17 00:00:00 2001 From: Michal Kuratczyk Date: Tue, 8 Jul 2025 16:34:13 +0200 Subject: [PATCH 57/60] Enable maybe_expre for OTP26 pipelines --- src/ra_kv.erl | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/ra_kv.erl b/src/ra_kv.erl index 3892bb23f..b40e08b35 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -1,4 +1,7 @@ -module(ra_kv). + +-feature(maybe_expr, enable). + -behaviour(ra_machine). -include("src/ra.hrl"). From 67b67a46f9702ac07cd5cc1297f3828dd466f1d3 Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Tue, 8 Jul 2025 15:55:17 +0100 Subject: [PATCH 58/60] fixes --- src/ra_kv.erl | 4 +- src/ra_log.erl | 110 ++++++++++++++++++++++------------------ test/ra_SUITE.erl | 1 + test/ra_log_2_SUITE.erl | 30 +++++++---- 4 files changed, 83 insertions(+), 62 deletions(-) diff --git a/src/ra_kv.erl b/src/ra_kv.erl index b40e08b35..6c26f4e19 100644 --- a/src/ra_kv.erl +++ b/src/ra_kv.erl @@ -1,10 +1,8 @@ -module(ra_kv). - -feature(maybe_expr, enable). - -behaviour(ra_machine). --include("src/ra.hrl"). +-include("src/ra.hrl"). -include_lib("eunit/include/eunit.hrl"). -export([ diff --git a/src/ra_log.erl b/src/ra_log.erl index 55311ed31..20bbeef6d 100644 --- a/src/ra_log.erl +++ b/src/ra_log.erl @@ -26,6 +26,7 @@ partial_read/3, execute_read_plan/4, read_plan_info/1, + previous_wal_index/1, last_index_term/1, set_last_index/2, handle_event/2, @@ -111,10 +112,10 @@ snapshot_state :: ra_snapshot:state(), current_snapshot :: option(ra_idxterm()), last_resend_time :: option({integer(), WalPid :: pid() | undefined}), - last_wal_write :: {pid(), Ms :: integer()}, + last_wal_write :: {pid(), Ms :: integer(), ra:index() | -1}, reader :: ra_log_segments:state(), mem_table :: ra_mt:state(), - tx = false :: boolean(), + tx = false :: false | {true, ra:range()}, pending = [] :: ra_seq:state(), live_indexes = [] :: ra_seq:state() }). @@ -265,6 +266,17 @@ init(#{uid := UId, {SnapIdx, Range}}) end end, + LastWalIdx = case ra_log_wal:last_writer_seq(Wal, UId) of + {ok, undefined} -> + -1; + {ok, Idx} -> + Idx; + {error, wal_down} -> + ?ERROR("~ts: ra_log:init/1 cannot complete as wal" + " process is down.", + [LogId]), + exit(wal_down) + end, Cfg = #cfg{directory = Dir, uid = UId, log_id = LogId, @@ -282,7 +294,7 @@ init(#{uid := UId, mem_table = Mt, snapshot_state = SnapshotState, current_snapshot = ra_snapshot:current(SnapshotState), - last_wal_write = {whereis(Wal), now_ms()}, + last_wal_write = {whereis(Wal), now_ms(), LastWalIdx}, live_indexes = LiveIndexes }, put_counter(Cfg, ?C_RA_SVR_METRIC_SNAPSHOT_INDEX, SnapIdx), @@ -316,18 +328,7 @@ init(#{uid := UId, {_, L} -> L end, - LastWrittenIdx = case ra_log_wal:last_writer_seq(Wal, UId) of - {ok, undefined} -> - %% take last segref index - max(SnapIdx, LastSegRefIdx); - {ok, Idx} -> - max(Idx, LastSegRefIdx); - {error, wal_down} -> - ?ERROR("~ts: ra_log:init/1 cannot complete as wal" - " process is down.", - [State2#?MODULE.cfg#cfg.log_id]), - exit(wal_down) - end, + LastWrittenIdx = lists:max([LastWalIdx, SnapIdx, LastSegRefIdx]), {LastWrittenTerm, State3} = case LastWrittenIdx of SnapIdx -> {SnapTerm, State2}; @@ -360,28 +361,32 @@ close(#?MODULE{cfg = #cfg{uid = _UId}, -spec begin_tx(state()) -> state(). begin_tx(State) -> - State#?MODULE{tx = true}. + State#?MODULE{tx = {true, undefined}}. -spec commit_tx(state()) -> {ok, state()} | {error, wal_down, state()}. commit_tx(#?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, - tx = true, + tx = {true, TxRange}, + range = Range, mem_table = Mt1} = State) -> {Entries, Mt} = ra_mt:commit(Mt1), Tid = ra_mt:tid(Mt), WriterId = {UId, self()}, - {WalCommands, Num} = - lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N}) -> + PrevIdx = previous_wal_index(State), + {WalCommands, Num, _} = + lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N, Prev}) -> Cmd = {ttb, term_to_iovec(Cmd0)}, - WalC = {append, WriterId, Tid, Idx-1, Idx, Term, Cmd}, - {[WalC | WC], N+1} - end, {[], 0}, Entries), + WalC = {append, WriterId, Tid, Prev, Idx, Term, Cmd}, + {[WalC | WC], N+1, Idx} + end, {[], 0, PrevIdx}, Entries), + {_, LastIdx} = Range, case ra_log_wal:write_batch(Wal, lists:reverse(WalCommands)) of {ok, Pid} -> ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, Num), {ok, State#?MODULE{tx = false, - last_wal_write = {Pid, now_ms()}, + range = ra_range:add(TxRange, Range), + last_wal_write = {Pid, now_ms(), LastIdx}, mem_table = Mt}}; {error, wal_down} -> %% still need to return the state here @@ -421,7 +426,7 @@ append({Idx, Term, Cmd0} = Entry, put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), State#?MODULE{range = ra_range:extend(Idx, Range), last_term = Term, - last_wal_write = {Pid, now_ms()}, + last_wal_write = {Pid, now_ms(), Idx}, pending = ra_seq:append(Idx, Pend), mem_table = Mt}; {error, wal_down} -> @@ -438,15 +443,14 @@ append({Idx, Term, Cmd0} = Entry, end; append({Idx, Term, _Cmd} = Entry, #?MODULE{cfg = Cfg, - range = Range, - tx = true, + tx = {true, TxRange}, pending = Pend0, mem_table = Mt0} = State) - when ?IS_NEXT_IDX(Idx, Range) -> + when ?IS_NEXT_IDX(Idx, TxRange) -> case ra_mt:stage(Entry, Mt0) of {ok, Mt} -> put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), - State#?MODULE{range = ra_range:extend(Idx, Range), + State#?MODULE{tx = {true, ra_range:extend(Idx, TxRange)}, last_term = Term, pending = ra_seq:append(Idx, Pend0), mem_table = Mt}; @@ -459,9 +463,10 @@ append({Idx, Term, _Cmd} = Entry, Cfg#cfg.uid, Mt0), append(Entry, State#?MODULE{mem_table = M0}) end; -append({Idx, _, _}, #?MODULE{range = Range}) -> - Msg = lists:flatten(io_lib:format("tried writing ~b - current range ~w", - [Idx, Range])), +append({Idx, _, _}, #?MODULE{range = Range, + tx = Tx}) -> + Msg = lists:flatten(io_lib:format("tried writing ~b - current range ~w tx ~p", + [Idx, Range, Tx])), exit({integrity_error, Msg}). -spec write(Entries :: [log_entry()], State :: state()) -> @@ -505,13 +510,7 @@ write_sparse({Idx, Term, _} = Entry, PrevIdx0, {ok, Mt} = ra_mt:insert_sparse(Entry, PrevIdx0, Mt0), ok = incr_counter(Cfg, ?C_RA_LOG_WRITE_OPS, 1), Tid = ra_mt:tid(Mt), - PrevIdx = case PrevIdx0 of - undefined -> - %% this is likely to always be accepted - 0; - _ -> - PrevIdx0 - end, + PrevIdx = previous_wal_index(State0), case ra_log_wal:write(Wal, {UId, self()}, Tid, PrevIdx, Idx, Term, Entry) of {ok, Pid} -> @@ -526,7 +525,7 @@ write_sparse({Idx, Term, _} = Entry, PrevIdx0, {ok, State0#?MODULE{range = NewRange, last_term = Term, mem_table = Mt, - last_wal_write = {Pid, now_ms()}}}; + last_wal_write = {Pid, now_ms(), Idx}}}; {error, wal_down} = Err-> Err end. @@ -704,6 +703,15 @@ read_plan_info(#read_plan{read = Read, num_segments => NumSegments}. +-spec previous_wal_index(state()) -> ra_idxterm() | -1. +previous_wal_index(#?MODULE{range = Range}) -> + case Range of + undefined -> + -1; + {_, LastIdx} -> + LastIdx + end. + -spec last_index_term(state()) -> option(ra_idxterm()). last_index_term(#?MODULE{range = {_, LastIdx}, last_term = LastTerm}) -> @@ -972,6 +980,8 @@ handle_event({down, _Pid, _Info}, #?MODULE{} = State) -> {State, []}. -spec next_index(state()) -> ra_index(). +next_index(#?MODULE{tx = {true, {_, Last}}}) -> + Last + 1; next_index(#?MODULE{range = {_, LastIdx}}) -> LastIdx + 1; next_index(#?MODULE{current_snapshot = {SnapIdx, _}}) -> @@ -1124,7 +1134,7 @@ promote_checkpoint(Idx, #?MODULE{cfg = Cfg, tick(Now, #?MODULE{cfg = #cfg{wal = Wal}, mem_table = Mt, last_written_index_term = {LastWrittenIdx, _}, - last_wal_write = {WalPid, Ms}} = State) -> + last_wal_write = {WalPid, Ms, _}} = State) -> CurWalPid = whereis(Wal), MtRange = ra_mt:range(Mt), case Now > Ms + ?WAL_RESEND_TIMEOUT andalso @@ -1262,7 +1272,7 @@ overview(#?MODULE{range = Range, snapshot_state = SnapshotState, current_snapshot = CurrSnap, reader = Reader, - last_wal_write = {_LastPid, LastMs}, + last_wal_write = {_LastPid, LastMs, LastWalIdx}, mem_table = Mt, pending = Pend } = State) -> @@ -1291,6 +1301,7 @@ overview(#?MODULE{range = Range, mem_table_range => ra_mt:range(Mt), mem_table_info => ra_mt:info(Mt), last_wal_write => LastMs, + last_wal_index => LastWalIdx, num_pending => ra_seq:length(Pend) }. @@ -1351,7 +1362,7 @@ release_resources(MaxOpenSegments, AccessPattern, %% only used by resend to wal functionality and doesn't update the mem table wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, wal = Wal} = Cfg, - range = _Range} = State, + last_wal_write = {_, _, _LastWalIdx}} = State, Tid, {Idx, Term, Cmd}) -> case ra_log_wal:write(Wal, {UId, self()}, Tid, Idx, Term, Cmd) of {ok, Pid} -> @@ -1359,7 +1370,7 @@ wal_rewrite(#?MODULE{cfg = #cfg{uid = UId, put_counter(Cfg, ?C_RA_SVR_METRIC_LAST_INDEX, Idx), State#?MODULE{%last_index = Idx, last_term = Term, - last_wal_write = {Pid, now_ms()} + last_wal_write = {Pid, now_ms(), Idx} }; {error, wal_down} -> error(wal_down) @@ -1372,14 +1383,15 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, mem_table = Mt0} = State, [{FstIdx, _, _} | _] = Entries) -> WriterId = {UId, self()}, + PrevIdx = previous_wal_index(State), %% all entries in a transaction are written to the same tid Tid = ra_mt:tid(Mt0), - {WalCommands, Num, Pend} = - lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N, P}) -> + {WalCommands, Num, LastIdx, Pend} = + lists:foldl(fun ({Idx, Term, Cmd0}, {WC, N, Prev, P}) -> Cmd = {ttb, term_to_iovec(Cmd0)}, - WalC = {append, WriterId, Tid, Idx-1, Idx, Term, Cmd}, - {[WalC | WC], N+1, ra_seq:append(Idx, P)} - end, {[], 0, Pend0}, Entries), + WalC = {append, WriterId, Tid, Prev, Idx, Term, Cmd}, + {[WalC | WC], N+1, Idx, ra_seq:append(Idx, P)} + end, {[], 0, PrevIdx, Pend0}, Entries), [{_, _, _, _PrevIdx, LastIdx, LastTerm, _} | _] = WalCommands, {_, Mt} = ra_mt:commit(Mt0), @@ -1395,7 +1407,7 @@ wal_write_batch(#?MODULE{cfg = #cfg{uid = UId, {ok, Pid} -> {ok, State#?MODULE{range = NewRange, last_term = LastTerm, - last_wal_write = {Pid, now_ms()}, + last_wal_write = {Pid, now_ms(), LastIdx}, mem_table = Mt, pending = Pend}}; {error, wal_down} = Err -> diff --git a/test/ra_SUITE.erl b/test/ra_SUITE.erl index c8b441f64..dd924a56b 100644 --- a/test/ra_SUITE.erl +++ b/test/ra_SUITE.erl @@ -709,6 +709,7 @@ wait_for_applied(Msg) -> false -> wait_for_applied(Msg) end after 10000 -> + flush(), error({timeout_waiting_for_applied, Msg}) end. diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 928adce37..5123f65eb 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -75,7 +75,8 @@ all_tests() -> write_config, sparse_write, overwritten_segment_is_cleared, - overwritten_segment_is_cleared_on_init + overwritten_segment_is_cleared_on_init, + snapshot_installation_with_live_indexes ]. groups() -> @@ -830,11 +831,6 @@ write_sparse_re_init(Config) -> ok. -write_sparse_after_snapshot_install(Config) -> - - - ok. - write_sparse_after_index_reset(Config) -> Log0 = ra_log_init(Config), Log1 = write_n(1, 6, 1, Log0), @@ -1371,19 +1367,33 @@ snapshot_installation_with_live_indexes(Config) -> %% create snapshot chunk Meta = meta(15, 2, [?N1]), - Chunk = create_snapshot_chunk(Config, Meta, #{}), + Chunk = create_snapshot_chunk(Config, Meta, [2, 9, 14], #{}), SnapState0 = ra_log:snapshot_state(Log2), {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), Machine = {machine, ?MODULE, #{}}, + + %% write a sparse one {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, Machine, SnapState1), run_effs(AEffs), + {ok, Log2b} = ra_log:write_sparse({14, 2, <<>>}, 9, Log2), {ok, Log3, Effs4} = ra_log:install_snapshot({15, 2}, ?MODULE, LiveIndexes, - ra_log:set_snapshot_state(SnapState, Log2)), + ra_log:set_snapshot_state(SnapState, Log2b)), + run_effs(Effs4), + ct:pal("o ~p", [ra_log:overview(Log3)]), {15, _} = ra_log:last_index_term(Log3), {15, _} = ra_log:last_written(Log3), + %% write the next index, bearning in mind the last index the WAL saw + %% was 14 + {ok, Log4} = ra_log:write([{16, 2, <<>>}], Log3), + Log = assert_log_events(Log4, + fun (L) -> + LW = ra_log:last_written(L), + {16, 2} == LW + end), + ct:pal("o ~p", [ra_log:overview(Log)]), ok. snapshot_installation(Config) -> @@ -2148,8 +2158,8 @@ meta(Idx, Term, Cluster) -> cluster => 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, Meta, Context) -> + create_snapshot_chunk(Config, Meta, <<"9">>, Context). create_snapshot_chunk(Config, #{index := Idx} = Meta, MacState, Context) -> OthDir = filename:join(?config(work_dir, Config), "snapshot_installation"), From b05a0dccea22bc599e81929ecd0d14b7302c5bed Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 9 Jul 2025 12:25:07 +0100 Subject: [PATCH 59/60] fix badkey --- src/ra_seq.erl | 14 +++++++------- src/ra_server_proc.erl | 4 ++-- src/ra_snapshot.erl | 4 +++- src/ra_worker.erl | 20 +++++++++++++------- 4 files changed, 25 insertions(+), 17 deletions(-) diff --git a/src/ra_seq.erl b/src/ra_seq.erl index da5161937..f7c8de6a2 100644 --- a/src/ra_seq.erl +++ b/src/ra_seq.erl @@ -71,14 +71,14 @@ floor(FloorIdxIncl, Seq) when is_list(Seq) -> -spec limit(ra:index(), state()) -> state(). -limit(CeilIdx, [Last | Rem]) +limit(CeilIdxIncl, [Last | Rem]) when is_integer(Last) andalso - Last > CeilIdx -> - limit(CeilIdx, Rem); -limit(CeilIdx, [{_, _} = T | Rem]) -> - case ra_range:limit(CeilIdx + 1, T) of + Last > CeilIdxIncl -> + limit(CeilIdxIncl, Rem); +limit(CeilIdxIncl, [{_, _} = T | Rem]) -> + case ra_range:limit(CeilIdxIncl + 1, T) of undefined -> - limit(CeilIdx, Rem); + limit(CeilIdxIncl, Rem); {I, I} -> [I | Rem]; {I, I2} when I == I2 - 1 -> @@ -86,7 +86,7 @@ limit(CeilIdx, [{_, _} = T | Rem]) -> NewRange -> [NewRange | Rem] end; -limit(_CeilIdx, Seq) -> +limit(_CeilIdxIncl, Seq) -> Seq. %% @doc adds two sequences together where To is diff --git a/src/ra_server_proc.erl b/src/ra_server_proc.erl index 67e9690c4..67b4ddeec 100644 --- a/src/ra_server_proc.erl +++ b/src/ra_server_proc.erl @@ -329,8 +329,8 @@ do_init(#{id := Id, true = ets:insert(ra_state, {Key, init, unknown}), process_flag(trap_exit, true), Config = #{counter := Counter, - system_config := #{names := Names} = SysConf} = maps:merge(config_defaults(Id), - Config0), + system_config := #{names := Names} = SysConf} = + maps:merge(config_defaults(Id), Config0), MsgQData = maps:get(message_queue_data, SysConf, off_heap), MinBinVheapSize = maps:get(server_min_bin_vheap_size, SysConf, ?MIN_BIN_VHEAP_SIZE), diff --git a/src/ra_snapshot.erl b/src/ra_snapshot.erl index 25778b1b1..0a78f670b 100644 --- a/src/ra_snapshot.erl +++ b/src/ra_snapshot.erl @@ -549,7 +549,9 @@ complete_accept(Chunk, Num, Machine, current = IdxTerm}, {ok, #{machine_version := SnapMacVer}, MacState} = recover(State), SnapMacMod = ra_machine:which_module(Machine, SnapMacVer), - LiveIndexes = ra_machine:live_indexes(SnapMacMod, MacState), + %% TODO: allow the ra machine to return a re_seq instead of a plain list + LiveIndexes = ra_seq:from_list( + ra_machine:live_indexes(SnapMacMod, MacState)), SnapDir = make_snapshot_dir(Dir, Idx, Term), ok = write_indexes(SnapDir, LiveIndexes), %% delete accepting marker file diff --git a/src/ra_worker.erl b/src/ra_worker.erl index cddd2e030..313755d97 100644 --- a/src/ra_worker.erl +++ b/src/ra_worker.erl @@ -8,6 +8,7 @@ -module(ra_worker). -behaviour(gen_server). +-include("ra.hrl"). -export([start_link/1, queue_work/3]). @@ -18,7 +19,8 @@ terminate/2, code_change/3]). --record(state, {}). +-record(state, {log_id = "" :: unicode:chardata() + }). %%% ra worker responsible for doing background work for a ra server. %%% @@ -39,9 +41,11 @@ queue_work(Pid, FunOrMfa, ErrFun) when is_pid(Pid) -> %%% gen_server callbacks %%%=================================================================== -init(Config) when is_map(Config) -> +init(#{id := Id} = Config) when is_map(Config) -> process_flag(trap_exit, true), - {ok, #state{}}. + LogId = maps:get(friendly_name, Config, + lists:flatten(io_lib:format("~w", [Id]))), + {ok, #state{log_id = LogId}}. handle_call(_, _From, State) -> {reply, ok, State}. @@ -52,8 +56,9 @@ handle_cast({work, FunOrMfa, ErrFun}, State) -> try erlang:apply(M, F, Args) of _ -> ok - catch Type:Err:_Stack -> - %% TODO: log + catch Type:Err:Stack -> + ?WARN("~ts: worker encounted error ~0p of type ~s, Stack:~n~p", + [State#state.log_id, Err, Type, Stack]), ErrFun({Type, Err}), ok end; @@ -61,8 +66,9 @@ handle_cast({work, FunOrMfa, ErrFun}, State) -> try FunOrMfa() of _ -> ok - catch Type:Err:_Stack -> - %% TODO: log + catch Type:Err:Stack -> + ?WARN("~ts: worker encounted error ~0p of type ~s, Stack:~n~p", + [State#state.log_id, Err, Type, Stack]), ErrFun({Type, Err}) end, ok From fbbe17318fdbf72b6e30104d66059c1bcd1d474f Mon Sep 17 00:00:00 2001 From: Karl Nilsson Date: Wed, 9 Jul 2025 12:51:42 +0100 Subject: [PATCH 60/60] handle missing segments event after follower recovery --- src/ra_server.erl | 3 ++- test/ra_log_2_SUITE.erl | 7 ++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/ra_server.erl b/src/ra_server.erl index 7a3a474e6..cb152e63f 100644 --- a/src/ra_server.erl +++ b/src/ra_server.erl @@ -405,6 +405,7 @@ init(#{id := Id, maps:get(membership, Config, voter)), #{cfg => Cfg, + leader_id => undefined, current_term => CurrentTerm, cluster => Cluster0, % There may be scenarios when a single server @@ -1339,8 +1340,8 @@ handle_follower(#heartbeat_rpc{leader_id = LeaderId, {follower, State, [cast_reply(Id, LeaderId, Reply)]}; handle_follower({ra_log_event, Evt}, #{log := Log0, cfg := #cfg{id = Id}, - leader_id := LeaderId, current_term := Term} = State0) -> + LeaderId = maps:get(leader_id, State0, undefined), % forward events to ra_log % if the last written changes then send an append entries reply LW = ra_log:last_written(Log0), diff --git a/test/ra_log_2_SUITE.erl b/test/ra_log_2_SUITE.erl index 5123f65eb..47ce1f5cd 100644 --- a/test/ra_log_2_SUITE.erl +++ b/test/ra_log_2_SUITE.erl @@ -1832,9 +1832,10 @@ sparse_write(Config) -> SnapState0 = ra_log:snapshot_state(Log2), {ok, SnapState1} = ra_snapshot:begin_accept(Meta, SnapState0), Machine = {machine, ?MODULE, #{}}, - {SnapState, _, LiveIndexes, AEffs} = ra_snapshot:complete_accept(Chunk, 1, - Machine, - SnapState1), + {SnapState, _, LiveIndexesSeq, AEffs} = ra_snapshot:complete_accept(Chunk, 1, + Machine, + SnapState1), + ?assertEqual(LiveIndexes, lists:reverse(ra_seq:expand(LiveIndexesSeq))), run_effs(AEffs), Log3 = ra_log:set_snapshot_state(SnapState, Log2), {ok, Log4, _} = ra_log:install_snapshot({15, 2}, ?MODULE,