diff --git a/README.md b/README.md index f02952cd..85dcbcb4 100644 --- a/README.md +++ b/README.md @@ -89,10 +89,6 @@ khepri:insert([emails, <<"alice">>], "alice@example.org"). khepri:insert("/:emails/alice", "alice@example.org"). ``` -The `khepri` module provides the "simple API". It has several functions to -cover the most common uses. For advanced uses, using the `khepri_machine` -module directly is preferred. - ### Read data back To get Alice's email address back, **query** the same path: @@ -147,9 +143,7 @@ khepri:transaction( %% There is less than 100 pieces of wood, or there is none %% at all (the node does not exist in Khepri). We need to %% request a new order. - {ok, _} = khepri_tx:put( - [order, wood], - #kpayload_data{data = 1000}), + {ok, _} = khepri_tx:put([order, wood], 1000), true end end). @@ -178,18 +172,16 @@ the database itself and automatically execute it after some event occurs. on_action => Action} = Props end, - khepri_machine:put( - StoreId, - StoredProcPath, - #kpayload_sproc{sproc = Fun}))}. + khepri:put(StoreId, StoredProcPath, Fun). ``` 2. Register a trigger using an event filter: ```erlang - EventFilter = #kevf_tree{path = [stock, wood, <<"oak">>]}, + %% A path is automatically considered a tree event filter. + EventFilter = [stock, wood, <<"oak">>], - ok = khepri_machine:register_trigger( + ok = khepri:register_trigger( StoreId, TriggerId, EventFilter, diff --git a/doc/overview.edoc b/doc/overview.edoc index 7c84e415..ded31ca7 100644 --- a/doc/overview.edoc +++ b/doc/overview.edoc @@ -44,9 +44,9 @@ Because RabbitMQ already uses an implementation of the Raft consensus algorithm for its quorum queues, it was decided to leverage that library for all metadata. That's how Khepri was borned. -Thanks to Ra and Raft, it is clear how Khepri will behave during and -recover from a network partition. This makes it more comfortable for -the RabbitMQ team and users, thanks to the absence of unknowns. +Thanks to Ra and Raft, it is clear how Khepri will behave during a +network partition and recover from it. This makes it more comfortable +for the RabbitMQ team and users, thanks to the absence of unknowns.
At the time of this writing, RabbitMQ does not use Khepri in a production @@ -89,29 +89,24 @@ A tree node may or may not have a payload. Khepri supports two types of payload, the data payload and the stored procedure payload. More payload types may be added in the future. -Payloads are represented using macros or helper functions: +When passed to {@link khepri:put/2}, the type of the payload is autodetected. +However if you need to prepare the payload before passing it to Khepri, you can +use the following functions:-
-Functions in {@link khepri_machine} have no assumption on the type of the -payload because they are a low-level API. Therefore, it must be specified -explicitly using the macros or helper functions mentioned above. - -Most functions in {@link khepri}, being a higher-level API, target more -specific use cases and detect the type of payload. - === Properties === Properties are:- `none' and {@link khepri:no_payload/0}
-- `#kpayload_data{data = Term}' and {@link khepri:data_payload/1}
-- `#kpayload_sproc{sproc = Fun}' and {@link khepri:sproc_payload/1}
+- {@link khepri_payload:none/0}
+- {@link khepri_payload:data/1}
+- {@link khepri_payload:sproc/1}
=== Addressing a tree node === @@ -189,68 +184,45 @@ KeepWhileCondition = #{[stock, wood] => #if_child_list_length{count = {gt, 0}}}. `keep_while' conditions on self (like the example above) are not evaluated on the first insert though. -== Khepri API == +== Stores == + +A Khepri store corresponds to one Ra cluster. In fact, the name of the Ra +cluster is the name of the Khepri store. It is possible to have multiple +database instances running on the same Erlang node or cluster by starting +multiple Ra clusters. Note that it is called a "Ra cluster" but it can have a +single member. -=== High-level API === +By default, {@link khepri:start/0} starts a default store called `khepri', +based on Ra's default system. You can start a simple store using {@link +khepri:start/1}. To configure a cluster, you need to use {@link +khepri_clustering} to add or remove members. -A high-level API is provided by the {@link khepri} module. It covers most -common use cases and should be straightforward to use. +== Khepri API == + +The essential part of the public API is provided by the {@link khepri} module. +It covers most common use cases and should be straightforward to use. ``` -khepri:insert([stock, wood, <<"lime tree">>], 150), +{ok, _} = khepri:put([stock, wood, <<"lime tree">>], 150), Ret = khepri:get([stock, wood, <<"lime tree">>]), {ok, #{[stock, wood, <<"lime tree">>] => - #{child_list_count => 0, - child_list_version => 1, - data => 150, - payload_version => 1}}} = Ret, + #{data => 150, + payload_version => 1, + child_list_count => 0, + child_list_version => 1}}} = Ret, true = khepri:exists([stock, wood, <<"lime tree">>]), -khepri:delete([stock, wood, <<"lime tree">>]). +{ok, _} = khepri:delete([stock, wood, <<"lime tree">>]). ''' -=== Low-level API === +Inside transaction funtions, {@link khepri_tx} must be used instead of {@link +khepri}. The former provides the same API, except for functions which don't +make sense in the context of a transaction function. The high-level API is built on top of a low-level API. The low-level API is -provided by the {@link khepri_machine} module. - -The low-level API provides just a handful of primitives. More advanced or -specific use cases may need to rely on that low-level API. - -``` -%% Unlike the high-level API's `khepri:insert/2' function, this low-level -%% insert returns whatever it replaced (if anything). In this case, there was -%% nothing before, so the returned value is empty. -Ret1 = khepri_machine:put( - StoreId, [stock, wood, <<"lime tree">>], - #kpayload_data{data = 150}), -{ok, #{}} = Ret1, - -Ret2 = khepri_machine:get(StoreId, [stock, wood, <<"lime tree">>]), -{ok, #{[stock, wood, <<"lime tree">>] => - #{child_list_count => 0, - child_list_version => 1, - data => 150, - payload_version => 1}}} = Ret2, - -%% Unlike the high-level API's `khepri:delete/2' function, this low-level -%% delete returns whatever it deleted. -Ret3 = khepri_machine:delete(StoreId, [stock, wood, <<"lime tree">>]), -{ok, #{[stock, wood, <<"lime tree">>] => - #{child_list_count => 0, - child_list_version => 1, - data => 150, - payload_version => 1}}} = Ret3. -''' - -=== Stores === - -It is possible to have multiple database instances running on the same Erlang -node or cluster. - -By default, Khepri starts a default store, based on Ra's default system. +provided by the private {@link khepri_machine} module. == Transactions == @@ -273,8 +245,7 @@ next section need to be taken into account. The nature of the anonymous function is passed as the `ReadWrite' argument to -{@link khepri:transaction/3} or {@link khepri_machine:transaction/3} -functions. +{@link khepri:transaction/3}. === The constraints imposed by Raft === @@ -344,9 +315,9 @@ outside of the changes to the tree nodes. If the transaction needs to have side effects, there are two options:
- The version of the payload, tracking the number of times it was modified -({@link khepri_machine:payload_version()}).
+({@link khepri:payload_version()}).- The version of the list of child nodes, tracking the number of times child -nodes were added or removed ({@link khepri_machine:child_list_version()}).
-- The number of child nodes ({@link khepri_machine:child_list_count()}).
+nodes were added or removed ({@link khepri:child_list_version()}). +- The number of child nodes ({@link khepri:child_list_count()}).
Here is an example of the second option: @@ -355,7 +326,7 @@ Here is an example of the second option: Path = [stock, wood, <<"lime tree">>], {ok, #{Path := #{data = Term, payload_version = PayloadVersion}}} = - khepri_machine:get(StoredId, Path), + khepri:get(StoredId, Path), %% Do anything with `Term` that depend on external factors and could have side %% effects. @@ -367,8 +338,7 @@ PathPattern = [stock, conditions = [ <<"lime tree">>, #if_payload_version{version = PayloadVersion}]}], -Payload = #kpayload_data{data = Term1}, -case khepri_machine:put(StoredId, PathPattern, Payload) of +case khepri:put(StoredId, PathPattern, Term1) of {ok, _} -> ok; %% `Term1` was stored successfully. {error, {mismatching_node, _}} -> @@ -399,14 +369,13 @@ The indicated stored procedure must have been stored in the tree first. === Storing an anonymous function === -This is possible to store an anonymous function as the payload of a tree node -using the {@link khepri_machine:payload_sproc()} record: +This is possible to store an anonymous function as the payload of a tree node: ``` -khepri_machine:put( +khepri:put( StoreId, StoredProcPath, - #kpayload_sproc{sproc = fun() -> do_something() end}))}. + fun() -> do_something() end). ''' The `StoredProcPath' can be any path in the @@ -420,40 +389,47 @@ A stored procedure can accept any numbers of arguments too. It is possible to execute a stored procedure directly without configuring any triggers. To execute a stored procedure, you can call {@link -khepri_machine:run_sproc/3}. Here is an example: +khepri:run_sproc/3}. Here is an example: ``` -Ret = khepri_machine:run_sproc( +Ret = khepri:run_sproc( StoreId, StoredProcPath, [] = _Args). ''' This works exactly like {@link erlang:apply/2}. The list of arguments passed -to {@link khepri_machine:run_sproc/3} must correspond to the stored procedure +to {@link khepri:run_sproc/3} must correspond to the stored procedure arity. === Configuring a trigger === Khepri uses event filters to associate a type of events with a stored procedure. Khepri supports tree changes events and thus only supports a single -event filter called {@link khepri_machine:event_filter_tree()}. +event filter called {@link khepri_evf:tree_event_filter()}. -An event filter is registered using {@link khepri_machine:register_trigger/4}: +An event filter is registered using {@link khepri:register_trigger/4}: ``` -EventFilter = #kevf_tree{path = [stock, wood, <<"oak">>], %% Required - props = #{on_actions => [delete], %% Optional - priority => 10}}, %% Optional - -ok = khepri_machine:register_trigger( +%% An event filter can be explicitly created using the `khepri_evf' +%% module. This is possible to specify properties at the same time. +EventFilter = khepri_evf:tree([stock, wood, <<"oak">>], %% Required + #{on_actions => [delete], %% Optional + priority => 10}), %% Optional + +%% For ease of use, some terms can be automatically converted to an event % +%filter. Here, a Unix-like path could be used as a tree event % filter, though +%it would have default properties unlike the previous line: +EventFilter = "/:stock/:wood/oak". + +ok = khepri:register_trigger( StoreId, TriggerId, EventFilter, StoredProcPath))}. ''' -In this example, the {@link khepri_machine:event_filter_tree()} record only +In this example, the {@link khepri_evf:tree_event_filter()} structure only requires the path to monitor. The path can be any path pattern and thus can have conditions to monitor several nodes at once. diff --git a/include/khepri.hrl b/include/khepri.hrl index 67613ef6..6b08eb6a 100644 --- a/include/khepri.hrl +++ b/include/khepri.hrl @@ -33,17 +33,6 @@ -define(IS_PATH_PATTERN(Path), (Path =:= [] orelse ?IS_PATH_CONDITION(hd(Path)))). -%% ------------------------------------------------------------------- -%% Payload types. -%% ------------------------------------------------------------------- - --record(kpayload_data, {data :: khepri_machine:data()}). --record(kpayload_sproc, {sproc :: khepri_fun:standalone_fun()}). - --define(IS_KHEPRI_PAYLOAD(Payload), (Payload =:= none orelse - is_record(Payload, kpayload_data) orelse - is_record(Payload, kpayload_sproc))). - %% ------------------------------------------------------------------- %% Path conditions. %% ------------------------------------------------------------------- @@ -73,14 +62,14 @@ {exists = true :: boolean()}). -record(if_payload_version, - {version = 0 :: khepri_machine:payload_version() | + {version = 0 :: khepri:payload_version() | khepri_condition:comparison_op( - khepri_machine:payload_version())}). + khepri:payload_version())}). -record(if_child_list_version, - {version = 0 :: khepri_machine:child_list_version() | + {version = 0 :: khepri:child_list_version() | khepri_condition:comparison_op( - khepri_machine:child_list_version())}). + khepri:child_list_version())}). -record(if_child_list_length, {count = 0 :: non_neg_integer() | @@ -94,14 +83,3 @@ -record(if_any, {conditions = [] :: [khepri_path:pattern_component()]}). - -%% ------------------------------------------------------------------- -%% Event filtering. -%% ------------------------------------------------------------------- - --record(kevf_tree, {path :: khepri_path:pattern(), - props = #{} :: #{on_actions => [create | update | delete], - priority => integer()}}). -%-record(kevf_process, {pid :: pid(), -% props = #{} :: #{on_reason => ets:match_pattern(), -% priority => integer()}}). diff --git a/src/internal.hrl b/src/internal.hrl index d914db85..0dee424e 100644 --- a/src/internal.hrl +++ b/src/internal.hrl @@ -5,6 +5,9 @@ %% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved. %% +-define(DEFAULT_RA_CLUSTER_NAME, khepri). +-define(DEFAULT_RA_FRIENDLY_NAME, "Khepri datastore"). + -define(INIT_DATA_VERSION, 1). -define(INIT_CHILD_LIST_VERSION, 1). -define(INIT_NODE_STAT, #{payload_version => ?INIT_DATA_VERSION, @@ -13,39 +16,49 @@ -define(TX_STATE_KEY, khepri_tx_machine_state). -define(TX_PROPS, khepri_tx_properties). +-define(NO_PAYLOAD, '$__NO_PAYLOAD__'). +-record(p_data, {data :: khepri:data()}). +-record(p_sproc, {sproc :: khepri_fun:standalone_fun()}). + +-define(IS_KHEPRI_PAYLOAD(Payload), (Payload =:= ?NO_PAYLOAD orelse + is_record(Payload, p_data) orelse + is_record(Payload, p_sproc))). + +-record(evf_tree, {path :: khepri_path:pattern(), + props = #{} :: khepri_evf:tree_event_filter_props()}). +%-record(evf_process, {pid :: pid(), +% props = #{} :: #{on_reason => ets:match_pattern(), +% priority => integer()}}). + +-define(IS_KHEPRI_EVENT_FILTER(EventFilter), + (is_record(EventFilter, evf_tree))). + %% Structure representing each node in the tree, including the root node. %% TODO: Rename stat to something more correct? -record(node, {stat = ?INIT_NODE_STAT :: khepri_machine:stat(), - payload = none :: khepri_machine:payload(), + payload = ?NO_PAYLOAD :: khepri_payload:payload(), child_nodes = #{} :: #{khepri_path:component() := #node{}}}). %% State machine commands. -record(put, {path :: khepri_path:pattern(), - payload = none :: khepri_machine:payload(), + payload = ?NO_PAYLOAD :: khepri_payload:payload(), extra = #{} :: #{keep_while => - khepri_machine:keep_while_conds_map()}}). + khepri:keep_while_conds_map()}}). -record(delete, {path :: khepri_path:pattern()}). -record(tx, {'fun' :: khepri_fun:standalone_fun()}). --record(register_trigger, {id :: khepri_machine:trigger_id(), - event_filter :: khepri_machine:event_filter(), +-record(register_trigger, {id :: khepri:trigger_id(), + event_filter :: khepri_evf:event_filter(), sproc :: khepri_path:path()}). -record(ack_triggered, {triggered :: [khepri_machine:triggered()]}). --record(triggered, {id :: khepri_machine:trigger_id(), +-record(triggered, {id :: khepri:trigger_id(), %% TODO: Do we need a ref to distinguish multiple %% instances of the same trigger? - event_filter :: khepri_machine:event_filter(), + event_filter :: khepri_evf:event_filter(), sproc :: khepri_fun:standalone_fun(), props = #{} :: map()}). - -%% Structure representing an anonymous function "extracted" as a compiled -%% module for storage. --record(standalone_fun, {module :: module(), - beam :: binary(), - arity :: arity(), - env :: list()}). diff --git a/src/khepri.erl b/src/khepri.erl index dc27bf30..8b365e68 100644 --- a/src/khepri.erl +++ b/src/khepri.erl @@ -5,913 +5,1739 @@ %% 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 == +%% The API is mainly made of the functions used to perform simple direct +%% atomic operations and queries on the database. In addition to that, {@link +%% transaction/1} are the starting point to run transaction functions. However +%% the API to use inside transaction functions is provided by {@link +%% khepri_tx}. %% -%% === Starting a Ra system === +%% This module also provides functions to start and stop (in the future) a +%% simple unclustered Khepri store. For more advanced setup and clustering, +%% see {@link khepri_cluster}. %% -%% 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. +%% == A Khepri store == %% -%% Here is a quick start example: +%% A Khepri store is one instance of Khepri running inside a Ra cluster (which +%% could be made of a single Erlang node). It is possible to run multiple +%% Khepri stores in parallel by creating multiple Ra clusters. %% -%% ``` -%% %% 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 Ra -%% documentation to learn more about Ra systems and Ra clusters. +%% A Khepri store is started and configured with {@link start/0}, {@link +%% start/1} or {@link start/3}. To setup a cluster, see {@link +%% khepri_cluster}. %% -%% === Managing Ra cluster members === +%% When a store is started, a store ID {@link store_id/0} is returned. This +%% store ID is then used by the rest of this module's API. The returned store +%% ID currently corresponds exactly to the Ra cluster name. Currently, it must +%% be an atom; other types are unsupported. %% -%% To add a member to your Ra cluster: +%% == Interacting with the Khepri store == %% -%% ``` -%% khepri:add_member( -%% RaSystem, -%% RaClusterName, -%% RaClusterFriendlyName, -%% NewMemberErlangNodename). -%% ''' +%% The API provides two ways to interact with a Khepri store: +%%
- Perform any side effects after the transaction.
-- Use {@link khepri_machine:put/3} with {@link -khepri_condition:if_payload_version()} conditions in the path and retry if the -put fails because the version changed in between.
+- Use {@link khepri:put/3} with {@link khepri_condition:if_payload_version()} +conditions in the path and retry if the put fails because the version changed +in between.
+%%
%% -%% To remove a member from your Ra cluster: +%% Simple operations are calls like: +%%- Direct atomic function for simple operations
+%%- Transactions for more complex operations
+%%+%%
%% +%% Transactions are like Mnesia ones. The caller passes an anonymous function +%% to {@link transaction/1}, etc.: %% ``` -%% khepri:remove_member( -%% RaClusterName, -%% MemberErlangNodenameToRemove). +%% khepri:transaction( +%% fun() -> +%% khepri_tx:put(Path, Value) +%% end). %% ''' %% -%% == Data manipulation == -%% -%% See individual functions for more details. +%% Simple operations are more efficient than transactions, but transactions are +%% more flexible. -module(khepri). -include_lib("kernel/include/logger.hrl"). -include("include/khepri.hrl"). +-include("src/khepri_fun.hrl"). -include("src/internal.hrl"). --export([start/0, +-export([ + %% Functions to start & stop (in the future) a Khepri store; for more + %% advanced functions, including clustering, see `khepri_cluster'. + 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, - create/2, create/3, - insert/2, insert/3, - update/2, update/3, - compare_and_swap/3, compare_and_swap/4, + %% Simple direct atomic operations & queries. + put/2, put/3, put/4, put/5, + create/2, create/3, create/4, create/5, + update/2, update/3, update/4, update/5, + compare_and_swap/3, compare_and_swap/4, compare_and_swap/5, + compare_and_swap/6, - clear_payload/1, clear_payload/2, - delete/1, delete/2, + clear_payload/1, clear_payload/2, clear_payload/3, clear_payload/4, + delete/1, delete/2, delete/3, + exists/1, exists/2, exists/3, get/1, get/2, get/3, - exists/1, exists/2, - has_data/1, has_data/2, - list/1, list/2, - find/2, find/3, + get_node_props/1, get_node_props/2, get_node_props/3, + has_data/1, has_data/2, has_data/3, + get_data/1, get_data/2, get_data/3, + has_sproc/1, has_sproc/2, has_sproc/3, + run_sproc/2, run_sproc/3, run_sproc/4, + register_trigger/3, register_trigger/4, register_trigger/5, - transaction/1, transaction/2, transaction/3, - run_sproc/2, run_sproc/3, + list/1, list/2, list/3, + find/2, find/3, find/4, - clear_store/0, clear_store/1, + clear_store/0, clear_store/1, clear_store/2, - no_payload/0, - data_payload/1, - sproc_payload/1, + %% Transactions; `khepri_tx' provides the API to use inside + %% transaction functions. + transaction/1, transaction/2, transaction/3, transaction/4, info/0, info/1]). -%% For internal use only. --export([forget_store_ids/0]). - --compile({no_auto_import, [get/2]}). --type store_id() :: ra:cluster_name(). +-compile({no_auto_import, [get/2, put/2, erase/1]}). + +%% FIXME: Dialyzer complains about several functions with "optional" arguments +%% (but not all). I believe the specs are correct, but can't figure out how to +%% please Dialyzer. So for now, let's disable this specific check for the +%% problematic functions. +-if(?OTP_RELEASE >= 24). +-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, + exists/2, + has_data/2, + get_data/2, + has_sproc/2, + run_sproc/3, + transaction/2, transaction/3]}). +-endif. + +%% FIXME: The code currently expects that the Ra cluster name is an atom. +%% However, Ra accepts binaries and strings as well. We should probably fix +%% that at some point. +-type store_id() :: atom(). % ra:cluster_name(). %% ID of a Khepri store. +%% +%% This is the same as the Ra cluster name hosting the Khepri store. + +-type error(Type) :: {error, Type}. +%% Return value of a failed command or query. + +-type data() :: any(). +%% Data stored in a node's payload. + +-type payload_version() :: pos_integer(). +%% Number of changes made to the payload of a node. +%% +%% The payload version starts at 1 when a node is created. It is increased by 1 +%% each time the payload is added, modified or removed. + +-type child_list_version() :: pos_integer(). +%% Number of changes made to the list of child nodes of a node (child nodes +%% added or removed). +%% +%% The child list version starts at 1 when a node is created. It is increased +%% by 1 each time a child is added or removed. Changes made to existing nodes +%% are not reflected in this version. + +-type child_list_length() :: non_neg_integer(). +%% Number of direct child nodes under a tree node. + +-type node_props() :: + #{data => data(), + sproc => khepri_fun:standalone_fun(), + payload_version => payload_version(), + child_list_version => child_list_version(), + child_list_length => child_list_length(), + child_nodes => #{khepri_path:node_id() => node_props()}}. +%% Structure used to return properties, payload and child nodes for a specific +%% node. +%% +%%- Queries: {@link get/1}, {@link exists/1}, {@link has_data/1}, etc.
+%%- Updates: {@link put/2}, {@link delete/1}, etc.
+%%+%%
+ +-type node_props_map() :: #{khepri_path:path() => node_props()}. +%% Structure used to return a map of nodes and their associated properties, +%% payload and child nodes. +%% +%% This structure is used in the return value of all commands and queries. + +-type result() :: khepri:ok(node_props_map()) | + khepri:error(). +%% Return value of a query or synchronous command. + +-type keep_while_conds_map() :: #{khepri_path:path() => + khepri_condition:keep_while()}. +%% Per-node `keep_while' conditions. +%% +%% When a node is put with `keep_while' conditions, this node will be kept in +%% the database while each condition remains true for their associated path. +%% +%% Example: +%% ``` +%% khepri:put( +%% StoreId, +%% [foo], +%% Payload, +%% #{keep_while => #{ +%% %% The node `[foo]' will be removed as soon as `[bar]' is removed +%% %% because the condition associated with `[bar]' will not be true +%% %% anymore. +%% [bar] => #if_node_exists{exists = true} +%% }} +%% ). +%% ''' + +-type trigger_id() :: atom(). +%% An ID to identify a registered trigger. + +-type async_option() :: boolean() | + ra_server:command_correlation() | + ra_server:command_priority() | + {ra_server:command_correlation(), + ra_server:command_priority()}. +%% Option to indicate if the command should be synchronous or asynchronous. +%% +%% Values are: +%%- Payload version, child list version, and child list count are always +%% included in the structure. The reason the type spec does not make them +%% mandatory is for {@link khepri_utils:flat_struct_to_tree/1} which may +%% construct fake node props without them.
+%%- Data is only included if there is data in the node's payload. Absence of +%% data is represented as no `data' entry in this structure.
+%%- Child nodes are only included if requested.
+%%+%%
+ +-type favor_option() :: consistency | compromise | low_latency. +%% Option to indicate where to put the cursor between freshness of the +%% returned data and low latency of queries. +%% +%% Values are: +%%- `true' to perform an asynchronous low-priority command without a +%% correlation ID.
+%%- `false' to perform a synchronous command.
+%%- A correlation ID to perform an asynchronous low-priority command with +%% that correlation ID.
+%%- A priority to perform an asynchronous command with the specified +%% priority but without a correlation ID.
+%%- A combination of a correlation ID and a priority to perform an +%% asynchronous command with the specified parameters.
+%%+%%
+ +-type command_options() :: #{async => async_option()}. +%% Options used in commands. +%% +%% Commands are {@link put/5}, {@link delete/3} and read-write {@link +%% transaction/4}. +%% +%%- `consistent' means that a "consistent query" will be used in Ra. It +%% will return the most up-to-date piece of data the cluster agreed on. Note +%% that it could block and eventually time out if there is no quorum in the Ra +%% cluster.
+%%- `compromise' performs "leader queries" most of the time to reduce +%% latency, but uses "consistent queries" every 10 seconds to verify that the +%% cluster is healthy on a regular basis. It should be faster but may block +%% and time out like `consistent' and still return slightly out-of-date +%% data.
+%%- `low_latency' means that "local queries" are used exclusively. They are +%% the fastest and have the lowest latency. However, the returned data is +%% whatever the local Ra server has. It could be out-of-date if it has +%% troubles keeping up with the Ra cluster. The chance of blocking and timing +%% out is very small.
+%%+%%
+ +-type query_options() :: #{expect_specific_node => boolean(), + include_child_names => boolean(), + favor => favor_option()}. +%% Options used in queries. +%% +%%- `async' indicates the synchronous or asynchronous nature of the +%% command; see {@link async_option()}.
+%%+%%
-type ok(Type) :: {ok, Type}. +%% The result of a function after a successful call, wrapped in an "ok" tuple. -type error() :: error(any()). - --type error(Type) :: {error, Type}. -%% Return value of a failed command or query. +%% The error tuple returned by a function after a failure. -export_type([store_id/0, ok/1, - error/0]). + error/0, + + data/0, + payload_version/0, + child_list_version/0, + child_list_length/0, + node_props/0, + node_props_map/0, + result/0, + keep_while_conds_map/0, + trigger_id/0, + + async_option/0, + favor_option/0, + command_options/0, + query_options/0]). %% ------------------------------------------------------------------- -%% Database management. +%% Service management. %% ------------------------------------------------------------------- --define(DEFAULT_RA_CLUSTER_NAME, ?MODULE). --define(DEFAULT_RA_FRIENDLY_NAME, "Khepri datastore"). - --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_cluster:start/0. start() -> - case application:ensure_all_started(ra) of - {ok, _} -> - RaSystem = default, - case ra_system:start_default() of - {ok, _} -> start(RaSystem); - {error, {already_started, _}} -> start(RaSystem); - {error, _} = Error -> Error - end; - {error, _} = Error -> - Error - end. + khepri_cluster: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_cluster:start/1. start(RaSystem) -> - start(RaSystem, ?DEFAULT_RA_CLUSTER_NAME, ?DEFAULT_RA_FRIENDLY_NAME). + khepri_cluster: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_cluster:start/3. start(RaSystem, ClusterName, FriendlyName) -> - case application:ensure_all_started(khepri) of - {ok, _} -> - case ensure_started(RaSystem, ClusterName, FriendlyName) of - ok -> - ok = remember_store_id(ClusterName), - {ok, ClusterName}; - Error -> - Error - end; - Error -> - Error - end. + khepri_cluster:start(RaSystem, ClusterName, FriendlyName). -ensure_started(RaSystem, ClusterName, FriendlyName) -> - ThisNode = node(), - ThisMember = node_to_member(ClusterName, ThisNode), - ?LOG_DEBUG( - "Check if a local Ra server is running for cluster \"~s\"", - [ClusterName], - #{domain => [khepri, clustering]}), - case whereis(ClusterName) of - undefined -> - ?LOG_DEBUG( - "No local Ra server running for cluster \"~s\", " - "try to restart it", - [ClusterName], - #{domain => [khepri, clustering]}), - Lock = {ClusterName, self()}, - global:set_lock(Lock), - Ret = case ra:restart_server(RaSystem, ThisMember) of - {error, Reason} - when Reason == not_started orelse - Reason == name_not_registered -> - ?LOG_DEBUG( - "Ra cluster not running, try to start it", - [], - #{domain => [khepri, clustering]}), - do_start( - RaSystem, ClusterName, FriendlyName, - [ThisMember]); - ok -> - ok; - {error, {already_started, _}} -> - ok; - _ -> - ok - end, - global:del_lock(Lock), - Ret; - _ -> - ?LOG_DEBUG( - "Local Ra server running, part of cluster \"~s\"", - [ClusterName], - #{domain => [khepri, clustering]}), - ok - end. +-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_cluster:reset/2. -do_start(RaSystem, ClusterName, FriendlyName, Members) -> - RaServerConfigs = [make_ra_server_config( - ClusterName, FriendlyName, Member, Members) - || Member <- Members], - ?LOG_DEBUG( - "Starting a cluster, named \"~s\", with the following Ra server " - "configuration:~n~p", - [ClusterName, hd(RaServerConfigs)], - #{domain => [khepri, clustering]}), - case ra:start_cluster(RaSystem, RaServerConfigs) of - {ok, Started, _} -> - ?LOG_DEBUG( - "Started Ra server for cluster \"~s\" on ~p", - [ClusterName, Started], - #{domain => [khepri, clustering]}), - ok; - {error, cluster_not_formed} = Error -> - ?LOG_ERROR( - "Failed to start Ra server for cluster \"~s\" using the " - "following Ra server configuration:~n~p", - [ClusterName, hd(RaServerConfigs)], - #{domain => [khepri, clustering]}), - Error - end. +reset(RaSystem, ClusterName) -> + khepri_cluster:reset(RaSystem, ClusterName). -add_member(RaSystem, NewNode) -> - add_member( - RaSystem, ?DEFAULT_RA_CLUSTER_NAME, ?DEFAULT_RA_FRIENDLY_NAME, - NewNode). - -add_member(RaSystem, ClusterName, FriendlyName, NewNode) -> - ?LOG_DEBUG( - "Querying members of cluster \"~s\"", - [ClusterName], - #{domain => [khepri, clustering]}), - case members(ClusterName) of - ExistingMembers when ExistingMembers =/= [] -> - NewMember = node_to_member(ClusterName, NewNode), - case lists:member(NewMember, ExistingMembers) of - false -> - start_ra_server_and_add_member( - RaSystem, ClusterName, FriendlyName, ExistingMembers, - NewMember); - true -> - ?LOG_DEBUG( - "Member ~p is already part of cluster \"~s\"", - [NewMember, ClusterName], - #{domain => [khepri, clustering]}), - ok - end; - [] -> - ?LOG_ERROR( - "Failed to query members of cluster \"~s\"", - [ClusterName], - #{domain => [khepri, clustering]}), - {error, failed_to_query_cluster_members} - end. +-spec get_store_ids() -> [StoreId] when + StoreId :: store_id(). +%% @doc Returns the list of running stores. +%% +%% @see khepri_cluster:get_store_ids/0. -start_ra_server_and_add_member( - RaSystem, ClusterName, FriendlyName, ExistingMembers, NewMember) -> - Lock = {ClusterName, self()}, - global:set_lock(Lock), - RaServerConfig = make_ra_server_config( - ClusterName, FriendlyName, NewMember, ExistingMembers), - ?LOG_DEBUG( - "Adding member ~p to cluster \"~s\" with the following " - "configuration:~n~p", - [NewMember, ClusterName, RaServerConfig], - #{domain => [khepri, clustering]}), - case ra:start_server(RaSystem, RaServerConfig) of - ok -> - %% TODO: Take the timeout as an argument (+ have a default). - Timeout = 30000, - Ret = do_add_member( - ClusterName, ExistingMembers, NewMember, Timeout), - global:del_lock(Lock), - Ret; - Error -> - global:del_lock(Lock), - ?LOG_ERROR( - "Failed to start member ~p, required to add it to " - "cluster \"~s\": ~p", - [NewMember, ClusterName, Error], - #{domain => [khepri, clustering]}), - Error - end. +get_store_ids() -> + khepri_cluster:get_store_ids(). -do_add_member(ClusterName, ExistingMembers, NewMember, Timeout) -> - T0 = erlang:monotonic_time(), - Ret = ra:add_member(ExistingMembers, NewMember), - case Ret of - {ok, _, _} -> - ok; - Error when Timeout >= 0 -> - ?LOG_NOTICE( - "Failed to add member ~p to cluster \"~s\": ~p; " - "will retry for ~b milliseconds", - [NewMember, ClusterName, Error, Timeout], - #{domain => [khepri, clustering]}), - timer:sleep(500), - T1 = erlang:monotonic_time(), - TDiff = erlang:convert_time_unit(T1 - T0, native, millisecond), - TimeLeft = Timeout - TDiff, - do_add_member( - ClusterName, ExistingMembers, NewMember, TimeLeft); - Error -> - ?LOG_ERROR( - "Failed to add member ~p to cluster \"~s\": ~p; " - "aborting", - [NewMember, ClusterName, Error], - #{domain => [khepri, clustering]}), - Error - end. +%% ------------------------------------------------------------------- +%% Data manipulation. +%% ------------------------------------------------------------------- -remove_member(NodeToRemove) -> - remove_member(?DEFAULT_RA_CLUSTER_NAME, NodeToRemove). - -remove_member(ClusterName, NodeToRemove) -> - ?LOG_DEBUG( - "Querying members of cluster \"~s\"", - [ClusterName], - #{domain => [khepri, clustering]}), - case members(ClusterName) of - ExistingMembers when ExistingMembers =/= [] -> - MemberToRemove = node_to_member(ClusterName, NodeToRemove), - case lists:member(MemberToRemove, ExistingMembers) of - true -> - do_remove_member( - ClusterName, ExistingMembers, MemberToRemove); - false -> - ?LOG_DEBUG( - "Member ~p is not part of cluster \"~s\"", - [MemberToRemove, ClusterName], - #{domain => [khepri, clustering]}), - ok - end; - [] -> - ?LOG_ERROR( - "Failed to query members of cluster \"~s\"", - [ClusterName], - #{domain => [khepri, clustering]}), - {error, failed_to_query_cluster_members} - end. +-spec put(PathPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Creates or modifies a specific tree node in the tree structure. +%% +%% Calling this function is the same as calling `put(StoreId, PathPattern, +%% Data)' with the default store ID. +%% +%% @see put/3. -do_remove_member(ClusterName, ExistingMembers, MemberToRemove) -> - case ra:remove_member(ExistingMembers, MemberToRemove) of - {ok, _, _} -> - ok; - Error -> - ?LOG_ERROR( - "Failed to remove member ~p from cluster \"~s\": ~p; " - "aborting", - [MemberToRemove, ClusterName, Error], - #{domain => [khepri, clustering]}), - Error - end. +put(PathPattern, Data) -> + put(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Data). -reset(RaSystem, ClusterName) -> - ThisNode = node(), - ThisMember = node_to_member(ClusterName, ThisNode), - ?LOG_DEBUG( - "Resetting member ~p in cluster \"~s\"", - [ThisMember, ClusterName], - #{domain => [khepri, clustering]}), - ra:force_delete_server(RaSystem, ThisMember). - -members(ClusterName) -> - Fun = fun ra:members/1, - do_query_members(ClusterName, Fun). - -locally_known_members(ClusterName) -> - Fun = fun(CN) -> ra:members({local, CN}) end, - do_query_members(ClusterName, Fun). - -do_query_members(ClusterName, Fun) -> - ThisNode = node(), - ThisMember = node_to_member(ClusterName, ThisNode), - ?LOG_DEBUG( - "Query members in cluster \"~s\"", - [ClusterName], - #{domain => [khepri, clustering]}), - case Fun(ThisMember) of - {ok, Members, _} -> - ?LOG_DEBUG( - "Found the following members in cluster \"~s\": ~p", - [ClusterName, Members], - #{domain => [khepri, clustering]}), - Members; - Error -> - ?LOG_WARNING( - "Failed to query members in cluster \"~s\": ~p", - [ClusterName, Error], - #{domain => [khepri, clustering]}), - [] - end. +-spec put(StoreId, PathPattern, Data) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Creates or modifies a specific tree node in the tree structure. +%% +%% Calling this function is the same as calling `put(StoreId, PathPattern, +%% Data, #{}, #{})'. +%% +%% @see put/5. -nodes(ClusterName) -> - [Node || {_, Node} <- members(ClusterName)]. +put(StoreId, PathPattern, Data) -> + put(StoreId, PathPattern, Data, #{}, #{}). -locally_known_nodes(ClusterName) -> - [Node || {_, Node} <- locally_known_members(ClusterName)]. +-spec put(StoreId, PathPattern, Data, Extra | Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Creates or modifies a specific tree node in the tree structure. +%% +%% Calling this function is the same as calling `put(StoreId, PathPattern, +%% Data, Extra, Options)' with an empty `Extra' or `Options'. +%% +%% @see put/5. -node_to_member(ClusterName, Node) -> - {ClusterName, Node}. +put(StoreId, PathPattern, Data, #{keep_while := _} = Extra) -> + put(StoreId, PathPattern, Data, Extra, #{}); +put(StoreId, PathPattern, Data, Options) -> + put(StoreId, PathPattern, Data, #{}, Options). -make_ra_server_config(ClusterName, FriendlyName, Member, Members) -> - UId = ra:new_uid(ra_lib:to_binary(ClusterName)), - #{cluster_name => ClusterName, - id => Member, - uid => UId, - friendly_name => FriendlyName, - initial_members => Members, - log_init_args => #{uid => UId}, - machine => {module, khepri_machine, #{store_id => ClusterName}}}. +-spec put(StoreId, PathPattern, Data, Extra, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Creates or modifies a specific tree node in the tree structure. +%% +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. +%% +%% The path or path pattern must target a specific tree node. In other words, +%% updating many nodes with the same payload is denied. That fact is checked +%% before the node is looked up: so if a condition in the path could +%% potentially match several nodes, an error is returned, even though only one +%% node would match at the time. +%% +%% When using a simple path (i.e. without conditions), if the target node does +%% not exist, it is created using the given payload. If the target node exists, +%% it is updated with the given payload and its payload version is increased by +%% one. Missing parent nodes are created on the way. +%% +%% When using a path pattern, the behavior is the same. However if a condition +%% in the path pattern is not met, an error is returned and the tree structure +%% is not modified. +%% +%% If the target node is modified, the returned structure in the "ok" tuple +%% will have a single key corresponding to the resolved path of the target +%% node. The path will be the same as the argument if it was a simple path, or +%% the final path after conditions were applied if it was a path pattern. That +%% key will point to a map containing the properties and payload (if any) of +%% the node before the modification. +%% +%% If the target node is created, the returned structure in the "ok" tuple will +%% have a single key corresponding to the path of the target node. That key +%% will point to an empty map, indicating there was no existing node (i.e. +%% there was no properties or payload to return). +%% +%% The payload must be one of the following form: +%%- `expect_specific_node' indicates if the path is expected to point to a +%% specific tree node or could match many nodes.
+%%- `include_child_names' indicates if child names should be included in +%% the returned node properties map.
+%%- `favor' indicates where to put the cursor between freshness of the +%% returned data and low latency of queries; see {@link favor_option()}.
+%%+%%
+%% +%% It is possible to wrap the payload in its internal structure explicitly +%% using the {@link khepri_payload} module directly. +%% +%% The `Extra' map may specify put-specific options: +%%- An explicit absense of payload ({@link khepri_payload:no_payload()}), +%% using the marker returned by {@link khepri_payload:none/0}, meaning there +%% will be no payload attached to the node and the existing payload will be +%% discarded if any
+%%- An anonymous function; it will be considered a stored procedure and will +%% be wrapped in a {@link khepri_payload:sproc()} record
+%%- Any other term; it will be wrapped in a {@link khepri_payload:data()} +%% record
+%%+%%
+%% +%% The `Options' map may specify command-level options; see {@link +%% command_options()}. +%% +%% Example: +%% ``` +%% %% Insert a node at `/:foo/:bar', overwriting the previous value. +%% Result = khepri:put(ra_cluster_name, [foo, bar], new_value), +%% +%% %% Here is the content of `Result'. +%% {ok, #{[foo, bar] => #{data => old_value, +%% payload_version => 1, +%% child_list_version => 1, +%% child_list_length => 0}}} = Result. +%% ''' +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the node to create or +%% modify. +%% @param Data the Erlang term or function to store, or a {@link +%% khepri_payload:payload()} structure. +%% @param Extra extra options such as `keep_while' conditions. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous put, an `{ok, Result}' tuple with a +%% map with one entry, or an `{error, Reason}' tuple; in the case of an +%% asynchronous put, always `ok' (the actual return value may be sent by a +%% message if a correlation ID was specified). + +put(StoreId, PathPattern, Data, Extra, Options) -> + do_put(StoreId, PathPattern, Data, Extra, Options). + +-spec create(PathPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Creates a specific tree node in the tree structure only if it does not +%% exist. +%% +%% Calling this function is the same as calling `create(StoreId, PathPattern, +%% Data)' with the default store ID. +%% +%% @see create/3. --define(PT_STORE_IDS, {khepri, store_ids}). +create(PathPattern, Data) -> + create(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Data). -remember_store_id(ClusterName) -> - StoreIds = persistent_term:get(?PT_STORE_IDS, #{}), - StoreIds1 = StoreIds#{ClusterName => true}, - persistent_term:put(?PT_STORE_IDS, StoreIds1), - ok. +-spec create(StoreId, PathPattern, Data) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Creates a specific tree node in the tree structure only if it does not +%% exist. +%% +%% Calling this function is the same as calling `create(StoreId, PathPattern, +%% Data, #{}, #{})'. +%% +%% @see create/5. --spec get_store_ids() -> [store_id()]. +create(StoreId, PathPattern, Data) -> + create(StoreId, PathPattern, Data, #{}, #{}). -get_store_ids() -> - maps:keys(persistent_term:get(?PT_STORE_IDS, #{})). +-spec create(StoreId, PathPattern, Data, Extra | Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Creates a specific tree node in the tree structure only if it does not +%% exist. +%% +%% Calling this function is the same as calling `create(StoreId, PathPattern, +%% Data, Extra, Options)' with an empty `Extra' or `Options'. +%% +%% @see create/5. + +create(StoreId, PathPattern, Data, #{keep_while := _} = Extra) -> + create(StoreId, PathPattern, Data, Extra, #{}); +create(StoreId, PathPattern, Data, Options) -> + create(StoreId, PathPattern, Data, #{}, Options). + +-spec create(StoreId, PathPattern, Data, Extra, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Creates a specific tree node in the tree structure only if it does not +%% exist. +%% +%% Internally, the `PathPattern' is modified to include an +%% `#if_node_exists{exists = false}' condition on its last component. +%% Otherwise, the behavior is that of {@link put/5}. +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the node to create or +%% modify. +%% @param Data the Erlang term or function to store, or a {@link +%% khepri_payload:payload()} structure. +%% @param Extra extra options such as `keep_while' conditions. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous put, an `{ok, Result}' tuple with a +%% map with one entry, or an `{error, Reason}' tuple; in the case of an +%% asynchronous put, always `ok' (the actual return value may be sent by a +%% message if a correlation ID was specified). +%% +%% @see put/5. + +create(StoreId, PathPattern, Data, Extra, Options) -> + PathPattern1 = khepri_path:from_string(PathPattern), + PathPattern2 = khepri_path:combine_with_conditions( + PathPattern1, [#if_node_exists{exists = false}]), + do_put(StoreId, PathPattern2, Data, Extra, Options). + +-spec update(PathPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists. +%% +%% Calling this function is the same as calling `update(StoreId, PathPattern, +%% Data)' with the default store ID. +%% +%% @see update/3. + +update(PathPattern, Data) -> + update(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Data). + +-spec update(StoreId, PathPattern, Data) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists. +%% +%% Calling this function is the same as calling `update(StoreId, PathPattern, +%% Data, #{}, #{})'. +%% +%% @see update/5. + +update(StoreId, PathPattern, Data) -> + update(StoreId, PathPattern, Data, #{}, #{}). + +-spec update(StoreId, PathPattern, Data, Extra | Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists. +%% +%% Calling this function is the same as calling `update(StoreId, PathPattern, +%% Data, Extra, Options)' with an empty `Extra' or `Options'. +%% +%% @see update/5. + +update(StoreId, PathPattern, Data, #{keep_while := _} = Extra) -> + update(StoreId, PathPattern, Data, Extra, #{}); +update(StoreId, PathPattern, Data, Options) -> + update(StoreId, PathPattern, Data, #{}, Options). + +-spec update(StoreId, PathPattern, Data, Extra, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists. +%% +%% Internally, the `PathPattern' is modified to include an +%% `#if_node_exists{exists = true}' condition on its last component. +%% Otherwise, the behavior is that of {@link put/5}. +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the node to create or +%% modify. +%% @param Data the Erlang term or function to store, or a {@link +%% khepri_payload:payload()} structure. +%% @param Extra extra options such as `keep_while' conditions. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous put, an `{ok, Result}' tuple with a +%% map with one entry, or an `{error, Reason}' tuple; in the case of an +%% asynchronous put, always `ok' (the actual return value may be sent by a +%% message if a correlation ID was specified). +%% +%% @see put/5. + +update(StoreId, PathPattern, Data, Extra, Options) -> + PathPattern1 = khepri_path:from_string(PathPattern), + PathPattern2 = khepri_path:combine_with_conditions( + PathPattern1, [#if_node_exists{exists = true}]), + do_put(StoreId, PathPattern2, Data, Extra, Options). + +-spec compare_and_swap(PathPattern, DataPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + DataPattern :: ets:match_pattern(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists and its data matches the given `DataPattern'. +%% +%% Calling this function is the same as calling `compare_and_swap(StoreId, +%% PathPattern, DataPattern, Data)' with the default store ID. +%% +%% @see compare_and_swap/4. + +compare_and_swap(PathPattern, DataPattern, Data) -> + compare_and_swap(?DEFAULT_RA_CLUSTER_NAME, PathPattern, DataPattern, Data). + +-spec compare_and_swap(StoreId, PathPattern, DataPattern, Data) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + DataPattern :: ets:match_pattern(), + Data :: khepri_payload:payload() | data() | fun(), + Result :: result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists and its data matches the given `DataPattern'. +%% +%% Calling this function is the same as calling `compare_and_swap(StoreId, +%% PathPattern, DataPattern, Data, #{}, #{})'. +%% +%% @see compare_and_swap/6. + +compare_and_swap(StoreId, PathPattern, DataPattern, Data) -> + compare_and_swap(StoreId, PathPattern, DataPattern, Data, #{}, #{}). + +-spec compare_and_swap( + StoreId, PathPattern, DataPattern, Data, Extra | Options) -> + Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + DataPattern :: ets:match_pattern(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists and its data matches the given `DataPattern'. +%% +%% Calling this function is the same as calling `compare_and_swap(StoreId, +%% PathPattern, DataPattern, Data, Extra, Options)' with an empty `Extra' or +%% `Options'. +%% +%% @see compare_and_swap/6. --spec forget_store_ids() -> ok. -%% @doc Clears the remembered store IDs. +compare_and_swap( + StoreId, PathPattern, DataPattern, Data, #{keep_while := _} = Extra) -> + compare_and_swap(StoreId, PathPattern, DataPattern, Data, Extra, #{}); +compare_and_swap(StoreId, PathPattern, DataPattern, Data, Options) -> + compare_and_swap(StoreId, PathPattern, DataPattern, Data, #{}, Options). + +-spec compare_and_swap( + StoreId, PathPattern, DataPattern, Data, Extra, Options) -> + Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + DataPattern :: ets:match_pattern(), + Data :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists and its data matches the given `DataPattern'. +%% +%% Internally, the `PathPattern' is modified to include an +%% `#if_data_matches{pattern = DataPattern}' condition on its last component. +%% Otherwise, the behavior is that of {@link put/5}. +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the node to create or +%% modify. +%% @param Data the Erlang term or function to store, or a {@link +%% khepri_payload:payload()} structure. +%% @param Extra extra options such as `keep_while' conditions. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous put, an `{ok, Result}' tuple with a +%% map with one entry, or an `{error, Reason}' tuple; in the case of an +%% asynchronous put, always `ok' (the actual return value may be sent by a +%% message if a correlation ID was specified). +%% +%% @see put/5. + +compare_and_swap(StoreId, PathPattern, DataPattern, Data, Extra, Options) -> + PathPattern1 = khepri_path:from_string(PathPattern), + PathPattern2 = khepri_path:combine_with_conditions( + PathPattern1, [#if_data_matches{pattern = DataPattern}]), + do_put(StoreId, PathPattern2, Data, Extra, Options). + +-spec do_put(StoreId, PathPattern, Payload, Extra, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Payload :: khepri_payload:payload() | data() | fun(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Prepares the payload and calls {@link khepri_machine:put/5}. %% %% @private -forget_store_ids() -> - _ = persistent_term:erase(?PT_STORE_IDS), - ok. +do_put(StoreId, PathPattern, Payload, Extra, Options) -> + Payload1 = khepri_payload:wrap(Payload), + khepri_machine:put(StoreId, PathPattern, Payload1, Extra, Options). -%% ------------------------------------------------------------------- -%% Data manipulation. -%% This is the simple API. The complete/advanced one is exposed by the -%% `khepri_machine' module. -%% ------------------------------------------------------------------- +-spec clear_payload(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: result(). +%% @doc Clears the payload of a specific tree node in the tree structure. +%% +%% Calling this function is the same as calling `clear_payload(StoreId, +%% PathPattern)' with the default store ID. +%% +%% @see clear_payload/2. --spec create(Path, Data) -> ok | error() when - Path :: khepri_path:pattern() | string(), - Data :: khepri_machine:data(). -%% @doc Creates a specific tree node in the tree structure only if it does not -%% exist. +clear_payload(PathPattern) -> + clear_payload(?DEFAULT_RA_CLUSTER_NAME, PathPattern). + +-spec clear_payload(StoreId, PathPattern) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Result :: result(). +%% @doc Clears the payload of a specific tree node in the tree structure. +%% +%% Calling this function is the same as calling `clear_payload(StoreId, +%% PathPattern, #{}, #{})'. +%% +%% @see clear_payload/4. + +clear_payload(StoreId, PathPattern) -> + clear_payload(StoreId, PathPattern, #{}, #{}). + +-spec clear_payload(StoreId, PathPattern, Extra | Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Clears the payload of a specific tree node in the tree structure. +%% +%% Calling this function is the same as calling `clear_payload(StoreId, +%% PathPattern, Extra, Options)' with an empty `Extra' or `Options'. +%% +%% @see clear_payload/4. + +clear_payload(StoreId, PathPattern, #{keep_while := _} = Extra) -> + clear_payload(StoreId, PathPattern, Extra, #{}); +clear_payload(StoreId, PathPattern, Options) -> + clear_payload(StoreId, PathPattern, #{}, Options). + +-spec clear_payload(StoreId, PathPattern, Extra, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Extra :: #{keep_while => keep_while_conds_map()}, + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Clears the payload of a specific tree node in the tree structure. +%% +%% In other words, the payload is set to {@link khepri_payload:no_payload()}. +%% Otherwise, the behavior is that of {@link put/5}. +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the node to create or +%% modify. +%% @param Extra extra options such as `keep_while' conditions. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous put, an `{ok, Result}' tuple with a +%% map with one entry, or an `{error, Reason}' tuple; in the case of an +%% asynchronous put, always `ok' (the actual return value may be sent by a +%% message if a correlation ID was specified). +%% +%% @see put/5. + +clear_payload(StoreId, PathPattern, Extra, Options) -> + khepri_machine:put( + StoreId, PathPattern, khepri_payload:none(), Extra, Options). + +-spec delete(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: result(). +%% @doc Deletes all tree nodes matching the path pattern. +%% +%% Calling this function is the same as calling `delete(StoreId, PathPattern)' +%% with the default store ID. +%% +%% @see delete/2. + +delete(PathPattern) -> + delete(?DEFAULT_RA_CLUSTER_NAME, PathPattern). + +-spec delete +(StoreId, PathPattern) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Result :: result(); +(PathPattern, Options) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Options :: command_options(), + Result :: result(). + +%% @doc Deletes all tree nodes matching the path pattern. +%% +%% This function accepts the following two forms: +%%- `keep_while': `keep_while' conditions to tie the life of the inserted +%% node to conditions on other nodes; see {@link +%% keep_while_conds_map()}.
+%%+%%
+%% +%% @see delete/3. + +delete(StoreId, PathPattern) when is_atom(StoreId) -> + delete(StoreId, PathPattern, #{}); +delete(PathPattern, Options) when is_map(Options) -> + delete(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). + +-spec delete(StoreId, PathPattern, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Options :: command_options(), + Result :: result() | NoRetIfAsync, + NoRetIfAsync :: ok. +%% @doc Deletes all tree nodes matching the path pattern. +%% +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. +%% +%% The returned structure in the "ok" tuple will have a key corresponding to +%% the path for each deleted node. Each key will point to a map containing the +%% properties and payload of that deleted node. +%% +%% Example: +%% ``` +%% %% Delete the node at `/:foo/:bar'. +%% Result = khepri:delete(ra_cluster_name, [foo, bar]), %% -%% Calling this function is the same as calling -%% `create(StoreId, Path, Data)' with the default store ID. +%% %% Here is the content of `Result'. +%% {ok, #{[foo, bar] => #{data => new_value, +%% payload_version => 2, +%% child_list_version => 1, +%% child_list_length => 0}}} = Result. +%% ''' +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to delete. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous delete, an `{ok, Result}' tuple with +%% a map with zero, one or more entries, or an `{error, Reason}' tuple; in the +%% case of an asynchronous put, always `ok' (the actual return value may be +%% sent by a message if a correlation ID was specified). + +delete(StoreId, PathPattern, Options) -> + khepri_machine:delete(StoreId, PathPattern, Options). + +-spec exists(PathPattern) -> Exists when + PathPattern :: khepri_path:pattern() | string(), + Exists :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path exists, +%% otherwise `false'. +%% +%% Calling this function is the same as calling `exists(StoreId, PathPattern)' +%% with the default store ID. +%% +%% @see exists/2. + +exists(PathPattern) -> + exists(?DEFAULT_RA_CLUSTER_NAME, PathPattern). + +-spec exists +(StoreId, PathPattern) -> Exists when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Exists :: boolean(); +(PathPattern, Options) -> Exists when + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Exists :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path exists, +%% otherwise `false'. +%% +%% This function accepts the following two forms: +%%- `delete(StoreId, PathPattern)'. Calling it is the same as calling +%% `delete(StoreId, PathPattern, #{})'.
+%%- `delete(PathPattern, Options)'. Calling it is the same as calling +%% `delete(StoreId, PathPattern, Options)' with the default store ID.
+%%+%%
+%% +%% @see exists/3. + +exists(StoreId, PathPattern) when is_atom(StoreId) -> + exists(StoreId, PathPattern, #{}); +exists(PathPattern, Options) when is_map(Options) -> + exists(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). + +-spec exists(StoreId, PathPattern, Options) -> Exists when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Exists :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path exists, +%% otherwise `false'. +%% +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. +%% +%% The `PathPattern' must point to a specific tree node and can't match +%% multiple nodes. +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to check. +%% @param Options query options such as `favor'. +%% +%% @returns `true' if tree the node exists, `false' if it does not exist or if +%% there was any error. +%% +%% @see get/3. + +exists(StoreId, PathPattern, Options) -> + Options1 = Options#{expect_specific_node => true}, + case get(StoreId, PathPattern, Options1) of + {ok, _} -> true; + _ -> false + end. + +-spec get(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: result(). +%% @doc Returns all tree nodes matching the path pattern. +%% +%% Calling this function is the same as calling `get(StoreId, PathPattern)' +%% with the default store ID. +%% +%% @see get/2. + +get(PathPattern) -> + get(?DEFAULT_RA_CLUSTER_NAME, PathPattern). + +-spec get +(StoreId, PathPattern) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Result :: result(); +(PathPattern, Options) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Result :: result(). +%% @doc Returns all tree nodes matching the path pattern. +%% +%% This function accepts the following two forms: +%%- `exists(StoreId, PathPattern)'. Calling it is the same as calling +%% `exists(StoreId, PathPattern, #{})'.
+%%- `exists(PathPattern, Options)'. Calling it is the same as calling +%% `exists(StoreId, PathPattern, Options)' with the default store ID.
+%%+%%
%% -%% @see create/3. +%% @see get/3. -create(Path, Data) -> - create(?DEFAULT_RA_CLUSTER_NAME, Path, Data). +get(StoreId, PathPattern) when is_atom(StoreId) -> + get(StoreId, PathPattern, #{}); +get(PathPattern, Options) when is_map(Options) -> + get(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). --spec create(StoreId, Path, Data) -> ok | error() when - StoreId :: store_id(), - Path :: khepri_path:pattern() | string(), - Data :: khepri_machine:data(). -%% @doc Creates a specific tree node in the tree structure only if it does not -%% exist. +-spec get(StoreId, PathPattern, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Result :: result(). +%% @doc Returns all tree nodes matching the path pattern. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. %% -%% The `Path' is the modified to include a `#if_node_exists{exists = false}' -%% condition on its last component. +%% The returned structure in the "ok" tuple will have a key corresponding to +%% the path for each node matching the path pattern. Each key will point to a +%% map containing the properties and payload of that matching node. %% -%% Once the path is possibly converted to a list of node names and conditions -%% and updated, it calls {@link khepri_machine:put/3}. +%% Example: +%% ``` +%% %% Query the node at `/:foo/:bar'. +%% Result = khepri:get(ra_cluster_name, [foo, bar]), +%% +%% %% Here is the content of `Result'. +%% {ok, #{[foo, bar] => #{data => new_value, +%% payload_version => 2, +%% child_list_version => 1, +%% child_list_length => 0}}} = Result. +%% ''' %% -%% @returns a single "ok" atom or an "error" tuple, unlike -%% {@link khepri_machine:put/3}. +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to get. +%% @param Options query options such as `favor'. %% -%% @see khepri_machine:put/3. +%% @returns an `{ok, Result}' tuple with a map with zero, one or more entries, +%% or an `{error, Reason}' tuple. -create(StoreId, Path, Data) -> - Path1 = khepri_path:maybe_from_string(Path), - Path2 = khepri_path:combine_with_conditions( - Path1, [#if_node_exists{exists = false}]), - do_put(StoreId, Path2, Data). +get(StoreId, PathPattern, Options) -> + khepri_machine:get(StoreId, PathPattern, Options). --spec insert(Path, Data) -> ok | error() when - Path :: khepri_path:pattern() | string(), - Data :: khepri_machine:data(). -%% @doc Creates or modifies a specific tree node in the tree structure. +-spec get_node_props(PathPattern) -> NodeProps when + PathPattern :: khepri_path:pattern() | string(), + NodeProps :: node_props(). +%% @doc Returns the tree node properties associated with the given node path. %% -%% Calling this function is the same as calling -%% `insert(StoreId, Path, Data)' with the default store ID. +%% Calling this function is the same as calling `get_node_props(StoreId, +%% PathPattern)' with the default store ID. %% -%% @see insert/3. +%% @see get_node_props/2. -insert(Path, Data) -> - insert(?DEFAULT_RA_CLUSTER_NAME, Path, Data). +get_node_props(PathPattern) -> + get_node_props(?DEFAULT_RA_CLUSTER_NAME, PathPattern). --spec insert(StoreId, Path, Data) -> ok | error() when +-spec get_node_props +(StoreId, PathPattern) -> NodeProps when StoreId :: store_id(), - Path :: khepri_path:pattern() | string(), - Data :: khepri_machine:data(). -%% @doc Creates or modifies a specific tree node in the tree structure. -%% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. -%% -%% Once the path is normalized to a list of tree node names and conditions and -%% updated, it calls {@link khepri_machine:put/3}. + PathPattern :: khepri_path:pattern() | string(), + NodeProps :: node_props(); +(PathPattern, Options) -> NodeProps when + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + NodeProps :: node_props(). +%% @doc Returns the tree node properties associated with the given node path. %% -%% @returns a single "ok" atom or an "error" tuple, unlike -%% {@link khepri_machine:put/3}. +%% This function accepts the following two forms: +%%- `get(StoreId, PathPattern)'. Calling it is the same as calling +%% `get(StoreId, PathPattern, #{})'.
+%%- `get(PathPattern, Options)'. Calling it is the same as calling +%% `get(StoreId, PathPattern, Options)' with the default store ID.
+%%+%%
%% -%% @see khepri_machine:put/3. +%% @see get_node_props/3. -insert(StoreId, Path, Data) -> - Path1 = khepri_path:maybe_from_string(Path), - do_put(StoreId, Path1, Data). +get_node_props(StoreId, PathPattern) when is_atom(StoreId) -> + get_node_props(StoreId, PathPattern, #{}); +get_node_props(PathPattern, Options) when is_map(Options) -> + get_node_props(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). --spec update(Path, Data) -> ok | error() when - Path :: khepri_path:pattern() | string(), - Data :: khepri_machine:data(). -%% @doc Updates a specific tree node in the tree structure only if it already -%% exists. +-spec get_node_props(StoreId, PathPattern, Options) -> NodeProps when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + NodeProps :: node_props(). +%% @doc Returns the tree node properties associated with the given node path. %% -%% Calling this function is the same as calling -%% `update(StoreId, Path, Data)' with the default store ID. +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. %% -%% @see update/3. - -update(Path, Data) -> - update(?DEFAULT_RA_CLUSTER_NAME, Path, Data). - --spec update(StoreId, Path, Data) -> ok | error() when - StoreId :: store_id(), - Path :: khepri_path:pattern() | string(), - Data :: khepri_machine:data(). -%% @doc Updates a specific tree node in the tree structure only if it already -%% exists. +%% The `PathPattern' must point to a specific tree node and can't match +%% multiple nodes. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% Unlike {@link get/3}, this function is optimistic and returns the +%% properties directly. If the node does not exist or if there are any errors, +%% an exception is raised. +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to check. +%% @param Options query options such as `favor'. %% -%% The `Path' is the modified to include a `#if_node_exists{exists = true}' -%% condition on its last component. +%% @returns the tree node properties if the node exists, or throws an +%% exception otherwise. %% -%% Once the path is possibly converted to a list of node names and conditions -%% and updated, it calls {@link khepri_machine:put/3}. +%% @see get/3. + +get_node_props(StoreId, PathPattern, Options) -> + Options1 = Options#{expect_specific_node => true}, + case get(StoreId, PathPattern, Options1) of + {ok, Result} -> + [{_Path, NodeProps}] = maps:to_list(Result), + NodeProps; + Error -> + throw(Error) + end. + +-spec has_data(PathPattern) -> HasData when + PathPattern :: khepri_path:pattern() | string(), + HasData :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path has data, +%% otherwise `false'. %% -%% @returns a single "ok" atom or an "error" tuple, unlike -%% {@link khepri_machine:put/3}. +%% Calling this function is the same as calling `has_data(StoreId, +%% PathPattern)' with the default store ID. %% -%% @see khepri_machine:put/3. +%% @see has_data/2. -update(StoreId, Path, Data) -> - Path1 = khepri_path:maybe_from_string(Path), - Path2 = khepri_path:combine_with_conditions( - Path1, [#if_node_exists{exists = true}]), - do_put(StoreId, Path2, Data). +has_data(PathPattern) -> + has_data(?DEFAULT_RA_CLUSTER_NAME, PathPattern). --spec compare_and_swap(Path, DataPattern, Data) -> ok | error() when - Path :: khepri_path:pattern() | string(), - DataPattern :: ets:match_pattern(), - Data :: khepri_machine:data(). -%% @doc Updates a specific tree node in the tree structure only if it already -%% exists and its data matches the given `DataPattern'. +-spec has_data +(StoreId, PathPattern) -> HasData when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + HasData :: boolean(); +(PathPattern, Options) -> HasData when + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + HasData :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path has data, +%% otherwise `false'. %% -%% Calling this function is the same as calling -%% `compare_and_swap(StoreId, Path, DataPattern, Data)' with the default store -%% ID. +%% This function accepts the following two forms: +%%- `get_node_props(StoreId, PathPattern)'. Calling it is the same as +%% calling `get_node_props(StoreId, PathPattern, #{})'.
+%%- `get_node_props(PathPattern, Options)'. Calling it is the same as +%% calling `get_node_props(StoreId, PathPattern, Options)' with the default +%% store ID.
+%%+%%
%% -%% @see create/3. +%% @see has_data/3. -compare_and_swap(Path, DataPattern, Data) -> - compare_and_swap(?DEFAULT_RA_CLUSTER_NAME, Path, DataPattern, Data). +has_data(StoreId, PathPattern) when is_atom(StoreId) -> + has_data(StoreId, PathPattern, #{}); +has_data(PathPattern, Options) when is_map(Options) -> + has_data(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). --spec compare_and_swap(StoreId, Path, DataPattern, Data) -> ok | error() when +-spec has_data(StoreId, PathPattern, Options) -> HasData when StoreId :: store_id(), - Path :: khepri_path:pattern() | string(), - DataPattern :: ets:match_pattern(), - Data :: khepri_machine:data(). -%% @doc Updates a specific tree node in the tree structure only if it already -%% exists and its data matches the given `DataPattern'. + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + HasData :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path has data, +%% otherwise `false'. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. %% -%% The `Path' is the modified to include a -%% `#if_data_matches{pattern = DataPattern}' condition on its last component. +%% The `PathPattern' must point to a specific tree node and can't match +%% multiple nodes. %% -%% Once the path is possibly converted to a list of node names and conditions -%% and updated, it calls {@link khepri_machine:put/3}. +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to check. +%% @param Options query options such as `favor'. %% -%% @returns a single "ok" atom or an "error" tuple, unlike -%% {@link khepri_machine:put/3}. +%% @returns `true' if tree the node holds data, `false' if it does not exist, +%% has no payload, holds a stored procedure or if there was any error. %% -%% @see khepri_machine:put/3. +%% @see get/3. -compare_and_swap(StoreId, Path, DataPattern, Data) -> - Path1 = khepri_path:maybe_from_string(Path), - Path2 = khepri_path:combine_with_conditions( - Path1, [#if_data_matches{pattern = DataPattern}]), - do_put(StoreId, Path2, Data). +has_data(StoreId, PathPattern, Options) -> + try + NodeProps = get_node_props(StoreId, PathPattern, Options), + maps:is_key(data, NodeProps) + catch + throw:{error, _} -> + false + end. --spec do_put( - store_id(), khepri_path:pattern() | string(), khepri_machine:data()) -> - ok | error(). -%% @doc Calls {@link khepri_machine:put/3} and simplifies the return value. -%% -%% The "ok" tuple is converted to a single "ok" atom, getting rid of the map -%% of entries. +-spec get_data(PathPattern) -> Data when + PathPattern :: khepri_path:pattern() | string(), + Data :: data(). +%% @doc Returns the data associated with the given node path. %% -%% The "error" tuple is left unmodified. +%% Calling this function is the same as calling `get_data(StoreId, +%% PathPattern)' with the default store ID. %% -%% @private +%% @see get_data/2. -do_put(StoreId, Path, Fun) when is_function(Fun) -> - case khepri_machine:put(StoreId, Path, #kpayload_sproc{sproc = Fun}) of - {ok, _} -> ok; - Error -> Error - end; -do_put(StoreId, Path, Data) -> - case khepri_machine:put(StoreId, Path, #kpayload_data{data = Data}) of - {ok, _} -> ok; - Error -> Error - end. +get_data(PathPattern) -> + get_data(?DEFAULT_RA_CLUSTER_NAME, PathPattern). --spec clear_payload(Path) -> ok | error() when - Path :: khepri_path:pattern() | string(). -%% @doc Clears the payload of an existing specific tree node in the tree structure. +-spec get_data +(StoreId, PathPattern) -> Data when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Data :: data(); +(PathPattern, Options) -> Data when + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Data :: data(). +%% @doc Returns the data associated with the given node path. %% -%% Calling this function is the same as calling -%% `clear_payload(StoreId, Path)' with the default store ID. +%% This function accepts the following two forms: +%%- `has_data(StoreId, PathPattern)'. Calling it is the same as calling +%% `has_data(StoreId, PathPattern, #{})'.
+%%- `has_data(PathPattern, Options)'. Calling it is the same as calling +%% `has_data(StoreId, PathPattern, Options)' with the default store ID.
+%%+%%
%% -%% @see create/3. +%% @see get_data/3. -clear_payload(Path) -> - clear_payload(?DEFAULT_RA_CLUSTER_NAME, Path). +get_data(StoreId, PathPattern) when is_atom(StoreId) -> + get_data(StoreId, PathPattern, #{}); +get_data(PathPattern, Options) when is_map(Options) -> + get_data(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). --spec clear_payload(StoreId, Path) -> ok | error() when +-spec get_data(StoreId, PathPattern, Options) -> Data when StoreId :: store_id(), - Path :: khepri_path:pattern() | string(). -%% @doc Clears the payload of an existing specific tree node in the tree structure. + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Data :: data(). +%% @doc Returns the data associated with the given node path. +%% +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. %% -%% In other words, the payload is set to `none'. +%% The `PathPattern' must point to a specific tree node and can't match +%% multiple nodes. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% Unlike {@link get/3}, this function is optimistic and returns the data +%% directly. An exception is raised for the following reasons: +%%- `get_data(StoreId, PathPattern)'. Calling it is the same as calling +%% `get_data(StoreId, PathPattern, #{})'.
+%%- `get_data(PathPattern, Options)'. Calling it is the same as calling +%% `get_data(StoreId, PathPattern, Options)' with the default store ID.
+%%+%%
%% -%% Once the path is possibly converted to a list of node names and conditions -%% and updated, it calls {@link khepri_machine:put/3}. +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to check. +%% @param Options query options such as `favor'. %% -%% @returns a single "ok" atom or an "error" tuple, unlike -%% {@link khepri_machine:put/3}. +%% @returns the data if the node has a data payload, or throws an exception if +%% it does not exist, has no payload, holds a stored procedure or if there was +%% any error. %% -%% @see khepri_machine:put/3. +%% @see get/3. -clear_payload(StoreId, Path) -> - Path1 = khepri_path:maybe_from_string(Path), - case khepri_machine:put(StoreId, Path1, none) of - {ok, _} -> ok; - Error -> Error +get_data(StoreId, PathPattern, Options) -> + NodeProps = get_node_props(StoreId, PathPattern, Options), + case NodeProps of + #{data := Data} -> Data; + _ -> throw({error, {no_data, NodeProps}}) end. --spec delete(PathPattern) -> ok | error() when - PathPattern :: khepri_path:pattern() | string(). -%% @doc Deletes all tree nodes matching the path pattern. +-spec has_sproc(PathPattern) -> HasStoredProc when + PathPattern :: khepri_path:pattern() | string(), + HasStoredProc :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path holds a +%% stored procedure, otherwise `false'. %% -%% Calling this function is the same as calling -%% `delete(StoreId, PathPattern)' with -%% the default store ID. +%% Calling this function is the same as calling `has_sproc(StoreId, +%% PathPattern)' with the default store ID. %% -%% @see delete/2. +%% @see has_sproc/2. -delete(Path) -> - delete(?DEFAULT_RA_CLUSTER_NAME, Path). +has_sproc(PathPattern) -> + has_sproc(?DEFAULT_RA_CLUSTER_NAME, PathPattern). --spec delete(StoreId, PathPattern) -> ok | error() when +-spec has_sproc +(StoreId, PathPattern) -> HasStoredProc when StoreId :: store_id(), - PathPattern :: khepri_path:pattern() | string(). -%% @doc Deletes all tree nodes matching the path pattern. + PathPattern :: khepri_path:pattern() | string(), + HasStoredProc :: boolean(); +(PathPattern, Options) -> HasStoredProc when + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + HasStoredProc :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path holds a +%% stored procedure, otherwise `false'. +%% +%% This function accepts the following two forms: +%%- the node does not exist
+%%- the node has no payload
+%%- the node holds a stored procedure
+%%- {@link get/3} returned an error
+%%+%%
+%% +%% @see has_sproc/3. + +has_sproc(StoreId, PathPattern) when is_atom(StoreId) -> + has_sproc(StoreId, PathPattern, #{}); +has_sproc(PathPattern, Options) when is_map(Options) -> + has_sproc(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). + +-spec has_sproc(StoreId, PathPattern, Options) -> HasStoredProc when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + HasStoredProc :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path holds a +%% stored procedure, otherwise `false'. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. %% -%% Once the path is possibly converted to a list of node names and conditions, -%% it calls {@link khepri_machine:delete/2}. +%% The `PathPattern' must point to a specific tree node and can't match +%% multiple nodes. %% -%% @returns a single "ok" atom or an "error" tuple, unlike -%% {@link khepri_machine:delete/2}. +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to check. +%% @param Options query options such as `favor'. %% -%% @see delete/2. +%% @returns `true' if the node holds a stored procedure, `false' if it does +%% not exist, has no payload, holds data or if there was any error. +%% +%% @see get/3. -delete(StoreId, Path) -> - Path1 = khepri_path:maybe_from_string(Path), - case khepri_machine:delete(StoreId, Path1) of - {ok, _} -> ok; - Error -> Error +has_sproc(StoreId, PathPattern, Options) -> + Options1 = Options#{expect_specific_node => true}, + case get(StoreId, PathPattern, Options1) of + {ok, Result} -> + [NodeProps] = maps:values(Result), + maps:is_key(sproc, NodeProps); + _ -> + false end. --spec get(PathPattern) -> Result when +-spec run_sproc(PathPattern, Args) -> Result when PathPattern :: khepri_path:pattern() | string(), - Result :: khepri_machine:result(). -%% @doc Returns all tree nodes matching the path pattern. + Args :: list(), + Result :: any(). +%% @doc Runs the stored procedure pointed to by the given path and returns the +%% result. %% -%% Calling this function is the same as calling -%% `get(StoreId, PathPattern)' with the default store ID. +%% Calling this function is the same as calling `run_sproc(StoreId, +%% PathPattern, Args)' with the default store ID. %% -%% @see get/3. +%% @see run_sproc/3. -get(Path) -> - get(?DEFAULT_RA_CLUSTER_NAME, Path). +run_sproc(PathPattern, Args) -> + run_sproc(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Args). --spec get -(StoreId, PathPattern) -> Result when +-spec run_sproc +(StoreId, PathPattern, Args) -> Result when StoreId :: store_id(), PathPattern :: khepri_path:pattern() | string(), - Result :: khepri_machine:result(); -(PathPattern, Options) -> Result when + Args :: list(), + Result :: any(); +(PathPattern, Args, Options) -> Result when PathPattern :: khepri_path:pattern() | string(), - Options :: khepri_machine:query_options(), - Result :: khepri_machine:result(). -%% @doc Returns all tree nodes matching the path pattern. + Args :: list(), + Options :: query_options(), + Result :: any(). +%% @doc Runs the stored procedure pointed to by the given path and returns the +%% result. %% %% This function accepts the following two forms: %%- `has_sproc(StoreId, PathPattern)'. Calling it is the same as calling +%% `has_sproc(StoreId, PathPattern, #{})'.
+%%- `has_sproc(PathPattern, Options)'. Calling it is the same as calling +%% `has_sproc(StoreId, PathPattern, Options)' with the default store ID.
+%%-%%
%% -%% @see get/3. +%% @see run_sproc/3. -get(StoreId, Path) when is_atom(StoreId) -> - get(StoreId, Path, #{}); -get(Path, Options) when is_map(Options) -> - get(?DEFAULT_RA_CLUSTER_NAME, Path, Options). +run_sproc(StoreId, PathPattern, Args) when is_atom(StoreId) -> + run_sproc(StoreId, PathPattern, Args, #{}); +run_sproc(PathPattern, Args, Options) when is_map(Options) -> + run_sproc(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Args, Options). --spec get(StoreId, PathPattern, Options) -> Result when +-spec run_sproc(StoreId, PathPattern, Args, Options) -> Result when StoreId :: store_id(), PathPattern :: khepri_path:pattern() | string(), - Options :: khepri_machine:query_options(), - Result :: khepri_machine:result(). -%% @doc Returns all tree nodes matching the path pattern. + Args :: list(), + Options :: query_options(), + Result :: any(). +%% @doc Runs the stored procedure pointed to by the given path and returns the +%% result. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. %% -%% Once the path is possibly converted to a list of node names and conditions, -%% it calls {@link khepri_machine:get/3}. +%% The `PathPattern' must point to a specific tree node and can't match +%% multiple nodes. %% -%% @see khepri_machine:get/3. - -get(StoreId, Path, Options) -> - Path1 = khepri_path:maybe_from_string(Path), - khepri_machine:get(StoreId, Path1, Options). - --spec exists(Path) -> Exists when - Path :: khepri_path:pattern() | string(), - Exists :: boolean(). -%% @doc Returns `true' if the tree node pointed to by the given path exists, -%% otherwise `false'. +%% The `Args' list must match the number of arguments expected by the stored +%% procedure. %% -%% Calling this function is the same as calling -%% `exists(StoreId, Path)' with the default store ID. +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to check. +%% @param Args the list of args to pass to the stored procedure; its length +%% must be equal to the stored procedure arity. +%% @param Options query options such as `favor'. +%% +%% @returns the result of the stored procedure execution, or throws an +%% exception if the node does not exist, does not hold a stored procedure or +%% if there was an error. + +run_sproc(StoreId, PathPattern, Args, Options) -> + khepri_machine:run_sproc(StoreId, PathPattern, Args, Options). + +-spec register_trigger(TriggerId, EventFilter, StoredProcPath) -> Ret when + TriggerId :: trigger_id(), + EventFilter :: khepri_evf:event_filter() | + khepri_path:pattern() | string(), + StoredProcPath :: khepri_path:path() | string(), + Ret :: ok | error(). +%% @doc Registers a trigger. +%% +%% Calling this function is the same as calling `register_trigger(StoreId, +%% TriggerId, EventFilter, StoredProcPath)' with the default store ID. +%% +%% @see register_trigger/4. + +register_trigger(TriggerId, EventFilter, StoredProcPath) -> + register_trigger( + ?DEFAULT_RA_CLUSTER_NAME, TriggerId, EventFilter, StoredProcPath). + +-spec register_trigger +(StoreId, TriggerId, EventFilter, StoredProcPath) -> Ret when + StoreId :: khepri:store_id(), + TriggerId :: trigger_id(), + EventFilter :: khepri_evf:event_filter() | + khepri_path:pattern() | string(), + StoredProcPath :: khepri_path:path() | string(), + Ret :: ok | error(); +(TriggerId, EventFilter, StoredProcPath, Options) -> Ret when + TriggerId :: trigger_id(), + EventFilter :: khepri_evf:event_filter() | + khepri_path:pattern() | string(), + StoredProcPath :: khepri_path:path() | string(), + Options :: command_options(), + Ret :: ok | error(). +%% @doc Registers a trigger. %% -%% @see exists/2. - -exists(Path) -> - exists(?DEFAULT_RA_CLUSTER_NAME, Path). - --spec exists(StoreId, Path) -> Exists when - StoreId :: store_id(), - Path :: khepri_path:pattern() | string(), - Exists :: boolean(). -%% @doc Returns `true' if the tree node pointed to by the given path exists, -%% otherwise `false'. +%% This function accepts the following two forms: +%%- `get(StoreId, Path)'. Calling it is the same as calling -%% `get(StoreId, PathPattern, #{})'.
-%%- `get(Path, Options'. Calling it is the same as calling -%% `get(StoreId, PathPattern, #{})' with the default store ID.
+%%- `run_sproc(StoreId, PathPattern, Args)'. Calling it is the same as +%% calling `run_sproc(StoreId, PathPattern, Args, #{})'.
+%%- `run_sproc(PathPattern, Args, Options)'. Calling it is the same as +%% calling `run_sproc(StoreId, PathPattern, Args, Options)' with the default +%% store ID.
%%+%%
%% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% @see register_trigger/5. + +register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath) + when is_atom(StoreId) -> + register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath, #{}); +register_trigger(TriggerId, EventFilter, StoredProcPath, Options) + when is_map(Options) -> + register_trigger( + ?DEFAULT_RA_CLUSTER_NAME, TriggerId, EventFilter, StoredProcPath, + Options). + +-spec register_trigger( + StoreId, TriggerId, EventFilter, StoredProcPath, Options) -> + Ret when + StoreId :: khepri:store_id(), + TriggerId :: trigger_id(), + EventFilter :: khepri_evf:event_filter() | + khepri_path:pattern() | string(), + StoredProcPath :: khepri_path:path() | string(), + Options :: command_options(), + Ret :: ok | error(). +%% @doc Registers a trigger. +%% +%% A trigger is based on an event filter. It associates an event with a stored +%% procedure. When an event matching the event filter is emitted, the stored +%% procedure is executed. +%% +%% The following event filters are documented by {@link +%% khepri_evf:event_filter()}. +%% +%% Here are examples of event filters: %% -%% The `Path' must point to a specific tree node and can't match multiple nodes. +%% ``` +%% %% An event filter can be explicitly created using the `khepri_evf' +%% %% module. This is possible to specify properties at the same time. +%% EventFilter = khepri_evf:tree([stock, wood, <<"oak">>], %% Required +%% #{on_actions => [delete], %% Optional +%% priority => 10}). %% Optional +%% ''' +%% ``` +%% %% For ease of use, some terms can be automatically converted to an event +%% %% filter. In this example, a Unix-like path can be used as a tree event +%% %% filter. +%% EventFilter = "/:stock/:wood/oak". +%% ''' %% -%% This function calls {@link get/3} and interpret its result. +%% The stored procedure is expected to accept a single argument. This argument +%% is a map containing the event properties. Here is an example: %% -%% @see get/3. +%% ``` +%% my_stored_procedure(Props) -> +%% #{path := Path}, +%% on_action => Action} = Props. +%% ''' +%% +%% The stored procedure is executed on the leader's Erlang node. +%% +%% It is guaranteed to run at least once. It could be executed multiple times +%% if the Ra leader changes, therefore the stored procedure must be +%% idempotent. +%% +%% @param StoreId the name of the Ra cluster. +%% @param TriggerId the name of the trigger. +%% @param EventFilter the event filter used to associate an event with a +%% stored procedure. +%% @param StoredProcPath the path to the stored procedure to execute when the +%% corresponding event occurs. +%% +%% @returns `ok' if the trigger was registered, an `{error, Reason}' tuple +%% otherwise. -exists(StoreId, Path) -> - case get(StoreId, Path, #{expect_specific_node => true}) of - {ok, _} -> true; - _ -> false - end. +register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath, Options) -> + khepri_machine:register_trigger( + StoreId, TriggerId, EventFilter, StoredProcPath, Options). --spec has_data(Path) -> HasData when - Path :: khepri_path:pattern() | string(), - HasData :: boolean(). -%% @doc Returns `true' if the tree node pointed to by the given path has a data -%% payload, otherwise `false'. +-spec list(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: result(). +%% @doc Returns all direct child nodes under the given path. %% -%% Calling this function is the same as calling -%% `has_data(StoreId, Path)' with the default store ID. +%% Calling this function is the same as calling `list(StoreId, PathPattern)' +%% with the default store ID. %% -%% @see has_data/2. +%% @see list/2. -has_data(Path) -> - has_data(?DEFAULT_RA_CLUSTER_NAME, Path). +list(PathPattern) -> + list(?DEFAULT_RA_CLUSTER_NAME, PathPattern). --spec has_data(StoreId, Path) -> HasData when +-spec list +(StoreId, PathPattern) -> Result when StoreId :: store_id(), - Path :: khepri_path:pattern() | string(), - HasData :: boolean(). -%% @doc Returns `true' if the tree node pointed to by the given path has a data -%% payload, otherwise `false'. + PathPattern :: khepri_path:pattern() | string(), + Result :: result(); +(PathPattern, Options) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Result :: result(). +%% @doc Returns all direct child nodes under the given path. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% This function accepts the following two forms: +%%- `register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath)'. +%% Calling it is the same as calling `register_trigger(StoreId, TriggerId, +%% EventFilter, StoredProcPath, #{})'.
+%%- `register_trigger(TriggerId, EventFilter, StoredProcPath, Options)'. +%% Calling it is the same as calling `register_trigger(StoreId, TriggerId, +%% EventFilter, StoredProcPath, Options)' with the default store ID.
+%%+%%
+%% +%% @see list/3. + +list(StoreId, PathPattern) when is_atom(StoreId) -> + list(StoreId, PathPattern, #{}); +list(PathPattern, Options) when is_map(Options) -> + list(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Options). + +-spec list(StoreId, PathPattern, Options) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Options :: query_options(), + Result :: result(). +%% @doc Returns all direct child nodes under the given path. %% -%% The `Path' must point to a specific tree node and can't match multiple nodes. +%% The `PathPattern' can be provided as native path (a list of node names and +%% conditions) or as a string. See {@link khepri_path:from_string/1}. %% -%% This function calls {@link get/3} and interpret its result. +%% Internally, an `#if_name_matches{regex = any}' condition is appended to the +%% `PathPattern'. Otherwise, the behavior is that of {@link get/3}. +%% +%% @param StoreId the name of the Ra cluster. +%% @param PathPattern the path (or path pattern) to the nodes to get. +%% @param Options query options such as `favor'. +%% +%% @returns an `{ok, Result}' tuple with a map with zero, one or more entries, +%% or an `{error, Reason}' tuple. %% %% @see get/3. -has_data(StoreId, Path) -> - case get(StoreId, Path, #{expect_specific_node => true}) of - {ok, Result} -> - [NodeProps] = maps:values(Result), - maps:is_key(data, NodeProps); - _ -> - false - end. - --spec list(khepri_path:pattern() | string()) -> - khepri_machine:result(). - -list(Path) -> - list(?DEFAULT_RA_CLUSTER_NAME, Path). +list(StoreId, PathPattern, Options) -> + PathPattern1 = khepri_path:from_string(PathPattern), + PathPattern2 = [?ROOT_NODE | PathPattern1] ++ [?STAR], + get(StoreId, PathPattern2, Options). --spec list(store_id(), khepri_path:pattern() | string()) -> - khepri_machine:result(). +-spec find(PathPattern, Condition) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Condition :: khepri_path:pattern_component(), + Result :: result(). +%% @doc Returns all tree nodes matching the path pattern. +%% +%% Calling this function is the same as calling `find(StoreId, PathPattern)' +%% with the default store ID. +%% +%% @see find/3. -list(StoreId, Path) -> - Path1 = khepri_path:maybe_from_string(Path), - Path2 = [?ROOT_NODE | Path1] ++ [?STAR], - khepri_machine:get(StoreId, Path2). +find(PathPattern, Condition) -> + find(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Condition). --spec find(Path, Condition) -> - Result when - Path :: khepri_path:pattern() | string(), +-spec find +(StoreId, PathPattern, Condition) -> Result when + StoreId :: store_id(), + PathPattern :: khepri_path:pattern() | string(), + Condition :: khepri_path:pattern_component(), + Result :: result(); +(PathPattern, Condition, Options) -> Result when + PathPattern :: khepri_path:pattern() | string(), Condition :: khepri_path:pattern_component(), - Result :: khepri_machine:result(). -%% @doc Finds tree nodes below `Path' which match the given `Condition'. + Options :: query_options(), + Result :: result(). +%% @doc Returns all tree nodes matching the path pattern. %% -%% This function operates on the default store. +%% This function accepts the following two forms: +%%- `list(StoreId, PathPattern)'. Calling it is the same as calling +%% `list(StoreId, PathPattern, #{})'.
+%%- `list(PathPattern, Options)'. Calling it is the same as calling +%% `list(StoreId, PathPattern, Options)' with the default store ID.
+%%+%%
%% -%% @see find/3. +%% @see find/4. -find(Path, Condition) -> - find(?DEFAULT_RA_CLUSTER_NAME, Path, Condition). +find(StoreId, PathPattern, Condition) when is_atom(StoreId) -> + find(StoreId, PathPattern, Condition, #{}); +find(PathPattern, Condition, Options) when is_map(Options) -> + find(?DEFAULT_RA_CLUSTER_NAME, PathPattern, Condition, Options). --spec find(StoreId, Path, Condition) -> - Result when +-spec find(StoreId, PathPattern, Condition, Options) -> Result when StoreId :: store_id(), - Path :: khepri_path:pattern() | string(), + PathPattern :: khepri_path:pattern() | string(), Condition :: khepri_path:pattern_component(), - Result :: khepri_machine:result(). -%% @doc Finds tree nodes under `Path' which match the given `Condition'. + Options :: query_options(), + Result :: result(). +%% @doc Finds tree nodes under `PathPattern' which match the given `Condition'. %% -%% The `Path' can be provided as a list of node names and conditions or as a -%% string. See {@link khepri_path:from_string/1}. +%% The `PathPattern' can be provided as a list of node names and conditions or +%% as a string. See {@link khepri_path:from_string/1}. %% -%% Nodes are searched deeply under the given `Path', not only among direct -%% child nodes. +%% Nodes are searched deeply under the given `PathPattern', not only among +%% direct child nodes. %% %% Example: %% ``` -%% %% Find nodes with data under `/foo/bar'. +%% %% Find nodes with data under `/:foo/:bar'. %% Result = khepri:find( %% ra_cluster_name, %% [foo, bar], @@ -929,23 +1755,29 @@ find(Path, Condition) -> %% ''' %% %% @param StoreId the name of the Ra cluster. -%% @param Path the path indicating where to start the search from. +%% @param PathPattern the path indicating where to start the search from. %% @param Condition the condition nodes must match to be part of the result. %% -%% @returns an "ok" tuple with a map with zero, one or more entries, or an -%% "error" tuple. +%% @returns an `{ok, Result}' tuple with a map with zero, one or more entries, +%% or an `{error, Reason}' tuple. -find(StoreId, Path, Condition) -> +find(StoreId, PathPattern, Condition, Options) -> Condition1 = #if_all{conditions = [?STAR_STAR, Condition]}, - Path1 = khepri_path:maybe_from_string(Path), - Path2 = [?ROOT_NODE | Path1] ++ [Condition1], - khepri_machine:get(StoreId, Path2). + PathPattern1 = khepri_path:from_string(PathPattern), + PathPattern2 = [?ROOT_NODE | PathPattern1] ++ [Condition1], + get(StoreId, PathPattern2, Options). -spec transaction(Fun) -> Ret when Fun :: khepri_tx:tx_fun(), Ret :: Atomic | Aborted, Atomic :: {atomic, khepri_tx:tx_fun_result()}, Aborted :: khepri_tx:tx_abort(). +%% @doc Runs a transaction and returns its result. +%% +%% Calling this function is the same as calling `transaction(StoreId, Fun)' +%% with the default store ID. +%% +%% @see transaction/2. transaction(Fun) -> transaction(?DEFAULT_RA_CLUSTER_NAME, Fun). @@ -957,74 +1789,207 @@ transaction(Fun) -> Ret :: Atomic | Aborted, Atomic :: {atomic, khepri_tx:tx_fun_result()}, Aborted :: khepri_tx:tx_abort(); -(Fun, ReadWrite) -> Ret when +(Fun, ReadWriteOrOptions) -> Ret when Fun :: khepri_tx:tx_fun(), + ReadWriteOrOptions :: ReadWrite | Options, ReadWrite :: ro | rw | auto, - Ret :: Atomic | Aborted, + Options :: command_options() | + query_options(), + Ret :: Atomic | Aborted | NoRetIfAsync, Atomic :: {atomic, khepri_tx:tx_fun_result()}, - Aborted :: khepri_tx:tx_abort(). + Aborted :: khepri_tx:tx_abort(), + NoRetIfAsync :: ok. +%% @doc Runs a transaction and returns its result. +%% +%% This function accepts the following two forms: +%%- `find(StoreId, PathPattern, Condition)'. Calling it is the same as +%% calling `find(StoreId, PathPattern, Condition, #{})'.
+%%- `find(PathPattern, Condition, Options)'. Calling it is the same as +%% calling `find(StoreId, PathPattern, Condition, Options)' with the default +%% store ID.
+%%+%%
+%% +%% @see transaction/3. transaction(StoreId, Fun) when is_function(Fun) -> transaction(StoreId, Fun, auto); -transaction(Fun, ReadWrite) when is_function(Fun) -> - transaction(?DEFAULT_RA_CLUSTER_NAME, Fun, ReadWrite). +transaction(Fun, ReadWriteOrOptions) when is_function(Fun) -> + transaction(?DEFAULT_RA_CLUSTER_NAME, Fun, ReadWriteOrOptions). --spec transaction(StoreId, Fun, ReadWrite) -> Ret when +-spec transaction +(StoreId, Fun, ReadWrite) -> Ret when StoreId :: store_id(), Fun :: khepri_tx:tx_fun(), ReadWrite :: ro | rw | auto, Ret :: Atomic | Aborted, Atomic :: {atomic, khepri_tx:tx_fun_result()}, - Aborted :: khepri_tx:tx_abort(). + Aborted :: khepri_tx:tx_abort(); +(StoreId, Fun, Options) -> Ret when + StoreId :: store_id(), + Fun :: khepri_tx:tx_fun(), + Options :: command_options() | + query_options(), + Ret :: Atomic | Aborted | NoRetIfAsync, + Atomic :: {atomic, khepri_tx:tx_fun_result()}, + Aborted :: khepri_tx:tx_abort(), + NoRetIfAsync :: ok; +(Fun, ReadWrite, Options) -> Ret when + Fun :: khepri_tx:tx_fun(), + ReadWrite :: ro | rw | auto, + Options :: command_options() | + query_options(), + Ret :: Atomic | Aborted | NoRetIfAsync, + Atomic :: {atomic, khepri_tx:tx_fun_result()}, + Aborted :: khepri_tx:tx_abort(), + NoRetIfAsync :: ok. +%% @doc Runs a transaction and returns its result. +%% +%% This function accepts the following three forms: +%%- `transaction(StoreId, Fun)'. Calling it is the same as calling +%% `transaction(StoreId, Fun, #{})'.
+%%- `transaction(Fun, Options)'. Calling it is the same as calling +%% `transaction(StoreId, Fun, Options)' with the default store ID.
+%%+%%
+%% +%% @see transaction/4. -transaction(StoreId, Fun, ReadWrite) -> - khepri_machine:transaction(StoreId, Fun, ReadWrite). +transaction(StoreId, Fun, ReadWrite) + when is_atom(StoreId) andalso is_atom(ReadWrite) -> + transaction(StoreId, Fun, ReadWrite, #{}); +transaction(StoreId, Fun, Options) + when is_atom(StoreId) andalso is_map(Options) -> + transaction(StoreId, Fun, auto, Options); +transaction(Fun, ReadWrite, Options) + when is_atom(ReadWrite) andalso is_map(Options) -> + transaction( + ?DEFAULT_RA_CLUSTER_NAME, Fun, ReadWrite, Options). -run_sproc(Path, Args) -> - run_sproc(?DEFAULT_RA_CLUSTER_NAME, Path, Args). +-spec transaction(StoreId, Fun, ReadWrite, Options) -> Ret when + StoreId :: store_id(), + Fun :: khepri_tx:tx_fun(), + ReadWrite :: ro | rw | auto, + Options :: command_options() | + query_options(), + Ret :: Atomic | Aborted | NoRetIfAsync, + Atomic :: {atomic, khepri_tx:tx_fun_result()}, + Aborted :: khepri_tx:tx_abort(), + NoRetIfAsync :: ok. +%% @doc Runs a transaction and returns its result. +%% +%% `Fun' is an arbitrary anonymous function which takes no arguments. +%% +%% The `ReadWrite' flag determines what the anonymous function is allowed to +%% do and in which context it runs: +%% +%%- `transaction(StoreId, PathPattern, ReadWrite)'. Calling it is the same +%% as calling `transaction(StoreId, PathPattern, ReadWrite, #{})'.
+%%- `transaction(StoreId, PathPattern, Options)'. Calling it is the same +%% as calling `transaction(StoreId, PathPattern, auto, Options)'.
+%%- `transaction(PathPattern, ReadWrite, Options)'. Calling it is the same +%% as calling `transaction(StoreId, PathPattern, ReadWrite, Options)' with the +%% default store ID.
+%%+%%
+%% +%% `Options' is relevant for both read-only and read-write transactions +%% (including audetected ones). However note that both types expect different +%% options. +%% +%% The result of `Fun' can be any term. That result is returned in an +%% `{atomic, Result}' tuple if the transaction is synchronous. The result is +%% sent by message if the transaction is asynchronous and a correlation ID was +%% specified. +%% +%% @param StoreId the name of the Ra cluster. +%% @param Fun an arbitrary anonymous function. +%% @param ReadWrite the read/write or read-only nature of the transaction. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous transaction, `{atomic, Result}' where +%% `Result' is the return value of `Fun', or `{aborted, Reason}' if the +%% anonymous function was aborted; in the case of an asynchronous transaction, +%% always `ok' (the actual return value may be sent by a message if a +%% correlation ID was specified). -run_sproc(StoreId, Path, Args) -> - khepri_machine:run_sproc(StoreId, Path, Args). +transaction(StoreId, Fun, ReadWrite, Options) -> + khepri_machine:transaction(StoreId, Fun, ReadWrite, Options). --spec clear_store() -> ok | error(). +-spec clear_store() -> Result when + Result :: result(). +%% @doc Wipes out the entire tree. +%% +%% Calling this function is the same as calling `clear_store(StoreId)' with +%% the default store ID. +%% +%% @see clear_store/1. clear_store() -> clear_store(?DEFAULT_RA_CLUSTER_NAME). --spec clear_store(store_id()) -> ok | error(). - -clear_store(StoreId) -> - delete(StoreId, [?STAR]). - --spec no_payload() -> none. -%% @doc Returns `none'. +-spec clear_store +(StoreId) -> Result when + StoreId :: store_id(), + Result :: result(); +(Options) -> Result when + Options :: command_options(), + Result :: result(). +%% @doc Wipes out the entire tree. +%% +%% This function accepts the following two forms: +%%- If `ReadWrite' is `ro', `Fun' can do whatever it wants, except modify +%% the content of the store. In other words, uses of {@link khepri_tx:put/2} +%% or {@link khepri_tx:delete/1} are forbidden and will abort the function. +%% `Fun' is executed from a process on the leader Ra member.
+%%- If `ReadWrite' is `rw', `Fun' can use the {@link khepri_tx} transaction +%% API as well as any calls to other modules as long as those functions or what +%% they do is permitted. See {@link khepri_tx} for more details. If `Fun' does +%% or calls something forbidden, the transaction will be aborted. `Fun' is +%% executed in the context of the state machine process on each Ra +%% members.
+%%- If `ReadWrite' is `auto', `Fun' is analyzed to determine if it calls +%% {@link khepri_tx:put/2} or {@link khepri_tx:delete/1}, or uses any denied +%% operations for a read/write transaction. If it does, this is the same as +%% setting `ReadWrite' to true. Otherwise, this is the equivalent of setting +%% `ReadWrite' to false.
+%%+%%
%% -%% This is a helper for cases where using records is inconvenient, like in an -%% Erlang shell. +%% @see clear_store/2. -no_payload() -> - none. +clear_store(StoreId) when is_atom(StoreId) -> + clear_store(StoreId, #{}); +clear_store(Options) when is_map(Options) -> + clear_store(?DEFAULT_RA_CLUSTER_NAME, Options). --spec data_payload(Term) -> Payload when - Term :: khepri_machine:data(), - Payload :: #kpayload_data{}. -%% @doc Returns `#kpayload_data{data = Term}'. +-spec clear_store(StoreId, Options) -> Result when + StoreId :: store_id(), + Options :: command_options(), + Result :: result(). +%% @doc Wipes out the entire tree. %% -%% This is a helper for cases where using macros is inconvenient, like in an -%% Erlang shell. - -data_payload(Term) -> - #kpayload_data{data = Term}. +%% Note that the root node will remain unmodified however. +%% +%% @param StoreId the name of the Ra cluster. +%% @param Options command options such as the command type. +%% +%% @returns in the case of a synchronous delete, an `{ok, Result}' tuple with +%% a map with zero, one or more entries, or an `{error, Reason}' tuple; in the +%% case of an asynchronous put, always `ok' (the actual return value may be +%% sent by a message if a correlation ID was specified). +%% +%% @see delete/3. -sproc_payload(Fun) when is_function(Fun) -> - #kpayload_sproc{sproc = Fun}; -sproc_payload(#standalone_fun{} = Fun) -> - #kpayload_sproc{sproc = Fun}. +clear_store(StoreId, Options) -> + delete(StoreId, [?STAR], Options). %% ------------------------------------------------------------------- %% Public helpers. %% ------------------------------------------------------------------- +-spec info() -> ok. +%% @doc Lists the running stores on stdout. + info() -> StoreIds = get_store_ids(), case StoreIds of @@ -1036,13 +2001,19 @@ info() -> fun(StoreId) -> io:format(" ~ts~n", [StoreId]) end, StoreIds) - end. + end, + ok. --spec info(store_id()) -> ok. +-spec info(StoreId) -> ok when + StoreId :: store_id(). +%% @doc Lists the content of specified store on stdout. +%% +%% @param StoreID the name of the Ra cluster. 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_cluster:members(StoreId)]), lists:foreach(fun(Node) -> io:format("~ts~n", [Node]) end, Nodes), case khepri_machine:get_keep_while_conds_state(StoreId) of @@ -1067,11 +2038,12 @@ info(StoreId) -> ok end, - case khepri_machine:get(StoreId, [#if_path_matches{regex = any}]) of + case get(StoreId, [?STAR_STAR]) of {ok, Result} -> io:format("~n\033[1;32m== TREE ==\033[0m~n~n●~n", []), Tree = khepri_utils:flat_struct_to_tree(Result), khepri_utils:display_tree(Tree); _ -> ok - end. + end, + ok. diff --git a/src/khepri_app.erl b/src/khepri_app.erl index 9221ba46..2978afd1 100644 --- a/src/khepri_app.erl +++ b/src/khepri_app.erl @@ -22,7 +22,7 @@ stop(_) -> lists:foreach( fun(StoreId) -> khepri_machine:clear_cache(StoreId) end, StoreIds), - khepri:forget_store_ids(), + khepri_cluster:forget_store_ids(), ok. config_change(_Changed, _New, _Removed) -> diff --git a/src/khepri_cluster.erl b/src/khepri_cluster.erl new file mode 100644 index 00000000..1991237a --- /dev/null +++ b/src/khepri_cluster.erl @@ -0,0 +1,461 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +%% @doc Khepri service and cluster management API. +%% +%% This module provides the public API for the service and cluster management. +%% For convenience, some functions of this API are repeated in the {@link +%% khepri} module. +%% +%% == 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 Ra +%% documentation to learn more about Ra systems and Ra clusters. +%% +%% == Managing Ra cluster members == +%% +%% To add a member to your Ra cluster: +%% +%% ``` +%% khepri_cluster:add_member( +%% RaSystem, +%% RaClusterName, +%% RaClusterFriendlyName, +%% NewMemberErlangNodename). +%% ''' +%% +%% To remove a member from your Ra cluster: +%% +%% ``` +%% khepri_cluster:remove_member( +%% RaClusterName, +%% MemberErlangNodenameToRemove). +%% ''' + +-module(khepri_cluster). + +-include_lib("kernel/include/logger.hrl"). + +-include("include/khepri.hrl"). +-include("src/internal.hrl"). + +-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, + forget_store_ids/0]). + +-if(?OTP_RELEASE >= 24). +-dialyzer({no_underspecs, [start/1]}). +-endif. + +%% ------------------------------------------------------------------- +%% Database management. +%% ------------------------------------------------------------------- + +-spec start() -> Ret when + Ret :: khepri:ok(StoreId) | khepri:error(), + StoreId :: khepri:store_id(). +%% @doc Starts a store on the default Ra system. +%% +%% The store uses the default Ra cluster name and cluster friendly name. + +start() -> + case application:ensure_all_started(ra) of + {ok, _} -> + RaSystem = default, + case ra_system:start_default() of + {ok, _} -> start(RaSystem); + {error, {already_started, _}} -> start(RaSystem); + {error, _} = Error -> Error + end; + {error, _} = Error -> + Error + end. + +-spec start(RaSystem) -> Ret when + RaSystem :: atom(), + Ret :: khepri:ok(StoreId) | khepri:error(), + StoreId :: khepri: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. + +start(RaSystem) -> + start(RaSystem, ?DEFAULT_RA_CLUSTER_NAME, ?DEFAULT_RA_FRIENDLY_NAME). + +-spec start(RaSystem, ClusterName, FriendlyName) -> Ret when + RaSystem :: atom(), + ClusterName :: ra:cluster_name(), + FriendlyName :: string(), + Ret :: khepri:ok(StoreId) | khepri:error(), + StoreId :: khepri: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. + +start(RaSystem, ClusterName, FriendlyName) -> + case application:ensure_all_started(khepri) of + {ok, _} -> + case ensure_started(RaSystem, ClusterName, FriendlyName) of + ok -> + ok = remember_store_id(ClusterName), + {ok, ClusterName}; + Error -> + Error + end; + Error -> + Error + end. + +ensure_started(RaSystem, ClusterName, FriendlyName) -> + ThisNode = node(), + ThisMember = node_to_member(ClusterName, ThisNode), + ?LOG_DEBUG( + "Check if a local Ra server is running for cluster \"~s\"", + [ClusterName], + #{domain => [khepri, clustering]}), + case whereis(ClusterName) of + undefined -> + ?LOG_DEBUG( + "No local Ra server running for cluster \"~s\", " + "try to restart it", + [ClusterName], + #{domain => [khepri, clustering]}), + Lock = {ClusterName, self()}, + global:set_lock(Lock), + Ret = case ra:restart_server(RaSystem, ThisMember) of + {error, Reason} + when Reason == not_started orelse + Reason == name_not_registered -> + ?LOG_DEBUG( + "Ra cluster not running, try to start it", + [], + #{domain => [khepri, clustering]}), + do_start( + RaSystem, ClusterName, FriendlyName, + [ThisMember]); + ok -> + ok; + {error, {already_started, _}} -> + ok; + _ -> + ok + end, + global:del_lock(Lock), + Ret; + _ -> + ?LOG_DEBUG( + "Local Ra server running, part of cluster \"~s\"", + [ClusterName], + #{domain => [khepri, clustering]}), + ok + end. + +do_start(RaSystem, ClusterName, FriendlyName, Members) -> + RaServerConfigs = [make_ra_server_config( + ClusterName, FriendlyName, Member, Members) + || Member <- Members], + ?LOG_DEBUG( + "Starting a cluster, named \"~s\", with the following Ra server " + "configuration:~n~p", + [ClusterName, hd(RaServerConfigs)], + #{domain => [khepri, clustering]}), + case ra:start_cluster(RaSystem, RaServerConfigs) of + {ok, Started, _} -> + ?LOG_DEBUG( + "Started Ra server for cluster \"~s\" on ~p", + [ClusterName, Started], + #{domain => [khepri, clustering]}), + ok; + {error, cluster_not_formed} = Error -> + ?LOG_ERROR( + "Failed to start Ra server for cluster \"~s\" using the " + "following Ra server configuration:~n~p", + [ClusterName, hd(RaServerConfigs)], + #{domain => [khepri, clustering]}), + Error + end. + +add_member(RaSystem, NewNode) -> + add_member( + RaSystem, ?DEFAULT_RA_CLUSTER_NAME, ?DEFAULT_RA_FRIENDLY_NAME, + NewNode). + +add_member(RaSystem, ClusterName, FriendlyName, NewNode) -> + ?LOG_DEBUG( + "Querying members of cluster \"~s\"", + [ClusterName], + #{domain => [khepri, clustering]}), + case members(ClusterName) of + ExistingMembers when ExistingMembers =/= [] -> + NewMember = node_to_member(ClusterName, NewNode), + case lists:member(NewMember, ExistingMembers) of + false -> + start_ra_server_and_add_member( + RaSystem, ClusterName, FriendlyName, ExistingMembers, + NewMember); + true -> + ?LOG_DEBUG( + "Member ~p is already part of cluster \"~s\"", + [NewMember, ClusterName], + #{domain => [khepri, clustering]}), + ok + end; + [] -> + ?LOG_ERROR( + "Failed to query members of cluster \"~s\"", + [ClusterName], + #{domain => [khepri, clustering]}), + {error, failed_to_query_cluster_members} + end. + +start_ra_server_and_add_member( + RaSystem, ClusterName, FriendlyName, ExistingMembers, NewMember) -> + Lock = {ClusterName, self()}, + global:set_lock(Lock), + RaServerConfig = make_ra_server_config( + ClusterName, FriendlyName, NewMember, ExistingMembers), + ?LOG_DEBUG( + "Adding member ~p to cluster \"~s\" with the following " + "configuration:~n~p", + [NewMember, ClusterName, RaServerConfig], + #{domain => [khepri, clustering]}), + case ra:start_server(RaSystem, RaServerConfig) of + ok -> + %% TODO: Take the timeout as an argument (+ have a default). + Timeout = 30000, + Ret = do_add_member( + ClusterName, ExistingMembers, NewMember, Timeout), + global:del_lock(Lock), + Ret; + Error -> + global:del_lock(Lock), + ?LOG_ERROR( + "Failed to start member ~p, required to add it to " + "cluster \"~s\": ~p", + [NewMember, ClusterName, Error], + #{domain => [khepri, clustering]}), + Error + end. + +do_add_member(ClusterName, ExistingMembers, NewMember, Timeout) -> + T0 = erlang:monotonic_time(), + Ret = ra:add_member(ExistingMembers, NewMember), + case Ret of + {ok, _, _} -> + ok; + Error when Timeout >= 0 -> + ?LOG_NOTICE( + "Failed to add member ~p to cluster \"~s\": ~p; " + "will retry for ~b milliseconds", + [NewMember, ClusterName, Error, Timeout], + #{domain => [khepri, clustering]}), + timer:sleep(500), + T1 = erlang:monotonic_time(), + TDiff = erlang:convert_time_unit(T1 - T0, native, millisecond), + TimeLeft = Timeout - TDiff, + do_add_member( + ClusterName, ExistingMembers, NewMember, TimeLeft); + Error -> + ?LOG_ERROR( + "Failed to add member ~p to cluster \"~s\": ~p; " + "aborting", + [NewMember, ClusterName, Error], + #{domain => [khepri, clustering]}), + Error + end. + +remove_member(NodeToRemove) -> + remove_member(?DEFAULT_RA_CLUSTER_NAME, NodeToRemove). + +remove_member(ClusterName, NodeToRemove) -> + ?LOG_DEBUG( + "Querying members of cluster \"~s\"", + [ClusterName], + #{domain => [khepri, clustering]}), + case members(ClusterName) of + ExistingMembers when ExistingMembers =/= [] -> + MemberToRemove = node_to_member(ClusterName, NodeToRemove), + case lists:member(MemberToRemove, ExistingMembers) of + true -> + do_remove_member( + ClusterName, ExistingMembers, MemberToRemove); + false -> + ?LOG_DEBUG( + "Member ~p is not part of cluster \"~s\"", + [MemberToRemove, ClusterName], + #{domain => [khepri, clustering]}), + ok + end; + [] -> + ?LOG_ERROR( + "Failed to query members of cluster \"~s\"", + [ClusterName], + #{domain => [khepri, clustering]}), + {error, failed_to_query_cluster_members} + end. + +do_remove_member(ClusterName, ExistingMembers, MemberToRemove) -> + case ra:remove_member(ExistingMembers, MemberToRemove) of + {ok, _, _} -> + ok; + Error -> + ?LOG_ERROR( + "Failed to remove member ~p from cluster \"~s\": ~p; " + "aborting", + [MemberToRemove, ClusterName, Error], + #{domain => [khepri, clustering]}), + Error + end. + +-spec reset(RaSystem, ClusterName) -> Ret when + RaSystem :: atom(), + ClusterName :: ra:cluster_name(), + Ret :: ok | khepri: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. + +reset(RaSystem, ClusterName) -> + ThisNode = node(), + ThisMember = node_to_member(ClusterName, ThisNode), + ?LOG_DEBUG( + "Resetting member ~p in cluster \"~s\"", + [ThisMember, ClusterName], + #{domain => [khepri, clustering]}), + ra:force_delete_server(RaSystem, ThisMember). + +members(ClusterName) -> + Fun = fun ra:members/1, + do_query_members(ClusterName, Fun). + +locally_known_members(ClusterName) -> + Fun = fun(CN) -> ra:members({local, CN}) end, + do_query_members(ClusterName, Fun). + +do_query_members(ClusterName, Fun) -> + ThisNode = node(), + ThisMember = node_to_member(ClusterName, ThisNode), + ?LOG_DEBUG( + "Query members in cluster \"~s\"", + [ClusterName], + #{domain => [khepri, clustering]}), + case Fun(ThisMember) of + {ok, Members, _} -> + ?LOG_DEBUG( + "Found the following members in cluster \"~s\": ~p", + [ClusterName, Members], + #{domain => [khepri, clustering]}), + Members; + Error -> + ?LOG_WARNING( + "Failed to query members in cluster \"~s\": ~p", + [ClusterName, Error], + #{domain => [khepri, clustering]}), + [] + end. + +nodes(ClusterName) -> + [Node || {_, Node} <- members(ClusterName)]. + +locally_known_nodes(ClusterName) -> + [Node || {_, Node} <- locally_known_members(ClusterName)]. + +node_to_member(ClusterName, Node) -> + {ClusterName, Node}. + +make_ra_server_config(ClusterName, FriendlyName, Member, Members) -> + UId = ra:new_uid(ra_lib:to_binary(ClusterName)), + #{cluster_name => ClusterName, + id => Member, + uid => UId, + friendly_name => FriendlyName, + initial_members => Members, + log_init_args => #{uid => UId}, + machine => {module, khepri_machine, #{store_id => ClusterName}}}. + +-define(PT_STORE_IDS, {khepri, store_ids}). + +remember_store_id(ClusterName) -> + StoreIds = persistent_term:get(?PT_STORE_IDS, #{}), + StoreIds1 = StoreIds#{ClusterName => true}, + persistent_term:put(?PT_STORE_IDS, StoreIds1), + ok. + +-spec get_store_ids() -> [StoreId] when + StoreId :: khepri:store_id(). +%% @doc Returns the list of running stores. + +get_store_ids() -> + maps:keys(persistent_term:get(?PT_STORE_IDS, #{})). + +-spec forget_store_ids() -> ok. +%% @doc Clears the remembered store IDs. +%% +%% @private + +forget_store_ids() -> + _ = persistent_term:erase(?PT_STORE_IDS), + ok. diff --git a/src/khepri_condition.erl b/src/khepri_condition.erl index 01d078ad..b1dd4df6 100644 --- a/src/khepri_condition.erl +++ b/src/khepri_condition.erl @@ -340,7 +340,7 @@ applies_to_grandchildren(_) -> -spec is_met(Condition, PathOrChildName, Child) -> IsMet when Condition :: khepri_path:pattern_component(), PathOrChildName :: khepri_path:path() | khepri_path:component(), - Child :: khepri_machine:tree_node() | khepri_machine:node_props(), + Child :: khepri_machine:tree_node() | khepri:node_props(), IsMet :: true | IsNotMet1 | IsNotMet2, IsNotMet1 :: {false, khepri_path:pattern_component()}, IsNotMet2 :: {false, {condition(), any()}}. @@ -385,17 +385,17 @@ is_met( _Child) -> eval_regex(Cond, SourceRegex, CompiledRegex, ChildName); is_met(#if_has_data{has_data = true}, - _ChildName, #node{payload = #kpayload_data{data = _}}) -> + _ChildName, #node{payload = #p_data{data = _}}) -> true; is_met(#if_has_data{has_data = false} = Cond, - _ChildName, #node{payload = #kpayload_data{data = _}}) -> + _ChildName, #node{payload = #p_data{data = _}}) -> {false, Cond}; is_met(#if_has_data{has_data = true} = Cond, _ChildName, _Child) -> {false, Cond}; is_met(#if_has_data{has_data = false}, _ChildName, _Child) -> true; is_met(#if_data_matches{compiled = CompMatchSpec} = Cond, - _ChildName, #node{payload = #kpayload_data{data = Data}}) -> + _ChildName, #node{payload = #p_data{data = Data}}) -> case term_matches(Data, CompMatchSpec) of true -> true; false -> {false, Cond} @@ -454,7 +454,7 @@ is_met(Cond, _, _) -> {false, Cond}. -spec term_matches(Term, MatchSpec) -> Matches when - Term :: khepri_machine:data(), + Term :: khepri:data(), MatchSpec :: ets:comp_match_spec(), Matches :: boolean(). %% @doc Returns true if the given match spec matches the given match term. diff --git a/src/khepri_evf.erl b/src/khepri_evf.erl new file mode 100644 index 00000000..3e637929 --- /dev/null +++ b/src/khepri_evf.erl @@ -0,0 +1,134 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +%% @doc Khepri event filters. + +-module(khepri_evf). + +-include("src/internal.hrl"). + +-export([tree/1, tree/2, + wrap/1, + get_priority/1, + set_priority/2]). + +-type tree_event_filter() :: #evf_tree{}. +%% A tree event filter. +%% +%% It takes a path pattern to monitor and optionally properties. + +-type tree_event_filter_props() :: #{on_actions => [create | update | delete], + priority => khepri_evf:priority()}. +%% Tree event filter properties. +%% +%% The properties are: +%%- `clear_store(StoreId)'. Calling it is the same as calling +%% `clear_store(StoreId, #{})'.
+%%- `clear_store(Options)'. Calling it is the same as calling +%% `clear_store(StoreId, Options)' with the default store ID.
+%%+%%
+%% +%% A Khepri path, wether it is a native path or a Unix-like path, can be used +%% as a tree event filter. It will be automatically converted to a tree event +%% filter with default properties. + +-type event_filter() :: tree_event_filter(). +%% An event filter. +%% +%% The following event filters are supported: +%%- `on_actions': a list of actions to filter among `create', `update' and +%% `delete'; the default is to react to all of them.
+%%- `priority': a {@link priority()}
+%%+%%
+%% +%% An event filter can be explicitly constructed using the functions provided +%% in this module. However, some common types will be automatically detected +%% and converted to an event filter with default properties. See each event +%% filter type for more details. + +-type priority() :: integer(). +%% An event filter priority. +%% +%% This is an integer to prioritize event filters: the greater the priority, +%% the more it is prioritized. Negative integers are allowed. +%% +%% The default priority is 0. + +-export_type([event_filter/0, + tree_event_filter/0, + tree_event_filter_props/0, + priority/0]). + +-spec tree(PathPattern) -> EventFilter when + PathPattern :: khepri_path:pattern() | string(), + EventFilter :: tree_event_filter(). +%% @doc Constructs a tree event filter. +%% +%% @see tree/2. + +tree(PathPattern) -> + tree(PathPattern, #{}). + +-spec tree(PathPattern, Props) -> EventFilter when + PathPattern :: khepri_path:pattern() | string(), + Props :: tree_event_filter_props(), + EventFilter :: tree_event_filter(). +%% @doc Constructs a tree event filter. +%% +%% @see tree_event_filter(). + +tree(PathPattern, Props) -> + PathPattern1 = khepri_path:from_string(PathPattern), + #evf_tree{path = PathPattern1, + props = Props}. + +-spec wrap(Input) -> EventFilter when + Input :: event_filter() | khepri_path:pattern() | string(), + EventFilter :: event_filter(). +%% @doc Automatically detects the event filter type and ensures it is wrapped +%% in one of the internal types. +%% +%% @param Input an already created event filter, or any term which can be +%% automatically converted to an event filter. +%% +%% @returns the created event filter. + +wrap(EventFilter) when ?IS_KHEPRI_EVENT_FILTER(EventFilter) -> + EventFilter; +wrap(PathPattern) when is_list(PathPattern) -> + tree(PathPattern). + +-spec get_priority(EventFilter) -> Priority when + EventFilter :: event_filter(), + Priority :: priority(). +%% @doc Returns the priority of the event filter. +%% +%% @param EventFilter the event filter to update. +%% +%% @returns the priority. + +get_priority(#evf_tree{props = Props}) -> + get_priority1(Props). + +get_priority1(#{priority := Priority}) -> Priority; +get_priority1(_) -> 0. + +-spec set_priority(EventFilter, Priority) -> EventFilter when + EventFilter :: event_filter(), + Priority :: priority(). +%% @doc Sets the priority of the event filter. +%% +%% @param EventFilter the event filter to update. +%% @param Priority the new priority. +%% +%% @returns the updated event filter. + +set_priority(#evf_tree{props = Props} = EventFilter, Priority) -> + Props1 = set_priority1(Props, Priority), + EventFilter#evf_tree{props = Props1}. + +set_priority1(Props, Priority) when is_integer(Priority) -> + Props#{priority => Priority}. diff --git a/src/khepri_fun.erl b/src/khepri_fun.erl index d8784dd3..beb98608 100644 --- a/src/khepri_fun.erl +++ b/src/khepri_fun.erl @@ -5,7 +5,7 @@ %% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved. %% -%% @doc Anonymous function extraction API. +%% @doc Anonymous function extraction private API. %% %% This module is responsible for extracting the code of an anonymous function. %% The goal is to be able to store the extracted function and execute it later, @@ -50,13 +50,17 @@ %% abstract code (i.e. after parsing but before compilation) is available in %% the `env'. We compile that abstract code and extract the assembly from that %% compiled beam. +%% +%% This module is private. The documentation is still visible because it may +%% help understand some implementation details. However, this module should +%% never be called directly outside of Khepri. -module(khepri_fun). -include_lib("kernel/include/logger.hrl"). -include_lib("stdlib/include/assert.hrl"). --include("src/internal.hrl"). +-include("src/khepri_fun.hrl"). -export([to_standalone_fun/1, to_standalone_fun/2, diff --git a/src/khepri_fun.hrl b/src/khepri_fun.hrl new file mode 100644 index 00000000..11ff2de9 --- /dev/null +++ b/src/khepri_fun.hrl @@ -0,0 +1,13 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +%% Structure representing an anonymous function "extracted" as a compiled +%% module for storage. +-record(standalone_fun, {module :: module(), + beam :: binary(), + arity :: arity(), + env :: list()}). diff --git a/src/khepri_machine.erl b/src/khepri_machine.erl index e926826b..e30bd058 100644 --- a/src/khepri_machine.erl +++ b/src/khepri_machine.erl @@ -6,66 +6,16 @@ %% %% @doc -%% Khepri low-level API. +%% Khepri private low-level API. %% -%% This module exposes the "low-level" API to the Khepri database and state -%% machine. All functions in {@link khepri} are built on top of this module. +%% This module exposes the private "low-level" API to the Khepri database and +%% state machine. Main functions correspond to Ra commands implemented by the +%% state machine. All functions in {@link khepri} are built on top of this +%% module. %% -%% The API is divided into two parts: -%%- Tree event filter ({@link tree_event_filter()}
+%%-%%
-%% -%% == The store ID == -%% -%% All functions require a store ID ({@link store_id/0}). The store ID -%% corresponds to the name of the Ra cluster Khepri was started with. -%% -%% See {@link khepri} for more details about Ra systems and clusters. -%% -%% == Direct manipulation on tree nodes == -%% -%% The API provides the following three functions: -%%- Functions to manipulate a simple set of tree nodes directly.
-%%- Functions to perform transactional queries and updates.
-%%-%%
-%% -%% All functions take a native path pattern. They do not accept Unix-like -%% paths. -%% -%% All functions, except asynchronous puts, deletes and R/W transactions, -%% return one of these tuples: -%%- {@link get/2} and {@link get/3}: returns all tree node matching the -%% given path pattern.
-%%- {@link put/3}, {@link put/4} and {@link put/5}: updates a single -%% specific tree node.
-%%- {@link delete/2}, {@link delete/3}: removes all tree node matching the -%% given path pattern.
-%%-%%
-%% -%% == Transactional queries and updates == -%% -%% Transactions are handled by {@link transaction/2}, {@link transaction/3} -%% and {@link transaction/4}. -%% -%% Both functions take an anonymous function. See {@link khepri_tx} for more -%% details about those functions and in particular their restrictions. -%% -%% The return value is whatever the anonymous function returns if it succeeded -%% or the reason why it aborted, similar to what {@link mnesia:transaction/1} -%% returns. +%% This module is private. The documentation is still visible because it may +%% help understand some implementation details. However, this module should +%% never be called directly outside of Khepri. -module(khepri_machine). -behaviour(ra_machine). @@ -74,15 +24,16 @@ -include_lib("stdlib/include/assert.hrl"). -include("include/khepri.hrl"). +-include("src/khepri_fun.hrl"). -include("src/internal.hrl"). -include("src/khepri_machine.hrl"). --export([put/3, put/4, put/5, - get/2, get/3, - delete/2, delete/3, - transaction/2, transaction/3, transaction/4, - run_sproc/3, - register_trigger/4]). +-export([put/5, + get/3, + delete/3, + transaction/4, + run_sproc/4, + register_trigger/5]). -export([get_keep_while_conds_state/1]). -export([init/1, apply/3, @@ -105,78 +56,12 @@ -compile({no_auto_import, [apply/3]}). --type data() :: any(). -%% Data stored in a node's payload. - --type payload_version() :: pos_integer(). -%% Number of changes made to the payload of a node. -%% -%% The payload version starts at 1 when a node is created. It is increased by 1 -%% each time the payload is added, modified or removed. - --type child_list_version() :: pos_integer(). -%% Number of changes made to the list of child nodes of a node (child nodes -%% added or removed). -%% -%% The child list version starts at 1 when a node is created. It is increased -%% by 1 each time a child is added or removed. Changes made to existing nodes -%% are not reflected in this version. - --type child_list_length() :: non_neg_integer(). -%% Number of direct child nodes under a tree node. - --type node_props() :: - #{data => data(), - sproc => khepri_fun:standalone_fun(), - payload_version => payload_version(), - child_list_version => child_list_version(), - child_list_length => child_list_length(), - child_nodes => #{khepri_path:node_id() => node_props()}}. -%% Structure used to return properties, payload and child nodes for a specific -%% node. -%% -%%- `{ok, NodePropsMap}' where `NodePropsMap' is a {@link -%% node_props_map/0}: -%%
-%%-%%
- The map returned by {@link get/2}, {@link get/3} and {@link delete/2} -%% contains one entry per node matching the path pattern.
-%%- The map returned by {@link put/3}, {@link put/4} and {@link put/5} -%% contains a single entry if the modified node existed before the update, or -%% no entry if it didn't.
-%%- `{error, Reason}' if an error occurred. In the case, no modifications to -%% the tree was performed.
-%%-%%
- --type node_props_map() :: #{khepri_path:path() => node_props()}. -%% Structure used to return a map of nodes and their associated properties, -%% payload and child nodes. -%% -%% This structure is used in the return value of all commands and queries. - --type result() :: khepri:ok(node_props_map()) | - khepri:error(). -%% Return value of a query or synchronous command. - --type stat() :: #{payload_version := payload_version(), - child_list_version := child_list_version()}. -%% Stats attached to each node in the tree structure. - --type payload_data() :: #kpayload_data{}. - --type payload_sproc() :: #kpayload_sproc{}. - --type payload() :: none | payload_data() | payload_sproc(). -%% All types of payload stored in the nodes of the tree structure. -%% -%% Beside the absence of payload, the only type of payload supported is data. - -type tree_node() :: #node{}. %% A node in the tree structure. --type trigger_id() :: atom(). -%% An ID to identify a registered trigger. - --type event_filter_tree() :: #kevf_tree{}. - --type event_filter() :: event_filter_tree(). +-type stat() :: #{payload_version := khepri:payload_version(), + child_list_version := khepri:child_list_version()}. +%% Stats attached to each node in the tree structure. -type triggered() :: #triggered{}. @@ -196,88 +81,18 @@ -type machine_config() :: #config{}. %% Configuration record, holding read-only or rarely changing fields. --type keep_while_conds_map() :: #{khepri_path:path() => - khepri_condition:keep_while()}. -%% Internal index of the per-node keep_while conditions. - -type keep_while_conds_revidx() :: #{khepri_path:path() => #{khepri_path:path() => ok}}. %% Internal reverse index of the keep_while conditions. If node A depends on a %% condition on node B, then this reverse index will have a "node B => node A" %% entry. --type keep_while_aftermath() :: #{khepri_path:path() => node_props() | delete}. +-type keep_while_aftermath() :: #{khepri_path:path() => + khepri:node_props() | delete}. %% Internal index of the per-node changes which happened during a traversal. %% This is used when the tree is walked back up to determine the list of tree %% nodes to remove after some keep_while condition evaluates to false. --type async_option() :: boolean() | - ra_server:command_correlation() | - ra_server:command_priority() | - {ra_server:command_correlation(), - ra_server:command_priority()}. -%% Option to indicate if the command should be synchronous or asynchronous. -%% -%% Values are: -%%- Payload version, child list version, and child list count are always -%% included in the structure. The reason the type spec does not make them -%% mandatory is for {@link khepri_utils:flat_struct_to_tree/1} which may -%% construct fake node props without them.
-%%- Data is only included if there is data in the node's payload. Absence of -%% data is represented as no `data' entry in this structure.
-%%- Child nodes are only included if requested.
-%%-%%
- --type favor_option() :: consistency | compromise | low_latency. -%% Option to indicate where to put the cursor between freshness of the -%% returned data and low latency of queries. -%% -%% Values are: -%%- `true' to perform an asynchronous low-priority command without a -%% correlation ID.
-%%- `false' to perform a synchronous command.
-%%- A correlation ID to perform an asynchronous low-priority command with -%% that correlation ID.
-%%- A priority to perform an asynchronous command with the specified -%% priority but without a correlation ID.
-%%- A combination of a correlation ID and a priority to perform an -%% asynchronous command with the specified parameters.
-%%-%%
- --type command_options() :: #{async => async_option()}. -%% Options used in commands. -%% -%% Commands are {@link put/5}, {@link delete/3} and read-write {@link -%% transaction/4}. -%% -%%- `consistent' means that a "consistent query" will be used in Ra. It -%% will return the most up-to-date piece of data the cluster agreed on. Note -%% that it could block and eventually time out if there is no quorum in the Ra -%% cluster.
-%%- `compromise' performs "leader queries" most of the time to reduce -%% latency, but uses "consistent queries" every 10 seconds to verify that the -%% cluster is healthy on a regular basis. It should be faster but may block -%% and time out like `consistent' and still return slightly out-of-date -%% data.
-%%- `low_latency' means that "local queries" are used exclusively. They are -%% the fastest and have the lowest latency. However, the returned data is -%% whatever the local Ra server has. It could be out-of-date if it has -%% troubles keeping up with the Ra cluster. The chance of blocking and timing -%% out is very small.
-%%-%%
- --type query_options() :: #{expect_specific_node => boolean(), - include_child_names => boolean(), - favor => favor_option()}. -%% Options used in queries. -%% -%%- `async' indicates the synchronous or asynchronous nature of the -%% command; see {@link async_option()}.
-%%-%%
- -type state() :: #?MODULE{}. %% State of this Ra state machine. @@ -287,7 +102,7 @@ -type walk_down_the_tree_extra() :: #{include_root_props => boolean(), keep_while_conds => - keep_while_conds_map(), + khepri:keep_while_conds_map(), keep_while_conds_revidx => keep_while_conds_revidx(), keep_while_aftermath => @@ -306,29 +121,11 @@ -type ok(Type1, Type2, Type3) :: {ok, Type1, Type2, Type3}. -type error(Type1, Type2) :: {error, Type1, Type2}. --export_type([data/0, - stat/0, - payload/0, - payload_data/0, - payload_sproc/0, - tree_node/0, - trigger_id/0, - event_filter/0, - event_filter_tree/0, - triggered/0, - payload_version/0, - child_list_version/0, - child_list_length/0, - node_props/0, - node_props_map/0, - result/0, - async_option/0, - favor_option/0, - command_options/0, - query_options/0]). -export_type([state/0, machine_config/0, - keep_while_conds_map/0, + tree_node/0, + stat/0, + triggered/0, keep_while_conds_revidx/0]). %% ------------------------------------------------------------------- @@ -338,90 +135,16 @@ %% TODO: Verify arguments carefully to avoid the construction of an invalid %% command. --spec put(StoreId, PathPattern, Payload) -> Result when - StoreId :: khepri:store_id(), - PathPattern :: khepri_path:pattern() | string(), - Payload :: payload(), - Result :: result() | NoRetIfAsync, - NoRetIfAsync :: ok. -%% @doc Creates or modifies a specific tree node in the tree structure. -%% -%% Calling this function is the same as calling -%% `put(StoreId, PathPattern, Payload, #{}, #{})'. -%% -%% @see put/5. - -put(StoreId, PathPattern, Payload) -> - put(StoreId, PathPattern, Payload, #{}, #{}). - --spec put(StoreId, PathPattern, Payload, Extra | Options) -> Result when - StoreId :: khepri:store_id(), - PathPattern :: khepri_path:pattern() | string(), - Payload :: payload(), - Extra :: #{keep_while => keep_while_conds_map()}, - Options :: command_options(), - Result :: result() | NoRetIfAsync, - NoRetIfAsync :: ok. -%% @doc Creates or modifies a specific tree node in the tree structure. -%% -%% @see put/5. - -put(StoreId, PathPattern, Payload, #{keep_while := _} = Extra) -> - put(StoreId, PathPattern, Payload, Extra, #{}); -put(StoreId, PathPattern, Payload, Options) -> - put(StoreId, PathPattern, Payload, #{}, Options). - -spec put(StoreId, PathPattern, Payload, Extra, Options) -> Result when StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern() | string(), - Payload :: payload(), - Extra :: #{keep_while => keep_while_conds_map()}, - Options :: command_options(), - Result :: result() | NoRetIfAsync, + Payload :: khepri_payload:payload(), + Extra :: #{keep_while => khepri:keep_while_conds_map()}, + Options :: khepri:command_options(), + Result :: khepri:result() | NoRetIfAsync, NoRetIfAsync :: ok. %% @doc Creates or modifies a specific tree node in the tree structure. %% -%% The path or path pattern must target a specific tree node. -%% -%% When using a simple path, if the target node does not exists, it is created -%% using the given payload. If the target node exists, it is updated with the -%% given payload and its payload version is increased by one. Missing parent -%% nodes are created on the way. -%% -%% When using a path pattern, the behavior is the same. However if a condition -%% in the path pattern is not met, an error is returned and the tree structure -%% is not modified. -%% -%% If the target node is modified, the returned structure in the "ok" tuple -%% will have a single key corresponding to the path of the target node. That -%% key will point to a map containing the properties and payload (if any) of -%% the node before the modification. -%% -%% If the target node is created, the returned structure in the "ok" tuple -%% will have a single key corresponding to the path of the target node. That -%% key will point to empty map, indicating there was no existing node (i.e. -%% there was no properties or payload to return). -%% -%% The payload must be one of the following form: -%%- `expect_specific_node' indicates if the path is expected to point to a -%% specific tree node or could match many nodes.
-%%- `include_child_names' indicates if child names should be included in -%% the returned node properties map.
-%%- `favor' indicates where to put the cursor between freshness of the -%% returned data and low latency of queries; see {@link favor_option()}.
-%%-%%
-%% -%% Example: -%% ``` -%% %% Insert a node at `/foo/bar', overwriting the previous value. -%% Result = khepri_machine:put( -%% ra_cluster_name, [foo, bar], #kpayload_data{data = new_value}), -%% -%% %% Here is the content of `Result'. -%% {ok, #{[foo, bar] => #{data => old_value, -%% payload_version => 1, -%% child_list_version => 1, -%% child_list_length => 0}}} = Result. -%% ''' -%% %% @param StoreId the name of the Ra cluster. %% @param PathPattern the path (or path pattern) to the node to create or %% modify. @@ -429,16 +152,18 @@ put(StoreId, PathPattern, Payload, Options) -> %% @param Extra extra options such as `keep_while' conditions. %% @param Options command options such as the command type. %% -%% @returns in the case of a synchronous put, an "ok" tuple with a map with one -%% entry, or an "error" tuple; in the case of an asynchronous put, always `ok' -%% (the actual return value may be sent by a message if a correlation ID was -%% specified). +%% @returns in the case of a synchronous put, an `{ok, Result}' tuple with a +%% map with one entry, or an `{error, Reason}' tuple; in the case of an +%% asynchronous put, always `ok' (the actual return value may be sent by a +%% message if a correlation ID was specified). +%% +%% @private put(StoreId, PathPattern, Payload, Extra, Options) when ?IS_KHEPRI_PAYLOAD(Payload) -> PathPattern1 = khepri_path:from_string(PathPattern), khepri_path:ensure_is_valid(PathPattern1), - Payload1 = prepare_payload(Payload), + Payload1 = khepri_payload:prepare(Payload), Command = #put{path = PathPattern1, payload = Payload1, extra = Extra}, @@ -446,60 +171,19 @@ put(StoreId, PathPattern, Payload, Extra, Options) put(_StoreId, PathPattern, Payload, _Extra, _Options) -> throw({invalid_payload, PathPattern, Payload}). -prepare_payload(none = Payload) -> - Payload; -prepare_payload(#kpayload_data{} = Payload) -> - Payload; -prepare_payload(#kpayload_sproc{sproc = Fun} = Payload) - when is_function(Fun) -> - StandaloneFun = khepri_sproc:to_standalone_fun(Fun), - Payload#kpayload_sproc{sproc = StandaloneFun}. - --spec get(StoreId, PathPattern) -> Result when - StoreId :: khepri:store_id(), - PathPattern :: khepri_path:pattern() | string(), - Result :: result(). -%% @doc Returns all tree nodes matching the path pattern. -%% -%% Calling this function is the same as calling -%% `get(StoreId, PathPattern, #{})'. -%% -%% @see get/3. - -get(StoreId, PathPattern) -> - get(StoreId, PathPattern, #{}). - -spec get(StoreId, PathPattern, Options) -> Result when StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern() | string(), - Options :: query_options(), - Result :: result(). + Options :: khepri:query_options(), + Result :: khepri:result(). %% @doc Returns all tree nodes matching the path pattern. %% -%% The returned structure in the "ok" tuple will have a key corresponding to -%% the path per node which matched the pattern. Each key will point to a map -%% containing the properties and payload of that matching node. -%% -%% Example: -%% ``` -%% %% Query the node at `/foo/bar'. -%% Result = khepri_machine:get(ra_cluster_name, [foo, bar]), -%% -%% %% Here is the content of `Result'. -%% {ok, #{[foo, bar] => #{data => new_value, -%% payload_version => 2, -%% child_list_version => 1, -%% child_list_length => 0}}} = Result. -%% ''' -%% %% @param StoreId the name of the Ra cluster. -%% @param PathPattern the path (or path pattern) to match against the nodes to -%% retrieve. -%% @param Options options to tune the tree traversal or the returned structure -%% content. +%% @param PathPattern the path (or path pattern) to the nodes to get. +%% @param Options query options such as `favor'. %% -%% @returns an "ok" tuple with a map with zero, one or more entries, or an -%% "error" tuple. +%% @returns an `{ok, Result}' tuple with a map with zero, one or more entries, +%% or an `{error, Reason}' tuple. get(StoreId, PathPattern, Options) -> PathPattern1 = khepri_path:from_string(PathPattern), @@ -509,54 +193,22 @@ get(StoreId, PathPattern, Options) -> end, process_query(StoreId, Query, Options). --spec delete(StoreId, PathPattern) -> Result when - StoreId :: khepri:store_id(), - PathPattern :: khepri_path:pattern() | string(), - Result :: result() | NoRetIfAsync, - NoRetIfAsync :: ok. -%% @doc Deletes all tree nodes matching the path pattern. -%% -%% Calling this function is the same as calling -%% `delete(StoreId, PathPattern, #{})'. -%% -%% @see delete/3. - -delete(StoreId, PathPattern) -> - delete(StoreId, PathPattern, #{}). - -spec delete(StoreId, PathPattern, Options) -> Result when StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern() | string(), - Options :: command_options(), - Result :: result() | NoRetIfAsync, + Options :: khepri:command_options(), + Result :: khepri:result() | NoRetIfAsync, NoRetIfAsync :: ok. %% @doc Deletes all tree nodes matching the path pattern. %% -%% The returned structure in the "ok" tuple will have a key corresponding to -%% the path per node which was deleted. Each key will point to a map containing -%% the properties and payload of that deleted node. -%% -%% Example: -%% ``` -%% %% Delete the node at `/foo/bar'. -%% Result = khepri_machine:delete(ra_cluster_name, [foo, bar]), -%% -%% %% Here is the content of `Result'. -%% {ok, #{[foo, bar] => #{data => new_value, -%% payload_version => 2, -%% child_list_version => 1, -%% child_list_length => 0}}} = Result. -%% ''' -%% %% @param StoreId the name of the Ra cluster. -%% @param PathPattern the path (or path pattern) to match against the nodes to -%% delete. +%% @param PathPattern the path (or path pattern) to the nodes to delete. %% @param Options command options such as the command type. %% -%% @returns in the case of a synchrnous delete, an "ok" tuple with a map with -%% zero, one or more entries, or an "error" tuple; in the case of an -%% asynchronous delete, always `ok' (the actual return value may be sent by a -%% message if a correlation ID was specified). +%% @returns in the case of a synchronous delete, an `{ok, Result}' tuple with +%% a map with zero, one or more entries, or an `{error, Reason}' tuple; in the +%% case of an asynchronous put, always `ok' (the actual return value may be +%% sent by a message if a correlation ID was specified). delete(StoreId, PathPattern, Options) -> PathPattern1 = khepri_path:from_string(PathPattern), @@ -564,83 +216,20 @@ delete(StoreId, PathPattern, Options) -> Command = #delete{path = PathPattern1}, process_command(StoreId, Command, Options). --spec transaction(StoreId, Fun) -> Ret when - StoreId :: khepri:store_id(), - Fun :: khepri_tx:tx_fun(), - Ret :: Atomic | Aborted | NoRetIfAsync, - Atomic :: {atomic, khepri_tx:tx_fun_result()}, - Aborted :: khepri_tx:tx_abort(), - NoRetIfAsync :: ok. -%% @doc Runs a transaction and returns the result. -%% -%% Calling this function is the same as calling -%% `transaction(StoreId, Fun, auto, #{})'. -%% -%% @see transaction/4. - -transaction(StoreId, Fun) -> - transaction(StoreId, Fun, auto, #{}). - --spec transaction(StoreId, Fun, ReadWrite | Options) -> Ret when - StoreId :: khepri:store_id(), - Fun :: khepri_tx:tx_fun(), - ReadWrite :: ro | rw | auto, - Options :: command_options() | query_options(), - Ret :: Atomic | Aborted | NoRetIfAsync, - Atomic :: {atomic, khepri_tx:tx_fun_result()}, - Aborted :: khepri_tx:tx_abort(), - NoRetIfAsync :: ok. -%% @doc Runs a transaction and returns the result. -%% -%% @see transaction/4. - -transaction(StoreId, Fun, ReadWrite) when is_atom(ReadWrite) -> - transaction(StoreId, Fun, ReadWrite, #{}); -transaction(StoreId, Fun, Options) when is_map(Options) -> - transaction(StoreId, Fun, auto, Options). - -spec transaction(StoreId, Fun, ReadWrite, Options) -> Ret when StoreId :: khepri:store_id(), Fun :: khepri_tx:tx_fun(), ReadWrite :: ro | rw | auto, - Options :: command_options() | query_options(), + Options :: khepri:command_options() | khepri:query_options(), Ret :: Atomic | Aborted | NoRetIfAsync, Atomic :: {atomic, khepri_tx:tx_fun_result()}, Aborted :: khepri_tx:tx_abort(), NoRetIfAsync :: ok. %% @doc Runs a transaction and returns the result. %% -%% `Fun' is an arbitrary anonymous function which takes no arguments. -%% -%% The `ReadWrite' flag determines what the anonymous function is allowed to -%% do and in which context it runs: -%% -%%- `none', meaning there will be no payload attached to the node
-%%- `#kpayload_data{data = Term}' to store any type of term in the -%% node
-%%-%%
-%% -%% `Options' is relevant for both read-only and read-write transactions -%% (including audetected ones). Note that both types expect different options. -%% -%% The result of `Fun' can be any term. That result is returned in an -%% `{atomic, Result}' tuple. -%% %% @param StoreId the name of the Ra cluster. %% @param Fun an arbitrary anonymous function. +%% @param ReadWrite the read/write or read-only nature of the transaction. %% @param Options command options such as the command type. %% %% @returns in the case of a synchronous transaction, `{atomic, Result}' where @@ -672,7 +261,7 @@ transaction(_StoreId, Term, _ReadWrite, _Options) -> -spec readonly_transaction(StoreId, Fun, Options) -> Ret when StoreId :: khepri:store_id(), Fun :: khepri_tx:tx_fun(), - Options :: query_options(), + Options :: khepri:query_options(), Ret :: Atomic | Aborted, Atomic :: {atomic, khepri_tx:tx_fun_result()}, Aborted :: khepri_tx:tx_abort(). @@ -697,7 +286,7 @@ readonly_transaction(StoreId, Fun, Options) when is_function(Fun, 0) -> -spec readwrite_transaction(StoreId, Fun, Options) -> Ret when StoreId :: khepri:store_id(), Fun :: khepri_fun:standalone_fun(), - Options :: command_options(), + Options :: khepri:command_options(), Ret :: Atomic | Aborted | NoRetIfAsync, Atomic :: {atomic, khepri_tx:tx_fun_result()}, Aborted :: khepri_tx:tx_abort(), @@ -720,10 +309,11 @@ readwrite_transaction(StoreId, StandaloneFun, Options) -> {atomic, Ret} end. --spec run_sproc(StoreId, PathPattern, Args) -> Ret when +-spec run_sproc(StoreId, PathPattern, Args, Options) -> Ret when StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern() | string(), Args :: [any()], + Options :: khepri:query_options(), Ret :: any(). %% @doc Executes a stored procedure. %% @@ -733,13 +323,17 @@ readwrite_transaction(StoreId, StandaloneFun, Options) -> %% @param StoreId the name of the Ra cluster. %% @param PathPattern the path to the stored procedure. %% @param Args the list of args to pass to the stored procedure; its length -%% must be equal to the stored procedure arity. +%% must be equal to the stored procedure arity. +%% @param Options options to tune the tree traversal or the returned structure +%% content. %% -%% @returns the return value of the stored procedure. +%% @returns the result of the stored procedure execution, or throws an +%% exception if the node does not exist, does not hold a stored procedure or +%% if there was an error. -run_sproc(StoreId, PathPattern, Args) when is_list(Args) -> - Options = #{expect_specific_node => true}, - case get(StoreId, PathPattern, Options) of +run_sproc(StoreId, PathPattern, Args, Options) when is_list(Args) -> + Options1 = Options#{expect_specific_node => true}, + case get(StoreId, PathPattern, Options1) of {ok, Result} -> [Value] = maps:values(Result), case Value of @@ -754,54 +348,36 @@ run_sproc(StoreId, PathPattern, Args) when is_list(Args) -> throw({invalid_sproc_fun, Error}) end. --spec register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath) -> +-spec register_trigger( + StoreId, TriggerId, EventFilter, StoredProcPath, Options) -> Ret when StoreId :: khepri:store_id(), - TriggerId :: trigger_id(), - EventFilter :: event_filter(), - StoredProcPath :: khepri_path:path(), + TriggerId :: khepri:trigger_id(), + EventFilter :: khepri_evf:event_filter() | + khepri_path:pattern() | string(), + StoredProcPath :: khepri_path:path() | string(), + Options :: khepri:command_options(), Ret :: ok | khepri:error(). %% @doc Registers a trigger. %% -%% A trigger is based on an event filter. It associates an event with a stored -%% procedure. When an event matching the event filter is emitted, the stored -%% procedure is executed. Here is an example of an event filter: -%% -%% ``` -%% EventFilter = #kevf_tree{path = [stock, wood, <<"oak">>], %% Required -%% props = #{on_actions => [delete], %% Optional -%% priority => 10}}, %% Optional -%% ''' -%% -%% The stored procedure is expected to accept a single argument. This argument -%% is a map containing the event properties. Here is an example: -%% -%% ``` -%% my_stored_procedure(Props) -> -%% #{path := Path}, -%% on_action => Action} = Props. -%% ''' -%% -%% The stored procedure is executed on the leader's Erlang node. -%% -%% It is guaranteed to run at least once. It could be executed multiple times -%% if the Ra leader changes, therefore the stored procedure must be -%% idempotent. -%% %% @param StoreId the name of the Ra cluster. %% @param TriggerId the name of the trigger. -%% @param EventFilter the event filter used to associate an event with a stored -%% procedure. +%% @param EventFilter the event filter used to associate an event with a +%% stored procedure. %% @param StoredProcPath the path to the stored procedure to execute when the -%% corresponding event occurs. +%% corresponding event occurs. %% -%% @returns `ok' if the trigger was registered, an "error" tuple otherwise. +%% @returns `ok' if the trigger was registered, an `{error, Reason}' tuple +%% otherwise. -register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath) -> +register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath, Options) -> + EventFilter1 = khepri_evf:wrap(EventFilter), + StoredProcPath1 = khepri_path:from_string(StoredProcPath), + khepri_path:ensure_is_valid(StoredProcPath1), Command = #register_trigger{id = TriggerId, - sproc = StoredProcPath, - event_filter = EventFilter}, - process_command(StoreId, Command, #{}). + sproc = StoredProcPath1, + event_filter = EventFilter1}, + process_command(StoreId, Command, Options). -spec ack_triggers_execution(StoreId, TriggeredStoredProcs) -> Ret when @@ -821,7 +397,7 @@ ack_triggers_execution(StoreId, TriggeredStoredProcs) -> -spec get_keep_while_conds_state(StoreId) -> Ret when StoreId :: khepri:store_id(), - Ret :: {ok, keep_while_conds_map()} | khepri:error(). + Ret :: {ok, khepri:keep_while_conds_map()} | khepri:error(). %% @doc Returns the `keep_while' conditions internal state. %% %% The returned state consists of all the `keep_while' condition set so far. @@ -842,7 +418,7 @@ get_keep_while_conds_state(StoreId) -> -spec process_command(StoreId, Command, Options) -> Ret when StoreId :: khepri:store_id(), Command :: command(), - Options :: command_options(), + Options :: khepri:command_options(), Ret :: any(). %% @doc Processes a command which is appended to the Ra log and processed by %% this state machine code. @@ -891,7 +467,7 @@ process_async_command(StoreId, Command, Correlation, Priority) -> ra:pipeline_command(LocalServerId, Command, Correlation, Priority). -spec select_command_type(Options) -> CommandType when - Options :: command_options(), + Options :: khepri:command_options(), CommandType :: sync | {async, Correlation, Priority}, Correlation :: ra_server:command_correlation(), Priority :: ra_server:command_priority(). @@ -926,7 +502,7 @@ select_command_type(#{async := {Correlation, Priority}}) -spec process_query(StoreId, QueryFun, Options) -> Ret when StoreId :: khepri:store_id(), QueryFun :: query_fun(), - Options :: query_options(), + Options :: khepri:query_options(), Ret :: any(). %% @doc Processes a query which is by the Ra leader. %% @@ -1072,7 +648,7 @@ cache_leader_if_changed(StoreId, _OldLeaderId, NewLeaderId) -> -spec select_query_type(StoreId, Options) -> QueryType when StoreId :: khepri:store_id(), - Options :: query_options(), + Options :: khepri:query_options(), QueryType :: local | leader | consistent. %% @doc Selects the query type depending on what the caller favors. %% @@ -1162,7 +738,9 @@ clear_cache(StoreId) -> %% ra_machine callbacks. %% ------------------------------------------------------------------- --spec init(machine_init_args()) -> state(). +-spec init(Params) -> State when + Params :: machine_init_args(), + State :: state(). %% @private init(#{store_id := StoreId} = Params) -> @@ -1226,9 +804,9 @@ apply( #?MODULE{triggers = Triggers} = State) -> StoredProcPath1 = khepri_path:realpath(StoredProcPath), EventFilter1 = case EventFilter of - #kevf_tree{path = Path} -> + #evf_tree{path = Path} -> Path1 = khepri_path:realpath(Path), - EventFilter#kevf_tree{path = Path1} + EventFilter#evf_tree{path = Path1} end, Triggers1 = Triggers#{TriggerId => #{sproc => StoredProcPath1, event_filter => EventFilter1}}, @@ -1307,7 +885,7 @@ state_enter(_StateName, _State) -> %% ------------------------------------------------------------------- -spec create_node_record(Payload) -> Node when - Payload :: payload(), + Payload :: khepri_payload:payload(), Node :: tree_node(). %% @private @@ -1315,8 +893,9 @@ create_node_record(Payload) -> #node{stat = ?INIT_NODE_STAT, payload = Payload}. --spec set_node_payload(tree_node(), payload()) -> - tree_node(). +-spec set_node_payload(Node, Payload) -> Node when + Node :: tree_node(), + Payload :: khepri_payload:payload(). %% @private set_node_payload(#node{payload = Payload} = Node, Payload) -> @@ -1326,19 +905,22 @@ set_node_payload(#node{stat = #{payload_version := DVersion} = Stat} = Node, Stat1 = Stat#{payload_version => DVersion + 1}, Node#node{stat = Stat1, payload = Payload}. --spec remove_node_payload(tree_node()) -> tree_node(). +-spec remove_node_payload(Node) -> Node when + Node :: tree_node(). %% @private remove_node_payload( - #node{payload = none} = Node) -> + #node{payload = ?NO_PAYLOAD} = Node) -> Node; remove_node_payload( #node{stat = #{payload_version := DVersion} = Stat} = Node) -> Stat1 = Stat#{payload_version => DVersion + 1}, - Node#node{stat = Stat1, payload = none}. + Node#node{stat = Stat1, payload = khepri_payload:none()}. --spec add_node_child(tree_node(), khepri_path:component(), tree_node()) -> - tree_node(). +-spec add_node_child(Node, ChildName, Child) -> Node when + Node :: tree_node(), + Child :: tree_node(), + ChildName :: khepri_path:component(). add_node_child(#node{stat = #{child_list_version := CVersion} = Stat, child_nodes = Children} = Node, @@ -1347,15 +929,18 @@ add_node_child(#node{stat = #{child_list_version := CVersion} = Stat, Stat1 = Stat#{child_list_version => CVersion + 1}, Node#node{stat = Stat1, child_nodes = Children1}. --spec update_node_child(tree_node(), khepri_path:component(), tree_node()) -> - tree_node(). +-spec update_node_child(Node, ChildName, Child) -> Node when + Node :: tree_node(), + Child :: tree_node(), + ChildName :: khepri_path:component(). update_node_child(#node{child_nodes = Children} = Node, ChildName, Child) -> Children1 = Children#{ChildName => Child}, Node#node{child_nodes = Children1}. --spec remove_node_child(tree_node(), khepri_path:component()) -> - tree_node(). +-spec remove_node_child(Node, ChildName) -> Node when + Node :: tree_node(), + ChildName :: khepri_path:component(). remove_node_child(#node{stat = #{child_list_version := CVersion} = Stat, child_nodes = Children} = Node, @@ -1365,7 +950,8 @@ remove_node_child(#node{stat = #{child_list_version := CVersion} = Stat, Children1 = maps:remove(ChildName, Children), Node#node{stat = Stat1, child_nodes = Children1}. --spec remove_node_child_nodes(tree_node()) -> tree_node(). +-spec remove_node_child_nodes(Node) -> Node when + Node :: tree_node(). remove_node_child_nodes( #node{child_nodes = Children} = Node) when Children =:= #{} -> @@ -1375,8 +961,10 @@ remove_node_child_nodes( Stat1 = Stat#{child_list_version => CVersion + 1}, Node#node{stat = Stat1, child_nodes = #{}}. --spec gather_node_props(tree_node(), command_options() | query_options()) -> - node_props(). +-spec gather_node_props(Node, Options) -> NodeProps when + Node :: tree_node(), + Options :: khepri:command_options() | khepri:query_options(), + NodeProps :: khepri:node_props(). gather_node_props(#node{stat = #{payload_version := DVersion, child_list_version := CVersion}, @@ -1393,8 +981,8 @@ gather_node_props(#node{stat = #{payload_version := DVersion, Result0 end, case Payload of - #kpayload_data{data = Data} -> Result1#{data => Data}; - #kpayload_sproc{sproc = Fun} -> Result1#{sproc => Fun}; + #p_data{data = Data} -> Result1#{data => Data}; + #p_sproc{sproc = Fun} -> Result1#{sproc => Fun}; _ -> Result1 end. @@ -1410,9 +998,10 @@ to_absolute_keep_while(BasePath, KeepWhile) -> Acc#{AbsPath => Cond} end, #{}, KeepWhile). --spec are_keep_while_conditions_met( - tree_node(), khepri_condition:keep_while()) -> - true | {false, any()}. +-spec are_keep_while_conditions_met(Node, KeepWhile) -> Ret when + Node :: tree_node(), + KeepWhile :: khepri_condition:keep_while(), + Ret :: true | {false, any()}. %% @private are_keep_while_conditions_met(_, KeepWhile) @@ -1455,9 +1044,12 @@ is_keep_while_condition_met_on_self(_, _, _) -> true. -spec update_keep_while_conds_revidx( - keep_while_conds_map(), keep_while_conds_revidx(), - khepri_path:path(), khepri_condition:keep_while()) -> - keep_while_conds_revidx(). + KeepWhileConds, KeepWhileCondsRevIdx, Watcher, KeepWhile) -> + KeepWhileConds when + KeepWhileConds :: khepri:keep_while_conds_map(), + KeepWhileCondsRevIdx :: keep_while_conds_revidx(), + Watcher :: khepri_path:path(), + KeepWhile :: khepri_condition:keep_while(). update_keep_while_conds_revidx( KeepWhileConds, KeepWhileCondsRevIdx, Watcher, KeepWhile) -> @@ -1481,11 +1073,11 @@ update_keep_while_conds_revidx( KWRevIdx#{Watched => Watchers1} end, KeepWhileCondsRevIdx1, KeepWhile). --spec find_matching_nodes( - tree_node(), - khepri_path:pattern(), - query_options()) -> - result(). +-spec find_matching_nodes(Root, PathPattern, Options) -> Result when + Root :: tree_node(), + PathPattern :: khepri_path:pattern(), + Options :: khepri:query_options(), + Result :: khepri:result(). %% @private find_matching_nodes(Root, PathPattern, Options) -> @@ -1518,10 +1110,13 @@ find_matching_nodes_cb( find_matching_nodes_cb(_, {interrupted, _, _}, _, Result) -> {ok, keep, Result}. --spec insert_or_update_node( - state(), khepri_path:pattern(), payload(), - #{keep_while => khepri_condition:keep_while()}) -> - {state(), result()} | {state(), result(), ra_machine:effects()}. +-spec insert_or_update_node(State, PathPattern, Payload, Extra) -> Ret when + State :: state(), + PathPattern :: khepri_path:pattern(), + Payload :: khepri_payload:payload(), + Extra :: #{keep_while => khepri_condition:keep_while()}, + Ret :: {State, Result} | {State, Result, ra_machine:effects()}, + Result :: khepri:result(). %% @private insert_or_update_node( @@ -1645,7 +1240,7 @@ insert_or_update_node_cb( NodeProps = #{}, {ok, Node, Result#{Path => NodeProps}}; true -> - Node = create_node_record(none), + Node = create_node_record(khepri_payload:none()), {ok, Node, Result}; false -> {error, {Reason, Info}} @@ -1670,8 +1265,11 @@ can_continue_update_after_node_not_found1(#if_any{conditions = Conds}) -> can_continue_update_after_node_not_found1(_) -> false. --spec delete_matching_nodes(state(), khepri_path:pattern()) -> - {state(), result()} | {state(), result(), ra_machine:effects()}. +-spec delete_matching_nodes(State, PathPattern) -> Ret when + State :: state(), + PathPattern :: khepri_path:pattern(), + Ret :: {State, Result} | {State, Result, ra_machine:effects()}, + Result :: khepri:result(). %% @private delete_matching_nodes( @@ -1770,7 +1368,7 @@ list_triggered_sprocs(Root, Changes, Triggers) -> (TriggerId, #{sproc := StoredProcPath, event_filter := - #kevf_tree{path = PathPattern, + #evf_tree{path = PathPattern, props = EventFilterProps} = EventFilter}, SPP1) -> %% For each trigger based on a tree event: @@ -1885,8 +1483,8 @@ sort_triggered_sprocs(TriggeredStoredProcs) -> lists:sort( fun(#triggered{id = IdA, event_filter = EventFilterA}, #triggered{id = IdB, event_filter = EventFilterB}) -> - PrioA = get_event_filter_priority(EventFilterA), - PrioB = get_event_filter_priority(EventFilterB), + PrioA = khepri_evf:get_priority(EventFilterA), + PrioB = khepri_evf:get_priority(EventFilterB), if PrioA =:= PrioB -> IdA =< IdB; true -> PrioA > PrioB @@ -1894,20 +1492,18 @@ sort_triggered_sprocs(TriggeredStoredProcs) -> end, TriggeredStoredProcs). -get_event_filter_priority(#kevf_tree{props = #{priority := Priority}}) - when is_integer(Priority) -> - Priority; -get_event_filter_priority(_EventFilter) -> - 0. - %% ------- -spec walk_down_the_tree( - tree_node(), khepri_path:pattern(), specific_node | many_nodes, - walk_down_the_tree_extra(), - walk_down_the_tree_fun(), any()) -> - ok(tree_node(), walk_down_the_tree_extra(), any()) | - khepri:error(). + Root, PathPattern, WorkOnWhat, Extra, Fun, FunAcc) -> Ret when + Root :: tree_node(), + PathPattern :: khepri_path:pattern(), + WorkOnWhat :: specific_node | many_nodes, + Extra :: walk_down_the_tree_extra(), + Fun :: walk_down_the_tree_fun(), + FunAcc :: any(), + Node :: tree_node(), + Ret :: ok(Node, Extra, FunAcc) | khepri:error(). %% @private walk_down_the_tree(Root, PathPattern, WorkOnWhat, Extra, Fun, FunAcc) -> @@ -1920,12 +1516,18 @@ walk_down_the_tree(Root, PathPattern, WorkOnWhat, Extra, Fun, FunAcc) -> Extra, Fun, FunAcc). -spec walk_down_the_tree1( - tree_node(), khepri_path:pattern(), specific_node | many_nodes, - khepri_path:pattern(), [tree_node() | {tree_node(), child_created}], - walk_down_the_tree_extra(), - walk_down_the_tree_fun(), any()) -> - ok(tree_node(), walk_down_the_tree_extra(), any()) | - khepri:error(). + Root, CompiledPathPattern, WorkOnWhat, + ReversedPath, ReversedParentTree, Extra, Fun, FunAcc) -> Ret when + Root :: tree_node(), + CompiledPathPattern :: khepri_path:pattern(), + WorkOnWhat :: specific_node | many_nodes, + ReversedPath :: khepri_path:pattern(), + ReversedParentTree :: [Node | {Node, child_created}], + Extra :: walk_down_the_tree_extra(), + Fun :: walk_down_the_tree_fun(), + FunAcc :: any(), + Node :: tree_node(), + Ret :: ok(Node, Extra, FunAcc) | khepri:error(). %% @private walk_down_the_tree1( @@ -2065,9 +1667,10 @@ walk_down_the_tree1( %% conditions on child nodes. {error, targets_dot_dot}; false -> - %% TODO: Should we provide more details about the error, like the - %% list of matching nodes? - {error, matches_many_nodes} + %% The caller expects that the path matches a single specific node + %% (no matter if it exists or not), but the condition could match + %% several nodes. + {error, {possibly_matching_many_nodes_denied, Condition}} end; walk_down_the_tree1( #node{child_nodes = Children} = CurrentNode, @@ -2173,24 +1776,26 @@ walk_down_the_tree1( Error end. --spec special_component_to_node_name( - ?ROOT_NODE | ?THIS_NODE, - khepri_path:pattern()) -> - khepri_path:component(). +-spec special_component_to_node_name(SpecialComponent, Path) -> NodeName when + SpecialComponent :: ?ROOT_NODE | ?THIS_NODE, + Path :: khepri_path:pattern(), + NodeName :: khepri_path:component(). special_component_to_node_name(?ROOT_NODE = NodeName, []) -> NodeName; special_component_to_node_name(?THIS_NODE, [NodeName | _]) -> NodeName; special_component_to_node_name(?THIS_NODE, []) -> ?ROOT_NODE. --spec starting_node_in_rev_parent_tree([tree_node()]) -> - tree_node(). +-spec starting_node_in_rev_parent_tree(ReversedParentTree) -> Node when + Node :: tree_node(), + ReversedParentTree :: [Node]. %% @private starting_node_in_rev_parent_tree(ReversedParentTree) -> hd(lists:reverse(ReversedParentTree)). --spec starting_node_in_rev_parent_tree([tree_node()], tree_node()) -> - tree_node(). +-spec starting_node_in_rev_parent_tree(ReversedParentTree, Node) -> Node when + Node :: tree_node(), + ReversedParentTree :: [Node]. %% @private starting_node_in_rev_parent_tree([], CurrentNode) -> @@ -2199,13 +1804,18 @@ starting_node_in_rev_parent_tree(ReversedParentTree, _) -> starting_node_in_rev_parent_tree(ReversedParentTree). -spec handle_branch( - tree_node(), khepri_path:component(), tree_node(), - khepri_path:pattern(), specific_node | many_nodes, - [tree_node() | {tree_node(), child_created}], - walk_down_the_tree_extra(), - walk_down_the_tree_fun(), any()) -> - ok(tree_node(), walk_down_the_tree_extra(), any()) | - khepri:error(). + Node, ChildName, Child, WholePathPattern, WorkOnWhat, + ReversedPath, Extra, Fun, FunAcc) -> Ret when + Node :: tree_node(), + ChildName :: khepri_path:component(), + Child :: tree_node(), + WholePathPattern :: khepri_path:pattern(), + WorkOnWhat :: specific_node | many_nodes, + ReversedPath :: [Node | {Node, child_created}], + Extra :: walk_down_the_tree_extra(), + Fun :: walk_down_the_tree_fun(), + FunAcc :: any(), + Ret :: ok(Node, Extra, FunAcc) | khepri:error(). %% @private handle_branch( @@ -2243,14 +1853,20 @@ handle_branch( end. -spec interrupted_walk_down( - mismatching_node | node_not_found, - map(), - khepri_path:pattern(), specific_node | many_nodes, - khepri_path:path(), [tree_node() | {tree_node(), child_created}], - walk_down_the_tree_extra(), - walk_down_the_tree_fun(), any()) -> - ok(tree_node(), walk_down_the_tree_extra(), any()) | - khepri:error(). + Reason, Info, PathPattern, WorkOnWhat, + ReversedPath, ReversedParentTree, + Extra, Fun, FunAcc) -> Ret when + Reason :: mismatching_node | node_not_found, + Info :: map(), + PathPattern :: khepri_path:pattern(), + WorkOnWhat :: specific_node | many_nodes, + ReversedPath :: khepri_path:path(), + Node :: tree_node(), + ReversedParentTree :: [Node | {Node, child_created}], + Extra :: walk_down_the_tree_extra(), + Fun :: walk_down_the_tree_fun(), + FunAcc :: any(), + Ret :: ok(Node, Extra, FunAcc) | khepri:error(). %% @private interrupted_walk_down( @@ -2297,7 +1913,8 @@ interrupted_walk_down( Error end. --spec reset_versions(tree_node()) -> tree_node(). +-spec reset_versions(Node) -> Node when + Node :: tree_node(). %% @private reset_versions(#node{stat = Stat} = CurrentNode) -> @@ -2305,7 +1922,10 @@ reset_versions(#node{stat = Stat} = CurrentNode) -> child_list_version => ?INIT_CHILD_LIST_VERSION}, CurrentNode#node{stat = Stat1}. --spec squash_version_bumps(tree_node(), tree_node()) -> tree_node(). +-spec squash_version_bumps(OldNode, NewNode) -> Node when + OldNode :: tree_node(), + NewNode :: tree_node(), + Node :: tree_node(). %% @private squash_version_bumps( @@ -2332,11 +1952,14 @@ squash_version_bumps( CurrentNode#node{stat = Stat1}. -spec walk_back_up_the_tree( - tree_node() | delete, khepri_path:path(), - [tree_node() | {tree_node(), child_created}], - walk_down_the_tree_extra(), - any()) -> - ok(tree_node(), walk_down_the_tree_extra(), any()). + Child, ReversedPath, ReversedParentTree, Extra, FunAcc) -> Ret when + Node :: tree_node(), + Child :: Node | delete, + ReversedPath :: khepri_path:path(), + ReversedParentTree :: [Node | {Node, child_created}], + Extra :: walk_down_the_tree_extra(), + FunAcc :: any(), + Ret :: ok(Node, Extra, FunAcc). %% @private walk_back_up_the_tree( @@ -2345,12 +1968,16 @@ walk_back_up_the_tree( Child, ReversedPath, ReversedParentTree, Extra, #{}, FunAcc). -spec walk_back_up_the_tree( - tree_node() | delete, khepri_path:path(), - [tree_node() | {tree_node(), child_created}], - walk_down_the_tree_extra(), - #{khepri_path:path() => tree_node() | delete}, - any()) -> - ok(tree_node(), walk_down_the_tree_extra(), any()). + Child, ReversedPath, ReversedParentTree, Extra, KeepWhileAftermath, + FunAcc) -> Ret when + Node :: tree_node(), + Child :: Node | delete, + ReversedPath :: khepri_path:path(), + ReversedParentTree :: [Node | {Node, child_created}], + Extra :: walk_down_the_tree_extra(), + KeepWhileAftermath :: #{khepri_path:path() => Node | delete}, + FunAcc :: any(), + Ret :: ok(Node, Extra, FunAcc). %% @private walk_back_up_the_tree( diff --git a/src/khepri_machine.hrl b/src/khepri_machine.hrl index a41c3289..64fc11aa 100644 --- a/src/khepri_machine.hrl +++ b/src/khepri_machine.hrl @@ -15,12 +15,12 @@ -record(khepri_machine, {config = #config{} :: khepri_machine:machine_config(), root = #node{stat = ?INIT_NODE_STAT} :: khepri_machine:tree_node(), - keep_while_conds = #{} :: khepri_machine:keep_while_conds_map(), + keep_while_conds = #{} :: khepri:keep_while_conds_map(), keep_while_conds_revidx = #{} :: khepri_machine:keep_while_conds_revidx(), triggers = #{} :: - #{khepri_machine:trigger_id() => + #{khepri:trigger_id() => #{sproc := khepri_path:path(), - event_filter := khepri_machine:event_filter()}}, + event_filter := khepri_evf:event_filter()}}, emitted_triggers = [] :: [khepri_machine:triggered()], metrics = #{} :: #{applied_command_count => non_neg_integer()}}). diff --git a/src/khepri_payload.erl b/src/khepri_payload.erl new file mode 100644 index 00000000..f797fa30 --- /dev/null +++ b/src/khepri_payload.erl @@ -0,0 +1,125 @@ +%% This Source Code Form is subject to the terms of the Mozilla Public +%% License, v. 2.0. If a copy of the MPL was not distributed with this +%% file, You can obtain one at https://mozilla.org/MPL/2.0/. +%% +%% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved. +%% + +%% @doc Khepri payloads. +%% +%% Payloads are the structure used to attach something to a tree node in the +%% store. Khepri supports the following payloads: +%%- If `ReadWrite' is `ro', `Fun' can do whatever it wants, except modify -%% the content of the store. In other words, uses of {@link khepri_tx:put/2} -%% or {@link khepri_tx:delete/1} are forbidden and will abort the function. -%% `Fun' is executed from a process on the leader Ra member.
-%%- If `ReadWrite' is `rw', `Fun' can use the {@link khepri_tx} transaction -%% API as well as any calls to other modules as long as those functions or what -%% they do is permitted. See {@link khepri_tx} for more details. If `Fun' does -%% or calls something forbidden, the transaction will be aborted. `Fun' is -%% executed in the context of the state machine process on each Ra -%% members.
-%%- If `ReadWrite' is `auto', `Fun' is analyzed to determine if it calls -%% {@link khepri_tx:put/2} or {@link khepri_tx:delete/1}, or uses any denied -%% operations for a read/write transaction. If it does, this is the same as -%% setting `ReadWrite' to true. Otherwise, this is the equivalent of setting -%% `ReadWrite' to false.
-%%+%%
+%% +%% Usually, there is no need to explicitly use this module as the type of +%% payload will be autodetected, thanks to the {@link wrap/1} function already +%% called internally. + +-module(khepri_payload). + +-include("src/khepri_fun.hrl"). +-include("src/internal.hrl"). + +-export([none/0, + data/1, + sproc/1, + wrap/1, + + prepare/1]). + +-type no_payload() :: ?NO_PAYLOAD. +%% Internal value used to mark that a tree node has no payload attached. + +-type data() :: #p_data{}. +%% Internal structure to wrap any Erlang term before it can be stored in a +%% tree node. +%% +%% The only constraint is the conversion to an Erlang binary must be supported +%% by this term. + +-type sproc() :: #p_sproc{}. +%% Internal structure to wrap an anonymous function before it can be stored in +%% a tree node and later executed. + +-type payload() :: no_payload() | data() | sproc(). +%% All types of payload stored in the nodes of the tree structure. +%% +%% Beside the absence of payload, the only type of payload supported is data. + +-export_type([no_payload/0, + data/0, + sproc/0, + payload/0]). + +-spec none() -> no_payload(). +%% @doc Returns the internal value used to mark that a tree node has no +%% payload attached. +%% +%% @see no_payload(). +none() -> + ?NO_PAYLOAD. + +-spec data(Term) -> Payload when + Term :: khepri:data(), + Payload :: data(). +%% @doc Returns the same term wrapped into an internal structure ready to be +%% stored in the tree. +%% +%% @see data(). + +data(Term) -> + #p_data{data = Term}. + +-spec sproc(Fun) -> Payload when + Fun :: khepri_fun:standalone_fun() | fun(), + Payload :: sproc(). +%% @doc Returns the same function wrapped into an internal structure ready to +%% be stored in the tree. +%% +%% @see sproc(). + +sproc(Fun) when is_function(Fun) -> + #p_sproc{sproc = Fun}; +sproc(#standalone_fun{} = Fun) -> + #p_sproc{sproc = Fun}. + +-spec wrap(Payload) -> WrappedPayload when + Payload :: payload() | khepri:data() | fun(), + WrappedPayload :: payload(). +%% @doc Automatically detects the payload type and ensures it is wrapped in +%% one of the internal types. +%% +%% The internal types make sure we avoid any collision between any +%% user-provided terms and internal structures. +%% +%% @param Payload an already wrapped payload, or any term which needs to be +%% wrapped. +%% +%% @returns the wrapped payload. + +wrap(Payload) when ?IS_KHEPRI_PAYLOAD(Payload) -> Payload; +wrap(Fun) when is_function(Fun) -> sproc(Fun); +wrap(Data) -> data(Data). + +-spec prepare(Payload) -> Payload when + Payload :: payload(). +%% @doc Finishes any needed changes to the payload before it is ready to be +%% stored. +%% +%% This currently only includes the conversion of anonymous functions to +%% standalone functions for stored procedures' payload records. +%% +%% @private + +prepare(?NO_PAYLOAD = Payload) -> + Payload; +prepare(#p_data{} = Payload) -> + Payload; +prepare(#p_sproc{sproc = Fun} = Payload) + when is_function(Fun) -> + StandaloneFun = khepri_sproc:to_standalone_fun(Fun), + Payload#p_sproc{sproc = StandaloneFun}. diff --git a/src/khepri_sproc.erl b/src/khepri_sproc.erl index 327606ed..0a07c8a2 100644 --- a/src/khepri_sproc.erl +++ b/src/khepri_sproc.erl @@ -6,12 +6,15 @@ %% %% @doc Khepri support code for stored procedures. +%% +%% @hidden -module(khepri_sproc). -include_lib("stdlib/include/assert.hrl"). -include("include/khepri.hrl"). +-include("src/khepri_fun.hrl"). -include("src/internal.hrl"). %% For internal user only. diff --git a/src/khepri_tx.erl b/src/khepri_tx.erl index fb1c1ecd..c87827e4 100644 --- a/src/khepri_tx.erl +++ b/src/khepri_tx.erl @@ -51,12 +51,22 @@ %% 2. If the function modifies the tree, it must be handled in %% `is_standalone_fun_still_needed()' is this file too. -export([put/2, put/3, + create/2, create/3, + update/2, update/3, + compare_and_swap/3, compare_and_swap/4, + + clear_payload/1, clear_payload/2, + delete/1, + + exists/1, exists/2, get/1, get/2, - exists/1, + get_node_props/1, get_node_props/2, has_data/1, - list/1, - find/2, - delete/1, + get_data/1, get_data/2, + + list/1, list/2, + find/2, find/3, + abort/1, is_transaction/0]). @@ -66,6 +76,16 @@ -compile({no_auto_import, [get/1, put/2, erase/1]}). +%% FIXME: Dialyzer complains about several functions with "optional" arguments +%% (but not all). I believe the specs are correct, but can't figure out how to +%% please Dialyzer. So for now, let's disable this specific check for the +%% problematic functions. +-if(?OTP_RELEASE >= 24). +-dialyzer({no_underspecs, [exists/1, + has_data/1, has_data/2, + get_data/1, get_data/2]}). +-endif. + -type tx_fun_result() :: any() | no_return(). -type tx_fun() :: fun(() -> tx_fun_result()). -type tx_fun_bindings() :: #{Name :: atom() => Value :: any()}. @@ -78,77 +98,136 @@ tx_fun_result/0, tx_abort/0]). --spec put(PathPattern, Payload) -> Result when - PathPattern :: khepri_path:pattern(), - Payload :: khepri_machine:payload(), - Result :: khepri_machine:result(). +%% ------------------------------------------------------------------- +%% Equivalent of `khepri' functions allowed in transactions. +%% ------------------------------------------------------------------- + +-spec put(PathPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | khepri:data(), + Result :: khepri:result(). %% @doc Creates or modifies a specific tree node in the tree structure. -put(PathPattern, Payload) -> - put(PathPattern, Payload, #{}). +put(PathPattern, Data) -> + put(PathPattern, Data, #{}). --spec put(PathPattern, Payload, Extra) -> Result when - PathPattern :: khepri_path:pattern(), - Payload :: khepri_machine:payload(), +-spec put(PathPattern, Data, Extra) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | khepri:data(), Extra :: #{keep_while => khepri_condition:keep_while()}, - Result :: khepri_machine:result(). + Result :: khepri:result(). %% @doc Creates or modifies a specific tree node in the tree structure. -put(PathPattern, Payload, Extra) when ?IS_KHEPRI_PAYLOAD(Payload) -> +put(PathPattern, Data, Extra) -> ensure_updates_are_allowed(), PathPattern1 = path_from_string(PathPattern), + Payload1 = khepri_payload:wrap(Data), {State, SideEffects} = get_tx_state(), Ret = khepri_machine:insert_or_update_node( - State, PathPattern1, Payload, Extra), + State, PathPattern1, Payload1, Extra), case Ret of {NewState, Result, NewSideEffects} -> set_tx_state(NewState, SideEffects ++ NewSideEffects); {NewState, Result} -> set_tx_state(NewState, SideEffects) end, - Result; -put(PathPattern, Payload, _Extra) -> - abort({invalid_payload, PathPattern, Payload}). + Result. -get(PathPattern) -> - get(PathPattern, #{}). +-spec create(PathPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | khepri:data(), + Result :: khepri:result(). +%% @doc Creates a specific tree node in the tree structure only if it does not +%% exist. -get(PathPattern, Options) -> +create(PathPattern, Data) -> + create(PathPattern, Data, #{}). + +-spec create(PathPattern, Data, Extra) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | khepri:data(), + Extra :: #{keep_while => khepri_condition:keep_while()}, + Result :: khepri:result(). +%% @doc Creates a specific tree node in the tree structure only if it does not +%% exist. + +create(PathPattern, Data, Extra) -> PathPattern1 = path_from_string(PathPattern), - {#khepri_machine{root = Root}, _SideEffects} = get_tx_state(), - khepri_machine:find_matching_nodes(Root, PathPattern1, Options). + PathPattern2 = khepri_path:combine_with_conditions( + PathPattern1, [#if_node_exists{exists = false}]), + put(PathPattern2, Data, Extra). + +-spec update(PathPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | khepri:data(), + Result :: khepri:result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists. + +update(PathPattern, Data) -> + update(PathPattern, Data, #{}). + +-spec update(PathPattern, Data, Extra) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri_payload:payload() | khepri:data(), + Extra :: #{keep_while => khepri_condition:keep_while()}, + Result :: khepri:result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists. --spec exists(Path) -> Exists when - Path :: khepri_path:pattern(), - Exists :: boolean(). +update(PathPattern, Data, Extra) -> + PathPattern1 = path_from_string(PathPattern), + PathPattern2 = khepri_path:combine_with_conditions( + PathPattern1, [#if_node_exists{exists = true}]), + put(PathPattern2, Data, Extra). + +-spec compare_and_swap(PathPattern, DataPattern, Data) -> Result when + PathPattern :: khepri_path:pattern() | string(), + DataPattern :: ets:match_pattern(), + Data :: khepri_payload:payload() | khepri:data(), + Result :: khepri:result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists and its data matches the given `DataPattern'. + +compare_and_swap(PathPattern, DataPattern, Data) -> + compare_and_swap(PathPattern, DataPattern, Data, #{}). + +-spec compare_and_swap(PathPattern, DataPattern, Data, Extra) -> Result when + PathPattern :: khepri_path:pattern() | string(), + DataPattern :: ets:match_pattern(), + Data :: khepri_payload:payload() | khepri:data(), + Extra :: #{keep_while => khepri_condition:keep_while()}, + Result :: khepri:result(). +%% @doc Updates a specific tree node in the tree structure only if it already +%% exists and its data matches the given `DataPattern'. -exists(Path) -> - case get(Path, #{expect_specific_node => true}) of - {ok, _} -> true; - _ -> false - end. +compare_and_swap(PathPattern, DataPattern, Data, Extra) -> + PathPattern1 = path_from_string(PathPattern), + PathPattern2 = khepri_path:combine_with_conditions( + PathPattern1, [#if_data_matches{pattern = DataPattern}]), + put(PathPattern2, Data, Extra). --spec has_data(Path) -> HasData when - Path :: khepri_path:pattern(), - HasData :: boolean(). +-spec clear_payload(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: khepri:result(). +%% @doc Clears the payload of a specific tree node in the tree structure. -has_data(Path) -> - case get(Path, #{expect_specific_node => true}) of - {ok, Result} -> - [NodeProps] = maps:values(Result), - maps:is_key(data, NodeProps); - _ -> - false - end. +clear_payload(PathPattern) -> + clear_payload(PathPattern, #{}). -list(Path) -> - Path1 = Path ++ [?STAR], - get(Path1). +-spec clear_payload(PathPattern, Extra) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Extra :: #{keep_while => khepri_condition:keep_while()}, + Result :: khepri:result(). +%% @doc Clears the payload of a specific tree node in the tree structure. -find(Path, Condition) -> - Condition1 = #if_all{conditions = [?STAR_STAR, Condition]}, - Path1 = Path ++ [Condition1], - get(Path1). +clear_payload(PathPattern, Extra) -> + put(PathPattern, khepri_payload:none(), Extra). + +-spec delete(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: khepri:result(). +%% @doc Deletes all tree nodes matching the path pattern. delete(PathPattern) -> ensure_updates_are_allowed(), @@ -163,13 +242,178 @@ delete(PathPattern) -> end, Result. +-spec exists(PathPattern) -> Exists when + PathPattern :: khepri_path:pattern() | string(), + Exists :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path exists, +%% otherwise `false'. + +exists(PathPattern) -> + exists(PathPattern, #{}). + +-spec exists(PathPattern, Options) -> Exists when + PathPattern :: khepri_path:pattern() | string(), + Options :: khepri:query_options(), + Exists :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path exists, +%% otherwise `false'. + +exists(PathPattern, Options) -> + Options1 = Options#{expect_specific_node => true}, + case get(PathPattern, Options1) of + {ok, _} -> true; + _ -> false + end. + +-spec get(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: khepri:result(). +%% @doc Returns all tree nodes matching the path pattern. + +get(PathPattern) -> + get(PathPattern, #{}). + +-spec get(PathPattern, Options) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Options :: khepri:query_options(), + Result :: khepri:result(). +%% @doc Returns all tree nodes matching the path pattern. + +get(PathPattern, Options) -> + PathPattern1 = path_from_string(PathPattern), + {#khepri_machine{root = Root}, _SideEffects} = get_tx_state(), + khepri_machine:find_matching_nodes(Root, PathPattern1, Options). + +-spec get_node_props(PathPattern) -> NodeProps when + PathPattern :: khepri_path:pattern() | string(), + NodeProps :: khepri:node_props(). +%% @doc Returns the tree node properties associated with the given node path. + +get_node_props(PathPattern) -> + get_node_props(PathPattern, #{}). + +-spec get_node_props(PathPattern, Options) -> NodeProps when + PathPattern :: khepri_path:pattern() | string(), + Options :: khepri:query_options(), + NodeProps :: khepri:node_props(). +%% @doc Returns the tree node properties associated with the given node path. + +get_node_props(PathPattern, Options) -> + Options1 = Options#{expect_specific_node => true}, + case get(PathPattern, Options1) of + {ok, Result} -> + [{_Path, NodeProps}] = maps:to_list(Result), + NodeProps; + Error -> + abort(Error) + end. + +-spec has_data(PathPattern) -> HasData when + PathPattern :: khepri_path:pattern() | string(), + HasData :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path has data, +%% otherwise `false'. + +has_data(PathPattern) -> + has_data(PathPattern, #{}). + +-spec has_data(PathPattern, Options) -> HasData when + PathPattern :: khepri_path:pattern() | string(), + Options :: khepri:query_options(), + HasData :: boolean(). +%% @doc Returns `true' if the tree node pointed to by the given path has data, +%% otherwise `false'. + +has_data(PathPattern, Options) -> + try + NodeProps = get_node_props(PathPattern, Options), + maps:is_key(data, NodeProps) + catch + throw:{aborted, _} -> + false + end. + +-spec get_data(PathPattern) -> Data when + PathPattern :: khepri_path:pattern() | string(), + Data :: khepri:data(). +%% @doc Returns the data associated with the given node path. + +get_data(PathPattern) -> + get_data(PathPattern, #{}). + +-spec get_data(PathPattern, Options) -> Data when + PathPattern :: khepri_path:pattern() | string(), + Options :: khepri:query_options(), + Data :: khepri:data(). +%% @doc Returns the data associated with the given node path. + +get_data(PathPattern, Options) -> + NodeProps = get_node_props(PathPattern, Options), + case NodeProps of + #{data := Data} -> Data; + _ -> abort({error, {no_data, NodeProps}}) + end. + +-spec list(PathPattern) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Result :: khepri:result(). +%% @doc Returns all direct child nodes under the given path. + +list(PathPattern) -> + list(PathPattern, #{}). + +-spec list(PathPattern, Options) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Options :: khepri:query_options(), + Result :: khepri:result(). +%% @doc Returns all direct child nodes under the given path. + +list(PathPattern, Options) -> + PathPattern1 = khepri_path:from_string(PathPattern), + PathPattern2 = [?ROOT_NODE | PathPattern1] ++ [?STAR], + get(PathPattern2, Options). + +-spec find(PathPattern, Condition) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Condition :: khepri_path:pattern_component(), + Result :: khepri:result(). +%% @doc Returns all tree nodes matching the path pattern. + +find(PathPattern, Condition) -> + find(PathPattern, Condition, #{}). + +-spec find(PathPattern, Condition, Options) -> Result when + PathPattern :: khepri_path:pattern() | string(), + Condition :: khepri_path:pattern_component(), + Options :: khepri:query_options(), + Result :: khepri:result(). +%% @doc Finds tree nodes under `PathPattern' which match the given `Condition'. + +find(PathPattern, Condition, Options) -> + Condition1 = #if_all{conditions = [?STAR_STAR, Condition]}, + PathPattern1 = khepri_path:from_string(PathPattern), + PathPattern2 = [?ROOT_NODE | PathPattern1] ++ [Condition1], + get(PathPattern2, Options). + -spec abort(Reason) -> no_return() when Reason :: any(). +%% @doc Aborts the transaction. +%% +%% Any changes so far are not committed to the store. +%% +%% {@link khepri:transaction/1} and friends will return `{atomic, Reason}'. +%% +%% @param Reason term to return to caller of the transaction. abort(Reason) -> throw({aborted, Reason}). -spec is_transaction() -> boolean(). +%% @doc Indicates if the calling function runs in the context of a transaction +%% function. +%% +%% @returns `true' if the calling code runs inside a transaction function, +%% `false' otherwise. is_transaction() -> StateAndSideEffects = erlang:get(?TX_STATE_KEY), @@ -178,10 +422,15 @@ is_transaction() -> _ -> false end. +%% ------------------------------------------------------------------- +%% Internal functions. +%% ------------------------------------------------------------------- + -spec to_standalone_fun(Fun, ReadWrite) -> StandaloneFun | no_return() when Fun :: fun(), ReadWrite :: ro | rw | auto, StandaloneFun :: khepri_fun:standalone_fun(). +%% @private to_standalone_fun(Fun, ReadWrite) when is_function(Fun, 0) andalso @@ -379,16 +628,22 @@ ensure_bif_is_valid(Bif, Arity) -> throw({call_denied, {Bif, Arity}}) end. -is_remote_call_valid(khepri, no_payload, 0) -> true; -is_remote_call_valid(khepri, data_payload, 1) -> true; +is_remote_call_valid(khepri_payload, no_payload, 0) -> true; +is_remote_call_valid(khepri_payload, data, 1) -> true; is_remote_call_valid(khepri_tx, put, _) -> true; -is_remote_call_valid(khepri_tx, get, _) -> true; +is_remote_call_valid(khepri_tx, create, _) -> true; +is_remote_call_valid(khepri_tx, update, _) -> true; +is_remote_call_valid(khepri_tx, compare_and_swap, _) -> true; +is_remote_call_valid(khepri_tx, clear_payload, _) -> true; +is_remote_call_valid(khepri_tx, delete, _) -> true; is_remote_call_valid(khepri_tx, exists, _) -> true; +is_remote_call_valid(khepri_tx, get, _) -> true; +is_remote_call_valid(khepri_tx, get_node_props, _) -> true; is_remote_call_valid(khepri_tx, has_data, _) -> true; +is_remote_call_valid(khepri_tx, get_data, _) -> true; is_remote_call_valid(khepri_tx, list, _) -> true; is_remote_call_valid(khepri_tx, find, _) -> true; -is_remote_call_valid(khepri_tx, delete, _) -> true; is_remote_call_valid(khepri_tx, abort, _) -> true; is_remote_call_valid(khepri_tx, is_transaction, _) -> true; @@ -534,10 +789,18 @@ is_standalone_fun_still_needed(_, rw) -> true; is_standalone_fun_still_needed(#{calls := Calls}, auto) -> ReadWrite = case Calls of - #{{khepri_tx, put, 2} := _} -> rw; - #{{khepri_tx, put, 3} := _} -> rw; - #{{khepri_tx, delete, 1} := _} -> rw; - _ -> ro + #{{khepri_tx, put, 2} := _} -> rw; + #{{khepri_tx, put, 3} := _} -> rw; + #{{khepri_tx, create, 2} := _} -> rw; + #{{khepri_tx, create, 3} := _} -> rw; + #{{khepri_tx, update, 2} := _} -> rw; + #{{khepri_tx, update, 3} := _} -> rw; + #{{khepri_tx, compare_and_swap, 3} := _} -> rw; + #{{khepri_tx, compare_and_swap, 4} := _} -> rw; + #{{khepri_tx, clear_payload, 1} := _} -> rw; + #{{khepri_tx, clear_payload, 2} := _} -> rw; + #{{khepri_tx, delete, 1} := _} -> rw; + _ -> ro end, ReadWrite =:= rw. @@ -609,6 +872,8 @@ get_tx_props() -> %% This is the same as calling {@link khepri_path:from_string/1} then {@link %% khepri_path:is_valid/1}, but the exception is caught to abort the %% transaction instead. +%% +%% @private path_from_string(PathPattern) -> try diff --git a/src/khepri_utils.erl b/src/khepri_utils.erl index ceb920eb..ada6da12 100644 --- a/src/khepri_utils.erl +++ b/src/khepri_utils.erl @@ -5,6 +5,8 @@ %% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved. %% +%% @hidden + -module(khepri_utils). -include_lib("stdlib/include/assert.hrl"). @@ -21,8 +23,8 @@ %% khepri:get_root/1 is unexported when compiled without `-DTEST'. -dialyzer(no_missing_calls). --spec flat_struct_to_tree(khepri_machine:node_props_map()) -> - khepri_machine:node_props(). +-spec flat_struct_to_tree(khepri:node_props_map()) -> + khepri:node_props(). flat_struct_to_tree(FlatStruct) -> NodeProps = maps:get([], FlatStruct, #{}), @@ -56,7 +58,7 @@ flat_struct_to_tree([ChildName], NodeProps, Tree) -> Tree#{ChildName => NodeProps} end. --spec display_tree(khepri_machine:node_props()) -> ok. +-spec display_tree(khepri:node_props()) -> ok. display_tree(Tree) -> display_tree(Tree, ""). diff --git a/test/async_option.erl b/test/async_option.erl index 98717f46..a56d9f46 100644 --- a/test/async_option.erl +++ b/test/async_option.erl @@ -21,12 +21,12 @@ async_unset_in_put_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put(?FUNCTION_NAME, [foo], none)), + khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), ?assertEqual( {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -38,13 +38,13 @@ async_false_in_put_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], none, #{async => false})), + khepri:put( + ?FUNCTION_NAME, [foo], ?NO_PAYLOAD, #{async => false})), ?assertEqual( {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -56,13 +56,13 @@ async_true_in_put_test_() -> begin ?assertEqual( ok, - khepri_machine:put( - ?FUNCTION_NAME, [foo], none, #{async => true})), + khepri:put( + ?FUNCTION_NAME, [foo], ?NO_PAYLOAD, #{async => true})), lists:foldl( fun (_, {ok, Result}) when Result =:= #{} -> timer:sleep(500), - khepri_machine:get(?FUNCTION_NAME, [foo]); + khepri:get(?FUNCTION_NAME, [foo]); (_, Ret) -> Ret end, {ok, #{}}, lists:seq(1, 60)), @@ -70,7 +70,7 @@ async_true_in_put_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -83,8 +83,9 @@ async_with_correlation_in_put_test_() -> Correlation = 1, ?assertEqual( ok, - khepri_machine:put( - ?FUNCTION_NAME, [foo], none, #{async => Correlation})), + khepri:put( + ?FUNCTION_NAME, [foo], ?NO_PAYLOAD, + #{async => Correlation})), Ret = receive {ra_event, _, {applied, [{Correlation, Reply}]}} -> Reply @@ -96,7 +97,7 @@ async_with_correlation_in_put_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -108,13 +109,13 @@ async_with_priority_in_put_test_() -> begin ?assertEqual( ok, - khepri_machine:put( - ?FUNCTION_NAME, [foo], none, #{async => low})), + khepri:put( + ?FUNCTION_NAME, [foo], ?NO_PAYLOAD, #{async => low})), lists:foldl( fun (_, {ok, Result}) when Result =:= #{} -> timer:sleep(500), - khepri_machine:get(?FUNCTION_NAME, [foo]); + khepri:get(?FUNCTION_NAME, [foo]); (_, Ret) -> Ret end, {ok, #{}}, lists:seq(1, 60)), @@ -122,7 +123,7 @@ async_with_priority_in_put_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -135,8 +136,8 @@ async_with_correlation_and_priority_in_put_test_() -> Correlation = 1, ?assertEqual( ok, - khepri_machine:put( - ?FUNCTION_NAME, [foo], none, + khepri:put( + ?FUNCTION_NAME, [foo], ?NO_PAYLOAD, #{async => {Correlation, low}})), Ret = receive {ra_event, _, {applied, [{Correlation, Reply}]}} -> @@ -149,7 +150,7 @@ async_with_correlation_and_priority_in_put_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -161,15 +162,15 @@ async_unset_in_delete_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put(?FUNCTION_NAME, [foo], none)), + khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), ?assertEqual( {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:delete(?FUNCTION_NAME, [foo])), + khepri:delete(?FUNCTION_NAME, [foo])), ?assertEqual( {ok, #{}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -181,16 +182,16 @@ async_false_in_delete_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put(?FUNCTION_NAME, [foo], none)), + khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), ?assertEqual( {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:delete( + khepri:delete( ?FUNCTION_NAME, [foo], #{async => false})), ?assertEqual( {ok, #{}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -202,16 +203,16 @@ async_true_in_delete_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put(?FUNCTION_NAME, [foo], none)), + khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), ?assertEqual( ok, - khepri_machine:delete( + khepri:delete( ?FUNCTION_NAME, [foo], #{async => true})), lists:foldl( fun (_, {ok, Result}) when Result =/= #{} -> timer:sleep(500), - khepri_machine:get(?FUNCTION_NAME, [foo]); + khepri:get(?FUNCTION_NAME, [foo]); (_, Ret) -> Ret end, @@ -221,7 +222,7 @@ async_true_in_delete_test_() -> lists:seq(1, 60)), ?assertEqual( {ok, #{}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -233,11 +234,11 @@ async_with_correlation_in_delete_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put(?FUNCTION_NAME, [foo], none)), + khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), Correlation = 1, ?assertEqual( ok, - khepri_machine:delete( + khepri:delete( ?FUNCTION_NAME, [foo], #{async => Correlation})), Ret = receive {ra_event, _, {applied, [{Correlation, Reply}]}} -> @@ -250,7 +251,7 @@ async_with_correlation_in_delete_test_() -> Ret), ?assertEqual( {ok, #{}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -262,16 +263,16 @@ async_with_priority_in_delete_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put(?FUNCTION_NAME, [foo], none)), + khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), ?assertEqual( ok, - khepri_machine:delete( + khepri:delete( ?FUNCTION_NAME, [foo], #{async => low})), lists:foldl( fun (_, {ok, Result}) when Result =/= #{} -> timer:sleep(500), - khepri_machine:get(?FUNCTION_NAME, [foo]); + khepri:get(?FUNCTION_NAME, [foo]); (_, Ret) -> Ret end, @@ -281,7 +282,7 @@ async_with_priority_in_delete_test_() -> lists:seq(1, 60)), ?assertEqual( {ok, #{}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -293,11 +294,11 @@ async_with_correlation_and_priority_in_delete_test_() -> begin ?assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put(?FUNCTION_NAME, [foo], none)), + khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), Correlation = 1, ?assertEqual( ok, - khepri_machine:delete( + khepri:delete( ?FUNCTION_NAME, [foo], #{async => {Correlation, low}})), Ret = receive {ra_event, _, {applied, [{Correlation, Reply}]}} -> @@ -310,7 +311,7 @@ async_with_correlation_and_priority_in_delete_test_() -> Ret), ?assertEqual( {ok, #{}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -320,15 +321,15 @@ async_unset_in_transaction_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_test( begin - Fun = fun() -> khepri_tx:put([foo], none) end, + Fun = fun() -> khepri_tx:put([foo], ?NO_PAYLOAD) end, ?assertEqual( {atomic, {ok, #{[foo] => #{}}}}, - khepri_machine:transaction(?FUNCTION_NAME, Fun)), + khepri:transaction(?FUNCTION_NAME, Fun)), ?assertEqual( {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -338,16 +339,16 @@ async_false_in_transaction_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_test( begin - Fun = fun() -> khepri_tx:put([foo], none) end, + Fun = fun() -> khepri_tx:put([foo], ?NO_PAYLOAD) end, ?assertEqual( {atomic, {ok, #{[foo] => #{}}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{async => false})), ?assertEqual( {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -357,16 +358,16 @@ async_true_in_transaction_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_test( begin - Fun = fun() -> khepri_tx:put([foo], none) end, + Fun = fun() -> khepri_tx:put([foo], ?NO_PAYLOAD) end, ?assertEqual( ok, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{async => true})), lists:foldl( fun (_, {ok, Result}) when Result =:= #{} -> timer:sleep(500), - khepri_machine:get(?FUNCTION_NAME, [foo]); + khepri:get(?FUNCTION_NAME, [foo]); (_, Ret) -> Ret end, {ok, #{}}, lists:seq(1, 60)), @@ -374,7 +375,7 @@ async_true_in_transaction_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -384,11 +385,11 @@ async_with_correlation_in_transaction_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_test( begin - Fun = fun() -> khepri_tx:put([foo], none) end, + Fun = fun() -> khepri_tx:put([foo], ?NO_PAYLOAD) end, Correlation = 1, ?assertEqual( ok, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{async => Correlation})), Ret = receive {ra_event, _, {applied, [{Correlation, Reply}]}} -> @@ -401,7 +402,7 @@ async_with_correlation_in_transaction_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -411,16 +412,16 @@ async_with_priority_in_transaction_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_test( begin - Fun = fun() -> khepri_tx:put([foo], none) end, + Fun = fun() -> khepri_tx:put([foo], ?NO_PAYLOAD) end, ?assertEqual( ok, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{async => low})), lists:foldl( fun (_, {ok, Result}) when Result =:= #{} -> timer:sleep(500), - khepri_machine:get(?FUNCTION_NAME, [foo]); + khepri:get(?FUNCTION_NAME, [foo]); (_, Ret) -> Ret end, {ok, #{}}, lists:seq(1, 60)), @@ -428,7 +429,7 @@ async_with_priority_in_transaction_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -438,11 +439,11 @@ async_with_correlation_and_priority_in_transaction_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_test( begin - Fun = fun() -> khepri_tx:put([foo], none) end, + Fun = fun() -> khepri_tx:put([foo], ?NO_PAYLOAD) end, Correlation = 1, ?assertEqual( ok, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{async => {Correlation, low}})), Ret = receive @@ -456,6 +457,6 @@ async_with_correlation_and_priority_in_transaction_test_() -> {ok, #{[foo] => #{payload_version => 1, child_list_version => 1, child_list_length => 0}}}, - khepri_machine:get(?FUNCTION_NAME, [foo])) + khepri:get(?FUNCTION_NAME, [foo])) end) ]}. diff --git a/test/conditions.erl b/test/conditions.erl index 620233ca..c8963629 100644 --- a/test/conditions.erl +++ b/test/conditions.erl @@ -180,11 +180,11 @@ if_has_data_matching_test() -> {false, #if_has_data{has_data = false}}, khepri_condition:is_met( khepri_condition:compile(#if_has_data{has_data = false}), - foo, #node{payload = #kpayload_data{data = foo}})), + foo, #node{payload = khepri_payload:data(foo)})), ?assert( khepri_condition:is_met( khepri_condition:compile(#if_has_data{has_data = true}), - foo, #node{payload = #kpayload_data{data = foo}})). + foo, #node{payload = khepri_payload:data(foo)})). if_data_matches_matching_test() -> CompiledCond1 = khepri_condition:compile( @@ -195,7 +195,7 @@ if_data_matches_matching_test() -> CompiledCond1, foo, #node{})), ?assert( khepri_condition:is_met( - CompiledCond1, foo, #node{payload = #kpayload_data{data = {a, b}}})), + CompiledCond1, foo, #node{payload = khepri_payload:data({a, b})})), CompiledCond2 = khepri_condition:compile( #if_data_matches{pattern = {a, '_'}}), @@ -213,18 +213,18 @@ if_data_matches_matching_test() -> ?assert( khepri_condition:is_met( - CompiledCond2, foo, #node{payload = #kpayload_data{data = {a, b}}})), + CompiledCond2, foo, #node{payload = khepri_payload:data({a, b})})), ?assert( khepri_condition:is_met( - CompiledCond2, foo, #node{payload = #kpayload_data{data = {a, c}}})), + CompiledCond2, foo, #node{payload = khepri_payload:data({a, c})})), ?assertEqual( {false, CompiledCond2}, khepri_condition:is_met( - CompiledCond2, foo, #node{payload = #kpayload_data{data = {b, c}}})), + CompiledCond2, foo, #node{payload = khepri_payload:data({b, c})})), ?assertEqual( {false, CompiledCond2}, khepri_condition:is_met( - CompiledCond2, foo, #node{payload = #kpayload_data{data = other}})). + CompiledCond2, foo, #node{payload = khepri_payload:data(other)})). if_payload_version_matching_test() -> ?assert( diff --git a/test/db_info.erl b/test/db_info.erl index f1c509ff..b72d715d 100644 --- a/test/db_info.erl +++ b/test/db_info.erl @@ -43,10 +43,10 @@ get_store_ids_with_running_store_test_() -> % fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, % fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, % [?_assertEqual( -% ok, +% {ok, #{}}, % khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), % ?_assertEqual( -% ok, +% {ok, #{}}, % khepri:create(?FUNCTION_NAME, [baz], baz_value)), % ?_assertEqual( % begin @@ -59,10 +59,10 @@ get_store_info_on_non_existing_store_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( "\n" @@ -78,10 +78,10 @@ get_store_info_on_running_store_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( "\n" @@ -111,8 +111,8 @@ get_store_info_with_keep_while_conds_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = foo_value}, + khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(foo_value), #{keep_while => KeepWhile})), ?_assertEqual( "\n" diff --git a/test/delete_command.erl b/test/delete_command.erl index 46e5c6c8..75ef901c 100644 --- a/test/delete_command.erl +++ b/test/delete_command.erl @@ -39,7 +39,7 @@ delete_non_existing_node_under_non_existing_parent_test() -> delete_existing_node_with_data_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [foo]}, @@ -61,7 +61,7 @@ delete_existing_node_with_data_test() -> delete_existing_node_with_data_using_dot_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [foo, ?THIS_NODE]}, @@ -83,7 +83,7 @@ delete_existing_node_with_data_using_dot_test() -> delete_existing_node_with_child_nodes_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [foo]}, @@ -104,7 +104,7 @@ delete_existing_node_with_child_nodes_test() -> delete_a_node_deep_into_the_tree_test() -> Commands = [#put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [foo, bar, baz]}, @@ -134,9 +134,9 @@ delete_a_node_deep_into_the_tree_test() -> delete_existing_node_with_condition_true_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [#if_data_matches{pattern = bar_value}]}, @@ -151,7 +151,7 @@ delete_existing_node_with_condition_true_test() -> child_nodes = #{foo => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}}}, + payload = khepri_payload:data(foo_value)}}}, Root), ?assertEqual({ok, #{[bar] => #{data => bar_value, payload_version => 1, @@ -161,9 +161,9 @@ delete_existing_node_with_condition_true_test() -> delete_existing_node_with_condition_false_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [#if_data_matches{pattern = other_value}]}, @@ -178,19 +178,19 @@ delete_existing_node_with_condition_false_test() -> child_nodes = #{foo => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, bar => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = bar_value}}}}, + payload = khepri_payload:data(bar_value)}}}, Root), ?assertEqual({ok, #{}}, Ret), ?assertEqual([], SE). delete_existing_node_with_condition_true_using_dot_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = @@ -209,7 +209,7 @@ delete_existing_node_with_condition_true_using_dot_test() -> child_nodes = #{foo => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}}}, + payload = khepri_payload:data(foo_value)}}}, Root), ?assertEqual({ok, #{[bar] => #{data => bar_value, payload_version => 1, @@ -219,9 +219,9 @@ delete_existing_node_with_condition_true_using_dot_test() -> delete_existing_node_with_condition_false_using_dot_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = @@ -240,21 +240,21 @@ delete_existing_node_with_condition_false_using_dot_test() -> child_nodes = #{foo => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, bar => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = bar_value}}}}, + payload = khepri_payload:data(bar_value)}}}, Root), ?assertEqual({ok, #{}}, Ret), ?assertEqual([], SE). delete_many_nodes_at_once_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}], + payload = khepri_payload:data(baz_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [#if_name_matches{regex = "a"}]}, @@ -269,7 +269,7 @@ delete_many_nodes_at_once_test() -> child_nodes = #{foo => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}}}, + payload = khepri_payload:data(foo_value)}}}, Root), ?assertEqual({ok, #{[bar] => #{data => bar_value, payload_version => 1, diff --git a/test/display_tree.erl b/test/display_tree.erl index 3beadfca..53a1f22b 100644 --- a/test/display_tree.erl +++ b/test/display_tree.erl @@ -18,13 +18,13 @@ complex_flat_struct_to_tree_test() -> Commands = [#put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = qux_value}}, + payload = khepri_payload:data(qux_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{data = pouet_value}}], + payload = khepri_payload:data(pouet_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -358,7 +358,7 @@ flat_struct_with_children_before_parents_test() -> display_simple_tree_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -378,27 +378,26 @@ display_simple_tree_test() -> display_large_tree_test() -> Commands = [#put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = qux_value}}, + payload = khepri_payload:data(qux_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{ - data = - [lorem, ipsum, dolor, sit, amet, - consectetur, adipiscing, elit, sed, do, - eiusmod, tempor, incididunt, ut, labore, - et, dolore, magna, aliqua, ut, enim, ad, - minim, veniam, quis, nostrud, exercitation, - ullamco, laboris, nisi, ut, aliquip, ex, - ea, commodo, consequat, duis, aute, irure, - dolor, in, reprehenderit, in, voluptate, - velit, esse, cillum, dolore, eu, fugiat, - nulla, pariatur, excepteur, sint, occaecat, - cupidatat, non, proident, sunt, in, culpa, - qui, officia, deserunt, mollit, anim, id, - est, laborum]}}], + payload = khepri_payload:data( + [lorem, ipsum, dolor, sit, amet, + consectetur, adipiscing, elit, sed, do, + eiusmod, tempor, incididunt, ut, labore, + et, dolore, magna, aliqua, ut, enim, ad, + minim, veniam, quis, nostrud, exercitation, + ullamco, laboris, nisi, ut, aliquip, ex, + ea, commodo, consequat, duis, aute, irure, + dolor, in, reprehenderit, in, voluptate, + velit, esse, cillum, dolore, eu, fugiat, + nulla, pariatur, excepteur, sint, occaecat, + cupidatat, non, proident, sunt, in, culpa, + qui, officia, deserunt, mollit, anim, id, + est, laborum])}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -436,27 +435,26 @@ display_large_tree_test() -> display_tree_with_plaintext_lines_test() -> Commands = [#put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = qux_value}}, + payload = khepri_payload:data(qux_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{ - data = - [lorem, ipsum, dolor, sit, amet, - consectetur, adipiscing, elit, sed, do, - eiusmod, tempor, incididunt, ut, labore, - et, dolore, magna, aliqua, ut, enim, ad, - minim, veniam, quis, nostrud, exercitation, - ullamco, laboris, nisi, ut, aliquip, ex, - ea, commodo, consequat, duis, aute, irure, - dolor, in, reprehenderit, in, voluptate, - velit, esse, cillum, dolore, eu, fugiat, - nulla, pariatur, excepteur, sint, occaecat, - cupidatat, non, proident, sunt, in, culpa, - qui, officia, deserunt, mollit, anim, id, - est, laborum]}}], + payload = khepri_payload:data( + [lorem, ipsum, dolor, sit, amet, + consectetur, adipiscing, elit, sed, do, + eiusmod, tempor, incididunt, ut, labore, + et, dolore, magna, aliqua, ut, enim, ad, + minim, veniam, quis, nostrud, exercitation, + ullamco, laboris, nisi, ut, aliquip, ex, + ea, commodo, consequat, duis, aute, irure, + dolor, in, reprehenderit, in, voluptate, + velit, esse, cillum, dolore, eu, fugiat, + nulla, pariatur, excepteur, sint, occaecat, + cupidatat, non, proident, sunt, in, culpa, + qui, officia, deserunt, mollit, anim, id, + est, laborum])}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -494,27 +492,26 @@ display_tree_with_plaintext_lines_test() -> display_tree_without_colors_test() -> Commands = [#put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = qux_value}}, + payload = khepri_payload:data(qux_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{ - data = - [lorem, ipsum, dolor, sit, amet, - consectetur, adipiscing, elit, sed, do, - eiusmod, tempor, incididunt, ut, labore, - et, dolore, magna, aliqua, ut, enim, ad, - minim, veniam, quis, nostrud, exercitation, - ullamco, laboris, nisi, ut, aliquip, ex, - ea, commodo, consequat, duis, aute, irure, - dolor, in, reprehenderit, in, voluptate, - velit, esse, cillum, dolore, eu, fugiat, - nulla, pariatur, excepteur, sint, occaecat, - cupidatat, non, proident, sunt, in, culpa, - qui, officia, deserunt, mollit, anim, id, - est, laborum]}}], + payload = khepri_payload:data( + [lorem, ipsum, dolor, sit, amet, + consectetur, adipiscing, elit, sed, do, + eiusmod, tempor, incididunt, ut, labore, + et, dolore, magna, aliqua, ut, enim, ad, + minim, veniam, quis, nostrud, exercitation, + ullamco, laboris, nisi, ut, aliquip, ex, + ea, commodo, consequat, duis, aute, irure, + dolor, in, reprehenderit, in, voluptate, + velit, esse, cillum, dolore, eu, fugiat, + nulla, pariatur, excepteur, sint, occaecat, + cupidatat, non, proident, sunt, in, culpa, + qui, officia, deserunt, mollit, anim, id, + est, laborum])}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -552,27 +549,26 @@ display_tree_without_colors_test() -> display_tree_with_plaintext_lines_and_without_colors_test() -> Commands = [#put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = qux_value}}, + payload = khepri_payload:data(qux_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{ - data = - [lorem, ipsum, dolor, sit, amet, - consectetur, adipiscing, elit, sed, do, - eiusmod, tempor, incididunt, ut, labore, - et, dolore, magna, aliqua, ut, enim, ad, - minim, veniam, quis, nostrud, exercitation, - ullamco, laboris, nisi, ut, aliquip, ex, - ea, commodo, consequat, duis, aute, irure, - dolor, in, reprehenderit, in, voluptate, - velit, esse, cillum, dolore, eu, fugiat, - nulla, pariatur, excepteur, sint, occaecat, - cupidatat, non, proident, sunt, in, culpa, - qui, officia, deserunt, mollit, anim, id, - est, laborum]}}], + payload = khepri_payload:data( + [lorem, ipsum, dolor, sit, amet, + consectetur, adipiscing, elit, sed, do, + eiusmod, tempor, incididunt, ut, labore, + et, dolore, magna, aliqua, ut, enim, ad, + minim, veniam, quis, nostrud, exercitation, + ullamco, laboris, nisi, ut, aliquip, ex, + ea, commodo, consequat, duis, aute, irure, + dolor, in, reprehenderit, in, voluptate, + velit, esse, cillum, dolore, eu, fugiat, + nulla, pariatur, excepteur, sint, occaecat, + cupidatat, non, proident, sunt, in, culpa, + qui, officia, deserunt, mollit, anim, id, + est, laborum])}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -611,9 +607,9 @@ display_tree_with_plaintext_lines_and_without_colors_test() -> display_tree_with_binary_key_test() -> Commands = [#put{path = [<<"foo">>], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -636,9 +632,9 @@ display_tree_with_binary_key_test() -> display_tree_with_similar_atom_and_binary_keys_test() -> Commands = [#put{path = [<<"foo">>], - payload = #kpayload_data{data = foo_binary}}, + payload = khepri_payload:data(foo_binary)}, #put{path = [foo], - payload = #kpayload_data{data = foo_atom}}], + payload = khepri_payload:data(foo_atom)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), diff --git a/test/favor_option.erl b/test/favor_option.erl index 4b8da4ad..a0256683 100644 --- a/test/favor_option.erl +++ b/test/favor_option.erl @@ -29,7 +29,7 @@ favor_compromise_in_get_test_() -> ?assertEqual( {ok, #{}}, - khepri_machine:get( + khepri:get( ?FUNCTION_NAME, [foo], #{favor => compromise})), ?assertEqual( @@ -44,7 +44,7 @@ favor_compromise_in_get_test_() -> ?assertEqual( {ok, #{}}, - khepri_machine:get( + khepri:get( ?FUNCTION_NAME, [foo], #{favor => compromise})), ?assertEqual( @@ -57,7 +57,7 @@ favor_compromise_in_get_test_() -> ?assertEqual( {ok, #{}}, - khepri_machine:get( + khepri:get( ?FUNCTION_NAME, [foo], #{favor => compromise})), ?assertEqual( @@ -86,7 +86,7 @@ favor_consistency_in_get_test_() -> ?assertEqual( {ok, #{}}, - khepri_machine:get( + khepri:get( ?FUNCTION_NAME, [foo], #{favor => consistency})), ?assertEqual( @@ -101,7 +101,7 @@ favor_consistency_in_get_test_() -> ?assertEqual( {ok, #{}}, - khepri_machine:get( + khepri:get( ?FUNCTION_NAME, [foo], #{favor => consistency})), ?assertEqual( @@ -130,7 +130,7 @@ favor_low_latency_in_get_test_() -> ?assertEqual( {ok, #{}}, - khepri_machine:get( + khepri:get( ?FUNCTION_NAME, [foo], #{favor => low_latency})), ?assertEqual( @@ -143,7 +143,7 @@ favor_low_latency_in_get_test_() -> ?assertEqual( {ok, #{}}, - khepri_machine:get( + khepri:get( ?FUNCTION_NAME, [foo], #{favor => low_latency})), ?assertEqual( @@ -176,7 +176,7 @@ favor_compromise_in_transaction_test_() -> ?assertEqual( {atomic, {ok, #{}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{favor => compromise})), ?assertEqual( @@ -191,7 +191,7 @@ favor_compromise_in_transaction_test_() -> ?assertEqual( {atomic, {ok, #{}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{favor => compromise})), ?assertEqual( @@ -204,7 +204,7 @@ favor_compromise_in_transaction_test_() -> ?assertEqual( {atomic, {ok, #{}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{favor => compromise})), ?assertEqual( @@ -235,7 +235,7 @@ favor_consistency_in_transaction_test_() -> ?assertEqual( {atomic, {ok, #{}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{favor => consistency})), ?assertEqual( @@ -250,7 +250,7 @@ favor_consistency_in_transaction_test_() -> ?assertEqual( {atomic, {ok, #{}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{favor => consistency})), ?assertEqual( @@ -281,7 +281,7 @@ favor_low_latency_in_transaction_test_() -> ?assertEqual( {atomic, {ok, #{}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{favor => low_latency})), ?assertEqual( @@ -294,7 +294,7 @@ favor_low_latency_in_transaction_test_() -> ?assertEqual( {atomic, {ok, #{}}}, - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, Fun, #{favor => low_latency})), ?assertEqual( diff --git a/test/keep_while_conditions.erl b/test/keep_while_conditions.erl index 00b87c49..f4639224 100644 --- a/test/keep_while_conditions.erl +++ b/test/keep_while_conditions.erl @@ -23,7 +23,7 @@ are_keep_while_conditions_met_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -61,13 +61,13 @@ are_keep_while_conditions_met_test() -> insert_when_keep_while_true_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), KeepWhile = #{[foo] => #if_node_exists{exists = true}}, Command = #put{path = [baz], - payload = #kpayload_data{data = baz_value}, + payload = khepri_payload:data(baz_value), extra = #{keep_while => KeepWhile}}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -83,11 +83,11 @@ insert_when_keep_while_true_test() -> #{foo => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, baz => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = baz_value}}}}, + payload = khepri_payload:data(baz_value)}}}, Root), ?assertEqual( #{[baz] => KeepWhile}, @@ -100,14 +100,14 @@ insert_when_keep_while_true_test() -> insert_when_keep_while_false_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), %% The targeted keep_while node does not exist. KeepWhile1 = #{[foo, bar] => #if_node_exists{exists = true}}, Command1 = #put{path = [baz], - payload = #kpayload_data{data = baz_value}, + payload = khepri_payload:data(baz_value), extra = #{keep_while => KeepWhile1}}, {S1, Ret1, SE1} = khepri_machine:apply(?META, Command1, S0), @@ -125,7 +125,7 @@ insert_when_keep_while_false_test() -> %% The targeted keep_while node exists but the condition is not verified. KeepWhile2 = #{[foo] => #if_child_list_length{count = 10}}, Command2 = #put{path = [baz], - payload = #kpayload_data{data = baz_value}, + payload = khepri_payload:data(baz_value), extra = #{keep_while => KeepWhile2}}, {S2, Ret2, SE2} = khepri_machine:apply(?META, Command2, S0), @@ -145,7 +145,7 @@ insert_when_keep_while_true_on_self_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), KeepWhile = #{[?THIS_NODE] => #if_child_list_length{count = 0}}, Command = #put{path = [foo], - payload = #kpayload_data{data = foo_value}, + payload = khepri_payload:data(foo_value), extra = #{keep_while => KeepWhile}}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -159,7 +159,7 @@ insert_when_keep_while_true_on_self_test() -> #{foo => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}}}, + payload = khepri_payload:data(foo_value)}}}, Root), ?assertEqual({ok, #{[foo] => #{}}}, Ret), ?assertEqual([], SE). @@ -168,7 +168,7 @@ insert_when_keep_while_false_on_self_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), KeepWhile = #{[?THIS_NODE] => #if_child_list_length{count = 1}}, Command = #put{path = [foo], - payload = #kpayload_data{data = foo_value}, + payload = khepri_payload:data(foo_value), extra = #{keep_while => KeepWhile}}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -182,7 +182,7 @@ insert_when_keep_while_false_on_self_test() -> #{foo => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}}}, + payload = khepri_payload:data(foo_value)}}}, Root), ?assertEqual({ok, #{[foo] => #{}}}, Ret), ?assertEqual([], SE). @@ -190,15 +190,15 @@ insert_when_keep_while_false_on_self_test() -> keep_while_still_true_after_command_test() -> KeepWhile = #{[foo] => #if_child_list_length{count = 0}}, Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}, + payload = khepri_payload:data(baz_value), extra = #{keep_while => KeepWhile}}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #put{path = [foo], - payload = #kpayload_data{data = new_foo_value}}, + payload = khepri_payload:data(new_foo_value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -212,11 +212,11 @@ keep_while_still_true_after_command_test() -> #node{ stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = new_foo_value}}, + payload = khepri_payload:data(new_foo_value)}, baz => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = baz_value}}}}, + payload = khepri_payload:data(baz_value)}}}, Root), ?assertEqual({ok, #{[foo] => #{data => foo_value, payload_version => 1, @@ -227,15 +227,15 @@ keep_while_still_true_after_command_test() -> keep_while_now_false_after_command_test() -> KeepWhile = #{[foo] => #if_child_list_length{count = 0}}, Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}, + payload = khepri_payload:data(baz_value), extra = #{keep_while => KeepWhile}}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -249,11 +249,11 @@ keep_while_now_false_after_command_test() -> #node{ stat = #{payload_version => 1, child_list_version => 2}, - payload = #kpayload_data{data = foo_value}, + payload = khepri_payload:data(foo_value), child_nodes = #{bar => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = bar_value}}}}}}, + payload = khepri_payload:data(bar_value)}}}}}, Root), ?assertEqual({ok, #{[foo, bar] => #{}}}, Ret), ?assertEqual([], SE). @@ -261,13 +261,13 @@ keep_while_now_false_after_command_test() -> recursive_automatic_cleanup_test() -> KeepWhile = #{[?THIS_NODE] => #if_child_list_length{count = {gt, 0}}}, Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}, + payload = khepri_payload:data(foo_value), extra = #{keep_while => KeepWhile}}, #put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}, + payload = khepri_payload:data(bar_value), extra = #{keep_while => KeepWhile}}, #put{path = [foo, bar, baz], - payload = #kpayload_data{data = baz_value}}], + payload = khepri_payload:data(baz_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -291,9 +291,9 @@ recursive_automatic_cleanup_test() -> keep_while_now_false_after_delete_command_test() -> KeepWhile = #{[foo] => #if_node_exists{exists = true}}, Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}, + payload = khepri_payload:data(baz_value), extra = #{keep_while => KeepWhile}}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), diff --git a/test/machine_code_called_from_ra.erl b/test/machine_code_called_from_ra.erl index 00c9baf9..51718eba 100644 --- a/test/machine_code_called_from_ra.erl +++ b/test/machine_code_called_from_ra.erl @@ -23,8 +23,8 @@ insert_a_node_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( {ok, #{[foo] => #{}}}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = foo_value}))]}. + khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(foo_value)))]}. query_a_node_test_() -> {setup, @@ -36,9 +36,9 @@ query_a_node_test_() -> child_list_version => 1, child_list_length => 0}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = foo_value}), - khepri_machine:get(?FUNCTION_NAME, [foo]) + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(foo_value)), + khepri:get(?FUNCTION_NAME, [foo]) end)]}. delete_a_node_test_() -> @@ -53,15 +53,15 @@ delete_a_node_test_() -> child_list_version => 1, child_list_length => 0}}}, begin - _ = khepri_machine:put( + _ = khepri:put( ?FUNCTION_NAME, [foo], - #kpayload_data{data = foo_value}), - khepri_machine:delete(?FUNCTION_NAME, [foo]) + khepri_payload:data(foo_value)), + khepri:delete(?FUNCTION_NAME, [foo]) end)}, {"Checking the deleted key is gone", ?_assertEqual( {ok, #{}}, - khepri_machine:get(?FUNCTION_NAME, [foo]))}]} + khepri:get(?FUNCTION_NAME, [foo]))}]} ]}. query_keep_while_conds_state_test_() -> @@ -73,10 +73,10 @@ query_keep_while_conds_state_test_() -> {ok, #{[foo] => #{[foo] => #if_child_list_length{count = {gt, 0}}}}}, begin - _ = khepri_machine:put( + _ = khepri:put( ?FUNCTION_NAME, [foo], - #kpayload_data{data = foo_value}, + khepri_payload:data(foo_value), #{keep_while => KeepWhile}), khepri_machine:get_keep_while_conds_state(?FUNCTION_NAME) end)]}. @@ -87,17 +87,17 @@ use_an_invalid_path_test_() -> fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertThrow( {invalid_path, #{path := not_a_list}}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, not_a_list, - none)), + ?NO_PAYLOAD)), ?_assertThrow( {invalid_path, #{path := ["not_a_component"], tail := ["not_a_component"]}}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, ["not_a_component"], - none))]}. + ?NO_PAYLOAD))]}. use_an_invalid_payload_test_() -> {setup, @@ -108,10 +108,12 @@ use_an_invalid_payload_test_() -> khepri_machine:put( ?FUNCTION_NAME, [foo], - invalid_payload)), + invalid_payload, + #{}, #{})), ?_assertThrow( {invalid_payload, [foo], {invalid_payload, in_a_tuple}}, khepri_machine:put( ?FUNCTION_NAME, [foo], - {invalid_payload, in_a_tuple}))]}. + {invalid_payload, in_a_tuple}, + #{}, #{}))]}. diff --git a/test/prop_state_machine.erl b/test/prop_state_machine.erl index c8637881..d7427af7 100644 --- a/test/prop_state_machine.erl +++ b/test/prop_state_machine.erl @@ -12,6 +12,7 @@ -include_lib("proper/include/proper.hrl"). -include("include/khepri.hrl"). +-include("src/internal.hrl"). -dialyzer([{[no_opaque, no_return], [prop_commands_with_simple_paths_work_in_any_order/0]}]). @@ -50,9 +51,9 @@ initial_state() -> #state{}. command(_State) -> - elements([{call, khepri_machine, put, [?STORE_ID, path(), payload()]}, - {call, khepri_machine, get, [?STORE_ID, path()]}, - {call, khepri_machine, delete, [?STORE_ID, path()]}]). + elements([{call, khepri, put, [?STORE_ID, path(), payload()]}, + {call, khepri, get, [?STORE_ID, path()]}, + {call, khepri, delete, [?STORE_ID, path()]}]). precondition(_State, _Command) -> true. @@ -60,36 +61,36 @@ precondition(_State, _Command) -> next_state( #state{} = State, _Result, - {call, khepri_machine, get, [_StoreId, _Path]}) -> + {call, khepri, get, [_StoreId, _Path]}) -> State; next_state( #state{entries = Entries} = State, _Result, - {call, khepri_machine, put, [_StoreId, Path, Payload]}) -> + {call, khepri, put, [_StoreId, Path, Payload]}) -> Entries1 = add_entry(Entries, Path, Payload), State#state{entries = Entries1, old_entries = Entries}; next_state( #state{entries = Entries} = State, _Result, - {call, khepri_machine, delete, [_StoreId, Path]}) -> + {call, khepri, delete, [_StoreId, Path]}) -> Entries1 = delete_entry(Entries, Path), State#state{entries = Entries1, old_entries = Entries}. postcondition( #state{entries = Entries}, - {call, khepri_machine, get, [_StoreId, Path]}, + {call, khepri, get, [_StoreId, Path]}, Result) -> result_is_ok(Result, Entries, Path, {ok, #{}}); postcondition( #state{entries = Entries}, - {call, khepri_machine, put, [_StoreId, Path, _Payload]}, + {call, khepri, put, [_StoreId, Path, _Payload]}, Result) -> result_is_ok(Result, Entries, Path, {ok, #{Path => #{}}}); postcondition( #state{entries = Entries}, - {call, khepri_machine, delete, [_StoreId, Path]}, + {call, khepri, delete, [_StoreId, Path]}, Result) -> result_is_ok(Result, Entries, Path, {ok, #{}}). @@ -110,19 +111,19 @@ add_entry(Entries, Path, Payload) -> {Entry0, true} end, Entry2 = case Payload of - #kpayload_data{data = Data} -> Entry1#{data => Data}; - none -> maps:remove(data, Entry1) + #p_data{data = Data} -> Entry1#{data => Data}; + ?NO_PAYLOAD -> maps:remove(data, Entry1) end, Entries1 = Entries#{Path => Entry2}, add_entry1(Entries1, tl(lists:reverse(Path)), New). -set_node_payload(#{data := Data} = Entry, #kpayload_data{data = Data}) -> +set_node_payload(#{data := Data} = Entry, #p_data{data = Data}) -> Entry; -set_node_payload(Entry, none) when not is_map_key(data, Entry) -> +set_node_payload(Entry, ?NO_PAYLOAD) when not is_map_key(data, Entry) -> Entry; -set_node_payload(Entry, #kpayload_data{data = Data}) -> +set_node_payload(Entry, #p_data{data = Data}) -> Entry#{data => Data}; -set_node_payload(Entry, none) -> +set_node_payload(Entry, ?NO_PAYLOAD) -> maps:remove(data, Entry). add_entry1(Entries, ReversedPath, New) -> @@ -226,9 +227,9 @@ payload() -> data_payload()]). no_payload() -> - none. + ?NO_PAYLOAD. data_payload() -> ?LET(Data, binary(), - #kpayload_data{data = Data}). + #p_data{data = Data}). diff --git a/test/put_command.erl b/test/put_command.erl index ae31a0fe..debe29e0 100644 --- a/test/put_command.erl +++ b/test/put_command.erl @@ -19,9 +19,9 @@ initialize_machine_with_genesis_data_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = foobar_value}}, + payload = khepri_payload:data(foobar_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}], + payload = khepri_payload:data(baz_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -39,18 +39,18 @@ initialize_machine_with_genesis_data_test() -> #{bar => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foobar_value}}}}, + payload = khepri_payload:data(foobar_value)}}}, baz => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = baz_value}} + payload = khepri_payload:data(baz_value)} }}, Root). insert_a_node_at_the_root_of_an_empty_db_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Command = #put{path = [foo], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -63,7 +63,7 @@ insert_a_node_at_the_root_of_an_empty_db_test() -> #{foo => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = value}}}}, + payload = khepri_payload:data(value)}}}, Root), ?assertEqual({ok, #{[foo] => #{}}}, Ret), ?assertEqual([], SE). @@ -76,7 +76,7 @@ insert_a_node_at_the_root_of_an_empty_db_with_conditions_test() -> [#if_node_exists{exists = false}, #if_payload_version{version = 1} ]}]}], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -89,19 +89,19 @@ insert_a_node_at_the_root_of_an_empty_db_with_conditions_test() -> #{foo => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = value}}}}, + payload = khepri_payload:data(value)}}}, Root), ?assertEqual({ok, #{[foo] => #{}}}, Ret), ?assertEqual([], SE). overwrite_an_existing_node_data_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #put{path = [foo], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -115,7 +115,7 @@ overwrite_an_existing_node_data_test() -> #node{ stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value2}}}}, + payload = khepri_payload:data(value2)}}}, Root), ?assertEqual({ok, #{[foo] => #{data => value1, payload_version => 1, @@ -126,7 +126,7 @@ overwrite_an_existing_node_data_test() -> insert_a_node_with_path_containing_dot_and_dot_dot_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Command = #put{path = [foo, ?PARENT_NODE, foo, bar, ?THIS_NODE], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -143,7 +143,7 @@ insert_a_node_with_path_containing_dot_and_dot_dot_test() -> #{bar => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = value}}}}}}, + payload = khepri_payload:data(value)}}}}}, Root), ?assertEqual({ok, #{[foo, bar] => #{}}}, Ret), ?assertEqual([], SE). @@ -151,7 +151,7 @@ insert_a_node_with_path_containing_dot_and_dot_dot_test() -> insert_a_node_under_an_nonexisting_parents_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Command = #put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -176,21 +176,21 @@ insert_a_node_under_an_nonexisting_parents_test() -> #{qux => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = value}}}}}}}}}}, + payload = khepri_payload:data(value)}}}}}}}}}, Root), ?assertEqual({ok, #{[foo, bar, baz, qux] => #{}}}, Ret), ?assertEqual([], SE). insert_a_node_with_condition_true_on_self_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #put{path = [#if_all{conditions = [foo, #if_data_matches{pattern = value1}]}], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -203,7 +203,7 @@ insert_a_node_with_condition_true_on_self_test() -> #node{ stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value2}}}}, + payload = khepri_payload:data(value2)}}}, Root), ?assertEqual({ok, #{[foo] => #{data => value1, payload_version => 1, @@ -213,7 +213,7 @@ insert_a_node_with_condition_true_on_self_test() -> insert_a_node_with_condition_false_on_self_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -221,7 +221,7 @@ insert_a_node_with_condition_false_on_self_test() -> %% version to make an exact match on the returned error later. Compiled = khepri_condition:compile(#if_data_matches{pattern = value2}), Command = #put{path = [#if_all{conditions = [foo, Compiled]}], - payload = #kpayload_data{data = value3}}, + payload = khepri_payload:data(value3)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), ?assertEqual(S0#khepri_machine.root, S1#khepri_machine.root), @@ -240,7 +240,7 @@ insert_a_node_with_condition_false_on_self_test() -> insert_a_node_with_condition_true_on_self_using_dot_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -248,7 +248,7 @@ insert_a_node_with_condition_true_on_self_using_dot_test() -> #if_all{conditions = [?THIS_NODE, #if_data_matches{pattern = value1}]}], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -261,7 +261,7 @@ insert_a_node_with_condition_true_on_self_using_dot_test() -> #node{ stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value2}}}}, + payload = khepri_payload:data(value2)}}}, Root), ?assertEqual({ok, #{[foo] => #{data => value1, payload_version => 1, @@ -271,7 +271,7 @@ insert_a_node_with_condition_true_on_self_using_dot_test() -> insert_a_node_with_condition_false_on_self_using_dot_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -280,7 +280,7 @@ insert_a_node_with_condition_false_on_self_using_dot_test() -> Compiled = khepri_condition:compile(#if_data_matches{pattern = value2}), Command = #put{path = [foo, #if_all{conditions = [?THIS_NODE, Compiled]}], - payload = #kpayload_data{data = value3}}, + payload = khepri_payload:data(value3)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), ?assertEqual(S0#khepri_machine.root, S1#khepri_machine.root), @@ -299,7 +299,7 @@ insert_a_node_with_condition_false_on_self_using_dot_test() -> insert_a_node_with_condition_true_on_parent_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -307,7 +307,7 @@ insert_a_node_with_condition_true_on_parent_test() -> [foo, #if_data_matches{pattern = value1}]}, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -320,19 +320,19 @@ insert_a_node_with_condition_true_on_parent_test() -> #node{ stat = #{payload_version => 1, child_list_version => 2}, - payload = #kpayload_data{data = value1}, + payload = khepri_payload:data(value1), child_nodes = #{bar => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = bar_value}}}}}}, + payload = khepri_payload:data(bar_value)}}}}}, Root), ?assertEqual({ok, #{[foo, bar] => #{}}}, Ret), ?assertEqual([], SE). insert_a_node_with_condition_false_on_parent_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -341,7 +341,7 @@ insert_a_node_with_condition_false_on_parent_test() -> Compiled = khepri_condition:compile(#if_data_matches{pattern = value2}), Command = #put{path = [#if_all{conditions = [foo, Compiled]}, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), ?assertEqual(S0#khepri_machine.root, S1#khepri_machine.root), @@ -364,14 +364,14 @@ insert_a_node_with_condition_false_on_parent_test() -> insert_a_node_with_if_node_exists_true_on_self_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command1 = #put{path = [#if_all{conditions = [foo, #if_node_exists{exists = true}]}], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S1, Ret1, SE1} = khepri_machine:apply(?META, Command1, S0), Root = khepri_machine:get_root(S1), @@ -385,7 +385,7 @@ insert_a_node_with_if_node_exists_true_on_self_test() -> #node{ stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value2}}}}, + payload = khepri_payload:data(value2)}}}, Root), ?assertEqual({ok, #{[foo] => #{data => value1, payload_version => 1, @@ -398,7 +398,7 @@ insert_a_node_with_if_node_exists_true_on_self_test() -> [baz, #if_node_exists{exists = true}]}), Command2 = #put{path = [Compiled], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S2, Ret2, SE2} = khepri_machine:apply(?META, Command2, S0), ?assertEqual(S0#khepri_machine.root, S2#khepri_machine.root), @@ -413,14 +413,14 @@ insert_a_node_with_if_node_exists_true_on_self_test() -> insert_a_node_with_if_node_exists_false_on_self_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command1 = #put{path = [#if_all{conditions = [foo, #if_node_exists{exists = false}]}], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S1, Ret1, SE1} = khepri_machine:apply(?META, Command1, S0), ?assertEqual(S0#khepri_machine.root, S1#khepri_machine.root), @@ -440,7 +440,7 @@ insert_a_node_with_if_node_exists_false_on_self_test() -> Command2 = #put{path = [#if_all{conditions = [baz, #if_node_exists{exists = false}]}], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S2, Ret2, SE2} = khepri_machine:apply(?META, Command2, S0), Root = khepri_machine:get_root(S2), @@ -453,18 +453,18 @@ insert_a_node_with_if_node_exists_false_on_self_test() -> #{foo => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = value1}}, + payload = khepri_payload:data(value1)}, baz => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = value2}}}}, + payload = khepri_payload:data(value2)}}}, Root), ?assertEqual({ok, #{[baz] => #{}}}, Ret2), ?assertEqual([], SE2). insert_a_node_with_if_node_exists_true_on_parent_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -472,7 +472,7 @@ insert_a_node_with_if_node_exists_true_on_parent_test() -> [foo, #if_node_exists{exists = true}]}, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, {S1, Ret1, SE1} = khepri_machine:apply(?META, Command1, S0), Root = khepri_machine:get_root(S1), @@ -485,12 +485,12 @@ insert_a_node_with_if_node_exists_true_on_parent_test() -> #node{ stat = #{payload_version => 1, child_list_version => 2}, - payload = #kpayload_data{data = value1}, + payload = khepri_payload:data(value1), child_nodes = #{bar => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = bar_value}}}}}}, + payload = khepri_payload:data(bar_value)}}}}}, Root), ?assertEqual({ok, #{[foo, bar] => #{}}}, Ret1), ?assertEqual([], SE1), @@ -501,7 +501,7 @@ insert_a_node_with_if_node_exists_true_on_parent_test() -> #if_node_exists{exists = true}]}), Command2 = #put{path = [Compiled, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, {S2, Ret2, SE2} = khepri_machine:apply(?META, Command2, S0), ?assertEqual(S0#khepri_machine.root, S2#khepri_machine.root), @@ -516,7 +516,7 @@ insert_a_node_with_if_node_exists_true_on_parent_test() -> insert_a_node_with_if_node_exists_false_on_parent_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value1}}], + payload = khepri_payload:data(value1)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), @@ -524,7 +524,7 @@ insert_a_node_with_if_node_exists_false_on_parent_test() -> [foo, #if_node_exists{exists = false}]}, bar], - payload = #kpayload_data{data = value2}}, + payload = khepri_payload:data(value2)}, {S1, Ret1, SE1} = khepri_machine:apply(?META, Command1, S0), ?assertEqual(S0#khepri_machine.root, S1#khepri_machine.root), @@ -545,7 +545,7 @@ insert_a_node_with_if_node_exists_false_on_parent_test() -> [baz, #if_node_exists{exists = false}]}, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, {S2, Ret2, SE2} = khepri_machine:apply(?META, Command2, S0), Root = khepri_machine:get_root(S2), @@ -558,7 +558,7 @@ insert_a_node_with_if_node_exists_false_on_parent_test() -> #{foo => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = value1}}, + payload = khepri_payload:data(value1)}, baz => #node{ stat = ?INIT_NODE_STAT, @@ -566,36 +566,39 @@ insert_a_node_with_if_node_exists_false_on_parent_test() -> #{bar => #node{ stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = bar_value}}}}}}, + payload = khepri_payload:data(bar_value)}}}}}, Root), ?assertEqual({ok, #{[baz, bar] => #{}}}, Ret2), ?assertEqual([], SE2). insert_with_a_path_matching_many_nodes_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}, + payload = khepri_payload:data(foo_value)}, #put{path = [bar], - payload = #kpayload_data{data = bar_value}}], + payload = khepri_payload:data(bar_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #put{path = [#if_name_matches{regex = any}], - payload = #kpayload_data{data = new_value}}, + payload = khepri_payload:data(new_value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), ?assertEqual(S0#khepri_machine.root, S1#khepri_machine.root), ?assertEqual(#{applied_command_count => 1}, S1#khepri_machine.metrics), - ?assertEqual({error, matches_many_nodes}, Ret), + ?assertEqual( + {error, + {possibly_matching_many_nodes_denied, #if_name_matches{regex = any}}}, + Ret), ?assertEqual([], SE). clear_payload_in_an_existing_node_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #put{path = [foo], - payload = none}, + payload = ?NO_PAYLOAD}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -609,7 +612,7 @@ clear_payload_in_an_existing_node_test() -> #node{ stat = #{payload_version => 2, child_list_version => 1}, - payload = none}}}, + payload = ?NO_PAYLOAD}}}, Root), ?assertEqual({ok, #{[foo] => #{data => value, payload_version => 1, @@ -619,7 +622,7 @@ clear_payload_in_an_existing_node_test() -> put_command_bumps_applied_command_count_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, snapshot_interval => 3, commands => Commands}), @@ -627,21 +630,21 @@ put_command_bumps_applied_command_count_test() -> ?assertEqual(#{}, S0#khepri_machine.metrics), Command1 = #put{path = [bar], - payload = none}, + payload = ?NO_PAYLOAD}, {S1, _, SE1} = khepri_machine:apply(?META, Command1, S0), ?assertEqual(#{applied_command_count => 1}, S1#khepri_machine.metrics), ?assertEqual([], SE1), Command2 = #put{path = [baz], - payload = none}, + payload = ?NO_PAYLOAD}, {S2, _, SE2} = khepri_machine:apply(?META, Command2, S1), ?assertEqual(#{applied_command_count => 2}, S2#khepri_machine.metrics), ?assertEqual([], SE2), Command3 = #put{path = [qux], - payload = none}, + payload = ?NO_PAYLOAD}, Meta = ?META, {S3, _, SE3} = khepri_machine:apply(Meta, Command3, S2), diff --git a/test/queries.erl b/test/queries.erl index 6ee3ddc4..0bf71d82 100644 --- a/test/queries.erl +++ b/test/queries.erl @@ -27,7 +27,7 @@ query_a_non_existing_node_test() -> query_an_existing_node_with_no_value_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -41,7 +41,7 @@ query_an_existing_node_with_no_value_test() -> query_an_existing_node_with_value_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -56,7 +56,7 @@ query_an_existing_node_with_value_test() -> query_a_node_with_matching_condition_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -75,7 +75,7 @@ query_a_node_with_matching_condition_test() -> query_a_node_with_non_matching_condition_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -91,9 +91,9 @@ query_a_node_with_non_matching_condition_test() -> query_child_nodes_of_a_specific_node_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}], + payload = khepri_payload:data(baz_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -117,9 +117,9 @@ query_child_nodes_of_a_specific_node_test() -> query_child_nodes_of_a_specific_node_with_condition_on_leaf_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}], + payload = khepri_payload:data(baz_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -137,13 +137,13 @@ query_child_nodes_of_a_specific_node_with_condition_on_leaf_test() -> query_many_nodes_with_condition_on_parent_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{data = pouet_value}}], + payload = khepri_payload:data(pouet_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -166,13 +166,13 @@ query_many_nodes_with_condition_on_parent_test() -> query_many_nodes_recursively_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{data = pouet_value}}], + payload = khepri_payload:data(pouet_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -214,13 +214,13 @@ query_many_nodes_recursively_test() -> query_many_nodes_recursively_using_regex_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{data = pouet_value}}], + payload = khepri_payload:data(pouet_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -245,13 +245,13 @@ query_many_nodes_recursively_using_regex_test() -> query_many_nodes_recursively_with_condition_on_leaf_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{data = pouet_value}}], + payload = khepri_payload:data(pouet_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -273,13 +273,13 @@ query_many_nodes_recursively_with_condition_on_leaf_test() -> query_many_nodes_recursively_with_condition_on_self_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{data = pouet_value}}], + payload = khepri_payload:data(pouet_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -310,13 +310,13 @@ query_many_nodes_recursively_with_condition_on_self_test() -> query_many_nodes_recursively_with_several_star_star_test() -> Commands = [#put{path = [foo, bar, baz, qux], - payload = #kpayload_data{data = qux_value}}, + payload = khepri_payload:data(qux_value)}, #put{path = [foo, youpi], - payload = #kpayload_data{data = youpi_value}}, + payload = khepri_payload:data(youpi_value)}, #put{path = [baz], - payload = #kpayload_data{data = baz_value}}, + payload = khepri_payload:data(baz_value)}, #put{path = [baz, pouet], - payload = #kpayload_data{data = pouet_value}}], + payload = khepri_payload:data(pouet_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -336,7 +336,7 @@ query_many_nodes_recursively_with_several_star_star_test() -> query_a_node_using_relative_path_components_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -352,9 +352,9 @@ query_a_node_using_relative_path_components_test() -> include_child_names_in_query_response_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [foo, quux], - payload = #kpayload_data{data = quux_value}}], + payload = khepri_payload:data(quux_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), diff --git a/test/root_node.erl b/test/root_node.erl index fe0da31a..4a6e91d8 100644 --- a/test/root_node.erl +++ b/test/root_node.erl @@ -51,7 +51,7 @@ query_root_node_using_dot_test() -> query_above_root_node_using_dot_dot_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Root = khepri_machine:get_root(S0), @@ -110,7 +110,7 @@ query_root_node_with_conditions_false_test() -> store_data_in_root_node_using_empty_path_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Command = #put{path = [], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -119,7 +119,7 @@ store_data_in_root_node_using_empty_path_test() -> stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, Root), ?assertEqual({ok, #{[] => #{payload_version => 1, child_list_version => 1, @@ -129,7 +129,7 @@ store_data_in_root_node_using_empty_path_test() -> store_data_in_root_node_using_root_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Command = #put{path = [?ROOT_NODE], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -138,7 +138,7 @@ store_data_in_root_node_using_root_test() -> stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, Root), ?assertEqual({ok, #{[] => #{payload_version => 1, child_list_version => 1, @@ -148,7 +148,7 @@ store_data_in_root_node_using_root_test() -> store_data_in_root_node_using_dot_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Command = #put{path = [?THIS_NODE], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -157,7 +157,7 @@ store_data_in_root_node_using_dot_test() -> stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, Root), ?assertEqual({ok, #{[] => #{payload_version => 1, child_list_version => 1, @@ -167,7 +167,7 @@ store_data_in_root_node_using_dot_test() -> store_data_in_root_node_using_dot_dot_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Command = #put{path = [?PARENT_NODE], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -176,7 +176,7 @@ store_data_in_root_node_using_dot_dot_test() -> stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, Root), ?assertEqual({ok, #{[] => #{payload_version => 1, child_list_version => 1, @@ -187,7 +187,7 @@ store_data_in_root_node_with_condition_true_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Compiled = khepri_condition:compile(#if_child_list_length{count = 0}), Command = #put{path = [#if_all{conditions = [?ROOT_NODE, Compiled]}], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -196,7 +196,7 @@ store_data_in_root_node_with_condition_true_test() -> stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, Root), ?assertEqual({ok, #{[] => #{payload_version => 1, child_list_version => 1, @@ -207,7 +207,7 @@ store_data_in_root_node_with_condition_true_using_dot_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Compiled = khepri_condition:compile(#if_child_list_length{count = 0}), Command = #put{path = [#if_all{conditions = [?THIS_NODE, Compiled]}], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -216,7 +216,7 @@ store_data_in_root_node_with_condition_true_using_dot_test() -> stat = #{payload_version => 2, child_list_version => 1}, - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, Root), ?assertEqual({ok, #{[] => #{payload_version => 1, child_list_version => 1, @@ -227,7 +227,7 @@ store_data_in_root_node_with_condition_false_test() -> S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME}), Compiled = khepri_condition:compile(#if_child_list_length{count = 1}), Command = #put{path = [#if_all{conditions = [?ROOT_NODE, Compiled]}], - payload = #kpayload_data{data = value}}, + payload = khepri_payload:data(value)}, {S1, Ret, SE} = khepri_machine:apply(?META, Command, S0), Root = khepri_machine:get_root(S1), @@ -267,7 +267,7 @@ delete_empty_root_node_test() -> delete_root_node_using_empty_path_test() -> Commands = [#put{path = [], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = []}, @@ -288,7 +288,7 @@ delete_root_node_using_empty_path_test() -> delete_root_node_using_root_test() -> Commands = [#put{path = [], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [?ROOT_NODE]}, @@ -309,7 +309,7 @@ delete_root_node_using_root_test() -> delete_root_node_using_dot_test() -> Commands = [#put{path = [], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [?THIS_NODE]}, @@ -330,7 +330,7 @@ delete_root_node_using_dot_test() -> delete_root_node_using_dot_dot_test() -> Commands = [#put{path = [], - payload = #kpayload_data{data = value}}], + payload = khepri_payload:data(value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = [?PARENT_NODE]}, @@ -351,9 +351,9 @@ delete_root_node_using_dot_dot_test() -> delete_root_node_with_child_nodes_test() -> Commands = [#put{path = [foo, bar], - payload = #kpayload_data{data = bar_value}}, + payload = khepri_payload:data(bar_value)}, #put{path = [baz, qux], - payload = #kpayload_data{data = qux_value}}], + payload = khepri_payload:data(qux_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Command = #delete{path = []}, @@ -373,7 +373,7 @@ delete_root_node_with_child_nodes_test() -> delete_root_node_with_condition_true_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Compiled = khepri_condition:compile(#if_child_list_length{count = 1}), @@ -394,7 +394,7 @@ delete_root_node_with_condition_true_test() -> delete_root_node_with_condition_true_using_dot_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Compiled = khepri_condition:compile(#if_child_list_length{count = 1}), @@ -415,7 +415,7 @@ delete_root_node_with_condition_true_using_dot_test() -> delete_root_node_with_condition_false_test() -> Commands = [#put{path = [foo], - payload = #kpayload_data{data = foo_value}}], + payload = khepri_payload:data(foo_value)}], S0 = khepri_machine:init(#{store_id => ?FUNCTION_NAME, commands => Commands}), Compiled = khepri_condition:compile(#if_child_list_length{count = 0}), @@ -431,7 +431,7 @@ delete_root_node_with_condition_false_test() -> child_nodes = #{foo => #node{stat = ?INIT_NODE_STAT, - payload = #kpayload_data{data = foo_value}}}}, + payload = khepri_payload:data(foo_value)}}}, Root), ?assertEqual({ok, #{}}, Ret), ?assertEqual([], SE). diff --git a/test/sf_cache.erl b/test/sf_cache.erl index 4fba662d..fee183b7 100644 --- a/test/sf_cache.erl +++ b/test/sf_cache.erl @@ -10,6 +10,7 @@ -include_lib("eunit/include/eunit.hrl"). -include("include/khepri.hrl"). +-include("src/khepri_fun.hrl"). -include("src/internal.hrl"). %% FIXME: compile:forms/2 is incorrectly specified and doesn't accept diff --git a/test/simple_delete.erl b/test/simple_delete.erl index 8805cfe3..7460515e 100644 --- a/test/simple_delete.erl +++ b/test/simple_delete.erl @@ -18,7 +18,7 @@ delete_non_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{}}, khepri:delete(?FUNCTION_NAME, [foo])), ?_assertEqual( {ok, #{}}, @@ -29,10 +29,13 @@ delete_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - ok, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, khepri:delete(?FUNCTION_NAME, [foo])), ?_assertEqual( {ok, #{}}, @@ -43,7 +46,7 @@ delete_non_existing_node_with_condition_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{}}, khepri:delete(?FUNCTION_NAME, [#if_name_matches{regex = "foo"}])), ?_assertEqual( {ok, #{}}, @@ -54,10 +57,13 @@ delete_existing_node_with_condition_true_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - ok, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, khepri:delete(?FUNCTION_NAME, [#if_name_matches{regex = "foo"}])), ?_assertEqual( {ok, #{}}, @@ -68,10 +74,10 @@ delete_existing_node_with_condition_false_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - ok, + {ok, #{}}, khepri:delete(?FUNCTION_NAME, [#if_name_matches{regex = "bar"}])), ?_assertEqual( {ok, #{[foo] => #{data => foo_value, @@ -85,10 +91,10 @@ clear_store_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, #{[] => #{payload_version => 1, @@ -106,8 +112,15 @@ clear_store_test_() -> child_list_version => 1, child_list_length => 0}}}, khepri:get(?FUNCTION_NAME, [?STAR_STAR])), + %% FIXME: SHould it return child nodes of `/:foo'? ?_assertEqual( - ok, + {ok, #{[foo] => #{payload_version => 1, + child_list_version => 1, + child_list_length => 1}, + [baz] => #{data => baz_value, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, khepri:clear_store(?FUNCTION_NAME)), ?_assertEqual( {ok, #{[] => #{payload_version => 1, diff --git a/test/simple_get.erl b/test/simple_get.erl index 625c8abb..84b1a8c1 100644 --- a/test/simple_get.erl +++ b/test/simple_get.erl @@ -32,7 +32,7 @@ get_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, #{[foo] => #{data => foo_value, @@ -46,10 +46,10 @@ get_many_nodes_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, #{[] => #{payload_version => 1, @@ -65,7 +65,9 @@ get_many_nodes_test_() -> khepri:get( ?FUNCTION_NAME, [?THIS_NODE, #if_name_matches{regex = any}])), ?_assertEqual( - {error, matches_many_nodes}, + {error, + {possibly_matching_many_nodes_denied, + #if_name_matches{regex = any}}}, khepri:get( ?FUNCTION_NAME, [?THIS_NODE, #if_name_matches{regex = any}], @@ -76,24 +78,143 @@ check_node_exists_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value)), ?_assert(khepri:exists(?FUNCTION_NAME, [foo])), ?_assertNot(khepri:exists(?FUNCTION_NAME, [bar]))]}. -check_node_has_data_test_() -> +check_node_has_data_on_non_existing_node_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertNot(khepri:has_data(?FUNCTION_NAME, [foo]))]}. + +check_node_has_data_on_existing_node_test_() -> {setup, fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertNot(khepri:has_data(?FUNCTION_NAME, [foo])), ?_assert(khepri:has_data(?FUNCTION_NAME, [baz]))]}. +check_node_has_sproc_on_non_existing_node_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertNot(khepri:has_sproc(?FUNCTION_NAME, [foo]))]}. + +check_node_has_sproc_on_existing_node_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo, bar] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo, bar], fun() -> bar_value end)), + ?_assertEqual( + {ok, #{[baz] => #{}}}, + khepri:create(?FUNCTION_NAME, [baz], fun() -> baz_value end)), + ?_assertNot(khepri:has_sproc(?FUNCTION_NAME, [foo])), + ?_assert(khepri:has_sproc(?FUNCTION_NAME, [baz]))]}. + +get_node_props_on_non_existing_node_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertThrow( + {error, {node_not_found, #{node_name := foo, + node_path := [foo], + node_is_target := true}}}, + khepri:get_node_props(?FUNCTION_NAME, [foo]))]}. + +get_node_props_on_existing_node_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo], foo_value)), + ?_assertEqual( + #{data => foo_value, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}, + khepri:get_node_props(?FUNCTION_NAME, [foo]))]}. + +get_node_props_on_many_nodes_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo, bar] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), + ?_assertEqual( + {ok, #{[baz] => #{}}}, + khepri:create(?FUNCTION_NAME, [baz], baz_value)), + ?_assertThrow( + {error, + {possibly_matching_many_nodes_denied, + #if_name_matches{regex = any}}}, + khepri:get_node_props( + ?FUNCTION_NAME, + [?THIS_NODE, #if_name_matches{regex = any}]))]}. + +get_data_on_non_existing_node_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertThrow( + {error, {node_not_found, #{node_name := foo, + node_path := [foo], + node_is_target := true}}}, + khepri:get_data(?FUNCTION_NAME, [foo]))]}. + +get_data_on_existing_node_with_data_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo], foo_value)), + ?_assertEqual( + foo_value, + khepri:get_data(?FUNCTION_NAME, [foo]))]}. + +get_data_on_existing_node_without_data_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo], khepri_payload:none())), + ?_assertThrow( + {error, {no_data, #{payload_version := 1, + child_list_version := 1, + child_list_length := 0}}}, + khepri:get_data(?FUNCTION_NAME, [foo]))]}. + +get_data_on_many_nodes_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo, bar] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), + ?_assertEqual( + {ok, #{[baz] => #{}}}, + khepri:create(?FUNCTION_NAME, [baz], baz_value)), + ?_assertThrow( + {error, + {possibly_matching_many_nodes_denied, + #if_name_matches{regex = any}}}, + khepri:get_data( + ?FUNCTION_NAME, + [?THIS_NODE, #if_name_matches{regex = any}]))]}. + list_non_existing_node_test_() -> {setup, fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, @@ -107,10 +228,10 @@ list_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, #{[] => #{payload_version => 1, @@ -138,10 +259,10 @@ find_node_by_name_in_filled_db_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, #{[foo] => #{payload_version => 1, @@ -154,10 +275,10 @@ find_node_by_condition_in_filled_db_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, #{[foo, bar] => #{data => bar_value, @@ -175,16 +296,16 @@ find_node_starting_from_subnode_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo, bar] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - ok, + {ok, #{[foo, bar, baz] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar, baz], baz_value)), ?_assertEqual( - ok, + {ok, #{[foo, bar, qux] => #{}}}, khepri:create(?FUNCTION_NAME, [foo, bar, qux], qux_value)), ?_assertEqual( - ok, + {ok, #{[baz] => #{}}}, khepri:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, #{[foo, bar, baz] => #{data => baz_value, diff --git a/test/simple_put.erl b/test/simple_put.erl index dc2e5dfc..d7464877 100644 --- a/test/simple_put.erl +++ b/test/simple_put.erl @@ -18,7 +18,7 @@ create_non_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, #{[foo] => #{data => foo_value, @@ -32,7 +32,7 @@ create_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( {error, @@ -52,8 +52,8 @@ insert_non_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, - khepri:insert(?FUNCTION_NAME, [foo], foo_value)), + {ok, #{[foo] => #{}}}, + khepri:put(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, #{[foo] => #{data => foo_value, payload_version => 1, @@ -66,11 +66,14 @@ insert_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( - ok, - khepri:insert(?FUNCTION_NAME, [foo], foo_value2)), + {ok, #{[foo] => #{data => foo_value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, + khepri:put(?FUNCTION_NAME, [foo], foo_value2)), ?_assertEqual( {ok, #{[foo] => #{data => foo_value2, payload_version => 2, @@ -98,10 +101,13 @@ update_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( - ok, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, khepri:update(?FUNCTION_NAME, [foo], foo_value2)), ?_assertEqual( {ok, #{[foo] => #{data => foo_value2, @@ -131,10 +137,13 @@ compare_and_swap_matching_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( - ok, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, khepri:compare_and_swap( ?FUNCTION_NAME, [foo], foo_value1, foo_value2)), ?_assertEqual( @@ -149,7 +158,7 @@ compare_and_swap_mismatching_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertMatch( {error, @@ -165,12 +174,42 @@ compare_and_swap_mismatching_node_test_() -> khepri:compare_and_swap( ?FUNCTION_NAME, [foo], foo_value2, foo_value3))]}. +compare_and_swap_with_keep_while_or_options_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo], foo_value1)), + ?_assertEqual( + {ok, #{[foo] => #{data => foo_value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, + khepri:compare_and_swap( + ?FUNCTION_NAME, [foo], foo_value1, foo_value2, + #{keep_while => #{}})), + ?_assertEqual( + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2, + child_list_version => 1, + child_list_length => 0}}}, + khepri:compare_and_swap( + ?FUNCTION_NAME, [foo], foo_value2, foo_value3, + #{async => false})), + ?_assertEqual( + {ok, #{[foo] => #{data => foo_value3, + payload_version => 3, + child_list_version => 1, + child_list_length => 0}}}, + khepri:get(?FUNCTION_NAME, [foo]))]}. + clear_payload_from_non_existing_node_test_() -> {setup, fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:clear_payload(?FUNCTION_NAME, [foo])), ?_assertEqual( {ok, #{[foo] => #{payload_version => 1, @@ -183,13 +222,54 @@ clear_payload_from_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - ok, + {ok, #{[foo] => #{}}}, khepri:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - ok, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, khepri:clear_payload(?FUNCTION_NAME, [foo])), ?_assertEqual( {ok, #{[foo] => #{payload_version => 2, child_list_version => 1, child_list_length => 0}}}, khepri:get(?FUNCTION_NAME, [foo]))]}. + +clear_payload_with_keep_while_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo], foo_value)), + ?_assertEqual( + {ok, #{[foo] => #{data => foo_value, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, + khepri:clear_payload(?FUNCTION_NAME, [foo], #{keep_while => #{}})), + ?_assertEqual( + {ok, #{[foo] => #{payload_version => 2, + child_list_version => 1, + child_list_length => 0}}}, + khepri:get(?FUNCTION_NAME, [foo]))]}. + +clear_payload_with_options_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {ok, #{[foo] => #{}}}, + khepri:create(?FUNCTION_NAME, [foo], foo_value)), + ?_assertEqual( + {ok, #{[foo] => #{data => foo_value, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}, + khepri:clear_payload(?FUNCTION_NAME, [foo], #{async => false})), + ?_assertEqual( + {ok, #{[foo] => #{payload_version => 2, + child_list_version => 1, + child_list_length => 0}}}, + khepri:get(?FUNCTION_NAME, [foo]))]}. diff --git a/test/stored_procs.erl b/test/stored_procs.erl index 7a2fd97a..8cdfc04f 100644 --- a/test/stored_procs.erl +++ b/test/stored_procs.erl @@ -10,6 +10,7 @@ -include_lib("eunit/include/eunit.hrl"). -include("include/khepri.hrl"). +-include("src/khepri_fun.hrl"). -include("src/internal.hrl"). -include("test/helpers.hrl"). @@ -22,14 +23,14 @@ execute_valid_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{sproc = fun() -> return_value end}))}, + fun() -> return_value end))}, {"Execute the stored procedure", ?_assertEqual( return_value, - khepri_machine:run_sproc( + khepri:run_sproc( ?FUNCTION_NAME, StoredProcPath, []))}] }]}. @@ -47,7 +48,7 @@ execute_nonexisting_sproc_test_() -> #{node_name := sproc, node_path := StoredProcPath, node_is_target := true}}}}, - khepri_machine:run_sproc( + khepri:run_sproc( ?FUNCTION_NAME, StoredProcPath, []))}] }]}. @@ -60,9 +61,9 @@ try_to_execute_data_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_data{data = value}))}, + value))}, {"Execute the stored procedure", ?_assertThrow( @@ -73,7 +74,7 @@ try_to_execute_data_test_() -> payload_version := 1, child_list_version := 1, child_list_length := 0}}}, - khepri_machine:run_sproc( + khepri:run_sproc( ?FUNCTION_NAME, StoredProcPath, []))}] }]}. @@ -87,14 +88,14 @@ execute_sproc_with_wrong_arity_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{sproc = fun() -> return_value end}))}, + fun() -> return_value end))}, {"Execute the stored procedure", ?_assertExit( {badarity, {#standalone_fun{arity = 0}, Args}}, - khepri_machine:run_sproc( + khepri:run_sproc( ?FUNCTION_NAME, StoredProcPath, Args))}] }]}. @@ -107,14 +108,14 @@ execute_crashing_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{sproc = fun() -> throw("Expected crash") end}))}, + fun() -> throw("Expected crash") end))}, {"Execute the stored procedure", ?_assertThrow( "Expected crash", - khepri_machine:run_sproc( + khepri:run_sproc( ?FUNCTION_NAME, StoredProcPath, []))}] }]}. @@ -130,10 +131,9 @@ crashing_sproc_stacktrace_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{ - sproc = fun mod_used_for_transactions:crashing_fun/0}))}, + fun mod_used_for_transactions:crashing_fun/0))}, {"Execute the stored procedure", ?_assertMatch( @@ -143,7 +143,7 @@ crashing_sproc_stacktrace_test_() -> {stored_procs, _, _, [{file, File2}, {line, _}]} | _]}, try - khepri_machine:run_sproc( + khepri:run_sproc( ?FUNCTION_NAME, StoredProcPath, []) catch Class:Reason:Stacktrace -> diff --git a/test/test_ra_server_helpers.erl b/test/test_ra_server_helpers.erl index f0959725..edd25a05 100644 --- a/test/test_ra_server_helpers.erl +++ b/test/test_ra_server_helpers.erl @@ -47,7 +47,7 @@ setup(Testcase) -> cleanup(#{ra_system := RaSystem, store_dir := StoreDir, store_id := StoreId}) -> - ServerIds = khepri:members(StoreId), + ServerIds = khepri_cluster:members(StoreId), _ = application:stop(khepri), %% FIXME: This monitoring can go away when/if the following pull request %% in Ra is merged: diff --git a/test/transactions.erl b/test/transactions.erl index b232e89b..5e6385f0 100644 --- a/test/transactions.erl +++ b/test/transactions.erl @@ -28,8 +28,7 @@ fun_taking_args_in_rw_transaction_test_/0, not_a_function_as_ro_transaction_test_/0, not_a_function_as_rw_transaction_test_/0, - use_an_invalid_path_in_tx_test_/0, - use_an_invalid_payload_in_tx_test_/0]}]). + use_an_invalid_path_in_tx_test_/0]}]). %% Used internally for a testcase. -export([really_do_get_root_path/0, @@ -83,7 +82,7 @@ is_transaction_test_() -> Fun = fun() -> khepri_tx:is_transaction() end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, ro) + khepri:transaction(?FUNCTION_NAME, Fun, ro) end), ?_assertEqual( {atomic, true}, @@ -91,7 +90,7 @@ is_transaction_test_() -> Fun = fun() -> khepri_tx:is_transaction() end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]} ]. @@ -105,7 +104,7 @@ noop_in_ro_transaction_test_() -> Fun = fun() -> ok end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, ro) + khepri:transaction(?FUNCTION_NAME, Fun, ro) end)]}. noop_in_rw_transaction_test_() -> @@ -118,7 +117,7 @@ noop_in_rw_transaction_test_() -> Fun = fun() -> ok end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]}. get_in_ro_transaction_test_() -> @@ -132,13 +131,13 @@ get_in_ro_transaction_test_() -> child_list_version => 1, child_list_length => 0}}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Fun = fun() -> khepri_tx:get([foo]) end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, ro) + khepri:transaction(?FUNCTION_NAME, Fun, ro) end)]}. get_in_rw_transaction_test_() -> @@ -152,13 +151,136 @@ get_in_rw_transaction_test_() -> child_list_version => 1, child_list_length => 0}}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Fun = fun() -> khepri_tx:get([foo]) end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +get_node_props_on_non_existing_node_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {aborted, + {error, {node_not_found, #{node_name => foo, + node_path => [foo], + node_is_target => true}}}}, + begin + Fun = fun() -> + khepri_tx:get_node_props([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +get_node_props_on_existing_node_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + #{data => value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:get_node_props([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +get_node_props_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + #{data => value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:get_node_props([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +get_data_on_non_existing_node_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {aborted, + {error, {node_not_found, #{node_name => foo, + node_path => [foo], + node_is_target => true}}}}, + begin + Fun = fun() -> + khepri_tx:get_data([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +get_data_on_existing_node_with_data_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, value1}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:get_data([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +get_data_on_existing_node_without_data_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {aborted, + {error, {no_data, #{payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:none()), + + Fun = fun() -> + khepri_tx:get_data([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +get_data_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, value1}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:get_data([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]}. put_in_ro_transaction_test_() -> @@ -168,20 +290,20 @@ put_in_ro_transaction_test_() -> [?_assertEqual( {aborted, store_update_denied}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Fun = fun() -> Path = [foo], case khepri_tx:get(Path) of {ok, #{Path := #{data := value1}}} -> khepri_tx:put( - Path, #kpayload_data{data = value2}); + Path, khepri_payload:data(value2)); Other -> Other end end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, ro) + khepri:transaction(?FUNCTION_NAME, Fun, ro) end)]}. put_in_rw_transaction_test_() -> @@ -195,20 +317,276 @@ put_in_rw_transaction_test_() -> child_list_version => 1, child_list_length => 0}}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Fun = fun() -> Path = [foo], case khepri_tx:get(Path) of {ok, #{Path := #{data := value1}}} -> khepri_tx:put( - Path, #kpayload_data{data = value2}); + Path, khepri_payload:data(value2)); Other -> Other end end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +create_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {aborted, store_update_denied}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:create( + [foo], khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +create_on_non_existing_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + {ok, #{[foo] => #{}}}}, + begin + Fun = fun() -> + khepri_tx:create( + [foo], khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +create_on_existing_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + {error, + {mismatching_node, + #{condition => #if_node_exists{exists = false}, + node_name => foo, + node_path => [foo], + node_is_target => true, + node_props => #{data => value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:create( + [foo], khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +update_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {aborted, store_update_denied}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:update( + [foo], khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +update_on_non_existing_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + {error, + {node_not_found, + #{condition => #if_all{conditions = + [foo, + #if_node_exists{exists = true}]}, + node_name => foo, + node_path => [foo], + node_is_target => true}}}}, + begin + Fun = fun() -> + khepri_tx:update( + [foo], khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +update_on_existing_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + {ok, #{[foo] => #{data => value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:update( + [foo], khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +compare_and_swap_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {aborted, store_update_denied}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:compare_and_swap( + [foo], value1, khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +compare_and_swap_on_non_existing_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertMatch( + {atomic, + {error, + {node_not_found, + #{condition := #if_all{conditions = + [foo, + #if_data_matches{pattern = value1}]}, + node_name := foo, + node_path := [foo], + node_is_target := true}}}}, + begin + Fun = fun() -> + khepri_tx:compare_and_swap( + [foo], value1, khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +compare_and_swap_on_existing_matching_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + {ok, #{[foo] => #{data => value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:compare_and_swap( + [foo], value1, khepri_payload:data(value2)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +compare_and_swap_on_existing_non_matching_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertMatch( + {atomic, + {error, + {mismatching_node, + #{condition := #if_data_matches{pattern = value2}, + node_name := foo, + node_path := [foo], + node_is_target := true, + node_props := #{data := value1, + payload_version := 1, + child_list_version := 1, + child_list_length := 0}}}}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:compare_and_swap( + [foo], value2, khepri_payload:data(value3)) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +clear_payload_in_ro_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {aborted, store_update_denied}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:clear_payload([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, ro) + end)]}. + +clear_payload_on_non_existing_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + {ok, #{[foo] => #{}}}}, + begin + Fun = fun() -> + khepri_tx:clear_payload([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) + end)]}. + +clear_payload_on_existing_node_in_rw_transaction_test_() -> + {setup, + fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, + fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, + [?_assertEqual( + {atomic, + {ok, #{[foo] => #{data => value1, + payload_version => 1, + child_list_version => 1, + child_list_length => 0}}}}, + begin + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), + + Fun = fun() -> + khepri_tx:clear_payload([foo]) + end, + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]}. delete_in_ro_transaction_test_() -> @@ -218,8 +596,8 @@ delete_in_ro_transaction_test_() -> [?_assertEqual( {aborted, store_update_denied}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Fun = fun() -> Path = [foo], @@ -230,7 +608,7 @@ delete_in_ro_transaction_test_() -> Other end end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, ro) + khepri:transaction(?FUNCTION_NAME, Fun, ro) end)]}. delete_in_rw_transaction_test_() -> @@ -244,8 +622,8 @@ delete_in_rw_transaction_test_() -> child_list_version => 1, child_list_length => 0}}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Fun = fun() -> Path = [foo], @@ -256,7 +634,7 @@ delete_in_rw_transaction_test_() -> Other end end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]}. exists_api_test_() -> @@ -267,16 +645,16 @@ exists_api_test_() -> {atomic, {false, true, false}}, begin - _ = khepri_machine:put( + _ = khepri:put( ?FUNCTION_NAME, [foo, bar], - #kpayload_data{data = bar_value}), + khepri_payload:data(bar_value)), Fun = fun() -> {khepri_tx:has_data([foo]), khepri_tx:has_data([foo, bar]), khepri_tx:has_data([foo, bar, baz])} end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]}. has_data_api_test_() -> @@ -287,14 +665,14 @@ has_data_api_test_() -> {atomic, {true, false}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = foo_value}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(foo_value)), Fun = fun() -> {khepri_tx:exists([foo]), khepri_tx:exists([bar])} end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]}. find_api_test_() -> @@ -308,13 +686,13 @@ find_api_test_() -> child_list_version => 1, child_list_length => 0}}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = foo_value}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(foo_value)), Fun = fun() -> khepri_tx:find([], #if_data_matches{pattern = '_'}) end, - khepri_machine:transaction(?FUNCTION_NAME, Fun, rw) + khepri:transaction(?FUNCTION_NAME, Fun, rw) end)]}. simple_api_test_() -> @@ -331,15 +709,15 @@ simple_api_test_() -> child_list_version => 1, child_list_length => 0}}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Fun = fun() -> Path = [foo], case khepri_tx:get(Path) of {ok, #{Path := #{data := value1}}} -> khepri_tx:put( - Path, #kpayload_data{data = value2}); + Path, khepri_payload:data(value2)); Other -> Other end @@ -357,7 +735,7 @@ case_abort_jump_instruction_test_() -> begin Fun = fun() -> Path = [foo], - case khepri_tx:put(Path, #kpayload_data{data = value2}) of + case khepri_tx:put(Path, khepri_payload:data(value2)) of {ok, _} -> ok; Error -> @@ -375,10 +753,10 @@ list_comprehension_test_() -> [?_assertEqual( {atomic, [bar_value, foo_value]}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = foo_value}), - _ = khepri_machine:put( - ?FUNCTION_NAME, [bar], #kpayload_data{data = bar_value}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(foo_value)), + _ = khepri:put( + ?FUNCTION_NAME, [bar], khepri_payload:data(bar_value)), Fun = fun() -> {ok, Nodes} = khepri_tx:list([?ROOT_NODE]), @@ -702,7 +1080,7 @@ use_an_invalid_path_in_tx_test_() -> {aborted, {invalid_path, #{path => not_a_list}}}, begin Fun = fun() -> - khepri_tx:put(not_a_list, none) + khepri_tx:put(not_a_list, ?NO_PAYLOAD) end, khepri:transaction(?FUNCTION_NAME, Fun) end), @@ -711,28 +1089,7 @@ use_an_invalid_path_in_tx_test_() -> tail => ["not_a_component"]}}}, begin Fun = fun() -> - khepri_tx:put(["not_a_component"], none) - end, - khepri:transaction(?FUNCTION_NAME, Fun) - end)]}. - -use_an_invalid_payload_in_tx_test_() -> - {setup, - fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, - fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, - [?_assertEqual( - {aborted, {invalid_payload, [foo], invalid_payload}}, - begin - Fun = fun() -> - khepri_tx:put([foo], invalid_payload) - end, - khepri:transaction(?FUNCTION_NAME, Fun) - end), - ?_assertEqual( - {aborted, {invalid_payload, [foo], {invalid_payload, in_a_tuple}}}, - begin - Fun = fun() -> - khepri_tx:put([foo], {invalid_payload, in_a_tuple}) + khepri_tx:put(["not_a_component"], ?NO_PAYLOAD) end, khepri:transaction(?FUNCTION_NAME, Fun) end)]}. @@ -743,12 +1100,12 @@ use_an_invalid_payload_in_tx_test_() -> {ok, #{Path := #{data := value1}}} -> khepri_tx:put( Path, - khepri:data_payload(value2)); + khepri_payload:data(value2)); Other -> Other end end, - khepri_machine:transaction( + khepri:transaction( " ++ atom_to_list(?FUNCTION_NAME) ++ ", Fun). "). @@ -767,8 +1124,8 @@ tx_from_the_shell_test_() -> child_list_version => 1, child_list_length => 0}}}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), Bindings = erl_eval:new_bindings(), {ok, Tokens, _EndLocation} = erl_scan:string(?TX_CODE), @@ -794,10 +1151,10 @@ tx_using_erl_eval_test_() -> [?_assertThrow( {invalid_tx_fun, {call_denied, _}}, begin - _ = khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}), + _ = khepri:put( + ?FUNCTION_NAME, [foo], khepri_payload:data(value1)), - khepri_machine:transaction( + khepri:transaction( ?FUNCTION_NAME, fun local_fun_using_erl_eval/0, rw) diff --git a/test/triggers.erl b/test/triggers.erl index b5a3a675..b2c92806 100644 --- a/test/triggers.erl +++ b/test/triggers.erl @@ -14,7 +14,7 @@ -include("test/helpers.hrl"). event_triggers_associated_sproc_test_() -> - EventFilter = #kevf_tree{path = [foo]}, + EventFilter = khepri_evf:tree([foo]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -24,15 +24,14 @@ event_triggers_associated_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -41,15 +40,14 @@ event_triggers_associated_sproc_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value}))}, + khepri:put(?FUNCTION_NAME, [foo], value))}, {"Checking the procedure was executed", ?_assertEqual(executed, receive_sproc_msg(Key))}] }]}. event_using_matching_pattern_triggers_associated_sproc_test_() -> - EventFilter = #kevf_tree{path = [foo, #if_child_list_length{count = 0}]}, + EventFilter = khepri_evf:tree([foo, #if_child_list_length{count = 0}]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -59,15 +57,14 @@ event_using_matching_pattern_triggers_associated_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -76,16 +73,15 @@ event_using_matching_pattern_triggers_associated_sproc_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo, bar], #kpayload_data{data = value}))}, + khepri:put( + ?FUNCTION_NAME, [foo, bar], value))}, {"Checking the procedure was executed", ?_assertEqual(executed, receive_sproc_msg(Key))}] }]}. event_using_non_matching_pattern1_does_not_trigger_associated_sproc_test_() -> - EventFilter = #kevf_tree{ - path = [?STAR, #if_child_list_length{count = 1}]}, + EventFilter = khepri_evf:tree([?STAR, #if_child_list_length{count = 1}]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -95,15 +91,14 @@ event_using_non_matching_pattern1_does_not_trigger_associated_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -112,15 +107,15 @@ event_using_non_matching_pattern1_does_not_trigger_associated_sproc_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo, bar], #kpayload_data{data = value}))}, + khepri:put( + ?FUNCTION_NAME, [foo, bar], value))}, {"Checking the procedure was executed", ?_assertEqual(timeout, receive_sproc_msg(Key))}] }]}. event_using_non_matching_pattern2_does_not_trigger_associated_sproc_test_() -> - EventFilter = #kevf_tree{path = [foo]}, + EventFilter = khepri_evf:tree([foo]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -130,15 +125,14 @@ event_using_non_matching_pattern2_does_not_trigger_associated_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -147,15 +141,15 @@ event_using_non_matching_pattern2_does_not_trigger_associated_sproc_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo, bar], #kpayload_data{data = value}))}, + khepri:put( + ?FUNCTION_NAME, [foo, bar], value))}, {"Checking the procedure was executed", ?_assertEqual(timeout, receive_sproc_msg(Key))}] }]}. event_using_non_matching_pattern3_does_not_trigger_associated_sproc_test_() -> - EventFilter = #kevf_tree{path = [foo, bar]}, + EventFilter = khepri_evf:tree([foo, bar]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -165,15 +159,14 @@ event_using_non_matching_pattern3_does_not_trigger_associated_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -182,15 +175,15 @@ event_using_non_matching_pattern3_does_not_trigger_associated_sproc_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value}))}, + khepri:put( + ?FUNCTION_NAME, [foo], value))}, {"Checking the procedure was executed", ?_assertEqual(timeout, receive_sproc_msg(Key))}] }]}. event_does_not_trigger_unassociated_sproc_test_() -> - EventFilter = #kevf_tree{path = [foo]}, + EventFilter = khepri_evf:tree([foo]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -200,15 +193,14 @@ event_does_not_trigger_unassociated_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -217,15 +209,15 @@ event_does_not_trigger_unassociated_sproc_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [bar], #kpayload_data{data = value}))}, + khepri:put( + ?FUNCTION_NAME, [bar], value))}, {"Checking the procedure was executed", ?_assertEqual(timeout, receive_sproc_msg(Key))}] }]}. event_does_not_trigger_non_existing_sproc_test_() -> - EventFilter = #kevf_tree{path = [foo]}, + EventFilter = khepri_evf:tree([foo]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -235,15 +227,14 @@ event_does_not_trigger_non_existing_sproc_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, [non_existing | StoredProcPath], - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -252,15 +243,15 @@ event_does_not_trigger_non_existing_sproc_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [bar], #kpayload_data{data = value}))}, + khepri:put( + ?FUNCTION_NAME, [bar], value))}, {"Checking the procedure was executed", ?_assertEqual(timeout, receive_sproc_msg(Key))}] }]}. event_does_not_trigger_data_node_test_() -> - EventFilter = #kevf_tree{path = [foo]}, + EventFilter = khepri_evf:tree([foo]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -270,15 +261,14 @@ event_does_not_trigger_data_node_test_() -> [{"Storing a procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath, - #kpayload_data{ - data = not_an_stored_proc}))}, + not_an_stored_proc))}, {"Registering a trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, ?FUNCTION_NAME, EventFilter, @@ -287,20 +277,17 @@ event_does_not_trigger_data_node_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [bar], #kpayload_data{data = value}))}, + khepri:put( + ?FUNCTION_NAME, [bar], value))}, {"Checking the procedure was executed", ?_assertEqual(timeout, receive_sproc_msg(Key))}] }]}. filter_on_change_type_test_() -> - CreatedEventFilter = #kevf_tree{path = [foo], - props = #{on_actions => [create]}}, - UpdatedEventFilter = #kevf_tree{path = [foo], - props = #{on_actions => [update]}}, - DeletedEventFilter = #kevf_tree{path = [foo], - props = #{on_actions => [delete]}}, + CreatedEventFilter = khepri_evf:tree([foo], #{on_actions => [create]}), + UpdatedEventFilter = khepri_evf:tree([foo], #{on_actions => [update]}), + DeletedEventFilter = khepri_evf:tree([foo], #{on_actions => [delete]}), StoredProcPath = [sproc], CreatedKey = {?FUNCTION_NAME, created}, UpdatedKey = {?FUNCTION_NAME, updated}, @@ -312,31 +299,28 @@ filter_on_change_type_test_() -> [{"Storing a procedure for `created` change", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath ++ [created], - #kpayload_sproc{ - sproc = make_sproc(self(), CreatedKey)}))}, + make_sproc(self(), CreatedKey)))}, {"Storing a procedure for `updated` change", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath ++ [updated], - #kpayload_sproc{ - sproc = make_sproc(self(), UpdatedKey)}))}, + make_sproc(self(), UpdatedKey)))}, {"Storing a procedure for `deleted` change", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath ++ [deleted], - #kpayload_sproc{ - sproc = make_sproc(self(), DeletedKey)}))}, + make_sproc(self(), DeletedKey)))}, {"Registering a `created` trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, created, CreatedEventFilter, @@ -345,7 +329,7 @@ filter_on_change_type_test_() -> {"Registering a `updated` trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, updated, UpdatedEventFilter, @@ -354,7 +338,7 @@ filter_on_change_type_test_() -> {"Registering a `deleted` trigger", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, deleted, DeletedEventFilter, @@ -363,8 +347,8 @@ filter_on_change_type_test_() -> {"Creating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value1}))}, + khepri:put( + ?FUNCTION_NAME, [foo], value1))}, {"Checking the `created` procedure was executed", ?_assertEqual(executed, receive_sproc_msg(CreatedKey))}, @@ -376,8 +360,8 @@ filter_on_change_type_test_() -> {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = value2}))}, + khepri:put( + ?FUNCTION_NAME, [foo], value2))}, {"Checking the `created` procedure was not executed", ?_assertEqual(timeout, receive_sproc_msg(CreatedKey))}, @@ -389,7 +373,7 @@ filter_on_change_type_test_() -> {"Deleting a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:delete( + khepri:delete( ?FUNCTION_NAME, [foo]))}, {"Checking the `created` procedure was not executed", @@ -401,7 +385,7 @@ filter_on_change_type_test_() -> }]}. a_buggy_sproc_does_not_crash_state_machine_test_() -> - EventFilter = #kevf_tree{path = [foo]}, + EventFilter = khepri_evf:tree([foo]), StoredProcPath = [sproc], Key = ?FUNCTION_NAME, {setup, @@ -411,23 +395,21 @@ a_buggy_sproc_does_not_crash_state_machine_test_() -> [{"Storing a working procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath ++ [good], - #kpayload_sproc{ - sproc = make_sproc(self(), Key)}))}, + make_sproc(self(), Key)))}, {"Storing a failing procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( + khepri:put( ?FUNCTION_NAME, StoredProcPath ++ [bad], - #kpayload_sproc{ - sproc = fun(_Props) -> throw("Expected crash") end}))}, + fun(_Props) -> throw("Expected crash") end))}, {"Registering trigger 1", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, good, EventFilter, @@ -436,17 +418,16 @@ a_buggy_sproc_does_not_crash_state_machine_test_() -> {"Registering trigger 2", ?_assertEqual( ok, - khepri_machine:register_trigger( + khepri:register_trigger( ?FUNCTION_NAME, bad, - EventFilter#kevf_tree{props = #{priority => 10}}, + khepri_evf:set_priority(EventFilter, 10), StoredProcPath ++ [bad]))}, {"Updating a node; should trigger the procedure", ?_assertMatch( {ok, _}, - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = 1}))}, + khepri:put(?FUNCTION_NAME, [foo], 1))}, {"Checking the procedure was executed", ?_assertEqual(executed, receive_sproc_msg(Key))}, @@ -456,8 +437,7 @@ a_buggy_sproc_does_not_crash_state_machine_test_() -> {ok, _}, begin timer:sleep(2000), - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = 2}) + khepri:put(?FUNCTION_NAME, [foo], 2) end)}, {"Checking the procedure was executed", @@ -468,8 +448,7 @@ a_buggy_sproc_does_not_crash_state_machine_test_() -> {ok, _}, begin timer:sleep(2000), - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = 3}) + khepri:put(?FUNCTION_NAME, [foo], 3) end)}, {"Checking the procedure was executed", @@ -480,8 +459,7 @@ a_buggy_sproc_does_not_crash_state_machine_test_() -> {ok, _}, begin timer:sleep(2000), - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = 4}) + khepri:put(?FUNCTION_NAME, [foo], 4) end)}, {"Checking the procedure was executed", @@ -492,8 +470,7 @@ a_buggy_sproc_does_not_crash_state_machine_test_() -> {ok, _}, begin timer:sleep(2000), - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = 5}) + khepri:put(?FUNCTION_NAME, [foo], 5) end)}, {"Checking the procedure was executed", @@ -504,8 +481,7 @@ a_buggy_sproc_does_not_crash_state_machine_test_() -> {ok, _}, begin timer:sleep(2000), - khepri_machine:put( - ?FUNCTION_NAME, [foo], #kpayload_data{data = 6}) + khepri:put(?FUNCTION_NAME, [foo], 6) end)}, {"Checking the procedure was executed", diff --git a/test/tx_funs.erl b/test/tx_funs.erl index 3a019d08..7fb8c51f 100644 --- a/test/tx_funs.erl +++ b/test/tx_funs.erl @@ -10,6 +10,7 @@ -include_lib("eunit/include/eunit.hrl"). -include("include/khepri.hrl"). +-include("src/khepri_fun.hrl"). -include("src/internal.hrl"). -include("src/khepri_machine.hrl"). @@ -49,8 +50,8 @@ noop_ok_test() -> allowed_khepri_tx_api_test() -> ?assertStandaloneFun( begin - _ = khepri_tx:put([foo], #kpayload_data{data = value}), - _ = khepri_tx:put([foo], #kpayload_data{data = value}, #{}), + _ = khepri_tx:put([foo], khepri_payload:data(value)), + _ = khepri_tx:put([foo], khepri_payload:data(value), #{}), _ = khepri_tx:get([foo]), _ = khepri_tx:get([foo], #{}), _ = khepri_tx:exists([foo]), @@ -831,7 +832,7 @@ when_readwrite_mode_is_true_test() -> ?assert( is_record(khepri_tx:to_standalone_fun( fun() -> - khepri_tx:put([foo], #kpayload_data{data = value}) + khepri_tx:put([foo], khepri_payload:data(value)) end, rw), standalone_fun)), @@ -847,7 +848,7 @@ when_readwrite_mode_is_true_test() -> {invalid_tx_fun, {call_denied, {self, 0}}}, khepri_tx:to_standalone_fun( fun() -> - _ = khepri_tx:put([foo], #kpayload_data{data = value}), + _ = khepri_tx:put([foo], khepri_payload:data(value)), self() ! message end, rw)), @@ -883,7 +884,7 @@ when_readwrite_mode_is_false_test() -> is_function(khepri_tx:to_standalone_fun( fun() -> khepri_tx:put( - [foo], #kpayload_data{data = value}) + [foo], khepri_payload:data(value)) end, ro), 0)), @@ -901,7 +902,7 @@ when_readwrite_mode_is_false_test() -> is_function(khepri_tx:to_standalone_fun( fun() -> _ = khepri_tx:put( - [foo], #kpayload_data{data = value}), + [foo], khepri_payload:data(value)), self() ! message end, ro), @@ -935,7 +936,7 @@ when_readwrite_mode_is_auto_test() -> ?assert( is_record(khepri_tx:to_standalone_fun( fun() -> - khepri_tx:put([foo], #kpayload_data{data = value}) + khepri_tx:put([foo], khepri_payload:data(value)) end, auto), standalone_fun)), @@ -951,7 +952,7 @@ when_readwrite_mode_is_auto_test() -> {invalid_tx_fun, {call_denied, {self, 0}}}, khepri_tx:to_standalone_fun( fun() -> - _ = khepri_tx:put([foo], #kpayload_data{data = value}), + _ = khepri_tx:put([foo], khepri_payload:data(value)), self() ! message end, auto)),- No payload at all ({@link no_payload()}
+%%- Data payload used to store any Erlang term ({@link data()})
+%%- Stored procedure payload used to store functions ({@link sproc()})
+%%