From 12904d797fe0677fc50b487c3b5387a63aa98fb3 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Mon, 25 Jul 2022 23:51:23 +0800 Subject: [PATCH 01/14] feat(resource): first commit for batching/async/caching mechanism --- apps/emqx_resource/include/emqx_resource.hrl | 1 + .../include/emqx_resource_utils.hrl | 5 +- apps/emqx_resource/src/emqx_resource.erl | 58 +- .../src/emqx_resource_manager.erl | 30 +- apps/emqx_resource/src/emqx_resource_sup.erl | 4 +- .../emqx_resource/src/emqx_resource_utils.erl | 17 + .../src/emqx_resource_worker.erl | 252 ++++++ apps/emqx_resource/src/replayq.erl | 779 ++++++++++++++++++ 8 files changed, 1091 insertions(+), 55 deletions(-) create mode 100644 apps/emqx_resource/src/emqx_resource_utils.erl create mode 100644 apps/emqx_resource/src/emqx_resource_worker.erl create mode 100644 apps/emqx_resource/src/replayq.erl diff --git a/apps/emqx_resource/include/emqx_resource.hrl b/apps/emqx_resource/include/emqx_resource.hrl index dd384af7c..d6f959510 100644 --- a/apps/emqx_resource/include/emqx_resource.hrl +++ b/apps/emqx_resource/include/emqx_resource.hrl @@ -54,3 +54,4 @@ -type after_query_fun() :: {fun((...) -> ok), Args :: [term()]}. -define(TEST_ID_PREFIX, "_test_:"). +-define(RES_METRICS, resource_metrics). diff --git a/apps/emqx_resource/include/emqx_resource_utils.hrl b/apps/emqx_resource/include/emqx_resource_utils.hrl index 8d94746eb..3df64b1e5 100644 --- a/apps/emqx_resource/include/emqx_resource_utils.hrl +++ b/apps/emqx_resource/include/emqx_resource_utils.hrl @@ -15,7 +15,7 @@ %%-------------------------------------------------------------------- -define(SAFE_CALL(_EXP_), - ?SAFE_CALL(_EXP_, ok) + ?SAFE_CALL(_EXP_, {error, {_EXCLASS_, _EXCPTION_, _ST_}}) ). -define(SAFE_CALL(_EXP_, _EXP_ON_FAIL_), @@ -24,8 +24,7 @@ (_EXP_) catch _EXCLASS_:_EXCPTION_:_ST_ -> - _EXP_ON_FAIL_, - {error, {_EXCLASS_, _EXCPTION_, _ST_}} + _EXP_ON_FAIL_ end end() ). diff --git a/apps/emqx_resource/src/emqx_resource.erl b/apps/emqx_resource/src/emqx_resource.erl index 33f0d0a3d..793b9f446 100644 --- a/apps/emqx_resource/src/emqx_resource.erl +++ b/apps/emqx_resource/src/emqx_resource.erl @@ -83,8 +83,7 @@ stop/1, %% query the instance query/2, - %% query the instance with after_query() - query/3 + query_async/3 ]). %% Direct calls to the callback module @@ -111,6 +110,8 @@ list_group_instances/1 ]). +-export([inc_metrics_funcs/1, inc_success/1, inc_failed/1]). + -optional_callbacks([ on_query/4, on_get_status/2 @@ -150,16 +151,16 @@ is_resource_mod(Module) -> -spec query_success(after_query()) -> ok. query_success(undefined) -> ok; -query_success({OnSucc, _}) -> apply_query_after_calls(OnSucc). +query_success({OnSucc, _}) -> exec_query_after_calls(OnSucc). -spec query_failed(after_query()) -> ok. query_failed(undefined) -> ok; -query_failed({_, OnFailed}) -> apply_query_after_calls(OnFailed). +query_failed({_, OnFailed}) -> exec_query_after_calls(OnFailed). -apply_query_after_calls(Funcs) -> +exec_query_after_calls(Funcs) -> lists:foreach( - fun({Fun, Args}) -> - safe_apply(Fun, Args) + fun({Fun, Arg}) -> + emqx_resource_utils:safe_exec(Fun, Arg) end, Funcs ). @@ -243,29 +244,12 @@ reset_metrics(ResId) -> %% ================================================================================= -spec query(resource_id(), Request :: term()) -> Result :: term(). query(ResId, Request) -> - query(ResId, Request, inc_metrics_funcs(ResId)). + emqx_resource_worker:query(ResId, Request). -%% same to above, also defines what to do when the Module:on_query success or failed -%% it is the duty of the Module to apply the `after_query()` functions. --spec query(resource_id(), Request :: term(), after_query()) -> Result :: term(). -query(ResId, Request, AfterQuery) -> - case emqx_resource_manager:ets_lookup(ResId) of - {ok, _Group, #{mod := Mod, state := ResourceState, status := connected}} -> - %% the resource state is readonly to Module:on_query/4 - %% and the `after_query()` functions should be thread safe - ok = emqx_metrics_worker:inc(resource_metrics, ResId, matched), - try - Mod:on_query(ResId, Request, AfterQuery, ResourceState) - catch - Err:Reason:ST -> - emqx_metrics_worker:inc(resource_metrics, ResId, exception), - erlang:raise(Err, Reason, ST) - end; - {ok, _Group, _Data} -> - query_error(not_connected, <<"resource not connected">>); - {error, not_found} -> - query_error(not_found, <<"resource not found">>) - end. +-spec query_async(resource_id(), Request :: term(), emqx_resource_worker:reply_fun()) -> + ok. +query_async(ResId, Request, ReplyFun) -> + emqx_resource_worker:query_async(ResId, Request, ReplyFun). -spec start(resource_id()) -> ok | {error, Reason :: term()}. start(ResId) -> @@ -429,16 +413,16 @@ check_and_do(ResourceType, RawConfig, Do) when is_function(Do) -> %% ================================================================================= +inc_success(ResId) -> + emqx_metrics_worker:inc(?RES_METRICS, ResId, success). + +inc_failed(ResId) -> + emqx_metrics_worker:inc(?RES_METRICS, ResId, failed). + filter_instances(Filter) -> [Id || #{id := Id, mod := Mod} <- list_instances_verbose(), Filter(Id, Mod)]. inc_metrics_funcs(ResId) -> - OnFailed = [{fun emqx_metrics_worker:inc/3, [resource_metrics, ResId, failed]}], - OnSucc = [{fun emqx_metrics_worker:inc/3, [resource_metrics, ResId, success]}], + OnSucc = [{fun ?MODULE:inc_success/1, ResId}], + OnFailed = [{fun ?MODULE:inc_failed/1, ResId}], {OnSucc, OnFailed}. - -safe_apply(Func, Args) -> - ?SAFE_CALL(erlang:apply(Func, Args)). - -query_error(Reason, Msg) -> - {error, {?MODULE, #{reason => Reason, msg => Msg}}}. diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index 82be9c58f..f1fa36173 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -109,9 +109,9 @@ create(MgrId, ResId, Group, ResourceType, Config, Opts) -> % The state machine will make the actual call to the callback/resource module after init ok = emqx_resource_manager_sup:ensure_child(MgrId, ResId, Group, ResourceType, Config, Opts), ok = emqx_metrics_worker:create_metrics( - resource_metrics, + ?RES_METRICS, ResId, - [matched, success, failed, exception], + [matched, success, failed, exception, resource_error], [matched] ), case maps:get(start_after_created, Opts, true) of @@ -207,12 +207,12 @@ ets_lookup(ResId) -> %% @doc Get the metrics for the specified resource get_metrics(ResId) -> - emqx_metrics_worker:get_metrics(resource_metrics, ResId). + emqx_metrics_worker:get_metrics(?RES_METRICS, ResId). %% @doc Reset the metrics for the specified resource -spec reset_metrics(resource_id()) -> ok. reset_metrics(ResId) -> - emqx_metrics_worker:reset_metrics(resource_metrics, ResId). + emqx_metrics_worker:reset_metrics(?RES_METRICS, ResId). %% @doc Returns the data for all resources -spec list_all() -> [resource_data()] | []. @@ -298,8 +298,7 @@ handle_event({call, From}, stop, stopped, _Data) -> {keep_state_and_data, [{reply, From, ok}]}; handle_event({call, From}, stop, _State, Data) -> Result = stop_resource(Data), - UpdatedData = Data#data{status = disconnected}, - {next_state, stopped, UpdatedData, [{reply, From, Result}]}; + {next_state, stopped, Data, [{reply, From, Result}]}; % Called when a resource is to be stopped and removed. handle_event({call, From}, {remove, ClearMetrics}, _State, Data) -> handle_remove_event(From, ClearMetrics, Data); @@ -315,9 +314,10 @@ handle_event({call, From}, health_check, _State, Data) -> handle_manually_health_check(From, Data); % State: CONNECTING handle_event(enter, _OldState, connecting, Data) -> + UpdatedData = Data#data{status = connected}, insert_cache(Data#data.id, Data#data.group, Data), Actions = [{state_timeout, 0, health_check}], - {keep_state_and_data, Actions}; + {keep_state, UpdatedData, Actions}; handle_event(internal, start_resource, connecting, Data) -> start_resource(Data, undefined); handle_event(state_timeout, health_check, connecting, Data) -> @@ -326,22 +326,24 @@ handle_event(state_timeout, health_check, connecting, Data) -> %% The connected state is entered after a successful on_start/2 of the callback mod %% and successful health_checks handle_event(enter, _OldState, connected, Data) -> - insert_cache(Data#data.id, Data#data.group, Data), + UpdatedData = Data#data{status = connected}, + insert_cache(Data#data.id, Data#data.group, UpdatedData), _ = emqx_alarm:deactivate(Data#data.id), Actions = [{state_timeout, ?HEALTHCHECK_INTERVAL, health_check}], - {next_state, connected, Data, Actions}; + {next_state, connected, UpdatedData, Actions}; handle_event(state_timeout, health_check, connected, Data) -> handle_connected_health_check(Data); %% State: DISCONNECTED handle_event(enter, _OldState, disconnected, Data) -> - insert_cache(Data#data.id, Data#data.group, Data), - handle_disconnected_state_enter(Data); + UpdatedData = Data#data{status = disconnected}, + insert_cache(Data#data.id, Data#data.group, UpdatedData), + handle_disconnected_state_enter(UpdatedData); handle_event(state_timeout, auto_retry, disconnected, Data) -> start_resource(Data, undefined); %% State: STOPPED %% The stopped state is entered after the resource has been explicitly stopped handle_event(enter, _OldState, stopped, Data) -> - UpdatedData = Data#data{status = disconnected}, + UpdatedData = Data#data{status = stopped}, insert_cache(Data#data.id, Data#data.group, UpdatedData), {next_state, stopped, UpdatedData}; % Ignore all other events @@ -415,7 +417,7 @@ handle_disconnected_state_enter(Data) -> handle_remove_event(From, ClearMetrics, Data) -> stop_resource(Data), case ClearMetrics of - true -> ok = emqx_metrics_worker:clear_metrics(resource_metrics, Data#data.id); + true -> ok = emqx_metrics_worker:clear_metrics(?RES_METRICS, Data#data.id); false -> ok end, {stop_and_reply, normal, [{reply, From, ok}]}. @@ -433,7 +435,7 @@ start_resource(Data, From) -> _ = maybe_alarm(disconnected, Data#data.id), %% Keep track of the error reason why the connection did not work %% so that the Reason can be returned when the verification call is made. - UpdatedData = Data#data{status = disconnected, error = Reason}, + UpdatedData = Data#data{error = Reason}, Actions = maybe_reply([], From, Err), {next_state, disconnected, UpdatedData, Actions} end. diff --git a/apps/emqx_resource/src/emqx_resource_sup.erl b/apps/emqx_resource/src/emqx_resource_sup.erl index 1120723c3..84458f0d5 100644 --- a/apps/emqx_resource/src/emqx_resource_sup.erl +++ b/apps/emqx_resource/src/emqx_resource_sup.erl @@ -15,6 +15,8 @@ %%-------------------------------------------------------------------- -module(emqx_resource_sup). +-include("emqx_resource.hrl"). + -behaviour(supervisor). -export([start_link/0]). @@ -29,7 +31,7 @@ start_link() -> init([]) -> SupFlags = #{strategy => one_for_one, intensity => 10, period => 10}, - Metrics = emqx_metrics_worker:child_spec(resource_metrics), + Metrics = emqx_metrics_worker:child_spec(?RES_METRICS), ResourceManager = #{ diff --git a/apps/emqx_resource/src/emqx_resource_utils.erl b/apps/emqx_resource/src/emqx_resource_utils.erl new file mode 100644 index 000000000..715691d2a --- /dev/null +++ b/apps/emqx_resource/src/emqx_resource_utils.erl @@ -0,0 +1,17 @@ +%%-------------------------------------------------------------------- +%% Copyright (c) 2020-2022 EMQ Technologies Co., Ltd. All Rights Reserved. +%% +%% Licensed under the Apache License, Version 2.0 (the "License"); +%% you may not use this file except in compliance with the License. +%% You may obtain a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, software +%% distributed under the License is distributed on an "AS IS" BASIS, +%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +%% See the License for the specific language governing permissions and +%% limitations under the License. +%%-------------------------------------------------------------------- + +-module(emqx_resource_utils). diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl new file mode 100644 index 000000000..6c3b05830 --- /dev/null +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -0,0 +1,252 @@ +%%-------------------------------------------------------------------- +%% Copyright (c) 2020-2022 EMQ Technologies Co., Ltd. All Rights Reserved. +%% +%% Licensed under the Apache License, Version 2.0 (the "License"); +%% you may not use this file except in compliance with the License. +%% You may obtain a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, software +%% distributed under the License is distributed on an "AS IS" BASIS, +%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +%% See the License for the specific language governing permissions and +%% limitations under the License. +%%-------------------------------------------------------------------- + +%% An FIFO queue using ETS-ReplayQ as backend. + +-module(emqx_resource_worker). + +-include("emqx_resource.hrl"). +-include("emqx_resource_utils.hrl"). +-include_lib("emqx/include/logger.hrl"). + +-behaviour(gen_statem). + +-export([ + start_link/2, + query/2, + query_async/3, + query_mfa/3 +]). + +-export([ + callback_mode/0, + init/1 +]). + +-export([do/3]). + +%% count +-define(DEFAULT_BATCH_SIZE, 100). +%% milliseconds +-define(DEFAULT_BATCH_TIME, 10). + +-define(QUERY(FROM, REQUEST), {FROM, REQUEST}). +-define(REPLY(FROM, REQUEST, RESULT), {FROM, REQUEST, RESULT}). +-define(EXPAND(RESULT, BATCH), [?REPLY(FROM, REQUEST, RESULT) || ?QUERY(FROM, REQUEST) <- BATCH]). + +-type id() :: binary(). +-type request() :: term(). +-type result() :: term(). +-type reply_fun() :: {fun((result(), Args :: term()) -> any()), Args :: term()}. +-type from() :: pid() | reply_fun(). + +-callback batcher_flush(Acc :: [{from(), request()}], CbState :: term()) -> + {{from(), result()}, NewCbState :: term()}. + +callback_mode() -> [state_functions]. + +start_link(Id, Opts) -> + gen_statem:start_link({local, name(Id)}, ?MODULE, {Id, Opts}, []). + +-spec query(id(), request()) -> ok. +query(Id, Request) -> + gen_statem:call(name(Id), {query, Request}). + +-spec query_async(id(), request(), reply_fun()) -> ok. +query_async(Id, Request, ReplyFun) -> + gen_statem:cast(name(Id), {query, Request, ReplyFun}). + +-spec name(id()) -> atom(). +name(Id) -> + Mod = atom_to_binary(?MODULE, utf8), + <>. + +disk_cache_dir(Id) -> + filename:join([emqx:data_dir(), Id, cache]). + +init({Id, Opts}) -> + BatchSize = maps:get(batch_size, Opts, ?DEFAULT_BATCH_SIZE), + Queue = + case maps:get(cache_enabled, Opts, true) of + true -> replayq:open(#{dir => disk_cache_dir(Id), seg_bytes => 10000000}); + false -> undefined + end, + St = #{ + id => Id, + batch_enabled => maps:get(batch_enabled, Opts, true), + batch_size => BatchSize, + batch_time => maps:get(batch_time, Opts, ?DEFAULT_BATCH_TIME), + cache_queue => Queue, + acc => [], + acc_left => BatchSize, + tref => undefined + }, + {ok, do, St}. + +do(cast, {query, Request, ReplyFun}, #{batch_enabled := true} = State) -> + do_acc(ReplyFun, Request, State); +do(cast, {query, Request, ReplyFun}, #{batch_enabled := false} = State) -> + do_query(ReplyFun, Request, State); +do({call, From}, {query, Request}, #{batch_enabled := true} = State) -> + do_acc(From, Request, State); +do({call, From}, {query, Request}, #{batch_enabled := false} = State) -> + do_query(From, Request, State); +do(info, {flush, Ref}, St = #{tref := {_TRef, Ref}}) -> + {keep_state, flush(St#{tref := undefined})}; +do(info, {flush, _Ref}, _St) -> + keep_state_and_data; +do(info, Info, _St) -> + ?SLOG(error, #{msg => unexpected_msg, info => Info}), + keep_state_and_data. + +do_acc(From, Request, #{acc := Acc, acc_left := Left} = St0) -> + Acc1 = [?QUERY(From, Request) | Acc], + St = St0#{acc := Acc1, acc_left := Left - 1}, + case Left =< 1 of + true -> {keep_state, flush(St)}; + false -> {keep_state, ensure_flush_timer(St)} + end. + +do_query(From, Request, #{id := Id, cache_queue := Q0} = St0) -> + Result = call_query(Id, Request), + Q1 = reply_caller(Id, Q0, ?REPLY(From, Request, Result)), + {keep_state, St0#{cache_queue := Q1}}. + +flush(#{acc := []} = St) -> + St; +flush( + #{ + id := Id, + acc := Batch, + batch_size := Size, + cache_queue := Q0 + } = St +) -> + BatchResults = call_batch_query(Id, Batch), + Q1 = batch_reply_caller(Id, Q0, BatchResults), + cancel_flush_timer( + St#{ + acc_left := Size, + acc := [], + cache_queue := Q1 + } + ). + +maybe_append_cache(undefined, _Request) -> undefined; +maybe_append_cache(Q, Request) -> replayq:append(Q, Request). + +batch_reply_caller(Id, Q, BatchResults) -> + lists:foldl( + fun(Reply, Q1) -> + reply_caller(Id, Q1, Reply) + end, + Q, + BatchResults + ). + +reply_caller(Id, Q, ?REPLY({ReplyFun, Args}, Request, Result)) when is_function(ReplyFun) -> + ?SAFE_CALL(ReplyFun(Result, Args)), + handle_query_result(Id, Q, Request, Result); +reply_caller(Id, Q, ?REPLY(From, Request, Result)) -> + gen_statem:reply(From, Result), + handle_query_result(Id, Q, Request, Result). + +handle_query_result(Id, Q, _Request, ok) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, success), + Q; +handle_query_result(Id, Q, _Request, {ok, _}) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, success), + Q; +handle_query_result(Id, Q, _Request, {error, _}) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, failed), + Q; +handle_query_result(Id, Q, Request, {error, {resource_error, #{reason := not_connected}}}) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, resource_error), + maybe_append_cache(Q, Request); +handle_query_result(Id, Q, _Request, {error, {resource_error, #{}}}) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, resource_error), + Q; +handle_query_result(Id, Q, Request, {error, {exception, _}}) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, exception), + maybe_append_cache(Q, Request). + +call_query(Id, Request) -> + ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched), + case emqx_resource_manager:ets_lookup(Id) of + {ok, _Group, #{mod := Mod, state := ResourceState, status := connected}} -> + try Mod:on_query(Id, Request, ResourceState) of + Result -> Result + catch + Err:Reason:ST -> + ModB = atom_to_binary(Mod, utf8), + Msg = <<"call failed, func: ", ModB/binary, ":on_query/3">>, + exception_error(Reason, Msg, {Err, Reason, ST}) + end; + {ok, _Group, #{status := stopped}} -> + resource_error(stopped, <<"resource stopped or disabled">>); + {ok, _Group, _Data} -> + resource_error(not_connected, <<"resource not connected">>); + {error, not_found} -> + resource_error(not_found, <<"resource not found">>) + end. + +call_batch_query(Id, Batch) -> + ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched, length(Batch)), + case emqx_resource_manager:ets_lookup(Id) of + {ok, _Group, #{mod := Mod, state := ResourceState, status := connected}} -> + try Mod:on_batch_query(Id, Batch, ResourceState) of + BatchResults -> BatchResults + catch + Err:Reason:ST -> + ModB = atom_to_binary(Mod, utf8), + Msg = <<"call failed, func: ", ModB/binary, ":on_batch_query/3">>, + ?EXPAND(exception_error(Reason, Msg, {Err, Reason, ST}), Batch) + end; + {ok, _Group, _Data} -> + ?EXPAND(resource_error(not_connected, <<"resource not connected">>), Batch); + {error, not_found} -> + ?EXPAND(resource_error(not_found, <<"resource not found">>), Batch) + end. + +resource_error(Reason, Msg) -> + {error, {resource_error, #{reason => Reason, msg => Msg}}}. +exception_error(Reason, Msg, Details) -> + {error, {exception, #{reason => Reason, msg => Msg, details => Details}}}. + +%% ========================================== +ensure_flush_timer(St = #{tref := undefined, batch_time := T}) -> + Ref = make_ref(), + TRef = erlang:send_after(T, self(), {flush, Ref}), + St#{tref => {TRef, Ref}}; +ensure_flush_timer(St) -> + St. + +cancel_flush_timer(St = #{tref := undefined}) -> + St; +cancel_flush_timer(St = #{tref := {TRef, _Ref}}) -> + _ = erlang:cancel_timer(TRef), + St#{tref => undefined}. + +query_mfa(InsertMode, Request, SyncTimeout) -> + {?MODULE, query_fun(InsertMode), query_args(InsertMode, Request, SyncTimeout)}. + +query_fun(<<"sync">>) -> query; +query_fun(<<"async">>) -> query_async. + +query_args(<<"sync">>, Request, SyncTimeout) -> + [Request, SyncTimeout]; +query_args(<<"async">>, Request, _) -> + [Request]. diff --git a/apps/emqx_resource/src/replayq.erl b/apps/emqx_resource/src/replayq.erl new file mode 100644 index 000000000..437702a5e --- /dev/null +++ b/apps/emqx_resource/src/replayq.erl @@ -0,0 +1,779 @@ +-module(replayq). + +-export([open/1, close/1]). +-export([append/2, pop/2, ack/2, ack_sync/2, peek/1, overflow/1]). +-export([count/1, bytes/1, is_empty/1, is_mem_only/1]). +%% exported for troubleshooting +-export([do_read_items/2]). + +%% internal exports for beam reload +-export([committer_loop/2, default_sizer/1, default_marshaller/1]). + +-export_type([config/0, q/0, ack_ref/0, sizer/0, marshaller/0]). + +-define(NOTHING_TO_ACK, nothing_to_ack). +-define(PENDING_ACKS(Ref), {replayq_pending_acks, Ref}). + +-type segno() :: pos_integer(). +-type item() :: term(). +-type count() :: non_neg_integer(). +-type id() :: count(). +-type bytes() :: non_neg_integer(). +-type filename() :: file:filename_all(). +-type dir() :: filename(). +-type ack_ref() :: ?NOTHING_TO_ACK | {segno(), ID :: pos_integer()}. +-type sizer() :: fun((item()) -> bytes()). +-type marshaller() :: fun((item()) -> binary()). + +-type config() :: #{ + dir => dir(), + seg_bytes => bytes(), + mem_only => boolean(), + max_total_bytes => bytes(), + offload => boolean(), + sizer => sizer(), + marshaller => marshaller() +}. +%% writer cursor +-define(NO_FD, no_fd). +-type w_cur() :: #{ + segno := segno(), + bytes := bytes(), + count := count(), + fd := ?NO_FD | file:fd() +}. + +-type stats() :: #{ + bytes := bytes(), + count := count() +}. + +-opaque q() :: #{ + config := mem_only | config(), + stats := stats(), + in_mem := queue:queue(in_mem_item()), + w_cur => w_cur(), + committer => pid(), + head_segno => segno(), + sizer := sizer(), + marshaller => marshaller(), + max_total_bytes := bytes() +}. + +-define(LAYOUT_VSN_0, 0). +-define(LAYOUT_VSN_1, 1). +-define(MAGIC, 841265288). +-define(SUFFIX, "replaylog"). +-define(DEFAULT_POP_BYTES_LIMIT, 2000000). +-define(DEFAULT_POP_COUNT_LIMIT, 1000). +-define(DEFAULT_REPLAYQ_LIMIT, 2000000000). +-define(COMMIT(SEGNO, ID, From), {commit, SEGNO, ID, From}). +-define(NO_COMMIT_HIST, no_commit_hist). +-define(FIRST_SEGNO, 1). +-define(NEXT_SEGNO(N), (N + 1)). +-define(STOP, stop). +-define(MEM_ONLY_ITEM(Bytes, Item), {Bytes, Item}). +-define(DISK_CP_ITEM(Id, Bytes, Item), {Id, Bytes, Item}). + +-type in_mem_item() :: + ?MEM_ONLY_ITEM(bytes(), item()) + | ?DISK_CP_ITEM(id(), bytes(), item()). + +-spec open(config()) -> q(). +open(#{mem_only := true} = C) -> + #{ + stats => #{bytes => 0, count => 0}, + in_mem => queue:new(), + sizer => get_sizer(C), + config => mem_only, + max_total_bytes => maps:get(max_total_bytes, C, ?DEFAULT_REPLAYQ_LIMIT) + }; +open(#{dir := Dir, seg_bytes := _} = Config) -> + ok = filelib:ensure_dir(filename:join(Dir, "foo")), + Sizer = get_sizer(Config), + Marshaller = get_marshaller(Config), + IsOffload = is_offload_mode(Config), + Q = + case delete_consumed_and_list_rest(Dir) of + [] -> + %% no old segments, start over from zero + #{ + stats => #{bytes => 0, count => 0}, + w_cur => init_writer(Dir, empty, IsOffload), + committer => spawn_committer(?FIRST_SEGNO, Dir), + head_segno => ?FIRST_SEGNO, + in_mem => queue:new() + }; + Segs -> + LastSegno = lists:last(Segs), + CommitHist = get_commit_hist(Dir), + Reader = fun(Seg, Ch) -> read_items(Dir, Seg, Ch, Sizer, Marshaller) end, + HeadItems = Reader(hd(Segs), CommitHist), + #{ + stats => collect_stats(HeadItems, tl(Segs), Reader), + w_cur => init_writer(Dir, LastSegno, IsOffload), + committer => spawn_committer(hd(Segs), Dir), + head_segno => hd(Segs), + in_mem => queue:from_list(HeadItems) + } + end, + Q#{ + sizer => Sizer, + marshaller => Marshaller, + config => maps:without([sizer, marshaller], Config), + max_total_bytes => maps:get(max_total_bytes, Config, ?DEFAULT_REPLAYQ_LIMIT) + }. + +-spec close(q() | w_cur()) -> ok | {error, any()}. +close(#{config := mem_only}) -> + ok; +close(#{w_cur := W_Cur, committer := Pid} = Q) -> + MRef = erlang:monitor(process, Pid), + Pid ! ?STOP, + unlink(Pid), + receive + {'DOWN', MRef, process, Pid, _Reason} -> + ok + end, + ok = maybe_dump_back_to_disk(Q), + do_close(W_Cur). + +do_close(#{fd := ?NO_FD}) -> ok; +do_close(#{fd := Fd}) -> file:close(Fd). + +%% In case of offload mode, dump the unacked (and un-popped) on disk +%% before close. this serves as a best-effort data loss protection +maybe_dump_back_to_disk(#{config := Config} = Q) -> + case is_offload_mode(Config) of + true -> dump_back_to_disk(Q); + false -> ok + end. + +dump_back_to_disk(#{ + config := #{dir := Dir}, + head_segno := ReaderSegno, + in_mem := InMem, + marshaller := Marshaller +}) -> + IoData0 = get_unacked(process_info(self(), dictionary), ReaderSegno, Marshaller), + Items1 = queue:to_list(InMem), + IoData1 = lists:map(fun(?DISK_CP_ITEM(_, _, I)) -> make_iodata(I, Marshaller) end, Items1), + %% ensure old segment file is deleted + ok = ensure_deleted(filename(Dir, ReaderSegno)), + %% rewrite the segment with what's currently in memory + IoData = [IoData0, IoData1], + case iolist_size(IoData) > 0 of + true -> + #{fd := Fd} = open_segment(Dir, ReaderSegno), + ok = file:write(Fd, [IoData0, IoData1]), + ok = file:close(Fd); + false -> + %% nothing to write + ok + end. + +get_unacked({dictionary, Dict}, ReaderSegno, Marshaller) -> + F = fun + ({?PENDING_ACKS(AckRef), Items}) -> + erase(?PENDING_ACKS(AckRef)), + {Segno, Id} = AckRef, + Segno =:= ReaderSegno andalso + {true, {Id, Items}}; + (_) -> + false + end, + Pendings0 = lists:filtermap(F, Dict), + Pendings = lists:keysort(1, Pendings0), + do_get_unacked(Pendings, Marshaller). + +do_get_unacked([], _Marshaller) -> + []; +do_get_unacked([{_, Items} | Rest], Marshaller) -> + [ + [make_iodata(I, Marshaller) || I <- Items] + | do_get_unacked(Rest, Marshaller) + ]. + +-spec append(q(), [item()]) -> q(). +append(Q, []) -> + Q; +append( + #{ + config := mem_only, + in_mem := InMem, + stats := #{bytes := Bytes0, count := Count0}, + sizer := Sizer + } = Q, + Items0 +) -> + {CountDiff, BytesDiff, Items} = transform(false, Items0, Sizer), + + Stats = #{count => Count0 + CountDiff, bytes => Bytes0 + BytesDiff}, + Q#{ + stats := Stats, + in_mem := append_in_mem(Items, InMem) + }; +append( + #{ + config := #{seg_bytes := BytesLimit, dir := Dir} = Config, + stats := #{bytes := Bytes0, count := Count0}, + w_cur := #{count := CountInSeg, segno := WriterSegno} = W_Cur0, + head_segno := ReaderSegno, + sizer := Sizer, + marshaller := Marshaller, + in_mem := HeadItems0 + } = Q, + Items0 +) -> + IoData = lists:map(fun(I) -> make_iodata(I, Marshaller) end, Items0), + {CountDiff, BytesDiff, Items} = transform(CountInSeg + 1, Items0, Sizer), + TotalBytes = Bytes0 + BytesDiff, + Stats = #{count => Count0 + CountDiff, bytes => TotalBytes}, + IsOffload = is_offload_mode(Config), + W_Cur1 = do_append(W_Cur0, CountDiff, BytesDiff, IoData), + W_Cur = + case is_segment_full(W_Cur1, TotalBytes, BytesLimit, ReaderSegno, IsOffload) of + true -> + ok = do_close(W_Cur1), + %% get ready for the next append + open_segment(Dir, ?NEXT_SEGNO(WriterSegno)); + false -> + W_Cur1 + end, + HeadItems = + case ReaderSegno =:= WriterSegno of + true -> append_in_mem(Items, HeadItems0); + false -> HeadItems0 + end, + Q#{ + stats := Stats, + w_cur := W_Cur, + in_mem := HeadItems + }. + +%% @doc pop out at least one item from the queue. +%% volume limited by `bytes_limit' and `count_limit'. +-spec pop(q(), #{bytes_limit => bytes(), count_limit => count()}) -> + {q(), ack_ref(), [item()]}. +pop(Q, Opts) -> + Bytes = maps:get(bytes_limit, Opts, ?DEFAULT_POP_BYTES_LIMIT), + Count = maps:get(count_limit, Opts, ?DEFAULT_POP_COUNT_LIMIT), + true = (Count > 0), + pop(Q, Bytes, Count, ?NOTHING_TO_ACK, []). + +%% @doc peek the queue front item. +-spec peek(q()) -> empty | item(). +peek(#{in_mem := HeadItems}) -> + case queue:peek(HeadItems) of + empty -> empty; + {value, ?MEM_ONLY_ITEM(_, Item)} -> Item; + {value, ?DISK_CP_ITEM(_, _, Item)} -> Item + end. + +%% @doc Asynch-ly write the consumed item Segment number + ID to a file. +-spec ack(q(), ack_ref()) -> ok. +ack(_, ?NOTHING_TO_ACK) -> + ok; +ack(#{committer := Pid}, {Segno, Id} = AckRef) -> + _ = erlang:erase(?PENDING_ACKS(AckRef)), + Pid ! ?COMMIT(Segno, Id, false), + ok. + +%% @hidden Synced ack, for deterministic tests only +-spec ack_sync(q(), ack_ref()) -> ok. +ack_sync(_, ?NOTHING_TO_ACK) -> + ok; +ack_sync(#{committer := Pid}, {Segno, Id} = AckRef) -> + _ = erlang:erase(?PENDING_ACKS(AckRef)), + Ref = make_ref(), + Pid ! ?COMMIT(Segno, Id, {self(), Ref}), + receive + {Ref, ok} -> ok + end. + +-spec count(q()) -> count(). +count(#{stats := #{count := Count}}) -> Count. + +-spec bytes(q()) -> bytes(). +bytes(#{stats := #{bytes := Bytes}}) -> Bytes. + +is_empty(#{config := mem_only, in_mem := All}) -> + queue:is_empty(All); +is_empty( + #{ + w_cur := #{segno := WriterSegno}, + head_segno := ReaderSegno, + in_mem := HeadItems + } = Q +) -> + Result = ((WriterSegno =:= ReaderSegno) andalso queue:is_empty(HeadItems)), + %% assert + Result = (count(Q) =:= 0). + +%% @doc Returns number of bytes the size of the queue has exceeded +%% total bytes limit. Result is negative when it is not overflow. +-spec overflow(q()) -> integer(). +overflow(#{ + max_total_bytes := MaxTotalBytes, + stats := #{bytes := Bytes} +}) -> + Bytes - MaxTotalBytes. + +-spec is_mem_only(q()) -> boolean(). +is_mem_only(#{config := mem_only}) -> + true; +is_mem_only(_) -> + false. + +%% internals ========================================================= + +transform(Id, Items, Sizer) -> + transform(Id, Items, Sizer, 0, 0, []). + +transform(_Id, [], _Sizer, Count, Bytes, Acc) -> + {Count, Bytes, lists:reverse(Acc)}; +transform(Id, [Item0 | Rest], Sizer, Count, Bytes, Acc) -> + Size = Sizer(Item0), + {NextId, Item} = + case Id of + false -> {false, ?MEM_ONLY_ITEM(Size, Item0)}; + N -> {N + 1, ?DISK_CP_ITEM(Id, Size, Item0)} + end, + transform(NextId, Rest, Sizer, Count + 1, Bytes + Size, [Item | Acc]). + +append_in_mem([], Q) -> Q; +append_in_mem([Item | Rest], Q) -> append_in_mem(Rest, queue:in(Item, Q)). + +pop(Q, _Bytes, 0, AckRef, Acc) -> + Result = lists:reverse(Acc), + ok = maybe_save_pending_acks(AckRef, Q, Result), + {Q, AckRef, Result}; +pop(#{config := Cfg} = Q, Bytes, Count, AckRef, Acc) -> + case is_empty(Q) of + true -> + {Q, AckRef, lists:reverse(Acc)}; + false when Cfg =:= mem_only -> + pop_mem(Q, Bytes, Count, Acc); + false -> + pop2(Q, Bytes, Count, AckRef, Acc) + end. + +pop_mem( + #{ + in_mem := InMem, + stats := #{count := TotalCount, bytes := TotalBytes} = Stats + } = Q, + Bytes, + Count, + Acc +) -> + case queue:out(InMem) of + {{value, ?MEM_ONLY_ITEM(Sz, _Item)}, _} when Sz > Bytes andalso Acc =/= [] -> + {Q, ?NOTHING_TO_ACK, lists:reverse(Acc)}; + {{value, ?MEM_ONLY_ITEM(Sz, Item)}, Rest} -> + NewQ = Q#{ + in_mem := Rest, + stats := Stats#{ + count := TotalCount - 1, + bytes := TotalBytes - Sz + } + }, + pop(NewQ, Bytes - Sz, Count - 1, ?NOTHING_TO_ACK, [Item | Acc]) + end. + +pop2( + #{ + head_segno := ReaderSegno, + in_mem := HeadItems, + stats := #{count := TotalCount, bytes := TotalBytes} = Stats, + w_cur := #{segno := WriterSegno} + } = Q, + Bytes, + Count, + AckRef, + Acc +) -> + case queue:out(HeadItems) of + {{value, ?DISK_CP_ITEM(_, Sz, _Item)}, _} when Sz > Bytes andalso Acc =/= [] -> + %% taking the head item would cause exceeding size limit + {Q, AckRef, lists:reverse(Acc)}; + {{value, ?DISK_CP_ITEM(Id, Sz, Item)}, Rest} -> + Q1 = Q#{ + in_mem := Rest, + stats := Stats#{ + count := TotalCount - 1, + bytes := TotalBytes - Sz + } + }, + %% read the next segment in case current is drained + NewQ = + case queue:is_empty(Rest) andalso ReaderSegno < WriterSegno of + true -> read_next_seg(Q1); + false -> Q1 + end, + NewAckRef = {ReaderSegno, Id}, + pop(NewQ, Bytes - Sz, Count - 1, NewAckRef, [Item | Acc]) + end. + +%% due to backward compatibility reasons for the ack api +%% we ca nnot track pending acks in q() -- reason to use process dictionary +maybe_save_pending_acks(?NOTHING_TO_ACK, _, _) -> + ok; +maybe_save_pending_acks(AckRef, #{config := Config}, Items) -> + case is_offload_mode(Config) of + true -> + _ = erlang:put(?PENDING_ACKS(AckRef), Items), + ok; + false -> + ok + end. + +read_next_seg( + #{ + config := #{dir := Dir} = Config, + head_segno := ReaderSegno, + w_cur := #{segno := WriterSegno, fd := Fd} = WCur0, + sizer := Sizer, + marshaller := Marshaller + } = Q +) -> + NextSegno = ReaderSegno + 1, + %% reader has caught up to latest segment + case NextSegno =:= WriterSegno of + true -> + %% force flush to disk so the next read can get all bytes + ok = file:sync(Fd); + false -> + ok + end, + IsOffload = is_offload_mode(Config), + WCur = + case IsOffload andalso NextSegno =:= WriterSegno of + true -> + %% reader has caught up to latest segment in offload mode, + %% close the writer's fd. Continue in mem-only mode for the head segment + ok = do_close(WCur0), + WCur0#{fd := ?NO_FD}; + false -> + WCur0 + end, + NextSegItems = read_items(Dir, NextSegno, ?NO_COMMIT_HIST, Sizer, Marshaller), + Q#{ + head_segno := NextSegno, + in_mem := queue:from_list(NextSegItems), + w_cur := WCur + }. + +delete_consumed_and_list_rest(Dir0) -> + Dir = unicode:characters_to_list(Dir0), + Segnos0 = lists:sort([parse_segno(N) || N <- filelib:wildcard("*." ?SUFFIX, Dir)]), + {SegnosToDelete, Segnos} = find_segnos_to_delete(Dir, Segnos0), + ok = lists:foreach(fun(Segno) -> ensure_deleted(filename(Dir, Segno)) end, SegnosToDelete), + case Segnos of + [] -> + %% delete commit file in case there is no segments left + %% segment number will start from 0 again. + ensure_deleted(commit_filename(Dir)), + []; + X -> + X + end. + +find_segnos_to_delete(Dir, Segnos) -> + CommitHist = get_commit_hist(Dir), + do_find_segnos_to_delete(Dir, Segnos, CommitHist). + +do_find_segnos_to_delete(_Dir, Segnos, ?NO_COMMIT_HIST) -> + {[], Segnos}; +do_find_segnos_to_delete(Dir, Segnos0, {CommittedSegno, CommittedId}) -> + {SegnosToDelete, Segnos} = lists:partition(fun(N) -> N < CommittedSegno end, Segnos0), + case + Segnos =/= [] andalso + hd(Segnos) =:= CommittedSegno andalso + is_all_consumed(Dir, CommittedSegno, CommittedId) + of + true -> + %% assert + CommittedSegno = hd(Segnos), + %% all items in the oldest segment have been consumed, + %% no need to keep this segment + {[CommittedSegno | SegnosToDelete], tl(Segnos)}; + _ -> + {SegnosToDelete, Segnos} + end. + +%% ALL items are consumed if the committed item ID is no-less than the number +%% of items in this segment +is_all_consumed(Dir, CommittedSegno, CommittedId) -> + CommittedId >= erlang:length(do_read_items(Dir, CommittedSegno)). + +ensure_deleted(Filename) -> + case file:delete(Filename) of + ok -> ok; + {error, enoent} -> ok + end. + +%% The committer writes consumer's acked segmeng number + item ID +%% to a file. The file is only read at start/restart. +spawn_committer(ReaderSegno, Dir) -> + Name = iolist_to_binary(filename:join([Dir, committer])), + %% register a name to avoid having two committers spawned for the same dir + RegName = binary_to_atom(Name, utf8), + Pid = erlang:spawn_link(fun() -> committer_loop(ReaderSegno, Dir) end), + true = erlang:register(RegName, Pid), + Pid. + +committer_loop(ReaderSegno, Dir) -> + receive + ?COMMIT(Segno0, Id0, false) -> + {Segno, Id} = collect_async_commits(Segno0, Id0), + ok = handle_commit(ReaderSegno, Dir, Segno, Id, false), + ?MODULE:committer_loop(Segno, Dir); + ?COMMIT(Segno, Id, From) -> + ok = handle_commit(ReaderSegno, Dir, Segno, Id, From), + ?MODULE:committer_loop(Segno, Dir); + ?STOP -> + ok; + Msg -> + exit({replayq_committer_unkown_msg, Msg}) + after 200 -> + ?MODULE:committer_loop(ReaderSegno, Dir) + end. + +handle_commit(ReaderSegno, Dir, Segno, Id, From) -> + IoData = io_lib:format("~p.\n", [#{segno => Segno, id => Id}]), + ok = do_commit(Dir, IoData), + case Segno > ReaderSegno of + true -> + SegnosToDelete = lists:seq(ReaderSegno, Segno - 1), + lists:foreach(fun(N) -> ok = ensure_deleted(filename(Dir, N)) end, SegnosToDelete); + false -> + ok + end, + ok = reply_ack_ok(From). + +%% Collect async acks which are already sent in the mailbox, +%% and keep only the last one for the current segment. +collect_async_commits(Segno, Id) -> + receive + ?COMMIT(Segno, AnotherId, false) -> + collect_async_commits(Segno, AnotherId) + after 0 -> + {Segno, Id} + end. + +reply_ack_ok({Pid, Ref}) -> + Pid ! {Ref, ok}, + ok; +reply_ack_ok(_) -> + ok. + +get_commit_hist(Dir) -> + CommitFile = commit_filename(Dir), + case filelib:is_regular(CommitFile) of + true -> + {ok, [#{segno := Segno, id := Id}]} = file:consult(CommitFile), + {Segno, Id}; + false -> + ?NO_COMMIT_HIST + end. + +do_commit(Dir, IoData) -> + TmpName = commit_filename(Dir, "COMMIT.tmp"), + Name = commit_filename(Dir), + ok = file:write_file(TmpName, IoData), + ok = file:rename(TmpName, Name). + +commit_filename(Dir) -> + commit_filename(Dir, "COMMIT"). + +commit_filename(Dir, Name) -> + filename:join([Dir, Name]). + +do_append( + #{fd := ?NO_FD, bytes := Bytes0, count := Count0} = Cur, + Count, + Bytes, + _IoData +) -> + %% offload mode, fd is not initialized yet + Cur#{ + bytes => Bytes0 + Bytes, + count => Count0 + Count + }; +do_append( + #{fd := Fd, bytes := Bytes0, count := Count0} = Cur, + Count, + Bytes, + IoData +) -> + ok = file:write(Fd, IoData), + Cur#{ + bytes => Bytes0 + Bytes, + count => Count0 + Count + }. + +read_items(Dir, Segno, CommitHist, Sizer, Marshaller) -> + Items0 = do_read_items(Dir, Segno), + Items = + case CommitHist of + ?NO_COMMIT_HIST -> + %% no commit hist, return all + Items0; + {CommitedSegno, _} when CommitedSegno < Segno -> + %% committed at an older segment + Items0; + {Segno, CommittedId} -> + %% committed at current segment keep only the tail + {_, R} = lists:splitwith(fun({I, _}) -> I =< CommittedId end, Items0), + R + end, + lists:map( + fun({Id, Bin}) -> + Item = Marshaller(Bin), + Size = Sizer(Item), + ?DISK_CP_ITEM(Id, Size, Item) + end, + Items + ). + +do_read_items(Dir, Segno) -> + Filename = filename(Dir, Segno), + {ok, Bin} = file:read_file(Filename), + case parse_items(Bin, 1, []) of + {Items, <<>>} -> + Items; + {Items, Corrupted} -> + error_logger:error_msg( + "corrupted replayq log: ~s, skipped ~p bytes", + [Filename, size(Corrupted)] + ), + Items + end. + +parse_items(<<>>, _Id, Acc) -> + {lists:reverse(Acc), <<>>}; +parse_items( + <> = All, + Id, + Acc +) -> + case CRC =:= erlang:crc32(Item) of + true -> parse_items(Rest, Id + 1, [{Id, Item} | Acc]); + false -> {lists:reverse(Acc), All} + end; +parse_items( + <> = All, + Id, + Acc +) -> + case CRC =:= erlang:crc32(Item) andalso Item =/= <<>> of + true -> parse_items(Rest, Id + 1, [{Id, Item} | Acc]); + false -> {lists:reverse(Acc), All} + end; +parse_items(Corrupted, _Id, Acc) -> + {lists:reverse(Acc), Corrupted}. + +make_iodata(Item0, Marshaller) -> + Item = Marshaller(Item0), + Size = size(Item), + CRC = erlang:crc32(Item), + [ + <>, + Item + ]. + +collect_stats(HeadItems, SegsOnDisk, Reader) -> + ItemF = fun(?DISK_CP_ITEM(_Id, Sz, _Item), {B, C}) -> + {B + Sz, C + 1} + end, + Acc0 = lists:foldl(ItemF, {0, 0}, HeadItems), + {Bytes, Count} = + lists:foldl( + fun(Segno, Acc) -> + Items = Reader(Segno, ?NO_COMMIT_HIST), + lists:foldl(ItemF, Acc, Items) + end, + Acc0, + SegsOnDisk + ), + #{bytes => Bytes, count => Count}. + +parse_segno(Filename) -> + [Segno, ?SUFFIX] = string:tokens(Filename, "."), + list_to_integer(Segno). + +filename(Dir, Segno) -> + Name = lists:flatten(io_lib:format("~10.10.0w." ?SUFFIX, [Segno])), + filename:join(Dir, Name). + +%% open the current segment for write if it is empty +%% otherwise rollout to the next segment +-spec init_writer(dir(), empty | segno(), boolean()) -> w_cur(). +init_writer(_Dir, empty, true) -> + %% clean start for offload mode + #{fd => ?NO_FD, segno => ?FIRST_SEGNO, bytes => 0, count => 0}; +init_writer(Dir, empty, false) -> + open_segment(Dir, ?FIRST_SEGNO); +init_writer(Dir, Segno, _IsOffload) when is_number(Segno) -> + Filename = filename(Dir, Segno), + case filelib:file_size(Filename) of + 0 -> open_segment(Dir, Segno); + _ -> open_segment(Dir, ?NEXT_SEGNO(Segno)) + end. + +-spec open_segment(dir(), segno()) -> w_cur(). +open_segment(Dir, Segno) -> + Filename = filename(Dir, Segno), + %% raw so there is no need to go through the single gen_server file_server + {ok, Fd} = file:open(Filename, [raw, read, write, binary, delayed_write]), + #{fd => Fd, segno => Segno, bytes => 0, count => 0}. + +get_sizer(C) -> + maps:get(sizer, C, fun ?MODULE:default_sizer/1). + +get_marshaller(C) -> + maps:get(marshaller, C, fun ?MODULE:default_marshaller/1). + +is_offload_mode(Config) when is_map(Config) -> + maps:get(offload, Config, false). + +default_sizer(I) when is_binary(I) -> erlang:size(I). + +default_marshaller(I) when is_binary(I) -> I. + +is_segment_full( + #{segno := WriterSegno, bytes := SegmentBytes}, + TotalBytes, + SegmentBytesLimit, + ReaderSegno, + true +) -> + %% in offload mode, when reader is lagging behind, we try + %% writer rolls to a new segment when file size limit is reached + %% when reader is reading off from the same segment as writer + %% i.e. the in memory queue, only start writing to segment file + %% when total bytes (in memory) is reached segment limit + %% + %% NOTE: we never shrink segment bytes, even when popping out + %% from the in-memory queue. + case ReaderSegno < WriterSegno of + true -> SegmentBytes >= SegmentBytesLimit; + false -> TotalBytes >= SegmentBytesLimit + end; +is_segment_full( + #{bytes := SegmentBytes}, + _TotalBytes, + SegmentBytesLimit, + _ReaderSegno, + false +) -> + %% here we check if segment size is greater than segment size limit + %% after append based on the assumption that the items are usually + %% very small in size comparing to segment size. + %% We can change implementation to split items list to avoid + %% segment overflow if really necessary + SegmentBytes >= SegmentBytesLimit. From d8d8d674e4ad7314e87926a9a488b5db82008583 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Wed, 27 Jul 2022 00:37:17 +0800 Subject: [PATCH 02/14] feat(resource): start emqx_resource_worker in pools --- .../src/emqx_resource_manager.erl | 9 +- apps/emqx_resource/src/emqx_resource_sup.erl | 80 ++++- .../src/emqx_resource_worker.erl | 316 +++++++++++------- 3 files changed, 278 insertions(+), 127 deletions(-) diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index f1fa36173..ee18a6836 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -111,12 +111,15 @@ create(MgrId, ResId, Group, ResourceType, Config, Opts) -> ok = emqx_metrics_worker:create_metrics( ?RES_METRICS, ResId, - [matched, success, failed, exception, resource_error], + [matched, success, failed, exception, resource_down], [matched] ), case maps:get(start_after_created, Opts, true) of - true -> wait_for_resource_ready(ResId, maps:get(wait_for_resource_ready, Opts, 5000)); - false -> ok + true -> + ok = emqx_resource_sup:start_workers(ResId, Opts), + wait_for_resource_ready(ResId, maps:get(wait_for_resource_ready, Opts, 5000)); + false -> + ok end, ok. diff --git a/apps/emqx_resource/src/emqx_resource_sup.erl b/apps/emqx_resource/src/emqx_resource_sup.erl index 84458f0d5..14db50d01 100644 --- a/apps/emqx_resource/src/emqx_resource_sup.erl +++ b/apps/emqx_resource/src/emqx_resource_sup.erl @@ -19,13 +19,10 @@ -behaviour(supervisor). --export([start_link/0]). +-export([start_link/0, start_workers/2, stop_workers/2]). -export([init/1]). -%% set a very large pool size in case all the workers busy --define(POOL_SIZE, 64). - start_link() -> supervisor:start_link({local, ?MODULE}, ?MODULE, []). @@ -43,3 +40,78 @@ init([]) -> modules => [emqx_resource_manager_sup] }, {ok, {SupFlags, [Metrics, ResourceManager]}}. + +start_workers(ResId, Opts) -> + PoolSize = pool_size(Opts), + _ = ensure_worker_pool(ResId, hash, [{size, PoolSize}]), + lists:foreach( + fun(Idx) -> + _ = ensure_worker_added(ResId, {ResId, Idx}, Idx), + ok = ensure_worker_started(ResId, Idx, Opts) + end, + lists:seq(1, PoolSize) + ). + +stop_workers(ResId, Opts) -> + PoolSize = pool_size(Opts), + lists:foreach( + fun(Idx) -> + ok = ensure_worker_stopped(ResId, Idx), + ok = ensure_worker_removed(ResId, {ResId, Idx}) + end, + lists:seq(1, PoolSize) + ), + _ = gproc_pool:delete(ResId), + ok. + +pool_size(Opts) -> + maps:get(worker_pool_size, Opts, erlang:system_info(schedulers_online)). + +ensure_worker_pool(Pool, Type, Opts) -> + try + gproc_pool:new(Pool, Type, Opts) + catch + error:exists -> ok + end, + ok. + +ensure_worker_added(Pool, Name, Slot) -> + try + gproc_pool:add_worker(Pool, Name, Slot) + catch + error:exists -> ok + end, + ok. + +ensure_worker_removed(Pool, Name) -> + _ = gproc_pool:remove_worker(Pool, Name), + ok. + +-define(CHILD_ID(MOD, RESID, INDEX), {MOD, RESID, INDEX}). +ensure_worker_started(ResId, Idx, Opts) -> + Mod = emqx_resource_worker, + Spec = #{ + id => ?CHILD_ID(Mod, ResId, Idx), + start => {Mod, start_link, [ResId, Idx, Opts]}, + restart => transient, + shutdown => 5000, + type => worker, + modules => [Mod] + }, + case supervisor:start_child(emqx_resource_sup, Spec) of + {ok, _Pid} -> ok; + {error, {already_started, _}} -> ok; + {error, already_present} -> ok; + {error, _} = Err -> Err + end. + +ensure_worker_stopped(ResId, Idx) -> + ChildId = ?CHILD_ID(emqx_resource_worker, ResId, Idx), + case supervisor:terminate_child(emqx_resource_sup, ChildId) of + ok -> + supervisor:delete_child(emqx_resource_sup, ChildId); + {error, not_found} -> + ok; + {error, Reason} -> + {error, Reason} + end. diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index 6c3b05830..3c5c7eefe 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -14,7 +14,8 @@ %% limitations under the License. %%-------------------------------------------------------------------- -%% An FIFO queue using ETS-ReplayQ as backend. +%% This module implements async message sending, disk message queuing, +%% and message batching using ReplayQ. -module(emqx_resource_worker). @@ -25,18 +26,23 @@ -behaviour(gen_statem). -export([ - start_link/2, - query/2, - query_async/3, - query_mfa/3 + start_link/3, + query/3, + query_async/4, + block/1, + resume/1 ]). -export([ callback_mode/0, - init/1 + init/1, + terminate/2, + code_change/3 ]). --export([do/3]). +-export([running/3, blocked/3]). + +-define(RESUME_INTERVAL, 15000). %% count -define(DEFAULT_BATCH_SIZE, 100). @@ -47,72 +53,136 @@ -define(REPLY(FROM, REQUEST, RESULT), {FROM, REQUEST, RESULT}). -define(EXPAND(RESULT, BATCH), [?REPLY(FROM, REQUEST, RESULT) || ?QUERY(FROM, REQUEST) <- BATCH]). +-define(RESOURCE_ERROR(Reason, Msg), {error, {resource_error, #{reason => Reason, msg => Msg}}}). +-define(RESOURCE_ERROR_M(Reason, Msg), {error, {resource_error, #{reason := Reason, msg := Msg}}}). + -type id() :: binary(). -type request() :: term(). -type result() :: term(). --type reply_fun() :: {fun((result(), Args :: term()) -> any()), Args :: term()}. +-type reply_fun() :: {fun((result(), Args :: term()) -> any()), Args :: term()} | undefined. -type from() :: pid() | reply_fun(). -callback batcher_flush(Acc :: [{from(), request()}], CbState :: term()) -> {{from(), result()}, NewCbState :: term()}. -callback_mode() -> [state_functions]. +callback_mode() -> [state_functions, state_enter]. -start_link(Id, Opts) -> - gen_statem:start_link({local, name(Id)}, ?MODULE, {Id, Opts}, []). +start_link(Id, Index, Opts) -> + gen_statem:start_link({local, name(Id, Index)}, ?MODULE, {Id, Index, Opts}, []). --spec query(id(), request()) -> ok. -query(Id, Request) -> - gen_statem:call(name(Id), {query, Request}). +-spec query(id(), term(), request()) -> ok. +query(Id, Key, Request) -> + gen_statem:call(pick(Id, Key), {query, Request}). --spec query_async(id(), request(), reply_fun()) -> ok. -query_async(Id, Request, ReplyFun) -> - gen_statem:cast(name(Id), {query, Request, ReplyFun}). +-spec query_async(id(), term(), request(), reply_fun()) -> ok. +query_async(Id, Key, Request, ReplyFun) -> + gen_statem:cast(pick(Id, Key), {query, Request, ReplyFun}). --spec name(id()) -> atom(). -name(Id) -> - Mod = atom_to_binary(?MODULE, utf8), - <>. +-spec block(pid() | atom()) -> ok. +block(ServerRef) -> + gen_statem:cast(ServerRef, block). -disk_cache_dir(Id) -> - filename:join([emqx:data_dir(), Id, cache]). +-spec resume(pid() | atom()) -> ok. +resume(ServerRef) -> + gen_statem:cast(ServerRef, resume). -init({Id, Opts}) -> +init({Id, Index, Opts}) -> + true = gproc_pool:connect_worker(Id, {Id, Index}), BatchSize = maps:get(batch_size, Opts, ?DEFAULT_BATCH_SIZE), Queue = - case maps:get(cache_enabled, Opts, true) of - true -> replayq:open(#{dir => disk_cache_dir(Id), seg_bytes => 10000000}); + case maps:get(queue_enabled, Opts, true) of + true -> replayq:open(#{dir => disk_queue_dir(Id), seg_bytes => 10000000}); false -> undefined end, St = #{ id => Id, + index => Index, batch_enabled => maps:get(batch_enabled, Opts, true), batch_size => BatchSize, batch_time => maps:get(batch_time, Opts, ?DEFAULT_BATCH_TIME), - cache_queue => Queue, + queue => Queue, acc => [], acc_left => BatchSize, tref => undefined }, - {ok, do, St}. + {ok, blocked, St, {next_event, cast, resume}}. -do(cast, {query, Request, ReplyFun}, #{batch_enabled := true} = State) -> - do_acc(ReplyFun, Request, State); -do(cast, {query, Request, ReplyFun}, #{batch_enabled := false} = State) -> - do_query(ReplyFun, Request, State); -do({call, From}, {query, Request}, #{batch_enabled := true} = State) -> - do_acc(From, Request, State); -do({call, From}, {query, Request}, #{batch_enabled := false} = State) -> - do_query(From, Request, State); -do(info, {flush, Ref}, St = #{tref := {_TRef, Ref}}) -> - {keep_state, flush(St#{tref := undefined})}; -do(info, {flush, _Ref}, _St) -> +running(enter, _, _St) -> keep_state_and_data; -do(info, Info, _St) -> +running(cast, resume, _St) -> + keep_state_and_data; +running(cast, block, St) -> + {next_state, block, St}; +running(cast, {query, Request, ReplyFun}, St) -> + query_or_acc(ReplyFun, Request, St); +running({call, From}, {query, Request}, St) -> + query_or_acc(From, Request, St); +running(info, {flush, Ref}, St = #{tref := {_TRef, Ref}}) -> + {keep_state, flush(St#{tref := undefined})}; +running(info, {flush, _Ref}, _St) -> + keep_state_and_data; +running(info, Info, _St) -> ?SLOG(error, #{msg => unexpected_msg, info => Info}), keep_state_and_data. -do_acc(From, Request, #{acc := Acc, acc_left := Left} = St0) -> +blocked(enter, _, _St) -> + keep_state_and_data; +blocked(cast, block, _St) -> + keep_state_and_data; +blocked(cast, resume, St) -> + do_resume(St); +blocked(state_timeout, resume, St) -> + do_resume(St); +blocked(cast, {query, Request, ReplyFun}, St) -> + handle_blocked(ReplyFun, Request, St); +blocked({call, From}, {query, Request}, St) -> + handle_blocked(From, Request, St). + +terminate(_Reason, #{id := Id, index := Index}) -> + gproc_pool:disconnect_worker(Id, {Id, Index}). + +code_change(_OldVsn, State, _Extra) -> + {ok, State}. + +%%============================================================================== +pick(Id, Key) -> + Pid = gproc_pool:pick_worker(Id, Key), + case is_pid(Pid) of + true -> Pid; + false -> error({failed_to_pick_worker, {Id, Key}}) + end. + +do_resume(#{queue := undefined} = St) -> + {next_state, running, St}; +do_resume(#{queue := Q, id := Id} = St) -> + case replayq:peek(Q) of + empty -> + {next_state, running, St, {state_timeout, ?RESUME_INTERVAL, resume}}; + First -> + Result = call_query(Id, First), + case handle_query_result(Id, false, Result) of + %% Send failed because resource down + true -> + {keep_state, St}; + %% Send ok or failed but the resource is working + false -> + %% We Send 'resume' to the end of the mailbox to give the worker + %% a chance to process 'query' requests. + {keep_state, St#{queue => drop_head(Q)}, {state_timeout, 0, resume}} + end + end. + +drop_head(Q) -> + {Q1, AckRef, _} = replayq:pop(Q, #{count_limit => 1}), + ok = replayq:ack(Q1, AckRef), + Q1. + +query_or_acc(From, Request, #{batch_enabled := true} = St) -> + acc_query(From, Request, St); +query_or_acc(From, Request, #{batch_enabled := false} = St) -> + send_query(From, Request, St). + +acc_query(From, Request, #{acc := Acc, acc_left := Left} = St0) -> Acc1 = [?QUERY(From, Request) | Acc], St = St0#{acc := Acc1, acc_left := Left - 1}, case Left =< 1 of @@ -120,10 +190,19 @@ do_acc(From, Request, #{acc := Acc, acc_left := Left} = St0) -> false -> {keep_state, ensure_flush_timer(St)} end. -do_query(From, Request, #{id := Id, cache_queue := Q0} = St0) -> +send_query(From, Request, #{id := Id, queue := Q} = St) -> Result = call_query(Id, Request), - Q1 = reply_caller(Id, Q0, ?REPLY(From, Request, Result)), - {keep_state, St0#{cache_queue := Q1}}. + case reply_caller(Id, Q, ?REPLY(From, Request, Result)) of + true -> + {keep_state, St#{queue := maybe_append_queue(Q, [Request])}}; + false -> + {next_state, blocked, St} + end. + +handle_blocked(From, Request, #{id := Id, queue := Q} = St) -> + Error = ?RESOURCE_ERROR(blocked, "resource is blocked"), + _ = reply_caller(Id, Q, ?REPLY(From, Request, Error)), + {keep_state, St#{queue := maybe_append_queue(Q, [Request])}}. flush(#{acc := []} = St) -> St; @@ -132,101 +211,109 @@ flush( id := Id, acc := Batch, batch_size := Size, - cache_queue := Q0 + queue := Q0 } = St ) -> - BatchResults = call_batch_query(Id, Batch), - Q1 = batch_reply_caller(Id, Q0, BatchResults), - cancel_flush_timer( - St#{ - acc_left := Size, - acc := [], - cache_queue := Q1 - } - ). + BatchResults = maybe_expand_batch_result(call_batch_query(Id, Batch), Batch), + St1 = cancel_flush_timer(St#{acc_left := Size, acc := []}), + case batch_reply_caller(Id, BatchResults) of + true -> + Q1 = maybe_append_queue(Q0, [Request || ?QUERY(_, Request) <- Batch]), + {keep_state, St1#{queue := Q1}}; + false -> + {next_state, blocked, St1} + end. -maybe_append_cache(undefined, _Request) -> undefined; -maybe_append_cache(Q, Request) -> replayq:append(Q, Request). +maybe_append_queue(undefined, _Query) -> undefined; +maybe_append_queue(Q, Query) -> replayq:append(Q, Query). -batch_reply_caller(Id, Q, BatchResults) -> +batch_reply_caller(Id, BatchResults) -> lists:foldl( - fun(Reply, Q1) -> - reply_caller(Id, Q1, Reply) + fun(Reply, BlockWorker) -> + reply_caller(Id, BlockWorker, Reply) end, - Q, + false, BatchResults ). -reply_caller(Id, Q, ?REPLY({ReplyFun, Args}, Request, Result)) when is_function(ReplyFun) -> +reply_caller(Id, BlockWorker, ?REPLY(undefined, _, Result)) -> + handle_query_result(Id, BlockWorker, Result); +reply_caller(Id, BlockWorker, ?REPLY({ReplyFun, Args}, _, Result)) -> ?SAFE_CALL(ReplyFun(Result, Args)), - handle_query_result(Id, Q, Request, Result); -reply_caller(Id, Q, ?REPLY(From, Request, Result)) -> + handle_query_result(Id, BlockWorker, Result); +reply_caller(Id, BlockWorker, ?REPLY(From, _, Result)) -> gen_statem:reply(From, Result), - handle_query_result(Id, Q, Request, Result). + handle_query_result(Id, BlockWorker, Result). -handle_query_result(Id, Q, _Request, ok) -> +handle_query_result(Id, BlockWorker, ok) -> emqx_metrics_worker:inc(?RES_METRICS, Id, success), - Q; -handle_query_result(Id, Q, _Request, {ok, _}) -> + BlockWorker; +handle_query_result(Id, BlockWorker, {ok, _}) -> emqx_metrics_worker:inc(?RES_METRICS, Id, success), - Q; -handle_query_result(Id, Q, _Request, {error, _}) -> - emqx_metrics_worker:inc(?RES_METRICS, Id, failed), - Q; -handle_query_result(Id, Q, Request, {error, {resource_error, #{reason := not_connected}}}) -> - emqx_metrics_worker:inc(?RES_METRICS, Id, resource_error), - maybe_append_cache(Q, Request); -handle_query_result(Id, Q, _Request, {error, {resource_error, #{}}}) -> - emqx_metrics_worker:inc(?RES_METRICS, Id, resource_error), - Q; -handle_query_result(Id, Q, Request, {error, {exception, _}}) -> + BlockWorker; +handle_query_result(Id, BlockWorker, ?RESOURCE_ERROR_M(exception, _)) -> emqx_metrics_worker:inc(?RES_METRICS, Id, exception), - maybe_append_cache(Q, Request). + BlockWorker; +handle_query_result(_Id, _, ?RESOURCE_ERROR_M(NotWorking, _)) when + NotWorking == not_connected; NotWorking == blocked +-> + true; +handle_query_result(_Id, BlockWorker, ?RESOURCE_ERROR_M(_, _)) -> + BlockWorker; +handle_query_result(Id, BlockWorker, {error, _}) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, failed), + BlockWorker; +handle_query_result(Id, _BlockWorker, {resource_down, _}) -> + emqx_metrics_worker:inc(?RES_METRICS, Id, resource_down), + true. call_query(Id, Request) -> - ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched), + do_call_query(on_query, Id, Request). + +call_batch_query(Id, Batch) -> + do_call_query(on_batch_query, Id, Batch). + +do_call_query(Fun, Id, Data) -> case emqx_resource_manager:ets_lookup(Id) of {ok, _Group, #{mod := Mod, state := ResourceState, status := connected}} -> - try Mod:on_query(Id, Request, ResourceState) of + ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched, length(Data)), + try Mod:Fun(Id, Data, ResourceState) of + %% if the callback module (connector) wants to return an error that + %% makes the current resource goes into the `error` state, it should + %% return `{resource_down, Reason}` Result -> Result catch Err:Reason:ST -> - ModB = atom_to_binary(Mod, utf8), - Msg = <<"call failed, func: ", ModB/binary, ":on_query/3">>, - exception_error(Reason, Msg, {Err, Reason, ST}) + Msg = io_lib:format( + "call query failed, func: ~s:~s/3, error: ~0p", + [Mod, Fun, {Err, Reason, ST}] + ), + ?RESOURCE_ERROR(exception, Msg) end; {ok, _Group, #{status := stopped}} -> - resource_error(stopped, <<"resource stopped or disabled">>); + ?RESOURCE_ERROR(stopped, "resource stopped or disabled"); {ok, _Group, _Data} -> - resource_error(not_connected, <<"resource not connected">>); + ?RESOURCE_ERROR(not_connected, "resource not connected"); {error, not_found} -> - resource_error(not_found, <<"resource not found">>) + ?RESOURCE_ERROR(not_found, "resource not found") end. -call_batch_query(Id, Batch) -> - ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched, length(Batch)), - case emqx_resource_manager:ets_lookup(Id) of - {ok, _Group, #{mod := Mod, state := ResourceState, status := connected}} -> - try Mod:on_batch_query(Id, Batch, ResourceState) of - BatchResults -> BatchResults - catch - Err:Reason:ST -> - ModB = atom_to_binary(Mod, utf8), - Msg = <<"call failed, func: ", ModB/binary, ":on_batch_query/3">>, - ?EXPAND(exception_error(Reason, Msg, {Err, Reason, ST}), Batch) - end; - {ok, _Group, _Data} -> - ?EXPAND(resource_error(not_connected, <<"resource not connected">>), Batch); - {error, not_found} -> - ?EXPAND(resource_error(not_found, <<"resource not found">>), Batch) - end. +%% the result is already expaned by the `Mod:on_query/3` +maybe_expand_batch_result(Results, _Batch) when is_list(Results) -> + Results; +%% the `Mod:on_query/3` returns a sinle result for a batch, so it is need expand +maybe_expand_batch_result(Result, Batch) -> + ?EXPAND(Result, Batch). -resource_error(Reason, Msg) -> - {error, {resource_error, #{reason => Reason, msg => Msg}}}. -exception_error(Reason, Msg, Details) -> - {error, {exception, #{reason => Reason, msg => Msg, details => Details}}}. +%%============================================================================== + +-spec name(id(), integer()) -> atom(). +name(Id, Index) -> + list_to_atom(lists:concat([?MODULE, ":", Id, ":", Index])). + +disk_queue_dir(Id) -> + filename:join([emqx:data_dir(), Id, queue]). -%% ========================================== ensure_flush_timer(St = #{tref := undefined, batch_time := T}) -> Ref = make_ref(), TRef = erlang:send_after(T, self(), {flush, Ref}), @@ -239,14 +326,3 @@ cancel_flush_timer(St = #{tref := undefined}) -> cancel_flush_timer(St = #{tref := {TRef, _Ref}}) -> _ = erlang:cancel_timer(TRef), St#{tref => undefined}. - -query_mfa(InsertMode, Request, SyncTimeout) -> - {?MODULE, query_fun(InsertMode), query_args(InsertMode, Request, SyncTimeout)}. - -query_fun(<<"sync">>) -> query; -query_fun(<<"async">>) -> query_async. - -query_args(<<"sync">>, Request, SyncTimeout) -> - [Request, SyncTimeout]; -query_args(<<"async">>, Request, _) -> - [Request]. From 0087b7c960a77c62f2caa873d425cafb22f0b866 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Wed, 27 Jul 2022 15:43:23 +0800 Subject: [PATCH 03/14] fix: remove the extra file replay.erl --- .../src/emqx_resource_manager.erl | 1 + .../src/emqx_resource_worker.erl | 25 +- apps/emqx_resource/src/replayq.erl | 779 ------------------ 3 files changed, 20 insertions(+), 785 deletions(-) delete mode 100644 apps/emqx_resource/src/replayq.erl diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index ee18a6836..b8a3812b5 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -423,6 +423,7 @@ handle_remove_event(From, ClearMetrics, Data) -> true -> ok = emqx_metrics_worker:clear_metrics(?RES_METRICS, Data#data.id); false -> ok end, + ok = emqx_resource_sup:stop_workers(Data#data.id, Data#data.opts), {stop_and_reply, normal, [{reply, From, ok}]}. start_resource(Data, From) -> diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index 3c5c7eefe..7acf2d0f9 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -27,7 +27,9 @@ -export([ start_link/3, + query/2, query/3, + query_async/3, query_async/4, block/1, resume/1 @@ -70,10 +72,18 @@ callback_mode() -> [state_functions, state_enter]. start_link(Id, Index, Opts) -> gen_statem:start_link({local, name(Id, Index)}, ?MODULE, {Id, Index, Opts}, []). +-spec query(id(), request()) -> ok. +query(Id, Request) -> + gen_statem:call(pick(Id, self()), {query, Request}). + -spec query(id(), term(), request()) -> ok. query(Id, Key, Request) -> gen_statem:call(pick(Id, Key), {query, Request}). +-spec query_async(id(), request(), reply_fun()) -> ok. +query_async(Id, Request, ReplyFun) -> + gen_statem:cast(pick(Id, self()), {query, Request, ReplyFun}). + -spec query_async(id(), term(), request(), reply_fun()) -> ok. query_async(Id, Key, Request, ReplyFun) -> gen_statem:cast(pick(Id, Key), {query, Request, ReplyFun}). @@ -91,7 +101,7 @@ init({Id, Index, Opts}) -> BatchSize = maps:get(batch_size, Opts, ?DEFAULT_BATCH_SIZE), Queue = case maps:get(queue_enabled, Opts, true) of - true -> replayq:open(#{dir => disk_queue_dir(Id), seg_bytes => 10000000}); + true -> replayq:open(#{dir => disk_queue_dir(Id, Index), seg_bytes => 10000000}); false -> undefined end, St = #{ @@ -157,13 +167,13 @@ do_resume(#{queue := undefined} = St) -> do_resume(#{queue := Q, id := Id} = St) -> case replayq:peek(Q) of empty -> - {next_state, running, St, {state_timeout, ?RESUME_INTERVAL, resume}}; + {next_state, running, St}; First -> Result = call_query(Id, First), case handle_query_result(Id, false, Result) of %% Send failed because resource down true -> - {keep_state, St}; + {keep_state, St, {state_timeout, ?RESUME_INTERVAL, resume}}; %% Send ok or failed but the resource is working false -> %% We Send 'resume' to the end of the mailbox to give the worker @@ -309,10 +319,13 @@ maybe_expand_batch_result(Result, Batch) -> -spec name(id(), integer()) -> atom(). name(Id, Index) -> - list_to_atom(lists:concat([?MODULE, ":", Id, ":", Index])). + Mod = atom_to_list(?MODULE), + Id1 = binary_to_list(Id), + Index1 = integer_to_list(Index), + list_to_atom(lists:concat([Mod, ":", Id1, ":", Index1])). -disk_queue_dir(Id) -> - filename:join([emqx:data_dir(), Id, queue]). +disk_queue_dir(Id, Index) -> + filename:join([node(), emqx:data_dir(), Id, "queue:" ++ integer_to_list(Index)]). ensure_flush_timer(St = #{tref := undefined, batch_time := T}) -> Ref = make_ref(), diff --git a/apps/emqx_resource/src/replayq.erl b/apps/emqx_resource/src/replayq.erl deleted file mode 100644 index 437702a5e..000000000 --- a/apps/emqx_resource/src/replayq.erl +++ /dev/null @@ -1,779 +0,0 @@ --module(replayq). - --export([open/1, close/1]). --export([append/2, pop/2, ack/2, ack_sync/2, peek/1, overflow/1]). --export([count/1, bytes/1, is_empty/1, is_mem_only/1]). -%% exported for troubleshooting --export([do_read_items/2]). - -%% internal exports for beam reload --export([committer_loop/2, default_sizer/1, default_marshaller/1]). - --export_type([config/0, q/0, ack_ref/0, sizer/0, marshaller/0]). - --define(NOTHING_TO_ACK, nothing_to_ack). --define(PENDING_ACKS(Ref), {replayq_pending_acks, Ref}). - --type segno() :: pos_integer(). --type item() :: term(). --type count() :: non_neg_integer(). --type id() :: count(). --type bytes() :: non_neg_integer(). --type filename() :: file:filename_all(). --type dir() :: filename(). --type ack_ref() :: ?NOTHING_TO_ACK | {segno(), ID :: pos_integer()}. --type sizer() :: fun((item()) -> bytes()). --type marshaller() :: fun((item()) -> binary()). - --type config() :: #{ - dir => dir(), - seg_bytes => bytes(), - mem_only => boolean(), - max_total_bytes => bytes(), - offload => boolean(), - sizer => sizer(), - marshaller => marshaller() -}. -%% writer cursor --define(NO_FD, no_fd). --type w_cur() :: #{ - segno := segno(), - bytes := bytes(), - count := count(), - fd := ?NO_FD | file:fd() -}. - --type stats() :: #{ - bytes := bytes(), - count := count() -}. - --opaque q() :: #{ - config := mem_only | config(), - stats := stats(), - in_mem := queue:queue(in_mem_item()), - w_cur => w_cur(), - committer => pid(), - head_segno => segno(), - sizer := sizer(), - marshaller => marshaller(), - max_total_bytes := bytes() -}. - --define(LAYOUT_VSN_0, 0). --define(LAYOUT_VSN_1, 1). --define(MAGIC, 841265288). --define(SUFFIX, "replaylog"). --define(DEFAULT_POP_BYTES_LIMIT, 2000000). --define(DEFAULT_POP_COUNT_LIMIT, 1000). --define(DEFAULT_REPLAYQ_LIMIT, 2000000000). --define(COMMIT(SEGNO, ID, From), {commit, SEGNO, ID, From}). --define(NO_COMMIT_HIST, no_commit_hist). --define(FIRST_SEGNO, 1). --define(NEXT_SEGNO(N), (N + 1)). --define(STOP, stop). --define(MEM_ONLY_ITEM(Bytes, Item), {Bytes, Item}). --define(DISK_CP_ITEM(Id, Bytes, Item), {Id, Bytes, Item}). - --type in_mem_item() :: - ?MEM_ONLY_ITEM(bytes(), item()) - | ?DISK_CP_ITEM(id(), bytes(), item()). - --spec open(config()) -> q(). -open(#{mem_only := true} = C) -> - #{ - stats => #{bytes => 0, count => 0}, - in_mem => queue:new(), - sizer => get_sizer(C), - config => mem_only, - max_total_bytes => maps:get(max_total_bytes, C, ?DEFAULT_REPLAYQ_LIMIT) - }; -open(#{dir := Dir, seg_bytes := _} = Config) -> - ok = filelib:ensure_dir(filename:join(Dir, "foo")), - Sizer = get_sizer(Config), - Marshaller = get_marshaller(Config), - IsOffload = is_offload_mode(Config), - Q = - case delete_consumed_and_list_rest(Dir) of - [] -> - %% no old segments, start over from zero - #{ - stats => #{bytes => 0, count => 0}, - w_cur => init_writer(Dir, empty, IsOffload), - committer => spawn_committer(?FIRST_SEGNO, Dir), - head_segno => ?FIRST_SEGNO, - in_mem => queue:new() - }; - Segs -> - LastSegno = lists:last(Segs), - CommitHist = get_commit_hist(Dir), - Reader = fun(Seg, Ch) -> read_items(Dir, Seg, Ch, Sizer, Marshaller) end, - HeadItems = Reader(hd(Segs), CommitHist), - #{ - stats => collect_stats(HeadItems, tl(Segs), Reader), - w_cur => init_writer(Dir, LastSegno, IsOffload), - committer => spawn_committer(hd(Segs), Dir), - head_segno => hd(Segs), - in_mem => queue:from_list(HeadItems) - } - end, - Q#{ - sizer => Sizer, - marshaller => Marshaller, - config => maps:without([sizer, marshaller], Config), - max_total_bytes => maps:get(max_total_bytes, Config, ?DEFAULT_REPLAYQ_LIMIT) - }. - --spec close(q() | w_cur()) -> ok | {error, any()}. -close(#{config := mem_only}) -> - ok; -close(#{w_cur := W_Cur, committer := Pid} = Q) -> - MRef = erlang:monitor(process, Pid), - Pid ! ?STOP, - unlink(Pid), - receive - {'DOWN', MRef, process, Pid, _Reason} -> - ok - end, - ok = maybe_dump_back_to_disk(Q), - do_close(W_Cur). - -do_close(#{fd := ?NO_FD}) -> ok; -do_close(#{fd := Fd}) -> file:close(Fd). - -%% In case of offload mode, dump the unacked (and un-popped) on disk -%% before close. this serves as a best-effort data loss protection -maybe_dump_back_to_disk(#{config := Config} = Q) -> - case is_offload_mode(Config) of - true -> dump_back_to_disk(Q); - false -> ok - end. - -dump_back_to_disk(#{ - config := #{dir := Dir}, - head_segno := ReaderSegno, - in_mem := InMem, - marshaller := Marshaller -}) -> - IoData0 = get_unacked(process_info(self(), dictionary), ReaderSegno, Marshaller), - Items1 = queue:to_list(InMem), - IoData1 = lists:map(fun(?DISK_CP_ITEM(_, _, I)) -> make_iodata(I, Marshaller) end, Items1), - %% ensure old segment file is deleted - ok = ensure_deleted(filename(Dir, ReaderSegno)), - %% rewrite the segment with what's currently in memory - IoData = [IoData0, IoData1], - case iolist_size(IoData) > 0 of - true -> - #{fd := Fd} = open_segment(Dir, ReaderSegno), - ok = file:write(Fd, [IoData0, IoData1]), - ok = file:close(Fd); - false -> - %% nothing to write - ok - end. - -get_unacked({dictionary, Dict}, ReaderSegno, Marshaller) -> - F = fun - ({?PENDING_ACKS(AckRef), Items}) -> - erase(?PENDING_ACKS(AckRef)), - {Segno, Id} = AckRef, - Segno =:= ReaderSegno andalso - {true, {Id, Items}}; - (_) -> - false - end, - Pendings0 = lists:filtermap(F, Dict), - Pendings = lists:keysort(1, Pendings0), - do_get_unacked(Pendings, Marshaller). - -do_get_unacked([], _Marshaller) -> - []; -do_get_unacked([{_, Items} | Rest], Marshaller) -> - [ - [make_iodata(I, Marshaller) || I <- Items] - | do_get_unacked(Rest, Marshaller) - ]. - --spec append(q(), [item()]) -> q(). -append(Q, []) -> - Q; -append( - #{ - config := mem_only, - in_mem := InMem, - stats := #{bytes := Bytes0, count := Count0}, - sizer := Sizer - } = Q, - Items0 -) -> - {CountDiff, BytesDiff, Items} = transform(false, Items0, Sizer), - - Stats = #{count => Count0 + CountDiff, bytes => Bytes0 + BytesDiff}, - Q#{ - stats := Stats, - in_mem := append_in_mem(Items, InMem) - }; -append( - #{ - config := #{seg_bytes := BytesLimit, dir := Dir} = Config, - stats := #{bytes := Bytes0, count := Count0}, - w_cur := #{count := CountInSeg, segno := WriterSegno} = W_Cur0, - head_segno := ReaderSegno, - sizer := Sizer, - marshaller := Marshaller, - in_mem := HeadItems0 - } = Q, - Items0 -) -> - IoData = lists:map(fun(I) -> make_iodata(I, Marshaller) end, Items0), - {CountDiff, BytesDiff, Items} = transform(CountInSeg + 1, Items0, Sizer), - TotalBytes = Bytes0 + BytesDiff, - Stats = #{count => Count0 + CountDiff, bytes => TotalBytes}, - IsOffload = is_offload_mode(Config), - W_Cur1 = do_append(W_Cur0, CountDiff, BytesDiff, IoData), - W_Cur = - case is_segment_full(W_Cur1, TotalBytes, BytesLimit, ReaderSegno, IsOffload) of - true -> - ok = do_close(W_Cur1), - %% get ready for the next append - open_segment(Dir, ?NEXT_SEGNO(WriterSegno)); - false -> - W_Cur1 - end, - HeadItems = - case ReaderSegno =:= WriterSegno of - true -> append_in_mem(Items, HeadItems0); - false -> HeadItems0 - end, - Q#{ - stats := Stats, - w_cur := W_Cur, - in_mem := HeadItems - }. - -%% @doc pop out at least one item from the queue. -%% volume limited by `bytes_limit' and `count_limit'. --spec pop(q(), #{bytes_limit => bytes(), count_limit => count()}) -> - {q(), ack_ref(), [item()]}. -pop(Q, Opts) -> - Bytes = maps:get(bytes_limit, Opts, ?DEFAULT_POP_BYTES_LIMIT), - Count = maps:get(count_limit, Opts, ?DEFAULT_POP_COUNT_LIMIT), - true = (Count > 0), - pop(Q, Bytes, Count, ?NOTHING_TO_ACK, []). - -%% @doc peek the queue front item. --spec peek(q()) -> empty | item(). -peek(#{in_mem := HeadItems}) -> - case queue:peek(HeadItems) of - empty -> empty; - {value, ?MEM_ONLY_ITEM(_, Item)} -> Item; - {value, ?DISK_CP_ITEM(_, _, Item)} -> Item - end. - -%% @doc Asynch-ly write the consumed item Segment number + ID to a file. --spec ack(q(), ack_ref()) -> ok. -ack(_, ?NOTHING_TO_ACK) -> - ok; -ack(#{committer := Pid}, {Segno, Id} = AckRef) -> - _ = erlang:erase(?PENDING_ACKS(AckRef)), - Pid ! ?COMMIT(Segno, Id, false), - ok. - -%% @hidden Synced ack, for deterministic tests only --spec ack_sync(q(), ack_ref()) -> ok. -ack_sync(_, ?NOTHING_TO_ACK) -> - ok; -ack_sync(#{committer := Pid}, {Segno, Id} = AckRef) -> - _ = erlang:erase(?PENDING_ACKS(AckRef)), - Ref = make_ref(), - Pid ! ?COMMIT(Segno, Id, {self(), Ref}), - receive - {Ref, ok} -> ok - end. - --spec count(q()) -> count(). -count(#{stats := #{count := Count}}) -> Count. - --spec bytes(q()) -> bytes(). -bytes(#{stats := #{bytes := Bytes}}) -> Bytes. - -is_empty(#{config := mem_only, in_mem := All}) -> - queue:is_empty(All); -is_empty( - #{ - w_cur := #{segno := WriterSegno}, - head_segno := ReaderSegno, - in_mem := HeadItems - } = Q -) -> - Result = ((WriterSegno =:= ReaderSegno) andalso queue:is_empty(HeadItems)), - %% assert - Result = (count(Q) =:= 0). - -%% @doc Returns number of bytes the size of the queue has exceeded -%% total bytes limit. Result is negative when it is not overflow. --spec overflow(q()) -> integer(). -overflow(#{ - max_total_bytes := MaxTotalBytes, - stats := #{bytes := Bytes} -}) -> - Bytes - MaxTotalBytes. - --spec is_mem_only(q()) -> boolean(). -is_mem_only(#{config := mem_only}) -> - true; -is_mem_only(_) -> - false. - -%% internals ========================================================= - -transform(Id, Items, Sizer) -> - transform(Id, Items, Sizer, 0, 0, []). - -transform(_Id, [], _Sizer, Count, Bytes, Acc) -> - {Count, Bytes, lists:reverse(Acc)}; -transform(Id, [Item0 | Rest], Sizer, Count, Bytes, Acc) -> - Size = Sizer(Item0), - {NextId, Item} = - case Id of - false -> {false, ?MEM_ONLY_ITEM(Size, Item0)}; - N -> {N + 1, ?DISK_CP_ITEM(Id, Size, Item0)} - end, - transform(NextId, Rest, Sizer, Count + 1, Bytes + Size, [Item | Acc]). - -append_in_mem([], Q) -> Q; -append_in_mem([Item | Rest], Q) -> append_in_mem(Rest, queue:in(Item, Q)). - -pop(Q, _Bytes, 0, AckRef, Acc) -> - Result = lists:reverse(Acc), - ok = maybe_save_pending_acks(AckRef, Q, Result), - {Q, AckRef, Result}; -pop(#{config := Cfg} = Q, Bytes, Count, AckRef, Acc) -> - case is_empty(Q) of - true -> - {Q, AckRef, lists:reverse(Acc)}; - false when Cfg =:= mem_only -> - pop_mem(Q, Bytes, Count, Acc); - false -> - pop2(Q, Bytes, Count, AckRef, Acc) - end. - -pop_mem( - #{ - in_mem := InMem, - stats := #{count := TotalCount, bytes := TotalBytes} = Stats - } = Q, - Bytes, - Count, - Acc -) -> - case queue:out(InMem) of - {{value, ?MEM_ONLY_ITEM(Sz, _Item)}, _} when Sz > Bytes andalso Acc =/= [] -> - {Q, ?NOTHING_TO_ACK, lists:reverse(Acc)}; - {{value, ?MEM_ONLY_ITEM(Sz, Item)}, Rest} -> - NewQ = Q#{ - in_mem := Rest, - stats := Stats#{ - count := TotalCount - 1, - bytes := TotalBytes - Sz - } - }, - pop(NewQ, Bytes - Sz, Count - 1, ?NOTHING_TO_ACK, [Item | Acc]) - end. - -pop2( - #{ - head_segno := ReaderSegno, - in_mem := HeadItems, - stats := #{count := TotalCount, bytes := TotalBytes} = Stats, - w_cur := #{segno := WriterSegno} - } = Q, - Bytes, - Count, - AckRef, - Acc -) -> - case queue:out(HeadItems) of - {{value, ?DISK_CP_ITEM(_, Sz, _Item)}, _} when Sz > Bytes andalso Acc =/= [] -> - %% taking the head item would cause exceeding size limit - {Q, AckRef, lists:reverse(Acc)}; - {{value, ?DISK_CP_ITEM(Id, Sz, Item)}, Rest} -> - Q1 = Q#{ - in_mem := Rest, - stats := Stats#{ - count := TotalCount - 1, - bytes := TotalBytes - Sz - } - }, - %% read the next segment in case current is drained - NewQ = - case queue:is_empty(Rest) andalso ReaderSegno < WriterSegno of - true -> read_next_seg(Q1); - false -> Q1 - end, - NewAckRef = {ReaderSegno, Id}, - pop(NewQ, Bytes - Sz, Count - 1, NewAckRef, [Item | Acc]) - end. - -%% due to backward compatibility reasons for the ack api -%% we ca nnot track pending acks in q() -- reason to use process dictionary -maybe_save_pending_acks(?NOTHING_TO_ACK, _, _) -> - ok; -maybe_save_pending_acks(AckRef, #{config := Config}, Items) -> - case is_offload_mode(Config) of - true -> - _ = erlang:put(?PENDING_ACKS(AckRef), Items), - ok; - false -> - ok - end. - -read_next_seg( - #{ - config := #{dir := Dir} = Config, - head_segno := ReaderSegno, - w_cur := #{segno := WriterSegno, fd := Fd} = WCur0, - sizer := Sizer, - marshaller := Marshaller - } = Q -) -> - NextSegno = ReaderSegno + 1, - %% reader has caught up to latest segment - case NextSegno =:= WriterSegno of - true -> - %% force flush to disk so the next read can get all bytes - ok = file:sync(Fd); - false -> - ok - end, - IsOffload = is_offload_mode(Config), - WCur = - case IsOffload andalso NextSegno =:= WriterSegno of - true -> - %% reader has caught up to latest segment in offload mode, - %% close the writer's fd. Continue in mem-only mode for the head segment - ok = do_close(WCur0), - WCur0#{fd := ?NO_FD}; - false -> - WCur0 - end, - NextSegItems = read_items(Dir, NextSegno, ?NO_COMMIT_HIST, Sizer, Marshaller), - Q#{ - head_segno := NextSegno, - in_mem := queue:from_list(NextSegItems), - w_cur := WCur - }. - -delete_consumed_and_list_rest(Dir0) -> - Dir = unicode:characters_to_list(Dir0), - Segnos0 = lists:sort([parse_segno(N) || N <- filelib:wildcard("*." ?SUFFIX, Dir)]), - {SegnosToDelete, Segnos} = find_segnos_to_delete(Dir, Segnos0), - ok = lists:foreach(fun(Segno) -> ensure_deleted(filename(Dir, Segno)) end, SegnosToDelete), - case Segnos of - [] -> - %% delete commit file in case there is no segments left - %% segment number will start from 0 again. - ensure_deleted(commit_filename(Dir)), - []; - X -> - X - end. - -find_segnos_to_delete(Dir, Segnos) -> - CommitHist = get_commit_hist(Dir), - do_find_segnos_to_delete(Dir, Segnos, CommitHist). - -do_find_segnos_to_delete(_Dir, Segnos, ?NO_COMMIT_HIST) -> - {[], Segnos}; -do_find_segnos_to_delete(Dir, Segnos0, {CommittedSegno, CommittedId}) -> - {SegnosToDelete, Segnos} = lists:partition(fun(N) -> N < CommittedSegno end, Segnos0), - case - Segnos =/= [] andalso - hd(Segnos) =:= CommittedSegno andalso - is_all_consumed(Dir, CommittedSegno, CommittedId) - of - true -> - %% assert - CommittedSegno = hd(Segnos), - %% all items in the oldest segment have been consumed, - %% no need to keep this segment - {[CommittedSegno | SegnosToDelete], tl(Segnos)}; - _ -> - {SegnosToDelete, Segnos} - end. - -%% ALL items are consumed if the committed item ID is no-less than the number -%% of items in this segment -is_all_consumed(Dir, CommittedSegno, CommittedId) -> - CommittedId >= erlang:length(do_read_items(Dir, CommittedSegno)). - -ensure_deleted(Filename) -> - case file:delete(Filename) of - ok -> ok; - {error, enoent} -> ok - end. - -%% The committer writes consumer's acked segmeng number + item ID -%% to a file. The file is only read at start/restart. -spawn_committer(ReaderSegno, Dir) -> - Name = iolist_to_binary(filename:join([Dir, committer])), - %% register a name to avoid having two committers spawned for the same dir - RegName = binary_to_atom(Name, utf8), - Pid = erlang:spawn_link(fun() -> committer_loop(ReaderSegno, Dir) end), - true = erlang:register(RegName, Pid), - Pid. - -committer_loop(ReaderSegno, Dir) -> - receive - ?COMMIT(Segno0, Id0, false) -> - {Segno, Id} = collect_async_commits(Segno0, Id0), - ok = handle_commit(ReaderSegno, Dir, Segno, Id, false), - ?MODULE:committer_loop(Segno, Dir); - ?COMMIT(Segno, Id, From) -> - ok = handle_commit(ReaderSegno, Dir, Segno, Id, From), - ?MODULE:committer_loop(Segno, Dir); - ?STOP -> - ok; - Msg -> - exit({replayq_committer_unkown_msg, Msg}) - after 200 -> - ?MODULE:committer_loop(ReaderSegno, Dir) - end. - -handle_commit(ReaderSegno, Dir, Segno, Id, From) -> - IoData = io_lib:format("~p.\n", [#{segno => Segno, id => Id}]), - ok = do_commit(Dir, IoData), - case Segno > ReaderSegno of - true -> - SegnosToDelete = lists:seq(ReaderSegno, Segno - 1), - lists:foreach(fun(N) -> ok = ensure_deleted(filename(Dir, N)) end, SegnosToDelete); - false -> - ok - end, - ok = reply_ack_ok(From). - -%% Collect async acks which are already sent in the mailbox, -%% and keep only the last one for the current segment. -collect_async_commits(Segno, Id) -> - receive - ?COMMIT(Segno, AnotherId, false) -> - collect_async_commits(Segno, AnotherId) - after 0 -> - {Segno, Id} - end. - -reply_ack_ok({Pid, Ref}) -> - Pid ! {Ref, ok}, - ok; -reply_ack_ok(_) -> - ok. - -get_commit_hist(Dir) -> - CommitFile = commit_filename(Dir), - case filelib:is_regular(CommitFile) of - true -> - {ok, [#{segno := Segno, id := Id}]} = file:consult(CommitFile), - {Segno, Id}; - false -> - ?NO_COMMIT_HIST - end. - -do_commit(Dir, IoData) -> - TmpName = commit_filename(Dir, "COMMIT.tmp"), - Name = commit_filename(Dir), - ok = file:write_file(TmpName, IoData), - ok = file:rename(TmpName, Name). - -commit_filename(Dir) -> - commit_filename(Dir, "COMMIT"). - -commit_filename(Dir, Name) -> - filename:join([Dir, Name]). - -do_append( - #{fd := ?NO_FD, bytes := Bytes0, count := Count0} = Cur, - Count, - Bytes, - _IoData -) -> - %% offload mode, fd is not initialized yet - Cur#{ - bytes => Bytes0 + Bytes, - count => Count0 + Count - }; -do_append( - #{fd := Fd, bytes := Bytes0, count := Count0} = Cur, - Count, - Bytes, - IoData -) -> - ok = file:write(Fd, IoData), - Cur#{ - bytes => Bytes0 + Bytes, - count => Count0 + Count - }. - -read_items(Dir, Segno, CommitHist, Sizer, Marshaller) -> - Items0 = do_read_items(Dir, Segno), - Items = - case CommitHist of - ?NO_COMMIT_HIST -> - %% no commit hist, return all - Items0; - {CommitedSegno, _} when CommitedSegno < Segno -> - %% committed at an older segment - Items0; - {Segno, CommittedId} -> - %% committed at current segment keep only the tail - {_, R} = lists:splitwith(fun({I, _}) -> I =< CommittedId end, Items0), - R - end, - lists:map( - fun({Id, Bin}) -> - Item = Marshaller(Bin), - Size = Sizer(Item), - ?DISK_CP_ITEM(Id, Size, Item) - end, - Items - ). - -do_read_items(Dir, Segno) -> - Filename = filename(Dir, Segno), - {ok, Bin} = file:read_file(Filename), - case parse_items(Bin, 1, []) of - {Items, <<>>} -> - Items; - {Items, Corrupted} -> - error_logger:error_msg( - "corrupted replayq log: ~s, skipped ~p bytes", - [Filename, size(Corrupted)] - ), - Items - end. - -parse_items(<<>>, _Id, Acc) -> - {lists:reverse(Acc), <<>>}; -parse_items( - <> = All, - Id, - Acc -) -> - case CRC =:= erlang:crc32(Item) of - true -> parse_items(Rest, Id + 1, [{Id, Item} | Acc]); - false -> {lists:reverse(Acc), All} - end; -parse_items( - <> = All, - Id, - Acc -) -> - case CRC =:= erlang:crc32(Item) andalso Item =/= <<>> of - true -> parse_items(Rest, Id + 1, [{Id, Item} | Acc]); - false -> {lists:reverse(Acc), All} - end; -parse_items(Corrupted, _Id, Acc) -> - {lists:reverse(Acc), Corrupted}. - -make_iodata(Item0, Marshaller) -> - Item = Marshaller(Item0), - Size = size(Item), - CRC = erlang:crc32(Item), - [ - <>, - Item - ]. - -collect_stats(HeadItems, SegsOnDisk, Reader) -> - ItemF = fun(?DISK_CP_ITEM(_Id, Sz, _Item), {B, C}) -> - {B + Sz, C + 1} - end, - Acc0 = lists:foldl(ItemF, {0, 0}, HeadItems), - {Bytes, Count} = - lists:foldl( - fun(Segno, Acc) -> - Items = Reader(Segno, ?NO_COMMIT_HIST), - lists:foldl(ItemF, Acc, Items) - end, - Acc0, - SegsOnDisk - ), - #{bytes => Bytes, count => Count}. - -parse_segno(Filename) -> - [Segno, ?SUFFIX] = string:tokens(Filename, "."), - list_to_integer(Segno). - -filename(Dir, Segno) -> - Name = lists:flatten(io_lib:format("~10.10.0w." ?SUFFIX, [Segno])), - filename:join(Dir, Name). - -%% open the current segment for write if it is empty -%% otherwise rollout to the next segment --spec init_writer(dir(), empty | segno(), boolean()) -> w_cur(). -init_writer(_Dir, empty, true) -> - %% clean start for offload mode - #{fd => ?NO_FD, segno => ?FIRST_SEGNO, bytes => 0, count => 0}; -init_writer(Dir, empty, false) -> - open_segment(Dir, ?FIRST_SEGNO); -init_writer(Dir, Segno, _IsOffload) when is_number(Segno) -> - Filename = filename(Dir, Segno), - case filelib:file_size(Filename) of - 0 -> open_segment(Dir, Segno); - _ -> open_segment(Dir, ?NEXT_SEGNO(Segno)) - end. - --spec open_segment(dir(), segno()) -> w_cur(). -open_segment(Dir, Segno) -> - Filename = filename(Dir, Segno), - %% raw so there is no need to go through the single gen_server file_server - {ok, Fd} = file:open(Filename, [raw, read, write, binary, delayed_write]), - #{fd => Fd, segno => Segno, bytes => 0, count => 0}. - -get_sizer(C) -> - maps:get(sizer, C, fun ?MODULE:default_sizer/1). - -get_marshaller(C) -> - maps:get(marshaller, C, fun ?MODULE:default_marshaller/1). - -is_offload_mode(Config) when is_map(Config) -> - maps:get(offload, Config, false). - -default_sizer(I) when is_binary(I) -> erlang:size(I). - -default_marshaller(I) when is_binary(I) -> I. - -is_segment_full( - #{segno := WriterSegno, bytes := SegmentBytes}, - TotalBytes, - SegmentBytesLimit, - ReaderSegno, - true -) -> - %% in offload mode, when reader is lagging behind, we try - %% writer rolls to a new segment when file size limit is reached - %% when reader is reading off from the same segment as writer - %% i.e. the in memory queue, only start writing to segment file - %% when total bytes (in memory) is reached segment limit - %% - %% NOTE: we never shrink segment bytes, even when popping out - %% from the in-memory queue. - case ReaderSegno < WriterSegno of - true -> SegmentBytes >= SegmentBytesLimit; - false -> TotalBytes >= SegmentBytesLimit - end; -is_segment_full( - #{bytes := SegmentBytes}, - _TotalBytes, - SegmentBytesLimit, - _ReaderSegno, - false -) -> - %% here we check if segment size is greater than segment size limit - %% after append based on the assumption that the items are usually - %% very small in size comparing to segment size. - %% We can change implementation to split items list to avoid - %% segment overflow if really necessary - SegmentBytes >= SegmentBytesLimit. From 2fb42e4d37442b141e1b0259bc71ae5807502704 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Thu, 28 Jul 2022 17:40:04 +0800 Subject: [PATCH 04/14] refactor: create emqx_resource_worker_sup for resource workers --- .../emqx_authn_jwks_connector.erl | 12 +- .../src/simple_authn/emqx_authn_mongodb.erl | 2 +- .../src/emqx_connector_http.erl | 29 ++-- .../src/emqx_connector_ldap.erl | 9 +- .../src/emqx_connector_mongo.erl | 10 +- .../src/emqx_connector_mqtt.erl | 10 +- .../src/emqx_connector_mysql.erl | 17 +-- .../src/emqx_connector_pgsql.erl | 13 +- .../src/emqx_connector_redis.erl | 9 +- apps/emqx_resource/include/emqx_resource.hrl | 14 +- apps/emqx_resource/src/emqx_resource.erl | 32 +---- .../src/emqx_resource_manager.erl | 15 +- apps/emqx_resource/src/emqx_resource_sup.erl | 85 ++--------- .../src/emqx_resource_worker.erl | 132 ++++++++++------- .../src/emqx_resource_worker_sup.erl | 136 ++++++++++++++++++ .../test/emqx_resource_SUITE.erl | 77 +++++----- .../emqx_resource/test/emqx_test_resource.erl | 27 ++-- 17 files changed, 332 insertions(+), 297 deletions(-) create mode 100644 apps/emqx_resource/src/emqx_resource_worker_sup.erl diff --git a/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl b/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl index 8f98e2f1e..cd8451ac9 100644 --- a/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl +++ b/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl @@ -24,7 +24,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2, connect/1 ]). @@ -45,7 +45,7 @@ on_start(InstId, Opts) -> on_stop(_InstId, #{pool_name := PoolName}) -> emqx_plugin_libs_pool:stop_pool(PoolName). -on_query(InstId, get_jwks, AfterQuery, #{pool_name := PoolName}) -> +on_query(InstId, get_jwks, #{pool_name := PoolName}) -> Result = ecpool:pick_and_do(PoolName, {emqx_authn_jwks_client, get_jwks, []}, no_handover), case Result of {error, Reason} -> @@ -54,20 +54,18 @@ on_query(InstId, get_jwks, AfterQuery, #{pool_name := PoolName}) -> connector => InstId, command => get_jwks, reason => Reason - }), - emqx_resource:query_failed(AfterQuery); + }); _ -> - emqx_resource:query_success(AfterQuery) + ok end, Result; -on_query(_InstId, {update, Opts}, AfterQuery, #{pool_name := PoolName}) -> +on_query(_InstId, {update, Opts}, #{pool_name := PoolName}) -> lists:foreach( fun({_, Worker}) -> ok = ecpool_worker:exec(Worker, {emqx_authn_jwks_client, update, [Opts]}, infinity) end, ecpool:workers(PoolName) ), - emqx_resource:query_success(AfterQuery), ok. on_get_status(_InstId, #{pool_name := PoolName}) -> diff --git a/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl b/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl index f7249ae57..ff9c97717 100644 --- a/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl +++ b/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl @@ -174,7 +174,7 @@ authenticate( reason => Reason }), ignore; - Doc -> + {ok, Doc} -> case check_password(Password, Doc, State) of ok -> {ok, is_superuser(Doc, State)}; diff --git a/apps/emqx_connector/src/emqx_connector_http.erl b/apps/emqx_connector/src/emqx_connector_http.erl index 59b4ddffa..e0d5ccfe0 100644 --- a/apps/emqx_connector/src/emqx_connector_http.erl +++ b/apps/emqx_connector/src/emqx_connector_http.erl @@ -28,7 +28,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -225,7 +225,7 @@ on_stop(InstId, #{pool_name := PoolName}) -> }), ehttpc_sup:stop_pool(PoolName). -on_query(InstId, {send_message, Msg}, AfterQuery, State) -> +on_query(InstId, {send_message, Msg}, State) -> case maps:get(request, State, undefined) of undefined -> ?SLOG(error, #{msg => "request_not_found", connector => InstId}); @@ -241,18 +241,16 @@ on_query(InstId, {send_message, Msg}, AfterQuery, State) -> on_query( InstId, {undefined, Method, {Path, Headers, Body}, Timeout, Retry}, - AfterQuery, State ) end; -on_query(InstId, {Method, Request}, AfterQuery, State) -> - on_query(InstId, {undefined, Method, Request, 5000, 2}, AfterQuery, State); -on_query(InstId, {Method, Request, Timeout}, AfterQuery, State) -> - on_query(InstId, {undefined, Method, Request, Timeout, 2}, AfterQuery, State); +on_query(InstId, {Method, Request}, State) -> + on_query(InstId, {undefined, Method, Request, 5000, 2}, State); +on_query(InstId, {Method, Request, Timeout}, State) -> + on_query(InstId, {undefined, Method, Request, Timeout, 2}, State); on_query( InstId, {KeyOrNum, Method, Request, Timeout, Retry}, - AfterQuery, #{pool_name := PoolName, base_path := BasePath} = State ) -> ?TRACE( @@ -275,32 +273,29 @@ on_query( of {error, Reason} -> ?SLOG(error, #{ - msg => "http_connector_do_reqeust_failed", + msg => "http_connector_do_request_failed", request => NRequest, reason => Reason, connector => InstId - }), - emqx_resource:query_failed(AfterQuery); + }); {ok, StatusCode, _} when StatusCode >= 200 andalso StatusCode < 300 -> - emqx_resource:query_success(AfterQuery); + ok; {ok, StatusCode, _, _} when StatusCode >= 200 andalso StatusCode < 300 -> - emqx_resource:query_success(AfterQuery); + ok; {ok, StatusCode, _} -> ?SLOG(error, #{ msg => "http connector do request, received error response", request => NRequest, connector => InstId, status_code => StatusCode - }), - emqx_resource:query_failed(AfterQuery); + }); {ok, StatusCode, _, _} -> ?SLOG(error, #{ msg => "http connector do request, received error response", request => NRequest, connector => InstId, status_code => StatusCode - }), - emqx_resource:query_failed(AfterQuery) + }) end, Result. diff --git a/apps/emqx_connector/src/emqx_connector_ldap.erl b/apps/emqx_connector/src/emqx_connector_ldap.erl index 195aa89a9..51d18b534 100644 --- a/apps/emqx_connector/src/emqx_connector_ldap.erl +++ b/apps/emqx_connector/src/emqx_connector_ldap.erl @@ -27,7 +27,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -99,7 +99,7 @@ on_stop(InstId, #{poolname := PoolName}) -> }), emqx_plugin_libs_pool:stop_pool(PoolName). -on_query(InstId, {search, Base, Filter, Attributes}, AfterQuery, #{poolname := PoolName} = State) -> +on_query(InstId, {search, Base, Filter, Attributes}, #{poolname := PoolName} = State) -> Request = {Base, Filter, Attributes}, ?TRACE( "QUERY", @@ -119,10 +119,9 @@ on_query(InstId, {search, Base, Filter, Attributes}, AfterQuery, #{poolname := P request => Request, connector => InstId, reason => Reason - }), - emqx_resource:query_failed(AfterQuery); + }); _ -> - emqx_resource:query_success(AfterQuery) + ok end, Result. diff --git a/apps/emqx_connector/src/emqx_connector_mongo.erl b/apps/emqx_connector/src/emqx_connector_mongo.erl index 5b07c5003..db8b1e632 100644 --- a/apps/emqx_connector/src/emqx_connector_mongo.erl +++ b/apps/emqx_connector/src/emqx_connector_mongo.erl @@ -27,7 +27,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -189,7 +189,6 @@ on_stop(InstId, #{poolname := PoolName}) -> on_query( InstId, {Action, Collection, Filter, Projector}, - AfterQuery, #{poolname := PoolName} = State ) -> Request = {Action, Collection, Filter, Projector}, @@ -212,14 +211,11 @@ on_query( reason => Reason, connector => InstId }), - emqx_resource:query_failed(AfterQuery), {error, Reason}; {ok, Cursor} when is_pid(Cursor) -> - emqx_resource:query_success(AfterQuery), - mc_cursor:foldl(fun(O, Acc2) -> [O | Acc2] end, [], Cursor, 1000); + {ok, mc_cursor:foldl(fun(O, Acc2) -> [O | Acc2] end, [], Cursor, 1000)}; Result -> - emqx_resource:query_success(AfterQuery), - Result + {ok, Result} end. -dialyzer({nowarn_function, [on_get_status/2]}). diff --git a/apps/emqx_connector/src/emqx_connector_mqtt.erl b/apps/emqx_connector/src/emqx_connector_mqtt.erl index 21e201504..98635de3f 100644 --- a/apps/emqx_connector/src/emqx_connector_mqtt.erl +++ b/apps/emqx_connector/src/emqx_connector_mqtt.erl @@ -37,7 +37,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -181,12 +181,12 @@ on_stop(_InstId, #{name := InstanceId}) -> }) end. -on_query(_InstId, {message_received, _Msg}, AfterQuery, _State) -> - emqx_resource:query_success(AfterQuery); -on_query(_InstId, {send_message, Msg}, AfterQuery, #{name := InstanceId}) -> +on_query(_InstId, {message_received, _Msg}, _State) -> + ok; +on_query(_InstId, {send_message, Msg}, #{name := InstanceId}) -> ?TRACE("QUERY", "send_msg_to_remote_node", #{message => Msg, connector => InstanceId}), emqx_connector_mqtt_worker:send_to_remote(InstanceId, Msg), - emqx_resource:query_success(AfterQuery). + ok. on_get_status(_InstId, #{name := InstanceId, bridge_conf := Conf}) -> AutoReconn = maps:get(auto_reconnect, Conf, true), diff --git a/apps/emqx_connector/src/emqx_connector_mysql.erl b/apps/emqx_connector/src/emqx_connector_mysql.erl index 409da4060..e818bc6ef 100644 --- a/apps/emqx_connector/src/emqx_connector_mysql.erl +++ b/apps/emqx_connector/src/emqx_connector_mysql.erl @@ -26,7 +26,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -122,14 +122,13 @@ on_stop(InstId, #{poolname := PoolName}) -> }), emqx_plugin_libs_pool:stop_pool(PoolName). -on_query(InstId, {TypeOrKey, SQLOrKey}, AfterQuery, State) -> - on_query(InstId, {TypeOrKey, SQLOrKey, [], default_timeout}, AfterQuery, State); -on_query(InstId, {TypeOrKey, SQLOrKey, Params}, AfterQuery, State) -> - on_query(InstId, {TypeOrKey, SQLOrKey, Params, default_timeout}, AfterQuery, State); +on_query(InstId, {TypeOrKey, SQLOrKey}, State) -> + on_query(InstId, {TypeOrKey, SQLOrKey, [], default_timeout}, State); +on_query(InstId, {TypeOrKey, SQLOrKey, Params}, State) -> + on_query(InstId, {TypeOrKey, SQLOrKey, Params, default_timeout}, State); on_query( InstId, {TypeOrKey, SQLOrKey, Params, Timeout}, - AfterQuery, #{poolname := PoolName, prepare_statement := Prepares} = State ) -> LogMeta = #{connector => InstId, sql => SQLOrKey, state => State}, @@ -147,7 +146,6 @@ on_query( ), %% kill the poll worker to trigger reconnection _ = exit(Conn, restart), - emqx_resource:query_failed(AfterQuery), Result; {error, not_prepared} -> ?SLOG( @@ -157,13 +155,12 @@ on_query( case prepare_sql(Prepares, PoolName) of ok -> %% not return result, next loop will try again - on_query(InstId, {TypeOrKey, SQLOrKey, Params, Timeout}, AfterQuery, State); + on_query(InstId, {TypeOrKey, SQLOrKey, Params, Timeout}, State); {error, Reason} -> ?SLOG( error, LogMeta#{msg => "mysql_connector_do_prepare_failed", reason => Reason} ), - emqx_resource:query_failed(AfterQuery), {error, Reason} end; {error, Reason} -> @@ -171,10 +168,8 @@ on_query( error, LogMeta#{msg => "mysql_connector_do_sql_query_failed", reason => Reason} ), - emqx_resource:query_failed(AfterQuery), Result; _ -> - emqx_resource:query_success(AfterQuery), Result end. diff --git a/apps/emqx_connector/src/emqx_connector_pgsql.erl b/apps/emqx_connector/src/emqx_connector_pgsql.erl index 6f89e7ff1..d31c1316f 100644 --- a/apps/emqx_connector/src/emqx_connector_pgsql.erl +++ b/apps/emqx_connector/src/emqx_connector_pgsql.erl @@ -29,7 +29,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -116,9 +116,9 @@ on_stop(InstId, #{poolname := PoolName}) -> }), emqx_plugin_libs_pool:stop_pool(PoolName). -on_query(InstId, {Type, NameOrSQL}, AfterQuery, #{poolname := _PoolName} = State) -> - on_query(InstId, {Type, NameOrSQL, []}, AfterQuery, State); -on_query(InstId, {Type, NameOrSQL, Params}, AfterQuery, #{poolname := PoolName} = State) -> +on_query(InstId, {Type, NameOrSQL}, #{poolname := _PoolName} = State) -> + on_query(InstId, {Type, NameOrSQL, []}, State); +on_query(InstId, {Type, NameOrSQL, Params}, #{poolname := PoolName} = State) -> ?SLOG(debug, #{ msg => "postgresql connector received sql query", connector => InstId, @@ -132,10 +132,9 @@ on_query(InstId, {Type, NameOrSQL, Params}, AfterQuery, #{poolname := PoolName} connector => InstId, sql => NameOrSQL, reason => Reason - }), - emqx_resource:query_failed(AfterQuery); + }); _ -> - emqx_resource:query_success(AfterQuery) + ok end, Result. diff --git a/apps/emqx_connector/src/emqx_connector_redis.erl b/apps/emqx_connector/src/emqx_connector_redis.erl index 67310dbac..4826a170b 100644 --- a/apps/emqx_connector/src/emqx_connector_redis.erl +++ b/apps/emqx_connector/src/emqx_connector_redis.erl @@ -28,7 +28,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -177,7 +177,7 @@ on_stop(InstId, #{poolname := PoolName, type := Type}) -> _ -> emqx_plugin_libs_pool:stop_pool(PoolName) end. -on_query(InstId, {cmd, Command}, AfterCommand, #{poolname := PoolName, type := Type} = State) -> +on_query(InstId, {cmd, Command}, #{poolname := PoolName, type := Type} = State) -> ?TRACE( "QUERY", "redis_connector_received", @@ -195,10 +195,9 @@ on_query(InstId, {cmd, Command}, AfterCommand, #{poolname := PoolName, type := T connector => InstId, sql => Command, reason => Reason - }), - emqx_resource:query_failed(AfterCommand); + }); _ -> - emqx_resource:query_success(AfterCommand) + ok end, Result. diff --git a/apps/emqx_resource/include/emqx_resource.hrl b/apps/emqx_resource/include/emqx_resource.hrl index d6f959510..13ffff587 100644 --- a/apps/emqx_resource/include/emqx_resource.hrl +++ b/apps/emqx_resource/include/emqx_resource.hrl @@ -21,7 +21,7 @@ -type resource_config() :: term(). -type resource_spec() :: map(). -type resource_state() :: term(). --type resource_status() :: connected | disconnected | connecting. +-type resource_status() :: connected | disconnected | connecting | stopped. -type resource_data() :: #{ id := resource_id(), mod := module(), @@ -45,13 +45,11 @@ %% periodically. auto_retry_interval => integer() }. --type after_query() :: - {[OnSuccess :: after_query_fun()], [OnFailed :: after_query_fun()]} - | undefined. - -%% the `after_query_fun()` is mainly for callbacks that increment counters or do some fallback -%% actions upon query failure --type after_query_fun() :: {fun((...) -> ok), Args :: [term()]}. +-type query_result() :: + ok + | {ok, term()} + | {error, term()} + | {resource_down, term()}. -define(TEST_ID_PREFIX, "_test_:"). -define(RES_METRICS, resource_metrics). diff --git a/apps/emqx_resource/src/emqx_resource.erl b/apps/emqx_resource/src/emqx_resource.erl index 793b9f446..081264315 100644 --- a/apps/emqx_resource/src/emqx_resource.erl +++ b/apps/emqx_resource/src/emqx_resource.erl @@ -23,13 +23,6 @@ -export([list_types/0]). -%% APIs for behaviour implementations - --export([ - query_success/1, - query_failed/1 -]). - %% APIs for instances -export([ @@ -113,7 +106,8 @@ -export([inc_metrics_funcs/1, inc_success/1, inc_failed/1]). -optional_callbacks([ - on_query/4, + on_query/3, + on_batch_query/3, on_get_status/2 ]). @@ -125,7 +119,9 @@ -callback on_stop(resource_id(), resource_state()) -> term(). %% when calling emqx_resource:query/3 --callback on_query(resource_id(), Request :: term(), after_query(), resource_state()) -> term(). +-callback on_query(resource_id(), Request :: term(), resource_state()) -> query_result(). + +-callback on_batch_query(resource_id(), Request :: term(), resource_state()) -> query_result(). %% when calling emqx_resource:health_check/2 -callback on_get_status(resource_id(), resource_state()) -> @@ -149,22 +145,6 @@ is_resource_mod(Module) -> proplists:get_value(behaviour, Info, []), lists:member(?MODULE, Behaviour). --spec query_success(after_query()) -> ok. -query_success(undefined) -> ok; -query_success({OnSucc, _}) -> exec_query_after_calls(OnSucc). - --spec query_failed(after_query()) -> ok. -query_failed(undefined) -> ok; -query_failed({_, OnFailed}) -> exec_query_after_calls(OnFailed). - -exec_query_after_calls(Funcs) -> - lists:foreach( - fun({Fun, Arg}) -> - emqx_resource_utils:safe_exec(Fun, Arg) - end, - Funcs - ). - %% ================================================================================= %% APIs for resource instances %% ================================================================================= @@ -247,7 +227,7 @@ query(ResId, Request) -> emqx_resource_worker:query(ResId, Request). -spec query_async(resource_id(), Request :: term(), emqx_resource_worker:reply_fun()) -> - ok. + Result :: term(). query_async(ResId, Request, ReplyFun) -> emqx_resource_worker:query_async(ResId, Request, ReplyFun). diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index b8a3812b5..b5bcbd330 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -53,7 +53,7 @@ -define(SHORT_HEALTHCHECK_INTERVAL, 1000). -define(HEALTHCHECK_INTERVAL, 15000). --define(ETS_TABLE, emqx_resource_manager). +-define(ETS_TABLE, ?MODULE). -define(WAIT_FOR_RESOURCE_DELAY, 100). -define(T_OPERATION, 5000). -define(T_LOOKUP, 1000). @@ -114,9 +114,9 @@ create(MgrId, ResId, Group, ResourceType, Config, Opts) -> [matched, success, failed, exception, resource_down], [matched] ), + ok = emqx_resource_worker_sup:start_workers(ResId, Opts), case maps:get(start_after_created, Opts, true) of true -> - ok = emqx_resource_sup:start_workers(ResId, Opts), wait_for_resource_ready(ResId, maps:get(wait_for_resource_ready, Opts, 5000)); false -> ok @@ -317,7 +317,7 @@ handle_event({call, From}, health_check, _State, Data) -> handle_manually_health_check(From, Data); % State: CONNECTING handle_event(enter, _OldState, connecting, Data) -> - UpdatedData = Data#data{status = connected}, + UpdatedData = Data#data{status = connecting}, insert_cache(Data#data.id, Data#data.group, Data), Actions = [{state_timeout, 0, health_check}], {keep_state, UpdatedData, Actions}; @@ -332,7 +332,7 @@ handle_event(enter, _OldState, connected, Data) -> UpdatedData = Data#data{status = connected}, insert_cache(Data#data.id, Data#data.group, UpdatedData), _ = emqx_alarm:deactivate(Data#data.id), - Actions = [{state_timeout, ?HEALTHCHECK_INTERVAL, health_check}], + Actions = [{state_timeout, health_check_interval(Data#data.opts), health_check}], {next_state, connected, UpdatedData, Actions}; handle_event(state_timeout, health_check, connected, Data) -> handle_connected_health_check(Data); @@ -423,7 +423,7 @@ handle_remove_event(From, ClearMetrics, Data) -> true -> ok = emqx_metrics_worker:clear_metrics(?RES_METRICS, Data#data.id); false -> ok end, - ok = emqx_resource_sup:stop_workers(Data#data.id, Data#data.opts), + ok = emqx_resource_worker_sup:stop_workers(Data#data.id, Data#data.opts), {stop_and_reply, normal, [{reply, From, ok}]}. start_resource(Data, From) -> @@ -487,7 +487,7 @@ handle_connected_health_check(Data) -> Data, fun (connected, UpdatedData) -> - Actions = [{state_timeout, ?HEALTHCHECK_INTERVAL, health_check}], + Actions = [{state_timeout, health_check_interval(Data#data.opts), health_check}], {keep_state, UpdatedData, Actions}; (Status, UpdatedData) -> ?SLOG(error, #{ @@ -510,6 +510,9 @@ with_health_check(Data, Func) -> insert_cache(ResId, UpdatedData#data.group, UpdatedData), Func(Status, UpdatedData). +health_check_interval(Opts) -> + maps:get(health_check_interval, Opts, ?HEALTHCHECK_INTERVAL). + maybe_alarm(connected, _ResId) -> ok; maybe_alarm(_Status, <>) -> diff --git a/apps/emqx_resource/src/emqx_resource_sup.erl b/apps/emqx_resource/src/emqx_resource_sup.erl index 14db50d01..920743101 100644 --- a/apps/emqx_resource/src/emqx_resource_sup.erl +++ b/apps/emqx_resource/src/emqx_resource_sup.erl @@ -19,7 +19,7 @@ -behaviour(supervisor). --export([start_link/0, start_workers/2, stop_workers/2]). +-export([start_link/0]). -export([init/1]). @@ -29,7 +29,6 @@ start_link() -> init([]) -> SupFlags = #{strategy => one_for_one, intensity => 10, period => 10}, Metrics = emqx_metrics_worker:child_spec(?RES_METRICS), - ResourceManager = #{ id => emqx_resource_manager_sup, @@ -39,79 +38,11 @@ init([]) -> type => supervisor, modules => [emqx_resource_manager_sup] }, - {ok, {SupFlags, [Metrics, ResourceManager]}}. - -start_workers(ResId, Opts) -> - PoolSize = pool_size(Opts), - _ = ensure_worker_pool(ResId, hash, [{size, PoolSize}]), - lists:foreach( - fun(Idx) -> - _ = ensure_worker_added(ResId, {ResId, Idx}, Idx), - ok = ensure_worker_started(ResId, Idx, Opts) - end, - lists:seq(1, PoolSize) - ). - -stop_workers(ResId, Opts) -> - PoolSize = pool_size(Opts), - lists:foreach( - fun(Idx) -> - ok = ensure_worker_stopped(ResId, Idx), - ok = ensure_worker_removed(ResId, {ResId, Idx}) - end, - lists:seq(1, PoolSize) - ), - _ = gproc_pool:delete(ResId), - ok. - -pool_size(Opts) -> - maps:get(worker_pool_size, Opts, erlang:system_info(schedulers_online)). - -ensure_worker_pool(Pool, Type, Opts) -> - try - gproc_pool:new(Pool, Type, Opts) - catch - error:exists -> ok - end, - ok. - -ensure_worker_added(Pool, Name, Slot) -> - try - gproc_pool:add_worker(Pool, Name, Slot) - catch - error:exists -> ok - end, - ok. - -ensure_worker_removed(Pool, Name) -> - _ = gproc_pool:remove_worker(Pool, Name), - ok. - --define(CHILD_ID(MOD, RESID, INDEX), {MOD, RESID, INDEX}). -ensure_worker_started(ResId, Idx, Opts) -> - Mod = emqx_resource_worker, - Spec = #{ - id => ?CHILD_ID(Mod, ResId, Idx), - start => {Mod, start_link, [ResId, Idx, Opts]}, - restart => transient, - shutdown => 5000, - type => worker, - modules => [Mod] + WorkerSup = #{ + id => emqx_resource_worker_sup, + start => {emqx_resource_worker_sup, start_link, []}, + restart => permanent, + shutdown => infinity, + type => supervisor }, - case supervisor:start_child(emqx_resource_sup, Spec) of - {ok, _Pid} -> ok; - {error, {already_started, _}} -> ok; - {error, already_present} -> ok; - {error, _} = Err -> Err - end. - -ensure_worker_stopped(ResId, Idx) -> - ChildId = ?CHILD_ID(emqx_resource_worker, ResId, Idx), - case supervisor:terminate_child(emqx_resource_sup, ChildId) of - ok -> - supervisor:delete_child(emqx_resource_sup, ChildId); - {error, not_found} -> - ok; - {error, Reason} -> - {error, Reason} - end. + {ok, {SupFlags, [Metrics, ResourceManager, WorkerSup]}}. diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index 7acf2d0f9..ae0d24313 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -44,6 +44,8 @@ -export([running/3, blocked/3]). +-export([queue_item_marshaller/1]). + -define(RESUME_INTERVAL, 15000). %% count @@ -51,11 +53,15 @@ %% milliseconds -define(DEFAULT_BATCH_TIME, 10). +-define(Q_ITEM(REQUEST), {q_item, REQUEST}). + -define(QUERY(FROM, REQUEST), {FROM, REQUEST}). -define(REPLY(FROM, REQUEST, RESULT), {FROM, REQUEST, RESULT}). -define(EXPAND(RESULT, BATCH), [?REPLY(FROM, REQUEST, RESULT) || ?QUERY(FROM, REQUEST) <- BATCH]). --define(RESOURCE_ERROR(Reason, Msg), {error, {resource_error, #{reason => Reason, msg => Msg}}}). +-define(RESOURCE_ERROR(Reason, Msg), + {error, {resource_error, #{reason => Reason, msg => iolist_to_binary(Msg)}}} +). -define(RESOURCE_ERROR_M(Reason, Msg), {error, {resource_error, #{reason := Reason, msg := Msg}}}). -type id() :: binary(). @@ -72,21 +78,21 @@ callback_mode() -> [state_functions, state_enter]. start_link(Id, Index, Opts) -> gen_statem:start_link({local, name(Id, Index)}, ?MODULE, {Id, Index, Opts}, []). --spec query(id(), request()) -> ok. +-spec query(id(), request()) -> Result :: term(). query(Id, Request) -> - gen_statem:call(pick(Id, self()), {query, Request}). + query(Id, self(), Request). --spec query(id(), term(), request()) -> ok. -query(Id, Key, Request) -> - gen_statem:call(pick(Id, Key), {query, Request}). +-spec query(id(), term(), request()) -> Result :: term(). +query(Id, PickKey, Request) -> + pick_query(call, Id, PickKey, {query, Request}). --spec query_async(id(), request(), reply_fun()) -> ok. +-spec query_async(id(), request(), reply_fun()) -> Result :: term(). query_async(Id, Request, ReplyFun) -> - gen_statem:cast(pick(Id, self()), {query, Request, ReplyFun}). + query_async(Id, self(), Request, ReplyFun). --spec query_async(id(), term(), request(), reply_fun()) -> ok. -query_async(Id, Key, Request, ReplyFun) -> - gen_statem:cast(pick(Id, Key), {query, Request, ReplyFun}). +-spec query_async(id(), term(), request(), reply_fun()) -> Result :: term(). +query_async(Id, PickKey, Request, ReplyFun) -> + pick_query(cast, Id, PickKey, {query, Request, ReplyFun}). -spec block(pid() | atom()) -> ok. block(ServerRef) -> @@ -97,17 +103,24 @@ resume(ServerRef) -> gen_statem:cast(ServerRef, resume). init({Id, Index, Opts}) -> + process_flag(trap_exit, true), true = gproc_pool:connect_worker(Id, {Id, Index}), BatchSize = maps:get(batch_size, Opts, ?DEFAULT_BATCH_SIZE), Queue = case maps:get(queue_enabled, Opts, true) of - true -> replayq:open(#{dir => disk_queue_dir(Id, Index), seg_bytes => 10000000}); - false -> undefined + true -> + replayq:open(#{ + dir => disk_queue_dir(Id, Index), + seg_bytes => 10000000, + marshaller => fun ?MODULE:queue_item_marshaller/1 + }); + false -> + undefined end, St = #{ id => Id, index => Index, - batch_enabled => maps:get(batch_enabled, Opts, true), + batch_enabled => maps:get(batch_enabled, Opts, false), batch_size => BatchSize, batch_time => maps:get(batch_time, Opts, ?DEFAULT_BATCH_TIME), queue => Queue, @@ -128,7 +141,7 @@ running(cast, {query, Request, ReplyFun}, St) -> running({call, From}, {query, Request}, St) -> query_or_acc(From, Request, St); running(info, {flush, Ref}, St = #{tref := {_TRef, Ref}}) -> - {keep_state, flush(St#{tref := undefined})}; + flush(St#{tref := undefined}); running(info, {flush, _Ref}, _St) -> keep_state_and_data; running(info, Info, _St) -> @@ -154,12 +167,21 @@ terminate(_Reason, #{id := Id, index := Index}) -> code_change(_OldVsn, State, _Extra) -> {ok, State}. +queue_item_marshaller(?Q_ITEM(_) = I) -> + term_to_binary(I); +queue_item_marshaller(Bin) when is_binary(Bin) -> + binary_to_term(Bin). + %%============================================================================== -pick(Id, Key) -> - Pid = gproc_pool:pick_worker(Id, Key), - case is_pid(Pid) of - true -> Pid; - false -> error({failed_to_pick_worker, {Id, Key}}) +pick_query(Fun, Id, Key, Query) -> + try gproc_pool:pick_worker(Id, Key) of + Pid when is_pid(Pid) -> + gen_statem:Fun(Pid, Query); + _ -> + ?RESOURCE_ERROR(not_created, "resource not found") + catch + error:badarg -> + ?RESOURCE_ERROR(not_created, "resource not found") end. do_resume(#{queue := undefined} = St) -> @@ -168,9 +190,9 @@ do_resume(#{queue := Q, id := Id} = St) -> case replayq:peek(Q) of empty -> {next_state, running, St}; - First -> + ?Q_ITEM(First) -> Result = call_query(Id, First), - case handle_query_result(Id, false, Result) of + case handle_query_result(Id, Result, false) of %% Send failed because resource down true -> {keep_state, St, {state_timeout, ?RESUME_INTERVAL, resume}}; @@ -182,6 +204,11 @@ do_resume(#{queue := Q, id := Id} = St) -> end end. +handle_blocked(From, Request, #{id := Id, queue := Q} = St) -> + Error = ?RESOURCE_ERROR(blocked, "resource is blocked"), + _ = reply_caller(Id, ?REPLY(From, Request, Error), false), + {keep_state, St#{queue := maybe_append_queue(Q, [?Q_ITEM(Request)])}}. + drop_head(Q) -> {Q1, AckRef, _} = replayq:pop(Q, #{count_limit => 1}), ok = replayq:ack(Q1, AckRef), @@ -196,26 +223,21 @@ acc_query(From, Request, #{acc := Acc, acc_left := Left} = St0) -> Acc1 = [?QUERY(From, Request) | Acc], St = St0#{acc := Acc1, acc_left := Left - 1}, case Left =< 1 of - true -> {keep_state, flush(St)}; + true -> flush(St); false -> {keep_state, ensure_flush_timer(St)} end. send_query(From, Request, #{id := Id, queue := Q} = St) -> Result = call_query(Id, Request), - case reply_caller(Id, Q, ?REPLY(From, Request, Result)) of + case reply_caller(Id, ?REPLY(From, Request, Result), false) of true -> - {keep_state, St#{queue := maybe_append_queue(Q, [Request])}}; + {next_state, blocked, St#{queue := maybe_append_queue(Q, [?Q_ITEM(Request)])}}; false -> - {next_state, blocked, St} + {keep_state, St} end. -handle_blocked(From, Request, #{id := Id, queue := Q} = St) -> - Error = ?RESOURCE_ERROR(blocked, "resource is blocked"), - _ = reply_caller(Id, Q, ?REPLY(From, Request, Error)), - {keep_state, St#{queue := maybe_append_queue(Q, [Request])}}. - flush(#{acc := []} = St) -> - St; + {keep_state, St}; flush( #{ id := Id, @@ -228,65 +250,65 @@ flush( St1 = cancel_flush_timer(St#{acc_left := Size, acc := []}), case batch_reply_caller(Id, BatchResults) of true -> - Q1 = maybe_append_queue(Q0, [Request || ?QUERY(_, Request) <- Batch]), - {keep_state, St1#{queue := Q1}}; + Q1 = maybe_append_queue(Q0, [?Q_ITEM(Request) || ?QUERY(_, Request) <- Batch]), + {next_state, blocked, St1#{queue := Q1}}; false -> - {next_state, blocked, St1} + {keep_state, St1} end. -maybe_append_queue(undefined, _Query) -> undefined; -maybe_append_queue(Q, Query) -> replayq:append(Q, Query). +maybe_append_queue(undefined, _Request) -> undefined; +maybe_append_queue(Q, Request) -> replayq:append(Q, Request). batch_reply_caller(Id, BatchResults) -> lists:foldl( fun(Reply, BlockWorker) -> - reply_caller(Id, BlockWorker, Reply) + reply_caller(Id, Reply, BlockWorker) end, false, BatchResults ). -reply_caller(Id, BlockWorker, ?REPLY(undefined, _, Result)) -> - handle_query_result(Id, BlockWorker, Result); -reply_caller(Id, BlockWorker, ?REPLY({ReplyFun, Args}, _, Result)) -> +reply_caller(Id, ?REPLY(undefined, _, Result), BlockWorker) -> + handle_query_result(Id, Result, BlockWorker); +reply_caller(Id, ?REPLY({ReplyFun, Args}, _, Result), BlockWorker) when is_function(ReplyFun) -> ?SAFE_CALL(ReplyFun(Result, Args)), - handle_query_result(Id, BlockWorker, Result); -reply_caller(Id, BlockWorker, ?REPLY(From, _, Result)) -> + handle_query_result(Id, Result, BlockWorker); +reply_caller(Id, ?REPLY(From, _, Result), BlockWorker) -> gen_statem:reply(From, Result), - handle_query_result(Id, BlockWorker, Result). + handle_query_result(Id, Result, BlockWorker). -handle_query_result(Id, BlockWorker, ok) -> +handle_query_result(Id, ok, BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, success), BlockWorker; -handle_query_result(Id, BlockWorker, {ok, _}) -> +handle_query_result(Id, {ok, _}, BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, success), BlockWorker; -handle_query_result(Id, BlockWorker, ?RESOURCE_ERROR_M(exception, _)) -> +handle_query_result(Id, ?RESOURCE_ERROR_M(exception, _), BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, exception), BlockWorker; -handle_query_result(_Id, _, ?RESOURCE_ERROR_M(NotWorking, _)) when +handle_query_result(_Id, ?RESOURCE_ERROR_M(NotWorking, _), _) when NotWorking == not_connected; NotWorking == blocked -> true; -handle_query_result(_Id, BlockWorker, ?RESOURCE_ERROR_M(_, _)) -> +handle_query_result(_Id, ?RESOURCE_ERROR_M(_, _), BlockWorker) -> BlockWorker; -handle_query_result(Id, BlockWorker, {error, _}) -> +handle_query_result(Id, {error, _}, BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, failed), BlockWorker; -handle_query_result(Id, _BlockWorker, {resource_down, _}) -> +handle_query_result(Id, {resource_down, _}, _BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, resource_down), true. call_query(Id, Request) -> - do_call_query(on_query, Id, Request). + do_call_query(on_query, Id, Request, 1). call_batch_query(Id, Batch) -> - do_call_query(on_batch_query, Id, Batch). + do_call_query(on_batch_query, Id, Batch, length(Batch)). -do_call_query(Fun, Id, Data) -> +do_call_query(Fun, Id, Data, Count) -> case emqx_resource_manager:ets_lookup(Id) of {ok, _Group, #{mod := Mod, state := ResourceState, status := connected}} -> - ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched, length(Data)), + ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched, Count), try Mod:Fun(Id, Data, ResourceState) of %% if the callback module (connector) wants to return an error that %% makes the current resource goes into the `error` state, it should diff --git a/apps/emqx_resource/src/emqx_resource_worker_sup.erl b/apps/emqx_resource/src/emqx_resource_worker_sup.erl new file mode 100644 index 000000000..a2b3a1ba5 --- /dev/null +++ b/apps/emqx_resource/src/emqx_resource_worker_sup.erl @@ -0,0 +1,136 @@ +%%-------------------------------------------------------------------- +%% Copyright (c) 2020-2022 EMQ Technologies Co., Ltd. All Rights Reserved. +%% +%% Licensed under the Apache License, Version 2.0 (the "License"); +%% you may not use this file except in compliance with the License. +%% You may obtain a copy of the License at +%% +%% http://www.apache.org/licenses/LICENSE-2.0 +%% +%% Unless required by applicable law or agreed to in writing, software +%% distributed under the License is distributed on an "AS IS" BASIS, +%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +%% See the License for the specific language governing permissions and +%% limitations under the License. +%%-------------------------------------------------------------------- +-module(emqx_resource_worker_sup). +-behaviour(supervisor). + +%%%============================================================================= +%%% Exports and Definitions +%%%============================================================================= + +%% External API +-export([start_link/0]). + +-export([start_workers/2, stop_workers/2]). + +%% Callbacks +-export([init/1]). + +-define(SERVER, ?MODULE). + +%%%============================================================================= +%%% API +%%%============================================================================= + +-spec start_link() -> supervisor:startlink_ret(). +start_link() -> + supervisor:start_link({local, ?SERVER}, ?MODULE, []). + +%%%============================================================================= +%%% Callbacks +%%%============================================================================= + +-spec init(list()) -> {ok, {supervisor:sup_flags(), [supervisor:child_spec()]}} | ignore. +init([]) -> + SupFlags = #{ + strategy => one_for_one, + intensity => 100, + period => 30 + }, + ChildSpecs = [], + {ok, {SupFlags, ChildSpecs}}. + +start_workers(ResId, Opts) -> + PoolSize = pool_size(Opts), + _ = ensure_worker_pool(ResId, hash, [{size, PoolSize}]), + lists:foreach( + fun(Idx) -> + _ = ensure_worker_added(ResId, Idx), + ok = ensure_worker_started(ResId, Idx, Opts) + end, + lists:seq(1, PoolSize) + ). + +stop_workers(ResId, Opts) -> + PoolSize = pool_size(Opts), + lists:foreach( + fun(Idx) -> + ensure_worker_removed(ResId, Idx) + end, + lists:seq(1, PoolSize) + ), + ensure_worker_pool_removed(ResId), + ok. + +%%%============================================================================= +%%% Internal +%%%============================================================================= +pool_size(Opts) -> + maps:get(worker_pool_size, Opts, erlang:system_info(schedulers_online)). + +ensure_worker_pool(ResId, Type, Opts) -> + try + gproc_pool:new(ResId, Type, Opts) + catch + error:exists -> ok + end, + ok. + +ensure_worker_added(ResId, Idx) -> + try + gproc_pool:add_worker(ResId, {ResId, Idx}, Idx) + catch + error:exists -> ok + end, + ok. + +-define(CHILD_ID(MOD, RESID, INDEX), {MOD, RESID, INDEX}). +ensure_worker_started(ResId, Idx, Opts) -> + Mod = emqx_resource_worker, + Spec = #{ + id => ?CHILD_ID(Mod, ResId, Idx), + start => {Mod, start_link, [ResId, Idx, Opts]}, + restart => transient, + shutdown => 5000, + type => worker, + modules => [Mod] + }, + case supervisor:start_child(emqx_resource_sup, Spec) of + {ok, _Pid} -> ok; + {error, {already_started, _}} -> ok; + {error, already_present} -> ok; + {error, _} = Err -> Err + end. + +ensure_worker_removed(ResId, Idx) -> + ChildId = ?CHILD_ID(emqx_resource_worker, ResId, Idx), + case supervisor:terminate_child(emqx_resource_sup, ChildId) of + ok -> + Res = supervisor:delete_child(emqx_resource_sup, ChildId), + _ = gproc_pool:remove_worker(ResId, {ResId, Idx}), + Res; + {error, not_found} -> + ok; + {error, Reason} -> + {error, Reason} + end. + +ensure_worker_pool_removed(ResId) -> + try + gproc_pool:delete(ResId) + catch + error:badarg -> ok + end, + ok. diff --git a/apps/emqx_resource/test/emqx_resource_SUITE.erl b/apps/emqx_resource/test/emqx_resource_SUITE.erl index 51e6bac43..915c59611 100644 --- a/apps/emqx_resource/test/emqx_resource_SUITE.erl +++ b/apps/emqx_resource/test/emqx_resource_SUITE.erl @@ -26,6 +26,7 @@ -define(TEST_RESOURCE, emqx_test_resource). -define(ID, <<"id">>). -define(DEFAULT_RESOURCE_GROUP, <<"default">>). +-define(RESOURCE_ERROR(REASON), {error, {resource_error, #{reason := REASON}}}). all() -> emqx_common_test_helpers:all(?MODULE). @@ -80,7 +81,7 @@ t_create_remove(_) -> #{name => test_resource}, #{} ), - #{pid := Pid} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid)), @@ -110,7 +111,7 @@ t_create_remove_local(_) -> #{name => test_resource}, #{} ), - #{pid := Pid} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid)), @@ -127,7 +128,7 @@ t_create_remove_local(_) -> {error, _} = emqx_resource:remove_local(?ID), ?assertMatch( - {error, {emqx_resource, #{reason := not_found}}}, + ?RESOURCE_ERROR(not_created), emqx_resource:query(?ID, get_state) ), ?assertNot(is_process_alive(Pid)). @@ -143,23 +144,23 @@ t_do_not_start_after_created(_) -> %% the resource should remain `disconnected` after created timer:sleep(200), ?assertMatch( - {error, {emqx_resource, #{reason := not_connected}}}, + ?RESOURCE_ERROR(stopped), emqx_resource:query(?ID, get_state) ), ?assertMatch( - {ok, _, #{status := disconnected}}, + {ok, _, #{status := stopped}}, emqx_resource:get_instance(?ID) ), %% start the resource manually.. ok = emqx_resource:start(?ID), - #{pid := Pid} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid)), %% restart the resource ok = emqx_resource:restart(?ID), ?assertNot(is_process_alive(Pid)), - #{pid := Pid2} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid2}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid2)), ok = emqx_resource:remove_local(?ID), @@ -174,23 +175,10 @@ t_query(_) -> #{name => test_resource} ), - Pid = self(), - Success = fun() -> Pid ! success end, - Failure = fun() -> Pid ! failure end, - - #{pid := _} = emqx_resource:query(?ID, get_state), - #{pid := _} = emqx_resource:query(?ID, get_state, {[{Success, []}], [{Failure, []}]}), - #{pid := _} = emqx_resource:query(?ID, get_state, undefined), - #{pid := _} = emqx_resource:query(?ID, get_state_failed, undefined), - - receive - Message -> ?assertEqual(success, Message) - after 100 -> - ?assert(false) - end, + {ok, #{pid := _}} = emqx_resource:query(?ID, get_state), ?assertMatch( - {error, {emqx_resource, #{reason := not_found}}}, + ?RESOURCE_ERROR(not_created), emqx_resource:query(<<"unknown">>, get_state) ), @@ -201,11 +189,14 @@ t_healthy_timeout(_) -> ?ID, ?DEFAULT_RESOURCE_GROUP, ?TEST_RESOURCE, - #{name => <<"test_resource">>}, - #{health_check_timeout => 200} + #{name => <<"bad_not_atom_name">>, register => true}, + %% the ?TEST_RESOURCE always returns the `Mod:on_get_status/2` 300ms later. + #{health_check_interval => 200} + ), + ?assertMatch( + ?RESOURCE_ERROR(not_connected), + emqx_resource:query(?ID, get_state) ), - timer:sleep(500), - ok = emqx_resource:remove_local(?ID). t_healthy(_) -> @@ -213,11 +204,9 @@ t_healthy(_) -> ?ID, ?DEFAULT_RESOURCE_GROUP, ?TEST_RESOURCE, - #{name => <<"test_resource">>} + #{name => test_resource} ), - timer:sleep(400), - - #{pid := Pid} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid}} = emqx_resource:query(?ID, get_state), timer:sleep(300), emqx_resource:set_resource_status_connecting(?ID), @@ -229,10 +218,10 @@ t_healthy(_) -> erlang:exit(Pid, shutdown), - ?assertEqual({ok, connecting}, emqx_resource:health_check(?ID)), + ?assertEqual({ok, disconnected}, emqx_resource:health_check(?ID)), ?assertMatch( - [#{status := connecting}], + [#{status := disconnected}], emqx_resource:list_instances_verbose() ), @@ -260,7 +249,7 @@ t_stop_start(_) -> #{} ), - #{pid := Pid0} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid0}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid0)), @@ -269,14 +258,14 @@ t_stop_start(_) -> ?assertNot(is_process_alive(Pid0)), ?assertMatch( - {error, {emqx_resource, #{reason := not_connected}}}, + ?RESOURCE_ERROR(stopped), emqx_resource:query(?ID, get_state) ), ok = emqx_resource:restart(?ID), timer:sleep(300), - #{pid := Pid1} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid1}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid1)). @@ -302,7 +291,7 @@ t_stop_start_local(_) -> #{} ), - #{pid := Pid0} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid0}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid0)), @@ -311,13 +300,13 @@ t_stop_start_local(_) -> ?assertNot(is_process_alive(Pid0)), ?assertMatch( - {error, {emqx_resource, #{reason := not_connected}}}, + ?RESOURCE_ERROR(stopped), emqx_resource:query(?ID, get_state) ), ok = emqx_resource:restart(?ID), - #{pid := Pid1} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid1}} = emqx_resource:query(?ID, get_state), ?assert(is_process_alive(Pid1)). @@ -368,17 +357,17 @@ create_dry_run_local_succ() -> ?assertEqual(undefined, whereis(test_resource)). t_create_dry_run_local_failed(_) -> - {Res1, _} = emqx_resource:create_dry_run_local( + Res1 = emqx_resource:create_dry_run_local( ?TEST_RESOURCE, - #{cteate_error => true} + #{create_error => true} ), - ?assertEqual(error, Res1), + ?assertMatch({error, _}, Res1), - {Res2, _} = emqx_resource:create_dry_run_local( + Res2 = emqx_resource:create_dry_run_local( ?TEST_RESOURCE, #{name => test_resource, health_check_error => true} ), - ?assertEqual(error, Res2), + ?assertMatch({error, _}, Res2), Res3 = emqx_resource:create_dry_run_local( ?TEST_RESOURCE, @@ -400,7 +389,7 @@ t_reset_metrics(_) -> #{name => test_resource} ), - #{pid := Pid} = emqx_resource:query(?ID, get_state), + {ok, #{pid := Pid}} = emqx_resource:query(?ID, get_state), emqx_resource:reset_metrics(?ID), ?assert(is_process_alive(Pid)), ok = emqx_resource:remove(?ID), diff --git a/apps/emqx_resource/test/emqx_test_resource.erl b/apps/emqx_resource/test/emqx_test_resource.erl index c23f87d50..569579d27 100644 --- a/apps/emqx_resource/test/emqx_test_resource.erl +++ b/apps/emqx_resource/test/emqx_test_resource.erl @@ -24,7 +24,7 @@ -export([ on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -50,24 +50,20 @@ on_start(_InstId, #{create_error := true}) -> error("some error"); on_start(InstId, #{name := Name, stop_error := true} = Opts) -> Register = maps:get(register, Opts, false), - {ok, #{ - name => Name, + {ok, Opts#{ id => InstId, stop_error => true, pid => spawn_dummy_process(Name, Register) }}; -on_start(InstId, #{name := Name, health_check_error := true} = Opts) -> +on_start(InstId, #{name := Name} = Opts) -> Register = maps:get(register, Opts, false), - {ok, #{ - name => Name, + {ok, Opts#{ id => InstId, - health_check_error => true, pid => spawn_dummy_process(Name, Register) }}; on_start(InstId, #{name := Name} = Opts) -> Register = maps:get(register, Opts, false), - {ok, #{ - name => Name, + {ok, Opts#{ id => InstId, pid => spawn_dummy_process(Name, Register) }}. @@ -78,12 +74,10 @@ on_stop(_InstId, #{pid := Pid}) -> erlang:exit(Pid, shutdown), ok. -on_query(_InstId, get_state, AfterQuery, State) -> - emqx_resource:query_success(AfterQuery), - State; -on_query(_InstId, get_state_failed, AfterQuery, State) -> - emqx_resource:query_failed(AfterQuery), - State. +on_query(_InstId, get_state, State) -> + {ok, State}; +on_query(_InstId, get_state_failed, State) -> + {error, State}. on_get_status(_InstId, #{health_check_error := true}) -> disconnected; @@ -91,10 +85,11 @@ on_get_status(_InstId, #{pid := Pid}) -> timer:sleep(300), case is_process_alive(Pid) of true -> connected; - false -> connecting + false -> disconnected end. spawn_dummy_process(Name, Register) -> + ct:pal("---- Register Name: ~p", [Name]), spawn( fun() -> true = From 0377d3cf61469aa51cbe367bdfb5dd1a2768e518 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Fri, 29 Jul 2022 18:49:57 +0800 Subject: [PATCH 05/14] fix: update existing testcases for new emqx_resource --- .../src/simple_authn/emqx_authn_mongodb.erl | 2 +- apps/emqx_authz/src/emqx_authz_mongodb.erl | 4 +- .../test/emqx_bridge_api_SUITE.erl | 116 ++++++++++-------- .../src/emqx_connector_mqtt.erl | 5 +- .../test/emqx_connector_mongo_SUITE.erl | 8 +- apps/emqx_resource/src/emqx_resource.app.src | 2 +- apps/emqx_resource/src/emqx_resource.erl | 5 +- .../src/emqx_resource_worker.erl | 26 ++-- 8 files changed, 98 insertions(+), 70 deletions(-) diff --git a/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl b/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl index ff9c97717..1351ae0dd 100644 --- a/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl +++ b/apps/emqx_authn/src/simple_authn/emqx_authn_mongodb.erl @@ -164,7 +164,7 @@ authenticate( ) -> Filter = emqx_authn_utils:render_deep(FilterTemplate, Credential), case emqx_resource:query(ResourceId, {find_one, Collection, Filter, #{}}) of - undefined -> + {ok, undefined} -> ignore; {error, Reason} -> ?TRACE_AUTHN_PROVIDER(error, "mongodb_query_failed", #{ diff --git a/apps/emqx_authz/src/emqx_authz_mongodb.erl b/apps/emqx_authz/src/emqx_authz_mongodb.erl index ac450e4cc..a1e1b8136 100644 --- a/apps/emqx_authz/src/emqx_authz_mongodb.erl +++ b/apps/emqx_authz/src/emqx_authz_mongodb.erl @@ -92,9 +92,9 @@ authorize( resource_id => ResourceID }), nomatch; - [] -> + {ok, []} -> nomatch; - Rows -> + {ok, Rows} -> Rules = [ emqx_authz_rule:compile({Permission, all, Action, Topics}) || #{ diff --git a/apps/emqx_bridge/test/emqx_bridge_api_SUITE.erl b/apps/emqx_bridge/test/emqx_bridge_api_SUITE.erl index c048a13fe..9346fb9c0 100644 --- a/apps/emqx_bridge/test/emqx_bridge_api_SUITE.erl +++ b/apps/emqx_bridge/test/emqx_bridge_api_SUITE.erl @@ -24,7 +24,7 @@ -include_lib("common_test/include/ct.hrl"). -define(CONF_DEFAULT, <<"bridges: {}">>). -define(BRIDGE_TYPE, <<"webhook">>). --define(BRIDGE_NAME, <<"test_bridge">>). +-define(BRIDGE_NAME, (atom_to_binary(?FUNCTION_NAME))). -define(URL(PORT, PATH), list_to_binary( io_lib:format( @@ -78,8 +78,12 @@ set_special_configs(_) -> init_per_testcase(_, Config) -> {ok, _} = emqx_cluster_rpc:start_link(node(), emqx_cluster_rpc, 1000), - Config. -end_per_testcase(_, _Config) -> + {Port, Sock, Acceptor} = start_http_server(fun handle_fun_200_ok/2), + [{port, Port}, {sock, Sock}, {acceptor, Acceptor} | Config]. +end_per_testcase(_, Config) -> + Sock = ?config(sock, Config), + Acceptor = ?config(acceptor, Config), + stop_http_server(Sock, Acceptor), clear_resources(), ok. @@ -95,31 +99,39 @@ clear_resources() -> %% HTTP server for testing %%------------------------------------------------------------------------------ start_http_server(HandleFun) -> + process_flag(trap_exit, true), Parent = self(), - spawn_link(fun() -> - {Port, Sock} = listen_on_random_port(), - Parent ! {port, Port}, - loop(Sock, HandleFun, Parent) + {Port, Sock} = listen_on_random_port(), + Acceptor = spawn_link(fun() -> + accept_loop(Sock, HandleFun, Parent) end), - receive - {port, Port} -> Port - after 2000 -> error({timeout, start_http_server}) - end. + timer:sleep(100), + {Port, Sock, Acceptor}. + +stop_http_server(Sock, Acceptor) -> + exit(Acceptor, kill), + gen_tcp:close(Sock). listen_on_random_port() -> Min = 1024, Max = 65000, + rand:seed(exsplus, erlang:timestamp()), Port = rand:uniform(Max - Min) + Min, - case gen_tcp:listen(Port, [{active, false}, {reuseaddr, true}, binary]) of + case + gen_tcp:listen(Port, [ + binary, {active, false}, {packet, raw}, {reuseaddr, true}, {backlog, 1000} + ]) + of {ok, Sock} -> {Port, Sock}; {error, eaddrinuse} -> listen_on_random_port() end. -loop(Sock, HandleFun, Parent) -> +accept_loop(Sock, HandleFun, Parent) -> + process_flag(trap_exit, true), {ok, Conn} = gen_tcp:accept(Sock), - Handler = spawn(fun() -> HandleFun(Conn, Parent) end), + Handler = spawn_link(fun() -> HandleFun(Conn, Parent) end), gen_tcp:controlling_process(Conn, Handler), - loop(Sock, HandleFun, Parent). + accept_loop(Sock, HandleFun, Parent). make_response(CodeStr, Str) -> B = iolist_to_binary(Str), @@ -138,7 +150,9 @@ handle_fun_200_ok(Conn, Parent) -> Parent ! {http_server, received, Req}, gen_tcp:send(Conn, make_response("200 OK", "Request OK")), handle_fun_200_ok(Conn, Parent); - {error, closed} -> + {error, Reason} -> + ct:pal("the http handler recv error: ~p", [Reason]), + timer:sleep(100), gen_tcp:close(Conn) end. @@ -153,24 +167,25 @@ parse_http_request(ReqStr0) -> %% Testcases %%------------------------------------------------------------------------------ -t_http_crud_apis(_) -> - Port = start_http_server(fun handle_fun_200_ok/2), +t_http_crud_apis(Config) -> + Port = ?config(port, Config), %% assert we there's no bridges at first {ok, 200, <<"[]">>} = request(get, uri(["bridges"]), []), %% then we add a webhook bridge, using POST %% POST /bridges/ will create a bridge URL1 = ?URL(Port, "path1"), + Name = ?BRIDGE_NAME, {ok, 201, Bridge} = request( post, uri(["bridges"]), - ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, ?BRIDGE_NAME) + ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, Name) ), %ct:pal("---bridge: ~p", [Bridge]), #{ <<"type">> := ?BRIDGE_TYPE, - <<"name">> := ?BRIDGE_NAME, + <<"name">> := Name, <<"enable">> := true, <<"status">> := _, <<"node_status">> := [_ | _], @@ -179,7 +194,7 @@ t_http_crud_apis(_) -> <<"url">> := URL1 } = jsx:decode(Bridge), - BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, ?BRIDGE_NAME), + BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, Name), %% send an message to emqx and the message should be forwarded to the HTTP server Body = <<"my msg">>, emqx:publish(emqx_message:make(<<"emqx_webhook/1">>, Body)), @@ -203,12 +218,12 @@ t_http_crud_apis(_) -> {ok, 200, Bridge2} = request( put, uri(["bridges", BridgeID]), - ?HTTP_BRIDGE(URL2, ?BRIDGE_TYPE, ?BRIDGE_NAME) + ?HTTP_BRIDGE(URL2, ?BRIDGE_TYPE, Name) ), ?assertMatch( #{ <<"type">> := ?BRIDGE_TYPE, - <<"name">> := ?BRIDGE_NAME, + <<"name">> := Name, <<"enable">> := true, <<"status">> := _, <<"node_status">> := [_ | _], @@ -225,7 +240,7 @@ t_http_crud_apis(_) -> [ #{ <<"type">> := ?BRIDGE_TYPE, - <<"name">> := ?BRIDGE_NAME, + <<"name">> := Name, <<"enable">> := true, <<"status">> := _, <<"node_status">> := [_ | _], @@ -242,7 +257,7 @@ t_http_crud_apis(_) -> ?assertMatch( #{ <<"type">> := ?BRIDGE_TYPE, - <<"name">> := ?BRIDGE_NAME, + <<"name">> := Name, <<"enable">> := true, <<"status">> := _, <<"node_status">> := [_ | _], @@ -275,7 +290,7 @@ t_http_crud_apis(_) -> {ok, 404, ErrMsg2} = request( put, uri(["bridges", BridgeID]), - ?HTTP_BRIDGE(URL2, ?BRIDGE_TYPE, ?BRIDGE_NAME) + ?HTTP_BRIDGE(URL2, ?BRIDGE_TYPE, Name) ), ?assertMatch( #{ @@ -286,29 +301,28 @@ t_http_crud_apis(_) -> ), ok. -t_start_stop_bridges(_) -> - lists:foreach( - fun(Type) -> - do_start_stop_bridges(Type) - end, - [node, cluster] - ). +t_start_stop_bridges_node(Config) -> + do_start_stop_bridges(node, Config). -do_start_stop_bridges(Type) -> +t_start_stop_bridges_cluster(Config) -> + do_start_stop_bridges(cluster, Config). + +do_start_stop_bridges(Type, Config) -> %% assert we there's no bridges at first {ok, 200, <<"[]">>} = request(get, uri(["bridges"]), []), - Port = start_http_server(fun handle_fun_200_ok/2), + Port = ?config(port, Config), URL1 = ?URL(Port, "abc"), + Name = atom_to_binary(Type), {ok, 201, Bridge} = request( post, uri(["bridges"]), - ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, ?BRIDGE_NAME) + ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, Name) ), %ct:pal("the bridge ==== ~p", [Bridge]), #{ <<"type">> := ?BRIDGE_TYPE, - <<"name">> := ?BRIDGE_NAME, + <<"name">> := Name, <<"enable">> := true, <<"status">> := <<"connected">>, <<"node_status">> := [_ | _], @@ -316,11 +330,11 @@ do_start_stop_bridges(Type) -> <<"node_metrics">> := [_ | _], <<"url">> := URL1 } = jsx:decode(Bridge), - BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, ?BRIDGE_NAME), + BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, Name), %% stop it {ok, 200, <<>>} = request(post, operation_path(Type, stop, BridgeID), <<"">>), {ok, 200, Bridge2} = request(get, uri(["bridges", BridgeID]), []), - ?assertMatch(#{<<"status">> := <<"disconnected">>}, jsx:decode(Bridge2)), + ?assertMatch(#{<<"status">> := <<"stopped">>}, jsx:decode(Bridge2)), %% start again {ok, 200, <<>>} = request(post, operation_path(Type, restart, BridgeID), <<"">>), {ok, 200, Bridge3} = request(get, uri(["bridges", BridgeID]), []), @@ -339,21 +353,22 @@ do_start_stop_bridges(Type) -> {ok, 204, <<>>} = request(delete, uri(["bridges", BridgeID]), []), {ok, 200, <<"[]">>} = request(get, uri(["bridges"]), []). -t_enable_disable_bridges(_) -> +t_enable_disable_bridges(Config) -> %% assert we there's no bridges at first {ok, 200, <<"[]">>} = request(get, uri(["bridges"]), []), - Port = start_http_server(fun handle_fun_200_ok/2), + Name = ?BRIDGE_NAME, + Port = ?config(port, Config), URL1 = ?URL(Port, "abc"), {ok, 201, Bridge} = request( post, uri(["bridges"]), - ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, ?BRIDGE_NAME) + ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, Name) ), %ct:pal("the bridge ==== ~p", [Bridge]), #{ <<"type">> := ?BRIDGE_TYPE, - <<"name">> := ?BRIDGE_NAME, + <<"name">> := Name, <<"enable">> := true, <<"status">> := <<"connected">>, <<"node_status">> := [_ | _], @@ -361,11 +376,11 @@ t_enable_disable_bridges(_) -> <<"node_metrics">> := [_ | _], <<"url">> := URL1 } = jsx:decode(Bridge), - BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, ?BRIDGE_NAME), + BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, Name), %% disable it {ok, 200, <<>>} = request(post, operation_path(cluster, disable, BridgeID), <<"">>), {ok, 200, Bridge2} = request(get, uri(["bridges", BridgeID]), []), - ?assertMatch(#{<<"status">> := <<"disconnected">>}, jsx:decode(Bridge2)), + ?assertMatch(#{<<"status">> := <<"stopped">>}, jsx:decode(Bridge2)), %% enable again {ok, 200, <<>>} = request(post, operation_path(cluster, enable, BridgeID), <<"">>), {ok, 200, Bridge3} = request(get, uri(["bridges", BridgeID]), []), @@ -391,21 +406,22 @@ t_enable_disable_bridges(_) -> {ok, 204, <<>>} = request(delete, uri(["bridges", BridgeID]), []), {ok, 200, <<"[]">>} = request(get, uri(["bridges"]), []). -t_reset_bridges(_) -> +t_reset_bridges(Config) -> %% assert we there's no bridges at first {ok, 200, <<"[]">>} = request(get, uri(["bridges"]), []), - Port = start_http_server(fun handle_fun_200_ok/2), + Name = ?BRIDGE_NAME, + Port = ?config(port, Config), URL1 = ?URL(Port, "abc"), {ok, 201, Bridge} = request( post, uri(["bridges"]), - ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, ?BRIDGE_NAME) + ?HTTP_BRIDGE(URL1, ?BRIDGE_TYPE, Name) ), %ct:pal("the bridge ==== ~p", [Bridge]), #{ <<"type">> := ?BRIDGE_TYPE, - <<"name">> := ?BRIDGE_NAME, + <<"name">> := Name, <<"enable">> := true, <<"status">> := <<"connected">>, <<"node_status">> := [_ | _], @@ -413,7 +429,7 @@ t_reset_bridges(_) -> <<"node_metrics">> := [_ | _], <<"url">> := URL1 } = jsx:decode(Bridge), - BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, ?BRIDGE_NAME), + BridgeID = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE, Name), {ok, 200, <<"Reset success">>} = request(put, uri(["bridges", BridgeID, "reset_metrics"]), []), %% delete the bridge diff --git a/apps/emqx_connector/src/emqx_connector_mqtt.erl b/apps/emqx_connector/src/emqx_connector_mqtt.erl index 98635de3f..0957e3c18 100644 --- a/apps/emqx_connector/src/emqx_connector_mqtt.erl +++ b/apps/emqx_connector/src/emqx_connector_mqtt.erl @@ -134,7 +134,8 @@ drop_bridge(Name) -> %% When use this bridge as a data source, ?MODULE:on_message_received will be called %% if the bridge received msgs from the remote broker. on_message_received(Msg, HookPoint, InstId) -> - _ = emqx_resource:query(InstId, {message_received, Msg}), + emqx_resource:inc_matched(InstId), + emqx_resource:inc_success(InstId), emqx:run_hook(HookPoint, [Msg]). %% =================================================================== @@ -181,8 +182,6 @@ on_stop(_InstId, #{name := InstanceId}) -> }) end. -on_query(_InstId, {message_received, _Msg}, _State) -> - ok; on_query(_InstId, {send_message, Msg}, #{name := InstanceId}) -> ?TRACE("QUERY", "send_msg_to_remote_node", #{message => Msg, connector => InstanceId}), emqx_connector_mqtt_worker:send_to_remote(InstanceId, Msg), diff --git a/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl b/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl index d76b8420a..2ad1f5f8e 100644 --- a/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl +++ b/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl @@ -85,8 +85,8 @@ perform_lifecycle_check(PoolName, InitialConfig) -> emqx_resource:get_instance(PoolName), ?assertEqual({ok, connected}, emqx_resource:health_check(PoolName)), % % Perform query as further check that the resource is working as expected - ?assertMatch([], emqx_resource:query(PoolName, test_query_find())), - ?assertMatch(undefined, emqx_resource:query(PoolName, test_query_find_one())), + ?assertMatch({ok, []}, emqx_resource:query(PoolName, test_query_find())), + ?assertMatch({ok, undefined}, emqx_resource:query(PoolName, test_query_find_one())), ?assertEqual(ok, emqx_resource:stop(PoolName)), % Resource will be listed still, but state will be changed and healthcheck will fail % as the worker no longer exists. @@ -108,8 +108,8 @@ perform_lifecycle_check(PoolName, InitialConfig) -> {ok, ?CONNECTOR_RESOURCE_GROUP, #{status := InitialStatus}} = emqx_resource:get_instance(PoolName), ?assertEqual({ok, connected}, emqx_resource:health_check(PoolName)), - ?assertMatch([], emqx_resource:query(PoolName, test_query_find())), - ?assertMatch(undefined, emqx_resource:query(PoolName, test_query_find_one())), + ?assertMatch({ok, []}, emqx_resource:query(PoolName, test_query_find())), + ?assertMatch({ok, undefined}, emqx_resource:query(PoolName, test_query_find_one())), % Stop and remove the resource in one go. ?assertEqual(ok, emqx_resource:remove_local(PoolName)), ?assertEqual({error, not_found}, ecpool:stop_sup_pool(ReturnedPoolName)), diff --git a/apps/emqx_resource/src/emqx_resource.app.src b/apps/emqx_resource/src/emqx_resource.app.src index 1bfd02323..b688e3c11 100644 --- a/apps/emqx_resource/src/emqx_resource.app.src +++ b/apps/emqx_resource/src/emqx_resource.app.src @@ -1,7 +1,7 @@ %% -*- mode: erlang -*- {application, emqx_resource, [ {description, "Manager for all external resources"}, - {vsn, "0.1.1"}, + {vsn, "0.1.2"}, {registered, []}, {mod, {emqx_resource_app, []}}, {applications, [ diff --git a/apps/emqx_resource/src/emqx_resource.erl b/apps/emqx_resource/src/emqx_resource.erl index 081264315..af047060c 100644 --- a/apps/emqx_resource/src/emqx_resource.erl +++ b/apps/emqx_resource/src/emqx_resource.erl @@ -103,7 +103,7 @@ list_group_instances/1 ]). --export([inc_metrics_funcs/1, inc_success/1, inc_failed/1]). +-export([inc_metrics_funcs/1, inc_matched/1, inc_success/1, inc_failed/1]). -optional_callbacks([ on_query/3, @@ -393,6 +393,9 @@ check_and_do(ResourceType, RawConfig, Do) when is_function(Do) -> %% ================================================================================= +inc_matched(ResId) -> + emqx_metrics_worker:inc(?RES_METRICS, ResId, matched). + inc_success(ResId) -> emqx_metrics_worker:inc(?RES_METRICS, ResId, success). diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index ae0d24313..9ab7fb749 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -44,7 +44,7 @@ -export([running/3, blocked/3]). --export([queue_item_marshaller/1]). +-export([queue_item_marshaller/1, estimate_size/1]). -define(RESUME_INTERVAL, 15000). @@ -112,6 +112,7 @@ init({Id, Index, Opts}) -> replayq:open(#{ dir => disk_queue_dir(Id, Index), seg_bytes => 10000000, + sizer => fun ?MODULE:estimate_size/1, marshaller => fun ?MODULE:queue_item_marshaller/1 }); false -> @@ -172,6 +173,9 @@ queue_item_marshaller(?Q_ITEM(_) = I) -> queue_item_marshaller(Bin) when is_binary(Bin) -> binary_to_term(Bin). +estimate_size(QItem) -> + size(queue_item_marshaller(QItem)). + %%============================================================================== pick_query(Fun, Id, Key, Query) -> try gproc_pool:pick_worker(Id, Key) of @@ -277,12 +281,6 @@ reply_caller(Id, ?REPLY(From, _, Result), BlockWorker) -> gen_statem:reply(From, Result), handle_query_result(Id, Result, BlockWorker). -handle_query_result(Id, ok, BlockWorker) -> - emqx_metrics_worker:inc(?RES_METRICS, Id, success), - BlockWorker; -handle_query_result(Id, {ok, _}, BlockWorker) -> - emqx_metrics_worker:inc(?RES_METRICS, Id, success), - BlockWorker; handle_query_result(Id, ?RESOURCE_ERROR_M(exception, _), BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, exception), BlockWorker; @@ -297,7 +295,12 @@ handle_query_result(Id, {error, _}, BlockWorker) -> BlockWorker; handle_query_result(Id, {resource_down, _}, _BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, resource_down), - true. + true; +handle_query_result(Id, Result, BlockWorker) -> + %% assert + true = is_ok_result(Result), + emqx_metrics_worker:inc(?RES_METRICS, Id, success), + BlockWorker. call_query(Id, Request) -> do_call_query(on_query, Id, Request, 1). @@ -339,6 +342,13 @@ maybe_expand_batch_result(Result, Batch) -> %%============================================================================== +is_ok_result(ok) -> + true; +is_ok_result(R) when is_tuple(R) -> + erlang:element(1, R) == ok; +is_ok_result(_) -> + false. + -spec name(id(), integer()) -> atom(). name(Id, Index) -> Mod = atom_to_list(?MODULE), From d3950b9534ae9e220ad7e6a550f3f549ca4baf19 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Fri, 29 Jul 2022 23:26:54 +0800 Subject: [PATCH 06/14] fix(resource): make option 'queue_enabled' disabled by default --- apps/emqx_authn/src/emqx_authn.app.src | 2 +- apps/emqx_authz/src/emqx_authz.app.src | 2 +- .../emqx_connector/src/emqx_connector.app.src | 2 +- .../src/emqx_resource_worker.erl | 27 +++++++------------ .../test/emqx_resource_SUITE.erl | 4 +++ 5 files changed, 17 insertions(+), 20 deletions(-) diff --git a/apps/emqx_authn/src/emqx_authn.app.src b/apps/emqx_authn/src/emqx_authn.app.src index 8087e822f..ef67b9a14 100644 --- a/apps/emqx_authn/src/emqx_authn.app.src +++ b/apps/emqx_authn/src/emqx_authn.app.src @@ -1,7 +1,7 @@ %% -*- mode: erlang -*- {application, emqx_authn, [ {description, "EMQX Authentication"}, - {vsn, "0.1.3"}, + {vsn, "0.1.4"}, {modules, []}, {registered, [emqx_authn_sup, emqx_authn_registry]}, {applications, [kernel, stdlib, emqx_resource, ehttpc, epgsql, mysql, jose]}, diff --git a/apps/emqx_authz/src/emqx_authz.app.src b/apps/emqx_authz/src/emqx_authz.app.src index ed19b15a8..e40b5e64c 100644 --- a/apps/emqx_authz/src/emqx_authz.app.src +++ b/apps/emqx_authz/src/emqx_authz.app.src @@ -1,7 +1,7 @@ %% -*- mode: erlang -*- {application, emqx_authz, [ {description, "An OTP application"}, - {vsn, "0.1.3"}, + {vsn, "0.1.4"}, {registered, []}, {mod, {emqx_authz_app, []}}, {applications, [ diff --git a/apps/emqx_connector/src/emqx_connector.app.src b/apps/emqx_connector/src/emqx_connector.app.src index 007962da3..cce266966 100644 --- a/apps/emqx_connector/src/emqx_connector.app.src +++ b/apps/emqx_connector/src/emqx_connector.app.src @@ -1,7 +1,7 @@ %% -*- mode: erlang -*- {application, emqx_connector, [ {description, "An OTP application"}, - {vsn, "0.1.2"}, + {vsn, "0.1.3"}, {registered, []}, {mod, {emqx_connector_app, []}}, {applications, [ diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index 9ab7fb749..055fbfc53 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -73,7 +73,7 @@ -callback batcher_flush(Acc :: [{from(), request()}], CbState :: term()) -> {{from(), result()}, NewCbState :: term()}. -callback_mode() -> [state_functions, state_enter]. +callback_mode() -> [state_functions]. start_link(Id, Index, Opts) -> gen_statem:start_link({local, name(Id, Index)}, ?MODULE, {Id, Index, Opts}, []). @@ -107,7 +107,7 @@ init({Id, Index, Opts}) -> true = gproc_pool:connect_worker(Id, {Id, Index}), BatchSize = maps:get(batch_size, Opts, ?DEFAULT_BATCH_SIZE), Queue = - case maps:get(queue_enabled, Opts, true) of + case maps:get(queue_enabled, Opts, false) of true -> replayq:open(#{ dir => disk_queue_dir(Id, Index), @@ -131,8 +131,6 @@ init({Id, Index, Opts}) -> }, {ok, blocked, St, {next_event, cast, resume}}. -running(enter, _, _St) -> - keep_state_and_data; running(cast, resume, _St) -> keep_state_and_data; running(cast, block, St) -> @@ -149,8 +147,6 @@ running(info, Info, _St) -> ?SLOG(error, #{msg => unexpected_msg, info => Info}), keep_state_and_data. -blocked(enter, _, _St) -> - keep_state_and_data; blocked(cast, block, _St) -> keep_state_and_data; blocked(cast, resume, St) -> @@ -218,28 +214,25 @@ drop_head(Q) -> ok = replayq:ack(Q1, AckRef), Q1. -query_or_acc(From, Request, #{batch_enabled := true} = St) -> - acc_query(From, Request, St); -query_or_acc(From, Request, #{batch_enabled := false} = St) -> - send_query(From, Request, St). - -acc_query(From, Request, #{acc := Acc, acc_left := Left} = St0) -> +query_or_acc(From, Request, #{batch_enabled := true, acc := Acc, acc_left := Left} = St0) -> Acc1 = [?QUERY(From, Request) | Acc], St = St0#{acc := Acc1, acc_left := Left - 1}, case Left =< 1 of true -> flush(St); false -> {keep_state, ensure_flush_timer(St)} - end. - -send_query(From, Request, #{id := Id, queue := Q} = St) -> - Result = call_query(Id, Request), - case reply_caller(Id, ?REPLY(From, Request, Result), false) of + end; +query_or_acc(From, Request, #{batch_enabled := false, queue := Q, id := Id} = St) -> + case send_query(From, Request, Id) of true -> {next_state, blocked, St#{queue := maybe_append_queue(Q, [?Q_ITEM(Request)])}}; false -> {keep_state, St} end. +send_query(From, Request, Id) -> + Result = call_query(Id, Request), + reply_caller(Id, ?REPLY(From, Request, Result), false). + flush(#{acc := []} = St) -> {keep_state, St}; flush( diff --git a/apps/emqx_resource/test/emqx_resource_SUITE.erl b/apps/emqx_resource/test/emqx_resource_SUITE.erl index 915c59611..278f556ef 100644 --- a/apps/emqx_resource/test/emqx_resource_SUITE.erl +++ b/apps/emqx_resource/test/emqx_resource_SUITE.erl @@ -347,6 +347,10 @@ t_create_dry_run_local(_) -> [] = ets:match(emqx_resource_manager, {{owner, '$1'}, '_'}). create_dry_run_local_succ() -> + case whereis(test_resource) of + undefined -> ok; + Pid -> exit(Pid, kill) + end, ?assertEqual( ok, emqx_resource:create_dry_run_local( From 75adba07813de06bbf50196943bc5919e6f296c9 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Sat, 30 Jul 2022 16:35:12 +0800 Subject: [PATCH 07/14] fix: increase resource metrics using the resource id --- .../emqx_connector/src/emqx_connector_mqtt.erl | 9 +++++---- .../src/emqx_resource_manager.erl | 18 +++++++++++++----- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/apps/emqx_connector/src/emqx_connector_mqtt.erl b/apps/emqx_connector/src/emqx_connector_mqtt.erl index 0957e3c18..66682caeb 100644 --- a/apps/emqx_connector/src/emqx_connector_mqtt.erl +++ b/apps/emqx_connector/src/emqx_connector_mqtt.erl @@ -133,9 +133,9 @@ drop_bridge(Name) -> %% =================================================================== %% When use this bridge as a data source, ?MODULE:on_message_received will be called %% if the bridge received msgs from the remote broker. -on_message_received(Msg, HookPoint, InstId) -> - emqx_resource:inc_matched(InstId), - emqx_resource:inc_success(InstId), +on_message_received(Msg, HookPoint, ResId) -> + emqx_resource:inc_matched(ResId), + emqx_resource:inc_success(ResId), emqx:run_hook(HookPoint, [Msg]). %% =================================================================== @@ -206,11 +206,12 @@ make_sub_confs(EmptyMap, _) when map_size(EmptyMap) == 0 -> make_sub_confs(undefined, _) -> undefined; make_sub_confs(SubRemoteConf, InstId) -> + ResId = emqx_resource_manager:manager_id_to_resource_id(InstId), case maps:take(hookpoint, SubRemoteConf) of error -> SubRemoteConf; {HookPoint, SubConf} -> - MFA = {?MODULE, on_message_received, [HookPoint, InstId]}, + MFA = {?MODULE, on_message_received, [HookPoint, ResId]}, SubConf#{on_message_received => MFA} end. diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index b5bcbd330..7dad85d5b 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -38,8 +38,12 @@ list_group/1, ets_lookup/1, get_metrics/1, - reset_metrics/1, - set_resource_status_connecting/1 + reset_metrics/1 +]). + +-export([ + set_resource_status_connecting/1, + manager_id_to_resource_id/1 ]). % Server @@ -64,6 +68,13 @@ %% API %%------------------------------------------------------------------------------ +make_manager_id(ResId) -> + emqx_resource:generate_id(ResId). + +manager_id_to_resource_id(MgrId) -> + [ResId, _Index] = string:split(MgrId, ":", trailing), + ResId. + %% @doc Called from emqx_resource when starting a resource instance. %% %% Triggers the emqx_resource_manager_sup supervisor to actually create @@ -455,9 +466,6 @@ stop_resource(Data) -> _ = maybe_clear_alarm(Data#data.id), ok. -make_manager_id(ResId) -> - emqx_resource:generate_id(ResId). - make_test_id() -> RandId = iolist_to_binary(emqx_misc:gen_id(16)), <>. From 8f0954837bdca5037bac40f799c531c8fe3e7a9e Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Sat, 30 Jul 2022 22:42:26 +0800 Subject: [PATCH 08/14] fix: update test cases for emqx_connector --- apps/emqx/src/emqx.app.src | 2 +- apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl | 2 +- apps/emqx_connector/test/emqx_connector_mysql_SUITE.erl | 2 +- apps/emqx_connector/test/emqx_connector_pgsql_SUITE.erl | 2 +- apps/emqx_connector/test/emqx_connector_redis_SUITE.erl | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/apps/emqx/src/emqx.app.src b/apps/emqx/src/emqx.app.src index fed9e6bc2..b7e65a042 100644 --- a/apps/emqx/src/emqx.app.src +++ b/apps/emqx/src/emqx.app.src @@ -3,7 +3,7 @@ {id, "emqx"}, {description, "EMQX Core"}, % strict semver, bump manually! - {vsn, "5.0.4"}, + {vsn, "5.0.5"}, {modules, []}, {registered, []}, {applications, [ diff --git a/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl b/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl index 2ad1f5f8e..e918be84a 100644 --- a/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl +++ b/apps/emqx_connector/test/emqx_connector_mongo_SUITE.erl @@ -95,7 +95,7 @@ perform_lifecycle_check(PoolName, InitialConfig) -> status := StoppedStatus }} = emqx_resource:get_instance(PoolName), - ?assertEqual(StoppedStatus, disconnected), + ?assertEqual(stopped, StoppedStatus), ?assertEqual({error, resource_is_stopped}, emqx_resource:health_check(PoolName)), % Resource healthcheck shortcuts things by checking ets. Go deeper by checking pool itself. ?assertEqual({error, not_found}, ecpool:stop_sup_pool(ReturnedPoolName)), diff --git a/apps/emqx_connector/test/emqx_connector_mysql_SUITE.erl b/apps/emqx_connector/test/emqx_connector_mysql_SUITE.erl index d7f5cec63..3fd7191b9 100644 --- a/apps/emqx_connector/test/emqx_connector_mysql_SUITE.erl +++ b/apps/emqx_connector/test/emqx_connector_mysql_SUITE.erl @@ -101,7 +101,7 @@ perform_lifecycle_check(PoolName, InitialConfig) -> status := StoppedStatus }} = emqx_resource:get_instance(PoolName), - ?assertEqual(StoppedStatus, disconnected), + ?assertEqual(stopped, StoppedStatus), ?assertEqual({error, resource_is_stopped}, emqx_resource:health_check(PoolName)), % Resource healthcheck shortcuts things by checking ets. Go deeper by checking pool itself. ?assertEqual({error, not_found}, ecpool:stop_sup_pool(ReturnedPoolName)), diff --git a/apps/emqx_connector/test/emqx_connector_pgsql_SUITE.erl b/apps/emqx_connector/test/emqx_connector_pgsql_SUITE.erl index d99d8ab6c..9442a1810 100644 --- a/apps/emqx_connector/test/emqx_connector_pgsql_SUITE.erl +++ b/apps/emqx_connector/test/emqx_connector_pgsql_SUITE.erl @@ -95,7 +95,7 @@ perform_lifecycle_check(PoolName, InitialConfig) -> status := StoppedStatus }} = emqx_resource:get_instance(PoolName), - ?assertEqual(StoppedStatus, disconnected), + ?assertEqual(stopped, StoppedStatus), ?assertEqual({error, resource_is_stopped}, emqx_resource:health_check(PoolName)), % Resource healthcheck shortcuts things by checking ets. Go deeper by checking pool itself. ?assertEqual({error, not_found}, ecpool:stop_sup_pool(ReturnedPoolName)), diff --git a/apps/emqx_connector/test/emqx_connector_redis_SUITE.erl b/apps/emqx_connector/test/emqx_connector_redis_SUITE.erl index 4770bbeee..a60702036 100644 --- a/apps/emqx_connector/test/emqx_connector_redis_SUITE.erl +++ b/apps/emqx_connector/test/emqx_connector_redis_SUITE.erl @@ -117,7 +117,7 @@ perform_lifecycle_check(PoolName, InitialConfig, RedisCommand) -> status := StoppedStatus }} = emqx_resource:get_instance(PoolName), - ?assertEqual(StoppedStatus, disconnected), + ?assertEqual(stopped, StoppedStatus), ?assertEqual({error, resource_is_stopped}, emqx_resource:health_check(PoolName)), % Resource healthcheck shortcuts things by checking ets. Go deeper by checking pool itself. ?assertEqual({error, not_found}, ecpool:stop_sup_pool(ReturnedPoolName)), From a2afdeeb48c3f7ee5c1313017fde4b6caf85281d Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Mon, 1 Aug 2022 13:06:28 +0800 Subject: [PATCH 09/14] feat: add test cases for batching query --- apps/emqx_resource/README.md | 2 +- ...t_resource.erl => emqx_connector_demo.erl} | 85 ++++++++++++++----- .../test/emqx_resource_SUITE.erl | 47 +++++++++- 3 files changed, 113 insertions(+), 21 deletions(-) rename apps/emqx_resource/test/{emqx_test_resource.erl => emqx_connector_demo.erl} (54%) diff --git a/apps/emqx_resource/README.md b/apps/emqx_resource/README.md index 04f3c2205..0f61df7ff 100644 --- a/apps/emqx_resource/README.md +++ b/apps/emqx_resource/README.md @@ -14,5 +14,5 @@ the config operations (like config validation, config dump back to files), and t And we put all the `specific` codes to the callback modules. See -* `test/emqx_test_resource.erl` for a minimal `emqx_resource` implementation; +* `test/emqx_connector_demo.erl` for a minimal `emqx_resource` implementation; * `test/emqx_resource_SUITE.erl` for examples of `emqx_resource` usage. diff --git a/apps/emqx_resource/test/emqx_test_resource.erl b/apps/emqx_resource/test/emqx_connector_demo.erl similarity index 54% rename from apps/emqx_resource/test/emqx_test_resource.erl rename to apps/emqx_resource/test/emqx_connector_demo.erl index 569579d27..e9c77e915 100644 --- a/apps/emqx_resource/test/emqx_test_resource.erl +++ b/apps/emqx_resource/test/emqx_connector_demo.erl @@ -14,7 +14,7 @@ %% limitations under the License. %%-------------------------------------------------------------------- --module(emqx_test_resource). +-module(emqx_connector_demo). -include_lib("typerefl/include/types.hrl"). @@ -25,9 +25,12 @@ on_start/2, on_stop/2, on_query/3, + on_batch_query/3, on_get_status/2 ]). +-export([counter_loop/1]). + %% callbacks for emqx_resource config schema -export([roots/0]). @@ -53,19 +56,19 @@ on_start(InstId, #{name := Name, stop_error := true} = Opts) -> {ok, Opts#{ id => InstId, stop_error => true, - pid => spawn_dummy_process(Name, Register) + pid => spawn_counter_process(Name, Register) }}; on_start(InstId, #{name := Name} = Opts) -> Register = maps:get(register, Opts, false), {ok, Opts#{ id => InstId, - pid => spawn_dummy_process(Name, Register) + pid => spawn_counter_process(Name, Register) }}; on_start(InstId, #{name := Name} = Opts) -> Register = maps:get(register, Opts, false), {ok, Opts#{ id => InstId, - pid => spawn_dummy_process(Name, Register) + pid => spawn_counter_process(Name, Register) }}. on_stop(_InstId, #{stop_error := true}) -> @@ -77,7 +80,44 @@ on_stop(_InstId, #{pid := Pid}) -> on_query(_InstId, get_state, State) -> {ok, State}; on_query(_InstId, get_state_failed, State) -> - {error, State}. + {error, State}; +on_query(_InstId, {inc_counter, N}, #{pid := Pid}) -> + Pid ! {inc, N}, + ok; +on_query(_InstId, get_counter, #{pid := Pid}) -> + ReqRef = make_ref(), + From = {self(), ReqRef}, + Pid ! {From, get}, + receive + {ReqRef, Num} -> {ok, Num} + after 1000 -> + {error, timeout} + end. + +on_batch_query(InstId, BatchReq, State) -> + %% Requests can be either 'get_counter' or 'inc_counter', but cannot be mixed. + case hd(BatchReq) of + {_From, {inc_counter, _}} -> + batch_inc_counter(InstId, BatchReq, State); + {_From, get_counter} -> + batch_get_counter(InstId, State) + end. + +batch_inc_counter(InstId, BatchReq, State) -> + TotalN = lists:foldl( + fun + ({_From, {inc_counter, N}}, Total) -> + Total + N; + ({_From, Req}, _Total) -> + error({mixed_requests_not_allowed, {inc_counter, Req}}) + end, + 0, + BatchReq + ), + on_query(InstId, {inc_counter, TotalN}, State). + +batch_get_counter(InstId, State) -> + on_query(InstId, get_counter, State). on_get_status(_InstId, #{health_check_error := true}) -> disconnected; @@ -88,18 +128,25 @@ on_get_status(_InstId, #{pid := Pid}) -> false -> disconnected end. -spawn_dummy_process(Name, Register) -> +spawn_counter_process(Name, Register) -> + Pid = spawn_link(?MODULE, counter_loop, [#{counter => 0}]), + true = maybe_register(Name, Pid, Register), + Pid. + +counter_loop(#{counter := Num} = State) -> + NewState = + receive + {inc, N} -> + #{counter => Num + N}; + {{FromPid, ReqRef}, get} -> + FromPid ! {ReqRef, Num}, + State + end, + counter_loop(NewState). + +maybe_register(Name, Pid, true) -> ct:pal("---- Register Name: ~p", [Name]), - spawn( - fun() -> - true = - case Register of - true -> register(Name, self()); - _ -> true - end, - Ref = make_ref(), - receive - Ref -> ok - end - end - ). + ct:pal("---- whereis(): ~p", [whereis(Name)]), + erlang:register(Name, Pid); +maybe_register(_Name, _Pid, false) -> + true. diff --git a/apps/emqx_resource/test/emqx_resource_SUITE.erl b/apps/emqx_resource/test/emqx_resource_SUITE.erl index 278f556ef..d05d4baf7 100644 --- a/apps/emqx_resource/test/emqx_resource_SUITE.erl +++ b/apps/emqx_resource/test/emqx_resource_SUITE.erl @@ -23,7 +23,7 @@ -include("emqx_resource.hrl"). -include_lib("stdlib/include/ms_transform.hrl"). --define(TEST_RESOURCE, emqx_test_resource). +-define(TEST_RESOURCE, emqx_connector_demo). -define(ID, <<"id">>). -define(DEFAULT_RESOURCE_GROUP, <<"default">>). -define(RESOURCE_ERROR(REASON), {error, {resource_error, #{reason := REASON}}}). @@ -184,6 +184,51 @@ t_query(_) -> ok = emqx_resource:remove_local(?ID). +t_query_counter(_) -> + {ok, _} = emqx_resource:create_local( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource, register => true} + ), + + {ok, 0} = emqx_resource:query(?ID, get_counter), + ok = emqx_resource:query(?ID, {inc_counter, 1}), + {ok, 1} = emqx_resource:query(?ID, get_counter), + ok = emqx_resource:query(?ID, {inc_counter, 5}), + {ok, 6} = emqx_resource:query(?ID, get_counter), + + ok = emqx_resource:remove_local(?ID). + +t_batch_query_counter(_) -> + {ok, _} = emqx_resource:create_local( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource, register => true, batch_enabled => true} + ), + + {ok, 0} = emqx_resource:query(?ID, get_counter), + Parent = self(), + Pids = [ + erlang:spawn(fun() -> + ok = emqx_resource:query(?ID, {inc_counter, 1}), + Parent ! {complete, self()} + end) + || _ <- lists:seq(1, 1000) + ], + [ + receive + {complete, Pid} -> ok + after 1000 -> + ct:fail({wait_for_query_timeout, Pid}) + end + || Pid <- Pids + ], + {ok, 1000} = emqx_resource:query(?ID, get_counter), + + ok = emqx_resource:remove_local(?ID). + t_healthy_timeout(_) -> {ok, _} = emqx_resource:create_local( ?ID, From f1419d52f1390036b1894ef494ff1f3c9f1628f2 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Tue, 2 Aug 2022 18:39:18 +0800 Subject: [PATCH 10/14] fix(resource): remove resource at the end of each test --- apps/emqx_resource/test/emqx_resource_SUITE.erl | 2 ++ 1 file changed, 2 insertions(+) diff --git a/apps/emqx_resource/test/emqx_resource_SUITE.erl b/apps/emqx_resource/test/emqx_resource_SUITE.erl index d05d4baf7..4c5a36f4b 100644 --- a/apps/emqx_resource/test/emqx_resource_SUITE.erl +++ b/apps/emqx_resource/test/emqx_resource_SUITE.erl @@ -36,6 +36,8 @@ groups() -> init_per_testcase(_, Config) -> Config. +end_per_testcase(_, _Config) -> + _ = emqx_resource:remove(?ID). init_per_suite(Config) -> code:ensure_loaded(?TEST_RESOURCE), From 35fe70b887e0d0f3bd9c235fabff822596a11e40 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Mon, 8 Aug 2022 17:52:44 +0800 Subject: [PATCH 11/14] feat: support aysnc callback to connector modules --- .../emqx_authn_jwks_connector.erl | 3 + .../test/emqx_authz_api_cache_SUITE.erl | 2 + .../src/emqx_connector_http.erl | 3 + .../src/emqx_connector_ldap.erl | 3 + .../src/emqx_connector_mongo.erl | 3 + .../src/emqx_connector_mqtt.erl | 3 + .../src/emqx_connector_mysql.erl | 3 + .../src/emqx_connector_pgsql.erl | 3 + .../src/emqx_connector_redis.erl | 3 + apps/emqx_resource/include/emqx_resource.hrl | 1 + apps/emqx_resource/src/emqx_resource.erl | 11 +- .../src/emqx_resource_manager.erl | 5 +- .../src/emqx_resource_worker.erl | 210 +++++++++++------- .../test/emqx_connector_demo.erl | 9 +- 14 files changed, 173 insertions(+), 89 deletions(-) diff --git a/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl b/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl index cd8451ac9..480950143 100644 --- a/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl +++ b/apps/emqx_authn/src/simple_authn/emqx_authn_jwks_connector.erl @@ -22,6 +22,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -31,6 +32,8 @@ -define(DEFAULT_POOL_SIZE, 8). +callback_mode() -> always_sync. + on_start(InstId, Opts) -> PoolName = emqx_plugin_libs_pool:pool_name(InstId), PoolOpts = [ diff --git a/apps/emqx_authz/test/emqx_authz_api_cache_SUITE.erl b/apps/emqx_authz/test/emqx_authz_api_cache_SUITE.erl index 306fe3f13..0c49cc03a 100644 --- a/apps/emqx_authz/test/emqx_authz_api_cache_SUITE.erl +++ b/apps/emqx_authz/test/emqx_authz_api_cache_SUITE.erl @@ -23,6 +23,8 @@ -include_lib("eunit/include/eunit.hrl"). -include_lib("common_test/include/ct.hrl"). +suite() -> [{timetrap, {seconds, 60}}]. + all() -> emqx_common_test_helpers:all(?MODULE). diff --git a/apps/emqx_connector/src/emqx_connector_http.erl b/apps/emqx_connector/src/emqx_connector_http.erl index e0d5ccfe0..c5a1b89db 100644 --- a/apps/emqx_connector/src/emqx_connector_http.erl +++ b/apps/emqx_connector/src/emqx_connector_http.erl @@ -26,6 +26,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -164,6 +165,8 @@ ref(Field) -> hoconsc:ref(?MODULE, Field). %% =================================================================== +callback_mode() -> always_sync. + on_start( InstId, #{ diff --git a/apps/emqx_connector/src/emqx_connector_ldap.erl b/apps/emqx_connector/src/emqx_connector_ldap.erl index 51d18b534..d53c0e41b 100644 --- a/apps/emqx_connector/src/emqx_connector_ldap.erl +++ b/apps/emqx_connector/src/emqx_connector_ldap.erl @@ -25,6 +25,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -42,6 +43,8 @@ roots() -> fields(_) -> []. %% =================================================================== +callback_mode() -> always_sync. + on_start( InstId, #{ diff --git a/apps/emqx_connector/src/emqx_connector_mongo.erl b/apps/emqx_connector/src/emqx_connector_mongo.erl index db8b1e632..07208545f 100644 --- a/apps/emqx_connector/src/emqx_connector_mongo.erl +++ b/apps/emqx_connector/src/emqx_connector_mongo.erl @@ -25,6 +25,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -139,6 +140,8 @@ mongo_fields() -> %% =================================================================== +callback_mode() -> always_sync. + on_start( InstId, Config = #{ diff --git a/apps/emqx_connector/src/emqx_connector_mqtt.erl b/apps/emqx_connector/src/emqx_connector_mqtt.erl index 66682caeb..e37f6a9a2 100644 --- a/apps/emqx_connector/src/emqx_connector_mqtt.erl +++ b/apps/emqx_connector/src/emqx_connector_mqtt.erl @@ -24,6 +24,7 @@ %% API and callbacks for supervisor -export([ + callback_mode/0, start_link/0, init/1, create_bridge/1, @@ -139,6 +140,8 @@ on_message_received(Msg, HookPoint, ResId) -> emqx:run_hook(HookPoint, [Msg]). %% =================================================================== +callback_mode() -> always_sync. + on_start(InstId, Conf) -> InstanceId = binary_to_atom(InstId, utf8), ?SLOG(info, #{ diff --git a/apps/emqx_connector/src/emqx_connector_mysql.erl b/apps/emqx_connector/src/emqx_connector_mysql.erl index e818bc6ef..b379e511c 100644 --- a/apps/emqx_connector/src/emqx_connector_mysql.erl +++ b/apps/emqx_connector/src/emqx_connector_mysql.erl @@ -24,6 +24,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -73,6 +74,8 @@ server(desc) -> ?DESC("server"); server(_) -> undefined. %% =================================================================== +callback_mode() -> always_sync. + -spec on_start(binary(), hoconsc:config()) -> {ok, state()} | {error, _}. on_start( InstId, diff --git a/apps/emqx_connector/src/emqx_connector_pgsql.erl b/apps/emqx_connector/src/emqx_connector_pgsql.erl index d31c1316f..4b188e5a5 100644 --- a/apps/emqx_connector/src/emqx_connector_pgsql.erl +++ b/apps/emqx_connector/src/emqx_connector_pgsql.erl @@ -27,6 +27,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -66,6 +67,8 @@ server(desc) -> ?DESC("server"); server(_) -> undefined. %% =================================================================== +callback_mode() -> always_sync. + on_start( InstId, #{ diff --git a/apps/emqx_connector/src/emqx_connector_redis.erl b/apps/emqx_connector/src/emqx_connector_redis.erl index 4826a170b..fae628d9e 100644 --- a/apps/emqx_connector/src/emqx_connector_redis.erl +++ b/apps/emqx_connector/src/emqx_connector_redis.erl @@ -26,6 +26,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -112,6 +113,8 @@ servers(desc) -> ?DESC("servers"); servers(_) -> undefined. %% =================================================================== +callback_mode() -> always_sync. + on_start( InstId, #{ diff --git a/apps/emqx_resource/include/emqx_resource.hrl b/apps/emqx_resource/include/emqx_resource.hrl index 13ffff587..c691789c2 100644 --- a/apps/emqx_resource/include/emqx_resource.hrl +++ b/apps/emqx_resource/include/emqx_resource.hrl @@ -25,6 +25,7 @@ -type resource_data() :: #{ id := resource_id(), mod := module(), + callback_mode := always_sync | async_if_possible, config := resource_config(), state := resource_state(), status := resource_status(), diff --git a/apps/emqx_resource/src/emqx_resource.erl b/apps/emqx_resource/src/emqx_resource.erl index af047060c..a54a77e19 100644 --- a/apps/emqx_resource/src/emqx_resource.erl +++ b/apps/emqx_resource/src/emqx_resource.erl @@ -75,8 +75,7 @@ %% stop the instance stop/1, %% query the instance - query/2, - query_async/3 + query/2 ]). %% Direct calls to the callback module @@ -224,12 +223,12 @@ reset_metrics(ResId) -> %% ================================================================================= -spec query(resource_id(), Request :: term()) -> Result :: term(). query(ResId, Request) -> - emqx_resource_worker:query(ResId, Request). + query(ResId, Request, #{}). --spec query_async(resource_id(), Request :: term(), emqx_resource_worker:reply_fun()) -> +-spec query(resource_id(), Request :: term(), emqx_resource_worker:query_opts()) -> Result :: term(). -query_async(ResId, Request, ReplyFun) -> - emqx_resource_worker:query_async(ResId, Request, ReplyFun). +query(ResId, Request, Opts) -> + emqx_resource_worker:query(ResId, Request, Opts). -spec start(resource_id()) -> ok | {error, Reason :: term()}. start(ResId) -> diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index 7dad85d5b..d6e5a1493 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -53,7 +53,7 @@ -export([init/1, callback_mode/0, handle_event/4, terminate/3]). % State record --record(data, {id, manager_id, group, mod, config, opts, status, state, error}). +-record(data, {id, manager_id, group, mod, callback_mode, config, opts, status, state, error}). -define(SHORT_HEALTHCHECK_INTERVAL, 1000). -define(HEALTHCHECK_INTERVAL, 15000). @@ -259,6 +259,7 @@ start_link(MgrId, ResId, Group, ResourceType, Config, Opts) -> manager_id = MgrId, group = Group, mod = ResourceType, + callback_mode = ResourceType:callback_mode(), config = Config, opts = Opts, status = connecting, @@ -559,10 +560,12 @@ maybe_reply(Actions, undefined, _Reply) -> maybe_reply(Actions, From, Reply) -> [{reply, From, Reply} | Actions]. +-spec data_record_to_external_map_with_metrics(#data{}) -> resource_data(). data_record_to_external_map_with_metrics(Data) -> #{ id => Data#data.id, mod => Data#data.mod, + callback_mode => Data#data.callback_mode, config => Data#data.config, status => Data#data.status, state => Data#data.state, diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index 055fbfc53..2115fed86 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -27,11 +27,9 @@ -export([ start_link/3, - query/2, query/3, - query_async/3, - query_async/4, block/1, + block/2, resume/1 ]). @@ -46,6 +44,8 @@ -export([queue_item_marshaller/1, estimate_size/1]). +-export([reply_after_query/4, batch_reply_after_query/4]). + -define(RESUME_INTERVAL, 15000). %% count @@ -55,8 +55,8 @@ -define(Q_ITEM(REQUEST), {q_item, REQUEST}). --define(QUERY(FROM, REQUEST), {FROM, REQUEST}). --define(REPLY(FROM, REQUEST, RESULT), {FROM, REQUEST, RESULT}). +-define(QUERY(FROM, REQUEST), {query, FROM, REQUEST}). +-define(REPLY(FROM, REQUEST, RESULT), {reply, FROM, REQUEST, RESULT}). -define(EXPAND(RESULT, BATCH), [?REPLY(FROM, REQUEST, RESULT) || ?QUERY(FROM, REQUEST) <- BATCH]). -define(RESOURCE_ERROR(Reason, Msg), @@ -65,10 +65,17 @@ -define(RESOURCE_ERROR_M(Reason, Msg), {error, {resource_error, #{reason := Reason, msg := Msg}}}). -type id() :: binary(). +-type query() :: {query, from(), request()}. -type request() :: term(). -type result() :: term(). -type reply_fun() :: {fun((result(), Args :: term()) -> any()), Args :: term()} | undefined. -type from() :: pid() | reply_fun(). +-type query_opts() :: #{ + %% The key used for picking a resource worker + pick_key => term() +}. + +-export_type([query_opts/0]). -callback batcher_flush(Acc :: [{from(), request()}], CbState :: term()) -> {{from(), result()}, NewCbState :: term()}. @@ -78,26 +85,20 @@ callback_mode() -> [state_functions]. start_link(Id, Index, Opts) -> gen_statem:start_link({local, name(Id, Index)}, ?MODULE, {Id, Index, Opts}, []). --spec query(id(), request()) -> Result :: term(). -query(Id, Request) -> - query(Id, self(), Request). - --spec query(id(), term(), request()) -> Result :: term(). -query(Id, PickKey, Request) -> - pick_query(call, Id, PickKey, {query, Request}). - --spec query_async(id(), request(), reply_fun()) -> Result :: term(). -query_async(Id, Request, ReplyFun) -> - query_async(Id, self(), Request, ReplyFun). - --spec query_async(id(), term(), request(), reply_fun()) -> Result :: term(). -query_async(Id, PickKey, Request, ReplyFun) -> - pick_query(cast, Id, PickKey, {query, Request, ReplyFun}). +-spec query(id(), request(), query_opts()) -> Result :: term(). +query(Id, Request, Opts) -> + PickKey = maps:get(pick_key, Opts, self()), + Timeout = maps:get(timeout, Opts, infinity), + pick_call(Id, PickKey, {query, Request}, Timeout). -spec block(pid() | atom()) -> ok. block(ServerRef) -> gen_statem:cast(ServerRef, block). +-spec block(pid() | atom(), [query()]) -> ok. +block(ServerRef, Query) -> + gen_statem:cast(ServerRef, {block, Query}). + -spec resume(pid() | atom()) -> ok. resume(ServerRef) -> gen_statem:cast(ServerRef, resume). @@ -121,6 +122,12 @@ init({Id, Index, Opts}) -> St = #{ id => Id, index => Index, + %% query_mode = dynamic | sync | async + %% TODO: + %% dynamic mode is async mode when things are going well, but becomes sync mode + %% if the resource worker is overloaded + query_mode => maps:get(query_mode, Opts, sync), + async_reply_fun => maps:get(async_reply_fun, Opts, undefined), batch_enabled => maps:get(batch_enabled, Opts, false), batch_size => BatchSize, batch_time => maps:get(batch_time, Opts, ?DEFAULT_BATCH_TIME), @@ -135,9 +142,11 @@ running(cast, resume, _St) -> keep_state_and_data; running(cast, block, St) -> {next_state, block, St}; -running(cast, {query, Request, ReplyFun}, St) -> - query_or_acc(ReplyFun, Request, St); -running({call, From}, {query, Request}, St) -> +running(cast, {block, [?QUERY(_, _) | _] = Batch}, #{queue := Q} = St) when is_list(Batch) -> + Q1 = maybe_append_queue(Q, [?Q_ITEM(Query) || Query <- Batch]), + {next_state, block, St#{queue := Q1}}; +running({call, From0}, {query, Request}, #{query_mode := QM, async_reply_fun := ReplyFun} = St) -> + From = maybe_quick_return(QM, From0, ReplyFun), query_or_acc(From, Request, St); running(info, {flush, Ref}, St = #{tref := {_TRef, Ref}}) -> flush(St#{tref := undefined}); @@ -149,13 +158,15 @@ running(info, Info, _St) -> blocked(cast, block, _St) -> keep_state_and_data; +blocked(cast, {block, [?QUERY(_, _) | _] = Batch}, #{queue := Q} = St) when is_list(Batch) -> + Q1 = maybe_append_queue(Q, [?Q_ITEM(Query) || Query <- Batch]), + {keep_state, St#{queue := Q1}}; blocked(cast, resume, St) -> do_resume(St); blocked(state_timeout, resume, St) -> do_resume(St); -blocked(cast, {query, Request, ReplyFun}, St) -> - handle_blocked(ReplyFun, Request, St); -blocked({call, From}, {query, Request}, St) -> +blocked({call, From0}, {query, Request}, #{query_mode := QM, async_reply_fun := ReplyFun} = St) -> + From = maybe_quick_return(QM, From0, ReplyFun), handle_blocked(From, Request, St). terminate(_Reason, #{id := Id, index := Index}) -> @@ -173,15 +184,23 @@ estimate_size(QItem) -> size(queue_item_marshaller(QItem)). %%============================================================================== -pick_query(Fun, Id, Key, Query) -> +maybe_quick_return(sync, From, _ReplyFun) -> + From; +maybe_quick_return(async, From, ReplyFun) -> + ok = gen_statem:reply(From), + ReplyFun. + +pick_call(Id, Key, Query, Timeout) -> try gproc_pool:pick_worker(Id, Key) of Pid when is_pid(Pid) -> - gen_statem:Fun(Pid, Query); + gen_statem:call(Pid, Query, {clean_timeout, Timeout}); _ -> ?RESOURCE_ERROR(not_created, "resource not found") catch error:badarg -> - ?RESOURCE_ERROR(not_created, "resource not found") + ?RESOURCE_ERROR(not_created, "resource not found"); + exit:{timeout, _} -> + ?RESOURCE_ERROR(timeout, "call resource timeout") end. do_resume(#{queue := undefined} = St) -> @@ -190,8 +209,8 @@ do_resume(#{queue := Q, id := Id} = St) -> case replayq:peek(Q) of empty -> {next_state, running, St}; - ?Q_ITEM(First) -> - Result = call_query(Id, First), + ?Q_ITEM(FirstQuery) -> + Result = call_query(sync, Id, FirstQuery, 1), case handle_query_result(Id, Result, false) of %% Send failed because resource down true -> @@ -206,8 +225,8 @@ do_resume(#{queue := Q, id := Id} = St) -> handle_blocked(From, Request, #{id := Id, queue := Q} = St) -> Error = ?RESOURCE_ERROR(blocked, "resource is blocked"), - _ = reply_caller(Id, ?REPLY(From, Request, Error), false), - {keep_state, St#{queue := maybe_append_queue(Q, [?Q_ITEM(Request)])}}. + _ = reply_caller(Id, ?REPLY(From, Request, Error)), + {keep_state, St#{queue := maybe_append_queue(Q, [?Q_ITEM(?QUERY(From, Request))])}}. drop_head(Q) -> {Q1, AckRef, _} = replayq:pop(Q, #{count_limit => 1}), @@ -221,17 +240,18 @@ query_or_acc(From, Request, #{batch_enabled := true, acc := Acc, acc_left := Lef true -> flush(St); false -> {keep_state, ensure_flush_timer(St)} end; -query_or_acc(From, Request, #{batch_enabled := false, queue := Q, id := Id} = St) -> - case send_query(From, Request, Id) of +query_or_acc(From, Request, #{batch_enabled := false, queue := Q, id := Id, query_mode := QM} = St) -> + case send_query(QM, From, Request, Id) of true -> - {next_state, blocked, St#{queue := maybe_append_queue(Q, [?Q_ITEM(Request)])}}; + Query = ?QUERY(From, Request), + {next_state, blocked, St#{queue := maybe_append_queue(Q, [?Q_ITEM(Query)])}}; false -> {keep_state, St} end. -send_query(From, Request, Id) -> - Result = call_query(Id, Request), - reply_caller(Id, ?REPLY(From, Request, Result), false). +send_query(QM, From, Request, Id) -> + Result = call_query(QM, Id, ?QUERY(From, Request), 1), + reply_caller(Id, ?REPLY(From, Request, Result)). flush(#{acc := []} = St) -> {keep_state, St}; @@ -240,31 +260,37 @@ flush( id := Id, acc := Batch, batch_size := Size, - queue := Q0 + queue := Q0, + query_mode := QM } = St ) -> - BatchResults = maybe_expand_batch_result(call_batch_query(Id, Batch), Batch), + Result = call_query(QM, Id, Batch, length(Batch)), St1 = cancel_flush_timer(St#{acc_left := Size, acc := []}), - case batch_reply_caller(Id, BatchResults) of + case batch_reply_caller(Id, Result, Batch) of true -> - Q1 = maybe_append_queue(Q0, [?Q_ITEM(Request) || ?QUERY(_, Request) <- Batch]), + Q1 = maybe_append_queue(Q0, [?Q_ITEM(Query) || Query <- Batch]), {next_state, blocked, St1#{queue := Q1}}; false -> {keep_state, St1} end. -maybe_append_queue(undefined, _Request) -> undefined; -maybe_append_queue(Q, Request) -> replayq:append(Q, Request). +maybe_append_queue(undefined, _Items) -> undefined; +maybe_append_queue(Q, Items) -> replayq:append(Q, Items). -batch_reply_caller(Id, BatchResults) -> +batch_reply_caller(Id, BatchResult, Batch) -> lists:foldl( fun(Reply, BlockWorker) -> reply_caller(Id, Reply, BlockWorker) end, false, - BatchResults + %% the `Mod:on_batch_query/3` returns a single result for a batch, + %% so we need to expand + ?EXPAND(BatchResult, Batch) ). +reply_caller(Id, Reply) -> + reply_caller(Id, Reply, false). + reply_caller(Id, ?REPLY(undefined, _, Result), BlockWorker) -> handle_query_result(Id, Result, BlockWorker); reply_caller(Id, ?REPLY({ReplyFun, Args}, _, Result), BlockWorker) when is_function(ReplyFun) -> @@ -295,45 +321,77 @@ handle_query_result(Id, Result, BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, success), BlockWorker. -call_query(Id, Request) -> - do_call_query(on_query, Id, Request, 1). - -call_batch_query(Id, Batch) -> - do_call_query(on_batch_query, Id, Batch, length(Batch)). - -do_call_query(Fun, Id, Data, Count) -> +call_query(QM, Id, Query, QueryLen) -> case emqx_resource_manager:ets_lookup(Id) of - {ok, _Group, #{mod := Mod, state := ResourceState, status := connected}} -> - ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched, Count), - try Mod:Fun(Id, Data, ResourceState) of - %% if the callback module (connector) wants to return an error that - %% makes the current resource goes into the `error` state, it should - %% return `{resource_down, Reason}` - Result -> Result - catch - Err:Reason:ST -> - Msg = io_lib:format( - "call query failed, func: ~s:~s/3, error: ~0p", - [Mod, Fun, {Err, Reason, ST}] - ), - ?RESOURCE_ERROR(exception, Msg) - end; + {ok, _Group, #{callback_mode := CM, mod := Mod, state := ResSt, status := connected}} -> + ok = emqx_metrics_worker:inc(?RES_METRICS, Id, matched, QueryLen), + apply_query_fun(call_mode(QM, CM), Mod, Id, Query, ResSt); {ok, _Group, #{status := stopped}} -> ?RESOURCE_ERROR(stopped, "resource stopped or disabled"); - {ok, _Group, _Data} -> + {ok, _Group, #{status := S}} when S == connecting; S == disconnected -> ?RESOURCE_ERROR(not_connected, "resource not connected"); {error, not_found} -> ?RESOURCE_ERROR(not_found, "resource not found") end. -%% the result is already expaned by the `Mod:on_query/3` -maybe_expand_batch_result(Results, _Batch) when is_list(Results) -> - Results; -%% the `Mod:on_query/3` returns a sinle result for a batch, so it is need expand -maybe_expand_batch_result(Result, Batch) -> - ?EXPAND(Result, Batch). +-define(APPLY_RESOURCE(EXPR, REQ), + try + %% if the callback module (connector) wants to return an error that + %% makes the current resource goes into the `error` state, it should + %% return `{resource_down, Reason}` + EXPR + catch + ERR:REASON:STACKTRACE -> + MSG = io_lib:format( + "call query failed, func: ~s, id: ~s, error: ~0p, Request: ~0p", + [??EXPR, Id, {ERR, REASON, STACKTRACE}, REQ], + [{chars_limit, 1024}] + ), + ?RESOURCE_ERROR(exception, MSG) + end +). + +apply_query_fun(sync, Mod, Id, ?QUERY(_From, Request), ResSt) -> + ?APPLY_RESOURCE(Mod:on_query(Id, Request, ResSt), Request); +apply_query_fun(async, Mod, Id, ?QUERY(_From, Request) = Query, ResSt) -> + ReplyFun = fun ?MODULE:reply_after_query/4, + ?APPLY_RESOURCE( + begin + _ = Mod:on_query_async(Id, Request, {ReplyFun, [self(), Id, Query]}, ResSt), + ok_async + end, + Request + ); +apply_query_fun(sync, Mod, Id, [?QUERY(_, _) | _] = Batch, ResSt) -> + Requests = [Request || ?QUERY(_From, Request) <- Batch], + ?APPLY_RESOURCE(Mod:on_batch_query(Id, Requests, ResSt), Batch); +apply_query_fun(async, Mod, Id, [?QUERY(_, _) | _] = Batch, ResSt) -> + Requests = [Request || ?QUERY(_From, Request) <- Batch], + ReplyFun = fun ?MODULE:batch_reply_after_query/4, + ?APPLY_RESOURCE( + begin + _ = Mod:on_batch_query_async(Id, Requests, {ReplyFun, [self(), Id, Batch]}, ResSt), + ok_async + end, + Batch + ). + +reply_after_query(Pid, Id, ?QUERY(From, Request) = Query, Result) -> + case reply_caller(Id, ?REPLY(From, Request, Result)) of + true -> ?MODULE:block(Pid, [Query]); + false -> ok + end. + +batch_reply_after_query(Pid, Id, Batch, Result) -> + case batch_reply_caller(Id, Result, Batch) of + true -> ?MODULE:block(Pid, Batch); + false -> ok + end. %%============================================================================== +call_mode(sync, _) -> sync; +call_mode(async, always_sync) -> sync; +call_mode(async, async_if_possible) -> async. is_ok_result(ok) -> true; diff --git a/apps/emqx_resource/test/emqx_connector_demo.erl b/apps/emqx_resource/test/emqx_connector_demo.erl index e9c77e915..740f110ec 100644 --- a/apps/emqx_resource/test/emqx_connector_demo.erl +++ b/apps/emqx_resource/test/emqx_connector_demo.erl @@ -22,6 +22,7 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, on_query/3, @@ -49,6 +50,8 @@ register(required) -> true; register(default) -> false; register(_) -> undefined. +callback_mode() -> always_sync. + on_start(_InstId, #{create_error := true}) -> error("some error"); on_start(InstId, #{name := Name, stop_error := true} = Opts) -> @@ -58,12 +61,6 @@ on_start(InstId, #{name := Name, stop_error := true} = Opts) -> stop_error => true, pid => spawn_counter_process(Name, Register) }}; -on_start(InstId, #{name := Name} = Opts) -> - Register = maps:get(register, Opts, false), - {ok, Opts#{ - id => InstId, - pid => spawn_counter_process(Name, Register) - }}; on_start(InstId, #{name := Name} = Opts) -> Register = maps:get(register, Opts, false), {ok, Opts#{ From 145ff66a9add21898ee6f6805273d72cb57ab58d Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Tue, 9 Aug 2022 08:58:42 +0800 Subject: [PATCH 12/14] fix: issues found by dialyzer and elvis --- apps/emqx_resource/include/emqx_resource.hrl | 3 +- apps/emqx_resource/src/emqx_resource.erl | 8 ++++- .../src/emqx_resource_manager.erl | 5 ++-- .../src/emqx_ee_connector_hstreamdb.erl | 29 +++++++++---------- .../src/emqx_ee_connector_influxdb.erl | 17 ++++++----- 5 files changed, 35 insertions(+), 27 deletions(-) diff --git a/apps/emqx_resource/include/emqx_resource.hrl b/apps/emqx_resource/include/emqx_resource.hrl index c691789c2..a59877a30 100644 --- a/apps/emqx_resource/include/emqx_resource.hrl +++ b/apps/emqx_resource/include/emqx_resource.hrl @@ -22,10 +22,11 @@ -type resource_spec() :: map(). -type resource_state() :: term(). -type resource_status() :: connected | disconnected | connecting | stopped. +-type callback_mode() :: always_sync | async_if_possible. -type resource_data() :: #{ id := resource_id(), mod := module(), - callback_mode := always_sync | async_if_possible, + callback_mode := callback_mode(), config := resource_config(), state := resource_state(), status := resource_status(), diff --git a/apps/emqx_resource/src/emqx_resource.erl b/apps/emqx_resource/src/emqx_resource.erl index a54a77e19..d17f4ce19 100644 --- a/apps/emqx_resource/src/emqx_resource.erl +++ b/apps/emqx_resource/src/emqx_resource.erl @@ -80,8 +80,10 @@ %% Direct calls to the callback module -%% start the instance -export([ + %% get the callback mode of a specific module + get_callback_mode/1, + %% start the instance call_start/3, %% verify if the resource is working normally call_health_check/3, @@ -285,6 +287,10 @@ generate_id(Name) when is_binary(Name) -> -spec list_group_instances(resource_group()) -> [resource_id()]. list_group_instances(Group) -> emqx_resource_manager:list_group(Group). +-spec get_callback_mode(module()) -> callback_mode(). +get_callback_mode(Mod) -> + Mod:callback_mode(). + -spec call_start(manager_id(), module(), resource_config()) -> {ok, resource_state()} | {error, Reason :: term()}. call_start(MgrId, Mod, Config) -> diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index d6e5a1493..3310555d1 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -54,6 +54,7 @@ % State record -record(data, {id, manager_id, group, mod, callback_mode, config, opts, status, state, error}). +-type data() :: #data{}. -define(SHORT_HEALTHCHECK_INTERVAL, 1000). -define(HEALTHCHECK_INTERVAL, 15000). @@ -259,7 +260,7 @@ start_link(MgrId, ResId, Group, ResourceType, Config, Opts) -> manager_id = MgrId, group = Group, mod = ResourceType, - callback_mode = ResourceType:callback_mode(), + callback_mode = emqx_resource:get_callback_mode(ResourceType), config = Config, opts = Opts, status = connecting, @@ -560,7 +561,7 @@ maybe_reply(Actions, undefined, _Reply) -> maybe_reply(Actions, From, Reply) -> [{reply, From, Reply} | Actions]. --spec data_record_to_external_map_with_metrics(#data{}) -> resource_data(). +-spec data_record_to_external_map_with_metrics(data()) -> resource_data(). data_record_to_external_map_with_metrics(Data) -> #{ id => Data#data.id, diff --git a/lib-ee/emqx_ee_connector/src/emqx_ee_connector_hstreamdb.erl b/lib-ee/emqx_ee_connector/src/emqx_ee_connector_hstreamdb.erl index 8ee37cd8a..3892b7fc0 100644 --- a/lib-ee/emqx_ee_connector/src/emqx_ee_connector_hstreamdb.erl +++ b/lib-ee/emqx_ee_connector/src/emqx_ee_connector_hstreamdb.erl @@ -13,9 +13,10 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -33,6 +34,7 @@ %% ------------------------------------------------------------------------------------------------- %% resource callback +callback_mode() -> always_sync. on_start(InstId, Config) -> start_client(InstId, Config). @@ -52,11 +54,10 @@ on_stop(InstId, #{client := Client, producer := Producer}) -> on_query( _InstId, {send_message, Data}, - AfterQuery, #{producer := Producer, ordering_key := OrderingKey, payload := Payload} ) -> Record = to_record(OrderingKey, Payload, Data), - do_append(AfterQuery, Producer, Record). + do_append(Producer, Record). on_get_status(_InstId, #{client := Client}) -> case is_alive(Client) of @@ -260,27 +261,26 @@ to_record(OrderingKey, Payload) when is_binary(OrderingKey) -> to_record(OrderingKey, Payload) -> hstreamdb:to_record(OrderingKey, raw, Payload). -do_append(AfterQuery, Producer, Record) -> - do_append(AfterQuery, false, Producer, Record). +do_append(Producer, Record) -> + do_append(false, Producer, Record). %% TODO: this append is async, remove or change it after we have better disk cache. -% do_append(AfterQuery, true, Producer, Record) -> +% do_append(true, Producer, Record) -> % case hstreamdb:append(Producer, Record) of % ok -> % ?SLOG(debug, #{ % msg => "hstreamdb producer async append success", % record => Record -% }), -% emqx_resource:query_success(AfterQuery); -% {error, Reason} -> +% }); +% {error, Reason} = Err -> % ?SLOG(error, #{ % msg => "hstreamdb producer async append failed", % reason => Reason, % record => Record % }), -% emqx_resource:query_failed(AfterQuery) +% Err % end; -do_append(AfterQuery, false, Producer, Record) -> +do_append(false, Producer, Record) -> %% TODO: this append is sync, but it does not support [Record], can only append one Record. %% Change it after we have better dick cache. case hstreamdb:append_flush(Producer, Record) of @@ -288,15 +288,14 @@ do_append(AfterQuery, false, Producer, Record) -> ?SLOG(debug, #{ msg => "hstreamdb producer sync append success", record => Record - }), - emqx_resource:query_success(AfterQuery); - {error, Reason} -> + }); + {error, Reason} = Err -> ?SLOG(error, #{ msg => "hstreamdb producer sync append failed", reason => Reason, record => Record }), - emqx_resource:query_failed(AfterQuery) + Err end. client_name(InstId) -> diff --git a/lib-ee/emqx_ee_connector/src/emqx_ee_connector_influxdb.erl b/lib-ee/emqx_ee_connector/src/emqx_ee_connector_influxdb.erl index 9582f1729..09b3d7350 100644 --- a/lib-ee/emqx_ee_connector/src/emqx_ee_connector_influxdb.erl +++ b/lib-ee/emqx_ee_connector/src/emqx_ee_connector_influxdb.erl @@ -13,9 +13,10 @@ %% callbacks of behaviour emqx_resource -export([ + callback_mode/0, on_start/2, on_stop/2, - on_query/4, + on_query/3, on_get_status/2 ]). @@ -28,6 +29,7 @@ %% ------------------------------------------------------------------------------------------------- %% resource callback +callback_mode() -> always_sync. on_start(InstId, Config) -> start_client(InstId, Config). @@ -35,8 +37,8 @@ on_start(InstId, Config) -> on_stop(_InstId, #{client := Client}) -> influxdb:stop_client(Client). -on_query(InstId, {send_message, Data}, AfterQuery, State) -> - do_query(InstId, {send_message, Data}, AfterQuery, State). +on_query(InstId, {send_message, Data}, State) -> + do_query(InstId, {send_message, Data}, State). on_get_status(_InstId, #{client := Client}) -> case influxdb:is_alive(Client) of @@ -308,7 +310,7 @@ ssl_config(SSL = #{enable := true}) -> %% ------------------------------------------------------------------------------------------------- %% Query -do_query(InstId, {send_message, Data}, AfterQuery, State = #{client := Client}) -> +do_query(InstId, {send_message, Data}, State = #{client := Client}) -> {Points, Errs} = data_to_points(Data, State), lists:foreach( fun({error, Reason}) -> @@ -326,15 +328,14 @@ do_query(InstId, {send_message, Data}, AfterQuery, State = #{client := Client}) msg => "influxdb write point success", connector => InstId, points => Points - }), - emqx_resource:query_success(AfterQuery); - {error, Reason} -> + }); + {error, Reason} = Err -> ?SLOG(error, #{ msg => "influxdb write point failed", connector => InstId, reason => Reason }), - emqx_resource:query_failed(AfterQuery) + Err end. %% ------------------------------------------------------------------------------------------------- From efd6c56dd9554a05b8b0772856008cdae40ab6ad Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Tue, 9 Aug 2022 13:00:21 +0800 Subject: [PATCH 13/14] fix: test cases for batch query sync --- .../src/emqx_resource_worker.erl | 7 +++- .../test/emqx_connector_demo.erl | 8 ++--- .../test/emqx_resource_SUITE.erl | 35 +++++++++++++++---- 3 files changed, 38 insertions(+), 12 deletions(-) diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index 2115fed86..e20345c2b 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -22,6 +22,7 @@ -include("emqx_resource.hrl"). -include("emqx_resource_utils.hrl"). -include_lib("emqx/include/logger.hrl"). +-include_lib("snabbkaffe/include/snabbkaffe.hrl"). -behaviour(gen_statem). @@ -351,9 +352,11 @@ call_query(QM, Id, Query, QueryLen) -> end ). -apply_query_fun(sync, Mod, Id, ?QUERY(_From, Request), ResSt) -> +apply_query_fun(sync, Mod, Id, ?QUERY(_From, Request) = _Query, ResSt) -> + ?tp(call_query, #{id => Id, mod => Mod, query => _Query, res_st => ResSt}), ?APPLY_RESOURCE(Mod:on_query(Id, Request, ResSt), Request); apply_query_fun(async, Mod, Id, ?QUERY(_From, Request) = Query, ResSt) -> + ?tp(call_query_async, #{id => Id, mod => Mod, query => Query, res_st => ResSt}), ReplyFun = fun ?MODULE:reply_after_query/4, ?APPLY_RESOURCE( begin @@ -363,9 +366,11 @@ apply_query_fun(async, Mod, Id, ?QUERY(_From, Request) = Query, ResSt) -> Request ); apply_query_fun(sync, Mod, Id, [?QUERY(_, _) | _] = Batch, ResSt) -> + ?tp(call_batch_query, #{id => Id, mod => Mod, batch => Batch, res_st => ResSt}), Requests = [Request || ?QUERY(_From, Request) <- Batch], ?APPLY_RESOURCE(Mod:on_batch_query(Id, Requests, ResSt), Batch); apply_query_fun(async, Mod, Id, [?QUERY(_, _) | _] = Batch, ResSt) -> + ?tp(call_batch_query_async, #{id => Id, mod => Mod, batch => Batch, res_st => ResSt}), Requests = [Request || ?QUERY(_From, Request) <- Batch], ReplyFun = fun ?MODULE:batch_reply_after_query/4, ?APPLY_RESOURCE( diff --git a/apps/emqx_resource/test/emqx_connector_demo.erl b/apps/emqx_resource/test/emqx_connector_demo.erl index 740f110ec..40734de68 100644 --- a/apps/emqx_resource/test/emqx_connector_demo.erl +++ b/apps/emqx_resource/test/emqx_connector_demo.erl @@ -94,18 +94,18 @@ on_query(_InstId, get_counter, #{pid := Pid}) -> on_batch_query(InstId, BatchReq, State) -> %% Requests can be either 'get_counter' or 'inc_counter', but cannot be mixed. case hd(BatchReq) of - {_From, {inc_counter, _}} -> + {inc_counter, _} -> batch_inc_counter(InstId, BatchReq, State); - {_From, get_counter} -> + get_counter -> batch_get_counter(InstId, State) end. batch_inc_counter(InstId, BatchReq, State) -> TotalN = lists:foldl( fun - ({_From, {inc_counter, N}}, Total) -> + ({inc_counter, N}, Total) -> Total + N; - ({_From, Req}, _Total) -> + (Req, _Total) -> error({mixed_requests_not_allowed, {inc_counter, Req}}) end, 0, diff --git a/apps/emqx_resource/test/emqx_resource_SUITE.erl b/apps/emqx_resource/test/emqx_resource_SUITE.erl index 4c5a36f4b..fb2bdfd7c 100644 --- a/apps/emqx_resource/test/emqx_resource_SUITE.erl +++ b/apps/emqx_resource/test/emqx_resource_SUITE.erl @@ -22,6 +22,7 @@ -include_lib("common_test/include/ct.hrl"). -include("emqx_resource.hrl"). -include_lib("stdlib/include/ms_transform.hrl"). +-include_lib("snabbkaffe/include/snabbkaffe.hrl"). -define(TEST_RESOURCE, emqx_connector_demo). -define(ID, <<"id">>). @@ -207,17 +208,40 @@ t_batch_query_counter(_) -> ?ID, ?DEFAULT_RESOURCE_GROUP, ?TEST_RESOURCE, - #{name => test_resource, register => true, batch_enabled => true} + #{name => test_resource, register => true}, + #{batch_enabled => true} ), - {ok, 0} = emqx_resource:query(?ID, get_counter), + ?check_trace( + #{timetrap => 10000, timeout => 1000}, + emqx_resource:query(?ID, get_counter), + fun(Result, Trace) -> + ?assertMatch({ok, 0}, Result), + QueryTrace = ?of_kind(call_batch_query, Trace), + ?assertMatch([#{batch := [{query, _, get_counter}]}], QueryTrace) + end + ), + + ?check_trace( + #{timetrap => 10000, timeout => 1000}, + inc_counter_in_parallel(1000), + fun(Trace) -> + QueryTrace = ?of_kind(call_batch_query, Trace), + ?assertMatch([#{batch := BatchReq} | _] when length(BatchReq) > 1, QueryTrace) + end + ), + {ok, 1000} = emqx_resource:query(?ID, get_counter), + + ok = emqx_resource:remove_local(?ID). + +inc_counter_in_parallel(N) -> Parent = self(), Pids = [ erlang:spawn(fun() -> ok = emqx_resource:query(?ID, {inc_counter, 1}), Parent ! {complete, self()} end) - || _ <- lists:seq(1, 1000) + || _ <- lists:seq(1, N) ], [ receive @@ -226,10 +250,7 @@ t_batch_query_counter(_) -> ct:fail({wait_for_query_timeout, Pid}) end || Pid <- Pids - ], - {ok, 1000} = emqx_resource:query(?ID, get_counter), - - ok = emqx_resource:remove_local(?ID). + ]. t_healthy_timeout(_) -> {ok, _} = emqx_resource:create_local( From 82550a585a4a0ce7a3e72733d67d683c03f00645 Mon Sep 17 00:00:00 2001 From: Shawn <506895667@qq.com> Date: Wed, 10 Aug 2022 00:30:42 +0800 Subject: [PATCH 14/14] fix: add test cases for query async --- apps/emqx_resource/include/emqx_resource.hrl | 6 + apps/emqx_resource/src/emqx_resource.erl | 13 +- .../src/emqx_resource_worker.erl | 48 +++++-- .../test/emqx_connector_demo.erl | 17 ++- .../test/emqx_resource_SUITE.erl | 122 +++++++++++++++--- 5 files changed, 171 insertions(+), 35 deletions(-) diff --git a/apps/emqx_resource/include/emqx_resource.hrl b/apps/emqx_resource/include/emqx_resource.hrl index a59877a30..75cba14ad 100644 --- a/apps/emqx_resource/include/emqx_resource.hrl +++ b/apps/emqx_resource/include/emqx_resource.hrl @@ -23,6 +23,12 @@ -type resource_state() :: term(). -type resource_status() :: connected | disconnected | connecting | stopped. -type callback_mode() :: always_sync | async_if_possible. +-type result() :: term(). +-type reply_fun() :: {fun((result(), Args :: term()) -> any()), Args :: term()} | undefined. +-type query_opts() :: #{ + %% The key used for picking a resource worker + pick_key => term() +}. -type resource_data() :: #{ id := resource_id(), mod := module(), diff --git a/apps/emqx_resource/src/emqx_resource.erl b/apps/emqx_resource/src/emqx_resource.erl index d17f4ce19..f3f2d5fb9 100644 --- a/apps/emqx_resource/src/emqx_resource.erl +++ b/apps/emqx_resource/src/emqx_resource.erl @@ -75,7 +75,10 @@ %% stop the instance stop/1, %% query the instance - query/2 + query/2, + %% query the instance without batching and queuing messages. + simple_sync_query/2, + simple_async_query/3 ]). %% Direct calls to the callback module @@ -232,6 +235,14 @@ query(ResId, Request) -> query(ResId, Request, Opts) -> emqx_resource_worker:query(ResId, Request, Opts). +-spec simple_sync_query(resource_id(), Request :: term()) -> Result :: term(). +simple_sync_query(ResId, Request) -> + emqx_resource_worker:simple_sync_query(ResId, Request). + +-spec simple_async_query(resource_id(), Request :: term(), reply_fun()) -> Result :: term(). +simple_async_query(ResId, Request, ReplyFun) -> + emqx_resource_worker:simple_async_query(ResId, Request, ReplyFun). + -spec start(resource_id()) -> ok | {error, Reason :: term()}. start(ResId) -> start(ResId, #{}). diff --git a/apps/emqx_resource/src/emqx_resource_worker.erl b/apps/emqx_resource/src/emqx_resource_worker.erl index e20345c2b..d19353a29 100644 --- a/apps/emqx_resource/src/emqx_resource_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_worker.erl @@ -34,6 +34,11 @@ resume/1 ]). +-export([ + simple_sync_query/2, + simple_async_query/3 +]). + -export([ callback_mode/0, init/1, @@ -68,13 +73,7 @@ -type id() :: binary(). -type query() :: {query, from(), request()}. -type request() :: term(). --type result() :: term(). --type reply_fun() :: {fun((result(), Args :: term()) -> any()), Args :: term()} | undefined. -type from() :: pid() | reply_fun(). --type query_opts() :: #{ - %% The key used for picking a resource worker - pick_key => term() -}. -export_type([query_opts/0]). @@ -92,6 +91,19 @@ query(Id, Request, Opts) -> Timeout = maps:get(timeout, Opts, infinity), pick_call(Id, PickKey, {query, Request}, Timeout). +%% simple query the resource without batching and queuing messages. +-spec simple_sync_query(id(), request()) -> Result :: term(). +simple_sync_query(Id, Request) -> + Result = call_query(sync, Id, ?QUERY(self(), Request), 1), + _ = handle_query_result(Id, Result, false), + Result. + +-spec simple_async_query(id(), request(), reply_fun()) -> Result :: term(). +simple_async_query(Id, Request, ReplyFun) -> + Result = call_query(async, Id, ?QUERY(ReplyFun, Request), 1), + _ = handle_query_result(Id, Result, false), + Result. + -spec block(pid() | atom()) -> ok. block(ServerRef) -> gen_statem:cast(ServerRef, block). @@ -188,7 +200,7 @@ estimate_size(QItem) -> maybe_quick_return(sync, From, _ReplyFun) -> From; maybe_quick_return(async, From, ReplyFun) -> - ok = gen_statem:reply(From), + gen_statem:reply(From, ok), ReplyFun. pick_call(Id, Key, Query, Timeout) -> @@ -295,7 +307,11 @@ reply_caller(Id, Reply) -> reply_caller(Id, ?REPLY(undefined, _, Result), BlockWorker) -> handle_query_result(Id, Result, BlockWorker); reply_caller(Id, ?REPLY({ReplyFun, Args}, _, Result), BlockWorker) when is_function(ReplyFun) -> - ?SAFE_CALL(ReplyFun(Result, Args)), + _ = + case Result of + {async_return, _} -> ok; + _ -> apply(ReplyFun, Args ++ [Result]) + end, handle_query_result(Id, Result, BlockWorker); reply_caller(Id, ?REPLY(From, _, Result), BlockWorker) -> gen_statem:reply(From, Result), @@ -316,6 +332,10 @@ handle_query_result(Id, {error, _}, BlockWorker) -> handle_query_result(Id, {resource_down, _}, _BlockWorker) -> emqx_metrics_worker:inc(?RES_METRICS, Id, resource_down), true; +handle_query_result(_Id, {async_return, {resource_down, _}}, _BlockWorker) -> + true; +handle_query_result(_Id, {async_return, ok}, BlockWorker) -> + BlockWorker; handle_query_result(Id, Result, BlockWorker) -> %% assert true = is_ok_result(Result), @@ -352,16 +372,16 @@ call_query(QM, Id, Query, QueryLen) -> end ). -apply_query_fun(sync, Mod, Id, ?QUERY(_From, Request) = _Query, ResSt) -> +apply_query_fun(sync, Mod, Id, ?QUERY(_, Request) = _Query, ResSt) -> ?tp(call_query, #{id => Id, mod => Mod, query => _Query, res_st => ResSt}), ?APPLY_RESOURCE(Mod:on_query(Id, Request, ResSt), Request); -apply_query_fun(async, Mod, Id, ?QUERY(_From, Request) = Query, ResSt) -> +apply_query_fun(async, Mod, Id, ?QUERY(_, Request) = Query, ResSt) -> ?tp(call_query_async, #{id => Id, mod => Mod, query => Query, res_st => ResSt}), ReplyFun = fun ?MODULE:reply_after_query/4, ?APPLY_RESOURCE( begin - _ = Mod:on_query_async(Id, Request, {ReplyFun, [self(), Id, Query]}, ResSt), - ok_async + Result = Mod:on_query_async(Id, Request, {ReplyFun, [self(), Id, Query]}, ResSt), + {async_return, Result} end, Request ); @@ -375,8 +395,8 @@ apply_query_fun(async, Mod, Id, [?QUERY(_, _) | _] = Batch, ResSt) -> ReplyFun = fun ?MODULE:batch_reply_after_query/4, ?APPLY_RESOURCE( begin - _ = Mod:on_batch_query_async(Id, Requests, {ReplyFun, [self(), Id, Batch]}, ResSt), - ok_async + Result = Mod:on_batch_query_async(Id, Requests, {ReplyFun, [self(), Id, Batch]}, ResSt), + {async_return, Result} end, Batch ). diff --git a/apps/emqx_resource/test/emqx_connector_demo.erl b/apps/emqx_resource/test/emqx_connector_demo.erl index 40734de68..3bea71993 100644 --- a/apps/emqx_resource/test/emqx_connector_demo.erl +++ b/apps/emqx_resource/test/emqx_connector_demo.erl @@ -26,11 +26,12 @@ on_start/2, on_stop/2, on_query/3, + on_query_async/4, on_batch_query/3, on_get_status/2 ]). --export([counter_loop/1]). +-export([counter_loop/1, set_callback_mode/1]). %% callbacks for emqx_resource config schema -export([roots/0]). @@ -50,7 +51,12 @@ register(required) -> true; register(default) -> false; register(_) -> undefined. -callback_mode() -> always_sync. +-define(CM_KEY, {?MODULE, callback_mode}). +callback_mode() -> + persistent_term:get(?CM_KEY). + +set_callback_mode(Mode) -> + persistent_term:put(?CM_KEY, Mode). on_start(_InstId, #{create_error := true}) -> error("some error"); @@ -91,6 +97,10 @@ on_query(_InstId, get_counter, #{pid := Pid}) -> {error, timeout} end. +on_query_async(_InstId, Query, ReplyFun, State) -> + Result = on_query(_InstId, Query, State), + apply_reply(ReplyFun, Result). + on_batch_query(InstId, BatchReq, State) -> %% Requests can be either 'get_counter' or 'inc_counter', but cannot be mixed. case hd(BatchReq) of @@ -147,3 +157,6 @@ maybe_register(Name, Pid, true) -> erlang:register(Name, Pid); maybe_register(_Name, _Pid, false) -> true. + +apply_reply({ReplyFun, Args}, Result) when is_function(ReplyFun) -> + apply(ReplyFun, Args ++ [Result]). diff --git a/apps/emqx_resource/test/emqx_resource_SUITE.erl b/apps/emqx_resource/test/emqx_resource_SUITE.erl index fb2bdfd7c..5177e792c 100644 --- a/apps/emqx_resource/test/emqx_resource_SUITE.erl +++ b/apps/emqx_resource/test/emqx_resource_SUITE.erl @@ -28,6 +28,7 @@ -define(ID, <<"id">>). -define(DEFAULT_RESOURCE_GROUP, <<"default">>). -define(RESOURCE_ERROR(REASON), {error, {resource_error, #{reason := REASON}}}). +-define(TRACE_OPTS, #{timetrap => 10000, timeout => 1000}). all() -> emqx_common_test_helpers:all(?MODULE). @@ -36,6 +37,7 @@ groups() -> []. init_per_testcase(_, Config) -> + emqx_connector_demo:set_callback_mode(always_sync), Config. end_per_testcase(_, _Config) -> _ = emqx_resource:remove(?ID). @@ -213,7 +215,7 @@ t_batch_query_counter(_) -> ), ?check_trace( - #{timetrap => 10000, timeout => 1000}, + ?TRACE_OPTS, emqx_resource:query(?ID, get_counter), fun(Result, Trace) -> ?assertMatch({ok, 0}, Result), @@ -223,7 +225,7 @@ t_batch_query_counter(_) -> ), ?check_trace( - #{timetrap => 10000, timeout => 1000}, + ?TRACE_OPTS, inc_counter_in_parallel(1000), fun(Trace) -> QueryTrace = ?of_kind(call_batch_query, Trace), @@ -234,23 +236,90 @@ t_batch_query_counter(_) -> ok = emqx_resource:remove_local(?ID). -inc_counter_in_parallel(N) -> - Parent = self(), - Pids = [ - erlang:spawn(fun() -> - ok = emqx_resource:query(?ID, {inc_counter, 1}), - Parent ! {complete, self()} - end) - || _ <- lists:seq(1, N) - ], - [ - receive - {complete, Pid} -> ok - after 1000 -> - ct:fail({wait_for_query_timeout, Pid}) +t_query_counter_async(_) -> + {ok, _} = emqx_resource:create_local( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource, register => true}, + #{query_mode => async} + ), + ?assertMatch({ok, 0}, emqx_resource:simple_sync_query(?ID, get_counter)), + ?check_trace( + ?TRACE_OPTS, + inc_counter_in_parallel(1000), + fun(Trace) -> + %% the callback_mode if 'emqx_connector_demo' is 'always_sync'. + QueryTrace = ?of_kind(call_query, Trace), + ?assertMatch([#{query := {query, _, {inc_counter, 1}}} | _], QueryTrace) end - || Pid <- Pids - ]. + ), + %% wait for 1s to make sure all the aysnc query is sent to the resource. + timer:sleep(1000), + %% simple query ignores the query_mode and batching settings in the resource_worker + ?check_trace( + ?TRACE_OPTS, + emqx_resource:simple_sync_query(?ID, get_counter), + fun(Result, Trace) -> + ?assertMatch({ok, 1000}, Result), + %% the callback_mode if 'emqx_connector_demo' is 'always_sync'. + QueryTrace = ?of_kind(call_query, Trace), + ?assertMatch([#{query := {query, _, get_counter}}], QueryTrace) + end + ), + {ok, _, #{metrics := #{counters := C}}} = emqx_resource:get_instance(?ID), + ?assertMatch(#{matched := 1002, success := 1002, failed := 0}, C), + ok = emqx_resource:remove_local(?ID). + +t_query_counter_async_2(_) -> + emqx_connector_demo:set_callback_mode(async_if_possible), + + Tab0 = ets:new(?FUNCTION_NAME, [bag, public]), + Insert = fun(Tab, Result) -> + ets:insert(Tab, {make_ref(), Result}) + end, + {ok, _} = emqx_resource:create_local( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource, register => true}, + #{query_mode => async, async_reply_fun => {Insert, [Tab0]}} + ), + ?assertMatch({ok, 0}, emqx_resource:simple_sync_query(?ID, get_counter)), + ?check_trace( + ?TRACE_OPTS, + inc_counter_in_parallel(1000), + fun(Trace) -> + QueryTrace = ?of_kind(call_query_async, Trace), + ?assertMatch([#{query := {query, _, {inc_counter, 1}}} | _], QueryTrace) + end + ), + + %% wait for 1s to make sure all the aysnc query is sent to the resource. + timer:sleep(1000), + %% simple query ignores the query_mode and batching settings in the resource_worker + ?check_trace( + ?TRACE_OPTS, + emqx_resource:simple_sync_query(?ID, get_counter), + fun(Result, Trace) -> + ?assertMatch({ok, 1000}, Result), + QueryTrace = ?of_kind(call_query, Trace), + ?assertMatch([#{query := {query, _, get_counter}}], QueryTrace) + end + ), + {ok, _, #{metrics := #{counters := C}}} = emqx_resource:get_instance(?ID), + ?assertMatch(#{matched := 1002, success := 1002, failed := 0}, C), + ?assertMatch(1000, ets:info(Tab0, size)), + ?assert( + lists:all( + fun + ({_, ok}) -> true; + (_) -> false + end, + ets:tab2list(Tab0) + ) + ), + ok = emqx_resource:remove_local(?ID). t_healthy_timeout(_) -> {ok, _} = emqx_resource:create_local( @@ -480,6 +549,23 @@ t_auto_retry(_) -> %%------------------------------------------------------------------------------ %% Helpers %%------------------------------------------------------------------------------ +inc_counter_in_parallel(N) -> + Parent = self(), + Pids = [ + erlang:spawn(fun() -> + emqx_resource:query(?ID, {inc_counter, 1}), + Parent ! {complete, self()} + end) + || _ <- lists:seq(1, N) + ], + [ + receive + {complete, Pid} -> ok + after 1000 -> + ct:fail({wait_for_query_timeout, Pid}) + end + || Pid <- Pids + ]. bin_config() -> <<"\"name\": \"test_resource\"">>.