├── .github
├── ISSUE_TEMPLATE
│ ├── config.yml
│ ├── feature-request.yml
│ └── bug-report.yml
├── codecov.yml
└── workflows
│ ├── test-job.yaml
│ └── test-and-release.yaml
├── doc
├── kmm-social-preview.png
├── _head.html
├── overview.edoc
├── stylesheet.css
├── kmm-favicon.svg
├── kmm-logo.svg
└── kmm-social-preview.svg
├── src
├── kmm_logging.hrl
├── khepri_mnesia_migration_app.erl
├── khepri_mnesia_migration.app.src
├── kmm_error.hrl
├── m2k_cluster_sync_sup.erl
├── khepri_mnesia_migration_sup.erl
├── m2k_table_copy_sup.erl
├── m2k_table_copy_sup_sup.erl
├── kmm_utils.erl
├── m2k_export.erl
├── mnesia_to_khepri_example_converter.erl
├── m2k_subscriber.erl
├── mnesia_to_khepri_converter.erl
├── m2k_cluster_sync.erl
├── mnesia_to_khepri.erl
└── m2k_table_copy.erl
├── .gitignore
├── test
├── handle_fallback.erl
├── controllable_converter.erl
├── cth_log_redirect_any_domains.erl
├── crashing_converter.erl
├── kmm_gen_servers.erl
└── helpers.erl
├── rebar.lock
├── mix.exs
├── rebar.config
├── README.md
├── LICENSE-Apache-2.0
└── LICENSE-MPL-2.0
/.github/ISSUE_TEMPLATE/config.yml:
--------------------------------------------------------------------------------
1 | blank_issues_enabled: true
2 |
--------------------------------------------------------------------------------
/doc/kmm-social-preview.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/rabbitmq/khepri_mnesia_migration/main/doc/kmm-social-preview.png
--------------------------------------------------------------------------------
/doc/_head.html:
--------------------------------------------------------------------------------
1 |
2 |
3 |
--------------------------------------------------------------------------------
/src/kmm_logging.hrl:
--------------------------------------------------------------------------------
1 | -define(KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN,
2 | [khepri_mnesia_migration, mnesia_to_khepri, cluster_sync]).
3 |
4 | -define(KMM_M2K_TABLE_COPY_LOG_DOMAIN,
5 | [khepri_mnesia_migration, mnesia_to_khepri, table_copy]).
6 |
--------------------------------------------------------------------------------
/.github/ISSUE_TEMPLATE/feature-request.yml:
--------------------------------------------------------------------------------
1 | name: Feature request
2 | description: Suggest an idea for this project
3 | labels: enhancement
4 |
5 | body:
6 | - type: textarea
7 | id: describe-problem
8 | attributes:
9 | label: Why
10 | validations:
11 | required: true
12 |
13 | - type: textarea
14 | id: describe-solution
15 | attributes:
16 | label: How
17 | validations:
18 | required: true
19 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | .rebar3
2 | _*
3 | .eunit
4 | *.o
5 | *.beam
6 | *.plt
7 | *.swp
8 | *.swo
9 | .erlang.cookie
10 | ebin
11 | log
12 | erl_crash.dump
13 | .rebar
14 | logs
15 | _build
16 | .idea
17 | *.iml
18 | rebar3.crashdump
19 | *~
20 | /_build
21 | /_checkouts
22 | /doc/*
23 | !/doc/_head.html
24 | !/doc/kmm-favicon.svg
25 | !/doc/kmm-logo.svg
26 | !/doc/kmm-social-preview.svg
27 | !/doc/kmm-social-preview.png
28 | !/doc/overview.edoc
29 | !/doc/stylesheet.css
30 | /mix.lock
31 |
--------------------------------------------------------------------------------
/.github/ISSUE_TEMPLATE/bug-report.yml:
--------------------------------------------------------------------------------
1 | name: Bug report
2 | description: Create a report to help us improve
3 | labels: bug
4 |
5 | body:
6 | - type: textarea
7 | id: describe-problem
8 | attributes:
9 | label: What does not work?
10 | validations:
11 | required: true
12 |
13 | - type: textarea
14 | id: expected-behavior
15 | attributes:
16 | label: Expected behavior
17 | validations:
18 | required: true
19 |
20 | - type: textarea
21 | id: reproduction-steps
22 | attributes:
23 | label: How to reproduce
24 | validations:
25 | required: false
26 |
--------------------------------------------------------------------------------
/.github/codecov.yml:
--------------------------------------------------------------------------------
1 | coverage:
2 | status:
3 | # The whole project must reach a coverage of 75% for the `codecov/project`
4 | # check to succeed.
5 | project:
6 | default:
7 | target: 75%
8 | threshold: 5% # How much the coverage can decrease.
9 | paths:
10 | - "!.github/"
11 |
12 | # The patch itself (i.e. the modified lines) must be 75% covered by tests
13 | # for the `codecov/patch` check to succeed.
14 | patch:
15 | default:
16 | target: 75%
17 | threshold: 5%
18 | paths:
19 | - "!.github/"
20 | informational: true
21 |
--------------------------------------------------------------------------------
/src/khepri_mnesia_migration_app.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(khepri_mnesia_migration_app).
12 |
13 | -behaviour(application).
14 |
15 | -export([start/2, stop/1]).
16 |
17 | start(_StartType, _StartArgs) ->
18 | khepri_mnesia_migration_sup:start_link().
19 |
20 | stop(_State) ->
21 | ok.
22 |
23 | %% internal functions
24 |
--------------------------------------------------------------------------------
/src/khepri_mnesia_migration.app.src:
--------------------------------------------------------------------------------
1 | %% vim:ft=erlang:sw=2:et:
2 | {application, khepri_mnesia_migration,
3 | [{description, "Tools to migrate between Mnesia and Khepri"},
4 | {vsn, "0.8.0"},
5 | {registered, [khepri_mnesia_migration_sup,
6 | m2k_cluster_sync_sup]},
7 | {mod, {khepri_mnesia_migration_app, []}},
8 | {applications,
9 | [kernel,
10 | stdlib
11 | ]},
12 | {env,[]},
13 | {files, [
14 | "README.md", "LICENSE-Apache-2.0", "LICENSE-MPL-2.0", "mix.exs",
15 | "rebar.config", "rebar.lock", "src"]},
16 | {modules, []},
17 |
18 | {licenses, ["Apache-2.0", "MPL-2.0"]},
19 | {links, [{"GitHub", "https://github.com/rabbitmq/khepri_mnesia_migration"}]},
20 | {build_tools, ["rebar3", "mix"]},
21 | {doc, "doc"}
22 | ]}.
23 |
--------------------------------------------------------------------------------
/doc/overview.edoc:
--------------------------------------------------------------------------------
1 | @author Jean-Sébastien Pédron
2 | @author Diana Parra Corbacho
3 | @author Michael Davis
4 | @author The RabbitMQ team
5 | @copyright 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom" refers to Broadcom Inc. and/or its subsidiaries.
6 | @title The Mnesia↔Khepri migration toolkit
7 | @version 0.8.0
8 |
9 | @doc
10 | `khepri_mnesia_migration' is a library providing helpers to migrate data
11 | between the Mnesia and Khepri databases.
12 |
13 | Currently, only the migration from Mnesia to Khepri is supported.
14 |
15 | Fork me on GitHub
16 |
--------------------------------------------------------------------------------
/doc/stylesheet.css:
--------------------------------------------------------------------------------
1 | body > h2.indextitle:first-of-type
2 | {
3 | font-size: 1.3em;
4 | text-align: center;
5 | border-bottom: 0px;
6 | font-family: "DejaVu Sans Mono", monospace;
7 | overflow-wrap: normal;
8 |
9 | padding-top: 115px;
10 | background-image: url("kmm-logo.svg");
11 | background-size: auto 100px;
12 | background-repeat: no-repeat;
13 | background-position: top;
14 | }
15 |
16 | .navbar
17 | {
18 | background-color: rgb(245, 242, 240);
19 | border-radius: 6px;
20 | }
21 |
22 | .navbar table tr
23 | {
24 | background-color: transparent;
25 | }
26 |
27 | .navbar table td:last-of-type
28 | {
29 | display: none;
30 | }
31 |
32 | .navbar + hr
33 | {
34 | display: none;
35 | }
36 |
37 | .markdown-body ul,
38 | .markdown-body ol
39 | {
40 | margin-top: .25em;
41 | }
42 |
--------------------------------------------------------------------------------
/src/kmm_error.hrl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | -define(
10 | kmm_error(Name, Props),
11 | {khepri_mnesia_migration, Name, Props}).
12 |
13 | -define(
14 | kmm_exception(Name, Props),
15 | {khepri_mnesia_migration_ex, Name, Props}).
16 |
17 | -define(
18 | kmm_misuse(Exception),
19 | erlang:error(Exception)).
20 |
21 | -define(
22 | kmm_misuse(Name, Props),
23 | ?kmm_misuse(?kmm_exception(Name, Props))).
24 |
25 | -define(
26 | kmm_raise_misuse(Name, Props, Stacktrace),
27 | erlang:raise(error, ?kmm_exception(Name, Props), Stacktrace)).
28 |
--------------------------------------------------------------------------------
/test/handle_fallback.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2024-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | -module(handle_fallback).
10 |
11 | -include_lib("eunit/include/eunit.hrl").
12 |
13 | can_detect_infinite_loop_test() ->
14 | ok = mnesia:start(),
15 | Table = some_table,
16 | ?assertEqual(
17 | {aborted, {no_exists, Table}},
18 | mnesia_to_khepri:handle_fallback(
19 | %% No need for a store, `is_migration_finished()' will return false.
20 | non_existing_store,
21 | <<"id">>,
22 | fun() ->
23 | mnesia:transaction(fun() -> mnesia:read(Table, some_key) end)
24 | end,
25 | ok)).
26 |
--------------------------------------------------------------------------------
/test/controllable_converter.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2024-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | -module(controllable_converter).
10 |
11 | -behaviour(mnesia_to_khepri_converter).
12 |
13 | -export([init_copy_to_khepri/4,
14 | copy_to_khepri/3,
15 | delete_from_khepri/3]).
16 |
17 | -record(?MODULE, {store_id,
18 | caller}).
19 |
20 | init_copy_to_khepri(StoreId, _MigrationId, _Tables, Caller) ->
21 | State = #?MODULE{store_id = StoreId,
22 | caller = Caller},
23 | Caller ! {?FUNCTION_NAME, StoreId, self()},
24 | receive proceed -> ok end,
25 | {ok, State}.
26 |
27 | copy_to_khepri(_Table, _Record, State) ->
28 | {ok, State}.
29 |
30 | delete_from_khepri(_Table, _Key, State) ->
31 | {ok, State}.
32 |
--------------------------------------------------------------------------------
/src/m2k_cluster_sync_sup.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(m2k_cluster_sync_sup).
12 |
13 | -behaviour(supervisor).
14 |
15 | -export([start_link/0,
16 | prepare_worker/1]).
17 |
18 | -export([init/1]).
19 |
20 | -define(SERVER, ?MODULE).
21 |
22 | start_link() ->
23 | supervisor:start_link({local, ?SERVER}, ?MODULE, []).
24 |
25 | prepare_worker(StoreId) ->
26 | supervisor:start_child(?SERVER, [#{khepri_store => StoreId}]).
27 |
28 | init([]) ->
29 | M2KClusterSync = #{id => m2k_cluster_sync,
30 | restart => temporary,
31 | start => {m2k_cluster_sync, start_link, []}},
32 |
33 | SupFlags = #{strategy => simple_one_for_one,
34 | intensity => 0,
35 | period => 1},
36 | ChildSpecs = [M2KClusterSync],
37 | {ok, {SupFlags, ChildSpecs}}.
38 |
--------------------------------------------------------------------------------
/test/cth_log_redirect_any_domains.erl:
--------------------------------------------------------------------------------
1 | -module(cth_log_redirect_any_domains).
2 |
3 | -export([log/2]).
4 |
5 | -define(BACKEND_MODULE, cth_log_redirect).
6 |
7 | %% Reversed behavior compared to `cth_log_redirect': log events with an
8 | %% unknown domain are sent to the `cth_log_redirect' server, others are
9 | %% dropped (as they are already handled by `cth_log_redirect').
10 | log(#{msg:={report,_Msg},meta:=#{domain:=[otp,sasl]}},_Config) ->
11 | ok;
12 | log(#{meta:=#{domain:=[otp]}},_Config) ->
13 | ok;
14 | log(#{meta:=#{domain:=_} = Meta}=Log,
15 | #{config := #{group_leader := GL}} = Config) ->
16 | Log1 = Log#{meta => Meta#{gl => GL}},
17 | do_log(add_log_category(Log1,error_logger),Config);
18 | log(_Log,_Config) ->
19 | ok.
20 |
21 | add_log_category(#{meta:=Meta}=Log,Category) ->
22 | Log#{meta=>Meta#{?BACKEND_MODULE=>#{category=>Category}}}.
23 |
24 | do_log(Log,Config) ->
25 | CthLogRedirect = case Config of
26 | #{config := #{group_leader_node := GLNode}} ->
27 | {?BACKEND_MODULE, GLNode};
28 | _ ->
29 | ?BACKEND_MODULE
30 | end,
31 | ok = gen_server:call(CthLogRedirect,{log,Log,Config}).
32 |
--------------------------------------------------------------------------------
/src/khepri_mnesia_migration_sup.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(khepri_mnesia_migration_sup).
12 |
13 | -behaviour(supervisor).
14 |
15 | -export([start_link/0]).
16 |
17 | -export([init/1]).
18 |
19 | -define(SERVER, ?MODULE).
20 |
21 | start_link() ->
22 | supervisor:start_link({local, ?SERVER}, ?MODULE, []).
23 |
24 | init([]) ->
25 | M2KClusterSyncSup = #{id => m2k_cluster_sync_sup,
26 | type => supervisor,
27 | start => {m2k_cluster_sync_sup, start_link, []}},
28 | M2KTableCopySupSup = #{id => m2k_table_copy_sup_sup,
29 | type => supervisor,
30 | start => {m2k_table_copy_sup_sup, start_link, []}},
31 |
32 | SupFlags = #{strategy => one_for_all,
33 | intensity => 0,
34 | period => 1},
35 | ChildSpecs = [M2KClusterSyncSup, M2KTableCopySupSup],
36 | {ok, {SupFlags, ChildSpecs}}.
37 |
--------------------------------------------------------------------------------
/src/m2k_table_copy_sup.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(m2k_table_copy_sup).
12 |
13 | -behaviour(supervisor).
14 |
15 | -export([start_link/1]).
16 |
17 | -export([init/1]).
18 |
19 | -define(SERVER, ?MODULE).
20 |
21 | start_link(Args) ->
22 | supervisor:start_link(?MODULE, [Args]).
23 |
24 | init([Args]) ->
25 | M2KSubscriber = #{id => m2k_subscriber,
26 | restart => transient,
27 | significant => true,
28 | start => {m2k_subscriber, start_link, [Args]}},
29 | M2KTableCopy = #{id => m2k_table_copy,
30 | restart => transient,
31 | significant => true,
32 | start => {m2k_table_copy, start_link, [Args]}},
33 |
34 | SupFlags = #{strategy => one_for_all,
35 | intensity => 0,
36 | period => 1,
37 | auto_shutdown => any_significant},
38 | ChildSpecs = [M2KSubscriber, M2KTableCopy],
39 | {ok, {SupFlags, ChildSpecs}}.
40 |
--------------------------------------------------------------------------------
/src/m2k_table_copy_sup_sup.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(m2k_table_copy_sup_sup).
12 |
13 | -behaviour(supervisor).
14 |
15 | -export([start_link/0,
16 | prepare_workers_sup/4]).
17 |
18 | -export([init/1]).
19 |
20 | -define(SERVER, ?MODULE).
21 |
22 | start_link() ->
23 | supervisor:start_link({local, ?SERVER}, ?MODULE, []).
24 |
25 | prepare_workers_sup(StoreId, MigrationId, Tables, Mod) ->
26 | supervisor:start_child(?SERVER, [#{khepri_store => StoreId,
27 | migration_id => MigrationId,
28 | tables => Tables,
29 | converter_mod => Mod}]).
30 |
31 | init([]) ->
32 | M2KTableCopySup = #{id => m2k_table_copy_sup,
33 | restart => temporary,
34 | start => {m2k_table_copy_sup, start_link, []}},
35 |
36 | SupFlags = #{strategy => simple_one_for_one,
37 | intensity => 0,
38 | period => 1},
39 | ChildSpecs = [M2KTableCopySup],
40 | {ok, {SupFlags, ChildSpecs}}.
41 |
--------------------------------------------------------------------------------
/rebar.lock:
--------------------------------------------------------------------------------
1 | {"1.2.0",
2 | [{<<"aten">>,{pkg,<<"aten">>,<<"0.6.0">>},2},
3 | {<<"gen_batch_server">>,{pkg,<<"gen_batch_server">>,<<"0.8.9">>},2},
4 | {<<"horus">>,{pkg,<<"horus">>,<<"0.3.1">>},1},
5 | {<<"khepri">>,{pkg,<<"khepri">>,<<"0.17.1">>},0},
6 | {<<"ra">>,{pkg,<<"ra">>,<<"2.16.7">>},1},
7 | {<<"seshat">>,{pkg,<<"seshat">>,<<"0.6.0">>},2}]}.
8 | [
9 | {pkg_hash,[
10 | {<<"aten">>, <<"7A57B275A6DAF515AC3683FB9853E280B4D0DCDD74292FD66AC4A01C8694F8C7">>},
11 | {<<"gen_batch_server">>, <<"1C6BC0F530BF8C17E8B4ACC20C2CC369FFA5BEE2B46DE01E21410745F24B1BC9">>},
12 | {<<"horus">>, <<"A5274C96E15924C28413752617B06050E4B08C04628B88209AFF9EA076F2BCB5">>},
13 | {<<"khepri">>, <<"B3F09238DF2240F29392CC894091C80711CB2FF4430E44B2828E75893C6484DD">>},
14 | {<<"ra">>, <<"1582982CC88C2FA4D6AFAE6DCF3E80E6AB4058A8BEEDA82A9A374216329B7B29">>},
15 | {<<"seshat">>, <<"3172EB1D7A2A4F66108CD6933A4E465AFF80F84AA90ED83F047B92F636123CCD">>}]},
16 | {pkg_hash_ext,[
17 | {<<"aten">>, <<"5F39A164206AE3F211EF5880B1F7819415686436E3229D30B6A058564FBAA168">>},
18 | {<<"gen_batch_server">>, <<"C8581FE4A4B6BCCF91E53CE6A8C7E6C27C8C591BAB5408B160166463F5579C22">>},
19 | {<<"horus">>, <<"D564D30EBC274F0D92C3D44A336D0B892F000BE159912AE4E6838701E85495EC">>},
20 | {<<"khepri">>, <<"A030F3F675E88B8727AF33A6D01441A54A7B1D9F1106EC33FBCAE26119DB0843">>},
21 | {<<"ra">>, <<"46F81F5EB98015F4574CEF523FA49B3BAC3035E9E9B8001D4C2D312ED94B61BD">>},
22 | {<<"seshat">>, <<"7CEF700F92831DD7CAE6A6DD223CCC55AC88ECCE0631EE9AB0F2B5FB70E79B90">>}]}
23 | ].
24 |
--------------------------------------------------------------------------------
/test/crashing_converter.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2024-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | -module(crashing_converter).
10 |
11 | -behaviour(mnesia_to_khepri_converter).
12 |
13 | -export([init_copy_to_khepri/3,
14 | copy_to_khepri/3,
15 | delete_from_khepri/3,
16 | finish_copy_to_khepri/1]).
17 |
18 | -record(?MODULE, {store_id}).
19 |
20 | init_copy_to_khepri(crash_during_init = StoreId, _MigrationId, _Tables) ->
21 | error({crash, StoreId});
22 | init_copy_to_khepri(StoreId, _MigrationId, _Tables) ->
23 | State = #?MODULE{store_id = StoreId},
24 | {ok, State}.
25 |
26 | copy_to_khepri(
27 | _Table, _Record,
28 | #?MODULE{store_id = crash_during_copy = StoreId}) ->
29 | error({crash, StoreId});
30 | copy_to_khepri(
31 | _Table, _Record,
32 | State) ->
33 | {ok, State}.
34 |
35 | delete_from_khepri(
36 | _Table, _Key,
37 | #?MODULE{store_id = crash_during_delete = StoreId}) ->
38 | error({crash, StoreId});
39 | delete_from_khepri(
40 | _Table, _Key,
41 | State) ->
42 | {ok, State}.
43 |
44 | finish_copy_to_khepri(#?MODULE{store_id = crash_during_finish = StoreId}) ->
45 | error({crash, StoreId});
46 | finish_copy_to_khepri(_State) ->
47 | ok.
48 |
--------------------------------------------------------------------------------
/test/kmm_gen_servers.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2024-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | -module(kmm_gen_servers).
10 |
11 | -include_lib("eunit/include/eunit.hrl").
12 |
13 | m2k_cluster_sync_test() ->
14 | Module = m2k_cluster_sync,
15 | test_gen_server(Module).
16 |
17 | m2k_table_copy_test() ->
18 | Module = m2k_table_copy,
19 | test_gen_server(Module).
20 |
21 | m2k_subscriber_test() ->
22 | Module = m2k_subscriber,
23 | test_gen_server(Module).
24 |
25 | test_gen_server(Module) ->
26 | ok = mnesia:start(),
27 | {atomic, ok} = mnesia:create_table(Module, []),
28 |
29 | RaSystem = Module,
30 | StoreId = RaSystem,
31 | StoreDir = helpers:store_dir_name(RaSystem),
32 | ?assertEqual({ok, StoreId}, khepri:start(StoreDir, StoreId)),
33 |
34 | Args = #{khepri_store => StoreId,
35 | migration_id => <<"id">>,
36 | tables => [Module],
37 | converter_mod => Module},
38 | {ok, Pid} = Module:start_link(Args),
39 |
40 | ?assertEqual(undefined, gen_server:call(Pid, unknown_call)),
41 | ?assertEqual(ok, gen_server:cast(Pid, unknown_cast)),
42 | Pid ! unknown_info,
43 |
44 | ?assert(erlang:is_process_alive(Pid)),
45 |
46 | ?assertEqual(ok, gen_server:stop(Pid)),
47 | ?assertNot(erlang:is_process_alive(Pid)),
48 |
49 | _ = helpers:remove_store_dir(StoreDir),
50 | ok.
51 |
--------------------------------------------------------------------------------
/mix.exs:
--------------------------------------------------------------------------------
1 | defmodule KhepriMnesiaMigration.MixProject do
2 | use Mix.Project
3 |
4 | def project do
5 | # To avoid duplication, we query the app file to learn the application
6 | # name, description and version.
7 | {:ok, [app]} = :file.consult("src/khepri_mnesia_migration.app.src")
8 | {:application, app_name, props} = app
9 |
10 | description = to_string(Keyword.get(props, :description))
11 | version = to_string(Keyword.get(props, :vsn))
12 |
13 | [
14 | app: app_name,
15 | description: description,
16 | version: version,
17 | language: :erlang,
18 | deps: deps()
19 | ]
20 | end
21 |
22 | def application do
23 | {:ok, [app]} = :file.consult("src/khepri_mnesia_migration.app.src")
24 | {:application, _app_name, props} = app
25 |
26 | Keyword.take(props, [:applications, :env, :mod, :registered])
27 | end
28 |
29 | defp deps() do
30 | # To avoid duplication, we query rebar.config to get the list of
31 | # dependencies and their version pinning.
32 | {:ok, terms} = :file.consult("rebar.config")
33 | deps = Keyword.get(terms, :deps)
34 |
35 | # The conversion to the mix.exs expected structure is basic, but that
36 | # should do it for our needs.
37 | for {app_name, version} <- deps do
38 | case version do
39 | _ when is_list(version) ->
40 | {app_name, to_string(version)}
41 |
42 | {:git, url} ->
43 | {app_name, git: to_string(url)}
44 |
45 | {:git, url, {:ref, ref}} ->
46 | {app_name, git: to_string(url), ref: to_string(ref)}
47 |
48 | {:git, url, {:branch, branch}} ->
49 | {app_name, git: to_string(url), branch: to_string(branch)}
50 |
51 | {:git, url, {:tag, tag}} ->
52 | {app_name, git: to_string(url), tag: to_string(tag)}
53 | end
54 | end
55 | end
56 | end
57 |
--------------------------------------------------------------------------------
/.github/workflows/test-job.yaml:
--------------------------------------------------------------------------------
1 | name: Single test job
2 |
3 | on:
4 | workflow_call:
5 | inputs:
6 | rebar_version:
7 | required: true
8 | type: string
9 | secrets:
10 | CODECOV_TOKEN:
11 | required: true
12 |
13 | jobs:
14 | test:
15 | name: "Erlang/OTP ${{ matrix.otp_version }} + ${{ matrix.os }}"
16 | runs-on: ${{ matrix.os }}
17 | strategy:
18 | fail-fast: false
19 | matrix:
20 | otp_version: ['26', '27']
21 | os: [ubuntu-latest, windows-latest]
22 |
23 | steps:
24 | - uses: actions/checkout@v4
25 | - uses: erlef/setup-beam@v1
26 | id: install-erlang
27 | with:
28 | otp-version: ${{ matrix.otp_version }}
29 | rebar3-version: ${{ inputs.rebar_version }}
30 |
31 | - name: Compile
32 | run: rebar3 compile
33 |
34 | - name: Xref
35 | run: rebar3 xref
36 | - name: EUnit (unit tests)
37 | run: env ERL_FLAGS='-enable-feature maybe_expr' rebar3 eunit --verbose --cover
38 | - name: Common test (integration tests)
39 | run: rebar3 ct --verbose --cover --sname ct
40 |
41 | - name: Upload common_test logs
42 | uses: actions/upload-artifact@v4
43 | if: ${{ always() }}
44 | with:
45 | name: common-test-logs-${{ matrix.otp_version }}-${{ matrix.os }}
46 | path: _build/test/logs
47 | include-hidden-files: true
48 | if-no-files-found: ignore
49 | retention-days: 5
50 |
51 | - name: Generate code coverage report
52 | run: rebar3 as test covertool generate
53 |
54 | - name: Upload code coverage to Codecov
55 | uses: codecov/codecov-action@v4
56 | with:
57 | token: ${{ secrets.CODECOV_TOKEN }}
58 | files: _build/test/covertool/khepri_mnesia_migration.covertool.xml
59 | flags: erlang-${{ matrix.otp_version }},os-${{ matrix.os }}
60 | name: Erlang/OTP ${{ matrix.otp_version }} on ${{ matrix.os }}
61 | verbose: true # optional (default = false)
62 |
--------------------------------------------------------------------------------
/src/kmm_utils.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(kmm_utils).
12 |
13 | -include("src/kmm_error.hrl").
14 |
15 | -export([is_mnesia_running/0,
16 | mnesia_nodes/0,
17 | %is_khepri_store_running/1,
18 | %khepri_nodes/1,
19 | erlang_node_uptime/0]).
20 |
21 | is_mnesia_running() ->
22 | mnesia:system_info(is_running) =:= yes.
23 |
24 | mnesia_nodes() ->
25 | case is_mnesia_running() of
26 | true ->
27 | AllNodes = lists:sort(mnesia:system_info(db_nodes)),
28 | RunningNodes = lists:sort(mnesia:system_info(running_db_nodes)),
29 | case AllNodes =:= RunningNodes of
30 | true ->
31 | AllNodes;
32 | false ->
33 | ?kmm_misuse(
34 | all_mnesia_nodes_must_run,
35 | #{all_nodes => AllNodes,
36 | running_nodes => RunningNodes})
37 | end;
38 | false ->
39 | ?kmm_misuse(mnesia_must_run, #{node => node()})
40 | end.
41 |
42 | %is_khepri_store_running(StoreId) ->
43 | % khepri_cluster:is_store_running(StoreId).
44 | %
45 | %khepri_nodes(StoreId) ->
46 | % case is_khepri_store_running(StoreId) of
47 | % true ->
48 | % AllNodes = lists:sort(khepri_cluster:nodes(StoreId)),
49 | % %% TODO: Ensure all nodes are running?
50 | % AllNodes;
51 | % false ->
52 | % ?kmm_misuse(
53 | % khepri_store_must_run,
54 | % #{node => node(),
55 | % store_id => StoreId})
56 | % end.
57 |
58 | erlang_node_uptime() ->
59 | CurrentTime = erlang:monotonic_time(),
60 | StartTime = erlang:system_info(start_time),
61 | erlang:convert_time_unit(CurrentTime - StartTime, native, millisecond).
62 |
--------------------------------------------------------------------------------
/rebar.config:
--------------------------------------------------------------------------------
1 | %% vim:ft=erlang:
2 | {minimum_otp_vsn, "26.0"}.
3 |
4 | {deps, [{khepri, "~> 0.17.0"}]}.
5 |
6 | {project_plugins, [covertool,
7 | rebar3_hex,
8 | {rebar3_edoc_extensions, "1.6.1"}]}.
9 |
10 | {erl_opts, [debug_info,
11 | warn_export_vars,
12 | warnings_as_errors]}.
13 | {erl_first_files, ["src/mnesia_to_khepri_converter.erl"]}.
14 |
15 | {dialyzer, [{warnings, [underspecs,
16 | unknown,
17 | unmatched_returns]},
18 | {plt_extra_apps, [mnesia, khepri]}]}.
19 |
20 | {xref_checks, [undefined_function_calls,
21 | undefined_functions,
22 | locals_not_used,
23 | deprecated_function_calls,
24 | deprecated_functions]}.
25 |
26 | {cover_enabled, true}.
27 | {cover_opts, [verbose]}.
28 | {cover_print_enabled, true}.
29 | {cover_export_enabled, true}.
30 | {covertool, [{coverdata_files, ["eunit.coverdata",
31 | "ct.coverdata"]}]}.
32 |
33 | {edoc_opts, [{stylesheet, "stylesheet.css"},
34 | {preprocess, true},
35 | {includes, ["."]},
36 | {sort_functions, false},
37 | {doclet, edoc_doclet_chunks},
38 | {layout, edoc_layout_chunks}]}.
39 |
40 | {alias, [{check, [xref,
41 | {ct, "--cover"},
42 | {eunit, "--cover"},
43 | {cover, "--verbose --min_coverage=75"},
44 | %% FIXME: Dialyzer is only executed on the library by
45 | %% default, not its testsuite. To run Dialyzer on the
46 | %% testsuites as well, the following command must be used:
47 | %% rebar as test dialyzer
48 | dialyzer,
49 | edoc]}]}.
50 |
51 | {profiles,
52 | [{test,
53 | [{deps, [%% FIXME: We need to add `cth_readable' as a dependency and an
54 | %% extra app for Dialyzer. That's because Rebar is using that
55 | %% application to override `ct:pal()' and Dialyzer complains it
56 | %% doesn't know this application.
57 | cth_readable,
58 | meck]},
59 | {dialyzer, [{plt_extra_apps, [common_test,
60 | cth_readable, %% <-- See comment above.
61 | edoc,
62 | eunit,
63 | khepri,
64 | mnesia,
65 | ra,
66 | tools]}]} %% <-- For `cover`.
67 | ]}]}.
68 |
69 | {hex, [{doc, edoc}]}.
70 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # `khepri_mnesia_migration`: Mnesia⬌Khepri migration toolkit
2 |
3 | [](https://hex.pm/packages/khepri_mnesia_migration/)
4 | [](https://github.com/rabbitmq/khepri_mnesia_migration/actions/workflows/test-and-release.yaml)
5 | [](https://codecov.io/gh/rabbitmq/khepri_mnesia_migration)
6 |
7 | `khepri_mnesia_migration` is a library providing helpers to migrate data
8 | between the Mnesia and Khepri databases.
9 |
10 | Currently, only the migration from Mnesia to Khepri is supported.
11 |
12 |
13 |
14 | ## Project maturity
15 |
16 | `khepri_mnesia_migration` is still under active development and should be
17 | considered *Alpha* at this stage.
18 |
19 | ## Documentation
20 |
21 | * A short tutorial in the [Getting started](#getting-started) section below
22 | * [Documentation and API reference](https://rabbitmq.github.io/khepri_mnesia_migration/)
23 |
24 | ## Getting started
25 |
26 | ### Add as a dependency
27 |
28 | Add `khepri_mnesia_migration` as a dependency of your project:
29 |
30 | Using Rebar:
31 |
32 | ```erlang
33 | %% In rebar.config
34 | {deps, [{khepri_mnesia_migration, "0.8.0"}]}.
35 | ```
36 |
37 | Using Erlang.mk:
38 |
39 | ```make
40 | # In your Makefile
41 | DEPS += khepri_mnesia_migration
42 | dep_khepri_mnesia_migration = hex 0.8.0
43 | ```
44 |
45 | Using Mix:
46 |
47 | ```elixir
48 | # In mix.exs
49 | defp deps do
50 | [
51 | {:khepri_mnesia_migration, "0.8.0"}
52 | ]
53 | end
54 | ```
55 |
56 | ### Synchronize cluster members
57 |
58 | To ensure a Khepri store has the same members as the Mnesia cluster, use
59 | `mnesia_to_khepri:synchronize_cluster_membership/{0,1}`:
60 |
61 | ```erlang
62 | mnesia_to_khepri:synchronize_cluster_membership(StoreId).
63 | ```
64 |
65 | ### Copy Mnesia tables to a Khepri store
66 |
67 | You can copy Mnesia tables records to a Khepri store using
68 | `mnesia_to_khepri:copy_tables/{2,3}`. It takes a converter module which takes
69 | care of actually processing each Mnesia records (if needed) and storing them in
70 | the Khepri store. A converter module called
71 | `mnesia_to_khepri_example_converter` is provided for common use cases and as an
72 | example.
73 |
74 | ```erlang
75 | mnesia_to_khepri:copy_all_tables(mnesia_to_khepri_example_converter).
76 | ```
77 |
78 | ## How to build
79 |
80 | ### Build
81 |
82 | ```
83 | rebar3 compile
84 | ```
85 |
86 | ### Build documentation
87 |
88 | ```
89 | rebar3 edoc
90 | ```
91 |
92 | ### Test
93 |
94 | ```
95 | rebar3 xref
96 | rebar3 eunit
97 | rebar3 ct --sname ct
98 | rebar3 as test dialyzer
99 | ```
100 |
101 | ## Copyright and License
102 |
103 | © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom" refers to
104 | Broadcom Inc. and/or its subsidiaries.
105 |
106 | This work is dual-licensed under the Apache License 2.0 and the Mozilla Public
107 | License 2.0. You can choose between one of them if you use this work.
108 |
109 | SPDX-License-Identifier: Apache-2.0 OR MPL-2.0
110 |
--------------------------------------------------------------------------------
/src/m2k_export.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(m2k_export).
12 |
13 | -include_lib("stdlib/include/assert.hrl").
14 | -include_lib("kernel/include/logger.hrl").
15 |
16 | -include("src/kmm_logging.hrl").
17 |
18 | -export([open_write/1,
19 | write/2,
20 | commit_write/1,
21 | abort_write/1]).
22 |
23 | open_write(#{table_copy_pid := _} = Args) ->
24 | State = Args#{table_to_record_name => #{}},
25 | {ok, State}.
26 |
27 | write(
28 | State,
29 | [LogHeader | Rest])
30 | when is_tuple(LogHeader) andalso
31 | element(1, LogHeader) =:= log_header andalso
32 | not is_map_key(table, State) ->
33 | ?LOG_DEBUG(
34 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] write: log_header",
35 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
36 | write(State, Rest);
37 | write(
38 | #{table_to_record_name := TableToRecordName} = State,
39 | [{schema, Table, TableInfo} | Rest]) ->
40 | State1 = case proplists:is_defined(record_name, TableInfo) of
41 | false ->
42 | ?LOG_DEBUG(
43 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] "
44 | "write: init_copy, table=~ts",
45 | [Table],
46 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
47 | State;
48 | true ->
49 | RecordName = proplists:get_value(record_name, TableInfo),
50 | ?LOG_DEBUG(
51 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] "
52 | "write: init_copy, table=~ts, record=~ts",
53 | [Table, RecordName],
54 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
55 | TableToRecordName1 = TableToRecordName#{
56 | Table => RecordName},
57 | State#{table_to_record_name => TableToRecordName1}
58 | end,
59 | write(State1, Rest);
60 | write(
61 | #{table_to_record_name := TableToRecordName,
62 | table_copy_pid := TableCopyPid} = State,
63 | [Record | Rest]) ->
64 | ?LOG_DEBUG(
65 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] write: record/~0p",
66 | [Record],
67 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
68 | Table = element(1, Record),
69 | Record1 = case TableToRecordName of
70 | #{Table := RecordName} -> setelement(1, Record, RecordName);
71 | _ -> Record
72 | end,
73 | TableCopyPid ! {?MODULE, self(), handle_record, Table, Record1},
74 | Result = receive
75 | {TableCopyPid, record_handled, Res} ->
76 | Res
77 | after
78 | 15_000 ->
79 | ?LOG_ERROR(
80 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] "
81 | "timeout: record/~0p",
82 | [Record],
83 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
84 | {error, timeout}
85 | end,
86 | case Result of
87 | ok ->
88 | write(State, Rest);
89 | Error ->
90 | Error
91 | end;
92 | write(State, []) ->
93 | {ok, State}.
94 |
95 | commit_write(State) ->
96 | ?LOG_DEBUG(
97 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] commit_write",
98 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
99 | {ok, State}.
100 |
101 | abort_write(State) ->
102 | ?LOG_DEBUG(
103 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] abort_write",
104 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
105 | {ok, State}.
106 |
--------------------------------------------------------------------------------
/src/mnesia_to_khepri_example_converter.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @doc Example converter module for use during Mnesia->Khepri copy.
10 | %%
11 | %% This converter module writes incoming Mnesia records to the Khepri store at
12 | %% the `[Table, Key]' path. However, the `Key' MUST be an atom or a binary.
13 |
14 | -module(mnesia_to_khepri_example_converter).
15 |
16 | -behaviour(mnesia_to_khepri_converter).
17 |
18 | -include_lib("kernel/include/logger.hrl").
19 |
20 | -include("src/kmm_logging.hrl").
21 |
22 | -export([init_copy_to_khepri/3,
23 | copy_to_khepri/3,
24 | delete_from_khepri/3]).
25 |
26 | -record(?MODULE, {store_id :: khepri:store_id()}).
27 |
28 | -spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when
29 | StoreId :: khepri:store_id(),
30 | MigrationId :: mnesia_to_khepri:migration_id(),
31 | Tables :: [mnesia_to_khepri:mnesia_table()],
32 | Ret :: {ok, Priv} | {error, Reason},
33 | Priv :: #?MODULE{},
34 | Reason :: any().
35 | %% @private
36 |
37 | init_copy_to_khepri(StoreId, _MigrationId, Tables) ->
38 | State = #?MODULE{store_id = StoreId},
39 | init_copy_to_khepri1(Tables, State).
40 |
41 | init_copy_to_khepri1([Table | Rest], State) ->
42 | case mnesia:table_info(Table, type) of
43 | set ->
44 | init_copy_to_khepri1(Rest, State);
45 | Type ->
46 | {error, {?MODULE, mnesia_table_type_unsupported,
47 | #{table => Table,
48 | type => Type}}}
49 | end;
50 | init_copy_to_khepri1([], State) ->
51 | {ok, State}.
52 |
53 | -spec copy_to_khepri(Table, Record, Priv) -> Ret when
54 | Table :: mnesia_to_khepri:mnesia_table(),
55 | Record :: tuple(),
56 | Priv :: #?MODULE{},
57 | Ret :: {ok, NewPriv} | {error, Reason},
58 | NewPriv :: #?MODULE{},
59 | Reason :: any().
60 | %% @private
61 |
62 | copy_to_khepri(
63 | Table, Record,
64 | #?MODULE{store_id = StoreId} = State) ->
65 | Key = element(2, Record),
66 | ?LOG_DEBUG(
67 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] key: ~0p",
68 | [Key],
69 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
70 | Supported = is_atom(Key) orelse is_binary(Key),
71 | case Supported of
72 | true ->
73 | Path = [Table, Key],
74 | ?LOG_DEBUG(
75 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] path: ~0p",
76 | [Path],
77 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
78 | case khepri:put(StoreId, Path, Record) of
79 | ok -> {ok, State};
80 | Error -> Error
81 | end;
82 | false ->
83 | {error, {?MODULE, mnesia_key_type_unsupported,
84 | #{table => Table,
85 | record => Record,
86 | key => Key}}}
87 | end.
88 |
89 | -spec delete_from_khepri(Table, Key, Priv) -> Ret when
90 | Table :: mnesia_to_khepri:mnesia_table(),
91 | Key :: any(),
92 | Priv :: #?MODULE{},
93 | Ret :: {ok, NewPriv} | {error, Reason},
94 | NewPriv :: #?MODULE{},
95 | Reason :: any().
96 | %% @private
97 |
98 | delete_from_khepri(
99 | Table, Key,
100 | #?MODULE{store_id = StoreId} = State) ->
101 | ?LOG_DEBUG(
102 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] key: ~0p",
103 | [Key],
104 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
105 | Supported = is_atom(Key) orelse is_binary(Key),
106 | case Supported of
107 | true ->
108 | Path = [Table, Key],
109 | ?LOG_DEBUG(
110 | "Mnesia->Khepri data copy: [" ?MODULE_STRING "] path: ~0p",
111 | [Path],
112 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
113 | case khepri:delete(StoreId, Path) of
114 | ok -> {ok, State};
115 | Error -> Error
116 | end;
117 | false ->
118 | {error, {?MODULE, mnesia_key_type_unsupported,
119 | #{table => Table,
120 | key => Key}}}
121 | end.
122 |
--------------------------------------------------------------------------------
/.github/workflows/test-and-release.yaml:
--------------------------------------------------------------------------------
1 | name: Test → Docs → Release
2 |
3 | on:
4 | - pull_request
5 | - push
6 |
7 | concurrency:
8 | group: ${{ github.workflow }}-${{ github.event.pull_request.number || github.ref }}
9 | cancel-in-progress: true
10 |
11 | env:
12 | REBAR_VERSION: '3.23.0'
13 | LATEST_ERLANG_VERSION: '27'
14 |
15 | jobs:
16 | # `env_to_output` works around a limitation of GitHub Actions that prevents
17 | # the use of environment variables in places such as a workflow call's `with`
18 | # arguments.
19 | #
20 | # https://github.com/actions/runner/issues/1189#issuecomment-1832389701
21 | env_to_output:
22 | name: Env. variable to outputs
23 | runs-on: ubuntu-latest
24 | outputs:
25 | REBAR_VERSION: ${{ steps.from_env.outputs.REBAR_VERSION }}
26 | steps:
27 | - id: from_env
28 | run: |
29 | vars="
30 | REBAR_VERSION
31 | "
32 | setOutput() {
33 | echo "${1}=${!1}" >> "${GITHUB_OUTPUT}"
34 | }
35 | for name in $vars; do
36 | setOutput $name
37 | done
38 |
39 | test:
40 | name: Test
41 | needs: env_to_output
42 | uses: ./.github/workflows/test-job.yaml
43 | with:
44 | rebar_version: ${{ needs.env_to_output.outputs.REBAR_VERSION }}
45 | secrets:
46 | CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }}
47 |
48 | dialyzer:
49 | name: Dialyzer
50 | runs-on: ubuntu-latest
51 | needs: env_to_output
52 |
53 | steps:
54 | - uses: actions/checkout@v4
55 | - uses: erlef/setup-beam@v1
56 | id: install-erlang
57 | with:
58 | otp-version: ${{ env.LATEST_ERLANG_VERSION }}
59 | rebar3-version: ${{ env.REBAR_VERSION }}
60 |
61 | - name: Restore Dialyzer PLT files from cache
62 | uses: actions/cache@v4
63 | with:
64 | path: _build/*/rebar3_*_plt
65 | key: dialyzer-plt-cache-${{ steps.install-erlang.outputs.otp-version }}-${{ runner.os }}-${{ hashFiles('rebar.config*') }}-v1
66 |
67 | - name: Dialyzer
68 | run: rebar3 clean && rebar3 as test dialyzer
69 |
70 | build_docs:
71 | name: Generate docs
72 | runs-on: ubuntu-latest
73 | needs:
74 | - test
75 | - dialyzer
76 |
77 | steps:
78 | - uses: actions/checkout@v4
79 | - uses: erlef/setup-beam@v1
80 | with:
81 | otp-version: ${{ env.LATEST_ERLANG_VERSION }}
82 | rebar3-version: ${{ env.REBAR_VERSION }}
83 |
84 | - name: Change doc version to "Development branch"
85 | run: sed -E -i -e 's/^@version.*/@version Development branch/' doc/overview.edoc
86 |
87 | - name: Generate
88 | run: rebar3 edoc
89 |
90 | - name: Ensure HTML files are there
91 | run: ls -l doc && test -f doc/index.html
92 |
93 | - name: Upload docs for next job
94 | uses: actions/upload-artifact@v4
95 | with:
96 | name: docs_dir
97 | path: ./doc
98 | if-no-files-found: error
99 |
100 | publish_docs:
101 | name: Publish docs
102 | runs-on: ubuntu-latest
103 | needs: build_docs
104 | if: github.repository == 'rabbitmq/khepri_mnesia_migration' && github.ref == 'refs/heads/main'
105 |
106 | steps:
107 | - name: Download docs from previous job
108 | uses: actions/download-artifact@v4
109 | with:
110 | name: docs_dir
111 | path: ./doc
112 |
113 | - name: Ensure HTML files are there
114 | run: ls -l doc && test -f doc/index.html
115 |
116 | - name: Publish
117 | uses: peaceiris/actions-gh-pages@v4
118 | with:
119 | github_token: ${{ secrets.GITHUB_TOKEN }}
120 | publish_dir: ./doc
121 |
122 | publish_release:
123 | name: Publish release
124 | runs-on: ubuntu-latest
125 | needs:
126 | - test
127 | - dialyzer
128 | - build_docs
129 | if: github.repository == 'rabbitmq/khepri_mnesia_migration' && (startsWith(github.ref, 'refs/tags/v0') || startsWith(github.ref, 'refs/tags/v1') || startsWith(github.ref, 'refs/tags/v2') || startsWith(github.ref, 'refs/tags/v3') || startsWith(github.ref, 'refs/tags/v4') || startsWith(github.ref, 'refs/tags/v5') || startsWith(github.ref, 'refs/tags/v6') || startsWith(github.ref, 'refs/tags/v7') || startsWith(github.ref, 'refs/tags/v8') || startsWith(github.ref, 'refs/tags/v9'))
130 |
131 | steps:
132 | - uses: actions/checkout@v4
133 | - uses: erlef/setup-beam@v1
134 | id: install-erlang
135 | with:
136 | otp-version: ${{ env.LATEST_ERLANG_VERSION }}
137 | rebar3-version: ${{ env.REBAR_VERSION }}
138 |
139 | - name: Publish to Hex.pm
140 | env:
141 | HEX_API_KEY: ${{ secrets.HEX_API_KEY }}
142 | run: rebar3 edoc && rebar3 hex publish -r hexpm --yes
143 |
--------------------------------------------------------------------------------
/src/m2k_subscriber.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(m2k_subscriber).
12 |
13 | -behaviour(gen_server).
14 |
15 | -include_lib("kernel/include/logger.hrl").
16 |
17 | -include("src/kmm_error.hrl").
18 | -include("src/kmm_logging.hrl").
19 |
20 | -export([start_link/1,
21 | subscribe/2,
22 | drain/1]).
23 | -export([init/1,
24 | handle_call/3,
25 | handle_cast/2,
26 | handle_info/2,
27 | terminate/2]).
28 |
29 | -record(?MODULE, {converter_mod :: mnesia_to_khepri:converter_mod(),
30 | subscribed_to = [] :: [mnesia_to_khepri:mnesia_table()],
31 | events = []}).
32 |
33 | subscribe(Pid, Tables) ->
34 | gen_server:call(Pid, {?FUNCTION_NAME, Tables}, infinity).
35 |
36 | drain(Pid) ->
37 | gen_server:call(Pid, ?FUNCTION_NAME, infinity).
38 |
39 | start_link(Args) ->
40 | gen_server:start_link(?MODULE, Args, []).
41 |
42 | %% -------------------------------------------------------------------
43 | %% `gen_server' callbacks.
44 | %% -------------------------------------------------------------------
45 |
46 | init(#{converter_mod := Mod}) ->
47 | erlang:process_flag(trap_exit, true),
48 | State = #?MODULE{converter_mod = Mod},
49 | {ok, State}.
50 |
51 | handle_call({subscribe, Tables}, _From, State) ->
52 | {Ret, State1} = do_subscribe(Tables, State),
53 | {reply, Ret, State1};
54 | handle_call(drain, _From, #?MODULE{events = Events} = State) ->
55 | State1 = State#?MODULE{events = []},
56 | State2 = do_unsubscribe(State1),
57 | %% FIXME: Empty the mailbox, it could contains Mnesia events!
58 | {reply, lists:reverse(Events), State2};
59 | handle_call(Request, _From, State) ->
60 | ?LOG_WARNING(
61 | ?MODULE_STRING ": Unhandled handle_call message: ~p",
62 | [Request],
63 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
64 | {reply, undefined, State}.
65 |
66 | handle_cast(Request, State) ->
67 | ?LOG_WARNING(
68 | ?MODULE_STRING ": Unhandled handle_cast message: ~p",
69 | [Request],
70 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
71 | {noreply, State}.
72 |
73 | handle_info(
74 | {mnesia_table_event, {write, Table, NewRecord, _, _}},
75 | #?MODULE{events = Events} = State) ->
76 | Event = {put, Table, NewRecord},
77 | Events1 = [Event | Events],
78 | State1 = State#?MODULE{events = Events1},
79 | {noreply, State1};
80 | handle_info(
81 | {mnesia_table_event, {delete, Table, {Table, Key}, _, _}},
82 | #?MODULE{events = Events} = State) ->
83 | Event = {delete, Table, Key},
84 | Events1 = [Event | Events],
85 | State1 = State#?MODULE{events = Events1},
86 | {noreply, State1};
87 | handle_info(
88 | {mnesia_table_event, {delete, Table, Record, _, _}},
89 | #?MODULE{events = Events} = State) ->
90 | Key = element(2, Record),
91 | Event = {delete, Table, Key},
92 | Events1 = [Event | Events],
93 | State1 = State#?MODULE{events = Events1},
94 | {noreply, State1};
95 | handle_info(Msg, State) ->
96 | ?LOG_WARNING(
97 | ?MODULE_STRING ": Unhandled handle_info message: ~p",
98 | [Msg],
99 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
100 | {noreply, State}.
101 |
102 | terminate(_Reason, State) ->
103 | _State1 = do_unsubscribe(State),
104 | ok.
105 |
106 | %% -------------------------------------------------------------------
107 | %% Internal functions.
108 | %% -------------------------------------------------------------------
109 |
110 | do_subscribe([Table | Rest], #?MODULE{subscribed_to = SubscribedTo} = State) ->
111 | ?LOG_DEBUG(
112 | "Mnesia->Khepri data copy: subscribe to changes to Mnesia table `~ts`",
113 | [Table],
114 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
115 |
116 | %% Ensure there is a local copy of the table we want to subscribe to.
117 | %% Otherwise, the subscribe call below will fail.
118 | _ = mnesia:add_table_copy(Table, node(), ram_copies),
119 |
120 | case mnesia:subscribe({table, Table, detailed}) of
121 | {ok, _} ->
122 | SubscribedTo1 = [Table | SubscribedTo],
123 | State1 = State#?MODULE{subscribed_to = SubscribedTo1},
124 | do_subscribe(Rest, State1);
125 | Error ->
126 | ?LOG_ERROR(
127 | "Mnesia->Khepri data copy: failed to subscribe to changes "
128 | "to Mnesia table `~ts`: ~p",
129 | [Table, Error],
130 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
131 | State1 = do_unsubscribe(State),
132 | {Error, State1}
133 | end;
134 | do_subscribe([], State) ->
135 | {ok, State}.
136 |
137 | do_unsubscribe(#?MODULE{subscribed_to = SubscribedTo} = State) ->
138 | do_unsubscribe1(SubscribedTo),
139 | State1 = State#?MODULE{subscribed_to = []},
140 | State1.
141 |
142 | do_unsubscribe1([Table | Rest]) ->
143 | ?LOG_DEBUG(
144 | "Mnesia->Khepri data copy: unsubscribe to changes to Mnesia table "
145 | "`~ts`",
146 | [Table],
147 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
148 | case mnesia:unsubscribe({table, Table, detailed}) of
149 | {ok, _} ->
150 | do_unsubscribe1(Rest);
151 | Error ->
152 | ?LOG_WARNING(
153 | "Mnesia->Khepri data copy: failed to unsubscribe to changes "
154 | "to Mnesia table `~ts`: ~p",
155 | [Table, Error],
156 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
157 | do_unsubscribe1(Rest)
158 | end;
159 | do_unsubscribe1([]) ->
160 | ok.
161 |
--------------------------------------------------------------------------------
/src/mnesia_to_khepri_converter.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2024-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @doc Behavior defining converter modules used during Mnesia->Khepri copy.
10 | %%
11 | %% All callbacks are run in the same process, so process-based features like
12 | %% the dictionary and mailbox are usable from the callback functions.
13 | %%
14 | %% Unlike what the "optional callback functions" line says above, at least
15 | %% one of {@link init_copy_to_khepri/3} or {@link init_copy_to_khepri/4} is
16 | %% required, depending on how the copy is started.
17 |
18 | -module(mnesia_to_khepri_converter).
19 |
20 | -callback init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when
21 | StoreId :: khepri:store_id(),
22 | MigrationId :: mnesia_to_khepri:migration_id(),
23 | Tables :: [mnesia_to_khepri:mnesia_table()],
24 | Ret :: {ok, Priv} | {error, Reason},
25 | Priv :: any(),
26 | Reason :: any().
27 |
28 | -callback init_copy_to_khepri(StoreId, MigrationId, Tables, Args) -> Ret when
29 | StoreId :: khepri:store_id(),
30 | MigrationId :: mnesia_to_khepri:migration_id(),
31 | Tables :: [mnesia_to_khepri:mnesia_table()],
32 | Args :: any(),
33 | Ret :: {ok, Priv} | {error, Reason},
34 | Priv :: any(),
35 | Reason :: any().
36 |
37 | -callback copy_to_khepri(Table, Record, Priv) -> Ret when
38 | Table :: mnesia_to_khepri:mnesia_table(),
39 | Record :: tuple(),
40 | Priv :: any(),
41 | Ret :: {ok, NewPriv} | {error, Reason},
42 | NewPriv :: any(),
43 | Reason :: any().
44 |
45 | -callback delete_from_khepri(Table, Key, Priv) -> Ret when
46 | Table :: mnesia_to_khepri:mnesia_table(),
47 | Key :: any(),
48 | Priv :: any(),
49 | Ret :: {ok, NewPriv} | {error, Reason},
50 | NewPriv :: any(),
51 | Reason :: any().
52 |
53 | -callback finish_copy_to_khepri(Priv) -> ok when
54 | Priv :: any().
55 |
56 | -optional_callbacks([init_copy_to_khepri/3,
57 | init_copy_to_khepri/4,
58 | finish_copy_to_khepri/1]).
59 |
60 | -export([init_copy_to_khepri/3, init_copy_to_khepri/4,
61 | copy_to_khepri/3,
62 | delete_from_khepri/3,
63 | finish_copy_to_khepri/1]).
64 |
65 | -spec init_copy_to_khepri(StoreId, MigrationId, Tables) -> Ret when
66 | StoreId :: khepri:store_id(),
67 | MigrationId :: mnesia_to_khepri:migration_id(),
68 | Tables :: [mnesia_to_khepri:mnesia_table()],
69 | Ret :: {ok, Priv},
70 | Priv :: priv.
71 | %% @doc Initializes the state of the converter module.
72 | %%
73 | %% This callback is used when a table copy is initiated like this:
74 | %% ```
75 | %% mnesia_to_khepri:copy_all_tables(
76 | %% StoreId,
77 | %% <<"my_migration">>,
78 | %% my_converter_mod).
79 | %% '''
80 | %%
81 | %% See {@link init_copy_to_khepri/4} if you want to pass arguments to the
82 | %% converter module.
83 | %%
84 | %% It it called after involved tables were marked as "being migrated" (i.e.
85 | %% copy in progress). This state is recorded in the destination Khepri store
86 | %% `StoreId'.
87 | %%
88 | %% Unlike what the "optional callback functions" line says above, at least
89 | %% one of {@link init_copy_to_khepri/3} or {@link init_copy_to_khepri/4} is
90 | %% required, depending on how the copy is started.
91 | %%
92 | %% @see init_copy_to_khepri/4.
93 |
94 | init_copy_to_khepri(_StoreId, _MigrationId, _Tables) ->
95 | {ok, priv}.
96 |
97 | -spec init_copy_to_khepri(StoreId, MigrationId, Tables, Args) -> Ret when
98 | StoreId :: khepri:store_id(),
99 | MigrationId :: mnesia_to_khepri:migration_id(),
100 | Tables :: [mnesia_to_khepri:mnesia_table()],
101 | Args :: any(),
102 | Ret :: {ok, Priv},
103 | Priv :: priv.
104 | %% @doc Initializes the state of the converter module, using `Args'.
105 | %%
106 | %% This callback is used when a table copy is initiated like this:
107 | %% ```
108 | %% mnesia_to_khepri:copy_all_tables(
109 | %% StoreId,
110 | %% <<"my_migration">>,
111 | %% {my_converter_mod, Args}).
112 | %% '''
113 | %%
114 | %% See {@link init_copy_to_khepri/3} if you don't need to pass arguments to
115 | %% the converter module.
116 | %%
117 | %% It it called after involved tables were marked as being migrated. This
118 | %% state is recorded in the destination Khepri store `StoreId'.
119 | %%
120 | %% Unlike what the "optional callback functions line says above, at least
121 | %% one of {@link init_copy_to_khepri/3} or {@link init_copy_to_khepri/4} is
122 | %% required, depending on how the copy is started.
123 | %%
124 | %% @see init_copy_to_khepri/3.
125 |
126 | init_copy_to_khepri(_StoreId, _MigrationId, _Tables, _Args) ->
127 | {ok, priv}.
128 |
129 | -spec copy_to_khepri(Table, Record, Priv) -> Ret when
130 | Table :: mnesia_to_khepri:mnesia_table(),
131 | Record :: tuple(),
132 | Priv :: priv,
133 | Ret :: {ok, Priv}.
134 | %% @doc Copies a record to a Khepri store.
135 | %%
136 | %% This callback is called for each record to copy. `Record' comes from table
137 | %% `Table' initially. It may not be called at all if there are not records to
138 | %% copy.
139 | %%
140 | %% The callback is responsible for deciding if the record should be written to
141 | %% Khepri and how. The destination Khepri store was passed to {@link
142 | %% init_copy_to_khepri/2} or {@link init_copy_to_khepri/3}.
143 |
144 | copy_to_khepri(_Table, _Record, Priv) ->
145 | {ok, Priv}.
146 |
147 | -spec delete_from_khepri(Table, Key, Priv) -> Ret when
148 | Table :: mnesia_to_khepri:mnesia_table(),
149 | Key :: any(),
150 | Priv :: priv,
151 | Ret :: {ok, Priv}.
152 | %% @doc Deletes a key from a Khepri store.
153 | %%
154 | %% This callback is called for each deletion which occurred in Mnesia
155 | %% concurrently to the copy in progress. Only the Mnesia key is available, not
156 | %% the record that was deleted. It may not be called at all if there were no
157 | %% concurrent deletions.
158 | %%
159 | %% The callback is responsible for deciding what to do in Khepri. The
160 | %% destination Khepri store was passed to {@link init_copy_to_khepri/3} or
161 | %% {@link init_copy_to_khepri/4}.
162 |
163 | delete_from_khepri(_Table, _Key, Priv) ->
164 | {ok, Priv}.
165 |
166 | -spec finish_copy_to_khepri(Priv) -> ok when
167 | Priv :: priv.
168 | %% @doc Performs any post-copy operations.
169 | %%
170 | %% It it called after involved tables were marked as "migrated" (i.e. copy is
171 | %% finished). This state is recorded in the destination Khepri store
172 | %% `StoreId'.
173 | %%
174 | %% This callback is optional.
175 |
176 | finish_copy_to_khepri(_Priv) ->
177 | ok.
178 |
--------------------------------------------------------------------------------
/doc/kmm-favicon.svg:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
207 |
--------------------------------------------------------------------------------
/doc/kmm-logo.svg:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | MNESIA
227 |
--------------------------------------------------------------------------------
/LICENSE-Apache-2.0:
--------------------------------------------------------------------------------
1 |
2 | Apache License
3 | Version 2.0, January 2004
4 | http://www.apache.org/licenses/
5 |
6 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
7 |
8 | 1. Definitions.
9 |
10 | "License" shall mean the terms and conditions for use, reproduction,
11 | and distribution as defined by Sections 1 through 9 of this document.
12 |
13 | "Licensor" shall mean the copyright owner or entity authorized by
14 | the copyright owner that is granting the License.
15 |
16 | "Legal Entity" shall mean the union of the acting entity and all
17 | other entities that control, are controlled by, or are under common
18 | control with that entity. For the purposes of this definition,
19 | "control" means (i) the power, direct or indirect, to cause the
20 | direction or management of such entity, whether by contract or
21 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
22 | outstanding shares, or (iii) beneficial ownership of such entity.
23 |
24 | "You" (or "Your") shall mean an individual or Legal Entity
25 | exercising permissions granted by this License.
26 |
27 | "Source" form shall mean the preferred form for making modifications,
28 | including but not limited to software source code, documentation
29 | source, and configuration files.
30 |
31 | "Object" form shall mean any form resulting from mechanical
32 | transformation or translation of a Source form, including but
33 | not limited to compiled object code, generated documentation,
34 | and conversions to other media types.
35 |
36 | "Work" shall mean the work of authorship, whether in Source or
37 | Object form, made available under the License, as indicated by a
38 | copyright notice that is included in or attached to the work
39 | (an example is provided in the Appendix below).
40 |
41 | "Derivative Works" shall mean any work, whether in Source or Object
42 | form, that is based on (or derived from) the Work and for which the
43 | editorial revisions, annotations, elaborations, or other modifications
44 | represent, as a whole, an original work of authorship. For the purposes
45 | of this License, Derivative Works shall not include works that remain
46 | separable from, or merely link (or bind by name) to the interfaces of,
47 | the Work and Derivative Works thereof.
48 |
49 | "Contribution" shall mean any work of authorship, including
50 | the original version of the Work and any modifications or additions
51 | to that Work or Derivative Works thereof, that is intentionally
52 | submitted to Licensor for inclusion in the Work by the copyright owner
53 | or by an individual or Legal Entity authorized to submit on behalf of
54 | the copyright owner. For the purposes of this definition, "submitted"
55 | means any form of electronic, verbal, or written communication sent
56 | to the Licensor or its representatives, including but not limited to
57 | communication on electronic mailing lists, source code control systems,
58 | and issue tracking systems that are managed by, or on behalf of, the
59 | Licensor for the purpose of discussing and improving the Work, but
60 | excluding communication that is conspicuously marked or otherwise
61 | designated in writing by the copyright owner as "Not a Contribution."
62 |
63 | "Contributor" shall mean Licensor and any individual or Legal Entity
64 | on behalf of whom a Contribution has been received by Licensor and
65 | subsequently incorporated within the Work.
66 |
67 | 2. Grant of Copyright License. Subject to the terms and conditions of
68 | this License, each Contributor hereby grants to You a perpetual,
69 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
70 | copyright license to reproduce, prepare Derivative Works of,
71 | publicly display, publicly perform, sublicense, and distribute the
72 | Work and such Derivative Works in Source or Object form.
73 |
74 | 3. Grant of Patent License. Subject to the terms and conditions of
75 | this License, each Contributor hereby grants to You a perpetual,
76 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
77 | (except as stated in this section) patent license to make, have made,
78 | use, offer to sell, sell, import, and otherwise transfer the Work,
79 | where such license applies only to those patent claims licensable
80 | by such Contributor that are necessarily infringed by their
81 | Contribution(s) alone or by combination of their Contribution(s)
82 | with the Work to which such Contribution(s) was submitted. If You
83 | institute patent litigation against any entity (including a
84 | cross-claim or counterclaim in a lawsuit) alleging that the Work
85 | or a Contribution incorporated within the Work constitutes direct
86 | or contributory patent infringement, then any patent licenses
87 | granted to You under this License for that Work shall terminate
88 | as of the date such litigation is filed.
89 |
90 | 4. Redistribution. You may reproduce and distribute copies of the
91 | Work or Derivative Works thereof in any medium, with or without
92 | modifications, and in Source or Object form, provided that You
93 | meet the following conditions:
94 |
95 | (a) You must give any other recipients of the Work or
96 | Derivative Works a copy of this License; and
97 |
98 | (b) You must cause any modified files to carry prominent notices
99 | stating that You changed the files; and
100 |
101 | (c) You must retain, in the Source form of any Derivative Works
102 | that You distribute, all copyright, patent, trademark, and
103 | attribution notices from the Source form of the Work,
104 | excluding those notices that do not pertain to any part of
105 | the Derivative Works; and
106 |
107 | (d) If the Work includes a "NOTICE" text file as part of its
108 | distribution, then any Derivative Works that You distribute must
109 | include a readable copy of the attribution notices contained
110 | within such NOTICE file, excluding those notices that do not
111 | pertain to any part of the Derivative Works, in at least one
112 | of the following places: within a NOTICE text file distributed
113 | as part of the Derivative Works; within the Source form or
114 | documentation, if provided along with the Derivative Works; or,
115 | within a display generated by the Derivative Works, if and
116 | wherever such third-party notices normally appear. The contents
117 | of the NOTICE file are for informational purposes only and
118 | do not modify the License. You may add Your own attribution
119 | notices within Derivative Works that You distribute, alongside
120 | or as an addendum to the NOTICE text from the Work, provided
121 | that such additional attribution notices cannot be construed
122 | as modifying the License.
123 |
124 | You may add Your own copyright statement to Your modifications and
125 | may provide additional or different license terms and conditions
126 | for use, reproduction, or distribution of Your modifications, or
127 | for any such Derivative Works as a whole, provided Your use,
128 | reproduction, and distribution of the Work otherwise complies with
129 | the conditions stated in this License.
130 |
131 | 5. Submission of Contributions. Unless You explicitly state otherwise,
132 | any Contribution intentionally submitted for inclusion in the Work
133 | by You to the Licensor shall be under the terms and conditions of
134 | this License, without any additional terms or conditions.
135 | Notwithstanding the above, nothing herein shall supersede or modify
136 | the terms of any separate license agreement you may have executed
137 | with Licensor regarding such Contributions.
138 |
139 | 6. Trademarks. This License does not grant permission to use the trade
140 | names, trademarks, service marks, or product names of the Licensor,
141 | except as required for reasonable and customary use in describing the
142 | origin of the Work and reproducing the content of the NOTICE file.
143 |
144 | 7. Disclaimer of Warranty. Unless required by applicable law or
145 | agreed to in writing, Licensor provides the Work (and each
146 | Contributor provides its Contributions) on an "AS IS" BASIS,
147 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
148 | implied, including, without limitation, any warranties or conditions
149 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
150 | PARTICULAR PURPOSE. You are solely responsible for determining the
151 | appropriateness of using or redistributing the Work and assume any
152 | risks associated with Your exercise of permissions under this License.
153 |
154 | 8. Limitation of Liability. In no event and under no legal theory,
155 | whether in tort (including negligence), contract, or otherwise,
156 | unless required by applicable law (such as deliberate and grossly
157 | negligent acts) or agreed to in writing, shall any Contributor be
158 | liable to You for damages, including any direct, indirect, special,
159 | incidental, or consequential damages of any character arising as a
160 | result of this License or out of the use or inability to use the
161 | Work (including but not limited to damages for loss of goodwill,
162 | work stoppage, computer failure or malfunction, or any and all
163 | other commercial damages or losses), even if such Contributor
164 | has been advised of the possibility of such damages.
165 |
166 | 9. Accepting Warranty or Additional Liability. While redistributing
167 | the Work or Derivative Works thereof, You may choose to offer,
168 | and charge a fee for, acceptance of support, warranty, indemnity,
169 | or other liability obligations and/or rights consistent with this
170 | License. However, in accepting such obligations, You may act only
171 | on Your own behalf and on Your sole responsibility, not on behalf
172 | of any other Contributor, and only if You agree to indemnify,
173 | defend, and hold each Contributor harmless for any liability
174 | incurred by, or claims asserted against, such Contributor by reason
175 | of your accepting any such warranty or additional liability.
176 |
177 | END OF TERMS AND CONDITIONS
178 |
179 | APPENDIX: How to apply the Apache License to your work.
180 |
181 | To apply the Apache License to your work, attach the following
182 | boilerplate notice, with the fields enclosed by brackets "[]"
183 | replaced with your own identifying information. (Don't include
184 | the brackets!) The text should be enclosed in the appropriate
185 | comment syntax for the file format. We also recommend that a
186 | file or class name and description of purpose be included on the
187 | same "printed page" as the copyright notice for easier
188 | identification within third-party archives.
189 |
190 | Copyright © 2021-2025 Broadcom. All Rights Reserved. The term "Broadcom" refers to Broadcom Inc. and/or its subsidiaries.
191 |
192 | Licensed under the Apache License, Version 2.0 (the "License");
193 | you may not use this file except in compliance with the License.
194 | You may obtain a copy of the License at
195 |
196 | http://www.apache.org/licenses/LICENSE-2.0
197 |
198 | Unless required by applicable law or agreed to in writing, software
199 | distributed under the License is distributed on an "AS IS" BASIS,
200 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
201 | See the License for the specific language governing permissions and
202 | limitations under the License.
203 |
--------------------------------------------------------------------------------
/test/helpers.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | -module(helpers).
10 |
11 | -include_lib("kernel/include/logger.hrl").
12 | -include_lib("stdlib/include/assert.hrl").
13 | -include_lib("common_test/include/ct.hrl").
14 |
15 | -export([init_list_of_modules_to_skip/0,
16 | start_erlang_node/1,
17 | stop_erlang_node/2,
18 | start_ra_system/1,
19 | reset_ra_system/1,
20 | stop_ra_system/1,
21 | store_dir_name/1,
22 | remove_store_dir/1,
23 | cluster_mnesia_nodes/1,
24 | mnesia_cluster_members/1,
25 | khepri_cluster_members/2,
26 | setup_node/1,
27 | basic_logger_config/0,
28 | start_epmd/0,
29 | start_n_nodes/3,
30 | stop_nodes/1,
31 | with_log/1,
32 | capture_log/1,
33 | silence_default_logger/0,
34 | restore_default_logger/1,
35 | %% For internal use only.
36 | log/2,
37 | format/2]).
38 |
39 | -define(CAPTURE_LOGGER_ID, capture_logger).
40 |
41 | init_list_of_modules_to_skip() ->
42 | _ = application:load(khepri),
43 | khepri_utils:init_list_of_modules_to_skip().
44 |
45 | start_ra_system(RaSystem) when is_atom(RaSystem) ->
46 | StoreDir = store_dir_name(RaSystem),
47 | Props = #{ra_system => RaSystem,
48 | store_dir => StoreDir},
49 | start_ra_system(Props);
50 | start_ra_system(#{ra_system := RaSystem, store_dir := StoreDir} = Props) ->
51 | _ = remove_store_dir(StoreDir),
52 | {ok, _} = application:ensure_all_started(ra),
53 | Default = ra_system:default_config(),
54 | RaSystemConfig = Default#{name => RaSystem,
55 | data_dir => StoreDir,
56 | wal_data_dir => StoreDir,
57 | wal_max_size_bytes => 16 * 1024,
58 | names => ra_system:derive_names(RaSystem)},
59 | case ra_system:start(RaSystemConfig) of
60 | {ok, RaSystemPid} ->
61 | Props#{ra_system_pid => RaSystemPid};
62 | {error, _} = Error ->
63 | throw(Error)
64 | end.
65 |
66 | reset_ra_system(Props) ->
67 | stop_ra_system(Props),
68 | start_ra_system(Props).
69 |
70 | stop_ra_system(#{ra_system := RaSystem,
71 | store_dir := StoreDir}) ->
72 | ?assertEqual(ok, supervisor:terminate_child(ra_systems_sup, RaSystem)),
73 | ?assertEqual(ok, supervisor:delete_child(ra_systems_sup, RaSystem)),
74 | _ = remove_store_dir(StoreDir),
75 | ok.
76 |
77 | store_dir_name(RaSystem) ->
78 | Node = node(),
79 | lists:flatten(
80 | io_lib:format("_test.khepri.~s.~s", [RaSystem, Node])).
81 |
82 | remove_store_dir(StoreDir) ->
83 | OnWindows = case os:type() of
84 | {win32, _} -> true;
85 | _ -> false
86 | end,
87 | case file:del_dir_r(StoreDir) of
88 | ok ->
89 | ok;
90 | {error, enoent} ->
91 | ok;
92 | {error, eexist} when OnWindows ->
93 | %% FIXME: Some files are not deleted on Windows... Are they still
94 | %% open in Ra?
95 | io:format(
96 | standard_error,
97 | "Files remaining in ~ts: ~p~n",
98 | [StoreDir, file:list_dir_all(StoreDir)]),
99 | ok;
100 | Error ->
101 | throw(Error)
102 | end.
103 |
104 | cluster_mnesia_nodes([FirstNode | _OtherNodes] = Nodes) ->
105 | ct:pal("Create Mnesia cluster"),
106 |
107 | lists:foreach(
108 | fun(Node) ->
109 | ?assertEqual(
110 | stopped,
111 | rpc:call(Node, mnesia, stop, [])),
112 | ?assertEqual(
113 | ok,
114 | rpc:call(Node, mnesia, delete_schema, [[Node]]))
115 | end, Nodes),
116 |
117 | ?assertMatch(
118 | ok,
119 | rpc:call(FirstNode, mnesia, create_schema, [Nodes])),
120 |
121 | lists:foreach(
122 | fun(Node) ->
123 | ?assertEqual(ok, rpc:call(Node, mnesia, start, []))
124 | end, Nodes),
125 |
126 | ?assertEqual(
127 | lists:sort(Nodes),
128 | mnesia_cluster_members(FirstNode)).
129 |
130 | mnesia_cluster_members(Node) ->
131 | Nodes = rpc:call(Node, mnesia, system_info, [db_nodes]),
132 | lists:sort(Nodes).
133 |
134 | khepri_cluster_members(Node, StoreId) ->
135 | {ok, Nodes} = rpc:call(Node, khepri_cluster, nodes, [StoreId]),
136 | lists:sort(Nodes).
137 |
138 | -define(LOGFMT_CONFIG, #{legacy_header => false,
139 | single_line => false,
140 | template => [time, " ", pid, ": ", msg, "\n"]}).
141 |
142 | setup_node(PrivDir) ->
143 | basic_logger_config(),
144 |
145 | %% We use an additional logger handler for messages tagged with a non-OTP
146 | %% domain because by default, `cth_log_redirect' drops them.
147 | GL = erlang:group_leader(),
148 | GLNode = node(GL),
149 | Ret = logger:add_handler(
150 | cth_log_redirect_any_domains, cth_log_redirect_any_domains,
151 | #{config => #{group_leader => GL,
152 | group_leader_node => GLNode}}),
153 | case Ret of
154 | ok -> ok;
155 | {error, {already_exist, _}} -> ok
156 | end,
157 | ok = logger:set_handler_config(
158 | cth_log_redirect_any_domains, formatter,
159 | {logger_formatter, ?LOGFMT_CONFIG}),
160 | ?LOG_INFO(
161 | "Extended logger configuration (~s):~n~p",
162 | [node(), logger:get_config()]),
163 |
164 | Node = node(),
165 | MnesiaBasename = lists:flatten(
166 | io_lib:format("_test.mnesia.~s", [Node])),
167 | MnesiaDir = filename:join(PrivDir, MnesiaBasename),
168 | ok = application:set_env(
169 | mnesia, dir, MnesiaDir, [{persistent, true}]),
170 | ok = mnesia:create_schema([Node]),
171 |
172 | ok = application:set_env(
173 | khepri, default_timeout, 5000, [{persistent, true}]),
174 |
175 | ok.
176 |
177 | basic_logger_config() ->
178 | _ = logger:set_primary_config(level, debug),
179 |
180 | HandlerIds = [HandlerId ||
181 | HandlerId <- logger:get_handler_ids(),
182 | HandlerId =:= default orelse
183 | HandlerId =:= cth_log_redirect],
184 | lists:foreach(
185 | fun(HandlerId) ->
186 | ok = logger:set_handler_config(
187 | HandlerId, formatter,
188 | {logger_formatter, ?LOGFMT_CONFIG}),
189 | _ = logger:add_handler_filter(
190 | HandlerId, progress,
191 | {fun logger_filters:progress/2,stop}),
192 | _ = logger:remove_handler_filter(
193 | HandlerId, remote_gl)
194 | end, HandlerIds),
195 | ?LOG_INFO(
196 | "Basic logger configuration (~s):~n~p",
197 | [node(), logger:get_config()]),
198 |
199 | ok.
200 |
201 | start_epmd() ->
202 | RootDir = code:root_dir(),
203 | ErtsVersion = erlang:system_info(version),
204 | ErtsDir = lists:flatten(io_lib:format("erts-~ts", [ErtsVersion])),
205 | EpmdPath0 = filename:join([RootDir, ErtsDir, "bin", "epmd"]),
206 | EpmdPath = case os:type() of
207 | {win32, _} -> EpmdPath0 ++ ".exe";
208 | _ -> EpmdPath0
209 | end,
210 | Port = erlang:open_port(
211 | {spawn_executable, EpmdPath},
212 | [{args, ["-daemon"]}]),
213 | erlang:port_close(Port),
214 | ok.
215 |
216 | start_n_nodes(Config, NamePrefix, Count) ->
217 | ct:pal("Start ~b Erlang nodes:", [Count]),
218 | Nodes = [begin
219 | Name = lists:flatten(
220 | io_lib:format(
221 | "~s-~s-~b", [?MODULE, NamePrefix, I])),
222 | ct:pal("- ~s", [Name]),
223 | start_erlang_node(Name)
224 | end || I <- lists:seq(1, Count)],
225 | ct:pal("Started nodes: ~p", [[Node || {Node, _Peer} <- Nodes]]),
226 |
227 | %% We add all nodes to the test coverage report.
228 | CoveredNodes = [Node || {Node, _Peer} <- Nodes],
229 | {ok, _} = cover:start([node() | CoveredNodes]),
230 |
231 | CodePath = code:get_path(),
232 | PrivDir = ?config(priv_dir, Config),
233 | lists:foreach(
234 | fun({Node, _Peer}) ->
235 | rpc:call(Node, code, add_pathsz, [CodePath]),
236 | ok = rpc:call(Node, ?MODULE, setup_node, [PrivDir])
237 | end, Nodes),
238 | Nodes.
239 |
240 | stop_nodes(Nodes) ->
241 | lists:foreach(
242 | fun({Node, Peer}) ->
243 | ok = cover:stop(Node),
244 | stop_erlang_node(Node, Peer)
245 | end,
246 | Nodes).
247 |
248 | -if(?OTP_RELEASE >= 25).
249 | start_erlang_node(Name) ->
250 | Name1 = list_to_atom(Name),
251 | {ok, Peer, Node} = peer:start(#{name => Name1,
252 | wait_boot => infinity}),
253 | {Node, Peer}.
254 |
255 | stop_erlang_node(_Node, Peer) ->
256 | try
257 | ok = peer:stop(Peer)
258 | catch
259 | _:noproc ->
260 | ok
261 | end.
262 | -else.
263 | start_erlang_node(Name) ->
264 | Name1 = list_to_atom(Name),
265 | Options = [{monitor_master, true}],
266 | {ok, Node} = ct_slave:start(Name1, Options),
267 | {Node, Node}.
268 |
269 | stop_erlang_node(_Node, Node) ->
270 | {ok, _} = ct_slave:stop(Node),
271 | ok.
272 | -endif.
273 |
274 | silence_default_logger() ->
275 | {ok, #{level := OldDefaultLoggerLevel}} =
276 | logger:get_handler_config(default),
277 | ok = logger:set_handler_config(default, level, none),
278 | OldDefaultLoggerLevel.
279 |
280 | restore_default_logger(OldDefaultLoggerLevel) ->
281 | ok = logger:set_handler_config(default, level, OldDefaultLoggerLevel).
282 |
283 | -spec with_log(Fun) -> {Result, Log}
284 | when
285 | Fun :: fun(() -> Result),
286 | Result :: term(),
287 | Log :: binary().
288 |
289 | %% @doc Returns the value of executing the given `Fun' and any log messages
290 | %% produced while executing it, concatenated into a binary.
291 | with_log(Fun) ->
292 | FormatterConfig = #{},
293 | HandlerConfig = #{config => self(),
294 | formatter => {?MODULE, FormatterConfig}},
295 | {ok, #{level := OldDefaultLogLevel}} = logger:get_handler_config(default),
296 | ok = logger:set_handler_config(default, level, none),
297 | ok = logger:add_handler(?CAPTURE_LOGGER_ID, ?MODULE, HandlerConfig),
298 | try
299 | Result = Fun(),
300 | Log = collect_logs(),
301 | {Result, Log}
302 | after
303 | _ = logger:remove_handler(?CAPTURE_LOGGER_ID),
304 | _ = logger:set_handler_config(default, level, OldDefaultLogLevel)
305 | end.
306 |
307 | -spec capture_log(Fun) -> Log
308 | when
309 | Fun :: fun(() -> any()),
310 | Log :: binary().
311 |
312 | %% @doc Returns the logger messages produced while executing the given `Fun'
313 | %% concatenated into a binary.
314 | capture_log(Fun) ->
315 | {_Result, Log} = with_log(Fun),
316 | Log.
317 |
318 | %% Implements the `log/2' callback for logger handlers
319 | log(LogEvent, Config) ->
320 | #{config := TestPid} = Config,
321 | Msg = case maps:get(msg, LogEvent) of
322 | {report, Report} ->
323 | {Format, Args} = logger:format_report(Report),
324 | iolist_to_binary(io_lib:format(Format, Args));
325 | {string, Chardata} ->
326 | unicode:characters_to_binary(Chardata);
327 | {Format, Args} ->
328 | iolist_to_binary(io_lib:format(Format, Args))
329 | end,
330 | TestPid ! {?MODULE, Msg},
331 | ok.
332 |
333 | %% Implements the `format/2' callback for logger formatters
334 | format(_LogEvent, _FormatConfig) ->
335 | %% No-op: print nothing to the console.
336 | ok.
337 |
338 | collect_logs() ->
339 | collect_logs(<<>>).
340 |
341 | collect_logs(Acc) ->
342 | receive
343 | {?MODULE, Msg} -> collect_logs(<>)
344 | after
345 | 50 -> Acc
346 | end.
347 |
--------------------------------------------------------------------------------
/doc/kmm-social-preview.svg:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | rabbitmq/khepri_mnesia_migration MNESIA
275 |
--------------------------------------------------------------------------------
/LICENSE-MPL-2.0:
--------------------------------------------------------------------------------
1 | Mozilla Public License Version 2.0
2 | ==================================
3 |
4 | 1. Definitions
5 | --------------
6 |
7 | 1.1. "Contributor"
8 | means each individual or legal entity that creates, contributes to
9 | the creation of, or owns Covered Software.
10 |
11 | 1.2. "Contributor Version"
12 | means the combination of the Contributions of others (if any) used
13 | by a Contributor and that particular Contributor's Contribution.
14 |
15 | 1.3. "Contribution"
16 | means Covered Software of a particular Contributor.
17 |
18 | 1.4. "Covered Software"
19 | means Source Code Form to which the initial Contributor has attached
20 | the notice in Exhibit A, the Executable Form of such Source Code
21 | Form, and Modifications of such Source Code Form, in each case
22 | including portions thereof.
23 |
24 | 1.5. "Incompatible With Secondary Licenses"
25 | means
26 |
27 | (a) that the initial Contributor has attached the notice described
28 | in Exhibit B to the Covered Software; or
29 |
30 | (b) that the Covered Software was made available under the terms of
31 | version 1.1 or earlier of the License, but not also under the
32 | terms of a Secondary License.
33 |
34 | 1.6. "Executable Form"
35 | means any form of the work other than Source Code Form.
36 |
37 | 1.7. "Larger Work"
38 | means a work that combines Covered Software with other material, in
39 | a separate file or files, that is not Covered Software.
40 |
41 | 1.8. "License"
42 | means this document.
43 |
44 | 1.9. "Licensable"
45 | means having the right to grant, to the maximum extent possible,
46 | whether at the time of the initial grant or subsequently, any and
47 | all of the rights conveyed by this License.
48 |
49 | 1.10. "Modifications"
50 | means any of the following:
51 |
52 | (a) any file in Source Code Form that results from an addition to,
53 | deletion from, or modification of the contents of Covered
54 | Software; or
55 |
56 | (b) any new file in Source Code Form that contains any Covered
57 | Software.
58 |
59 | 1.11. "Patent Claims" of a Contributor
60 | means any patent claim(s), including without limitation, method,
61 | process, and apparatus claims, in any patent Licensable by such
62 | Contributor that would be infringed, but for the grant of the
63 | License, by the making, using, selling, offering for sale, having
64 | made, import, or transfer of either its Contributions or its
65 | Contributor Version.
66 |
67 | 1.12. "Secondary License"
68 | means either the GNU General Public License, Version 2.0, the GNU
69 | Lesser General Public License, Version 2.1, the GNU Affero General
70 | Public License, Version 3.0, or any later versions of those
71 | licenses.
72 |
73 | 1.13. "Source Code Form"
74 | means the form of the work preferred for making modifications.
75 |
76 | 1.14. "You" (or "Your")
77 | means an individual or a legal entity exercising rights under this
78 | License. For legal entities, "You" includes any entity that
79 | controls, is controlled by, or is under common control with You. For
80 | purposes of this definition, "control" means (a) the power, direct
81 | or indirect, to cause the direction or management of such entity,
82 | whether by contract or otherwise, or (b) ownership of more than
83 | fifty percent (50%) of the outstanding shares or beneficial
84 | ownership of such entity.
85 |
86 | 2. License Grants and Conditions
87 | --------------------------------
88 |
89 | 2.1. Grants
90 |
91 | Each Contributor hereby grants You a world-wide, royalty-free,
92 | non-exclusive license:
93 |
94 | (a) under intellectual property rights (other than patent or trademark)
95 | Licensable by such Contributor to use, reproduce, make available,
96 | modify, display, perform, distribute, and otherwise exploit its
97 | Contributions, either on an unmodified basis, with Modifications, or
98 | as part of a Larger Work; and
99 |
100 | (b) under Patent Claims of such Contributor to make, use, sell, offer
101 | for sale, have made, import, and otherwise transfer either its
102 | Contributions or its Contributor Version.
103 |
104 | 2.2. Effective Date
105 |
106 | The licenses granted in Section 2.1 with respect to any Contribution
107 | become effective for each Contribution on the date the Contributor first
108 | distributes such Contribution.
109 |
110 | 2.3. Limitations on Grant Scope
111 |
112 | The licenses granted in this Section 2 are the only rights granted under
113 | this License. No additional rights or licenses will be implied from the
114 | distribution or licensing of Covered Software under this License.
115 | Notwithstanding Section 2.1(b) above, no patent license is granted by a
116 | Contributor:
117 |
118 | (a) for any code that a Contributor has removed from Covered Software;
119 | or
120 |
121 | (b) for infringements caused by: (i) Your and any other third party's
122 | modifications of Covered Software, or (ii) the combination of its
123 | Contributions with other software (except as part of its Contributor
124 | Version); or
125 |
126 | (c) under Patent Claims infringed by Covered Software in the absence of
127 | its Contributions.
128 |
129 | This License does not grant any rights in the trademarks, service marks,
130 | or logos of any Contributor (except as may be necessary to comply with
131 | the notice requirements in Section 3.4).
132 |
133 | 2.4. Subsequent Licenses
134 |
135 | No Contributor makes additional grants as a result of Your choice to
136 | distribute the Covered Software under a subsequent version of this
137 | License (see Section 10.2) or under the terms of a Secondary License (if
138 | permitted under the terms of Section 3.3).
139 |
140 | 2.5. Representation
141 |
142 | Each Contributor represents that the Contributor believes its
143 | Contributions are its original creation(s) or it has sufficient rights
144 | to grant the rights to its Contributions conveyed by this License.
145 |
146 | 2.6. Fair Use
147 |
148 | This License is not intended to limit any rights You have under
149 | applicable copyright doctrines of fair use, fair dealing, or other
150 | equivalents.
151 |
152 | 2.7. Conditions
153 |
154 | Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted
155 | in Section 2.1.
156 |
157 | 3. Responsibilities
158 | -------------------
159 |
160 | 3.1. Distribution of Source Form
161 |
162 | All distribution of Covered Software in Source Code Form, including any
163 | Modifications that You create or to which You contribute, must be under
164 | the terms of this License. You must inform recipients that the Source
165 | Code Form of the Covered Software is governed by the terms of this
166 | License, and how they can obtain a copy of this License. You may not
167 | attempt to alter or restrict the recipients' rights in the Source Code
168 | Form.
169 |
170 | 3.2. Distribution of Executable Form
171 |
172 | If You distribute Covered Software in Executable Form then:
173 |
174 | (a) such Covered Software must also be made available in Source Code
175 | Form, as described in Section 3.1, and You must inform recipients of
176 | the Executable Form how they can obtain a copy of such Source Code
177 | Form by reasonable means in a timely manner, at a charge no more
178 | than the cost of distribution to the recipient; and
179 |
180 | (b) You may distribute such Executable Form under the terms of this
181 | License, or sublicense it under different terms, provided that the
182 | license for the Executable Form does not attempt to limit or alter
183 | the recipients' rights in the Source Code Form under this License.
184 |
185 | 3.3. Distribution of a Larger Work
186 |
187 | You may create and distribute a Larger Work under terms of Your choice,
188 | provided that You also comply with the requirements of this License for
189 | the Covered Software. If the Larger Work is a combination of Covered
190 | Software with a work governed by one or more Secondary Licenses, and the
191 | Covered Software is not Incompatible With Secondary Licenses, this
192 | License permits You to additionally distribute such Covered Software
193 | under the terms of such Secondary License(s), so that the recipient of
194 | the Larger Work may, at their option, further distribute the Covered
195 | Software under the terms of either this License or such Secondary
196 | License(s).
197 |
198 | 3.4. Notices
199 |
200 | You may not remove or alter the substance of any license notices
201 | (including copyright notices, patent notices, disclaimers of warranty,
202 | or limitations of liability) contained within the Source Code Form of
203 | the Covered Software, except that You may alter any license notices to
204 | the extent required to remedy known factual inaccuracies.
205 |
206 | 3.5. Application of Additional Terms
207 |
208 | You may choose to offer, and to charge a fee for, warranty, support,
209 | indemnity or liability obligations to one or more recipients of Covered
210 | Software. However, You may do so only on Your own behalf, and not on
211 | behalf of any Contributor. You must make it absolutely clear that any
212 | such warranty, support, indemnity, or liability obligation is offered by
213 | You alone, and You hereby agree to indemnify every Contributor for any
214 | liability incurred by such Contributor as a result of warranty, support,
215 | indemnity or liability terms You offer. You may include additional
216 | disclaimers of warranty and limitations of liability specific to any
217 | jurisdiction.
218 |
219 | 4. Inability to Comply Due to Statute or Regulation
220 | ---------------------------------------------------
221 |
222 | If it is impossible for You to comply with any of the terms of this
223 | License with respect to some or all of the Covered Software due to
224 | statute, judicial order, or regulation then You must: (a) comply with
225 | the terms of this License to the maximum extent possible; and (b)
226 | describe the limitations and the code they affect. Such description must
227 | be placed in a text file included with all distributions of the Covered
228 | Software under this License. Except to the extent prohibited by statute
229 | or regulation, such description must be sufficiently detailed for a
230 | recipient of ordinary skill to be able to understand it.
231 |
232 | 5. Termination
233 | --------------
234 |
235 | 5.1. The rights granted under this License will terminate automatically
236 | if You fail to comply with any of its terms. However, if You become
237 | compliant, then the rights granted under this License from a particular
238 | Contributor are reinstated (a) provisionally, unless and until such
239 | Contributor explicitly and finally terminates Your grants, and (b) on an
240 | ongoing basis, if such Contributor fails to notify You of the
241 | non-compliance by some reasonable means prior to 60 days after You have
242 | come back into compliance. Moreover, Your grants from a particular
243 | Contributor are reinstated on an ongoing basis if such Contributor
244 | notifies You of the non-compliance by some reasonable means, this is the
245 | first time You have received notice of non-compliance with this License
246 | from such Contributor, and You become compliant prior to 30 days after
247 | Your receipt of the notice.
248 |
249 | 5.2. If You initiate litigation against any entity by asserting a patent
250 | infringement claim (excluding declaratory judgment actions,
251 | counter-claims, and cross-claims) alleging that a Contributor Version
252 | directly or indirectly infringes any patent, then the rights granted to
253 | You by any and all Contributors for the Covered Software under Section
254 | 2.1 of this License shall terminate.
255 |
256 | 5.3. In the event of termination under Sections 5.1 or 5.2 above, all
257 | end user license agreements (excluding distributors and resellers) which
258 | have been validly granted by You or Your distributors under this License
259 | prior to termination shall survive termination.
260 |
261 | ************************************************************************
262 | * *
263 | * 6. Disclaimer of Warranty *
264 | * ------------------------- *
265 | * *
266 | * Covered Software is provided under this License on an "as is" *
267 | * basis, without warranty of any kind, either expressed, implied, or *
268 | * statutory, including, without limitation, warranties that the *
269 | * Covered Software is free of defects, merchantable, fit for a *
270 | * particular purpose or non-infringing. The entire risk as to the *
271 | * quality and performance of the Covered Software is with You. *
272 | * Should any Covered Software prove defective in any respect, You *
273 | * (not any Contributor) assume the cost of any necessary servicing, *
274 | * repair, or correction. This disclaimer of warranty constitutes an *
275 | * essential part of this License. No use of any Covered Software is *
276 | * authorized under this License except under this disclaimer. *
277 | * *
278 | ************************************************************************
279 |
280 | ************************************************************************
281 | * *
282 | * 7. Limitation of Liability *
283 | * -------------------------- *
284 | * *
285 | * Under no circumstances and under no legal theory, whether tort *
286 | * (including negligence), contract, or otherwise, shall any *
287 | * Contributor, or anyone who distributes Covered Software as *
288 | * permitted above, be liable to You for any direct, indirect, *
289 | * special, incidental, or consequential damages of any character *
290 | * including, without limitation, damages for lost profits, loss of *
291 | * goodwill, work stoppage, computer failure or malfunction, or any *
292 | * and all other commercial damages or losses, even if such party *
293 | * shall have been informed of the possibility of such damages. This *
294 | * limitation of liability shall not apply to liability for death or *
295 | * personal injury resulting from such party's negligence to the *
296 | * extent applicable law prohibits such limitation. Some *
297 | * jurisdictions do not allow the exclusion or limitation of *
298 | * incidental or consequential damages, so this exclusion and *
299 | * limitation may not apply to You. *
300 | * *
301 | ************************************************************************
302 |
303 | 8. Litigation
304 | -------------
305 |
306 | Any litigation relating to this License may be brought only in the
307 | courts of a jurisdiction where the defendant maintains its principal
308 | place of business and such litigation shall be governed by laws of that
309 | jurisdiction, without reference to its conflict-of-law provisions.
310 | Nothing in this Section shall prevent a party's ability to bring
311 | cross-claims or counter-claims.
312 |
313 | 9. Miscellaneous
314 | ----------------
315 |
316 | This License represents the complete agreement concerning the subject
317 | matter hereof. If any provision of this License is held to be
318 | unenforceable, such provision shall be reformed only to the extent
319 | necessary to make it enforceable. Any law or regulation which provides
320 | that the language of a contract shall be construed against the drafter
321 | shall not be used to construe this License against a Contributor.
322 |
323 | 10. Versions of the License
324 | ---------------------------
325 |
326 | 10.1. New Versions
327 |
328 | Mozilla Foundation is the license steward. Except as provided in Section
329 | 10.3, no one other than the license steward has the right to modify or
330 | publish new versions of this License. Each version will be given a
331 | distinguishing version number.
332 |
333 | 10.2. Effect of New Versions
334 |
335 | You may distribute the Covered Software under the terms of the version
336 | of the License under which You originally received the Covered Software,
337 | or under the terms of any subsequent version published by the license
338 | steward.
339 |
340 | 10.3. Modified Versions
341 |
342 | If you create software not governed by this License, and you want to
343 | create a new license for such software, you may create and use a
344 | modified version of this License if you rename the license and remove
345 | any references to the name of the license steward (except to note that
346 | such modified license differs from this License).
347 |
348 | 10.4. Distributing Source Code Form that is Incompatible With Secondary
349 | Licenses
350 |
351 | If You choose to distribute Source Code Form that is Incompatible With
352 | Secondary Licenses under the terms of this version of the License, the
353 | notice described in Exhibit B of this License must be attached.
354 |
355 | Exhibit A - Source Code Form License Notice
356 | -------------------------------------------
357 |
358 | This Source Code Form is subject to the terms of the Mozilla Public
359 | License, v. 2.0. If a copy of the MPL was not distributed with this
360 | file, You can obtain one at http://mozilla.org/MPL/2.0/.
361 |
362 | If it is not possible or desirable to put the notice in a particular
363 | file, then You may include the notice in a location (such as a LICENSE
364 | file in a relevant directory) where a recipient would be likely to look
365 | for such a notice.
366 |
367 | You may add additional accurate notices of copyright ownership.
368 |
369 | Exhibit B - "Incompatible With Secondary Licenses" Notice
370 | ---------------------------------------------------------
371 |
372 | This Source Code Form is "Incompatible With Secondary Licenses", as
373 | defined by the Mozilla Public License, v. 2.0.
374 |
--------------------------------------------------------------------------------
/src/m2k_cluster_sync.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(m2k_cluster_sync).
12 |
13 | -behaviour(gen_server).
14 |
15 | -include_lib("kernel/include/logger.hrl").
16 | -include_lib("stdlib/include/assert.hrl").
17 | -include_lib("khepri/include/khepri.hrl").
18 |
19 | -include("src/kmm_error.hrl").
20 | -include("src/kmm_logging.hrl").
21 |
22 | -export([start_link/1,
23 | proceed/1]).
24 | -export([init/1,
25 | handle_call/3,
26 | handle_cast/2,
27 | handle_info/2,
28 | terminate/2]).
29 |
30 | -ifdef(TEST).
31 | -export([sort_khepri_clusters/2]).
32 | -endif.
33 |
34 | -record(?MODULE, {khepri_store}).
35 |
36 | proceed(Pid) ->
37 | case gen_server:call(Pid, ?FUNCTION_NAME, infinity) of
38 | {exception, ?kmm_exception(_, _) = Exception} ->
39 | ?kmm_misuse(Exception);
40 | Ret ->
41 | Ret
42 | end.
43 |
44 | start_link(Args) ->
45 | gen_server:start_link(?MODULE, Args, []).
46 |
47 | %% -------------------------------------------------------------------
48 | %% `gen_server' callbacks.
49 | %% -------------------------------------------------------------------
50 |
51 | init(#{khepri_store := StoreId}) ->
52 | erlang:process_flag(trap_exit, true),
53 | State = #?MODULE{khepri_store = StoreId},
54 | {ok, State}.
55 |
56 | handle_call(proceed, _From, State) ->
57 | Ret = try
58 | do_sync_cluster(State)
59 | catch
60 | throw:?kmm_error(_, _) = Reason ->
61 | ?LOG_ERROR(
62 | "Failed to synchronize Mnesia->Khepri clusters: ~0p",
63 | [Reason],
64 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
65 | {error, Reason};
66 | error:?kmm_exception(_, _) = Exception ->
67 | ?LOG_ERROR(
68 | "Exception during Mnesia->Khepri clusters sync: ~0p",
69 | [Exception],
70 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
71 | {exception, Exception}
72 | end,
73 | {reply, Ret, State};
74 | handle_call(Request, _From, State) ->
75 | ?LOG_WARNING(
76 | ?MODULE_STRING ": Unhandled handle_call message: ~p",
77 | [Request],
78 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
79 | {reply, undefined, State}.
80 |
81 | handle_cast(Request, State) ->
82 | ?LOG_WARNING(
83 | ?MODULE_STRING ": Unhandled handle_cast message: ~p",
84 | [Request],
85 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
86 | {noreply, State}.
87 |
88 | handle_info(Msg, State) ->
89 | ?LOG_WARNING(
90 | ?MODULE_STRING ": Unhandled handle_info message: ~p",
91 | [Msg],
92 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
93 | {noreply, State}.
94 |
95 | terminate(_Reason, _State) ->
96 | ok.
97 |
98 | %% -------------------------------------------------------------------
99 | %% Internal functions.
100 | %% -------------------------------------------------------------------
101 |
102 | do_sync_cluster(#?MODULE{khepri_store = StoreId} = State) ->
103 | Lock = {{?MODULE, StoreId}, self()},
104 | global:set_lock(Lock),
105 | try
106 | do_sync_cluster_locked(State)
107 | after
108 | global:del_lock(Lock)
109 | end.
110 |
111 | do_sync_cluster_locked(#?MODULE{khepri_store = StoreId}) ->
112 | ?LOG_INFO(
113 | "Syncing Mnesia->Khepri clusters membership",
114 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
115 |
116 | MnesiaCluster = kmm_utils:mnesia_nodes(),
117 | ?LOG_DEBUG(
118 | "Mnesia->Khepri cluster sync: Mnesia cluster: ~0p",
119 | [MnesiaCluster],
120 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
121 |
122 | NodesToConsider = case MnesiaCluster of
123 | [SingleNode] ->
124 | %% If the node is unclustered according to
125 | %% Mnesia, we consider connected nodes that run
126 | %% the Khepri store already and have this node
127 | %% among the cluster members they know about.
128 | %%
129 | %% This allows to repair a cluster where a node
130 | %% lost its disk for instance. In ths situation,
131 | %% Mnesia thinks it's unclustered. Khepri on
132 | %% other nodes will think this lost node is
133 | %% already clustered though.
134 | %%
135 | %% See `find_largest_khepri_cluster/2' for the
136 | %% rest of the logic.
137 | PossibleNodes = list_possible_nodes(StoreId),
138 | ?LOG_DEBUG(
139 | "Mnesia->Khepri cluster sync: "
140 | "Connected nodes to consider: ~0p",
141 | [PossibleNodes],
142 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
143 | [SingleNode | PossibleNodes];
144 | _ ->
145 | MnesiaCluster
146 | end,
147 |
148 | LargestKhepriCluster = find_largest_khepri_cluster(
149 | NodesToConsider, StoreId),
150 | ?LOG_DEBUG(
151 | "Mnesia->Khepri cluster sync: Largest Khepri cluster: ~0p",
152 | [LargestKhepriCluster],
153 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
154 |
155 | NodesToAdd = MnesiaCluster -- LargestKhepriCluster,
156 | ?LOG_DEBUG(
157 | "Mnesia->Khepri cluster sync: Khepri nodes joining the largest "
158 | "Khepri cluster: ~0p",
159 | [NodesToAdd],
160 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
161 |
162 | add_nodes_to_khepri_cluster(NodesToAdd, LargestKhepriCluster, StoreId),
163 |
164 | KhepriCluster = khepri_cluster_on_node(hd(LargestKhepriCluster), StoreId),
165 | NodesToRemove = KhepriCluster -- NodesToConsider,
166 | ?LOG_DEBUG(
167 | "Mnesia->Khepri cluster sync: Khepri nodes being removed from the "
168 | "expanded Khepri cluster: ~0p",
169 | [NodesToRemove],
170 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
171 |
172 | remove_nodes_from_khepri_cluster(NodesToRemove, StoreId).
173 |
174 | list_possible_nodes(StoreId) ->
175 | %% To detect if this node needs to be added back to an existing cluster, we
176 | %% check all connected Erlang nodes against the following conditions:
177 | %% 1. A Khepri store named `StoreId' must be running
178 | %% 2. That Khepri store must think that this node is part of the cluster
179 | ConnectedNodes = nodes(),
180 | ThisMember = khepri_cluster:this_member(StoreId),
181 | lists:filter(
182 | fun(Node) ->
183 | try
184 | IsKhepriRunning = erpc:call(
185 | Node, khepri_cluster, is_store_running,
186 | [StoreId]),
187 | case IsKhepriRunning of
188 | true ->
189 | case locally_known_members(Node, StoreId) of
190 | {ok, Members} ->
191 | lists:member(ThisMember, Members);
192 | _ ->
193 | false
194 | end;
195 | false ->
196 | false
197 | end
198 | catch
199 | _:_ ->
200 | false
201 | end
202 | end, ConnectedNodes).
203 |
204 | locally_known_members(Node, StoreId) ->
205 | %% Khepri 0.17.0 replaces `khepri_cluster:locally_known_members/1' (Khepri
206 | %% 0.16.0 and lower) with `khepri_cluster:members/2' and the `favor'
207 | %% option set to `low_latency'.
208 | %%
209 | %% Prior to 0.17.0, `khepri_cluster:members/2' did exist but took
210 | %% different arguments. Thus exist (only `khepri_cluster:members/1') so we
211 | %% can use a `function_clause' exception to detect when the remote member
212 | %% is running Khepri 0.16.0 or lower.
213 | try
214 | erpc:call(
215 | Node, khepri_cluster, members, [StoreId, #{favor => low_latency}])
216 | catch
217 | error:{exception, function_clause, _} ->
218 | erpc:call(
219 | Node, khepri_cluster, locally_known_members, [StoreId])
220 | end.
221 |
222 | find_largest_khepri_cluster(Nodes, StoreId) ->
223 | KhepriClusters0 = list_all_khepri_clusters(Nodes, StoreId),
224 | KhepriClusters1 = remove_khepri_nodes_not_in_mnesia_cluster(
225 | Nodes, KhepriClusters0),
226 | KhepriClusters2 = discard_nodes_who_lost_their_data(KhepriClusters1),
227 | SortedKhepriClusters = sort_khepri_clusters(KhepriClusters2, StoreId),
228 | ?LOG_DEBUG(
229 | "Mnesia->Khepri cluster sync: Khepri clusters: ~0p",
230 | [SortedKhepriClusters],
231 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
232 | LargestKhepriCluster = hd(SortedKhepriClusters),
233 | LargestKhepriCluster.
234 |
235 | list_all_khepri_clusters(Nodes, StoreId) ->
236 | KhepriClusters = lists:foldl(
237 | fun(Node, Acc) ->
238 | Cluster = khepri_cluster_on_node(Node, StoreId),
239 | Acc#{Cluster => true}
240 | end, #{}, Nodes),
241 | maps:keys(KhepriClusters).
242 |
243 | khepri_cluster_on_node(Node, StoreId) ->
244 | case rpc:call(Node, khepri_cluster, nodes, [StoreId]) of
245 | {ok, AllNodes} when is_list(AllNodes) andalso AllNodes =/= [] ->
246 | SortedNodes = lists:sort(AllNodes),
247 | SortedNodes;
248 | {error, noproc} ->
249 | ?kmm_misuse(
250 | khepri_store_must_run,
251 | #{node => Node,
252 | store_id => StoreId});
253 | {error, _Reason} = Error ->
254 | throw(
255 | ?kmm_error(
256 | failed_to_query_khepri_nodes,
257 | #{node => Node,
258 | store_id => StoreId,
259 | error => Error}))
260 | end.
261 |
262 | remove_khepri_nodes_not_in_mnesia_cluster(MnesiaCluster, KhepriClusters) ->
263 | %% We need to leave Khepri nodes that are not part of the Mnesia cluster
264 | %% alone.
265 | lists:filtermap(
266 | fun(KhepriCluster) ->
267 | KhepriCluster1 = remove_khepri_nodes_not_in_mnesia_cluster1(
268 | MnesiaCluster, KhepriCluster),
269 | case KhepriCluster1 of
270 | [] -> false;
271 | _ -> {true, KhepriCluster1}
272 | end
273 | end, KhepriClusters).
274 |
275 | remove_khepri_nodes_not_in_mnesia_cluster1(MnesiaCluster, KhepriCluster) ->
276 | lists:filter(
277 | fun(KhepriNode) ->
278 | lists:member(KhepriNode, MnesiaCluster)
279 | end, KhepriCluster).
280 |
281 | discard_nodes_who_lost_their_data(KhepriClusters) ->
282 | discard_nodes_who_lost_their_data(KhepriClusters, KhepriClusters, []).
283 |
284 | discard_nodes_who_lost_their_data(
285 | [[SingleNode] | Rest],
286 | KhepriClusters,
287 | LostNodes) ->
288 | %% We check if a standalore node is also a member of another cluster. It
289 | %% means the standalore node lost its state and no longer knows that it is
290 | %% already clustered. Other members consider that it is already clustered
291 | %% and don't know the node lost its state.
292 | %%
293 | %% If we find such a node, we discard it from the list of Khepri clusters
294 | %% and delete if from the other clusters. This way, the rest of the logic
295 | %% will consider that the lost node is unclustered.
296 | IsMemberElsewhere = lists:any(
297 | fun
298 | (KhepriCluster)
299 | when length(KhepriCluster) =:= 1 ->
300 | false;
301 | (KhepriCluster) ->
302 | lists:member(SingleNode, KhepriCluster)
303 | end, KhepriClusters),
304 | LostNodes1 = case IsMemberElsewhere of
305 | false -> LostNodes;
306 | true -> [SingleNode | LostNodes]
307 | end,
308 | discard_nodes_who_lost_their_data(Rest, KhepriClusters, LostNodes1);
309 | discard_nodes_who_lost_their_data(
310 | [_KhepriCluster | Rest],
311 | KhepriClusters,
312 | LostNodes) ->
313 | discard_nodes_who_lost_their_data(Rest, KhepriClusters, LostNodes);
314 | discard_nodes_who_lost_their_data([], KhepriClusters, []) ->
315 | KhepriClusters;
316 | discard_nodes_who_lost_their_data([], KhepriClusters, LostNodes) ->
317 | ?LOG_DEBUG(
318 | "Mnesia->Khepri cluster sync: "
319 | "Nodes who might have lost their data; "
320 | "they will be considered unclustered: ~0p",
321 | [lists:sort(LostNodes)],
322 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN}),
323 | lists:filtermap(
324 | fun(KhepriCluster) ->
325 | KhepriCluster1 = KhepriCluster -- LostNodes,
326 | case KhepriCluster1 of
327 | [] -> false;
328 | _ -> {true, KhepriCluster1}
329 | end
330 | end, KhepriClusters).
331 |
332 | -define(KHEPRI_MACHINE_VERSIONS_KEY, kmm_khepri_machine_versions).
333 | -define(TREE_NODES_COUNTS_KEY, kmm_tree_nodes_counts).
334 | -define(ERLANG_NODES_UPTIMES_KEY, kmm_erlang_node_uptimes).
335 |
336 | sort_khepri_clusters(KhepriClusters, StoreId) ->
337 | _ = erlang:put(?KHEPRI_MACHINE_VERSIONS_KEY, #{}),
338 | _ = erlang:put(?TREE_NODES_COUNTS_KEY, #{}),
339 | _ = erlang:put(?ERLANG_NODES_UPTIMES_KEY, #{}),
340 | SortedNodes = do_sort_khepri_clusters(KhepriClusters, StoreId),
341 | _ = erlang:erase(?ERLANG_NODES_UPTIMES_KEY),
342 | _ = erlang:erase(?TREE_NODES_COUNTS_KEY),
343 | _ = erlang:erase(?KHEPRI_MACHINE_VERSIONS_KEY),
344 | SortedNodes.
345 |
346 | do_sort_khepri_clusters(KhepriClusters, StoreId) ->
347 | Criterias = [fun compare_members_counts/3,
348 | fun compare_khepri_machine_versions/3,
349 | fun compare_tree_nodes_counts/3,
350 | fun compare_erlang_node_uptimes/3,
351 | fun compare_erlang_node_names/3],
352 | lists:sort(
353 | fun(A, B) ->
354 | Sort = lists:foldl(
355 | fun
356 | (Criteria, undefined) -> Criteria(A, B, StoreId);
357 | (_Criteria, Result) -> Result
358 | end, undefined, Criterias),
359 | ?assertNotEqual(undefined, Sort),
360 | Sort
361 | end,
362 | KhepriClusters).
363 |
364 | compare_members_counts(A, B, _StoreId) ->
365 | AMembersCount = length(A),
366 | BMembersCount = length(B),
367 | if
368 | AMembersCount =:= BMembersCount -> undefined;
369 | true -> length(A) > length(B)
370 | end.
371 |
372 | compare_khepri_machine_versions(A, B, _StoreId) ->
373 | AMacVer = get_khepri_machine_versions(A),
374 | BMacVer = get_khepri_machine_versions(B),
375 | if
376 | AMacVer =:= BMacVer -> undefined;
377 | true -> AMacVer < BMacVer
378 | end.
379 |
380 | compare_tree_nodes_counts(A, B, StoreId) ->
381 | ANodesCount = get_tree_nodes_count(A, StoreId),
382 | BNodesCount = get_tree_nodes_count(B, StoreId),
383 | if
384 | ANodesCount =:= BNodesCount -> undefined;
385 | true -> ANodesCount > BNodesCount
386 | end.
387 |
388 | compare_erlang_node_uptimes(A, B, _StoreId) ->
389 | ALongestUptime = get_longest_erlang_node_uptime(A),
390 | BLongestUptime = get_longest_erlang_node_uptime(B),
391 | if
392 | ALongestUptime =:= BLongestUptime -> undefined;
393 | true -> ALongestUptime > BLongestUptime
394 | end.
395 |
396 | compare_erlang_node_names(A, B, _StoreId) ->
397 | A =< B.
398 |
399 | get_khepri_machine_versions(Nodes) ->
400 | KhepriMacVers = erlang:get(?KHEPRI_MACHINE_VERSIONS_KEY),
401 | case KhepriMacVers of
402 | #{Nodes := KhepriMacVer} ->
403 | KhepriMacVer;
404 | _ ->
405 | Rets = erpc:multicall(Nodes, khepri_machine, version, []),
406 | MacVers = lists:map(
407 | fun
408 | ({ok, MacVer}) ->
409 | MacVer;
410 | (_Error) ->
411 | ?kmm_misuse(
412 | failed_to_query_khepri_machine_versions,
413 | #{nodes => Nodes,
414 | returns => Rets})
415 | end, Rets),
416 | MacVer = lists:min(MacVers),
417 | KhepriMacVers1 = KhepriMacVers#{Nodes => MacVer},
418 | _ = erlang:put(?KHEPRI_MACHINE_VERSIONS_KEY, KhepriMacVers1),
419 | MacVer
420 | end.
421 |
422 | get_tree_nodes_count(Nodes, StoreId) ->
423 | TreeNodesCounts = erlang:get(?TREE_NODES_COUNTS_KEY),
424 | case TreeNodesCounts of
425 | #{Nodes := TreeNodesCount} ->
426 | TreeNodesCount;
427 | _ ->
428 | Node = hd(Nodes),
429 | Ret = rpc:call(Node, khepri, count, [StoreId, "**"]),
430 | case Ret of
431 | {ok, TreeNodesCount} ->
432 | TreeNodesCounts1 = TreeNodesCounts#{
433 | Nodes => TreeNodesCount},
434 | _ = erlang:put(?TREE_NODES_COUNTS_KEY, TreeNodesCounts1),
435 | TreeNodesCount;
436 | Error ->
437 | throw(
438 | ?kmm_error(
439 | failed_to_query_khepri_tree_nodes_count,
440 | #{node => Node,
441 | store_id => StoreId,
442 | error => Error}))
443 | end
444 | end.
445 |
446 | get_longest_erlang_node_uptime(Nodes) ->
447 | NodeUptimes = erlang:get(?ERLANG_NODES_UPTIMES_KEY),
448 | case NodeUptimes of
449 | #{Nodes := Uptime} ->
450 | Uptime;
451 | _ ->
452 | Rets = erpc:multicall(Nodes, kmm_utils, erlang_node_uptime, []),
453 | Uptimes = lists:map(
454 | fun
455 | ({ok, Uptime}) ->
456 | Uptime;
457 | (_Error) ->
458 | ?kmm_misuse(
459 | failed_to_query_erlang_node_uptimes,
460 | #{nodes => Nodes,
461 | returns => Rets})
462 | end, Rets),
463 | Uptime = lists:max(Uptimes),
464 | NodeUptimes1 = NodeUptimes#{Nodes => Uptime},
465 | _ = erlang:put(?ERLANG_NODES_UPTIMES_KEY, NodeUptimes1),
466 | Uptime
467 | end.
468 |
469 | add_nodes_to_khepri_cluster([Node | Rest], KhepriCluster, StoreId) ->
470 | case lists:member(Node, KhepriCluster) of
471 | false ->
472 | ClusteredNode = hd(KhepriCluster),
473 | Ret = rpc:call(
474 | Node,
475 | khepri_cluster, join, [StoreId, ClusteredNode]),
476 | case Ret of
477 | ok ->
478 | add_nodes_to_khepri_cluster(Rest, KhepriCluster, StoreId);
479 | Error ->
480 | throw(
481 | ?kmm_error(
482 | failed_to_cluster_khepri_node,
483 | #{node => Node,
484 | khepri_cluster => KhepriCluster,
485 | error => Error}))
486 | end;
487 | true ->
488 | add_nodes_to_khepri_cluster(Rest, KhepriCluster, StoreId)
489 | end;
490 | add_nodes_to_khepri_cluster([], _KhepriCluster, _StoreId) ->
491 | ok.
492 |
493 | remove_nodes_from_khepri_cluster([Node | Rest], StoreId) ->
494 | try
495 | case erpc:call(Node, khepri_cluster, reset, [StoreId]) of
496 | ok ->
497 | remove_nodes_from_khepri_cluster(Rest, StoreId);
498 | {error, ?khepri_error(not_a_khepri_store, _)} ->
499 | ?LOG_DEBUG(
500 | "Mnesia->Khepri cluster sync: Node ~0p does not run the "
501 | "Khepri store, skipping its removal from the Khepri "
502 | "cluster",
503 | [Node],
504 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN});
505 | Error ->
506 | throw(
507 | ?kmm_error(
508 | failed_to_reset_khepri_node,
509 | #{node => Node,
510 | error => Error}))
511 | end
512 | catch
513 | error:{erpc, noconnection} ->
514 | ?LOG_DEBUG(
515 | "Mnesia->Khepri cluster sync: Node ~0p unreachable, "
516 | "skipping its removal from the Khepri cluster",
517 | [Node],
518 | #{domain => ?KMM_M2K_CLUSTER_SYNC_LOG_DOMAIN})
519 | end;
520 | remove_nodes_from_khepri_cluster([], _StoreId) ->
521 | ok.
522 |
--------------------------------------------------------------------------------
/src/mnesia_to_khepri.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @doc Tools to migrate from Mnesia to Khepri.
10 | %%
11 | %% The migration from Mnesia to Khepri implemented in this module is divided in
12 | %% two distinct parts:
13 | %%
14 | %% the cluster membership
15 | %% the tables content
16 | %%
17 | %%
18 | %% Both parts can be used independently.
19 | %%
20 | %% == Cluster membership synchronization ==
21 | %%
22 | %% For the first part, {@link sync_cluster_membership/0} and {@link
23 | %% sync_cluster_membership/1} ensure the default/specified Khepri store has the
24 | %% same cluster members as Mnesia.
25 | %%
26 | %% All "instances" of the Khepri store on unclustered nodes will be reset
27 | %% except one. The selected surviving Khepri store is determined using several
28 | %% heuristics which are explained in {@link sync_cluster_membership/1}.
29 | %%
30 | %% == Tables copy ==
31 | %%
32 | %% For the second part, {@link copy_tables/3}, {@link copy_tables/4}, {@link
33 | %% copy_all_tables/2} and {@link copy_all_tables/3} take care of copying
34 | %% records to the designated Khepri store.
35 | %%
36 | %% The functions take a converter module which is responsible for actually
37 | %% writing each record to wherever it wants in the Khepri store. This allows
38 | %% the caller to filter and convert the records. The converter module
39 | %% interface is defined by the {@link mnesia_to_khepri_converter} behavior.
40 | %% There is an example converter module called {@link
41 | %% mnesia_to_khepri_example_converter} provided.
42 | %%
43 | %% The copy works while Mnesia tables are still being used and updated. To
44 | %% allow that, the copy follows several stages which are explained in {@link
45 | %% copy_tables/4}.
46 | %%
47 | %% The functions {@link is_migration_finished/1}, {@link
48 | %% is_migration_finished/2}, {@link wait_for_migration/2} and {@link
49 | %% wait_for_migration/3} can be used to follow an on-going copy of the tables.
50 | %%
51 | %% Finally to help with the use of the Mnesia tables and Khepri store
52 | %% concurrently with a tables copy, you can use {@link handle_fallback/3} and
53 | %% {@link handle_fallback/4}.
54 |
55 | -module(mnesia_to_khepri).
56 |
57 | -include_lib("kernel/include/logger.hrl").
58 |
59 | -include("src/kmm_error.hrl").
60 | -include("src/kmm_logging.hrl").
61 |
62 | -export([sync_cluster_membership/0, sync_cluster_membership/1,
63 | copy_tables/3, copy_tables/4,
64 | copy_all_tables/2, copy_all_tables/3,
65 | is_migration_finished/1, is_migration_finished/2,
66 | wait_for_migration/2, wait_for_migration/3,
67 | cleanup_after_table_copy/1, cleanup_after_table_copy/2,
68 | rollback_table_copy/1, rollback_table_copy/2,
69 | handle_fallback/3, handle_fallback/4]).
70 |
71 | -type migration_id() :: binary().
72 | %% MigrationId of a migration.
73 | %%
74 | %% This is used to semantically identify a migration that covers a set of
75 | %% Mnesia tables and an associated converter module.
76 | %%
77 | %% A migration is idempotent, based on this identifier. In other words, a
78 | %% migration identifier by this identifier can happen only once, and there
79 | %% can't be concurrent migration processes with that same identifier.
80 |
81 | -type mnesia_table() :: atom().
82 | %% The name of a Mnesia table.
83 | %%
84 | %% This is the same type as `mnesia:table()' which is not exported
85 | %% unfortunately.
86 |
87 | -type converter_mod() :: module() | {module(), any()}.
88 | %% A converter module, possibly with a private term to initliaze it.
89 | %%
90 | %% A converter module is a module implementing the {@link
91 | %% mnesia_to_khepri_converter} behavior. The private term is passed as is to
92 | %% its @{link mnesia_to_khepri_converter:init_copy_to_khepri/3} callback.
93 |
94 | -export_type([migration_id/0,
95 | mnesia_table/0,
96 | converter_mod/0]).
97 |
98 | %% -------------------------------------------------------------------
99 | %% Cluster membership.
100 | %% -------------------------------------------------------------------
101 |
102 | -spec sync_cluster_membership() -> ok.
103 | %% @doc Ensures the default Khepri store has the same members as the Mnesia
104 | %% cluster.
105 | %%
106 | %% @see sync_cluster_membership/1.
107 |
108 | sync_cluster_membership() ->
109 | StoreId = khepri_cluster:get_default_store_id(),
110 | sync_cluster_membership(StoreId).
111 |
112 | -spec sync_cluster_membership(StoreId) -> ok when
113 | StoreId :: khepri:store_id().
114 | %% @doc Ensures the Khepri store named `StoreId' has the same members as the
115 | %% Mnesia cluster.
116 | %%
117 | %% The synchronization is split into the following steps:
118 | %%
119 | %% Mnesia is queried to list all its members. We ensure that all Mnesia
120 | %% members are running at this point: if some members are down, there is
121 | %% little chance that Khepri will work on those anyway.
122 | %% All Mnesia members are queried to learn about the membership status of
123 | %% all "instances" of the Khepri store named `StoreId'.
124 | %% Among all instances of the Khepri store, one instance is selected to be
125 | %% the one that will be joined by other nodes. The selection process is
126 | %% described below.
127 | %% Other nodes are reset, meaning their data is dropped, and they are
128 | %% added to the selected Khepri store.
129 | %%
130 | %%
131 | %% The synchronization process has to select a store instance that will grow
132 | %% and other store instances that will be reset. The criterias below are
133 | %% evaluated to sort the list of store instances, then the first instance in
134 | %% that list is selected as the winning instance.
135 | %%
136 | %% Criterias are evaluated in order. For a given criteria, if both sides are
137 | %% equal, the next criteria is evaluated until one criteria gives an instance
138 | %% "greater" than another.
139 | %%
140 | %% Here is the ordered list of criterias:
141 | %%
142 | %% A Khepri store instance with more members gets precedence.
143 | %% Then, a Khepri store instance with more tree nodes (more records) gets
144 | %% precedence.
145 | %% Then, a Khepri store instance where a member's Erlang node runs for the
146 | %% longest time gets precedence.
147 | %% Then, a Khepri store instance where its members list sorts before
148 | %% another members list gets precedence.
149 | %%
150 | %%
151 | %% Here are some examples:
152 | %%
153 | %% A Khepri store instance with 3 members will be selected before another
154 | %% Khepri store instance with 1 member.
155 | %% If they have the same number of members, a Khepri store instance with
156 | %% 125 tree nodes will be selected before another Khepri store instance with
157 | %% 34 tree nodes.
158 | %% If they have the same number of members and the same number of tree
159 | %% nodes, a Khepri store instance where the oldest Erlang node runs for 67
160 | %% days will be selected before another Khepri store instance where the oldest
161 | %% Erlang node runs for 3 days.
162 | %%
163 |
164 | sync_cluster_membership(StoreId) ->
165 | case m2k_cluster_sync_sup:prepare_worker(StoreId) of
166 | {ok, Pid} when is_pid(Pid) -> m2k_cluster_sync:proceed(Pid);
167 | {error, _} = Error -> Error
168 | end.
169 |
170 | %% -------------------------------------------------------------------
171 | %% Tables copy.
172 | %% -------------------------------------------------------------------
173 |
174 | -spec copy_tables(MigrationId, Tables, Mod) -> Ret when
175 | MigrationId :: migration_id(),
176 | Tables :: [mnesia_table()],
177 | Mod :: converter_mod() | {converter_mod(), ModArgs},
178 | ModArgs :: any(),
179 | Ret :: ok | {error, any()}.
180 | %% @doc Copies records from Mnesia tables `Tables' to the default Khepri
181 | %% store.
182 | %%
183 | %% @see copy_tables/3.
184 |
185 | copy_tables(MigrationId, Tables, Mod) ->
186 | StoreId = khepri_cluster:get_default_store_id(),
187 | copy_tables(StoreId, MigrationId, Tables, Mod).
188 |
189 | -spec copy_tables(StoreId, MigrationId, Tables, Mod) -> Ret when
190 | StoreId :: khepri:store_id(),
191 | MigrationId :: migration_id(),
192 | Tables :: [mnesia_table()],
193 | Mod :: converter_mod() | {converter_mod(), ModArgs},
194 | ModArgs :: any(),
195 | Ret :: ok | {error, any()}.
196 | %% @doc Copies records from Mnesia tables `Tables' to the Khepri store named
197 | %% `StoreId'.
198 | %%
199 | %% The converter module `Mod' is responsible for storing each Mnesia record in
200 | %% the Khepri store. How it is called is described below. {@link
201 | %% mnesia_to_khepri_example_converter} can be used as the default converter
202 | %% module or as an example to write a new one.
203 | %%
204 | %% The copy is split into the following steps:
205 | %%
206 | %% The PID of the Erlang process working on the copy is stored in the
207 | %% Khepri store. This serves as an indicator that tables are being copied and
208 | %% this prevents concurrent copy from happening.
209 | %% The converter module state is initialized using its
210 | %% `Mod:init_copy_to_khepri/3' function, or `Mod:init_copy_to_khepri/4' if
211 | %% `ModArgs' is set.
212 | %% The copy process subscribes to changes from Mnesia.
213 | %% The copy process copies records from Mnesia using its Backup &
214 | %% Restore API. The process uses `Mod:copy_to_khepri/3' for each Mnesia
215 | %% record.
216 | %% The copy process marks all involved Mnesia tables as read-only.
217 | %% The copy process consumes all Mnesia events it received during the
218 | %% initial copy and calls `Mod:copy_to_khepri/3' and
219 | %% `Mod:delete_from_khepri/3' to update the Khepri store accordingly.
220 | %% The copy process calls `Mod:finish_copy_to_khepri/1' to let the
221 | %% converter module do any cleanups.
222 | %% The "migration in progress" marker is removed from the Khepri
223 | %% store.
224 | %%
225 | %%
226 | %% Copied Mnesia tables continue to be available during the process, except
227 | %% after they are marked as read-only. See {@link handle_fallback/2} and {@link
228 | %% handle_fallback/3} for helpers to use Mnesia while records are being copied.
229 |
230 | copy_tables(StoreId, MigrationId, Tables, Mod)
231 | when is_binary(MigrationId) ->
232 | ExistingTables = filter_out_non_existing_tables(Tables),
233 | copy_tables1(StoreId, MigrationId, ExistingTables, Mod).
234 |
235 | copy_tables1(_StoreId, _MigrationId, [], _Mod) ->
236 | ok;
237 | copy_tables1(StoreId, MigrationId, Tables, Mod)
238 | when is_binary(MigrationId) andalso
239 | is_list(Tables) andalso
240 | Tables =/= [] andalso
241 | (is_atom(Mod) orelse
242 | (is_tuple(Mod) andalso size(Mod) =:= 2 andalso
243 | is_atom(element(1, Mod)))) ->
244 | Ret = m2k_table_copy_sup_sup:prepare_workers_sup(
245 | StoreId, MigrationId, Tables, Mod),
246 | case Ret of
247 | {ok, Pid} when is_pid(Pid) ->
248 | copy_tables2(StoreId, MigrationId, Tables, Mod, Pid);
249 | {error, _} = Error ->
250 | Error
251 | end.
252 |
253 | filter_out_non_existing_tables(Tables) ->
254 | %% This filtering is not atomic w.r.t. the upcoming copy process, but a
255 | %% table could be added or removed at any time anyway.
256 | lists:filter(
257 | fun(Table) ->
258 | try
259 | _ = mnesia:table_info(Table, type),
260 | true
261 | catch
262 | exit:{aborted, {no_exists, Table, type}} ->
263 | %% The table doesn't exist in the first place, there is
264 | %% nothing to migrate.
265 | ?LOG_DEBUG(
266 | "Mnesia->Khepri data copy: Table `~ts` does not "
267 | "exist, skipping its migration",
268 | [Table],
269 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
270 | false
271 | end
272 | end,
273 | Tables).
274 |
275 | copy_tables2(StoreId, MigrationId, Tables, Mod, Pid) ->
276 | case m2k_table_copy:proceed(Pid) of
277 | ok ->
278 | ok;
279 | {error, {already_started, OtherPid}} ->
280 | MRef = erlang:monitor(process, OtherPid),
281 | receive
282 | {'DOWN', MRef, process, OtherPid, normal} ->
283 | copy_tables1(StoreId, MigrationId, Tables, Mod);
284 | {'DOWN', MRef, process, OtherPid, noproc} ->
285 | %% The previous migration crashed or the node was killed
286 | %% for instance. The `in_flight' marker is still there
287 | %% but the process is gone. We can clear the marker and
288 | %% retry.
289 | m2k_table_copy:clear_migration_marker(
290 | StoreId, MigrationId, OtherPid, '_'),
291 | copy_tables1(StoreId, MigrationId, Tables, Mod);
292 | {'DOWN', MRef, process, OtherPid, Info} ->
293 | {error, Info}
294 | end;
295 | {error, _} = Error ->
296 | Error
297 | end.
298 |
299 | -spec copy_all_tables(MigrationId, Mod) -> Ret when
300 | MigrationId :: migration_id(),
301 | Mod :: converter_mod() | {converter_mod(), ModArgs},
302 | ModArgs :: any(),
303 | Ret :: ok | {error, any()}.
304 | %% @doc Copies records from all Mnesia tables to the default Khepri store.
305 | %%
306 | %% @see copy_all_tables/3.
307 |
308 | copy_all_tables(MigrationId, Mod) ->
309 | StoreId = khepri_cluster:get_default_store_id(),
310 | copy_all_tables(StoreId, MigrationId, Mod).
311 |
312 | -spec copy_all_tables(StoreId, MigrationId, Mod) -> Ret when
313 | StoreId :: khepri:store_id(),
314 | MigrationId :: migration_id(),
315 | Mod :: converter_mod() | {converter_mod(), ModArgs},
316 | ModArgs :: any(),
317 | Ret :: ok | {error, any()}.
318 | %% @doc Copies records from all Mnesia tables to the Khepri store named
319 | %% `StoreId'.
320 | %%
321 | %% @see copy_tables/4.
322 |
323 | copy_all_tables(StoreId, MigrationId, Mod) ->
324 | Tables = list_all_tables(),
325 | copy_tables(StoreId, MigrationId, Tables, Mod).
326 |
327 | -spec list_all_tables() -> Tables when
328 | Tables :: [mnesia_table()].
329 | %% @private
330 |
331 | list_all_tables() ->
332 | Tables0 = lists:sort(mnesia:system_info(tables)),
333 | Tables1 = Tables0 -- [schema],
334 | Tables1.
335 |
336 | -spec is_migration_finished(MigrationId) -> Migrated when
337 | MigrationId :: mnesia_to_khepri:migration_id(),
338 | Migrated :: boolean() | {in_flight, pid()} | undefined.
339 | %% @doc Returns the migration status of the specified migration identifier.
340 | %%
341 | %% The default Khepri store is queried to get the migration status. It must
342 | %% correspond to the Khepri store passed to {@link copy_tables/3} and similar
343 | %% functions.
344 | %%
345 | %% @see is_migration_finished/2.
346 |
347 | is_migration_finished(MigrationId) ->
348 | StoreId = khepri_cluster:get_default_store_id(),
349 | is_migration_finished(StoreId, MigrationId).
350 |
351 | -spec is_migration_finished(StoreId, MigrationId) -> Migrated when
352 | StoreId :: khepri:store_id(),
353 | MigrationId :: mnesia_to_khepri:migration_id(),
354 | Migrated :: boolean() | {in_flight, pid()} | undefined.
355 | %% @doc Returns the migration status of the specified migration identifier.
356 | %%
357 | %% The Khepri store named `StoreId' is queried to get the migration status. It
358 | %% must correspond to the Khepri store passed to {@link copy_tables/3} and
359 | %% similar functions.
360 | %%
361 | %% @returns `true' if the migration is finished, `{in_flight, Pid}' if the
362 | %% migration is in progress and handled by the `Pid' process, `false' if the
363 | %% migration has not started or `undefined' if the query of the Khepri store
364 | %% where the status is recorded failed.
365 |
366 | is_migration_finished(StoreId, MigrationId) when is_binary(MigrationId) ->
367 | m2k_table_copy:is_migration_finished(StoreId, MigrationId).
368 |
369 | -spec wait_for_migration(MigrationId, Timeout) -> Ret when
370 | MigrationId :: mnesia_to_khepri:migration_id(),
371 | Timeout :: timeout(),
372 | Ret :: boolean() | timeout.
373 | %% @doc Waits for migration `MigrationId' to be finish.
374 | %%
375 | %% The default Khepri store is queried to get the migration status. It must
376 | %% correspond to the Khepri store passed to {@link copy_tables/3} and similar
377 | %% functions.
378 | %%
379 | %% @see wait_for_migration/3.
380 |
381 | wait_for_migration(MigrationId, Timeout) ->
382 | StoreId = khepri_cluster:get_default_store_id(),
383 | wait_for_migration(StoreId, MigrationId, Timeout).
384 |
385 | -spec wait_for_migration(StoreId, MigrationId, Timeout) -> Ret when
386 | StoreId :: khepri:store_id(),
387 | MigrationId :: mnesia_to_khepri:migration_id(),
388 | Timeout :: timeout(),
389 | Ret :: boolean() | timeout.
390 | %% @doc Waits for migration `MigrationId' to be finish.
391 | %%
392 | %% If the migration has not started, it returns `false' immediately.
393 | %%
394 | %% If the migration is finished, it returns `true' immediately.
395 | %%
396 | %% If the migration is in progress or the status is undefined (see {@link
397 | %% is_migration_finished/3}), it waits until the status is known to be
398 | %% "finished" or "not started" or until `Timeout' milliseconds.
399 | %%
400 | %% The Khepri store named `StoreId' is queried to get the migration status. It
401 | %% must correspond to the Khepri store passed to {@link copy_tables/3} and
402 | %% similar functions.
403 |
404 | wait_for_migration(_StoreId, MigrationId, 0)
405 | when is_binary(MigrationId) ->
406 | timeout;
407 | wait_for_migration(StoreId, MigrationId, Timeout)
408 | when is_binary(MigrationId) ->
409 | T0 = khepri_utils:start_timeout_window(Timeout),
410 | case is_migration_finished(StoreId, MigrationId) of
411 | true ->
412 | true;
413 | false ->
414 | false;
415 | _ ->
416 | timer:sleep(250),
417 | NewTimeout = khepri_utils:end_timeout_window(Timeout, T0),
418 | wait_for_migration(StoreId, MigrationId, NewTimeout)
419 | end.
420 |
421 | -spec cleanup_after_table_copy(MigrationId) -> Ret when
422 | MigrationId :: mnesia_to_khepri:migration_id(),
423 | Ret :: ok | {error, any()}.
424 | %% @doc Performs any cleanups after a migration has finished.
425 | %%
426 | %% @see cleanup_after_table_copy/2.
427 |
428 | cleanup_after_table_copy(MigrationId) ->
429 | StoreId = khepri_cluster:get_default_store_id(),
430 | cleanup_after_table_copy(StoreId, MigrationId).
431 |
432 | -spec cleanup_after_table_copy(StoreId, MigrationId) -> Ret when
433 | StoreId :: khepri:store_id(),
434 | MigrationId :: mnesia_to_khepri:migration_id(),
435 | Ret :: ok | {error, any()}.
436 | %% @doc Performs any cleanups after a migration has finished.
437 | %%
438 | %% Currently this includes the deletion of the copied Mnesia tables.
439 | %%
440 | %% This is a separate step from {@link copy_tables/4} because the caller might
441 | %% want to record some post-migration states before comitting to delete the
442 | %% Mnesia tables.
443 |
444 | cleanup_after_table_copy(StoreId, MigrationId)
445 | when is_binary(MigrationId) ->
446 | m2k_table_copy:cleanup(StoreId, MigrationId).
447 |
448 | -spec rollback_table_copy(MigrationId) -> Ret when
449 | MigrationId :: mnesia_to_khepri:migration_id(),
450 | Ret :: ok | {error, any()}.
451 | %% @doc Rolls back a migration.
452 | %%
453 | %% @see rollback_table_copy/2.
454 |
455 | rollback_table_copy(MigrationId) ->
456 | StoreId = khepri_cluster:get_default_store_id(),
457 | rollback_table_copy(StoreId, MigrationId).
458 |
459 | -spec rollback_table_copy(StoreId, MigrationId) -> Ret when
460 | StoreId :: khepri:store_id(),
461 | MigrationId :: mnesia_to_khepri:migration_id(),
462 | Ret :: ok | {error, any()}.
463 | %% @doc Rolls back a migration.
464 | %%
465 | %% This function puts Mnesia tables back to read-write mode and deletes the
466 | %% "migration in progress" marker.
467 | %%
468 | %% Rollback is not possible once {@link cleanup_after_table_copy/2} is used.
469 |
470 | rollback_table_copy(StoreId, MigrationId)
471 | when is_binary(MigrationId) ->
472 | m2k_table_copy:rollback(StoreId, MigrationId).
473 |
474 | -spec handle_fallback(MigrationId, MnesiaFun, KhepriFunOrRet) ->
475 | Ret when
476 | MigrationId :: mnesia_to_khepri:migration_id(),
477 | MnesiaFun :: fun(() -> Ret),
478 | KhepriFunOrRet :: fun(() -> Ret) | Ret,
479 | Ret :: any().
480 | %% @doc Runs `MnesiaFun' or evaluates `KhepriFunOrRet' depending on the status
481 | %% of the migration.
482 | %%
483 | %% The default Khepri store is queried to get the migration status. It must
484 | %% correspond to the Khepri store passed to {@link copy_tables/3} and similar
485 | %% functions.
486 | %%
487 | %% @see handle_fallback/4.
488 |
489 | handle_fallback(MigrationId, MnesiaFun, KhepriFunOrRet) ->
490 | StoreId = khepri_cluster:get_default_store_id(),
491 | handle_fallback(StoreId, MigrationId, MnesiaFun, KhepriFunOrRet).
492 |
493 | -spec handle_fallback(StoreId, MigrationId, MnesiaFun, KhepriFunOrRet) ->
494 | Ret when
495 | StoreId :: khepri:store_id(),
496 | MigrationId :: mnesia_to_khepri:migration_id(),
497 | MnesiaFun :: fun(() -> Ret),
498 | KhepriFunOrRet :: fun(() -> Ret) | Ret,
499 | Ret :: any().
500 | %% @doc Runs `MnesiaFun' or evaluates `KhepriFunOrRet' depending on the status
501 | %% of the migration.
502 | %%
503 | %% If the migration is finished it executes `KhepriFunOrRet' if it is a
504 | %% function with an arity of 0 or returns the term directly otherwise.
505 | %%
506 | %% If the migration is not finished, the function tries to execute
507 | %% `MnesiaFun'. It should fail by returning or raising `{aborted, {no_exists,
508 | %% Table}}'. When this happens, the function waits for the migration to finish
509 | %% using {@link wait_for_table_migration/3}, then it starts again.
510 | %%
511 | %% The Khepri store named `StoreId' is queried to get the migration status. It
512 | %% must correspond to the Khepri store passed to {@link copy_tables/3} and
513 | %% similar functions.
514 |
515 | -define(UNKNOWN_TABLE, 0).
516 | -define(CONSIDER_INFINITE_LOOP_AFTER, 100).
517 |
518 | handle_fallback(StoreId, MigrationId, MnesiaFun, KhepriFunOrRet)
519 | when is_binary(MigrationId) andalso is_function(MnesiaFun, 0) ->
520 | handle_fallback(StoreId, MigrationId, MnesiaFun, KhepriFunOrRet, 1).
521 |
522 | handle_fallback(StoreId, MigrationId, MnesiaFun, KhepriFunOrRet, Attempts)
523 | when Attempts < ?CONSIDER_INFINITE_LOOP_AFTER ->
524 | case is_migration_finished(StoreId, MigrationId) of
525 | true when is_function(KhepriFunOrRet, 0) ->
526 | KhepriFunOrRet();
527 | true ->
528 | KhepriFunOrRet;
529 | _ ->
530 | try
531 | case MnesiaFun() of
532 | {aborted, NoExists1}
533 | when is_tuple(NoExists1) andalso
534 | element(1, NoExists1) =:= no_exists ->
535 | {_, Stacktrace1} = erlang:process_info(
536 | self(), current_stacktrace),
537 | log_no_exists_error(NoExists1, Stacktrace1),
538 |
539 | _ = wait_for_migration(StoreId, MigrationId, 1000),
540 | handle_fallback(
541 | StoreId, MigrationId, MnesiaFun, KhepriFunOrRet,
542 | Attempts + 1);
543 | Ret ->
544 | Ret
545 | end
546 | catch
547 | _:{_aborted, NoExists2}:Stacktrace2
548 | when is_tuple(NoExists2) andalso
549 | element(1, NoExists2) =:= no_exists ->
550 | log_no_exists_error(NoExists2, Stacktrace2),
551 |
552 | _ = wait_for_migration(StoreId, MigrationId, 1000),
553 | handle_fallback(
554 | StoreId, MigrationId, MnesiaFun, KhepriFunOrRet,
555 | Attempts + 1)
556 | end
557 | end;
558 | handle_fallback(
559 | _StoreId, _MigrationId, MnesiaFun, _KhepriFunOrRet, Attempts) ->
560 | ?LOG_WARNING(
561 | "Mnesia->Khepri fallback handling: Mnesia function failed ~b times. "
562 | "Possibly an infinite retry loop; trying one last time",
563 | [Attempts - 1]),
564 | MnesiaFun().
565 |
566 | log_no_exists_error(NoExists, Stacktrace) ->
567 | case table_name_from_no_exists(NoExists) of
568 | ?UNKNOWN_TABLE ->
569 | ?LOG_DEBUG(
570 | "Mnesia->Khepri fallback handling: Mnesia function failed "
571 | "because at least one table is missing or read-only. "
572 | "Migration could be in progress; waiting for migration to "
573 | "progress and trying again~n"
574 | "~p",
575 | [Stacktrace]),
576 | ok;
577 | Table ->
578 | ?LOG_DEBUG(
579 | "Mnesia->Khepri fallback handling: Mnesia function failed "
580 | "because table `~ts` is missing or read-only. Migration "
581 | "could be in progress; waiting for migration to progress "
582 | "and trying again~n"
583 | "~p",
584 | [Table, Stacktrace]),
585 | ok
586 | end.
587 |
588 | table_name_from_no_exists({no_exists, Table}) when is_atom(Table) ->
589 | Table;
590 | table_name_from_no_exists({no_exists, [Table | _]}) when is_atom(Table) ->
591 | Table;
592 | table_name_from_no_exists({no_exists, Table, _}) when is_atom(Table) ->
593 | Table;
594 | table_name_from_no_exists(_) ->
595 | ?UNKNOWN_TABLE.
596 |
--------------------------------------------------------------------------------
/src/m2k_table_copy.erl:
--------------------------------------------------------------------------------
1 | %% This Source Code Form is subject to the terms of the Mozilla Public
2 | %% License, v. 2.0. If a copy of the MPL was not distributed with this
3 | %% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4 | %%
5 | %% Copyright © 2022-2025 Broadcom. All Rights Reserved. The term "Broadcom"
6 | %% refers to Broadcom Inc. and/or its subsidiaries.
7 | %%
8 |
9 | %% @private
10 |
11 | -module(m2k_table_copy).
12 |
13 | -behaviour(gen_server).
14 |
15 | -include_lib("kernel/include/logger.hrl").
16 | -include_lib("stdlib/include/assert.hrl").
17 | -include_lib("khepri/include/khepri.hrl").
18 |
19 | -include("src/kmm_error.hrl").
20 | -include("src/kmm_logging.hrl").
21 |
22 | -export([start_link/1,
23 | proceed/1,
24 | is_migration_finished/2,
25 | cleanup/2,
26 | rollback/2,
27 | clear_migration_marker/4]).
28 | -export([init/1,
29 | handle_call/3,
30 | handle_cast/2,
31 | handle_info/2,
32 | terminate/2]).
33 |
34 | -ifdef(TEST).
35 | -export([migration_recorded_state/2,
36 | marker_path/1]).
37 | -endif.
38 |
39 | -record(migration, {progress :: {in_flight, pid()} | finished,
40 | tables :: [mnesia_to_khepri:mnesia_table()] | '_'}).
41 |
42 | -record(?MODULE, {khepri_store :: khepri:store_id(),
43 | migration_id :: mnesia_to_khepri:migration_id(),
44 | tables :: [mnesia_to_khepri:mnesia_table()],
45 | record_defs :: #{mnesia_to_khepri:mnesia_table() =>
46 | {atom(), arity()}},
47 | converter_mod :: mnesia_to_khepri:converter_mod() |
48 | {mnesia_to_khepri:converter_mod(), any()},
49 | converter_mod_priv :: any() | undefined,
50 | subscriber :: pid() | undefined,
51 | backup_pid :: pid() | undefined,
52 | progress :: #migration{}}).
53 |
54 | -define(PROJECTION_NAME, kmm_m2k_table_copy_projection).
55 |
56 | proceed(SupPid) ->
57 | [{m2k_subscriber, SubscriberPid, _, _},
58 | {m2k_table_copy, TableCopyPid, _, _}] =
59 | lists:sort(supervisor:which_children(SupPid)),
60 |
61 | Ret = gen_server:call(
62 | TableCopyPid, {?FUNCTION_NAME, SubscriberPid}, infinity),
63 | case Ret of
64 | {exception, ?kmm_exception(_, _) = Exception} ->
65 | ?kmm_misuse(Exception);
66 | _ ->
67 | Ret
68 | end.
69 |
70 | -spec is_migration_finished(StoreId, MigrationId) -> IsFinished when
71 | StoreId :: khepri:store_id(),
72 | MigrationId :: mnesia_to_khepri:migration_id(),
73 | IsFinished :: boolean() | {in_flight, pid()} | undefined.
74 |
75 | is_migration_finished(StoreId, MigrationId) when is_binary(MigrationId) ->
76 | %% If the Khepri store is not running, we can assume that the migration
77 | %% didn't take place yet.
78 | case khepri_cluster:is_store_running(StoreId) of
79 | true -> is_migration_finished1(StoreId, MigrationId);
80 | false -> false
81 | end.
82 |
83 | is_migration_finished1(StoreId, MigrationId) ->
84 | ProjectionName = ?PROJECTION_NAME,
85 | try
86 | case ets:lookup(ProjectionName, MigrationId) of
87 | [{_, #migration{progress = finished}}] ->
88 | true;
89 | [{_, #migration{progress = {in_flight, _} = InFlight}}] ->
90 | InFlight;
91 | [] ->
92 | false
93 | end
94 | catch
95 | error:badarg ->
96 | case setup_projection(StoreId, ProjectionName) of
97 | ok ->
98 | is_migration_finished1(StoreId, MigrationId);
99 | %% Before Khepri v0.13.0, `khepri:register_projection/1,2,3`
100 | %% would return `{error, exists}` for projections which already
101 | %% exist.
102 | {error, exists} ->
103 | is_migration_finished1(StoreId, MigrationId);
104 | %% In v0.13.0+, Khepri returns a `?khepri_error(..)` instead.
105 | {error, {khepri, projection_already_exists, _Info}} ->
106 | is_migration_finished1(StoreId, MigrationId);
107 | Error ->
108 | Key = {?MODULE, ?FUNCTION_NAME, StoreId},
109 | case persistent_term:get(Key, undefined) of
110 | Error ->
111 | ok;
112 | _ ->
113 | ?LOG_INFO(
114 | "Mnesia->Khepri fallback handling: couldn't "
115 | "setup Khepri projection for migration "
116 | "\"~ts\"; that's ok but expect slightly "
117 | "slower versions of `is_migration_finished()` "
118 | "and `handle_fallback()`~n~p",
119 | [MigrationId, Error],
120 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
121 | persistent_term:put(Key, Error)
122 | end,
123 | is_migration_finished_slow(StoreId, MigrationId)
124 | end
125 | end.
126 |
127 | setup_projection(StoreId, ProjectionName) ->
128 | %% In case this function is called many times concurrently, for instance
129 | %% because many processes use `mnesia_to_khepri:handle_fallback()' at the
130 | %% same time, we use a lock and check if the ETS table already exists
131 | %% inside the lock before registering the projection.
132 | %%
133 | %% This avoids that all these processes register the same projection many
134 | %% times, causing many Khepri/Ra commands to be sent to the leader.
135 | Lock = {{?MODULE, StoreId}, self()},
136 | global:set_lock(Lock, [node()]),
137 | try
138 | ProjectionName = ?PROJECTION_NAME,
139 | case ets:whereis(ProjectionName) of
140 | undefined ->
141 | ?LOG_DEBUG(
142 | "Mnesia->Khepri data copy: setup Khepri projection "
143 | "(name: \"~s\")",
144 | [ProjectionName],
145 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
146 | PathPattern = marker_path(?KHEPRI_WILDCARD_STAR),
147 | Options = #{type => set, read_concurrency => true},
148 | ProjectionFun = fun(Path, Progress) ->
149 | {lists:last(Path), Progress}
150 | end,
151 | Projection = khepri_projection:new(
152 | ProjectionName, ProjectionFun, Options),
153 | khepri:register_projection(StoreId, PathPattern, Projection);
154 | _ ->
155 | ok
156 | end
157 | after
158 | global:del_lock(Lock, [node()])
159 | end.
160 |
161 | is_migration_finished_slow(StoreId, MigrationId) ->
162 | Path = marker_path(MigrationId),
163 | case khepri:get_or(StoreId, Path, false) of
164 | {ok, #migration{progress = finished}} -> true;
165 | {ok, #migration{progress = {in_flight, _} = InFlight}} -> InFlight;
166 | {ok, false} -> false;
167 | _ -> undefined
168 | end.
169 |
170 | -spec cleanup(StoreId, MigrationId) -> Ret when
171 | StoreId :: khepri:store_id(),
172 | MigrationId :: mnesia_to_khepri:migration_id(),
173 | Ret :: ok | {error, any()}.
174 |
175 | cleanup(StoreId, MigrationId) ->
176 | ?LOG_DEBUG(
177 | "Mnesia->Khepri data copy: deleting Mnesia tables after "
178 | "migration \"~ts\" finished",
179 | [MigrationId],
180 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
181 | Path = marker_path(MigrationId),
182 | case khepri:get(StoreId, Path) of
183 | {ok, #migration{progress = finished,
184 | tables = Tables}} ->
185 | do_cleanup(Tables);
186 | {ok, #migration{progress = InFlight}} ->
187 | {error, InFlight};
188 | {error, {khepri, node_not_found, _}} ->
189 | {error, {no_such_migration, MigrationId}};
190 | {error, _} = Error ->
191 | Error
192 | end.
193 |
194 | do_cleanup(Tables) ->
195 | lists:foreach(
196 | fun(Table) ->
197 | ?LOG_DEBUG(
198 | "Mnesia->Khepri data copy: marking Mnesia table `~ts` back "
199 | "as read-write",
200 | [Table],
201 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
202 | case mnesia:change_table_access_mode(Table, read_write) of
203 | Ret when Ret =:= {atomic, ok} orelse
204 | Ret =:= {aborted,
205 | {already_exists, Table, read_write}} ->
206 | ?LOG_DEBUG(
207 | "Mnesia->Khepri data copy: deleting Mnesia table "
208 | "`~ts`",
209 | [Table],
210 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
211 | case mnesia:delete_table(Table) of
212 | {atomic, ok} ->
213 | ok;
214 | {aborted, {no_exists, _}} ->
215 | ?LOG_DEBUG(
216 | "Mnesia->Khepri data copy: Mnesia table "
217 | "`~ts` already deleted",
218 | [Table],
219 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
220 | ok;
221 | {aborted, Reason2} ->
222 | ?LOG_DEBUG(
223 | "Mnesia->Khepri data copy: failed to delete "
224 | "Mnesia table `~ts`: ~0p",
225 | [Table, Reason2],
226 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
227 | ok
228 | end;
229 | {aborted, {no_exists, _}} ->
230 | ?LOG_DEBUG(
231 | "Mnesia->Khepri data copy: Mnesia table `~ts` "
232 | "already deleted",
233 | [Table],
234 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
235 | ok;
236 | {aborted, Reason1} ->
237 | ?LOG_DEBUG(
238 | "Mnesia->Khepri data copy: failed to mark Mnesia "
239 | "table `~ts` as read-write: ~0p",
240 | [Table, Reason1],
241 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
242 | ok
243 | end
244 | end, Tables).
245 |
246 | -spec rollback(StoreId, MigrationId) -> Ret when
247 | StoreId :: khepri:store_id(),
248 | MigrationId :: mnesia_to_khepri:migration_id(),
249 | Ret :: ok | {error, any()}.
250 |
251 | rollback(StoreId, MigrationId) ->
252 | ?LOG_DEBUG(
253 | "Mnesia->Khepri data copy: marking Mnesia tables back as read-write "
254 | "as part of migration \"~ts\" rollback",
255 | [MigrationId],
256 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
257 | Path = marker_path(MigrationId),
258 | case khepri:get(StoreId, Path) of
259 | {ok, #migration{progress = finished,
260 | tables = Tables} = Progress} ->
261 | make_tables_readwrite(Tables),
262 | clear_migration_marker(StoreId, MigrationId, Progress),
263 | wait_for_projected_record_deletion(MigrationId),
264 | ?assertEqual(false, is_migration_finished(StoreId, MigrationId)),
265 | ok;
266 | {ok, #migration{progress = InFlight}} ->
267 | {error, InFlight};
268 | {error, {khepri, node_not_found, _}} ->
269 | {error, {no_such_migration, MigrationId}};
270 | {error, _} = Error ->
271 | Error
272 | end.
273 |
274 | wait_for_projected_record_deletion(MigrationId) ->
275 | Retry = try
276 | case ets:lookup(?PROJECTION_NAME, MigrationId) of
277 | [_] -> true;
278 | [] -> false
279 | end
280 | catch
281 | error:badarg ->
282 | false
283 | end,
284 | case Retry of
285 | true ->
286 | timer:sleep(100),
287 | wait_for_projected_record_deletion(MigrationId);
288 | false ->
289 | ok
290 | end.
291 |
292 | start_link(Args) ->
293 | gen_server:start_link(?MODULE, Args, []).
294 |
295 | %% -------------------------------------------------------------------
296 | %% `gen_server' callbacks.
297 | %% -------------------------------------------------------------------
298 |
299 | init(#{khepri_store := StoreId,
300 | migration_id := MigrationId,
301 | tables := Tables,
302 | converter_mod := Mod}) ->
303 | erlang:process_flag(trap_exit, true),
304 | Progress = migration_recorded_state(self(), Tables),
305 | RecordDefs = query_table_record_definitions(Tables),
306 | State = #?MODULE{khepri_store = StoreId,
307 | migration_id = MigrationId,
308 | tables = Tables,
309 | record_defs = RecordDefs,
310 | converter_mod = Mod,
311 | progress = Progress},
312 | {ok, State}.
313 |
314 | handle_call({proceed, SubscriberPid}, From, State) ->
315 | State1 = State#?MODULE{subscriber = SubscriberPid},
316 | try
317 | State2 = do_copy_data(State1),
318 | gen_server:reply(From, ok),
319 |
320 | {stop, normal, State2}
321 | catch
322 | throw:ok ->
323 | {stop, normal, ok, State1};
324 | throw:{already_started, OtherPid} = Reason ->
325 | Error = {error, Reason},
326 | ?LOG_INFO(
327 | "Mnesia->Khepri data copy already in progress by PID ~p",
328 | [OtherPid],
329 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
330 | {stop, normal, Error, State1};
331 | error:?kmm_exception(_, _) = Exception:Stacktrace ->
332 | ?LOG_ERROR(
333 | "Exception during Mnesia->Khepri data copy: ~0p~n~p",
334 | [Exception, Stacktrace],
335 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
336 | {stop, normal, {exception, Exception}, State1};
337 | throw:Reason:Stacktrace ->
338 | Error = {error, Reason},
339 | ?LOG_ERROR(
340 | "Failed to copy Mnesia->Khepri data: ~0p~n~p",
341 | [Error, Stacktrace],
342 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
343 | {stop, normal, Error, State1}
344 | end;
345 | handle_call(Request, _From, State) ->
346 | ?LOG_WARNING(
347 | ?MODULE_STRING ": Unhandled handle_call message: ~p",
348 | [Request],
349 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
350 | {reply, undefined, State}.
351 |
352 | handle_cast(Request, State) ->
353 | ?LOG_WARNING(
354 | ?MODULE_STRING ": Unhandled handle_cast message: ~p",
355 | [Request],
356 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
357 | {noreply, State}.
358 |
359 | handle_info(Msg, State) ->
360 | ?LOG_WARNING(
361 | ?MODULE_STRING ": Unhandled handle_info message: ~p",
362 | [Msg],
363 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
364 | {noreply, State}.
365 |
366 | terminate(_Reason, State) ->
367 | clear_migration_marker(State),
368 | ok.
369 |
370 | %% -------------------------------------------------------------------
371 | %% Internal functions.
372 | %% -------------------------------------------------------------------
373 |
374 | query_table_record_definitions(Tables) ->
375 | query_table_record_definitions(Tables, #{}).
376 |
377 | query_table_record_definitions([Table | Rest], RecordDefs) ->
378 | RecordName = mnesia:table_info(Table, record_name),
379 | Arity = mnesia:table_info(Table, arity),
380 | RecordDefs1 = RecordDefs#{Table => {RecordName, Arity}},
381 | query_table_record_definitions(Rest, RecordDefs1);
382 | query_table_record_definitions([], RecordDefs) ->
383 | RecordDefs.
384 |
385 | do_copy_data(#?MODULE{migration_id = MigrationId, tables = Tables} = State) ->
386 | ?LOG_INFO(
387 | "Mnesia->Khepri data copy: "
388 | "starting migration \"~ts\" from Mnesia to Khepri; tables: ~1p",
389 | [MigrationId, Tables],
390 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
391 |
392 | mark_tables_as_being_migrated(State),
393 |
394 | State1 = init_converter_mod(State),
395 | subscribe_to_mnesia_changes(State1),
396 | State2 = start_copy_from_mnesia_to_khepri(State1),
397 | State3 = handle_migration_records(State2),
398 | State4 = final_sync_from_mnesia_to_khepri(State3),
399 | State5 = finish_converter_mod(State4),
400 |
401 | mark_tables_as_migrated(State5),
402 |
403 | ?LOG_INFO(
404 | "Mnesia->Khepri data copy: "
405 | "migration \"~ts\" from Mnesia to Khepri finished",
406 | [MigrationId],
407 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
408 |
409 | State5.
410 |
411 | init_converter_mod(
412 | #?MODULE{tables = Tables,
413 | khepri_store = StoreId,
414 | migration_id = MigrationId,
415 | converter_mod = Mod} = State) ->
416 | ?LOG_DEBUG(
417 | "Mnesia->Khepri data copy: initialize converter mod ~s for Mnesia "
418 | "tables: ~0p",
419 | [actual_mod(Mod), Tables],
420 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
421 | try
422 | Ret = case Mod of
423 | {ActualMod, ModArgs} ->
424 | ActualMod:init_copy_to_khepri(
425 | StoreId, MigrationId, Tables, ModArgs);
426 | _ ->
427 | Mod:init_copy_to_khepri(
428 | StoreId, MigrationId, Tables)
429 | end,
430 | case Ret of
431 | {ok, ModPriv} ->
432 | State#?MODULE{converter_mod_priv = ModPriv};
433 | Error ->
434 | throw(
435 | ?kmm_error(
436 | converter_mod_error,
437 | #{converter_mod => Mod,
438 | tables => Tables,
439 | error => Error}))
440 | end
441 | catch
442 | Class:Reason:Stacktrace ->
443 | ?kmm_misuse(
444 | converter_mod_exception,
445 | #{converter_mod => Mod,
446 | tables => Tables,
447 | class => Class,
448 | reason => Reason,
449 | stacktrace => Stacktrace})
450 | end.
451 |
452 | subscribe_to_mnesia_changes(
453 | #?MODULE{tables = Tables, subscriber = SubscriberPid}) ->
454 | ?LOG_DEBUG(
455 | "Mnesia->Khepri data copy: subscribe to Mnesia changes",
456 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
457 | case m2k_subscriber:subscribe(SubscriberPid, Tables) of
458 | ok ->
459 | ok;
460 | Error ->
461 | throw(
462 | ?kmm_error(
463 | failed_to_subscribe_to_mnesia_changes,
464 | #{tables => Tables,
465 | error => Error}))
466 | end.
467 |
468 | start_copy_from_mnesia_to_khepri(#?MODULE{tables = Tables} = State) ->
469 | ?LOG_DEBUG(
470 | "Mnesia->Khepri data copy: start actual data copy",
471 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
472 | CheckpointOptions = [{min, Tables},
473 | {ram_overrides_dump, true}],
474 | case mnesia:activate_checkpoint(CheckpointOptions) of
475 | {ok, Checkpoint, _Nodes} ->
476 | Self = self(),
477 | Args = #{table_copy_pid => Self},
478 | BackupPid = spawn_link(
479 | fun() ->
480 | Ret = mnesia:backup_checkpoint(
481 | Checkpoint, Args, m2k_export),
482 | _ = mnesia:deactivate_checkpoint(Checkpoint),
483 | Self ! {self(), done, Ret},
484 | unlink(Self),
485 | exit(normal)
486 | end),
487 | State#?MODULE{backup_pid = BackupPid};
488 | Error ->
489 | throw(
490 | ?kmm_error(
491 | failed_to_activate_mnesia_checkpoint,
492 | #{tables => Tables,
493 | error => Error}))
494 | end.
495 |
496 | handle_migration_records(
497 | #?MODULE{backup_pid = BackupPid,
498 | converter_mod = Mod,
499 | converter_mod_priv = ModPriv,
500 | tables = Tables} = State) ->
501 | receive
502 | {m2k_export, ExportPid, handle_record, Table, Record} ->
503 | {State2, Reply} =
504 | try
505 | ActualMod = actual_mod(Mod),
506 | Ret = case Record of
507 | {_RecordName, Key} ->
508 | ActualMod:delete_from_khepri(
509 | Table, Key, ModPriv);
510 | _ ->
511 | ActualMod:copy_to_khepri(
512 | Table, Record, ModPriv)
513 | end,
514 | case Ret of
515 | {ok, ModPriv1} ->
516 | State1 = State#?MODULE{converter_mod_priv = ModPriv1},
517 | {State1, ok};
518 | Error ->
519 | {State, Error}
520 | end
521 | catch
522 | Class:Reason:Stacktrace ->
523 | Exception = ?kmm_exception(
524 | converter_mod_exception,
525 | #{converter_mod => Mod,
526 | tables => Tables,
527 | class => Class,
528 | reason => Reason,
529 | stacktrace =>
530 | Stacktrace}),
531 | {State, {error, Exception}}
532 | end,
533 | ExportPid ! {self(), record_handled, Reply},
534 | handle_migration_records(State2);
535 | {BackupPid, done, Ret} ->
536 | case Ret of
537 | ok ->
538 | State;
539 | {error,
540 | {'EXIT',
541 | {error,
542 | {error,
543 | {_, {error, ?kmm_exception(_, _) = Exception}}}}}} ->
544 | ?kmm_misuse(Exception);
545 | Error ->
546 | throw(
547 | ?kmm_error(
548 | converter_mod_error,
549 | #{converter_mod => Mod,
550 | error => Error}))
551 | end;
552 | {'EXIT', BackupPid, Reason} ->
553 | throw(
554 | ?kmm_error(
555 | backup_process_error,
556 | #{converter_mod => Mod,
557 | error => Reason}))
558 | after
559 | 15_000 ->
560 | throw(
561 | ?kmm_error(
562 | converter_mod_error,
563 | #{converter_mod => Mod,
564 | error => timeout}))
565 | end.
566 |
567 | final_sync_from_mnesia_to_khepri(
568 | #?MODULE{tables = Tables,
569 | subscriber = SubscriberPid} = State) ->
570 | ?LOG_DEBUG(
571 | "Mnesia->Khepri data copy: final sync",
572 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
573 | %% Switch all tables to read-only. All concurrent and future Mnesia
574 | %% transactions involving a write to one of them will fail with the
575 | %% `{no_exists, Table}' exception.
576 | make_tables_readonly(Tables),
577 |
578 | try
579 | Events = m2k_subscriber:drain(SubscriberPid),
580 | consume_mnesia_events(Events, State)
581 | catch
582 | Class:Reason:Stacktrace ->
583 | make_tables_readwrite(Tables),
584 | erlang:raise(Class, Reason, Stacktrace)
585 | end.
586 |
587 | finish_converter_mod(
588 | #?MODULE{tables = Tables,
589 | converter_mod = Mod,
590 | converter_mod_priv = ModPriv} = State) ->
591 | ActualMod = actual_mod(Mod),
592 | case erlang:function_exported(ActualMod, finish_copy_to_khepri, 1) of
593 | true ->
594 | try
595 | _ = ActualMod:finish_copy_to_khepri(ModPriv),
596 | State#?MODULE{converter_mod_priv = undefined}
597 | catch
598 | Class:Reason:Stacktrace ->
599 | make_tables_readwrite(Tables),
600 | ?kmm_misuse(
601 | converter_mod_exception,
602 | #{converter_mod => Mod,
603 | tables => Tables,
604 | class => Class,
605 | reason => Reason,
606 | stacktrace => Stacktrace})
607 | end;
608 | false ->
609 | State
610 | end.
611 |
612 | actual_mod({Mod, _ModArgs}) when is_atom(Mod) ->
613 | Mod;
614 | actual_mod(Mod) when is_atom(Mod) ->
615 | Mod.
616 |
617 | mark_tables_as_being_migrated(
618 | #?MODULE{khepri_store = StoreId,
619 | migration_id = MigrationId,
620 | progress = Progress}) ->
621 | Path = marker_path(MigrationId),
622 | case khepri:create(StoreId, Path, Progress) of
623 | ok ->
624 | ok;
625 |
626 | {error,
627 | {khepri, mismatching_node,
628 | #{node_props := #{data :=
629 | #migration{
630 | progress = finished}}}}} ->
631 | throw(ok);
632 |
633 | {error,
634 | {khepri, mismatching_node,
635 | #{node_props := #{data :=
636 | #migration{
637 | progress = {in_flight, OtherPid}}}}}} ->
638 | throw({already_started, OtherPid});
639 |
640 | {error, Reason} ->
641 | throw(Reason)
642 | end.
643 |
644 | mark_tables_as_migrated(
645 | #?MODULE{khepri_store = StoreId,
646 | migration_id = MigrationId,
647 | progress = Progress}) ->
648 | Condition = #if_data_matches{pattern = Progress},
649 | Path = marker_path(#if_all{conditions = [MigrationId, Condition]}),
650 | Progress1 = Progress#migration{progress = finished},
651 | case khepri:update(StoreId, Path, Progress1) of
652 | ok -> ok;
653 | {error, Reason} -> throw(Reason)
654 | end.
655 |
656 | clear_migration_marker(
657 | #?MODULE{khepri_store = StoreId,
658 | migration_id = MigrationId,
659 | progress = Progress}) ->
660 | clear_migration_marker(StoreId, MigrationId, Progress).
661 |
662 | clear_migration_marker(StoreId, MigrationId, Pid, Tables) ->
663 | Progress = migration_recorded_state(Pid, Tables),
664 | Condition = #if_data_matches{pattern = Progress},
665 | Path = marker_path(#if_all{conditions = [MigrationId, Condition]}),
666 | _ = khepri:delete(StoreId, Path),
667 | ok.
668 |
669 | clear_migration_marker(StoreId, MigrationId, Progress) ->
670 | Condition = #if_data_matches{pattern = Progress},
671 | Path = marker_path(#if_all{conditions = [MigrationId, Condition]}),
672 | _ = khepri:delete(StoreId, Path),
673 | ok.
674 |
675 | migration_recorded_state(Pid, Tables) when is_pid(Pid) ->
676 | #migration{progress = {in_flight, Pid},
677 | tables = Tables}.
678 |
679 | marker_path(PathComponent) ->
680 | ['__khepri_mnesia_migration', ?MODULE, PathComponent].
681 |
682 | make_tables_readonly(Tables) ->
683 | make_tables_readonly(Tables, []).
684 |
685 | make_tables_readonly([Table | Rest], AlreadyMarked) ->
686 | ?LOG_DEBUG(
687 | "Mnesia->Khepri data copy: mark Mnesia table `~ts` as read-only",
688 | [Table],
689 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
690 | case mnesia:change_table_access_mode(Table, read_only) of
691 | {atomic, ok} ->
692 | make_tables_readonly(Rest, [Table | AlreadyMarked]);
693 | {aborted, {already_exists, _, read_only}} ->
694 | make_tables_readonly(Rest, [Table | AlreadyMarked]);
695 | {aborted, _} = Error ->
696 | _ = make_tables_readwrite(AlreadyMarked),
697 | throw(Error)
698 | end;
699 | make_tables_readonly([], _AlreadyMarked) ->
700 | ok.
701 |
702 | make_tables_readwrite([Table | Rest]) ->
703 | ?LOG_DEBUG(
704 | "Mnesia->Khepri data copy: mark Mnesia table `~ts` as read-write after "
705 | "a failed copy or a rollback",
706 | [Table],
707 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
708 | _ = mnesia:change_table_access_mode(Table, read_write),
709 | make_tables_readwrite(Rest);
710 | make_tables_readwrite([]) ->
711 | ok.
712 |
713 | consume_mnesia_events(
714 | Events,
715 | #?MODULE{tables = Tables,
716 | converter_mod = Mod,
717 | converter_mod_priv = ModPriv} = State) ->
718 | ?LOG_DEBUG(
719 | "Mnesia->Khepri data copy: consuming ~b Mnesia events from tables ~0p",
720 | [length(Events), Tables],
721 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN}),
722 | ActualMod = actual_mod(Mod),
723 | ModPriv1 = consume_mnesia_events1(Events, ActualMod, ModPriv, State),
724 | State#?MODULE{converter_mod_priv = ModPriv1}.
725 |
726 | consume_mnesia_events1(
727 | [{put, Table, Record} | Rest], Mod, ModPriv, State) ->
728 | ModPriv2 = case Mod:copy_to_khepri(Table, Record, ModPriv) of
729 | {ok, ModPriv1} -> ModPriv1;
730 | Error -> throw(Error)
731 | end,
732 | Remaining = length(Rest),
733 | if
734 | Remaining rem 100 =:= 0 ->
735 | ?LOG_DEBUG(
736 | "Mnesia->Khepri data copy: ~b Mnesia events left",
737 | [Remaining],
738 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN});
739 | true ->
740 | ok
741 | end,
742 | consume_mnesia_events1(Rest, Mod, ModPriv2, State);
743 | consume_mnesia_events1(
744 | [{delete, Table, Key} | Rest], Mod, ModPriv, State) ->
745 | ModPriv2 = case Mod:delete_from_khepri(Table, Key, ModPriv) of
746 | {ok, ModPriv1} ->
747 | ModPriv1;
748 | Error ->
749 | throw(Error)
750 | end,
751 | Remaining = length(Rest),
752 | if
753 | Remaining rem 100 =:= 0 ->
754 | ?LOG_DEBUG(
755 | "Mnesia->Khepri data copy: ~b Mnesia events left",
756 | [Remaining],
757 | #{domain => ?KMM_M2K_TABLE_COPY_LOG_DOMAIN});
758 | true ->
759 | ok
760 | end,
761 | consume_mnesia_events1(Rest, Mod, ModPriv2, State);
762 | consume_mnesia_events1([], _Mod, ModPriv, _State) ->
763 | ModPriv.
764 |
--------------------------------------------------------------------------------