fix(emqx_cluster_rpc): fail fast on stale state

Due to:

* Cluster RPC MFA is not idempotent!
* There is a lack of rollback for callback's side-effects

For instance, when two nodes try to add a cluster-singleton
concurrently, one of them will have to wait for the table lock
then try to catch-up, then try to apply MFA.
The catch-up will have the singleton created, but the initiated
initiated multicall apply will fail causing the commit to rollback,
but not to 'undo' the singleton creation.
Later, the retries will fail indefinitely.
This commit is contained in:
Zaiming (Stone) Shi 2022-06-11 20:47:46 +02:00
parent b92708726a
commit 2065be569e
18 changed files with 202 additions and 209 deletions

View File

@ -233,7 +233,8 @@ put(Config) ->
erase(RootName) -> erase(RootName) ->
persistent_term:erase(?PERSIS_KEY(?CONF, bin(RootName))), persistent_term:erase(?PERSIS_KEY(?CONF, bin(RootName))),
persistent_term:erase(?PERSIS_KEY(?RAW_CONF, bin(RootName))). persistent_term:erase(?PERSIS_KEY(?RAW_CONF, bin(RootName))),
ok.
-spec put(emqx_map_lib:config_key_path(), term()) -> ok. -spec put(emqx_map_lib:config_key_path(), term()) -> ok.
put(KeyPath, Config) -> put(KeyPath, Config) ->

View File

@ -163,8 +163,7 @@ get_counters(Name, Id) ->
reset_counters(Name, Id) -> reset_counters(Name, Id) ->
Indexes = maps:values(get_indexes(Name, Id)), Indexes = maps:values(get_indexes(Name, Id)),
Ref = get_ref(Name, Id), Ref = get_ref(Name, Id),
[counters:put(Ref, Idx, 0) || Idx <- Indexes], lists:foreach(fun(Idx) -> counters:put(Ref, Idx, 0) end, Indexes).
ok.
-spec get_metrics(handler_name(), metric_id()) -> metrics(). -spec get_metrics(handler_name(), metric_id()) -> metrics().
get_metrics(Name, Id) -> get_metrics(Name, Id) ->

View File

@ -53,8 +53,7 @@
-define(RPC_MODULES, "gen_rpc, erpc, rpc, emqx_rpc"). -define(RPC_MODULES, "gen_rpc, erpc, rpc, emqx_rpc").
%% List of known functions also known to do RPC: %% List of known functions also known to do RPC:
-define(RPC_FUNCTIONS, -define(RPC_FUNCTIONS,
"emqx_cluster_rpc:multicall/3, emqx_cluster_rpc:multicall/5, " "emqx_cluster_rpc:multicall/3, emqx_cluster_rpc:multicall/5"
"emqx_plugin_libs_rule:cluster_call/3"
). ).
%% List of functions in the RPC backend modules that we can ignore: %% List of functions in the RPC backend modules that we can ignore:
@ -63,11 +62,9 @@
%% List of business-layer functions that are exempt from the checks: %% List of business-layer functions that are exempt from the checks:
%% erlfmt-ignore %% erlfmt-ignore
-define(EXEMPTIONS, -define(EXEMPTIONS,
"emqx_mgmt_api:do_query/6," % Reason: legacy code. A fun and a QC query are "emqx_mgmt_api:do_query/6" % Reason: legacy code. A fun and a QC query are
% passed in the args, it's futile to try to statically % passed in the args, it's futile to try to statically
% check it % check it
"emqx_plugin_libs_rule:cluster_call/3" % Reason: some sort of external plugin API that we
% don't want to break?
). ).
-define(XREF, myxref). -define(XREF, myxref).

View File

@ -30,7 +30,11 @@
skip_failed_commit/1, skip_failed_commit/1,
fast_forward_to_commit/2 fast_forward_to_commit/2
]). ]).
-export([get_node_tnx_id/1, latest_tnx_id/0]). -export([
get_node_tnx_id/1,
latest_tnx_id/0,
make_initiate_call_req/3
]).
-export([ -export([
init/1, init/1,
@ -44,7 +48,7 @@
-export([get_tables_status/0]). -export([get_tables_status/0]).
-export_type([txn_id/0, succeed_num/0, multicall_return/1, multicall_return/0]). -export_type([tnx_id/0, succeed_num/0]).
-ifdef(TEST). -ifdef(TEST).
-compile(export_all). -compile(export_all).
@ -56,19 +60,21 @@
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-include("emqx_conf.hrl"). -include("emqx_conf.hrl").
-define(INITIATE(MFA, LatestIdLastSeen), {initiate, MFA, LatestIdLastSeen}).
-define(CATCH_UP, catch_up). -define(CATCH_UP, catch_up).
-define(TIMEOUT, timer:minutes(1)). -define(TIMEOUT, timer:minutes(1)).
-type txn_id() :: pos_integer(). -type tnx_id() :: pos_integer().
-type succeed_num() :: pos_integer() | all. -type succeed_num() :: pos_integer() | all.
-type multicall_return(Result) :: -type multicall_return(Result) ::
{ok, txn_id(), Result} {ok, tnx_id(), Result}
| {error, term()} | {init_failure, term()}
| {retry, txn_id(), Result, node()}. | {peers_lagging, tnx_id(), Result, [node()]}.
-type multicall_return() :: multicall_return(_). -type multicall_return() :: multicall_return(_).
-type init_call_req() :: ?INITIATE({module(), atom(), list()}, tnx_id()).
%%%=================================================================== %%%===================================================================
%%% API %%% API
@ -102,27 +108,73 @@ start_link(Node, Name, RetryMs) ->
{error, Reason} {error, Reason}
end. end.
%% @doc return {ok, TnxId, MFARes} the first MFA result when all MFA run ok. %% @doc Initiate a local call (or core node),
%% return {error, MFARes} when the first MFA result is no ok or {ok, term()}. %% then async-ly replicate the call to peer nodes in the cluster.
%% return {retry, TnxId, MFARes, Nodes} when some Nodes failed and some Node ok. %% The evaluation result of the provided MFA is returned,
-spec multicall(module(), atom(), list()) -> multicall_return(). %% the result is expected to be `ok | {ok, _}' to indicate success,
%% and `{error, _}' to indicate failure.
%%
%% The excpetion of the MFA evaluation is captured and translated
%% into an `{error, _}' tuple.
%% This call tries to wait for all peer nodes to be in-sync before
%% returning the result.
%%
%% In case of partial success, an `error' level log is emitted
%% but the initial localy apply result is returned.
-spec multicall(module(), atom(), list()) -> term().
multicall(M, F, A) -> multicall(M, F, A) ->
multicall(M, F, A, all, timer:minutes(2)). multicall(M, F, A, all, timer:minutes(2)).
-spec multicall(module(), atom(), list(), succeed_num(), timeout()) -> multicall_return(). -spec multicall(module(), atom(), list(), succeed_num(), timeout()) -> term().
multicall(M, F, A, RequireNum, Timeout) when RequireNum =:= all orelse RequireNum >= 1 -> multicall(M, F, A, RequiredSyncs, Timeout) when RequiredSyncs =:= all orelse RequiredSyncs >= 1 ->
MFA = {initiate, {M, F, A}}, case do_multicall(M, F, A, RequiredSyncs, Timeout) of
{ok, _TxnId, Result} ->
Result;
{init_failure, Error} ->
Error;
{peers_lagging, TnxId, Res, Nodes} ->
%% The init MFA return ok, but some other nodes failed.
?SLOG(error, #{
msg => "cluster_rpc_peers_lagging",
lagging_nodes => Nodes,
tnx_id => TnxId
}),
Res
end.
%% Return {ok, TnxId, MFARes} the first MFA result when all MFA run ok.
%% return {init_failure, Error} when the initial MFA result is no ok or {ok, term()}.
%% return {peers_lagging, TnxId, MFARes, Nodes} when some Nodes failed and some Node ok.
-spec do_multicall(module(), atom(), list(), succeed_num(), timeout()) -> multicall_return().
do_multicall(M, F, A, RequiredSyncs, Timeout) when
RequiredSyncs =:= all orelse RequiredSyncs >= 1
->
%% Idealy 'LatestId' should be provided by the multicall originator,
%% which is the viewer of the state e.g.
%% * Sysadmin who issues CLI-commands or REST-API calls to make config changes
%% * Dashboard viewer who is making decision based on what they can see from the UI
%% To reach the ideal state, it would require adding transaction ID to each and
%% every view/GET requests and also provide the ID as a part of the view/GET responses.
%%
%% To keep things simple, we try to get the 'old' view when a multicall request
%% is received as early as possible.
%%
%% Reason to do this:
%% The 'initiate' call handler tries to take a table lock (cluster-wide) before
%% bumping the transaction ID. While waiting for the lock, the ID might have been
%% bumpped by another node in the cluster.
InitReq = make_initiate_call_req(M, F, A),
Begin = erlang:monotonic_time(), Begin = erlang:monotonic_time(),
InitRes = InitRes =
case mria_rlog:role() of case mria_rlog:role() of
core -> core ->
gen_server:call(?MODULE, MFA, Timeout); gen_server:call(?MODULE, InitReq, Timeout);
replicant -> replicant ->
%% the initiate transaction must happened on core node %% the initiate transaction must happened on core node
%% make sure MFA(in the transaction) and the transaction on the same node %% make sure MFA(in the transaction) and the transaction on the same node
%% don't need rpc again inside transaction. %% don't need rpc again inside transaction.
case mria_status:upstream_node(?CLUSTER_RPC_SHARD) of case mria_status:upstream_node(?CLUSTER_RPC_SHARD) of
{ok, Node} -> gen_server:call({?MODULE, Node}, MFA, Timeout); {ok, Node} -> gen_server:call({?MODULE, Node}, InitReq, Timeout);
disconnected -> {error, disconnected} disconnected -> {error, disconnected}
end end
end, end,
@ -132,23 +184,23 @@ multicall(M, F, A, RequireNum, Timeout) when RequireNum =:= all orelse RequireNu
RetryTimeout = ceil(3 * max(MinDelay, get_retry_ms())), RetryTimeout = ceil(3 * max(MinDelay, get_retry_ms())),
OkOrFailed = OkOrFailed =
case InitRes of case InitRes of
{ok, _TnxId, _} when RequireNum =:= 1 -> {ok, _TnxId, _} when RequiredSyncs =:= 1 ->
ok; ok;
{ok, TnxId, _} when RequireNum =:= all -> {ok, TnxId, _} when RequiredSyncs =:= all ->
wait_for_all_nodes_commit(TnxId, MinDelay, RetryTimeout); wait_for_all_nodes_commit(TnxId, MinDelay, RetryTimeout);
{ok, TnxId, _} when is_integer(RequireNum) -> {ok, TnxId, _} when is_integer(RequiredSyncs) ->
wait_for_nodes_commit(RequireNum, TnxId, MinDelay, RetryTimeout); wait_for_nodes_commit(RequiredSyncs, TnxId, MinDelay, RetryTimeout);
Error -> Error ->
Error Error
end, end,
case OkOrFailed of case OkOrFailed of
ok -> ok ->
InitRes; InitRes;
{error, Error0} -> {init_failure, Error0} ->
{error, Error0}; {init_failure, Error0};
{retry, Node0} -> {peers_lagging, Nodes} ->
{ok, TnxId0, MFARes} = InitRes, {ok, TnxId0, MFARes} = InitRes,
{retry, TnxId0, MFARes, Node0} {peers_lagging, TnxId0, MFARes, Nodes}
end. end.
-spec query(pos_integer()) -> {'atomic', map()} | {'aborted', Reason :: term()}. -spec query(pos_integer()) -> {'atomic', map()} | {'aborted', Reason :: term()}.
@ -167,6 +219,11 @@ latest_tnx_id() ->
{atomic, TnxId} = transaction(fun get_latest_id/0, []), {atomic, TnxId} = transaction(fun get_latest_id/0, []),
TnxId. TnxId.
-spec make_initiate_call_req(module(), atom(), list()) -> init_call_req().
make_initiate_call_req(M, F, A) ->
TnxId = get_latest_id(dirty),
?INITIATE({M, F, A}, TnxId).
-spec get_node_tnx_id(node()) -> integer(). -spec get_node_tnx_id(node()) -> integer().
get_node_tnx_id(Node) -> get_node_tnx_id(Node) ->
case mnesia:wread({?CLUSTER_COMMIT, Node}) of case mnesia:wread({?CLUSTER_COMMIT, Node}) of
@ -232,12 +289,12 @@ handle_call(reset, _From, State) ->
_ = mria:clear_table(?CLUSTER_COMMIT), _ = mria:clear_table(?CLUSTER_COMMIT),
_ = mria:clear_table(?CLUSTER_MFA), _ = mria:clear_table(?CLUSTER_MFA),
{reply, ok, State, {continue, ?CATCH_UP}}; {reply, ok, State, {continue, ?CATCH_UP}};
handle_call({initiate, MFA}, _From, State = #{node := Node}) -> handle_call(?INITIATE(MFA, LatestId), _From, State = #{node := Node}) ->
case transaction(fun init_mfa/2, [Node, MFA]) of case transaction(fun init_mfa/3, [Node, MFA, LatestId]) of
{atomic, {ok, TnxId, Result}} -> {atomic, {ok, TnxId, Result}} ->
{reply, {ok, TnxId, Result}, State, {continue, ?CATCH_UP}}; {reply, {ok, TnxId, Result}, State, {continue, ?CATCH_UP}};
{aborted, Reason} -> {aborted, Error} ->
{reply, {error, Reason}, State, {continue, ?CATCH_UP}} {reply, {init_failure, Error}, State, {continue, ?CATCH_UP}}
end; end;
handle_call(skip_failed_commit, _From, State = #{node := Node}) -> handle_call(skip_failed_commit, _From, State = #{node := Node}) ->
Timeout = catch_up(State, true), Timeout = catch_up(State, true),
@ -273,7 +330,7 @@ catch_up(#{node := Node, retry_interval := RetryMs} = State, SkipResult) ->
{atomic, caught_up} -> {atomic, caught_up} ->
?TIMEOUT; ?TIMEOUT;
{atomic, {still_lagging, NextId, MFA}} -> {atomic, {still_lagging, NextId, MFA}} ->
{Succeed, _} = apply_mfa(NextId, MFA), {Succeed, _} = apply_mfa(NextId, MFA, catch_up),
case Succeed orelse SkipResult of case Succeed orelse SkipResult of
true -> true ->
case transaction(fun commit/2, [Node, NextId]) of case transaction(fun commit/2, [Node, NextId]) of
@ -316,35 +373,6 @@ read_next_mfa(Node) ->
[#cluster_rpc_mfa{mfa = MFA}] -> {still_lagging, NextId, MFA} [#cluster_rpc_mfa{mfa = MFA}] -> {still_lagging, NextId, MFA}
end. end.
do_catch_up(ToTnxId, Node) ->
case mnesia:wread({?CLUSTER_COMMIT, Node}) of
[] ->
commit(Node, ToTnxId),
caught_up;
[#cluster_rpc_commit{tnx_id = LastAppliedId}] when ToTnxId =:= LastAppliedId ->
caught_up;
[#cluster_rpc_commit{tnx_id = LastAppliedId}] when ToTnxId > LastAppliedId ->
CurTnxId = LastAppliedId + 1,
[#cluster_rpc_mfa{mfa = MFA}] = mnesia:read(?CLUSTER_MFA, CurTnxId),
case apply_mfa(CurTnxId, MFA) of
{true, _Result} -> ok = commit(Node, CurTnxId);
{false, Error} -> mnesia:abort(Error)
end;
[#cluster_rpc_commit{tnx_id = LastAppliedId}] ->
Reason = lists:flatten(
io_lib:format(
"~p catch up failed by LastAppliedId(~p) > ToTnxId(~p)",
[Node, LastAppliedId, ToTnxId]
)
),
?SLOG(error, #{
msg => "catch_up_failed!",
last_applied_id => LastAppliedId,
to_tnx_id => ToTnxId
}),
mnesia:abort(Reason)
end.
commit(Node, TnxId) -> commit(Node, TnxId) ->
ok = mnesia:write(?CLUSTER_COMMIT, #cluster_rpc_commit{node = Node, tnx_id = TnxId}, write). ok = mnesia:write(?CLUSTER_COMMIT, #cluster_rpc_commit{node = Node, tnx_id = TnxId}, write).
@ -365,33 +393,44 @@ do_fast_forward_to_commit(ToTnxId, State = #{node := Node}) ->
end. end.
get_latest_id() -> get_latest_id() ->
case mnesia:last(?CLUSTER_MFA) of get_latest_id(tnx).
get_latest_id(IsolationLevel) ->
F =
case IsolationLevel of
tnx -> fun mnesia:last/1;
dirty -> fun mnesia:dirty_last/1
end,
case F(?CLUSTER_MFA) of
'$end_of_table' -> 0; '$end_of_table' -> 0;
Id -> Id Id -> Id
end. end.
init_mfa(Node, MFA) -> init_mfa(Node, MFA, LatestIdLastSeen) ->
mnesia:write_lock_table(?CLUSTER_MFA), mnesia:write_lock_table(?CLUSTER_MFA),
LatestId = get_latest_id(), LatestId = get_latest_id(),
ok = do_catch_up_in_one_trans(LatestId, Node), case LatestIdLastSeen =:= LatestId of
TnxId = LatestId + 1, true ->
MFARec = #cluster_rpc_mfa{ TnxId = LatestId + 1,
tnx_id = TnxId, MFARec = #cluster_rpc_mfa{
mfa = MFA, tnx_id = TnxId,
initiator = Node, mfa = MFA,
created_at = erlang:localtime() initiator = Node,
}, created_at = erlang:localtime()
ok = mnesia:write(?CLUSTER_MFA, MFARec, write), },
ok = commit(Node, TnxId), ok = mnesia:write(?CLUSTER_MFA, MFARec, write),
case apply_mfa(TnxId, MFA) of ok = commit(Node, TnxId),
{true, Result} -> {ok, TnxId, Result}; case apply_mfa(TnxId, MFA, init) of
{false, Error} -> mnesia:abort(Error) {true, Result} -> {ok, TnxId, Result};
end. {false, Error} -> mnesia:abort(Error)
end;
do_catch_up_in_one_trans(LatestId, Node) -> false ->
case do_catch_up(LatestId, Node) of ?SLOG(error, #{
caught_up -> ok; msg => stale_view_of_cluster_state,
ok -> do_catch_up_in_one_trans(LatestId, Node) tnx_id => LatestId,
last_seen_tnx_id => LatestIdLastSeen
}),
mnesia:abort({error, stale_view_of_cluster_state})
end. end.
transaction(Func, Args) -> transaction(Func, Args) ->
@ -433,7 +472,7 @@ trans_query(TnxId) ->
-define(TO_BIN(_B_), iolist_to_binary(io_lib:format("~p", [_B_]))). -define(TO_BIN(_B_), iolist_to_binary(io_lib:format("~p", [_B_]))).
apply_mfa(TnxId, {M, F, A}) -> apply_mfa(TnxId, {M, F, A}, Kind) ->
Res = Res =
try try
erlang:apply(M, F, A) erlang:apply(M, F, A)
@ -444,7 +483,7 @@ apply_mfa(TnxId, {M, F, A}) ->
{error, #{exception => Class, reason => Reason, stacktrace => Stacktrace}} {error, #{exception => Class, reason => Reason, stacktrace => Stacktrace}}
end, end,
%% Do not log args as it might be sensitive information %% Do not log args as it might be sensitive information
Meta = #{tnx_id => TnxId, entrypoint => format_mfa(M, F, length(A))}, Meta = #{kind => Kind, tnx_id => TnxId, entrypoint => format_mfa(M, F, length(A))},
IsSuccess = is_success(Res), IsSuccess = is_success(Res),
log_and_alarm(IsSuccess, Res, Meta), log_and_alarm(IsSuccess, Res, Meta),
{IsSuccess, Res}. {IsSuccess, Res}.
@ -475,21 +514,21 @@ wait_for_all_nodes_commit(TnxId, Delay, Remain) ->
[] -> [] ->
ok; ok;
Nodes -> Nodes ->
{retry, Nodes} {peers_lagging, Nodes}
end. end.
wait_for_nodes_commit(RequiredNum, TnxId, Delay, Remain) -> wait_for_nodes_commit(RequiredSyncs, TnxId, Delay, Remain) ->
ok = timer:sleep(Delay), ok = timer:sleep(Delay),
case length(synced_nodes(TnxId)) >= RequiredNum of case length(synced_nodes(TnxId)) >= RequiredSyncs of
true -> true ->
ok; ok;
false when Remain > 0 -> false when Remain > 0 ->
wait_for_nodes_commit(RequiredNum, TnxId, Delay, Remain - Delay); wait_for_nodes_commit(RequiredSyncs, TnxId, Delay, Remain - Delay);
false -> false ->
case lagging_node(TnxId) of case lagging_node(TnxId) of
%% All commit but The succeedNum > length(nodes()). %% All commit but The succeedNum > length(nodes()).
[] -> ok; [] -> ok;
Nodes -> {retry, Nodes} Nodes -> {peers_lagging, Nodes}
end end
end. end.

View File

@ -92,7 +92,7 @@ get_node_and_config(KeyPath) ->
) -> ) ->
{ok, emqx_config:update_result()} | {error, emqx_config:update_error()}. {ok, emqx_config:update_result()} | {error, emqx_config:update_error()}.
update(KeyPath, UpdateReq, Opts) -> update(KeyPath, UpdateReq, Opts) ->
check_cluster_rpc_result(emqx_conf_proto_v1:update(KeyPath, UpdateReq, Opts)). emqx_conf_proto_v1:update(KeyPath, UpdateReq, Opts).
%% @doc Update the specified node's key path in local-override.conf. %% @doc Update the specified node's key path in local-override.conf.
-spec update( -spec update(
@ -111,7 +111,7 @@ update(Node, KeyPath, UpdateReq, Opts) ->
-spec remove(emqx_map_lib:config_key_path(), emqx_config:update_opts()) -> -spec remove(emqx_map_lib:config_key_path(), emqx_config:update_opts()) ->
{ok, emqx_config:update_result()} | {error, emqx_config:update_error()}. {ok, emqx_config:update_result()} | {error, emqx_config:update_error()}.
remove(KeyPath, Opts) -> remove(KeyPath, Opts) ->
check_cluster_rpc_result(emqx_conf_proto_v1:remove_config(KeyPath, Opts)). emqx_conf_proto_v1:remove_config(KeyPath, Opts).
%% @doc remove the specified node's key path in local-override.conf. %% @doc remove the specified node's key path in local-override.conf.
-spec remove(node(), emqx_map_lib:config_key_path(), emqx_config:update_opts()) -> -spec remove(node(), emqx_map_lib:config_key_path(), emqx_config:update_opts()) ->
@ -125,7 +125,7 @@ remove(Node, KeyPath, Opts) ->
-spec reset(emqx_map_lib:config_key_path(), emqx_config:update_opts()) -> -spec reset(emqx_map_lib:config_key_path(), emqx_config:update_opts()) ->
{ok, emqx_config:update_result()} | {error, emqx_config:update_error()}. {ok, emqx_config:update_result()} | {error, emqx_config:update_error()}.
reset(KeyPath, Opts) -> reset(KeyPath, Opts) ->
check_cluster_rpc_result(emqx_conf_proto_v1:reset(KeyPath, Opts)). emqx_conf_proto_v1:reset(KeyPath, Opts).
%% @doc reset the specified node's key path in local-override.conf. %% @doc reset the specified node's key path in local-override.conf.
-spec reset(node(), emqx_map_lib:config_key_path(), emqx_config:update_opts()) -> -spec reset(node(), emqx_map_lib:config_key_path(), emqx_config:update_opts()) ->
@ -208,27 +208,6 @@ gen_example(File, SchemaModule, I18nFile, Lang) ->
Example = hocon_schema_example:gen(SchemaModule, Opts), Example = hocon_schema_example:gen(SchemaModule, Opts),
file:write_file(File, Example). file:write_file(File, Example).
check_cluster_rpc_result(Result) ->
case Result of
{ok, _TnxId, Res} ->
Res;
{retry, TnxId, Res, Nodes} ->
%% The init MFA return ok, but other nodes failed.
%% We return ok and alert an alarm.
?SLOG(error, #{
msg => "failed_to_update_config_in_cluster",
nodes => Nodes,
tnx_id => TnxId
}),
Res;
%% all MFA return not ok or {ok, term()}.
{error, Error} ->
%% a lot of the callers do not handle
%% this error return, some even ignore
%% throw here to ensure the code will not proceed
erlang:throw(Error)
end.
%% Only gen hot_conf schema, not all configuration fields. %% Only gen hot_conf schema, not all configuration fields.
gen_hot_conf_schema(File) -> gen_hot_conf_schema(File) ->
{ApiSpec0, Components0} = emqx_dashboard_swagger:spec( {ApiSpec0, Components0} = emqx_dashboard_swagger:spec(

View File

@ -61,7 +61,7 @@ get_all(KeyPath) ->
update_config_key_path(), update_config_key_path(),
emqx_config:update_request(), emqx_config:update_request(),
emqx_config:update_opts() emqx_config:update_opts()
) -> emqx_cluster_rpc:multicall_return(). ) -> {ok, emqx_config:update_result()} | {error, emqx_config:update_error()}.
update(KeyPath, UpdateReq, Opts) -> update(KeyPath, UpdateReq, Opts) ->
emqx_cluster_rpc:multicall(emqx, update_config, [KeyPath, UpdateReq, Opts]). emqx_cluster_rpc:multicall(emqx, update_config, [KeyPath, UpdateReq, Opts]).
@ -78,7 +78,7 @@ update(Node, KeyPath, UpdateReq, Opts) ->
rpc:call(Node, emqx, update_config, [KeyPath, UpdateReq, Opts], 5000). rpc:call(Node, emqx, update_config, [KeyPath, UpdateReq, Opts], 5000).
-spec remove_config(update_config_key_path(), emqx_config:update_opts()) -> -spec remove_config(update_config_key_path(), emqx_config:update_opts()) ->
emqx_cluster_rpc:multicall_result(). {ok, emqx_config:update_result()} | {error, emqx_config:update_error()}.
remove_config(KeyPath, Opts) -> remove_config(KeyPath, Opts) ->
emqx_cluster_rpc:multicall(emqx, remove_config, [KeyPath, Opts]). emqx_cluster_rpc:multicall(emqx, remove_config, [KeyPath, Opts]).
@ -90,7 +90,7 @@ remove_config(Node, KeyPath, Opts) ->
rpc:call(Node, emqx, remove_config, [KeyPath, Opts], 5000). rpc:call(Node, emqx, remove_config, [KeyPath, Opts], 5000).
-spec reset(update_config_key_path(), emqx_config:update_opts()) -> -spec reset(update_config_key_path(), emqx_config:update_opts()) ->
emqx_cluster_rpc:multicall_return(). {ok, emqx_config:update_result()} | {error, emqx_config:update_error()}.
reset(KeyPath, Opts) -> reset(KeyPath, Opts) ->
emqx_cluster_rpc:multicall(emqx, reset_config, [KeyPath, Opts]). emqx_cluster_rpc:multicall(emqx, reset_config, [KeyPath, Opts]).

View File

@ -69,13 +69,13 @@ t_base_test(_Config) ->
?assertEqual(emqx_cluster_rpc:status(), {atomic, []}), ?assertEqual(emqx_cluster_rpc:status(), {atomic, []}),
Pid = self(), Pid = self(),
MFA = {M, F, A} = {?MODULE, echo, [Pid, test]}, MFA = {M, F, A} = {?MODULE, echo, [Pid, test]},
{ok, TnxId, ok} = emqx_cluster_rpc:multicall(M, F, A), {ok, TnxId, ok} = multicall(M, F, A),
{atomic, Query} = emqx_cluster_rpc:query(TnxId), {atomic, Query} = emqx_cluster_rpc:query(TnxId),
?assertEqual(MFA, maps:get(mfa, Query)), ?assertEqual(MFA, maps:get(mfa, Query)),
?assertEqual(node(), maps:get(initiator, Query)), ?assertEqual(node(), maps:get(initiator, Query)),
?assert(maps:is_key(created_at, Query)), ?assert(maps:is_key(created_at, Query)),
?assertEqual(ok, receive_msg(3, test)), ?assertEqual(ok, receive_msg(3, test)),
?assertEqual({ok, 2, ok}, emqx_cluster_rpc:multicall(M, F, A)), ?assertEqual({ok, 2, ok}, multicall(M, F, A)),
{atomic, Status} = emqx_cluster_rpc:status(), {atomic, Status} = emqx_cluster_rpc:status(),
case length(Status) =:= 3 of case length(Status) =:= 3 of
true -> true ->
@ -95,7 +95,7 @@ t_commit_fail_test(_Config) ->
emqx_cluster_rpc:reset(), emqx_cluster_rpc:reset(),
{atomic, []} = emqx_cluster_rpc:status(), {atomic, []} = emqx_cluster_rpc:status(),
{M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE2)]}, {M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE2)]},
{error, "MFA return not ok"} = emqx_cluster_rpc:multicall(M, F, A), {init_failure, "MFA return not ok"} = multicall(M, F, A),
?assertEqual({atomic, []}, emqx_cluster_rpc:status()), ?assertEqual({atomic, []}, emqx_cluster_rpc:status()),
ok. ok.
@ -103,7 +103,7 @@ t_commit_crash_test(_Config) ->
emqx_cluster_rpc:reset(), emqx_cluster_rpc:reset(),
{atomic, []} = emqx_cluster_rpc:status(), {atomic, []} = emqx_cluster_rpc:status(),
{M, F, A} = {?MODULE, no_exist_function, []}, {M, F, A} = {?MODULE, no_exist_function, []},
{error, {error, Meta}} = emqx_cluster_rpc:multicall(M, F, A), {init_failure, {error, Meta}} = multicall(M, F, A),
?assertEqual(undef, maps:get(reason, Meta)), ?assertEqual(undef, maps:get(reason, Meta)),
?assertEqual(error, maps:get(exception, Meta)), ?assertEqual(error, maps:get(exception, Meta)),
?assertEqual(true, maps:is_key(stacktrace, Meta)), ?assertEqual(true, maps:is_key(stacktrace, Meta)),
@ -114,21 +114,23 @@ t_commit_ok_but_apply_fail_on_other_node(_Config) ->
emqx_cluster_rpc:reset(), emqx_cluster_rpc:reset(),
{atomic, []} = emqx_cluster_rpc:status(), {atomic, []} = emqx_cluster_rpc:status(),
MFA = {M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE1)]}, MFA = {M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE1)]},
{ok, _, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, _, ok} = multicall(M, F, A, 1, 1000),
{atomic, [Status]} = emqx_cluster_rpc:status(), {atomic, [Status]} = emqx_cluster_rpc:status(),
?assertEqual(MFA, maps:get(mfa, Status)), ?assertEqual(MFA, maps:get(mfa, Status)),
?assertEqual(node(), maps:get(node, Status)), ?assertEqual(node(), maps:get(node, Status)),
erlang:send(?NODE2, test), erlang:send(?NODE2, test),
Res = gen_server:call(?NODE2, {initiate, {M, F, A}}), Call = emqx_cluster_rpc:make_initiate_call_req(M, F, A),
?assertEqual({error, "MFA return not ok"}, Res), Res = gen_server:call(?NODE2, Call),
?assertEqual({init_failure, "MFA return not ok"}, Res),
ok. ok.
t_catch_up_status_handle_next_commit(_Config) -> t_catch_up_status_handle_next_commit(_Config) ->
emqx_cluster_rpc:reset(), emqx_cluster_rpc:reset(),
{atomic, []} = emqx_cluster_rpc:status(), {atomic, []} = emqx_cluster_rpc:status(),
{M, F, A} = {?MODULE, failed_on_node_by_odd, [erlang:whereis(?NODE1)]}, {M, F, A} = {?MODULE, failed_on_node_by_odd, [erlang:whereis(?NODE1)]},
{ok, 1, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, 1, ok} = multicall(M, F, A, 1, 1000),
{ok, 2} = gen_server:call(?NODE2, {initiate, {M, F, A}}), Call = emqx_cluster_rpc:make_initiate_call_req(M, F, A),
{ok, 2} = gen_server:call(?NODE2, Call),
ok. ok.
t_commit_ok_apply_fail_on_other_node_then_recover(_Config) -> t_commit_ok_apply_fail_on_other_node_then_recover(_Config) ->
@ -138,19 +140,19 @@ t_commit_ok_apply_fail_on_other_node_then_recover(_Config) ->
ets:insert(test, {other_mfa_result, failed}), ets:insert(test, {other_mfa_result, failed}),
ct:pal("111:~p~n", [ets:tab2list(cluster_rpc_commit)]), ct:pal("111:~p~n", [ets:tab2list(cluster_rpc_commit)]),
{M, F, A} = {?MODULE, failed_on_other_recover_after_retry, [erlang:whereis(?NODE1)]}, {M, F, A} = {?MODULE, failed_on_other_recover_after_retry, [erlang:whereis(?NODE1)]},
{ok, 1, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, 1, ok} = multicall(M, F, A, 1, 1000),
ct:pal("222:~p~n", [ets:tab2list(cluster_rpc_commit)]), ct:pal("222:~p~n", [ets:tab2list(cluster_rpc_commit)]),
ct:pal("333:~p~n", [emqx_cluster_rpc:status()]), ct:pal("333:~p~n", [emqx_cluster_rpc:status()]),
{atomic, [_Status | L]} = emqx_cluster_rpc:status(), {atomic, [_Status | L]} = emqx_cluster_rpc:status(),
?assertEqual([], L), ?assertEqual([], L),
ets:insert(test, {other_mfa_result, ok}), ets:insert(test, {other_mfa_result, ok}),
{ok, 2, ok} = emqx_cluster_rpc:multicall(io, format, ["test"], 1, 1000), {ok, 2, ok} = multicall(io, format, ["test"], 1, 1000),
ct:sleep(1000), ct:sleep(1000),
{atomic, NewStatus} = emqx_cluster_rpc:status(), {atomic, NewStatus} = emqx_cluster_rpc:status(),
?assertEqual(3, length(NewStatus)), ?assertEqual(3, length(NewStatus)),
Pid = self(), Pid = self(),
MFAEcho = {M1, F1, A1} = {?MODULE, echo, [Pid, test]}, MFAEcho = {M1, F1, A1} = {?MODULE, echo, [Pid, test]},
{ok, TnxId, ok} = emqx_cluster_rpc:multicall(M1, F1, A1), {ok, TnxId, ok} = multicall(M1, F1, A1),
{atomic, Query} = emqx_cluster_rpc:query(TnxId), {atomic, Query} = emqx_cluster_rpc:query(TnxId),
?assertEqual(MFAEcho, maps:get(mfa, Query)), ?assertEqual(MFAEcho, maps:get(mfa, Query)),
?assertEqual(node(), maps:get(initiator, Query)), ?assertEqual(node(), maps:get(initiator, Query)),
@ -167,7 +169,7 @@ t_del_stale_mfa(_Config) ->
Ids = Ids =
[ [
begin begin
{ok, TnxId, ok} = emqx_cluster_rpc:multicall(M, F, A), {ok, TnxId, ok} = multicall(M, F, A),
TnxId TnxId
end end
|| _ <- Keys || _ <- Keys
@ -176,7 +178,7 @@ t_del_stale_mfa(_Config) ->
Ids2 = Ids2 =
[ [
begin begin
{ok, TnxId, ok} = emqx_cluster_rpc:multicall(M, F, A), {ok, TnxId, ok} = multicall(M, F, A),
TnxId TnxId
end end
|| _ <- Keys2 || _ <- Keys2
@ -203,7 +205,7 @@ t_del_stale_mfa(_Config) ->
t_skip_failed_commit(_Config) -> t_skip_failed_commit(_Config) ->
emqx_cluster_rpc:reset(), emqx_cluster_rpc:reset(),
{atomic, []} = emqx_cluster_rpc:status(), {atomic, []} = emqx_cluster_rpc:status(),
{ok, 1, ok} = emqx_cluster_rpc:multicall(io, format, ["test~n"], all, 1000), {ok, 1, ok} = multicall(io, format, ["test~n"], all, 1000),
ct:sleep(180), ct:sleep(180),
{atomic, List1} = emqx_cluster_rpc:status(), {atomic, List1} = emqx_cluster_rpc:status(),
Node = node(), Node = node(),
@ -212,7 +214,7 @@ t_skip_failed_commit(_Config) ->
tnx_ids(List1) tnx_ids(List1)
), ),
{M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE1)]}, {M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE1)]},
{ok, 2, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, 2, ok} = multicall(M, F, A, 1, 1000),
2 = gen_server:call(?NODE2, skip_failed_commit, 5000), 2 = gen_server:call(?NODE2, skip_failed_commit, 5000),
{atomic, List2} = emqx_cluster_rpc:status(), {atomic, List2} = emqx_cluster_rpc:status(),
?assertEqual( ?assertEqual(
@ -224,7 +226,7 @@ t_skip_failed_commit(_Config) ->
t_fast_forward_commit(_Config) -> t_fast_forward_commit(_Config) ->
emqx_cluster_rpc:reset(), emqx_cluster_rpc:reset(),
{atomic, []} = emqx_cluster_rpc:status(), {atomic, []} = emqx_cluster_rpc:status(),
{ok, 1, ok} = emqx_cluster_rpc:multicall(io, format, ["test~n"], all, 1000), {ok, 1, ok} = multicall(io, format, ["test~n"], all, 1000),
ct:sleep(180), ct:sleep(180),
{atomic, List1} = emqx_cluster_rpc:status(), {atomic, List1} = emqx_cluster_rpc:status(),
Node = node(), Node = node(),
@ -233,11 +235,11 @@ t_fast_forward_commit(_Config) ->
tnx_ids(List1) tnx_ids(List1)
), ),
{M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE1)]}, {M, F, A} = {?MODULE, failed_on_node, [erlang:whereis(?NODE1)]},
{ok, 2, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, 2, ok} = multicall(M, F, A, 1, 1000),
{ok, 3, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, 3, ok} = multicall(M, F, A, 1, 1000),
{ok, 4, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, 4, ok} = multicall(M, F, A, 1, 1000),
{ok, 5, ok} = emqx_cluster_rpc:multicall(M, F, A, 1, 1000), {ok, 5, ok} = multicall(M, F, A, 1, 1000),
{retry, 6, ok, _} = emqx_cluster_rpc:multicall(M, F, A, 2, 1000), {peers_lagging, 6, ok, _} = multicall(M, F, A, 2, 1000),
3 = gen_server:call(?NODE2, {fast_forward_to_commit, 3}, 5000), 3 = gen_server:call(?NODE2, {fast_forward_to_commit, 3}, 5000),
4 = gen_server:call(?NODE2, {fast_forward_to_commit, 4}, 5000), 4 = gen_server:call(?NODE2, {fast_forward_to_commit, 4}, 5000),
6 = gen_server:call(?NODE2, {fast_forward_to_commit, 7}, 5000), 6 = gen_server:call(?NODE2, {fast_forward_to_commit, 7}, 5000),
@ -333,3 +335,9 @@ failed_on_other_recover_after_retry(Pid) ->
[{_, Res}] = ets:lookup(test, other_mfa_result), [{_, Res}] = ets:lookup(test, other_mfa_result),
Res Res
end. end.
multicall(M, F, A, N, T) ->
emqx_cluster_rpc:do_multicall(M, F, A, N, T).
multicall(M, F, A) ->
multicall(M, F, A, all, timer:minutes(2)).

View File

@ -70,6 +70,7 @@ end_per_testcase(_, _Config) ->
ok. ok.
t_list_raw_empty(_) -> t_list_raw_empty(_) ->
ok = emqx_config:erase(hd(emqx_connector:config_key_path())),
Result = emqx_connector:list_raw(), Result = emqx_connector:list_raw(),
?assertEqual([], Result). ?assertEqual([], Result).

View File

@ -556,7 +556,7 @@ with_gateway(GwName0, Fun) ->
end, end,
case emqx_gateway:lookup(GwName) of case emqx_gateway:lookup(GwName) of
undefined -> undefined ->
return_http_error(404, "Gateway not load"); return_http_error(404, "Gateway not loaded");
Gateway -> Gateway ->
Fun(GwName, Gateway) Fun(GwName, Gateway)
end end

View File

@ -317,7 +317,7 @@ get_plugins() ->
upload_install(post, #{body := #{<<"plugin">> := Plugin}}) when is_map(Plugin) -> upload_install(post, #{body := #{<<"plugin">> := Plugin}}) when is_map(Plugin) ->
[{FileName, Bin}] = maps:to_list(maps:without([type], Plugin)), [{FileName, Bin}] = maps:to_list(maps:without([type], Plugin)),
%% File bin is too large, we use rpc:multicall instead of cluster_rpc:multicall %% File bin is too large, we use rpc:multicall instead of cluster_rpc:multicall
%% TODO what happened when a new node join in? %% TODO what happens when a new node join in?
%% emqx_plugins_monitor should copy plugins from other core node when boot-up. %% emqx_plugins_monitor should copy plugins from other core node when boot-up.
case emqx_plugins:describe(string:trim(FileName, trailing, ".tar.gz")) of case emqx_plugins:describe(string:trim(FileName, trailing, ".tar.gz")) of
{error, #{error := "bad_info_file", return := {enoent, _}}} -> {error, #{error := "bad_info_file", return := {enoent, _}}} ->
@ -358,16 +358,11 @@ upload_install(post, #{}) ->
}}. }}.
do_install_package(FileName, Bin) -> do_install_package(FileName, Bin) ->
{Res, _} = emqx_mgmt_api_plugins_proto_v1:install_package(FileName, Bin), %% TODO: handle bad nodes
case lists:filter(fun(R) -> R =/= ok end, Res) of {[_ | _] = Res, []} = emqx_mgmt_api_plugins_proto_v1:install_package(FileName, Bin),
[] -> %% TODO: handle non-OKs
{200}; [] = lists:filter(fun(R) -> R =/= ok end, Res),
[{error, Reason} | _] -> {200}.
{400, #{
code => 'UNEXPECTED_ERROR',
message => iolist_to_binary(io_lib:format("~p", [Reason]))
}}
end.
plugin(get, #{bindings := #{name := Name}}) -> plugin(get, #{bindings := #{name := Name}}) ->
{Plugins, _} = emqx_mgmt_api_plugins_proto_v1:describe_package(Name), {Plugins, _} = emqx_mgmt_api_plugins_proto_v1:describe_package(Name),
@ -376,11 +371,11 @@ plugin(get, #{bindings := #{name := Name}}) ->
[] -> {404, #{code => 'NOT_FOUND', message => Name}} [] -> {404, #{code => 'NOT_FOUND', message => Name}}
end; end;
plugin(delete, #{bindings := #{name := Name}}) -> plugin(delete, #{bindings := #{name := Name}}) ->
{ok, _TnxId, Res} = emqx_mgmt_api_plugins_proto_v1:delete_package(Name), Res = emqx_mgmt_api_plugins_proto_v1:delete_package(Name),
return(204, Res). return(204, Res).
update_plugin(put, #{bindings := #{name := Name, action := Action}}) -> update_plugin(put, #{bindings := #{name := Name, action := Action}}) ->
{ok, _TnxId, Res} = emqx_mgmt_api_plugins_proto_v1:ensure_action(Name, Action), Res = emqx_mgmt_api_plugins_proto_v1:ensure_action(Name, Action),
return(204, Res). return(204, Res).
update_boot_order(post, #{bindings := #{name := Name}, body := Body}) -> update_boot_order(post, #{bindings := #{name := Name}, body := Body}) ->
@ -422,7 +417,8 @@ delete_package(Name) ->
ok -> ok ->
_ = emqx_plugins:ensure_disabled(Name), _ = emqx_plugins:ensure_disabled(Name),
_ = emqx_plugins:ensure_uninstalled(Name), _ = emqx_plugins:ensure_uninstalled(Name),
_ = emqx_plugins:delete_package(Name); _ = emqx_plugins:delete_package(Name),
ok;
Error -> Error ->
Error Error
end. end.
@ -430,20 +426,19 @@ delete_package(Name) ->
%% for RPC plugin update %% for RPC plugin update
ensure_action(Name, start) -> ensure_action(Name, start) ->
_ = emqx_plugins:ensure_enabled(Name), _ = emqx_plugins:ensure_enabled(Name),
_ = emqx_plugins:ensure_started(Name); _ = emqx_plugins:ensure_started(Name),
ok;
ensure_action(Name, stop) -> ensure_action(Name, stop) ->
_ = emqx_plugins:ensure_stopped(Name), _ = emqx_plugins:ensure_stopped(Name),
_ = emqx_plugins:ensure_disabled(Name); _ = emqx_plugins:ensure_disabled(Name),
ok;
ensure_action(Name, restart) -> ensure_action(Name, restart) ->
_ = emqx_plugins:ensure_enabled(Name), _ = emqx_plugins:ensure_enabled(Name),
_ = emqx_plugins:restart(Name). _ = emqx_plugins:restart(Name),
ok.
return(Code, ok) -> return(Code, ok) ->
{Code}; {Code};
return(Code, {ok, Result}) ->
{Code, Result};
return(_, {error, #{error := "bad_info_file", return := {enoent, _}, path := Path}}) ->
{404, #{code => 'NOT_FOUND', message => Path}};
return(_, {error, Reason}) -> return(_, {error, Reason}) ->
{400, #{code => 'PARAM_ERROR', message => iolist_to_binary(io_lib:format("~p", [Reason]))}}. {400, #{code => 'PARAM_ERROR', message => iolist_to_binary(io_lib:format("~p", [Reason]))}}.

View File

@ -43,11 +43,10 @@ install_package(Filename, Bin) ->
describe_package(Name) -> describe_package(Name) ->
rpc:multicall(emqx_mgmt_api_plugins, describe_package, [Name], 10000). rpc:multicall(emqx_mgmt_api_plugins, describe_package, [Name], 10000).
-spec delete_package(binary() | string()) -> emqx_cluster_rpc:multicall_return(). -spec delete_package(binary() | string()) -> ok | {error, any()}.
delete_package(Name) -> delete_package(Name) ->
emqx_cluster_rpc:multicall(emqx_mgmt_api_plugins, delete_package, [Name], all, 10000). emqx_cluster_rpc:multicall(emqx_mgmt_api_plugins, delete_package, [Name], all, 10000).
-spec ensure_action(binary() | string(), 'restart' | 'start' | 'stop') -> -spec ensure_action(binary() | string(), 'restart' | 'start' | 'stop') -> ok | {error, any()}.
emqx_cluster_rpc:multicall_return().
ensure_action(Name, Action) -> ensure_action(Name, Action) ->
emqx_cluster_rpc:multicall(emqx_mgmt_api_plugins, ensure_action, [Name, Action], all, 10000). emqx_cluster_rpc:multicall(emqx_mgmt_api_plugins, ensure_action, [Name, Action], all, 10000).

View File

@ -61,8 +61,6 @@
can_topic_match_oneof/2 can_topic_match_oneof/2
]). ]).
-export([cluster_call/3]).
-compile({no_auto_import, [float/1]}). -compile({no_auto_import, [float/1]}).
-define(EX_PLACE_HOLDER, "(\\$\\{[a-zA-Z0-9\\._]+\\})"). -define(EX_PLACE_HOLDER, "(\\$\\{[a-zA-Z0-9\\._]+\\})").
@ -307,7 +305,3 @@ can_topic_match_oneof(Topic, Filters) ->
end, end,
Filters Filters
). ).
cluster_call(Module, Func, Args) ->
{ok, _TnxId, Result} = emqx_cluster_rpc:multicall(Module, Func, Args),
Result.

View File

@ -337,7 +337,7 @@ do_ensure_started(NameVsn) ->
). ).
%% try the function, catch 'throw' exceptions as normal 'error' return %% try the function, catch 'throw' exceptions as normal 'error' return
%% other exceptions with stacktrace returned. %% other exceptions with stacktrace logged.
tryit(WhichOp, F) -> tryit(WhichOp, F) ->
try try
F() F()

View File

@ -125,20 +125,10 @@ purge_test() ->
meck_emqx() -> meck_emqx() ->
meck:new(emqx, [unstick, passthrough]), meck:new(emqx, [unstick, passthrough]),
meck:expect( meck:expect(
emqx, emqx_conf,
update_config, update,
fun(Path, Values, _Opts) -> fun(Path, Values, _Opts) ->
emqx_config:put(Path, Values) emqx_config:put(Path, Values)
end end
), ),
%meck:expect(emqx, get_config,
% fun(KeyPath, Default) ->
% Map = emqx:get_raw_config(KeyPath, Default),
% Map1 = emqx_map_lib:safe_atom_key_map(Map),
% case Map1 of
% #{states := Plugins} ->
% Map1#{states => [emqx_map_lib:safe_atom_key_map(P) ||P <- Plugins]};
% _ -> Map1
% end
% end),
ok. ok.

View File

@ -174,7 +174,7 @@ create(InstId, Group, ResourceType, Config) ->
-spec create(instance_id(), resource_group(), resource_type(), resource_config(), create_opts()) -> -spec create(instance_id(), resource_group(), resource_type(), resource_config(), create_opts()) ->
{ok, resource_data() | 'already_created'} | {error, Reason :: term()}. {ok, resource_data() | 'already_created'} | {error, Reason :: term()}.
create(InstId, Group, ResourceType, Config, Opts) -> create(InstId, Group, ResourceType, Config, Opts) ->
wrap_rpc(emqx_resource_proto_v1:create(InstId, Group, ResourceType, Config, Opts)). emqx_resource_proto_v1:create(InstId, Group, ResourceType, Config, Opts).
% -------------------------------------------- % --------------------------------------------
-spec create_local(instance_id(), resource_group(), resource_type(), resource_config()) -> -spec create_local(instance_id(), resource_group(), resource_type(), resource_config()) ->
@ -196,7 +196,7 @@ create_local(InstId, Group, ResourceType, Config, Opts) ->
-spec create_dry_run(resource_type(), resource_config()) -> -spec create_dry_run(resource_type(), resource_config()) ->
ok | {error, Reason :: term()}. ok | {error, Reason :: term()}.
create_dry_run(ResourceType, Config) -> create_dry_run(ResourceType, Config) ->
wrap_rpc(emqx_resource_proto_v1:create_dry_run(ResourceType, Config)). emqx_resource_proto_v1:create_dry_run(ResourceType, Config).
-spec create_dry_run_local(resource_type(), resource_config()) -> -spec create_dry_run_local(resource_type(), resource_config()) ->
ok | {error, Reason :: term()}. ok | {error, Reason :: term()}.
@ -211,7 +211,7 @@ recreate(InstId, ResourceType, Config) ->
-spec recreate(instance_id(), resource_type(), resource_config(), create_opts()) -> -spec recreate(instance_id(), resource_type(), resource_config(), create_opts()) ->
{ok, resource_data()} | {error, Reason :: term()}. {ok, resource_data()} | {error, Reason :: term()}.
recreate(InstId, ResourceType, Config, Opts) -> recreate(InstId, ResourceType, Config, Opts) ->
wrap_rpc(emqx_resource_proto_v1:recreate(InstId, ResourceType, Config, Opts)). emqx_resource_proto_v1:recreate(InstId, ResourceType, Config, Opts).
-spec recreate_local(instance_id(), resource_type(), resource_config()) -> -spec recreate_local(instance_id(), resource_type(), resource_config()) ->
{ok, resource_data()} | {error, Reason :: term()}. {ok, resource_data()} | {error, Reason :: term()}.
@ -225,7 +225,7 @@ recreate_local(InstId, ResourceType, Config, Opts) ->
-spec remove(instance_id()) -> ok | {error, Reason :: term()}. -spec remove(instance_id()) -> ok | {error, Reason :: term()}.
remove(InstId) -> remove(InstId) ->
wrap_rpc(emqx_resource_proto_v1:remove(InstId)). emqx_resource_proto_v1:remove(InstId).
-spec remove_local(instance_id()) -> ok | {error, Reason :: term()}. -spec remove_local(instance_id()) -> ok | {error, Reason :: term()}.
remove_local(InstId) -> remove_local(InstId) ->
@ -237,7 +237,7 @@ reset_metrics_local(InstId) ->
-spec reset_metrics(instance_id()) -> ok | {error, Reason :: term()}. -spec reset_metrics(instance_id()) -> ok | {error, Reason :: term()}.
reset_metrics(InstId) -> reset_metrics(InstId) ->
wrap_rpc(emqx_resource_proto_v1:reset_metrics(InstId)). emqx_resource_proto_v1:reset_metrics(InstId).
%% ================================================================================= %% =================================================================================
-spec query(instance_id(), Request :: term()) -> Result :: term(). -spec query(instance_id(), Request :: term()) -> Result :: term().
@ -430,11 +430,5 @@ inc_metrics_funcs(InstId) ->
safe_apply(Func, Args) -> safe_apply(Func, Args) ->
?SAFE_CALL(erlang:apply(Func, Args)). ?SAFE_CALL(erlang:apply(Func, Args)).
wrap_rpc(Ret) ->
case Ret of
{ok, _TxnId, Result} -> Result;
Failed -> Failed
end.
query_error(Reason, Msg) -> query_error(Reason, Msg) ->
{error, {?MODULE, #{reason => Reason, msg => Msg}}}. {error, {?MODULE, #{reason => Reason, msg => Msg}}}.

View File

@ -40,7 +40,7 @@ introduced_in() ->
resource_config(), resource_config(),
create_opts() create_opts()
) -> ) ->
emqx_cluster_rpc:multicall_return(resource_data()). {ok, resource_data() | 'already_created'} | {error, Reason :: term()}.
create(InstId, Group, ResourceType, Config, Opts) -> create(InstId, Group, ResourceType, Config, Opts) ->
emqx_cluster_rpc:multicall(emqx_resource, create_local, [ emqx_cluster_rpc:multicall(emqx_resource, create_local, [
InstId, Group, ResourceType, Config, Opts InstId, Group, ResourceType, Config, Opts
@ -50,7 +50,7 @@ create(InstId, Group, ResourceType, Config, Opts) ->
resource_type(), resource_type(),
resource_config() resource_config()
) -> ) ->
emqx_cluster_rpc:multicall_return(resource_data()). ok | {error, Reason :: term()}.
create_dry_run(ResourceType, Config) -> create_dry_run(ResourceType, Config) ->
emqx_cluster_rpc:multicall(emqx_resource, create_dry_run_local, [ResourceType, Config]). emqx_cluster_rpc:multicall(emqx_resource, create_dry_run_local, [ResourceType, Config]).
@ -60,16 +60,14 @@ create_dry_run(ResourceType, Config) ->
resource_config(), resource_config(),
create_opts() create_opts()
) -> ) ->
emqx_cluster_rpc:multicall_return(resource_data()). {ok, resource_data()} | {error, Reason :: term()}.
recreate(InstId, ResourceType, Config, Opts) -> recreate(InstId, ResourceType, Config, Opts) ->
emqx_cluster_rpc:multicall(emqx_resource, recreate_local, [InstId, ResourceType, Config, Opts]). emqx_cluster_rpc:multicall(emqx_resource, recreate_local, [InstId, ResourceType, Config, Opts]).
-spec remove(instance_id()) -> -spec remove(instance_id()) -> ok | {error, Reason :: term()}.
emqx_cluster_rpc:multicall_return(ok).
remove(InstId) -> remove(InstId) ->
emqx_cluster_rpc:multicall(emqx_resource, remove_local, [InstId]). emqx_cluster_rpc:multicall(emqx_resource, remove_local, [InstId]).
-spec reset_metrics(instance_id()) -> -spec reset_metrics(instance_id()) -> ok | {error, any()}.
emqx_cluster_rpc:multicall_return(ok).
reset_metrics(InstId) -> reset_metrics(InstId) ->
emqx_cluster_rpc:multicall(emqx_resource, reset_metrics_local, [InstId]). emqx_cluster_rpc:multicall(emqx_resource, reset_metrics_local, [InstId]).

View File

@ -321,7 +321,7 @@ replace_sql_clrf(#{<<"sql">> := SQL} = Params) ->
end. end.
'/rules/:id/reset_metrics'(put, #{bindings := #{id := RuleId}}) -> '/rules/:id/reset_metrics'(put, #{bindings := #{id := RuleId}}) ->
case emqx_rule_engine_proto_v1:reset_metrics(RuleId) of case emqx_rule_engine_proto_v1:reset_metrics(RuleId) of
{ok, _TxnId, _Result} -> ok ->
{200, <<"Reset Success">>}; {200, <<"Reset Success">>};
Failed -> Failed ->
{400, #{ {400, #{

View File

@ -30,7 +30,6 @@
introduced_in() -> introduced_in() ->
"5.0.0". "5.0.0".
-spec reset_metrics(rule_id()) -> -spec reset_metrics(rule_id()) -> ok | {error, any()}.
emqx_cluster_rpc:multicall_return(ok).
reset_metrics(RuleId) -> reset_metrics(RuleId) ->
emqx_cluster_rpc:multicall(emqx_rule_engine, reset_metrics_for_rule, [RuleId]). emqx_cluster_rpc:multicall(emqx_rule_engine, reset_metrics_for_rule, [RuleId]).