diff --git a/src/khepri.erl b/src/khepri.erl
index 001acf33..07892a07 100644
--- a/src/khepri.erl
+++ b/src/khepri.erl
@@ -203,6 +203,22 @@
-type child_list_length() :: non_neg_integer().
%% Number of direct child nodes under a tree node.
+-type delete_reason() :: explicit | keep_while.
+%% The reason why a tree node was removed from the store.
+%%
+%%
+%% - `explicit' means that the tree node was removed from the store because a
+%% deletion command targeted that tree node or its ancestor. This reason is
+%% used when a tree node's path is provided to {@link khepri_adv:delete/3} for
+%% example.
+%% - `keep_while': the tree node was removed because the keep-while condition
+%% set when the node was created became unsatisfied. Note that adding or
+%% updating a tree node can cause a keep-while condition to become unsatisfied,
+%% so a put operation may result in a tree node being deleted with this delete
+%% reason. See {@link khepri_condition:keep_while()} and {@link
+%% khepri:put_options()}.
+%%
+
-type node_props() ::
#{data => khepri:data(),
has_data => boolean(),
@@ -211,7 +227,8 @@
payload_version => khepri:payload_version(),
child_list_version => khepri:child_list_version(),
child_list_length => khepri:child_list_length(),
- child_names => [khepri_path:node_id()]}.
+ child_names => [khepri_path:node_id()],
+ delete_reason => khepri:delete_reason()}.
%% Structure used to return properties, payload and child nodes for a specific
%% tree node.
%%
@@ -337,7 +354,8 @@
child_names |
payload |
has_payload |
- raw_payload],
+ raw_payload |
+ delete_reason],
include_root_props => boolean()}.
%% Options used during tree traversal.
%%
@@ -460,6 +478,7 @@
payload_version/0,
child_list_version/0,
child_list_length/0,
+ delete_reason/0,
node_props/0,
trigger_id/0,
diff --git a/src/khepri_node.hrl b/src/khepri_node.hrl
index df88ecb7..2c5ed6a1 100644
--- a/src/khepri_node.hrl
+++ b/src/khepri_node.hrl
@@ -16,7 +16,8 @@
child_list_version => ?INIT_CHILD_LIST_VERSION}).
-define(DEFAULT_PROPS_TO_RETURN, [payload,
- payload_version]).
+ payload_version,
+ delete_reason]).
-record(node, {props = ?INIT_NODE_PROPS :: khepri_machine:props(),
payload = ?NO_PAYLOAD :: khepri_payload:payload(),
diff --git a/src/khepri_tree.erl b/src/khepri_tree.erl
index 48f66c68..22f76236 100644
--- a/src/khepri_tree.erl
+++ b/src/khepri_tree.erl
@@ -233,7 +233,9 @@ gather_node_props(#node{props = #{payload_version := PVersion,
_ -> Acc
end;
(raw_payload, Acc) ->
- Acc#{raw_payload => Payload}
+ Acc#{raw_payload => Payload};
+ (delete_reason, Acc) ->
+ Acc
end, #{}, WantedProps);
gather_node_props(#node{}, _Options) ->
#{}.
@@ -528,22 +530,41 @@ find_matching_nodes_cb(_, {interrupted, _, _}, _, Acc, _) ->
delete_matching_nodes(Tree, PathPattern, AppliedChanges, TreeOptions) ->
Fun = fun(Path, Node, Result) ->
- delete_matching_nodes_cb(Path, Node, TreeOptions, Result)
+ delete_matching_nodes_cb(
+ Path, Node, TreeOptions, explicit, Result)
end,
walk_down_the_tree(
Tree, PathPattern, TreeOptions, AppliedChanges, Fun, #{}).
-delete_matching_nodes_cb([] = Path, #node{} = Node, TreeOptions, Result) ->
+delete_matching_nodes_cb(
+ [] = Path, #node{} = Node, TreeOptions, DeleteReason, Result) ->
Node1 = remove_node_payload(Node),
Node2 = remove_node_child_nodes(Node1),
- NodeProps = gather_node_props(Node, TreeOptions),
- {ok, Node2, Result#{Path => NodeProps}};
-delete_matching_nodes_cb(Path, #node{} = Node, TreeOptions, Result) ->
- NodeProps = gather_node_props(Node, TreeOptions),
- {ok, delete, Result#{Path => NodeProps}};
-delete_matching_nodes_cb(_, {interrupted, _, _}, _Options, Result) ->
+ NodeProps1 = gather_node_props(Node, TreeOptions),
+ NodeProps2 = maybe_add_delete_reason_prop(
+ NodeProps1, TreeOptions, DeleteReason),
+ {ok, Node2, Result#{Path => NodeProps2}};
+delete_matching_nodes_cb(
+ Path, #node{} = Node, TreeOptions, DeleteReason, Result) ->
+ NodeProps1 = gather_node_props(Node, TreeOptions),
+ NodeProps2 = maybe_add_delete_reason_prop(
+ NodeProps1, TreeOptions, DeleteReason),
+ {ok, delete, Result#{Path => NodeProps2}};
+delete_matching_nodes_cb(
+ _, {interrupted, _, _}, _Options, _DeleteReason, Result) ->
{ok, keep, Result}.
+maybe_add_delete_reason_prop(
+ NodeProps, #{props_to_return := WantedProps}, DeleteReason) ->
+ case lists:member(delete_reason, WantedProps) of
+ true ->
+ NodeProps#{delete_reason => DeleteReason};
+ false ->
+ NodeProps
+ end;
+maybe_add_delete_reason_prop(NodeProps, _TreeOptions, _DeleteReason) ->
+ NodeProps.
+
%% -------------------------------------------------------------------
%% Insert or update a tree node.
%% -------------------------------------------------------------------
diff --git a/test/advanced_delete.erl b/test/advanced_delete.erl
index 76d187e2..3d6ccae9 100644
--- a/test/advanced_delete.erl
+++ b/test/advanced_delete.erl
@@ -36,7 +36,8 @@ delete_existing_node_test_() ->
khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)),
?_assertEqual(
{ok, #{[foo] => #{data => foo_value,
- payload_version => 1}}},
+ payload_version => 1,
+ delete_reason => explicit}}},
khepri_adv:delete(?FUNCTION_NAME, [foo])),
?_assertEqual(
{error, ?khepri_error(node_not_found, #{node_name => foo,
@@ -77,7 +78,8 @@ delete_many_on_existing_node_with_condition_true_test_() ->
khepri_adv:create(?FUNCTION_NAME, [foo], foo_value)),
?_assertEqual(
{ok, #{[foo] => #{data => foo_value,
- payload_version => 1}}},
+ payload_version => 1,
+ delete_reason => explicit}}},
khepri_adv:delete_many(
?FUNCTION_NAME, [#if_name_matches{regex = "foo"}])),
?_assertEqual(
diff --git a/test/advanced_tx_delete.erl b/test/advanced_tx_delete.erl
index 483c4bbe..cf9b64b7 100644
--- a/test/advanced_tx_delete.erl
+++ b/test/advanced_tx_delete.erl
@@ -50,7 +50,8 @@ delete_existing_node_test_() ->
?_assertEqual(
{ok,
{ok, #{[foo] => #{data => foo_value,
- payload_version => 1}}}},
+ payload_version => 1,
+ delete_reason => explicit}}}},
begin
Fun = fun() ->
khepri_tx_adv:delete([foo])
@@ -116,7 +117,8 @@ delete_many_on_existing_node_with_condition_true_test_() ->
?_assertEqual(
{ok,
{ok, #{[foo] => #{data => foo_value,
- payload_version => 1}}}},
+ payload_version => 1,
+ delete_reason => explicit}}}},
begin
Fun = fun() ->
khepri_tx_adv:delete_many(
diff --git a/test/async_option.erl b/test/async_option.erl
index 3264fc96..d4046918 100644
--- a/test/async_option.erl
+++ b/test/async_option.erl
@@ -289,7 +289,8 @@ async_with_correlation_and_priority_in_delete_test_() ->
RaEvent = receive {ra_event, _, _} = Event -> Event end,
?assertEqual(
[{Correlation, {ok, #{[foo] =>
- #{payload_version => 1}}}}],
+ #{payload_version => 1,
+ delete_reason => explicit}}}}],
khepri:handle_async_ret(?FUNCTION_NAME, RaEvent)),
?assertEqual(
{error,
diff --git a/test/cluster_SUITE.erl b/test/cluster_SUITE.erl
index 0a413569..16d785a0 100644
--- a/test/cluster_SUITE.erl
+++ b/test/cluster_SUITE.erl
@@ -1693,7 +1693,8 @@ can_use_default_store_on_single_node(_Config) ->
{ok, #{[bar] => #{payload_version => 2}}},
khepri_adv:clear_many_payloads([bar])),
?assertEqual(
- {ok, #{[bar] => #{payload_version => 2}}},
+ {ok, #{[bar] => #{payload_version => 2,
+ delete_reason => explicit}}},
khepri_adv:delete([bar])),
?assertMatch(
{ok, #{}},
diff --git a/test/delete_command.erl b/test/delete_command.erl
index 4bb9f0e3..6cc84d53 100644
--- a/test/delete_command.erl
+++ b/test/delete_command.erl
@@ -127,7 +127,8 @@ delete_a_node_deep_into_the_tree_test() ->
options = #{props_to_return => [payload,
payload_version,
child_list_version,
- child_list_length]}},
+ child_list_length,
+ delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -138,9 +139,12 @@ delete_a_node_deep_into_the_tree_test() ->
child_list_version => 3},
child_nodes = #{}},
Root),
- ?assertEqual({ok, #{[foo, bar, baz] => #{payload_version => 1,
- child_list_version => 1,
- child_list_length => 1}}}, Ret),
+ ?assertEqual(
+ {ok, #{[foo, bar, baz] => #{payload_version => 1,
+ child_list_version => 1,
+ child_list_length => 1,
+ delete_reason => explicit}}},
+ Ret),
?assertEqual([], SE).
delete_existing_node_with_condition_true_test() ->
@@ -275,7 +279,8 @@ delete_many_nodes_at_once_test() ->
options = #{props_to_return => [payload,
payload_version,
child_list_version,
- child_list_length]}},
+ child_list_length,
+ delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -292,11 +297,13 @@ delete_many_nodes_at_once_test() ->
?assertEqual({ok, #{[bar] => #{data => bar_value,
payload_version => 1,
child_list_version => 1,
- child_list_length => 0},
+ child_list_length => 0,
+ delete_reason => explicit},
[baz] => #{data => baz_value,
payload_version => 1,
child_list_version => 1,
- child_list_length => 0}}}, Ret),
+ child_list_length => 0,
+ delete_reason => explicit}}}, Ret),
?assertEqual([], SE).
delete_command_bumps_applied_command_count_test() ->
diff --git a/test/keep_while_conditions.erl b/test/keep_while_conditions.erl
index 7dec366d..ee450b0a 100644
--- a/test/keep_while_conditions.erl
+++ b/test/keep_while_conditions.erl
@@ -202,7 +202,8 @@ keep_while_still_true_after_command_test() ->
options = #{props_to_return => [payload,
payload_version,
child_list_version,
- child_list_length]}},
+ child_list_length,
+ delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -238,7 +239,8 @@ keep_while_now_false_after_command_test() ->
S0 = khepri_machine:init(?MACH_PARAMS(Commands)),
Command = #put{path = [foo, bar],
- payload = khepri_payload:data(bar_value)},
+ payload = khepri_payload:data(bar_value),
+ options = #{props_to_return => [delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -278,7 +280,8 @@ recursive_automatic_cleanup_test() ->
options = #{props_to_return => [payload,
payload_version,
child_list_version,
- child_list_length]}},
+ child_list_length,
+ delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -289,10 +292,13 @@ recursive_automatic_cleanup_test() ->
child_list_version => 3},
child_nodes = #{}},
Root),
- ?assertEqual({ok, #{[foo, bar, baz] => #{data => baz_value,
- payload_version => 1,
- child_list_version => 1,
- child_list_length => 0}}}, Ret),
+ ?assertEqual(
+ {ok, #{[foo, bar, baz] => #{data => baz_value,
+ payload_version => 1,
+ child_list_version => 1,
+ child_list_length => 0,
+ delete_reason => explicit}}},
+ Ret),
?assertEqual([], SE).
keep_while_now_false_after_delete_command_test() ->
@@ -308,7 +314,8 @@ keep_while_now_false_after_delete_command_test() ->
options = #{props_to_return => [payload,
payload_version,
child_list_version,
- child_list_length]}},
+ child_list_length,
+ delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -322,14 +329,16 @@ keep_while_now_false_after_delete_command_test() ->
?assertEqual({ok, #{[foo] => #{data => foo_value,
payload_version => 1,
child_list_version => 1,
- child_list_length => 0}}}, Ret),
+ child_list_length => 0,
+ delete_reason => explicit}}}, Ret),
?assertEqual([], SE).
automatic_reclaim_of_useless_nodes_works_test() ->
Commands = [#put{path = [foo, bar, baz, qux],
payload = khepri_payload:data(value)}],
S0 = khepri_machine:init(?MACH_PARAMS(Commands)),
- Command = #delete{path = [foo, bar, baz]},
+ Command = #delete{path = [foo, bar, baz],
+ options = #{props_to_return => [delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -340,7 +349,8 @@ automatic_reclaim_of_useless_nodes_works_test() ->
child_list_version => 3},
child_nodes = #{}},
Root),
- ?assertEqual({ok, #{[foo, bar, baz] => #{}}}, Ret),
+ ?assertEqual(
+ {ok, #{[foo, bar, baz] => #{delete_reason => explicit}}}, Ret),
?assertEqual([], SE).
automatic_reclaim_keeps_relevant_nodes_1_test() ->
@@ -351,7 +361,8 @@ automatic_reclaim_keeps_relevant_nodes_1_test() ->
#put{path = [foo],
payload = khepri_payload:data(relevant)}],
S0 = khepri_machine:init(?MACH_PARAMS(Commands)),
- Command = #delete{path = [foo, bar, baz]},
+ Command = #delete{path = [foo, bar, baz],
+ options = #{props_to_return => [delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -368,7 +379,8 @@ automatic_reclaim_keeps_relevant_nodes_1_test() ->
payload = khepri_payload:data(relevant),
child_nodes = #{}}}},
Root),
- ?assertEqual({ok, #{[foo, bar, baz] => #{}}}, Ret),
+ ?assertEqual(
+ {ok, #{[foo, bar, baz] => #{delete_reason => explicit}}}, Ret),
?assertEqual([], SE).
automatic_reclaim_keeps_relevant_nodes_2_test() ->
@@ -379,7 +391,8 @@ automatic_reclaim_keeps_relevant_nodes_2_test() ->
#put{path = [foo, bar, baz, qux],
payload = khepri_payload:data(qux_value)}],
S0 = khepri_machine:init(?MACH_PARAMS(Commands)),
- Command = #delete{path = [foo, bar, baz, qux]},
+ Command = #delete{path = [foo, bar, baz, qux],
+ options = #{props_to_return => [delete_reason]}},
{S1, Ret, SE} = khepri_machine:apply(?META, Command, S0),
Root = khepri_machine:get_root(S1),
@@ -401,5 +414,7 @@ automatic_reclaim_keeps_relevant_nodes_2_test() ->
payload = khepri_payload:data(bar_value),
child_nodes = #{}}}}}},
Root),
- ?assertEqual({ok, #{[foo, bar, baz, qux] => #{}}}, Ret),
+ ?assertEqual(
+ {ok, #{[foo, bar, baz, qux] => #{delete_reason => explicit}}},
+ Ret),
?assertEqual([], SE).
diff --git a/test/prop_state_machine.erl b/test/prop_state_machine.erl
index 965d5f35..8e575b14 100644
--- a/test/prop_state_machine.erl
+++ b/test/prop_state_machine.erl
@@ -102,7 +102,7 @@ postcondition(
#state{entries = Entries},
{call, khepri_adv, delete_many, [_StoreId, Path, _Options]},
Result) ->
- result_is_ok(Result, Entries, Path, {ok, #{}}).
+ result_is_ok_after_delete(Result, Entries, Path).
add_entry(Entries, Path, Payload) ->
{Entry1, New} = case Entries of
@@ -183,6 +183,18 @@ result_is_ok(Result, Entries, Path, Default) ->
Default =:= Result
end.
+result_is_ok_after_delete({ok, NodePropsMap}, Entries, Path) ->
+ case Entries of
+ #{Path := NodeProps} ->
+ DeletedProps = maps:get(Path, NodePropsMap, #{}),
+ DeletedProps1 = maps:remove(delete_reason, DeletedProps),
+ DeletedProps1 =:= NodeProps;
+ _ ->
+ NodePropsMap =:= #{}
+ end;
+result_is_ok_after_delete(_, _Entries, _Path) ->
+ false.
+
result_is_ok_after_put(Result, Entries, Path, Payload, Default) ->
case Entries of
#{Path := #{data := Payload} = NodeProps} ->