diff --git a/deps/rabbit/src/rabbit_db_binding.erl b/deps/rabbit/src/rabbit_db_binding.erl index a35b07e73e9d..fde322bfaa95 100644 --- a/deps/rabbit/src/rabbit_db_binding.erl +++ b/deps/rabbit/src/rabbit_db_binding.erl @@ -413,11 +413,11 @@ get_all_in_mnesia() -> end). get_all_in_khepri() -> - case ets:whereis(?KHEPRI_BINDINGS_PROJECTION) of - undefined -> - []; - Table -> - [B || #route{binding = B} <- ets:tab2list(Table)] + try + [B || #route{binding = B} <- ets:tab2list(?KHEPRI_BINDINGS_PROJECTION)] + catch + error:badarg -> + [] end. -spec get_all(VHostName) -> [Binding] when @@ -444,15 +444,16 @@ get_all_in_mnesia(VHost) -> [B || #route{binding = B} <- rabbit_db:list_in_mnesia(?MNESIA_TABLE, Match)]. get_all_in_khepri(VHost) -> - case ets:whereis(?KHEPRI_BINDINGS_PROJECTION) of - undefined -> - []; - Table -> - VHostResource = rabbit_misc:r(VHost, '_'), - Match = #route{binding = #binding{source = VHostResource, - destination = VHostResource, - _ = '_'}}, - [B || #route{binding = B} <- ets:match_object(Table, Match)] + try + VHostResource = rabbit_misc:r(VHost, '_'), + Match = #route{binding = #binding{source = VHostResource, + destination = VHostResource, + _ = '_'}}, + [B || #route{binding = B} <- ets:match_object( + ?KHEPRI_BINDINGS_PROJECTION, Match)] + catch + error:badarg -> + [] end. -spec get_all(Src, Dst, Reverse) -> [Binding] when @@ -481,14 +482,15 @@ get_all_in_mnesia(SrcName, DstName, Reverse) -> mnesia:async_dirty(Fun). get_all_in_khepri(SrcName, DstName) -> - case ets:whereis(?KHEPRI_BINDINGS_PROJECTION) of - undefined -> - []; - Table -> - MatchHead = #route{binding = #binding{source = SrcName, - destination = DstName, - _ = '_'}}, - [B || #route{binding = B} <- ets:match_object(Table, MatchHead)] + try + MatchHead = #route{binding = #binding{source = SrcName, + destination = DstName, + _ = '_'}}, + [B || #route{binding = B} <- ets:match_object( + ?KHEPRI_BINDINGS_PROJECTION, MatchHead)] + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -528,12 +530,13 @@ list_for_route(Route, true) -> end. get_all_for_source_in_khepri(Resource) -> - case ets:whereis(?KHEPRI_BINDINGS_PROJECTION) of - undefined -> - []; - Table -> - Route = #route{binding = #binding{source = Resource, _ = '_'}}, - [B || #route{binding = B} <- ets:match_object(Table, Route)] + try + Route = #route{binding = #binding{source = Resource, _ = '_'}}, + [B || #route{binding = B} <- ets:match_object( + ?KHEPRI_BINDINGS_PROJECTION, Route)] + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -563,13 +566,14 @@ get_all_for_destination_in_mnesia(Dst) -> mnesia:async_dirty(Fun). get_all_for_destination_in_khepri(Destination) -> - case ets:whereis(?KHEPRI_BINDINGS_PROJECTION) of - undefined -> - []; - Table -> - Match = #route{binding = #binding{destination = Destination, - _ = '_'}}, - [B || #route{binding = B} <- ets:match_object(Table, Match)] + try + Match = #route{binding = #binding{destination = Destination, + _ = '_'}}, + [B || #route{binding = B} <- ets:match_object( + ?KHEPRI_BINDINGS_PROJECTION, Match)] + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -644,15 +648,16 @@ match_in_mnesia(SrcName, Match) -> Routes, Match(Binding)]. match_in_khepri(SrcName, Match) -> - case ets:whereis(?KHEPRI_BINDINGS_PROJECTION) of - undefined -> - []; - Table -> - MatchHead = #route{binding = #binding{source = SrcName, - _ = '_'}}, - Routes = ets:select(Table, [{MatchHead, [], [['$_']]}]), - [Dest || [#route{binding = Binding = #binding{destination = Dest}}] <- - Routes, Match(Binding)] + try + MatchHead = #route{binding = #binding{source = SrcName, + _ = '_'}}, + Routes = ets:select( + ?KHEPRI_BINDINGS_PROJECTION, [{MatchHead, [], [['$_']]}]), + [Dest || [#route{binding = Binding = #binding{destination = Dest}}] <- + Routes, Match(Binding)] + catch + error:badarg -> + [] end. %% Routing - HOT CODE PATH @@ -686,26 +691,22 @@ match_routing_key_in_mnesia(SrcName, RoutingKeys, UseIndex) -> route_in_mnesia_v1(SrcName, RoutingKeys) end. +match_routing_key_in_khepri(Src, ['_']) -> + try + MatchHead = #index_route{source_key = {Src, '_'}, + destination = '$1', + _ = '_'}, + ets:select(?KHEPRI_INDEX_ROUTE_PROJECTION, [{MatchHead, [], ['$1']}]) + catch + error:badarg -> + [] + end; match_routing_key_in_khepri(Src, RoutingKeys) -> - case ets:whereis(?KHEPRI_INDEX_ROUTE_PROJECTION) of - undefined -> - []; - Table -> - do_match_routing_key_in_khepri(Table, Src, RoutingKeys) - end. - -do_match_routing_key_in_khepri(Table, Src, ['_']) -> - MatchHead = #index_route{source_key = {Src, '_'}, - destination = '$1', - _ = '_'}, - ets:select(Table, [{MatchHead, [], ['$1']}]); - -do_match_routing_key_in_khepri(Table, Src, RoutingKeys) -> lists:foldl( fun(RK, Acc) -> try Dst = ets:lookup_element( - Table, + ?KHEPRI_INDEX_ROUTE_PROJECTION, {Src, RK}, #index_route.destination), Dst ++ Acc diff --git a/deps/rabbit/src/rabbit_db_exchange.erl b/deps/rabbit/src/rabbit_db_exchange.erl index 6d912dc71cf2..1b0a2382b544 100644 --- a/deps/rabbit/src/rabbit_db_exchange.erl +++ b/deps/rabbit/src/rabbit_db_exchange.erl @@ -183,14 +183,12 @@ get_in_mnesia(Name) -> rabbit_mnesia:dirty_read({?MNESIA_TABLE, Name}). get_in_khepri(Name) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - {error, not_found}; - Table -> - case ets:lookup(Table, Name) of - [X] -> {ok, X}; - [] -> {error, not_found} - end + try ets:lookup(?KHEPRI_PROJECTION, Name) of + [X] -> {ok, X}; + [] -> {error, not_found} + catch + error:badarg -> + {error, not_found} end. %% ------------------------------------------------------------------- @@ -233,11 +231,11 @@ get_many_in_mnesia(Table, Names) when is_list(Names) -> lists:append([ets:lookup(Table, Name) || Name <- Names]). get_many_in_khepri(Names) when is_list(Names) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - lists:append([ets:lookup(Table, Name) || Name <- Names]) + try + lists:append([ets:lookup(?KHEPRI_PROJECTION, Name) || Name <- Names]) + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_db_queue.erl b/deps/rabbit/src/rabbit_db_queue.erl index ed10e9c2a86d..f2d7b512406b 100644 --- a/deps/rabbit/src/rabbit_db_queue.erl +++ b/deps/rabbit/src/rabbit_db_queue.erl @@ -110,15 +110,15 @@ get_all_in_mnesia() -> end). get_all_in_khepri() -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - list_with_possible_retry_in_khepri( - fun() -> - ets:tab2list(Table) - end) - end. + list_with_possible_retry_in_khepri( + fun() -> + try + ets:tab2list(?KHEPRI_PROJECTION) + catch + error:badarg -> + [] + end + end). -spec get_all(VHostName) -> [Queue] when VHostName :: vhost:name(), @@ -144,16 +144,16 @@ get_all_in_mnesia(VHostName) -> end). get_all_in_khepri(VHostName) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - list_with_possible_retry_in_khepri( - fun() -> - Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHostName, queue)), - ets:match_object(Table, Pattern) - end) - end. + list_with_possible_retry_in_khepri( + fun() -> + try + Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHostName, queue)), + ets:match_object(?KHEPRI_PROJECTION, Pattern) + catch + error:badarg -> + [] + end + end). %% ------------------------------------------------------------------- %% get_all_durable(). @@ -181,16 +181,16 @@ get_all_durable_in_mnesia() -> end). get_all_durable_in_khepri() -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - list_with_possible_retry_in_khepri( - fun() -> - Pattern = amqqueue:pattern_match_on_durable(true), - ets:match_object(Table, Pattern) - end) - end. + list_with_possible_retry_in_khepri( + fun() -> + try + Pattern = amqqueue:pattern_match_on_durable(true), + ets:match_object(?KHEPRI_PROJECTION, Pattern) + catch + error:badarg -> + [] + end + end). -spec get_all_durable_by_type(Type) -> [Queue] when Type :: atom(), @@ -213,12 +213,12 @@ get_all_durable_by_type_in_mnesia(Type) -> rabbit_db:list_in_mnesia(?MNESIA_DURABLE_TABLE, Pattern). get_all_durable_by_type_in_khepri(Type) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - Pattern = amqqueue:pattern_match_on_type_and_durable(Type, true), - ets:match_object(Table, Pattern) + try + Pattern = amqqueue:pattern_match_on_type_and_durable(Type, true), + ets:match_object(?KHEPRI_PROJECTION, Pattern) + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -250,18 +250,18 @@ filter_all_durable_in_mnesia(FilterFun) -> end). filter_all_durable_in_khepri(FilterFun) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - ets:foldl( - fun(Q, Acc0) -> - case amqqueue:is_durable(Q) andalso FilterFun(Q) of - true -> [Q | Acc0]; - false -> Acc0 - end - end, - [], Table) + try + ets:foldl( + fun(Q, Acc0) -> + case amqqueue:is_durable(Q) andalso FilterFun(Q) of + true -> [Q | Acc0]; + false -> Acc0 + end + end, + [], ?KHEPRI_PROJECTION) + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -287,12 +287,12 @@ list_in_mnesia() -> mnesia:dirty_all_keys(?MNESIA_TABLE). list_in_khepri() -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - Pattern = amqqueue:pattern_match_on_name('$1'), - ets:select(Table, [{Pattern, [], ['$1']}]) + try + Pattern = amqqueue:pattern_match_on_name('$1'), + ets:select(?KHEPRI_PROJECTION, [{Pattern, [], ['$1']}]) + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -318,11 +318,12 @@ count_in_mnesia() -> mnesia:table_info(?MNESIA_TABLE, size). count_in_khepri() -> - case ets:whereis(?KHEPRI_PROJECTION) of + case ets:info(?KHEPRI_PROJECTION, size) of undefined -> + %% `ets:info/2` on a table that does not exist returns `undefined`. 0; - Table -> - ets:info(Table, size) + Size -> + Size end. -spec count(VHostName) -> Count when @@ -361,12 +362,12 @@ list_for_count_in_mnesia(VHostName) -> end). list_for_count_in_khepri(VHostName) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - 0; - Table -> - Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHostName, queue)), - ets:select_count(Table, [{Pattern, [], [true]}]) + try + Pattern = amqqueue:pattern_match_on_name(rabbit_misc:r(VHostName, queue)), + ets:select_count(?KHEPRI_PROJECTION, [{Pattern, [], [true]}]) + catch + error:badarg -> + 0 end. %% ------------------------------------------------------------------- @@ -466,11 +467,11 @@ get_many(Names) when is_list(Names) -> }). get_many_in_khepri(Names) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - get_many_in_ets(Table, Names) + try + get_many_in_ets(?KHEPRI_PROJECTION, Names) + catch + error:badarg -> + [] end. get_many_in_ets(Table, [{Name, RouteInfos}]) @@ -512,14 +513,12 @@ get_in_mnesia(Name) -> rabbit_mnesia:dirty_read({?MNESIA_TABLE, Name}). get_in_khepri(Name) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - {error, not_found}; - Table -> - case ets:lookup(Table, Name) of - [Q] -> {ok, Q}; - [] -> {error, not_found} - end + try ets:lookup(?KHEPRI_PROJECTION, Name) of + [Q] -> {ok, Q}; + [] -> {error, not_found} + catch + error:badarg -> + {error, not_found} end. %% ------------------------------------------------------------------- @@ -568,12 +567,12 @@ get_many_durable_in_mnesia(Names) -> get_many_in_ets(?MNESIA_DURABLE_TABLE, Names). get_many_durable_in_khepri(Names) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - Queues = get_many_in_ets(Table, Names), - [Q || Q <- Queues, amqqueue:is_durable(Q)] + try + Queues = get_many_in_ets(?KHEPRI_PROJECTION, Names), + [Q || Q <- Queues, amqqueue:is_durable(Q)] + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -783,11 +782,11 @@ exists_in_mnesia(QName) -> ets:member(?MNESIA_TABLE, QName). exists_in_khepri(QName) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - false; - Table -> - ets:member(Table, QName) + try + ets:member(?KHEPRI_PROJECTION, QName) + catch + error:badarg -> + false end. %% ------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_db_rtparams.erl b/deps/rabbit/src/rabbit_db_rtparams.erl index 1e0682421f6a..0f07bf82b483 100644 --- a/deps/rabbit/src/rabbit_db_rtparams.erl +++ b/deps/rabbit/src/rabbit_db_rtparams.erl @@ -151,14 +151,12 @@ get_in_mnesia(Key) -> end. get_in_khepri(Key) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - undefined; - Table -> - case ets:lookup(Table, Key) of - [] -> undefined; - [Record] -> Record - end + try ets:lookup(?KHEPRI_PROJECTION, Key) of + [] -> undefined; + [Record] -> Record + catch + error:badarg -> + undefined end. %% ------------------------------------------------------------------- @@ -182,11 +180,11 @@ get_all_in_mnesia() -> rabbit_mnesia:dirty_read_all(?MNESIA_TABLE). get_all_in_khepri() -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - ets:tab2list(Table) + try + ets:tab2list(?KHEPRI_PROJECTION) + catch + error:badarg -> + [] end. -spec get_all(VHostName, Comp) -> Ret when @@ -224,13 +222,13 @@ get_all_in_khepri(VHostName, Comp) -> '_' -> ok; _ -> rabbit_vhost:assert(VHostName) end, - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - Match = #runtime_parameters{key = {VHostName, Comp, '_'}, - _ = '_'}, - ets:match_object(Table, Match) + try + Match = #runtime_parameters{key = {VHostName, Comp, '_'}, + _ = '_'}, + ets:match_object(?KHEPRI_PROJECTION, Match) + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_db_topic_exchange.erl b/deps/rabbit/src/rabbit_db_topic_exchange.erl index 640530bb3e5b..6d9affd55598 100644 --- a/deps/rabbit/src/rabbit_db_topic_exchange.erl +++ b/deps/rabbit/src/rabbit_db_topic_exchange.erl @@ -492,54 +492,57 @@ ensure_topic_deletion_ets() -> %% Khepri topic graph trie_match_in_khepri(X, Words, BKeys) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - trie_match_in_khepri(Table, X, root, Words, BKeys, []) + try + trie_match_in_khepri(X, root, Words, BKeys, []) + catch + error:badarg -> + [] end. -trie_match_in_khepri(Table, X, Node, [], BKeys, ResAcc0) -> - Destinations = trie_bindings_in_khepri(Table, X, Node, BKeys), +trie_match_in_khepri(X, Node, [], BKeys, ResAcc0) -> + Destinations = trie_bindings_in_khepri(X, Node, BKeys), ResAcc = add_matched(Destinations, BKeys, ResAcc0), trie_match_part_in_khepri( - Table, X, Node, <<"#">>, - fun trie_match_skip_any_in_khepri/6, [], BKeys, ResAcc); -trie_match_in_khepri(Table, X, Node, [W | RestW] = Words, BKeys, ResAcc) -> + X, Node, <<"#">>, + fun trie_match_skip_any_in_khepri/5, [], BKeys, ResAcc); +trie_match_in_khepri(X, Node, [W | RestW] = Words, BKeys, ResAcc) -> lists:foldl(fun ({WArg, MatchFun, RestWArg}, Acc) -> trie_match_part_in_khepri( - Table, X, Node, WArg, MatchFun, RestWArg, BKeys, Acc) - end, ResAcc, [{W, fun trie_match_in_khepri/6, RestW}, - {<<"*">>, fun trie_match_in_khepri/6, RestW}, + X, Node, WArg, MatchFun, RestWArg, BKeys, Acc) + end, ResAcc, [{W, fun trie_match_in_khepri/5, RestW}, + {<<"*">>, fun trie_match_in_khepri/5, RestW}, {<<"#">>, - fun trie_match_skip_any_in_khepri/6, Words}]). + fun trie_match_skip_any_in_khepri/5, Words}]). -trie_match_part_in_khepri( - Table, X, Node, Search, MatchFun, RestW, BKeys, ResAcc) -> - case trie_child_in_khepri(Table, X, Node, Search) of - {ok, NextNode} -> MatchFun(Table, X, NextNode, RestW, BKeys, ResAcc); +trie_match_part_in_khepri(X, Node, Search, MatchFun, RestW, BKeys, ResAcc) -> + case trie_child_in_khepri(X, Node, Search) of + {ok, NextNode} -> MatchFun(X, NextNode, RestW, BKeys, ResAcc); error -> ResAcc end. -trie_match_skip_any_in_khepri(Table, X, Node, [], BKeys, ResAcc) -> - trie_match_in_khepri(Table, X, Node, [], BKeys, ResAcc); -trie_match_skip_any_in_khepri(Table, X, Node, [_ | RestW] = Words, BKeys, ResAcc) -> +trie_match_skip_any_in_khepri(X, Node, [], BKeys, ResAcc) -> + trie_match_in_khepri(X, Node, [], BKeys, ResAcc); +trie_match_skip_any_in_khepri(X, Node, [_ | RestW] = Words, BKeys, ResAcc) -> trie_match_skip_any_in_khepri( - Table, X, Node, RestW, BKeys, - trie_match_in_khepri(Table, X, Node, Words, BKeys, ResAcc)). - -trie_child_in_khepri(Table, X, Node, Word) -> - case ets:lookup(Table, #trie_edge{exchange_name = X, - node_id = Node, - word = Word}) of + X, Node, RestW, BKeys, + trie_match_in_khepri(X, Node, Words, BKeys, ResAcc)). + +trie_child_in_khepri(X, Node, Word) -> + case ets:lookup( + ?KHEPRI_PROJECTION, + #trie_edge{exchange_name = X, + node_id = Node, + word = Word}) of [#topic_trie_edge{node_id = NextNode}] -> {ok, NextNode}; [] -> error end. -trie_bindings_in_khepri(Table,X, Node, BKeys) -> - case ets:lookup(Table, #trie_edge{exchange_name = X, - node_id = Node, - word = bindings}) of +trie_bindings_in_khepri(X, Node, BKeys) -> + case ets:lookup( + ?KHEPRI_PROJECTION, + #trie_edge{exchange_name = X, + node_id = Node, + word = bindings}) of [#topic_trie_edge{node_id = {bindings, Bindings}}] -> [case BKeys of true -> diff --git a/deps/rabbit/src/rabbit_db_user.erl b/deps/rabbit/src/rabbit_db_user.erl index 73c0828e7184..fb00b01a5daa 100644 --- a/deps/rabbit/src/rabbit_db_user.erl +++ b/deps/rabbit/src/rabbit_db_user.erl @@ -187,14 +187,12 @@ get_in_mnesia(Username) -> end. get_in_khepri(Username) -> - case ets:whereis(?KHEPRI_USERS_PROJECTION) of - undefined -> - undefined; - Table -> - case ets:lookup(Table, Username) of - [User] -> User; - _ -> undefined - end + try ets:lookup(?KHEPRI_USERS_PROJECTION, Username) of + [User] -> User; + _ -> undefined + catch + error:badarg -> + undefined end. %% ------------------------------------------------------------------- @@ -297,18 +295,16 @@ get_user_permissions_in_mnesia(Username, VHostName) -> end. get_user_permissions_in_khepri(Username, VHostName) -> - case ets:whereis(?KHEPRI_PERMISSIONS_PROJECTION) of - undefined -> - undefined; - Table -> - UserVHost = #user_vhost{username = Username, - virtual_host = VHostName}, - case ets:lookup(Table, UserVHost) of - [UserPermission] -> - UserPermission; - _ -> - undefined - end + UserVHost = #user_vhost{username = Username, + virtual_host = VHostName}, + try ets:lookup(?KHEPRI_PERMISSIONS_PROJECTION, UserVHost) of + [UserPermission] -> + UserPermission; + _ -> + undefined + catch + error:badarg -> + undefined end. %% ------------------------------------------------------------------- diff --git a/deps/rabbit/src/rabbit_db_vhost.erl b/deps/rabbit/src/rabbit_db_vhost.erl index 62441b76e0ab..faf7643e527b 100644 --- a/deps/rabbit/src/rabbit_db_vhost.erl +++ b/deps/rabbit/src/rabbit_db_vhost.erl @@ -242,11 +242,11 @@ exists_in_mnesia(VHostName) -> mnesia:dirty_read({?MNESIA_TABLE, VHostName}) /= []. exists_in_khepri(VHostName) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - false; - Table -> - ets:member(Table, VHostName) + try + ets:member(?KHEPRI_PROJECTION, VHostName) + catch + error:badarg -> + false end. %% ------------------------------------------------------------------- @@ -275,14 +275,12 @@ get_in_mnesia(VHostName) -> end. get_in_khepri(VHostName) -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - undefined; - Table -> - case ets:lookup(Table, VHostName) of - [Record] -> Record; - _ -> undefined - end + try ets:lookup(?KHEPRI_PROJECTION, VHostName) of + [Record] -> Record; + _ -> undefined + catch + error:badarg -> + undefined end. %% ------------------------------------------------------------------- @@ -306,11 +304,11 @@ get_all_in_mnesia() -> mnesia:dirty_match_object(?MNESIA_TABLE, vhost:pattern_match_all()). get_all_in_khepri() -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - ets:tab2list(Table) + try + ets:tab2list(?KHEPRI_PROJECTION) + catch + error:badarg -> + [] end. %% ------------------------------------------------------------------- @@ -334,11 +332,12 @@ list_in_mnesia() -> mnesia:dirty_all_keys(?MNESIA_TABLE). list_in_khepri() -> - case ets:whereis(?KHEPRI_PROJECTION) of - undefined -> - []; - Table -> - ets:select(Table, [{vhost:pattern_match_names(), [], ['$1']}]) + try + ets:select( + ?KHEPRI_PROJECTION, [{vhost:pattern_match_names(), [], ['$1']}]) + catch + error:badarg -> + [] end. %% -------------------------------------------------------------------