Skip to content

Commit

Permalink
khepri_clustering: Move cluster management API to this new module
Browse files Browse the repository at this point in the history
  • Loading branch information
dumbbell committed Apr 11, 2022
1 parent ab82ae9 commit 06e6d83
Show file tree
Hide file tree
Showing 3 changed files with 177 additions and 121 deletions.
173 changes: 60 additions & 113 deletions src/khepri.erl
Original file line number Diff line number Diff line change
Expand Up @@ -5,81 +5,13 @@
%% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved.
%%

%% @doc Khepri high-level and cluster management API.
%% @doc Khepri database API.
%%
%% This module exposes the high-level API to manipulate data and the cluster
%% management API.
%% This module exposes the database API to manipulate data.
%%
%% == Cluster management ==
%%
%% === Starting a Ra system ===
%%
%% The default store is based on Ra's default system. You need to change the
%% Ra application configuration if you want to set settings. That said, it is
%% recommended to start your own Ra system. This way, even though Ra is
%% already running, you can choose where the Khepri data should be stored.
%% This is also required if you need to run multiple database instances in
%% parallel.
%%
%% Here is a quick start example:
%%
%% ```
%% %% We start Khepri. Ra is also started because Khepri depends on it.
%% {ok, _} = application:ensure_all_started(khepri),
%%
%% %% We define the configuration of the Ra system for our database. Here, we
%% %% only care about the directory where data will be written.
%% RaSystem = my_ra_system,
%% RaSystemDataDir = "/path/to/storage/dir",
%% DefaultSystemConfig = ra_system:default_config(),
%% RaSystemConfig = DefaultSystemConfig#{name => RaSystem,
%% data_dir => RaSystemDataDir,
%% wal_data_dir => RaSystemDataDir,
%% names => ra_system:derive_names(
%% RaSystem)},
%%
%% %% The configuration is ready, let's start the Ra system.
%% {ok, _RaSystemPid} = ra_system:start(RaSystemConfig),
%%
%% %% At last we can start Khepri! We need to choose a name for the Ra cluster
%% %% running in the Ra system started above. This must be an atom.
%% RaClusterName = my_khepri_db,
%% RaClusterFriendlyName = "My Khepri DB",
%% {ok, StoreId} = khepri:start(
%% RaSystem,
%% RaClusterName,
%% RaClusterFriendlyName),
%%
%% %% The Ra cluster name is our "store ID" used everywhere in the Khepri API.
%% khepri:insert(StoreId, [stock, wood], 156).
%% '''
%%
%% Please refer to <a href="https://github.com/rabbitmq/ra">Ra
%% documentation</a> to learn more about Ra systems and Ra clusters.
%%
%% === Managing Ra cluster members ===
%%
%% To add a member to your Ra cluster:
%%
%% ```
%% khepri:add_member(
%% RaSystem,
%% RaClusterName,
%% RaClusterFriendlyName,
%% NewMemberErlangNodename).
%% '''
%%
%% To remove a member from your Ra cluster:
%%
%% ```
%% khepri:remove_member(
%% RaClusterName,
%% MemberErlangNodenameToRemove).
%% '''
%%
%% == Data manipulation ==
%%
%% See individual functions for more details.
%% As a convenience, some functions of the service and cluster management API
%% are also repeated in this module. See the {@link khepri_clustering} module
%% for the documentation.

-module(khepri).

Expand All @@ -92,15 +24,7 @@
-export([start/0,
start/1,
start/3,
add_member/2,
add_member/4,
remove_member/1,
remove_member/2,
reset/2,
members/1,
locally_known_members/1,
nodes/1,
locally_known_nodes/1,
get_store_ids/0,

put/2, put/3, put/4, put/5,
Expand Down Expand Up @@ -143,7 +67,9 @@
%% please Dialyzer. So for now, let's disable this specific check for the
%% problematic functions.
-if(?OTP_RELEASE >= 24).
-dialyzer({no_underspecs, [put/2, put/3,
-dialyzer({no_underspecs, [start/1, start/3,

put/2, put/3,
create/2, create/3,
update/2, update/3,
compare_and_swap/3, compare_and_swap/4,
Expand Down Expand Up @@ -177,54 +103,74 @@
error/0]).

%% -------------------------------------------------------------------
%% Database management.
%% Service management.
%% -------------------------------------------------------------------

-spec start() -> {ok, store_id()} | {error, any()}.
-spec start() -> Ret when
Ret :: ok(StoreId) | error(),
StoreId :: store_id().
%% @doc Starts a store on the default Ra system.
%%
%% The store uses the default Ra cluster name and cluster friendly name.
%%
%% @see khepri_clustering:start/0.

start() ->
khepri_clustering:start().

-spec start(atom()) ->
{ok, store_id()} | {error, any()}.
-spec start(RaSystem) -> Ret when
RaSystem :: atom(),
Ret :: ok(StoreId) | error(),
StoreId :: store_id().
%% @doc Starts a store on the specified Ra system.
%%
%% The store uses the default Ra cluster name and cluster friendly name.
%%
%% @param RaSystem the name of the Ra system.
%%
%% @see khepri_clustering:start/1.

start(RaSystem) ->
khepri_clustering:start(RaSystem).

-spec start(atom(), ra:cluster_name(), string()) ->
{ok, store_id()} | {error, any()}.
-spec start(RaSystem, ClusterName, FriendlyName) -> Ret when
RaSystem :: atom(),
ClusterName :: ra:cluster_name(),
FriendlyName :: string(),
Ret :: ok(StoreId) | error(),
StoreId :: store_id().
%% @doc Starts a store on the specified Ra system.
%%
%% @param RaSystem the name of the Ra system.
%% @param ClusterName the name of the Ra cluster.
%% @param FriendlyName the friendly name of the Ra cluster.
%%
%% @see khepri_clustering:start/3.

start(RaSystem, ClusterName, FriendlyName) ->
khepri_clustering:start(RaSystem, ClusterName, FriendlyName).

add_member(RaSystem, NewNode) ->
khepri_clustering:add_member(RaSystem, NewNode).

add_member(RaSystem, ClusterName, FriendlyName, NewNode) ->
khepri_clustering:add_member(RaSystem, ClusterName, FriendlyName, NewNode).

remove_member(NodeToRemove) ->
khepri_clustering:remove_member(NodeToRemove).

remove_member(ClusterName, NodeToRemove) ->
khepri_clustering:remove_member(ClusterName, NodeToRemove).
-spec reset(RaSystem, ClusterName) -> Ret when
RaSystem :: atom(),
ClusterName :: ra:cluster_name(),
Ret :: ok | error() | {badrpc, any()}.
%% @doc Resets the store on this Erlang node.
%%
%% It does that by force-deleting the Ra local server.
%%
%% @param RaSystem the name of the Ra system.
%% @param ClusterName the name of the Ra cluster.
%%
%% @see khepri_clustering:reset/2.

reset(RaSystem, ClusterName) ->
khepri_clustering:reset(RaSystem, ClusterName).

members(ClusterName) ->
khepri_clustering:members(ClusterName).

locally_known_members(ClusterName) ->
khepri_clustering:locally_known_members(ClusterName).

nodes(ClusterName) ->
khepri_clustering:nodes(ClusterName).

locally_known_nodes(ClusterName) ->
khepri_clustering:locally_known_nodes(ClusterName).

-spec get_store_ids() -> [store_id()].
-spec get_store_ids() -> [StoreId] when
StoreId :: store_id().
%% @doc Returns the list of running stores.
%%
%% @see khepri_clustering:get_store_ids/0.

get_store_ids() ->
khepri_clustering:get_store_ids().
Expand Down Expand Up @@ -1891,7 +1837,8 @@ info() ->

info(StoreId) ->
io:format("~n\033[1;32m== CLUSTER MEMBERS ==\033[0m~n~n", []),
Nodes = lists:sort([Node || {_, Node} <- members(StoreId)]),
Nodes = lists:sort(
[Node || {_, Node} <- khepri_clustering:members(StoreId)]),
lists:foreach(fun(Node) -> io:format("~ts~n", [Node]) end, Nodes),

case khepri_machine:get_keep_while_conds_state(StoreId) of
Expand Down
Loading

0 comments on commit 06e6d83

Please sign in to comment.