Skip to content

Commit

Permalink
Make #kevf_*{} records private
Browse files Browse the repository at this point in the history
The public API is now provided by the new `khepri_evf` module to
construct the event filters.

Before:

    EventFilter = #kevf_tree{path = Path, props = Props}.

Now:

    EventFilter = khepri_evf:tree(Path, Props).

The reason is the same as for the payload records: if we change the
event filter records, users are not forced to recompile their code.

The `khepri_machine:register_trigger/5` now tries to autodetect and
convert common types to event filter, thanks to `khepri_evf:wrap/1`.
This is the case for native and Unix-like paths which are converted to
tree event filters. This allows users to skip the call to `khepri_evf`
and make it easier to use the API with a path directly.
  • Loading branch information
dumbbell committed Apr 20, 2022
1 parent 7f5e377 commit d3d4ac9
Show file tree
Hide file tree
Showing 9 changed files with 197 additions and 53 deletions.
3 changes: 2 additions & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,8 @@ the database itself and automatically execute it after some event occurs.
2. Register a trigger using an event filter:

```erlang
EventFilter = #kevf_tree{path = [stock, wood, <<"oak">>]},
%% A path is automatically considered a tree event filter.
EventFilter = [stock, wood, <<"oak">>],

ok = khepri:register_trigger(
StoreId,
Expand Down
17 changes: 12 additions & 5 deletions doc/overview.edoc
Original file line number Diff line number Diff line change
Expand Up @@ -406,14 +406,21 @@ arity.

Khepri uses <em>event filters</em> to associate a type of events with a stored
procedure. Khepri supports tree changes events and thus only supports a single
event filter called {@link khepri:event_filter_tree()}.
event filter called {@link khepri_evf:tree_event_filter()}.

An event filter is registered using {@link khepri:register_trigger/4}:

```
EventFilter = #kevf_tree{path = [stock, wood, <<"oak">>], %% Required
props = #{on_actions => [delete], %% Optional
priority => 10}}, %% Optional
%% An event filter can be explicitly created using the `khepri_evf'
%% module. This is possible to specify properties at the same time.
EventFilter = khepri_evf:tree([stock, wood, <<"oak">>], %% Required
#{on_actions => [delete], %% Optional
priority => 10}), %% Optional

%% For ease of use, some terms can be automatically converted to an event %
%filter. Here, a Unix-like path could be used as a tree event % filter, though
%it would have default properties unlike the previous line:
EventFilter = "/:stock/:wood/oak".

ok = khepri:register_trigger(
StoreId,
Expand All @@ -422,7 +429,7 @@ ok = khepri:register_trigger(
StoredProcPath))}.
'''

In this example, the {@link khepri:event_filter_tree()} record only
In this example, the {@link khepri_evf:tree_event_filter()} structure only
requires the path to monitor. The path can be any path pattern and thus can
have conditions to monitor several nodes at once.

Expand Down
14 changes: 0 additions & 14 deletions include/khepri.hrl
Original file line number Diff line number Diff line change
Expand Up @@ -83,17 +83,3 @@

-record(if_any,
{conditions = [] :: [khepri_path:pattern_component()]}).

%% -------------------------------------------------------------------
%% Event filtering.
%% -------------------------------------------------------------------

-record(kevf_tree, {path :: khepri_path:pattern(),
props = #{} :: #{on_actions => [create | update | delete],
priority => integer()}}).
%-record(kevf_process, {pid :: pid(),
% props = #{} :: #{on_reason => ets:match_pattern(),
% priority => integer()}}).

-define(IS_KHEPRI_EVENT_FILTER(EventFilter),
(is_record(EventFilter, kevf_tree))).
13 changes: 11 additions & 2 deletions src/internal.hrl
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,15 @@
is_record(Payload, p_data) orelse
is_record(Payload, p_sproc))).

-record(evf_tree, {path :: khepri_path:pattern(),
props = #{} :: khepri_evf:tree_event_filter_props()}).
%-record(evf_process, {pid :: pid(),
% props = #{} :: #{on_reason => ets:match_pattern(),
% priority => integer()}}).

-define(IS_KHEPRI_EVENT_FILTER(EventFilter),
(is_record(EventFilter, evf_tree))).

%% Structure representing each node in the tree, including the root node.
%% TODO: Rename stat to something more correct?
-record(node, {stat = ?INIT_NODE_STAT :: khepri_machine:stat(),
Expand All @@ -42,14 +51,14 @@
-record(tx, {'fun' :: khepri_fun:standalone_fun()}).

-record(register_trigger, {id :: khepri:trigger_id(),
event_filter :: khepri:event_filter(),
event_filter :: khepri_evf:event_filter(),
sproc :: khepri_path:path()}).

-record(ack_triggered, {triggered :: [khepri_machine:triggered()]}).

-record(triggered, {id :: khepri:trigger_id(),
%% TODO: Do we need a ref to distinguish multiple
%% instances of the same trigger?
event_filter :: khepri:event_filter(),
event_filter :: khepri_evf:event_filter(),
sproc :: khepri_fun:standalone_fun(),
props = #{} :: map()}).
39 changes: 25 additions & 14 deletions src/khepri.erl
Original file line number Diff line number Diff line change
Expand Up @@ -215,10 +215,6 @@
-type trigger_id() :: atom().
%% An ID to identify a registered trigger.

-type event_filter_tree() :: #kevf_tree{}.

-type event_filter() :: event_filter_tree().

-type async_option() :: boolean() |
ra_server:command_correlation() |
ra_server:command_priority() |
Expand Down Expand Up @@ -305,8 +301,6 @@
result/0,
keep_while_conds_map/0,
trigger_id/0,
event_filter_tree/0,
event_filter/0,

async_option/0,
favor_option/0,
Expand Down Expand Up @@ -1501,7 +1495,8 @@ run_sproc(StoreId, PathPattern, Args, Options) ->

-spec register_trigger(TriggerId, EventFilter, StoredProcPath) -> Ret when
TriggerId :: trigger_id(),
EventFilter :: event_filter(),
EventFilter :: khepri_evf:event_filter() |
khepri_path:pattern() | string(),
StoredProcPath :: khepri_path:path() | string(),
Ret :: ok | error().
%% @doc Registers a trigger.
Expand All @@ -1519,12 +1514,14 @@ register_trigger(TriggerId, EventFilter, StoredProcPath) ->
(StoreId, TriggerId, EventFilter, StoredProcPath) -> Ret when
StoreId :: khepri:store_id(),
TriggerId :: trigger_id(),
EventFilter :: event_filter(),
EventFilter :: khepri_evf:event_filter() |
khepri_path:pattern() | string(),
StoredProcPath :: khepri_path:path() | string(),
Ret :: ok | error();
(TriggerId, EventFilter, StoredProcPath, Options) -> Ret when
TriggerId :: trigger_id(),
EventFilter :: event_filter(),
EventFilter :: khepri_evf:event_filter() |
khepri_path:pattern() | string(),
StoredProcPath :: khepri_path:path() | string(),
Options :: command_options(),
Ret :: ok | error().
Expand Down Expand Up @@ -1556,20 +1553,34 @@ register_trigger(TriggerId, EventFilter, StoredProcPath, Options)
Ret when
StoreId :: khepri:store_id(),
TriggerId :: trigger_id(),
EventFilter :: event_filter(),
EventFilter :: khepri_evf:event_filter() |
khepri_path:pattern() | string(),
StoredProcPath :: khepri_path:path() | string(),
Options :: command_options(),
Ret :: ok | error().
%% @doc Registers a trigger.
%%
%% A trigger is based on an event filter. It associates an event with a stored
%% procedure. When an event matching the event filter is emitted, the stored
%% procedure is executed. Here is an example of an event filter:
%% procedure is executed.
%%
%% The following event filters are documented by {@link
%% khepri_evf:event_filter()}.
%%
%% Here are examples of event filters:
%%
%% ```
%% %% An event filter can be explicitly created using the `khepri_evf'
%% %% module. This is possible to specify properties at the same time.
%% EventFilter = khepri_evf:tree([stock, wood, <<"oak">>], %% Required
%% #{on_actions => [delete], %% Optional
%% priority => 10}). %% Optional
%% '''
%% ```
%% EventFilter = #kevf_tree{path = [stock, wood, <<"oak">>], %% Required
%% props = #{on_actions => [delete], %% Optional
%% priority => 10}}, %% Optional
%% %% For ease of use, some terms can be automatically converted to an event
%% %% filter. In this example, a Unix-like path can be used as a tree event
%% %% filter.
%% EventFilter = "/:stock/:wood/oak".
%% '''
%%
%% The stored procedure is expected to accept a single argument. This argument
Expand Down
134 changes: 134 additions & 0 deletions src/khepri_evf.erl
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021-2022 VMware, Inc. or its affiliates. All rights reserved.
%%

%% @doc Khepri event filters.

-module(khepri_evf).

-include("src/internal.hrl").

-export([tree/1, tree/2,
wrap/1,
get_priority/1,
set_priority/2]).

-type tree_event_filter() :: #evf_tree{}.
%% A tree event filter.
%%
%% It takes a path pattern to monitor and optionally properties.

-type tree_event_filter_props() :: #{on_actions => [create | update | delete],
priority => khepri_evf:priority()}.
%% Tree event filter properties.
%%
%% The properties are:
%% <ul>
%% <li>`on_actions': a list of actions to filter among `create', `update' and
%% `delete'; the default is to react to all of them.</li>
%% <li>`priority': a {@link priority()}</li>
%% </ul>
%%
%% A Khepri path, wether it is a native path or a Unix-like path, can be used
%% as a tree event filter. It will be automatically converted to a tree event
%% filter with default properties.

-type event_filter() :: tree_event_filter().
%% An event filter.
%%
%% The following event filters are supported:
%% <ul>
%% <li>Tree event filter ({@link tree_event_filter()}</li>
%% </ul>
%%
%% An event filter can be explicitly constructed using the functions provided
%% in this module. However, some common types will be automatically detected
%% and converted to an event filter with default properties. See each event
%% filter type for more details.

-type priority() :: integer().
%% An event filter priority.
%%
%% This is an integer to prioritize event filters: the greater the priority,
%% the more it is prioritized. Negative integers are allowed.
%%
%% The default priority is 0.

-export_type([event_filter/0,
tree_event_filter/0,
tree_event_filter_props/0,
priority/0]).

-spec tree(PathPattern) -> EventFilter when
PathPattern :: khepri_path:pattern() | string(),
EventFilter :: tree_event_filter().
%% @doc Constructs a tree event filter.
%%
%% @see tree/2.

tree(PathPattern) ->
tree(PathPattern, #{}).

-spec tree(PathPattern, Props) -> EventFilter when
PathPattern :: khepri_path:pattern() | string(),
Props :: tree_event_filter_props(),
EventFilter :: tree_event_filter().
%% @doc Constructs a tree event filter.
%%
%% @see tree_event_filter().

tree(PathPattern, Props) ->
PathPattern1 = khepri_path:from_string(PathPattern),
#evf_tree{path = PathPattern1,
props = Props}.

-spec wrap(Input) -> EventFilter when
Input :: event_filter() | khepri_path:pattern() | string(),
EventFilter :: event_filter().
%% @doc Automatically detects the event filter type and ensures it is wrapped
%% in one of the internal types.
%%
%% @param Input an already created event filter, or any term which can be
%% automatically converted to an event filter.
%%
%% @returns the created event filter.

wrap(EventFilter) when ?IS_KHEPRI_EVENT_FILTER(EventFilter) ->
EventFilter;
wrap(PathPattern) when is_list(PathPattern) ->
tree(PathPattern).

-spec get_priority(EventFilter) -> Priority when
EventFilter :: event_filter(),
Priority :: priority().
%% @doc Returns the priority of the event filter.
%%
%% @param EventFilter the event filter to update.
%%
%% @returns the priority.

get_priority(#evf_tree{props = Props}) ->
get_priority1(Props).

get_priority1(#{priority := Priority}) -> Priority;
get_priority1(_) -> 0.

-spec set_priority(EventFilter, Priority) -> EventFilter when
EventFilter :: event_filter(),
Priority :: priority().
%% @doc Sets the priority of the event filter.
%%
%% @param EventFilter the event filter to update.
%% @param Priority the new priority.
%%
%% @returns the updated event filter.

set_priority(#evf_tree{props = Props} = EventFilter, Priority) ->
Props1 = set_priority1(Props, Priority),
EventFilter#evf_tree{props = Props1}.

set_priority1(Props, Priority) when is_integer(Priority) ->
Props#{priority => Priority}.
25 changes: 10 additions & 15 deletions src/khepri_machine.erl
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,8 @@ run_sproc(StoreId, PathPattern, Args, Options) when is_list(Args) ->
Ret when
StoreId :: khepri:store_id(),
TriggerId :: khepri:trigger_id(),
EventFilter :: khepri:event_filter(),
EventFilter :: khepri_evf:event_filter() |
khepri_path:pattern() | string(),
StoredProcPath :: khepri_path:path() | string(),
Options :: khepri:command_options(),
Ret :: ok | khepri:error().
Expand All @@ -369,13 +370,13 @@ run_sproc(StoreId, PathPattern, Args, Options) when is_list(Args) ->
%% @returns `ok' if the trigger was registered, an `{error, Reason}' tuple
%% otherwise.

register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath, Options)
when ?IS_KHEPRI_EVENT_FILTER(EventFilter) ->
register_trigger(StoreId, TriggerId, EventFilter, StoredProcPath, Options) ->
EventFilter1 = khepri_evf:wrap(EventFilter),
StoredProcPath1 = khepri_path:from_string(StoredProcPath),
khepri_path:ensure_is_valid(StoredProcPath1),
Command = #register_trigger{id = TriggerId,
sproc = StoredProcPath1,
event_filter = EventFilter},
event_filter = EventFilter1},
process_command(StoreId, Command, Options).

-spec ack_triggers_execution(StoreId, TriggeredStoredProcs) ->
Expand Down Expand Up @@ -803,9 +804,9 @@ apply(
#?MODULE{triggers = Triggers} = State) ->
StoredProcPath1 = khepri_path:realpath(StoredProcPath),
EventFilter1 = case EventFilter of
#kevf_tree{path = Path} ->
#evf_tree{path = Path} ->
Path1 = khepri_path:realpath(Path),
EventFilter#kevf_tree{path = Path1}
EventFilter#evf_tree{path = Path1}
end,
Triggers1 = Triggers#{TriggerId => #{sproc => StoredProcPath1,
event_filter => EventFilter1}},
Expand Down Expand Up @@ -1367,7 +1368,7 @@ list_triggered_sprocs(Root, Changes, Triggers) ->
(TriggerId,
#{sproc := StoredProcPath,
event_filter :=
#kevf_tree{path = PathPattern,
#evf_tree{path = PathPattern,
props = EventFilterProps} = EventFilter},
SPP1) ->
%% For each trigger based on a tree event:
Expand Down Expand Up @@ -1482,21 +1483,15 @@ sort_triggered_sprocs(TriggeredStoredProcs) ->
lists:sort(
fun(#triggered{id = IdA, event_filter = EventFilterA},
#triggered{id = IdB, event_filter = EventFilterB}) ->
PrioA = get_event_filter_priority(EventFilterA),
PrioB = get_event_filter_priority(EventFilterB),
PrioA = khepri_evf:get_priority(EventFilterA),
PrioB = khepri_evf:get_priority(EventFilterB),
if
PrioA =:= PrioB -> IdA =< IdB;
true -> PrioA > PrioB
end
end,
TriggeredStoredProcs).

get_event_filter_priority(#kevf_tree{props = #{priority := Priority}})
when is_integer(Priority) ->
Priority;
get_event_filter_priority(_EventFilter) ->
0.

%% -------

-spec walk_down_the_tree(
Expand Down
2 changes: 1 addition & 1 deletion src/khepri_machine.hrl
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,6 @@
triggers = #{} ::
#{khepri:trigger_id() =>
#{sproc := khepri_path:path(),
event_filter := khepri:event_filter()}},
event_filter := khepri_evf:event_filter()}},
emitted_triggers = [] :: [khepri_machine:triggered()],
metrics = #{} :: #{applied_command_count => non_neg_integer()}}).
Loading

0 comments on commit d3d4ac9

Please sign in to comment.