diff --git a/src/khepri.erl b/src/khepri.erl index 3bc706c2..001acf33 100644 --- a/src/khepri.erl +++ b/src/khepri.erl @@ -430,8 +430,7 @@ %% %% `undefined' is returned if a tree node has no payload attached to it. --type async_ret() :: khepri_adv:single_result() | - khepri_adv:many_results() | +-type async_ret() :: khepri_adv:many_results() | khepri_tx:tx_fun_result() | khepri:error({not_leader, ra:server_id()}). %% The value returned from of a command function which was executed @@ -446,11 +445,10 @@ %% commands which were applied, or `{error, {not_leader, LeaderId}}' if the %% commands could not be applied since they were sent to a non-leader member. %% -%% Note that when commands are successfully applied, the return values are in -%% the {@link khepri_adv} formats - {@link khepri_adv:single_result()} or -%% {@link khepri_adv:many_results()} - rather than {@link -%% khepri:minimal_ret()}, even if the command was sent using a function from -%% the {@link khepri} API such as {@link khepri:put/4}. +%% Note that when commands are successfully applied, the return values are +%% {@link khepri_adv:many_results()} rather than {@link khepri:minimal_ret()}, +%% even if the command was sent using a function from the {@link khepri} API +%% such as {@link khepri:put/4}. %% %% See {@link khepri:handle_async_ret/2}. @@ -778,10 +776,12 @@ get(PathPattern, Options) when is_map(Options) -> get(StoreId, PathPattern, Options) -> case khepri_adv:get(StoreId, PathPattern, Options) of - {ok, #{data := Data}} -> {ok, Data}; - {ok, #{sproc := StandaloneFun}} -> {ok, StandaloneFun}; - {ok, _} -> {ok, undefined}; - Error -> Error + {ok, NodePropsMap} -> + NodeProps = khepri_utils:get_single_node_props(NodePropsMap), + Payload = khepri_utils:node_props_to_payload(NodeProps, undefined), + {ok, Payload}; + {error, _} = Error -> + Error end. %% ------------------------------------------------------------------- @@ -892,11 +892,14 @@ get_or(PathPattern, Default, Options) when is_map(Options) -> get_or(StoreId, PathPattern, Default, Options) -> case khepri_adv:get(StoreId, PathPattern, Options) of - {ok, #{data := Data}} -> {ok, Data}; - {ok, #{sproc := StandaloneFun}} -> {ok, StandaloneFun}; - {ok, _} -> {ok, Default}; - {error, ?khepri_error(node_not_found, _)} -> {ok, Default}; - Error -> Error + {ok, NodePropsMap} -> + NodeProps = khepri_utils:get_single_node_props(NodePropsMap), + Payload = khepri_utils:node_props_to_payload(NodeProps, Default), + {ok, Payload}; + {error, ?khepri_error(node_not_found, _)} -> + {ok, Default}; + {error, _} = Error -> + Error end. %% ------------------------------------------------------------------- @@ -1889,7 +1892,7 @@ filter(StoreId, PathPattern, Pred, Options) when is_function(Pred, 2) -> FoldFun = fun(Path, NodeProps, Acc) -> case Pred(Path, NodeProps) of true -> - Payload = node_props_to_payload( + Payload = khepri_utils:node_props_to_payload( NodeProps, undefined), Acc#{Path => Payload}; false -> @@ -1898,10 +1901,6 @@ filter(StoreId, PathPattern, Pred, Options) when is_function(Pred, 2) -> end, fold(StoreId, PathPattern, FoldFun, #{}, Options). -node_props_to_payload(#{data := Data}, _Default) -> Data; -node_props_to_payload(#{sproc := StandaloneFun}, _Default) -> StandaloneFun; -node_props_to_payload(_NodeProps, Default) -> Default. - %% ------------------------------------------------------------------- %% run_sproc(). %% ------------------------------------------------------------------- @@ -1992,14 +1991,18 @@ run_sproc(PathPattern, Args, Options) when is_map(Options) -> run_sproc(StoreId, PathPattern, Args, Options) -> case khepri_adv:get(StoreId, PathPattern, Options) of - {ok, #{sproc := StandaloneFun}} -> - khepri_sproc:run(StandaloneFun, Args); - {ok, NodeProps} -> - throw(?khepri_exception( - denied_execution_of_non_sproc_node, - #{path => PathPattern, - args => Args, - node_props => NodeProps})); + {ok, NodePropsMap} -> + NodeProps = khepri_utils:get_single_node_props(NodePropsMap), + case NodeProps of + #{sproc := StandaloneFun} -> + khepri_sproc:run(StandaloneFun, Args); + _ -> + throw(?khepri_exception( + denied_execution_of_non_sproc_node, + #{path => PathPattern, + args => Args, + node_props => NodeProps})) + end; {error, Reason} -> throw(?khepri_error( failed_to_get_sproc, diff --git a/src/khepri_adv.erl b/src/khepri_adv.erl index 0a75e2da..d98d9080 100644 --- a/src/khepri_adv.erl +++ b/src/khepri_adv.erl @@ -52,16 +52,10 @@ %% Structure used to return a map of nodes and their associated properties, %% payload and child nodes. --type single_result() :: khepri:ok(khepri:node_props() | #{}) | - khepri:error(). -%% Return value of a query or synchronous command targeting one specific tree -%% node. - -type many_results() :: khepri_machine:common_ret(). %% Return value of a query or synchronous command targeting many tree nodes. -export_type([node_props_map/0, - single_result/0, many_results/0]). %% ------------------------------------------------------------------- @@ -70,7 +64,7 @@ -spec get(PathPattern) -> Ret when PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Returns the properties and payload of the tree node pointed to by the %% given path pattern. %% @@ -89,11 +83,11 @@ get(PathPattern) -> (StoreId, PathPattern) -> Ret when StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(); + Ret :: khepri_adv:many_results(); (PathPattern, Options) -> Ret when PathPattern :: khepri_path:pattern(), Options :: khepri:query_options() | khepri:tree_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Returns the properties and payload of the tree node pointed to by the %% given path pattern. %% @@ -118,7 +112,7 @@ get(PathPattern, Options) when is_map(Options) -> StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern(), Options :: khepri:query_options() | khepri:tree_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Returns the properties and payload of the tree node pointed to by the %% given path pattern. %% @@ -161,15 +155,14 @@ get(PathPattern, Options) when is_map(Options) -> %% @param PathPattern the path (or path pattern) to the tree node to get. %% @param Options query options. %% -%% @returns an `{ok, NodeProps}' tuple or an `{error, Reason}' tuple. +%% @returns an `{ok, NodePropsMap}' tuple or an `{error, Reason}' tuple. %% %% @see get_many/3. %% @see khepri:get/3. get(StoreId, PathPattern, Options) -> Options1 = Options#{expect_specific_node => true}, - Ret = get_many(StoreId, PathPattern, Options1), - ?common_ret_to_single_result_ret(Ret). + get_many(StoreId, PathPattern, Options1). %% ------------------------------------------------------------------- %% get_many(). @@ -274,7 +267,7 @@ get_many(StoreId, PathPattern, Options) -> PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), Ret :: khepri:minimal_ret() | - khepri_adv:single_result(). + khepri_adv:many_results(). %% @doc Sets the payload of the tree node pointed to by the given path %% pattern. %% @@ -294,7 +287,7 @@ put(PathPattern, Data) -> PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), Ret :: khepri:minimal_ret() | - khepri_adv:single_result(). + khepri_adv:many_results(). %% @doc Sets the payload of the tree node pointed to by the given path %% pattern. %% @@ -314,7 +307,7 @@ put(StoreId, PathPattern, Data) -> khepri:tree_options() | khepri:put_options(), Ret :: khepri:minimal_ret() | - khepri_adv:single_result() | + khepri_adv:many_results() | khepri_machine:async_ret(). %% @doc Sets the payload of the tree node pointed to by the given path %% pattern. @@ -385,8 +378,8 @@ put(StoreId, PathPattern, Data) -> %% khepri_payload:payload()} structure. %% @param Options command options. %% -%% @returns in the case of a synchronous call, an `{ok, NodeProps}' tuple or -%% an `{error, Reason}' tuple; in the case of an asynchronous call, always +%% @returns in the case of a synchronous call, an `{ok, NodePropsMap}' tuple +%% or an `{error, Reason}' tuple; in the case of an asynchronous call, always %% `ok' (the actual return value may be sent by a message if a correlation ID %% was specified). %% @@ -397,8 +390,7 @@ put(StoreId, PathPattern, Data) -> put(StoreId, PathPattern, Data, Options) -> Options1 = Options#{expect_specific_node => true}, - Ret = put_many(StoreId, PathPattern, Data, Options1), - ?common_ret_to_single_result_ret(Ret). + put_many(StoreId, PathPattern, Data, Options1). %% ------------------------------------------------------------------- %% put_many(). @@ -527,7 +519,7 @@ put_many(StoreId, PathPattern, Data, Options) -> -spec create(PathPattern, Data) -> Ret when PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Creates a tree node with the given payload. %% %% Calling this function is the same as calling `create(StoreId, PathPattern, @@ -545,7 +537,7 @@ create(PathPattern, Data) -> StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Creates a tree node with the given payload. %% %% Calling this function is the same as calling `create(StoreId, PathPattern, @@ -563,7 +555,7 @@ create(StoreId, PathPattern, Data) -> Options :: khepri:command_options() | khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result() | khepri_machine:async_ret(). + Ret :: khepri_adv:many_results() | khepri_machine:async_ret(). %% @doc Creates a tree node with the given payload. %% %% The behavior is the same as {@link put/4} except that if the tree node @@ -579,8 +571,8 @@ create(StoreId, PathPattern, Data) -> %% khepri_payload:payload()} structure. %% @param Options command options. %% -%% @returns in the case of a synchronous call, an `{ok, NodeProps}' tuple or -%% an `{error, Reason}' tuple; in the case of an asynchronous call, always +%% @returns in the case of a synchronous call, an `{ok, NodePropsMap}' tuple +%% or an `{error, Reason}' tuple; in the case of an asynchronous call, always %% `ok' (the actual return value may be sent by a message if a correlation ID %% was specified). %% @@ -593,13 +585,7 @@ create(StoreId, PathPattern, Data, Options) -> PathPattern2 = khepri_path:combine_with_conditions( PathPattern1, [#if_node_exists{exists = false}]), Options1 = Options#{expect_specific_node => true}, - case do_put(StoreId, PathPattern2, Data, Options1) of - {ok, NodePropsMaps} -> - [NodeProps] = maps:values(NodePropsMaps), - {ok, NodeProps}; - Error -> - Error - end. + do_put(StoreId, PathPattern2, Data, Options1). %% ------------------------------------------------------------------- %% update(). @@ -608,7 +594,7 @@ create(StoreId, PathPattern, Data, Options) -> -spec update(PathPattern, Data) -> Ret when PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload. %% %% Calling this function is the same as calling `update(StoreId, PathPattern, @@ -626,7 +612,7 @@ update(PathPattern, Data) -> StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload. %% %% Calling this function is the same as calling `update(StoreId, PathPattern, @@ -644,7 +630,7 @@ update(StoreId, PathPattern, Data) -> Options :: khepri:command_options() | khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result() | khepri_machine:async_ret(). + Ret :: khepri_adv:many_results() | khepri_machine:async_ret(). %% @doc Updates an existing tree node with the given payload. %% %% The behavior is the same as {@link put/4} except that if the tree node @@ -661,8 +647,8 @@ update(StoreId, PathPattern, Data) -> %% @param Extra extra options such as `keep_while' conditions. %% @param Options command options. %% -%% @returns in the case of a synchronous call, an `{ok, NodeProps}' tuple or -%% an `{error, Reason}' tuple; in the case of an asynchronous call, always +%% @returns in the case of a synchronous call, an `{ok, NodePropsMap}' tuple +%% or an `{error, Reason}' tuple; in the case of an asynchronous call, always %% `ok' (the actual return value may be sent by a message if a correlation ID %% was specified). %% @@ -675,8 +661,7 @@ update(StoreId, PathPattern, Data, Options) -> PathPattern2 = khepri_path:combine_with_conditions( PathPattern1, [#if_node_exists{exists = true}]), Options1 = Options#{expect_specific_node => true}, - Ret = do_put(StoreId, PathPattern2, Data, Options1), - ?common_ret_to_single_result_ret(Ret). + do_put(StoreId, PathPattern2, Data, Options1). %% ------------------------------------------------------------------- %% compare_and_swap(). @@ -686,7 +671,7 @@ update(StoreId, PathPattern, Data, Options) -> PathPattern :: khepri_path:pattern(), DataPattern :: ets:match_pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload only if its data %% matches the given pattern. %% @@ -706,7 +691,7 @@ compare_and_swap(PathPattern, DataPattern, Data) -> PathPattern :: khepri_path:pattern(), DataPattern :: ets:match_pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload only if its data %% matches the given pattern. %% @@ -727,7 +712,7 @@ compare_and_swap(StoreId, PathPattern, DataPattern, Data) -> Options :: khepri:command_options() | khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result() | khepri_machine:async_ret(). + Ret :: khepri_adv:many_results() | khepri_machine:async_ret(). %% @doc Updates an existing tree node with the given payload only if its data %% matches the given pattern. %% @@ -745,8 +730,8 @@ compare_and_swap(StoreId, PathPattern, DataPattern, Data) -> %% @param Extra extra options such as `keep_while' conditions. %% @param Options command options. %% -%% @returns in the case of a synchronous call, an `{ok, NodeProps}' tuple or -%% an `{error, Reason}' tuple; in the case of an asynchronous call, always +%% @returns in the case of a synchronous call, an `{ok, NodePropsMap}' tuple +%% or an `{error, Reason}' tuple; in the case of an asynchronous call, always %% `ok' (the actual return value may be sent by a message if a correlation ID %% was specified). %% @@ -758,8 +743,7 @@ compare_and_swap(StoreId, PathPattern, DataPattern, Data, Options) -> PathPattern2 = khepri_path:combine_with_conditions( PathPattern1, [#if_data_matches{pattern = DataPattern}]), Options1 = Options#{expect_specific_node => true}, - Ret = do_put(StoreId, PathPattern2, Data, Options1), - ?common_ret_to_single_result_ret(Ret). + do_put(StoreId, PathPattern2, Data, Options1). %% ------------------------------------------------------------------- %% do_put(). @@ -787,7 +771,7 @@ do_put(StoreId, PathPattern, Payload, Options) -> -spec delete(PathPattern) -> Ret when PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the tree node pointed to by the given path pattern. %% %% Calling this function is the same as calling `delete(StoreId, PathPattern)' @@ -805,11 +789,11 @@ delete(PathPattern) -> (StoreId, PathPattern) -> Ret when StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(); + Ret :: khepri_adv:many_results(); (PathPattern, Options) -> Ret when PathPattern :: khepri_path:pattern(), Options :: khepri:command_options() | khepri:tree_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the tree node pointed to by the given path pattern. %% %% This function accepts the following two forms: @@ -833,7 +817,7 @@ delete(PathPattern, Options) when is_map(Options) -> StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern(), Options :: khepri:command_options() | khepri:tree_options(), - Ret :: khepri_adv:single_result() | khepri_machine:async_ret(). + Ret :: khepri_adv:many_results() | khepri_machine:async_ret(). %% @doc Deletes the tree node pointed to by the given path pattern. %% %% The `PathPattern' can be provided as a native path pattern (a list of tree @@ -865,8 +849,8 @@ delete(PathPattern, Options) when is_map(Options) -> %% @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 call, an `{ok, NodeProps}' tuple or -%% an `{error, Reason}' tuple; in the case of an asynchronous call, always +%% @returns in the case of a synchronous call, an `{ok, NodePropsMap}' tuple +%% or an `{error, Reason}' tuple; in the case of an asynchronous call, always %% `ok' (the actual return value may be sent by a message if a correlation ID %% was specified). %% @@ -876,19 +860,7 @@ delete(PathPattern, Options) when is_map(Options) -> delete(StoreId, PathPattern, Options) -> %% TODO: Not handled by khepri_machine:delete/3... Options1 = Options#{expect_specific_node => true}, - case khepri_machine:delete(StoreId, PathPattern, Options1) of - {ok, NodePropsMap} -> - %% It's ok to delete a non-existing tree node. The returned result - %% will be an empty map, in which case we return `#{}' as the - %% "node properties". - NodeProps = case maps:values(NodePropsMap) of - [NP] -> NP; - [] -> #{} - end, - {ok, NodeProps}; - Error -> - Error - end. + khepri_machine:delete(StoreId, PathPattern, Options1). %% ------------------------------------------------------------------- %% delete_many(). @@ -988,7 +960,7 @@ delete_many(StoreId, PathPattern, Options) -> -spec clear_payload(PathPattern) -> Ret when PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the payload of the tree node pointed to by the given path %% pattern. %% @@ -1006,7 +978,7 @@ clear_payload(PathPattern) -> -spec clear_payload(StoreId, PathPattern) -> Ret when StoreId :: khepri:store_id(), PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the payload of the tree node pointed to by the given path %% pattern. %% @@ -1024,7 +996,7 @@ clear_payload(StoreId, PathPattern) -> Options :: khepri:command_options() | khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result() | khepri_machine:async_ret(). + Ret :: khepri_adv:many_results() | khepri_machine:async_ret(). %% @doc Deletes the payload of the tree node pointed to by the given path %% pattern. %% @@ -1036,8 +1008,8 @@ clear_payload(StoreId, PathPattern) -> %% @param Extra extra options such as `keep_while' conditions. %% @param Options command options. %% -%% @returns in the case of a synchronous call, an `{ok, NodeProps}' tuple or -%% an `{error, Reason}' tuple; in the case of an asynchronous call, always +%% @returns in the case of a synchronous call, an `{ok, NodePropsMap}' tuple +%% or an `{error, Reason}' tuple; in the case of an asynchronous call, always %% `ok' (the actual return value may be sent by a message if a correlation ID %% was specified). %% diff --git a/src/khepri_projection.erl b/src/khepri_projection.erl index 7a64f0d6..a05def99 100644 --- a/src/khepri_projection.erl +++ b/src/khepri_projection.erl @@ -189,7 +189,8 @@ new(Name, ProjectionFun, Options) {_CustomFunOptions, _EtsOptions} = Value -> Value end, - EtsOptions1 = maps:fold(fun to_ets_options/3, ?DEFAULT_ETS_OPTS, EtsOptions), + EtsOptions1 = maps:fold( + fun to_ets_options/3, ?DEFAULT_ETS_OPTS, EtsOptions), ShouldProcessFunction = if is_function(ProjectionFun, 2) -> diff --git a/src/khepri_ret.hrl b/src/khepri_ret.hrl index 84626713..88f6bfd8 100644 --- a/src/khepri_ret.hrl +++ b/src/khepri_ret.hrl @@ -6,17 +6,6 @@ %% refers to Broadcom Inc. and/or its subsidiaries. %% --define(common_ret_to_single_result_ret(__Ret), - case (__Ret) of - {ok, __NodePropsMap} -> - [__NodeProps] = maps:values(__NodePropsMap), - {ok, __NodeProps}; - {error, ?khepri_exception(_, _) = __Exception} -> - ?khepri_misuse(__Exception); - __Error -> - __Error - end). - -define(result_ret_to_minimal_ret(__Ret), case (__Ret) of {ok, _} -> ok; diff --git a/src/khepri_tx.erl b/src/khepri_tx.erl index 6eda0245..a9a4d83c 100644 --- a/src/khepri_tx.erl +++ b/src/khepri_tx.erl @@ -180,10 +180,12 @@ get(PathPattern) -> get(PathPattern, Options) -> case khepri_tx_adv:get(PathPattern, Options) of - {ok, #{data := Data}} -> {ok, Data}; - {ok, #{sproc := StandaloneFun}} -> {ok, StandaloneFun}; - {ok, _} -> {ok, undefined}; - Error -> Error + {ok, NodePropsMap} -> + NodeProps = khepri_utils:get_single_node_props(NodePropsMap), + Payload = khepri_utils:node_props_to_payload(NodeProps, undefined), + {ok, Payload}; + {error, _} = Error -> + Error end. %% ------------------------------------------------------------------- @@ -220,11 +222,14 @@ get_or(PathPattern, Default) -> get_or(PathPattern, Default, Options) -> case khepri_tx_adv:get(PathPattern, Options) of - {ok, #{data := Data}} -> {ok, Data}; - {ok, #{sproc := StandaloneFun}} -> {ok, StandaloneFun}; - {ok, _} -> {ok, Default}; - {error, ?khepri_error(node_not_found, _)} -> {ok, Default}; - Error -> Error + {ok, NodePropsMap} -> + NodeProps = khepri_utils:get_single_node_props(NodePropsMap), + Payload = khepri_utils:node_props_to_payload(NodeProps, Default), + {ok, Payload}; + {error, ?khepri_error(node_not_found, _)} -> + {ok, Default}; + {error, _} = Error -> + Error end. %% ------------------------------------------------------------------- diff --git a/src/khepri_tx_adv.erl b/src/khepri_tx_adv.erl index 0f3fc9dd..5ab4efa0 100644 --- a/src/khepri_tx_adv.erl +++ b/src/khepri_tx_adv.erl @@ -64,7 +64,7 @@ -spec get(PathPattern) -> Ret when PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Returns the payload of the tree node pointed to by the given path %% pattern. %% @@ -79,7 +79,7 @@ get(PathPattern) -> -spec get(PathPattern, Options) -> Ret when PathPattern :: khepri_path:pattern(), Options :: khepri:tree_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Returns the payload of the tree node pointed to by the given path %% pattern. %% @@ -90,8 +90,7 @@ get(PathPattern) -> get(PathPattern, Options) -> Options1 = Options#{expect_specific_node => true}, - Ret = get_many(PathPattern, Options1), - ?common_ret_to_single_result_ret(Ret). + get_many(PathPattern, Options1). %% ------------------------------------------------------------------- %% get_many(). @@ -148,7 +147,7 @@ do_get_many(PathPattern, Fun, Acc, Options) -> -spec put(PathPattern, Data) -> Ret when PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Sets the payload of the tree node pointed to by the given path %% pattern. %% @@ -164,7 +163,7 @@ put(PathPattern, Data) -> PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), Options :: khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Sets the payload of the tree node pointed to by the given path %% pattern. %% @@ -175,8 +174,7 @@ put(PathPattern, Data) -> put(PathPattern, Data, Options) -> Options1 = Options#{expect_specific_node => true}, - Ret = put_many(PathPattern, Data, Options1), - ?common_ret_to_single_result_ret(Ret). + put_many(PathPattern, Data, Options1). %% ------------------------------------------------------------------- %% put_many(). @@ -231,7 +229,7 @@ put_many(PathPattern, Data, Options) -> -spec create(PathPattern, Data) -> Ret when PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Creates a tree node with the given payload. %% %% This is the same as {@link khepri_adv:create/3} but inside the context of a @@ -246,7 +244,7 @@ create(PathPattern, Data) -> PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), Options :: khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Creates a tree node with the given payload. %% %% This is the same as {@link khepri_adv:create/4} but inside the context of a @@ -259,8 +257,7 @@ create(PathPattern, Data, Options) -> PathPattern2 = khepri_path:combine_with_conditions( PathPattern1, [#if_node_exists{exists = false}]), Options1 = Options#{expect_specific_node => true}, - Ret = put_many(PathPattern2, Data, Options1), - ?common_ret_to_single_result_ret(Ret). + put_many(PathPattern2, Data, Options1). %% ------------------------------------------------------------------- %% update(). @@ -269,7 +266,7 @@ create(PathPattern, Data, Options) -> -spec update(PathPattern, Data) -> Ret when PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload. %% %% This is the same as {@link khepri_adv:update/3} but inside the context of a @@ -284,7 +281,7 @@ update(PathPattern, Data) -> PathPattern :: khepri_path:pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), Options :: khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload. %% %% This is the same as {@link khepri_adv:update/4} but inside the context of a @@ -297,8 +294,7 @@ update(PathPattern, Data, Options) -> PathPattern2 = khepri_path:combine_with_conditions( PathPattern1, [#if_node_exists{exists = true}]), Options1 = Options#{expect_specific_node => true}, - Ret = put_many(PathPattern2, Data, Options1), - ?common_ret_to_single_result_ret(Ret). + put_many(PathPattern2, Data, Options1). %% ------------------------------------------------------------------- %% compare_and_swap(). @@ -308,7 +304,7 @@ update(PathPattern, Data, Options) -> PathPattern :: khepri_path:pattern(), DataPattern :: ets:match_pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload only if its data %% matches the given pattern. %% @@ -326,7 +322,7 @@ compare_and_swap(PathPattern, DataPattern, Data) -> DataPattern :: ets:match_pattern(), Data :: khepri_payload:payload() | khepri:data() | fun(), Options :: khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Updates an existing tree node with the given payload only if its data %% matches the given pattern. %% @@ -340,8 +336,7 @@ compare_and_swap(PathPattern, DataPattern, Data, Options) -> PathPattern2 = khepri_path:combine_with_conditions( PathPattern1, [#if_data_matches{pattern = DataPattern}]), Options1 = Options#{expect_specific_node => true}, - Ret = put_many(PathPattern2, Data, Options1), - ?common_ret_to_single_result_ret(Ret). + put_many(PathPattern2, Data, Options1). %% ------------------------------------------------------------------- %% delete(). @@ -349,7 +344,7 @@ compare_and_swap(PathPattern, DataPattern, Data, Options) -> -spec delete(PathPattern) -> Ret when PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the tree node pointed to by the given path pattern. %% %% This is the same as {@link khepri_adv:delete/2} but inside the context of a @@ -363,7 +358,7 @@ delete(PathPattern) -> -spec delete(PathPattern, Options) -> Ret when PathPattern :: khepri_path:pattern(), Options :: khepri:tree_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the tree node pointed to by the given path pattern. %% %% This is the same as {@link khepri_adv:delete/3} but inside the context of a @@ -373,19 +368,7 @@ delete(PathPattern) -> delete(PathPattern, Options) -> Options1 = Options#{expect_specific_node => true}, - case delete_many(PathPattern, Options1) of - {ok, NodePropsMap} -> - %% It's ok to delete a non-existing tree node. The returned result - %% will be an empty map, in which case we return `#{}' as the - %% "node properties". - NodeProps = case maps:values(NodePropsMap) of - [NP] -> NP; - [] -> #{} - end, - {ok, NodeProps}; - Error -> - Error - end. + delete_many(PathPattern, Options1). %% ------------------------------------------------------------------- %% delete_many(). @@ -434,7 +417,7 @@ delete_many(PathPattern, Options) -> -spec clear_payload(PathPattern) -> Ret when PathPattern :: khepri_path:pattern(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the payload of the tree node pointed to by the given path %% pattern. %% @@ -449,7 +432,7 @@ clear_payload(PathPattern) -> -spec clear_payload(PathPattern, Options) -> Ret when PathPattern :: khepri_path:pattern(), Options :: khepri:tree_options() | khepri:put_options(), - Ret :: khepri_adv:single_result(). + Ret :: khepri_adv:many_results(). %% @doc Deletes the payload of the tree node pointed to by the given path %% pattern. %% diff --git a/src/khepri_utils.erl b/src/khepri_utils.erl index fb0f978d..511b2a7c 100644 --- a/src/khepri_utils.erl +++ b/src/khepri_utils.erl @@ -20,6 +20,7 @@ sleep/2, is_ra_server_alive/1, + get_single_node_props/1, node_props_to_payload/2, flat_struct_to_tree/1, @@ -90,6 +91,10 @@ sleep(Time, Timeout) when Time > Timeout -> is_ra_server_alive({RegName, Node}) when Node =:= node() -> is_pid(erlang:whereis(RegName)). +get_single_node_props(NodePropsMap) -> + [NodeProps] = maps:values(NodePropsMap), + NodeProps. + node_props_to_payload(#{data := Data}, _Default) -> Data; node_props_to_payload(#{sproc := StandaloneFun}, _Default) -> StandaloneFun; node_props_to_payload(_NodeProps, Default) -> Default. diff --git a/test/advanced_delete.erl b/test/advanced_delete.erl index 22208306..76d187e2 100644 --- a/test/advanced_delete.erl +++ b/test/advanced_delete.erl @@ -32,11 +32,11 @@ delete_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:delete(?FUNCTION_NAME, [foo])), ?_assertEqual( {error, ?khepri_error(node_not_found, #{node_name => foo, @@ -73,7 +73,7 @@ delete_many_on_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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, #{[foo] => #{data => foo_value, @@ -91,15 +91,15 @@ delete_many_on_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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, #{}}, khepri_adv:delete_many( ?FUNCTION_NAME, [#if_name_matches{regex = "bar"}])), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_payload_from_non_existing_node_test_() -> @@ -120,14 +120,14 @@ 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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 2}}}, khepri_adv:clear_payload(?FUNCTION_NAME, [foo])), ?_assertEqual( - {ok, #{payload_version => 2}}, + {ok, #{[foo] => #{payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_payload_with_keep_while_test_() -> @@ -135,15 +135,15 @@ clear_payload_with_keep_while_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 2}}}, khepri_adv:clear_payload( ?FUNCTION_NAME, [foo], #{keep_while => #{}})), ?_assertEqual( - {ok, #{payload_version => 2}}, + {ok, #{[foo] => #{payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_payload_with_options_test_() -> @@ -151,14 +151,14 @@ clear_payload_with_options_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 2}}}, khepri_adv:clear_payload(?FUNCTION_NAME, [foo], #{async => false})), ?_assertEqual( - {ok, #{payload_version => 2}}, + {ok, #{[foo] => #{payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_many_payloads_from_non_existing_node_test_() -> @@ -179,10 +179,10 @@ clear_many_payloads_from_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo1] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo1], foo1_value)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo2, bar] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo2, bar], bar_value)), ?_assertEqual( {ok, #{[foo1] => #{data => foo1_value, diff --git a/test/advanced_get.erl b/test/advanced_get.erl index 5c97fd70..77e89a3e 100644 --- a/test/advanced_get.erl +++ b/test/advanced_get.erl @@ -31,11 +31,11 @@ get_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. get_existing_node_with_sproc_test_() -> @@ -43,11 +43,12 @@ get_existing_node_with_sproc_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], fun() -> ok end)), ?_assertMatch( - {ok, #{sproc := Fun, - payload_version := 1}} when ?IS_HORUS_STANDALONE_FUN(Fun), + {ok, #{[foo] := #{sproc := Fun, + payload_version := 1}}} + when ?IS_HORUS_STANDALONE_FUN(Fun), khepri_adv:get(?FUNCTION_NAME, [foo]))]}. get_existing_node_with_no_payload_test_() -> @@ -55,10 +56,10 @@ get_existing_node_with_no_payload_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo, bar] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. invalid_get_call_test_() -> @@ -84,10 +85,10 @@ get_many_existing_nodes_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo, bar] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[baz] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, #{[foo] => #{payload_version => 1}, diff --git a/test/advanced_put.erl b/test/advanced_put.erl index b9f3d4ad..f251c09a 100644 --- a/test/advanced_put.erl +++ b/test/advanced_put.erl @@ -20,11 +20,11 @@ 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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. create_existing_node_test_() -> @@ -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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( {error, @@ -73,11 +73,11 @@ 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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:put(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. insert_existing_node_test_() -> @@ -85,19 +85,19 @@ insert_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( - {ok, #{data => foo_value1, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}, khepri_adv:put(?FUNCTION_NAME, [foo], foo_value2)), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:put(?FUNCTION_NAME, [foo], foo_value2, #{})), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. invalid_put_call_test_() -> @@ -115,10 +115,10 @@ insert_many_non_existing_nodes_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[a] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [a], ?NO_PAYLOAD)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[b] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [b], ?NO_PAYLOAD)), ?_assertEqual( {ok, #{[a, foo] => #{payload_version => 1}, @@ -138,10 +138,10 @@ insert_many_existing_nodes_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[a, foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [a, foo], foo_value_a)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[b, foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [b, foo], foo_value_b)), ?_assertEqual( {ok, #{[a, foo] => #{data => foo_value_a, @@ -196,15 +196,15 @@ update_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( - {ok, #{data => foo_value1, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}, khepri_adv:update(?FUNCTION_NAME, [foo], foo_value2)), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. invalid_update_call_test_() -> @@ -240,16 +240,16 @@ 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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( - {ok, #{data => foo_value1, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}, khepri_adv:compare_and_swap( ?FUNCTION_NAME, [foo], foo_value1, foo_value2)), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. compare_and_swap_mismatching_node_test_() -> @@ -257,7 +257,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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertMatch( {error, @@ -277,17 +277,17 @@ compare_and_swap_with_options_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( - {ok, #{data => foo_value1, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}, khepri_adv:compare_and_swap( ?FUNCTION_NAME, [foo], foo_value1, foo_value2, #{async => false})), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. invalid_compare_and_swap_call_test_() -> diff --git a/test/advanced_tx_delete.erl b/test/advanced_tx_delete.erl index 3c87f464..483c4bbe 100644 --- a/test/advanced_tx_delete.erl +++ b/test/advanced_tx_delete.erl @@ -37,7 +37,7 @@ delete_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), @@ -49,8 +49,8 @@ delete_existing_node_test_() -> end), ?_assertEqual( {ok, - {ok, #{data => foo_value, - payload_version => 1}}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}}, begin Fun = fun() -> khepri_tx_adv:delete([foo]) @@ -102,7 +102,7 @@ delete_many_on_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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), @@ -135,7 +135,7 @@ delete_many_on_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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, {ok, #{}}}, @@ -147,8 +147,8 @@ delete_many_on_existing_node_with_condition_false_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_payload_from_non_existing_node_test_() -> @@ -174,7 +174,7 @@ 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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), @@ -186,8 +186,8 @@ clear_payload_from_existing_node_test_() -> end), ?_assertEqual( {ok, - {ok, #{data => foo_value, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:clear_payload([foo]) @@ -195,7 +195,7 @@ clear_payload_from_existing_node_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{payload_version => 2}}, + {ok, #{[foo] => #{payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_payload_with_keep_while_test_() -> @@ -203,12 +203,12 @@ clear_payload_with_keep_while_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, - {ok, #{data => foo_value, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:clear_payload( @@ -217,7 +217,7 @@ clear_payload_with_keep_while_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{payload_version => 2}}, + {ok, #{[foo] => #{payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_payload_with_options_test_() -> @@ -225,12 +225,12 @@ clear_payload_with_options_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, - {ok, #{data => foo_value, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:clear_payload( @@ -239,7 +239,7 @@ clear_payload_with_options_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{payload_version => 2}}, + {ok, #{[foo] => #{payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. clear_many_payloads_from_non_existing_node_test_() -> @@ -265,10 +265,10 @@ clear_many_payloads_from_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo1] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo1], foo1_value)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo2, bar] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo2, bar], bar_value)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), diff --git a/test/advanced_tx_get.erl b/test/advanced_tx_get.erl index a10e9dd3..539c6d61 100644 --- a/test/advanced_tx_get.erl +++ b/test/advanced_tx_get.erl @@ -37,12 +37,12 @@ get_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)), ?_assertEqual( {ok, - {ok, #{data => foo_value, - payload_version => 1}}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}}, begin Fun = fun() -> khepri_tx_adv:get([foo]) @@ -51,8 +51,8 @@ get_existing_node_test_() -> end), ?_assertEqual( {ok, - {ok, #{data => foo_value, - payload_version => 1}}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}}, begin Fun = fun() -> khepri_tx_adv:get([foo]) @@ -68,8 +68,8 @@ get_existing_node_with_sproc_test_() -> ok, khepri:create(?FUNCTION_NAME, [foo], fun() -> ok end)), ?_assertMatch( - {ok, {ok, #{sproc := StoredFun, - payload_version := 1}}} + {ok, {ok, #{[foo] := #{sproc := StoredFun, + payload_version := 1}}}} when ?IS_HORUS_STANDALONE_FUN(StoredFun), begin Fun = fun() -> @@ -83,11 +83,11 @@ get_existing_node_with_no_payload_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo, bar] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( {ok, - {ok, #{payload_version => 1}}}, + {ok, #{[foo] => #{payload_version => 1}}}}, begin Fun = fun() -> khepri_tx_adv:get([foo]) @@ -128,10 +128,10 @@ get_many_existing_nodes_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo, bar] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo, bar], bar_value)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[baz] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [baz], baz_value)), ?_assertEqual( {ok, diff --git a/test/advanced_tx_put.erl b/test/advanced_tx_put.erl index 9ccce28e..3b61217d 100644 --- a/test/advanced_tx_put.erl +++ b/test/advanced_tx_put.erl @@ -28,7 +28,7 @@ create_non_existing_node_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, ro) end), ?_assertEqual( - {ok, {ok, #{payload_version => 1}}}, + {ok, {ok, #{[foo] => #{payload_version => 1}}}}, begin Fun = fun() -> khepri_tx_adv:create([foo], foo_value) @@ -36,8 +36,8 @@ create_non_existing_node_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. create_existing_node_test_() -> @@ -45,7 +45,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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( {ok, @@ -103,7 +103,7 @@ 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, {ok, #{payload_version => 1}}}, + {ok, {ok, #{[foo] => #{payload_version => 1}}}}, begin Fun = fun() -> khepri_tx_adv:put([foo], foo_value) @@ -111,8 +111,8 @@ insert_non_existing_node_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{data => foo_value, - payload_version => 1}}, + {ok, #{[foo] => #{data => foo_value, + payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. insert_existing_node_test_() -> @@ -120,7 +120,7 @@ insert_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), @@ -132,8 +132,8 @@ insert_existing_node_test_() -> end), ?_assertEqual( {ok, - {ok, #{data => foo_value1, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:put([foo], foo_value2) @@ -142,8 +142,8 @@ insert_existing_node_test_() -> end), ?_assertEqual( {ok, - {ok, #{data => foo_value2, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:put([foo], foo_value2, #{}) @@ -151,8 +151,8 @@ insert_existing_node_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. invalid_put_call_test_() -> @@ -176,10 +176,10 @@ insert_many_non_existing_nodes_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[a] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [a], ?NO_PAYLOAD)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[b] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [b], ?NO_PAYLOAD)), ?_assertEqual( {ok, @@ -204,10 +204,10 @@ insert_many_existing_nodes_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[a, foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [a, foo], foo_value_a)), ?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[b, foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [b, foo], foo_value_b)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), @@ -295,7 +295,7 @@ update_existing_node_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), @@ -307,8 +307,8 @@ update_existing_node_test_() -> end), ?_assertEqual( {ok, - {ok, #{data => foo_value1, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:update([foo], foo_value2) @@ -316,8 +316,8 @@ update_existing_node_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. invalid_update_call_test_() -> @@ -364,7 +364,7 @@ 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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertError( ?khepri_exception(denied_update_in_readonly_tx, #{}), @@ -377,8 +377,8 @@ compare_and_swap_matching_node_test_() -> end), ?_assertEqual( {ok, - {ok, #{data => foo_value1, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:compare_and_swap( @@ -387,8 +387,8 @@ compare_and_swap_matching_node_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. compare_and_swap_mismatching_node_test_() -> @@ -396,7 +396,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, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertMatch( {ok, @@ -422,12 +422,12 @@ compare_and_swap_with_options_test_() -> fun() -> test_ra_server_helpers:setup(?FUNCTION_NAME) end, fun(Priv) -> test_ra_server_helpers:cleanup(Priv) end, [?_assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:create(?FUNCTION_NAME, [foo], foo_value1)), ?_assertEqual( {ok, - {ok, #{data => foo_value1, - payload_version => 2}}}, + {ok, #{[foo] => #{data => foo_value1, + payload_version => 2}}}}, begin Fun = fun() -> khepri_tx_adv:compare_and_swap( @@ -437,8 +437,8 @@ compare_and_swap_with_options_test_() -> khepri:transaction(?FUNCTION_NAME, Fun, rw) end), ?_assertEqual( - {ok, #{data => foo_value2, - payload_version => 2}}, + {ok, #{[foo] => #{data => foo_value2, + payload_version => 2}}}, khepri_adv:get(?FUNCTION_NAME, [foo]))]}. invalid_compare_and_swap_call_test_() -> diff --git a/test/async_option.erl b/test/async_option.erl index e7d12ccf..3264fc96 100644 --- a/test/async_option.erl +++ b/test/async_option.erl @@ -25,7 +25,7 @@ async_unset_in_put_test_() -> ok, khepri:put(?FUNCTION_NAME, [foo], ?NO_PAYLOAD)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -41,7 +41,7 @@ async_false_in_put_test_() -> khepri:put( ?FUNCTION_NAME, [foo], ?NO_PAYLOAD, #{async => false})), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -66,7 +66,7 @@ async_true_in_put_test_() -> end, khepri:exists(?FUNCTION_NAME, [foo]), lists:seq(1, 60)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -88,7 +88,7 @@ async_with_correlation_in_put_test_() -> [{Correlation, {ok, #{[foo] => #{}}}}], khepri:handle_async_ret(?FUNCTION_NAME, RaEvent)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -113,7 +113,7 @@ async_with_priority_in_put_test_() -> end, khepri:exists(?FUNCTION_NAME, [foo]), lists:seq(1, 60)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -135,7 +135,7 @@ async_with_correlation_and_priority_in_put_test_() -> [{Correlation, {ok, #{[foo] => #{}}}}], khepri:handle_async_ret(?FUNCTION_NAME, RaEvent)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -311,7 +311,7 @@ async_unset_in_transaction_test_() -> {ok, ok}, khepri:transaction(?FUNCTION_NAME, Fun)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -328,7 +328,7 @@ async_false_in_transaction_test_() -> khepri:transaction( ?FUNCTION_NAME, Fun, #{async => false})), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -354,7 +354,7 @@ async_true_in_transaction_test_() -> end, khepri:exists(?FUNCTION_NAME, [foo]), lists:seq(1, 60)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -376,7 +376,7 @@ async_with_correlation_in_transaction_test_() -> [{Correlation, ok}], khepri:handle_async_ret(?FUNCTION_NAME, RaEvent)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -432,7 +432,7 @@ async_with_priority_in_transaction_test_() -> end, khepri:exists(?FUNCTION_NAME, [foo]), lists:seq(1, 60)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. @@ -455,7 +455,7 @@ async_with_correlation_and_priority_in_transaction_test_() -> [{Correlation, ok}], khepri:handle_async_ret(?FUNCTION_NAME, RaEvent)), ?assertEqual( - {ok, #{payload_version => 1}}, + {ok, #{[foo] => #{payload_version => 1}}}, khepri_adv:get(?FUNCTION_NAME, [foo])) end) ]}. diff --git a/test/cluster_SUITE.erl b/test/cluster_SUITE.erl index a38cde37..0a413569 100644 --- a/test/cluster_SUITE.erl +++ b/test/cluster_SUITE.erl @@ -1522,20 +1522,20 @@ can_use_default_store_on_single_node(_Config) -> {error, ?khepri_error(mismatching_node, _)}, khepri_adv:create([foo], value1)), ?assertEqual( - {ok, #{data => value4, - payload_version => 5}}, + {ok, #{[foo] => #{data => value4, + payload_version => 5}}}, khepri_adv:put([foo], value2)), ?assertEqual( {ok, #{[foo] => #{data => value2, payload_version => 5}}}, khepri_adv:put_many([foo], value2)), ?assertEqual( - {ok, #{data => value2, - payload_version => 6}}, + {ok, #{[foo] => #{data => value2, + payload_version => 6}}}, khepri_adv:update([foo], value3)), ?assertEqual( - {ok, #{data => value3, - payload_version => 7}}, + {ok, #{[foo] => #{data => value3, + payload_version => 7}}}, khepri_adv:compare_and_swap([foo], value3, value4)), ?assertEqual(true, khepri:exists([foo])), @@ -1579,12 +1579,12 @@ can_use_default_store_on_single_node(_Config) -> khepri:filter([foo], fun(_P, _NP) -> true end, #{})), ?assertEqual( - {ok, #{data => value4, - payload_version => 7}}, + {ok, #{[foo] => #{data => value4, + payload_version => 7}}}, khepri_adv:get([foo])), ?assertEqual( - {ok, #{data => value4, - payload_version => 7}}, + {ok, #{[foo] => #{data => value4, + payload_version => 7}}}, khepri_adv:get([foo], #{})), ?assertEqual( {ok, #{[foo] => #{data => value4, @@ -1641,7 +1641,9 @@ can_use_default_store_on_single_node(_Config) -> ?assertEqual( false, khepri:has_projection(ProjectionName1, #{favor => consistency})), - Projection1 = khepri_projection:new(ProjectionName1, copy), + Projection1 = khepri_projection:new( + ProjectionName1, + fun(Path, Data) -> {Path, Data} end), ?assertEqual(ok, khepri:register_projection("/**", Projection1)), ?assertEqual( {error, ?khepri_error( @@ -1655,7 +1657,8 @@ can_use_default_store_on_single_node(_Config) -> ProjectionName2 = projection2, ?assertEqual(false, khepri:has_projection(ProjectionName2)), Projection2 = khepri_projection:new( - ProjectionName2, copy, + ProjectionName2, + fun(Path, Data) -> {Data, Path} end, #{read_concurrency => true, keypos => 2}), ?assertEqual(ok, khepri:register_projection("/**", Projection2, #{})), ?assertEqual(true, khepri:has_projection(ProjectionName2)), @@ -1683,14 +1686,14 @@ can_use_default_store_on_single_node(_Config) -> ?assertEqual(ok, khepri:create([bar], value1)), ?assertEqual( - {ok, #{data => value1, - payload_version => 2}}, + {ok, #{[bar] => #{data => value1, + payload_version => 2}}}, khepri_adv:clear_payload([bar])), ?assertEqual( {ok, #{[bar] => #{payload_version => 2}}}, khepri_adv:clear_many_payloads([bar])), ?assertEqual( - {ok, #{payload_version => 2}}, + {ok, #{[bar] => #{payload_version => 2}}}, khepri_adv:delete([bar])), ?assertMatch( {ok, #{}},