[
  {
    "path": ".github/workflows/build.yml",
    "content": "name: Build\non:\n  push:\n    branches:\n      - '*'\n  pull_request:\n    types:\n      - opened\n      - synchronize\njobs:\n  linux:\n    strategy:\n      matrix:\n        platform: [ubuntu-latest]\n        otp-version: [24]\n    runs-on: ${{ matrix.platform }}\n    container:\n      image: erlang:${{ matrix.otp-version }}\n    steps:\n    - name: Checkout\n      uses: actions/checkout@v2\n    - name: Cache Hex packages\n      uses: actions/cache@v1\n      with:\n        path: ~/.cache/rebar3/hex/hexpm/packages\n        key: ${{ runner.os }}-hex-${{ hashFiles(format('{0}{1}', github.workspace, '/rebar.lock')) }}\n        restore-keys: |\n          ${{ runner.os }}-hex-\n    - name: Cache Dialyzer PLTs\n      uses: actions/cache@v1\n      with:\n        path: ~/.cache/rebar3/rebar3_*_plt\n        key: ${{ runner.os }}-dialyzer-${{ hashFiles(format('{0}{1}', github.workspace, '/rebar.config')) }}\n        restore-keys: |\n          ${{ runner.os }}-dialyzer-\n    - name: Compile\n      run: rebar3 compile\n    - name: Generate Dialyzer PLT\n      run: dialyzer --build_plt --apps erts kernel stdlib\n    - name: Run CT Tests\n      run: rebar3 ct\n    - name: Run Checks\n      run: rebar3 do dialyzer, xref\n    - name: Produce Documentation\n      run: rebar3 edoc\n      if: ${{ matrix.otp-version == '24' }}\n"
  },
  {
    "path": ".gitignore",
    "content": ".DS_Store\n.rebar3\n.eunit\n*.o\n*.beam\n*.plt\n*.swp\n*.swo\nebin\nlog\nerl_crash.dump\n.rebar\n_build\n.idea\nrebar3.crashdump\n.edts\n*.coverdata\n*.log\n*.log.*\ndoc\n# Emacs Backup files\n*~\n# Emacs temporary files\n.#*\n*#\n"
  },
  {
    "path": "CODE_OF_CONDUCT.md",
    "content": "# Code of Conduct\n\nMeta has adopted a Code of Conduct that we expect project participants to adhere to.\nPlease read the [full text](https://code.fb.com/codeofconduct/)\nso that you can understand what actions will and will not be tolerated.\n"
  },
  {
    "path": "CONTRIBUTING.md",
    "content": "# Contributing to WhatsApp Raft\nWe want to make contributing to this project as easy and transparent as\npossible.\n\n## Our Development Process\nWe expect to ship changes to existing setup scripts and add new setup scripts on an ongoing basis.\n\n## Pull Requests\nWe actively welcome your pull requests.\n\n1. Fork the repo and create your branch from `main`.\n2. Make sure your changes lint and work with all past and present versions of WhatsApp RAFT.\n3. If you haven't already, complete the Contributor License Agreement (\"CLA\").\n\n## Contributor License Agreement (\"CLA\")\nIn order to accept your pull request, we need you to submit a CLA. You only need\nto do this once to work on any of Meta's open source projects.\n\nComplete your CLA here: <https://code.facebook.com/cla>\n\n## Issues\nWe use GitHub issues to track public bugs. Please ensure your description is\nclear and has sufficient instructions to be able to reproduce the issue.\n\nFor issues on your integration with WhatsApp Raft, please use our\nsupport channel at <https://business.facebook.com/direct-support>.\n\nMeta has a [bounty program](https://www.facebook.com/whitehat/) for the safe\ndisclosure of security bugs. In those cases, please go through the process\noutlined on that page and do not file a public issue.\n\n## License\nBy contributing to WhatsApp Raft, you agree that your contributions will be licensed\nunder the LICENSE file in the root directory of this source tree.\n"
  },
  {
    "path": "LICENSE",
    "content": "Apache License\n                           Version 2.0, January 2004\n                        http://www.apache.org/licenses/\n\n   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION\n\n   1. Definitions.\n\n      \"License\" shall mean the terms and conditions for use, reproduction,\n      and distribution as defined by Sections 1 through 9 of this document.\n\n      \"Licensor\" shall mean the copyright owner or entity authorized by\n      the copyright owner that is granting the License.\n\n      \"Legal Entity\" shall mean the union of the acting entity and all\n      other entities that control, are controlled by, or are under common\n      control with that entity. For the purposes of this definition,\n      \"control\" means (i) the power, direct or indirect, to cause the\n      direction or management of such entity, whether by contract or\n      otherwise, or (ii) ownership of fifty percent (50%) or more of the\n      outstanding shares, or (iii) beneficial ownership of such entity.\n\n      \"You\" (or \"Your\") shall mean an individual or Legal Entity\n      exercising permissions granted by this License.\n\n      \"Source\" form shall mean the preferred form for making modifications,\n      including but not limited to software source code, documentation\n      source, and configuration files.\n\n      \"Object\" form shall mean any form resulting from mechanical\n      transformation or translation of a Source form, including but\n      not limited to compiled object code, generated documentation,\n      and conversions to other media types.\n\n      \"Work\" shall mean the work of authorship, whether in Source or\n      Object form, made available under the License, as indicated by a\n      copyright notice that is included in or attached to the work\n      (an example is provided in the Appendix below).\n\n      \"Derivative Works\" shall mean any work, whether in Source or Object\n      form, that is based on (or derived from) the Work and for which the\n      editorial revisions, annotations, elaborations, or other modifications\n      represent, as a whole, an original work of authorship. For the purposes\n      of this License, Derivative Works shall not include works that remain\n      separable from, or merely link (or bind by name) to the interfaces of,\n      the Work and Derivative Works thereof.\n\n      \"Contribution\" shall mean any work of authorship, including\n      the original version of the Work and any modifications or additions\n      to that Work or Derivative Works thereof, that is intentionally\n      submitted to Licensor for inclusion in the Work by the copyright owner\n      or by an individual or Legal Entity authorized to submit on behalf of\n      the copyright owner. For the purposes of this definition, \"submitted\"\n      means any form of electronic, verbal, or written communication sent\n      to the Licensor or its representatives, including but not limited to\n      communication on electronic mailing lists, source code control systems,\n      and issue tracking systems that are managed by, or on behalf of, the\n      Licensor for the purpose of discussing and improving the Work, but\n      excluding communication that is conspicuously marked or otherwise\n      designated in writing by the copyright owner as \"Not a Contribution.\"\n\n      \"Contributor\" shall mean Licensor and any individual or Legal Entity\n      on behalf of whom a Contribution has been received by Licensor and\n      subsequently incorporated within the Work.\n\n   2. Grant of Copyright License. Subject to the terms and conditions of\n      this License, each Contributor hereby grants to You a perpetual,\n      worldwide, non-exclusive, no-charge, royalty-free, irrevocable\n      copyright license to reproduce, prepare Derivative Works of,\n      publicly display, publicly perform, sublicense, and distribute the\n      Work and such Derivative Works in Source or Object form.\n\n   3. Grant of Patent License. Subject to the terms and conditions of\n      this License, each Contributor hereby grants to You a perpetual,\n      worldwide, non-exclusive, no-charge, royalty-free, irrevocable\n      (except as stated in this section) patent license to make, have made,\n      use, offer to sell, sell, import, and otherwise transfer the Work,\n      where such license applies only to those patent claims licensable\n      by such Contributor that are necessarily infringed by their\n      Contribution(s) alone or by combination of their Contribution(s)\n      with the Work to which such Contribution(s) was submitted. If You\n      institute patent litigation against any entity (including a\n      cross-claim or counterclaim in a lawsuit) alleging that the Work\n      or a Contribution incorporated within the Work constitutes direct\n      or contributory patent infringement, then any patent licenses\n      granted to You under this License for that Work shall terminate\n      as of the date such litigation is filed.\n\n   4. Redistribution. You may reproduce and distribute copies of the\n      Work or Derivative Works thereof in any medium, with or without\n      modifications, and in Source or Object form, provided that You\n      meet the following conditions:\n\n      (a) You must give any other recipients of the Work or\n          Derivative Works a copy of this License; and\n\n      (b) You must cause any modified files to carry prominent notices\n          stating that You changed the files; and\n\n      (c) You must retain, in the Source form of any Derivative Works\n          that You distribute, all copyright, patent, trademark, and\n          attribution notices from the Source form of the Work,\n          excluding those notices that do not pertain to any part of\n          the Derivative Works; and\n\n      (d) If the Work includes a \"NOTICE\" text file as part of its\n          distribution, then any Derivative Works that You distribute must\n          include a readable copy of the attribution notices contained\n          within such NOTICE file, excluding those notices that do not\n          pertain to any part of the Derivative Works, in at least one\n          of the following places: within a NOTICE text file distributed\n          as part of the Derivative Works; within the Source form or\n          documentation, if provided along with the Derivative Works; or,\n          within a display generated by the Derivative Works, if and\n          wherever such third-party notices normally appear. The contents\n          of the NOTICE file are for informational purposes only and\n          do not modify the License. You may add Your own attribution\n          notices within Derivative Works that You distribute, alongside\n          or as an addendum to the NOTICE text from the Work, provided\n          that such additional attribution notices cannot be construed\n          as modifying the License.\n\n      You may add Your own copyright statement to Your modifications and\n      may provide additional or different license terms and conditions\n      for use, reproduction, or distribution of Your modifications, or\n      for any such Derivative Works as a whole, provided Your use,\n      reproduction, and distribution of the Work otherwise complies with\n      the conditions stated in this License.\n\n   5. Submission of Contributions. Unless You explicitly state otherwise,\n      any Contribution intentionally submitted for inclusion in the Work\n      by You to the Licensor shall be under the terms and conditions of\n      this License, without any additional terms or conditions.\n      Notwithstanding the above, nothing herein shall supersede or modify\n      the terms of any separate license agreement you may have executed\n      with Licensor regarding such Contributions.\n\n   6. Trademarks. This License does not grant permission to use the trade\n      names, trademarks, service marks, or product names of the Licensor,\n      except as required for reasonable and customary use in describing the\n      origin of the Work and reproducing the content of the NOTICE file.\n\n   7. Disclaimer of Warranty. Unless required by applicable law or\n      agreed to in writing, Licensor provides the Work (and each\n      Contributor provides its Contributions) on an \"AS IS\" BASIS,\n      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or\n      implied, including, without limitation, any warranties or conditions\n      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A\n      PARTICULAR PURPOSE. You are solely responsible for determining the\n      appropriateness of using or redistributing the Work and assume any\n      risks associated with Your exercise of permissions under this License.\n\n   8. Limitation of Liability. In no event and under no legal theory,\n      whether in tort (including negligence), contract, or otherwise,\n      unless required by applicable law (such as deliberate and grossly\n      negligent acts) or agreed to in writing, shall any Contributor be\n      liable to You for damages, including any direct, indirect, special,\n      incidental, or consequential damages of any character arising as a\n      result of this License or out of the use or inability to use the\n      Work (including but not limited to damages for loss of goodwill,\n      work stoppage, computer failure or malfunction, or any and all\n      other commercial damages or losses), even if such Contributor\n      has been advised of the possibility of such damages.\n\n   9. Accepting Warranty or Additional Liability. While redistributing\n      the Work or Derivative Works thereof, You may choose to offer,\n      and charge a fee for, acceptance of support, warranty, indemnity,\n      or other liability obligations and/or rights consistent with this\n      License. However, in accepting such obligations, You may act only\n      on Your own behalf and on Your sole responsibility, not on behalf\n      of any other Contributor, and only if You agree to indemnify,\n      defend, and hold each Contributor harmless for any liability\n      incurred by, or claims asserted against, such Contributor by reason\n      of your accepting any such warranty or additional liability.\n\n   END OF TERMS AND CONDITIONS\n\n   APPENDIX: How to apply the Apache License to your work.\n\n      To apply the Apache License to your work, attach the following\n      boilerplate notice, with the fields enclosed by brackets \"[]\"\n      replaced with your own identifying information. (Don't include\n      the brackets!)  The text should be enclosed in the appropriate\n      comment syntax for the file format. We also recommend that a\n      file or class name and description of purpose be included on the\n      same \"printed page\" as the copyright notice for easier\n      identification within third-party archives.\n\n   Copyright [yyyy] [name of copyright owner]\n\n   Licensed under the Apache License, Version 2.0 (the \"License\");\n   you may not use this file except in compliance with the License.\n   You may obtain a copy of the License at\n\n       http://www.apache.org/licenses/LICENSE-2.0\n\n   Unless required by applicable law or agreed to in writing, software\n   distributed under the License is distributed on an \"AS IS\" BASIS,\n   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n   See the License for the specific language governing permissions and\n   limitations under the License.\n\n"
  },
  {
    "path": "README.md",
    "content": "# WhatsApp Raft - WARaft\n\nWARaft is a Raft library in Erlang by WhatsApp. It provides an Erlang implementation to obtain consensus among replicated state machines. Consensus is a fundamental problem in fault-tolerant distributed systems. WARaft has been used as consensus provider in WhatsApp message storage, which is a large scale strongly consistent storage system across 5+ datacenters.\n\n## Features\n\n* Full implementation of Raft consensus algorithm defined in https://raft.github.io/\n* Extensible framework. It offers pluggable component interface for log, state machines and transport layer. Users are also allowed provide their own implementation to customize .\n* Performant. It is highly optimized for large volume transactions user cases. It could support up to 200K/s transactions with in a 5 node cluster.\n* Distributed key value store. WARaft provides components needed to build a distributed key-value storage.\n\n## Get Started\n\nThe following code snippet gives a quick glance about how WARaft works. It creates a single-node WARaft cluster and writes and reads a record.\n\n```erlang\n% Setup the WARaft application and the host application\nrr(wa_raft_server).\napplication:ensure_all_started(wa_raft).\napplication:set_env(test_app, raft_database, \".\").\n% Create a spec for partition 1 of the RAFT table \"test\" and start it.\nSpec = wa_raft_sup:child_spec(test_app, [#{table => test, partition => 1}]).\n% Here we add WARaft to the kernel's supervisor, but you should place WARaft's\n% child spec underneath your application's supervisor in a real deployment.\nsupervisor:start_child(kernel_sup, Spec).\n% Check that the RAFT server started successfully\nwa_raft_server:status(raft_server_test_1).\n% Make a cluster configuration with the current node as the only member\nConfig = wa_raft_server:make_config([#raft_identity{name = raft_server_test_1, node = node()}]).\n% Bootstrap the RAFT server to get it started\nwa_raft_server:bootstrap(raft_server_test_1, #raft_log_pos{index = 1, term = 1}, Config, #{}).\n% Wait for the RAFT server to become the leader\nwa_raft_server:status(raft_server_test_1).\n% Read and write against a key\nwa_raft_acceptor:commit(raft_acceptor_test_1, {make_ref(), {write, test, key, 1000}}).\nwa_raft_acceptor:read(raft_acceptor_test_1, {read, test, key}).\n```\n\nA typical output would look like the following:\n\n```erlang\n1> % Setup the WARaft application and the host application\n   rr(wa_raft_server).\n[raft_application,raft_identifier,raft_identity,raft_log,\n raft_log_pos,raft_options,raft_state]\n2> application:ensure_all_started(wa_raft).\n{ok,[wa_raft]}\n3> application:set_env(test_app, raft_database, \".\").\nok\n4> % Create a spec for partition 1 of the RAFT table \"test\" and start it.\n   Spec = wa_raft_sup:child_spec(test_app, [#{table => test, partition => 1}]).\n#{id => wa_raft_sup,restart => permanent,shutdown => infinity,\n  start =>\n      {wa_raft_sup,start_link,\n                   [test_app,[#{table => test,partition => 1}],#{}]},\n  type => supervisor,\n  modules => [wa_raft_sup]}\n5> % Here we add WARaft to the kernel's supervisor, but you should place WARaft's\n   % child spec underneath your application's supervisor in a real deployment.\n   supervisor:start_child(kernel_sup, Spec).\n{ok,<0.101.0>}\n6> % Check that the RAFT server started successfully\n   wa_raft_server:status(raft_server_test_1).\n[{state,stalled},\n {id,nonode@nohost},\n {table,test},\n {partition,1},\n {data_dir,\"./test.1\"},\n {current_term,0},\n {voted_for,undefined},\n {commit_index,0},\n {last_applied,0},\n {leader_id,undefined},\n {next_index,#{}},\n {match_index,#{}},\n {log_module,wa_raft_log_ets},\n {log_first,0},\n {log_last,0},\n {votes,#{}},\n {inflight_applies,0},\n {disable_reason,undefined},\n {config,#{version => 1,membership => [],witness => []}},\n {config_index,0},\n {witness,false}]\n7> % Make a cluster configuration with the current node as the only member\n   Config = wa_raft_server:make_config([#raft_identity{name = raft_server_test_1, node = node()}]).\n#{version => 1,\n  membership => [{raft_server_test_1,nonode@nohost}],\n  witness => []}\n8> % Bootstrap the RAFT server to get it started\n   wa_raft_server:bootstrap(raft_server_test_1, #raft_log_pos{index = 1, term = 1}, Config, #{}).\nok\n9> % Wait for the RAFT server to become the leader\n   wa_raft_server:status(raft_server_test_1).\n[{state,leader},\n {id,nonode@nohost},\n {table,test},\n {partition,1},\n {data_dir,\"./test.1\"},\n {current_term,1},\n {voted_for,nonode@nohost},\n {commit_index,2},\n {last_applied,2},\n {leader_id,nonode@nohost},\n {next_index,#{}},\n {match_index,#{}},\n {log_module,wa_raft_log_ets},\n {log_first,1},\n {log_last,2},\n {votes,#{}},\n {inflight_applies,0},\n {disable_reason,undefined},\n {config,#{version => 1,\n           membership => [{raft_server_test_1,nonode@nohost}],\n           witness => []}},\n {config_index,1},\n {witness,false}]\n10> % Read and write against a key\n    wa_raft_acceptor:commit(raft_acceptor_test_1, {make_ref(), {write, test, key, 1000}}).\nok\n11> wa_raft_acceptor:read(raft_acceptor_test_1, {read, test, key}).\n{ok,1000}\n```\n\nThe [example directory](https://github.com/WhatsApp/waraft/tree/main/examples/kvstore/src) contains an example generic key-value store built on top of WARaft.\n\n## License\n\nWARaft is [Apache licensed](./LICENSE).\n"
  },
  {
    "path": "examples/kvstore/src/kvstore.app.src",
    "content": "%% % @format\n\n%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%\n%% This source code is licensed under the Apache 2.0 license found in\n%% the LICENSE file in the root directory of this source tree.\n\n{application, kvstore, [\n    {description, \"Distributed Key-Value Storage\"},\n    {vsn, \"1.0.0\"},\n    {modules, []},\n    {registered, [kvstore_sup]},\n    {applications, [\n        kernel,\n        stdlib,\n        wa_raft\n    ]},\n    {env, [\n        % Specify where you want your data to be stored here\n        {raft_database, \"/mnt/kvstore\"},\n        % Specify your own implementations here\n        {raft_log_module, wa_raft_log_ets},\n        {raft_storage_module, wa_raft_storage_ets},\n        {raft_distribution_module, wa_raft_distribution},\n        {raft_transport_module, wa_raft_transport}\n    ]},\n    {mod, {kvstore_app, []}}\n]}.\n"
  },
  {
    "path": "examples/kvstore/src/kvstore_app.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n\n-module(kvstore_app).\n-compile(warn_missing_spec_all).\n\n-behaviour(application).\n\n%% API\n-export([\n    start/2,\n    stop/1\n]).\n\n-spec start(application:start_type(), term()) -> {ok, pid()}.\nstart(normal, _Args) ->\n    {ok, _Pid} = kvstore_sup:start_link().\n\n-spec stop(term()) -> ok.\nstop(_State) ->\n    ok.\n"
  },
  {
    "path": "examples/kvstore/src/kvstore_client.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n\n%%%\n%%% This module offers APIs to access the storage.\n%%%\n-module(kvstore_client).\n-compile(warn_missing_spec_all).\n\n-export([\n    read/1,\n    write/2,\n    delete/1\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n-define(CALL_TIMEOUT, 5000).\n-define(TABLE, kvstore).\n-define(NUM_PARTITIONS, 4).\n\n%% Read value for a given key. It's a blocking call.\n-spec read(term()) ->  {ok, term()} | wa_raft_acceptor:read_error().\nread(Key) ->\n    Acceptor = ?RAFT_ACCEPTOR_NAME(?TABLE, partition(Key)),\n    wa_raft_acceptor:read(Acceptor, {read, ?TABLE, Key}, ?CALL_TIMEOUT).\n\n%% Write a key/value pair to storage. It's a blocking call.\n-spec write(term(), term()) ->  ok | wa_raft_acceptor:commit_error().\nwrite(Key, Value) ->\n    commit(Key, {write, ?TABLE, Key, Value}).\n\n%% Delete a key/value pair. It's a blocking call.\n-spec delete(term()) ->  ok | wa_raft_acceptor:commit_error().\ndelete(Key) ->\n    commit(Key, {delete, ?TABLE, Key}).\n\n-spec commit(term(), term()) -> term() | wa_raft_acceptor:commit_error().\ncommit(Key, Command) ->\n    Acceptor = ?RAFT_ACCEPTOR_NAME(?TABLE, partition(Key)),\n    wa_raft_acceptor:commit(Acceptor, {make_ref(), Command}, ?CALL_TIMEOUT).\n\n-spec partition(term()) -> number().\npartition(Key) ->\n    erlang:phash2(Key, ?NUM_PARTITIONS) + 1.\n"
  },
  {
    "path": "examples/kvstore/src/kvstore_sup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n\n%%%\n%%% This supervisor starts 4 RAFT partitions under itself.\n%%%\n\n-module(kvstore_sup).\n-compile(warn_missing_spec_all).\n\n-behaviour(supervisor).\n\n-export([\n    start_link/0,\n    init/1\n]).\n\n-spec start_link() -> supervisor:startlink_ret().\nstart_link() ->\n    supervisor:start_link({local, ?MODULE}, ?MODULE, []).\n\n-spec init(term()) -> {ok, {supervisor:sup_flags(), [supervisor:child_spec()]}}.\ninit([]) ->\n    Partitions = [1, 2, 3, 4],\n    Args = [raft_args(P) || P <- Partitions],\n    ChildSpecs = [\n        wa_raft_sup:child_spec(Args)\n    ],\n    {ok, {#{}, ChildSpecs}}.\n\n% Construct a RAFT \"args\" for a partition.\n-spec raft_args(Partition :: wa_raft:partition()) -> wa_raft:args().\nraft_args(Partition) ->\n    % RAFT clusters are primarily identified by their table and partition number\n    #{table => kvstore, partition => Partition}.\n"
  },
  {
    "path": "include/wa_raft.hrl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This file defines general macros and data structures shared across modules.\n\n%% The name of the RAFT application.\n-define(RAFT_APPLICATION, wa_raft).\n\n%%-------------------------------------------------------------------\n%% Registered information about applications and partitions\n%%-------------------------------------------------------------------\n\n%% Name of the application environment key that is expected to contain a path\n%% to the directory in which RAFT should place the partition paths for the\n%% RAFT partitions under a RAFT client application. This environment value uses\n%% the application search order to determine the value to use.\n-define(RAFT_DATABASE, raft_database).\n%% Default location containing databases for RAFT partitions part of a RAFT client application\n-define(RAFT_DATABASE_PATH(Application), (wa_raft_env:database_path(Application))).\n%% Registered database location for the specified RAFT partition\n-define(RAFT_PARTITION_PATH(Table, Partition), (wa_raft_part_sup:registered_partition_path(Table, Partition))).\n\n%% Registered name of the RAFT partition supervisor for a RAFT partition\n-define(RAFT_SUPERVISOR_NAME(Table, Partition), (wa_raft_part_sup:registered_name(Table, Partition))).\n%% Registered name of the RAFT acceptor server for a RAFT partition\n-define(RAFT_ACCEPTOR_NAME(Table, Partition), (wa_raft_acceptor:registered_name(Table, Partition))).\n%% Registered name of the RAFT log server for a RAFT partition\n-define(RAFT_LOG_NAME(Table, Partition), (wa_raft_log:registered_name(Table, Partition))).\n%% Registered name of the RAFT server for a RAFT partition\n-define(RAFT_SERVER_NAME(Table, Partition), (wa_raft_server:registered_name(Table, Partition))).\n%% Registered name of the RAFT storage server for a RAFT partition\n-define(RAFT_STORAGE_NAME(Table, Partition), (wa_raft_storage:registered_name(Table, Partition))).\n\n%% Default distribution provider module\n-define(RAFT_DEFAULT_DISTRIBUTION_MODULE, wa_raft_distribution).\n%% Default log provider module\n-define(RAFT_DEFAULT_LOG_MODULE, wa_raft_log_ets).\n%% Default storage provider module\n-define(RAFT_DEFAULT_STORAGE_MODULE, wa_raft_storage_ets).\n%% Default module for handling outgoing transports\n-define(RAFT_DEFAULT_TRANSPORT_MODULE, wa_raft_dist_transport).\n%% Default module for log labeling\n-define(RAFT_DEFAULT_LABEL_MODULE, undefined).\n\n%% RAFT election max weight\n-define(RAFT_ELECTION_MAX_WEIGHT, 10).\n%% Raft election default weight\n-define(RAFT_ELECTION_DEFAULT_WEIGHT, ?RAFT_ELECTION_MAX_WEIGHT).\n\n%% Name of server state persist file\n-define(STATE_FILE_NAME, \"state\").\n%% Name prefix for snapshots\n-define(SNAPSHOT_PREFIX, \"snapshot\").\n%% Snapshot name\n-define(SNAPSHOT_NAME(Index, Term), (?SNAPSHOT_PREFIX \".\" ++ integer_to_list(Index) ++ \".\" ++ integer_to_list(Term))).\n\n%% Witness Snapshot name\n-define(WITNESS_SNAPSHOT_NAME(Index, Term), (?SNAPSHOT_PREFIX \".\" ++ integer_to_list(Index) ++ \".\" ++ integer_to_list(Term) ++ \".witness\")).\n\n%% Location of a snapshot\n-define(RAFT_SNAPSHOT_PATH(Path, Name), (filename:join(Path, Name))).\n-define(RAFT_SNAPSHOT_PATH(Table, Partition, Name), ?RAFT_SNAPSHOT_PATH(?RAFT_PARTITION_PATH(Table, Partition), Name)).\n-define(RAFT_SNAPSHOT_PATH(Table, Partition, Index, Term), ?RAFT_SNAPSHOT_PATH(Table, Partition, ?SNAPSHOT_NAME(Index, Term))).\n\n%% [Transport] Atomics - field index for update timestamp\n-define(RAFT_TRANSPORT_ATOMICS_UPDATED_TS, 1).\n%% [Transport] Transport atomics - field count\n-define(RAFT_TRANSPORT_TRANSPORT_ATOMICS_COUNT, 1).\n%% [Transport] File atomics - field count\n-define(RAFT_TRANSPORT_FILE_ATOMICS_COUNT, 1).\n\n-define(READ_OP, '$read').\n\n%%-------------------------------------------------------------------\n%% Metrics\n%%-------------------------------------------------------------------\n\n-define(RAFT_METRICS_MODULE_KEY, {?RAFT_APPLICATION, raft_metrics_module}).\n-define(RAFT_METRICS_MODULE, (persistent_term:get(?RAFT_METRICS_MODULE_KEY, wa_raft_metrics))).\n-define(RAFT_COUNT(Table, Metric), ?RAFT_METRICS_MODULE:count({raft, Table, Metric})).\n-define(RAFT_COUNTV(Table, Metric, Value), ?RAFT_METRICS_MODULE:countv({raft, Table, Metric}, Value)).\n-define(RAFT_GATHER(Table, Metric, Value), ?RAFT_METRICS_MODULE:gather({raft, Table, Metric}, Value)).\n-define(RAFT_GATHER_LATENCY(Table, Metric, Value), ?RAFT_METRICS_MODULE:gather_latency({raft, Table, Metric}, Value)).\n\n%%-------------------------------------------------------------------\n%% Global Configuration\n%%-------------------------------------------------------------------\n\n%% Get global config\n-define(RAFT_CONFIG(Name), (application:get_env(?RAFT_APPLICATION, Name))).\n-define(RAFT_CONFIG(Name, Default), (application:get_env(?RAFT_APPLICATION, Name, Default))).\n\n%% Default metrics module\n-define(RAFT_METRICS_MODULE(), ?RAFT_CONFIG(raft_metrics_module)).\n\n%% Default Call timeout for all cross node gen_server:call\n-define(RAFT_RPC_CALL_TIMEOUT(), ?RAFT_CONFIG(raft_rpc_call_timeout, 10000)).\n%% Default call timeout for storage related operation (we need bigger default since storage can be slower)\n-define(RAFT_STORAGE_CALL_TIMEOUT(), ?RAFT_CONFIG(raft_storage_call_timeout, 60000)).\n\n%% Maximum number of concurrent outgoing snapshot transfers initiated by leaders.\n-define(RAFT_MAX_CONCURRENT_SNAPSHOT_CATCHUP(), ?RAFT_CONFIG(raft_max_snapshot_catchup, 5)).\n%% Maximum number of concurrent incoming snapshot transfers.\n-define(RAFT_MAX_CONCURRENT_INCOMING_SNAPSHOT_TRANSFERS(), ?RAFT_CONFIG(raft_max_incoming_snapshot_transfers, 5)).\n%% Maximum number of concurrent incoming snapshot transfers of witness snapshots.\n-define(RAFT_MAX_CONCURRENT_INCOMING_WITNESS_SNAPSHOT_TRANSFERS(), ?RAFT_CONFIG(raft_max_incoming_witness_snapshot_transfers, 10)).\n\n%% Default cross-node call timeout for heartbeats made for bulk logs catchup\n-define(RAFT_CATCHUP_HEARTBEAT_TIMEOUT(), ?RAFT_CONFIG(raft_catchup_rpc_timeout_ms, 5000)).\n\n%% Number of workers to use for transports\n-define(RAFT_TRANSPORT_THREADS(), ?RAFT_CONFIG(raft_transport_threads, 1)).\n%% Time in seconds after which a transport that has not made progress should be considered failed\n-define(RAFT_TRANSPORT_IDLE_TIMEOUT(), ?RAFT_CONFIG(transport_idle_timeout_secs, 30)).\n\n%% Maximum number of previous inactive transports to retain info for.\n-define(RAFT_TRANSPORT_INACTIVE_INFO_LIMIT(), ?RAFT_CONFIG(raft_transport_inactive_info_limit, 30)).\n\n%% Size in bytes of individual chunks (messages containing file data) to be sent during transports\n%% using the dist transport provider\n-define(RAFT_DIST_TRANSPORT_CHUNK_SIZE(), ?RAFT_CONFIG(dist_transport_chunk_size, 1 * 1024 * 1024)).\n%% Maximum number of chunks that can be sent by the dist transport provider without being\n%% acknowledged by the recipient\n-define(RAFT_DIST_TRANSPORT_MAX_INFLIGHT(), ?RAFT_CONFIG(dist_transport_max_inflight, 4)).\n\n%%-------------------------------------------------------------------\n%% Configuration\n%%-------------------------------------------------------------------\n\n%% Get application-scoped config\n-define(RAFT_APP_CONFIG(App, Name, Default), (wa_raft_env:get_env(App, Name, Default))).\n\n%% Get table-scoped config (supports {table_overrides, #{Table => Value}, AppDefault})\n-define(RAFT_TABLE_CONFIG(App, Table, Name, Default), (wa_raft_env:get_table_env(App, Table, Name, Default))).\n\n%%-------------------------------------------------------------------\n%% Application-scoped Configuration\n%%-------------------------------------------------------------------\n\n%% Whether or not this node is eligible to be leader.\n-define(RAFT_LEADER_ELIGIBLE, raft_leader_eligible).\n-define(RAFT_LEADER_ELIGIBLE(App), (?RAFT_APP_CONFIG(App, ?RAFT_LEADER_ELIGIBLE, true) =/= false)).\n%% Relative \"weight\" at which this node will trigger elections and thereby be elected.\n-define(RAFT_ELECTION_WEIGHT, raft_election_weight).\n-define(RAFT_ELECTION_WEIGHT(App), ?RAFT_APP_CONFIG(App, ?RAFT_ELECTION_WEIGHT, ?RAFT_ELECTION_DEFAULT_WEIGHT)).\n\n%% Time in seconds to retain transport destination directories after use\n-define(RAFT_TRANSPORT_RETAIN_INTERVAL, transport_retain_min_secs).\n-define(RAFT_TRANSPORT_RETAIN_INTERVAL(App), ?RAFT_APP_CONFIG(App, ?RAFT_TRANSPORT_RETAIN_INTERVAL, 300)).\n\n%%-------------------------------------------------------------------\n%% Table-scoped Configuration\n%%-------------------------------------------------------------------\n\n%% Maximum number of pending applies for any single RAFT partition\n-define(RAFT_MAX_PENDING_APPLIES, raft_max_pending_applies).\n-define(RAFT_MAX_PENDING_APPLIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, {?RAFT_MAX_PENDING_APPLIES, raft_apply_queue_max_size}, 1000)).\n%% Maximum bytes of pending applies for any single RAFT partition\n-define(RAFT_MAX_PENDING_APPLY_BYTES, raft_max_pending_apply_bytes).\n-define(RAFT_MAX_PENDING_APPLY_BYTES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_MAX_PENDING_APPLY_BYTES, 32_000_000)).\n%% Maximum number of pending high priority commits for any single RAFT partition\n-define(RAFT_MAX_PENDING_HIGH_PRIORITY_COMMITS, raft_max_pending_high_priority_commits).\n-define(RAFT_MAX_PENDING_HIGH_PRIORITY_COMMITS(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_MAX_PENDING_HIGH_PRIORITY_COMMITS, 1500)).\n%% Maximum number of pending low priority commits for any single RAFT partition\n-define(RAFT_MAX_PENDING_LOW_PRIORITY_COMMITS, raft_max_pending_low_priority_commits).\n-define(RAFT_MAX_PENDING_LOW_PRIORITY_COMMITS(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_MAX_PENDING_LOW_PRIORITY_COMMITS, 250)).\n%% Maximum number of pending reads for any single RAFT partition\n-define(RAFT_MAX_PENDING_READS, raft_max_pending_reads).\n-define(RAFT_MAX_PENDING_READS(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_MAX_PENDING_READS, 5000)).\n\n%% Time in milliseconds during which a leader was unable to replicate heartbeats to a\n%% quorum of followers before considering the leader to be stale.\n-define(RAFT_LEADER_STALE_INTERVAL, raft_max_heartbeat_age_msecs).\n-define(RAFT_LEADER_STALE_INTERVAL(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_LEADER_STALE_INTERVAL, 180 * 1000)).\n%% Interval in milliseconds between heartbeats sent by RAFT leaders with no pending log entries\n-define(RAFT_HEARTBEAT_INTERVAL, raft_heartbeat_interval_ms).\n-define(RAFT_HEARTBEAT_INTERVAL(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_HEARTBEAT_INTERVAL, 120)).\n%% Maximum number of log entries to include in a single heartbeat\n-define(RAFT_HEARTBEAT_MAX_ENTRIES, raft_max_log_entries_per_heartbeat).\n-define(RAFT_HEARTBEAT_MAX_ENTRIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_HEARTBEAT_MAX_ENTRIES, 15)).\n%% Maximum bytes of log entries to include in a single heartbeat\n-define(RAFT_HEARTBEAT_MAX_BYTES, raft_max_heartbeat_size).\n-define(RAFT_HEARTBEAT_MAX_BYTES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_HEARTBEAT_MAX_BYTES, 1 * 1024 * 1024)).\n%% Time in milliseconds to wait to collect pending log entries into a single heartbeat before\n%% triggering a heartbeat due to having pending log entries\n-define(RAFT_COMMIT_BATCH_INTERVAL, raft_commit_batch_interval_ms).\n-define(RAFT_COMMIT_BATCH_INTERVAL(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_COMMIT_BATCH_INTERVAL, 2)).\n%% Maximum number of pending log entries to collect before a heartbeat is forced. This should\n%% be at most equal to the maximum number of log entries permitted per heartbeat.\n-define(RAFT_COMMIT_BATCH_MAX_ENTRIES, raft_commit_batch_max).\n-define(RAFT_COMMIT_BATCH_MAX_ENTRIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_COMMIT_BATCH_MAX_ENTRIES, 15)).\n%% Maximum number of log entries to speculatively retain in the log due to followers\n%% not yet reporting having replicated the log entry locally\n-define(RAFT_MAX_RETAINED_ENTRIES, raft_max_retained_entries).\n-define(RAFT_MAX_RETAINED_ENTRIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, {?RAFT_MAX_RETAINED_ENTRIES, max_log_rotate_delay}, 1500000)).\n\n%% Maximum number of log entries to queue for application by storage at once before\n%% continuing to process the incoming message queue on the RAFT server.\n-define(RAFT_MAX_CONSECUTIVE_APPLY_ENTRIES, raft_apply_log_batch_size).\n-define(RAFT_MAX_CONSECUTIVE_APPLY_ENTRIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_MAX_CONSECUTIVE_APPLY_ENTRIES, 200)).\n%% Maximum bytes of log entries to queue for application by storage at once before\n%% continuing to process the incoming message queue on the RAFT server.\n-define(RAFT_MAX_CONSECUTIVE_APPLY_BYTES, raft_apply_batch_max_bytes).\n-define(RAFT_MAX_CONSECUTIVE_APPLY_BYTES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_MAX_CONSECUTIVE_APPLY_BYTES, 200 * 4 * 1024)).\n\n%% Minimum time in milliseconds since the receiving the last valid leader heartbeat\n%% before triggering a new election due to term timeout. This time should be much\n%% greater than the maximum expected network delay.\n-define(RAFT_ELECTION_TIMEOUT_MIN, raft_election_timeout_ms).\n-define(RAFT_ELECTION_TIMEOUT_MIN(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_ELECTION_TIMEOUT_MIN, 5000)).\n%% Maximum time in milliseconds since the receiving the last valid leader heartbeat\n%% before triggering a new election due to term timeout. The difference between this\n%% time and the minimum election timeout should be much greater than the expected\n%% variance in network delay.\n-define(RAFT_ELECTION_TIMEOUT_MAX, raft_election_timeout_ms_max).\n-define(RAFT_ELECTION_TIMEOUT_MAX(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_ELECTION_TIMEOUT_MAX, 7500)).\n\n%% The maximum time in milliseconds during which a leader can continue to be considered live without\n%% receiving an updated heartbeat response quorum from replicas or during which a follower or witness\n%% can be considered live without receiving a heartbeat from a valid leader of the current term.\n-define(RAFT_LIVENESS_GRACE_PERIOD_MS, raft_liveness_grace_period_ms).\n-define(RAFT_LIVENESS_GRACE_PERIOD_MS(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_LIVENESS_GRACE_PERIOD_MS, 30_000)).\n%% The maximum number of log entries that can be not yet applied to a follower or witnesse's log\n%% compared to the leader's commit index before the replica is considered stale.\n-define(RAFT_STALE_GRACE_PERIOD_ENTRIES, raft_stale_grace_period_entries).\n-define(RAFT_STALE_GRACE_PERIOD_ENTRIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_STALE_GRACE_PERIOD_ENTRIES, 5_000)).\n\n%% Minium amount of time in seconds since the last successfully received\n%% heartbeat from a leader of a term for non-forced promotion to be allowed.\n-define(RAFT_PROMOTION_GRACE_PERIOD, raft_promotion_grace_period_secs).\n-define(RAFT_PROMOTION_GRACE_PERIOD(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_PROMOTION_GRACE_PERIOD, 60)).\n\n%% Maximum number of log entries to include in a Handover RPC to pass\n%% leadership to another peer. A limit is enforced to prevent a handover\n%% trying to send huge numbers of logs to catchup a peer during handover.\n-define(RAFT_HANDOVER_MAX_ENTRIES, raft_max_handover_log_entries).\n-define(RAFT_HANDOVER_MAX_ENTRIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_HANDOVER_MAX_ENTRIES, 200)).\n%% Maximum number of total log entries from the leader's current log that a\n%% peer has not yet confirmed to be applied. This limit helps prevent nodes who\n%% may have already received all the current log entries but are behind in\n%% actually applying them to the underlying storage from becoming leader due to\n%% handover before they are ready. This defaults to equal to the maximum number\n%% of missing log entries. (See `?RAFT_HANDOVER_MAX_ENTRIES`.)\n-define(RAFT_HANDOVER_MAX_UNAPPLIED_ENTRIES, raft_handover_max_unapplied_entries).\n-define(RAFT_HANDOVER_MAX_UNAPPLIED_ENTRIES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_HANDOVER_MAX_UNAPPLIED_ENTRIES, undefined)).\n%% Maximum total byte size of log entries to include in a Handover RPC.\n-define(RAFT_HANDOVER_MAX_BYTES, raft_max_handover_log_size).\n-define(RAFT_HANDOVER_MAX_BYTES(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_HANDOVER_MAX_BYTES, 50 * 1024 * 1024)).\n%% Time in milliseconds to wait before considering a previously triggered handover failed.\n-define(RAFT_HANDOVER_TIMEOUT, raft_handover_timeout_ms).\n-define(RAFT_HANDOVER_TIMEOUT(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_HANDOVER_TIMEOUT, 600)).\n\n%% Minimum nubmer of log entries past the minimum kept by the RAFT server before triggering\n%% log rotation\n-define(RAFT_LOG_ROTATION_INTERVAL, raft_max_log_records_per_file).\n-define(RAFT_LOG_ROTATION_INTERVAL(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_LOG_ROTATION_INTERVAL, 200000)).\n%% Maximum number of log entries past the minimum kept by the RAFT server to retain in\n%% the log after rotation\n-define(RAFT_LOG_ROTATION_KEEP, raft_max_log_records).\n-define(RAFT_LOG_ROTATION_KEEP(App, Table, Interval), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_LOG_ROTATION_KEEP, Interval * 10)).\n%% Whether log rotation should be controlled by local log length or by\n%% leader-announced cluster trimming index\n-define(RAFT_LOG_ROTATION_BY_TRIM_INDEX, raft_rotate_by_trim_index).\n-define(RAFT_LOG_ROTATION_BY_TRIM_INDEX(App, Table), (?RAFT_TABLE_CONFIG(App, Table, {?RAFT_LOG_ROTATION_BY_TRIM_INDEX, use_trim_index}, false) =:= true)).\n\n%% Whether or not the log should return entries in external term format\n%% when log entries are fetched for heartbeats\n-define(RAFT_LOG_HEARTBEAT_BINARY_ENTRIES, raft_log_heartbeat_binary_entries).\n-define(RAFT_LOG_HEARTBEAT_BINARY_ENTRIES(App, Table),\n    (?RAFT_TABLE_CONFIG(App, Table, ?RAFT_LOG_HEARTBEAT_BINARY_ENTRIES, false) =:= true)\n).\n\n%% The number of log entries that have yet to be applied on a follower after\n%% which leaders should send a storage snapshot in lieu of continuing regular\n%% replication using log entries in heartbeats.\n-define(RAFT_SNAPSHOT_CATCHUP_THRESHOLD, raft_snapshot_catchup_threshold).\n-define(RAFT_SNAPSHOT_CATCHUP_THRESHOLD(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_SNAPSHOT_CATCHUP_THRESHOLD, 100000)).\n%% Number of milliseconds to wait before attempting to send a new storage snapshot\n%% to a follower that previously rejected a snapshot due to being overloaded.\n-define(RAFT_SNAPSHOT_CATCHUP_OVERLOADED_BACKOFF_MS, raft_snapshot_catchup_overloaded_backoff_ms).\n-define(RAFT_SNAPSHOT_CATCHUP_OVERLOADED_BACKOFF_MS(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_SNAPSHOT_CATCHUP_OVERLOADED_BACKOFF_MS, 1000)).\n%% Number of milliseconds to wait before attempting to send a new storage snapshot\n%% to a follower that previously successfully received a storage snapshot.\n-define(RAFT_SNAPSHOT_CATCHUP_COMPLETED_BACKOFF_MS, raft_snapshot_catchup_completed_backoff_ms).\n-define(RAFT_SNAPSHOT_CATCHUP_COMPLETED_BACKOFF_MS(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_SNAPSHOT_CATCHUP_COMPLETED_BACKOFF_MS, 20 * 1000)).\n%% Number of milliseconds to wait before attempting to send a new storage snapshot\n%% to a follower that previously failed to receive a storage snapshot.\n-define(RAFT_SNAPSHOT_CATCHUP_FAILED_BACKOFF_MS, raft_snapshot_catchup_failed_backoff_ms).\n-define(RAFT_SNAPSHOT_CATCHUP_FAILED_BACKOFF_MS(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_SNAPSHOT_CATCHUP_FAILED_BACKOFF_MS, 10 * 1000)).\n\n%% Number of omitted log entries to skip actually applying to storage when\n%% operating as a witness.\n-define(RAFT_STORAGE_WITNESS_APPLY_INTERVAL, raft_storage_witness_apply_interval).\n-define(RAFT_STORAGE_WITNESS_APPLY_INTERVAL(App, Table), ?RAFT_TABLE_CONFIG(App, Table, ?RAFT_STORAGE_WITNESS_APPLY_INTERVAL, 5000)).\n\n%% Whether or not the storage server should request more log entries\n%% when the apply queue is empty.\n-define(RAFT_STORAGE_NOTIFY_COMPLETE, raft_storage_notify_complete).\n-define(RAFT_STORAGE_NOTIFY_COMPLETE(App, Table), (?RAFT_TABLE_CONFIG(App, Table, ?RAFT_STORAGE_NOTIFY_COMPLETE, true) =:= true)).\n\n%%-------------------------------------------------------------------\n%% Records\n%%-------------------------------------------------------------------\n\n%% Log position\n-record(raft_log_pos, {\n    %% log sequence number\n    index = 0 :: wa_raft_log:log_index(),\n    %% leader's term when log entry is created\n    term = 0 :: wa_raft_log:log_term()\n}).\n\n%% Log handle.\n-record(raft_log, {\n    name :: wa_raft_log:log_name(),\n    application :: atom(),\n    table :: wa_raft:table(),\n    partition :: wa_raft:partition(),\n    provider :: module()\n}).\n\n%% This record represents the identity of a RAFT replica, usable to\n%% distinguish different RAFT replicas from one another. This record\n%% is not guaranteed to remain structurally compatible between versions\n%% of RAFT and so should not be persisted between runtimes nor sent\n%% between RAFT servers. It is generally allowed to inspect the fields\n%% of this record, however, similarly, this record is subject to change\n%% at any time.\n-record(raft_identity, {\n    % The service name (registered name) of the RAFT server that this\n    % identity record refers to.\n    name :: atom(),\n    % The node that the RAFT server that this identity record refers\n    % to is located on.\n    node :: node()\n}).\n\n%% This record represents a RAFT instance identifier.\n-record(raft_identifier, {\n    application :: atom(),\n    table :: wa_raft:table(),\n    partition :: wa_raft:partition()\n}).\n\n%%-------------------------------------------------------------------\n%% Records for registered application and partition information\n%%-------------------------------------------------------------------\n\n%% Information about an application that has started a RAFT supervisor.\n-record(raft_application, {\n    % Application name\n    name :: atom(),\n    % Config search path\n    config_search_apps :: [atom()]\n}).\n\n%% Normalized options produced by `wa_raft_part_sup` for passing into RAFT processes.\n%% Not to be created externally.\n-record(raft_options, {\n    % General options\n    application :: atom(),\n    table :: wa_raft:table(),\n    partition :: wa_raft:partition(),\n    self :: #raft_identity{},\n    identifier :: #raft_identifier{},\n    database :: file:filename(),\n\n    % Acceptor options\n    acceptor_name :: atom(),\n\n    % Distribution options\n    distribution_module :: module(),\n\n    % Label options\n    label_module :: undefined | module(),\n\n    % Log options\n    log_name :: atom(),\n    log_module :: module(),\n\n    % Queue options\n    queue_name :: atom(),\n    queue_counters :: atomics:atomics_ref(),\n    queue_reads :: atom(),\n\n    % Server options\n    server_name :: atom(),\n\n    % Storage options\n    storage_name :: atom(),\n    storage_module :: module(),\n\n    % Partition supervisor options\n    supervisor_name :: atom(),\n\n    % Transport options\n    transport_cleanup_name :: atom(),\n    transport_directory :: file:filename(),\n    transport_module :: module()\n}).\n\n%%-------------------------------------------------------------------\n%% Internal server states\n%%-------------------------------------------------------------------\n\n%% Raft runtime state\n-record(raft_state, {\n    %% Owning application\n    application :: atom(),\n    %% RAFT server name\n    name :: atom(),\n    %% RAFT server's cluster identity\n    self :: #raft_identity{},\n    %% RAFT replica's local identifier\n    identifier :: #raft_identifier{},\n    %% Table name\n    table :: wa_raft:table(),\n    %% Partition number\n    partition :: wa_raft:partition(),\n    %% Local path to partition data\n    partition_path :: string(),\n\n    %% Current view into this RAFT replica's log state\n    log_view :: wa_raft_log:view(),\n    %% Current queue handle\n    queues :: wa_raft_queue:queues(),\n\n    %% Active module for distribution of RPCs\n    distribution_module :: module(),\n    %% Active module for labeling of log entries\n    label_module :: module() | undefined,\n\n    %% Name of this RAFT replica's storage server\n    storage :: atom(),\n\n    %% The index of the latest log entry in the local log that is known to\n    %% match the log entries committed by the cluster\n    commit_index = 0 :: non_neg_integer(),\n    %% The index of the latest log entry that has been sent to storage to be\n    %% applied\n    last_applied = 0 :: non_neg_integer(),\n\n    %% The most recently written RAFT configuration and the index at which it\n    %% was written if a configuration exists in storage\n    cached_config :: undefined | {wa_raft_log:log_index(), wa_raft_server:config()},\n    %% [Leader] The label of the last log entry in the current log\n    last_label :: undefined | term(),\n    %% The timestamp (milliseconds monotonic clock) of the most recently\n    %% received (follower) or sent (leader) heartbeat.\n    leader_heartbeat_ts :: undefined | integer(),\n\n    %% The largest RAFT term that has been observed in the cluster or reached\n    %% by this RAFT replica\n    current_term = 0 :: non_neg_integer(),\n    %% The peer that this RAFT replica voted for in the current term\n    voted_for :: undefined | node(),\n    %% The affirmative votes for leadership this RAFT replica has received from\n    %% the cluster in the current term\n    votes = #{} :: #{node() => true},\n    %% The leader of the current RAFT term if known\n    leader_id :: undefined | node(),\n\n    %% The timestamp (milliseconds monotonic clock) that the current state of\n    %% this RAFT replica was reached\n    state_start_ts :: non_neg_integer(),\n\n    %% [Leader] The list of pending operations in the current commit batch\n    %%          that are in queue to be appended and replicated after a short\n    %%          wait to see if multiple commits can be handled at once to\n    %%          reduce overhead\n    pending_high = [] :: [{gen_server:from(), wa_raft_acceptor:op()}],\n\n    pending_low = [] :: [{gen_server:from(), wa_raft_acceptor:op()}],\n\n    %% [Leader] Whether or not a read has been accepted and is waiting for the\n    %%          leader to establish a new quorum to be handled.\n    pending_read = false :: boolean(),\n    %% [Leader] The queue of accepted commit requests that are waiting to be\n    %%          committed and applied for response to the client.\n    queued = #{} :: #{wa_raft_log:log_index() => {gen_server:from(), wa_raft_acceptor:priority()}},\n    %% [Leader] The index of the next log entry to send in the next heartbeat\n    %%          to each peer\n    next_indices = #{} :: #{node() => wa_raft_log:log_index()},\n    %% [Leader] The index of the latest log entry in each peer's log that is\n    %%          confirmed by a heartbeat response to match the local log\n    match_indices = #{} :: #{node() => wa_raft_log:log_index()},\n    %% [Leader] The index of the latest log entry that has been applied to\n    %%          each peer's underlying storage state\n    last_applied_indices = #{} :: #{node() => wa_raft_log:log_index()},\n\n    %% [Leader] The timestamp (milliseconds monotonic clock) of the last time\n    %%          each peer was sent a heartbeat\n    last_heartbeat_ts = #{} :: #{node() => integer()},\n    %% [Leader] The timestamp (milliseconds monotonic clock) of the last time\n    %%          each peer responded to this RAFT replica with a heartbeat\n    %%          response\n    heartbeat_response_ts = #{} :: #{node() => integer()},\n    %% [Leader] The log index of the first log entry appended to the log that\n    %%          has a log term matching the current term\n    first_current_term_log_index = 0 :: wa_raft_log:log_index(),\n    %% [Leader] Information about a currently pending handover of leadership to\n    %%          a peer\n    handover :: undefined | {node(), reference(), integer()},\n\n    %% [Disabled] The reason for which this RAFT replica was disabled\n    disable_reason :: term()\n}).\n"
  },
  {
    "path": "include/wa_raft_logger.hrl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n\n-include_lib(\"kernel/include/logger.hrl\").\n\n-define(RAFT_LOG_OPTS, #{domain => [whatsapp, wa_raft]}).\n\n-define(RAFT_LOG_ERROR(Message), ?LOG_ERROR(Message, ?RAFT_LOG_OPTS)).\n-define(RAFT_LOG_ERROR(Format, Args), ?LOG_ERROR(Format, Args, ?RAFT_LOG_OPTS)).\n\n-define(RAFT_LOG_WARNING(Message), ?LOG_WARNING(Message, ?RAFT_LOG_OPTS)).\n-define(RAFT_LOG_WARNING(Format, Args), ?LOG_WARNING(Format, Args, ?RAFT_LOG_OPTS)).\n\n-define(RAFT_LOG_NOTICE(Message), ?LOG_NOTICE(Message, ?RAFT_LOG_OPTS)).\n-define(RAFT_LOG_NOTICE(Format, Args), ?LOG_NOTICE(Format, Args, ?RAFT_LOG_OPTS)).\n\n-define(RAFT_LOG_INFO(Message), ?LOG_INFO(Message, ?RAFT_LOG_OPTS)).\n-define(RAFT_LOG_INFO(Format, Args), ?LOG_INFO(Format, Args, ?RAFT_LOG_OPTS)).\n\n-define(RAFT_LOG_DEBUG(Message), ?LOG_DEBUG(Message, ?RAFT_LOG_OPTS)).\n-define(RAFT_LOG_DEBUG(Format, Args), ?LOG_DEBUG(Format, Args, ?RAFT_LOG_OPTS)).\n\n-define(RAFT_LOG(Level, Message), ?LOG(Level, Message, ?RAFT_LOG_OPTS)).\n-define(RAFT_LOG(Level, Format, Args), ?LOG(Level, Format, Args, ?RAFT_LOG_OPTS)).\n"
  },
  {
    "path": "include/wa_raft_rpc.hrl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This file contains macros defining the form of all RPCs and API\n%%% calls used as part of the RAFT protocol and RAFT server and storage API.\n\n%%-------------------------------------------------------------------\n%% RAFT Server RPC Formats\n%%-------------------------------------------------------------------\n%% As the RAFT process that is intended to performs the cross-node\n%% communication required to provide durability against failure,\n%% RAFT servers across nodes must agree on the RPC formats in use.\n%% This means that RPC formats should not be changed once created.\n%%-------------------------------------------------------------------\n\n-define(RAFT_NAMED_RPC(Type, Term, SenderName, SenderNode, Payload), {rpc, Type, Term, SenderName, SenderNode, Payload}).\n\n%% These two RPCs are used by RAFT catchup to receive the status of\n%% the RAFT server being sent to and should not change.\n-define(LEGACY_RAFT_RPC(Type, Term, SenderId, Payload), {rpc, Type, Term, SenderId, Payload}).\n-define(LEGACY_APPEND_ENTRIES_RESPONSE_RPC(Term, SenderId, PrevLogIndex, Success, LastIndex),\n    ?LEGACY_RAFT_RPC(append_entries_response, Term, SenderId, {PrevLogIndex, Success, LastIndex})).\n\n%%-------------------------------------------------------------------\n%% RAFT Server Procedures\n%%-------------------------------------------------------------------\n%% An RPC received from a peer is intended to trigger one of the\n%% procedures listed below.\n%%-------------------------------------------------------------------\n\n-define(APPEND_ENTRIES,          append_entries).\n-define(APPEND_ENTRIES_RESPONSE, append_entries_response).\n-define(REQUEST_VOTE,            request_vote).\n-define(VOTE,                    vote).\n-define(HANDOVER,                handover).\n-define(HANDOVER_FAILED,         handover_failed).\n-define(NOTIFY_TERM,             notify_term).\n\n%% Definitions of each of the standard procedures.\n-define(PROCEDURE(Type, Payload), {procedure, Type, Payload}).\n-define(APPEND_ENTRIES(PrevLogIndex, PrevLogTerm, Entries, CommitIndex, TrimIndex),   ?PROCEDURE(?APPEND_ENTRIES, {PrevLogIndex, PrevLogTerm, Entries, CommitIndex, TrimIndex})).\n-define(APPEND_ENTRIES_RESPONSE(PrevLogIndex, Success, MatchIndex, LastAppliedIndex), ?PROCEDURE(?APPEND_ENTRIES_RESPONSE, {PrevLogIndex, Success, MatchIndex, LastAppliedIndex})).\n-define(REQUEST_VOTE(ElectionType, LastLogIndex, LastLogTerm),                        ?PROCEDURE(?REQUEST_VOTE, {ElectionType, LastLogIndex, LastLogTerm})).\n-define(VOTE(Vote),                                                                   ?PROCEDURE(?VOTE, {Vote})).\n-define(HANDOVER(Ref, PrevLogIndex, PrevLogTerm, Entries),                            ?PROCEDURE(?HANDOVER, {Ref, PrevLogIndex, PrevLogTerm, Entries})).\n-define(HANDOVER_FAILED(Ref),                                                         ?PROCEDURE(?HANDOVER_FAILED, {Ref})).\n-define(NOTIFY_TERM(),                                                                ?PROCEDURE(?NOTIFY_TERM, {})).\n\n%% A request to execute a particular procedure. This request could\n%% have been issued locally or as a result of a remote procedure\n%% call. The peer (if exists and could be oneself) that issued the\n%% procedure call will be provided as the sender.\n-define(REMOTE(Sender, Call), {remote, Sender, Call}).\n\n%%-------------------------------------------------------------------\n%% RAFT Server Internal Events\n%%-------------------------------------------------------------------\n%% An event produced internally within the RAFT server.\n%%-------------------------------------------------------------------\n\n-define(ADVANCE_TERM(Term), {advance_term, Term}).\n-define(FORCE_ELECTION(Term), {force_election, Term}).\n\n%%-------------------------------------------------------------------\n%% RAFT Server API\n%%-------------------------------------------------------------------\n%% The RAFT server also accepts commands issued from other processes\n%% on the local node. These commands are not guaranteed to have the\n%% same format between versions and so should only be used locally.\n%% Prefer to use `wa_raft_server` module exports when possible.\n%%-------------------------------------------------------------------\n\n-define(RAFT_COMMAND(Type, Payload), {command, Type, Payload}).\n\n-define(COMMIT_COMMAND(From, Op, Priority),         ?RAFT_COMMAND(commit, {From, Op, Priority})).\n-define(READ_COMMAND(Op),                           ?RAFT_COMMAND(read, Op)).\n\n-define(CURRENT_CONFIG_COMMAND,                     ?RAFT_COMMAND(current_config, undefined)).\n-define(STATUS_COMMAND,                             ?RAFT_COMMAND(status, undefined)).\n-define(TRIGGER_ELECTION_COMMAND(TermOrOffset),     ?RAFT_COMMAND(trigger_election, {TermOrOffset})).\n-define(PROMOTE_COMMAND(TermOrOffset, Force),       ?RAFT_COMMAND(promote, {TermOrOffset, Force})).\n-define(RESIGN_COMMAND,                             ?RAFT_COMMAND(resign, undefined)).\n\n-define(ADJUST_CONFIG_COMMAND(Action, Index),       ?ADJUST_CONFIG_COMMAND(undefined, Action, Index)).\n-define(ADJUST_CONFIG_COMMAND(From, Action, Index), ?RAFT_COMMAND(adjust_config, {From, Action, Index})).\n-define(REFRESH_CONFIG_COMMAND(),                   ?RAFT_COMMAND(refresh_config, undefined)).\n\n-define(SNAPSHOT_AVAILABLE_COMMAND(Root, Position), ?RAFT_COMMAND(snapshot_available, {Root, Position})).\n\n-define(HANDOVER_CANDIDATES_COMMAND,                ?RAFT_COMMAND(handover_candidates, undefined)).\n-define(HANDOVER_COMMAND(Peer),                     ?RAFT_COMMAND(handover, Peer)).\n-define(IS_PEER_READY_COMMAND(Peer),                ?RAFT_COMMAND(is_peer_ready, Peer)).\n\n-define(ENABLE_COMMAND,                             ?RAFT_COMMAND(enable, undefined)).\n-define(DISABLE_COMMAND(Reason),                    ?RAFT_COMMAND(disable, Reason)).\n\n-define(BOOTSTRAP_COMMAND(Position, Config, Data),  ?RAFT_COMMAND(bootstrap, {Position, Config, Data})).\n\n-define(NOTIFY_COMPLETE_COMMAND(),                  ?RAFT_COMMAND(notify_complete, undefined)).\n"
  },
  {
    "path": "rebar.config",
    "content": "{erl_opts, [ debug_info\n           , warnings_as_errors\n           , warn_export_vars\n           , warn_unused_import\n           ]\n}.\n\n{deps, []}.\n\n{dialyzer, [ {warnings, [unknown]}\n           , {plt_apps, all_deps}\n           ]}.\n\n{xref_checks, [ undefined_function_calls\n              , undefined_functions\n              , locals_not_used\n              , deprecated_function_calls\n              , deprecated_functions\n              ]}.\n"
  },
  {
    "path": "src/wa_raft.app.src",
    "content": "%% % @format\n\n%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n\n{application, wa_raft, [\n    {description, \"Erlang implementation of RAFT Consensus Protocol\"},\n    {vsn, \"1.0.0\"},\n    {modules, []},\n    %% NOTE: No more dependency is expected for this app\n    {applications, [\n        kernel,\n        stdlib\n    ]},\n    {env, []},\n    {mod, {wa_raft_app, []}}\n]}.\n"
  },
  {
    "path": "src/wa_raft.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This file defines dialyzer types.\n\n-module(wa_raft).\n-compile(warn_missing_spec_all).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n%% Public Types\n-export_type([\n    table/0,\n    partition/0,\n    args/0,\n    identity/0\n]).\n\n-type table() :: atom().\n-type partition() :: pos_integer().\n\n%% Specification for starting a RAFT partition.\n-type args() ::\n    #{\n        % Table name\n        table := table(),\n        % Partition number\n        partition := partition(),\n        % Distribution module\n        distribution_module => module(),\n        % Log module\n        log_module => module(),\n        % Log label module\n        label_module => module(),\n        % Storage module\n        storage_module => module(),\n        % Transport module\n        transport_module => module()\n    }.\n\n-type identity() :: #raft_identity{}.\n"
  },
  {
    "path": "src/wa_raft_acceptor.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module implements the front-end process for accepting commits / reads\n\n-module(wa_raft_acceptor).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n%% OTP supervisor\n-export([\n    child_spec/1,\n    start_link/1\n]).\n\n%% Client API - data access\n-export([\n    commit/2,\n    commit/3,\n    commit/4,\n    commit_async/3,\n    commit_async/4,\n    read/2,\n    read/3\n]).\n\n%% Client API - RAFT apis\n-export([\n    adjust_config/3,\n    adjust_config_async/3,\n    adjust_config_async/4\n]).\n\n%% Internal API\n-export([\n    default_name/2,\n    registered_name/2\n]).\n\n%% gen_server callbacks\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    terminate/2\n]).\n\n-export_type([\n    command/0,\n    key/0,\n    op/0,\n    read_op/0,\n    priority/0\n]).\n\n-export_type([\n    call_error_type/0,\n    call_error/0,\n    call_result/0,\n    read_error/0,\n    read_error_type/0,\n    read_result/0,\n    commit_error_type/0,\n    commit_error/0,\n    commit_result/0\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%% Request type macros\n-define(READ_REQUEST(Command), {read, Command}).\n-define(COMMIT_REQUEST(Op, Priority), {commit, Op, Priority}).\n-define(COMMIT_ASYNC_REQUEST(From, Op, Priority), {commit, From, Op, Priority}).\n\n%% Commit op type macros\n-define(OP_DEFAULT(Op), {default, Op}).\n-define(OP_ADJUST_CONFIG(Action, Index), {adjust_config, Action, Index}).\n\n-type command() :: noop_command() | noop_omitted_command() | config_command() | dynamic().\n-type noop_command() :: noop.\n-type noop_omitted_command() :: noop_omitted.\n-type config_command() :: {config, Config :: wa_raft_server:config()}.\n\n-type key() :: term().\n-type op() :: {Key :: key(), Command :: command()}.\n-type read_op() :: {From :: gen_server:from(), Command :: command()}.\n-type priority() :: high | low.\n\n-type call_error_type() :: timeout | unreachable | {call_error, Reason :: term()}.\n-type call_error() :: {error, call_error_type()}.\n-type call_result() :: Result :: dynamic() | Error :: call_error().\n\n-type read_request() :: ?READ_REQUEST(Command :: command()).\n-type read_error_type() :: not_leader | read_queue_full | apply_queue_full | {notify_redirect, Peer :: node()}.\n-type read_error() :: {error, read_error_type()}.\n-type read_result() :: Result :: dynamic() | Error :: read_error() | call_error().\n\n-type commit_op() :: default_op() | adjust_config_op().\n-type default_op() :: ?OP_DEFAULT(Op :: op()).\n-type adjust_config_op() :: ?OP_ADJUST_CONFIG(Action :: wa_raft_server:config_action(), Index :: wa_raft_log:log_index() | undefined).\n-type commit_request() :: ?COMMIT_REQUEST(Op :: commit_op(), Priority :: priority()).\n-type commit_async_request() :: ?COMMIT_ASYNC_REQUEST(From :: gen_server:from(), Op :: commit_op(), Priority :: priority()).\n-type commit_error_type() ::\n    not_supported |\n    not_leader |\n    commit_queue_full |\n    apply_queue_full |\n    {notify_redirect, Peer :: node()} |\n    commit_stalled |\n    cancelled.\n-type commit_error() :: {error, commit_error_type()}.\n-type commit_result() :: Result :: dynamic() | Error :: commit_error() | call_error().\n\n%% Acceptor state\n-record(state, {\n    % Acceptor service name\n    name :: atom(),\n    % RAFT table\n    table :: wa_raft:table(),\n    % Server service name\n    server :: atom(),\n    % Queues handle\n    queues :: wa_raft_queue:queues()\n}).\n\n%%-------------------------------------------------------------------\n%% OTP Supervision\n%%-------------------------------------------------------------------\n\n%%-------------------------------------------------------------------\n%% OTP Supervision\n%%-------------------------------------------------------------------\n\n-spec child_spec(Options :: #raft_options{}) -> supervisor:child_spec().\nchild_spec(Options) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Options]},\n        restart => transient,\n        shutdown => 30000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Options :: #raft_options{}) -> gen_server:start_ret().\nstart_link(#raft_options{acceptor_name = Name} = Options) ->\n    gen_server:start_link({local, Name}, ?MODULE, Options, []).\n\n%%-------------------------------------------------------------------\n%% Public API\n%%-------------------------------------------------------------------\n\n%% Request that the specified RAFT server commit the provided command. The commit can only be\n%% successful if the requested RAFT server is the active leader of the RAFT partition it is a\n%% part of. Returns either the result returned by the storage module when applying the command\n%% or an error indicating some reason for which the command was not able to be committed or\n%% should be retried.\n-spec commit(ServerRef :: gen_server:server_ref(), Op :: op()) -> commit_result().\ncommit(ServerRef, Op) ->\n    commit(ServerRef, Op, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec commit(ServerRef :: gen_server:server_ref(), Op :: op(), Timeout :: timeout()) -> commit_result().\ncommit(ServerRef, Op, Timeout) ->\n    commit(ServerRef, Op, Timeout, high).\n\n-spec commit(ServerRef :: gen_server:server_ref(), Op :: op(), Timeout :: timeout(), Priority :: priority()) -> commit_result().\ncommit(ServerRef, Op, Timeout, Priority) ->\n    call(ServerRef, ?COMMIT_REQUEST(?OP_DEFAULT(Op), Priority), Timeout).\n\n-spec commit_async(ServerRef :: gen_server:server_ref(), From :: {pid(), term()}, Op :: op()) -> ok.\ncommit_async(ServerRef, From, Op) ->\n    commit_async(ServerRef, From, Op, high).\n\n-spec commit_async(ServerRef :: gen_server:server_ref(), From :: {pid(), term()}, Op :: op(), Priority :: priority()) -> ok.\ncommit_async(ServerRef, From, Op, Priority) ->\n    gen_server:cast(ServerRef, ?COMMIT_ASYNC_REQUEST(From, ?OP_DEFAULT(Op), Priority)).\n\n% Strong-read\n-spec read(ServerRef :: gen_server:server_ref(), Command :: command()) -> read_result().\nread(ServerRef, Command) ->\n    read(ServerRef, Command, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec read(ServerRef :: gen_server:server_ref(), Command :: command(), Timeout :: timeout()) -> read_result().\nread(ServerRef, Command, Timeout) ->\n    call(ServerRef, ?READ_REQUEST(Command), Timeout).\n\n-spec adjust_config(\n    ServerRef :: gen_server:server_ref(),\n    Action :: wa_raft_server:config_action(),\n    Index :: wa_raft_log:log_index() | undefined\n) -> commit_result().\nadjust_config(ServerRef, Action, Index) ->\n    adjust_config(ServerRef, Action, Index, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec adjust_config(\n    ServerRef :: gen_server:server_ref(),\n    Action :: wa_raft_server:config_action(),\n    Index :: wa_raft_log:log_index() | undefined,\n    Timeout :: timeout()\n) -> commit_result().\nadjust_config(ServerRef, Action, Index, Timeout) ->\n    call(ServerRef, ?COMMIT_REQUEST(?OP_ADJUST_CONFIG(Action, Index), high), Timeout).\n\n-spec adjust_config_async(\n    ServerRef :: gen_server:server_ref(),\n    From :: gen_server:from(),\n    Action :: wa_raft_server:config_action()\n) -> ok.\nadjust_config_async(ServerRef, From, Action) ->\n    adjust_config_async(ServerRef, From, Action, undefined).\n\n-spec adjust_config_async(\n    ServerRef :: gen_server:server_ref(),\n    From :: gen_server:from(),\n    Action :: wa_raft_server:config_action(),\n    Index :: wa_raft_log:log_index() | undefined\n) -> ok.\nadjust_config_async(ServerRef, From, Action, Index) ->\n    gen_server:cast(ServerRef, ?COMMIT_ASYNC_REQUEST(From, ?OP_ADJUST_CONFIG(Action, Index), high)).\n\n-spec call(ServerRef :: gen_server:server_ref(), Request :: term(), Timeout :: timeout()) -> call_result().\ncall(ServerRef, Request, Timeout) ->\n    try\n        gen_server:call(ServerRef, Request, Timeout)\n    catch\n        exit:{timeout, _}       -> {error, timeout};\n        exit:{noproc, _}        -> {error, unreachable};\n        exit:{{nodedown, _}, _} -> {error, unreachable};\n        exit:{shutdown, _}      -> {error, unreachable};\n        exit:{Other, _}         -> {error, {call_error, Other}}\n    end.\n\n%%-------------------------------------------------------------------\n%% Internal API\n%%-------------------------------------------------------------------\n\n%% Get the default name for the RAFT acceptor server associated with the\n%% provided RAFT partition.\n-spec default_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom, one per table/partition at startup\n    binary_to_atom(<<\"raft_acceptor_\", (atom_to_binary(Table))/binary, \"_\", (integer_to_binary(Partition))/binary>>).\n\n%% Get the registered name for the RAFT acceptor server associated with the\n%% provided RAFT partition or the default name if no registration exists.\n-spec registered_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\nregistered_name(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> default_name(Table, Partition);\n        Options   -> Options#raft_options.acceptor_name\n    end.\n\n%%-------------------------------------------------------------------\n%% RAFT Acceptor - Server Callbacks\n%%-------------------------------------------------------------------\n\n-spec init(Options :: #raft_options{}) -> {ok, #state{}}.\ninit(#raft_options{table = Table, partition = Partition, acceptor_name = Name, server_name = Server} = Options) ->\n    process_flag(trap_exit, true),\n\n    ?RAFT_LOG_NOTICE(\"Acceptor[~0p] starting for partition ~0p/~0p\", [Name, Table, Partition]),\n\n    {ok, #state{\n        name = Name,\n        table = Table,\n        server = Server,\n        queues = wa_raft_queue:queues(Options)\n    }}.\n\n-spec handle_call(read_request(), gen_server:from(), #state{}) -> {reply, read_result(), #state{}} | {noreply, #state{}};\n                 (commit_request(), gen_server:from(), #state{}) -> {reply, commit_result(), #state{}} | {noreply, #state{}}.\nhandle_call(?READ_REQUEST(Command), From, State) ->\n    case read_impl(From, Command, State) of\n        continue           -> {noreply, State};\n        {error, _} = Error -> {reply, Error, State}\n    end;\nhandle_call(?COMMIT_REQUEST(Op, Priority), From, State) ->\n    case commit_impl(From, Op, Priority, State) of\n        continue           -> {noreply, State};\n        {error, _} = Error -> {reply, Error, State}\n    end;\nhandle_call(Request, From, #state{name = Name} = State) ->\n    ?RAFT_LOG_ERROR(\"Acceptor[~0p] received unexpected call ~0P from ~0p.\", [Name, Request, 30, From]),\n    {noreply, State}.\n\n-spec handle_cast(commit_async_request(), #state{}) -> {noreply, #state{}}.\nhandle_cast(?COMMIT_ASYNC_REQUEST(From, Op, Priority), State) ->\n    Result = commit_impl(From, Op, Priority, State),\n    Result =/= continue andalso gen_server:reply(From, Result),\n    {noreply, State};\nhandle_cast(Request, #state{name = Name} = State) ->\n    ?RAFT_LOG_ERROR(\"Acceptor[~0p] received unexpected cast ~0P.\", [Name, Request, 30]),\n    {noreply, State}.\n\n-spec terminate(Reason :: term(), State :: #state{}) -> ok.\nterminate(Reason, #state{name = Name}) ->\n    ?RAFT_LOG_NOTICE(\"Acceptor[~0p] terminating with reason ~0P\", [Name, Reason, 30]),\n    ok.\n\n%%-------------------------------------------------------------------\n%% RAFT Acceptor - Implementations\n%%-------------------------------------------------------------------\n\n%% Enqueue a commit.\n-spec commit_impl(\n    From :: gen_server:from(),\n    CommitOp :: commit_op(),\n    Priority :: priority(),\n    State :: #state{}\n) -> continue | commit_error().\ncommit_impl(From, CommitOp, Priority, #state{table = Table, name = Name, server = Server, queues = Queues}) ->\n    StartTUsec = erlang:monotonic_time(microsecond),\n    ?RAFT_LOG_DEBUG(\"Acceptor[~0p] starts to handle commit of ~0P from ~0p.\", [Name, CommitOp, 30, From]),\n    try\n        case wa_raft_queue:commit_started(Queues, Priority) of\n            ok ->\n                case CommitOp of\n                    ?OP_DEFAULT(Op) ->\n                        wa_raft_server:commit(Server, From, Op, Priority),\n                        continue;\n                    ?OP_ADJUST_CONFIG(Action, Index) ->\n                        wa_raft_server:adjust_config(Server, From, Action, Index),\n                        continue;\n                    _ ->\n                        ?RAFT_LOG_WARNING(\n                            \"Acceptor[~0p] does not know how to handle commit op ~0P.\",\n                            [Name, CommitOp, 20]\n                        ),\n                        {error, not_supported}\n                end;\n            Reason ->\n                ?RAFT_COUNT(Table, {'acceptor.error', Reason, Priority}),\n                ?RAFT_LOG_WARNING(\n                    \"Acceptor[~0p] is rejecting commit request from ~0p due to ~0p.\",\n                    [Name, From, Reason]\n                ),\n                {error, Reason}\n        end\n    after\n        ?RAFT_GATHER(Table, 'acceptor.commit.func', erlang:monotonic_time(microsecond) - StartTUsec)\n    end.\n\n%% Enqueue a strongly-consistent read.\n-spec read_impl(gen_server:from(), command(), #state{}) -> continue | read_error().\nread_impl(From, Command, #state{table = Table, name = Name, server = Server, queues = Queues}) ->\n    StartTUsec = erlang:monotonic_time(microsecond),\n    ?RAFT_LOG_DEBUG(\"Acceptor[~p] starts to handle read of ~0P from ~0p.\", [Name, Command, 100, From]),\n    try\n        case wa_raft_queue:reserve_read(Queues) of\n            ok ->\n                wa_raft_server:read(Server, {From, Command}),\n                continue;\n            Reason ->\n                ?RAFT_COUNT(Table, {'acceptor.strong_read.error', Reason}),\n                ?RAFT_LOG_WARNING(\n                    \"Acceptor[~0p] is rejecting read request from ~0p due to ~0p.\",\n                    [Name, From, Reason]\n                ),\n                {error, Reason}\n        end\n    after\n        ?RAFT_GATHER(Table, 'acceptor.strong_read.func', erlang:monotonic_time(microsecond) - StartTUsec)\n    end.\n"
  },
  {
    "path": "src/wa_raft_app.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% Application implementation for wa_raft.\n\n-module(wa_raft_app).\n-compile(warn_missing_spec_all).\n-behaviour(application).\n\n%% Application callbacks\n-export([\n    start/2,\n    stop/1\n]).\n\n-spec start(StartType :: application:start_type(), StartArgs :: term()) -> {ok, pid()}.\nstart(normal, _Args) ->\n    {ok, _Pid} = wa_raft_app_sup:start_link().\n\n-spec stop(State :: term()) -> ok.\nstop(_State) ->\n    ok.\n"
  },
  {
    "path": "src/wa_raft_app_sup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% Application supervisor to be started by the wa_raft application for\n%%% supervising services and resources shared between application-started\n%%% RAFT processes.\n\n-module(wa_raft_app_sup).\n-compile(warn_missing_spec_all).\n-behaviour(supervisor).\n\n%% API\n-export([\n    start_link/0\n]).\n\n%% Supervisor callbacks\n-export([\n    init/1\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n-spec start_link() -> supervisor:startlink_ret().\nstart_link() ->\n    supervisor:start_link({local, ?MODULE}, ?MODULE, []).\n\n-spec init(Arg :: term()) -> {ok, {supervisor:sup_flags(), [supervisor:child_spec()]}}.\ninit(_) ->\n    % Cache certain commonly used configuration values.\n    case ?RAFT_METRICS_MODULE() of\n        {ok, Module} -> wa_raft_metrics:install(Module);\n        _Other       -> ok\n    end,\n\n    % Setup tables used by shared services.\n    wa_raft_info:init_tables(),\n    wa_raft_transport:setup_tables(),\n\n    % Configure startup of shared services.\n    ChildSpecs = [\n        wa_raft_transport:child_spec(),\n        wa_raft_transport_sup:child_spec(),\n        wa_raft_dist_transport:child_spec(),\n        wa_raft_snapshot_catchup:child_spec()\n    ],\n\n    {ok, {#{strategy => one_for_one, intensity => 5, period => 1}, lists:flatten(ChildSpecs)}}.\n"
  },
  {
    "path": "src/wa_raft_dist_transport.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module implements transport interface by using erlang OTP dist.\n\n-module(wa_raft_dist_transport).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n-behaviour(wa_raft_transport).\n\n-export([\n    child_spec/0,\n    start_link/0\n]).\n\n-export([\n    transport_init/1,\n    transport_send/3\n]).\n\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    terminate/2\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n-record(sender_state, {\n}).\n-record(receiver_state, {\n    fds = #{} :: #{{ID :: wa_raft_transport:transport_id(), FileID :: wa_raft_transport:file_id()} => Fd :: file:fd()}\n}).\n\n-spec child_spec() -> supervisor:child_spec().\nchild_spec() ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, []},\n        restart => transient,\n        shutdown => 5000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link() -> gen_server:start_ret().\nstart_link() ->\n    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).\n\n-spec transport_init(Node :: node()) -> {ok, State :: #sender_state{}}.\ntransport_init(_Node) ->\n    {ok, #sender_state{}}.\n\n-spec transport_send(ID :: wa_raft_transport:transport_id(), FileID :: wa_raft_transport:file_id(), State :: #sender_state{}) ->\n    {ok, NewState :: #sender_state{}} |\n    {stop, Reason :: term(), NewState :: #sender_state{}}.\ntransport_send(ID, FileID, State) ->\n    ?RAFT_LOG_DEBUG(\"wa_raft_dist_transport starting to send file ~p/~p\", [ID, FileID]),\n    case wa_raft_transport:transport_info(ID) of\n        {ok, #{peer := Peer}} ->\n            case wa_raft_transport:file_info(ID, FileID) of\n                {ok, #{name := File, path := Path}} ->\n                    case prim_file:open(Path, [binary, read]) of\n                        {ok, Fd} ->\n                            try\n                                try prim_file:advise(Fd, 0, 0, sequential)\n                                catch _:_ -> ok\n                                end,\n                                case transport_send_loop(ID, FileID, Fd, Peer, State) of\n                                    {ok, NewState} ->\n                                        {ok, NewState};\n                                    {error, Reason, NewState} ->\n                                        {stop, Reason, NewState}\n                                end\n                            after\n                                prim_file:close(Fd)\n                            end;\n                        {error, Reason} ->\n                            ?RAFT_LOG_ERROR(\n                                \"wa_raft_dist_transport failed to open file ~p/~p (~s) due to ~p\",\n                                [ID, FileID, File, Reason]\n                            ),\n                            {stop, {failed_to_open_file, ID, FileID, Reason}, State}\n                    end;\n                _ ->\n                    {stop, {invalid_file, ID, FileID}, State}\n            end;\n        _ ->\n            {stop, {invalid_transport, ID}, State}\n    end.\n\n-spec transport_send_loop(\n    wa_raft_transport:transport_id(),\n    wa_raft_transport:file_id(),\n    file:fd(),\n    node(),\n    #sender_state{}\n) -> {ok, #sender_state{}} | {error, term(), #sender_state{}}.\ntransport_send_loop(ID, FileID, Fd, Peer, State) ->\n    ChunkSize = ?RAFT_DIST_TRANSPORT_CHUNK_SIZE(),\n    MaxInflight = ?RAFT_DIST_TRANSPORT_MAX_INFLIGHT(),\n    transport_send_loop(ID, FileID, Fd, 0, Peer, [], ChunkSize, MaxInflight, State).\n\n-spec transport_send_loop(\n    wa_raft_transport:transport_id(),\n    wa_raft_transport:file_id(),\n    file:fd(),\n    non_neg_integer() | eof,\n    node(),\n    [gen_server:request_id()],\n    pos_integer(),\n    pos_integer(),\n    #sender_state{}\n) -> {ok, #sender_state{}} | {error, term(), #sender_state{}}.\ntransport_send_loop(ID, FileID, _Fd, eof, Peer, [], _ChunkSize, _MaxInflight, State) ->\n    gen_server:cast({?MODULE, Peer}, {complete, ID, FileID}),\n    {ok, State};\ntransport_send_loop(ID, FileID, Fd, Offset, Peer, [RequestId | Chunks], ChunkSize, MaxInflight, State)\n        when Offset =:= eof ; length(Chunks) >= MaxInflight ->\n    case gen_server:wait_response(RequestId, 5000) of\n        {reply, ok} ->\n            transport_send_loop(ID, FileID, Fd, Offset, Peer, Chunks, ChunkSize, MaxInflight, State);\n        {reply, {error, Reason}} ->\n            ?RAFT_LOG_ERROR(\"wa_raft_dist_transport failed to send file ~p/~p due to receiver error ~p\", [ID, FileID, Reason]),\n            {error, {receiver_error, ID, FileID, Reason}, State};\n        timeout ->\n            ?RAFT_LOG_ERROR(\"wa_raft_dist_transport timed out while sending file ~p/~p\", [ID, FileID]),\n            {error, {send_timed_out, ID, FileID}, State};\n        {error, {Reason, _}} ->\n            ?RAFT_LOG_ERROR(\"wa_raft_dist_transport failed to send file ~p/~p due to ~p\", [ID, FileID, Reason]),\n            {error, {send_failed, ID, FileID, Reason}, State}\n    end;\ntransport_send_loop(ID, FileID, Fd, Offset, Peer, Chunks, ChunkSize, MaxInflight, State) when is_integer(Offset) ->\n    case prim_file:read(Fd, ChunkSize) of\n        {ok, Data} ->\n            RequestId = gen_server:send_request({?MODULE, Peer}, {chunk, ID, FileID, Offset, Data}),\n            wa_raft_transport:update_file_info(ID, FileID,\n                fun (#{completed_bytes := Completed} = Info) ->\n                    Info#{completed_bytes := Completed + byte_size(Data)}\n                end),\n            transport_send_loop(ID, FileID, Fd, Offset + byte_size(Data), Peer, Chunks ++ [RequestId], ChunkSize, MaxInflight, State);\n        eof ->\n            transport_send_loop(ID, FileID, Fd, eof, Peer, Chunks, ChunkSize, MaxInflight, State);\n        {error, Reason} ->\n            ?RAFT_LOG_ERROR(\"wa_raft_dist_transport failed to read file ~p/~p due to ~p\", [ID, FileID, Reason]),\n            {error, {read_failed, ID, FileID, Reason}, State}\n    end.\n\n-spec init(Args :: []) -> {ok, State :: #receiver_state{}}.\ninit([]) ->\n    process_flag(trap_exit, true),\n    {ok, #receiver_state{}}.\n\n-spec handle_call(Request, From :: term(), State :: #receiver_state{}) ->\n    {reply, Reply :: term(), NewState :: #receiver_state{}} | {noreply, NewState :: #receiver_state{}}\n    when Request :: {chunk, wa_raft_transport:transport_id(), wa_raft_transport:file_id(), integer(), binary()}.\nhandle_call({chunk, ID, FileID, Offset, Data}, _From, #receiver_state{} = State0) ->\n    {Reply, NewState} = case open_file(ID, FileID, State0) of\n        {ok, Fd, State1} ->\n            case prim_file:pwrite(Fd, Offset, Data) of\n                ok ->\n                    wa_raft_transport:update_file_info(ID, FileID,\n                        fun (#{completed_bytes := Completed} = Info) ->\n                            Info#{completed_bytes := Completed + byte_size(Data)}\n                        end),\n\n                    {ok, State1};\n                {error, Reason} ->\n                    ?RAFT_LOG_WARNING(\n                        \"wa_raft_dist_transport receiver failed to write file chunk ~p/~p @ ~p due to ~p\",\n                        [ID, FileID, Offset, Reason]\n                    ),\n                    {{write_failed, Reason}, State1}\n            end;\n        {error, Reason, State1} ->\n            ?RAFT_LOG_WARNING(\n                \"wa_raft_dist_transport receiver failed to handle file chunk ~p/~p @ ~p due to open failing due to ~p\",\n                [ID, FileID, Offset, Reason]\n            ),\n            {{open_failed, Reason}, State1}\n    end,\n    {reply, Reply, NewState};\nhandle_call(Request, From, #receiver_state{} = State) ->\n    ?RAFT_LOG_NOTICE(\"wa_raft_dist_transport got unrecognized call ~p from ~p\", [Request, From]),\n    {noreply, State}.\n\n-spec handle_cast(Request, State :: #receiver_state{}) -> {noreply, NewState :: #receiver_state{}}\n    when Request :: {complete,  wa_raft_transport:transport_id(), wa_raft_transport:file_id()}.\nhandle_cast({complete, ID, FileID}, #receiver_state{} = State0) ->\n    case open_file(ID, FileID, State0) of\n        {ok, _Fd, State1} ->\n            {ok, State2} = close_file(ID, FileID, State1),\n            wa_raft_transport:complete(ID, FileID, ok),\n            {noreply, State2};\n        {error, Reason, State1} ->\n            ?RAFT_LOG_WARNING(\n                \"wa_raft_dist_transport receiver failed to handle file complete ~p/~p due to open failing due to ~p\",\n                [ID, FileID, Reason]\n            ),\n            {noreply, State1}\n    end;\nhandle_cast(Request, #receiver_state{} = State) ->\n    ?RAFT_LOG_NOTICE(\"wa_raft_dist_transport got unrecognized cast ~p\", [Request]),\n    {noreply, State}.\n\n-spec terminate(Reason :: dynamic(), State :: #receiver_state{}) -> ok.\nterminate(Reason, #receiver_state{}) ->\n    ?RAFT_LOG_NOTICE(\"wa_raft_dist_transport terminating due to ~p\", [Reason]),\n    ok.\n\n-spec open_file(ID :: wa_raft_transport:transport_id(), FileID :: wa_raft_transport:file_id(), State :: #receiver_state{}) ->\n    {ok, Fd :: file:fd(), NewState :: #receiver_state{}} | {error, Reason :: term(), NewState :: #receiver_state{}}.\nopen_file(ID, FileID, #receiver_state{fds = Fds} = State0) ->\n    case Fds of\n        #{{ID, FileID} := Fd} ->\n            {ok, Fd, State0};\n        #{} ->\n            case wa_raft_transport:file_info(ID, FileID) of\n                {ok, #{name := File, path := Path}} ->\n                    try filelib:ensure_dir(Path)\n                    catch _:_ -> ok\n                    end,\n                    case prim_file:open(Path, [binary, write]) of\n                        {ok, Fd} ->\n                            State1 = State0#receiver_state{fds = Fds#{{ID, FileID} => Fd}},\n                            {ok, Fd, State1};\n                        {error, Reason} ->\n                            ?RAFT_LOG_WARNING(\n                                \"wa_raft_dist_transport receiver failed to open file ~p/~p (~p) due to ~p\",\n                                [ID, FileID, File, Reason]\n                            ),\n                            {error, {open_failed, Reason}, State0}\n                    end;\n                _ ->\n                    {error, invalid_file, State0}\n            end\n    end.\n\n-spec close_file(ID :: wa_raft_transport:transport_id(), FileID :: wa_raft_transport:file_id(), State :: #receiver_state{}) ->\n    {ok, NewState :: #receiver_state{}}.\nclose_file(ID, FileID, #receiver_state{fds = Fds} = State0) ->\n    case Fds of\n        #{{ID, FileID} := Fd} ->\n            try prim_file:close(Fd)\n            catch _:_ -> ok\n            end,\n            State1 = State0#receiver_state{fds = maps:remove({ID, FileID}, Fds)},\n            {ok, State1};\n        _ ->\n            {ok, State0}\n    end.\n"
  },
  {
    "path": "src/wa_raft_distribution.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% Pluggable distribution interface. The default implementation uses Erlang\n%%% distribution.\n\n-module(wa_raft_distribution).\n-compile(warn_missing_spec_all).\n\n-export([\n    cast/3\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n-type dest_addr() :: {Name :: atom(), Node :: node()}.\n\n-export_type([\n    dest_addr/0\n]).\n\n%%% ------------------------------------------------------------------------\n%%%  Behaviour callbacks\n%%%\n\n-callback cast(dest_addr(), #raft_identifier{}, term()) -> ok | term().\n\n%%% ------------------------------------------------------------------------\n%%%  Erlang distribution default implementation\n%%%\n\n-spec cast(DestAddr :: dest_addr(), Identifier :: #raft_identifier{}, Message :: term()) -> ok | {error, Reason} when\n    Reason :: noconnect | nosuspend.\ncast(DestAddr, _Identifier, Message) ->\n    case erlang:send(DestAddr, {'$gen_cast', Message}, [noconnect, nosuspend]) of\n        ok -> ok;\n        Reason -> {error, Reason}\n    end.\n"
  },
  {
    "path": "src/wa_raft_durable_state.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module implements functions for storing / loading persistent state.\n\n-module(wa_raft_durable_state).\n-compile(warn_missing_spec_all).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n-export([\n    load/1,\n    store/1,\n    sync/1\n]).\n\n-spec load(StateIn :: #raft_state{}) -> {ok, StateOut :: #raft_state{}} | no_state | {error, Reason :: term()}.\nload(#raft_state{name = Name, partition_path = PartitionPath} = State) ->\n    StateItems = [\n        {current_term,   fun is_integer/1, fun (V, S) -> S#raft_state{current_term = V} end,   required},\n        {voted_for,      fun is_atom/1,    fun (V, S) -> S#raft_state{voted_for = V} end,      required},\n        {disable_reason, undefined,        fun (V, S) -> S#raft_state{disable_reason = V} end, undefined}\n    ],\n    StateFile = filename:join(PartitionPath, ?STATE_FILE_NAME),\n    case file:consult(StateFile) of\n        {ok, [{crc, SavedCRC} | StateTerms]} ->\n            case erlang:crc32(term_to_binary(StateTerms, [{minor_version, 1}, deterministic])) of\n                SavedCRC ->\n                    try\n                        {ok, lists:foldl(\n                            fun ({Item, Validator, Updater, Default}, StateN) ->\n                                    case proplists:lookup(Item, StateTerms) of\n                                        none when Default =:= required ->\n                                            ?RAFT_LOG_ERROR(\"~p read state file but cannot find ~p.\", [Name, Item]),\n                                            throw({error, {missing, Item}});\n                                        none ->\n                                            Updater(Default, StateN);\n                                        {Item, Value} ->\n                                            case Validator =:= undefined orelse Validator(Value) of\n                                                true ->\n                                                    Updater(Value, StateN);\n                                                false ->\n                                                    ?RAFT_LOG_ERROR(\"~p read state file but ~p has an invalid value `~p`.\", [Name, Item, Value]),\n                                                    throw({error, {invalid, Item}})\n                                            end\n                                    end\n                            end, State, StateItems)}\n                    catch\n                        throw:{error, Reason} -> {error, Reason}\n                    end;\n                InvalidCRC ->\n                    ?RAFT_LOG_ERROR(\"~p read state file but CRCs did not match. (saved crc: ~p, computed crc: ~p)\", [Name, SavedCRC, InvalidCRC]),\n                    {error, invalid_crc}\n            end;\n        {ok, _} ->\n            ?RAFT_LOG_ERROR(\"~p read state file but no CRC was found\", [Name]),\n            {error, no_crc};\n        {error, enoent} ->\n            ?RAFT_LOG_NOTICE(\"~p is not loading non-existant state file.\", [Name]),\n            no_state;\n        {error, Reason} ->\n            ?RAFT_LOG_ERROR(\"~p could not read state file due to ~p.\", [Name, Reason]),\n            {error, Reason}\n    end.\n\n-spec store(#raft_state{}) -> ok | {error, Reason :: term()}.\nstore(#raft_state{name = Name, table = Table, partition_path = PartitionPath, current_term = CurrentTerm, voted_for = VotedFor, disable_reason = DisableReason}) ->\n    StateList = [\n        {current_term, CurrentTerm},\n        {voted_for, VotedFor},\n        {disable_reason, DisableReason}\n    ],\n    StateListWithCRC = [{crc, erlang:crc32(term_to_binary(StateList, [{minor_version, 1}, deterministic]))} | StateList],\n    StateIO = [io_lib:format(\"~p.~n\", [Term]) || Term <- StateListWithCRC],\n    StateFile = filename:join(PartitionPath, ?STATE_FILE_NAME),\n    StateFileTemp = [StateFile, \".temp\"],\n    case filelib:ensure_dir(StateFile) of\n        ok ->\n            case prim_file:write_file(StateFileTemp, StateIO) of\n                ok ->\n                    case file:rename(StateFileTemp, StateFile) of\n                        ok ->\n                            ok;\n                        {error, Reason} ->\n                            ?RAFT_COUNT(Table, {'server.persist_state.error.rename', Reason}),\n                            ?RAFT_LOG_ERROR(\"~p failed to rename temporary state file due to ~p.\", [Name, Reason]),\n                            {error, {rename, Reason}}\n                    end;\n                {error, Reason} ->\n                    ?RAFT_COUNT(Table, {'server.persist_state.error.write', Reason}),\n                    ?RAFT_LOG_ERROR(\"~p failed to write current state to temporary file due to ~p.\", [Name, Reason]),\n                    {error, {write, Reason}}\n            end;\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, {'server.persist_state.error.ensure_dir', Reason}),\n            ?RAFT_LOG_ERROR(\"~p failed to ensure directory exists due to ~p.\", [Name, Reason]),\n            {error, {ensure_dir, Reason}}\n    end.\n\n-spec sync(StateIn :: #raft_state{}) -> ok.\nsync(#raft_state{partition_path = PartitionPath}) ->\n    StateFile = filename:join(PartitionPath, ?STATE_FILE_NAME),\n    case prim_file:open(StateFile, [read, binary]) of\n        {ok, Fd} ->\n            prim_file:sync(Fd),\n            prim_file:close(Fd),\n            ok;\n        _ ->\n            ok\n    end.\n"
  },
  {
    "path": "src/wa_raft_env.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module contains utility functions for consulting application\n%%% configuration in the OTP application environment according to the search\n%%% order configured for each RAFT partition.\n\n-module(wa_raft_env).\n-compile(warn_missing_spec_all).\n\n%% Config API\n-export([\n    database_path/1\n]).\n\n%% Internal API\n-export([\n    get_env/2,\n    get_env/3,\n    get_table_env/3,\n    get_table_env/4\n]).\n\n-type scope() :: Application :: atom() | {Table :: wa_raft:table(), Partition :: wa_raft:partition()} | SearchApps :: [atom()].\n-type key() :: Key :: atom() | {Primary :: atom(), Fallback :: atom()}.\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n%%-------------------------------------------------------------------\n%% Config API\n%%-------------------------------------------------------------------\n\n-spec database_path(Scope :: scope()) -> Root :: file:filename().\ndatabase_path(Scope) ->\n    case get_env(Scope, ?RAFT_DATABASE) of\n        {ok, Root} -> Root;\n        undefined  -> error({no_configured_database_path, Scope})\n    end.\n\n%%-------------------------------------------------------------------\n%% Internal API\n%%-------------------------------------------------------------------\n\n-spec get_env(Scope :: scope(), Key :: key()) -> {ok, Value :: dynamic()} | undefined.\nget_env(Scope, Key) ->\n    get_env_impl(search_apps(Scope), key(Key), fallback(Key)).\n\n-spec get_env(Scope :: scope(), Key :: key(), Default :: Value) -> Value.\nget_env(Scope, Key, Default) ->\n    case get_env(Scope, Key) of\n        {ok, Value} -> Value;\n        undefined   -> Default\n    end.\n\n-spec get_env_impl(SearchApps :: [atom()], Key :: atom(), FallbackKey :: atom()) -> {ok, Value :: dynamic()} | undefined.\nget_env_impl([], _Key, FallbackKey) ->\n    ?RAFT_CONFIG(FallbackKey);\nget_env_impl([App | SearchApps], Key, FallbackKey) ->\n    case application:get_env(App, Key) of\n        {ok, Value} -> {ok, Value};\n        undefined   -> get_env_impl(SearchApps, Key, FallbackKey)\n    end.\n\n-doc \"\"\"\nGet a configuration value with table-level overrides.\n\nTable-level configuration is supported by storing the value as a tagged tuple:\n  `{table_overrides, #{Table => Value}, AppDefault}`\nWhen this format is found, the table is looked up in the map first.\nIf the table is not in the map, AppDefault is used. If the value is a plain\n(untagged) term, it is used directly as the app-level value for all tables.\n\"\"\".\n-spec get_table_env(Scope :: scope(), Table :: wa_raft:table(), Key :: key()) -> {ok, Value :: dynamic()} | undefined.\nget_table_env(Scope, Table, Key) ->\n    get_table_env_impl(search_apps(Scope), Table, key(Key), fallback(Key)).\n\n-spec get_table_env(Scope :: scope(), Table :: wa_raft:table(), Key :: key(), Default :: Value) -> Value.\nget_table_env(Scope, Table, Key, Default) ->\n    case get_table_env(Scope, Table, Key) of\n        {ok, Value} -> Value;\n        undefined   -> Default\n    end.\n\n-spec get_table_env_impl(SearchApps :: [atom()], Table :: wa_raft:table(), Key :: atom(), FallbackKey :: atom()) ->\n    {ok, Value :: dynamic()} | undefined.\nget_table_env_impl([], _Table, _Key, FallbackKey) ->\n    ?RAFT_CONFIG(FallbackKey);\nget_table_env_impl([App | SearchApps], Table, Key, FallbackKey) ->\n    case application:get_env(App, Key) of\n        {ok, {table_overrides, TableOverrides, AppDefault}} ->\n            {ok, maps:get(Table, TableOverrides, AppDefault)};\n        {ok, Value} ->\n            {ok, Value};\n        undefined ->\n            get_table_env_impl(SearchApps, Table, Key, FallbackKey)\n    end.\n\n-spec search_apps(Scope :: scope()) -> SearchApps :: [atom()].\nsearch_apps(Application) when is_atom(Application) ->\n    case wa_raft_sup:options(Application) of\n        undefined       -> [];\n        RaftApplication -> RaftApplication#raft_application.config_search_apps\n    end;\nsearch_apps({Table, Partition}) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> [];\n        Options   -> search_apps(Options#raft_options.application)\n    end;\nsearch_apps(SearchApps) ->\n    SearchApps.\n\n-spec key(key()) -> atom().\nkey({Key, _}) -> Key;\nkey(Key) -> Key.\n\n-spec fallback(key()) -> atom().\nfallback({_, Fallback}) -> Fallback;\nfallback(Key) -> Key.\n"
  },
  {
    "path": "src/wa_raft_info.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% API for accessing certain useful information about the state of local\n%%% RAFT servers without requiring a status request against the RAFT server\n%%% itself.\n\n-module(wa_raft_info).\n-compile(warn_missing_spec_all).\n\n%% Public API\n-export([\n    get_current_term/2,\n    get_leader/2,\n    get_current_term_and_leader/2,\n    get_membership/2,\n    get_live/2,\n    get_stale/2,\n    get_state/2,\n    get_message_queue_length/1\n]).\n\n%% Internal API\n-export([\n    init_tables/0,\n    delete_state/2,\n    set_current_term_and_leader/4,\n    set_membership/3,\n    set_live/3,\n    set_stale/3,\n    set_state/3,\n    set_message_queue_length/1,\n    set_message_queue_length/2\n]).\n\n%% Local RAFT server's current FSM state\n-define(RAFT_SERVER_STATE_KEY(Table, Partition), {state, Table, Partition}).\n%% Local RAFT server's most recently known term and leader\n-define(RAFT_CURRENT_TERM_AND_LEADER_KEY(Table, Partition), {term, Table, Partition}).\n%% Local RAFT server's current live flag - indicates if the server thinks it is part of a live cluster\n-define(RAFT_LIVE_KEY(Table, Partition), {live, Table, Partition}).\n%% Local RAFT server's current stale flag - indicates if the server thinks its data is stale\n-define(RAFT_STALE_KEY(Table, Partition), {stale, Table, Partition}).\n%% Local RAFT server's message queue length\n-define(RAFT_MSG_QUEUE_LENGTH_KEY(Name), {msg_queue_length, Name}).\n%% Local RAFT server's most recently known membership\n-define(RAFT_MEMBERSHIP_KEY(Table, Partition), {membership, Table, Partition}).\n\n%%-------------------------------------------------------------------\n%% RAFT Info - Public API\n%%-------------------------------------------------------------------\n\n-spec get(term(), Default) -> Default.\nget(Key, Default) ->\n    try\n        ets:lookup_element(?MODULE, Key, 2, Default)\n    catch\n        error:badarg ->\n            Default\n    end.\n\n-spec get_leader(wa_raft:table(), wa_raft:partition()) -> node() | undefined.\nget_leader(Table, Partition) ->\n    {_, Leader} = get(?RAFT_CURRENT_TERM_AND_LEADER_KEY(Table, Partition), {undefined, undefined}),\n    Leader.\n\n-spec get_current_term(wa_raft:table(), wa_raft:partition()) -> wa_raft_log:log_term() | undefined.\nget_current_term(Table, Partition) ->\n    {Term, _} = get(?RAFT_CURRENT_TERM_AND_LEADER_KEY(Table, Partition), {undefined, undefined}),\n    Term.\n\n%% The RAFT server always sets both the known term and leader together, so that\n%% the atomic read performed by this method will not return a known leader for\n%% a different term.\n-spec get_current_term_and_leader(wa_raft:table(), wa_raft:partition()) ->\n    {wa_raft_log:log_term() | undefined, node() | undefined}.\nget_current_term_and_leader(Table, Partition) ->\n    get(?RAFT_CURRENT_TERM_AND_LEADER_KEY(Table, Partition), {undefined, undefined}).\n\n-spec get_state(wa_raft:table(), wa_raft:partition()) -> wa_raft_server:state() | undefined.\nget_state(Table, Partition) ->\n    get(?RAFT_SERVER_STATE_KEY(Table, Partition), undefined).\n\n-spec get_live(wa_raft:table(), wa_raft:partition()) -> boolean().\nget_live(Table, Partition) ->\n    get(?RAFT_LIVE_KEY(Table, Partition), false).\n\n-spec get_stale(wa_raft:table(), wa_raft:partition()) -> boolean().\nget_stale(Table, Partition) ->\n    get(?RAFT_STALE_KEY(Table, Partition), true).\n\n-spec get_message_queue_length(atom()) -> undefined | non_neg_integer().\nget_message_queue_length(Name) ->\n    get(?RAFT_MSG_QUEUE_LENGTH_KEY(Name), undefined).\n\n-spec get_membership(wa_raft:table(), wa_raft:partition()) -> wa_raft_server:membership() | undefined.\nget_membership(Table, Partition) ->\n    get(?RAFT_MEMBERSHIP_KEY(Table, Partition), undefined).\n\n%%-------------------------------------------------------------------\n%% RAFT Info - Internal API\n%%-------------------------------------------------------------------\n\n-spec init_tables() -> ok.\ninit_tables() ->\n    ets:new(?MODULE, [set, public, named_table, {write_concurrency, true}, {read_concurrency, true}]),\n    ok.\n\n-spec set(term(), term()) -> true.\nset(Key, Value) ->\n    ets:update_element(?MODULE, Key, {2, Value}) orelse ets:insert(?MODULE, {Key, Value}).\n\n-spec delete(term()) -> true.\ndelete(Key) ->\n    ets:delete(?MODULE, Key).\n\n-spec set_current_term_and_leader(wa_raft:table(), wa_raft:partition(), wa_raft_log:log_term(), node()) -> true.\nset_current_term_and_leader(Table, Partition, Term, Leader) ->\n    set(?RAFT_CURRENT_TERM_AND_LEADER_KEY(Table, Partition), {Term, Leader}).\n\n-spec set_state(wa_raft:table(), wa_raft:partition(), wa_raft_server:state()) -> true.\nset_state(Table, Partition, State) ->\n    set(?RAFT_SERVER_STATE_KEY(Table, Partition), State).\n\n-spec delete_state(wa_raft:table(), wa_raft:partition()) -> true.\ndelete_state(Table, Partition) ->\n    delete(?RAFT_SERVER_STATE_KEY(Table, Partition)).\n\n-spec set_live(wa_raft:table(), wa_raft:partition(), boolean()) -> true.\nset_live(Table, Partition, Live) ->\n    set(?RAFT_LIVE_KEY(Table, Partition), Live).\n\n-spec set_stale(wa_raft:table(), wa_raft:partition(), boolean()) -> true.\nset_stale(Table, Partition, Stale) ->\n    set(?RAFT_STALE_KEY(Table, Partition), Stale).\n\n-spec set_membership(wa_raft:table(), wa_raft:partition(), wa_raft_server:membership()) -> true.\nset_membership(Table, Partition, Membership) ->\n    set(?RAFT_MEMBERSHIP_KEY(Table, Partition), Membership).\n\n-spec set_message_queue_length(Name :: atom()) -> true.\nset_message_queue_length(Name) ->\n    {message_queue_len, Length} = process_info(self(), message_queue_len),\n    set_message_queue_length(Name, Length).\n\n-spec set_message_queue_length(Name :: atom(), Length :: non_neg_integer()) -> true.\nset_message_queue_length(Name, Length) ->\n    set(?RAFT_MSG_QUEUE_LENGTH_KEY(Name), Length).\n"
  },
  {
    "path": "src/wa_raft_label.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% Pluggable module for labeling log entries before adding them to the RAFT log.\n\n-module(wa_raft_label).\n-compile(warn_missing_spec_all).\n\n    -type label() :: dynamic().\n\n-export_type([label/0]).\n\n%%% ------------------------------------------------------------------------\n%%%  Behaviour callbacks\n%%%\n\n% Produce a label for a new log record based on the log payload and the label of the preceding log entry.\n-callback new_label(LastLabel :: label(), Command :: wa_raft_acceptor:command()) -> NewLabel :: label().\n"
  },
  {
    "path": "src/wa_raft_log.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module is the interface for raft log. It defines the callbacks\n%%% required by the specific log implementations.\n\n-module(wa_raft_log).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n%% OTP supervision\n-export([\n    child_spec/1,\n    start_link/1\n]).\n\n%% APIs for writing new log data\n-export([\n    append/2,\n    try_append/2,\n    try_append/3,\n    check_heartbeat/3\n]).\n\n%% APIs for accessing log data\n-export([\n    first_index/1,\n    last_index/1,\n\n    fold/5,\n    fold/6,\n    fold_binary/5,\n    fold_binary/6,\n    fold_terms/5,\n\n    term/2,\n    get/2,\n    get/3,\n    get/4,\n    get_terms/3,\n\n    entries/3,\n    entries/4,\n\n    config/1\n]).\n\n%% APIs for managing logs and log data\n-export([\n    open/2,\n    reset/2,\n    truncate/2,\n    trim/2,\n    rotate/2, rotate/4,\n    flush/1\n]).\n\n%% Internal API\n-export([\n    default_name/2,\n    registered_name/2\n]).\n\n%% Internal API\n-export([\n    log/1,\n    log_name/1,\n    provider/1\n]).\n\n%% gen_server callbacks\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    terminate/2\n]).\n\n-export_type([\n    log/0,\n    log_name/0,\n    log_pos/0,\n    log_op/0,\n    log_index/0,\n    log_term/0,\n    log_entry/0,\n    log_record/0,\n    view/0\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%% Atom indicating that the provider has not been opened yet.\n-define(PROVIDER_NOT_OPENED, '$not_opened').\n\n%% A view of a RAFT log that is backed by a particular\n%% log provider. This view keeps track of its own logical\n%% start and end indices as well as a batch of pending\n%% log entries so that the RAFT server is always able to\n%% access a consistent view of the RAFT log given simple\n%% RAFT log provider implementations.\n-record(log_view, {\n    log :: log(),\n    first = 0 :: log_index(),\n    last = 0 :: log_index(),\n    config :: undefined | {log_index(), wa_raft_server:config()}\n}).\n\n%% The state stored by the RAFT log server which is\n%% responsible for synchronizing destructive operations\n%% on the RAFT log with operations that are performed\n%% asynchronously to the RAFT server.\n-record(log_state, {\n    log :: log(),\n    state = ?PROVIDER_NOT_OPENED :: term()\n}).\n\n%% Name of a raft log.\n-type log() :: #raft_log{}.\n-type log_name() :: atom().\n-type log_index() :: non_neg_integer().\n-type log_term() :: non_neg_integer().\n-type log_pos() :: #raft_log_pos{}.\n-type log_op() ::\n    undefined\n    | {wa_raft_acceptor:key(), wa_raft_acceptor:command()}\n    | {wa_raft_acceptor:key(), wa_raft_label:label(), wa_raft_acceptor:command()}.\n-type log_entry() :: {log_term(), log_op()}.\n-type log_record() :: {log_index(), log_entry()}.\n\n%% A view of a RAFT log.\n-opaque view() :: #log_view{}.\n\n%%-------------------------------------------------------------------\n%% RAFT log provider interface for accessing log data\n%%-------------------------------------------------------------------\n\n%% Gets the first index of the RAFT log. If there are no log entries,\n%% then return 'undefined'.\n-callback first_index(Log :: log()) -> undefined | FirstIndex :: log_index() | {error, Reason :: term()}.\n\n%% Gets the last index of the RAFT log. If there are no log entries,\n%% then return 'undefined'.\n-callback last_index(Log :: log()) -> undefined | LastIndex :: log_index() | {error, Reason :: term()}.\n\n%% Call the provided combining function on successive log entries in the\n%% specified log starting from the specified start index (inclusive) and ending\n%% at the specified end index (also inclusive) or until the total cumulative\n%% size of log entries for which the combining function has been called upon\n%% is at least the specified size limit. The combining function must always be\n%% called with log entries in log order starting with the first log entry that\n%% exists within the provided range. The combining function should not be\n%% called for those log indices within the provided range that do not have a\n%% stored log entry. It is suggested that the external term size of the entire\n%% log entry be used as the size provided to the combining function and for\n%% tracking the size limit; however, implementations are free to use any value.\n%%\n%% The combining function may raise an error. Implementations should take care\n%% to release any resources held in the case that the fold needs to terminate\n%% early.\n-callback fold(Log :: log(),\n               Start :: log_index(),\n               End :: log_index(),\n               SizeLimit :: non_neg_integer() | infinity,\n               Func :: fun((Index :: log_index(), Size :: non_neg_integer(), Entry :: log_entry(), Acc) -> Acc),\n               Acc) -> {ok, Acc} | {error, Reason :: term()}.\n\n%% Call the provided combining function on the external term format of\n%% successive log entries in the specified log starting from the specified\n%% start index (inclusive) and ending at the specified end index (also\n%% inclusive) or until the total cumulative size of log entries for which the\n%% combining function has been called upon is at least the specified size\n%% limit. The combining function must always be called with log entries in log\n%% order starting with the first log entry that exists within the provided\n%% range. The combining function should not be called for those log indices\n%% within the provided range that do not have a stored log entry. The byte\n%% size of the binary provided to the combining function must be used as the\n%% size of the entry for tracking of the size limit.\n%%\n%% The combining function may raise an error. Implementations should take care\n%% to release any resources held in the case that the fold needs to terminate\n%% early.\n-callback fold_binary(\n    Log :: log(),\n    Start :: log_index(),\n    End :: log_index(),\n    SizeLimit :: non_neg_integer() | infinity,\n    Func :: fun((Index :: log_index(), Entry :: binary(), Acc) -> Acc),\n    Acc\n) -> {ok, Acc} | {error, Reason :: term()}.\n-optional_callbacks([fold_binary/6]).\n\n%% Call the provided combining function on the log term of successive log\n%% entries in the specified log starting from the specified start index\n%% (inclusive) and ending at the specified end index (also inclusive). The\n%% combining function must always be called with log entries in log order\n%% starting with the first log entry that exists within the provided range. The\n%% combining function should not be called for those log indices within the\n%% provided range that do not have a stored log entry.\n%%\n%% The combining function may raise an error. Implementations should take care\n%% to release any resources held in the case that the fold needs to terminate\n%% early.\n-callback fold_terms(Log :: log(),\n                     Start :: log_index(),\n                     End :: log_index(),\n                     Func :: fun((Index :: log_index(), Term :: log_term(), Acc) -> Acc),\n                     Acc) -> {ok, Acc} | {error, Reason :: term()}.\n\n%% Get a single log entry at the specified index. This API is specified\n%% separately because some implementations may have more efficient ways to\n%% get log entries when only one log entry is required. If the log entry\n%% does not exist, then return 'not_found'.\n-callback get(Log :: log(), Index :: log_index()) -> {ok, Entry :: log_entry()} | not_found | {error, Reason :: term()}.\n\n%% Get only the term of a specific log entry. This API is specified\n%% separately because some implementations may have more efficient ways to\n%% get just the term of a particular log entry. If the log entry does not\n%% exist, then return 'not_found'.\n-callback term(Log :: log(), Index :: log_index()) -> {ok, Term :: log_term()} | not_found | {error, Reason :: term()}.\n\n%% Get the most recent configuration stored in the log. Log providers\n%% should ensure that configuration operations are indexed so that this\n%% call does not require a scan of the log.\n-callback config(Log :: log()) -> {ok, Index :: log_index(), Config :: wa_raft_server:config()} | not_found | {error, Reason :: term()}.\n\n%%-------------------------------------------------------------------\n%% RAFT log provider interface for writing new log data\n%%-------------------------------------------------------------------\n\n%% Append new log entries to the end of the RAFT log.\n%%  - This function should never overwrite existing log entries.\n%%  - If the new log entries were written successfully, return 'ok'.\n%%  - Log entries may be provided either as terms directly or as a\n%%    binary encoding a log entry in external term format.\n%%  - In 'strict' mode, the append should always succeed or return an\n%%    error on failure.\n%%  - In 'relaxed' mode, if there are transient conditions that would\n%%    make it difficult to append to the log without blocking, then\n%%    the append should be skipped and 'skipped' returned. Otherwise,\n%%    the same conditions as the 'strict' mode apply.\n-callback append(View :: view(), Entries :: [log_entry() | binary()], Mode :: strict | relaxed, Priority :: wa_raft_acceptor:priority()) ->\n    ok | skipped | {error, Reason :: term()}.\n\n%%-------------------------------------------------------------------\n%% RAFT log provider interface for managing underlying RAFT log\n%%-------------------------------------------------------------------\n\n%% Perform any first time setup operations before opening the RAFT log.\n%% This function is called from the RAFT log server and is only called\n%% once per incarnation of a RAFT partition.\n%% If this setup fails such that the log is not usable, implementations\n%% should raise an error or exit to interrupt the startup process.\n-callback init(Log :: wa_raft_log:log()) -> ok.\n\n%% Open the RAFT log and return a state term that will be provided to\n%% subsequent calls made from the RAFT log server. During the opening\n%% process, the log will be inspected to see if it contains a record\n%% corresponding to the last applied index of the storage backing this\n%% RAFT partition and whether or not the term of this entry matches\n%% that reported by the storage. If so, then opening proceeeds normally.\n%% If there is a mismatch, then the log will be reinitialized using\n%% `reset/3`.\n%% If this setup fails such that the log is not usable, implementations\n%% should raise an error or exit to interrupt the opening process.\n-callback open(Log :: wa_raft_log:log()) -> {ok, State :: term()} | {error, Reason :: term()}.\n\n%% Close the RAFT log and release any resources used by it. This\n%% is called when the RAFT log server is terminating.\n-callback close(Log :: log(), State :: term()) -> term().\n\n%% Completely clear the RAFT log and setup a new log with an initial entry\n%% at the provided index with the provided term and an undefined op.\n-callback reset(Log :: log(), Position :: log_pos(), State :: term()) -> {ok, NewState :: term()} | {error, Reason :: term()}.\n\n%% Truncate the RAFT log to the given position so that all log entries\n%% including and after the provided index are completely deleted from\n%% the RAFT log. If the truncation failed but the log state was not\n%% changed, then an error can be returned. Otherwise, a error should\n%% be raised.\n-callback truncate(Log :: log(), Index :: log_index(), State :: term()) -> {ok, NewState :: term()} | {error, Reason :: term()}.\n\n%% Optionally, trim the RAFT log up to the given index.\n%%  - This means that all log entries before the given index can be\n%%    deleted (both term and op information can be removed) and the\n%%    log entry at the given index can have its op removed (keeping\n%%    only the term information).\n%%  - Implementations are not required to always trim the log to exactly\n%%    the provided index but must not trim past the provided index and\n%%    must always ensure that if the log were to be reloaded from disk\n%%    at any time that the log always remains contiguous, meaning that\n%%    only the first entry in the log can be missing op information and\n%%    that the indices of all log entries in the log are contiguous.\n-callback trim(Log :: log(), Index :: log_index(), State :: term()) -> {ok, NewState :: term()} | {error, Reason :: term()}.\n\n%% Flush log to disk on a best-effort basis. The return value is\n%% ignored.\n-callback flush(Log :: log()) -> term().\n\n%%-------------------------------------------------------------------\n%% RAFT log provider interface for writing new log data\n%%-------------------------------------------------------------------\n\n-spec child_spec(Options :: #raft_options{}) -> supervisor:child_spec().\nchild_spec(Options) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Options]},\n        restart => permanent,\n        shutdown => 30000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Options :: #raft_options{}) -> {ok, Pid :: pid()} | ignore | {error, Reason :: term()}.\nstart_link(#raft_options{log_name = Name} = Options) ->\n    gen_server:start_link({local, Name}, ?MODULE, Options, []).\n\n%%-------------------------------------------------------------------\n%% APIs for writing new log data\n%%-------------------------------------------------------------------\n\n%% Append new log entries to the end of the log.\n-spec append(View :: view(), Entries :: [log_entry() | binary()]) ->\n    {ok, NewView :: view()} | {error, Reason :: term()}.\nappend(View, Entries) ->\n    % eqwalizer:ignore - strict append cannot return skipped\n    append(View, Entries, strict, high).\n\n%% Attempt to append new log entries to the end of the log if an append can be\n%% serviced immediately.\n-spec try_append(View :: view(), Entries :: [log_entry() | binary()]) ->\n    {ok, NewView :: view()} | skipped | {error, Reason :: term()}.\ntry_append(View, Entries) ->\n    try_append(View, Entries, high).\n\n-spec try_append(View :: view(), Entries :: [log_entry() | binary()], Priority :: wa_raft_acceptor:priority()) ->\n    {ok, NewView :: view()} | skipped | {error, Reason :: term()}.\ntry_append(View, Entries, Priority) ->\n    append(View, Entries, relaxed, Priority).\n\n%% Append new log entries to the end of the log.\n-spec append(View :: view(), Entries :: [log_entry() | binary()], Mode :: strict | relaxed, Priority :: wa_raft_acceptor:priority()) ->\n    {ok, NewView :: view()} | skipped | {error, Reason :: term()}.\nappend(#log_view{log = #raft_log{table = Table}, last = Last} = View, Entries, Mode, Priority) ->\n    ?RAFT_COUNT(Table, 'log.append'),\n    Provider = provider(View),\n    case Provider:append(View, Entries, Mode, Priority) of\n        ok ->\n            ?RAFT_COUNT(Table, 'log.append.ok'),\n            {ok, refresh_config(View#log_view{last = Last + length(Entries)})};\n        skipped when Mode =:= relaxed ->\n            ?RAFT_COUNT(Table, 'log.append.skipped'),\n            skipped;\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.append.error'),\n            {error, Reason}\n    end.\n\n%% Compare the provided heartbeat log entries to the local log at the provided\n%% starting position in preparation for an append operation:\n%%  * If the provided starting position is before the start of the log or past\n%%    the end of the log, the comparison will fail with an `out_of_range`\n%%    error.\n%%  * If there is a conflict between the provided heartbeat log entries and any\n%%    local log entries due to a term mismatch, then the comparison will fail\n%%    with a `conflict` tuple that contains the log index of the first log\n%%    entry with a conflicting term and the list containing the corresponding\n%%    heartbeat log entry and all subsequent heartbeat log entries.\n%%  * Otherwise, the comparison will succeed. Any new log entries not already\n%%    in the local log will be returned.\n-spec check_heartbeat(View :: view(), Start :: log_index(), Entries :: [log_entry() | binary()]) ->\n    {ok, NewEntries :: [log_entry() | binary()]} |\n    {conflict, ConflictIndex :: log_index(), NewEntries :: [log_entry() | binary()]} |\n    {invalid, out_of_range} |\n    {error, term()}.\ncheck_heartbeat(#log_view{first = First, last = Last}, Start, _Entries) when Start =< 0; Start < First; Start > Last ->\n    {invalid, out_of_range};\ncheck_heartbeat(#log_view{log = #raft_log{table = Table} = Log, last = Last}, Start, Entries) ->\n    Provider = provider(Log),\n    End = Start + length(Entries) - 1,\n    try Provider:fold_terms(Log, Start, End, fun check_heartbeat_terms/3, {Start, Entries}) of\n        % The fold should not terminate early if the provider is well-behaved.\n        {ok, {Next, []}} when Next =:= End + 1 ->\n            {ok, []};\n        {ok, {Next, NewEntries}} when Next =:= Last + 1 ->\n            {ok, NewEntries};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.heartbeat.error'),\n            {error, Reason}\n    catch\n        throw:{conflict, ConflictIndex, ConflictEntries} ->\n            {conflict, ConflictIndex, ConflictEntries};\n        throw:{missing, Index} ->\n            ?RAFT_COUNT(Table, 'log.heartbeat.corruption'),\n            {error, {missing, Index}}\n    end.\n\n-spec check_heartbeat_terms(Index :: wa_raft_log:log_index(), Term :: wa_raft_log:log_term(), Acc) -> Acc\n    when Acc :: {Next :: wa_raft_log:log_index(), Entries :: [log_entry() | binary()]}.\ncheck_heartbeat_terms(Index, Term, {Next, [Binary | Entries]}) when is_binary(Binary) ->\n    check_heartbeat_terms(Index, Term, {Next, [binary_to_term(Binary) | Entries]});\ncheck_heartbeat_terms(Index, Term, {Index, [{Term, _} | Entries]}) ->\n    {Index + 1, Entries};\ncheck_heartbeat_terms(Index, _, {Index, [_ | _] = Entries}) ->\n    throw({conflict, Index, Entries});\ncheck_heartbeat_terms(_, _, {Index, [_ | _]}) ->\n    throw({missing, Index}).\n\n%%-------------------------------------------------------------------\n%% APIs for accessing log data\n%%-------------------------------------------------------------------\n\n%% Gets the first index of the log view or as reported by the log provider.\n-spec first_index(LogOrView :: log() | view()) -> FirstIndex :: log_index().\nfirst_index(#log_view{first = First}) ->\n    First;\nfirst_index(Log) ->\n    Provider = provider(Log),\n    Provider:first_index(Log).\n\n%% Gets the last index of the log view or as reported by the log provider.\n-spec last_index(LogOrView :: log() | view()) -> LastIndex :: log_index().\nlast_index(#log_view{last = Last}) ->\n    Last;\nlast_index(Log) ->\n    Provider = provider(Log),\n    Provider:last_index(Log).\n\n-spec fold(LogOrView :: log() | view(),\n           First :: log_index(),\n           Last :: log_index() | infinity,\n           Func :: fun((Index :: log_index(), Entry :: log_entry(), Acc) -> Acc),\n           Acc) -> {ok, Acc} | {error, term()}.\nfold(LogOrView, First, Last, Func, Acc) ->\n    fold(LogOrView, First, Last, infinity, Func, Acc).\n\n%% Call the provided combining function on successive log entries in the\n%% specified log starting from the specified start index (inclusive) and ending\n%% at the specified end index (also inclusive). The combining function will\n%% always be called with log entries in log order starting with the first log\n%% entry that exists within the provided range. The combining function will\n%% not be called for those log indices within the provided range that do not\n%% have a stored log entry. The size provided to the combining function when\n%% requested is determined by the underlying log provider.\n-spec fold(\n    LogOrView :: log() | view(),\n    First :: log_index(),\n    Last :: log_index() | infinity,\n    SizeLimit :: non_neg_integer() | infinity,\n    Func ::\n        fun((Index :: log_index(), Entry :: log_entry(), Acc) -> Acc)\n        | fun((Index :: log_index(), Size :: non_neg_integer(), Entry :: log_entry(), Acc) -> Acc),\n    Acc\n) -> {ok, Acc} | {error, Reason :: term()}.\nfold(LogOrView, RawFirst, RawLast, SizeLimit, Func, Acc) ->\n    #raft_log{table = Table} = Log = log(LogOrView),\n    First = max(RawFirst, first_index(LogOrView)),\n    Last = min(RawLast, last_index(LogOrView)),\n    ?RAFT_COUNT(Table, 'log.fold'),\n    ?RAFT_COUNTV(Table, 'log.fold.total', Last - First + 1),\n    AdjFunc = if\n        is_function(Func, 3) -> fun (Index, _Size, Entry, InnerAcc) -> Func(Index, Entry, InnerAcc) end;\n        is_function(Func, 4) -> Func\n    end,\n    Provider = provider(Log),\n    case Provider:fold(Log, First, Last, SizeLimit, AdjFunc, Acc) of\n        {ok, AccOut} ->\n            {ok, AccOut};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.fold.error'),\n            {error, Reason}\n    end.\n\n-spec fold_binary(\n    LogOrView :: log() | view(),\n    First :: log_index(),\n    Last :: log_index() | infinity,\n    Func :: fun((Index :: log_index(), Entry :: binary(), Acc) -> Acc),\n    Acc\n) -> {ok, Acc} | {error, term()}.\nfold_binary(LogOrView, First, Last, Func, Acc) ->\n    fold_binary(LogOrView, First, Last, infinity, Func, Acc).\n\n%% Call the provided combining function on the external term format of\n%% successive log entries in the specified log starting from the specified\n%% start index (inclusive) and ending at the specified end index (also\n%% inclusive). The combining function will always be called with log entries\n%% in log order starting with the first log entry that exists within the\n%% provided range. The combining function will not be called for those log\n%% indices within the provided range that do not have a stored log entry. The\n%% size provided to the combining function when requested is determined by the\n%% underlying log provider.\n-spec fold_binary(\n    LogOrView :: log() | view(),\n    First :: log_index(),\n    Last :: log_index() | infinity,\n    SizeLimit :: non_neg_integer() | infinity,\n    Func :: fun((Index :: log_index(), Entry :: binary(), Acc) -> Acc),\n    Acc\n) -> {ok, Acc} | {error, term()}.\nfold_binary(LogOrView, RawFirst, RawLast, SizeLimit, Func, Acc) ->\n    #raft_log{table = Table} = Log = log(LogOrView),\n    First = max(RawFirst, first_index(LogOrView)),\n    Last = min(RawLast, last_index(LogOrView)),\n    ?RAFT_COUNT(Table, 'log.fold_binary'),\n    ?RAFT_COUNTV(Table, 'log.fold_binary.total', Last - First + 1),\n    Provider = provider(Log),\n    case Provider:fold_binary(Log, First, Last, SizeLimit, Func, Acc) of\n        {ok, AccOut} ->\n            {ok, AccOut};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.fold_binary.error'),\n            {error, Reason}\n    end.\n\n%% Folds over the terms in the log view of raw entries from the log provider\n%% between the provided first and last log indices (inclusive).\n%% If there exists a log term between the provided first and last indices then\n%% the accumulator function will be called on at least that term.\n%% This API provides no validation of the log indices and term passed by the\n%% provider to the callback function.\n-spec fold_terms(LogOrView :: log() | view(),\n                 First :: log_index(),\n                 Last :: log_index(),\n                 Func :: fun((Index :: log_index(), Term :: log_term(), Acc) -> Acc),\n                 Acc) ->\n    {ok, Acc} | {error, term()}.\nfold_terms(#log_view{log = Log, first = LogFirst, last = LogLast}, First, Last, Func, Acc) ->\n    fold_terms_impl(Log, max(First, LogFirst), min(Last, LogLast), Func, Acc);\nfold_terms(Log, First, Last, Func, Acc) ->\n    Provider = provider(Log),\n    LogFirst = Provider:first_index(Log),\n    LogLast = Provider:last_index(Log),\n    fold_terms_impl(Log, max(First, LogFirst), min(Last, LogLast), Func, Acc).\n\n-spec fold_terms_impl(\n    Log :: log(),\n    First :: log_index(),\n    Last :: log_index(),\n    Func :: fun((Index :: log_index(), Term :: log_term(), Acc) -> Acc),\n    Acc :: term()\n) -> {ok, Acc} | {error, term()}.\nfold_terms_impl(#raft_log{table = Table} = Log, First, Last, Func, AccIn) ->\n    ?RAFT_COUNT(Table, 'log.fold_terms'),\n    ?RAFT_COUNTV(Table, 'log.fold_terms.total', Last - First + 1),\n    Provider = provider(Log),\n    case Provider:fold_terms(Log, First, Last, Func, AccIn) of\n        {ok, AccOut} ->\n            {ok, AccOut};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.fold_terms.error'),\n            {error, Reason}\n    end.\n\n%% Gets the term of entry at the provided log index. When using a log view\n%% this function may return 'not_found' even if the underlying log entry still\n%% exists if the entry is outside of the log view.\n-spec term(LogOrView :: log() | view(), Index :: log_index()) -> {ok, Term :: log_term()} | not_found | {error, term()}.\nterm(#log_view{first = First, last = Last}, Index) when Index < First; Last < Index ->\n    not_found;\nterm(#log_view{log = Log}, Index) ->\n    Provider = provider(Log),\n    Provider:term(Log, Index);\nterm(Log, Index) ->\n    Provider = provider(Log),\n    Provider:term(Log, Index).\n\n%% Gets the log entry at the provided log index. When using a log view\n%% this function may return 'not_found' even if the underlying log entry still\n%% exists if the entry is outside of the log view.\n-spec get(LogOrView :: log() | view(), Index :: log_index()) -> {ok, Entry :: log_entry()} | not_found | {error, term()}.\nget(#log_view{first = First, last = Last}, Index) when Index < First; Last < Index ->\n    not_found;\nget(#log_view{log = #raft_log{table = Table} = Log}, Index) ->\n    ?RAFT_COUNT(Table, 'log.get'),\n    Provider = provider(Log),\n    Provider:get(Log, Index);\nget(#raft_log{table = Table} = Log, Index) ->\n    ?RAFT_COUNT(Table, 'log.get'),\n    Provider = provider(Log),\n    Provider:get(Log, Index).\n\n%% Fetch a contiguous range of log entries containing up to the specified\n%% number of log entries starting at the provided index. When using a log view,\n%% only those log entries that fall within the provided view will be returned.\n-spec get(LogOrView :: log() | view(), Start :: log_index(), CountLimit :: non_neg_integer()) ->\n    {ok, Entries :: [log_entry()]} | {error, term()}.\nget(LogOrView, Start, CountLimit) ->\n    get(LogOrView, Start, CountLimit, infinity).\n\n%% Fetch a contiguous range of log entries containing up to the specified\n%% number of log entries or the specified maximum total number of bytes (based\n%% on the byte sizes reported by the underlying log provider) starting at the\n%% provided index. If log entries exist at the provided starting index, then\n%% at least one log entry will be returned. When using a log view, only those\n%% log entries that fall within the provided view will be returned.\n-spec get(\n    LogOrView :: log() | view(),\n    Start :: log_index(),\n    CountLimit :: non_neg_integer(),\n    SizeLimit :: non_neg_integer() | infinity\n) -> {ok, Entries :: [log_entry()]} | {error, term()}.\nget(LogOrView, Start, CountLimit, SizeLimit) ->\n    End = Start + CountLimit - 1,\n    try fold(LogOrView, Start, End, SizeLimit, fun get_method/3, {Start, []}) of\n        {ok, {_, EntriesRev}} ->\n            {ok, lists:reverse(EntriesRev)};\n        {error, Reason} ->\n            {error, Reason}\n    catch\n        throw:{missing, Index} ->\n            ?RAFT_LOG_WARNING(\n                \"[~0p] detected missing log entry ~0p while folding range ~0p ~~ ~0p\",\n                [log_name(LogOrView), Index, Start, End]\n            ),\n            {error, corruption}\n    end.\n\n-spec get_method(Index :: log_index(), Entry :: log_entry(), Acc) -> Acc when\n    Acc :: {AccIndex :: log_index(), AccEntries :: [log_entry()]}.\nget_method(Index, Entry, {Index, AccEntries}) ->\n    {Index + 1, [Entry | AccEntries]};\nget_method(_, _, {AccIndex, _}) ->\n    throw({missing, AccIndex}).\n\n-spec get_terms(LogOrView :: log() | view(), Start :: log_index(), CountLimit :: non_neg_integer()) ->\n    {ok, Terms :: [wa_raft_log:log_term()]} | {error, term()}.\nget_terms(LogOrView, Start, CountLimit) ->\n    End = Start + CountLimit - 1,\n    try fold_terms(LogOrView, Start, End, fun get_terms_method/3, {Start, []}) of\n        {ok, {_, TermsRev}} ->\n            {ok, lists:reverse(TermsRev)};\n        {error, Reason} ->\n            {error, Reason}\n    catch\n        throw:{missing, Index} ->\n            ?RAFT_LOG_WARNING(\n                \"[~0p] detected missing log entry ~0p while folding range ~0p ~~ ~0p for terms\",\n                [log_name(LogOrView), Index, Start, End]\n            ),\n            {error, corruption}\n    end.\n\n-spec get_terms_method(Index :: log_index(), Terms :: log_term(), Acc) -> Acc when\n    Acc :: {AccIndex :: log_index(), AccTerms :: [log_term()]}.\nget_terms_method(Index, Entry, {Index, AccTerms}) ->\n    {Index + 1, [Entry | AccTerms]};\nget_terms_method(_, _, {AccIndex, _}) ->\n    throw({missing, AccIndex}).\n\n%% Produce a list of log entries in a format appropriate for inclusion within\n%% a heartbeat to a peer containing up to the specified number of log entries\n%% starting at the provided index. When using a log view, only those log\n%% entries that fall within the provided view will be returned.\n-spec entries(LogOrView :: log() | view(), Start :: log_index(), CountLimit :: non_neg_integer()) ->\n    {ok, Entries :: [log_entry() | binary()]} | {error, term()}.\nentries(LogOrView, First, Count) ->\n    entries(LogOrView, First, Count, infinity).\n\n%% Produce a list of log entries in a format appropriate for inclusion within\n%% a heartbeat to a peer containing up to the specified number of log entries\n%% or the specified maximum total number of bytes (based on the byte sizes\n%% reported by the underlying log provider when returning log entries or the\n%% byte size of each log entry binary when returning binaries) starting at the\n%% provided index. If log entries exist at the provided starting index, then\n%% at least one log entry will be returned. When using a log view, only those\n%% log entries that fall within the provided view will be returned.\n-spec entries(\n    LogOrView :: log() | view(),\n    Start :: log_index(),\n    CountLimit :: non_neg_integer(),\n    SizeLimit :: non_neg_integer() | infinity\n) -> {ok, Entries :: [log_entry() | binary()]} | {error, term()}.\nentries(LogOrView, Start, CountLimit, SizeLimit) ->\n    App = app(LogOrView),\n    Table = table(LogOrView),\n    Provider = provider(LogOrView),\n    End = Start + CountLimit - 1,\n    try\n        case erlang:function_exported(Provider, fold_binary, 6) andalso ?RAFT_LOG_HEARTBEAT_BINARY_ENTRIES(App, Table) of\n            true -> fold_binary(LogOrView, Start, End, SizeLimit, fun entries_method/3, {Start, []});\n            false -> fold(LogOrView, Start, End, SizeLimit, fun entries_method/3, {Start, []})\n        end\n    of\n        {ok, {_, EntriesRev}} ->\n            {ok, lists:reverse(EntriesRev)};\n        {error, Reason} ->\n            {error, Reason}\n    catch\n        throw:{missing, Index} ->\n            ?RAFT_LOG_WARNING(\n                \"[~0p] detected missing log entry ~0p while folding range ~0p ~~ ~0p for heartbeat\",\n                [log_name(LogOrView), Index, Start, End]\n            ),\n            {error, corruption}\n    end.\n\n-spec entries_method(Index :: log_index(), Entry :: log_entry() | binary(), Acc) -> Acc when\n    Acc :: {AccIndex :: log_index(), AccEntries :: [log_entry() | binary()]}.\nentries_method(Index, Entry, {Index, AccEntries}) ->\n    {Index + 1, [Entry | AccEntries]};\nentries_method(_, _, {AccIndex, _}) ->\n    throw({missing, AccIndex}).\n\n-spec config(LogOrView :: log() | view()) -> {ok, Index :: log_index(), Config :: wa_raft_server:config()} | not_found.\nconfig(#log_view{config = undefined}) ->\n    not_found;\nconfig(#log_view{first = First, config = {Index, _}}) when First > Index ->\n    % After trims, it is possible that we have a cached config from before the start\n    % of the log view. Don't return the cached config in this case.\n    not_found;\nconfig(#log_view{config = {Index, Config}}) ->\n    {ok, Index, Config};\nconfig(Log) ->\n    Provider = provider(Log),\n    case Provider:config(Log) of\n        {ok, Index, Config} -> {ok, Index, wa_raft_server:normalize_config(Config)};\n        Other -> Other\n    end.\n\n%%-------------------------------------------------------------------\n%% APIs for managing logs and log data\n%%-------------------------------------------------------------------\n\n%% Open the specified log (registered name or pid) at the provided position.\n%% If the log does not contain the provided position, then the log is reset\n%% to include it. Otherwise, the log is opened as is and may contain entries\n%% before and after the provided position.\n-spec open(PidOrName :: pid() | log_name(), Position :: log_pos()) -> {ok, View :: view()} | {error, term()}.\nopen(PidOrName, Position) ->\n    gen_server:call(PidOrName, {open, Position}, infinity).\n\n%% Reset the log backing the provided log view to contain only the provided\n%% position. The log entry data at the provided position will be 'undefined'.\n-spec reset(View :: view(), Position :: log_pos()) -> {ok, NewView :: view()} | {error, term()}.\nreset(#log_view{log = Log} = View, Position) ->\n    gen_server:call(log_name(Log), {reset, Position, View}, infinity).\n\n%% Truncate the log by deleting all log entries in the log at and after the\n%% provided log index. This operation is required to delete all data for the\n%% affected log indices.\n-spec truncate(View :: view(), Index :: log_index()) -> {ok, NewView :: view()} | {error, term()}.\ntruncate(#log_view{log = Log} = View, Index) ->\n    gen_server:call(log_name(Log), {truncate, Index, View}, infinity).\n\n%% Trim the log by removing log entries before the provided log index.\n%% This operation is not required to remove all data before the\n%% provided log index immediately and can defer this work to future\n%% trimming operations. This operation is asynchronous.\n-spec trim(View :: view(), Index :: log_index()) -> {ok, NewView :: view()}.\ntrim(#log_view{log = Log, first = First} = View, Index) ->\n    gen_server:cast(log_name(Log), {trim, Index}),\n    {ok, View#log_view{first = max(Index, First)}}.\n\n%% Perform a batched trimming (rotate) of the underlying log according\n%% to application environment configuration values.\n-spec rotate(View :: view(), Index :: log_index()) -> {ok, NewView :: view()}.\nrotate(#log_view{log = #raft_log{application = App, table = Table}} = View, Index) ->\n    % Current rotation configuration is based on two configuration values,\n    % 'raft_max_log_records_per_file' which indicates after how many outstanding extra\n    % log entries are in the log should we trim and 'raft_max_log_records' which\n    % indicates how many additional log entries after the fully replicated index should\n    % be considered not extraneous and be kept by rotation.\n    Interval = ?RAFT_LOG_ROTATION_INTERVAL(App, Table),\n    Keep = ?RAFT_LOG_ROTATION_KEEP(App, Table, Interval),\n    rotate(View, Index, Interval, Keep).\n\n%% Perform a batched trimming (rotate) of the underlying log where\n%% we keep some number of log entries and only trigger trimming operations\n%% every so often.\n-spec rotate(View :: view(), Index :: log_index(), Interval :: pos_integer(), Keep :: non_neg_integer()) -> {ok, NewView :: view()}.\nrotate(#log_view{log = #raft_log{table = Table}, first = First} = View, Index, Interval, Keep) when Index - Keep - First >= Interval ->\n    ?RAFT_COUNT(Table, 'log.rotate'),\n    trim(View, Index - Keep);\nrotate(#log_view{log = #raft_log{table = Table}} = View, _Index, _Interval, _Keep) ->\n    ?RAFT_COUNT(Table, 'log.rotate'),\n    {ok, View}.\n\n%% Try to flush any underlying log data that is not yet on disk to disk.\n-spec flush(LogOrView :: log() | view()) -> ok.\nflush(#log_view{log = Log}) ->\n    gen_server:cast(log_name(Log), flush);\nflush(Log) ->\n    gen_server:cast(log_name(Log), flush).\n\n%%-------------------------------------------------------------------\n%% Internal API\n%%-------------------------------------------------------------------\n\n%% Get the default name for the RAFT log server associated with the\n%% provided RAFT partition.\n-spec default_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom, one per table/partition at startup\n    binary_to_atom(<<\"raft_log_\", (atom_to_binary(Table))/binary, \"_\", (integer_to_binary(Partition))/binary>>).\n\n%% Get the registered name for the RAFT log server associated with the\n%% provided RAFT partition or the default name if no registration exists.\n-spec registered_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\nregistered_name(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> default_name(Table, Partition);\n        Options   -> Options#raft_options.log_name\n    end.\n\n-spec app(LogOrView :: log() | view()) -> App :: atom().\napp(LogOrView) ->\n    (log(LogOrView))#raft_log.application.\n\n-spec table(LogOrView :: log() | view()) -> Table :: wa_raft:table().\ntable(LogOrView) ->\n    (log(LogOrView))#raft_log.table.\n\n-spec log(LogOrView :: log() | view()) -> Log :: log().\nlog(#log_view{log = Log}) ->\n    Log;\nlog(#raft_log{} = Log) ->\n    Log.\n\n-spec log_name(LogOrView :: log() | view()) -> Name :: log_name().\nlog_name(#log_view{log = #raft_log{name = Name}}) ->\n    Name;\nlog_name(#raft_log{name = Name}) ->\n    Name.\n\n-spec provider(LogOrView :: log() | view()) -> Provider :: module().\nprovider(#log_view{log = #raft_log{provider = Provider}}) ->\n    Provider;\nprovider(#raft_log{provider = Provider}) ->\n    Provider.\n\n-spec refresh_config(View :: view()) -> NewView :: view().\nrefresh_config(#log_view{log = Log} = View) ->\n    Provider = provider(Log),\n    case Provider:config(Log) of\n        {ok, Index, Config} ->\n            View#log_view{config = {Index, wa_raft_server:normalize_config(Config)}};\n        not_found ->\n            View#log_view{config = undefined}\n    end.\n\n%%-------------------------------------------------------------------\n%% gen_server Callbacks\n%%-------------------------------------------------------------------\n\n-spec init(Options :: #raft_options{}) -> {ok, State :: #log_state{}}.\ninit(#raft_options{application = Application, table = Table, partition = Partition, log_name = Name, log_module = Provider}) ->\n    process_flag(trap_exit, true),\n\n    Log = #raft_log{\n       name = Name,\n       application = Application,\n       table = Table,\n       partition = Partition,\n       provider = Provider\n    },\n    ok = Provider:init(Log),\n\n    {ok, #log_state{log = Log}}.\n\n-spec handle_call(Request, From :: term(), State :: #log_state{}) ->\n    {reply, Reply :: term(), NewState :: #log_state{}} |\n    {noreply, NewState :: #log_state{}}\n    when Request ::\n        {open, Position :: log_pos()} |\n        {reset, Position :: log_pos(), View :: view()} |\n        {truncate, Index :: log_index(), View :: view()}.\nhandle_call({open, Position}, _From, State) ->\n    {Reply, NewState} = handle_open(Position, State),\n    {reply, Reply, NewState};\nhandle_call({reset, Position, View}, _From, State) ->\n    case handle_reset(Position, View, State) of\n        {ok, NewView, NewState} ->\n            {reply, {ok, NewView}, NewState};\n        {error, Reason} ->\n            {reply, {error, Reason}, State}\n    end;\nhandle_call({truncate, Index, View}, _From, State) ->\n    case handle_truncate(Index, View, State) of\n        {ok, NewView, NewState} ->\n            {reply, {ok, NewView}, NewState};\n        {error, Reason} ->\n            {reply, {error, Reason}, State}\n    end;\nhandle_call(Request, From, #log_state{log = Log} = State) ->\n    ?RAFT_LOG_NOTICE(\"[~p] got unrecognized call ~p from ~p\", [log_name(Log), Request, From]),\n    {noreply, State}.\n\n-spec handle_cast(Request, State :: #log_state{}) -> {noreply, NewState :: #log_state{}}\n    when Request :: flush | {trim, Index :: log_index()}.\nhandle_cast(flush, #log_state{log = #raft_log{table = Table} = Log} = State) ->\n    ?RAFT_COUNT(Table, 'log.flush'),\n    Provider = provider(Log),\n    Provider:flush(Log),\n    {noreply, State};\nhandle_cast({trim, Index}, #log_state{log = Log} = State) ->\n    case handle_trim(Index, State) of\n        {ok, NewState} ->\n            {noreply, NewState};\n        {error, Reason} ->\n            ?RAFT_LOG_WARNING(\"[~p] failed to trim log due to ~p\", [log_name(Log), Reason]),\n            {noreply, State}\n    end;\nhandle_cast(Request, #log_state{log = Log} = State) ->\n    ?RAFT_LOG_NOTICE(\"[~p] got unrecognized cast ~p\", [log_name(Log), Request]),\n    {noreply, State}.\n\n-spec terminate(Reason :: term(), State :: #log_state{}) -> term().\nterminate(Reason, #log_state{log = Log, state = State}) ->\n    Provider = provider(Log),\n    ?RAFT_LOG_NOTICE(\"[~p] terminating due to ~p\", [Log, Reason]),\n    State =/= ?PROVIDER_NOT_OPENED andalso Provider:close(Log, State).\n\n%%-------------------------------------------------------------------\n%% RAFT Log Server Logic\n%%-------------------------------------------------------------------\n\n-spec handle_open(Position :: log_pos(), State :: #log_state{}) ->\n    {{ok, NewView :: view()} | {error, Reason :: term()}, NewState :: #log_state{}}.\nhandle_open(#raft_log_pos{index = Index, term = Term} = Position,\n            #log_state{log = #raft_log{name = Name, table = Table, provider = Provider} = Log} = State0) ->\n    ?RAFT_COUNT(Table, 'log.open'),\n    ?RAFT_LOG_NOTICE(\"[~p] opening log at position ~p:~p\", [Name, Index, Term]),\n    case Provider:open(Log) of\n        {ok, ProviderState} ->\n            Action = case Provider:get(Log, Index) of\n                {ok, {Term, _Op}} ->\n                    none;\n                {ok, {MismatchTerm, _Op}} ->\n                    ?RAFT_LOG_WARNING(\n                        \"[~p] resetting log due to expecting term ~p at ~p but log contains term ~p\",\n                        [Name, Term, Index, MismatchTerm]\n                    ),\n                    reset;\n                not_found ->\n                    reset;\n                Other ->\n                    {failed, Other}\n            end,\n\n            State1 = State0#log_state{state = ProviderState},\n            View0 = #log_view{log = Log},\n            case Action of\n                none ->\n                    ?RAFT_COUNT(Table, 'log.open.normal'),\n                    View1 = case Provider:first_index(Log) of\n                        undefined ->\n                            ?RAFT_LOG_WARNING(\n                                \"[~p] opened log normally but the first index was not set\",\n                                [Name]\n                            ),\n                            View0;\n                        FirstIndex ->\n                            View0#log_view{first = FirstIndex}\n                    end,\n                    View2 = case Provider:last_index(Log) of\n                        undefined ->\n                            ?RAFT_LOG_WARNING(\n                                \"[~p] opened log normally but the last index was not set\",\n                                [Name]\n                            ),\n                            View1;\n                        LastIndex ->\n                            View1#log_view{last = LastIndex}\n                    end,\n                    View3 = refresh_config(View2),\n                    {{ok, View3}, State1};\n                reset ->\n                    ?RAFT_COUNT(Table, 'log.open.reset'),\n                    case handle_reset(Position, View0, State1) of\n                        {ok, View1, State2} ->\n                            {{ok, View1}, State2};\n                        {error, Reason} ->\n                            ?RAFT_COUNT(Table, 'log.open.reset.error'),\n                            {{error, Reason}, State1}\n                    end;\n                {failed, Return} ->\n                    ?RAFT_COUNT(Table, 'log.open.error'),\n                    {Return, State1}\n            end;\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.open.error'),\n            {{error, Reason}, State0#log_state{state = ?PROVIDER_NOT_OPENED}}\n    end.\n\n-spec handle_reset(Position :: log_pos(), View :: view(), State :: #log_state{}) ->\n    {ok, NewView :: view(), NewState :: #log_state{}} | {error, Reason :: term()}.\nhandle_reset(_Position, _View, #log_state{state = ?PROVIDER_NOT_OPENED}) ->\n    {error, not_open};\nhandle_reset(#raft_log_pos{index = 0, term = Term}, _View, #log_state{log = Log}) when Term =/= 0 ->\n    ?RAFT_LOG_ERROR(\"[~p] rejects reset to index 0 with non-zero term ~p\", [log_name(Log), Term]),\n    {error, invalid_position};\nhandle_reset(#raft_log_pos{index = Index, term = Term} = Position, View0,\n             #log_state{log = #raft_log{table = Table} = Log, state = ProviderState} = State0) ->\n    ?RAFT_COUNT(Table, 'log.reset'),\n    ?RAFT_LOG_NOTICE(\"[~p] resetting log to position ~p:~p\", [log_name(Log), Index, Term]),\n    Provider = provider(Log),\n    case Provider:reset(Log, Position, ProviderState) of\n        {ok, NewProviderState} ->\n            View1 = View0#log_view{first = Index, last = Index, config = undefined},\n            State1 = State0#log_state{state = NewProviderState},\n            {ok, View1, State1};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.reset.error'),\n            {error, Reason}\n    end.\n\n-spec handle_truncate(Index :: log_index(), View :: view(), State :: #log_state{}) ->\n    {ok, NewView :: view(), NewState :: #log_state{}} | {error, Reason :: term()}.\nhandle_truncate(_Index, _View, #log_state{state = ?PROVIDER_NOT_OPENED}) ->\n    {error, not_open};\nhandle_truncate(Index, #log_view{first = First}, #log_state{log = Log}) when Index =< First ->\n    ?RAFT_LOG_ERROR(\"[~p] rejects log deletion by truncation to ~p for log starting at ~p\", [log_name(Log), Index, First]),\n    {error, invalid_position};\nhandle_truncate(Index, #log_view{last = Last} = View0, #log_state{log = #raft_log{table = Table} = Log, state = ProviderState} = State0) ->\n    ?RAFT_COUNT(Table, 'log.truncate'),\n    ?RAFT_LOG_NOTICE(\"[~p] truncating log from ~p to past ~p\", [log_name(Log), Last, Index]),\n    Provider = provider(Log),\n    case Provider:truncate(Log, Index, ProviderState) of\n        {ok, NewProviderState} ->\n            View1 = View0#log_view{last = min(Last, Index - 1)},\n            View2 = refresh_config(View1),\n            State1 = State0#log_state{state = NewProviderState},\n            {ok, View2, State1};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.truncate.error'),\n            {error, Reason}\n    end.\n\n%% Trim is an asynchronous operation so we do not use the view here.\n%% Rather, the wa_raft_log:trim/2 API will assume that the trim succeeded and\n%% optimistically update the view to advance the start of the log to the provided index.\n-spec handle_trim(Index :: log_index(), State :: #log_state{}) ->\n    {ok, NewState :: #log_state{}} | {error, Reason :: term()}.\nhandle_trim(_Index, #log_state{state = ?PROVIDER_NOT_OPENED}) ->\n    {error, not_open};\nhandle_trim(Index, #log_state{log = #raft_log{table = Table} = Log, state = ProviderState} = State) ->\n    ?RAFT_COUNT(Table, 'log.trim'),\n    ?RAFT_LOG_DEBUG(\"[~p] trimming log to ~p\", [log_name(Log), Index]),\n    Provider = provider(Log),\n    case Provider:trim(Log, Index, ProviderState) of\n        {ok, NewProviderState} ->\n            {ok, State#log_state{state = NewProviderState}};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, 'log.trim.error'),\n            {error, Reason}\n    end.\n"
  },
  {
    "path": "src/wa_raft_log_ets.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module is an implementation of a completely in-memory RAFT\n%%% log provider that uses ETS as a backing store for the log data.\n%%% This module is only suitable as a log provider for an fully\n%%% in-memory RAFT cluster and should not be used when any durability\n%%% guarantees are required against node shutdown.\n\n-module(wa_raft_log_ets).\n-compile(warn_missing_spec_all).\n-behaviour(wa_raft_log).\n\n%% RAFT log provider interface for accessing log data\n-export([\n    first_index/1,\n    last_index/1,\n    fold/6,\n    fold_terms/5,\n    get/2,\n    term/2,\n    config/1\n]).\n\n%% RAFT log provider interface for writing new log data\n-export([\n    append/4\n]).\n\n%% RAFT log provider interface for managing underlying RAFT log\n-export([\n    init/1,\n    open/1,\n    close/2,\n    reset/3,\n    truncate/3,\n    trim/3,\n    flush/1\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n-type state() :: undefined.\n\n%%-------------------------------------------------------------------\n%% RAFT log provider interface for accessing log data\n%%-------------------------------------------------------------------\n\n-spec first_index(Log :: wa_raft_log:log()) -> undefined | wa_raft_log:log_index().\nfirst_index(#raft_log{name = Name}) ->\n    case ets:first(Name) of\n        '$end_of_table' -> undefined;\n        Key             -> Key\n    end.\n\n-spec last_index(Log :: wa_raft_log:log()) -> undefined | wa_raft_log:log_index().\nlast_index(#raft_log{name = Name}) ->\n    case ets:last(Name) of\n        '$end_of_table' -> undefined;\n        Key             -> Key\n    end.\n\n-spec fold(Log :: wa_raft_log:log(),\n           Start :: wa_raft_log:log_index() | '$end_of_table',\n           End :: wa_raft_log:log_index(),\n           SizeLimit :: non_neg_integer() | infinity,\n           Func :: fun((Index :: wa_raft_log:log_index(), Size :: non_neg_integer(), Entry :: wa_raft_log:log_entry(), Acc) -> Acc),\n           Acc) -> {ok, Acc}.\nfold(Log, Start, End, SizeLimit, Func, Acc) ->\n    fold_impl(Log, Start, End, 0, SizeLimit, Func, Acc).\n\n-spec fold_impl(\n    Log :: wa_raft_log:log(),\n    Start :: wa_raft_log:log_index() | '$end_of_table',\n    End :: wa_raft_log:log_index(),\n    Size :: non_neg_integer(),\n    SizeLimit :: non_neg_integer() | infinity,\n    Func :: fun((Index :: wa_raft_log:log_index(), Size :: non_neg_integer(), Entry :: wa_raft_log:log_entry(), Acc) -> Acc),\n    Acc\n) -> {ok, Acc}.\nfold_impl(_Log, Start, End, Size, SizeLimit, _Func, Acc) when End < Start; Size >= SizeLimit ->\n    {ok, Acc};\nfold_impl(#raft_log{name = Name} = Log, Start, End, Size, SizeLimit, Func, Acc) ->\n    case ets:lookup(Name, Start) of\n        [{Start, Entry}] ->\n            EntrySize = erlang:external_size(Entry),\n            fold_impl(Log, ets:next(Name, Start), End, Size + EntrySize, SizeLimit, Func, Func(Start, EntrySize, Entry, Acc));\n        [] ->\n            fold_impl(Log, ets:next(Name, Start), End, Size, SizeLimit, Func, Acc)\n    end.\n\n-spec fold_terms(Log :: wa_raft_log:log(),\n    Start :: wa_raft_log:log_index() | '$end_of_table',\n    End :: wa_raft_log:log_index(),\n    Func :: fun((Index :: wa_raft_log:log_index(), Entry :: wa_raft_log:log_term(), Acc) -> Acc),\n    Acc) -> {ok, Acc}.\nfold_terms(Log, Start, End, Func, Acc) ->\n    fold_terms_impl(Log, Start, End, Func, Acc).\n\n-spec fold_terms_impl(\n    Log :: wa_raft_log:log(),\n    Start :: wa_raft_log:log_index() | '$end_of_table',\n    End :: wa_raft_log:log_index(),\n    Func :: fun((Index :: wa_raft_log:log_index(), Term :: wa_raft_log:log_term(), Acc) -> Acc),\n    Acc\n    ) -> {ok, Acc}.\nfold_terms_impl(_Log, Start, End, _Func, Acc) when End < Start ->\n    {ok, Acc};\nfold_terms_impl(#raft_log{name = Name} = Log, Start, End, Func, Acc) ->\n    case ets:lookup(Name, Start) of\n        [{Start, {Term, _Op}}] ->\n            fold_terms_impl(Log, ets:next(Name, Start), End, Func, Func(Start, Term, Acc));\n        [] ->\n            fold_terms_impl(Log, ets:next(Name, Start), End, Func, Acc)\n        end.\n\n-spec get(Log :: wa_raft_log:log(), Index :: wa_raft_log:log_index()) -> {ok, Entry :: wa_raft_log:log_entry()} | not_found.\nget(#raft_log{name = Name}, Index) ->\n    case ets:lookup(Name, Index) of\n        [{Index, Entry}] -> {ok, Entry};\n        []               -> not_found\n    end.\n\n-spec term(Log :: wa_raft_log:log(), Index :: wa_raft_log:log_index()) -> {ok, Term :: wa_raft_log:log_term()} | not_found.\nterm(Log, Index) ->\n    case get(Log, Index) of\n        {ok, {Term, _Op}} -> {ok, Term};\n        not_found         -> not_found\n    end.\n\n-spec config(Log :: wa_raft_log:log()) -> {ok, Index :: wa_raft_log:log_index(), Entry :: wa_raft_server:config()} | not_found.\nconfig(#raft_log{name = Name}) ->\n    case ets:select_reverse(Name, [{{'$1', {'_', {'_', {config, '$2'}}}}, [], [{{'$1', '$2'}}]}], 1) of\n        {[{Index, Config}], _Cont} -> {ok, Index, Config};\n        _                          -> not_found\n    end.\n\n%%-------------------------------------------------------------------\n%% RAFT log provider interface for writing new log data\n%%-------------------------------------------------------------------\n\n-spec append(View :: wa_raft_log:view(), Entries :: [wa_raft_log:log_entry() | binary()], Mode :: strict | relaxed, Priority :: wa_raft_acceptor:priority()) -> ok.\nappend(View, Entries, _Mode, _Priority) ->\n    Name = wa_raft_log:log_name(View),\n    Last = wa_raft_log:last_index(View),\n    true = ets:insert(Name, append_decode(Last + 1, Entries)),\n    ok.\n\n-spec append_decode(Index :: wa_raft_log:log_index(), Entries :: [wa_raft_log:log_entry() | binary()]) ->\n    [{wa_raft_log:log_index(), wa_raft_log:log_entry()}].\nappend_decode(_, []) ->\n    [];\nappend_decode(Index, [Entry | Entries]) ->\n    NewEntry =\n        case is_binary(Entry) of\n            true -> binary_to_term(Entry);\n            false -> Entry\n        end,\n    [{Index, NewEntry} | append_decode(Index + 1, Entries)].\n\n%%-------------------------------------------------------------------\n%% RAFT log provider interface for managing underlying RAFT log\n%%-------------------------------------------------------------------\n\n-spec init(Log :: wa_raft_log:log()) -> ok.\ninit(#raft_log{name = LogName}) ->\n    ets:new(LogName, [ordered_set, public, named_table]),\n    ok.\n\n-spec open(Log :: wa_raft_log:log()) -> {ok, State :: state()}.\nopen(_Log) ->\n    {ok, undefined}.\n\n-spec close(Log :: wa_raft_log:log(), State :: state()) -> ok.\nclose(_Log, _State) ->\n    ok.\n\n-spec reset(Log :: wa_raft_log:log(), Position :: wa_raft_log:log_pos(), State :: state()) ->\n    {ok, NewState :: state()}.\nreset(#raft_log{name = Name}, #raft_log_pos{index = Index, term = Term}, State) ->\n    true = ets:delete_all_objects(Name),\n    true = ets:insert(Name, {Index, {Term, undefined}}),\n    {ok, State}.\n\n-spec truncate(Log :: wa_raft_log:log(), Index :: wa_raft_log:log_index() | '$end_of_table', State :: state()) ->\n    {ok, NewState :: state()}.\ntruncate(_Log, '$end_of_table', State) ->\n    {ok, State};\ntruncate(#raft_log{name = Name} = Log, Index, State) ->\n    true = ets:delete(Name, Index),\n    truncate(Log, ets:next(Name, Index), State).\n\n-spec trim(Log :: wa_raft_log:log(), Index :: wa_raft_log:log_index(), State :: state()) ->\n    {ok, NewState :: state()}.\ntrim(Log, Index, State) ->\n    trim_impl(Log, Index - 1),\n    {ok, State}.\n\n-spec trim_impl(Log :: wa_raft_log:log(), Index :: wa_raft_log:log_index() | '$end_of_table') -> ok.\ntrim_impl(_Log, '$end_of_table') ->\n    ok;\ntrim_impl(#raft_log{name = Name} = Log, Index) ->\n    true = ets:delete(Name, Index),\n    trim_impl(Log, ets:prev(Name, Index)).\n\n-spec flush(Log :: wa_raft_log:log()) -> ok.\nflush(_Log) ->\n    ok.\n"
  },
  {
    "path": "src/wa_raft_metrics.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% Pluggable metrics interface to allow integration with different metrics system.\n%%% The default implementation skips metrics logging and does nothing.\n\n-module(wa_raft_metrics).\n-compile(warn_missing_spec_all).\n\n%% Public API\n-export([\n    install/1\n]).\n\n%% Default Implementation\n-export([\n    count/1,\n    countv/2,\n    gather/2,\n    gather_latency/2\n]).\n\n%% Public Types\n-export_type([\n    metric/0,\n    value/0\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n%%-------------------------------------------------------------------\n%% RAFT Metrics Behaviour\n%%-------------------------------------------------------------------\n\n%% Report a single occurrence of some metric.\n-callback count(metric()) -> ok.\n%% Report a number of occurrences of some metric.\n-callback countv(metric(), value()) -> ok.\n%% Report the measured value of an occurrence of some metric.\n-callback gather(metric(), value()) -> ok.\n%% Report the measured latency of an occurrence of some metric.\n-callback gather_latency(metric(), value()) -> ok.\n\n%%-------------------------------------------------------------------\n%% Public Types\n%%-------------------------------------------------------------------\n\n-type metric() :: atom() | tuple().\n-type value() :: integer().\n\n%%-------------------------------------------------------------------\n%% Public API\n%%-------------------------------------------------------------------\n\n%% Replace the previously installed or default module used to report\n%% RAFT metrics with the provided module.\n-spec install(Module :: module()) -> ok.\ninstall(Module) ->\n    persistent_term:put(?RAFT_METRICS_MODULE_KEY, Module).\n\n%%-------------------------------------------------------------------\n%% Default Implementation\n%%-------------------------------------------------------------------\n\n-spec count(metric()) -> ok.\ncount(_Metric) ->\n    ok.\n\n-spec countv(metric(), value()) -> ok.\ncountv(_Metric, _Value) ->\n    ok.\n\n-spec gather(metric(), value()) -> ok.\ngather(_Metric, _Value) ->\n    ok.\n\n-spec gather_latency(metric(), value()) -> ok.\ngather_latency(_Metric, _Value) ->\n    ok.\n"
  },
  {
    "path": "src/wa_raft_part_sup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% OTP Supervisor for monitoring RAFT processes. Correctness of RAFT\n%%% relies on the consistency of the signaling between the processes,\n%%% this supervisor is configured to restart all RAFT processes\n%%% when any of them exits abnormally.\n\n-module(wa_raft_part_sup).\n-compile(warn_missing_spec_all).\n-behaviour(supervisor).\n\n%% OTP Supervision\n-export([\n    child_spec/1,\n    child_spec/2,\n    start_link/2\n]).\n\n%% Options API\n-export([\n    default_name/2,\n    registered_name/2\n]).\n\n%% Options API\n-export([\n    partition_path/2,\n    default_partition_path/3,\n    registered_partition_path/2\n]).\n\n%% Options API\n-export([\n    options/2\n]).\n\n%% Supervisor callbacks\n-export([\n    init/1\n]).\n\n%% Test API\n-export([\n    prepare_spec/2\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%% Key in persistent_term for the options associated with a RAFT partition.\n-define(OPTIONS_KEY(Table, Partition), {?MODULE, Table, Partition}).\n\n%%-------------------------------------------------------------------\n%% OTP supervision\n%%-------------------------------------------------------------------\n\n%% Returns a spec suitable for use with a `simple_one_for_one` supervisor.\n-spec child_spec(Application :: atom()) -> supervisor:child_spec().\nchild_spec(Application) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Application]},\n        restart => permanent,\n        shutdown => infinity,\n        type => supervisor,\n        modules => [?MODULE]\n    }.\n\n-spec child_spec(Application :: atom(), Spec :: wa_raft:args()) -> supervisor:child_spec().\nchild_spec(Application, Spec) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Application, Spec]},\n        restart => permanent,\n        shutdown => infinity,\n        type => supervisor,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Application :: atom(), Spec :: wa_raft:args()) -> supervisor:startlink_ret().\nstart_link(Application, Spec) ->\n    %% First normalize the provided specification into a full options record.\n    Options = #raft_options{table = Table, partition = Partition, supervisor_name = Name} = normalize_spec(Application, Spec),\n\n    %% Then put the declared options for the current RAFT partition into\n    %% persistent term for access by shared resources and other services.\n    %% The RAFT options for a table are not expected to change during the\n    %% runtime of the RAFT application and so repeated updates should not\n    %% result in any GC load. Warn if this is case.\n    PrevOptions = persistent_term:get(?OPTIONS_KEY(Table, Partition), Options),\n    PrevOptions =/= Options andalso\n        ?RAFT_LOG_WARNING(\n            ?MODULE_STRING \" storing changed options for RAFT partitition ~0p/~0p\",\n            [Table, Partition]\n        ),\n    ok = persistent_term:put(?OPTIONS_KEY(Table, Partition), Options),\n\n    supervisor:start_link({local, Name}, ?MODULE, Options).\n\n%%-------------------------------------------------------------------\n%% Internal API\n%%-------------------------------------------------------------------\n\n%% Get the default name for the RAFT partition supervisor associated with the\n%% provided RAFT partition.\n-spec default_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom for supervisor name\n    list_to_atom(\"raft_sup_\" ++ atom_to_list(Table) ++ \"_\" ++ integer_to_list(Partition)).\n\n%% Get the registered name for the RAFT partition supervisor associated with the\n%% provided RAFT partition or the default name if no registration exists.\n-spec registered_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\nregistered_name(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> default_name(Table, Partition);\n        Options   -> Options#raft_options.supervisor_name\n    end.\n\n%% Get the database directory that should be used by the RAFT partition that\n%% is started under the provided application with the provided arguments.\n-spec partition_path(Application :: atom(), Spec :: wa_raft:args()) -> Database :: file:filename().\npartition_path(Application, #{table := Table, partition := Partition}) ->\n    Root = wa_raft_env:database_path(Application),\n    default_partition_path(Root, Table, Partition).\n\n%% Get the default location for the database directory associated with the\n%% provided RAFT partition given the database of the RAFT root.\n-spec default_partition_path(Root :: file:filename(), Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Database :: file:filename().\ndefault_partition_path(Root, Table, Partition) ->\n    filename:join(Root, atom_to_list(Table) ++ \".\" ++ integer_to_list(Partition)).\n\n%% Get the registered database directory for the provided RAFT partition. An\n%% error is raised if no registration exists.\n-spec registered_partition_path(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Database :: file:filename().\nregistered_partition_path(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> error({not_registered, Table, Partition});\n        Options   -> Options#raft_options.database\n    end.\n\n-spec options(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> #raft_options{} | undefined.\noptions(Table, Partition) ->\n    persistent_term:get(?OPTIONS_KEY(Table, Partition), undefined).\n\n-spec normalize_spec(Application :: atom(), Spec :: wa_raft:args()) -> #raft_options{}.\nnormalize_spec(Application, #{table := Table, partition := Partition} = Spec) ->\n    Database = partition_path(Application, Spec),\n    ServerName = wa_raft_server:default_name(Table, Partition),\n    LogName = wa_raft_log:default_name(Table, Partition),\n    StorageName = wa_raft_storage:default_name(Table, Partition),\n    AcceptorName = wa_raft_acceptor:default_name(Table, Partition),\n    #raft_options{\n        application = Application,\n        table = Table,\n        partition = Partition,\n        % RAFT identity always uses the default RAFT server name for the partition\n        self = #raft_identity{name = wa_raft_server:default_name(Table, Partition), node = node()},\n        identifier = #raft_identifier{application = Application, table = Table, partition = Partition},\n        database = Database,\n        acceptor_name = AcceptorName,\n        distribution_module = maps:get(distribution_module, Spec, wa_raft_env:get_env(Application, raft_distribution_module, ?RAFT_DEFAULT_DISTRIBUTION_MODULE)),\n        log_name = LogName,\n        log_module = maps:get(log_module, Spec, wa_raft_env:get_env(Application, raft_log_module, ?RAFT_DEFAULT_LOG_MODULE)),\n        label_module = maps:get(label_module, Spec, wa_raft_env:get_env(Application, raft_label_module, ?RAFT_DEFAULT_LABEL_MODULE)),\n        queue_name = wa_raft_queue:default_name(Table, Partition),\n        queue_counters = wa_raft_queue:default_counters(),\n        queue_reads = wa_raft_queue:default_read_queue_name(Table, Partition),\n        server_name = ServerName,\n        storage_name = StorageName,\n        storage_module = maps:get(storage_module, Spec, wa_raft_env:get_env(Application, raft_storage_module, ?RAFT_DEFAULT_STORAGE_MODULE)),\n        supervisor_name = default_name(Table, Partition),\n        transport_cleanup_name = wa_raft_transport_cleanup:default_name(Table, Partition),\n        transport_directory = wa_raft_transport:default_directory(Database),\n        transport_module = maps:get(transport_module, Spec, wa_raft_env:get_env(Application, {raft_transport_module, transport_module}, ?RAFT_DEFAULT_TRANSPORT_MODULE))\n    }.\n\n%%-------------------------------------------------------------------\n%% Test API\n%%-------------------------------------------------------------------\n\n-spec prepare_spec(Application :: atom(), Spec :: wa_raft:args()) -> #raft_options{}.\nprepare_spec(Application, Spec) ->\n    Options = #raft_options{table = Table, partition = Partition} = normalize_spec(Application, Spec),\n    ok = persistent_term:put(?OPTIONS_KEY(Table, Partition), Options),\n    Options.\n\n%%-------------------------------------------------------------------\n%% Supervisor callbacks\n%%-------------------------------------------------------------------\n\n-spec init(Options :: #raft_options{}) -> {ok, {supervisor:sup_flags(), [supervisor:child_spec()]}}.\ninit(Options) ->\n    ChildSpecs = [\n        wa_raft_queue:child_spec(Options),\n        wa_raft_storage:child_spec(Options),\n        wa_raft_log:child_spec(Options),\n        wa_raft_server:child_spec(Options),\n        wa_raft_acceptor:child_spec(Options),\n        wa_raft_transport_cleanup:child_spec(Options)\n    ],\n    {ok, {#{strategy => one_for_all, intensity => 10, period => 1}, ChildSpecs}}.\n"
  },
  {
    "path": "src/wa_raft_queue.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module implements tracking of pending requests and queue limits.\n\n-module(wa_raft_queue).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n%% PUBLIC API\n-export([\n    queues/1,\n    queues/2,\n    commit_queue_size/2,\n    commit_queue_size/3,\n    commit_queue_full/2,\n    commit_queue_full/3,\n    apply_queue_size/1,\n    apply_queue_size/2,\n    apply_queue_byte_size/1,\n    apply_queue_byte_size/2,\n    apply_queue_full/1,\n    apply_queue_full/2\n]).\n\n%% INTERNAL API\n-export([\n    default_name/2,\n    default_counters/0,\n    default_read_queue_name/2,\n    registered_name/2\n]).\n\n%% PENDING COMMIT QUEUE API\n-export([\n    commit_started/2,\n    commit_cancelled/4,\n    commit_completed/4\n]).\n\n%% PENDING READ API\n-export([\n    reserve_read/1,\n    submit_read/4,\n    query_reads/2,\n    fulfill_read/3,\n    fulfill_incomplete_read/3,\n    fulfill_all_reads/2\n]).\n\n%% APPLY QUEUE API\n-export([\n    reserve_apply/2,\n    fulfill_apply/2\n]).\n\n%% OTP SUPERVISION\n-export([\n    child_spec/1,\n    start_link/1\n]).\n\n%% QUEUE SERVER CALLBACKS\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    terminate/2\n]).\n\n%% TYPES\n-export_type([\n    queues/0\n]).\n\n-include_lib(\"stdlib/include/ms_transform.hrl\"). % used by ets:fun2ms\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%%-------------------------------------------------------------------\n\n%% ETS table creation options shared by all queue tables\n-define(RAFT_QUEUE_TABLE_OPTIONS, [named_table, public, {read_concurrency, true}, {write_concurrency, true}]).\n\n%% Total number of counters for RAFT partition specfic counters\n-define(RAFT_NUMBER_OF_QUEUE_SIZE_COUNTERS, 5).\n%% Index into counter reference for counter tracking apply queue size\n-define(RAFT_APPLY_QUEUE_SIZE_COUNTER, 1).\n%% Index into counter reference for counter tracking apply total byte size\n-define(RAFT_APPLY_QUEUE_BYTE_SIZE_COUNTER, 2).\n%% Index into counter reference for counter tracking high priority commit queue size\n-define(RAFT_HIGH_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 3).\n%% Index into counter reference for counter tracking low priority commit queue size\n-define(RAFT_LOW_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 4).\n%% Index into counter reference for counter tracking read queue size\n-define(RAFT_READ_QUEUE_SIZE_COUNTER, 5).\n\n%%-------------------------------------------------------------------\n%% INTERNAL TYPES\n%%-------------------------------------------------------------------\n\n-record(state, {\n    name :: atom()\n}).\n\n-record(queues, {\n    application :: atom(),\n    table :: wa_raft:table(),\n    counters :: atomics:atomics_ref(),\n    reads :: atom()\n}).\n-opaque queues() :: #queues{}.\n\n%%-------------------------------------------------------------------\n%% PUBLIC API\n%%-------------------------------------------------------------------\n\n-spec queues(Options :: #raft_options{}) -> Queues :: queues().\nqueues(Options) ->\n    #queues{\n        application = Options#raft_options.application,\n        table = Options#raft_options.table,\n        counters = Options#raft_options.queue_counters,\n        reads = Options#raft_options.queue_reads\n    }.\n\n-spec queues(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Queues :: queues() | undefined.\nqueues(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> undefined;\n        Options -> queues(Options)\n    end.\n\n-spec commit_queue_size(Queues :: queues(), Priority :: wa_raft_acceptor:priority()) -> non_neg_integer().\ncommit_queue_size(#queues{counters = Counters}, high) ->\n    atomics:get(Counters, ?RAFT_HIGH_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER);\ncommit_queue_size(#queues{counters = Counters}, low) ->\n    atomics:get(Counters, ?RAFT_LOW_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER).\n\n-spec commit_queue_size(Table :: wa_raft:table(), Partition :: wa_raft:partition(), Priority :: wa_raft_acceptor:priority()) -> non_neg_integer().\ncommit_queue_size(Table, Partition, Priority) ->\n    case queues(Table, Partition) of\n        undefined -> 0;\n        Queue     -> commit_queue_size(Queue, Priority)\n    end.\n\n-spec commit_queue_full(Queues :: queues(), Priority :: wa_raft_acceptor:priority()) -> boolean().\ncommit_queue_full(#queues{application = App, table = Table, counters = Counters}, high) ->\n    atomics:get(Counters, ?RAFT_HIGH_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER) >= ?RAFT_MAX_PENDING_HIGH_PRIORITY_COMMITS(App, Table);\ncommit_queue_full(#queues{application = App, table = Table, counters = Counters}, low) ->\n    atomics:get(Counters, ?RAFT_LOW_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER) >= ?RAFT_MAX_PENDING_LOW_PRIORITY_COMMITS(App, Table).\n\n-spec commit_queue_full(Table :: wa_raft:table(), Partition :: wa_raft:partition(), Priority :: wa_raft_acceptor:priority()) -> boolean().\ncommit_queue_full(Table, Partition, Priority) ->\n    case queues(Table, Partition) of\n        undefined -> false;\n        Queues    -> commit_queue_full(Queues, Priority)\n    end.\n\n-spec apply_queue_size(Queues :: queues()) -> non_neg_integer().\napply_queue_size(#queues{counters = Counters}) ->\n    atomics:get(Counters, ?RAFT_APPLY_QUEUE_SIZE_COUNTER).\n\n-spec apply_queue_size(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> non_neg_integer().\napply_queue_size(Table, Partition) ->\n    case queues(Table, Partition) of\n        undefined -> 0;\n        Queues    -> apply_queue_size(Queues)\n    end.\n\n-spec apply_queue_byte_size(Queues :: queues()) -> non_neg_integer().\napply_queue_byte_size(#queues{counters = Counters}) ->\n    atomics:get(Counters, ?RAFT_APPLY_QUEUE_BYTE_SIZE_COUNTER).\n\n-spec apply_queue_byte_size(wa_raft:table(), wa_raft:partition()) -> non_neg_integer().\napply_queue_byte_size(Table, Partition) ->\n    case queues(Table, Partition) of\n        undefined -> 0;\n        Queues    -> apply_queue_byte_size(Queues)\n    end.\n\n-spec apply_queue_full(Queues :: queues()) -> boolean().\napply_queue_full(#queues{application = App, table = Table, counters = Counters}) ->\n    atomics:get(Counters, ?RAFT_APPLY_QUEUE_SIZE_COUNTER) >= ?RAFT_MAX_PENDING_APPLIES(App, Table) orelse\n        atomics:get(Counters, ?RAFT_APPLY_QUEUE_BYTE_SIZE_COUNTER) >= ?RAFT_MAX_PENDING_APPLY_BYTES(App, Table).\n\n-spec apply_queue_full(wa_raft:table(), wa_raft:partition()) -> boolean().\napply_queue_full(Table, Partition) ->\n    case queues(Table, Partition) of\n        undefined -> false;\n        Queues    -> apply_queue_full(Queues)\n    end.\n\n%%-------------------------------------------------------------------\n%% INTERNAL API\n%%-------------------------------------------------------------------\n\n%% Get the default name for the RAFT queue server associated with the\n%% provided RAFT partition.\n-spec default_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom, one per table/partition at startup\n    binary_to_atom(<<\"raft_queue_\", (atom_to_binary(Table))/bytes, \"_\", (integer_to_binary(Partition))/bytes>>).\n\n%% Create a properly-sized atomics array for use by a RAFT queue\n-spec default_counters() -> Counters :: atomics:atomics_ref().\ndefault_counters() ->\n    atomics:new(?RAFT_NUMBER_OF_QUEUE_SIZE_COUNTERS, []).\n\n%% Get the default name for the RAFT read queue ETS table associated with the\n%% provided RAFT partition.\n-spec default_read_queue_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_read_queue_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom, one per table/partition at startup\n    binary_to_atom(<<\"raft_read_queue_\", (atom_to_binary(Table))/bytes, \"_\", (integer_to_binary(Partition))/bytes>>).\n\n%% Get the registered name for the RAFT queue server associated with the\n%% provided RAFT partition or the default name if no registration exists.\n-spec registered_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\nregistered_name(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> default_name(Table, Partition);\n        Options   -> Options#raft_options.queue_name\n    end.\n\n%%-------------------------------------------------------------------\n%% PENDING COMMIT QUEUE API\n%%-------------------------------------------------------------------\n\n-spec commit_started(Queues :: queues(), Priority :: wa_raft_acceptor:priority()) -> ok | apply_queue_full | commit_queue_full.\ncommit_started(#queues{table = Table, counters = Counters} = Queues, Priority) ->\n    case commit_queue_full(Queues, Priority) of\n        true ->\n            ?RAFT_COUNT(Table, {'acceptor.error.commit_queue_full', Priority}),\n            commit_queue_full;\n        false ->\n            case apply_queue_full(Queues) of\n                true ->\n                    apply_queue_full;\n                false ->\n                    PendingCommits =\n                        case Priority of\n                            high ->\n                                atomics:add_get(Counters, ?RAFT_HIGH_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 1);\n                            low ->\n                                atomics:add_get(Counters, ?RAFT_LOW_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 1)\n                        end,\n                    ?RAFT_GATHER(Table, {'acceptor.commit.request.pending', Priority}, PendingCommits),\n                    ok\n            end\n    end.\n\n\n-spec commit_cancelled(Queues :: queues(), From :: gen_server:from(), Reason :: wa_raft_acceptor:commit_error() | undefined, Priority :: wa_raft_acceptor:priority()) -> ok.\ncommit_cancelled(#queues{counters = Counters}, From, Reason, Priority) ->\n    case Priority of\n        high ->\n            atomics:sub(Counters, ?RAFT_HIGH_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 1);\n        low ->\n            atomics:sub(Counters, ?RAFT_LOW_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 1)\n    end,\n    Reason =/= undefined andalso gen_server:reply(From, Reason),\n    ok.\n\n-spec commit_completed(Queues :: queues(), From :: gen_server:from(), Reply :: term(), Priority :: wa_raft_acceptor:priority()) -> ok.\ncommit_completed(#queues{counters = Counters}, From, Reply, Priority) ->\n    case Priority of\n        high ->\n            atomics:sub(Counters, ?RAFT_HIGH_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 1);\n        low ->\n            atomics:sub(Counters, ?RAFT_LOW_PRIORITY_COMMIT_QUEUE_SIZE_COUNTER, 1)\n    end,\n    gen_server:reply(From, Reply),\n    ok.\n\n%%-------------------------------------------------------------------\n%% PENDING READ QUEUE API\n%%-------------------------------------------------------------------\n\n% Inspects the read and apply queues to check if a strong read is allowed\n% to be submitted to the RAFT server currently. If so, then returns 'ok'\n% and increments the read counter. Inspecting the queues and actually\n% adding the read request to the table are done in two stages for reads\n% because the acceptor does not have enough information to add the read\n% to the ETS table directly.\n-spec reserve_read(Queues :: queues()) -> ok | read_queue_full | apply_queue_full.\nreserve_read(#queues{application = App, table = Table, counters = Counters}) ->\n    PendingReads = atomics:get(Counters, ?RAFT_READ_QUEUE_SIZE_COUNTER),\n    case PendingReads >= ?RAFT_MAX_PENDING_READS(App, Table) of\n        true -> read_queue_full;\n        false ->\n            case atomics:get(Counters, ?RAFT_APPLY_QUEUE_SIZE_COUNTER) >= ?RAFT_MAX_PENDING_APPLIES(App, Table) of\n                true -> apply_queue_full;\n                false ->\n                    ?RAFT_GATHER(Table, 'acceptor.strong_read.request.pending', PendingReads + 1),\n                    atomics:add(Counters, ?RAFT_READ_QUEUE_SIZE_COUNTER, 1),\n                    ok\n            end\n    end.\n\n% Called from the RAFT server once it knows the proper ReadIndex for the\n% read request to add the read request to the reads table for storage\n% to handle upon applying.\n-spec submit_read(Queues :: queues(), wa_raft_log:log_index(), From :: gen_server:from(), Command :: wa_raft_acceptor:command()) -> ok.\nsubmit_read(#queues{reads = Reads}, ReadIndex, From, Command) ->\n    ets:insert(Reads, {{ReadIndex, make_ref()}, From, Command}),\n    ok.\n\n-spec query_reads(Queues :: queues(), wa_raft_log:log_index() | infinity) -> [{{wa_raft_log:log_index(), reference()}, wa_raft_acceptor:command()}].\nquery_reads(#queues{reads = Reads}, MaxLogIndex) ->\n    MatchSpec = ets:fun2ms(\n        fun({{LogIndex, Reference}, _, Command}) when LogIndex =< MaxLogIndex ->\n            {{LogIndex, Reference}, Command}\n        end\n    ),\n    ets:select(Reads, MatchSpec).\n\n-spec fulfill_read(Queues :: queues(), {wa_raft_log:log_index(), reference()}, dynamic()) -> ok | not_found.\nfulfill_read(#queues{counters = Counters, reads = Reads}, Reference, Reply) ->\n    case ets:take(Reads, Reference) of\n        [{Reference, From, _}] ->\n            atomics:sub(Counters, ?RAFT_READ_QUEUE_SIZE_COUNTER, 1),\n            gen_server:reply(From, Reply);\n        [] ->\n            not_found\n    end.\n\n% Complete a read that was reserved by the RAFT acceptor but was rejected\n% before it could be added to the read queue and so has no reference.\n-spec fulfill_incomplete_read(Queues :: queues(), gen_server:from(), wa_raft_acceptor:read_error()) -> ok.\nfulfill_incomplete_read(#queues{counters = Counters}, From, Reply) ->\n    atomics:sub(Counters, ?RAFT_READ_QUEUE_SIZE_COUNTER, 1),\n    gen_server:reply(From, Reply).\n\n% Fulfill a pending reads with an error that indicates that the read was not completed.\n-spec fulfill_all_reads(Queues :: queues(), wa_raft_acceptor:read_error()) -> ok.\nfulfill_all_reads(#queues{counters = Counters, reads = Reads}, Reply) ->\n    lists:foreach(\n        fun ({Reference, _, _}) ->\n            case ets:take(Reads, Reference) of\n                [{Reference, From, _}] ->\n                    atomics:sub(Counters, ?RAFT_READ_QUEUE_SIZE_COUNTER, 1),\n                    gen_server:reply(From, Reply);\n                [] ->\n                    ok\n            end\n        end, ets:tab2list(Reads)).\n\n%%-------------------------------------------------------------------\n%% APPLY QUEUE API\n%%-------------------------------------------------------------------\n\n-spec reserve_apply(Queues :: queues(), non_neg_integer()) -> ok.\nreserve_apply(#queues{counters = Counters}, Size) ->\n    atomics:add(Counters, ?RAFT_APPLY_QUEUE_SIZE_COUNTER, 1),\n    atomics:add(Counters, ?RAFT_APPLY_QUEUE_BYTE_SIZE_COUNTER, Size).\n\n-spec fulfill_apply(Queues :: queues(), non_neg_integer()) -> ok.\nfulfill_apply(#queues{counters = Counters}, Size) ->\n    atomics:sub(Counters, ?RAFT_APPLY_QUEUE_SIZE_COUNTER, 1),\n    atomics:sub(Counters, ?RAFT_APPLY_QUEUE_BYTE_SIZE_COUNTER, Size).\n\n%%-------------------------------------------------------------------\n%% OTP SUPERVISION\n%%-------------------------------------------------------------------\n\n-spec child_spec(Options :: #raft_options{}) -> supervisor:child_spec().\nchild_spec(Options) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Options]},\n        restart => transient,\n        shutdown => 1000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Options :: #raft_options{}) -> gen_server:start_ret().\nstart_link(#raft_options{queue_name = Name} = Options) ->\n    gen_server:start_link({local, Name}, ?MODULE, Options, []).\n\n%%-------------------------------------------------------------------\n%% QUEUE SERVER CALLBACKS\n%%-------------------------------------------------------------------\n\n-spec init(Options :: #raft_options{}) -> {ok, #state{}}.\ninit(\n    #raft_options{\n        table = Table,\n        partition = Partition,\n        queue_name = Name,\n        queue_counters = Counters,\n        queue_reads = ReadsName\n    }\n) ->\n    process_flag(trap_exit, true),\n\n    ?RAFT_LOG_NOTICE(\n        \"Queue[~p] starting for partition ~0p/~0p with read queue ~0p\",\n        [Name, Table, Partition, ReadsName]\n    ),\n\n    % The queue process is the first process in the supervision for a single\n    % RAFT partition. The supervisor is configured to restart all processes if\n    % even a single process fails. Since the queue process is starting up, all\n    % queues tracked should be empty so reset all counters.\n    [atomics:put(Counters, Index, 0) || Index <- lists:seq(1, ?RAFT_NUMBER_OF_QUEUE_SIZE_COUNTERS)],\n\n    % Create ETS table for pending reads.\n    ets:new(ReadsName, [ordered_set | ?RAFT_QUEUE_TABLE_OPTIONS]),\n\n    {ok, #state{name = Name}}.\n\n-spec handle_call(Request :: term(), From :: gen_server:from(), State :: #state{}) -> {noreply, #state{}}.\nhandle_call(Request, From, #state{name = Name} = State) ->\n    ?RAFT_LOG_NOTICE(\"Queue[~p] got unexpected request ~0P from ~0p\", [Name, Request, 100, From]),\n    {noreply, State}.\n\n-spec handle_cast(Request :: term(), State :: #state{}) -> {noreply, #state{}}.\nhandle_cast(Request, #state{name = Name} = State) ->\n    ?RAFT_LOG_NOTICE(\"Queue[~p] got unexpected call ~0P\", [Name, Request, 100]),\n    {noreply, State}.\n\n-spec terminate(Reason :: dynamic(), State :: #state{}) -> ok.\nterminate(Reason, #state{name = Name}) ->\n    ?RAFT_LOG_NOTICE(\"Queue[~p] terminating due to ~0P\", [Name, Reason, 100]).\n"
  },
  {
    "path": "src/wa_raft_server.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module implements RPC of raft consensus protocol. See raft spec\n%%% on https://raft.github.io/. A wa_raft instance is a participant in\n%%% a consensus group. Each participant plays a certain role (follower,\n%%% leader or candidate). The mission of a consensus group is to\n%%% implement a replicated state machine in a distributed cluster.\n\n-module(wa_raft_server).\n-compile(warn_missing_spec_all).\n-behaviour(gen_statem).\n-compile({inline, [require_valid_state/1]}).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - OTP Supervision\n%%------------------------------------------------------------------------------\n\n-export([\n    child_spec/1,\n    start_link/1\n]).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Public APIs - RAFT Cluster Configuration\n%%------------------------------------------------------------------------------\n\n-export([\n    latest_config_version/0\n]).\n\n%% Inspection of cluster configuration\n-export([\n    get_config_version/1,\n    get_config_participants/1,\n    get_config_members/1,\n    get_config_full_members/1,\n    get_config_witness_members/1,\n    get_config_witnesses/1,\n    is_data_replica/2,\n    is_witness/2\n]).\n\n%% Creation and modification of cluster configuration\n-export([\n    make_config/0,\n    make_config/1,\n    make_config/2,\n    make_config/3,\n    normalize_config/1\n]).\n\n% Stubbing log entries for witnesses\n-export([\n    stub_entries_for_witness/1\n]).\n\n%% Modification of cluster configuration\n-export([\n    set_config_members/2,\n    set_config_members/3,\n    set_config_members/4\n]).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Public APIs\n%%------------------------------------------------------------------------------\n\n-export([\n    get_current_config/1\n]).\n\n-export([\n    status/1,\n    status/2,\n    membership/1\n]).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Internal APIs - Local Options\n%%------------------------------------------------------------------------------\n\n-export([\n    default_name/2,\n    registered_name/2\n]).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Internal APIs - RPC Handling\n%%------------------------------------------------------------------------------\n\n-export([\n    make_rpc/3,\n    parse_rpc/2\n]).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Internal APIs - Commands\n%%------------------------------------------------------------------------------\n\n-export([\n    commit/4,\n    read/2,\n    snapshot_available/3,\n    adjust_config/2,\n    adjust_config/3,\n    adjust_config/4,\n    adjust_membership/3,\n    adjust_membership/4,\n    refresh_config/1,\n    trigger_election/1,\n    trigger_election/2,\n    promote/2,\n    promote/3,\n    resign/1,\n    handover/1,\n    handover/2,\n    handover_candidates/1,\n    is_peer_ready/2,\n    disable/2,\n    enable/1,\n    bootstrap/4,\n    notify_complete/1\n]).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation\n%%------------------------------------------------------------------------------\n\n%% General callbacks\n-export([\n    init/1,\n    callback_mode/0,\n    terminate/3\n]).\n\n%% State-specific callbacks\n-export([\n    stalled/3,\n    leader/3,\n    follower/3,\n    candidate/3,\n    disabled/3,\n    witness/3\n]).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Test Exports\n%%------------------------------------------------------------------------------\n\n-ifdef(TEST).\n-export([\n    compute_quorum/3,\n    config/1,\n    max_index_to_apply/3,\n    leader_adjust_config/2\n]).\n-endif.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Public Types\n%%------------------------------------------------------------------------------\n\n-export_type([\n    state/0,\n    config/0,\n    config_all/0,\n    membership/0,\n    status/0,\n    config_action/0\n]).\n\n%%------------------------------------------------------------------------------\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_rpc.hrl\").\n\n%%------------------------------------------------------------------------------\n\n%% Section 5.2. Randomized election timeout for fast election and to avoid split votes\n-define(ELECTION_TIMEOUT(State), {state_timeout, random_election_timeout(State), election}).\n\n%% Timeout in milliseconds before the next heartbeat is to be sent by a RAFT leader with no pending log entries\n-define(HEARTBEAT_TIMEOUT(State),    {state_timeout, ?RAFT_HEARTBEAT_INTERVAL(State#raft_state.application, State#raft_state.table), heartbeat}).\n%% Timeout in milliseconds before the next heartbeat is to be sent by a RAFT leader with pending log entries\n-define(COMMIT_BATCH_TIMEOUT(State), {state_timeout, ?RAFT_COMMIT_BATCH_INTERVAL(State#raft_state.application, State#raft_state.table), batch_commit}).\n\n%%------------------------------------------------------------------------------\n\n-define(SERVER_LOG_PREFIX, \"Server[~0p, term ~0p, ~0p] \").\n-define(SERVER_LOG_FORMAT(Format), ?SERVER_LOG_PREFIX Format).\n\n-define(SERVER_LOG_ARGS(State, Data, Args), [(Data)#raft_state.name, (Data)#raft_state.current_term, require_valid_state(State) | Args]).\n\n% elp:ignore W0002 (unused_macro) - Keeping for consistency\n-define(SERVER_LOG_ERROR(Data, Format, Args), ?SERVER_LOG_ERROR(?FUNCTION_NAME, Data, Format, Args)).\n-define(SERVER_LOG_ERROR(State, Data, Format, Args), ?RAFT_LOG_ERROR(?SERVER_LOG_FORMAT(Format), ?SERVER_LOG_ARGS(State, Data, Args))).\n\n-define(SERVER_LOG_WARNING(Data, Format, Args), ?SERVER_LOG_WARNING(?FUNCTION_NAME, Data, Format, Args)).\n-define(SERVER_LOG_WARNING(State, Data, Format, Args), ?RAFT_LOG_WARNING(?SERVER_LOG_FORMAT(Format), ?SERVER_LOG_ARGS(State, Data, Args))).\n\n-define(SERVER_LOG_NOTICE(Data, Format, Args), ?SERVER_LOG_NOTICE(?FUNCTION_NAME, Data, Format, Args)).\n-define(SERVER_LOG_NOTICE(State, Data, Format, Args), ?RAFT_LOG_NOTICE(?SERVER_LOG_FORMAT(Format), ?SERVER_LOG_ARGS(State, Data, Args))).\n\n-define(SERVER_LOG_DEBUG(Data, Format, Args), ?SERVER_LOG_DEBUG(?FUNCTION_NAME, Data, Format, Args)).\n-define(SERVER_LOG_DEBUG(State, Data, Format, Args), ?RAFT_LOG_DEBUG(?SERVER_LOG_FORMAT(Format), ?SERVER_LOG_ARGS(State, Data, Args))).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Public Types\n%%------------------------------------------------------------------------------\n\n-type state() ::\n    stalled |\n    leader |\n    follower |\n    candidate |\n    disabled |\n    witness.\n\n-type term_or_offset() :: wa_raft_log:log_term() | current | next | {next, Offset :: pos_integer()}.\n\n-type peer() :: {Name :: atom(), Node :: node()}.\n-type membership() :: [peer()].\n\n-type config() ::\n    #{\n        version := 1,\n        participants := membership(),\n        membership := membership(),\n        witness := membership()\n    }.\n-type config_all() :: config_v1_all().\n\n-type config_v1_all() ::\n    #{\n        version := 1,\n        participants => membership(),\n        membership => membership(),\n        witness => membership()\n    }.\n\n-type status() :: [status_element()].\n-type status_element() ::\n      {state, state()}\n    | {id, atom()}\n    | {peers, [{atom(), {node(), atom()}}]}\n    | {partition, wa_raft:partition()}\n    | {partition_path, file:filename_all()}\n    | {current_term, wa_raft_log:log_term()}\n    | {voted_for, node()}\n    | {commit_index, wa_raft_log:log_index()}\n    | {last_applied, wa_raft_log:log_index()}\n    | {leader_id, node()}\n    | {pending_high, non_neg_integer()}\n    | {pending_low, non_neg_integer()}\n    | {pending_read, boolean()}\n    | {queued, non_neg_integer()}\n    | {next_indices, #{node() => wa_raft_log:log_index()}}\n    | {match_indices, #{node() => wa_raft_log:log_index()}}\n    | {log_module, module()}\n    | {log_first, wa_raft_log:log_index()}\n    | {log_last, wa_raft_log:log_index()}\n    | {votes, #{node() => boolean()}}\n    | {inflight_applies, non_neg_integer()}\n    | {disable_reason, string()}\n    | {witness, boolean()}\n    | {config, config()}\n    | {config_index, wa_raft_log:log_index()}.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Private Types\n%%------------------------------------------------------------------------------\n\n-type event() :: rpc() | remote(normalized_procedure()) | command() | internal_event() | timeout_type().\n\n-type rpc() :: rpc_named() | legacy_rpc().\n-type legacy_rpc() :: ?LEGACY_RAFT_RPC(atom(), wa_raft_log:log_term(), node(), undefined | tuple()).\n-type rpc_named() :: ?RAFT_NAMED_RPC(atom(), wa_raft_log:log_term(), atom(), node(), undefined | tuple()).\n\n-type command() :: commit_command() | read_command() | current_config_command() | status_command() |\n                   trigger_election_command() | promote_command() | resign_command() | adjust_config_command() |\n                   snapshot_available_command() | handover_candidates_command() | handover_command() |\n                   enable_command() | disable_command() | bootstrap_command() | notify_complete_command() |\n                   is_peer_ready_command().\n\n-type commit_command()              :: ?COMMIT_COMMAND(gen_server:from(), wa_raft_acceptor:op(), wa_raft_acceptor:priority()).\n-type read_command()                :: ?READ_COMMAND(wa_raft_acceptor:read_op()).\n-type current_config_command()      :: ?CURRENT_CONFIG_COMMAND.\n-type status_command()              :: ?STATUS_COMMAND.\n-type trigger_election_command()    :: ?TRIGGER_ELECTION_COMMAND(term_or_offset()).\n-type promote_command()             :: ?PROMOTE_COMMAND(term_or_offset(), boolean()).\n-type resign_command()              :: ?RESIGN_COMMAND.\n-type adjust_config_command()       :: ?ADJUST_CONFIG_COMMAND(gen_server:from() | undefined, config_action(), wa_raft_log:log_index() | undefined).\n-type snapshot_available_command()  :: ?SNAPSHOT_AVAILABLE_COMMAND(string(), wa_raft_log:log_pos()).\n-type handover_candidates_command() :: ?HANDOVER_CANDIDATES_COMMAND.\n-type handover_command()            :: ?HANDOVER_COMMAND(node()).\n-type enable_command()              :: ?ENABLE_COMMAND.\n-type disable_command()             :: ?DISABLE_COMMAND(term()).\n-type bootstrap_command()           :: ?BOOTSTRAP_COMMAND(wa_raft_log:log_pos(), config(), dynamic()).\n-type notify_complete_command()     :: ?NOTIFY_COMPLETE_COMMAND().\n-type is_peer_ready_command()       :: ?IS_PEER_READY_COMMAND(peer()).\n\n-type internal_event() :: advance_term_event() | force_election_event().\n-type advance_term_event() :: ?ADVANCE_TERM(wa_raft_log:log_term()).\n-type force_election_event() :: ?FORCE_ELECTION(wa_raft_log:log_term()).\n\n-type timeout_type() :: election | heartbeat.\n\n-type refresh_action() :: refresh.\n-type membership_action() ::\n    %% Add a new peer to the cluster as a voting member or promote an existing\n    %% non-voting participant to a voting member.\n    {add, Peer :: peer()} |\n    %% Add a new peer or existing non-voting witness participant to the\n    %% cluster as a voting witness member.\n    {add_witness, Peer :: peer()} |\n    %% Add a new peer to the cluster as a non-voting participant.\n    {add_participant, Peer :: peer()} |\n    %% Promote a non-voting participant to a voting member if the participant\n    %% is ready. A participant is ready if it would be eligible for a handover\n    %% if it were a voting member.\n    {promote_participant_if_ready, Peer :: peer()} |\n    %% Remove a voting member's membership and participation or a non-voting\n    %% participant's participation from the cluster.\n    {remove, Peer :: peer()} |\n    %% Remove a voting witness member's membership and participation from the\n    %% cluster.\n    {remove_witness, Peer :: peer()} |\n    %% Demote an existing voting member to a non-voting participant or a\n    %% voting witness member to a non-voting witness participant.\n    {remove_membership, Peer :: peer()} |\n    %% Demote an existing voting member or non-voting participant to a voting\n    %% witness member or non-voting witness participant.\n    {demote_to_witness, Peer :: peer()}.\n\n-type config_action() :: refresh_action() | membership_action().\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - OTP Supervision\n%%------------------------------------------------------------------------------\n\n-spec child_spec(Options :: #raft_options{}) -> supervisor:child_spec().\nchild_spec(Options) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Options]},\n        restart => transient,\n        shutdown => 30000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Options :: #raft_options{}) -> gen_statem:start_ret().\nstart_link(#raft_options{server_name = Name} = Options) ->\n    gen_statem:start_link({local, Name}, ?MODULE, Options, []).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Public APIs - RAFT Cluster Configuration\n%%------------------------------------------------------------------------------\n\n%% Returns the version number for the latest cluster configuration format that\n%% is supported by the current RAFT implementation. All cluster configurations\n%% returned by methods used to create or modify cluster configurations in this\n%% module will return cluster configurations of this version.\n-spec latest_config_version() -> pos_integer().\nlatest_config_version() ->\n    1.\n\n-spec get_config_version(Config :: config() | config_all()) -> pos_integer().\nget_config_version(#{version := Version}) ->\n    Version.\n\n-spec get_config_participants(Config :: config() | config_all()) -> [#raft_identity{}].\nget_config_participants(#{version := 1, participants := Participants}) ->\n    [#raft_identity{name = Name, node = Node} || {Name, Node} <- Participants];\nget_config_participants(Config) ->\n    get_config_members(Config).\n\n-spec get_config_members(Config :: config() | config_all()) -> [#raft_identity{}].\nget_config_members(#{version := 1, membership := Members}) ->\n    [#raft_identity{name = Name, node = Node} || {Name, Node} <- Members];\nget_config_members(_) ->\n    [].\n\n-spec get_config_full_members(Config :: config() | config_all()) -> [#raft_identity{}].\nget_config_full_members(#{version := 1, membership := Members, witness := Witnesses}) ->\n    [#raft_identity{name = Name, node = Node} || {Name, Node} <- Members -- Witnesses];\nget_config_full_members(Config) ->\n    get_config_members(Config).\n\n-spec get_config_witness_members(Config :: config() | config_all()) -> [#raft_identity{}].\nget_config_witness_members(#{version := 1, membership := Members, witness := Witnesses}) ->\n    MembersMap = maps:from_keys(Members, []),\n    [#raft_identity{name = Name, node = Node} || {Name, Node} = Witness <- Witnesses, maps:is_key(Witness, MembersMap)];\nget_config_witness_members(_) ->\n    [].\n\n-spec get_config_witnesses(Config :: config() | config_all()) -> [#raft_identity{}].\nget_config_witnesses(#{version := 1, witness := Witnesses}) ->\n    [#raft_identity{name = Name, node = Node} || {Name, Node} <- Witnesses];\nget_config_witnesses(_) ->\n    [].\n\n-spec is_data_replica(Identity :: #raft_identity{}, Config :: config() | config_all()) -> boolean().\nis_data_replica(Identity, Config) ->\n    lists:member(Identity, get_config_participants(Config)) andalso not lists:member(Identity, get_config_witnesses(Config)).\n\n-spec is_witness(Identity :: #raft_identity{}, Config :: config() | config_all()) -> boolean().\nis_witness(Identity, Config) ->\n    lists:member(Identity, get_config_witnesses(Config)).\n\n%% Create a new cluster configuration with no members.\n%% Without any members, this cluster configuration should not be used as\n%% the active configuration for a RAFT cluster.\n-spec make_config() -> config().\nmake_config() ->\n    #{\n        version => 1,\n        participants => [],\n        membership => [],\n        witness => []\n    }.\n\n%% Create a new cluster configuration with the provided members.\n-spec make_config(Members :: [#raft_identity{}]) -> config().\nmake_config(Members) ->\n    set_config_members(Members, make_config()).\n\n%% Create a new cluster configuration with the provided members and witnesses.\n-spec make_config(Members :: [#raft_identity{}], Witnesses :: [#raft_identity{}]) -> config().\nmake_config(Members, Witnesses) ->\n    set_config_members(Members, Witnesses, make_config()).\n\n%% Create a new cluster configuration with the provided participants, members, and witnesses.\n%% Any members that are not in the participants list will be included.\n-spec make_config(Participants :: [#raft_identity{}], Members :: [#raft_identity{}], Witnesses :: [#raft_identity{}]) -> config().\nmake_config(Participants, Members, Witnesses) ->\n    set_config_members(Participants, Members, Witnesses, make_config()).\n\n%% Replace the set of members in the provided cluster configuration.\n%% After replacement, the set of participants will be equal to the provided list of members.\n%% After replacement, no witnesses will be set.\n%% Will upgrade the cluster configuration to the latest version.\n-spec set_config_members(Members :: [#raft_identity{}], ConfigAll :: config() | config_all()) -> config().\nset_config_members(Members, ConfigAll) ->\n    set_config_members(Members, [], ConfigAll).\n\n%% Replace the set of participants, members and witnesses in the provided cluster configuration.\n%% After replacement, the set of participants will be equal to the provided list of members.\n%% Will upgrade the cluster configuration to the latest version.\n-spec set_config_members(Members :: [#raft_identity{}], Witnesses :: [#raft_identity{}], ConfigAll :: config() | config_all()) -> config().\nset_config_members(Members, Witnesses, ConfigAll) ->\n    set_config_members(Members, Members, Witnesses, ConfigAll).\n\n%% Replace the set of participants, members, and witnesses in the provided cluster configuration.\n%% After replacement, the set of participants will include at least all members.\n%% Will upgrade the cluster configuration to the latest version.\n-spec set_config_members(Participants :: [#raft_identity{}], Members :: [#raft_identity{}], Witnesses :: [#raft_identity{}], ConfigAll :: config() | config_all()) -> config().\nset_config_members(Participants, Members, Witnesses, ConfigAll) ->\n    ParticipantPeers = lists:usort([{Name, Node} || #raft_identity{name = Name, node = Node} <- Participants]),\n    MemberPeers = lists:usort([{Name, Node} || #raft_identity{name = Name, node = Node} <- Members]),\n    WitnessPeers = lists:usort([{Name, Node} || #raft_identity{name = Name, node = Node} <- Witnesses]),\n    Config = normalize_config(ConfigAll),\n    Config#{\n        participants => lists:umerge(ParticipantPeers, MemberPeers),\n        membership => MemberPeers,\n        witness => WitnessPeers\n    }.\n\n%% Attempt to upgrade any configuration from an older configuration version to the\n%% latest configuration version if possible.\n-spec normalize_config(ConfigAll :: config() | config_all()) -> Config :: config().\nnormalize_config(#{version := 1} = Config) ->\n    Membership = maps:get(membership, Config, []),\n    #{\n        version => 1,\n        participants => maps:get(participants, Config, Membership),\n        membership => Membership,\n        witness => maps:get(witness, Config, [])\n    };\nnormalize_config(#{version := Version}) ->\n    % All valid configurations will contain at least their own version; however,\n    % we do not know how to handle configurations with newer versions.\n    error({unsupported_version, Version});\nnormalize_config(#{}) ->\n    error(no_version).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Public APIs\n%%------------------------------------------------------------------------------\n\n-spec get_current_config(Server :: gen_statem:server_ref()) -> config().\nget_current_config(Server) ->\n    gen_statem:call(Server, ?CURRENT_CONFIG_COMMAND, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec status(Server :: gen_statem:server_ref()) -> status().\nstatus(Server) ->\n    gen_statem:call(Server, ?STATUS_COMMAND, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec status\n    (Server :: gen_statem:server_ref(), Key :: atom()) -> Value :: dynamic();\n    (Server :: gen_statem:server_ref(), Keys :: [atom()]) -> Value :: [dynamic()].\nstatus(Server, Key) when is_atom(Key) ->\n    hd(status(Server, [Key]));\nstatus(Server, Keys) when is_list(Keys) ->\n    case status(Server) of\n        [_|_] = Status ->\n            [proplists:get_value(Key, Status, undefined) || Key <- Keys];\n        _ ->\n            lists:duplicate(length(Keys), undefined)\n    end.\n\n-spec membership(Service :: gen_statem:server_ref()) -> undefined | [#raft_identity{}].\nmembership(Service) ->\n    case proplists:get_value(config, status(Service), undefined) of\n        undefined -> undefined;\n        Config    -> get_config_members(Config)\n    end.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Internal APIs - Local Options\n%%------------------------------------------------------------------------------\n\n%% Get the default name for the RAFT server associated with the provided\n%% RAFT partition.\n-spec default_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom, one per table/partition at startup\n    binary_to_atom(<<\"raft_server_\", (atom_to_binary(Table))/binary, \"_\", (integer_to_binary(Partition))/binary>>).\n\n%% Get the registered name for the RAFT server associated with the provided\n%% RAFT partition or the default name if no registration exists.\n-spec registered_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\nregistered_name(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> default_name(Table, Partition);\n        Options   -> Options#raft_options.server_name\n    end.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Internal APIs - RPC Handling\n%%------------------------------------------------------------------------------\n\n-spec make_rpc(Self :: #raft_identity{}, Term :: wa_raft_log:log_term(), Procedure :: normalized_procedure()) -> rpc().\nmake_rpc(#raft_identity{name = Name, node = Node}, Term, ?PROCEDURE(Procedure, Payload)) ->\n    % For compatibility with legacy versions that expect RPCs sent with no arguments to have payload 'undefined' instead of {}.\n    PayloadOrUndefined = case Payload of\n        {} -> undefined;\n        _  -> Payload\n    end,\n    ?RAFT_NAMED_RPC(Procedure, Term, Name, Node, PayloadOrUndefined).\n\n-spec parse_rpc(Self :: #raft_identity{}, RPC :: rpc()) -> {Term :: wa_raft_log:log_term(), Sender :: #raft_identity{}, Procedure :: procedure()}.\nparse_rpc(_, ?RAFT_NAMED_RPC(Key, Term, SenderName, SenderNode, PayloadOrUndefined)) ->\n    Payload = case PayloadOrUndefined of\n        undefined -> {};\n        _         -> PayloadOrUndefined\n    end,\n    #{Key := ?PROCEDURE(Procedure, Defaults)} = protocol(),\n    {Term, #raft_identity{name = SenderName, node = SenderNode}, ?PROCEDURE(Procedure, defaultize_payload(Defaults, Payload))};\nparse_rpc(#raft_identity{name = Name} = Self, ?LEGACY_RAFT_RPC(Procedure, Term, SenderId, Payload)) ->\n    parse_rpc(Self, ?RAFT_NAMED_RPC(Procedure, Term, Name, SenderId, Payload)).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - Internal APIs - Commands\n%%------------------------------------------------------------------------------\n\n-spec commit(\n    Server :: gen_statem:server_ref(),\n    From :: gen_server:from(),\n    Op :: wa_raft_acceptor:op(),\n    Priority :: wa_raft_acceptor:priority()\n) -> ok.\ncommit(Server, From, Op, Priority) ->\n    gen_statem:cast(Server, ?COMMIT_COMMAND(From, Op, Priority)).\n\n-spec read(\n    Server :: gen_statem:server_ref(),\n    Op :: wa_raft_acceptor:read_op()\n) -> ok.\nread(Server, Op) ->\n    gen_statem:cast(Server, ?READ_COMMAND(Op)).\n\n-spec snapshot_available(\n    Server :: gen_statem:server_ref(),\n    Root :: file:filename(),\n    Position :: wa_raft_log:log_pos()\n) -> ok | {error, Reason :: term()}.\nsnapshot_available(Server, Root, Position) ->\n    % Use the storage call timeout because this command requires the RAFT\n    % server to make a potentially expensive call against the RAFT storage\n    % server to complete.\n    gen_statem:call(Server, ?SNAPSHOT_AVAILABLE_COMMAND(Root, Position), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec adjust_config(Server :: gen_statem:server_ref(), Action :: config_action()) ->\n    {ok, Position :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\nadjust_config(Server, Action) ->\n    adjust_config(Server, Action, undefined).\n\n-spec adjust_config(\n    Server :: gen_statem:server_ref(),\n    Action :: config_action(),\n    Index :: wa_raft_log:log_index() | undefined\n) -> {ok, Position :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\nadjust_config(Server, Action, Index) ->\n    gen_statem:call(Server, ?ADJUST_CONFIG_COMMAND(undefined, Action, Index), ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec adjust_config(\n    Server :: gen_statem:server_ref(),\n    From :: gen_server:from(),\n    Action :: config_action(),\n    Index :: wa_raft_log:log_index() | undefined\n) -> ok.\nadjust_config(Server, From, Action, Index) ->\n    gen_statem:cast(Server, ?ADJUST_CONFIG_COMMAND(From, Action, Index)).\n\n-spec adjust_membership(\n    Server :: gen_statem:server_ref(),\n    Action :: add | add_witness | remove | remove_witness,\n    Peer :: peer()\n) -> {ok, Position :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\nadjust_membership(Server, Action, Peer) ->\n    adjust_config(Server, {Action, Peer}).\n\n-spec adjust_membership(\n    Server :: gen_statem:server_ref(),\n    Action :: add | add_witness | remove | remove_witness,\n    Peer :: peer(),\n    ConfigIndex :: wa_raft_log:log_index() | undefined\n) -> {ok, Position :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\nadjust_membership(Server, Action, Peer, ConfigIndex) ->\n    adjust_config(Server, {Action, Peer}, ConfigIndex).\n\n-spec refresh_config(Server :: gen_statem:server_ref()) ->\n    {ok, Position :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\nrefresh_config(Server) ->\n    adjust_config(Server, refresh).\n\n%% Request the specified RAFT server to start an election in the next term.\n-spec trigger_election(Server :: gen_statem:server_ref()) -> ok | {error, Reason :: term()}.\ntrigger_election(Server) ->\n    trigger_election(Server, current).\n\n%% Request the specified RAFT server to trigger a new election in the term *after* the specified term.\n-spec trigger_election(Server :: gen_statem:server_ref(), Term :: term_or_offset()) -> ok | {error, Reason :: term()}.\ntrigger_election(Server, Term) ->\n    gen_statem:call(Server, ?TRIGGER_ELECTION_COMMAND(Term), ?RAFT_RPC_CALL_TIMEOUT()).\n\n%% Request the specified RAFT server to promote itself to leader of the specified term.\n-spec promote(Server :: gen_statem:server_ref(), Term :: term_or_offset()) -> ok | {error, Reason :: term()}.\npromote(Server, Term) ->\n    promote(Server, Term, false).\n\n-spec promote(Server :: gen_statem:server_ref(), Term :: term_or_offset(), Force :: boolean()) -> ok | {error, Reason :: term()}.\npromote(Server, Term, Force) ->\n    gen_statem:call(Server, ?PROMOTE_COMMAND(Term, Force), ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec resign(Server :: gen_statem:server_ref()) -> ok | {error, Reason :: term()}.\nresign(Server) ->\n    gen_statem:call(Server, ?RESIGN_COMMAND, ?RAFT_RPC_CALL_TIMEOUT()).\n\n%% Instruct a RAFT leader to attempt a handover to a random handover candidate.\n-spec handover(Server :: gen_statem:server_ref()) -> ok.\nhandover(Server) ->\n    gen_statem:cast(Server, ?HANDOVER_COMMAND(undefined)).\n\n%% Instruct a RAFT leader to attempt a handover to the specified peer node.\n%% If an `undefined` peer node is specified, then handover to a random handover candidate.\n%% Returns which peer node the handover was sent to or otherwise an error.\n-spec handover(Server :: gen_statem:server_ref(), Peer :: node() | undefined) -> {ok, Peer :: node()} | {error, Reason :: term()}.\nhandover(Server, Peer) ->\n    gen_statem:call(Server, ?HANDOVER_COMMAND(Peer), ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec handover_candidates(Server :: gen_statem:server_ref()) -> {ok, Candidates :: [node()]} | {error, Reason :: term()}.\nhandover_candidates(Server) ->\n    gen_statem:call(Server, ?HANDOVER_CANDIDATES_COMMAND, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec is_peer_ready(Server :: gen_statem:server_ref(), Peer :: peer()) -> ok | {error, Reason :: term()}.\nis_peer_ready(Server, Peer) ->\n    gen_statem:call(Server, ?IS_PEER_READY_COMMAND(Peer), ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec disable(Server :: gen_statem:server_ref(), Reason :: term()) -> ok | {error, ErrorReason :: atom()}.\ndisable(Server, Reason) ->\n    gen_statem:call(Server, ?DISABLE_COMMAND(Reason), ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec enable(Server :: gen_statem:server_ref()) -> ok | {error, ErrorReason :: atom()}.\nenable(Server) ->\n    gen_statem:call(Server, ?ENABLE_COMMAND, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec bootstrap(\n    Server :: gen_statem:server_ref(),\n    Position :: wa_raft_log:log_pos(),\n    Config :: config(),\n    Data :: dynamic()\n) -> ok | {error, Reason :: term()}.\nbootstrap(Server, Position, Config, Data) ->\n    gen_statem:call(Server, ?BOOTSTRAP_COMMAND(Position, Config, Data), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec notify_complete(Server :: gen_statem:server_ref()) -> ok.\nnotify_complete(Server) ->\n    gen_statem:cast(Server, ?NOTIFY_COMPLETE_COMMAND()).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Logging\n%%------------------------------------------------------------------------------\n\n-spec require_valid_state(state()) -> state().\nrequire_valid_state(State) ->\n    State.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - General Callbacks\n%%------------------------------------------------------------------------------\n\n-spec init(Options :: #raft_options{}) -> gen_statem:init_result(state()).\ninit(\n    #raft_options{\n        application = Application,\n        table = Table,\n        partition = Partition,\n        self = Self,\n        identifier = Identifier,\n        database = PartitionPath,\n        label_module = LabelModule,\n        distribution_module = DistributionModule,\n        log_name = Log,\n        server_name = Name,\n        storage_name = Storage\n    } = Options\n) ->\n    process_flag(trap_exit, true),\n\n    ?RAFT_LOG_NOTICE(\"Server[~0p] starting with options ~0p\", [Name, Options]),\n\n    % This increases the potential overhead of sending messages to server;\n    % however, can protect the server from GC overhead\n    % and other memory-related issues (most notably when receiving log entries\n    % when undergoing a fast log catchup).\n    ?RAFT_CONFIG(raft_server_message_queue_off_heap, true) andalso\n        process_flag(message_queue_data, off_heap),\n\n    % Open storage and the log\n    {ok, Last} = wa_raft_storage:open(Storage),\n    {ok, View} = wa_raft_log:open(Log, Last),\n\n    Now = erlang:monotonic_time(millisecond),\n    State0 = #raft_state{\n        application = Application,\n        name = Name,\n        self = Self,\n        identifier = Identifier,\n        table = Table,\n        partition = Partition,\n        partition_path = PartitionPath,\n        log_view = View,\n        queues = wa_raft_queue:queues(Options),\n        label_module = LabelModule,\n        last_label = undefined,\n        distribution_module = DistributionModule,\n        storage = Storage,\n        commit_index = Last#raft_log_pos.index,\n        last_applied = Last#raft_log_pos.index,\n        current_term = Last#raft_log_pos.term,\n        state_start_ts = Now\n    },\n\n    State1 = load_config(State0),\n    rand:seed(exsp, {erlang:monotonic_time(), erlang:time_offset(), erlang:unique_integer()}),\n    % TODO T246543655 When we have proper error handling for data corruption\n    %                 vs. stalled server then handle {error, Reason} type\n    %                 returns from load_state.\n    State2 = case wa_raft_durable_state:load(State1) of\n        {ok, NewState} -> NewState;\n        _              -> State1\n    end,\n    true = wa_raft_info:set_current_term_and_leader(Table, Partition, State2#raft_state.current_term, undefined),\n    % 1. Begin as disabled if a disable reason is set\n    % 2. Begin as stalled if there is no data\n    % 3. Begin as witness if configured\n    % 4. Begin as follower otherwise\n    case {State2#raft_state.last_applied, State2#raft_state.disable_reason} of\n        {0, undefined}    -> {ok, stalled, State2};\n        {_, undefined}    -> {ok, follower_or_witness_state(State2), State2};\n        {_, _}            -> {ok, disabled, State2}\n    end.\n\n-spec callback_mode() -> gen_statem:callback_mode_result().\ncallback_mode() ->\n    [state_functions, state_enter].\n\n-spec terminate(Reason :: term(), State :: state(), Data :: #raft_state{}) -> ok.\nterminate(Reason, State, #raft_state{name = Name, table = Table, partition = Partition, handover = Handover} = Data0) ->\n    ?SERVER_LOG_NOTICE(State, Data0, \"terminating due to ~0P\", [Reason, 20]),\n    case Handover of\n        {Peer, _, _} -> cancel_pending_and_queued({error, {notify_redirect, Peer}}, Data0);\n        undefined    -> cancel_pending_and_queued({error, not_leader}, Data0)\n    end,\n    wa_raft_durable_state:sync(Data0),\n    wa_raft_info:delete_state(Table, Partition),\n    wa_raft_info:set_live(Table, Partition, false),\n    wa_raft_info:set_stale(Table, Partition, true),\n    wa_raft_info:set_message_queue_length(Name, 0),\n    ok.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Procedure Call Marshalling\n%%------------------------------------------------------------------------------\n\n%% A macro that destructures the identity record indicating that the\n%% relevant procedure should be refactored to treat identities\n%% opaquely.\n-define(IDENTITY_REQUIRES_MIGRATION(Name, Node), #raft_identity{name = Name, node = Node}).\n\n-type remote(Call) :: ?REMOTE(#raft_identity{}, Call).\n-type procedure()  :: ?PROCEDURE(atom(), tuple()).\n\n-type normalized_procedure() :: append_entries() | append_entries_response() | request_vote() | vote() | handover() | handover_failed() | notify_term().\n-type append_entries()          :: ?APPEND_ENTRIES         (wa_raft_log:log_index(), wa_raft_log:log_term(), [wa_raft_log:log_entry() | binary()], wa_raft_log:log_index(), wa_raft_log:log_index()).\n-type append_entries_response() :: ?APPEND_ENTRIES_RESPONSE(wa_raft_log:log_index(), boolean(), wa_raft_log:log_index(), wa_raft_log:log_index() | undefined).\n-type request_vote()            :: ?REQUEST_VOTE           (election_type(), wa_raft_log:log_index(), wa_raft_log:log_term()).\n-type vote()                    :: ?VOTE                   (boolean()).\n-type handover()                :: ?HANDOVER               (reference(), wa_raft_log:log_index(), wa_raft_log:log_term(), [wa_raft_log:log_entry() | binary()]).\n-type handover_failed()         :: ?HANDOVER_FAILED        (reference()).\n-type notify_term()             :: ?NOTIFY_TERM            ().\n\n-type election_type() :: normal | force | allowed.\n\n-spec protocol() -> #{atom() => procedure()}.\nprotocol() ->\n    #{\n        ?APPEND_ENTRIES          => ?APPEND_ENTRIES(0, 0, [], 0, 0),\n        ?APPEND_ENTRIES_RESPONSE => ?APPEND_ENTRIES_RESPONSE(0, false, 0, undefined),\n        ?REQUEST_VOTE            => ?REQUEST_VOTE(normal, 0, 0),\n        ?VOTE                    => ?VOTE(false),\n        ?HANDOVER                => ?HANDOVER(undefined, 0, 0, []),\n        ?HANDOVER_FAILED         => ?HANDOVER_FAILED(undefined)\n    }.\n\n-spec handle_rpc(\n    Type :: gen_statem:event_type(),\n    RPC :: rpc(),\n    State :: state(),\n    Data :: #raft_state{}\n) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\nhandle_rpc(Type, ?RAFT_NAMED_RPC(Procedure, Term, SenderName, SenderNode, Payload) = Event, State, #raft_state{} = Data) ->\n    handle_rpc_impl(Type, Event, Procedure, Term, #raft_identity{name = SenderName, node = SenderNode}, Payload, State, Data);\nhandle_rpc(Type, ?LEGACY_RAFT_RPC(Procedure, Term, SenderId, Payload) = Event, State, #raft_state{name = Name} = Data) ->\n    handle_rpc_impl(Type, Event, Procedure, Term, #raft_identity{name = Name, node = SenderId}, Payload, State, Data);\nhandle_rpc(_, RPC, State, #raft_state{table = Table} = Data) ->\n    ?RAFT_COUNT(Table, {'rpc.unrecognized', State}),\n    ?SERVER_LOG_NOTICE(State, Data, \"receives unknown RPC format ~0P\", [RPC, 20]),\n    keep_state_and_data.\n\n-spec handle_rpc_impl(\n    Type :: gen_statem:event_type(),\n    Event :: rpc(),\n    Key :: atom(),\n    Term :: wa_raft_log:log_term(),\n    Sender :: #raft_identity{},\n    Payload :: undefined | tuple(),\n    State :: state(),\n    Data :: #raft_state{}\n) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\n%% [Protocol] Undefined payload should be treated as an empty tuple\nhandle_rpc_impl(Type, Event, Key, Term, Sender, undefined, State, Data) ->\n    handle_rpc_impl(Type, Event, Key, Term, Sender, {}, State, Data);\n%% [General Rules] Discard any incoming RPCs with a term older than the current term\nhandle_rpc_impl(_, _, Key, Term, Sender, _, State, #raft_state{current_term = CurrentTerm} = Data) when Term < CurrentTerm ->\n    ?SERVER_LOG_NOTICE(State, Data, \"dropping stale ~0p from ~0p with old term ~0p.\", [Key, Sender, Term]),\n    State =/= disabled andalso send_rpc(Sender, ?NOTIFY_TERM(), Data),\n    keep_state_and_data;\n%% [RequestVote RPC] RAFT servers should ignore vote requests with reason `normal`\n%%                   if it knows about a currently active leader even if the vote\n%%                   request has a newer term. A leader is only active if it is\n%%                   replicating to peers so we check if we have recently received\n%%                   a heartbeat. (4.2.3)\nhandle_rpc_impl(Type, Event, ?REQUEST_VOTE, Term, Sender, Payload, State,\n                #raft_state{application = App, table = Table, leader_heartbeat_ts = LeaderHeartbeatTs} = Data) when is_tuple(Payload), element(1, Payload) =:= normal ->\n    AllowedDelay = ?RAFT_ELECTION_TIMEOUT_MIN(App, Table) div 2,\n    Delay = case LeaderHeartbeatTs of\n        undefined -> infinity;\n        _         -> erlang:monotonic_time(millisecond) - LeaderHeartbeatTs\n    end,\n    case Delay > AllowedDelay of\n        true ->\n            % We have not gotten a heartbeat from the leader recently so allow this vote request\n            % to go through by reraising it with the special 'allowed' election type.\n            handle_rpc_impl(Type, Event, ?REQUEST_VOTE, Term, Sender, setelement(1, Payload, allowed), State, Data);\n        false ->\n            % We have gotten a heartbeat recently so drop this vote request.\n            % Log this at debug level because we may end up with alot of these when we have\n            % removed a server from the cluster but not yet shut it down.\n            ?RAFT_COUNT(Table, 'server.request_vote.drop'),\n            ?SERVER_LOG_DEBUG(State, Data, \"dropping normal vote request from ~p because leader was still active ~p ms ago (allowed ~p ms).\",\n                [Sender, Delay, AllowedDelay]),\n            keep_state_and_data\n    end;\n%% [General Rules] Advance to the newer term and reset state when seeing a newer term in an incoming RPC\nhandle_rpc_impl(Type, Event, _, Term, _, _, _, #raft_state{current_term = CurrentTerm}) when Term > CurrentTerm ->\n    {keep_state_and_data, [{next_event, internal, ?ADVANCE_TERM(Term)}, {next_event, Type, Event}]};\n%% [NotifyTerm RPC] Drop NotifyTerm RPCs with matching term\nhandle_rpc_impl(_, _, ?NOTIFY_TERM, _, _, _, _, #raft_state{}) ->\n    keep_state_and_data;\n%% [Protocol] Convert any valid remote procedure call to the appropriate local procedure call.\nhandle_rpc_impl(Type, _, Key, _, Sender, Payload, State, #raft_state{table = Table} = Data) when is_tuple(Payload) ->\n    case protocol() of\n        #{Key := ?PROCEDURE(Procedure, Defaults)} ->\n            handle_procedure(Type, ?REMOTE(Sender, ?PROCEDURE(Procedure, defaultize_payload(Defaults, Payload))), State, Data);\n        #{} ->\n            ?RAFT_COUNT(Table, {'rpc.unknown', State}),\n            ?SERVER_LOG_DEBUG(State, Data, \"receives unknown RPC type ~0p with payload ~0P\", [Key, Payload, 25]),\n            keep_state_and_data\n    end.\n\n-spec handle_procedure(\n    Type :: gen_statem:event_type(),\n    ProcedureCall :: remote(procedure()),\n    State :: state(),\n    Data :: #raft_state{}\n) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\n%% [AppendEntries RPC] If we haven't discovered leader for this term, record it\nhandle_procedure(Type, ?REMOTE(Sender, ?APPEND_ENTRIES(_, _, _, _, _)) = Procedure, State, #raft_state{leader_id = undefined} = Data) ->\n    {keep_state, set_leader(State, Sender, Data), {next_event, Type, Procedure}};\n%% [Handover][Handover RPC] If we haven't discovered leader for this term, record it\nhandle_procedure(Type, ?REMOTE(Sender, ?HANDOVER(_, _, _, _)) = Procedure, State, #raft_state{leader_id = undefined} = Data) ->\n    {keep_state, set_leader(State, Sender, Data), {next_event, Type, Procedure}};\nhandle_procedure(Type, Procedure, _, #raft_state{}) ->\n    {keep_state_and_data, {next_event, Type, Procedure}}.\n\n-spec defaultize_payload(Defaults :: tuple(), Payload :: tuple()) -> tuple().\ndefaultize_payload(Defaults, Payload) ->\n    defaultize_payload(Defaults, Payload, tuple_size(Defaults), tuple_size(Payload)).\n\n-spec defaultize_payload(tuple(), tuple(), non_neg_integer(), non_neg_integer()) -> tuple().\ndefaultize_payload(_, Payload, N, N) ->\n    Payload;\ndefaultize_payload(Defaults, Payload, N, M) when N > M ->\n    defaultize_payload(Defaults, erlang:insert_element(M + 1, Payload, element(M + 1, Defaults)), N, M + 1);\ndefaultize_payload(Defaults, Payload, N, M) when N < M ->\n    defaultize_payload(Defaults, erlang:delete_element(M, Payload), N, M - 1).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Stalled State\n%%------------------------------------------------------------------------------\n%% The stalled state is an extension to the RAFT protocol designed to handle\n%% situations in which a replica of the FSM is lost or replaced within a RAFT\n%% cluster without being removed from the cluster membership. As the replica of\n%% the FSM stored before the machine was lost or replaced could have been a\n%% critical member of a quorum, it is important to ensure that the replacement\n%% does not support a different result for any quorums before it receives a\n%% fresh copy of the FSM state and log that is guaranteed to reflect any\n%% quorums that the machine supported before it was lost or replaced.\n%%\n%% This is achieved by preventing a stalled node from participating in quorum\n%% for both log entries and election. A leader of the cluster must provide a\n%% fresh copy of its FSM state before the stalled node can return to normal\n%% operation.\n%%------------------------------------------------------------------------------\n\n-spec stalled\n    (\n        Type :: enter,\n        PreviousStateName :: state(),\n        Data :: #raft_state{}\n    ) -> gen_statem:state_enter_result(state(), #raft_state{});\n    (\n        Type :: gen_statem:event_type(),\n        Event :: event(),\n        Data :: #raft_state{}\n    ) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\nstalled(enter, PreviousStateName, #raft_state{table = Table} = State) ->\n    ?RAFT_COUNT(Table, 'stalled.enter'),\n    ?SERVER_LOG_NOTICE(State, \"becomes stalled from state ~0p.\", [PreviousStateName]),\n    {keep_state, enter_state(?FUNCTION_NAME, State)};\n\n%% [Internal] Advance to newer term when requested\nstalled(internal, ?ADVANCE_TERM(NewTerm), #raft_state{table = Table, current_term = CurrentTerm} = State) when NewTerm > CurrentTerm ->\n    ?RAFT_COUNT(Table, 'stalled.advance_term'),\n    ?SERVER_LOG_NOTICE(State, \"advancing to new term ~0p.\", [NewTerm]),\n    {repeat_state, advance_term(?FUNCTION_NAME, NewTerm, undefined, State)};\n\n%% [Protocol] Parse any RPCs in network formats\nstalled(Type, Event, #raft_state{} = State) when is_tuple(Event), element(1, Event) =:= rpc ->\n    handle_rpc(Type, Event, ?FUNCTION_NAME, State);\n\n%% [AppendEntries RPC] Stalled nodes always discard heartbeats\nstalled(_, ?REMOTE(Sender, ?APPEND_ENTRIES(PrevLogIndex, _, _, _, _)), #raft_state{} = State) ->\n    NewState = State#raft_state{leader_heartbeat_ts = erlang:monotonic_time(millisecond)},\n    send_rpc(Sender, ?APPEND_ENTRIES_RESPONSE(PrevLogIndex, false, 0, 0), NewState),\n    {keep_state, NewState};\n\nstalled({call, From}, ?TRIGGER_ELECTION_COMMAND(_), #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"cannot start an election.\", []),\n    {keep_state_and_data, {reply, From, {error, invalid_state}}};\n\nstalled({call, From}, ?PROMOTE_COMMAND(_, _), #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"cannot be promoted to leader.\", []),\n    {keep_state_and_data, {reply, From, {error, invalid_state}}};\n\nstalled(\n    {call, From},\n    ?BOOTSTRAP_COMMAND(#raft_log_pos{index = Index, term = Term} = Position, Config, Data),\n    #raft_state{\n        self = Self,\n        partition_path = PartitionPath,\n        storage = Storage,\n        current_term = CurrentTerm,\n        last_applied = LastApplied\n    } = State0\n) ->\n    case LastApplied =:= 0 of\n        true ->\n            ?SERVER_LOG_NOTICE(State0, \"attempting bootstrap at ~0p:~0p with config ~0p and data ~0P.\", [Index, Term, Config, Data, 30]),\n            Path = filename:join(PartitionPath, io_lib:format(\"snapshot.~0p.~0p.bootstrap.tmp\", [Index, Term])),\n            try\n                ok = wa_raft_storage:make_empty_snapshot(Storage, Path, Position, Config, Data),\n                State1 = open_snapshot(Path, Position, State0),\n                AdjustedTerm = max(1, Term),\n                case AdjustedTerm > CurrentTerm of\n                    true ->\n                        case is_single_member(Self, config(State1)) of\n                            true ->\n                                State2 = advance_term(?FUNCTION_NAME, AdjustedTerm, node(), State1),\n                                ?SERVER_LOG_NOTICE(State2, \"switching to leader as sole member after successful bootstrap.\", []),\n                                {next_state, leader, State2, {reply, From, ok}};\n                            false ->\n                                State2 = advance_term(?FUNCTION_NAME, AdjustedTerm, undefined, State1),\n                                ?SERVER_LOG_NOTICE(State2, \"switching to follower after successful bootstrap.\", []),\n                                {next_state, follower_or_witness_state(State2), State2, {reply, From, ok}}\n                        end;\n                    false ->\n                        ?SERVER_LOG_NOTICE(State1, \"switching to follower after successful bootstrap.\", []),\n                        {next_state, follower_or_witness_state(State1), State1, {reply, From, ok}}\n                end\n            catch\n                _:Reason ->\n                    ?SERVER_LOG_WARNING(State0, \"failed to bootstrap due to ~0P.\", [Reason, 20]),\n                    {keep_state_and_data, {reply, From, {error, Reason}}}\n            after\n                try file:del_dir_r(Path)\n                catch _:_ -> ok\n                end\n            end;\n        false ->\n            ?SERVER_LOG_NOTICE(State0, \"at ~0p rejecting request to bootstrap with data.\", [LastApplied]),\n            {keep_state_and_data, {reply, From, {error, rejected}}}\n    end;\n\nstalled(\n    Type,\n    ?SNAPSHOT_AVAILABLE_COMMAND(Root, #raft_log_pos{index = SnapshotIndex, term = SnapshotTerm} = SnapshotPos),\n    #raft_state{\n        current_term = CurrentTerm,\n        last_applied = LastApplied\n    } = State0\n) ->\n    case SnapshotIndex > LastApplied orelse LastApplied =:= 0 of\n        true ->\n            try\n                ?SERVER_LOG_NOTICE(State0, \"applying snapshot at ~0p:~0p.\", [SnapshotIndex, SnapshotTerm]),\n                State1 = open_snapshot(Root, SnapshotPos, State0),\n                State2 = case SnapshotTerm > CurrentTerm of\n                    true -> advance_term(?FUNCTION_NAME, SnapshotTerm, undefined, State1);\n                    false -> State1\n                end,\n                % At this point, we assume that we received some cluster membership configuration from\n                % our peer so it is safe to transition to an operational state.\n                reply(Type, ok),\n                {next_state, follower_or_witness_state(State2), State2}\n            catch\n                _:Reason ->\n                    ?SERVER_LOG_WARNING(State0, \"failed to load available snapshot ~0p due to ~0P\", [Root, Reason, 20]),\n                    reply(Type, {error, Reason}),\n                    keep_state_and_data\n            end;\n        false ->\n            ?SERVER_LOG_NOTICE(State0, \"at ~0p ignoring old snapshot at ~0p:~0p\", [LastApplied, SnapshotIndex, SnapshotTerm]),\n            reply(Type, {error, rejected}),\n            keep_state_and_data\n    end;\n\n%% [Command] Defer to common handling for generic RAFT server commands\nstalled(Type, ?RAFT_COMMAND(_, _) = Event, #raft_state{} = State) ->\n    command(?FUNCTION_NAME, Type, Event, State);\n\n%% [Fallback] Report unhandled events\nstalled(Type, Event, #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"did not know how to handle ~0p event ~0P\", [Type, Event, 20]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Leader State\n%%------------------------------------------------------------------------------\n%% In a RAFT cluster, the leader of a RAFT term is a replica that has received\n%% a quorum of votes from the cluster in that RAFT term, establishing it as the\n%% unique coordinator for that RAFT term. The leader is responsible for\n%% accepting and replicating new log entries to progress the state of the FSM.\n%%------------------------------------------------------------------------------\n\n-spec leader\n    (\n        Type :: enter,\n        PreviousStateName :: state(),\n        Data :: #raft_state{}\n    ) -> gen_statem:state_enter_result(state(), #raft_state{});\n    (\n        Type :: gen_statem:event_type(),\n        Event :: event(),\n        Data :: #raft_state{}\n    ) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\nleader(enter, PreviousStateName, #raft_state{table = Table, self = Self, log_view = View0} = State0) ->\n    ?RAFT_COUNT(Table, 'leader.enter'),\n    ?RAFT_COUNT(Table, 'leader.elected'),\n    ?SERVER_LOG_NOTICE(State0, \"becomes leader from state ~0p.\", [PreviousStateName]),\n\n    % Setup leader state and announce leadership\n    State1 = enter_state(?FUNCTION_NAME, State0),\n    State2 = set_leader(?FUNCTION_NAME, Self, State1),\n\n    % Attempt to refresh the label state as necessary for new log entries\n    LastLogIndex = wa_raft_log:last_index(View0),\n    State3 = case wa_raft_log:get(View0, LastLogIndex) of\n        {ok, {_, {_, LastLabel, _}}} ->\n            State2#raft_state{last_label = LastLabel};\n        {ok, {_, undefined}} ->\n            % The RAFT log could have been reset (i.e. after snapshot installation).\n            % In such case load the log label state from storage.\n            LastLabel = load_label_state(State2),\n            State2#raft_state{last_label = LastLabel};\n        {ok, _} ->\n            State2#raft_state{last_label = undefined}\n    end,\n\n    % At the start of a new term, the leader should append a new log\n    % entry that will start the process of establishing the first\n    % quorum in the new term by starting replication and clearing out\n    % any log mismatches on follower replicas.\n    {LogEntry, State4} = make_log_entry({make_ref(), noop}, State3),\n    {ok, View1} = wa_raft_log:append(View0, [LogEntry]),\n    TermStartIndex = wa_raft_log:last_index(View1),\n    State5 = State4#raft_state{log_view = View1, first_current_term_log_index = TermStartIndex},\n\n    % Perform initial heartbeat and log entry resolution\n    State6 = append_entries_to_followers(State5),\n    State7 = apply_single_node_cluster(State6), % apply immediately for single node cluster\n    {keep_state, State7, ?HEARTBEAT_TIMEOUT(State7)};\n\n%% [Internal] Advance to newer term when requested\nleader(\n    internal,\n    ?ADVANCE_TERM(NewTerm),\n    #raft_state{table = Table, current_term = CurrentTerm} = State0\n) when NewTerm > CurrentTerm ->\n    ?RAFT_COUNT(Table, 'leader.advance_term'),\n    ?SERVER_LOG_NOTICE(State0, \"advancing to new term ~0p.\", [NewTerm]),\n    State1 = advance_term(?FUNCTION_NAME, NewTerm, undefined, State0),\n    {next_state, follower_or_witness_state(State1), State1};\n\n%% [Protocol] Parse any RPCs in network formats\nleader(Type, Event, #raft_state{} = State) when is_tuple(Event), element(1, Event) =:= rpc ->\n    handle_rpc(Type, Event, ?FUNCTION_NAME, State);\n\n%% [AppendEntries RPC] Leaders should not act upon any incoming heartbeats (5.1, 5.2)\nleader(_, ?REMOTE(_, ?APPEND_ENTRIES(_, _, _, _, _)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Leader] Handle AppendEntries RPC responses (5.2, 5.3, 7).\n%% Handle normal-case successes\nleader(\n    cast,\n    ?REMOTE(\n        ?IDENTITY_REQUIRES_MIGRATION(_, FollowerId) = Sender,\n        ?APPEND_ENTRIES_RESPONSE(_, true, FollowerMatchIndex, FollowerLastAppliedIndex)\n    ),\n    #raft_state{\n        table = Table,\n        commit_index = CommitIndex,\n        next_indices = NextIndices,\n        match_indices = MatchIndices,\n        last_applied_indices = LastAppliedIndices,\n        heartbeat_response_ts = HeartbeatResponse0,\n        first_current_term_log_index = TermStartIndex\n    } = State0\n) ->\n    StartTUsec = erlang:monotonic_time(microsecond),\n    ?SERVER_LOG_DEBUG(State0, \"at commit index ~0p completed append to ~0p whose log now matches up to ~0p.\",\n        [CommitIndex, Sender, FollowerMatchIndex]),\n    HeartbeatResponse1 = HeartbeatResponse0#{FollowerId => erlang:monotonic_time(millisecond)},\n    State1 = State0#raft_state{heartbeat_response_ts = HeartbeatResponse1},\n\n    NextIndex = maps:get(FollowerId, NextIndices, TermStartIndex),\n    NewMatchIndices = MatchIndices#{FollowerId => FollowerMatchIndex},\n    NewNextIndices = NextIndices#{FollowerId => max(NextIndex, FollowerMatchIndex + 1)},\n    NewLastAppliedIndices = case FollowerLastAppliedIndex of\n        undefined -> LastAppliedIndices;\n        _ -> LastAppliedIndices#{FollowerId => FollowerLastAppliedIndex}\n    end,\n\n    State2 = State1#raft_state{\n        next_indices = NewNextIndices,\n        match_indices = NewMatchIndices,\n        last_applied_indices = NewLastAppliedIndices\n    },\n    State3 = maybe_advance(State2),\n    State4 = apply_log_leader(State3),\n    ?RAFT_GATHER(Table, 'leader.apply.func', erlang:monotonic_time(microsecond) - StartTUsec),\n    {keep_state, maybe_heartbeat(State4), ?HEARTBEAT_TIMEOUT(State4)};\n\n%% and failures.\nleader(\n    cast,\n    ?REMOTE(\n        ?IDENTITY_REQUIRES_MIGRATION(_, FollowerId) = Sender,\n        ?APPEND_ENTRIES_RESPONSE(_, false, FollowerEndIndex, FollowerLastAppliedIndex)\n    ),\n    #raft_state{\n        table = Table,\n        commit_index = CommitIndex,\n        next_indices = NextIndices,\n        last_applied_indices = LastAppliedIndices\n    } = State0\n) ->\n    ?RAFT_COUNT(Table, 'leader.append.failure'),\n    ?SERVER_LOG_DEBUG(State0, \"at commit index ~0p failed append to ~0p whose log now ends at ~0p.\",\n        [CommitIndex, Sender, FollowerEndIndex]),\n\n    % Check to see if we should request a snapshot due to this failure\n    leader_maybe_request_snapshot(FollowerId, FollowerEndIndex, FollowerLastAppliedIndex, State0),\n\n    % We must trust the follower's last log index here because the follower may have\n    % applied a snapshot since the last successful heartbeat. In such case, we need\n    % to fast-forward the follower's next index so that we resume replication at the\n    % point after the snapshot.\n    NewNextIndices = NextIndices#{FollowerId => FollowerEndIndex + 1},\n    NewLastAppliedIndices = case FollowerLastAppliedIndex of\n        undefined -> LastAppliedIndices;\n        _ -> LastAppliedIndices#{FollowerId => FollowerLastAppliedIndex}\n    end,\n\n    State1 = State0#raft_state{\n        next_indices = NewNextIndices,\n        last_applied_indices = NewLastAppliedIndices\n    },\n    State2 = apply_log_leader(State1),\n    {keep_state, maybe_heartbeat(State2), ?HEARTBEAT_TIMEOUT(State2)};\n\n%% [RequestVote RPC] Ignore any vote requests as leadership is aleady established (5.1, 5.2)\nleader(_, ?REMOTE(_, ?REQUEST_VOTE(_, _, _)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Vote RPC] We are already leader, so we don't need to consider any more votes (5.1)\nleader(_, ?REMOTE(_, ?VOTE(_)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Handover][Handover RPC] We are already leader so ignore any handover requests.\nleader(_, ?REMOTE(Sender, ?HANDOVER(Reference, _, _, _)), #raft_state{} = State) ->\n    send_rpc(Sender, ?HANDOVER_FAILED(Reference), State),\n    keep_state_and_data;\n\n%% [Handover][HandoverFailed RPC] Our handover failed, so clear the handover status.\nleader(\n    _,\n    ?REMOTE(?IDENTITY_REQUIRES_MIGRATION(_, NodeId) = Sender, ?HANDOVER_FAILED(Reference)),\n    #raft_state{\n        handover = {NodeId, Reference, _}\n    } = State\n) ->\n    ?SERVER_LOG_NOTICE(State, \"resuming normal operations after failed handover to ~0p.\", [Sender]),\n    {keep_state, State#raft_state{handover = undefined}};\n\n%% [Handover][HandoverFailed RPC] Got a handover failed with an unknown ID. Ignore.\nleader(_, ?REMOTE(_, ?HANDOVER_FAILED(_)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Timeout] Suspend periodic heartbeat to followers while handover is active\nleader(state_timeout = Type, Event, #raft_state{handover = {Peer, _, Timeout}} = State) ->\n    NowMillis = erlang:monotonic_time(millisecond),\n    case NowMillis > Timeout of\n        true ->\n            ?SERVER_LOG_NOTICE(State, \"handover to ~0p times out.\", [Peer]),\n            {keep_state, State#raft_state{handover = undefined}, {next_event, Type, Event}};\n        false ->\n            check_leader_liveness(State),\n            {keep_state_and_data, ?HEARTBEAT_TIMEOUT(State)}\n    end;\n\n%% [Timeout] Periodic heartbeat to followers\nleader(state_timeout, _, State0) ->\n    case leader_eligible(State0) of\n        true ->\n            State1 = append_entries_to_followers(State0),\n            State2 = apply_single_node_cluster(State1),\n            check_leader_liveness(State2),\n            {keep_state, State2, ?HEARTBEAT_TIMEOUT(State2)};\n        false ->\n            ?SERVER_LOG_NOTICE(State0, \"resigns from leadership because this node is ineligible.\", []),\n            State1 = clear_leader(?FUNCTION_NAME, State0),\n            {next_state, follower_or_witness_state(State1), State1}\n    end;\n\n%% [Commit]\n%%   If a handover is in progress, reject the commit immediately with a\n%%   notify_redirect error so the client can redirect to the new leader.\n%%   Otherwise, add the commit to the pending list and append if enough\n%%   have accumulated.\nleader(\n    cast,\n    ?COMMIT_COMMAND(From, _Op, Priority),\n    #raft_state{\n        table = Table,\n        queues = Queues,\n        handover = {Peer, _, _}\n    } = _State\n) ->\n    ?RAFT_COUNT(Table, 'commit.rejected.handover'),\n    wa_raft_queue:commit_cancelled(Queues, From, {error, {notify_redirect, Peer}}, Priority),\n    keep_state_and_data;\nleader(\n    cast,\n    ?COMMIT_COMMAND(From, Op, Priority),\n    #raft_state{\n        application = App,\n        table = Table,\n        pending_high = PendingHigh,\n        pending_low = PendingLow\n    } = State0\n) ->\n    % No size limit is imposed here as the pending queue cannot grow larger\n    % than the limit on the number of pending commits.\n    ?RAFT_COUNT(Table, {'commit', Priority}),\n    State1 = case Priority of\n        high ->\n            State0#raft_state{pending_high = [{From, Op} | PendingHigh]};\n        low ->\n            State0#raft_state{pending_low = [{From, Op} | PendingLow]}\n    end,\n    State2 = apply_single_node_cluster(State1),\n    PendingCount = length(State2#raft_state.pending_high) + length(State2#raft_state.pending_low),\n    case ?RAFT_COMMIT_BATCH_INTERVAL(App, Table) > 0 andalso PendingCount =< ?RAFT_COMMIT_BATCH_MAX_ENTRIES(App, Table) of\n        true ->\n            ?RAFT_COUNT(Table, 'commit.batch.delay'),\n            {keep_state, State2, ?COMMIT_BATCH_TIMEOUT(State2)};\n        false ->\n            State3 = append_entries_to_followers(State2),\n            {keep_state, State3, ?HEARTBEAT_TIMEOUT(State3)}\n    end;\n\n%% [Strong Read]\n%%   If a handover is in progress, reject the read immediately with a\n%%   notify_redirect error so the client can redirect to the new leader.\nleader(\n    cast,\n    ?READ_COMMAND({From, _}),\n    #raft_state{\n        table = Table,\n        queues = Queues,\n        handover = {Peer, _, _}\n    } = _State\n) ->\n    ?RAFT_COUNT(Table, 'read.rejected.handover'),\n    wa_raft_queue:fulfill_incomplete_read(Queues, From, {error, {notify_redirect, Peer}}),\n    keep_state_and_data;\nleader(\n    cast,\n    ?READ_COMMAND({From, Command}),\n    #raft_state{\n        self = Self,\n        queues = Queues,\n        storage = Storage,\n        commit_index = CommitIndex,\n        last_applied = LastApplied,\n        pending_high = PendingHigh,\n        pending_low = PendingLow,\n        first_current_term_log_index = FirstLogIndex\n    } = State0\n) ->\n    ReadIndex = max(CommitIndex, FirstLogIndex),\n    case is_single_member(Self, config(State0)) of\n        % If we are a single node cluster and we are fully-applied, then immediately dispatch.\n        true when PendingHigh =:= [], PendingLow =:= [], ReadIndex =< LastApplied ->\n            wa_raft_storage:apply_read(Storage, From, Command),\n            {keep_state, State0};\n        _ ->\n            ok = wa_raft_queue:submit_read(Queues, ReadIndex, From, Command),\n            % Regardless of whether or not the read index is an existing log entry, indicate that\n            % a read is pending as the leader must establish a new quorum to be able to serve the\n            % read request.\n            {keep_state, State0#raft_state{pending_read = true}}\n    end;\n\n%% [Resign] Leader resigns by switching to follower state.\nleader({call, From}, ?RESIGN_COMMAND, #raft_state{} = State0) ->\n    ?SERVER_LOG_NOTICE(State0, \"resigns.\", []),\n    State1 = clear_leader(?FUNCTION_NAME, State0),\n    {next_state, follower_or_witness_state(State1), State1, {reply, From, ok}};\n\n%% [Adjust Membership] Leader attempts to commit a single-node membership change.\nleader(Type, ?ADJUST_CONFIG_COMMAND(From, Action, Index), #raft_state{queues = Queues} = State0) ->\n    maybe\n        ok ?= leader_config_change_allowed(Index, State0),\n        {ok, NewConfig} ?= leader_adjust_config(Action, State0),\n        % With all checks completed, we can now attempt to append the new\n        % configuration to the log. If successful, a round of heartbeats is\n        % immediately started to replicate the change as soon as possible.\n        {ok, #raft_log_pos{index = NewConfigIndex} = NewConfigPosition, State1} ?=\n            leader_change_config(NewConfig, From, State0),\n\n        ?SERVER_LOG_NOTICE(\n            State1,\n            \"is attempting to change configuration from ~0p to ~0p at ~0p.\",\n            [config(State0), NewConfig, NewConfigIndex]\n        ),\n        State2 = apply_single_node_cluster(State1),\n        State3 = append_entries_to_followers(State2),\n        reply(Type, {ok, NewConfigPosition}),\n        {keep_state, State3, ?HEARTBEAT_TIMEOUT(State3)}\n    else\n        {error, Reason} ->\n            ?SERVER_LOG_NOTICE(\n                State0,\n                \"failed to apply action ~0p to current configuration ~0p due to ~0P.\",\n                [Action, config(State0), Reason, 20]\n            ),\n            From =/= undefined andalso\n                wa_raft_queue:commit_completed(Queues, From, {error, Reason}, high),\n            reply(Type, {error, Reason}),\n            keep_state_and_data\n    end;\n\n%% [Handover Candidates] Return list of handover candidates (peers that are not lagging too much)\nleader({call, From}, ?HANDOVER_CANDIDATES_COMMAND, #raft_state{} = State) ->\n    {keep_state_and_data, {reply, From, {ok, get_handover_candidates(State)}}};\n\n%% [Is Peer Ready] Check if participant is caught up\nleader({call, From}, ?IS_PEER_READY_COMMAND(Peer), #raft_state{} = State) ->\n    Config = config(State),\n    IsParticipant = lists:member(Peer, config_participants(Config)),\n    IsReady = is_eligible_for_handover(Peer, State),\n    Result =\n        if\n            not IsParticipant -> {error, not_a_participant};\n            not IsReady -> {error, not_ready};\n            true -> ok\n        end,\n    {keep_state_and_data, {reply, From, Result}};\n\n%% [Handover] With peer 'undefined' randomly select a valid candidate to handover to\nleader(Type, ?HANDOVER_COMMAND(undefined), #raft_state{} = State) ->\n    case get_handover_candidates(State) of\n        [] ->\n            ?SERVER_LOG_NOTICE(State, \"has no valid peer to handover to.\", []),\n            reply(Type, {error, no_valid_peer}),\n            keep_state_and_data;\n        Candidates ->\n            Peer = lists:nth(rand:uniform(length(Candidates)), Candidates),\n            leader(Type, ?HANDOVER_COMMAND(Peer), State)\n    end;\n\n%% [Handover] Handover to self results in no-op\nleader(Type, ?HANDOVER_COMMAND(Peer), #raft_state{} = State) when Peer =:= node() ->\n    ?SERVER_LOG_WARNING(State, \"dropping handover to self.\", []),\n    reply(Type, {ok, Peer}),\n    {keep_state, State};\n\n%% [Handover] Attempt to start a handover to the specified peer\nleader(\n    Type,\n    ?HANDOVER_COMMAND(Peer),\n    #raft_state{\n        application = App,\n        table = Table,\n        name = Name,\n        log_view = View,\n        match_indices = MatchIndices,\n        handover = undefined\n    } = State0\n) ->\n    % TODO T246543673 For the time being, assume that all members of the\n    %                 cluster use the same server name.\n    case is_member({Name, Peer}, config(State0)) of\n        false ->\n            ?SERVER_LOG_WARNING(State0, \"dropping handover to unknown peer ~0p.\", [Peer]),\n            reply(Type, {error, invalid_peer}),\n            keep_state_and_data;\n        true ->\n            PeerMatchIndex = maps:get(Peer, MatchIndices, 0),\n            FirstIndex = wa_raft_log:first_index(View),\n            PeerSendIndex = max(PeerMatchIndex + 1, FirstIndex + 1),\n            LastIndex = wa_raft_log:last_index(View),\n            MaxHandoverBatchSize = ?RAFT_HANDOVER_MAX_ENTRIES(App, Table),\n            MaxHandoverBytes = ?RAFT_HANDOVER_MAX_BYTES(App, Table),\n\n            case LastIndex - PeerSendIndex =< MaxHandoverBatchSize of\n                true ->\n                    ?RAFT_COUNT(Table, 'leader.handover'),\n                    ?SERVER_LOG_NOTICE(State0, \"starting handover to ~p.\", [Peer]),\n\n                    PrevLogIndex = PeerSendIndex - 1,\n                    {ok, PrevLogTerm} = wa_raft_log:term(View, PrevLogIndex),\n                    {ok, LogEntries} = wa_raft_log:entries(View, PeerSendIndex, MaxHandoverBatchSize, MaxHandoverBytes),\n\n                    % The request to load the log may result in not all required log entries being loaded\n                    % if we hit the byte size limit. Ensure that we have loaded all required log entries\n                    % before initiating a handover.\n                    case PrevLogIndex + length(LogEntries) of\n                        LastIndex ->\n                            Ref = make_ref(),\n                            Timeout = erlang:monotonic_time(millisecond) + ?RAFT_HANDOVER_TIMEOUT(App, Table),\n                            State1 = State0#raft_state{handover = {Peer, Ref, Timeout}},\n                            send_rpc(?IDENTITY_REQUIRES_MIGRATION(Name, Peer), ?HANDOVER(Ref, PrevLogIndex, PrevLogTerm, LogEntries), State1),\n                            reply(Type, {ok, Peer}),\n                            {keep_state, State1};\n                        _ ->\n                            ?RAFT_COUNT(Table, 'leader.handover.oversize'),\n                            ?SERVER_LOG_WARNING(State0, \"handover to peer ~0p would require an oversized RPC.\", [Peer]),\n                            reply(Type, {error, oversize}),\n                            keep_state_and_data\n                    end;\n                false ->\n                    ?RAFT_COUNT(Table, 'leader.handover.peer_lagging'),\n                    ?SERVER_LOG_WARNING(State0, \"determines that peer ~0p is not eligible for handover because it is ~0p entries behind.\",\n                        [Peer, LastIndex - PeerSendIndex]),\n                    reply(Type, {error, peer_lagging}),\n                    keep_state_and_data\n            end\n    end;\n\n%% [Handover] Reject starting a handover when a handover is still in progress\nleader({call, From}, ?HANDOVER_COMMAND(Peer), #raft_state{handover = {Node, _, _}} = State) ->\n    ?SERVER_LOG_WARNING(State, \"rejecting duplicate handover to ~0p with running handover to ~0p.\", [Peer, Node]),\n    {keep_state_and_data, {reply, From, {error, duplicate}}};\n\n%% [Command] Defer to common handling for generic RAFT server commands\nleader(Type, ?RAFT_COMMAND(_, _) = Event, #raft_state{} = State) ->\n    command(?FUNCTION_NAME, Type, Event, State);\n\n%% [Fallback] Report unhandled events\nleader(Type, Event, #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"did not know how to handle ~0p event ~0P\", [Type, Event, 20]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Follower State\n%%------------------------------------------------------------------------------\n%% In a RAFT cluster, a follower is a replica that is receiving replicated log\n%% entries from the leader of a RAFT term. The follower participates in quorum\n%% decisions about log entries received from the leader by appending those log\n%% entries to its own local copy of the RAFT log.\n%%------------------------------------------------------------------------------\n\n-spec follower\n    (\n        Type :: enter,\n        PreviousStateName :: state(),\n        Data :: #raft_state{}\n    ) -> gen_statem:state_enter_result(state(), #raft_state{});\n    (\n        Type :: gen_statem:event_type(),\n        Event :: event(),\n        Data :: #raft_state{}\n    ) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\nfollower(enter, PreviousStateName, #raft_state{table = Table} = State) ->\n    ?RAFT_COUNT(Table, 'follower.enter'),\n    ?SERVER_LOG_NOTICE(State, \"becomes follower from state ~0p.\", [PreviousStateName]),\n    {keep_state, enter_state(?FUNCTION_NAME, State), ?ELECTION_TIMEOUT(State)};\n\n%% [Internal] Advance to newer term when requested\nfollower(internal, ?ADVANCE_TERM(NewTerm), #raft_state{table = Table, current_term = CurrentTerm} = State) when NewTerm > CurrentTerm ->\n    ?RAFT_COUNT(Table, 'follower.advance_term'),\n    ?SERVER_LOG_NOTICE(State, \"advancing to new term ~0p.\", [NewTerm]),\n    {repeat_state, advance_term(?FUNCTION_NAME, NewTerm, undefined, State)};\n\n%% [Protocol] Parse any RPCs in network formats\nfollower(Type, Event, #raft_state{} = State) when is_tuple(Event), element(1, Event) =:= rpc ->\n    handle_rpc(Type, Event, ?FUNCTION_NAME, State);\n\n%% [AppendEntries RPC] Handle incoming heartbeats (5.2, 5.3)\nfollower(\n    Type,\n    ?REMOTE(Leader, ?APPEND_ENTRIES(PrevLogIndex, PrevLogTerm, Entries, CommitIndex, TrimIndex)),\n    #raft_state{} = State\n) ->\n    handle_heartbeat(?FUNCTION_NAME, Type, Leader, PrevLogIndex, PrevLogTerm, Entries, CommitIndex, TrimIndex, State);\n\n%% [AppendEntriesResponse RPC] Followers should not act upon any incoming heartbeat responses (5.2)\nfollower(_, ?REMOTE(_, ?APPEND_ENTRIES_RESPONSE(_, _, _, _)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [RequestVote RPC] Handle incoming vote requests (5.2)\nfollower(_, ?REMOTE(Candidate, ?REQUEST_VOTE(_, CandidateIndex, CandidateTerm)), #raft_state{} = State) ->\n    request_vote_impl(?FUNCTION_NAME, Candidate, CandidateIndex, CandidateTerm, State);\n\n%% [Handover][Handover RPC] The leader is requesting this follower to take over leadership in a new term\nfollower(\n    _,\n    ?REMOTE(\n        ?IDENTITY_REQUIRES_MIGRATION(_, LeaderId) = Sender,\n        ?HANDOVER(Ref, PrevLogIndex, PrevLogTerm, LogEntries)\n    ),\n    #raft_state{\n        application = App,\n        table = Table,\n        leader_id = LeaderId\n    } = State0\n) ->\n    ?RAFT_COUNT(Table, 'follower.handover'),\n    ?SERVER_LOG_NOTICE(State0, \"evaluating handover RPC from ~0p.\", [Sender]),\n    case ?RAFT_LEADER_ELIGIBLE(App) andalso ?RAFT_ELECTION_WEIGHT(App) =/= 0 of\n        true ->\n            case append_entries(?FUNCTION_NAME, PrevLogIndex, PrevLogTerm, LogEntries, length(LogEntries), State0) of\n                {ok, true, _, State1} ->\n                    ?SERVER_LOG_NOTICE(State1, \"immediately starting new election due to append success during handover RPC.\", []),\n                    candidate_or_witness_state_transition(State1);\n                {ok, false, _, State1} ->\n                    ?RAFT_COUNT(Table, 'follower.handover.rejected'),\n                    ?SERVER_LOG_WARNING(State1, \"failing handover request because append was rejected.\", []),\n                    send_rpc(Sender, ?HANDOVER_FAILED(Ref), State1),\n                    {keep_state, State1};\n                {fatal, Reason} ->\n                    ?RAFT_COUNT(Table, 'follower.handover.fatal'),\n                    ?SERVER_LOG_WARNING(State0, \"failing handover request because append was fatal due to ~0P.\", [Reason, 30]),\n                    send_rpc(Sender, ?HANDOVER_FAILED(Ref), State0),\n                    State1 = State0#raft_state{disable_reason = Reason},\n                    wa_raft_durable_state:store(State1),\n                    {next_state, disabled, State1}\n            end;\n        false ->\n            ?SERVER_LOG_NOTICE(State0, \"not considering handover RPC due to being inelgibile for leadership.\", []),\n            send_rpc(Sender, ?HANDOVER_FAILED(Ref), State0),\n            {keep_state, State0}\n    end;\n\n%% [Handover][HandoverFailed RPC] Followers should not act upon any incoming failed handover\nfollower(_, ?REMOTE(_, ?HANDOVER_FAILED(_)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Follower] handle timeout\n%% follower doesn't receive any heartbeat. starting a new election\nfollower(state_timeout, _, #raft_state{\n        table = Table,\n        leader_id = LeaderId,\n        log_view = View,\n        leader_heartbeat_ts = HeartbeatTs\n    } = State\n) ->\n    WaitingMs = case HeartbeatTs of\n        undefined -> undefined;\n        _         -> erlang:monotonic_time(millisecond) - HeartbeatTs\n    end,\n    ?RAFT_COUNT(Table, 'follower.timeout'),\n    case candidate_eligible(State) of\n        true ->\n            ?SERVER_LOG_NOTICE(State, \"times out and starts election at ~0p after waiting for leader ~0p for ~0p ms.\",\n                [wa_raft_log:last_index(View), WaitingMs, LeaderId]),\n            {next_state, candidate, State};\n        false ->\n            ?SERVER_LOG_NOTICE(State, \"is not timing out due to being ineligible or having zero election weight.\", []),\n            {repeat_state, State}\n    end;\n\n%% [Command] Defer to common handling for generic RAFT server commands\nfollower(Type, ?RAFT_COMMAND(_, _) = Event, #raft_state{} = State) ->\n    command(?FUNCTION_NAME, Type, Event, State);\n\n%% [Fallback] Report unhandled events\nfollower(Type, Event, #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"did not know how to handle ~0p event ~0P\", [Type, Event, 20]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Candidate State\n%%------------------------------------------------------------------------------\n%% In a RAFT cluster, a candidate is a replica that is attempting to become the\n%% leader of a RAFT term. It is waiting for responses from the other members of\n%% the RAFT cluster to determine if it has received enough votes to assume the\n%% leadership of the RAFT term.\n%%------------------------------------------------------------------------------\n\n-spec candidate\n    (\n        Type :: enter,\n        PreviousStateName :: state(),\n        Data :: #raft_state{}\n    ) -> gen_statem:state_enter_result(state(), #raft_state{});\n    (\n        Type :: gen_statem:event_type(),\n        Event :: event(),\n        Data :: #raft_state{}\n    ) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\n%% [Enter] Node starts a new election upon entering the candidate state.\ncandidate(\n    enter,\n    PreviousStateName,\n    #raft_state{\n        table = Table,\n        self = Self,\n        current_term = CurrentTerm,\n        log_view = View\n    } = State0\n) ->\n    ?RAFT_COUNT(Table, 'leader.election_started'),\n    ?RAFT_COUNT(Table, 'candidate.enter'),\n    ?SERVER_LOG_NOTICE(State0, \"becomes candidate from state ~0p.\", [PreviousStateName]),\n\n    % Entering the candidate state means that we are starting a new election, thus\n    % advance the term and set VotedFor to the current node. (Candidates always\n    % implicitly vote for themselves.)\n    State1 = enter_state(?FUNCTION_NAME, State0),\n    State2 = advance_term(?FUNCTION_NAME, CurrentTerm + 1, node(), State1),\n\n    % Determine the log index and term at which the election will occur.\n    LastLogIndex = wa_raft_log:last_index(View),\n    {ok, LastLogTerm} = wa_raft_log:term(View, LastLogIndex),\n\n    ?SERVER_LOG_NOTICE(State2, \"advances to new term and starts election at ~0p:~0p.\",\n        [LastLogIndex, LastLogTerm]),\n\n    % Broadcast vote requests and also send a vote-for-self.\n    % (Candidates always implicitly vote for themselves.)\n    send_rpc_to_all_members(?REQUEST_VOTE(normal, LastLogIndex, LastLogTerm), State2),\n    send_rpc(Self, ?VOTE(true), State2),\n\n    {keep_state, State2, ?ELECTION_TIMEOUT(State2)};\n\n%% [Internal] Advance to newer term when requested\ncandidate(\n    internal,\n    ?ADVANCE_TERM(NewTerm),\n    #raft_state{\n        table = Table,\n        current_term = CurrentTerm\n    } = State\n) when NewTerm > CurrentTerm ->\n    ?RAFT_COUNT(Table, 'candidate.advance_term'),\n    ?SERVER_LOG_NOTICE(State, \"advancing to new term ~0p.\", [NewTerm]),\n    State1 = advance_term(?FUNCTION_NAME, NewTerm, undefined, State),\n    {next_state, follower_or_witness_state(State1), State1};\n\n%% [ForceElection] Resend vote requests with the 'force' type to force an election even if an active leader is available.\ncandidate(\n    internal,\n    ?FORCE_ELECTION(Term),\n    #raft_state{\n        log_view = View,\n        current_term = CurrentTerm\n    } = State\n) when Term + 1 =:= CurrentTerm ->\n    ?SERVER_LOG_NOTICE(State, \"accepts request to force election issued by the immediately prior term ~0p.\", [Term]),\n    % No changes to the log are expected during an election so we can just reload these values from the log view.\n    LastLogIndex = wa_raft_log:last_index(View),\n    {ok, LastLogTerm} = wa_raft_log:term(View, LastLogIndex),\n    send_rpc_to_all_members(?REQUEST_VOTE(force, LastLogIndex, LastLogTerm), State),\n    keep_state_and_data;\n\n%% [Protocol] Parse any RPCs in network formats\ncandidate(Type, Event, #raft_state{} = State) when is_tuple(Event), element(1, Event) =:= rpc ->\n    handle_rpc(Type, Event, ?FUNCTION_NAME, State);\n\n%% [AppendEntries RPC] Switch to follower because current term now has a leader (5.2, 5.3)\ncandidate(Type, ?REMOTE(Sender, ?APPEND_ENTRIES(_, _, _, _, _)) = Event, #raft_state{} = State) ->\n    ?SERVER_LOG_NOTICE(State, \"switching to follower after receiving heartbeat from ~0p.\", [Sender]),\n    {next_state, follower_or_witness_state(State), State, {next_event, Type, Event}};\n\n%% [RequestVote RPC] Candidates should ignore incoming vote requests as they always vote for themselves (5.2)\ncandidate(_, ?REMOTE(_, ?REQUEST_VOTE(_, _, _)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Vote RPC] Candidate receives an affirmative vote (5.2)\ncandidate(\n    cast,\n    ?REMOTE(?IDENTITY_REQUIRES_MIGRATION(_, NodeId), ?VOTE(true)),\n    #raft_state{\n        table = Table,\n        log_view = View,\n        state_start_ts = StateStartTs,\n        heartbeat_response_ts = HeartbeatResponse0,\n        votes = Votes0\n    } = State0\n) ->\n    HeartbeatResponse1 = HeartbeatResponse0#{NodeId => erlang:monotonic_time(millisecond)},\n    Votes1 = Votes0#{NodeId => true},\n    State1 = State0#raft_state{heartbeat_response_ts = HeartbeatResponse1, votes = Votes1},\n    case compute_quorum(Votes1, false, config(State1)) of\n        true ->\n            Duration = erlang:monotonic_time(millisecond) - StateStartTs,\n            LastIndex = wa_raft_log:last_index(View),\n            {ok, LastTerm} = wa_raft_log:term(View, LastIndex),\n            EstablishedQuorum = [Peer || Peer := true <- Votes1],\n            ?SERVER_LOG_NOTICE(State1, \"is becoming leader after ~0p ms with log at ~0p:~0p and votes from ~0p.\",\n                [Duration, LastIndex, LastTerm, EstablishedQuorum]),\n            ?RAFT_GATHER(Table, 'candidate.election.duration', Duration),\n            {next_state, leader, State1};\n        false ->\n            {keep_state, State1}\n    end;\n\n%% [Vote RPC] Candidates should ignore negative votes (5.2)\ncandidate(cast, ?REMOTE(_, ?VOTE(_)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Handover][Handover RPC] Switch to follower because current term now has a leader (5.2, 5.3)\ncandidate(Type, ?REMOTE(_, ?HANDOVER(_, _, _, _)) = Event, #raft_state{} = State) ->\n    ?SERVER_LOG_NOTICE(State, \"ends election to handle handover.\", []),\n    {next_state, follower_or_witness_state(State), State, {next_event, Type, Event}};\n\n%% [Handover][HandoverFailed RPC] Candidates should not act upon any incoming failed handover\ncandidate(_, ?REMOTE(_, ?HANDOVER_FAILED(_)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Candidate] Handle Election Timeout (5.2)\n%% Candidate doesn't get enough votes after a period of time, restart election or fallback\n%% to follower if the local replica is no longer eligible.\ncandidate(state_timeout, _, #raft_state{votes = Votes} = State) ->\n    ?SERVER_LOG_NOTICE(State, \"election timed out with votes ~0p.\", [Votes]),\n    case candidate_eligible(State) of\n        true -> {repeat_state, State};\n        false -> {next_state, follower, State}\n    end;\n\n%% [Command] Defer to common handling for generic RAFT server commands\ncandidate(Type, ?RAFT_COMMAND(_, _) = Event, #raft_state{} = State) ->\n    command(?FUNCTION_NAME, Type, Event, State);\n\n%% [Fallback] Report unhandled events\ncandidate(Type, Event, #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"did not know how to handle ~0p event ~0P.\", [Type, Event, 20]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Disabled State\n%%------------------------------------------------------------------------------\n%% The disabled state is an extension to the RAFT protocol used to hold any\n%% replicas of an FSM that have for some reason or another identified that some\n%% deficiency or malfunction that makes them unfit to either enforce any prior\n%% quorum decisions or properly participate in future quorum decisions. Common\n%% reasons include the detection of corruptions or inconsistencies within the\n%% FSM state or RAFT log. The reason for which the replica was disabled is kept\n%% in persistent so that the replica will remain disabled even when restarted.\n%%------------------------------------------------------------------------------\n\n-spec disabled\n    (\n        Type :: enter,\n        PreviousStateName :: state(),\n        Data :: #raft_state{}\n    ) -> gen_statem:state_enter_result(state(), #raft_state{});\n    (\n        Type :: gen_statem:event_type(),\n        Event :: event(),\n        Data :: #raft_state{}\n    ) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\ndisabled(enter, PreviousStateName, #raft_state{table = Table, disable_reason = DisableReason} = State0) ->\n    ?RAFT_COUNT(Table, 'disabled.enter'),\n    ?SERVER_LOG_NOTICE(State0, \"becomes disabled from state ~0p with reason ~0p.\", [PreviousStateName, DisableReason]),\n    State1 = case DisableReason of\n        undefined -> State0#raft_state{disable_reason = \"No reason specified.\"};\n        _         -> State0\n    end,\n    {keep_state, enter_state(?FUNCTION_NAME, State1)};\n\n%% [Internal] Advance to newer term when requested\ndisabled(\n    internal,\n    ?ADVANCE_TERM(NewTerm),\n    #raft_state{\n        table = Table,\n        current_term = CurrentTerm\n    } = State\n) when NewTerm > CurrentTerm ->\n    ?RAFT_COUNT(Table, 'disabled.advance_term'),\n    ?SERVER_LOG_NOTICE(State, \"advancing to new term ~0p.\", [NewTerm]),\n    {keep_state, advance_term(?FUNCTION_NAME, NewTerm, undefined, State)};\n\n%% [Protocol] Parse any RPCs in network formats\ndisabled(Type, Event, #raft_state{} = State) when is_tuple(Event), element(1, Event) =:= rpc ->\n    handle_rpc(Type, Event, ?FUNCTION_NAME, State);\n\n%% [AppendEntries RPC] Disabled servers should not act upon any incoming heartbeats as they should\n%%                     behave as if dead to the cluster\ndisabled(_, ?REMOTE(_, ?APPEND_ENTRIES(_, _, _, _, _)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [RequestVote RPC] Disabled servers should not act upon any vote requests as they should behave\n%%                   as if dead to the cluster\ndisabled(_, ?REMOTE(_, ?REQUEST_VOTE(_, _, _)), #raft_state{}) ->\n    keep_state_and_data;\n\ndisabled({call, From}, ?TRIGGER_ELECTION_COMMAND(_), #raft_state{}) ->\n    {keep_state_and_data, {reply, From, {error, invalid_state}}};\n\ndisabled({call, From}, ?PROMOTE_COMMAND(_, _), #raft_state{}) ->\n    {keep_state_and_data, {reply, From, {error, invalid_state}}};\n\ndisabled({call, From}, ?ENABLE_COMMAND, #raft_state{} = State0) ->\n    ?SERVER_LOG_NOTICE(State0, \"re-enabling by request from ~0p by moving to stalled state.\", [From]),\n    State1 = State0#raft_state{disable_reason = undefined},\n    wa_raft_durable_state:store(State1),\n    {next_state, stalled, State1, {reply, From, ok}};\n\n%% [Command] Defer to common handling for generic RAFT server commands\ndisabled(Type, ?RAFT_COMMAND(_, _) = Event, #raft_state{} = State) ->\n    command(?FUNCTION_NAME, Type, Event, State);\n\n%% [Fallback] Report unhandled events\ndisabled(Type, Event, #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"did not know how to handle ~0p event ~0P\", [Type, Event, 20]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Witness State\n%%------------------------------------------------------------------------------\n%% The witness state is an extension to the RAFT protocol that identifies a\n%% replica as a special \"witness replica\" that participates in quorum decisions\n%% but does not retain a full copy of the actual underlying FSM. These replicas\n%% can use significantly fewer system resources to operate however it is not\n%% recommended for more than 25% of the replicas in a RAFT cluster to be\n%% witness replicas as having more than such a number of witness replicas can\n%% result in significantly reduced chance of data durability in the face of\n%% unexpected replica loss.\n%%------------------------------------------------------------------------------\n\n-spec witness\n    (\n        Type :: enter,\n        PreviousStateName :: state(),\n        Data :: #raft_state{}\n    ) -> gen_statem:state_enter_result(state(), #raft_state{});\n    (\n        Type :: gen_statem:event_type(),\n        Event :: event(),\n        Data :: #raft_state{}\n    ) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\nwitness(enter, PreviousStateName, #raft_state{table = Table} = State) ->\n    ?RAFT_COUNT(Table, 'witness.enter'),\n    ?SERVER_LOG_NOTICE(State, \"becomes witness from state ~0p.\", [PreviousStateName]),\n    {keep_state, enter_state(?FUNCTION_NAME, State), ?ELECTION_TIMEOUT(State)};\n\n%% [Internal] Advance to newer term when requested\nwitness(\n    internal,\n    ?ADVANCE_TERM(NewTerm),\n    #raft_state{\n        table = Table,\n        current_term = CurrentTerm\n    } = State\n) when NewTerm > CurrentTerm ->\n    ?RAFT_COUNT(Table, 'witness.advance_term'),\n    ?SERVER_LOG_NOTICE(State, \"advancing to new term ~0p.\", [NewTerm]),\n    {keep_state, advance_term(?FUNCTION_NAME, NewTerm, undefined, State)};\n\n%% [Protocol] Parse any RPCs in network formats\nwitness(Type, Event, #raft_state{} = State) when is_tuple(Event), element(1, Event) =:= rpc ->\n    handle_rpc(Type, Event, ?FUNCTION_NAME, State);\n\n%% [AppendEntries RPC] Handle incoming heartbeats (5.2, 5.3)\nwitness(\n    Type,\n    ?REMOTE(Leader, ?APPEND_ENTRIES(PrevLogIndex, PrevLogTerm, Entries, CommitIndex, TrimIndex)),\n    #raft_state{} = State\n) ->\n    handle_heartbeat(?FUNCTION_NAME, Type, Leader, PrevLogIndex, PrevLogTerm, Entries, CommitIndex, TrimIndex, State);\n\n%% [AppendEntriesResponse RPC] Witnesses should not act upon any incoming heartbeat responses (5.2)\nwitness(_, ?REMOTE(_, ?APPEND_ENTRIES_RESPONSE(_, _, _, _)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [Handover][Handover RPC] Witnesses should not receive handover requests\nwitness(_, ?REMOTE(Sender, ?HANDOVER(Reference, _, _, _)), #raft_state{} = State) ->\n    send_rpc(Sender, ?HANDOVER_FAILED(Reference), State),\n    keep_state_and_data;\n\n%% [Handover][HandoverFailed RPC] Witnesses should not act upon any incoming failed handover\nwitness(_, ?REMOTE(_, ?HANDOVER_FAILED(_)), #raft_state{}) ->\n    keep_state_and_data;\n\nwitness({call, From}, ?TRIGGER_ELECTION_COMMAND(_), #raft_state{}) ->\n    {keep_state_and_data, {reply, From, {error, invalid_state}}};\n\nwitness({call, From}, ?PROMOTE_COMMAND(_, _), #raft_state{}) ->\n    {keep_state_and_data, {reply, From, {error, invalid_state}}};\n\n%% [RequestVote RPC] Handle incoming vote requests (5.2)\nwitness(_, ?REMOTE(Candidate, ?REQUEST_VOTE(_, CandidateIndex, CandidateTerm)), #raft_state{} = State) ->\n    request_vote_impl(?FUNCTION_NAME, Candidate, CandidateIndex, CandidateTerm, State);\n\n%% [Vote RPC] Witnesses should not act upon any incoming votes as they cannot become leader\nwitness(_, ?REMOTE(_, ?VOTE(_)), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [State Timeout] Check liveness, but do not restart state.\nwitness(state_timeout, _, #raft_state{} = State) ->\n    check_follower_liveness(?FUNCTION_NAME, State),\n    {keep_state_and_data, ?ELECTION_TIMEOUT(State)};\n\n%% [Command] Defer to common handling for generic RAFT server commands\nwitness(Type, ?RAFT_COMMAND(_, _) = Event, #raft_state{} = State) ->\n    command(?FUNCTION_NAME, Type, Event, State);\n\n%% [Fallback] Report unhandled events\nwitness(Type, Event, #raft_state{} = State) ->\n    ?SERVER_LOG_WARNING(State, \"did not know how to handle ~0p event ~0P\", [Type, Event, 20]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Command Handlers\n%%------------------------------------------------------------------------------\n%% Fallbacks for command calls to the RAFT server for when there is no special\n%% handling for a command defined within the state-specific callback itself.\n%%------------------------------------------------------------------------------\n\n-spec command(\n    State :: state(),\n    Type :: gen_statem:event_type(),\n    Command :: command(),\n    Data :: #raft_state{}\n) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\n%% [Commit] Non-leader nodes should fail commits with {error, not_leader}.\ncommand(\n    State,\n    cast,\n    ?COMMIT_COMMAND(From, _Op, Priority),\n    #raft_state{\n        queues = Queues,\n        leader_id = LeaderId\n    } = Data\n) when State =/= leader ->\n    ?SERVER_LOG_WARNING(State, Data, \"commit fails as leader is currently ~0p.\", [LeaderId]),\n    wa_raft_queue:commit_cancelled(Queues, From, {error, not_leader}, Priority),\n    keep_state_and_data;\n\n%% [Strong Read] Non-leader nodes are not eligible for strong reads.\ncommand(\n    State,\n    cast,\n    ?READ_COMMAND({From, _}),\n    #raft_state{\n        queues = Queues,\n        leader_id = LeaderId\n    } = Data\n) when State =/= leader ->\n    ?SERVER_LOG_WARNING(State, Data, \"strong read fails. Leader is ~p.\", [LeaderId]),\n    wa_raft_queue:fulfill_incomplete_read(Queues, From, {error, not_leader}),\n    keep_state_and_data;\n\n%% [Notify Complete] Attempt to send more log entries to storage if applicable.\ncommand(\n    State,\n    cast,\n    ?NOTIFY_COMPLETE_COMMAND(),\n    #raft_state{\n        queues = Queues\n    } = Data\n) when State =:= leader; State =:= follower; State =:= witness ->\n    case wa_raft_queue:apply_queue_size(Queues) of\n        0 ->\n            NewState = case State of\n                leader -> apply_log_leader(Data);\n                _ -> apply_log(State, infinity, Data)\n            end,\n            {keep_state, NewState};\n        _ ->\n            keep_state_and_data\n    end;\ncommand(_, cast, ?NOTIFY_COMPLETE_COMMAND(), #raft_state{}) ->\n    keep_state_and_data;\n\n%% [CurrentConfig] Get replica's effective RAFT cluster configuration\ncommand(_, Type, ?CURRENT_CONFIG_COMMAND, #raft_state{} = Data) ->\n    reply(Type, config(Data)),\n    keep_state_and_data;\n\n%% [Status] Get status of node.\ncommand(State, {call, From}, ?STATUS_COMMAND, #raft_state{} = Data) ->\n    Status = [\n        {state, State},\n        {id, Data#raft_state.self#raft_identity.node},\n        {table, Data#raft_state.table},\n        {partition, Data#raft_state.partition},\n        {partition_path, Data#raft_state.partition_path},\n        {current_term, Data#raft_state.current_term},\n        {voted_for, Data#raft_state.voted_for},\n        {commit_index, Data#raft_state.commit_index},\n        {last_applied, Data#raft_state.last_applied},\n        {leader_id, Data#raft_state.leader_id},\n        {pending_high, length(Data#raft_state.pending_high)},\n        {pending_low, length(Data#raft_state.pending_low)},\n        {pending_read, Data#raft_state.pending_read},\n        {queued, maps:size(Data#raft_state.queued)},\n        {next_indices, Data#raft_state.next_indices},\n        {match_indices, Data#raft_state.match_indices},\n        {log_module, wa_raft_log:provider(Data#raft_state.log_view)},\n        {log_first, wa_raft_log:first_index(Data#raft_state.log_view)},\n        {log_last, wa_raft_log:last_index(Data#raft_state.log_view)},\n        {votes, Data#raft_state.votes},\n        {inflight_applies, wa_raft_queue:apply_queue_size(Data#raft_state.table, Data#raft_state.partition)},\n        {disable_reason, Data#raft_state.disable_reason},\n        {config, config(Data)},\n        {config_index, config_index(Data)},\n        {witness, is_self_witness(Data)}\n    ],\n    {keep_state_and_data, {reply, From, Status}};\n\n%% [Promote] Request full replica nodes to start a new election.\ncommand(\n    State,\n    {call, From},\n    ?TRIGGER_ELECTION_COMMAND(TermOrOffset),\n    #raft_state{\n        application = App,\n        current_term = CurrentTerm\n    } = Data\n) when State =/= stalled, State =/= witness, State =/= disabled ->\n    Term = case TermOrOffset of\n        current -> CurrentTerm;\n        next -> CurrentTerm + 1;\n        {next, Offset} -> CurrentTerm + Offset;\n        _ -> TermOrOffset\n    end,\n    case is_integer(Term) andalso Term >= CurrentTerm of\n        true ->\n            case ?RAFT_LEADER_ELIGIBLE(App) of\n                true ->\n                    ?SERVER_LOG_NOTICE(State, Data, \"switching to candidate after promotion request.\", []),\n                    NewState = case Term > CurrentTerm of\n                        true -> advance_term(State, Term, undefined, Data);\n                        false -> Data\n                    end,\n                    case State of\n                        candidate -> {repeat_state, NewState, {reply, From, ok}};\n                        _         -> {next_state, candidate, NewState, {reply, From, ok}}\n                    end;\n                false ->\n                    ?SERVER_LOG_WARNING(State, Data, \"cannot be promoted as candidate while ineligible.\", []),\n                    {keep_state_and_data, {reply, From, {error, ineligible}}}\n            end;\n        false ->\n            ?SERVER_LOG_WARNING(State, Data, \"refusing to promote to current, older, or invalid term ~0p.\", [Term]),\n            {keep_state_and_data, {reply, From, {error, rejected}}}\n    end;\n\n%% [Promote] Non-disabled nodes check if eligible to promote and then promote to leader.\ncommand(\n    State,\n    {call, From},\n    ?PROMOTE_COMMAND(TermOrOffset, Force),\n    #raft_state{\n        application = App,\n        table = Table,\n        current_term = CurrentTerm,\n        leader_heartbeat_ts = HeartbeatTs,\n        leader_id = LeaderId\n    } = Data\n) when State =/= stalled, State =/= witness, State =/= disabled ->\n    Now = erlang:monotonic_time(millisecond),\n    Eligible = ?RAFT_LEADER_ELIGIBLE(App),\n    HeartbeatGracePeriodMs = ?RAFT_PROMOTION_GRACE_PERIOD(App, Table) * 1000,\n    Term = case TermOrOffset of\n        current -> CurrentTerm;\n        next -> CurrentTerm + 1;\n        {next, Offset} -> CurrentTerm + Offset;\n        _ -> TermOrOffset\n    end,\n    Membership = get_config_members(config(Data)),\n    Allowed = if\n        % Prevent promotions to older or invalid terms\n        not is_integer(Term) orelse Term < CurrentTerm ->\n            ?SERVER_LOG_WARNING(State, Data, \"cannot attempt promotion to current, older, or invalid term ~0p.\", [Term]),\n            invalid_term;\n        Term =:= CurrentTerm andalso LeaderId =/= undefined ->\n            ?SERVER_LOG_WARNING(State, Data, \"refusing to promote to leader of current term already led by ~0p.\", [LeaderId]),\n            invalid_term;\n        % Prevent promotions that will immediately result in a resignation.\n        not Eligible ->\n            ?SERVER_LOG_WARNING(State, Data, \"cannot promote to leader as the node is ineligible.\", []),\n            ineligible;\n        State =:= witness ->\n            ?SERVER_LOG_WARNING(State, Data, \"cannot promote a witness node.\", []),\n            invalid_state;\n        % Prevent promotions to any operational state when there is no cluster membership configuration.\n        Membership =:= [] ->\n            ?SERVER_LOG_WARNING(State, Data, \"cannot promote to leader with no existing membership.\", []),\n            invalid_configuration;\n        Force ->\n            true;\n        HeartbeatTs =:= undefined ->\n            true;\n        Now - HeartbeatTs >= HeartbeatGracePeriodMs ->\n            true;\n        true ->\n            ?SERVER_LOG_WARNING(State, Data, \"rejecting request to promote to leader as a valid heartbeat was recently received.\", []),\n            rejected\n    end,\n    case Allowed of\n        true ->\n            ?SERVER_LOG_NOTICE(State, Data, \"is promoting to leader of term ~0p.\", [Term]),\n            NewState = case Term > CurrentTerm of\n                true -> advance_term(State, Term, node(), Data);\n                false -> Data\n            end,\n            case State of\n                leader -> {repeat_state, NewState, {reply, From, ok}};\n                _      -> {next_state, leader, NewState, {reply, From, ok}}\n            end;\n        Reason ->\n            {keep_state_and_data, {reply, From, {error, Reason}}}\n    end;\n\n%% [Resign] Non-leader nodes cannot resign.\ncommand(State, {call, From}, ?RESIGN_COMMAND, #raft_state{} = Data) when State =/= leader ->\n    ?SERVER_LOG_NOTICE(State, Data, \"not resigning because we are not leader.\", []),\n    {keep_state_and_data, {reply, From, {error, not_leader}}};\n\n%% [AdjustMembership] Non-leader nodes cannot adjust their config.\ncommand(\n    State,\n    Type,\n    ?ADJUST_CONFIG_COMMAND(From, Action, _),\n    #raft_state{queues = Queues} = Data\n) when State =/= leader ->\n    ?SERVER_LOG_NOTICE(State, Data, \"refusing to adjust config with action ~0p because we are not leader.\", [Action]),\n    From =/= undefined andalso\n        wa_raft_queue:commit_cancelled(Queues, From, {error, not_leader}, high),\n    reply(Type, {error, not_leader}),\n    {keep_state, Data};\n\n%% [Snapshot Available] Follower and candidate nodes might switch to stalled to install snapshot.\ncommand(\n    State,\n    Type,\n    ?SNAPSHOT_AVAILABLE_COMMAND(_, #raft_log_pos{index = SnapshotIndex}) = Event,\n    #raft_state{\n        last_applied = LastAppliedIndex\n    } = Data\n) when State =:= follower; State =:= candidate; State =:= witness ->\n    case SnapshotIndex > LastAppliedIndex of\n        true ->\n            ?SERVER_LOG_NOTICE(State, Data, \"at ~0p is notified of a newer snapshot at ~0p.\", [LastAppliedIndex, SnapshotIndex]),\n            {next_state, stalled, Data, {next_event, Type, Event}};\n        false ->\n            ?SERVER_LOG_NOTICE(State, Data, \"at ~0p is ignoring an older snapshot at ~0p.\", [LastAppliedIndex, SnapshotIndex]),\n            reply(Type, {error, rejected}),\n            keep_state_and_data\n    end;\n\n%% [Snapshot Available] Leader and disabled nodes should not install snapshots.\ncommand(\n    State,\n    Type,\n    ?SNAPSHOT_AVAILABLE_COMMAND(_, _),\n    #raft_state{}\n) when State =:= leader; State =:= disabled ->\n    reply(Type, {error, rejected}),\n    keep_state_and_data;\n\n%% [Handover Candidates] Non-leader nodes cannot serve handovers.\ncommand(State, {call, From}, ?HANDOVER_CANDIDATES_COMMAND, #raft_state{}) when State =/= leader ->\n    {keep_state_and_data, {reply, From, {error, not_leader}}};\n\n%% [Is Peer Ready] Non-leader nodes cannot check peer readiness.\ncommand(State, {call, From}, ?IS_PEER_READY_COMMAND(_), #raft_state{}) when State =/= leader ->\n    {keep_state_and_data, {reply, From, {error, not_leader}}};\n\n%% [Handover] Non-leader nodes cannot serve handovers.\ncommand(State, Type, ?HANDOVER_COMMAND(_), #raft_state{}) when State =/= leader ->\n    reply(Type, {error, not_leader}),\n    keep_state_and_data;\n\n%% [Enable] Non-disabled nodes are already enabled.\ncommand(State, {call, From}, ?ENABLE_COMMAND, #raft_state{}) when State =/= disabled ->\n    {keep_state_and_data, {reply, From, {error, already_enabled}}};\n\n%% [Disable] All nodes should disable by setting RAFT state disable_reason.\ncommand(\n    State,\n    {call, From},\n    ?DISABLE_COMMAND(Reason),\n    #raft_state{\n        self = ?IDENTITY_REQUIRES_MIGRATION(_, NodeId),\n        leader_id = LeaderId\n    } = Data0\n) ->\n    ?SERVER_LOG_NOTICE(State, Data0, \"disabling due to ~0p.\", [Reason]),\n    Data1 = Data0#raft_state{disable_reason = Reason},\n    Data2 = case NodeId =:= LeaderId of\n        true  -> clear_leader(State, Data1);\n        false -> Data1\n    end,\n    wa_raft_durable_state:store(Data2),\n    {next_state, disabled, Data2, {reply, From, ok}};\n\n%% [Bootstrap] Non-stalled nodes are already bootstrapped.\ncommand(_State, {call, From}, ?BOOTSTRAP_COMMAND(_Position, _Config, _Data), #raft_state{}) ->\n    {keep_state_and_data, {reply, From, {error, already_bootstrapped}}};\n\n%% [Fallback] Drop unknown command calls.\ncommand(State, Type, Event, #raft_state{} = Data) ->\n    ?SERVER_LOG_NOTICE(State, Data, \"dropping unhandled ~0p command ~0P\", [Type, Event, 20]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Cluster Configuration Helpers\n%%------------------------------------------------------------------------------\n\n%% Return whether or not the specified peer is a member of in the provided configuration.\n%% Raises an error if the membership list is missing or empty.\n-spec is_member(Peer :: peer(), Config :: config()) -> boolean().\nis_member(Peer, Config) ->\n    lists:member(Peer, config_membership(Config)).\n\n-spec is_self(Peer :: peer(), Data :: #raft_state{}) -> boolean().\nis_self({Name, Node}, #raft_state{self = #raft_identity{name = Name, node = Node}}) -> true;\nis_self(_, _) -> false.\n\n-spec is_self_member(Data :: #raft_state{}) -> boolean().\nis_self_member(#raft_state{self = #raft_identity{name = Name, node = Node}} = Data) ->\n    is_member({Name, Node}, config(Data)).\n\n-spec is_self_witness(Data :: #raft_state{}) -> boolean().\nis_self_witness(#raft_state{self = #raft_identity{name = Name, node = Node}} = Data) ->\n    lists:member({Name, Node}, config_witnesses(config(Data))).\n\n%% Returns true only if the membership of the current configuration contains exactly\n%% the provided peer and that the provided peer is not specified as a witness.\n-spec is_single_member(Peer :: #raft_identity{} | peer(), Config :: config()) -> IsSingleMember :: boolean().\nis_single_member(#raft_identity{name = Name, node = Node}, Config) ->\n    is_single_member({Name, Node}, Config);\nis_single_member(Peer, #{membership := Membership, witness := Witnesses}) ->\n    Membership =:= [Peer] andalso not lists:member(Peer, Witnesses).\n\n%% Get the non-empty participants list from the provided config. Falls back to the\n%% membership list if the participants list is missing or empty. Raises an error if\n%% both the participants and membership lists are missing or empty.\n-spec config_participants(Config :: config()) -> Participants :: membership().\nconfig_participants(#{participants := Participants}) when Participants =/= [] ->\n    Participants;\nconfig_participants(Config) ->\n    config_membership(Config).\n\n%% Get the non-empty membership list from the provided config. Raises an error\n%% if the membership list is missing or empty.\n-spec config_membership(Config :: config()) -> Membership :: membership().\nconfig_membership(#{membership := Membership}) when Membership =/= [] ->\n    Membership;\nconfig_membership(_) ->\n    error(membership_not_set).\n\n-spec config_witnesses(Config :: config()) -> Witnesses :: [peer()].\nconfig_witnesses(#{witness := Witnesses}) ->\n    Witnesses.\n\n-spec config_participant_identities(Config :: config()) -> Peers :: [#raft_identity{}].\nconfig_participant_identities(Config) ->\n    [#raft_identity{name = Name, node = Node} || {Name, Node} <- config_participants(Config)].\n\n-spec config_member_identities(Config :: config()) -> Peers :: [#raft_identity{}].\nconfig_member_identities(Config) ->\n    [#raft_identity{name = Name, node = Node} || {Name, Node} <- config_membership(Config)].\n\n-spec config_full_member_identities(Config :: config()) -> Replicas :: [#raft_identity{}].\nconfig_full_member_identities(Config) ->\n    FullMembers = config_membership(Config) -- config_witnesses(Config),\n    [#raft_identity{name = Name, node = Node} || {Name, Node} <- FullMembers].\n\n%% Returns the current effective RAFT configuration. This is the most recent configuration\n%% stored in either the RAFT log or the RAFT storage.\n-spec config(State :: #raft_state{}) -> Config :: config().\nconfig(#raft_state{log_view = View, cached_config = {ConfigIndex, Config}}) ->\n    case wa_raft_log:config(View) of\n        {ok, LogConfigIndex, LogConfig} when LogConfigIndex > ConfigIndex ->\n            LogConfig;\n        {ok, _, _} ->\n            % This case will normally only occur when the log has leftover log entries from\n            % previous incarnations of the RAFT server that have been applied but not yet\n            % trimmed this incarnation.\n            Config;\n        not_found ->\n            Config\n    end;\nconfig(#raft_state{log_view = View}) ->\n    case wa_raft_log:config(View) of\n        {ok, _, LogConfig} -> LogConfig;\n        not_found -> make_config()\n    end.\n\n-spec config_index(State :: #raft_state{}) -> ConfigIndex :: wa_raft_log:log_index().\nconfig_index(#raft_state{log_view = View, cached_config = {ConfigIndex, _}}) ->\n    case wa_raft_log:config(View) of\n        {ok, LogConfigIndex, _} ->\n            % The case where the log contains a config that is already applied generally\n            % only occurs after a restart as any log entries whose trim was deferred\n            % will become visible again.\n            max(LogConfigIndex, ConfigIndex);\n        not_found ->\n            ConfigIndex\n    end;\nconfig_index(#raft_state{log_view = View}) ->\n    case wa_raft_log:config(View) of\n        {ok, LogConfigIndex, _} -> LogConfigIndex;\n        not_found -> 0\n    end.\n\n%% Loads and caches the current configuration stored in the RAFT storage.\n%% This configuration is used whenever there is no newer configuration\n%% available in the RAFT log and so needs to be kept in sync with what\n%% the RAFT server expects is in storage.\n-spec load_config(State :: #raft_state{}) -> NewState :: #raft_state{}.\nload_config(#raft_state{storage = Storage, table = Table, partition = Partition} = State) ->\n    case wa_raft_storage:config(Storage) of\n        {ok, #raft_log_pos{index = ConfigIndex}, Config} ->\n            wa_raft_info:set_membership(Table, Partition, maps:get(membership, Config, [])),\n            State#raft_state{cached_config = {ConfigIndex, normalize_config(Config)}};\n        undefined ->\n            State#raft_state{cached_config = undefined};\n        {error, Reason} ->\n            error({could_not_load_config, Reason})\n    end.\n\n-spec load_label_state(State :: #raft_state{}) -> LabelState :: wa_raft_label:label().\nload_label_state(#raft_state{storage = Storage, label_module = LabelModule}) when LabelModule =/= undefined ->\n    case wa_raft_storage:label(Storage) of\n        {ok, Label} ->\n            Label;\n        {error, Reason} ->\n            error({failed_to_load_label_state, Reason})\n    end;\nload_label_state(_) ->\n    undefined.\n\n%% Add a new peer to the participants list of the provided cluster\n%% configuration.\n-spec config_add_participant(Peer :: peer(), Config :: config()) -> config().\nconfig_add_participant(Peer, Config) ->\n    Config#{\n        participants => lists:umerge([Peer], config_participants(Config))\n    }.\n\n%% Add a new peer to the participants and membership lists of the provided\n%% cluster configuration.\n-spec config_add_member(Peer :: peer(), Config :: config()) -> config().\nconfig_add_member(Peer, Config) ->\n    Config#{\n        participants => lists:umerge([Peer], config_participants(Config)),\n        membership => lists:umerge([Peer], config_membership(Config))\n    }.\n\n%% Add a new peer to the participants, membership, and witness lists of the\n%% provided cluster configuration.\n-spec config_add_witness(Peer :: peer(), Config :: config()) -> config().\nconfig_add_witness(Peer, Config) ->\n    Config#{\n        participants => lists:umerge([Peer], config_participants(Config)),\n        membership => lists:umerge([Peer], config_membership(Config)),\n        witness => lists:umerge([Peer], config_witnesses(Config))\n    }.\n\n%% Add a peer to just the witness list of the provided cluster configuration.\n-spec config_add_witness_only(Peer :: peer(), Config :: config()) -> config().\nconfig_add_witness_only(Peer, Config) ->\n    Config#{\n        witness => lists:umerge([Peer], config_witnesses(Config))\n    }.\n\n%% Remove a peer from the participants, membership, and witness lists of the\n%% provided cluster configuration.\n-spec config_remove_participant(Peer :: peer(), Config :: config()) -> config().\nconfig_remove_participant(Peer, Config) ->\n    Config#{\n        participants => config_participants(Config) -- [Peer],\n        membership => config_membership(Config) -- [Peer],\n        witness => config_witnesses(Config) -- [Peer]\n    }.\n\n%% Remove a peer from the membership list of the provided\n%% cluster configuration.\n-spec config_remove_member(Peer :: peer(), Config :: config()) -> config().\nconfig_remove_member(Peer, Config) ->\n    Config#{\n        membership => config_membership(Config) -- [Peer]\n    }.\n\n%% After an apply is sent to storage, check to see if it is a new configuration\n%% being applied. If it is, then update the cached configuration.\n-spec maybe_update_config(\n    Index :: wa_raft_log:log_index(),\n    Term :: wa_raft_log:log_term(),\n    Op :: wa_raft_acceptor:command(),\n    State :: #raft_state{}\n) -> NewState :: #raft_state{}.\nmaybe_update_config(Index, _, {config, Config}, #raft_state{table = Table, partition = Partition} = State) ->\n    wa_raft_info:set_membership(Table, Partition, maps:get(membership, Config, [])),\n    State#raft_state{cached_config = {Index, Config}};\nmaybe_update_config(_, _, _, #raft_state{} = State) ->\n    State.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Private Functions\n%%------------------------------------------------------------------------------\n\n-spec random_election_timeout(#raft_state{}) -> non_neg_integer().\nrandom_election_timeout(#raft_state{application = App, table = Table}) ->\n    Max = ?RAFT_ELECTION_TIMEOUT_MAX(App, Table),\n    Min = ?RAFT_ELECTION_TIMEOUT_MIN(App, Table),\n    Timeout =\n        case Max > Min of\n            true -> Min + rand:uniform(Max - Min);\n            false -> Min\n        end,\n    case ?RAFT_ELECTION_WEIGHT(App) of\n        Weight when Weight > 0, Weight =< ?RAFT_ELECTION_MAX_WEIGHT ->\n            % higher weight, shorter timeout so it has more chances to initiate an leader election\n            round(Timeout * ?RAFT_ELECTION_MAX_WEIGHT div Weight);\n        _ ->\n            Timeout * ?RAFT_ELECTION_DEFAULT_WEIGHT\n    end.\n\n-spec make_log_entry(Op :: wa_raft_acceptor:op(), Data :: #raft_state{}) -> {wa_raft_log:log_entry(), #raft_state{}}.\nmake_log_entry(Op, #raft_state{last_label = LastLabel} = Data) ->\n    {Entry, NewLabel} = make_log_entry_impl(Op, LastLabel, Data),\n    {Entry, Data#raft_state{last_label = NewLabel}}.\n\n-spec make_log_entry_impl(\n    Op :: wa_raft_acceptor:op(),\n    Data :: #raft_state{}\n) -> {wa_raft_log:log_entry(), wa_raft_label:label() | undefined}.\nmake_log_entry_impl(Op, #raft_state{last_label = LastLabel} = Data) ->\n    make_log_entry_impl(Op, LastLabel, Data).\n\n-spec make_log_entry_impl(\n    Op :: wa_raft_acceptor:op(),\n    LastLabel :: wa_raft_label:label() | undefined,\n    Data :: #raft_state{}\n) -> {wa_raft_log:log_entry(), wa_raft_label:label() | undefined}.\nmake_log_entry_impl({Key, Command}, LastLabel, #raft_state{label_module = undefined} = Data) ->\n    {make_log_entry_impl(Key, LastLabel, Command, Data), LastLabel};\nmake_log_entry_impl({Key, Command}, LastLabel, #raft_state{label_module = LabelModule} = Data) ->\n    NewLabel = case requires_new_label(Command) of\n        true -> LabelModule:new_label(LastLabel, Command);\n        false -> LastLabel\n    end,\n    {make_log_entry_impl(Key, NewLabel, Command, Data), NewLabel}.\n\n-spec make_log_entry_impl(\n    Key :: wa_raft_acceptor:key(),\n    Label :: wa_raft_label:label() | undefined,\n    Command :: wa_raft_acceptor:command(),\n    Data :: #raft_state{}\n) -> wa_raft_log:log_entry().\nmake_log_entry_impl(Key, undefined, Command, #raft_state{current_term = CurrentTerm}) ->\n    {CurrentTerm, {Key, Command}};\nmake_log_entry_impl(Key, Label, Command, #raft_state{current_term = CurrentTerm}) ->\n    {CurrentTerm, {Key, Label, Command}}.\n\n-spec requires_new_label(Command :: wa_raft_acceptor:command()) -> boolean().\nrequires_new_label(noop) -> false;\nrequires_new_label({config, _}) -> false;\nrequires_new_label(_) -> true.\n\n-spec apply_single_node_cluster(Data :: #raft_state{}) -> #raft_state{}.\napply_single_node_cluster(#raft_state{self = Self} = Data0) ->\n    case is_single_member(Self, config(Data0)) of\n        true ->\n            Data1 = commit_pending(Data0, high),\n            Data2 = commit_pending(Data1, low),\n            Data3 = maybe_advance(Data2),\n            apply_log_leader(Data3);\n        false ->\n            Data0\n    end.\n\n%% Leader - check quorum and advance commit index if possible\n-spec maybe_advance(Data :: #raft_state{}) -> #raft_state{}.\nmaybe_advance(\n    #raft_state{\n        table = Table,\n        log_view = View,\n        commit_index = CommitIndex,\n        match_indices = MatchIndices,\n        first_current_term_log_index = TermStartIndex\n    } = Data\n) ->\n    LogLast = wa_raft_log:last_index(View),\n    case max_index_to_apply(MatchIndices, LogLast, config(Data)) of\n        % Raft paper section 5.4.3 - Only log entries from the leader's current term are committed\n        % by counting replicas; once an entry from the current term has been committed in this way,\n        % then all prior entries are committed indirectly because of the View Matching Property\n        QuorumIndex when QuorumIndex < TermStartIndex ->\n            ?RAFT_COUNT(Table, 'apply.delay.old'),\n            ?SERVER_LOG_WARNING(leader, Data, \"cannot establish quorum at ~0p before start of term at ~0p.\",\n                [QuorumIndex, TermStartIndex]),\n            Data;\n        QuorumIndex when QuorumIndex > CommitIndex ->\n            Data#raft_state{commit_index = QuorumIndex};\n        _ ->\n            Data\n    end.\n\n% Return the max index to potentially apply on the leader. This is the latest log index that\n% has achieved replication on at least a quorum of nodes in the current RAFT cluster.\n% NOTE: We do not need to enforce that the leader should not commit entries from previous\n%       terms here (5.4.2) because we only update the CommitIndex broadcast by the leader\n%       when we actually apply the log entry on the leader. (See `apply_log` for information\n%       about how this rule is enforced there.)\n-spec max_index_to_apply(\n    MatchIndices :: #{node() => wa_raft_log:log_index()},\n    LastIndex :: wa_raft_log:log_index(),\n    Config :: config()\n) -> wa_raft_log:log_index().\nmax_index_to_apply(MatchIndices, LastIndex, Config) ->\n    compute_quorum(MatchIndices#{node() => LastIndex}, 0, Config).\n\n%% Create a new list with exactly one element for each member in the membership\n%% defined in the provided configuration taking the value mapped to each member in\n%% the provided map or a provided default if a pairing is not available.\n-spec to_member_list(\n    Mapping :: #{node() => Value},\n    Default :: Value,\n    Config :: config()\n) -> Normalized :: [Value].\nto_member_list(Mapping, Default, Config) ->\n    [maps:get(Node, Mapping, Default) || {_, Node} <- config_membership(Config)].\n\n%% Compute the quorum maximum value for the current membership given a config for\n%% the values represented by the given a mapping of peers (see note on config about\n%% RAFT RPC ids) to values assuming a default value for peers who are not represented\n%% in the mapping.\n-spec compute_quorum(\n    Mapping :: #{node() => Value},\n    Default :: Value,\n    Config :: config()\n ) -> Quorum :: Value.\ncompute_quorum(Mapping, Default, Config) ->\n    compute_quorum(to_member_list(Mapping, Default, Config)).\n\n%% Given a set of values $V$, compute the greatest $Q$ s.t. there exists\n%% at least a quorum of values $v_i \\in V$ for which $v_i \\ge Q$.\n-spec compute_quorum(Values :: [Value]) -> Quorum :: Value.\ncompute_quorum([_|_] = Values) ->\n    %% When taking element $k$ from a sorted list $|V| = n$, we know that all elements\n    %% of the sorted list $v_k ... v_n$ will be at least $v_k$. With $k = ceil(|V| / 2)$,\n    %% we can compute the quorum.\n    Index = (length(Values) + 1) div 2,\n    lists:nth(Index, lists:sort(Values)).\n\n-spec apply_log_leader(Data :: #raft_state{}) -> #raft_state{}.\napply_log_leader(\n    #raft_state{\n        last_applied = LastApplied,\n        match_indices = MatchIndices\n    } = Data\n) ->\n    TrimIndex = lists:min(to_member_list(MatchIndices#{node() => LastApplied}, 0, config(Data))),\n    apply_log(leader, TrimIndex, Data).\n\n-spec apply_log(\n    State :: state(),\n    TrimIndex :: wa_raft_log:log_index() | infinity,\n    Data :: #raft_state{}\n) -> #raft_state{}.\napply_log(\n    State,\n    TrimIndex,\n    #raft_state{\n        application = App,\n        table = Table,\n        queues = Queues,\n        log_view = View,\n        commit_index = CommitIndex,\n        last_applied = LastApplied\n    } = Data0\n) when CommitIndex > LastApplied ->\n    StartTUsec = erlang:monotonic_time(microsecond),\n    case wa_raft_queue:apply_queue_full(Queues) of\n        false ->\n            % Apply a limited number of log entries (both count and total byte size limited)\n            LimitedIndex = min(CommitIndex, LastApplied + ?RAFT_MAX_CONSECUTIVE_APPLY_ENTRIES(App, Table)),\n            LimitBytes = ?RAFT_MAX_CONSECUTIVE_APPLY_BYTES(App, Table),\n            {ok, {_, #raft_state{log_view = View1, last_applied = NewLastApplied} = Data1}} = wa_raft_log:fold(View, LastApplied + 1, LimitedIndex, LimitBytes,\n                fun (Index, Size, Entry, {Index, AccData}) ->\n                    wa_raft_queue:reserve_apply(Queues, Size),\n                    {Index + 1, apply_op(State, Index, Size, Entry, AccData)}\n                end, {LastApplied + 1, Data0}),\n\n            % Perform log trimming since we've now applied some log entries, only keeping\n            % at maximum MaxRotateDelay log entries.\n            MaxRotateDelay = ?RAFT_MAX_RETAINED_ENTRIES(App, Table),\n            RotateIndex = max(LimitedIndex - MaxRotateDelay, min(NewLastApplied, TrimIndex)),\n            RotateIndex =/= infinity orelse error(bad_state),\n            {ok, View2} = wa_raft_log:rotate(View1, RotateIndex),\n            Data2 = Data1#raft_state{log_view = View2},\n            ?RAFT_GATHER(Table, 'apply_log.latency_us', erlang:monotonic_time(microsecond) - StartTUsec),\n            Data2;\n        true ->\n            ApplyQueueSize = wa_raft_queue:apply_queue_size(Queues),\n            ?RAFT_COUNT(Table, 'apply.delay'),\n            ?RAFT_GATHER(Table, 'apply.queue', ApplyQueueSize),\n            ?RAFT_GATHER(Table, 'apply_log.latency_us', erlang:monotonic_time(microsecond) - StartTUsec),\n            Data0\n    end;\napply_log(_, _, #raft_state{} = Data) ->\n    Data.\n\n-spec apply_op(\n    State :: state(),\n    Index :: wa_raft_log:log_index(),\n    Size :: non_neg_integer(),\n    Entry :: wa_raft_log:log_entry() | undefined,\n    Data :: #raft_state{}\n) -> #raft_state{}.\napply_op(State, LogIndex, _, _, #raft_state{last_applied = LastApplied} = Data) when LogIndex =< LastApplied ->\n    ?SERVER_LOG_WARNING(State, Data, \"is skipping applying log entry ~0p because log entries up to ~0p are already applied.\", [LogIndex, LastApplied]),\n    Data;\napply_op(_, Index, Size, {Term, {Key, Command}}, #raft_state{} = Data) ->\n    Record = {Index, {Term, {Key, undefined, Command}}},\n    NewData = send_op_to_storage(Index, Record, Size, Data),\n    maybe_update_config(Index, Term, Command, NewData);\napply_op(_, Index, Size, {Term, {_, _, Command}} = Entry, #raft_state{} = Data) ->\n    Record = {Index, Entry},\n    NewData = send_op_to_storage(Index, Record, Size, Data),\n    maybe_update_config(Index, Term, Command, NewData);\napply_op(State, LogIndex, _, undefined, #raft_state{table = Table, log_view = View} = Data) ->\n    ?RAFT_COUNT(Table, 'server.missing.log.entry'),\n    ?SERVER_LOG_ERROR(State, Data, \"failed to apply ~0p because log entry is missing from log covering ~0p to ~0p.\",\n        [LogIndex, wa_raft_log:first_index(View), wa_raft_log:last_index(View)]),\n    exit({invalid_op, LogIndex});\napply_op(State, LogIndex, _, Entry, #raft_state{table = Table} = Data) ->\n    ?RAFT_COUNT(Table, 'server.corrupted.log.entry'),\n    ?SERVER_LOG_ERROR(State, Data, \"failed to apply unrecognized entry ~0P at ~0p.\", [Entry, 20, LogIndex]),\n    exit({invalid_op, LogIndex, Entry}).\n\n-spec send_op_to_storage(\n    Index :: wa_raft_log:log_index(),\n    Record :: wa_raft_log:log_record(),\n    Size :: non_neg_integer(),\n    Data :: #raft_state{}\n) -> #raft_state{}.\nsend_op_to_storage(Index, Record, Size, #raft_state{storage = Storage, queued = Queued} = Data) ->\n    {{From, Priority}, NewQueued} =\n        case maps:take(Index, Queued) of\n            error -> {{undefined, high}, Queued};\n            Value -> Value\n        end,\n    wa_raft_storage:apply(Storage, From, Record, Size, Priority),\n    Data#raft_state{last_applied = Index, queued = NewQueued}.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - State Management\n%%------------------------------------------------------------------------------\n\n-spec set_leader(State :: state(), Leader :: #raft_identity{}, Data :: #raft_state{}) -> #raft_state{}.\nset_leader(_, ?IDENTITY_REQUIRES_MIGRATION(_, Node), #raft_state{leader_id = Node} = Data) ->\n    Data;\nset_leader(\n    State,\n    ?IDENTITY_REQUIRES_MIGRATION(_, Node),\n    #raft_state{\n        table = Table,\n        partition = Partition,\n        storage = Storage,\n        current_term = CurrentTerm\n    } = Data\n) ->\n    ?SERVER_LOG_NOTICE(State, Data, \"changes leader to ~0p.\", [Node]),\n    wa_raft_info:set_current_term_and_leader(Table, Partition, CurrentTerm, Node),\n    NewData = Data#raft_state{leader_id = Node, pending_read = false},\n    wa_raft_storage:cancel(Storage),\n    cancel_pending({error, not_leader}, NewData).\n\n-spec clear_leader(state(), #raft_state{}) -> #raft_state{}.\nclear_leader(_, #raft_state{leader_id = undefined} = Data) ->\n    Data;\nclear_leader(State, #raft_state{table = Table, partition = Partition, current_term = CurrentTerm} = Data) ->\n    ?SERVER_LOG_NOTICE(State, Data, \"clears leader record.\", []),\n    wa_raft_info:set_current_term_and_leader(Table, Partition, CurrentTerm, undefined),\n    Data#raft_state{leader_id = undefined}.\n\n%% Setup the RAFT state upon entry into a new RAFT server state.\n-spec enter_state(State :: state(), Data :: #raft_state{}) -> #raft_state{}.\nenter_state(State, #raft_state{table = Table, partition = Partition} = Data0) ->\n    Now = erlang:monotonic_time(millisecond),\n    Data1 = Data0#raft_state{state_start_ts = Now},\n    true = wa_raft_info:set_state(Table, Partition, State),\n    ok = check_stale_upon_entry(State, Data1),\n    Data1.\n\n-spec check_stale_upon_entry(State :: state(), Data :: #raft_state{}) -> ok.\n%% Followers and candidates are live upon entry if they've received a timely heartbeat\n%% and inherit their staleness from the previous state.\ncheck_stale_upon_entry(State, Data) when State =:= follower; State =:= candidate ->\n    check_follower_liveness(State, Data);\n%% Witnesses are live upon entry if they've received a timely heartbeat but are always stale.\ncheck_stale_upon_entry(witness, #raft_state{table = Table, partition = Partition} = Data) ->\n    check_follower_liveness(witness, Data),\n    wa_raft_info:set_stale(Table, Partition, true),\n    ok;\n%% Leaders are always live and never stale upon entry.\ncheck_stale_upon_entry(leader, #raft_state{name = Name, table = Table, partition = Partition}) ->\n    wa_raft_info:set_live(Table, Partition, true),\n    wa_raft_info:set_stale(Table, Partition, false),\n    wa_raft_info:set_message_queue_length(Name),\n    ok;\n%% Stalled and disabled servers are never live and always stale.\ncheck_stale_upon_entry(_, #raft_state{name = Name, table = Table, partition = Partition}) ->\n    wa_raft_info:set_live(Table, Partition, false),\n    wa_raft_info:set_stale(Table, Partition, true),\n    wa_raft_info:set_message_queue_length(Name),\n    ok.\n\n%% Set a new current term and voted-for peer and clear any state that is associated with the previous term.\n-spec advance_term(\n    State :: state(),\n    NewTerm :: wa_raft_log:log_term(),\n    VotedFor :: undefined | node(),\n    Data :: #raft_state{}\n) -> #raft_state{}.\nadvance_term(\n    State,\n    NewTerm,\n    VotedFor,\n    #raft_state{\n        current_term = CurrentTerm\n    } = Data0\n) when NewTerm > CurrentTerm ->\n    Data1 = Data0#raft_state{\n        current_term = NewTerm,\n        voted_for = VotedFor,\n        votes = #{},\n        next_indices = #{},\n        match_indices = #{},\n        last_applied_indices = #{},\n        last_heartbeat_ts = #{},\n        heartbeat_response_ts = #{},\n        handover = undefined\n    },\n    Data2 = clear_leader(State, Data1),\n    ok = wa_raft_durable_state:store(Data2),\n    Data2.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Elections\n%%------------------------------------------------------------------------------\n\n%% Whether or not the local replica is eligible to maintain leadership.\n%% Generally, this requires that the replica is a member of cluster and is\n%% eligible to be leader.\n-spec leader_eligible(Data :: #raft_state{}) -> boolean().\nleader_eligible(#raft_state{application = App} = Data) ->\n    ?RAFT_LEADER_ELIGIBLE(App) andalso is_self_member(Data).\n\n%% Whether or not the local replica is eligible to start elections.\n%% Generally, this requires that the replica is a member of cluster, is\n%% eligible to be leader and has a non-zero election weight.\n-spec candidate_eligible(Data :: #raft_state{}) -> boolean().\ncandidate_eligible(#raft_state{application = App} = Data) ->\n    leader_eligible(Data) andalso ?RAFT_ELECTION_WEIGHT(App) =/= 0.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Leader Methods\n%%------------------------------------------------------------------------------\n\n-spec append_entries_to_followers(Data :: #raft_state{}) -> #raft_state{}.\nappend_entries_to_followers(Data0) ->\n    Data1 = commit_pending(Data0, high),\n    Data2 = commit_pending(Data1, low),\n    Data3 = lists:foldl(fun heartbeat/2, Data2, config_participant_identities(config(Data2))),\n    Data3.\n\n-spec commit_pending(Data :: #raft_state{}, Priority :: wa_raft_acceptor:priority()) -> #raft_state{}.\ncommit_pending(#raft_state{pending_high = [], pending_low = [], pending_read = false} = Data, _Priority) ->\n    Data;\ncommit_pending(#raft_state{table = Table, log_view = View, pending_high = PendingHigh, pending_low = PendingLow, queued = Queued} = Data, Priority) ->\n    Pending =\n        case Priority of\n            high ->\n                PendingHigh;\n            low ->\n                PendingLow\n        end,\n    {Entries, NewLabel} = collect_pending(Data, Priority),\n    case Entries of\n        [_ | _] ->\n            % If we have processed at least one new log entry\n            % with the given priority, we try to append to the log.\n            case wa_raft_log:try_append(View, Entries, Priority) of\n                {ok, NewView} ->\n                    % Add the newly appended log entries to the pending queue (which\n                    % is kept in reverse order).\n                    Last = wa_raft_log:last_index(NewView),\n                    {_, NewQueued} =\n                        lists:foldl(\n                            fun ({From, _Op}, {Index, AccQueued}) ->\n                                {Index - 1, AccQueued#{Index => {From, Priority}}}\n                            end,\n                            {Last, Queued},\n                            Pending\n                        ),\n                    % We can clear pending read flag as we've successfully added at\n                    % least one new log entry so the leader will proceed to replicate\n                    % and establish a new quorum is it is still up to date.\n                    Data1 = Data#raft_state{\n                        log_view = NewView,\n                        last_label = NewLabel,\n                        pending_read = false,\n                        queued = NewQueued\n                    },\n                    case Priority of\n                        high ->\n                            Data1#raft_state{pending_high = []};\n                        low ->\n                            Data1#raft_state{pending_low = []}\n                    end;\n                skipped ->\n                    % Since the append failed, we do not advance to the new label.\n                    % We cancel all pending commits that are less than or equal to the current priority.\n                    ?RAFT_COUNTV(Table, {'commit.skipped', Priority}, length(Entries)),\n                    ?SERVER_LOG_WARNING(leader, Data, \"skipped pre-heartbeat sync for ~0p log entr(ies).\", [length(Entries)]),\n                    cancel_pending({error, commit_stalled}, Data);\n                {error, Error} ->\n                    ?RAFT_COUNTV(Table, {'commit.error', Priority}, length(Entries)),\n                    ?SERVER_LOG_ERROR(leader, Data, \"sync failed due to ~0P.\", [Error, 20]),\n                    error(Error)\n            end;\n        _ ->\n            Data\n    end.\n\n-spec collect_pending(Data :: #raft_state{}, Priority :: wa_raft_acceptor:priority()) -> {[wa_raft_log:log_entry()], wa_raft_label:label() | undefined}.\ncollect_pending(#raft_state{pending_high = [], pending_low = [], pending_read = true} = Data, _Priority) ->\n    % If the pending queues are empty, then we have at least one pending\n    % read but no commits to go along with it.\n    {ReadEntry, NewLabel} = make_log_entry_impl({?READ_OP, noop}, Data),\n    {[ReadEntry], NewLabel};\ncollect_pending(#raft_state{last_label = LastLabel, pending_high = PendingHigh, pending_low = PendingLow} = Data, Priority) ->\n    % Otherwise, the pending queues are kept in reverse order so fold\n    % from the right to ensure that the log entries are labeled\n    % in the correct order.\n    Pending =\n        case Priority of\n            high ->\n                PendingHigh;\n            low ->\n                PendingLow\n        end,\n    {Entries, NewLabel} = lists:foldr(\n        fun ({_, Op}, {AccEntries, AccLabel}) ->\n            {Entry, NewAccLabel} = make_log_entry_impl(Op, AccLabel, Data),\n            {[Entry | AccEntries], NewAccLabel}\n        end,\n        {[], LastLabel},\n        Pending\n    ),\n    {lists:reverse(Entries), NewLabel}.\n\n-spec cancel_pending(Reason :: wa_raft_acceptor:commit_error(), Data :: #raft_state{}) -> #raft_state{}.\ncancel_pending(_, #raft_state{pending_high = [], pending_low = []} = Data) ->\n    Data;\ncancel_pending(Reason, #raft_state{queues = Queues, pending_high = PendingHigh, pending_low = PendingLow} = Data) ->\n    % Pending commits are kept in reverse order.\n    [wa_raft_queue:commit_cancelled(Queues, From, Reason, high) || {From, _Op} <- lists:reverse(PendingHigh)],\n    [wa_raft_queue:commit_cancelled(Queues, From, Reason, low) || {From, _Op} <- lists:reverse(PendingLow)],\n    Data#raft_state{pending_high = [], pending_low = []}.\n\n%% Cancel all pending and queued commits with the given error reason.\n-spec cancel_pending_and_queued(Reason :: wa_raft_acceptor:commit_error(), Data :: #raft_state{}) -> #raft_state{}.\ncancel_pending_and_queued(Reason, #raft_state{queues = Queues, queued = Queued} = Data) ->\n    NewData = cancel_pending(Reason, Data),\n    [wa_raft_queue:commit_cancelled(Queues, From, Reason, Priority) || _ := {From, Priority} <- maps:iterator(Queued, ordered)],\n    NewData#raft_state{queued = #{}}.\n\n-spec heartbeat(Peer :: #raft_identity{}, State :: #raft_state{}) -> #raft_state{}.\nheartbeat(Self, #raft_state{self = Self} = Data) ->\n    % Skip sending heartbeat to self\n    Data;\nheartbeat(\n    ?IDENTITY_REQUIRES_MIGRATION(_, FollowerId) = Sender,\n    #raft_state{\n        application = App,\n        table = Table,\n        name = Name,\n        log_view = View,\n        commit_index = CommitIndex,\n        next_indices = NextIndices,\n        match_indices = MatchIndices,\n        last_heartbeat_ts = LastHeartbeatTs,\n        first_current_term_log_index = TermStartIndex\n    } = State0\n) ->\n    FollowerNextIndex = maps:get(FollowerId, NextIndices, TermStartIndex),\n    PrevLogIndex = FollowerNextIndex - 1,\n    PrevLogTermRes = wa_raft_log:term(View, PrevLogIndex),\n    FollowerMatchIndex = maps:get(FollowerId, MatchIndices, 0),\n    FollowerMatchIndex =/= 0 andalso\n        ?RAFT_GATHER(Table, 'leader.follower.lag', CommitIndex - FollowerMatchIndex),\n    NowTs = erlang:monotonic_time(millisecond),\n    LastFollowerHeartbeatTs = maps:get(FollowerId, LastHeartbeatTs, undefined),\n    State1 = State0#raft_state{last_heartbeat_ts = LastHeartbeatTs#{FollowerId => NowTs}, leader_heartbeat_ts = NowTs},\n    LastIndex = wa_raft_log:last_index(View),\n    case PrevLogTermRes of %% no log entry to replicate\n        not_found ->\n            ?RAFT_COUNT(Table, 'leader.heartbeat.not_ready'),\n            ?SERVER_LOG_DEBUG(leader, State1, \"at ~0p sends empty heartbeat to follower ~0p at ~0p.\",\n                [CommitIndex, FollowerId, FollowerNextIndex]),\n            % Send append entries request.\n            {ok, LastTerm} = wa_raft_log:term(View, LastIndex),\n            send_rpc(Sender, ?APPEND_ENTRIES(LastIndex, LastTerm, [], CommitIndex, 0), State1),\n            LastFollowerHeartbeatTs =/= undefined andalso\n                ?RAFT_GATHER(Table, 'leader.heartbeat.interval_ms', erlang:monotonic_time(millisecond) - LastFollowerHeartbeatTs),\n            State1;\n        _ ->\n            MaxLogEntries = ?RAFT_HEARTBEAT_MAX_ENTRIES(App, Table),\n            MaxHeartbeatSize = ?RAFT_HEARTBEAT_MAX_BYTES(App, Table),\n            Witnesses = config_witnesses(config(State0)),\n            Entries = case lists:member({Name, FollowerId}, Witnesses) of\n                true ->\n                    {ok, RawEntries} = wa_raft_log:get(View, FollowerNextIndex, MaxLogEntries, MaxHeartbeatSize),\n                    stub_entries_for_witness(RawEntries);\n                false ->\n                    {ok, RawEntries} = wa_raft_log:entries(View, FollowerNextIndex, MaxLogEntries, MaxHeartbeatSize),\n                    RawEntries\n            end,\n            {ok, PrevLogTerm} = PrevLogTermRes,\n            % track when we send out a heartbeat that is empty but also not at the end of the log\n            Entries =:= [] andalso PrevLogIndex =/= LastIndex andalso ?RAFT_COUNT(Table, 'leader.heartbeat.empty'),\n            ?RAFT_GATHER(Table, 'leader.heartbeat.size', length(Entries)),\n            ?SERVER_LOG_DEBUG(leader, State1, \"at ~0p sends heartbeat to follower ~0p at ~0p with ~0p entr(ies).\",\n                [CommitIndex, FollowerId, FollowerNextIndex, length(Entries)]),\n            % Compute trim index.\n            TrimIndex = lists:min(to_member_list(MatchIndices#{node() => LastIndex}, 0, config(State1))),\n            % Send append entries request.\n            CastResult = send_rpc(Sender, ?APPEND_ENTRIES(PrevLogIndex, PrevLogTerm, Entries, CommitIndex, TrimIndex), State1),\n            NewNextIndices =\n                case CastResult of\n                    ok ->\n                        % pipelining - move NextIndex after sending out logs. If a packet is lost, follower's AppendEntriesResponse\n                        % will return send back its correct index\n                        NextIndices#{FollowerId => PrevLogIndex + length(Entries) + 1};\n                    _ ->\n                        NextIndices\n                end,\n            LastFollowerHeartbeatTs =/= undefined andalso\n                ?RAFT_GATHER(Table, 'leader.heartbeat.interval_ms', erlang:monotonic_time(millisecond) - LastFollowerHeartbeatTs),\n            State1#raft_state{next_indices = NewNextIndices}\n    end.\n\n-spec stub_entries_for_witness([wa_raft_log:log_entry() | binary()]) -> [wa_raft_log:log_entry() | binary()].\nstub_entries_for_witness(Entries) ->\n    [stub_entry(E) || E <- Entries].\n\n-spec stub_entry(wa_raft_log:log_entry() | binary()) -> wa_raft_log:log_entry() | binary().\nstub_entry(Binary) when is_binary(Binary) ->\n    Binary;\nstub_entry({Term, undefined}) ->\n    {Term, undefined};\nstub_entry({Term, {Key, Cmd}}) ->\n    {Term, {Key, stub_command(Cmd)}};\nstub_entry({Term, {Key, Label, Cmd}}) ->\n    {Term, {Key, Label, stub_command(Cmd)}}.\n\n-spec stub_command(wa_raft_acceptor:command()) -> wa_raft_acceptor:command().\nstub_command(noop) -> noop;\nstub_command({config, _} = ConfigCmd) -> ConfigCmd;\nstub_command(_) -> noop_omitted.\n\n-spec get_handover_eligibility_match_cutoff(State :: #raft_state{}) -> wa_raft_log:log_index().\nget_handover_eligibility_match_cutoff(#raft_state{application = App, table = Table, log_view = View}) ->\n    wa_raft_log:last_index(View) - ?RAFT_HANDOVER_MAX_ENTRIES(App, Table).\n\n-spec get_handover_eligibility_apply_cutoff(State :: #raft_state{}) -> wa_raft_log:log_index().\nget_handover_eligibility_apply_cutoff(#raft_state{application = App, table = Table, log_view = View}) ->\n    LastIndex = wa_raft_log:last_index(View),\n    case ?RAFT_HANDOVER_MAX_UNAPPLIED_ENTRIES(App, Table) of\n        undefined -> LastIndex - ?RAFT_HANDOVER_MAX_ENTRIES(App, Table);\n        Limit -> LastIndex - Limit\n    end.\n\n-spec get_handover_candidates(State :: #raft_state{}) -> [node()].\nget_handover_candidates(State) ->\n    Replicas = config_full_member_identities(config(State)),\n    MatchCutoffIndex = get_handover_eligibility_match_cutoff(State),\n    ApplyCutoffIndex = get_handover_eligibility_apply_cutoff(State),\n    [Peer || ?IDENTITY_REQUIRES_MIGRATION(_, Peer) <- Replicas,\n        Peer =/= node(),\n        is_eligible_for_handover_impl(Peer, MatchCutoffIndex, ApplyCutoffIndex, State)].\n\n-spec is_eligible_for_handover(Peer :: peer(), Data :: #raft_state{}) -> boolean().\nis_eligible_for_handover({_, Node}, Data) ->\n    MatchCutoffIndex = get_handover_eligibility_match_cutoff(Data),\n    ApplyCutoffIndex = get_handover_eligibility_apply_cutoff(Data),\n    is_eligible_for_handover_impl(Node, MatchCutoffIndex, ApplyCutoffIndex, Data).\n\n-spec is_eligible_for_handover_impl(\n    Node :: node(),\n    MatchCutoffIndex :: wa_raft_log:log_index(),\n    ApplyCutoffIndex :: wa_raft_log:log_index(),\n    Data :: #raft_state{}\n) -> boolean().\nis_eligible_for_handover_impl(\n    Node,\n    MatchCutoffIndex,\n    ApplyCutoffIndex,\n    #raft_state{\n        match_indices = MatchIndices,\n        last_applied_indices = LastAppliedIndices\n    }\n) ->\n    % A peer whose matching index or last applied index is unknown should not be eligible for handovers.\n    case {maps:find(Node, MatchIndices), maps:find(Node, LastAppliedIndices)} of\n        {{ok, MatchIndex}, {ok, LastAppliedIndex}} ->\n            MatchIndex >= MatchCutoffIndex andalso LastAppliedIndex >= ApplyCutoffIndex;\n        _ ->\n            false\n    end.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Configuration Changes\n%%------------------------------------------------------------------------------\n\n-spec leader_config_change_allowed(\n    Index :: wa_raft_log:log_index() | undefined,\n    State :: #raft_state{}\n) -> ok | {error, Reason :: term()}.\nleader_config_change_allowed(\n    Index,\n    #raft_state{\n        log_view = View,\n        commit_index = CommitIndex,\n        last_applied = LastApplied,\n        cached_config = {CachedConfigIndex, _},\n        first_current_term_log_index = TermStartIndex\n    } = State\n) ->\n    % A leader must establish quorum on at least one log entry in the current\n    % term because it is ready for a configuration change.\n    QuorumReady = CommitIndex >= TermStartIndex,\n    % No two configuration changes can be in the log at the same time\n    % and both be not yet committed.\n    ConfigIndex = case wa_raft_log:config(View) of\n        {ok, LogConfigIndex, _} -> erlang:max(CachedConfigIndex, LogConfigIndex);\n        _ -> CachedConfigIndex\n    end,\n    if\n        not QuorumReady ->\n            ?SERVER_LOG_NOTICE(\n                leader,\n                State,\n                \"at ~0p is not ready for a new configuration because it has not established a quorum in the current term.\",\n                [CommitIndex]\n            ),\n            {error, no_quorum};\n        ConfigIndex > LastApplied ->\n            ?SERVER_LOG_NOTICE(\n                leader,\n                State,\n                \"at ~0p is not ready for a new configuration because a new configuration is not yet committed at ~0p.\",\n                [CommitIndex, ConfigIndex]\n            ),\n            {error, not_ready};\n        Index =:= undefined ->\n            ok;\n        Index =/= ConfigIndex ->\n            ?SERVER_LOG_NOTICE(\n                leader,\n                State,\n                \"refuses a new configuration because the configuration at ~0p is not at the required ~0p\",\n                [ConfigIndex, Index]\n            ),\n            {error, config_index_mismatch};\n        true ->\n            ok\n    end.\n\n-spec leader_change_config(NewConfig :: wa_raft_server:config(), From :: undefined | gen_server:from(), State :: #raft_state{}) ->\n    {ok, NewConfigPosition :: wa_raft_log:log_pos(), NewState :: #raft_state{}} | {error, Reason :: term()}.\nleader_change_config(NewConfig, From, #raft_state{log_view = View, current_term = CurrentTerm} = State0) ->\n    {LogEntry, State1} = make_log_entry({make_ref(), {config, NewConfig}}, State0),\n    case wa_raft_log:try_append(View, [LogEntry]) of\n        {ok, NewView} ->\n            NewConfigIndex = wa_raft_log:last_index(NewView),\n            NewConfigPosition = #raft_log_pos{index = NewConfigIndex, term = CurrentTerm},\n            State2 = State1#raft_state{log_view = NewView},\n            % When initiated from the commit flow (via wa_raft_acceptor), From is\n            % non-undefined and needs to be stored in the queued map so that the\n            % storage can reply to the caller when the config change is applied.\n            State3 = case From of\n                undefined ->\n                    State2;\n                _ ->\n                    Queued = State2#raft_state.queued,\n                    State2#raft_state{queued = Queued#{NewConfigIndex => {From, high}}}\n            end,\n            {ok, NewConfigPosition, State3};\n        skipped ->\n            {error, commit_stalled};\n        {error, Reason} ->\n            {error, Reason}\n    end.\n\n-spec leader_adjust_config(Action :: config_action(), State :: #raft_state{}) ->\n    {ok, NewConfig :: config()} | {error, Reason :: atom()}.\nleader_adjust_config(refresh, Data) ->\n    {ok, config(Data)};\nleader_adjust_config({Action, Peer}, Data) ->\n    Config = config(Data),\n    IsSelf = is_self(Peer, Data),\n    IsReady = is_eligible_for_handover(Peer, Data),\n    IsMember = lists:member(Peer, config_membership(Config)),\n    IsWitness = lists:member(Peer, config_witnesses(Config)),\n    IsParticipant = lists:member(Peer, config_participants(Config)),\n    case Action of\n        add ->\n            if\n                IsMember -> {error, already_member};\n                IsWitness -> {error, already_witness};\n                true -> {ok, config_add_member(Peer, Config)}\n            end;\n        add_witness ->\n            if\n                IsMember andalso IsWitness -> {error, already_witness};\n                IsMember -> {error, already_member};\n                IsParticipant andalso not IsWitness -> {error, not_a_witness};\n                true -> {ok, config_add_witness(Peer, Config)}\n            end;\n        add_participant ->\n            if\n                IsMember -> {error, already_member};\n                IsWitness -> {error, already_witness};\n                IsParticipant -> {error, already_participating};\n                true -> {ok, config_add_participant(Peer, Config)}\n            end;\n        promote_participant_if_ready ->\n            if\n                IsMember -> {error, already_member};\n                IsWitness -> {error, already_witness};\n                not IsParticipant -> {error, not_a_participant};\n                not IsReady -> {error, not_ready};\n                true -> {ok, config_add_member(Peer, Config)}\n            end;\n        remove ->\n            if\n                IsSelf -> {error, cannot_remove_self};\n                not IsParticipant -> {error, not_a_participant};\n                true -> {ok, config_remove_participant(Peer, Config)}\n            end;\n        remove_witness ->\n            if\n                IsSelf -> {error, cannot_remove_self};\n                not IsWitness -> {error, not_a_witness};\n                true -> {ok, config_remove_participant(Peer, Config)}\n            end;\n        remove_membership ->\n            if\n                IsSelf -> {error, cannot_remove_self};\n                not IsMember -> {error, not_a_member};\n                true -> {ok, config_remove_member(Peer, Config)}\n            end;\n        demote_to_witness ->\n            if\n                IsSelf -> {error, cannot_demote_self};\n                not IsParticipant -> {error, not_a_participant};\n                IsWitness -> {error, already_witness};\n                true -> {ok, config_add_witness_only(Peer, Config)}\n            end\n    end.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Log State\n%%------------------------------------------------------------------------------\n\n-spec reset_log(Position :: wa_raft_log:log_pos(), Data :: #raft_state{}) -> #raft_state{}.\nreset_log(#raft_log_pos{index = Index} = Position, #raft_state{queues = Queues, log_view = View, queued = Queued} = Data) ->\n    {ok, NewView} = wa_raft_log:reset(View, Position),\n    [wa_raft_queue:commit_cancelled(Queues, From, {error, cancelled}, Priority) || _ := {From, Priority} <- maps:iterator(Queued, ordered)],\n    NewData = Data#raft_state{\n        log_view = NewView,\n        last_applied = Index,\n        commit_index = Index,\n        queued = #{}\n    },\n    load_config(NewData).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Snapshots\n%%------------------------------------------------------------------------------\n\n-spec open_snapshot(Root :: string(), Position :: wa_raft_log:log_pos(), Data :: #raft_state{}) -> #raft_state{}.\nopen_snapshot(Root, Position, #raft_state{storage = Storage} = Data) ->\n    ok = wa_raft_storage:open_snapshot(Storage, Root, Position),\n    reset_log(Position, Data).\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Heartbeat\n%%------------------------------------------------------------------------------\n\n%% Attempt to append the log entries declared by a leader in a heartbeat,\n%% apply committed but not yet applied log entries, trim the log, and reset\n%% the election timeout timer as necessary.\n-spec handle_heartbeat(\n    State :: state(),\n    Event :: gen_statem:event_type(),\n    Leader :: #raft_identity{},\n    PrevLogIndex :: wa_raft_log:log_index(),\n    PrevLogTerm :: wa_raft_log:log_term(),\n    Entries :: [wa_raft_log:log_entry() | binary()],\n    CommitIndex :: wa_raft_log:log_index(),\n    TrimIndex :: wa_raft_log:log_index(),\n    Data :: #raft_state{}\n) -> gen_statem:event_handler_result(state(), #raft_state{}).\n\nhandle_heartbeat(\n    State,\n    _Event,\n    Leader,\n    PrevLogIndex,\n    PrevLogTerm,\n    Entries,\n    LeaderCommitIndex,\n    TrimIndex,\n    #raft_state{\n        application = App,\n        table = Table,\n        queues = Queues,\n        log_view = View,\n        commit_index = CommitIndex,\n        last_applied = LastApplied\n    } = Data0\n) ->\n    EntryCount = length(Entries),\n\n    ?RAFT_GATHER(Table, {'heartbeat.size', State}, EntryCount),\n    EntryCount =/= 0 andalso\n        ?SERVER_LOG_DEBUG(State, Data0, \"considering appending ~0p log entries in range ~0p to ~0p to log ending at ~0p.\",\n            [EntryCount, PrevLogIndex + 1, PrevLogIndex + EntryCount, wa_raft_log:last_index(View)]),\n\n    case append_entries(State, PrevLogIndex, PrevLogTerm, Entries, EntryCount, Data0) of\n        {ok, Accepted, NewMatchIndex, Data1} ->\n            AdjustedLastApplied = max(0, LastApplied - wa_raft_queue:apply_queue_size(Queues)),\n            send_rpc(Leader, ?APPEND_ENTRIES_RESPONSE(PrevLogIndex, Accepted, NewMatchIndex, AdjustedLastApplied), Data1),\n\n            Data2 = Data1#raft_state{leader_heartbeat_ts = erlang:monotonic_time(millisecond)},\n            Data3 = case Accepted of\n                true ->\n                    LocalTrimIndex = case ?RAFT_LOG_ROTATION_BY_TRIM_INDEX(App, Table) of\n                        true  -> TrimIndex;\n                        false -> infinity\n                    end,\n                    NewCommitIndex = max(CommitIndex, min(LeaderCommitIndex, NewMatchIndex)),\n                    apply_log(State, LocalTrimIndex, Data2#raft_state{commit_index = NewCommitIndex});\n                _ ->\n                    Data2\n            end,\n            refresh_follower_liveness(State, Data3),\n            check_follower_lagging(State, CommitIndex, Data3),\n            case follower_or_witness_state(Data3) of\n                State ->\n                    {keep_state, Data3, ?ELECTION_TIMEOUT(Data3)};\n                NewState ->\n                    {next_state, NewState, Data3, ?ELECTION_TIMEOUT(Data3)}\n            end;\n        {fatal, Reason} ->\n            Data1 = Data0#raft_state{disable_reason = Reason},\n            wa_raft_durable_state:store(Data1),\n            {next_state, disabled, Data1}\n    end.\n\n%% Append the provided range of the log entries to the local log only if the\n%% term of the previous log matches the term stored by the local log,\n%% otherwise, truncate the log if the term does not match or do nothing if\n%% the previous log entry is not available locally. If an unrecoverable error\n%% is encountered, returns a diagnostic that can be used as a reason to\n%% disable the current replica.\n-spec append_entries(\n    State :: state(),\n    PrevLogIndex :: wa_raft_log:log_index(),\n    PrevLogTerm :: wa_raft_log:log_term(),\n    Entries :: [wa_raft_log:log_entry() | binary()],\n    EntryCount :: non_neg_integer(),\n    Data :: #raft_state{}\n) -> {ok, Accepted :: boolean(), NewMatchIndex :: wa_raft_log:log_index(), NewData :: #raft_state{}} | {fatal, Reason :: term()}.\n\nappend_entries(\n    State,\n    PrevLogIndex,\n    PrevLogTerm,\n    Entries,\n    EntryCount,\n    #raft_state{\n        log_view = View0,\n        table = Table,\n        queues = Queues,\n        commit_index = CommitIndex,\n        current_term = CurrentTerm,\n        leader_id = LeaderId,\n        queued = Queued\n    } = Data\n) ->\n    % Compare the incoming heartbeat with the local log to determine what\n    % actions need to be taken as part of handling this heartbeat.\n    case wa_raft_log:check_heartbeat(View0, PrevLogIndex, [{PrevLogTerm, undefined} | Entries]) of\n        {ok, []} ->\n            % No append is required as all the log entries in the heartbeat\n            % are already in the local log.\n            {ok, true, PrevLogIndex + EntryCount, Data};\n        {ok, NewEntries} ->\n            % No conflicting log entries were found in the heartbeat, but the\n            % heartbeat does contain new log entries to be appended to the end\n            % of the log.\n            case wa_raft_log:try_append(View0, NewEntries) of\n                {ok, View1} ->\n                    {ok, true, PrevLogIndex + EntryCount, Data#raft_state{log_view = View1}};\n                skipped ->\n                    NewCount = length(NewEntries),\n                    Last = wa_raft_log:last_index(View0),\n                    ?SERVER_LOG_WARNING(State, Data, \"is not ready to append ~0p log entries in range ~0p to ~0p to log ending at ~0p.\",\n                        [NewCount, Last + 1, Last + NewCount, Last]),\n                    {ok, false, Last, Data}\n            end;\n        {conflict, ConflictIndex, [ConflictEntry | _]} when ConflictIndex =< CommitIndex ->\n            % A conflict is detected that would result in the truncation of a\n            % log entry that the local replica has committed. We cannot validly\n            % delete log entries that are already committed because doing so\n            % may potenially cause the log entry to be no longer present on a\n            % majority of replicas.\n            {ok, LocalTerm} = wa_raft_log:term(View0, ConflictIndex),\n            {ConflictTerm, _} = if\n                is_binary(ConflictEntry) -> binary_to_term(ConflictEntry);\n                true -> ConflictEntry\n            end,\n            ?RAFT_COUNT(Table, {'heartbeat.error.corruption.excessive_truncation', State}),\n            ?SERVER_LOG_WARNING(State, Data, \"refuses heartbeat at ~0p to ~0p that requires truncation past ~0p (term ~0p vs ~0p) when log entries up to ~0p are already committed.\",\n                [PrevLogIndex, PrevLogIndex + EntryCount, ConflictIndex, ConflictTerm, LocalTerm, CommitIndex]),\n            Fatal = io_lib:format(\"A heartbeat at ~0p to ~0p from ~0p in term ~0p required truncating past ~0p (term ~0p vs ~0p) when log entries up to ~0p were already committed.\",\n                [PrevLogIndex, PrevLogIndex + EntryCount, LeaderId, CurrentTerm, ConflictIndex, ConflictTerm, LocalTerm, CommitIndex]),\n            {fatal, lists:flatten(Fatal)};\n        {conflict, ConflictIndex, NewEntries} when ConflictIndex >= PrevLogIndex ->\n            % A truncation is required as there is a conflict between the local\n            % log and the incoming heartbeat.\n            Last = wa_raft_log:last_index(View0),\n            ?SERVER_LOG_NOTICE(State, Data, \"handling heartbeat at ~0p by truncating local log ending at ~0p to past ~0p.\",\n                [PrevLogIndex, Last, ConflictIndex]),\n            case wa_raft_log:truncate(View0, ConflictIndex) of\n                {ok, View1} ->\n                    NewQueued = cancel_queued(Queues, ConflictIndex, Last, {error, not_leader}, Queued),\n                    case ConflictIndex =:= PrevLogIndex of\n                        true ->\n                            % If the conflict precedes the heartbeat's log\n                            % entries then no append can be performed.\n                            {ok, false, wa_raft_log:last_index(View1), Data#raft_state{log_view = View1, queued = NewQueued}};\n                        false ->\n                            % Otherwise, we can replace the truncated log\n                            % entries with those from the current heartbeat.\n                            case wa_raft_log:try_append(View1, NewEntries) of\n                                {ok, View2} ->\n                                    {ok, true, PrevLogIndex + EntryCount, Data#raft_state{log_view = View2, queued = NewQueued}};\n                                skipped ->\n                                    NewCount = length(NewEntries),\n                                    NewLast = wa_raft_log:last_index(View1),\n                                    ?SERVER_LOG_WARNING(State, Data, \"is not ready to append ~0p log entries in range ~0p to ~0p to log ending at ~0p.\",\n                                        [NewCount, NewLast + 1, NewLast + NewCount, NewLast]),\n                                    {ok, false, NewLast, Data}\n                            end\n                    end;\n                {error, Reason} ->\n                    ?RAFT_COUNT(Table, {'heartbeat.truncate.error', State}),\n                    ?SERVER_LOG_WARNING(State, Data, \"fails to truncate past ~0p while handling heartbeat at ~0p to ~0p due to ~0P\",\n                        [ConflictIndex, PrevLogIndex, PrevLogIndex + EntryCount, Reason, 30]),\n                    {ok, false, wa_raft_log:last_index(View0), Data}\n            end;\n        {invalid, out_of_range} ->\n            % If the heartbeat is out of range (generally past the end of the\n            % log) then ignore and notify the leader of what log entry is\n            % required by this replica.\n            ?RAFT_COUNT(Table, {'heartbeat.skip.out_of_range', State}),\n            EntryCount =/= 0 andalso\n                ?SERVER_LOG_WARNING(State, Data, \"refuses out of range heartbeat at ~0p to ~0p with local log covering ~0p to ~0p.\",\n                    [PrevLogIndex, PrevLogIndex + EntryCount, wa_raft_log:first_index(View0), wa_raft_log:last_index(View0)]),\n            {ok, false, wa_raft_log:last_index(View0), Data};\n        {error, Reason} ->\n            ?RAFT_COUNT(Table, {'heartbeat.skip.error', State}),\n            ?SERVER_LOG_WARNING(State, Data, \"fails to check heartbeat at ~0p to ~0p for validity due to ~0P\",\n                [PrevLogIndex, PrevLogIndex + EntryCount, Reason, 30]),\n            {ok, false, wa_raft_log:last_index(View0), Data}\n    end.\n\n-spec cancel_queued(\n    Queues :: wa_raft_queue:queues(),\n    Start :: wa_raft_log:log_index(),\n    End :: wa_raft_log:log_index(),\n    Reason :: wa_raft_acceptor:commit_error() | undefined,\n    Queued :: #{wa_raft_log:log_index() => {gen_server:from(), wa_raft_acceptor:priority()}}\n) -> #{wa_raft_log:log_index() => {gen_server:from(), wa_raft_acceptor:priority()}}.\ncancel_queued(_, Start, End, _, Queued) when Start > End; Queued =:= #{} ->\n    Queued;\ncancel_queued(Queues, Start, End, Reason, Queued) ->\n    case maps:take(Start, Queued) of\n        {{From, Priority}, NewQueued} ->\n            wa_raft_queue:commit_cancelled(Queues, From, Reason, Priority),\n            cancel_queued(Queues, Start + 1, End, Reason, NewQueued);\n        error ->\n            cancel_queued(Queues, Start + 1, End, Reason, Queued)\n    end.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Vote Requests\n%%------------------------------------------------------------------------------\n\n%% [RequestVote RPC]\n-spec request_vote_impl(\n    State :: state(),\n    Candidate :: #raft_identity{},\n    CandidateIndex :: wa_raft_log:log_index(),\n    CandidateTerm :: wa_raft_log:log_term(),\n    Data :: #raft_state{}\n) -> gen_statem:event_handler_result(state(), #raft_state{}).\n%% A replica with an available vote in the current term should allocate its vote\n%% if the candidate's log is at least as up-to-date as the local log. (5.4.1)\nrequest_vote_impl(\n    State,\n    ?IDENTITY_REQUIRES_MIGRATION(_, CandidateId) = Candidate,\n    CandidateIndex,\n    CandidateTerm,\n    #raft_state{\n        log_view = View,\n        voted_for = VotedFor\n    } = Data\n) when VotedFor =:= undefined; VotedFor =:= CandidateId ->\n    Index = wa_raft_log:last_index(View),\n    {ok, Term} = wa_raft_log:term(View, Index),\n    case {CandidateTerm, CandidateIndex} >= {Term, Index} of\n        true ->\n            ?SERVER_LOG_NOTICE(State, Data, \"decides to vote for candidate ~0p with up-to-date log at ~0p:~0p versus local log at ~0p:~0p.\",\n                [Candidate, CandidateIndex, CandidateTerm, Index, Term]),\n            case VotedFor of\n                undefined ->\n                    % If this vote request causes the current replica to allocate its vote, then\n                    % persist the vote before responding. (Fig. 2)\n                    NewData = Data#raft_state{voted_for = CandidateId},\n                    wa_raft_durable_state:store(NewData),\n                    send_rpc(Candidate, ?VOTE(true), NewData),\n                    {keep_state, NewData};\n                CandidateId ->\n                    % Otherwise, the vote allocation did not change, so just send the response.\n                    send_rpc(Candidate, ?VOTE(true), Data),\n                    keep_state_and_data\n            end;\n        false ->\n            ?SERVER_LOG_NOTICE(State, Data, \"refuses to vote for candidate ~0p with outdated log at ~0p:~0p versus local log at ~0p:~0p.\",\n                [Candidate, CandidateIndex, CandidateTerm, Index, Term]),\n            keep_state_and_data\n    end;\n%% A replica that was already allocated its vote to a specific candidate in the\n%% current term should ignore vote requests from other candidates. (5.4.1)\nrequest_vote_impl(State, Candidate, _, _,  #raft_state{voted_for = VotedFor} = Data) ->\n    ?SERVER_LOG_NOTICE(State, Data, \"refusing to vote for candidate ~0p after previously voting for candidate ~0p in the current term.\",\n        [Candidate, VotedFor]),\n    keep_state_and_data.\n\n%%------------------------------------------------------------------------------\n%% RAFT Server - State Machine Implementation - Helpers\n%%------------------------------------------------------------------------------\n\n%% Generic reply function for non-RPC requests that operates based on event type.\n-spec reply(Type :: enter | gen_statem:event_type(), Message :: term()) -> ok | {error, Reason :: term()}.\nreply(cast, _) ->\n    ok;\nreply({call, From}, Message) ->\n    gen_statem:reply(From, Message);\nreply(Type, Message) ->\n    ?RAFT_LOG_WARNING(\"Attempted to reply to non-reply event type ~0p with message ~0P.\", [Type, Message, 20]),\n    ok.\n\n-spec send_rpc(\n    Destination :: #raft_identity{},\n    ProcedureCall :: normalized_procedure(),\n    State :: #raft_state{}\n) -> ok | {error, term()}.\nsend_rpc(Destination, Procedure, #raft_state{self = Self, current_term = Term} = State) ->\n    cast(Destination, make_rpc(Self, Term, Procedure), State).\n\n-spec send_rpc_to_all_members(ProcedureCall :: normalized_procedure(), State :: #raft_state{}) -> [ok | {error, term()}].\nsend_rpc_to_all_members(ProcedureCall, #raft_state{self = Self} = State) ->\n    [send_rpc(Peer, ProcedureCall, State) || Peer <- config_member_identities(config(State)), Peer =/= Self].\n\n-spec cast(Destination :: #raft_identity{}, RPC :: rpc(), State :: #raft_state{}) -> ok | {error, term()}.\ncast(\n    #raft_identity{\n        name = Name,\n        node = Node\n    } = Destination,\n    Message,\n    #raft_state{\n        table = Table,\n        identifier = Identifier,\n        distribution_module = Distribution\n    }\n) ->\n    try\n        ok = Distribution:cast({Name, Node}, Identifier, Message)\n    catch\n        _:E ->\n            ?RAFT_COUNT(Table, {'server.cast.error', E}),\n            ?RAFT_LOG_DEBUG(\"Cast to ~p error ~100p\", [Destination, E]),\n            {error, E}\n    end.\n\n-spec maybe_heartbeat(#raft_state{}) -> #raft_state{}.\nmaybe_heartbeat(#raft_state{table = Table} = State) ->\n    case should_heartbeat(State) of\n        true ->\n            ?RAFT_COUNT(Table, 'leader.heartbeat'),\n            append_entries_to_followers(State);\n        false ->\n            State\n    end.\n\n-spec should_heartbeat(#raft_state{}) -> boolean().\nshould_heartbeat(#raft_state{handover = Handover}) when Handover =/= undefined ->\n    false;\nshould_heartbeat(#raft_state{application = App, table = Table, last_heartbeat_ts = LastHeartbeatTs}) ->\n    Latest = lists:max(maps:values(LastHeartbeatTs)),\n    Current = erlang:monotonic_time(millisecond),\n    Current - Latest > ?RAFT_HEARTBEAT_INTERVAL(App, Table).\n\n-spec refresh_follower_liveness(State :: state(), Data :: #raft_state{}) -> ok.\nrefresh_follower_liveness(State, #raft_state{table = Table, partition = Partition} = Data) ->\n    case wa_raft_info:get_live(Table, Partition) of\n        true ->\n            ok;\n        false ->\n            ?SERVER_LOG_NOTICE(State, Data, \"is live\", []),\n            wa_raft_info:set_live(Table, Partition, true)\n    end,\n    ok.\n\n%% Check the timestamp of a follower or witness's last received heartbeat to determine if\n%% the replica is live.\n-spec check_follower_liveness(State :: state(), Data :: #raft_state{}) -> ok.\ncheck_follower_liveness(\n    State,\n    #raft_state{\n        application = App,\n        name = Name,\n        table = Table,\n        partition = Partition,\n        leader_heartbeat_ts = LeaderHeartbeatTs\n    } = Data\n) ->\n    NowTs = erlang:monotonic_time(millisecond),\n    GracePeriod = ?RAFT_LIVENESS_GRACE_PERIOD_MS(App, Table),\n    Liveness = wa_raft_info:get_live(Table, Partition),\n    Live = LeaderHeartbeatTs =/= undefined andalso LeaderHeartbeatTs + GracePeriod >= NowTs,\n    case Live of\n        Liveness ->\n            ok;\n        true ->\n            ?SERVER_LOG_NOTICE(State, Data, \"is live\", []),\n            wa_raft_info:set_live(Table, Partition, true);\n        false ->\n            ?SERVER_LOG_NOTICE(State, Data, \"is no longer live after last leader heartbeat at ~0p\", [LeaderHeartbeatTs]),\n            wa_raft_info:set_live(Table, Partition, false),\n            wa_raft_info:get_stale(Table, Partition) orelse begin\n                ?SERVER_LOG_NOTICE(State, Data, \"is now stale due to liveness\", []),\n                wa_raft_info:set_stale(Table, Partition, true)\n            end\n    end,\n    wa_raft_info:set_message_queue_length(Name),\n    ok.\n\n%% Check the state of a follower or witness's last applied log entry versus the leader's\n%% commit index to determine if the replica is lagging behind and adjust the partition's\n%% staleness if needed.\n-spec check_follower_lagging(State :: state(), LeaderCommit :: pos_integer(), State :: #raft_state{}) -> ok.\ncheck_follower_lagging(\n    State,\n    LeaderCommit,\n    #raft_state{\n        application = App,\n        name = Name,\n        table = Table,\n        partition = Partition,\n        last_applied = LastApplied\n    } = Data\n) ->\n    Lagging = LeaderCommit - LastApplied,\n    ?RAFT_GATHER(Table, 'follower.lagging', Lagging),\n\n    % Witnesses are always considered stale and so do not re-check their staleness.\n    State =/= witness andalso begin\n        Stale = wa_raft_info:get_stale(Table, Partition),\n        case Lagging >= ?RAFT_STALE_GRACE_PERIOD_ENTRIES(App, Table) of\n            Stale ->\n                ok;\n            true ->\n                ?SERVER_LOG_NOTICE(State, Data, \"last applied at ~0p is ~0p behind leader's commit at ~0p.\",\n                    [LastApplied, Lagging, LeaderCommit]),\n                wa_raft_info:set_stale(Table, Partition, true);\n            false ->\n                ?SERVER_LOG_NOTICE(State, Data, \"catches up.\", []),\n                wa_raft_info:set_stale(Table, Partition, false)\n        end\n    end,\n    wa_raft_info:set_message_queue_length(Name),\n\n    ok.\n\n%% As leader, compute the quorum of the most recent timestamps of follower's\n%% acknowledgement of heartbeats and update the partition's staleness and\n%% liveness when necessary.\n-spec check_leader_liveness(#raft_state{}) -> true.\ncheck_leader_liveness(\n    #raft_state{\n        application = App,\n        name = Name,\n        table = Table,\n        partition = Partition,\n        heartbeat_response_ts = HeartbeatResponse\n    } = State\n) ->\n    NowTs = erlang:monotonic_time(millisecond),\n    QuorumTs = compute_quorum(HeartbeatResponse#{node() => NowTs}, 0, config(State)),\n\n    % If the quorum of the most recent timestamps of follower's acknowledgement\n    % of heartbeats is too old, then the leader is considered stale.\n    QuorumAge = NowTs - QuorumTs,\n    MaxAge = ?RAFT_LEADER_STALE_INTERVAL(App, Table),\n    ShouldBeStale = QuorumAge >= MaxAge,\n    ShouldBeLive = not ShouldBeStale,\n\n    % Update liveness if necessary\n    wa_raft_info:get_live(Table, Partition) =/= ShouldBeLive andalso\n        wa_raft_info:set_live(Table, Partition, ShouldBeLive),\n\n    % Update staleness if necessary\n    Stale = wa_raft_info:get_stale(Table, Partition),\n    case ShouldBeStale of\n        Stale ->\n            ok;\n        true ->\n            ?SERVER_LOG_NOTICE(leader, State, \"is now stale due to last heartbeat quorum age being ~0p ms >= ~0p ms max\", [QuorumAge, MaxAge]),\n            wa_raft_info:set_stale(Table, Partition, ShouldBeStale);\n        false ->\n            ?SERVER_LOG_NOTICE(leader, State, \"is no longer stale after heartbeat quorum age drops to ~0p ms < ~0p ms max\", [QuorumAge, MaxAge]),\n            wa_raft_info:set_stale(Table, Partition, ShouldBeStale)\n    end,\n\n    % Update message queue length\n    wa_raft_info:set_message_queue_length(Name).\n\n%% Check if the leader should send a storage snapshot to this follower and request\n%% if necessary.\n-spec leader_maybe_request_snapshot(\n    Follower :: node(),\n    FollowerLastIndex :: wa_raft_log:log_index(),\n    FollowerLastAppliedIndex :: wa_raft_log:log_index() | undefined,\n    State :: #raft_state{}\n) -> ok.\nleader_maybe_request_snapshot(\n    Follower,\n    FollowerLastIndex,\n    FollowerLastAppliedIndex,\n    #raft_state{log_view = View} = State\n) ->\n    FirstIndex = wa_raft_log:first_index(View),\n    leader_should_send_snapshot(FirstIndex, FollowerLastIndex, FollowerLastAppliedIndex, State) andalso\n        request_snapshot_for_follower(Follower, State),\n    ok.\n\n%% Leaders should send a storage snapshot to a follower instead of replicating\n%% when the follower:\n%%  * is stalled or otherwise missing their entire log,\n%%  * requires a log entry that is already rotated out, or\n%%  * is very far behind (past the snapshot catchup threshold).\n-spec leader_should_send_snapshot(\n    FirstIndex :: wa_raft_log:log_index(),\n    FollowerLastIndex :: wa_raft_log:log_index(),\n    FollowerLastAppliedIndex :: wa_raft_log:log_index() | undefined,\n    State :: #raft_state{}\n) -> boolean().\nleader_should_send_snapshot(_, 0, _, _) ->\n    true;\nleader_should_send_snapshot(FirstIndex, FollowerLastIndex, _, _) when FirstIndex > FollowerLastIndex ->\n    true;\nleader_should_send_snapshot(_, _, undefined, _) ->\n    false;\nleader_should_send_snapshot(_, FollowerLastIndex, FollowerLastAppliedIndex, #raft_state{application = App, table = Table}) ->\n    SnapshotCatchupThreshold = ?RAFT_SNAPSHOT_CATCHUP_THRESHOLD(App, Table),\n    FollowerLastIndex - FollowerLastAppliedIndex > SnapshotCatchupThreshold.\n\n%% Try to start a snapshot transport to a follower if the snapshot transport\n%% service is available. If the follower is a witness or too many snapshot\n%% transports have been started then no transport is created. This function\n%% always performs this request asynchronously.\n-spec request_snapshot_for_follower(Follower :: node(), State :: #raft_state{}) -> ok.\nrequest_snapshot_for_follower(\n    FollowerId,\n    #raft_state{\n        application = App,\n        name = Name,\n        table = Table,\n        partition = Partition\n    } = State\n) ->\n    Witness = lists:member({Name, FollowerId}, config_witnesses(config(State))),\n    wa_raft_snapshot_catchup:catchup(App, Name, FollowerId, Table, Partition, Witness).\n\n-spec follower_or_witness_state(State :: #raft_state{}) -> state().\nfollower_or_witness_state(State) ->\n    case is_self_witness(State) of\n        true -> witness;\n        false -> follower\n    end.\n\n-spec candidate_or_witness_state_transition(State :: #raft_state{}) ->\n    gen_statem:event_handler_result(state(), #raft_state{}).\ncandidate_or_witness_state_transition(#raft_state{current_term = CurrentTerm} = State) ->\n    case is_self_witness(State) of\n        true -> {next_state, witness, State};\n        false -> {next_state, candidate, State, {next_event, internal, ?FORCE_ELECTION(CurrentTerm)}}\n    end.\n"
  },
  {
    "path": "src/wa_raft_snapshot_catchup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% This module manages requests to trigger snapshot catchup across all\n%%% local RAFT partitions.\n\n-module(wa_raft_snapshot_catchup).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%% Supervisor callbacks\n-export([\n    child_spec/0,\n    start_link/0\n]).\n\n%% Public API\n-export([\n    which_transports/0\n]).\n\n%% Internal API\n-export([\n    catchup/6\n]).\n\n%% Snapshot catchup server implementation\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    handle_info/2,\n    terminate/2\n]).\n\n%% Testing API\n-export([\n    init_tables/0\n]).\n\n-define(SCAN_EVERY_MS, 500).\n\n-define(PENDING_KEY(Name, Node), {pending, Name, Node}).\n\n-define(WHICH_TRANSPORTS, which_transports).\n-define(CATCHUP(App, Name, Node, Table, Partition, Witness), {catchup, App, Name, Node, Table, Partition, Witness}).\n\n-type key() :: {Name :: atom(), Node :: node()}.\n-type snapshot_key() :: {Table :: wa_raft:table(), Partition :: wa_raft:partition(), Position :: wa_raft_log:log_pos(), Witness :: boolean()}.\n\n-type which_transports() :: ?WHICH_TRANSPORTS.\n-type call() :: which_transports().\n\n-type catchup() :: ?CATCHUP(atom(), atom(), node(), wa_raft:table(), wa_raft:partition(), boolean()).\n-type cast() :: catchup().\n\n-record(transport, {\n    app :: atom(),\n    table :: wa_raft:table(),\n    partition :: wa_raft:partition(),\n    id :: wa_raft_transport:transport_id(),\n    snapshot :: {wa_raft_log:log_pos(), Witness :: boolean()}\n}).\n-record(state, {\n    % currently active transports\n    transports = #{} :: #{key() => #transport{}},\n    % counts of active transports that are using a particular snapshot\n    snapshots = #{} :: #{snapshot_key() => pos_integer()},\n    % timestamps (ms) after which transports to previously overloaded nodes can be retried\n    overload_backoffs = #{} :: #{node() => integer()},\n    % timestamps (ms) after which repeat transports can be retried\n    retry_backoffs = #{} :: #{key() => integer()}\n}).\n\n-spec child_spec() -> supervisor:child_spec().\nchild_spec() ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, []},\n        restart => permanent,\n        shutdown => 30000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link() -> supervisor:startlink_ret().\nstart_link() ->\n    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).\n\n-spec which_transports() -> [wa_raft_transport:transport_id()].\nwhich_transports() ->\n    gen_server:call(?MODULE, ?WHICH_TRANSPORTS).\n\n-spec catchup(\n    App :: atom(),\n    Name :: atom(),\n    Node :: node(),\n    Table :: wa_raft:table(),\n    Partition :: wa_raft:partition(),\n    Witness :: boolean()\n) -> ok.\ncatchup(App, Name, Node, Table, Partition, Witness) ->\n    try\n        % Check ETS to avoid putting duplicate requests into the message queue.\n        ets:insert_new(?MODULE, {?PENDING_KEY(Name, Node)}) andalso\n            gen_server:cast(?MODULE, ?CATCHUP(App, Name, Node, Table, Partition, Witness)),\n        ok\n    catch\n        error:badarg ->\n            ok\n    end.\n\n-spec init(Args :: term()) -> {ok, #state{}}.\ninit([]) ->\n    process_flag(trap_exit, true),\n    init_tables(),\n    schedule_scan(),\n    {ok, #state{}}.\n\n-spec init_tables() -> ok.\ninit_tables() ->\n    ?MODULE = ets:new(?MODULE, [set, public, named_table]),\n    ok.\n\n-spec handle_call(Request :: call(), From :: gen_server:from(), State :: #state{}) -> {noreply, #state{}} | {reply, term(), #state{}}.\nhandle_call(?WHICH_TRANSPORTS, _From, #state{transports = Transports} = State) ->\n    {reply, [ID || _ := #transport{id = ID} <- Transports], State};\nhandle_call(Request, From, #state{} = State) ->\n    ?RAFT_LOG_NOTICE(\"received unrecognized call ~P from ~0p\", [Request, 25, From]),\n    {noreply, State}.\n\n-spec handle_cast(Request :: cast(), State :: #state{}) -> {noreply, #state{}}.\nhandle_cast(?CATCHUP(App, Name, Node, Table, Partition, Witness), State0) ->\n    % Just immediately remove the pending key from the ETS. Doing this here is simpler\n    % but permits a bounded number of extra requests to remain in the queue.\n    ets:delete(?MODULE, ?PENDING_KEY(Name, Node)),\n    Now = erlang:monotonic_time(millisecond),\n    case allowed(Now, Name, Node, State0) of\n        {true, #state{transports = Transports, snapshots = Snapshots, overload_backoffs = OverloadBackoffs} = State1} ->\n            try\n                {#raft_log_pos{index = Index, term = Term} = LogPos, Path} = create_snapshot(Table, Partition, Witness),\n                case wa_raft_transport:start_snapshot_transfer(Node, Table, Partition, LogPos, Path, Witness, infinity) of\n                    {error, receiver_overloaded} ->\n                        ?RAFT_LOG_NOTICE(\n                            \"destination node ~0p is overloaded, abort new transport for ~0p:~0p and try again later\",\n                            [Node, Table, Partition]\n                        ),\n                        NewOverloadBackoff = Now + ?RAFT_SNAPSHOT_CATCHUP_OVERLOADED_BACKOFF_MS(App, Table),\n                        NewOverloadBackoffs = OverloadBackoffs#{Node => NewOverloadBackoff},\n                        {noreply, State1#state{overload_backoffs = NewOverloadBackoffs}};\n                    {ok, ID} ->\n                        ?RAFT_LOG_NOTICE(\n                            \"started sending snapshot for ~0p:~0p at ~0p:~0p over transport ~0p\",\n                            [Table, Partition, Index, Term, ID]\n                        ),\n                        NewTransport = #transport{\n                            app = App,\n                            table = Table,\n                            partition = Partition,\n                            id = ID,\n                            snapshot = {LogPos, Witness}\n                        },\n                        NewTransports = Transports#{{Name, Node} => NewTransport},\n                        NewSnapshots = maps:update_with({Table, Partition, LogPos, Witness}, fun(V) -> V + 1 end, 1, Snapshots),\n                        {noreply, State1#state{transports = NewTransports, snapshots = NewSnapshots}}\n                end\n            catch\n                _T:_E:S ->\n                    ?RAFT_LOG_ERROR(\n                        \"failed to start accepted snapshot transport of ~0p:~0p to ~0p at ~p\",\n                        [Table, Partition, Node, S]\n                    ),\n                    {noreply, State1}\n            end;\n        {false, State1} ->\n            {noreply, State1}\n    end;\nhandle_cast(Request, #state{} = State) ->\n    ?RAFT_LOG_NOTICE(\"received unrecognized cast ~P\", [Request, 25]),\n    {noreply, State}.\n\n-spec handle_info(term(), #state{}) -> {noreply, #state{}}.\nhandle_info(scan, #state{transports = Transports} = State) ->\n    NewState = maps:fold(fun scan_transport/3, State, Transports),\n    schedule_scan(),\n    {noreply, NewState};\nhandle_info(Info, #state{} = State) ->\n    ?RAFT_LOG_NOTICE(\"received unrecognized info ~P\", [Info, 25]),\n    {noreply, State}.\n\n-spec terminate(Reason :: dynamic(), #state{}) -> ok.\nterminate(_Reason, #state{transports = Transports, snapshots = Snapshots}) ->\n    [\n        wa_raft_transport:cancel(ID, terminating)\n     || {_Name, _Node} := #transport{id = ID} <- Transports\n    ],\n    [\n        delete_snapshot(Table, Partition, LogPos, Witness)\n     || {Table, Partition, LogPos, Witness} := _ <- Snapshots\n    ],\n    ok.\n\n-spec allowed(Now :: integer(), Name :: atom(), Node :: node(), State :: #state{}) -> {boolean(), #state{}}.\nallowed(Now, Name, Node, #state{transports = Transports, overload_backoffs = OverloadBackoffs, retry_backoffs = RetryBackoffs} = State0) ->\n    Key = {Name, Node},\n    Limited = maps:size(Transports) >= ?RAFT_MAX_CONCURRENT_SNAPSHOT_CATCHUP(),\n    Exists = maps:is_key(Key, Transports),\n    Overloaded = maps:get(Node, OverloadBackoffs, Now) > Now,\n    Blocked = maps:get(Key, RetryBackoffs, Now) > Now,\n    Allowed = not (Limited orelse Exists orelse Overloaded orelse Blocked),\n    State1 = case Overloaded of\n        true -> State0;\n        false -> State0#state{overload_backoffs = maps:remove(Node, OverloadBackoffs)}\n    end,\n    State2 = case Blocked of\n        true -> State1;\n        false -> State1#state{retry_backoffs = maps:remove(Key, RetryBackoffs)}\n    end,\n    {Allowed, State2}.\n\n-spec scan_transport(Key :: key(), Transport :: #transport{}, #state{}) -> #state{}.\nscan_transport(Key, #transport{app = App, table = Table, id = ID} = Transport, State) ->\n    Status = case wa_raft_transport:transport_info(ID) of\n        {ok, #{status := S}} -> S;\n        _                    -> undefined\n    end,\n    case Status of\n        requested ->\n            State;\n        running ->\n            State;\n        completed ->\n            finish_transport(Key, Transport, ?RAFT_SNAPSHOT_CATCHUP_COMPLETED_BACKOFF_MS(App, Table), State);\n        _Other ->\n            finish_transport(Key, Transport, ?RAFT_SNAPSHOT_CATCHUP_FAILED_BACKOFF_MS(App, Table), State)\n    end.\n\n-spec finish_transport(Key :: key(), Transport :: #transport{}, Backoff :: pos_integer(), State :: #state{}) -> #state{}.\nfinish_transport(Key, #transport{table = Table, partition = Partition, snapshot = {LogPos, Witness}}, Backoff,\n                 #state{transports = Transports, snapshots = Snapshots, retry_backoffs = RetryBackoffs} = State) ->\n    Now = erlang:monotonic_time(millisecond),\n    SnapshotKey = {Table, Partition, LogPos, Witness},\n    NewSnapshots = case Snapshots of\n        #{SnapshotKey := 1} ->\n            % try to delete a snapshot if it is the last transport using it\n            delete_snapshot(Table, Partition, LogPos, Witness),\n            maps:remove(SnapshotKey, Snapshots);\n        #{SnapshotKey := Count} ->\n            % otherwise decrement the reference count for the snapshot\n            Snapshots#{SnapshotKey => Count - 1};\n        #{} ->\n            % unexpected that the snapshot is missing, but just ignore\n            Snapshots\n    end,\n    NewRetryBackoffs = RetryBackoffs#{Key => Now + Backoff},\n    State#state{transports = maps:remove(Key, Transports), snapshots = NewSnapshots, retry_backoffs = NewRetryBackoffs}.\n\n-spec delete_snapshot(Table :: wa_raft:table(), Partition :: wa_raft:partition(),\n                      Position :: wa_raft_log:log_pos(), Witness :: boolean()) -> ok.\ndelete_snapshot(Table, Partition, Position, Witness) ->\n    Storage = wa_raft_storage:registered_name(Table, Partition),\n    wa_raft_storage:delete_snapshot(Storage, snapshot_name(Position, Witness)).\n\n-spec schedule_scan() -> reference().\nschedule_scan() ->\n    erlang:send_after(?SCAN_EVERY_MS, self(), scan).\n\n-spec snapshot_name(LogPos :: wa_raft_log:log_pos(), Witness :: boolean()) -> string().\nsnapshot_name(#raft_log_pos{index = Index, term = Term}, false) ->\n    ?SNAPSHOT_NAME(Index, Term);\nsnapshot_name(#raft_log_pos{index = Index, term = Term}, true) ->\n    ?WITNESS_SNAPSHOT_NAME(Index, Term).\n\n-spec create_snapshot(\n    Table :: wa_raft:table(),\n    Partition :: wa_raft:partition(),\n    Witness :: boolean()\n) -> {LogPos :: wa_raft_log:log_pos(), Path :: string()}.\ncreate_snapshot(Table, Partition, Witness) ->\n    StorageRef = wa_raft_storage:registered_name(Table, Partition),\n    {ok, LogPos} = case Witness of\n        false ->\n            wa_raft_storage:create_snapshot(StorageRef);\n        true ->\n            wa_raft_storage:create_witness_snapshot(StorageRef)\n    end,\n    Path = ?RAFT_SNAPSHOT_PATH(Table, Partition, snapshot_name(LogPos, Witness)),\n    {LogPos, Path}.\n"
  },
  {
    "path": "src/wa_raft_storage.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% The RAFT storage server provides functionality for handling the\n%%% state machine replicated by RAFT in a way suitable for implementing\n%%% storage solutions on top the RAFT consensus algorithm.\n\n-module(wa_raft_storage).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n%% OTP Supervision\n-export([\n    child_spec/1,\n    start_link/1\n]).\n\n%% Public API\n-export([\n    status/1,\n    position/1,\n    label/1,\n    config/1,\n    read/2\n]).\n\n%% Internal API\n-export([\n    open/1,\n    cancel/1,\n    apply/5,\n    apply_read/3\n]).\n\n%% Internal API\n-export([\n    open_snapshot/3,\n    create_snapshot/1,\n    create_snapshot/2,\n    create_witness_snapshot/1,\n    create_witness_snapshot/2,\n    create_snapshot/3,\n    delete_snapshot/2,\n    make_empty_snapshot/5\n]).\n\n%% Internal API\n-export([\n    default_name/2,\n    registered_name/2\n]).\n\n%% gen_server callbacks\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    terminate/2\n]).\n\n%% For tests\n-ifdef(TEST).\n-export([\n    list_snapshots/1\n]).\n-endif.\n\n-export_type([\n    storage_handle/0,\n    snapshot_options/0,\n    metadata/0,\n    status/0\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage\n%%-----------------------------------------------------------------------------\n%% The RAFT consensus algorithm provides sequential consistency guarantees by\n%% ensuring the consistent replication of the \"RAFT log\", which is a sequence\n%% of \"write commands\", or \"log entries\". The RAFT algorithm intends for these\n%% entries to be applied sequentially against an underlying state machine. As\n%% this implementation of RAFT is primarily designed for implementation of\n%% storage solutions, we call the underlying state machine the \"storage\" and\n%% the state of the state machine after the application of each log entry the\n%% \"storage state\". As the sequence of commands is the same on each replica,\n%% the observable storage state after each log entry should also be the same.\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider\n%%-----------------------------------------------------------------------------\n%% This RAFT implementation provides the opportunity for users to define how\n%% exactly the \"storage\" should be implemented by defining a \"storage provider\"\n%% module when setting up a RAFT partition.\n%%\n%% Apart from certain expectations of the \"position\" of the storage state and\n%% metadata stored on behalf of the RAFT implementation, storage providers are\n%% free to handle commands in any way they see fit. However, to take advantage\n%% of the consistency guarantees provided by the RAFT algorithm, it is best to\n%% ensure a fundamental level of consistency, atomicity, and durability.\n%%\n%% The RAFT storage server is designed to be able to tolerate crashes caused\n%% by storage providers. If any callback could not be handled in a way in\n%% which it would be safe to continue operation, then storage providers are\n%% expected to raise an error to reset the RAFT replica to a known good state.\n%%-----------------------------------------------------------------------------\n\n%% Open the storage state for the specified RAFT partition.\n-callback storage_open(Options :: #raft_options{}, Path :: file:filename()) -> Handle :: storage_handle().\n\n%% Get any custom status to be reported alongside the status reported by the\n%% RAFT storage server.\n-callback storage_status(Handle :: storage_handle()) -> [{atom(), term()}].\n-optional_callbacks([storage_status/1]).\n\n%% Close a previously opened storage state.\n-callback storage_close(Handle :: storage_handle()) -> term().\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider - Position\n%%-----------------------------------------------------------------------------\n%% The position of a storage state is the log position of the write command\n%% that was most recently applied against the state. This position should be\n%% available anytime immediately after the storage is opened and after any\n%% write command is applied.\n%%-----------------------------------------------------------------------------\n\n%% Issue a read command to get the position of the current storage state.\n-callback storage_position(Handle :: storage_handle()) -> Position :: wa_raft_log:log_pos().\n\n%% Issue a read command to get the label associated with the most\n%% recent command that was applied with a label. See the optional\n%% callback `storage_apply/4` for details.\n-callback storage_label(Handle :: storage_handle()) -> {ok, Label :: wa_raft_label:label()} | {error, Reason :: term()}.\n-optional_callbacks([storage_label/1]).\n\n\n%% Issue a read command to get the config of the current storage state.\n-callback storage_config(Handle :: storage_handle()) -> {ok, Version :: wa_raft_log:log_pos(), Config :: wa_raft_server:config()} | undefined.\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider - Write Commands\n%%-----------------------------------------------------------------------------\n%% A \"write command\" is one that may cause the results of future read or write\n%% commands to produce different results. All write commands are synchronized\n%% by being added to the RAFT log and replicated. The RAFT protocol guarantees\n%% that all replicas will apply all write commands in the same order without\n%% omission. For best behaviour, the handling of write commands should ensure\n%% a fundamental level of consistency, atomicity, and durability.\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider - Consistency\n%%-----------------------------------------------------------------------------\n%% For most practical applications, it is sufficient to ensure that, regardless\n%% of the internal details of the starting and intermediate storage states,\n%% two independent applications of the same sequence of write commands produces\n%% a storage state that will continue to produce the same results when any\n%% valid sequence of future commands is applied to both identically.\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider - Atomicity and Durability against Failures\n%%-----------------------------------------------------------------------------\n%% As part of ensuring that a RAFT replica can recover from sudden unexpected\n%% failure, storage providers should be able to gracefully recover from the\n%% unexpected termination of the RAFT storage server or node resulting in\n%% the opening of a storage state that was not previously closed or whose\n%% operation was interrupted in the middle of a previous command.\n%%\n%% Generally, ensuring these qualities requires that implementations make\n%% changes that may be saved to a durable media that will persist between\n%% openings of the storage to be performed atomically (either actually or\n%% effectively) so that it is not possible for opening the storage to\n%% result in observing any intermediate state. On the other hand, that any\n%% applied changes are made durable against restart is only necessary insofar\n%% as the log of commands still retains those log entries necessary tt\n%% reproduce the lost changes.\n%%-----------------------------------------------------------------------------\n\n%% Apply a write command against the storage state, updating the state as\n%% required if a standard command or as desired for custom commands.\n%% If the command could not be applied in a manner so as to preserve the\n%% desired consistency guarantee then implementations can raise an error to\n%% cause the apply to be aborted safely.\n-callback storage_apply(Command :: wa_raft_acceptor:command(), Position :: wa_raft_log:log_pos(), Handle :: storage_handle()) -> {Result :: dynamic(), NewHandle :: storage_handle()}.\n\n%% Apply a write command against the storage state, in the same way as the\n%% above `storage_apply/3` callback. The provided label should be maintained\n%% in the storage state so that it is returned by subsequent calls to\n%% `storage_label/1`. If this callback is defined, `storage_label/1` must\n%% also be defined.\n-callback storage_apply(Command :: wa_raft_acceptor:command(), Position :: wa_raft_log:log_pos(), Label :: wa_raft_label:label(), Handle :: storage_handle()) -> {Result :: dynamic(), NewHandle :: storage_handle()}.\n-optional_callbacks([storage_apply/4]).\n\n%% Apply a write command to update the raft config stored by the storage provider\n%% on behalf of the RAFT implementation. Subsequent calls to read the config\n%% should return the updated version and value.\n%% If the command could not be applied in a manner so as to preserve the\n%% desired consistency guarantee then implementations can raise an error to be\n%% aborted safely.\n-callback storage_apply_config(Config :: wa_raft_server:config(), Position :: wa_raft_log:log_pos(), Handle :: storage_handle()) -> {Result :: ok | {error, Reason :: term()}, NewHandle :: storage_handle()}.\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider - Read Commands\n%%-----------------------------------------------------------------------------\n%% In some cases, the RAFT implementation may request a storage provider to\n%% handle commands that could require consulting the storage state but are not\n%% commands that were replicated and committed by the RAFT protocol. Such\n%% commands are called \"read commands\".\n%%\n%% Storage providers are recommended to ensure that the execution of read\n%% commands produce no externally visible side-effects. Ideally, the insertion\n%% or removal of a read command anywhere into the RAFT log (with any number\n%% of other read commands already inserted) would not affect the result\n%% returned by any other command or the results of any future commands.\n%%\n%% Implicitly, use of the `storage_position/1` callback is non-synchronized\n%% access of the storage state and should be considered to be read commands.\n%%\n%% Not exhaustively, the RAFT implementation uses read commands to access\n%% metadata stored by in the storage state on behalf of the RAFT implementation\n%% or to serve strong read requests issued by users.\n%%-----------------------------------------------------------------------------\n\n%% Apply a read command against the storage state, returning the result of\n%% the read command.\n-callback storage_read(Command :: wa_raft_acceptor:command(), Position :: wa_raft_log:log_pos(), Handle :: storage_handle()) -> Result :: dynamic().\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider - Snapshots\n%%-----------------------------------------------------------------------------\n%% A snapshot is a representation of a storage state that can be saved to disk\n%% and transmitted as a set of regular files to another RAFT replica using the\n%% same storage provider and loaded to reproduce an identical storage state.\n%%\n%% Not exhaustively, the RAFT implementation uses snapshots to quickly rebuild\n%% replicas that have fallen significantly behind in replication.\n%%-----------------------------------------------------------------------------\n\n%% Create a new snapshot of the current storage state at the provided path,\n%% producing a directory tree rooted at the provided path that represents the\n%% current storage state. The produced snapshot should retain the current\n%% position when loaded.\n-callback storage_create_snapshot(Path :: file:filename(), Handle :: storage_handle()) -> ok | {error, Reason :: term()}.\n\n%% Create a new snapshot of the current storage state at the provided path,\n%% with the given options. The options are opaque to the RAFT implementation\n%% and are passed through to the storage provider for interpretation.\n-callback storage_create_snapshot(Path :: file:filename(), Options :: snapshot_options(), Handle :: storage_handle()) -> ok | {error, Reason :: term()}.\n-optional_callbacks([storage_create_snapshot/3]).\n\n%% Create a new witness snapshot at the provided path which must contain the current\n%% position in storage and configuration.\n%% The snapshot will be empty (without actual storage data) but will retain all\n%% necessary metadata. When loaded, this witness snapshot will reflect the exact\n%% position state of the original storage without the storage contents.\n-callback storage_create_witness_snapshot(Path :: file:filename(), Handle :: storage_handle()) -> ok | {error, Reason :: term()}.\n-optional_callback([storage_create_witness_snapshot/2]).\n\n%% Load a snapshot previously created by the same storage provider, possibly\n%% copied, rooted at the provided path. If successful, the current storage\n%% state should be replaced by the storage state represented by the snapshot.\n%% If a recoverable error occured, the storage state should remain unchanged.\n%% If the storage state is no longer suitable for use, an error should be\n%% raised.\n-callback storage_open_snapshot(Path :: file:filename(), ExpectedPosition :: wa_raft_log:log_pos(), Handle :: storage_handle()) -> {ok, NewHandle :: storage_handle()} | {error, Reason :: term()}.\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage Provider - Bootstrapping\n%%-----------------------------------------------------------------------------\n\n%% Create a new snapshot at the provided path that contains some directory\n%% tree that when subsequently loaded using `storage_open_snapshot` results in\n%% a storage state with the provided last applied position and for which\n%% subsequent calls to `storage_config` returns the provided position as the\n%% version and the config as the value. Extra data may be used by implementors\n%% to provide extra state via arguments to external APIs that use this\n%% endpoint, such as the partition bootstrapping API.\n-callback storage_make_empty_snapshot(Options :: #raft_options{}, Path :: file:filename(), Position :: wa_raft_log:log_pos(), Config :: wa_raft_server:config(), Data :: dynamic()) -> ok | {error, Reason :: term()}.\n-optional_callback([storage_make_empty_snapshot/5]).\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage - Types\n%%-----------------------------------------------------------------------------\n\n-type metadata() :: config | atom().\n-type storage_handle() :: dynamic().\n-type snapshot_options() :: map().\n\n-type status() :: [status_element()].\n-type status_element() ::\n      {name, atom()}\n    | {table, wa_raft:table()}\n    | {partition, wa_raft:partition()}\n    | {module, module()}\n    | {last_applied, wa_raft_log:log_index()}\n    | ModuleSpecificStatus :: {atom(), term()}.\n\n-record(state, {\n    application :: atom(),\n    name :: atom(),\n    table :: wa_raft:table(),\n    partition :: wa_raft:partition(),\n    self :: #raft_identity{},\n    options :: #raft_options{},\n    path :: file:filename(),\n    server :: atom(),\n    queues :: wa_raft_queue:queues(),\n    module :: module(),\n    handle :: storage_handle(),\n    position :: wa_raft_log:log_pos(),\n    config :: undefined | {ok, wa_raft_log:log_pos(), wa_raft_server:config()},\n    witness = false :: boolean(),\n    skipped = 0 :: non_neg_integer()\n}).\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage - Private Types\n%%-----------------------------------------------------------------------------\n\n-define(STATUS_REQUEST, status).\n-define(POSITION_REQUEST, position).\n-define(LABEL_REQUEST, label).\n-define(CONFIG_REQUEST, config).\n\n-define(READ_REQUEST(Command), {read, Command}).\n\n-define(OPEN_REQUEST, open).\n-define(CANCEL_REQUEST, cancel).\n-define(FULFILL_REQUEST(Key, Result), {fulfill, Key, Result}).\n-define(APPLY_REQUEST(From, Record, Size, Priority), {apply, From, Record, Size, Priority}).\n-define(APPLY_READ_REQUEST(From, Command), {apply_read, From, Command}).\n\n-define(CREATE_SNAPSHOT_REQUEST(), create_snapshot).\n-define(CREATE_SNAPSHOT_REQUEST(Name), {create_snapshot, Name}).\n-define(CREATE_WITNESS_SNAPSHOT_REQUEST(), create_witness_snapshot).\n-define(CREATE_WITNESS_SNAPSHOT_REQUEST(Name), {create_witness_snapshot, Name}).\n-define(CREATE_SNAPSHOT_REQUEST(Name, Options), {create_snapshot, Name, Options}).\n-define(OPEN_SNAPSHOT_REQUEST(Path, Position), {open_snapshot, Path, Position}).\n-define(DELETE_SNAPSHOT_REQUEST(Name), {delete_snapshot, Name}).\n\n-define(MAKE_EMPTY_SNAPSHOT_REQUEST(Path, Position, Config, Data), {make_empty_snapshot, Path, Position, Config, Data}).\n\n-type call() :: status_request() | position_request() | label_request() | config_request() | read_request() |\n                open_request() | create_snapshot_request() | create_witness_snapshot_request() |\n                open_snapshot_request() | make_empty_snapshot_request().\n-type cast() :: cancel_request() | fulfill_request() | apply_request() | apply_read_request() | delete_snapshot_request().\n\n-type status_request() :: ?STATUS_REQUEST.\n-type position_request() :: ?POSITION_REQUEST.\n-type label_request() :: ?LABEL_REQUEST.\n-type config_request() :: ?CONFIG_REQUEST.\n\n-type read_request() :: ?READ_REQUEST(Command :: wa_raft_acceptor:command()).\n\n-type open_request() :: ?OPEN_REQUEST.\n-type cancel_request() :: ?CANCEL_REQUEST.\n-type fulfill_request() :: ?FULFILL_REQUEST(Key :: wa_raft_acceptor:key(), Result :: wa_raft_acceptor:commit_result()).\n-type apply_request() :: ?APPLY_REQUEST(From :: gen_server:from() | undefined, Record :: wa_raft_log:log_record(), Size :: non_neg_integer(), Priority :: wa_raft_acceptor:priority()).\n-type apply_read_request() :: ?APPLY_READ_REQUEST(From :: gen_server:from(), Comman :: wa_raft_acceptor:command()).\n\n-type create_snapshot_request() :: ?CREATE_SNAPSHOT_REQUEST() | ?CREATE_SNAPSHOT_REQUEST(Name :: string()) | ?CREATE_SNAPSHOT_REQUEST(Name :: string(), Options :: snapshot_options()).\n-type create_witness_snapshot_request() :: ?CREATE_WITNESS_SNAPSHOT_REQUEST() | ?CREATE_WITNESS_SNAPSHOT_REQUEST(Name :: string()).\n-type open_snapshot_request() :: ?OPEN_SNAPSHOT_REQUEST(Path :: string(), Position :: wa_raft_log:log_pos()).\n-type delete_snapshot_request() :: ?DELETE_SNAPSHOT_REQUEST(Name :: string()).\n\n-type make_empty_snapshot_request() :: ?MAKE_EMPTY_SNAPSHOT_REQUEST(Path :: string(), Position :: wa_raft_log:log_pos(), Config :: wa_raft_server:config(), Data :: dynamic()).\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage - OTP Supervision\n%%-----------------------------------------------------------------------------\n\n-spec child_spec(Options :: #raft_options{}) -> supervisor:child_spec().\nchild_spec(Options) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Options]},\n        restart => transient,\n        shutdown => 30000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Options :: #raft_options{}) -> gen_server:start_ret().\nstart_link(#raft_options{storage_name = Name} = Options) ->\n    gen_server:start_link({local, Name}, ?MODULE, Options, []).\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage - Public API\n%%-----------------------------------------------------------------------------\n\n-spec status(Storage :: gen_server:server_ref()) -> status().\nstatus(Storage) ->\n    gen_server:call(Storage, ?STATUS_REQUEST, ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec position(Storage :: gen_server:server_ref()) -> Position :: wa_raft_log:log_pos().\nposition(Storage) ->\n    gen_server:call(Storage, ?POSITION_REQUEST, ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec label(Storage :: gen_server:server_ref()) -> {ok, Label :: wa_raft_label:label()} | {error, Reason :: term()}.\nlabel(Storage) ->\n    gen_server:call(Storage, ?LABEL_REQUEST, ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec config(Storage :: gen_server:server_ref()) -> {ok, wa_raft_log:log_pos(), wa_raft_server:config()} | undefined | {error, Reason :: term()}.\nconfig(Storage) ->\n    gen_server:call(Storage, ?CONFIG_REQUEST, ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec read(Storage :: gen_server:server_ref(), Command :: wa_raft_acceptor:command()) -> ok.\nread(Storage, Command) ->\n    gen_server:call(Storage, ?READ_REQUEST(Command), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n%%-----------------------------------------------------------------------------\n%% RAFT Storage - Internal API\n%%-----------------------------------------------------------------------------\n\n-spec open(Storage :: gen_server:server_ref()) -> {ok, LastApplied :: wa_raft_log:log_pos()}.\nopen(Storage) ->\n    gen_server:call(Storage, ?OPEN_REQUEST, ?RAFT_RPC_CALL_TIMEOUT()).\n\n-spec cancel(Storage :: gen_server:server_ref()) -> ok.\ncancel(Storage) ->\n    gen_server:cast(Storage, ?CANCEL_REQUEST).\n\n-spec apply(\n    Storage :: gen_server:server_ref(),\n    From :: gen_server:from() | undefined,\n    Record :: wa_raft_log:log_record(),\n    Size :: non_neg_integer(),\n    Priority :: wa_raft_acceptor:priority()\n) -> ok.\napply(Storage, From, Record, Size, Priority) ->\n    gen_server:cast(Storage, ?APPLY_REQUEST(From, Record, Size, Priority)).\n\n-spec apply_read(Storage :: gen_server:server_ref(), From :: gen_server:from(), Command :: wa_raft_acceptor:command()) -> ok.\napply_read(Storage, From, Command) ->\n    gen_server:cast(Storage, ?APPLY_READ_REQUEST(From, Command)).\n\n-spec open_snapshot(Storage :: gen_server:server_ref(), Path :: file:filename(), Position :: wa_raft_log:log_pos()) -> ok | {error, Reason :: term()}.\nopen_snapshot(Storage, Path, Position) ->\n    gen_server:call(Storage, ?OPEN_SNAPSHOT_REQUEST(Path, Position), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec create_snapshot(Storage :: gen_server:server_ref()) -> {ok, Pos :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\ncreate_snapshot(Storage) ->\n    gen_server:call(Storage, ?CREATE_SNAPSHOT_REQUEST(), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n%% Be careful when using the same name for two snapshots as the RAFT storage\n%% server will not recreate an existing snapshot even if the storage state has\n%% advanced since the snapshot was created; however, this method will always\n%% return the current position upon success.\n-spec create_snapshot(Storage :: gen_server:server_ref(), Name :: string()) -> {ok, Pos :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\ncreate_snapshot(Storage, Name) ->\n    gen_server:call(Storage, ?CREATE_SNAPSHOT_REQUEST(Name), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec create_witness_snapshot(Storage :: gen_server:server_ref()) -> {ok, Pos :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\ncreate_witness_snapshot(Storage) ->\n    gen_server:call(Storage, ?CREATE_WITNESS_SNAPSHOT_REQUEST(), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n%% Be careful when using the same name for two snapshots as the RAFT storage\n%% server will not recreate an existing snapshot even if the storage state has\n%% advanced since the snapshot was created; however, this method will always\n%% return the current position upon success.\n-spec create_witness_snapshot(Storage :: gen_server:server_ref(), Name :: string()) -> {ok, Pos :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\ncreate_witness_snapshot(Storage, Name) ->\n    gen_server:call(Storage, ?CREATE_WITNESS_SNAPSHOT_REQUEST(Name), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n%% Be careful when using the same name for two snapshots as the RAFT storage\n%% server will not recreate an existing snapshot even if the storage state has\n%% advanced since the snapshot was created; however, this method will always\n%% return the current position upon success.\n%% Options are opaque to the RAFT implementation and are passed through to the\n%% storage provider for interpretation.\n%% Snaphots created with custom options should not be created using the default\n%% snapshot name as RAFT may confuse it with a generic snapshot and incorrectly use\n%% it for transport or other RAFT operations.\n-spec create_snapshot(Storage :: gen_server:server_ref(), Name :: string(), Options :: snapshot_options()) -> {ok, Pos :: wa_raft_log:log_pos()} | {error, Reason :: term()}.\ncreate_snapshot(Storage, Name, Options) ->\n    gen_server:call(Storage, ?CREATE_SNAPSHOT_REQUEST(Name, Options), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n-spec delete_snapshot(Storage :: gen_server:server_ref(), Name :: string()) -> ok.\ndelete_snapshot(Storage, Name) ->\n    gen_server:cast(Storage, ?DELETE_SNAPSHOT_REQUEST(Name)).\n\n-spec make_empty_snapshot(Storage :: gen_server:server_ref(), Path :: file:filename(), Position :: wa_raft_log:log_pos(), Config :: wa_raft_server:config(), Data :: term()) -> ok | {error, Reason :: term()}.\nmake_empty_snapshot(Storage, Path, Position, Config, Data) ->\n    gen_server:call(Storage, ?MAKE_EMPTY_SNAPSHOT_REQUEST(Path, Position, Config, Data), ?RAFT_STORAGE_CALL_TIMEOUT()).\n\n%%-------------------------------------------------------------------\n%% RAFT Storage - Internal API\n%%-------------------------------------------------------------------\n\n%% Get the default name for the RAFT storage server associated with the\n%% provided RAFT partition.\n-spec default_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom, one per table/partition at startup\n    binary_to_atom(<<\"raft_storage_\", (atom_to_binary(Table))/binary, \"_\", (integer_to_binary(Partition))/binary>>).\n\n%% Get the registered name for the RAFT storage server associated with the\n%% provided RAFT partition or the default name if no registration exists.\n-spec registered_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\nregistered_name(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> default_name(Table, Partition);\n        Options   -> Options#raft_options.storage_name\n    end.\n\n%%-------------------------------------------------------------------\n%% RAFT Storage - Server Callbacks\n%%-------------------------------------------------------------------\n\n-spec init(Options :: #raft_options{}) -> {ok, #state{}}.\ninit(#raft_options{application = Application, table = Table, partition = Partition, self = Self, database = Path, server_name = Server, storage_name = Name, storage_module = Module} = Options) ->\n    process_flag(trap_exit, true),\n\n    % This increases the potential overhead of sending log entries to storage\n    % to be applied; however, can protect the storage server from GC overhead\n    % and other memory-related issues.\n    process_flag(message_queue_data, off_heap),\n\n    ?RAFT_LOG_NOTICE(\n        \"Storage[~0p] starting for partition ~0p/~0p at ~0p using ~0p\",\n        [Name, Table, Partition, Path, Module]\n    ),\n\n    Handle = Module:storage_open(Options, Path),\n    Position = Module:storage_position(Handle),\n\n    ?RAFT_LOG_NOTICE(\"Storage[~0p] opened at position ~0p.\", [Name, Position]),\n\n    State = #state{\n        application = Application,\n        name = Name,\n        table = Table,\n        partition = Partition,\n        self = Self,\n        options = Options,\n        path = Path,\n        server = Server,\n        queues = wa_raft_queue:queues(Options),\n        module = Module,\n        handle = Handle,\n        position = Position\n    },\n    {ok, refresh_config(State)}.\n\n%% The interaction between the RAFT server and the RAFT storage server is designed to be\n%% as asynchronous as possible since the RAFT storage server may be caught up in handling\n%% a long running I/O request while it is working on applying new log entries.\n%% If you are adding a new call to the RAFT storage server, make sure that it is either\n%% guaranteed to not be used when the storage server is busy (and may not reply in time)\n%% or timeouts and other failures are handled properly.\n-spec handle_call(Request :: call(), From :: gen_server:from(), State :: #state{}) ->\n    {noreply, NewState :: #state{}} |\n    {reply, Reply :: term(), NewState :: #state{}} |\n    {stop, Reason :: term(), Reply :: term(), NewState :: #state{}}.\n\nhandle_call(?STATUS_REQUEST, _From, #state{module = Module, handle = Handle} = State) ->\n    BaseStatus = [\n        {name, State#state.name},\n        {table, State#state.table},\n        {partition, State#state.partition},\n        {module, State#state.module},\n        {last_applied, State#state.position#raft_log_pos.index}\n    ],\n    ModuleStatus = case erlang:function_exported(Module, storage_status, 1) of\n        true  -> Module:storage_status(Handle);\n        false -> []\n    end,\n    {reply, BaseStatus ++ ModuleStatus, State};\n\nhandle_call(?POSITION_REQUEST, _From, #state{table = Table, module = Module, handle = Handle} = State) ->\n    ?RAFT_COUNT(Table, 'storage.position'),\n    Result = Module:storage_position(Handle),\n    {reply, Result, State};\n\nhandle_call(?LABEL_REQUEST, _From, #state{table = Table, module = Module, handle = Handle} = State) ->\n    ?RAFT_COUNT(Table, 'storage.label'),\n    Result = Module:storage_label(Handle),\n    {reply, Result, State};\n\nhandle_call(?CONFIG_REQUEST, _From, #state{config = Config} = State) ->\n    {reply, Config, State};\n\nhandle_call(?READ_REQUEST(Command), _From, #state{module = Module, handle = Handle, position = Position} = State) ->\n    {reply, Module:storage_read(Command, Position, Handle), State};\n\nhandle_call(?OPEN_REQUEST, _From, #state{position = Position} = State) ->\n    {reply, {ok, Position}, State};\n\nhandle_call(?OPEN_SNAPSHOT_REQUEST(SnapshotPath, SnapshotPosition), _From, #state{name = Name, module = Module, handle = Handle, position = Position} = State) ->\n    ?RAFT_LOG_NOTICE(\"Storage[~0p] at ~0p is opening snapshot ~0p.\", [Name, Position, SnapshotPosition]),\n    case Module:storage_open_snapshot(SnapshotPath, SnapshotPosition, Handle) of\n        {ok, NewHandle} ->\n            {reply, ok, refresh_config(State#state{position = SnapshotPosition, handle = NewHandle})};\n        {error, Reason} ->\n            {reply, {error, Reason}, State}\n    end;\n\nhandle_call(?CREATE_SNAPSHOT_REQUEST(), _From, #state{position = #raft_log_pos{index = Index, term = Term}} = State) ->\n    Name = ?SNAPSHOT_NAME(Index, Term),\n    {reply, handle_create_snapshot(Name, State), State};\n\nhandle_call(?CREATE_SNAPSHOT_REQUEST(Name), _From, #state{} = State) ->\n    {reply, handle_create_snapshot(Name, State), State};\n\nhandle_call(?CREATE_WITNESS_SNAPSHOT_REQUEST(), _From, #state{position = #raft_log_pos{index = Index, term = Term}} = State) ->\n    Name = ?WITNESS_SNAPSHOT_NAME(Index, Term),\n    {reply, handle_create_witness_snapshot(Name, State), State};\n\nhandle_call(?CREATE_WITNESS_SNAPSHOT_REQUEST(Name), _From, #state{} = State) ->\n    {reply, handle_create_witness_snapshot(Name, State), State};\n\nhandle_call(?CREATE_SNAPSHOT_REQUEST(Name, Options), _From, #state{} = State) ->\n    {reply, handle_create_snapshot(Name, Options, State), State};\n\nhandle_call(?MAKE_EMPTY_SNAPSHOT_REQUEST(SnapshotPath, SnapshotPosition, Config, Data), _From, #state{name = Name, options = Options, module = Module} = State) ->\n    ?RAFT_LOG_NOTICE(\n        \"Storage[~0p] making bootstrap snapshot ~0p at ~0p with config ~0p and data ~0P.\",\n        [Name, SnapshotPath, SnapshotPosition, Config, Data, 30]\n    ),\n    case erlang:function_exported(Module, storage_make_empty_snapshot, 5) of\n        true -> {reply, Module:storage_make_empty_snapshot(Options, SnapshotPath, SnapshotPosition, Config, Data), State};\n        false -> {reply, {error, not_supported}, State}\n    end;\n\nhandle_call(Request, From, #state{name = Name} = State) ->\n    ?RAFT_LOG_WARNING(\"Storage[~0p] received unexpected call ~0P from ~0p.\", [Name, Request, 20, From]),\n    {noreply, State}.\n\n-spec handle_cast(Request :: cast(), State :: #state{}) ->\n    {noreply, NewState :: #state{}}.\n\nhandle_cast(?CANCEL_REQUEST, #state{name = Name, queues = Queues} = State) ->\n    ?RAFT_LOG_NOTICE(\"Storage[~0p] cancels all pending reads.\", [Name]),\n    wa_raft_queue:fulfill_all_reads(Queues, {error, not_leader}),\n    {noreply, State};\n\nhandle_cast(?APPLY_REQUEST(From, {LogIndex, {LogTerm, {_, Label, Command}}}, Size, Priority), #state{name = Name, queues = Queues} = State0) ->\n    wa_raft_queue:fulfill_apply(Queues, Size),\n    LogPosition = #raft_log_pos{index = LogIndex, term = LogTerm},\n    ?RAFT_LOG_DEBUG(\"Storage[~0p] is starting to apply ~0p\", [Name, LogPosition]),\n    {noreply, handle_apply(From, LogPosition, Label, Command, Priority, State0)};\n\nhandle_cast(?APPLY_READ_REQUEST(From, Command), #state{module = Module, handle = Handle, position = Position} = State) ->\n    gen_server:reply(From, Module:storage_read(Command, Position, Handle)),\n    {noreply, State};\n\nhandle_cast(?DELETE_SNAPSHOT_REQUEST(SnapshotName), #state{name = Name, path = Path} = State) ->\n    Result = try file:del_dir_r(filename:join(Path, SnapshotName))\n             catch Class:Reason -> {Class, Reason}\n             end,\n    ?RAFT_LOG_NOTICE(\"Storage[~0p] deletes snapshot ~0p: ~0P.\", [Name, SnapshotName, Result, 20]),\n    {noreply, State};\n\nhandle_cast(Request, #state{name = Name} = State) ->\n    ?RAFT_LOG_WARNING(\"Storage[~0p] received unexpected cast ~0P.\", [Name, Request, 20]),\n    {noreply, State}.\n\n-spec terminate(Reason :: term(), State :: #state{}) -> term().\nterminate(Reason, #state{name = Name, module = Module, handle = Handle, position = Position}) ->\n    ?RAFT_LOG_NOTICE(\"Storage[~0p] terminating at ~0p with reason ~0P.\", [Name, Position, Reason, 30]),\n    Module:storage_close(Handle).\n\n%%-------------------------------------------------------------------\n%% RAFT Storage - Implementation\n%%-------------------------------------------------------------------\n\n-spec handle_apply(\n    From :: gen_server:from() | undefined,\n    LogPosition :: wa_raft_log:log_pos(),\n    Label :: wa_raft_label:label(),\n    Command :: wa_raft_acceptor:command(),\n    Priority :: wa_raft_acceptor:priority(),\n    State :: #state{}\n) -> NewState :: #state{}.\n%% In the case that a log entry is reapplied, fulfill any new pending reads at that index.\nhandle_apply(\n    _From,\n    #raft_log_pos{index = LogIndex},\n    _Label,\n    _Command,\n    _Priority,\n    #state{position = #raft_log_pos{index = Index}} = State\n) when LogIndex =:= Index ->\n    handle_delayed_reads(State),\n    State;\n%% Issue an apply request to storage when the next log entry is to be applied,\n%% and respond if the current effect term is equal to the log entry's term.\nhandle_apply(\n    From,\n    #raft_log_pos{index = LogIndex} = LogPosition,\n    Label,\n    Command,\n    Priority,\n    #state{\n        application = Application,\n        table = Table,\n        name = Name,\n        server = Server,\n        queues = Queues,\n        position = #raft_log_pos{index = Index}\n    } = State\n) when LogIndex =:= Index + 1 ->\n    ?RAFT_COUNT(Table, 'storage.apply'),\n    StartTUsec = erlang:monotonic_time(microsecond),\n    {Reply, NewState} = handle_command(Label, Command, LogPosition, State),\n    From =/= undefined andalso wa_raft_queue:commit_completed(Queues, From, Reply, Priority),\n    handle_delayed_reads(NewState),\n    wa_raft_queue:apply_queue_size(Queues) =:= 0 andalso ?RAFT_STORAGE_NOTIFY_COMPLETE(Application, Table) andalso\n        wa_raft_server:notify_complete(Server),\n    ?RAFT_LOG_DEBUG(\"Storage[~0p] finishes applying ~0p.\", [Name, LogPosition]),\n    ?RAFT_GATHER(Table, 'storage.apply.func', erlang:monotonic_time(microsecond) - StartTUsec),\n    NewState;\n%% Otherwise, the apply is out of order.\nhandle_apply(_From, LogPosition, _Label, _Command, _Priority, #state{name = Name, position = Position}) ->\n    ?RAFT_LOG_ERROR(\"Storage[~0p] at ~0p received an out-of-order operation at ~0p.\", [Name, Position, LogPosition]),\n    error(out_of_order_apply).\n\n-spec handle_command(\n    Label :: wa_raft_label:label(),\n    Command :: wa_raft_acceptor:command(),\n    Position :: wa_raft_log:log_pos(),\n    State :: #state{}\n) -> {Result :: term(), #state{}}.\nhandle_command(Label, noop = Command, Position, #state{} = State) ->\n    handle_command_impl(Label, Command, Position, State);\nhandle_command(_Label, {config, Config}, Position, #state{name = Name, module = Module, handle = Handle} = State) ->\n    ?RAFT_LOG_INFO(\"Storage[~0p] is applying a new configuration ~0p at ~0p.\", [Name, Config, Position]),\n    {Reply, NewHandle} =\n        case Module:storage_apply_config(Config, Position, Handle) of\n            {ok, H} -> {{ok, Position}, H};\n            {Other, H} -> {Other, H}\n        end,\n    {Reply, refresh_config(State#state{handle = NewHandle, position = Position})};\nhandle_command(Label, _Command, Position, #state{application = Application, table = Table, witness = true, skipped = Skipped} = State) ->\n    case Skipped >= ?RAFT_STORAGE_WITNESS_APPLY_INTERVAL(Application, Table) of\n        true ->\n            {Reply, NewState} = handle_command_impl(Label, noop_omitted, Position, State),\n            {Reply, NewState#state{skipped = 0}};\n        false ->\n            {ok, State#state{position = Position, skipped = Skipped + 1}}\n    end;\nhandle_command(Label, Command, Position, #state{} = State) ->\n    handle_command_impl(Label, Command, Position, State).\n\n-spec handle_command_impl(\n    Label :: wa_raft_label:label(),\n    Command :: wa_raft_acceptor:command(),\n    Position :: wa_raft_log:log_pos(),\n    State :: #state{}\n) -> {Result :: term(), #state{}}.\nhandle_command_impl(Label, Command, Position, #state{module = Module, handle = Handle} = State) ->\n    {Reply, NewHandle} = case Label of\n        undefined -> Module:storage_apply(Command, Position, Handle);\n        _         -> Module:storage_apply(Command, Position, Label, Handle)\n    end,\n    {Reply, State#state{handle = NewHandle, position = Position}}.\n\n-spec handle_delayed_reads(State :: #state{}) -> ok.\nhandle_delayed_reads(#state{queues = Queues, module = Module, handle = Handle, position = #raft_log_pos{index = Index} = Position}) ->\n    [\n        begin\n            Reply = Module:storage_read(Command, Position, Handle),\n            wa_raft_queue:fulfill_read(Queues, Reference, Reply)\n        end || {Reference, Command} <- wa_raft_queue:query_reads(Queues, Index)\n    ],\n    ok.\n\n-spec handle_create_snapshot(SnapshotName :: string(), Storage :: #state{}) -> {ok, wa_raft_log:log_pos()} | {error, Reason :: term()}.\nhandle_create_snapshot(SnapshotName, #state{name = Name, path = Path, module = Module, handle = Handle, position = Position} = State) ->\n    SnapshotPath = filename:join(Path, SnapshotName),\n    case filelib:is_dir(SnapshotPath, prim_file) of\n        true ->\n            ?RAFT_LOG_NOTICE(\"Storage[~0p] skips recreating existing snapshot ~0p.\", [Name, SnapshotName]),\n            {ok, Position};\n        false ->\n            cleanup_snapshots(State),\n            ?RAFT_LOG_NOTICE(\"Storage[~0p] is creating snapshot ~0p.\", [Name, SnapshotName]),\n            case Module:storage_create_snapshot(SnapshotPath, Handle) of\n                ok -> {ok, Position};\n                Other -> Other\n            end\n    end.\n\n-spec handle_create_witness_snapshot(SnapshotName :: string(), Storage :: #state{}) -> {ok, wa_raft_log:log_pos()} | {error, Reason :: term()}.\nhandle_create_witness_snapshot(SnapshotName, #state{name = Name, path = Path, module = Module, handle = Handle, position = Position} = State) ->\n    SnapshotPath = filename:join(Path, SnapshotName),\n    case filelib:is_dir(SnapshotPath, prim_file) of\n        true ->\n            ?RAFT_LOG_NOTICE(\"Storage[~0p] skips recreating existing witness snapshot ~0p.\", [Name, SnapshotName]),\n            {ok, Position};\n        false ->\n            cleanup_snapshots(State),\n            ?RAFT_LOG_NOTICE(\"Storage[~0p] is creating witness snapshot ~0p.\", [Name, SnapshotName]),\n            case erlang:function_exported(Module, storage_create_witness_snapshot, 2) of\n                true ->\n                    case Module:storage_create_witness_snapshot(SnapshotPath, Handle) of\n                        ok -> {ok, Position};\n                        Other -> Other\n                    end;\n                false ->\n                    {error, not_supported}\n            end\n    end.\n\n-spec handle_create_snapshot(SnapshotName :: string(), Options :: snapshot_options(), Storage :: #state{}) -> {ok, wa_raft_log:log_pos()} | {error, Reason :: term()}.\nhandle_create_snapshot(SnapshotName, Options, #state{name = Name, path = Path, module = Module, handle = Handle, position = Position} = State) ->\n    SnapshotPath = filename:join(Path, SnapshotName),\n    case filelib:is_dir(SnapshotPath, prim_file) of\n        true ->\n            ?RAFT_LOG_NOTICE(\"Storage[~0p] skips recreating existing snapshot ~0p.\", [Name, SnapshotName]),\n            {ok, Position};\n        false ->\n            cleanup_snapshots(State),\n            ?RAFT_LOG_NOTICE(\"Storage[~0p] is creating snapshot ~0p with options ~0p.\", [Name, SnapshotName, Options]),\n            case erlang:function_exported(Module, storage_create_snapshot, 3) of\n                true ->\n                    case Module:storage_create_snapshot(SnapshotPath, Options, Handle) of\n                        ok -> {ok, Position};\n                        Other -> Other\n                    end;\n                false ->\n                    {error, not_supported}\n            end\n    end.\n\n-spec refresh_config(Storage :: #state{}) -> #state{}.\nrefresh_config(#state{self = Self, module = Module, handle = Handle} = Storage) ->\n    case Module:storage_config(Handle) of\n        {ok, Version, Config} ->\n            Storage#state{\n                config = {ok, Version, wa_raft_server:normalize_config(Config)},\n                witness = wa_raft_server:is_witness(Self, Config)\n            };\n        undefined ->\n            Storage#state{\n                config = undefined,\n                witness = false\n            }\n    end.\n\n-define(MAX_RETAINED_SNAPSHOT, 1).\n\n-spec cleanup_snapshots(#state{}) -> ok.\ncleanup_snapshots(#state{path = Path}) ->\n    Snapshots = list_snapshots(Path),\n    case length(Snapshots) > ?MAX_RETAINED_SNAPSHOT of\n        true ->\n            lists:foreach(\n                fun ({_, SnapshotName}) ->\n                    SnapshotPath = filename:join(Path, SnapshotName),\n                    ?RAFT_LOG_NOTICE(\"Removing snapshot \\\"~s\\\".\", [SnapshotPath]),\n                    file:del_dir_r(SnapshotPath)\n                end, lists:sublist(Snapshots, length(Snapshots) - ?MAX_RETAINED_SNAPSHOT)),\n            ok;\n        _ ->\n            ok\n    end.\n\n%% Private functions\n-spec list_snapshots(Path :: string()) -> [{wa_raft_log:log_pos(), file:filename()}].\nlist_snapshots(Path) ->\n    SnapshotNames = filelib:wildcard(?SNAPSHOT_PREFIX ++ \".*\", Path, prim_file),\n    Snapshots = lists:filtermap(fun decode_snapshot_name/1, SnapshotNames),\n    lists:keysort(1, Snapshots).\n\n-spec decode_snapshot_name(SnapshotName :: string()) -> {true, {wa_raft_log:log_pos(), file:filename()}} | false.\ndecode_snapshot_name(SnapshotName) ->\n    case string:lexemes(SnapshotName, \".\") of\n        [?SNAPSHOT_PREFIX, IndexStr, TermStr | _] ->\n            case {list_to_integer(IndexStr), list_to_integer(TermStr)} of\n                {Index, Term} when Index >= 0, Term >= 0 ->\n                    {true, {#raft_log_pos{index = Index, term = Term}, SnapshotName}};\n                _ ->\n                    ?RAFT_LOG_WARNING(\n                        \"Invalid snapshot with invalid index (~p) and/or term (~p). (full name ~p)\",\n                        [IndexStr, TermStr, SnapshotName]\n                    ),\n                    false\n            end;\n        _ ->\n            ?RAFT_LOG_WARNING(\"Invalid snapshot dir name ~p\", [SnapshotName]),\n            false\n    end.\n"
  },
  {
    "path": "src/wa_raft_storage_ets.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% An example implementation of the RAFT storage provider behaviour that\n%%% uses ETS as a backing store. This implementation is for demonstration\n%%% purposes only and should not be used in actual applications.\n\n-module(wa_raft_storage_ets).\n-compile(warn_missing_spec_all).\n-behaviour(wa_raft_storage).\n\n-export([\n    storage_open/2,\n    storage_close/1,\n    storage_label/1,\n    storage_position/1,\n    storage_config/1,\n    storage_apply/3,\n    storage_apply/4,\n    storage_apply_config/3,\n    storage_read/3,\n    storage_create_snapshot/2,\n    storage_create_witness_snapshot/2,\n    storage_open_snapshot/3,\n    storage_make_empty_snapshot/5\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n%% Options used for the ETS table\n-define(OPTIONS, [set, public, {read_concurrency, true}, {write_concurrency, true}]).\n\n%% Filename used for the actual ETS table file in a snapshot\n-define(SNAPSHOT_FILENAME, \"data\").\n\n%% Tag used in keys for metadata stored on the behalf of RAFT\n-define(METADATA_TAG, '$metadata').\n%% Tag used for label metadata stored on behalf of RAFT.\n-define(LABEL_TAG, '$label').\n%% Tag used for recording the current storage position\n-define(POSITION_TAG, '$position').\n%% Tag used for tracking if the current storage is incomplete.\n-define(INCOMPLETE_TAG, '$incomplete').\n\n-record(state, {\n    name :: atom(),\n    table :: wa_raft:table(),\n    partition :: wa_raft:partition(),\n    self :: #raft_identity{},\n    storage :: ets:table()\n}).\n\n-spec storage_open(#raft_options{}, file:filename()) -> #state{}.\nstorage_open(#raft_options{table = Table, partition = Partition, self = Self, storage_name = Name}, _RootDir) ->\n    Storage = ets:new(Name, ?OPTIONS),\n    #state{name = Name, table = Table, partition = Partition, self = Self, storage = Storage}.\n\n-spec storage_close(#state{}) -> ok.\nstorage_close(#state{storage = Storage}) ->\n    true = ets:delete(Storage),\n    ok.\n\n-spec storage_position(#state{}) -> wa_raft_log:log_pos().\nstorage_position(#state{storage = Storage}) ->\n    ets:lookup_element(Storage, ?POSITION_TAG, 2, #raft_log_pos{}).\n\n-spec storage_label(#state{}) -> {ok, Label :: wa_raft_label:label()}.\nstorage_label(#state{storage = Storage}) ->\n    case ets:lookup(Storage, ?LABEL_TAG) of\n        [{_, Label}] -> {ok, Label};\n        []           -> {ok, undefined}\n    end.\n\n-spec storage_config(#state{}) -> {ok, wa_raft_log:log_pos(), wa_raft_server:config()} | undefined.\nstorage_config(#state{storage = Storage}) ->\n    case ets:lookup(Storage, {?METADATA_TAG, config}) of\n        [{_, {Version, Value}}] -> {ok, Version, Value};\n        []                      -> undefined\n    end.\n\n-spec storage_incomplete(#state{}) -> boolean().\nstorage_incomplete(#state{storage = Storage}) ->\n    ets:lookup_element(Storage, ?INCOMPLETE_TAG, 2, false).\n\n-spec storage_apply(Command :: wa_raft_acceptor:command(), Position :: wa_raft_log:log_pos(), Label :: wa_raft_label:label(), Storage :: #state{}) -> {ok, #state{}}.\nstorage_apply(Command, Position, Label, #state{storage = Storage} = State) ->\n    true = ets:insert(Storage, {?LABEL_TAG, Label}),\n    storage_apply(Command, Position, State).\n\n-spec storage_apply(Command :: wa_raft_acceptor:command(), Position :: wa_raft_log:log_pos(), Storage :: #state{}) -> {ok, #state{}}.\nstorage_apply(noop, Position, #state{storage = Storage} = State) ->\n    true = ets:insert(Storage, {?POSITION_TAG, Position}),\n    {ok, State};\nstorage_apply(noop_omitted, Position, #state{storage = Storage} = State) ->\n    true = ets:insert(Storage, [{?INCOMPLETE_TAG, true}, {?POSITION_TAG, Position}]),\n    {ok, State};\nstorage_apply({write, _Table, Key, Value}, Position, #state{storage = Storage} = State) ->\n    true = ets:insert(Storage, [{Key, Value}, {?POSITION_TAG, Position}]),\n    {ok, State};\nstorage_apply({delete, _Table, Key}, Position, #state{storage = Storage} = State) ->\n    true = ets:delete(Storage, Key),\n    true = ets:insert(Storage, {?POSITION_TAG, Position}),\n    {ok, State}.\n\n-spec storage_apply_config(\n    Config :: wa_raft_server:config(),\n    LogPos :: wa_raft_log:log_pos(),\n    State :: #state{}\n) -> {ok | {error, Reason :: term()}, #state{}}.\nstorage_apply_config(Config, LogPos, State) ->\n    storage_check_config(Config, State),\n    storage_apply_config(Config, LogPos, LogPos, State).\n\n-spec storage_apply_config(\n    Config :: wa_raft_server:config(),\n    ConfigPos :: wa_raft_log:log_pos(),\n    LogPos :: wa_raft_log:log_pos(),\n    State :: #state{}\n) -> {ok | {error, Reason :: term()}, #state{}}.\nstorage_apply_config(Config, ConfigPos, LogPos, #state{storage = Storage} = State) ->\n    true = ets:insert(Storage, [{{?METADATA_TAG, config}, {ConfigPos, Config}}, {?POSITION_TAG, LogPos}]),\n    {ok, State}.\n\n-spec storage_read(Command :: wa_raft_acceptor:command(), Position :: wa_raft_log:log_pos(), State :: #state{}) -> ok | {ok, Value :: dynamic()} | not_found.\nstorage_read(noop, _Position, #state{}) ->\n    ok;\nstorage_read({read, _Table, Key}, _Position, #state{storage = Storage}) ->\n    case ets:lookup(Storage, Key) of\n        [{_, Value}] -> {ok, Value};\n        []           -> not_found\n    end.\n\n-spec storage_create_snapshot(file:filename(), #state{}) -> ok | {error, Reason :: term()}.\nstorage_create_snapshot(SnapshotPath, #state{storage = Storage}) ->\n    case filelib:ensure_path(SnapshotPath) of\n        ok              -> ets:tab2file(Storage, filename:join(SnapshotPath, ?SNAPSHOT_FILENAME));\n        {error, Reason} -> {error, Reason}\n    end.\n\n-spec storage_create_witness_snapshot(file:filename(), #state{}) -> ok | {error, Reason :: term()}.\nstorage_create_witness_snapshot(SnapshotPath, #state{name = Name, table = Table, partition = Partition, self = Self} = State) ->\n    {ok, ConfigPosition, Config} = storage_config(State),\n    SnapshotPosition = storage_position(State),\n    storage_make_empty_snapshot(Name, Table, Partition, Self, SnapshotPath, SnapshotPosition, Config, ConfigPosition, #{}).\n\n-spec storage_open_snapshot(file:filename(), wa_raft_log:log_pos(), #state{}) -> {ok, #state{}} | {error, Reason :: term()}.\nstorage_open_snapshot(SnapshotPath, SnapshotPosition, #state{storage = Storage} = State) ->\n    SnapshotData = filename:join(SnapshotPath, ?SNAPSHOT_FILENAME),\n    case ets:file2tab(SnapshotData) of\n        {ok, NewStorage} ->\n            case ets:lookup_element(NewStorage, ?POSITION_TAG, 2, #raft_log_pos{}) of\n                SnapshotPosition ->\n                    NewState = State#state{storage = NewStorage},\n                    storage_check_config(NewState),\n                    try ets:delete(Storage) catch _:_ -> ok end,\n                    {ok, NewState};\n                _IncorrectPosition ->\n                    try ets:delete(NewStorage) catch _:_ -> ok end,\n                    {error, bad_position}\n            end;\n        {error, Reason} ->\n            {error, Reason}\n    end.\n\n-spec storage_check_config(#state{}) -> ok.\nstorage_check_config(State) ->\n    case storage_config(State) of\n        {ok, _, Config} -> storage_check_config(Config, State);\n        undefined -> ok\n    end.\n\n-spec storage_check_config(wa_raft_server:config(), #state{}) -> ok.\nstorage_check_config(Config, #state{self = Self} = State) ->\n    case storage_incomplete(State) andalso wa_raft_server:is_data_replica(Self, Config) of\n        true -> error(invalid_incomplete_replica);\n        false -> ok\n    end.\n\n-spec storage_make_empty_snapshot(#raft_options{}, file:filename(), wa_raft_log:log_pos(), wa_raft_server:config(), dynamic()) -> ok | {error, Reason :: term()}.\nstorage_make_empty_snapshot(#raft_options{table = Table, partition = Partition, self = Self, storage_name = Name}, SnapshotPath, SnapshotPosition, Config, Data) ->\n    storage_make_empty_snapshot(Name, Table, Partition, Self, SnapshotPath, SnapshotPosition, Config, SnapshotPosition, Data).\n\n-spec storage_make_empty_snapshot(atom(), wa_raft:table(), wa_raft:partition(), #raft_identity{}, file:filename(), wa_raft_log:log_pos(), wa_raft_server:config(), wa_raft_log:log_pos(), dynamic()) -> ok | {error, Reason :: term()}.\nstorage_make_empty_snapshot(Name, Table, Partition, Self, SnapshotPath, SnapshotPosition, Config, ConfigPosition, _Data) ->\n    Storage = ets:new(Name, ?OPTIONS),\n    State = #state{name = Name, table = Table, partition = Partition, self = Self, storage = Storage},\n    storage_apply_config(Config, ConfigPosition, SnapshotPosition, State),\n    storage_create_snapshot(SnapshotPath, State).\n"
  },
  {
    "path": "src/wa_raft_sup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% Supervisor for supervising RAFT partitions started by a client application.\n%%% As a `simple_one_for_one` supervisor, this supervisor can dynamically\n%%% start and stop partitions and will stop partitions in parallel during\n%%% shutdown.\n\n-module(wa_raft_sup).\n-compile(warn_missing_spec_all).\n-behaviour(supervisor).\n\n%% OTP supervision\n-export([\n    child_spec/1,\n    child_spec/2,\n    child_spec/3,\n    start_link/3\n]).\n\n%% API\n-export([\n    start_partition/2,\n    start_partition_under_application/2,\n    stop_partition/2,\n    stop_partition/3,\n    stop_partition_under_application/2,\n    stop_partition_under_application/3\n]).\n\n%% Internal API\n-export([\n    default_name/1,\n    default_config_apps/1,\n    registered_config_apps/1\n]).\n\n%% Internal API\n-export([\n    options/1\n]).\n\n%% Test API\n-export([\n    prepare_application/1,\n    prepare_application/2\n]).\n\n%% Supervisor callbacks\n-export([\n    init/1\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n%% Key in persistent_term for the application options associated with an\n%% application that has started a RAFT supervisor.\n-define(OPTIONS_KEY(Application), {?MODULE, Application}).\n\n%% Options for RAFT client applications\n-type options() :: #{\n    % RAFT will search for environment variables from applications in this order\n    config_search_apps => [atom()]\n}.\n\n%%-------------------------------------------------------------------\n%% OTP supervision\n%%-------------------------------------------------------------------\n\n-spec child_spec(Specs :: [wa_raft:args()]) -> supervisor:child_spec().\nchild_spec(Specs) ->\n    {ok, Application} = application:get_application(),\n    child_spec(Application, Specs, #{}).\n\n-spec child_spec(Application :: atom(), Specs :: [wa_raft:args()]) -> supervisor:child_spec().\nchild_spec(Application, RaftArgs) when is_list(RaftArgs) ->\n    child_spec(Application, RaftArgs, #{});\nchild_spec(RaftArgs, Options) ->\n    {ok, Application} = application:get_application(),\n    child_spec(Application, RaftArgs, Options).\n\n-spec child_spec(Application :: atom(), Specs :: [wa_raft:args()], Options :: options()) -> supervisor:child_spec().\nchild_spec(Application, RaftArgs, Options) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Application, RaftArgs, Options]},\n        restart => permanent,\n        shutdown => infinity,\n        type => supervisor,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Application :: atom(), Specs :: [wa_raft:args()], Options :: options()) -> supervisor:startlink_ret().\nstart_link(Application, RaftArgs, Options) ->\n    ok = persistent_term:put(?OPTIONS_KEY(Application), normalize_spec(Application, Options)),\n    case supervisor:start_link({local, default_name(Application)}, ?MODULE, Application) of\n        {ok, Pid} = Result ->\n            [\n                case start_partition(Pid, Spec) of\n                    {error, Reason} -> error(Reason);\n                    _Other          -> ok\n                end || Spec <- RaftArgs\n            ],\n            Result;\n        Else ->\n            Else\n    end.\n\n%%-------------------------------------------------------------------\n%% API\n%%-------------------------------------------------------------------\n\n-spec start_partition(Supervisor :: atom() | pid(), Spec :: wa_raft:args()) -> supervisor:startchild_ret().\nstart_partition(Supervisor, Spec) ->\n    supervisor:start_child(Supervisor, [Spec]).\n\n-spec start_partition_under_application(Application :: atom(), Spec :: wa_raft:args()) -> supervisor:startchild_ret().\nstart_partition_under_application(Application, Spec) ->\n    start_partition(default_name(Application), Spec).\n\n-spec stop_partition(Supervisor :: atom() | pid(), Pid :: pid()) -> ok | {error, atom()}.\nstop_partition(Supervisor, Pid) ->\n    supervisor:terminate_child(Supervisor, Pid).\n\n-spec stop_partition(Supervisor :: atom() | pid(), Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> ok | {error, atom()}.\nstop_partition(Supervisor, Table, Partition) ->\n    case whereis(wa_raft_part_sup:registered_name(Table, Partition)) of\n        Pid when is_pid(Pid) -> stop_partition(Supervisor, Pid);\n        _                    -> {error, not_found}\n    end.\n\n-spec stop_partition_under_application(Application :: atom(), Pid :: pid()) -> ok | {error, atom()}.\nstop_partition_under_application(Application, Pid) ->\n    stop_partition(default_name(Application), Pid).\n\n-spec stop_partition_under_application(Application :: atom(), Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> ok | {error, atom()}.\nstop_partition_under_application(Application, Table, Partition) ->\n    stop_partition(default_name(Application), Table, Partition).\n\n%%-------------------------------------------------------------------\n%% Internal API\n%%-------------------------------------------------------------------\n\n-spec default_name(Application :: atom()) -> atom().\ndefault_name(Application) ->\n    % elp:ignore W0023 bounded atom for supervisor name\n    list_to_atom(\"raft_sup_\" ++ atom_to_list(Application)).\n\n-spec default_config_apps(Application :: atom()) -> [atom()].\ndefault_config_apps(Application) ->\n    [Application, ?RAFT_APPLICATION].\n\n-spec registered_config_apps(Application :: atom()) -> [atom()].\nregistered_config_apps(Application) ->\n    case options(Application) of\n        undefined -> error({raft_not_started, Application});\n        Options   -> Options#raft_application.config_search_apps\n    end.\n\n-spec options(Application :: atom()) -> #raft_application{} | undefined.\noptions(Application) ->\n    persistent_term:get(?OPTIONS_KEY(Application), undefined).\n\n-spec normalize_spec(Application :: atom(), Options :: options()) -> #raft_application{}.\nnormalize_spec(Application, Options) ->\n    #raft_application{\n        name = Application,\n        config_search_apps = maps:get(config_search_apps, Options, [Application])\n    }.\n\n%%-------------------------------------------------------------------\n%% Test API\n%%-------------------------------------------------------------------\n\n-spec prepare_application(Application :: atom()) -> ok.\nprepare_application(Application) ->\n    prepare_application(Application, #{}).\n\n-spec prepare_application(Application :: atom(), Options :: options()) -> ok.\nprepare_application(Application, Options) ->\n    RaftApplication = normalize_spec(Application, Options),\n    ok = persistent_term:put(?OPTIONS_KEY(Application), RaftApplication).\n\n%%-------------------------------------------------------------------\n%% Supervisor callbacks\n%%-------------------------------------------------------------------\n\n-spec init(Application :: atom()) -> {ok, {supervisor:sup_flags(), [supervisor:child_spec()]}}.\ninit(Application) ->\n    {ok, {#{strategy => simple_one_for_one, intensity => 10, period => 1}, [wa_raft_part_sup:child_spec(Application)]}}.\n"
  },
  {
    "path": "src/wa_raft_transport.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n\n-module(wa_raft_transport).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n-include_lib(\"kernel/include/file.hrl\").\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%% OTP supervision\n-export([\n    child_spec/0,\n    start_link/0\n]).\n\n%% Bulk Transfer API\n-export([\n    start_transfer/4,\n    start_transfer/5,\n    transfer/5\n]).\n\n%% Snapshot Transfer API\n-export([\n    start_snapshot_transfer/6,\n    start_snapshot_transfer/7,\n    transfer_snapshot/7\n]).\n\n%% Transport API\n-export([\n    cancel/2,\n    complete/3\n]).\n\n%% ETS API\n-export([\n    setup_tables/0,\n    transports/0,\n    transport_info/1,\n    transport_info/2,\n    file_info/2,\n    update_file_info/3\n]).\n\n%% Internal API - Configuration\n-export([\n    default_directory/1,\n    registered_directory/2,\n    registered_module/2\n]).\n\n%% Internal API - Transport Workers\n-export([\n    pop_file/1\n]).\n\n%% gen_server callbacks\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    handle_info/2\n]).\n\n-export_type([\n    transport_id/0,\n    transport_info/0,\n    file_id/0,\n    file_info/0\n]).\n\n%% Name of the ETS table to keep records for transports\n-define(TRANSPORT_TABLE, wa_raft_transport_transports).\n%% Name of the ETS table to keep records for files\n-define(FILE_TABLE, wa_raft_transport_files).\n\n-define(RAFT_TRANSPORT_PARTITION_SUBDIRECTORY, \"transport\").\n\n-define(RAFT_TRANSPORT_SCAN_INTERVAL_SECS, 30).\n\n%% Number of counters\n-define(RAFT_TRANSPORT_COUNTERS, 2).\n\n%% Counter - inflight receives\n-define(RAFT_TRANSPORT_COUNTER_ACTIVE_RECEIVES, 1).\n\n%% Counter - inflight witness receives\n-define(RAFT_TRANSPORT_COUNTER_ACTIVE_WITNESS_RECEIVES, 2).\n\n-type transport_id() :: pos_integer().\n-type transport_info() :: #{\n    type := sender | receiver,\n    status := requested | running | completed | cancelled | timed_out | failed,\n    atomics := atomics:atomics_ref(),\n\n    peer := atom(),\n    module := module(),\n    meta := meta(),\n    notify => gen_server:from(),\n\n    root := string(),\n\n    start_ts := Millis :: integer(),\n    end_ts => Millis :: integer(),\n\n    total_files := non_neg_integer(),\n    completed_files := non_neg_integer(),\n    queue => ets:table(),\n\n    error => term()\n}.\n\n-type meta() :: meta_transfer() | meta_snapshot().\n-type meta_transfer() :: #{\n    type := transfer,\n    table := wa_raft:table(),\n    partition := wa_raft:partition()\n}.\n-type meta_snapshot() :: #{\n    type := snapshot,\n    table := wa_raft:table(),\n    partition := wa_raft:partition(),\n    position := wa_raft_log:log_pos(),\n    witness := boolean()\n}.\n\n-type file_id() :: pos_integer().\n-type file_info() :: #{\n    status := requested | sending | receiving | completed | cancelled | failed,\n    atomics := {Transport :: atomics:atomics_ref(), File :: atomics:atomics_ref()},\n\n    name := string(),\n    path := string(),\n    mtime => integer(),\n\n    start_ts => Millis :: integer(),\n    end_ts => Millis :: integer(),\n    retries => non_neg_integer(),\n\n    total_bytes := non_neg_integer(),\n    completed_bytes := non_neg_integer(),\n\n    meta => map(),\n    error => Reason :: term()\n}.\n\n%%% ------------------------------------------------------------------------\n\n-record(state, {\n    counters :: counters:counters_ref()\n}).\n\n%%% ------------------------------------------------------------------------\n%%%  Behaviour callbacks\n%%%\n\n%% Perform any setup required before transport can be started.\n-callback transport_init(Node :: node()) -> {ok, State :: term()} | {stop, Reason :: term()}.\n\n%% Send a file to the target peer.\n-callback transport_send(ID :: transport_id(), FileID :: file_id(), State :: term()) ->\n    {ok, NewState :: term()} |\n    {continue, NewState :: term()} |\n    {stop, Reason :: term(), NewState :: term()}.\n\n%% Optional callback for performing any shutdown operations.\n-callback transport_terminate(Reason :: term(), State :: term()) -> term().\n\n-optional_callbacks([\n    transport_terminate/2\n]).\n\n%%% ------------------------------------------------------------------------\n%%%  OTP supervision callbacks\n%%%\n-spec child_spec() -> supervisor:child_spec().\nchild_spec() ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, []},\n        restart => permanent,\n        shutdown => 5000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link() -> {ok, Pid :: pid()} | ignore | {error, Reason :: term()}.\nstart_link() ->\n    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).\n\n%%% ------------------------------------------------------------------------\n%%%  Internal API\n%%%\n\n-spec start_transport(Peer :: atom(), Meta :: meta(), Root :: string(), Timeout :: timeout()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\nstart_transport(Peer, Meta, Root, Timeout) ->\n    gen_server:call(?MODULE, {start, Peer, Meta, Root}, Timeout).\n\n-spec start_transport_and_wait(Peer :: atom(), Meta :: meta(), Root :: string(), Timeout :: timeout()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\nstart_transport_and_wait(Peer, Meta, Root, Timeout) ->\n    gen_server:call(?MODULE, {start_wait, Peer, Meta, Root}, Timeout).\n\n%%% ------------------------------------------------------------------------\n%%%  Bulk Transfer API\n%%%\n\n-spec start_transfer(Peer :: atom(), Table :: wa_raft:table(), Partition :: wa_raft:partition(), Root :: string()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\nstart_transfer(Peer, Table, Partition, Root) ->\n    start_transfer(Peer, Table, Partition, Root, 10000).\n\n-spec start_transfer(Peer :: atom(), Table :: wa_raft:table(), Partition :: wa_raft:partition(), Root :: string(), Timeout :: timeout()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\nstart_transfer(Peer, Table, Partition, Root, Timeout) ->\n    start_transport(Peer, #{type => transfer, table => Table, partition => Partition}, Root, Timeout).\n\n-spec transfer(Peer :: atom(), Table :: wa_raft:table(), Partition :: wa_raft:partition(), Root :: string(), Timeout :: timeout()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\ntransfer(Peer, Table, Partition, Root, Timeout) ->\n    start_transport_and_wait(Peer, #{type => transfer, table => Table, partition => Partition}, Root, Timeout).\n\n%%% ------------------------------------------------------------------------\n%%%  Snapshot Transfer API\n%%%\n\n-spec start_snapshot_transfer(Peer :: atom(), Table :: wa_raft:table(), Partition :: wa_raft:partition(), LogPos :: wa_raft_log:log_pos(), Root :: string(), Witness :: boolean()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\nstart_snapshot_transfer(Peer, Table, Partition, LogPos, Root, Witness) ->\n    start_snapshot_transfer(Peer, Table, Partition, LogPos, Root, Witness, 10000).\n\n-spec start_snapshot_transfer(Peer :: atom(), Table :: wa_raft:table(), Partition :: wa_raft:partition(), LogPos :: wa_raft_log:log_pos(), Root :: string(), Witness :: boolean(), Timeout :: timeout()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\nstart_snapshot_transfer(Peer, Table, Partition, LogPos, Root, Witness, Timeout) ->\n    start_transport(Peer, #{type => snapshot, table => Table, partition => Partition, position => LogPos, witness => Witness}, Root, Timeout).\n\n-spec transfer_snapshot(Peer :: atom(), Table :: wa_raft:table(), Partition :: wa_raft:partition(), LogPos :: wa_raft_log:log_pos(), Root :: string(), Witness :: boolean(), Timeout :: timeout()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\ntransfer_snapshot(Peer, Table, Partition, LogPos, Root, Witness, Timeout) ->\n    start_transport_and_wait(Peer, #{type => snapshot, table => Table, partition => Partition, position => LogPos, witness => Witness}, Root, Timeout).\n\n%%% ------------------------------------------------------------------------\n%%%  Transport API\n%%%\n\n-spec cancel(ID :: transport_id(), Reason :: term()) -> ok | {error, Reason :: term()}.\ncancel(ID, Reason) ->\n    gen_server:call(?MODULE, {cancel, ID, Reason}).\n\n-spec complete(ID :: transport_id(), FileID :: file_id(), Status :: dynamic()) -> ok.\ncomplete(ID, FileID, Status) ->\n    gen_server:cast(?MODULE, {complete, ID, FileID, Status}).\n\n%%% ------------------------------------------------------------------------\n%%%  ETS table helper functions\n%%%\n\n-spec setup_tables() -> ok.\nsetup_tables() ->\n    ?TRANSPORT_TABLE = ets:new(?TRANSPORT_TABLE, [named_table, set, public]),\n    ?FILE_TABLE = ets:new(?FILE_TABLE, [named_table, set, public]),\n    ok.\n\n-spec transports() -> [transport_id()].\ntransports() ->\n    ets:select(?TRANSPORT_TABLE, [{{'$1', '_'}, [], ['$1']}]).\n\n-spec transport_info(ID :: transport_id()) -> {ok, Info :: transport_info()} | not_found.\ntransport_info(ID) ->\n    case ets:lookup_element(?TRANSPORT_TABLE, ID, 2, not_found) of\n        not_found -> not_found;\n        Info      -> {ok, Info}\n    end.\n\n-spec transport_info(ID :: transport_id(), Item :: atom()) -> Info :: term() | undefined.\ntransport_info(ID, Item) ->\n    case transport_info(ID) of\n        {ok, #{Item := Value}} -> Value;\n        _                      -> undefined\n    end.\n\n% This function should only be called from the \"gen_server\" process since it does not\n% provide any atomicity guarantees.\n-spec set_transport_info(ID :: transport_id(), Info :: transport_info(), Counters :: counters:counters_ref()) -> ok.\nset_transport_info(ID, #{atomics := TransportAtomics} = Info, Counters) ->\n    true = ets:insert(?TRANSPORT_TABLE, {ID, Info}),\n    maybe_update_active_inbound_transport_counts(undefined, Info, Counters),\n    ok = atomics:put(TransportAtomics, ?RAFT_TRANSPORT_ATOMICS_UPDATED_TS, erlang:system_time(millisecond)).\n\n% This function should only be called from the \"gen_server\" process since it does not\n% provide any atomicity guarantees.\n-spec update_and_get_transport_info(\n    ID :: transport_id(),\n    Fun :: fun((Info :: transport_info()) -> NewInfo :: transport_info()),\n    Counters :: counters:counters_ref()\n) -> {ok, NewOrExistingInfo :: transport_info()} | not_found.\nupdate_and_get_transport_info(ID, Fun, Counters) ->\n    case transport_info(ID) of\n        {ok, #{atomics := TransportAtomics} = Info} ->\n            case Fun(Info) of\n                Info ->\n                    {ok, Info};\n                NewInfo ->\n                    true = ets:insert(?TRANSPORT_TABLE, {ID, NewInfo}),\n                    ok = atomics:put(TransportAtomics, ?RAFT_TRANSPORT_ATOMICS_UPDATED_TS, erlang:system_time(millisecond)),\n                    ok = maybe_update_active_inbound_transport_counts(Info, NewInfo, Counters),\n                    {ok, NewInfo}\n            end;\n        not_found ->\n            not_found\n    end.\n\n-spec delete_transport_info(ID :: transport_id()) -> ok | not_found.\ndelete_transport_info(ID) ->\n    case transport_info(ID) of\n        {ok, #{total_files := TotalFiles} = Info} ->\n            lists:foreach(fun (FileID) -> delete_file_info(ID, FileID) end, lists:seq(1, TotalFiles)),\n            ets:delete(?TRANSPORT_TABLE, ID),\n            Queue = maps:get(queue, Info, undefined),\n            Queue =/= undefined andalso catch ets:delete(Queue),\n            ok;\n        not_found ->\n            not_found\n    end.\n\n-spec file_info(ID :: transport_id(), FileID :: file_id()) -> {ok, Info :: file_info()} | not_found.\nfile_info(ID, FileID) ->\n    case ets:lookup_element(?FILE_TABLE, {ID, FileID}, 2, not_found) of\n        not_found -> not_found;\n        Info      -> {ok, Info}\n    end.\n\n-spec maybe_update_active_inbound_transport_counts(OldInfo :: transport_info() | undefined, NewInfo :: transport_info(), Counters :: counters:counters_ref()) -> ok.\nmaybe_update_active_inbound_transport_counts(OldInfo, #{meta := #{witness := true}} = NewInfo, Counters) ->\n    maybe_update_active_inbound_transport_counts_impl(OldInfo, NewInfo, ?RAFT_TRANSPORT_COUNTER_ACTIVE_WITNESS_RECEIVES, Counters);\nmaybe_update_active_inbound_transport_counts(OldInfo, NewInfo, Counters) ->\n    maybe_update_active_inbound_transport_counts_impl(OldInfo, NewInfo, ?RAFT_TRANSPORT_COUNTER_ACTIVE_RECEIVES, Counters).\n\n-spec maybe_update_active_inbound_transport_counts_impl(OldInfo :: transport_info() | undefined, NewInfo :: transport_info(), Counter :: non_neg_integer(), Counters :: counters:counters_ref()) -> ok.\nmaybe_update_active_inbound_transport_counts_impl(undefined, #{type := receiver, status := running}, Counter, Counters) ->\n    counters:add(Counters, Counter, 1);\nmaybe_update_active_inbound_transport_counts_impl(#{type := receiver, status := OldStatus}, #{status := running}, Counter, Counters) when OldStatus =/= running ->\n    counters:add(Counters, Counter, 1);\nmaybe_update_active_inbound_transport_counts_impl(#{type := receiver, status := running}, #{status := NewStatus}, Counter, Counters) when NewStatus =/= running ->\n    counters:sub(Counters, Counter, 1);\nmaybe_update_active_inbound_transport_counts_impl(_, _, _, _) ->\n    ok.\n\n% This function should only be called from the \"worker\" process responsible for the\n% transport of the specified file since it does not provide any atomicity guarantees.\n-spec set_file_info(ID :: transport_id(), FileID :: file_id(), Info :: file_info()) -> ok.\nset_file_info(ID, FileID, #{atomics := {TransportAtomics, FileAtomics}} = Info) ->\n    true = ets:insert(?FILE_TABLE, {{ID, FileID}, Info}),\n    NowMillis = erlang:system_time(millisecond),\n    ok = atomics:put(TransportAtomics, ?RAFT_TRANSPORT_ATOMICS_UPDATED_TS, NowMillis),\n    ok = atomics:put(FileAtomics, ?RAFT_TRANSPORT_ATOMICS_UPDATED_TS, NowMillis).\n\n% This function should only be called from the \"worker\" process responsible for the\n% transport of the specified file since it does not provide any atomicity guarantees.\n-spec update_file_info(ID :: transport_id(), FileID :: file_id(), Fun :: fun((Info :: file_info()) -> NewInfo :: file_info())) -> ok | not_found.\nupdate_file_info(ID, FileID, Fun) ->\n    case file_info(ID, FileID) of\n        {ok, #{atomics := {TransportAtomics, FileAtomics}} = Info} ->\n            case Fun(Info) of\n                Info ->\n                    ok;\n                NewInfo ->\n                    true = ets:insert(?FILE_TABLE, {{ID, FileID}, NewInfo}),\n                    NowMillis = erlang:system_time(millisecond),\n                    ok = atomics:put(TransportAtomics, ?RAFT_TRANSPORT_ATOMICS_UPDATED_TS, NowMillis),\n                    ok = atomics:put(FileAtomics, ?RAFT_TRANSPORT_ATOMICS_UPDATED_TS, NowMillis),\n                    ok\n            end;\n        not_found ->\n            not_found\n    end.\n\n-spec delete_file_info(ID :: transport_id(), FileID :: file_id()) -> ok.\ndelete_file_info(ID, FileID) ->\n    ets:delete(?FILE_TABLE, {ID, FileID}),\n    ok.\n\n%%-------------------------------------------------------------------\n%% Internal API - Configuration\n%%-------------------------------------------------------------------\n\n%% Get the default directory for incoming transports associated with the\n%% provided RAFT partition given that RAFT partition's database directory.\n-spec default_directory(Database :: file:filename()) -> Directory :: file:filename().\ndefault_directory(Database) ->\n    filename:join(Database, ?RAFT_TRANSPORT_PARTITION_SUBDIRECTORY).\n\n%% Get the registered directory for incoming transports associated with the\n%% provided RAFT partition or 'undefined' if no registration exists.\n-spec registered_directory(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Directory :: file:filename() | undefined.\nregistered_directory(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> undefined;\n        Options   -> Options#raft_options.transport_directory\n    end.\n\n%% Get the registered module for outgoing transports associated with the\n%% provided RAFT partition or the default transport module if no registration exists.\n-spec registered_module(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Module :: module() | undefined.\nregistered_module(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> ?RAFT_DEFAULT_TRANSPORT_MODULE;\n        Options   -> Options#raft_options.transport_module\n    end.\n\n%%-------------------------------------------------------------------\n%% Internal API - Transport Workers\n%%-------------------------------------------------------------------\n\n-spec pop_file(ID :: transport_id()) -> {ok, FileID :: file_id()} | empty | not_found.\npop_file(ID) ->\n    case transport_info(ID) of\n        {ok, #{queue := Queue}} -> try_pop_file(Queue);\n        _Other                  -> not_found\n    end.\n\n-spec try_pop_file(Queue :: ets:table()) -> {ok, FileID :: file_id()} | empty | not_found.\ntry_pop_file(Queue) ->\n    try ets:first(Queue) of\n        '$end_of_table' ->\n            empty;\n        FileID ->\n            try ets:select_delete(Queue, [{{FileID}, [], [true]}]) of\n                0 -> try_pop_file(Queue);\n                1 -> {ok, FileID}\n            catch\n                error:badarg -> not_found\n            end\n    catch\n        error:badarg -> not_found\n    end.\n\n%%% ------------------------------------------------------------------------\n%%%  gen_server callbacks\n%%%\n\n-spec init(Args :: []) -> {ok, State :: #state{}}.\ninit(_) ->\n    process_flag(trap_exit, true),\n    Counters = counters:new(?RAFT_TRANSPORT_COUNTERS, [atomics]),\n    schedule_scan(),\n    {ok, #state{counters = Counters}}.\n\n-spec handle_call(Request, From :: gen_server:from(), State :: #state{}) -> {reply, Reply :: term(), NewState :: #state{}} | {noreply, NewState :: #state{}}\n    when\n        Request ::\n            {start, Peer :: node(), Meta :: meta(), Root :: string()} |\n            {start_wait, Peer :: node(), Meta :: meta(), Root :: string()} |\n            {transport, ID :: transport_id(), Peer :: node(), Module :: module(), Meta :: meta(), Files :: [{file_id(), RelPath :: string(), Size :: integer()}]} |\n            {cancel, ID :: transport_id(), Reason :: term()}.\nhandle_call({start, Peer, Meta, Root}, _From, #state{counters = Counters} = State) ->\n    {reply, handle_transport_start(undefined, Peer, Meta, Root, Counters), State};\nhandle_call({start_wait, Peer, Meta, Root}, From, #state{counters = Counters} = State) ->\n    case handle_transport_start(From, Peer, Meta, Root, Counters) of\n        {ok, _ID}       -> {noreply, State};\n        {error, Reason} -> {reply, {error, Reason}, State}\n    end;\nhandle_call({transport, ID, Peer, Module, Meta, Files}, From, #state{counters = Counters} = State) ->\n    Table = maps:get(table, Meta, undefined),\n    try\n        IsWitness = maps:get(witness, Meta, false),\n        {MaxIncomingSnapshotTransfers, NumActiveReceives} = case IsWitness of\n            true  -> {?RAFT_MAX_CONCURRENT_INCOMING_WITNESS_SNAPSHOT_TRANSFERS(), counters:get(Counters, ?RAFT_TRANSPORT_COUNTER_ACTIVE_WITNESS_RECEIVES)};\n            false -> {?RAFT_MAX_CONCURRENT_INCOMING_SNAPSHOT_TRANSFERS(), counters:get(Counters, ?RAFT_TRANSPORT_COUNTER_ACTIVE_RECEIVES)}\n        end,\n        ShouldThrottle = NumActiveReceives >= MaxIncomingSnapshotTransfers,\n        case {transport_info(ID), ShouldThrottle} of\n            {{ok, _Info}, _} ->\n                ?RAFT_LOG_WARNING(\"wa_raft_transport got duplicate transport receive start for ~p from ~p\", [ID, From]),\n                {reply, duplicate, State};\n            {not_found, true} ->\n                {reply, {error, receiver_overloaded}, State};\n            {not_found, _} ->\n                ?RAFT_COUNT(Table, 'transport.receive'),\n                ?RAFT_LOG_NOTICE(\"wa_raft_transport starting transport receive for ~p\", [ID]),\n\n                TransportAtomics = atomics:new(?RAFT_TRANSPORT_TRANSPORT_ATOMICS_COUNT, []),\n                RootDir = transport_destination(ID, Meta),\n                NowMillis = erlang:system_time(millisecond),\n                TotalFiles = length(Files),\n\n                % Force the receiving directory to always exist\n                catch filelib:ensure_dir([RootDir, $/]),\n\n                % Setup overall transport info\n                set_transport_info(ID, #{\n                    type => receiver,\n                    status => running,\n                    atomics => TransportAtomics,\n                    peer => Peer,\n                    module => Module,\n                    meta => Meta,\n                    root => RootDir,\n                    start_ts => NowMillis,\n                    total_files => TotalFiles,\n                    completed_files => 0\n                }, Counters),\n\n                % Setup file info for each file\n                [\n                    begin\n                        FileAtomics = atomics:new(?RAFT_TRANSPORT_FILE_ATOMICS_COUNT, []),\n                        set_file_info(ID, FileID, #{\n                            status => requested,\n                            atomics => {TransportAtomics, FileAtomics},\n                            name => RelativePath,\n                            path => filename:join(RootDir, RelativePath),\n                            total_bytes => Size,\n                            completed_bytes => 0\n                        })\n                    end || {FileID, RelativePath, Size} <- Files\n                ],\n\n                % If the transport is empty, then immediately complete it\n                TotalFiles =:= 0 andalso\n                    update_and_get_transport_info(\n                        ID,\n                        fun (Info0) ->\n                            Info1 = Info0#{status => completed, end_ts => NowMillis},\n                            Info2 = case maybe_notify_complete(ID, Info1, State) of\n                                ok              -> Info1;\n                                {error, Reason} -> Info1#{status => failed, error => {notify_failed, Reason}}\n                            end,\n                            maybe_notify(ID, Info2)\n                        end,\n                        Counters\n                    ),\n\n                {reply, ok, State}\n        end\n    catch\n        T:E:S ->\n            ?RAFT_COUNT(Table, 'transport.receive.error'),\n            ?RAFT_LOG_WARNING(\"wa_raft_transport failed to accept transport ~p due to ~p ~p: ~n~p\", [ID, T, E, S]),\n            update_and_get_transport_info(\n                ID,\n                fun (Info) ->\n                    Info#{\n                        status => failed,\n                        end_ts => erlang:system_time(millisecond),\n                        error => {receive_failed, {T, E, S}}\n                    }\n                end,\n                Counters\n            ),\n            {reply, {error, failed}, State}\n    end;\nhandle_call({cancel, ID, Reason}, _From, #state{counters = Counters} = State) ->\n    ?RAFT_LOG_NOTICE(\"wa_raft_transport got cancellation request for ~p for reason ~p\", [ID, Reason]),\n    Reply =\n        case\n            update_and_get_transport_info(\n                ID,\n                fun\n                    (#{status := running} = Info) ->\n                        NowMillis = erlang:system_time(millisecond),\n                        Info#{status := cancelled, end_ts => NowMillis, error => {cancelled, Reason}};\n                    (Info) ->\n                        Info\n                end,\n                Counters\n            )\n        of\n            {ok, _Info} -> ok;\n            not_found   -> {error, not_found}\n        end,\n    {reply, Reply, State};\nhandle_call(Request, _From, #state{} = State) ->\n    ?RAFT_LOG_WARNING(\"wa_raft_transport received unrecognized call ~p\", [Request]),\n    {noreply, State}.\n\n-spec handle_cast(Request, State :: #state{}) -> {noreply, NewState :: #state{}}\n    when Request :: {complete, ID :: transport_id(), FileID :: file_id(), Status :: term()}.\nhandle_cast({complete, ID, FileID, Status}, #state{counters = Counters} = State) ->\n    NowMillis = erlang:system_time(millisecond),\n    Table = case transport_info(ID) of\n        {ok, #{meta := Meta}} -> maps:get(table, Meta, undefined);\n        _                     -> undefined\n    end,\n    ?RAFT_COUNT(Table, {'transport.file.send', normalize_status(Status)}),\n    Result0 = update_file_info(ID, FileID,\n        fun (Info) ->\n            case Info of\n                #{start_ts := StartMillis} ->\n                    ?RAFT_GATHER_LATENCY(Table, {'transport.file.send.latency_ms', Status}, NowMillis - StartMillis);\n                _ ->\n                    ok\n            end,\n            case Status of\n                ok -> Info#{status => completed, end_ts => NowMillis};\n                _  -> Info#{status => failed, end_ts => NowMillis, error => Status}\n            end\n        end),\n    Result0 =:= not_found andalso\n        ?RAFT_LOG_WARNING(\"wa_raft_transport got complete report for unknown file ~p:~p\", [ID, FileID]),\n    Result1 =\n        update_and_get_transport_info(\n            ID,\n            fun\n                (#{status := running, completed_files := CompletedFiles, total_files := TotalFiles} = Info0) ->\n                    Info1 = Info0#{completed_files => CompletedFiles + 1},\n                    Info2 = case CompletedFiles + 1 of\n                        TotalFiles -> Info1#{status => completed, end_ts => NowMillis};\n                        _          -> Info1\n                    end,\n                    Info3 = case Status of\n                        ok -> Info2;\n                        _  -> Info2#{status => failed, end_ts => NowMillis, error => {file, FileID, Status}}\n                    end,\n                    Info4 = case maybe_notify_complete(ID, Info3, State) of\n                        ok              -> Info3;\n                        {error, Reason} -> Info3#{status => failed, error => {notify_failed, Reason}}\n                    end,\n                    maybe_notify(ID, Info4);\n                (Info) ->\n                    Info\n            end,\n            Counters\n        ),\n    Result1 =:= not_found andalso\n        ?RAFT_LOG_WARNING(\"wa_raft_transport got complete report for unknown transfer ~p\", [ID]),\n    {noreply, State};\nhandle_cast(Request, State) ->\n    ?RAFT_LOG_NOTICE(\"wa_raft_transport got unrecognized cast ~p\", [Request]),\n    {noreply, State}.\n\n-spec handle_info(Info :: term(), State :: #state{}) -> {noreply, NewState :: #state{}}.\nhandle_info(scan, #state{counters = Counters} = State) ->\n    InactiveTransports =\n        lists:filter(\n            fun (ID) ->\n                case update_and_get_transport_info(ID, fun (Info) -> scan_transport(ID, Info) end, Counters) of\n                    {ok, #{status := Status}} -> Status =/= requested andalso Status =/= running;\n                    not_found                 -> false\n                end\n            end, transports()),\n    ExcessTransports = length(InactiveTransports) - ?RAFT_TRANSPORT_INACTIVE_INFO_LIMIT(),\n    ExcessTransports > 0 andalso begin\n        ExcessTransportIDs = lists:sublist(lists:sort(InactiveTransports), ExcessTransports),\n        lists:foreach(fun delete_transport_info/1, ExcessTransportIDs)\n    end,\n\n    schedule_scan(),\n    {noreply, State};\nhandle_info(Info, State) ->\n    ?RAFT_LOG_NOTICE(\"wa_raft_transport got unrecognized info ~p\", [Info]),\n    {noreply, State}.\n\n%%% ------------------------------------------------------------------------\n%%%  Helper functions\n%%%\n\n-spec make_id() -> non_neg_integer().\nmake_id() ->\n    NowMicros = erlang:system_time(microsecond),\n    ID = NowMicros * 1000000 + rand:uniform(1000000) - 1,\n    case transport_info(ID) of\n        {ok, _Info} -> make_id();\n        not_found   -> ID\n    end.\n\n-spec handle_transport_start(From :: gen_server:from() | undefined, Peer :: node(), Meta :: meta(), Root :: string(), Counters :: counters:counters_ref()) -> {ok, ID :: transport_id()} | {error, Reason :: term()}.\nhandle_transport_start(From, Peer, Meta, Root, Counters) ->\n    ID = make_id(),\n    Table = maps:get(table, Meta, undefined),\n\n    ?RAFT_COUNT(Table, 'transport.start'),\n    ?RAFT_LOG_NOTICE(\n        \"wa_raft_transport starting transport ~p of ~p to ~p with metadata ~p\",\n        [ID, Root, Peer, Meta]\n    ),\n\n    try\n        Files = collect_files(Root),\n        TransportAtomics = atomics:new(?RAFT_TRANSPORT_TRANSPORT_ATOMICS_COUNT, []),\n        Module = transport_module(Meta),\n        TotalFiles = length(Files),\n        NowMillis = erlang:system_time(millisecond),\n        Queue = ets:new(?MODULE, [ordered_set, public]),\n\n        % Setup overall transport info\n        set_transport_info(ID, #{\n            type => sender,\n            status => requested,\n            atomics => TransportAtomics,\n            peer => Peer,\n            module => Module,\n            meta => Meta,\n            root => Root,\n            start_ts => NowMillis,\n            total_files => TotalFiles,\n            completed_files => 0,\n            queue => Queue\n        }, Counters),\n\n        % Setup file info for each file\n        [\n            begin\n                FileAtomics = atomics:new(?RAFT_TRANSPORT_FILE_ATOMICS_COUNT, []),\n                set_file_info(ID, FileID, #{\n                    status => requested,\n                    atomics => {TransportAtomics, FileAtomics},\n                    name => Filename,\n                    path => Path,\n                    mtime => MTime,\n                    total_bytes => Size,\n                    completed_bytes => 0\n                })\n            end || {FileID, Filename, Path, MTime, Size} <- Files\n        ],\n\n        % Notify peer node of incoming transport\n        FileData = [{FileID, Filename, Size} || {FileID, Filename, _, _, Size} <- Files],\n        case gen_server:call({?MODULE, Peer}, {transport, ID, node(), Module, Meta, FileData}, ?RAFT_RPC_CALL_TIMEOUT()) of\n            ok ->\n                % Add all files to the queue\n                ets:insert(Queue, [{FileID} || {FileID, _, _, _, _} <- Files]),\n\n                % Start workers\n                update_and_get_transport_info(\n                    ID,\n                    fun (Info0) ->\n                        Info1 = case From of\n                            undefined -> Info0;\n                            _         -> Info0#{notify => From}\n                        end,\n                        case TotalFiles of\n                            0 ->\n                                Info2 = Info1#{status => completed, end_ts => NowMillis},\n                                maybe_notify(ID, Info2);\n                            _ ->\n                                Sup = wa_raft_transport_sup:get_or_start(Peer),\n                                [gen_server:cast(Pid, {notify, ID, Table}) || {_Id, Pid, _Type, _Modules} <- supervisor:which_children(Sup), is_pid(Pid)],\n                                Info1#{status => running}\n                        end\n                    end,\n                    Counters\n                ),\n                {ok, ID};\n            {error, receiver_overloaded} ->\n                ?RAFT_COUNT(Table, 'transport.rejected.receiver_overloaded'),\n                ?RAFT_LOG_WARNING(\"wa_raft_transport peer ~p rejected transport ~p because of overload\", [Peer, ID]),\n                update_and_get_transport_info(\n                    ID,\n                    fun (Info) ->\n                        Info#{\n                            status => failed,\n                            end_ts => NowMillis,\n                            error => {rejected, receiver_overloaded}\n                        }\n                    end,\n                    Counters\n                ),\n                {error, receiver_overloaded};\n            Error ->\n                ?RAFT_COUNT(Table, 'transport.rejected'),\n                ?RAFT_LOG_WARNING(\"wa_raft_transport peer ~p rejected transport ~p with error ~p\", [Peer, ID, Error]),\n                update_and_get_transport_info(\n                    ID,\n                    fun (Info) ->\n                        Info#{\n                            status => failed,\n                            end_ts => NowMillis,\n                            error => {rejected, Error}\n                        }\n                    end,\n                    Counters\n                ),\n                {error, Error}\n        end\n    catch\n        T:E:S ->\n            ?RAFT_COUNT(Table, 'transport.start.error'),\n            ?RAFT_LOG_WARNING(\n                \"wa_raft_transport failed to start transport ~p due to ~p ~p: ~n~p\",\n                [ID, T, E, S]\n            ),\n            update_and_get_transport_info(\n                ID,\n                fun (Info) ->\n                    Info#{\n                        status => failed,\n                        end_ts => erlang:system_time(millisecond),\n                        error => {start, {T, E, S}}\n                    }\n                end,\n                Counters\n            ),\n            {error, failed}\n    end.\n\n-spec transport_module(Meta :: meta()) -> module().\ntransport_module(#{table := Table, partition := Partition}) ->\n    wa_raft_transport:registered_module(Table, Partition);\ntransport_module(_Meta) ->\n    ?RAFT_DEFAULT_TRANSPORT_MODULE.\n\n-spec transport_destination(ID :: transport_id(), Meta :: meta()) -> string().\ntransport_destination(ID, #{type := transfer, table := Table, partition := Partition}) ->\n    filename:join(wa_raft_transport:registered_directory(Table, Partition), integer_to_list(ID));\ntransport_destination(ID, #{type := snapshot, table := Table, partition := Partition}) ->\n    filename:join(wa_raft_transport:registered_directory(Table, Partition), integer_to_list(ID)).\n\n-spec collect_files(string()) -> [{non_neg_integer(), string(), string(), integer(), non_neg_integer()}].\ncollect_files(Root) ->\n    {_, Files} = collect_files_impl(Root, [\"\"],\n        fun (Filename, Path, #file_info{size = Size, mtime = MTime}, {FileID, Acc}) ->\n            {FileID + 1, [{FileID, filename:flatten(Filename), filename:flatten(Path), MTime, Size} | Acc]}\n        end, {1, []}),\n    Files.\n\n-spec collect_files_impl(\n    string(), list(), fun(), {integer(), [{non_neg_integer(), string(), string(), integer(), non_neg_integer()}]}\n) -> {integer(), [{non_neg_integer(), string(), string(), integer(), non_neg_integer()}]}.\ncollect_files_impl(_Root, [], _Fun, Acc) ->\n    Acc;\ncollect_files_impl(Root, [Filename | Queue], Fun, Acc0) ->\n    Path = [Root, $/, Filename],\n    case prim_file:read_file_info(Path, [{time, posix}]) of\n        {ok, #file_info{type = regular} = Info} ->\n            Acc1 = Fun(Filename, Path, Info, Acc0),\n            collect_files_impl(Root, Queue, Fun, Acc1);\n        {ok, #file_info{type = directory}} ->\n            case prim_file:list_dir(Path) of\n                {ok, Files} ->\n                    NewQueue = lists:foldl(fun (Subfile, Acc) -> [join_names(Filename, Subfile) | Acc] end, Queue, Files),\n                    collect_files_impl(Root, NewQueue, Fun, Acc0);\n                {error, Reason} ->\n                    ?RAFT_LOG_ERROR(\"wa_raft_transport failed to list files in ~p due to ~p\", [filename:flatten(Path), Reason]),\n                    throw({list_dir, Reason})\n            end;\n        {ok, #file_info{type = Type}} ->\n            ?RAFT_LOG_WARNING(\"wa_raft_transport skipping file ~p with unknown type ~p\", [filename:flatten(Path), Type]),\n            collect_files_impl(Root, Queue, Fun, Acc0);\n        {error, Reason} ->\n            ?RAFT_LOG_ERROR(\"wa_raft_transport failed to read info of file ~p due to ~p\", [filename:flatten(Path), Reason]),\n            throw({read_file_info, Reason})\n    end.\n\n-spec join_names(string(), string()) -> string() | [string() | char()].\njoin_names(\"\", Name) -> Name;\njoin_names(Dir, Name) -> [Dir, $/, Name].\n\n-spec maybe_notify_complete(transport_id(), transport_info(), #state{}) -> ok | {error, term()}.\nmaybe_notify_complete(_ID, #{type := sender}, _State) ->\n    ok;\nmaybe_notify_complete(_ID, #{status := Status}, _State) when Status =/= completed ->\n    ok;\nmaybe_notify_complete(ID, #{type := receiver, root := Root, meta := #{type := snapshot, table := Table, partition := Partition, position := LogPos}}, #state{}) ->\n    try wa_raft_server:snapshot_available(wa_raft_server:registered_name(Table, Partition), Root, LogPos) of\n        ok ->\n            ok;\n        {error, Reason} ->\n            ?RAFT_LOG_NOTICE(\n                \"wa_raft_transport failed to notify ~p of transport ~p completion due to ~p\",\n                [wa_raft_server:registered_name(Table, Partition), ID, Reason]\n            ),\n            {error, Reason}\n    catch\n        T:E:S ->\n            ?RAFT_LOG_NOTICE(\n                \"wa_raft_transport failed to notify ~p of transport ~p completion due to ~p ~p: ~n~p\",\n                [wa_raft_server:registered_name(Table, Partition), ID, T, E, S]\n            ),\n            {error, {T, E, S}}\n    end;\nmaybe_notify_complete(ID, _Info, #state{}) ->\n    ?RAFT_LOG_NOTICE(\"wa_raft_transport finished transport ~p but does not know what to do with it\", [ID]).\n\n-spec maybe_notify(transport_id(), transport_info()) -> transport_info().\nmaybe_notify(ID, #{status := Status, notify := Notify, start_ts := Start, end_ts := End} = Info) when Status =/= requested, Status =/= running ->\n    Table = maps:get(table, maps:get(meta, Info, #{}), undefined),\n    ?RAFT_COUNT(Table, {'transport', Status}),\n    ?RAFT_GATHER_LATENCY(Table, {'transport.latency_ms', Status}, End - Start),\n    gen_server:reply(Notify, {ok, ID}),\n    maps:remove(notify, Info);\nmaybe_notify(_ID, Info) ->\n    Info.\n\n-spec scan_transport(ID :: transport_id(), Info :: transport_info()) -> NewInfo :: transport_info().\nscan_transport(ID, #{status := running, atomics := TransportAtomics} = Info) ->\n    LastUpdateTs = atomics:get(TransportAtomics, ?RAFT_TRANSPORT_ATOMICS_UPDATED_TS),\n    NowMillis = erlang:system_time(millisecond),\n    case NowMillis - LastUpdateTs >= ?RAFT_TRANSPORT_IDLE_TIMEOUT() * 1000 of\n        true  -> maybe_notify(ID, Info#{status := timed_out, end_ts => NowMillis});\n        false -> Info\n    end;\nscan_transport(_ID, Info) ->\n    Info.\n\n-spec schedule_scan() -> reference().\nschedule_scan() ->\n    erlang:send_after(?RAFT_TRANSPORT_SCAN_INTERVAL_SECS * 1000, self(), scan).\n\n-spec normalize_status(term()) -> atom().\nnormalize_status(Status) when is_atom(Status) ->\n    Status;\nnormalize_status({_Error, Reason}) when is_atom(Reason) ->\n    Reason;\nnormalize_status({_Error, Reason}) when is_tuple(Reason) ->\n    normalize_status(element(1, Reason));\nnormalize_status({Error, _Reason}) when is_atom(Error) ->\n    Error;\nnormalize_status(_) ->\n    unknown.\n"
  },
  {
    "path": "src/wa_raft_transport_cleanup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n\n-module(wa_raft_transport_cleanup).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%% OTP supervision\n-export([\n    child_spec/1,\n    start_link/1\n]).\n\n%% Internal API\n-export([\n    default_name/2,\n    registered_name/2\n]).\n\n%% Server Callbacks\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    handle_info/2\n]).\n\n-define(RAFT_TRANSPORT_CLEANUP_SCAN_INTERVAL_SECS, 30).\n\n-record(state, {\n    application :: atom(),\n    name :: atom(),\n    directory :: file:filename()\n}).\n\n%%-------------------------------------------------------------------\n%% OTP Supervision\n%%-------------------------------------------------------------------\n\n-spec child_spec(Options :: #raft_options{}) -> supervisor:child_spec().\nchild_spec(Options) ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, [Options]},\n        restart => permanent,\n        shutdown => 5000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Options :: #raft_options{}) ->  gen_server:start_ret().\nstart_link(#raft_options{transport_cleanup_name = Name} = Options) ->\n    gen_server:start_link({local, Name}, ?MODULE, Options, []).\n\n%%-------------------------------------------------------------------\n%% Internal API\n%%-------------------------------------------------------------------\n\n%% Get the default name for the RAFT acceptor server associated with the\n%% provided RAFT partition.\n-spec default_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\ndefault_name(Table, Partition) ->\n    % elp:ignore W0023 bounded atom, one per table/partition at startup\n    binary_to_atom(<<\"raft_transport_cleanup_\", (atom_to_binary(Table))/binary, \"_\", (integer_to_binary(Partition))/binary>>).\n\n%% Get the registered name for the RAFT acceptor server associated with the\n%% provided RAFT partition or the default name if no registration exists.\n-spec registered_name(Table :: wa_raft:table(), Partition :: wa_raft:partition()) -> Name :: atom().\nregistered_name(Table, Partition) ->\n    case wa_raft_part_sup:options(Table, Partition) of\n        undefined -> default_name(Table, Partition);\n        Options   -> Options#raft_options.transport_cleanup_name\n    end.\n\n%%-------------------------------------------------------------------\n%% Server Callbacks\n%%-------------------------------------------------------------------\n\n-spec init(Options :: #raft_options{}) -> {ok, State :: #state{}}.\ninit(#raft_options{application = Application, transport_directory = Directory, transport_cleanup_name = Name}) ->\n    process_flag(trap_exit, true),\n    schedule_scan(),\n    {ok, #state{application = Application, name = Name, directory = Directory}}.\n\n-spec handle_call(Request :: term(), From :: gen_server:from(), State :: #state{}) -> {noreply, NewState :: #state{}}.\nhandle_call(Request, From, #state{name = Name} = State) ->\n    ?RAFT_LOG_WARNING(\"~p received unrecognized call ~0P from ~0p\", [Name, Request, 25, From]),\n    {noreply, State}.\n\n-spec handle_cast(Request :: term(), State :: #state{}) -> {noreply, NewState :: #state{}}.\nhandle_cast(Request, #state{name = Name} = State) ->\n    ?RAFT_LOG_NOTICE(\"~p got unrecognized cast ~0P\", [Name, Request, 25]),\n    {noreply, State}.\n\n-spec handle_info(Info :: term(), State :: #state{}) -> {noreply, NewState :: #state{}}.\nhandle_info(scan, #state{} = State) ->\n    maybe_cleanup(State),\n    schedule_scan(),\n    {noreply, State};\nhandle_info(Info, #state{name = Name} = State) ->\n    ?RAFT_LOG_NOTICE(\"~p got unrecognized info ~p\", [Name, Info]),\n    {noreply, State}.\n\n-spec maybe_cleanup(State :: #state{}) -> ok | {error, term()}.\nmaybe_cleanup(#state{application = App, name = Name, directory = Directory} = State) ->\n    case prim_file:list_dir(Directory) of\n        {ok, Files} ->\n            RetainMillis = ?RAFT_TRANSPORT_RETAIN_INTERVAL(App) * 1000,\n            NowMillis = erlang:system_time(millisecond),\n            lists:foreach(\n                fun (Filename) ->\n                    Path = filename:join(Directory, Filename),\n                    ID = list_to_integer(Filename),\n                    case wa_raft_transport:transport_info(ID) of\n                        {ok, #{end_ts := EndTs}} when NowMillis - EndTs > RetainMillis ->\n                            ?RAFT_LOG_NOTICE(\n                                \"~p deleting ~p due to expiring after transport ended\",\n                                [Name, Filename]\n                            ),\n                            cleanup(ID, Path, State);\n                        {ok, _Info} ->\n                            ok;\n                        not_found ->\n                            ?RAFT_LOG_NOTICE(\n                                \"~p deleting ~p due to having no associated transport\",\n                                [Name, Filename]\n                            ),\n                            cleanup(ID, Path, State)\n                    end\n                end, Files);\n        {error, enoent} ->\n            ok;\n        {error, Reason} ->\n            ?RAFT_LOG_WARNING(\n                \"~p failed to list transports for cleanup due to ~p\",\n                [Name, Reason]\n            ),\n            {error, Reason}\n    end.\n\n-spec cleanup(non_neg_integer(), string(), #state{}) -> ok | {error, term()}.\ncleanup(ID, Path, #state{name = Name}) ->\n    case file:del_dir_r(Path) of\n        ok ->\n            ok;\n        {error, Reason} ->\n            ?RAFT_LOG_WARNING(\n                \"~p failed to cleanup transport ~p due to ~p\",\n                [Name, ID, Reason]\n            ),\n            {error, Reason}\n    end.\n\n-spec schedule_scan() -> reference().\nschedule_scan() ->\n    erlang:send_after(?RAFT_TRANSPORT_CLEANUP_SCAN_INTERVAL_SECS * 1000, self(), scan).\n"
  },
  {
    "path": "src/wa_raft_transport_sup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% OTP supervisor for handling workers responsible for actual data\n%%% send and receive for RAFT transport mechanisms.\n\n-module(wa_raft_transport_sup).\n-compile(warn_missing_spec_all).\n-behaviour(supervisor).\n\n%% Internal API\n-export([\n    get_or_start/1\n]).\n\n%% OTP supervision callbacks\n-export([\n    child_spec/0,\n    start_link/0\n]).\n\n%% supervisor callbacks\n-export([\n    init/1\n]).\n\n%%% ------------------------------------------------------------------------\n%%%  OTP supervision callbacks\n%%%\n\n-spec get_or_start(node()) -> atom().\nget_or_start(Node) ->\n    Name = wa_raft_transport_target_sup:name(Node),\n    not is_pid(whereis(Name)) andalso\n        supervisor:start_child(?MODULE, wa_raft_transport_target_sup:child_spec(Node)),\n    Name.\n\n%%% ------------------------------------------------------------------------\n%%%  OTP supervision callbacks\n%%%\n\n-spec child_spec() -> supervisor:child_spec().\nchild_spec() ->\n    #{\n        id => ?MODULE,\n        start => {?MODULE, start_link, []},\n        restart => permanent,\n        shutdown => infinity,\n        type => supervisor,\n        modules => [?MODULE]\n    }.\n\n-spec start_link() -> supervisor:startlink_ret().\nstart_link() ->\n    supervisor:start_link({local, ?MODULE}, ?MODULE, []).\n\n%%% ------------------------------------------------------------------------\n%%%  supervisor callbacks\n%%%\n\n-spec init(term()) -> {ok, {supervisor:sup_flags(), [supervisor:child_spec()]}}.\ninit(_) ->\n    {ok, {#{strategy => one_for_one, intensity => 5, period => 1}, []}}.\n"
  },
  {
    "path": "src/wa_raft_transport_target_sup.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n%%%\n%%% Supervisor responsible for managing workers responsible for the\n%%% transport to a particular target node.\n\n-module(wa_raft_transport_target_sup).\n-compile(warn_missing_spec_all).\n-behaviour(supervisor).\n\n%% Internal API\n-export([\n    name/1\n]).\n\n%% OTP supervision callbacks\n-export([\n    child_spec/1,\n    start_link/1\n]).\n\n%% Supervisor callbacks\n-export([\n    init/1\n]).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n\n%%% ------------------------------------------------------------------------\n%%%  Internal API\n%%%\n\n-spec name(node()) -> atom().\nname(Name) ->\n    % elp:ignore W0023 bounded atom for supervisor name\n    binary_to_atom(<<\"raft_transport_target_sup_\", (atom_to_binary(Name))/binary>>).\n\n%%% ------------------------------------------------------------------------\n%%%  OTP supervision callbacks\n%%%\n\n-spec child_spec(node()) -> supervisor:child_spec().\nchild_spec(Node) ->\n    #{\n        id => Node,\n        start => {?MODULE, start_link, [Node]},\n        restart => temporary,\n        shutdown => infinity,\n        type => supervisor,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(node()) -> supervisor:startlink_ret().\nstart_link(Node) ->\n    supervisor:start_link({local, name(Node)}, ?MODULE, Node).\n\n%%% ------------------------------------------------------------------------\n%%%  supervisor callbacks\n%%%\n\n-spec init(node()) -> {ok, {supervisor:sup_flags(), [supervisor:child_spec()]}}.\ninit(Node) ->\n    NumThreads = ?RAFT_TRANSPORT_THREADS(),\n    Specs = [wa_raft_transport_worker:child_spec(Node, N) || N <- lists:seq(1, NumThreads)],\n    {ok, {#{strategy => one_for_all, intensity => 5, period => 1}, Specs}}.\n"
  },
  {
    "path": "src/wa_raft_transport_worker.erl",
    "content": "%%% Copyright (c) Meta Platforms, Inc. and affiliates. All rights reserved.\n%%%\n%%% This source code is licensed under the Apache 2.0 license found in\n%%% the LICENSE file in the root directory of this source tree.\n\n-module(wa_raft_transport_worker).\n-compile(warn_missing_spec_all).\n-behaviour(gen_server).\n\n-include_lib(\"wa_raft/include/wa_raft.hrl\").\n-include_lib(\"wa_raft/include/wa_raft_logger.hrl\").\n\n%% OTP supervision\n-export([\n    child_spec/2,\n    start_link/2\n]).\n\n%% gen_server callbacks\n-export([\n    init/1,\n    handle_call/3,\n    handle_cast/2,\n    handle_info/2,\n    terminate/2\n]).\n\n-define(CONTINUE_TIMEOUT, 0).\n\n-record(state, {\n    node :: node(),\n    number :: non_neg_integer(),\n    jobs = queue:new() :: queue:queue(job()),\n    states = #{} :: #{module() => state()}\n}).\n-type state() :: #state{}.\n\n-record(transport, {\n    id :: wa_raft_transport:transport_id(),\n    table :: wa_raft:table() | undefined\n}).\n-record(file, {\n    id :: wa_raft_transport:transport_id(),\n    table :: wa_raft:table() | undefined,\n    file :: wa_raft_transport:file_id()\n}).\n-type job() :: #transport{} | #file{}.\n\n%%% ------------------------------------------------------------------------\n%%%  OTP supervision callbacks\n%%%\n\n-spec child_spec(Node :: node(), Number :: non_neg_integer()) -> supervisor:child_spec().\nchild_spec(Node, Number) ->\n    #{\n        id => {?MODULE, Node, Number},\n        start => {?MODULE, start_link, [Node, Number]},\n        restart => permanent,\n        shutdown => 5000,\n        modules => [?MODULE]\n    }.\n\n-spec start_link(Node :: node(), Number :: non_neg_integer()) -> gen_server:start_ret().\nstart_link(Node, Number) ->\n    gen_server:start_link(?MODULE, {Node, Number}, []).\n\n%%% ------------------------------------------------------------------------\n%%%  gen_server callbacks\n%%%\n\n-spec init(Args :: {node(), non_neg_integer()}) -> {ok, State :: state(), Timeout :: timeout()}.\ninit({Node, Number}) ->\n    {ok, #state{node = Node, number = Number}, ?CONTINUE_TIMEOUT}.\n\n-spec handle_call(Request :: term(), From :: {Pid :: pid(), Tag :: term()}, State :: state()) ->\n    {noreply, NewState :: state(), Timeout :: timeout()}.\nhandle_call(Request, From, #state{number = Number} = State) ->\n    ?RAFT_LOG_WARNING(\"[~p] received unrecognized call ~p from ~p\", [Number, Request, From]),\n    {noreply, State, ?CONTINUE_TIMEOUT}.\n\n-spec handle_cast(Request, State :: state()) -> {noreply, NewState :: state(), Timeout :: timeout()}\n    when Request :: {notify, wa_raft_transport:transport_id(), wa_raft:table() | undefined}.\nhandle_cast({notify, ID, Table}, #state{jobs = Jobs} = State) ->\n    {noreply, State#state{jobs = queue:in(#transport{id = ID, table = Table}, Jobs)}, ?CONTINUE_TIMEOUT};\nhandle_cast(Request, #state{number = Number} = State) ->\n    ?RAFT_LOG_WARNING(\"[~p] received unrecognized cast ~p\", [Number, Request]),\n    {noreply, State, ?CONTINUE_TIMEOUT}.\n\n-spec handle_info(Info :: term(), State :: state()) ->\n      {noreply, NewState :: state()}\n    | {noreply, NewState :: state(), Timeout :: timeout() | hibernate}.\nhandle_info(timeout, #state{number = Number, jobs = Jobs, states = States} = State) ->\n    case queue:out(Jobs) of\n        {empty, NewJobs} ->\n            {noreply, State#state{jobs = NewJobs}, hibernate};\n        {{value, #transport{id = ID, table = Table}}, NewJobs} ->\n            case wa_raft_transport:pop_file(ID) of\n                {ok, FileID} ->\n                    ?RAFT_COUNT(Table, 'transport.file.send'),\n                    wa_raft_transport:update_file_info(ID, FileID,\n                        fun (Info) -> Info#{status => sending, start_ts => erlang:system_time(millisecond)} end),\n                    NewJob = #file{id = ID, table = Table, file = FileID},\n                    {noreply, State#state{jobs = queue:in(NewJob, NewJobs)}, ?CONTINUE_TIMEOUT};\n                _Other ->\n                    {noreply, State#state{jobs = NewJobs}, ?CONTINUE_TIMEOUT}\n            end;\n        {{value, #file{id = ID, file = FileID} = Job}, NewJobs} ->\n            {Result, NewState} = case wa_raft_transport:transport_info(ID) of\n                {ok, #{module := Module}} ->\n                    try get_module_state(Module, State) of\n                        {ok, ModuleState0} ->\n                            try Module:transport_send(ID, FileID, ModuleState0) of\n                                {ok, ModuleState1} ->\n                                    {ok, State#state{states = States#{Module => ModuleState1}}};\n                                {continue, ModuleState1} ->\n                                    {continue, State#state{states = States#{Module => ModuleState1}}};\n                                {stop, Reason, ModuleState1} ->\n                                    {{stop, Reason}, State#state{states = States#{Module => ModuleState1}}}\n                            catch\n                                T:E:S ->\n                                    ?RAFT_LOG_WARNING(\n                                        \"[~p] module ~p failed to send file ~p:~p due to ~p ~p: ~p\",\n                                        [Number, Module, ID, FileID, T, E, S]\n                                    ),\n                                    {{T, E}, State}\n                            end;\n                        Other ->\n                            {Other, State}\n                    catch\n                        T:E:S ->\n                            ?RAFT_LOG_WARNING(\n                                \"[~p] module ~p failed to get/init module state due to ~p ~p: ~p\",\n                                [Number, Module, T, E, S]\n                            ),\n                            {{T, E}, State}\n                    end;\n                _ ->\n                    ?RAFT_LOG_WARNING(\"[~p] trying to send for unknown transfer ~p\", [Number, ID]),\n                    {{stop, invalid_transport}, State}\n            end,\n            case Result =:= continue of\n                true ->\n                    {noreply, NewState#state{jobs = queue:in(Job, NewJobs)}, ?CONTINUE_TIMEOUT};\n                false ->\n                    wa_raft_transport:complete(ID, FileID, Result),\n                    {noreply, NewState#state{jobs = queue:in(#transport{id = ID}, NewJobs)}, ?CONTINUE_TIMEOUT}\n            end\n    end;\nhandle_info(Info, #state{number = Number} = State) ->\n    ?RAFT_LOG_WARNING(\"[~p] received unrecognized info ~p\", [Number, Info]),\n    {noreply, State, ?CONTINUE_TIMEOUT}.\n\n-spec terminate(term(), state()) -> ok.\nterminate(Reason, #state{states = States}) ->\n    [\n        case erlang:function_exported(Module, transport_terminate, 2) of\n            true  -> Module:transport_terminate(Reason, State);\n            false -> ok\n        end\n     || Module := State <- States\n    ],\n    ok.\n\n-spec get_module_state(module(), state()) -> {ok, state()} | {stop, term()}.\nget_module_state(Module, #state{node = Node, states = States}) ->\n    case States of\n        #{Module := ModuleState} -> {ok, ModuleState};\n        _                        -> Module:transport_init(Node)\n    end.\n"
  }
]