Skip to content

Commit

Permalink
Add delete_reason prop to deleted nodes' props maps
Browse files Browse the repository at this point in the history
This change adds a new `delete_reason` key to the `node_props()` map
which is either `direct` or `keep_while`. This commit changes
`khepri_tree` to add the `direct` delete reason for tree nodes which
were deleted because they were the target of a delete command. The
`keep_while` reason will be added in the child commit.

The motivation to include a delete reason is that in the child commit
we will include tree nodes deleted because their keep-while conditions
became unsatisfied. `delete_reason` enables the caller to distinguish
between tree nodes deleted "directly" (the target of the delete command)
and the others which were expired. It also enables callers of put
operations to distinguish between puts (tree node creations and updates)
and deletions, since a put operation could also trigger a keep-while
expiration.
  • Loading branch information
the-mikedavis committed Oct 16, 2024
1 parent 5e89ae2 commit a11cfc5
Show file tree
Hide file tree
Showing 10 changed files with 122 additions and 41 deletions.
23 changes: 21 additions & 2 deletions src/khepri.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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.
%%
%% <ul>
%% <li>`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.</li>
%% <li>`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()}.</li>
%% </ul>

-type node_props() ::
#{data => khepri:data(),
has_data => boolean(),
Expand All @@ -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.
%%
Expand Down Expand Up @@ -337,7 +354,8 @@
child_names |
payload |
has_payload |
raw_payload],
raw_payload |
delete_reason],
include_root_props => boolean()}.
%% Options used during tree traversal.
%%
Expand Down Expand Up @@ -460,6 +478,7 @@
payload_version/0,
child_list_version/0,
child_list_length/0,
delete_reason/0,
node_props/0,
trigger_id/0,

Expand Down
3 changes: 2 additions & 1 deletion src/khepri_node.hrl
Original file line number Diff line number Diff line change
Expand Up @@ -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(),
Expand Down
39 changes: 30 additions & 9 deletions src/khepri_tree.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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) ->
#{}.
Expand Down Expand Up @@ -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.
%% -------------------------------------------------------------------
Expand Down
6 changes: 4 additions & 2 deletions test/advanced_delete.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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(
Expand Down
6 changes: 4 additions & 2 deletions test/advanced_tx_delete.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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])
Expand Down Expand Up @@ -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(
Expand Down
3 changes: 2 additions & 1 deletion test/async_option.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
3 changes: 2 additions & 1 deletion test/cluster_SUITE.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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, #{}},
Expand Down
21 changes: 14 additions & 7 deletions test/delete_command.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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),

Expand All @@ -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() ->
Expand Down Expand Up @@ -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),

Expand All @@ -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() ->
Expand Down
45 changes: 30 additions & 15 deletions test/keep_while_conditions.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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),

Expand Down Expand Up @@ -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),

Expand Down Expand Up @@ -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),

Expand All @@ -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() ->
Expand All @@ -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),

Expand All @@ -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),

Expand All @@ -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() ->
Expand All @@ -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),

Expand All @@ -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() ->
Expand All @@ -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),

Expand All @@ -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).
14 changes: 13 additions & 1 deletion test/prop_state_machine.erl
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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} ->
Expand Down

0 comments on commit a11cfc5

Please sign in to comment.