From 6fa6c679bb7f5c3dbe74a3450984d9875d63ca2a Mon Sep 17 00:00:00 2001 From: Thales Macedo Garitezi Date: Thu, 19 Jan 2023 18:07:08 -0300 Subject: [PATCH 01/15] feat(buffer_worker): add expiration time to requests With this, we avoid performing work or replying to callers that are no longer waiting on a result. Also introduces two new counters: - `dropped.expired` :: happens when a request expires before being sent downstream - `late_reply` :: when a response is receive from downstream, but the caller is no longer for a reply because the request has expired, and the caller might even have retried it. --- apps/emqx_bridge/include/emqx_bridge.hrl | 10 +- apps/emqx_bridge/src/emqx_bridge_api.erl | 12 +- .../test/emqx_bridge_mqtt_SUITE.erl | 15 +- apps/emqx_resource/include/emqx_resource.hrl | 2 + .../src/emqx_resource_buffer_worker.erl | 319 ++++++-- .../src/emqx_resource_manager.erl | 2 + .../src/emqx_resource_metrics.erl | 37 + .../test/emqx_connector_demo.erl | 2 +- .../test/emqx_resource_SUITE.erl | 770 +++++++++++++++++- .../test/emqx_ee_bridge_gcp_pubsub_SUITE.erl | 8 +- .../src/emqx_ee_connector_gcp_pubsub.erl | 1 - 11 files changed, 1090 insertions(+), 88 deletions(-) diff --git a/apps/emqx_bridge/include/emqx_bridge.hrl b/apps/emqx_bridge/include/emqx_bridge.hrl index d8229cc77..d062b4b7f 100644 --- a/apps/emqx_bridge/include/emqx_bridge.hrl +++ b/apps/emqx_bridge/include/emqx_bridge.hrl @@ -16,19 +16,21 @@ -define(EMPTY_METRICS, ?METRICS( - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 ) ). -define(METRICS( Dropped, DroppedOther, + DroppedExpired, DroppedQueueFull, DroppedResourceNotFound, DroppedResourceStopped, Matched, Queued, Retried, + LateReply, SentFailed, SentInflight, SentSucc, @@ -40,12 +42,14 @@ #{ 'dropped' => Dropped, 'dropped.other' => DroppedOther, + 'dropped.expired' => DroppedExpired, 'dropped.queue_full' => DroppedQueueFull, 'dropped.resource_not_found' => DroppedResourceNotFound, 'dropped.resource_stopped' => DroppedResourceStopped, 'matched' => Matched, 'queuing' => Queued, 'retried' => Retried, + 'late_reply' => LateReply, 'failed' => SentFailed, 'inflight' => SentInflight, 'success' => SentSucc, @@ -59,12 +63,14 @@ -define(metrics( Dropped, DroppedOther, + DroppedExpired, DroppedQueueFull, DroppedResourceNotFound, DroppedResourceStopped, Matched, Queued, Retried, + LateReply, SentFailed, SentInflight, SentSucc, @@ -76,12 +82,14 @@ #{ 'dropped' := Dropped, 'dropped.other' := DroppedOther, + 'dropped.expired' := DroppedExpired, 'dropped.queue_full' := DroppedQueueFull, 'dropped.resource_not_found' := DroppedResourceNotFound, 'dropped.resource_stopped' := DroppedResourceStopped, 'matched' := Matched, 'queuing' := Queued, 'retried' := Retried, + 'late_reply' := LateReply, 'failed' := SentFailed, 'inflight' := SentInflight, 'success' := SentSucc, diff --git a/apps/emqx_bridge/src/emqx_bridge_api.erl b/apps/emqx_bridge/src/emqx_bridge_api.erl index 50505effc..2c43ce5d7 100644 --- a/apps/emqx_bridge/src/emqx_bridge_api.erl +++ b/apps/emqx_bridge/src/emqx_bridge_api.erl @@ -751,11 +751,11 @@ aggregate_metrics(AllMetrics) -> fun( #{ metrics := ?metrics( - M1, M2, M3, M4, M5, M6, M7, M8, M9, M10, M11, M12, M13, M14, M15 + M1, M2, M3, M4, M5, M6, M7, M8, M9, M10, M11, M12, M13, M14, M15, M16, M17 ) }, ?metrics( - N1, N2, N3, N4, N5, N6, N7, N8, N9, N10, N11, N12, N13, N14, N15 + N1, N2, N3, N4, N5, N6, N7, N8, N9, N10, N11, N12, N13, N14, N15, N16, N17 ) ) -> ?METRICS( @@ -773,7 +773,9 @@ aggregate_metrics(AllMetrics) -> M12 + N12, M13 + N13, M14 + N14, - M15 + N15 + M15 + N15, + M16 + N16, + M17 + N17 ) end, InitMetrics, @@ -805,11 +807,13 @@ format_metrics(#{ counters := #{ 'dropped' := Dropped, 'dropped.other' := DroppedOther, + 'dropped.expired' := DroppedExpired, 'dropped.queue_full' := DroppedQueueFull, 'dropped.resource_not_found' := DroppedResourceNotFound, 'dropped.resource_stopped' := DroppedResourceStopped, 'matched' := Matched, 'retried' := Retried, + 'late_reply' := LateReply, 'failed' := SentFailed, 'success' := SentSucc, 'received' := Rcvd @@ -824,12 +828,14 @@ format_metrics(#{ ?METRICS( Dropped, DroppedOther, + DroppedExpired, DroppedQueueFull, DroppedResourceNotFound, DroppedResourceStopped, Matched, Queued, Retried, + LateReply, SentFailed, SentInflight, SentSucc, diff --git a/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl b/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl index 1b4eac73e..1f5b06fab 100644 --- a/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl +++ b/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl @@ -830,7 +830,8 @@ t_mqtt_conn_bridge_egress_reconnect(_) -> <<"resource_opts">> => #{ <<"worker_pool_size">> => 2, <<"query_mode">> => <<"sync">>, - <<"request_timeout">> => <<"500ms">>, + %% using a long time so we can test recovery + <<"request_timeout">> => <<"15s">>, %% to make it check the healthy quickly <<"health_check_interval">> => <<"0.5s">> } @@ -898,8 +899,10 @@ t_mqtt_conn_bridge_egress_reconnect(_) -> ), Payload1 = <<"hello2">>, Payload2 = <<"hello3">>, - emqx:publish(emqx_message:make(LocalTopic, Payload1)), - emqx:publish(emqx_message:make(LocalTopic, Payload2)), + %% we need to to it in other processes because it'll block due to + %% the long timeout + spawn(fun() -> emqx:publish(emqx_message:make(LocalTopic, Payload1)) end), + spawn(fun() -> emqx:publish(emqx_message:make(LocalTopic, Payload2)) end), {ok, _} = snabbkaffe:receive_events(SRef), %% verify the metrics of the bridge, the message should be queued @@ -917,9 +920,9 @@ t_mqtt_conn_bridge_egress_reconnect(_) -> <<"matched">> := Matched, <<"success">> := 1, <<"failed">> := 0, - <<"queuing">> := 1, - <<"inflight">> := 1 - } when Matched >= 3, + <<"queuing">> := Queuing, + <<"inflight">> := Inflight + } when Matched >= 3 andalso Inflight + Queuing == 2, maps:get(<<"metrics">>, DecodedMetrics1) ), diff --git a/apps/emqx_resource/include/emqx_resource.hrl b/apps/emqx_resource/include/emqx_resource.hrl index d7b080ae8..4f2a4883b 100644 --- a/apps/emqx_resource/include/emqx_resource.hrl +++ b/apps/emqx_resource/include/emqx_resource.hrl @@ -29,6 +29,8 @@ -type query_opts() :: #{ %% The key used for picking a resource worker pick_key => term(), + timeout => timeout(), + expire_at => infinity | integer(), async_reply_fun => reply_fun() }. -type resource_data() :: #{ diff --git a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl index 4b16ed3d5..8b79ce5a8 100644 --- a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl @@ -60,21 +60,23 @@ -define(COLLECT_REQ_LIMIT, 1000). -define(SEND_REQ(FROM, REQUEST), {'$send_req', FROM, REQUEST}). --define(QUERY(FROM, REQUEST, SENT), {query, FROM, REQUEST, SENT}). +-define(QUERY(FROM, REQUEST, SENT, EXPIRE_AT), {query, FROM, REQUEST, SENT, EXPIRE_AT}). -define(REPLY(FROM, REQUEST, SENT, RESULT), {reply, FROM, REQUEST, SENT, RESULT}). -define(EXPAND(RESULT, BATCH), [ ?REPLY(FROM, REQUEST, SENT, RESULT) - || ?QUERY(FROM, REQUEST, SENT) <- BATCH + || ?QUERY(FROM, REQUEST, SENT, _EXPIRE_AT) <- BATCH ]). -define(INFLIGHT_ITEM(Ref, BatchOrQuery, IsRetriable, WorkerMRef), {Ref, BatchOrQuery, IsRetriable, WorkerMRef} ). +-define(ITEM_IDX, 2). -define(RETRY_IDX, 3). -define(WORKER_MREF_IDX, 4). -type id() :: binary(). -type index() :: pos_integer(). --type queue_query() :: ?QUERY(from(), request(), HasBeenSent :: boolean()). +-type expire_at() :: infinity | integer(). +-type queue_query() :: ?QUERY(from(), request(), HasBeenSent :: boolean(), expire_at()). -type request() :: term(). -type from() :: pid() | reply_fun() | request_from(). -type request_from() :: undefined | gen_statem:from(). @@ -98,14 +100,18 @@ start_link(Id, Index, Opts) -> gen_statem:start_link(?MODULE, {Id, Index, Opts}, []). -spec sync_query(id(), request(), query_opts()) -> Result :: term(). -sync_query(Id, Request, Opts) -> +sync_query(Id, Request, Opts0) -> + Opts1 = ensure_timeout_query_opts(Opts0, sync), + Opts = ensure_expire_at(Opts1), PickKey = maps:get(pick_key, Opts, self()), - Timeout = maps:get(timeout, Opts, timer:seconds(15)), + Timeout = maps:get(timeout, Opts), emqx_resource_metrics:matched_inc(Id), pick_call(Id, PickKey, {query, Request, Opts}, Timeout). -spec async_query(id(), request(), query_opts()) -> Result :: term(). -async_query(Id, Request, Opts) -> +async_query(Id, Request, Opts0) -> + Opts1 = ensure_timeout_query_opts(Opts0, async), + Opts = ensure_expire_at(Opts1), PickKey = maps:get(pick_key, Opts, self()), emqx_resource_metrics:matched_inc(Id), pick_cast(Id, PickKey, {query, Request, Opts}). @@ -120,11 +126,15 @@ simple_sync_query(Id, Request) -> %% would mess up the metrics anyway. `undefined' is ignored by %% `emqx_resource_metrics:*_shift/3'. Index = undefined, - QueryOpts = #{simple_query => true}, + QueryOpts0 = #{simple_query => true, timeout => infinity}, + QueryOpts = #{expire_at := ExpireAt} = ensure_expire_at(QueryOpts0), emqx_resource_metrics:matched_inc(Id), Ref = make_message_ref(), - Result = call_query(sync, Id, Index, Ref, ?QUERY(self(), Request, false), QueryOpts), HasBeenSent = false, + From = self(), + Result = call_query( + sync, Id, Index, Ref, ?QUERY(From, Request, HasBeenSent, ExpireAt), QueryOpts + ), _ = handle_query_result(Id, Result, HasBeenSent), Result. @@ -179,9 +189,14 @@ init({Id, Index, Opts}) -> ?tp(buffer_worker_init, #{id => Id, index => Index}), {ok, running, Data}. -running(enter, _, St) -> +running(enter, _, Data) -> ?tp(buffer_worker_enter_running, #{}), - maybe_flush(St); + %% According to `gen_statem' laws, we mustn't call `maybe_flush' + %% directly because it may decide to return `{next_state, blocked, _}', + %% and that's an invalid response for a state enter call. + %% Returning a next event from a state enter call is also + %% prohibited. + {keep_state, ensure_flush_timer(Data, 0)}; running(cast, resume, _St) -> keep_state_and_data; running(cast, flush, Data) -> @@ -243,9 +258,9 @@ code_change(_OldVsn, State, _Extra) -> {ok, State}. %%============================================================================== --define(PICK(ID, KEY, EXPR), +-define(PICK(ID, KEY, PID, EXPR), try gproc_pool:pick_worker(ID, KEY) of - Pid when is_pid(Pid) -> + PID when is_pid(PID) -> EXPR; _ -> ?RESOURCE_ERROR(worker_not_created, "resource not created") @@ -258,7 +273,7 @@ code_change(_OldVsn, State, _Extra) -> ). pick_call(Id, Key, Query, Timeout) -> - ?PICK(Id, Key, begin + ?PICK(Id, Key, Pid, begin Caller = self(), MRef = erlang:monitor(process, Pid, [{alias, reply_demonitor}]), From = {Caller, MRef}, @@ -281,15 +296,21 @@ pick_call(Id, Key, Query, Timeout) -> end). pick_cast(Id, Key, Query) -> - ?PICK(Id, Key, begin + ?PICK(Id, Key, Pid, begin From = undefined, erlang:send(Pid, ?SEND_REQ(From, Query)), ok end). resume_from_blocked(Data) -> - #{inflight_tid := InflightTID} = Data, - case inflight_get_first_retriable(InflightTID) of + ?tp(buffer_worker_resume_from_blocked_enter, #{}), + #{ + id := Id, + index := Index, + inflight_tid := InflightTID + } = Data, + Now = now_(), + case inflight_get_first_retriable(InflightTID, Now) of none -> case is_inflight_full(InflightTID) of true -> @@ -297,14 +318,32 @@ resume_from_blocked(Data) -> false -> {next_state, running, Data} end; - {Ref, FirstQuery} -> + {expired, Ref, Batch} -> + IsAcked = ack_inflight(InflightTID, Ref, Id, Index), + IsAcked andalso emqx_resource_metrics:dropped_expired_inc(Id, length(Batch)), + ?tp(buffer_worker_retry_expired, #{expired => Batch}), + resume_from_blocked(Data); + {single, Ref, Query} -> %% We retry msgs in inflight window sync, as if we send them %% async, they will be appended to the end of inflight window again. case is_inflight_full(InflightTID) of true -> {keep_state, Data}; false -> - retry_inflight_sync(Ref, FirstQuery, Data) + retry_inflight_sync(Ref, Query, Data) + end; + {batch, Ref, NotExpired, Expired} -> + update_inflight_item(InflightTID, Ref, NotExpired), + NumExpired = length(Expired), + emqx_resource_metrics:dropped_expired_inc(Id, NumExpired), + NumExpired > 0 andalso ?tp(buffer_worker_retry_expired, #{expired => Expired}), + %% We retry msgs in inflight window sync, as if we send them + %% async, they will be appended to the end of inflight window again. + case is_inflight_full(InflightTID) of + true -> + {keep_state, Data}; + false -> + retry_inflight_sync(Ref, NotExpired, Data) end end. @@ -320,10 +359,10 @@ retry_inflight_sync(Ref, QueryOrBatch, Data0) -> Result = call_query(sync, Id, Index, Ref, QueryOrBatch, QueryOpts), ReplyResult = case QueryOrBatch of - ?QUERY(From, CoreReq, HasBeenSent) -> + ?QUERY(From, CoreReq, HasBeenSent, _ExpireAt) -> Reply = ?REPLY(From, CoreReq, HasBeenSent, Result), reply_caller_defer_metrics(Id, Reply, QueryOpts); - [?QUERY(_, _, _) | _] = Batch -> + [?QUERY(_, _, _, _) | _] = Batch -> batch_reply_caller_defer_metrics(Id, Result, Batch, QueryOpts) end, case ReplyResult of @@ -378,10 +417,12 @@ collect_and_enqueue_query_requests(Request0, Data0) -> (?SEND_REQ(undefined = _From, {query, Req, Opts})) -> ReplyFun = maps:get(async_reply_fun, Opts, undefined), HasBeenSent = false, - ?QUERY(ReplyFun, Req, HasBeenSent); - (?SEND_REQ(From, {query, Req, _Opts})) -> + ExpireAt = maps:get(expire_at, Opts), + ?QUERY(ReplyFun, Req, HasBeenSent, ExpireAt); + (?SEND_REQ(From, {query, Req, Opts})) -> HasBeenSent = false, - ?QUERY(From, Req, HasBeenSent) + ExpireAt = maps:get(expire_at, Opts), + ?QUERY(From, Req, HasBeenSent, ExpireAt) end, Requests ), @@ -406,6 +447,8 @@ maybe_flush(Data0) -> -spec flush(data()) -> gen_statem:event_handler_result(state(), data()). flush(Data0) -> #{ + id := Id, + index := Index, batch_size := BatchSize, inflight_tid := InflightTID, queue := Q0 @@ -419,25 +462,45 @@ flush(Data0) -> Data2 = ensure_flush_timer(Data1), {keep_state, Data2}; {_, false} -> + ?tp(buffer_worker_flush_before_pop, #{}), {Q1, QAckRef, Batch} = replayq:pop(Q0, #{count_limit => BatchSize}), - IsBatch = BatchSize =/= 1, - %% We *must* use the new queue, because we currently can't - %% `nack' a `pop'. - %% Maybe we could re-open the queue? Data2 = Data1#{queue := Q1}, - Ref = make_message_ref(), - do_flush(Data2, #{ - new_queue => Q1, - is_batch => IsBatch, - batch => Batch, - ref => Ref, - ack_ref => QAckRef - }) + ?tp(buffer_worker_flush_before_sieve_expired, #{}), + Now = now_(), + %% if the request has expired, the caller is no longer + %% waiting for a response. + case sieve_expired_requests(Batch, Now) of + all_expired -> + ok = replayq:ack(Q1, QAckRef), + emqx_resource_metrics:dropped_expired_inc(Id, length(Batch)), + emqx_resource_metrics:queuing_set(Id, Index, queue_count(Q1)), + ?tp(buffer_worker_flush_all_expired, #{batch => Batch}), + flush(Data2); + {NotExpired, Expired} -> + NumExpired = length(Expired), + emqx_resource_metrics:dropped_expired_inc(Id, NumExpired), + IsBatch = BatchSize =/= 1, + %% We *must* use the new queue, because we currently can't + %% `nack' a `pop'. + %% Maybe we could re-open the queue? + ?tp( + buffer_worker_flush_potentially_partial, + #{expired => Expired, not_expired => NotExpired} + ), + Ref = make_message_ref(), + do_flush(Data2, #{ + new_queue => Q1, + is_batch => IsBatch, + batch => NotExpired, + ref => Ref, + ack_ref => QAckRef + }) + end end. -spec do_flush(data(), #{ is_batch := boolean(), - batch := [?QUERY(from(), request(), boolean())], + batch := [queue_query()], ack_ref := replayq:ack_ref(), ref := inflight_key(), new_queue := replayq:q() @@ -459,7 +522,7 @@ do_flush( inflight_tid := InflightTID } = Data0, %% unwrap when not batching (i.e., batch size == 1) - [?QUERY(From, CoreReq, HasBeenSent) = Request] = Batch, + [?QUERY(From, CoreReq, HasBeenSent, _ExpireAt) = Request] = Batch, QueryOpts = #{inflight_tid => InflightTID, simple_query => false}, Result = call_query(configured, Id, Index, Ref, Request, QueryOpts), Reply = ?REPLY(From, CoreReq, HasBeenSent, Result), @@ -812,10 +875,10 @@ call_query(QM0, Id, Index, Ref, Query, QueryOpts) -> end ). -apply_query_fun(sync, Mod, Id, _Index, _Ref, ?QUERY(_, Request, _) = _Query, ResSt, _QueryOpts) -> +apply_query_fun(sync, Mod, Id, _Index, _Ref, ?QUERY(_, Request, _, _) = _Query, ResSt, _QueryOpts) -> ?tp(call_query, #{id => Id, mod => Mod, query => _Query, res_st => ResSt, call_mode => sync}), ?APPLY_RESOURCE(call_query, Mod:on_query(Id, Request, ResSt), Request); -apply_query_fun(async, Mod, Id, Index, Ref, ?QUERY(_, Request, _) = Query, ResSt, QueryOpts) -> +apply_query_fun(async, Mod, Id, Index, Ref, ?QUERY(_, Request, _, _) = Query, ResSt, QueryOpts) -> ?tp(call_query_async, #{ id => Id, mod => Mod, query => Query, res_st => ResSt, call_mode => async }), @@ -834,13 +897,13 @@ apply_query_fun(async, Mod, Id, Index, Ref, ?QUERY(_, Request, _) = Query, ResSt end, Request ); -apply_query_fun(sync, Mod, Id, _Index, _Ref, [?QUERY(_, _, _) | _] = Batch, ResSt, _QueryOpts) -> +apply_query_fun(sync, Mod, Id, _Index, _Ref, [?QUERY(_, _, _, _) | _] = Batch, ResSt, _QueryOpts) -> ?tp(call_batch_query, #{ id => Id, mod => Mod, batch => Batch, res_st => ResSt, call_mode => sync }), - Requests = [Request || ?QUERY(_From, Request, _) <- Batch], + Requests = [Request || ?QUERY(_From, Request, _, _ExpireAt) <- Batch], ?APPLY_RESOURCE(call_batch_query, Mod:on_batch_query(Id, Requests, ResSt), Batch); -apply_query_fun(async, Mod, Id, Index, Ref, [?QUERY(_, _, _) | _] = Batch, ResSt, QueryOpts) -> +apply_query_fun(async, Mod, Id, Index, Ref, [?QUERY(_, _, _, _) | _] = Batch, ResSt, QueryOpts) -> ?tp(call_batch_query_async, #{ id => Id, mod => Mod, batch => Batch, res_st => ResSt, call_mode => async }), @@ -850,7 +913,7 @@ apply_query_fun(async, Mod, Id, Index, Ref, [?QUERY(_, _, _) | _] = Batch, ResSt begin ReplyFun = fun ?MODULE:batch_reply_after_query/8, ReplyFunAndArgs = {ReplyFun, [self(), Id, Index, InflightTID, Ref, Batch, QueryOpts]}, - Requests = [Request || ?QUERY(_From, Request, _) <- Batch], + Requests = [Request || ?QUERY(_From, Request, _, _ExpireAt) <- Batch], IsRetriable = false, WorkerMRef = undefined, InflightItem = ?INFLIGHT_ITEM(Ref, Batch, IsRetriable, WorkerMRef), @@ -862,7 +925,41 @@ apply_query_fun(async, Mod, Id, Index, Ref, [?QUERY(_, _, _) | _] = Batch, ResSt ). reply_after_query( - Pid, Id, Index, InflightTID, Ref, ?QUERY(From, Request, HasBeenSent), QueryOpts, Result + Pid, + Id, + Index, + InflightTID, + Ref, + ?QUERY(_From, _Request, _HasBeenSent, ExpireAt) = Query, + QueryOpts, + Result +) -> + ?tp( + buffer_worker_reply_after_query_enter, + #{batch_or_query => [Query], ref => Ref} + ), + Now = now_(), + case is_expired(ExpireAt, Now) of + true -> + IsFullBefore = is_inflight_full(InflightTID), + IsAcked = ack_inflight(InflightTID, Ref, Id, Index), + IsAcked andalso emqx_resource_metrics:late_reply_inc(Id), + IsFullBefore andalso ?MODULE:flush_worker(Pid), + ?tp(buffer_worker_reply_after_query_expired, #{expired => [Query]}), + ok; + false -> + do_reply_after_query(Pid, Id, Index, InflightTID, Ref, Query, QueryOpts, Result) + end. + +do_reply_after_query( + Pid, + Id, + Index, + InflightTID, + Ref, + ?QUERY(From, Request, HasBeenSent, _ExpireAt), + QueryOpts, + Result ) -> %% NOTE: 'inflight' is the count of messages that were sent async %% but received no ACK, NOT the number of messages queued in the @@ -875,7 +972,7 @@ reply_after_query( %% Keep retrying. ?tp(buffer_worker_reply_after_query, #{ action => Action, - batch_or_query => ?QUERY(From, Request, HasBeenSent), + batch_or_query => ?QUERY(From, Request, HasBeenSent, _ExpireAt), ref => Ref, result => Result }), @@ -884,7 +981,7 @@ reply_after_query( ack -> ?tp(buffer_worker_reply_after_query, #{ action => Action, - batch_or_query => ?QUERY(From, Request, HasBeenSent), + batch_or_query => ?QUERY(From, Request, HasBeenSent, _ExpireAt), ref => Ref, result => Result }), @@ -896,6 +993,34 @@ reply_after_query( end. batch_reply_after_query(Pid, Id, Index, InflightTID, Ref, Batch, QueryOpts, Result) -> + ?tp( + buffer_worker_reply_after_query_enter, + #{batch_or_query => Batch, ref => Ref} + ), + Now = now_(), + case sieve_expired_requests(Batch, Now) of + all_expired -> + IsFullBefore = is_inflight_full(InflightTID), + IsAcked = ack_inflight(InflightTID, Ref, Id, Index), + IsAcked andalso emqx_resource_metrics:late_reply_inc(Id), + IsFullBefore andalso ?MODULE:flush_worker(Pid), + ?tp(buffer_worker_reply_after_query_expired, #{expired => Batch}), + ok; + {NotExpired, Expired} -> + NumExpired = length(Expired), + emqx_resource_metrics:late_reply_inc(Id, NumExpired), + NumExpired > 0 andalso + ?tp(buffer_worker_reply_after_query_expired, #{expired => Expired}), + do_batch_reply_after_query( + Pid, Id, Index, InflightTID, Ref, NotExpired, QueryOpts, Result + ) + end. + +do_batch_reply_after_query(Pid, Id, Index, InflightTID, Ref, Batch, QueryOpts, Result) -> + ?tp( + buffer_worker_reply_after_query_enter, + #{batch_or_query => Batch, ref => Ref} + ), %% NOTE: 'inflight' is the count of messages that were sent async %% but received no ACK, NOT the number of messages queued in the %% inflight window. @@ -986,9 +1111,12 @@ inflight_new(InfltWinSZ, Id, Index) -> ), TableId. --spec inflight_get_first_retriable(ets:tid()) -> - none | {integer(), [?QUERY(_, _, _)] | ?QUERY(_, _, _)}. -inflight_get_first_retriable(InflightTID) -> +-spec inflight_get_first_retriable(ets:tid(), integer()) -> + none + | {expired, inflight_key(), [queue_query()]} + | {single, inflight_key(), queue_query()} + | {batch, inflight_key(), _NotExpired :: [queue_query()], _Expired :: [queue_query()]}. +inflight_get_first_retriable(InflightTID, Now) -> MatchSpec = ets:fun2ms( fun(?INFLIGHT_ITEM(Ref, BatchOrQuery, IsRetriable, _WorkerMRef)) when @@ -1000,8 +1128,22 @@ inflight_get_first_retriable(InflightTID) -> case ets:select(InflightTID, MatchSpec, _Limit = 1) of '$end_of_table' -> none; - {[{Ref, BatchOrQuery}], _Continuation} -> - {Ref, BatchOrQuery} + {[{Ref, Query = ?QUERY(_From, _CoreReq, _HasBeenSent, ExpireAt)}], _Continuation} -> + case is_expired(ExpireAt, Now) of + true -> + {expired, Ref, [Query]}; + false -> + {single, Ref, Query} + end; + {[{Ref, Batch = [_ | _]}], _Continuation} -> + %% batch is non-empty because we check that in + %% `sieve_expired_requests'. + case sieve_expired_requests(Batch, Now) of + all_expired -> + {expired, Ref, Batch}; + {NotExpired, Expired} -> + {batch, Ref, NotExpired, Expired} + end end. is_inflight_full(undefined) -> @@ -1030,7 +1172,7 @@ inflight_append(undefined, _InflightItem, _Id, _Index) -> ok; inflight_append( InflightTID, - ?INFLIGHT_ITEM(Ref, [?QUERY(_, _, _) | _] = Batch0, IsRetriable, WorkerMRef), + ?INFLIGHT_ITEM(Ref, [?QUERY(_, _, _, _) | _] = Batch0, IsRetriable, WorkerMRef), Id, Index ) -> @@ -1044,7 +1186,9 @@ inflight_append( ok; inflight_append( InflightTID, - ?INFLIGHT_ITEM(Ref, ?QUERY(_From, _Req, _HasBeenSent) = Query0, IsRetriable, WorkerMRef), + ?INFLIGHT_ITEM( + Ref, ?QUERY(_From, _Req, _HasBeenSent, _ExpireAt) = Query0, IsRetriable, WorkerMRef + ), Id, Index ) -> @@ -1106,9 +1250,9 @@ ack_inflight(undefined, _Ref, _Id, _Index) -> ack_inflight(InflightTID, Ref, Id, Index) -> Count = case ets:take(InflightTID, Ref) of - [?INFLIGHT_ITEM(Ref, ?QUERY(_, _, _), _IsRetriable, _WorkerMRef)] -> + [?INFLIGHT_ITEM(Ref, ?QUERY(_, _, _, _), _IsRetriable, _WorkerMRef)] -> 1; - [?INFLIGHT_ITEM(Ref, [?QUERY(_, _, _) | _] = Batch, _IsRetriable, _WorkerMRef)] -> + [?INFLIGHT_ITEM(Ref, [?QUERY(_, _, _, _) | _] = Batch, _IsRetriable, _WorkerMRef)] -> length(Batch); _ -> 0 @@ -1133,6 +1277,12 @@ mark_inflight_items_as_retriable(Data, WorkerMRef) -> ?tp(buffer_worker_worker_down_update, #{num_affected => _NumAffected}), ok. +%% used to update a batch after dropping expired individual queries. +update_inflight_item(InflightTID, Ref, NewBatch) -> + _ = ets:update_element(InflightTID, Ref, {?ITEM_IDX, NewBatch}), + ?tp(buffer_worker_worker_update_inflight_item, #{ref => Ref}), + ok. + %%============================================================================== inc_sent_failed(Id, _HasBeenSent = true) -> @@ -1180,11 +1330,14 @@ clear_disk_queue_dir(Id, Index) -> Res end. -ensure_flush_timer(Data = #{tref := undefined, batch_time := T}) -> +ensure_flush_timer(Data = #{batch_time := T}) -> + ensure_flush_timer(Data, T). + +ensure_flush_timer(Data = #{tref := undefined}, T) -> Ref = make_ref(), TRef = erlang:send_after(T, self(), {flush, Ref}), Data#{tref => {TRef, Ref}}; -ensure_flush_timer(Data) -> +ensure_flush_timer(Data, _T) -> Data. cancel_flush_timer(St = #{tref := undefined}) -> @@ -1195,7 +1348,7 @@ cancel_flush_timer(St = #{tref := {TRef, _Ref}}) -> -spec make_message_ref() -> inflight_key(). make_message_ref() -> - erlang:monotonic_time(nanosecond). + now_(). collect_requests(Acc, Limit) -> Count = length(Acc), @@ -1213,9 +1366,9 @@ do_collect_requests(Acc, Count, Limit) -> mark_as_sent(Batch) when is_list(Batch) -> lists:map(fun mark_as_sent/1, Batch); -mark_as_sent(?QUERY(From, Req, _)) -> +mark_as_sent(?QUERY(From, Req, _HasBeenSent, ExpireAt)) -> HasBeenSent = true, - ?QUERY(From, Req, HasBeenSent). + ?QUERY(From, Req, HasBeenSent, ExpireAt). is_unrecoverable_error({error, {unrecoverable_error, _}}) -> true; @@ -1235,3 +1388,49 @@ is_async_return({async_return, _}) -> true; is_async_return(_) -> false. + +sieve_expired_requests(Batch, Now) -> + {Expired, NotExpired} = + lists:partition( + fun(?QUERY(_From, _CoreReq, _HasBeenSent, ExpireAt)) -> + is_expired(ExpireAt, Now) + end, + Batch + ), + case {NotExpired, Expired} of + {[], []} -> + %% Should be impossible for batch_size >= 1. + all_expired; + {[], [_ | _]} -> + all_expired; + {[_ | _], _} -> + {NotExpired, Expired} + end. + +-spec is_expired(infinity | integer(), integer()) -> boolean(). +is_expired(infinity = _ExpireAt, _Now) -> + false; +is_expired(ExpireAt, Now) -> + Now > ExpireAt. + +now_() -> + erlang:monotonic_time(nanosecond). + +-spec ensure_timeout_query_opts(query_opts(), sync | async) -> query_opts(). +ensure_timeout_query_opts(#{timeout := _} = Opts, _SyncOrAsync) -> + Opts; +ensure_timeout_query_opts(#{} = Opts0, sync) -> + TimeoutMS = timer:seconds(15), + Opts0#{timeout => TimeoutMS}; +ensure_timeout_query_opts(#{} = Opts0, async) -> + Opts0#{timeout => infinity}. + +-spec ensure_expire_at(query_opts()) -> query_opts(). +ensure_expire_at(#{expire_at := _} = Opts) -> + Opts; +ensure_expire_at(#{timeout := infinity} = Opts) -> + Opts#{expire_at => infinity}; +ensure_expire_at(#{timeout := TimeoutMS} = Opts) -> + TimeoutNS = erlang:convert_time_unit(TimeoutMS, millisecond, nanosecond), + ExpireAt = now_() + TimeoutNS, + Opts#{expire_at => ExpireAt}. diff --git a/apps/emqx_resource/src/emqx_resource_manager.erl b/apps/emqx_resource/src/emqx_resource_manager.erl index 95d1ed1d2..c01158b0a 100644 --- a/apps/emqx_resource/src/emqx_resource_manager.erl +++ b/apps/emqx_resource/src/emqx_resource_manager.erl @@ -134,8 +134,10 @@ create(MgrId, ResId, Group, ResourceType, Config, Opts) -> 'retried.success', 'retried.failed', 'success', + 'late_reply', 'failed', 'dropped', + 'dropped.expired', 'dropped.queue_full', 'dropped.resource_not_found', 'dropped.resource_stopped', diff --git a/apps/emqx_resource/src/emqx_resource_metrics.erl b/apps/emqx_resource/src/emqx_resource_metrics.erl index 455be9c22..28507e291 100644 --- a/apps/emqx_resource/src/emqx_resource_metrics.erl +++ b/apps/emqx_resource/src/emqx_resource_metrics.erl @@ -34,6 +34,9 @@ dropped_other_inc/1, dropped_other_inc/2, dropped_other_get/1, + dropped_expired_inc/1, + dropped_expired_inc/2, + dropped_expired_get/1, dropped_queue_full_inc/1, dropped_queue_full_inc/2, dropped_queue_full_get/1, @@ -46,6 +49,9 @@ failed_inc/1, failed_inc/2, failed_get/1, + late_reply_inc/1, + late_reply_inc/2, + late_reply_get/1, matched_inc/1, matched_inc/2, matched_get/1, @@ -75,9 +81,11 @@ events() -> [?TELEMETRY_PREFIX, Event] || Event <- [ dropped_other, + dropped_expired, dropped_queue_full, dropped_resource_not_found, dropped_resource_stopped, + late_reply, failed, inflight, matched, @@ -114,6 +122,9 @@ handle_telemetry_event( dropped_other -> emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped', Val), emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped.other', Val); + dropped_expired -> + emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped', Val), + emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped.expired', Val); dropped_queue_full -> emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped', Val), emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped.queue_full', Val); @@ -123,6 +134,8 @@ handle_telemetry_event( dropped_resource_stopped -> emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped', Val), emqx_metrics_worker:inc(?RES_METRICS, ID, 'dropped.resource_stopped', Val); + late_reply -> + emqx_metrics_worker:inc(?RES_METRICS, ID, 'late_reply', Val); failed -> emqx_metrics_worker:inc(?RES_METRICS, ID, 'failed', Val); matched -> @@ -211,6 +224,30 @@ dropped_other_inc(ID, Val) -> dropped_other_get(ID) -> emqx_metrics_worker:get(?RES_METRICS, ID, 'dropped.other'). +%% @doc Count of messages dropped due to being expired before being sent. +dropped_expired_inc(ID) -> + dropped_expired_inc(ID, 1). + +dropped_expired_inc(ID, Val) -> + telemetry:execute([?TELEMETRY_PREFIX, dropped_expired], #{counter_inc => Val}, #{ + resource_id => ID + }). + +dropped_expired_get(ID) -> + emqx_metrics_worker:get(?RES_METRICS, ID, 'dropped.expired'). + +%% @doc Count of messages that were sent but received a late reply. +late_reply_inc(ID) -> + late_reply_inc(ID, 1). + +late_reply_inc(ID, Val) -> + telemetry:execute([?TELEMETRY_PREFIX, late_reply], #{counter_inc => Val}, #{ + resource_id => ID + }). + +late_reply_get(ID) -> + emqx_metrics_worker:get(?RES_METRICS, ID, 'late_reply'). + %% @doc Count of messages dropped because the queue was full dropped_queue_full_inc(ID) -> dropped_queue_full_inc(ID, 1). diff --git a/apps/emqx_resource/test/emqx_connector_demo.erl b/apps/emqx_resource/test/emqx_connector_demo.erl index c2b0c5733..4e3423808 100644 --- a/apps/emqx_resource/test/emqx_connector_demo.erl +++ b/apps/emqx_resource/test/emqx_connector_demo.erl @@ -260,7 +260,7 @@ counter_loop( ?tp(connector_demo_inc_counter_async, #{n => N}), State#{counter => Num + N}; {big_payload, _Payload, ReplyFun} when Status == blocked -> - apply_reply(ReplyFun, {error, blocked}), + apply_reply(ReplyFun, {error, {recoverable_error, blocked}}), State; {{FromPid, ReqRef}, {inc, N}} when Status == running -> %ct:pal("sync counter recv: ~p", [{inc, N}]), diff --git a/apps/emqx_resource/test/emqx_resource_SUITE.erl b/apps/emqx_resource/test/emqx_resource_SUITE.erl index c9325af2b..9b2af74f6 100644 --- a/apps/emqx_resource/test/emqx_resource_SUITE.erl +++ b/apps/emqx_resource/test/emqx_resource_SUITE.erl @@ -232,7 +232,7 @@ t_batch_query_counter(_) -> fun(Result, Trace) -> ?assertMatch({ok, 0}, Result), QueryTrace = ?of_kind(call_batch_query, Trace), - ?assertMatch([#{batch := [{query, _, get_counter, _}]}], QueryTrace) + ?assertMatch([#{batch := [{query, _, get_counter, _, _}]}], QueryTrace) end ), @@ -284,7 +284,7 @@ t_query_counter_async_query(_) -> fun(Trace) -> %% the callback_mode of 'emqx_connector_demo' is 'always_sync'. QueryTrace = ?of_kind(call_query, Trace), - ?assertMatch([#{query := {query, _, {inc_counter, 1}, _}} | _], QueryTrace) + ?assertMatch([#{query := {query, _, {inc_counter, 1}, _, _}} | _], QueryTrace) end ), %% simple query ignores the query_mode and batching settings in the resource_worker @@ -295,7 +295,7 @@ t_query_counter_async_query(_) -> ?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) + ?assertMatch([#{query := {query, _, get_counter, _, _}}], QueryTrace) end ), {ok, _, #{metrics := #{counters := C}}} = emqx_resource:get_instance(?ID), @@ -337,7 +337,7 @@ t_query_counter_async_callback(_) -> end, fun(Trace) -> QueryTrace = ?of_kind(call_query_async, Trace), - ?assertMatch([#{query := {query, _, {inc_counter, 1}, _}} | _], QueryTrace) + ?assertMatch([#{query := {query, _, {inc_counter, 1}, _, _}} | _], QueryTrace) end ), @@ -348,7 +348,7 @@ t_query_counter_async_callback(_) -> fun(Result, Trace) -> ?assertMatch({ok, 1000}, Result), QueryTrace = ?of_kind(call_query, Trace), - ?assertMatch([#{query := {query, _, get_counter, _}}], QueryTrace) + ?assertMatch([#{query := {query, _, get_counter, _, _}}], QueryTrace) end ), {ok, _, #{metrics := #{counters := C}}} = emqx_resource:get_instance(?ID), @@ -419,7 +419,7 @@ t_query_counter_async_inflight(_) -> ), fun(Trace) -> QueryTrace = ?of_kind(call_query_async, Trace), - ?assertMatch([#{query := {query, _, {inc_counter, 1}, _}} | _], QueryTrace) + ?assertMatch([#{query := {query, _, {inc_counter, 1}, _, _}} | _], QueryTrace) end ), tap_metrics(?LINE), @@ -476,7 +476,7 @@ t_query_counter_async_inflight(_) -> end, fun(Trace) -> QueryTrace = ?of_kind(call_query_async, Trace), - ?assertMatch([#{query := {query, _, {inc_counter, _}, _}} | _], QueryTrace), + ?assertMatch([#{query := {query, _, {inc_counter, _}, _, _}} | _], QueryTrace), ?assertEqual(WindowSize + Num, ets:info(Tab0, size), #{tab => ets:tab2list(Tab0)}), tap_metrics(?LINE), ok @@ -495,7 +495,7 @@ t_query_counter_async_inflight(_) -> ), fun(Trace) -> QueryTrace = ?of_kind(call_query_async, Trace), - ?assertMatch([#{query := {query, _, {inc_counter, 1}, _}} | _], QueryTrace) + ?assertMatch([#{query := {query, _, {inc_counter, 1}, _, _}} | _], QueryTrace) end ), @@ -605,8 +605,8 @@ t_query_counter_async_inflight_batch(_) -> [ #{ batch := [ - {query, _, {inc_counter, 1}, _}, - {query, _, {inc_counter, 1}, _} + {query, _, {inc_counter, 1}, _, _}, + {query, _, {inc_counter, 1}, _, _} ] } | _ @@ -687,7 +687,7 @@ t_query_counter_async_inflight_batch(_) -> fun(Trace) -> QueryTrace = ?of_kind(call_batch_query_async, Trace), ?assertMatch( - [#{batch := [{query, _, {inc_counter, _}, _} | _]} | _], + [#{batch := [{query, _, {inc_counter, _}, _, _} | _]} | _], QueryTrace ) end @@ -712,7 +712,7 @@ t_query_counter_async_inflight_batch(_) -> fun(Trace) -> QueryTrace = ?of_kind(call_batch_query_async, Trace), ?assertMatch( - [#{batch := [{query, _, {inc_counter, _}, _} | _]} | _], + [#{batch := [{query, _, {inc_counter, _}, _, _} | _]} | _], QueryTrace ) end @@ -1499,9 +1499,680 @@ t_async_pool_worker_death(_Config) -> ), ok. +t_expiration_sync_before_sending(_Config) -> + emqx_connector_demo:set_callback_mode(always_sync), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => sync, + batch_size => 1, + worker_pool_size => 1, + resume_interval => 1_000 + } + ), + do_t_expiration_before_sending(sync). + +t_expiration_sync_batch_before_sending(_Config) -> + emqx_connector_demo:set_callback_mode(always_sync), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => sync, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 1_000 + } + ), + do_t_expiration_before_sending(sync). + +t_expiration_async_before_sending(_Config) -> + emqx_connector_demo:set_callback_mode(async_if_possible), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => async, + batch_size => 1, + worker_pool_size => 1, + resume_interval => 1_000 + } + ), + do_t_expiration_before_sending(async). + +t_expiration_async_batch_before_sending(_Config) -> + emqx_connector_demo:set_callback_mode(async_if_possible), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => async, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 1_000 + } + ), + do_t_expiration_before_sending(async). + +do_t_expiration_before_sending(QueryMode) -> + ?check_trace( + begin + ok = emqx_resource:simple_sync_query(?ID, block), + + ?force_ordering( + #{?snk_kind := buffer_worker_flush_before_pop}, + #{?snk_kind := delay_enter} + ), + ?force_ordering( + #{?snk_kind := delay}, + #{?snk_kind := buffer_worker_flush_before_sieve_expired} + ), + + TimeoutMS = 100, + spawn_link(fun() -> + case QueryMode of + sync -> + ?assertError( + timeout, + emqx_resource:query(?ID, {inc_counter, 99}, #{timeout => TimeoutMS}) + ); + async -> + ?assertEqual( + ok, emqx_resource:query(?ID, {inc_counter, 99}, #{timeout => TimeoutMS}) + ) + end + end), + spawn_link(fun() -> + ?tp(delay_enter, #{}), + ct:sleep(2 * TimeoutMS), + ?tp(delay, #{}), + ok + end), + + {ok, _} = ?block_until(#{?snk_kind := buffer_worker_flush_all_expired}, 4 * TimeoutMS), + ok + end, + fun(Trace) -> + ?assertMatch( + [#{batch := [{query, _, {inc_counter, 99}, _, _}]}], + ?of_kind(buffer_worker_flush_all_expired, Trace) + ), + Metrics = tap_metrics(?LINE), + ?assertMatch( + #{ + counters := #{ + matched := 2, + %% the block call + success := 1, + dropped := 1, + 'dropped.expired' := 1, + retried := 0, + failed := 0 + } + }, + Metrics + ), + ok + end + ), + ok. + +t_expiration_sync_before_sending_partial_batch(_Config) -> + emqx_connector_demo:set_callback_mode(always_sync), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => sync, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 1_000 + } + ), + install_telemetry_handler(?FUNCTION_NAME), + do_t_expiration_before_sending_partial_batch(sync). + +t_expiration_async_before_sending_partial_batch(_Config) -> + emqx_connector_demo:set_callback_mode(async_if_possible), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => async, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 1_000 + } + ), + install_telemetry_handler(?FUNCTION_NAME), + do_t_expiration_before_sending_partial_batch(async). + +do_t_expiration_before_sending_partial_batch(QueryMode) -> + ?check_trace( + begin + ok = emqx_resource:simple_sync_query(?ID, block), + + ?force_ordering( + #{?snk_kind := buffer_worker_flush_before_pop}, + #{?snk_kind := delay_enter} + ), + ?force_ordering( + #{?snk_kind := delay}, + #{?snk_kind := buffer_worker_flush_before_sieve_expired} + ), + + Pid0 = + spawn_link(fun() -> + ?assertEqual( + ok, emqx_resource:query(?ID, {inc_counter, 99}, #{timeout => infinity}) + ), + ?tp(infinity_query_returned, #{}) + end), + TimeoutMS = 100, + Pid1 = + spawn_link(fun() -> + case QueryMode of + sync -> + ?assertError( + timeout, + emqx_resource:query(?ID, {inc_counter, 199}, #{timeout => TimeoutMS}) + ); + async -> + ?assertEqual( + ok, + emqx_resource:query(?ID, {inc_counter, 199}, #{timeout => TimeoutMS}) + ) + end + end), + Pid2 = + spawn_link(fun() -> + ?tp(delay_enter, #{}), + ct:sleep(2 * TimeoutMS), + ?tp(delay, #{}), + ok + end), + + {ok, _} = ?block_until( + #{?snk_kind := buffer_worker_flush_potentially_partial}, 4 * TimeoutMS + ), + ok = emqx_resource:simple_sync_query(?ID, resume), + case QueryMode of + async -> + {ok, _} = ?block_until( + #{ + ?snk_kind := buffer_worker_reply_after_query, + action := ack, + batch_or_query := [{query, _, {inc_counter, 99}, _, _}] + }, + 10 * TimeoutMS + ); + sync -> + %% more time because it needs to retry if sync + {ok, _} = ?block_until(#{?snk_kind := infinity_query_returned}, 20 * TimeoutMS) + end, + + lists:foreach( + fun(Pid) -> + unlink(Pid), + exit(Pid, kill) + end, + [Pid0, Pid1, Pid2] + ), + ok + end, + fun(Trace) -> + ?assertMatch( + [ + #{ + expired := [{query, _, {inc_counter, 199}, _, _}], + not_expired := [{query, _, {inc_counter, 99}, _, _}] + } + ], + ?of_kind(buffer_worker_flush_potentially_partial, Trace) + ), + wait_until_gauge_is(inflight, 0, 500), + Metrics = tap_metrics(?LINE), + case QueryMode of + async -> + ?assertMatch( + #{ + counters := #{ + matched := 4, + %% the block call, the request with + %% infinity timeout, and the resume + %% call. + success := 3, + dropped := 1, + 'dropped.expired' := 1, + %% was sent successfully and held by + %% the test connector. + retried := 0, + failed := 0 + } + }, + Metrics + ); + sync -> + ?assertMatch( + #{ + counters := #{ + matched := 4, + %% the block call, the request with + %% infinity timeout, and the resume + %% call. + success := 3, + dropped := 1, + 'dropped.expired' := 1, + %% currently, the test connector + %% replies with an error that may make + %% the buffer worker retry. + retried := Retried, + failed := 0 + } + } when Retried =< 1, + Metrics + ) + end, + ok + end + ), + ok. + +t_expiration_async_after_reply(_Config) -> + emqx_connector_demo:set_callback_mode(async_if_possible), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => async, + batch_size => 1, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 1_000 + } + ), + install_telemetry_handler(?FUNCTION_NAME), + do_t_expiration_async_after_reply(single). + +t_expiration_async_batch_after_reply(_Config) -> + emqx_connector_demo:set_callback_mode(async_if_possible), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => async, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 2_000 + } + ), + install_telemetry_handler(?FUNCTION_NAME), + do_t_expiration_async_after_reply(batch). + +do_t_expiration_async_after_reply(IsBatch) -> + ?check_trace( + begin + NAcks = + case IsBatch of + batch -> 1; + single -> 2 + end, + ?force_ordering( + #{?snk_kind := buffer_worker_flush_ack}, + NAcks, + #{?snk_kind := delay_enter}, + _Guard = true + ), + ?force_ordering( + #{?snk_kind := delay}, + #{ + ?snk_kind := buffer_worker_reply_after_query_enter, + batch_or_query := [{query, _, {inc_counter, 199}, _, _} | _] + } + ), + + TimeoutMS = 100, + ?assertEqual( + ok, + emqx_resource:query(?ID, {inc_counter, 199}, #{timeout => TimeoutMS}) + ), + ?assertEqual( + ok, emqx_resource:query(?ID, {inc_counter, 99}, #{timeout => infinity}) + ), + Pid0 = + spawn_link(fun() -> + ?tp(delay_enter, #{}), + ct:sleep(2 * TimeoutMS), + ?tp(delay, #{}), + ok + end), + + {ok, _} = ?block_until( + #{?snk_kind := buffer_worker_flush_potentially_partial}, 4 * TimeoutMS + ), + {ok, _} = ?block_until( + #{?snk_kind := buffer_worker_reply_after_query_expired}, 10 * TimeoutMS + ), + + unlink(Pid0), + exit(Pid0, kill), + ok + end, + fun(Trace) -> + ?assertMatch( + [ + #{ + expired := [{query, _, {inc_counter, 199}, _, _}] + } + ], + ?of_kind(buffer_worker_reply_after_query_expired, Trace) + ), + wait_telemetry_event(success, #{n_events => 1, timeout => 4_000}), + Metrics = tap_metrics(?LINE), + ?assertMatch( + #{ + counters := #{ + matched := 2, + %% the request with infinity timeout. + success := 1, + dropped := 0, + late_reply := 1, + retried := 0, + failed := 0 + } + }, + Metrics + ), + ok + end + ), + ok. + +t_expiration_batch_all_expired_after_reply(_Config) -> + ResumeInterval = 300, + emqx_connector_demo:set_callback_mode(async_if_possible), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => async, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => ResumeInterval + } + ), + ?check_trace( + begin + ?force_ordering( + #{?snk_kind := buffer_worker_flush_ack}, + #{?snk_kind := delay_enter} + ), + ?force_ordering( + #{?snk_kind := delay}, + #{ + ?snk_kind := buffer_worker_reply_after_query_enter, + batch_or_query := [{query, _, {inc_counter, 199}, _, _} | _] + } + ), + + TimeoutMS = 200, + ?assertEqual( + ok, + emqx_resource:query(?ID, {inc_counter, 199}, #{timeout => TimeoutMS}) + ), + Pid0 = + spawn_link(fun() -> + ?tp(delay_enter, #{}), + ct:sleep(2 * TimeoutMS), + ?tp(delay, #{}), + ok + end), + + {ok, _} = ?block_until( + #{?snk_kind := buffer_worker_reply_after_query_expired}, 10 * TimeoutMS + ), + + unlink(Pid0), + exit(Pid0, kill), + ok + end, + fun(Trace) -> + ?assertMatch( + [ + #{ + expired := [{query, _, {inc_counter, 199}, _, _}] + } + ], + ?of_kind(buffer_worker_reply_after_query_expired, Trace) + ), + Metrics = tap_metrics(?LINE), + ?assertMatch( + #{ + counters := #{ + matched := 1, + success := 0, + dropped := 0, + late_reply := 1, + retried := 0, + failed := 0 + } + }, + Metrics + ), + ok + end + ), + ok. + +t_expiration_retry(_Config) -> + emqx_connector_demo:set_callback_mode(always_sync), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => sync, + batch_size => 1, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 300 + } + ), + do_t_expiration_retry(single). + +t_expiration_retry_batch(_Config) -> + emqx_connector_demo:set_callback_mode(always_sync), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => sync, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => 300 + } + ), + do_t_expiration_retry(batch). + +do_t_expiration_retry(IsBatch) -> + ResumeInterval = 300, + ?check_trace( + begin + ok = emqx_resource:simple_sync_query(?ID, block), + + {ok, SRef0} = snabbkaffe:subscribe( + ?match_event(#{?snk_kind := buffer_worker_flush_nack}), + 1, + 200 + ), + TimeoutMS = 100, + %% the request that expires must be first, so it's the + %% head of the inflight table (and retriable). + {ok, SRef1} = snabbkaffe:subscribe( + ?match_event(#{?snk_kind := buffer_worker_appended_to_queue}), + 1, + ResumeInterval * 2 + ), + spawn_link(fun() -> + ?assertError( + timeout, + emqx_resource:query( + ?ID, + {inc_counter, 1}, + #{timeout => TimeoutMS} + ) + ) + end), + Pid1 = + spawn_link(fun() -> + receive + go -> ok + end, + ?assertEqual( + ok, + emqx_resource:query( + ?ID, + {inc_counter, 2}, + #{timeout => infinity} + ) + ) + end), + {ok, _} = snabbkaffe:receive_events(SRef1), + Pid1 ! go, + {ok, _} = snabbkaffe:receive_events(SRef0), + + {ok, _} = + ?block_until( + #{?snk_kind := buffer_worker_retry_expired}, + ResumeInterval * 10 + ), + + SuccessEventKind = + case IsBatch of + batch -> buffer_worker_retry_inflight_succeeded; + single -> buffer_worker_flush_ack + end, + {ok, {ok, _}} = + ?wait_async_action( + emqx_resource:simple_sync_query(?ID, resume), + #{?snk_kind := SuccessEventKind}, + ResumeInterval * 5 + ), + + ok + end, + fun(Trace) -> + ?assertMatch( + [#{expired := [{query, _, {inc_counter, 1}, _, _}]}], + ?of_kind(buffer_worker_retry_expired, Trace) + ), + ok + end + ), + ok. + +t_expiration_retry_batch_multiple_times(_Config) -> + ResumeInterval = 300, + emqx_connector_demo:set_callback_mode(always_sync), + {ok, _} = emqx_resource:create( + ?ID, + ?DEFAULT_RESOURCE_GROUP, + ?TEST_RESOURCE, + #{name => test_resource}, + #{ + query_mode => sync, + batch_size => 2, + batch_time => 100, + worker_pool_size => 1, + resume_interval => ResumeInterval + } + ), + ?check_trace( + begin + ok = emqx_resource:simple_sync_query(?ID, block), + + {ok, SRef} = snabbkaffe:subscribe( + ?match_event(#{?snk_kind := buffer_worker_flush_nack}), + 1, + 200 + ), + TimeoutMS = 100, + spawn_link(fun() -> + ?assertError( + timeout, + emqx_resource:query( + ?ID, + {inc_counter, 1}, + #{timeout => TimeoutMS} + ) + ) + end), + spawn_link(fun() -> + ?assertError( + timeout, + emqx_resource:query( + ?ID, + {inc_counter, 2}, + #{timeout => ResumeInterval + TimeoutMS} + ) + ) + end), + {ok, _} = snabbkaffe:receive_events(SRef), + + {ok, _} = + snabbkaffe:block_until( + ?match_n_events(2, #{?snk_kind := buffer_worker_retry_expired}), + ResumeInterval * 10 + ), + + ok + end, + fun(Trace) -> + ?assertMatch( + [ + #{expired := [{query, _, {inc_counter, 1}, _, _}]}, + #{expired := [{query, _, {inc_counter, 2}, _, _}]} + ], + ?of_kind(buffer_worker_retry_expired, Trace) + ), + ok + end + ), + ok. + %%------------------------------------------------------------------------------ %% Helpers %%------------------------------------------------------------------------------ + inc_counter_in_parallel(N) -> inc_counter_in_parallel(N, #{}). @@ -1564,6 +2235,81 @@ tap_metrics(Line) -> ct:pal("metrics (l. ~b): ~p", [Line, #{counters => C, gauges => G}]), #{counters => C, gauges => G}. +install_telemetry_handler(TestCase) -> + Tid = ets:new(TestCase, [ordered_set, public]), + HandlerId = TestCase, + TestPid = self(), + _ = telemetry:attach_many( + HandlerId, + emqx_resource_metrics:events(), + fun(EventName, Measurements, Metadata, _Config) -> + Data = #{ + name => EventName, + measurements => Measurements, + metadata => Metadata + }, + ets:insert(Tid, {erlang:monotonic_time(), Data}), + TestPid ! {telemetry, Data}, + ok + end, + unused_config + ), + on_exit(fun() -> + telemetry:detach(HandlerId), + ets:delete(Tid) + end), + put({?MODULE, telemetry_table}, Tid), + Tid. + +wait_until_gauge_is(GaugeName, ExpectedValue, Timeout) -> + Events = receive_all_events(GaugeName, Timeout), + case length(Events) > 0 andalso lists:last(Events) of + #{measurements := #{gauge_set := ExpectedValue}} -> + ok; + #{measurements := #{gauge_set := Value}} -> + ct:fail( + "gauge ~p didn't reach expected value ~p; last value: ~p", + [GaugeName, ExpectedValue, Value] + ); + false -> + ct:pal("no ~p gauge events received!", [GaugeName]) + end. + +receive_all_events(EventName, Timeout) -> + receive_all_events(EventName, Timeout, []). + +receive_all_events(EventName, Timeout, Acc) -> + receive + {telemetry, #{name := [_, _, EventName]} = Event} -> + receive_all_events(EventName, Timeout, [Event | Acc]) + after Timeout -> + lists:reverse(Acc) + end. + +wait_telemetry_event(EventName) -> + wait_telemetry_event(EventName, #{timeout => 5_000, n_events => 1}). + +wait_telemetry_event( + EventName, + Opts0 +) -> + DefaultOpts = #{timeout => 5_000, n_events => 1}, + #{timeout := Timeout, n_events := NEvents} = maps:merge(DefaultOpts, Opts0), + wait_n_events(NEvents, Timeout, EventName). + +wait_n_events(NEvents, _Timeout, _EventName) when NEvents =< 0 -> + ok; +wait_n_events(NEvents, Timeout, EventName) -> + TelemetryTable = get({?MODULE, telemetry_table}), + receive + {telemetry, #{name := [_, _, EventName]}} -> + wait_n_events(NEvents - 1, Timeout, EventName) + after Timeout -> + RecordedEvents = ets:tab2list(TelemetryTable), + ct:pal("recorded events: ~p", [RecordedEvents]), + error({timeout_waiting_for_telemetry, EventName}) + end. + assert_sync_retry_fail_then_succeed_inflight(Trace) -> ct:pal(" ~p", [Trace]), ?assert( diff --git a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl index 47a4646de..c9560739f 100644 --- a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl +++ b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl @@ -350,10 +350,12 @@ service_account_json(PrivateKeyPEM) -> metrics_mapping() -> #{ dropped => fun emqx_resource_metrics:dropped_get/1, + dropped_expired => fun emqx_resource_metrics:dropped_expired_get/1, dropped_other => fun emqx_resource_metrics:dropped_other_get/1, dropped_queue_full => fun emqx_resource_metrics:dropped_queue_full_get/1, dropped_resource_not_found => fun emqx_resource_metrics:dropped_resource_not_found_get/1, dropped_resource_stopped => fun emqx_resource_metrics:dropped_resource_stopped_get/1, + late_reply => fun emqx_resource_metrics:late_reply_get/1, failed => fun emqx_resource_metrics:failed_get/1, inflight => fun emqx_resource_metrics:inflight_get/1, matched => fun emqx_resource_metrics:matched_get/1, @@ -1117,9 +1119,6 @@ do_econnrefused_or_timeout_test(Config, Error) -> CurrentMetrics ); {timeout, async} -> - wait_telemetry_event(TelemetryTable, success, ResourceId, #{ - timeout => 10_000, n_events => 2 - }), wait_until_gauge_is(inflight, 0, _Timeout = 400), wait_until_gauge_is(queuing, 0, _Timeout = 400), assert_metrics( @@ -1130,7 +1129,8 @@ do_econnrefused_or_timeout_test(Config, Error) -> matched => 2, queuing => 0, retried => 0, - success => 2 + success => 0, + late_reply => 2 }, ResourceId ); diff --git a/lib-ee/emqx_ee_connector/src/emqx_ee_connector_gcp_pubsub.erl b/lib-ee/emqx_ee_connector/src/emqx_ee_connector_gcp_pubsub.erl index 041bdec08..898c36fe0 100644 --- a/lib-ee/emqx_ee_connector/src/emqx_ee_connector_gcp_pubsub.erl +++ b/lib-ee/emqx_ee_connector/src/emqx_ee_connector_gcp_pubsub.erl @@ -58,7 +58,6 @@ %% emqx_resource API %%------------------------------------------------------------------------------------------------- -%% TODO: check is_buffer_supported() -> false. callback_mode() -> async_if_possible. From ca4a262b758ba7adc537813292f76242bf4f96c1 Mon Sep 17 00:00:00 2001 From: Thales Macedo Garitezi Date: Fri, 20 Jan 2023 08:24:53 -0300 Subject: [PATCH 02/15] refactor: re-organize dealing with unrecoverable errors --- .../test/emqx_bridge_mqtt_SUITE.erl | 11 ++- .../src/emqx_connector_pgsql.erl | 6 +- .../src/emqx_resource_buffer_worker.erl | 81 ++++++++++--------- .../test/emqx_ee_bridge_gcp_pubsub_SUITE.erl | 15 ++-- .../test/emqx_ee_bridge_influxdb_SUITE.erl | 33 +++++++- .../test/emqx_ee_bridge_mysql_SUITE.erl | 2 +- .../test/emqx_ee_bridge_pgsql_SUITE.erl | 26 ++++-- .../src/emqx_ee_connector_influxdb.erl | 34 +++++++- 8 files changed, 141 insertions(+), 67 deletions(-) diff --git a/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl b/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl index 1f5b06fab..a99f06f20 100644 --- a/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl +++ b/apps/emqx_bridge/test/emqx_bridge_mqtt_SUITE.erl @@ -957,12 +957,11 @@ assert_mqtt_msg_received(Topic, Payload) -> receive {deliver, Topic, #message{payload = Payload}} -> ct:pal("Got mqtt message: ~p on topic ~p", [Payload, Topic]), - ok; - Msg -> - ct:pal("Unexpected Msg: ~p", [Msg]), - assert_mqtt_msg_received(Topic, Payload) - after 100 -> - ct:fail("timeout waiting for ~p on topic ~p", [Payload, Topic]) + ok + after 300 -> + {messages, Messages} = process_info(self(), messages), + Msg = io_lib:format("timeout waiting for ~p on topic ~p", [Payload, Topic]), + error({Msg, #{messages => Messages}}) end. request(Method, Url, Body) -> diff --git a/apps/emqx_connector/src/emqx_connector_pgsql.erl b/apps/emqx_connector/src/emqx_connector_pgsql.erl index 34defb5e5..890227b9d 100644 --- a/apps/emqx_connector/src/emqx_connector_pgsql.erl +++ b/apps/emqx_connector/src/emqx_connector_pgsql.erl @@ -189,8 +189,8 @@ on_batch_query( Datas2 = [emqx_plugin_libs_rule:proc_sql(TokenList, Data) || Data <- Datas], St = maps:get(BinKey, Sts), case on_sql_query(InstId, PoolName, execute_batch, St, Datas2) of - {error, Error} -> - {error, Error}; + {error, _Error} = Result -> + handle_result(Result); {_Column, Results} -> handle_batch_result(Results, 0) end @@ -417,6 +417,8 @@ to_bin(Bin) when is_binary(Bin) -> to_bin(Atom) when is_atom(Atom) -> erlang:atom_to_binary(Atom). +handle_result({error, disconnected}) -> + {error, {recoverable_error, disconnected}}; handle_result({error, Error}) -> {error, {unrecoverable_error, Error}}; handle_result(Res) -> diff --git a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl index 8b79ce5a8..669b8e474 100644 --- a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl @@ -783,45 +783,26 @@ handle_query_result_pure(Id, ?RESOURCE_ERROR_M(Reason, _), _HasBeenSent) -> ok end, {nack, PostFn}; -handle_query_result_pure(Id, {error, {unrecoverable_error, Reason}}, HasBeenSent) -> - PostFn = fun() -> - ?SLOG(error, #{id => Id, msg => unrecoverable_error, reason => Reason}), - inc_sent_failed(Id, HasBeenSent), - ok - end, - {ack, PostFn}; -handle_query_result_pure(Id, {error, {recoverable_error, Reason}}, _HasBeenSent) -> - %% the message will be queued in replayq or inflight window, - %% i.e. the counter 'queuing' or 'dropped' will increase, so we pretend that we have not - %% sent this message. - PostFn = fun() -> - ?SLOG(warning, #{id => Id, msg => recoverable_error, reason => Reason}), - ok - end, - {nack, PostFn}; -handle_query_result_pure(Id, {error, Reason}, _HasBeenSent) -> - PostFn = fun() -> - ?SLOG(error, #{id => Id, msg => send_error, reason => Reason}), - ok - end, - {nack, PostFn}; -handle_query_result_pure(Id, {async_return, {error, {unrecoverable_error, Reason}}}, HasBeenSent) -> - PostFn = fun() -> - ?SLOG(error, #{id => Id, msg => unrecoverable_error, reason => Reason}), - inc_sent_failed(Id, HasBeenSent), - ok - end, - {ack, PostFn}; -handle_query_result_pure(Id, {async_return, {error, Msg}}, _HasBeenSent) -> - PostFn = fun() -> - ?SLOG(error, #{id => Id, msg => async_send_error, info => Msg}), - ok - end, - {nack, PostFn}; -handle_query_result_pure(_Id, {async_return, ok}, _HasBeenSent) -> - {ack, fun() -> ok end}; -handle_query_result_pure(_Id, {async_return, {ok, Pid}}, _HasBeenSent) when is_pid(Pid) -> - {ack, fun() -> ok end}; +handle_query_result_pure(Id, {error, Reason} = Error, HasBeenSent) -> + case is_unrecoverable_error(Error) of + true -> + PostFn = + fun() -> + ?SLOG(error, #{id => Id, msg => unrecoverable_error, reason => Reason}), + inc_sent_failed(Id, HasBeenSent), + ok + end, + {ack, PostFn}; + false -> + PostFn = + fun() -> + ?SLOG(error, #{id => Id, msg => send_error, reason => Reason}), + ok + end, + {nack, PostFn} + end; +handle_query_result_pure(Id, {async_return, Result}, HasBeenSent) -> + handle_query_async_result_pure(Id, Result, HasBeenSent); handle_query_result_pure(Id, Result, HasBeenSent) -> PostFn = fun() -> assert_ok_result(Result), @@ -830,6 +811,28 @@ handle_query_result_pure(Id, Result, HasBeenSent) -> end, {ack, PostFn}. +handle_query_async_result_pure(Id, {error, Reason} = Error, HasBeenSent) -> + case is_unrecoverable_error(Error) of + true -> + PostFn = + fun() -> + ?SLOG(error, #{id => Id, msg => unrecoverable_error, reason => Reason}), + inc_sent_failed(Id, HasBeenSent), + ok + end, + {ack, PostFn}; + false -> + PostFn = fun() -> + ?SLOG(error, #{id => Id, msg => async_send_error, reason => Reason}), + ok + end, + {nack, PostFn} + end; +handle_query_async_result_pure(_Id, {ok, Pid}, _HasBeenSent) when is_pid(Pid) -> + {ack, fun() -> ok end}; +handle_query_async_result_pure(_Id, ok, _HasBeenSent) -> + {ack, fun() -> ok end}. + handle_async_worker_down(Data0, Pid) -> #{async_workers := AsyncWorkers0} = Data0, {WorkerMRef, AsyncWorkers} = maps:take(Pid, AsyncWorkers0), diff --git a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl index c9560739f..247b7799b 100644 --- a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl +++ b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_gcp_pubsub_SUITE.erl @@ -1135,8 +1135,6 @@ do_econnrefused_or_timeout_test(Config, Error) -> ResourceId ); {_, sync} -> - %% even waiting, hard to avoid flakiness... simpler to just sleep - %% a bit until stabilization. wait_until_gauge_is(queuing, 0, 500), wait_until_gauge_is(inflight, 1, 500), assert_metrics( @@ -1336,12 +1334,19 @@ t_unrecoverable_error(Config) -> ), wait_until_gauge_is(queuing, 0, _Timeout = 400), - wait_until_gauge_is(inflight, 1, _Timeout = 400), + %% TODO: once temporary clause in + %% `emqx_resource_buffer_worker:is_unrecoverable_error' + %% that marks all unknown errors as unrecoverable is + %% removed, this inflight should be 1, because we retry if + %% the worker is killed. + wait_until_gauge_is(inflight, 0, _Timeout = 400), assert_metrics( #{ dropped => 0, - failed => 0, - inflight => 1, + %% FIXME: see comment above; failed should be 0 + %% and inflight should be 1. + failed => 1, + inflight => 0, matched => 1, queuing => 0, retried => 0, diff --git a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_influxdb_SUITE.erl b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_influxdb_SUITE.erl index fc7dce418..e1899b1b2 100644 --- a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_influxdb_SUITE.erl +++ b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_influxdb_SUITE.erl @@ -277,6 +277,7 @@ influxdb_config(apiv1 = Type, InfluxDBHost, InfluxDBPort, Config) -> " precision = ns\n" " write_syntax = \"~s\"\n" " resource_opts = {\n" + " request_timeout = 1s\n" " query_mode = ~s\n" " batch_size = ~b\n" " }\n" @@ -313,6 +314,7 @@ influxdb_config(apiv2 = Type, InfluxDBHost, InfluxDBPort, Config) -> " precision = ns\n" " write_syntax = \"~s\"\n" " resource_opts = {\n" + " request_timeout = 1s\n" " query_mode = ~s\n" " batch_size = ~b\n" " }\n" @@ -906,9 +908,23 @@ t_write_failure(Config) -> emqx_common_test_helpers:with_failure(down, ProxyName, ProxyHost, ProxyPort, fun() -> case QueryMode of sync -> - ?assertError(timeout, send_message(Config, SentData)); + {_, {ok, _}} = + ?wait_async_action( + try + send_message(Config, SentData) + catch + error:timeout -> + {error, timeout} + end, + #{?snk_kind := buffer_worker_flush_nack}, + 1_000 + ); async -> - ?assertEqual(ok, send_message(Config, SentData)) + ?wait_async_action( + ?assertEqual(ok, send_message(Config, SentData)), + #{?snk_kind := buffer_worker_reply_after_query}, + 1_000 + ) end end), fun(Trace0) -> @@ -920,11 +936,20 @@ t_write_failure(Config) -> ?assert( {error, {error, {closed, "The connection was lost."}}} =:= Result orelse {error, {error, closed}} =:= Result orelse - {error, {error, econnrefused}} =:= Result, + {error, {recoverable_error, {error, econnrefused}}} =:= Result, #{got => Result} ); async -> - ok + Trace = ?of_kind(buffer_worker_reply_after_query, Trace0), + ?assertMatch([#{action := nack} | _], Trace), + [#{result := Result} | _] = Trace, + ?assert( + {error, {recoverable_error, {closed, "The connection was lost."}}} =:= + Result orelse + {error, {error, closed}} =:= Result orelse + {error, {recoverable_error, econnrefused}} =:= Result, + #{got => Result} + ) end, ok end diff --git a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_mysql_SUITE.erl b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_mysql_SUITE.erl index 3bac01c66..57792b366 100644 --- a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_mysql_SUITE.erl +++ b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_mysql_SUITE.erl @@ -228,7 +228,7 @@ query_resource(Config, Request) -> Name = ?config(mysql_name, Config), BridgeType = ?config(mysql_bridge_type, Config), ResourceID = emqx_bridge_resource:resource_id(BridgeType, Name), - emqx_resource:query(ResourceID, Request). + emqx_resource:query(ResourceID, Request, #{timeout => 500}). unprepare(Config, Key) -> Name = ?config(mysql_name, Config), diff --git a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_pgsql_SUITE.erl b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_pgsql_SUITE.erl index bdbbed8cf..25752f685 100644 --- a/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_pgsql_SUITE.erl +++ b/lib-ee/emqx_ee_bridge/test/emqx_ee_bridge_pgsql_SUITE.erl @@ -249,7 +249,7 @@ query_resource(Config, Request) -> Name = ?config(pgsql_name, Config), BridgeType = ?config(pgsql_bridge_type, Config), ResourceID = emqx_bridge_resource:resource_id(BridgeType, Name), - emqx_resource:query(ResourceID, Request). + emqx_resource:query(ResourceID, Request, #{timeout => 1_000}). connect_direct_pgsql(Config) -> Opts = #{ @@ -422,12 +422,22 @@ t_write_failure(Config) -> SentData = #{payload => Val, timestamp => 1668602148000}, ?check_trace( emqx_common_test_helpers:with_failure(down, ProxyName, ProxyHost, ProxyPort, fun() -> - case QueryMode of - sync -> - ?assertError(timeout, send_message(Config, SentData)); - async -> - send_message(Config, SentData) - end + {_, {ok, _}} = + ?wait_async_action( + case QueryMode of + sync -> + try + send_message(Config, SentData) + catch + error:timeout -> + {error, timeout} + end; + async -> + send_message(Config, SentData) + end, + #{?snk_kind := buffer_worker_flush_nack}, + 1_000 + ) end), fun(Trace0) -> ct:pal("trace: ~p", [Trace0]), @@ -437,7 +447,7 @@ t_write_failure(Config) -> case Error of {resource_error, _} -> ok; - disconnected -> + {recoverable_error, disconnected} -> ok; _ -> ct:fail("unexpected error: ~p", [Error]) 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 0037242b3..1ae4d9874 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 @@ -26,6 +26,7 @@ on_batch_query_async/4, on_get_status/2 ]). +-export([reply_callback/2]). -export([ namespace/0, @@ -353,7 +354,12 @@ do_query(InstId, Client, Points) -> connector => InstId, reason => Reason }), - Err + case is_unrecoverable_error(Err) of + true -> + {error, {unrecoverable_error, Reason}}; + false -> + {error, {recoverable_error, Reason}} + end end. do_async_query(InstId, Client, Points, ReplyFunAndArgs) -> @@ -362,7 +368,20 @@ do_async_query(InstId, Client, Points, ReplyFunAndArgs) -> connector => InstId, points => Points }), - {ok, _WorkerPid} = influxdb:write_async(Client, Points, ReplyFunAndArgs). + WrappedReplyFunAndArgs = {fun ?MODULE:reply_callback/2, [ReplyFunAndArgs]}, + {ok, _WorkerPid} = influxdb:write_async(Client, Points, WrappedReplyFunAndArgs). + +reply_callback(ReplyFunAndArgs, {error, Reason} = Error) -> + case is_unrecoverable_error(Error) of + true -> + Result = {error, {unrecoverable_error, Reason}}, + emqx_resource:apply_reply_fun(ReplyFunAndArgs, Result); + false -> + Result = {error, {recoverable_error, Reason}}, + emqx_resource:apply_reply_fun(ReplyFunAndArgs, Result) + end; +reply_callback(ReplyFunAndArgs, Result) -> + emqx_resource:apply_reply_fun(ReplyFunAndArgs, Result). %% ------------------------------------------------------------------------------------------------- %% Tags & Fields Config Trans @@ -583,6 +602,17 @@ str(B) when is_binary(B) -> str(S) when is_list(S) -> S. +is_unrecoverable_error({error, {unrecoverable_error, _}}) -> + true; +is_unrecoverable_error({error, {recoverable_error, _}}) -> + false; +is_unrecoverable_error({error, {error, econnrefused}}) -> + false; +is_unrecoverable_error({error, econnrefused}) -> + false; +is_unrecoverable_error(_) -> + false. + %%=================================================================== %% eunit tests %%=================================================================== From 140cda2f134d4d82b1794235743216c60d9e79de Mon Sep 17 00:00:00 2001 From: "Zaiming (Stone) Shi" Date: Mon, 23 Jan 2023 19:38:08 +0100 Subject: [PATCH 03/15] fix(emqx_connection): crash when idle_timeout is set to infinity --- apps/emqx/i18n/emqx_schema_i18n.conf | 11 ++++-- apps/emqx/src/emqx.app.src | 2 +- apps/emqx/src/emqx_connection.erl | 9 +++-- apps/emqx/test/emqx_client_SUITE.erl | 52 ++++++++++++++++++++++++---- changes/v5.0.16/fix-9834.en.md | 1 + changes/v5.0.16/fix-9834.zh.md | 1 + 6 files changed, 64 insertions(+), 12 deletions(-) create mode 100644 changes/v5.0.16/fix-9834.en.md create mode 100644 changes/v5.0.16/fix-9834.zh.md diff --git a/apps/emqx/i18n/emqx_schema_i18n.conf b/apps/emqx/i18n/emqx_schema_i18n.conf index 0665cfb09..0ab781280 100644 --- a/apps/emqx/i18n/emqx_schema_i18n.conf +++ b/apps/emqx/i18n/emqx_schema_i18n.conf @@ -656,8 +656,15 @@ mqtt 下所有的配置作为全局的默认值存在,它可以被 zone< mqtt_idle_timeout { desc { - en: """After the TCP connection is established, if the MQTT CONNECT packet from the client is not received within the time specified by idle_timeout, the connection will be disconnected.""" - zh: """TCP 连接建立后,如果在 idle_timeout 指定的时间内未收到客户端的 MQTT CONNECT 报文,则连接将被断开。""" + en: """After the TCP connection is established, if the MQTT CONNECT packet from the client is +not received within the time specified by idle_timeout, the connection will be disconnected. +After the CONNECT packet has been accepted by EMQX, if the connection idles for this long time, +then the Erlang process is put to hibernation to save OS resources. Note: long idle_timeout +interval may impose risk at the system if large number of malicious clients only establish connections +but do not send any data.""" + zh: """TCP 连接建立后,如果在 idle_timeout 指定的时间内未收到客户端的 MQTT CONNECT 报文,则连接将被断开。 +如果连接在 CONNECT 报文被 EMQX 接受之后空闲超过该时长,那么服务这个连接的 Erlang 进程会进入休眠以节省系统资源。 +注意,该配置值如果设置过大的情况下,如果大量恶意客户端只连接,但不发任何数据,可能会导致系统资源被恶意消耗。""" } label: { en: """Idle Timeout""" diff --git a/apps/emqx/src/emqx.app.src b/apps/emqx/src/emqx.app.src index 40d2796cd..270d36a5e 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.15"}, + {vsn, "5.0.16"}, {modules, []}, {registered, []}, {applications, [ diff --git a/apps/emqx/src/emqx_connection.erl b/apps/emqx/src/emqx_connection.erl index 0c6481399..5ed302a6f 100644 --- a/apps/emqx/src/emqx_connection.erl +++ b/apps/emqx/src/emqx_connection.erl @@ -403,14 +403,19 @@ exit_on_sock_error(Reason) -> recvloop( Parent, State = #state{ - idle_timeout = IdleTimeout, + idle_timeout = IdleTimeout0, zone = Zone } ) -> + IdleTimeout = + case IdleTimeout0 of + infinity -> infinity; + _ -> IdleTimeout0 + 100 + end, receive Msg -> handle_recv(Msg, Parent, State) - after IdleTimeout + 100 -> + after IdleTimeout -> case emqx_olp:backoff_hibernation(Zone) of true -> recvloop(Parent, State); diff --git a/apps/emqx/test/emqx_client_SUITE.erl b/apps/emqx/test/emqx_client_SUITE.erl index 79c934b47..8effb60ef 100644 --- a/apps/emqx/test/emqx_client_SUITE.erl +++ b/apps/emqx/test/emqx_client_SUITE.erl @@ -43,6 +43,8 @@ <<"TopicA/#">> ]). +-define(WAIT(EXPR, ATTEMPTS), wait(fun() -> EXPR end, ATTEMPTS)). + all() -> [ {group, mqttv3}, @@ -85,6 +87,12 @@ init_per_suite(Config) -> end_per_suite(_Config) -> emqx_common_test_helpers:stop_apps([]). +init_per_testcase(_Case, Config) -> + Config. + +end_per_testcase(_Case, _Config) -> + emqx_config:put_zone_conf(default, [mqtt, idle_timeout], 15000). + %%-------------------------------------------------------------------- %% Test cases for MQTT v3 %%-------------------------------------------------------------------- @@ -101,16 +109,35 @@ t_basic_v4(_Config) -> t_cm(_) -> emqx_config:put_zone_conf(default, [mqtt, idle_timeout], 1000), - ClientId = <<"myclient">>, + ClientId = atom_to_binary(?FUNCTION_NAME), {ok, C} = emqtt:start_link([{clientid, ClientId}]), {ok, _} = emqtt:connect(C), - ct:sleep(500), - #{clientinfo := #{clientid := ClientId}} = emqx_cm:get_chan_info(ClientId), + ?WAIT(#{clientinfo := #{clientid := ClientId}} = emqx_cm:get_chan_info(ClientId), 2), emqtt:subscribe(C, <<"mytopic">>, 0), - ct:sleep(1200), - Stats = emqx_cm:get_chan_stats(ClientId), - ?assertEqual(1, proplists:get_value(subscriptions_cnt, Stats)), - emqx_config:put_zone_conf(default, [mqtt, idle_timeout], 15000). + ?WAIT( + begin + Stats = emqx_cm:get_chan_stats(ClientId), + ?assertEqual(1, proplists:get_value(subscriptions_cnt, Stats)) + end, + 2 + ), + ok. + +t_idle_timeout_infinity(_) -> + emqx_config:put_zone_conf(default, [mqtt, idle_timeout], infinity), + ClientId = atom_to_binary(?FUNCTION_NAME), + {ok, C} = emqtt:start_link([{clientid, ClientId}]), + {ok, _} = emqtt:connect(C), + ?WAIT(#{clientinfo := #{clientid := ClientId}} = emqx_cm:get_chan_info(ClientId), 2), + emqtt:subscribe(C, <<"mytopic">>, 0), + ?WAIT( + begin + Stats = emqx_cm:get_chan_stats(ClientId), + ?assertEqual(1, proplists:get_value(subscriptions_cnt, Stats)) + end, + 2 + ), + ok. t_cm_registry(_) -> Children = supervisor:which_children(emqx_cm_sup), @@ -363,3 +390,14 @@ tls_certcn_as_clientid(TLSVsn, RequiredTLSVsn) -> #{clientinfo := #{clientid := CN}} = emqx_cm:get_chan_info(CN), confirm_tls_version(Client, RequiredTLSVsn), emqtt:disconnect(Client). + +wait(F, 1) -> + F(); +wait(F, Attempts) when Attempts > 0 -> + try + F() + catch + _:_ -> + timer:sleep(1000), + wait(F, Attempts - 1) + end. diff --git a/changes/v5.0.16/fix-9834.en.md b/changes/v5.0.16/fix-9834.en.md new file mode 100644 index 000000000..d5ad1f67a --- /dev/null +++ b/changes/v5.0.16/fix-9834.en.md @@ -0,0 +1 @@ +Allow `mqtt.idle_timeout` to be set to `infinity` diff --git a/changes/v5.0.16/fix-9834.zh.md b/changes/v5.0.16/fix-9834.zh.md new file mode 100644 index 000000000..06eafc1a0 --- /dev/null +++ b/changes/v5.0.16/fix-9834.zh.md @@ -0,0 +1 @@ +允许配置项 `mqtt.idle_timeout` 设置成 `infinity` From 7575120ea694e0ebacb5b43bdfcc685ae3c29c7e Mon Sep 17 00:00:00 2001 From: "Zaiming (Stone) Shi" Date: Tue, 24 Jan 2023 10:54:20 +0100 Subject: [PATCH 04/15] test: use snabbkaffe retry macro --- apps/emqx/test/emqx_client_SUITE.erl | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/apps/emqx/test/emqx_client_SUITE.erl b/apps/emqx/test/emqx_client_SUITE.erl index 8effb60ef..2f433c73d 100644 --- a/apps/emqx/test/emqx_client_SUITE.erl +++ b/apps/emqx/test/emqx_client_SUITE.erl @@ -24,6 +24,7 @@ -include_lib("emqx/include/emqx_mqtt.hrl"). -include_lib("eunit/include/eunit.hrl"). -include_lib("common_test/include/ct.hrl"). +-include_lib("snabbkaffe/include/snabbkaffe.hrl"). -define(TOPICS, [ <<"TopicA">>, @@ -43,7 +44,7 @@ <<"TopicA/#">> ]). --define(WAIT(EXPR, ATTEMPTS), wait(fun() -> EXPR end, ATTEMPTS)). +-define(WAIT(EXPR, ATTEMPTS), ?retry(1000, ATTEMPTS, EXPR)). all() -> [ @@ -390,14 +391,3 @@ tls_certcn_as_clientid(TLSVsn, RequiredTLSVsn) -> #{clientinfo := #{clientid := CN}} = emqx_cm:get_chan_info(CN), confirm_tls_version(Client, RequiredTLSVsn), emqtt:disconnect(Client). - -wait(F, 1) -> - F(); -wait(F, Attempts) when Attempts > 0 -> - try - F() - catch - _:_ -> - timer:sleep(1000), - wait(F, Attempts - 1) - end. From 502b826396377ae5741b265c14aa0798c6461f2e Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Mon, 23 Jan 2023 16:18:04 +0100 Subject: [PATCH 05/15] test: add influxdb docker test files --- scripts/test/influx/certs/server.key | 28 ++++++ scripts/test/influx/certs/server_combined.pem | 69 ++++++++++++++ scripts/test/influx/docker-compose.yaml | 77 +++++++++++++++ scripts/test/influx/emqx.conf | 94 +++++++++++++++++++ scripts/test/influx/influx-bridge.conf | 46 +++++++++ scripts/test/influx/start.sh | 34 +++++++ 6 files changed, 348 insertions(+) create mode 100644 scripts/test/influx/certs/server.key create mode 100644 scripts/test/influx/certs/server_combined.pem create mode 100644 scripts/test/influx/docker-compose.yaml create mode 100644 scripts/test/influx/emqx.conf create mode 100644 scripts/test/influx/influx-bridge.conf create mode 100755 scripts/test/influx/start.sh diff --git a/scripts/test/influx/certs/server.key b/scripts/test/influx/certs/server.key new file mode 100644 index 000000000..e99276f95 --- /dev/null +++ b/scripts/test/influx/certs/server.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQDstiij2PRYssNx +P5k+0YlWLgx57xOz5zpzjyjavDvFUmboEJH2VG2A7ITQtXlTqrqB3rxbGEDYjltm +RG/tjm4g5GQ3RgfupZdsNommgzE4/bH0PT6PwYTlt83596Vu9hFqUF5a0rtYF8RJ +gXHXzDeraz25JoLcuNgGlSrpgioryx6NZB/oBkbuHbGL7fmYuRO8zQSxknHtg/Jr +K4YqQaw66VFQoydyzcGp9b8KjMYjyBVjNvt7xbvvCLrMZ6XMAhPYyquRhQC00ydJ +OHavzcauARAQH20noHpKU8/W8HSwBOKMsYdh0vxLnFwnrP9qOjMoB1G6DtpL8yGc +dsWSkXJTAgMBAAECggEAI4uxy/OcXbrFFpB9AViTIwSEjcWU/oL5Mrk2DVgO3FwP +ZUciZAyG6mvvVdf/1syLIqLBQ3yqNYV1vC8ID8g/nrjX7mTvHqscc1I37BqZ8vub +FBjOTZAQgN56aSWttgfwKDgsCGofWo8sbqKoPHDyOyTo0+YwkyyUDrOqX1OojtgG +Vv+0jKZfc6zzGWoG7HXAAMHYlnLDcO/XqFGiqWG0jVQtKaHOYndYzbWQHg/h4kOr +NcpOCryAUOWWZWjO212qINAYmN/v+ietoemrNKInYzkF6na0TCjOyhzurMdhO1VF +H0D/q0QKZp0BurWMBgoVMEu/L+5NKnVAdS417DojoQKBgQDydlesHeoAlTEKfSOf +4Un/n2ylU7Xu6V17RV+oTUESFlxl+2IeQJ1FuTtnW7LWaT/+jCkqNoxpcj+Aq1Ov +tsecy894hEy1cKOTD8g0BHqGMQ96El/vy4cphBGrmJ1BHd8t+B7u2kuDJO+Jap/r +R0FMtQvkcsZX/JlVnKl2j/OVdwKBgQD57Z4riMKCyEh99y0e5dK2cDjm+NhEErxs +C3AV5ye1rApf5HR4qYaEwdfqMk1mU1eQKQG3nLMEYNDx+ofMNxnJ8xWFtlLOCeRA +bAaD/O9lqvVNV6HDamT85xxrgi6glJRVmlglr1qDutvqKEbt5Cy81hVrGg09prmw +0DdKXlpxBQKBgQC4xicaK9Hgj9HHxIYZmX7jJ4OMmrbTU1TCVKQJhpZISnIv5dto +YqV4NSloIyGu9JaoTfgxC1EGKuLqE6a7CfglLcz2cxxiSWze7fAMdD1Ldqfsx4QF +In3mzZcJxkxRC4rBZY1tST6yursxZehpTOweoPpnJnCxLPdNnw4mJJ3lEQKBgF+2 +Wh5970+6d9iVslfk1786VvBNWDUpn7MY5kHdFl4Clq6GhPw4n+VzfsaXoQCQ5jW1 +Ho08m2oP0bT5psvURjIKv7FHU4rISvtI423kjaG3yN1ns/Fsa2RW9ZFOd6DThcMK +qrsz3kwYYWPZLOCpw2BF9Enh3jRAB1jpRGNcovIpAoGAQJJSNcZDr1JY3NppBZ9U +7OqcQypQ8PP7jBpszQrWid8BGGK082UjO+okG7f3wQjmRYaPpuS/Rqa50ikmLscZ +aO6Y4+rUGmZH6G7r/hk9Pu8AO3JDRH/gNImZy6X1Ae8NDoeHRYzXtz57aGlOY+gU +rcgGpevWU5XzAtEudxOMRsk= +-----END PRIVATE KEY----- diff --git a/scripts/test/influx/certs/server_combined.pem b/scripts/test/influx/certs/server_combined.pem new file mode 100644 index 000000000..8d0efa591 --- /dev/null +++ b/scripts/test/influx/certs/server_combined.pem @@ -0,0 +1,69 @@ +-----BEGIN CERTIFICATE----- +MIIDvTCCAqWgAwIBAgIFFnRIM4UwDQYJKoZIhvcNAQELBQAwbTELMAkGA1UEBhMC +U0UxEjAQBgNVBAgMCVN0b2NraG9sbTESMBAGA1UECgwJTXlPcmdOYW1lMRkwFwYD +VQQLDBBNeUludGVybWVkaWF0ZUNBMRswGQYDVQQDDBJNeUludGVybWVkaWF0ZUNB +LTEwHhcNMjMwMTIzMTQxNjI1WhcNMzMwMTIwMTQxNjI1WjBrMQswCQYDVQQGEwJT +RTESMBAGA1UECAwJU3RvY2tob2xtMRIwEAYDVQQKDAlNeU9yZ05hbWUxEjAQBgNV +BAsMCU15U2VydmljZTEgMB4GA1UEAwwXaW5mbHV4ZGIuZW1xeC5pbmZsdXguaW8w +ggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQDstiij2PRYssNxP5k+0YlW +Lgx57xOz5zpzjyjavDvFUmboEJH2VG2A7ITQtXlTqrqB3rxbGEDYjltmRG/tjm4g +5GQ3RgfupZdsNommgzE4/bH0PT6PwYTlt83596Vu9hFqUF5a0rtYF8RJgXHXzDer +az25JoLcuNgGlSrpgioryx6NZB/oBkbuHbGL7fmYuRO8zQSxknHtg/JrK4YqQaw6 +6VFQoydyzcGp9b8KjMYjyBVjNvt7xbvvCLrMZ6XMAhPYyquRhQC00ydJOHavzcau +ARAQH20noHpKU8/W8HSwBOKMsYdh0vxLnFwnrP9qOjMoB1G6DtpL8yGcdsWSkXJT +AgMBAAGjZjBkMCIGA1UdEQQbMBmCF2luZmx1eGRiLmVtcXguaW5mbHV4LmlvMB0G +A1UdDgQWBBSz+wgHJ2TZNGasVR+BSMqRpg3pSTAfBgNVHSMEGDAWgBRfr50a+u+U +2sPlCD7ft2gOjNIdOjANBgkqhkiG9w0BAQsFAAOCAQEAAInxCAHkEVbPQhijpl+C +vblGHxS4aPGGAmIU7rS2CsZQaXRLZNC++syMZQzOKtLOIzLT8WSgs026/tLsFQk0 +5VCLbgnQqS2wblFylGribUiT/dSMmiMvGylc6MHJnbEyiNKtlB88paTLuDC81lIZ +KDBEYSQTfaykm3MXjeZGB5sHBcKeBWCkDY9O1YvsP/5a9zyZiC1c+XTTlB5Lnzgo +aL7SWB88J+mF0cn5iwitwTg0ryZtO5fG8qmqaAs+G8u9oVytnxi+WBmjukjnFdzC +SlRFYGY0KZmg1ZGlLCzBS3Bi3q+ck8yfv2rj2WBqklOISzSnp5VaRC6gcUDNE0XY +1g== +-----END CERTIFICATE----- +-----BEGIN CERTIFICATE----- +MIIEODCCAyCgAwIBAgIFFnRIM4UwDQYJKoZIhvcNAQELBQAwbzELMAkGA1UEBhMC +U0UxEjAQBgNVBAgMCVN0b2NraG9sbTESMBAGA1UEBwwJU3RvY2tob2xtMRIwEAYD +VQQKDAlNeU9yZ05hbWUxETAPBgNVBAsMCE15Um9vdENBMREwDwYDVQQDDAhNeVJv +b3RDQTAeFw0yMzAxMjMxNDE2MjVaFw0zMzAxMjAxNDE2MjVaMG0xCzAJBgNVBAYT +AlNFMRIwEAYDVQQIDAlTdG9ja2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTEZMBcG +A1UECwwQTXlJbnRlcm1lZGlhdGVDQTEbMBkGA1UEAwwSTXlJbnRlcm1lZGlhdGVD +QS0xMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEApYFU7RYTvI8eH6Y6 +yVgjwWWMWSqU3Q97G8l2zHcgOCrph8xQRM8tp0wsoPpvj6SYvohzPJyIxK6tFiQg +FvPrk4JQdQWNEK7ogRwqbeFX017vVsQIDMAsO5V5cqoRjOghV1QvHkz5+oVQt09w +whwk47IrsKhwiXtQrFuGi3w7WNTkwh1FX3V8ZrqCVr02/f1GmODYxPqvuXVtf42B +iewWOvnHs58vyMPvts/Ngh4HBzQncSudeI9GDsPscAn6I5YmjBQkGsQQb7cygd4p +u3OrMw/Sw2e85N6fBM6+b4ZTUEaz6EC2ymBNiJtkLDzkM3ZaP19GluYy17PBi8ZB +UO+rowIDAQABo4HcMIHZMA4GA1UdDwEB/wQEAwIBBjAPBgNVHRMBAf8EBTADAQH/ +MB0GA1UdDgQWBBRfr50a+u+U2sPlCD7ft2gOjNIdOjCBlgYDVR0jBIGOMIGLoXOk +cTBvMQswCQYDVQQGEwJTRTESMBAGA1UECAwJU3RvY2tob2xtMRIwEAYDVQQHDAlT +dG9ja2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTERMA8GA1UECwwITXlSb290Q0Ex +ETAPBgNVBAMMCE15Um9vdENBghRfGvHfslLmWduK/DoD5zdiKKaPsDANBgkqhkiG +9w0BAQsFAAOCAQEAKoEj6H7tBDiD/sqPuxABGJI4dT6jqSFskF2An95FlwTHLu4b +kBxfb1r3emaqM6t+dgEjupqa6Mte9/rQmynGn+U9IFn3dOo95FPmf8igQGGcO7Tz +Wl1pTm2S7m5+fQAgC94jmBk8JUXO+fGJe33Mx+os1ZaE26KvbhQzTk8NpYfK+cp2 ++K4HsaVogr+mR139YNbKD00xexGQdxPzdBO4EHEsCI+6W7ELFz/iaj7S6XTe9YW2 +Uj858vOWhL6bg2zWhb/ER2MqbB05gOQhf8Rw+4iL+45AiGFrdyD4jXaqMy7lR4dv +fBlPwX60owxqgtKc4QC8/rSKLteSey01WV1SXA== +-----END CERTIFICATE----- +-----BEGIN CERTIFICATE----- +MIIDZTCCAk0CFF8a8d+yUuZZ24r8OgPnN2Iopo+wMA0GCSqGSIb3DQEBCwUAMG8x +CzAJBgNVBAYTAlNFMRIwEAYDVQQIDAlTdG9ja2hvbG0xEjAQBgNVBAcMCVN0b2Nr +aG9sbTESMBAGA1UECgwJTXlPcmdOYW1lMREwDwYDVQQLDAhNeVJvb3RDQTERMA8G +A1UEAwwITXlSb290Q0EwHhcNMjMwMTIzMTQxNjI0WhcNMzMwMTIwMTQxNjI0WjBv +MQswCQYDVQQGEwJTRTESMBAGA1UECAwJU3RvY2tob2xtMRIwEAYDVQQHDAlTdG9j +a2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTERMA8GA1UECwwITXlSb290Q0ExETAP +BgNVBAMMCE15Um9vdENBMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEA +4aktTF0LaTO6XYvZCGM7vI1XaXIh0eEvYmnLW0di2JD/CajTAqyGbLkpHCDVJDrE +zIXP4v+u872RtfOHDcXnK8fU0hYv4r6KG4YW8PAQouPDuP992WbpuxircOq4b7Z8 +ED5RIxKqZxFJYLmkrvk+cJyqOOIUdbre1+82Wh7TJzBkNJNVRYVb6PzX3qCoVxgy +3tyZCW7K/2wjs8Rde3eyTJuOFsKRaGk5ycS4OMAPqFc3Y1uw5yIDTkWMp/dtbBIK +jVDg6DcY79Vlz293u7Y03Je4xK1HksxpxcU/sMqgtsUB+HKbUBDhLUNkxRRkCw1N ++5PVkP8rmxGuQbIgYxOMOQIDAQABMA0GCSqGSIb3DQEBCwUAA4IBAQBQ9hrNHYcL +m9Vh5Q94Pg+hPk6Q97wRv9lKsimQgVutLXBjC8+NfHGwJchMRlnKrcRHeYTzYIUi +TXxLrRbAMLnAeyeKlGhjqqw4VDORJFvt0I82ia6jrKNlbwGqQ6v+SKoXu85HuJv+ +Er8Iz9X6l1aZ2hEBFy4W4IuYy+fPBlrN74aUdSLcekgLbLj8n0mjQjWD5wzQ4Gsf +8dwm0rsbh6UoX8/XOMkSXP9SlKsNyzA4ZHTZ9aZoehoe0G5sUbfDumUaONmp4oOj +EFntuk5AsbxqzQTSYP4jrLc7/D2hOCJBn93nfZ+raMyD6mjttfk52aeYwFLuPihL +FoanUYSck91R +-----END CERTIFICATE----- diff --git a/scripts/test/influx/docker-compose.yaml b/scripts/test/influx/docker-compose.yaml new file mode 100644 index 000000000..60b4c3421 --- /dev/null +++ b/scripts/test/influx/docker-compose.yaml @@ -0,0 +1,77 @@ +version: '3.9' + +services: + influxdb_server: + container_name: influxdb_server + image: influxdb:2.5.0 + hostname: influxdb.emqx.influx.io + expose: + - "8086" + - "8083" + ports: + - "8086:8086" + - "8083:8083" + environment: + DOCKER_INFLUXDB_INIT_MODE: setup + DOCKER_INFLUXDB_INIT_USERNAME: root + DOCKER_INFLUXDB_INIT_PASSWORD: emqx@123 + DOCKER_INFLUXDB_INIT_ORG: emqx + DOCKER_INFLUXDB_INIT_BUCKET: mqtt + DOCKER_INFLUXDB_INIT_ADMIN_TOKEN: abcdefg + volumes: + - "${PWD}/certs/server_combined.pem:/etc/influxdb/cert.pem" + - "${PWD}/certs/server.key:/etc/influxdb/key.pem" +# - "../../../.ci/docker-compose-file/certs/server.crt:/etc/influxdb/cert.pem" +# - "../../../.ci/docker-compose-file/certs/server.key:/etc/influxdb/key.pem" + - "../../../.ci/docker-compose-file/influxdb/setup-v1.sh:/docker-entrypoint-initdb.d/setup-v1.sh" + restart: always + command: + - influxd + - --tls-cert=/etc/influxdb/cert.pem + - --tls-key=/etc/influxdb/key.pem + networks: + - emqx_test + + emqx_1: + container_name: emqx_1 + image: emqx/emqx-enterprise:${EMQX_TAG} + hostname: emqx_1.emqx.influx.io + ports: + - "1883:1883" + - "18083:18083" + environment: + EMQX_LOG__CONSOLE_HANDLER__LEVEL: warning + EMQX_NODE_NAME: "emqx@emqx_1.emqx.influx.io" + volumes: + - "${PWD}/emqx.conf:/opt/emqx/etc/emqx.conf" + - "${PWD}/influx-bridge.conf:/opt/emqx/etc/influx-bridge.conf" + networks: + - emqx_test + + emqx_2: + container_name: emqx_2 + image: emqx/emqx-enterprise:${EMQX_TAG} + hostname: emqx_1.emqx.influx.io + environment: + EMQX_LOG__CONSOLE_HANDLER__LEVEL: warning + EMQX_NODE_NAME: "emqx@emqx_2.emqx.influx.io" + ports: + - "1884:1883" + - "18084:18083" + volumes: + - "${PWD}/emqx.conf:/opt/emqx/etc/emqx.conf" + - "${PWD}/influx-bridge.conf:/opt/emqx/etc/influx-bridge.conf" + networks: + - emqx_test + +networks: + emqx_test: + driver: bridge + name: emqx.influx.io + ipam: + driver: default + config: + - subnet: 172.100.240.0/24 + gateway: 172.100.240.1 + - subnet: 2001:3200:3110::/64 + gateway: 2001:3200:3210::1 diff --git a/scripts/test/influx/emqx.conf b/scripts/test/influx/emqx.conf new file mode 100644 index 000000000..8fd114f50 --- /dev/null +++ b/scripts/test/influx/emqx.conf @@ -0,0 +1,94 @@ +## NOTE: +## Configs in this file might be overridden by: +## 1. Environment variables which start with 'EMQX_' prefix +## 2. File $EMQX_NODE__DATA_DIR/configs/cluster-override.conf +## 3. File $EMQX_NODE__DATA_DIR/configs/local-override.conf +## +## The *-override.conf files are overwritten at runtime when changes +## are made from EMQX dashboard UI, management HTTP API, or CLI. +## All configuration details can be found in emqx.conf.example + +node { + cookie = "influx" + data_dir = "data" +} + +log { + file_handlers.default { + level = warning + file = "log/emqx.log" + } +} + +cluster { + name = emqxcl + discovery_strategy = manual +} + + +listeners.tcp.default { + bind = "0.0.0.0:1883" + max_connections = 1024000 +} + +listeners.ssl.default { + bind = "0.0.0.0:8883" + max_connections = 512000 + ssl_options { + keyfile = "etc/certs/key.pem" + certfile = "etc/certs/cert.pem" + cacertfile = "etc/certs/cacert.pem" + } +} + +listeners.ws.default { + bind = "0.0.0.0:8083" + max_connections = 1024000 + websocket.mqtt_path = "/mqtt" +} + +listeners.wss.default { + bind = "0.0.0.0:8084" + max_connections = 512000 + websocket.mqtt_path = "/mqtt" + ssl_options { + keyfile = "etc/certs/key.pem" + certfile = "etc/certs/cert.pem" + cacertfile = "etc/certs/cacert.pem" + } +} + +# listeners.quic.default { +# enabled = true +# bind = "0.0.0.0:14567" +# max_connections = 1024000 +# keyfile = "etc/certs/key.pem" +# certfile = "etc/certs/cert.pem" +#} + +dashboard { + listeners.http { + bind = 18083 + } + default_username = "admin" + default_password = "public" +} + +authorization { + deny_action = ignore + no_match = allow + cache = { enable = true } + sources = [ + { + type = file + enable = true + # This file is immutable to EMQX. + # Once new rules are created from dashboard UI or HTTP API, + # the file 'data/authz/acl.conf' is used instead of this one + path = "etc/acl.conf" + } + ] +} + +include emqx-enterprise.conf +include influx-bridge.conf \ No newline at end of file diff --git a/scripts/test/influx/influx-bridge.conf b/scripts/test/influx/influx-bridge.conf new file mode 100644 index 000000000..b513b962f --- /dev/null +++ b/scripts/test/influx/influx-bridge.conf @@ -0,0 +1,46 @@ +bridges { + "influxdb_api_v2" { + test { + bucket = "mqtt" + enable = true + org = "emqx" + precision = "ms" + resource_opts { + async_inflight_window = 100 + auto_restart_interval = "60s" + batch_size = 100 + batch_time = "10ms" + health_check_interval = "15s" + max_queue_bytes = "1GB" + query_mode = "sync" + request_timeout = "15s" + start_after_created = "true" + start_timeout = "5s" + worker_pool_size = 4 + } + server = "influxdb.emqx.influx.io:8086" + ssl { + ciphers = [] + depth = 10 + enable = false + reuse_sessions = true + secure_renegotiate = true + user_lookup_fun = "emqx_tls_psk:lookup" + verify = "verify_peer" + versions = ["tlsv1.3", "tlsv1.2", "tlsv1.1", "tlsv1"] + } + token = "abcdefg" + write_syntax = "mqtt,clientid=${clientid} value=${payload.value}" + } + } +} +log { + console_handler {enable = true, level = "warning"} + file_handlers { + default { + enable = false + file = "log/emqx.log" + level = "warning" + } + } +} \ No newline at end of file diff --git a/scripts/test/influx/start.sh b/scripts/test/influx/start.sh new file mode 100755 index 000000000..7ce9c399c --- /dev/null +++ b/scripts/test/influx/start.sh @@ -0,0 +1,34 @@ +#!/usr/bin/env bash + +NET='emqx.influx.io' +NODE1="emqx@emqx_1.$NET" +NODE2="emqx@emqx_2.$NET" +export EMQX_TAG='5.0.0-rc.1-g92797d72' + +cd -P -- "$(dirname -- "${BASH_SOURCE[0]}")" + +docker rm -f emqx_1 emqx_2 influxdb_server +docker-compose up -d + +wait_limit=60 +wait_for_emqx() { + container="$1" + wait_limit="$2" + wait_sec=0 + while ! docker exec "$container" emqx_ctl status >/dev/null 2>&1; do + wait_sec=$(( wait_sec + 1 )) + if [ $wait_sec -gt "$wait_limit" ]; then + echo "timeout wait for EMQX" + exit 1 + fi + echo -n '.' + sleep 1 + done +} + +wait_for_emqx emqx_1 30 +wait_for_emqx emqx_2 30 + +echo + +docker exec emqx_1 emqx_ctl cluster join "$NODE2" From 9fc586f670f75d2ddf3aea38d865256e168ad79d Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Mon, 23 Jan 2023 16:44:59 +0100 Subject: [PATCH 06/15] test: add influx docker test certs --- scripts/test/influx/certs/ca.pem | 21 ++++++++++++ scripts/test/influx/certs/client.key | 28 ++++++++++++++++ scripts/test/influx/certs/client.pem | 46 ++++++++++++++++++++++++++ scripts/test/influx/certs/server.pem | 48 ++++++++++++++++++++++++++++ 4 files changed, 143 insertions(+) create mode 100644 scripts/test/influx/certs/ca.pem create mode 100644 scripts/test/influx/certs/client.key create mode 100644 scripts/test/influx/certs/client.pem create mode 100644 scripts/test/influx/certs/server.pem diff --git a/scripts/test/influx/certs/ca.pem b/scripts/test/influx/certs/ca.pem new file mode 100644 index 000000000..0044e44ea --- /dev/null +++ b/scripts/test/influx/certs/ca.pem @@ -0,0 +1,21 @@ +-----BEGIN CERTIFICATE----- +MIIDZTCCAk0CFF8a8d+yUuZZ24r8OgPnN2Iopo+wMA0GCSqGSIb3DQEBCwUAMG8x +CzAJBgNVBAYTAlNFMRIwEAYDVQQIDAlTdG9ja2hvbG0xEjAQBgNVBAcMCVN0b2Nr +aG9sbTESMBAGA1UECgwJTXlPcmdOYW1lMREwDwYDVQQLDAhNeVJvb3RDQTERMA8G +A1UEAwwITXlSb290Q0EwHhcNMjMwMTIzMTQxNjI0WhcNMzMwMTIwMTQxNjI0WjBv +MQswCQYDVQQGEwJTRTESMBAGA1UECAwJU3RvY2tob2xtMRIwEAYDVQQHDAlTdG9j +a2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTERMA8GA1UECwwITXlSb290Q0ExETAP +BgNVBAMMCE15Um9vdENBMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEA +4aktTF0LaTO6XYvZCGM7vI1XaXIh0eEvYmnLW0di2JD/CajTAqyGbLkpHCDVJDrE +zIXP4v+u872RtfOHDcXnK8fU0hYv4r6KG4YW8PAQouPDuP992WbpuxircOq4b7Z8 +ED5RIxKqZxFJYLmkrvk+cJyqOOIUdbre1+82Wh7TJzBkNJNVRYVb6PzX3qCoVxgy +3tyZCW7K/2wjs8Rde3eyTJuOFsKRaGk5ycS4OMAPqFc3Y1uw5yIDTkWMp/dtbBIK +jVDg6DcY79Vlz293u7Y03Je4xK1HksxpxcU/sMqgtsUB+HKbUBDhLUNkxRRkCw1N ++5PVkP8rmxGuQbIgYxOMOQIDAQABMA0GCSqGSIb3DQEBCwUAA4IBAQBQ9hrNHYcL +m9Vh5Q94Pg+hPk6Q97wRv9lKsimQgVutLXBjC8+NfHGwJchMRlnKrcRHeYTzYIUi +TXxLrRbAMLnAeyeKlGhjqqw4VDORJFvt0I82ia6jrKNlbwGqQ6v+SKoXu85HuJv+ +Er8Iz9X6l1aZ2hEBFy4W4IuYy+fPBlrN74aUdSLcekgLbLj8n0mjQjWD5wzQ4Gsf +8dwm0rsbh6UoX8/XOMkSXP9SlKsNyzA4ZHTZ9aZoehoe0G5sUbfDumUaONmp4oOj +EFntuk5AsbxqzQTSYP4jrLc7/D2hOCJBn93nfZ+raMyD6mjttfk52aeYwFLuPihL +FoanUYSck91R +-----END CERTIFICATE----- diff --git a/scripts/test/influx/certs/client.key b/scripts/test/influx/certs/client.key new file mode 100644 index 000000000..931b36442 --- /dev/null +++ b/scripts/test/influx/certs/client.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQDK1dJ41zpZ6Sha +RuzL6CqqCXO0R6n5FlQVLIaY+0JCEDJbFqlZ9OtIj96Gh5vlHuxvf2s3fnKBdp0t +pov5ekwavEmEeT0L3h80E21HwGraVHILf2WuBKzpwJlO3Ilwv3s0qfADv7caKPtP +3iQ8EpLMZ2oJrl4jdJ/9Ss3jhV5yTqmxJgY2g1HAOQtpXbbzynoBC3tu8abVd/0k +euimPwg0sr3maphxfbKjUQSi7cn/NzP+BurlTxfy+c3ajeqdFYoG4M1TBpqGn1xb +7gmq37TFxXFvfjmW5Jx9yNtx78drpggOJHcRYXWzF78gHEEf+VbMMBBiQ6B59sOY +x6vn2nM7AgMBAAECggEAO7Iid54gdkXAspEdMAXsSI7vlqdgxwZjSLgKBQwtu/O4 +XkaVIPyMIpb1DvLqVsx7GXWOwI5edOvL5puVpp7vtw6jNHHaHZwJ4vG0v+FJkWg5 +CFk+O7D0kvc0J5mQebZhNGavVOsfoHtfO2pNj8CHo9DrzF2dJyIz1AAoCEZAPbOS +EIiOD7935iZS3YrEdwdVMSq9RfJ95IN58r6Gqqk4tnTazpq9Qe/+8sJJdBqfYRQ2 +R3DxV86uIxbVncPK5SuCXiC4GFYFSoQO2g25pw6AUH7meZUNekXcQtH+i+OU2+lv +Y8W/JIEg1HdxmHDMZd8t25B8BtmWEJCVdv3JbfyVwQKBgQDLhEiZAX3dT1E7xLgM +ecQRdbSIFXoPIsfmOJQY2VOnwSqcqB1lqEo9rmyLLJLsx5AEa5/npm5lbVPQp4Yc +8ng1WUdmZm0A5uKp8mM6XvDOdnIJXv9+onz7aJWs6Xk4d2r3QI/B36q+CoOjmgTO +m/zGB4MWrzYSBMPwlJWLlw6m1wKBgQD/JIxS3tX/P/ro+AekuBcFyk4oiVuClNVF +4G2JEZ6QV76wilvn+R/ocaM5Wdg3a9iRa96PmwqqbcxPiLL08LbtIYG/W7B11Y1t +vSa9LrtN9HI2XaQePfF4aCZQ23e3alNWSCaPeG0QWcoSmzCMOpGXvV8LcLjdo7kr +C5XS3DyePQKBgQDKyw3sKOhaVmcDv4oGJG+TVk6GK6Bs5KN8bHRmLmiFiJ4FbB+e +5IrV27CdHdVpy5FeVkAIfHcU6lPSXpvszE+sGUJkbzu+4lKKNdVanLAniy8m11qP +vSn8+sHqgtsfwR79MYAOF+nzZFa39wlDN3wxd9rQB9dBLS5x1B0P833OEQKBgBUg +vqoCH/ByKOHv4Boo0Gc/WHeM4FIQ5xRRQGDQX7QOQBE4c+HixFyoRF70bK4yNQ3d +w+FacmiojPmtFTypTkyWsLra6eLoGz0SI8oYu5P6ARc1a+jBMmTJIxshAWnj2SDU ++rR4wXdQKCagSUXjd7j8ConU/MoFy8ifvflnGXH1AoGAE8LwsVp/+sn0o1cZl8+Z +aQJDXHyf61F0GGzf7rHV4L/dQhZDxTLNHRMCSQDrPBEoge0OjHGDkJqo8e4ugHNm +150I/FW/AGd3jZgZ8FPApjbCbD9CoxsV+pFpIqE1AJ9TkrdAxTXGIJa0a/hoA0et +S8eNqGQ9RrWQyOJdKhRHjl4= +-----END PRIVATE KEY----- diff --git a/scripts/test/influx/certs/client.pem b/scripts/test/influx/certs/client.pem new file mode 100644 index 000000000..6c633deb4 --- /dev/null +++ b/scripts/test/influx/certs/client.pem @@ -0,0 +1,46 @@ +-----BEGIN CERTIFICATE----- +MIIDazCCAlMCFHQbLq+eVR/mJLDEkxXP6CdXs+9oMA0GCSqGSIb3DQEBCwUAMG0x +CzAJBgNVBAYTAlNFMRIwEAYDVQQIDAlTdG9ja2hvbG0xEjAQBgNVBAoMCU15T3Jn +TmFtZTEZMBcGA1UECwwQTXlJbnRlcm1lZGlhdGVDQTEbMBkGA1UEAwwSTXlJbnRl +cm1lZGlhdGVDQS0yMB4XDTIzMDEyMzE0MTYyNVoXDTMzMDEyMDE0MTYyNVowdzEL +MAkGA1UEBhMCU0UxEjAQBgNVBAgMCVN0b2NraG9sbTESMBAGA1UEBwwJU3RvY2to +b2xtMRIwEAYDVQQKDAlNeU9yZ05hbWUxGDAWBgNVBAsMD015U2VydmljZUNsaWVu +dDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB +CgKCAQEAytXSeNc6WekoWkbsy+gqqglztEep+RZUFSyGmPtCQhAyWxapWfTrSI/e +hoeb5R7sb39rN35ygXadLaaL+XpMGrxJhHk9C94fNBNtR8Bq2lRyC39lrgSs6cCZ +TtyJcL97NKnwA7+3Gij7T94kPBKSzGdqCa5eI3Sf/UrN44Veck6psSYGNoNRwDkL +aV2288p6AQt7bvGm1Xf9JHropj8INLK95mqYcX2yo1EEou3J/zcz/gbq5U8X8vnN +2o3qnRWKBuDNUwaahp9cW+4Jqt+0xcVxb345luScfcjbce/Ha6YIDiR3EWF1sxe/ +IBxBH/lWzDAQYkOgefbDmMer59pzOwIDAQABMA0GCSqGSIb3DQEBCwUAA4IBAQB/ +Y2uaU1KQOQILzLog0i5cUh76j3TegFaQhqQTNH1GDnPmkPaphWSb3Pm8jEODBT/+ +UpoGG7gfHEii6u+BkDiQLDOU5yS9pG98xbyTQL8E/o4l+I3CiM/YYRjGE20J8lP6 +BHgx/8GPAzV2hfIOqfJ1rb+BZ1LVvUh2brLVvyPZYyjpTiJcK0kwFL4KDOq0A3r1 +/rSs5wMOAhQIWy8vWw7TaDOgBkTJMqJUhpAfpGCRlVADg+i7fMe84ZsQBY4lpoM3 +OI2ulf03LqsyNRz/hgF2EgCWu3Qo/jRqHmWvhNxy9KQZ46raFc01wMz8j3nucxQS +0QULe+f7knniBqwddXFt +-----END CERTIFICATE----- +-----BEGIN CERTIFICATE----- +MIIEODCCAyCgAwIBAgIFFnRIM4UwDQYJKoZIhvcNAQELBQAwbzELMAkGA1UEBhMC +U0UxEjAQBgNVBAgMCVN0b2NraG9sbTESMBAGA1UEBwwJU3RvY2tob2xtMRIwEAYD +VQQKDAlNeU9yZ05hbWUxETAPBgNVBAsMCE15Um9vdENBMREwDwYDVQQDDAhNeVJv +b3RDQTAeFw0yMzAxMjMxNDE2MjVaFw0zMzAxMjAxNDE2MjVaMG0xCzAJBgNVBAYT +AlNFMRIwEAYDVQQIDAlTdG9ja2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTEZMBcG +A1UECwwQTXlJbnRlcm1lZGlhdGVDQTEbMBkGA1UEAwwSTXlJbnRlcm1lZGlhdGVD +QS0yMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAvjw+/i2D1FOz5tbr +FZds+l2PN7NCfq2hQkoI+7z3iibJwqpTG9FoJl5j+zsJPlAx7oPg3XFXiVyJ0ayg +FxJsfLn033uHvmQ4gsUxH7TuY8tdpIOa3klHAydU7ePHyGg/HLQCCxQw+JjYR/AX +Zje/yqnuvpXB5OHrtf9G16itVJRpovm4QuoO5n//9W/ImEpxnAjiIEAd168kVzIG +c3CKdvB/jRO63GX3o+nIYGtnJpJiB7MmzMLAnGT/UBoAR4//1vrbLQOjPg4PH/FO +YeyUshidVG8DZkwTTiswDvEYLTiGLPgpwAj9dXaY8X45OPicDarM26EKhX/QFCrv +5Vk7MwIDAQABo4HcMIHZMA4GA1UdDwEB/wQEAwIBBjAPBgNVHRMBAf8EBTADAQH/ +MB0GA1UdDgQWBBQe8N1/KwZug32rsZgUD1ZENKDfPjCBlgYDVR0jBIGOMIGLoXOk +cTBvMQswCQYDVQQGEwJTRTESMBAGA1UECAwJU3RvY2tob2xtMRIwEAYDVQQHDAlT +dG9ja2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTERMA8GA1UECwwITXlSb290Q0Ex +ETAPBgNVBAMMCE15Um9vdENBghRfGvHfslLmWduK/DoD5zdiKKaPsDANBgkqhkiG +9w0BAQsFAAOCAQEA09xjxwue/Sx4nPVp01FwNF0PEq04u8wc108TvTExjsPxaeKR +hAdbUf4UlrD/dwJVExI3jNhis9KP9OG7PCxKMkvOjmsxAK8tsY8MnPtzcUfRjxqN +i3UmF1hZjQ3MDm89Gqmg58muBiQJt4apFSQSD0hQjGjEzzTAmjN41vgbTRVdpqLM +Uf3b7cb26xtloEu2FALs5MLOJ1DTbRsjzJ8fmjGlEWw2h2HapnqRTqQSOUx2Nema +Ms/rDPsIt0Yavlp9VBKg213zRZRMqlFEg0GI5XoPouloUZQoxEgW13+iUD63e6ag +EfuWhS+zhXYrFrGDzg+ckx1/rWYLL3QrfK6+9Q== +-----END CERTIFICATE----- diff --git a/scripts/test/influx/certs/server.pem b/scripts/test/influx/certs/server.pem new file mode 100644 index 000000000..4c9201188 --- /dev/null +++ b/scripts/test/influx/certs/server.pem @@ -0,0 +1,48 @@ +-----BEGIN CERTIFICATE----- +MIIDvTCCAqWgAwIBAgIFFnRIM4UwDQYJKoZIhvcNAQELBQAwbTELMAkGA1UEBhMC +U0UxEjAQBgNVBAgMCVN0b2NraG9sbTESMBAGA1UECgwJTXlPcmdOYW1lMRkwFwYD +VQQLDBBNeUludGVybWVkaWF0ZUNBMRswGQYDVQQDDBJNeUludGVybWVkaWF0ZUNB +LTEwHhcNMjMwMTIzMTQxNjI1WhcNMzMwMTIwMTQxNjI1WjBrMQswCQYDVQQGEwJT +RTESMBAGA1UECAwJU3RvY2tob2xtMRIwEAYDVQQKDAlNeU9yZ05hbWUxEjAQBgNV +BAsMCU15U2VydmljZTEgMB4GA1UEAwwXaW5mbHV4ZGIuZW1xeC5pbmZsdXguaW8w +ggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQDstiij2PRYssNxP5k+0YlW +Lgx57xOz5zpzjyjavDvFUmboEJH2VG2A7ITQtXlTqrqB3rxbGEDYjltmRG/tjm4g +5GQ3RgfupZdsNommgzE4/bH0PT6PwYTlt83596Vu9hFqUF5a0rtYF8RJgXHXzDer +az25JoLcuNgGlSrpgioryx6NZB/oBkbuHbGL7fmYuRO8zQSxknHtg/JrK4YqQaw6 +6VFQoydyzcGp9b8KjMYjyBVjNvt7xbvvCLrMZ6XMAhPYyquRhQC00ydJOHavzcau +ARAQH20noHpKU8/W8HSwBOKMsYdh0vxLnFwnrP9qOjMoB1G6DtpL8yGcdsWSkXJT +AgMBAAGjZjBkMCIGA1UdEQQbMBmCF2luZmx1eGRiLmVtcXguaW5mbHV4LmlvMB0G +A1UdDgQWBBSz+wgHJ2TZNGasVR+BSMqRpg3pSTAfBgNVHSMEGDAWgBRfr50a+u+U +2sPlCD7ft2gOjNIdOjANBgkqhkiG9w0BAQsFAAOCAQEAAInxCAHkEVbPQhijpl+C +vblGHxS4aPGGAmIU7rS2CsZQaXRLZNC++syMZQzOKtLOIzLT8WSgs026/tLsFQk0 +5VCLbgnQqS2wblFylGribUiT/dSMmiMvGylc6MHJnbEyiNKtlB88paTLuDC81lIZ +KDBEYSQTfaykm3MXjeZGB5sHBcKeBWCkDY9O1YvsP/5a9zyZiC1c+XTTlB5Lnzgo +aL7SWB88J+mF0cn5iwitwTg0ryZtO5fG8qmqaAs+G8u9oVytnxi+WBmjukjnFdzC +SlRFYGY0KZmg1ZGlLCzBS3Bi3q+ck8yfv2rj2WBqklOISzSnp5VaRC6gcUDNE0XY +1g== +-----END CERTIFICATE----- +-----BEGIN CERTIFICATE----- +MIIEODCCAyCgAwIBAgIFFnRIM4UwDQYJKoZIhvcNAQELBQAwbzELMAkGA1UEBhMC +U0UxEjAQBgNVBAgMCVN0b2NraG9sbTESMBAGA1UEBwwJU3RvY2tob2xtMRIwEAYD +VQQKDAlNeU9yZ05hbWUxETAPBgNVBAsMCE15Um9vdENBMREwDwYDVQQDDAhNeVJv +b3RDQTAeFw0yMzAxMjMxNDE2MjVaFw0zMzAxMjAxNDE2MjVaMG0xCzAJBgNVBAYT +AlNFMRIwEAYDVQQIDAlTdG9ja2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTEZMBcG +A1UECwwQTXlJbnRlcm1lZGlhdGVDQTEbMBkGA1UEAwwSTXlJbnRlcm1lZGlhdGVD +QS0xMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEApYFU7RYTvI8eH6Y6 +yVgjwWWMWSqU3Q97G8l2zHcgOCrph8xQRM8tp0wsoPpvj6SYvohzPJyIxK6tFiQg +FvPrk4JQdQWNEK7ogRwqbeFX017vVsQIDMAsO5V5cqoRjOghV1QvHkz5+oVQt09w +whwk47IrsKhwiXtQrFuGi3w7WNTkwh1FX3V8ZrqCVr02/f1GmODYxPqvuXVtf42B +iewWOvnHs58vyMPvts/Ngh4HBzQncSudeI9GDsPscAn6I5YmjBQkGsQQb7cygd4p +u3OrMw/Sw2e85N6fBM6+b4ZTUEaz6EC2ymBNiJtkLDzkM3ZaP19GluYy17PBi8ZB +UO+rowIDAQABo4HcMIHZMA4GA1UdDwEB/wQEAwIBBjAPBgNVHRMBAf8EBTADAQH/ +MB0GA1UdDgQWBBRfr50a+u+U2sPlCD7ft2gOjNIdOjCBlgYDVR0jBIGOMIGLoXOk +cTBvMQswCQYDVQQGEwJTRTESMBAGA1UECAwJU3RvY2tob2xtMRIwEAYDVQQHDAlT +dG9ja2hvbG0xEjAQBgNVBAoMCU15T3JnTmFtZTERMA8GA1UECwwITXlSb290Q0Ex +ETAPBgNVBAMMCE15Um9vdENBghRfGvHfslLmWduK/DoD5zdiKKaPsDANBgkqhkiG +9w0BAQsFAAOCAQEAKoEj6H7tBDiD/sqPuxABGJI4dT6jqSFskF2An95FlwTHLu4b +kBxfb1r3emaqM6t+dgEjupqa6Mte9/rQmynGn+U9IFn3dOo95FPmf8igQGGcO7Tz +Wl1pTm2S7m5+fQAgC94jmBk8JUXO+fGJe33Mx+os1ZaE26KvbhQzTk8NpYfK+cp2 ++K4HsaVogr+mR139YNbKD00xexGQdxPzdBO4EHEsCI+6W7ELFz/iaj7S6XTe9YW2 +Uj858vOWhL6bg2zWhb/ER2MqbB05gOQhf8Rw+4iL+45AiGFrdyD4jXaqMy7lR4dv +fBlPwX60owxqgtKc4QC8/rSKLteSey01WV1SXA== +-----END CERTIFICATE----- From 28718edbfd7f7bad68bcdc6bc3e8ae25071be862 Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Tue, 24 Jan 2023 14:12:34 +0100 Subject: [PATCH 07/15] chore: bump application VSNs --- apps/emqx_bridge/src/emqx_bridge.app.src | 2 +- apps/emqx_connector/src/emqx_connector.app.src | 2 +- apps/emqx_resource/src/emqx_resource.app.src | 2 +- lib-ee/emqx_ee_bridge/src/emqx_ee_bridge.app.src | 2 +- lib-ee/emqx_ee_connector/src/emqx_ee_connector.app.src | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/apps/emqx_bridge/src/emqx_bridge.app.src b/apps/emqx_bridge/src/emqx_bridge.app.src index 39cb1b18b..ca0001319 100644 --- a/apps/emqx_bridge/src/emqx_bridge.app.src +++ b/apps/emqx_bridge/src/emqx_bridge.app.src @@ -1,7 +1,7 @@ %% -*- mode: erlang -*- {application, emqx_bridge, [ {description, "EMQX bridges"}, - {vsn, "0.1.9"}, + {vsn, "0.1.10"}, {registered, []}, {mod, {emqx_bridge_app, []}}, {applications, [ diff --git a/apps/emqx_connector/src/emqx_connector.app.src b/apps/emqx_connector/src/emqx_connector.app.src index 9a82bda27..2e955a5e9 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, "EMQX Data Integration Connectors"}, - {vsn, "0.1.12"}, + {vsn, "0.1.13"}, {registered, []}, {mod, {emqx_connector_app, []}}, {applications, [ diff --git a/apps/emqx_resource/src/emqx_resource.app.src b/apps/emqx_resource/src/emqx_resource.app.src index e3a37fd10..6bd9fd213 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.5"}, + {vsn, "0.1.6"}, {registered, []}, {mod, {emqx_resource_app, []}}, {applications, [ diff --git a/lib-ee/emqx_ee_bridge/src/emqx_ee_bridge.app.src b/lib-ee/emqx_ee_bridge/src/emqx_ee_bridge.app.src index 11831ab06..b2a3c80c6 100644 --- a/lib-ee/emqx_ee_bridge/src/emqx_ee_bridge.app.src +++ b/lib-ee/emqx_ee_bridge/src/emqx_ee_bridge.app.src @@ -1,6 +1,6 @@ {application, emqx_ee_bridge, [ {description, "EMQX Enterprise data bridges"}, - {vsn, "0.1.3"}, + {vsn, "0.1.4"}, {registered, []}, {applications, [ kernel, diff --git a/lib-ee/emqx_ee_connector/src/emqx_ee_connector.app.src b/lib-ee/emqx_ee_connector/src/emqx_ee_connector.app.src index 15cafa6a4..56d128601 100644 --- a/lib-ee/emqx_ee_connector/src/emqx_ee_connector.app.src +++ b/lib-ee/emqx_ee_connector/src/emqx_ee_connector.app.src @@ -1,6 +1,6 @@ {application, emqx_ee_connector, [ {description, "EMQX Enterprise connectors"}, - {vsn, "0.1.3"}, + {vsn, "0.1.4"}, {registered, []}, {applications, [ kernel, From fbfd4a25651db6c2f1a87e4b020dd906511676f5 Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Tue, 24 Jan 2023 14:13:03 +0100 Subject: [PATCH 08/15] test: use EMQX_IMAGE_TAG for influx test script --- scripts/test/influx/docker-compose.yaml | 4 ++-- scripts/test/influx/start.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/scripts/test/influx/docker-compose.yaml b/scripts/test/influx/docker-compose.yaml index 60b4c3421..4ff7ec6bf 100644 --- a/scripts/test/influx/docker-compose.yaml +++ b/scripts/test/influx/docker-compose.yaml @@ -34,7 +34,7 @@ services: emqx_1: container_name: emqx_1 - image: emqx/emqx-enterprise:${EMQX_TAG} + image: emqx/emqx-enterprise:${EMQX_IMAGE_TAG} hostname: emqx_1.emqx.influx.io ports: - "1883:1883" @@ -50,7 +50,7 @@ services: emqx_2: container_name: emqx_2 - image: emqx/emqx-enterprise:${EMQX_TAG} + image: emqx/emqx-enterprise:${EMQX_IMAGE_TAG} hostname: emqx_1.emqx.influx.io environment: EMQX_LOG__CONSOLE_HANDLER__LEVEL: warning diff --git a/scripts/test/influx/start.sh b/scripts/test/influx/start.sh index 7ce9c399c..d06a2d44a 100755 --- a/scripts/test/influx/start.sh +++ b/scripts/test/influx/start.sh @@ -3,7 +3,7 @@ NET='emqx.influx.io' NODE1="emqx@emqx_1.$NET" NODE2="emqx@emqx_2.$NET" -export EMQX_TAG='5.0.0-rc.1-g92797d72' +export EMQX_IMAGE_TAG="${EMQX_IMAGE_TAG:-latest}" cd -P -- "$(dirname -- "${BASH_SOURCE[0]}")" From 9d20431257a45af08e0ed4e43df1c4675d00b5b1 Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Tue, 24 Jan 2023 14:13:35 +0100 Subject: [PATCH 09/15] fix(emqx_resource): fix crash while flushing queue We used next_event for flushing the queue in emqx_resource, but this leads to a crash. We now call flush_worker/1 instead. --- .../src/emqx_resource_buffer_worker.erl | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl index 669b8e474..634b0e954 100644 --- a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl @@ -583,10 +583,11 @@ do_flush( ), case queue_count(Q1) > 0 of true -> - {keep_state, Data1, [{next_event, internal, flush}]}; + flush_worker(self()); false -> - {keep_state, Data1} - end + ok + end, + {keep_state, Data1} end; do_flush(Data0, #{ is_batch := true, @@ -659,15 +660,17 @@ do_flush(Data0, #{ } ), CurrentCount = queue_count(Q1), - case {CurrentCount > 0, CurrentCount >= BatchSize} of - {false, _} -> - {keep_state, Data1}; - {true, true} -> - {keep_state, Data1, [{next_event, internal, flush}]}; - {true, false} -> - Data2 = ensure_flush_timer(Data1), - {keep_state, Data2} - end + Data2 = + case {CurrentCount > 0, CurrentCount >= BatchSize} of + {false, _} -> + Data1; + {true, true} -> + flush_worker(self()), + Data1; + {true, false} -> + ensure_flush_timer(Data1) + end, + {keep_state, Data2} end. batch_reply_caller(Id, BatchResult, Batch, QueryOpts) -> From 7603ab490b2e01a0f65612437edeccc7cbdd4f31 Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Mon, 23 Jan 2023 15:02:23 +0100 Subject: [PATCH 10/15] fix(emqx_ee_connector): fix passing of influxdb ssl opts --- lib-ee/emqx_ee_connector/src/emqx_ee_connector_influxdb.erl | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 1ae4d9874..824233a6d 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 @@ -324,8 +324,9 @@ ssl_config(#{enable := false}) -> ssl_config(SSL = #{enable := true}) -> [ {https_enabled, true}, - {transport, ssl} - ] ++ maps:to_list(maps:remove(enable, SSL)). + {transport, ssl}, + {transport_opts, maps:to_list(maps:remove(enable, SSL))} + ]. username(#{username := Username}) -> [{username, str(Username)}]; From 21e933e9bb20b92280569b578f348560d79f2c2d Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Tue, 24 Jan 2023 14:37:15 +0100 Subject: [PATCH 11/15] test: add rule to influx-bridge.conf and add newlines --- scripts/test/influx/emqx.conf | 2 +- scripts/test/influx/influx-bridge.conf | 19 ++++++++++++++++--- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/scripts/test/influx/emqx.conf b/scripts/test/influx/emqx.conf index 8fd114f50..cd8571f98 100644 --- a/scripts/test/influx/emqx.conf +++ b/scripts/test/influx/emqx.conf @@ -91,4 +91,4 @@ authorization { } include emqx-enterprise.conf -include influx-bridge.conf \ No newline at end of file +include influx-bridge.conf diff --git a/scripts/test/influx/influx-bridge.conf b/scripts/test/influx/influx-bridge.conf index b513b962f..31ddeaf79 100644 --- a/scripts/test/influx/influx-bridge.conf +++ b/scripts/test/influx/influx-bridge.conf @@ -22,11 +22,11 @@ bridges { ssl { ciphers = [] depth = 10 - enable = false + enable = true reuse_sessions = true secure_renegotiate = true user_lookup_fun = "emqx_tls_psk:lookup" - verify = "verify_peer" + verify = "verify_none" versions = ["tlsv1.3", "tlsv1.2", "tlsv1.1", "tlsv1"] } token = "abcdefg" @@ -43,4 +43,17 @@ log { level = "warning" } } -} \ No newline at end of file +} +rule_engine { + ignore_sys_message = true + jq_function_default_timeout = "10s" + jq_implementation_module = "jq_nif" + rules { + "rule_o75y" { + actions = ["influxdb_api_v2:test"] + description = "" + metadata {created_at = 1674565304418} + sql = "SELECT\n *\nFROM\n \"t/#\"" + } + } +} From 424d6ed34b9d1e76a11dfea68f3ea9e3e7a1298b Mon Sep 17 00:00:00 2001 From: Erik Timan Date: Tue, 24 Jan 2023 14:42:52 +0100 Subject: [PATCH 12/15] test: improve start.sh script for influxdb tests --- scripts/test/influx/start.sh | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/scripts/test/influx/start.sh b/scripts/test/influx/start.sh index d06a2d44a..f430d353a 100755 --- a/scripts/test/influx/start.sh +++ b/scripts/test/influx/start.sh @@ -1,13 +1,15 @@ #!/usr/bin/env bash NET='emqx.influx.io' -NODE1="emqx@emqx_1.$NET" -NODE2="emqx@emqx_2.$NET" +NODE2_FQDN="emqx@emqx_2.$NET" +NODE1_CONTAINER_NAME="emqx_1" +NODE2_CONTAINER_NAME="emqx_2" +INFLUXDB_CONTAINER_NAME="influxdb_server" export EMQX_IMAGE_TAG="${EMQX_IMAGE_TAG:-latest}" -cd -P -- "$(dirname -- "${BASH_SOURCE[0]}")" +cd -P -- "$(dirname -- "${BASH_SOURCE[0]}")" || exit -docker rm -f emqx_1 emqx_2 influxdb_server +docker rm -f "$NODE1_CONTAINER_NAME" "$NODE2_CONTAINER_NAME" "$INFLUXDB_CONTAINER_NAME" docker-compose up -d wait_limit=60 @@ -26,9 +28,9 @@ wait_for_emqx() { done } -wait_for_emqx emqx_1 30 -wait_for_emqx emqx_2 30 +wait_for_emqx "$NODE1_CONTAINER_NAME" 30 +wait_for_emqx "$NODE2_CONTAINER_NAME" 30 echo -docker exec emqx_1 emqx_ctl cluster join "$NODE2" +docker exec "$NODE1_CONTAINER_NAME" emqx_ctl cluster join "$NODE2_FQDN" From 2416aeebc76447b7efa661f06daf236592e29cbf Mon Sep 17 00:00:00 2001 From: "Zaiming (Stone) Shi" Date: Mon, 23 Jan 2023 19:55:14 +0100 Subject: [PATCH 13/15] build: allow user-specified image tag for make docker build --- build | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/build b/build index 87c719a53..06b1d29e2 100755 --- a/build +++ b/build @@ -332,13 +332,15 @@ make_docker() { if [[ "$PROFILE" = *-elixir ]]; then PKG_VSN="$PKG_VSN-elixir" fi + local default_tag="emqx/${PROFILE%%-elixir}:${PKG_VSN}" + EMQX_IMAGE_TAG="${EMQX_IMAGE_TAG:-$default_tag}" set -x docker build --no-cache --pull \ --build-arg BUILD_FROM="${EMQX_BUILDER}" \ --build-arg RUN_FROM="${EMQX_RUNNER}" \ --build-arg EMQX_NAME="$PROFILE" \ - --tag "emqx/${PROFILE%%-elixir}:${PKG_VSN}" \ + --tag "${EMQX_IMAGE_TAG}" \ -f "${EMQX_DOCKERFILE}" . } From feb02389118fe3d5196e45d9a9e3c48512170f00 Mon Sep 17 00:00:00 2001 From: "Zaiming (Stone) Shi" Date: Tue, 24 Jan 2023 12:54:42 +0100 Subject: [PATCH 14/15] fix(influxdb): ensure client is stopped before returning error --- lib-ee/emqx_ee_connector/src/emqx_ee_connector_influxdb.erl | 2 ++ 1 file changed, 2 insertions(+) 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 824233a6d..0ee27c5c4 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 @@ -248,6 +248,8 @@ do_start_client( client => Client, reason => "client is not alive" }), + %% no leak + _ = influxdb:stop_client(Client), {error, influxdb_client_not_alive} end; {error, {already_started, Client0}} -> From 5fdf7fd24c0fc9053b7ad99debbb197c9887b447 Mon Sep 17 00:00:00 2001 From: "Zaiming (Stone) Shi" Date: Thu, 19 Jan 2023 12:39:42 +0100 Subject: [PATCH 15/15] fix(kafka): use async callback to bump success counters some telemetry events from wolff are discarded: * dropped: this is double counted in wolff, we now only subscribe to the dropped_queue_full event * retried_failed: it has different meanings in wolff, in wolff, it means it's the 2nd (or onward) produce attempt in EMQX, it means it's eventually failed after some retries * retried_success since we are going to handle the success counters in callbac this having this reported from wolff will only make things harder to understand * failed wolff never fails (unelss drop which is a different counter) --- apps/emqx_bridge/src/emqx_bridge.erl | 16 ++- apps/emqx_resource/include/emqx_resource.hrl | 2 + apps/emqx_resource/src/emqx_resource.erl | 5 +- .../src/emqx_resource_buffer_worker.erl | 70 ++++++++----- .../src/kafka/emqx_bridge_impl_kafka.erl | 8 +- .../kafka/emqx_bridge_impl_kafka_producer.erl | 98 ++++++++----------- .../emqx_bridge_impl_kafka_producer_SUITE.erl | 84 +++++++++++----- scripts/ct/run.sh | 2 +- 8 files changed, 172 insertions(+), 113 deletions(-) diff --git a/apps/emqx_bridge/src/emqx_bridge.erl b/apps/emqx_bridge/src/emqx_bridge.erl index 5b3fe796b..fb199522d 100644 --- a/apps/emqx_bridge/src/emqx_bridge.erl +++ b/apps/emqx_bridge/src/emqx_bridge.erl @@ -171,14 +171,22 @@ send_message(BridgeId, Message) -> not_found -> {error, {bridge_not_found, BridgeId}}; #{enable := true} = Config -> - Timeout = emqx_map_lib:deep_get( - [resource_opts, request_timeout], Config, timer:seconds(15) - ), - emqx_resource:query(ResId, {send_message, Message}, #{timeout => Timeout}); + QueryOpts = query_opts(Config), + emqx_resource:query(ResId, {send_message, Message}, QueryOpts); #{enable := false} -> {error, {bridge_stopped, BridgeId}} end. +query_opts(Config) -> + case emqx_map_lib:deep_get([resource_opts, request_timeout], Config, false) of + Timeout when is_integer(Timeout) -> + %% request_timeout is configured + #{timeout => Timeout}; + _ -> + %% emqx_resource has a default value (15s) + #{} + end. + config_key_path() -> [bridges]. diff --git a/apps/emqx_resource/include/emqx_resource.hrl b/apps/emqx_resource/include/emqx_resource.hrl index 03be8fae8..7464eb4f8 100644 --- a/apps/emqx_resource/include/emqx_resource.hrl +++ b/apps/emqx_resource/include/emqx_resource.hrl @@ -89,6 +89,8 @@ -define(DEFAULT_QUEUE_SIZE, 100 * 1024 * 1024). -define(DEFAULT_QUEUE_SIZE_RAW, <<"100MB">>). +-define(DEFAULT_REQUEST_TIMEOUT, timer:seconds(15)). + %% count -define(DEFAULT_BATCH_SIZE, 1). diff --git a/apps/emqx_resource/src/emqx_resource.erl b/apps/emqx_resource/src/emqx_resource.erl index bb27b6acd..ad7f30b47 100644 --- a/apps/emqx_resource/src/emqx_resource.erl +++ b/apps/emqx_resource/src/emqx_resource.erl @@ -255,7 +255,7 @@ reset_metrics(ResId) -> query(ResId, Request) -> query(ResId, Request, #{}). --spec query(resource_id(), Request :: term(), emqx_resource_buffer_worker:query_opts()) -> +-spec query(resource_id(), Request :: term(), query_opts()) -> Result :: term(). query(ResId, Request, Opts) -> case emqx_resource_manager:ets_lookup(ResId) of @@ -263,7 +263,8 @@ query(ResId, Request, Opts) -> IsBufferSupported = is_buffer_supported(Module), case {IsBufferSupported, QM} of {true, _} -> - emqx_resource_buffer_worker:simple_sync_query(ResId, Request); + %% only Kafka so far + emqx_resource_buffer_worker:simple_async_query(ResId, Request); {false, sync} -> emqx_resource_buffer_worker:sync_query(ResId, Request, Opts); {false, async} -> diff --git a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl index 634b0e954..11d3753f0 100644 --- a/apps/emqx_resource/src/emqx_resource_buffer_worker.erl +++ b/apps/emqx_resource/src/emqx_resource_buffer_worker.erl @@ -38,7 +38,8 @@ ]). -export([ - simple_sync_query/2 + simple_sync_query/2, + simple_async_query/2 ]). -export([ @@ -61,6 +62,7 @@ -define(COLLECT_REQ_LIMIT, 1000). -define(SEND_REQ(FROM, REQUEST), {'$send_req', FROM, REQUEST}). -define(QUERY(FROM, REQUEST, SENT, EXPIRE_AT), {query, FROM, REQUEST, SENT, EXPIRE_AT}). +-define(SIMPLE_QUERY(REQUEST), ?QUERY(undefined, REQUEST, false, infinity)). -define(REPLY(FROM, REQUEST, SENT, RESULT), {reply, FROM, REQUEST, SENT, RESULT}). -define(EXPAND(RESULT, BATCH), [ ?REPLY(FROM, REQUEST, SENT, RESULT) @@ -116,8 +118,8 @@ async_query(Id, Request, Opts0) -> emqx_resource_metrics:matched_inc(Id), pick_cast(Id, PickKey, {query, Request, Opts}). -%% simple query the resource without batching and queuing messages. --spec simple_sync_query(id(), request()) -> Result :: term(). +%% simple query the resource without batching and queuing. +-spec simple_sync_query(id(), request()) -> term(). simple_sync_query(Id, Request) -> %% Note: since calling this function implies in bypassing the %% buffer workers, and each buffer worker index is used when @@ -126,18 +128,27 @@ simple_sync_query(Id, Request) -> %% would mess up the metrics anyway. `undefined' is ignored by %% `emqx_resource_metrics:*_shift/3'. Index = undefined, - QueryOpts0 = #{simple_query => true, timeout => infinity}, - QueryOpts = #{expire_at := ExpireAt} = ensure_expire_at(QueryOpts0), + QueryOpts = simple_query_opts(), emqx_resource_metrics:matched_inc(Id), Ref = make_message_ref(), - HasBeenSent = false, - From = self(), - Result = call_query( - sync, Id, Index, Ref, ?QUERY(From, Request, HasBeenSent, ExpireAt), QueryOpts - ), - _ = handle_query_result(Id, Result, HasBeenSent), + Result = call_query(sync, Id, Index, Ref, ?SIMPLE_QUERY(Request), QueryOpts), + _ = handle_query_result(Id, Result, _HasBeenSent = false), Result. +%% simple async-query the resource without batching and queuing. +-spec simple_async_query(id(), request()) -> term(). +simple_async_query(Id, Request) -> + Index = undefined, + QueryOpts = simple_query_opts(), + emqx_resource_metrics:matched_inc(Id), + Ref = make_message_ref(), + Result = call_query(async, Id, Index, Ref, ?SIMPLE_QUERY(Request), QueryOpts), + _ = handle_query_result(Id, Result, _HasBeenSent = false), + Result. + +simple_query_opts() -> + ensure_expire_at(#{simple_query => true, timeout => infinity}). + -spec block(pid()) -> ok. block(ServerRef) -> gen_statem:cast(ServerRef, block). @@ -848,9 +859,9 @@ call_query(QM0, Id, Index, Ref, Query, QueryOpts) -> case emqx_resource_manager:ets_lookup(Id) of {ok, _Group, #{mod := Mod, state := ResSt, status := connected} = Data} -> QM = - case QM0 of - configured -> maps:get(query_mode, Data); - _ -> QM0 + case QM0 =:= configured of + true -> maps:get(query_mode, Data); + false -> QM0 end, CBM = maps:get(callback_mode, Data), CallMode = call_mode(QM, CBM), @@ -991,11 +1002,7 @@ do_reply_after_query( ref => Ref, result => Result }), - IsFullBefore = is_inflight_full(InflightTID), - IsAcked = ack_inflight(InflightTID, Ref, Id, Index), - IsAcked andalso PostFn(), - IsFullBefore andalso ?MODULE:flush_worker(Pid), - ok + do_ack(InflightTID, Ref, Id, Index, PostFn, Pid, QueryOpts) end. batch_reply_after_query(Pid, Id, Index, InflightTID, Ref, Batch, QueryOpts, Result) -> @@ -1049,13 +1056,23 @@ do_batch_reply_after_query(Pid, Id, Index, InflightTID, Ref, Batch, QueryOpts, R ref => Ref, result => Result }), - IsFullBefore = is_inflight_full(InflightTID), - IsAcked = ack_inflight(InflightTID, Ref, Id, Index), - IsAcked andalso PostFn(), - IsFullBefore andalso ?MODULE:flush_worker(Pid), - ok + do_ack(InflightTID, Ref, Id, Index, PostFn, Pid, QueryOpts) end. +do_ack(InflightTID, Ref, Id, Index, PostFn, WorkerPid, QueryOpts) -> + IsFullBefore = is_inflight_full(InflightTID), + IsKnownRef = ack_inflight(InflightTID, Ref, Id, Index), + case maps:get(simple_query, QueryOpts, false) of + true -> + PostFn(); + false when IsKnownRef -> + PostFn(); + false -> + ok + end, + IsFullBefore andalso ?MODULE:flush_worker(WorkerPid), + ok. + %%============================================================================== %% operations for queue queue_item_marshaller(Bin) when is_binary(Bin) -> @@ -1113,7 +1130,7 @@ inflight_new(InfltWinSZ, Id, Index) -> inflight_append(TableId, {?SIZE_REF, 0}, Id, Index), inflight_append(TableId, {?INITIAL_TIME_REF, erlang:system_time()}, Id, Index), inflight_append( - TableId, {?INITIAL_MONOTONIC_TIME_REF, erlang:monotonic_time(nanosecond)}, Id, Index + TableId, {?INITIAL_MONOTONIC_TIME_REF, make_message_ref()}, Id, Index ), TableId. @@ -1426,8 +1443,7 @@ now_() -> ensure_timeout_query_opts(#{timeout := _} = Opts, _SyncOrAsync) -> Opts; ensure_timeout_query_opts(#{} = Opts0, sync) -> - TimeoutMS = timer:seconds(15), - Opts0#{timeout => TimeoutMS}; + Opts0#{timeout => ?DEFAULT_REQUEST_TIMEOUT}; ensure_timeout_query_opts(#{} = Opts0, async) -> Opts0#{timeout => infinity}. diff --git a/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka.erl b/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka.erl index 747cd187d..49ca9fb86 100644 --- a/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka.erl +++ b/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka.erl @@ -12,6 +12,7 @@ on_start/2, on_stop/2, on_query/3, + on_query_async/4, on_get_status/2, is_buffer_supported/0 ]). @@ -26,8 +27,11 @@ on_start(InstId, Config) -> on_stop(InstId, State) -> emqx_bridge_impl_kafka_producer:on_stop(InstId, State). -on_query(InstId, Msg, State) -> - emqx_bridge_impl_kafka_producer:on_query(InstId, Msg, State). +on_query(InstId, Req, State) -> + emqx_bridge_impl_kafka_producer:on_query(InstId, Req, State). + +on_query_async(InstId, Req, ReplyFn, State) -> + emqx_bridge_impl_kafka_producer:on_query_async(InstId, Req, ReplyFn, State). on_get_status(InstId, State) -> emqx_bridge_impl_kafka_producer:on_get_status(InstId, State). diff --git a/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka_producer.erl b/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka_producer.erl index 25741b6cd..18e27b775 100644 --- a/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka_producer.erl +++ b/lib-ee/emqx_ee_bridge/src/kafka/emqx_bridge_impl_kafka_producer.erl @@ -11,6 +11,7 @@ on_start/2, on_stop/2, on_query/3, + on_query_async/4, on_get_status/2 ]). @@ -140,19 +141,48 @@ on_stop(_InstanceID, #{client_id := ClientID, producers := Producers, resource_i } ). +on_query( + _InstId, + {send_message, Message}, + #{message_template := Template, producers := Producers} +) -> + KafkaMessage = render_message(Template, Message), + %% TODO: this function is not used so far, + %% timeout should be configurable + %% or the on_query/3 should be on_query/4 instead. + try + {_Partition, _Offset} = wolff:send_sync(Producers, [KafkaMessage], 5000), + ok + catch + error:{producer_down, _} = Reason -> + {error, Reason}; + error:timeout -> + {error, timeout} + end. + %% @doc The callback API for rule-engine (or bridge without rules) %% The input argument `Message' is an enriched format (as a map()) %% of the original #message{} record. %% The enrichment is done by rule-engine or by the data bridge framework. %% E.g. the output of rule-engine process chain %% or the direct mapping from an MQTT message. -on_query(_InstId, {send_message, Message}, #{message_template := Template, producers := Producers}) -> +on_query_async( + _InstId, + {send_message, Message}, + AsyncReplyFn, + #{message_template := Template, producers := Producers} +) -> KafkaMessage = render_message(Template, Message), + %% * Must be a batch because wolff:send and wolff:send_sync are batch APIs + %% * Must be a single element batch because wolff books calls, but not batch sizes + %% for counters and gauges. + Batch = [KafkaMessage], %% The retuned information is discarded here. %% If the producer process is down when sending, this function would %% raise an error exception which is to be caught by the caller of this callback - {_Partition, _Pid} = wolff:send(Producers, [KafkaMessage], {fun ?MODULE:on_kafka_ack/3, [#{}]}), - {async_return, ok}. + {_Partition, Pid} = wolff:send(Producers, Batch, {fun ?MODULE:on_kafka_ack/3, [AsyncReplyFn]}), + %% this Pid is so far never used because Kafka producer is by-passing the buffer worker + {ok, Pid}. compile_message_template(T) -> KeyTemplate = maps:get(key, T, <<"${.clientid}">>), @@ -194,9 +224,14 @@ render_timestamp(Template, Message) -> erlang:system_time(millisecond) end. -on_kafka_ack(_Partition, _Offset, _Extra) -> - %% Do nothing so far. - %% Maybe need to bump some counters? +%% Wolff producer never gives up retrying +%% so there can only be 'ok' results. +on_kafka_ack(_Partition, Offset, {ReplyFn, Args}) when is_integer(Offset) -> + %% the ReplyFn is emqx_resource_worker:reply_after_query/8 + apply(ReplyFn, Args ++ [ok]); +on_kafka_ack(_Partition, buffer_overflow_discarded, _Callback) -> + %% wolff should bump the dropped_queue_full counter + %% do not apply the callback (which is basically to bump success or fail counter) ok. on_get_status(_InstId, _State) -> @@ -345,27 +380,13 @@ get_required(Field, Config, Throw) -> %% we *must* match the bridge id in the event metadata with that in %% the handler config; otherwise, multiple kafka producer bridges will %% install multiple handlers to the same wolff events, multiplying the -handle_telemetry_event( - [wolff, dropped], - #{counter_inc := Val}, - #{bridge_id := ID}, - #{bridge_id := ID} -) when is_integer(Val) -> - emqx_resource_metrics:dropped_inc(ID, Val); handle_telemetry_event( [wolff, dropped_queue_full], #{counter_inc := Val}, #{bridge_id := ID}, #{bridge_id := ID} ) when is_integer(Val) -> - %% When wolff emits a `dropped_queue_full' event due to replayq - %% overflow, it also emits a `dropped' event (at the time of - %% writing, wolff is 1.7.4). Since we already bump `dropped' when - %% `dropped.queue_full' occurs, we have to correct it here. This - %% correction will have to be dropped if wolff stops also emitting - %% `dropped'. - emqx_resource_metrics:dropped_queue_full_inc(ID, Val), - emqx_resource_metrics:dropped_inc(ID, -Val); + emqx_resource_metrics:dropped_queue_full_inc(ID, Val); handle_telemetry_event( [wolff, queuing], #{gauge_set := Val}, @@ -380,13 +401,6 @@ handle_telemetry_event( #{bridge_id := ID} ) when is_integer(Val) -> emqx_resource_metrics:retried_inc(ID, Val); -handle_telemetry_event( - [wolff, failed], - #{counter_inc := Val}, - #{bridge_id := ID}, - #{bridge_id := ID} -) when is_integer(Val) -> - emqx_resource_metrics:failed_inc(ID, Val); handle_telemetry_event( [wolff, inflight], #{gauge_set := Val}, @@ -394,27 +408,6 @@ handle_telemetry_event( #{bridge_id := ID} ) when is_integer(Val) -> emqx_resource_metrics:inflight_set(ID, PartitionID, Val); -handle_telemetry_event( - [wolff, retried_failed], - #{counter_inc := Val}, - #{bridge_id := ID}, - #{bridge_id := ID} -) when is_integer(Val) -> - emqx_resource_metrics:retried_failed_inc(ID, Val); -handle_telemetry_event( - [wolff, retried_success], - #{counter_inc := Val}, - #{bridge_id := ID}, - #{bridge_id := ID} -) when is_integer(Val) -> - emqx_resource_metrics:retried_success_inc(ID, Val); -handle_telemetry_event( - [wolff, success], - #{counter_inc := Val}, - #{bridge_id := ID}, - #{bridge_id := ID} -) when is_integer(Val) -> - emqx_resource_metrics:success_inc(ID, Val); handle_telemetry_event(_EventId, _Metrics, _MetaData, _HandlerConfig) -> %% Event that we do not handle ok. @@ -437,15 +430,10 @@ maybe_install_wolff_telemetry_handlers(ResourceID) -> %% unique handler id telemetry_handler_id(ResourceID), [ - [wolff, dropped], [wolff, dropped_queue_full], [wolff, queuing], [wolff, retried], - [wolff, failed], - [wolff, inflight], - [wolff, retried_failed], - [wolff, retried_success], - [wolff, success] + [wolff, inflight] ], fun ?MODULE:handle_telemetry_event/4, %% we *must* keep track of the same id that is handed down to diff --git a/lib-ee/emqx_ee_bridge/test/emqx_bridge_impl_kafka_producer_SUITE.erl b/lib-ee/emqx_ee_bridge/test/emqx_bridge_impl_kafka_producer_SUITE.erl index 14567dd39..10a6d8d9a 100644 --- a/lib-ee/emqx_ee_bridge/test/emqx_bridge_impl_kafka_producer_SUITE.erl +++ b/lib-ee/emqx_ee_bridge/test/emqx_bridge_impl_kafka_producer_SUITE.erl @@ -46,7 +46,14 @@ %%------------------------------------------------------------------------------ all() -> - emqx_common_test_helpers:all(?MODULE). + [ + {group, on_query}, + {group, on_query_async} + ]. + +groups() -> + All = emqx_common_test_helpers:all(?MODULE), + [{on_query, All}, {on_query_async, All}]. wait_until_kafka_is_up() -> wait_until_kafka_is_up(0). @@ -89,6 +96,12 @@ end_per_suite(_Config) -> _ = application:stop(emqx_connector), ok. +init_per_group(GroupName, Config) -> + [{query_api, GroupName} | Config]. + +end_per_group(_, _) -> + ok. + set_special_configs(emqx_management) -> Listeners = #{http => #{port => 8081}}, Config = #{ @@ -106,23 +119,23 @@ set_special_configs(_) -> %% Test cases for all combinations of SSL, no SSL and authentication types %%------------------------------------------------------------------------------ -t_publish_no_auth(_CtConfig) -> - publish_with_and_without_ssl("none"). +t_publish_no_auth(CtConfig) -> + publish_with_and_without_ssl(CtConfig, "none"). -t_publish_no_auth_key_dispatch(_CtConfig) -> - publish_with_and_without_ssl("none", #{"partition_strategy" => "key_dispatch"}). +t_publish_no_auth_key_dispatch(CtConfig) -> + publish_with_and_without_ssl(CtConfig, "none", #{"partition_strategy" => "key_dispatch"}). -t_publish_sasl_plain(_CtConfig) -> - publish_with_and_without_ssl(valid_sasl_plain_settings()). +t_publish_sasl_plain(CtConfig) -> + publish_with_and_without_ssl(CtConfig, valid_sasl_plain_settings()). -t_publish_sasl_scram256(_CtConfig) -> - publish_with_and_without_ssl(valid_sasl_scram256_settings()). +t_publish_sasl_scram256(CtConfig) -> + publish_with_and_without_ssl(CtConfig, valid_sasl_scram256_settings()). -t_publish_sasl_scram512(_CtConfig) -> - publish_with_and_without_ssl(valid_sasl_scram512_settings()). +t_publish_sasl_scram512(CtConfig) -> + publish_with_and_without_ssl(CtConfig, valid_sasl_scram512_settings()). -t_publish_sasl_kerberos(_CtConfig) -> - publish_with_and_without_ssl(valid_sasl_kerberos_settings()). +t_publish_sasl_kerberos(CtConfig) -> + publish_with_and_without_ssl(CtConfig, valid_sasl_kerberos_settings()). %%------------------------------------------------------------------------------ %% Test cases for REST api @@ -350,7 +363,7 @@ kafka_bridge_rest_api_helper(Config) -> %% exists and it will. This is specially bad if the %% original crash was due to misconfiguration and we are %% trying to fix it... -t_failed_creation_then_fix(_Config) -> +t_failed_creation_then_fix(Config) -> HostsString = kafka_hosts_string_sasl(), ValidAuthSettings = valid_sasl_plain_settings(), WrongAuthSettings = ValidAuthSettings#{"password" := "wrong"}, @@ -394,7 +407,7 @@ t_failed_creation_then_fix(_Config) -> }, {ok, Offset} = resolve_kafka_offset(kafka_hosts(), KafkaTopic, 0), ct:pal("base offset before testing ~p", [Offset]), - ?assertEqual({async_return, ok}, ?PRODUCER:on_query(ResourceId, {send_message, Msg}, State)), + ok = send(Config, ResourceId, Msg, State), {ok, {_, [KafkaMsg]}} = brod:fetch(kafka_hosts(), KafkaTopic, 0, Offset), ?assertMatch(#kafka_message{key = BinTime}, KafkaMsg), %% TODO: refactor those into init/end per testcase @@ -406,11 +419,37 @@ t_failed_creation_then_fix(_Config) -> %% Helper functions %%------------------------------------------------------------------------------ -publish_with_and_without_ssl(AuthSettings) -> - publish_with_and_without_ssl(AuthSettings, #{}). +send(Config, ResourceId, Msg, State) when is_list(Config) -> + Ref = make_ref(), + ok = do_send(Ref, Config, ResourceId, Msg, State), + receive + {ack, Ref} -> + ok + after 10000 -> + error(timeout) + end. -publish_with_and_without_ssl(AuthSettings, Config) -> +do_send(Ref, Config, ResourceId, Msg, State) when is_list(Config) -> + Caller = self(), + F = fun(ok) -> + Caller ! {ack, Ref}, + ok + end, + case proplists:get_value(query_api, Config) of + on_query -> + ok = ?PRODUCER:on_query(ResourceId, {send_message, Msg}, State), + F(ok); + on_query_async -> + {ok, _} = ?PRODUCER:on_query_async(ResourceId, {send_message, Msg}, {F, []}, State), + ok + end. + +publish_with_and_without_ssl(CtConfig, AuthSettings) -> + publish_with_and_without_ssl(CtConfig, AuthSettings, #{}). + +publish_with_and_without_ssl(CtConfig, AuthSettings, Config) -> publish_helper( + CtConfig, #{ auth_settings => AuthSettings, ssl_settings => #{} @@ -418,6 +457,7 @@ publish_with_and_without_ssl(AuthSettings, Config) -> Config ), publish_helper( + CtConfig, #{ auth_settings => AuthSettings, ssl_settings => valid_ssl_settings() @@ -426,10 +466,11 @@ publish_with_and_without_ssl(AuthSettings, Config) -> ), ok. -publish_helper(AuthSettings) -> - publish_helper(AuthSettings, #{}). +publish_helper(CtConfig, AuthSettings) -> + publish_helper(CtConfig, AuthSettings, #{}). publish_helper( + CtConfig, #{ auth_settings := AuthSettings, ssl_settings := SSLSettings @@ -477,8 +518,7 @@ publish_helper( ct:pal("base offset before testing ~p", [Offset]), StartRes = ?PRODUCER:on_start(InstId, Conf), {ok, State} = StartRes, - OnQueryRes = ?PRODUCER:on_query(InstId, {send_message, Msg}, State), - {async_return, ok} = OnQueryRes, + ok = send(CtConfig, InstId, Msg, State), {ok, {_, [KafkaMsg]}} = brod:fetch(kafka_hosts(), KafkaTopic, 0, Offset), ?assertMatch(#kafka_message{key = BinTime}, KafkaMsg), ok = ?PRODUCER:on_stop(InstId, State), diff --git a/scripts/ct/run.sh b/scripts/ct/run.sh index 07d45efe1..69427c7c3 100755 --- a/scripts/ct/run.sh +++ b/scripts/ct/run.sh @@ -195,7 +195,7 @@ fi echo "Fixing file owners and permissions for $UID_GID" # rebar and hex cache directory need to be writable by $UID -docker exec -i $TTY -u root:root "$ERLANG_CONTAINER" bash -c "mkdir -p /.cache && chown $UID_GID /.cache && chown -R $UID_GID /emqx" +docker exec -i $TTY -u root:root "$ERLANG_CONTAINER" bash -c "mkdir -p /.cache && chown $UID_GID /.cache && chown -R $UID_GID /emqx/.git /emqx/.ci /emqx/_build/default/lib" # need to initialize .erlang.cookie manually here because / is not writable by $UID docker exec -i $TTY -u root:root "$ERLANG_CONTAINER" bash -c "openssl rand -base64 16 > /.erlang.cookie && chown $UID_GID /.erlang.cookie && chmod 0400 /.erlang.cookie"