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)
This commit is contained in:
parent
6175076f6f
commit
5fdf7fd24c
|
@ -171,14 +171,22 @@ send_message(BridgeId, Message) ->
|
||||||
not_found ->
|
not_found ->
|
||||||
{error, {bridge_not_found, BridgeId}};
|
{error, {bridge_not_found, BridgeId}};
|
||||||
#{enable := true} = Config ->
|
#{enable := true} = Config ->
|
||||||
Timeout = emqx_map_lib:deep_get(
|
QueryOpts = query_opts(Config),
|
||||||
[resource_opts, request_timeout], Config, timer:seconds(15)
|
emqx_resource:query(ResId, {send_message, Message}, QueryOpts);
|
||||||
),
|
|
||||||
emqx_resource:query(ResId, {send_message, Message}, #{timeout => Timeout});
|
|
||||||
#{enable := false} ->
|
#{enable := false} ->
|
||||||
{error, {bridge_stopped, BridgeId}}
|
{error, {bridge_stopped, BridgeId}}
|
||||||
end.
|
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() ->
|
config_key_path() ->
|
||||||
[bridges].
|
[bridges].
|
||||||
|
|
||||||
|
|
|
@ -89,6 +89,8 @@
|
||||||
-define(DEFAULT_QUEUE_SIZE, 100 * 1024 * 1024).
|
-define(DEFAULT_QUEUE_SIZE, 100 * 1024 * 1024).
|
||||||
-define(DEFAULT_QUEUE_SIZE_RAW, <<"100MB">>).
|
-define(DEFAULT_QUEUE_SIZE_RAW, <<"100MB">>).
|
||||||
|
|
||||||
|
-define(DEFAULT_REQUEST_TIMEOUT, timer:seconds(15)).
|
||||||
|
|
||||||
%% count
|
%% count
|
||||||
-define(DEFAULT_BATCH_SIZE, 1).
|
-define(DEFAULT_BATCH_SIZE, 1).
|
||||||
|
|
||||||
|
|
|
@ -255,7 +255,7 @@ reset_metrics(ResId) ->
|
||||||
query(ResId, Request) ->
|
query(ResId, Request) ->
|
||||||
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().
|
Result :: term().
|
||||||
query(ResId, Request, Opts) ->
|
query(ResId, Request, Opts) ->
|
||||||
case emqx_resource_manager:ets_lookup(ResId) of
|
case emqx_resource_manager:ets_lookup(ResId) of
|
||||||
|
@ -263,7 +263,8 @@ query(ResId, Request, Opts) ->
|
||||||
IsBufferSupported = is_buffer_supported(Module),
|
IsBufferSupported = is_buffer_supported(Module),
|
||||||
case {IsBufferSupported, QM} of
|
case {IsBufferSupported, QM} of
|
||||||
{true, _} ->
|
{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} ->
|
{false, sync} ->
|
||||||
emqx_resource_buffer_worker:sync_query(ResId, Request, Opts);
|
emqx_resource_buffer_worker:sync_query(ResId, Request, Opts);
|
||||||
{false, async} ->
|
{false, async} ->
|
||||||
|
|
|
@ -38,7 +38,8 @@
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export([
|
-export([
|
||||||
simple_sync_query/2
|
simple_sync_query/2,
|
||||||
|
simple_async_query/2
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export([
|
-export([
|
||||||
|
@ -61,6 +62,7 @@
|
||||||
-define(COLLECT_REQ_LIMIT, 1000).
|
-define(COLLECT_REQ_LIMIT, 1000).
|
||||||
-define(SEND_REQ(FROM, REQUEST), {'$send_req', FROM, REQUEST}).
|
-define(SEND_REQ(FROM, REQUEST), {'$send_req', FROM, REQUEST}).
|
||||||
-define(QUERY(FROM, REQUEST, SENT, EXPIRE_AT), {query, FROM, REQUEST, SENT, EXPIRE_AT}).
|
-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(REPLY(FROM, REQUEST, SENT, RESULT), {reply, FROM, REQUEST, SENT, RESULT}).
|
||||||
-define(EXPAND(RESULT, BATCH), [
|
-define(EXPAND(RESULT, BATCH), [
|
||||||
?REPLY(FROM, REQUEST, SENT, RESULT)
|
?REPLY(FROM, REQUEST, SENT, RESULT)
|
||||||
|
@ -116,8 +118,8 @@ async_query(Id, Request, Opts0) ->
|
||||||
emqx_resource_metrics:matched_inc(Id),
|
emqx_resource_metrics:matched_inc(Id),
|
||||||
pick_cast(Id, PickKey, {query, Request, Opts}).
|
pick_cast(Id, PickKey, {query, Request, Opts}).
|
||||||
|
|
||||||
%% simple query the resource without batching and queuing messages.
|
%% simple query the resource without batching and queuing.
|
||||||
-spec simple_sync_query(id(), request()) -> Result :: term().
|
-spec simple_sync_query(id(), request()) -> term().
|
||||||
simple_sync_query(Id, Request) ->
|
simple_sync_query(Id, Request) ->
|
||||||
%% Note: since calling this function implies in bypassing the
|
%% Note: since calling this function implies in bypassing the
|
||||||
%% buffer workers, and each buffer worker index is used when
|
%% 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
|
%% would mess up the metrics anyway. `undefined' is ignored by
|
||||||
%% `emqx_resource_metrics:*_shift/3'.
|
%% `emqx_resource_metrics:*_shift/3'.
|
||||||
Index = undefined,
|
Index = undefined,
|
||||||
QueryOpts0 = #{simple_query => true, timeout => infinity},
|
QueryOpts = simple_query_opts(),
|
||||||
QueryOpts = #{expire_at := ExpireAt} = ensure_expire_at(QueryOpts0),
|
|
||||||
emqx_resource_metrics:matched_inc(Id),
|
emqx_resource_metrics:matched_inc(Id),
|
||||||
Ref = make_message_ref(),
|
Ref = make_message_ref(),
|
||||||
HasBeenSent = false,
|
Result = call_query(sync, Id, Index, Ref, ?SIMPLE_QUERY(Request), QueryOpts),
|
||||||
From = self(),
|
_ = handle_query_result(Id, Result, _HasBeenSent = false),
|
||||||
Result = call_query(
|
|
||||||
sync, Id, Index, Ref, ?QUERY(From, Request, HasBeenSent, ExpireAt), QueryOpts
|
|
||||||
),
|
|
||||||
_ = handle_query_result(Id, Result, HasBeenSent),
|
|
||||||
Result.
|
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.
|
-spec block(pid()) -> ok.
|
||||||
block(ServerRef) ->
|
block(ServerRef) ->
|
||||||
gen_statem:cast(ServerRef, block).
|
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
|
case emqx_resource_manager:ets_lookup(Id) of
|
||||||
{ok, _Group, #{mod := Mod, state := ResSt, status := connected} = Data} ->
|
{ok, _Group, #{mod := Mod, state := ResSt, status := connected} = Data} ->
|
||||||
QM =
|
QM =
|
||||||
case QM0 of
|
case QM0 =:= configured of
|
||||||
configured -> maps:get(query_mode, Data);
|
true -> maps:get(query_mode, Data);
|
||||||
_ -> QM0
|
false -> QM0
|
||||||
end,
|
end,
|
||||||
CBM = maps:get(callback_mode, Data),
|
CBM = maps:get(callback_mode, Data),
|
||||||
CallMode = call_mode(QM, CBM),
|
CallMode = call_mode(QM, CBM),
|
||||||
|
@ -991,11 +1002,7 @@ do_reply_after_query(
|
||||||
ref => Ref,
|
ref => Ref,
|
||||||
result => Result
|
result => Result
|
||||||
}),
|
}),
|
||||||
IsFullBefore = is_inflight_full(InflightTID),
|
do_ack(InflightTID, Ref, Id, Index, PostFn, Pid, QueryOpts)
|
||||||
IsAcked = ack_inflight(InflightTID, Ref, Id, Index),
|
|
||||||
IsAcked andalso PostFn(),
|
|
||||||
IsFullBefore andalso ?MODULE:flush_worker(Pid),
|
|
||||||
ok
|
|
||||||
end.
|
end.
|
||||||
|
|
||||||
batch_reply_after_query(Pid, Id, Index, InflightTID, Ref, Batch, QueryOpts, Result) ->
|
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,
|
ref => Ref,
|
||||||
result => Result
|
result => Result
|
||||||
}),
|
}),
|
||||||
IsFullBefore = is_inflight_full(InflightTID),
|
do_ack(InflightTID, Ref, Id, Index, PostFn, Pid, QueryOpts)
|
||||||
IsAcked = ack_inflight(InflightTID, Ref, Id, Index),
|
|
||||||
IsAcked andalso PostFn(),
|
|
||||||
IsFullBefore andalso ?MODULE:flush_worker(Pid),
|
|
||||||
ok
|
|
||||||
end.
|
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
|
%% operations for queue
|
||||||
queue_item_marshaller(Bin) when is_binary(Bin) ->
|
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, {?SIZE_REF, 0}, Id, Index),
|
||||||
inflight_append(TableId, {?INITIAL_TIME_REF, erlang:system_time()}, Id, Index),
|
inflight_append(TableId, {?INITIAL_TIME_REF, erlang:system_time()}, Id, Index),
|
||||||
inflight_append(
|
inflight_append(
|
||||||
TableId, {?INITIAL_MONOTONIC_TIME_REF, erlang:monotonic_time(nanosecond)}, Id, Index
|
TableId, {?INITIAL_MONOTONIC_TIME_REF, make_message_ref()}, Id, Index
|
||||||
),
|
),
|
||||||
TableId.
|
TableId.
|
||||||
|
|
||||||
|
@ -1426,8 +1443,7 @@ now_() ->
|
||||||
ensure_timeout_query_opts(#{timeout := _} = Opts, _SyncOrAsync) ->
|
ensure_timeout_query_opts(#{timeout := _} = Opts, _SyncOrAsync) ->
|
||||||
Opts;
|
Opts;
|
||||||
ensure_timeout_query_opts(#{} = Opts0, sync) ->
|
ensure_timeout_query_opts(#{} = Opts0, sync) ->
|
||||||
TimeoutMS = timer:seconds(15),
|
Opts0#{timeout => ?DEFAULT_REQUEST_TIMEOUT};
|
||||||
Opts0#{timeout => TimeoutMS};
|
|
||||||
ensure_timeout_query_opts(#{} = Opts0, async) ->
|
ensure_timeout_query_opts(#{} = Opts0, async) ->
|
||||||
Opts0#{timeout => infinity}.
|
Opts0#{timeout => infinity}.
|
||||||
|
|
||||||
|
|
|
@ -12,6 +12,7 @@
|
||||||
on_start/2,
|
on_start/2,
|
||||||
on_stop/2,
|
on_stop/2,
|
||||||
on_query/3,
|
on_query/3,
|
||||||
|
on_query_async/4,
|
||||||
on_get_status/2,
|
on_get_status/2,
|
||||||
is_buffer_supported/0
|
is_buffer_supported/0
|
||||||
]).
|
]).
|
||||||
|
@ -26,8 +27,11 @@ on_start(InstId, Config) ->
|
||||||
on_stop(InstId, State) ->
|
on_stop(InstId, State) ->
|
||||||
emqx_bridge_impl_kafka_producer:on_stop(InstId, State).
|
emqx_bridge_impl_kafka_producer:on_stop(InstId, State).
|
||||||
|
|
||||||
on_query(InstId, Msg, State) ->
|
on_query(InstId, Req, State) ->
|
||||||
emqx_bridge_impl_kafka_producer:on_query(InstId, Msg, 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) ->
|
on_get_status(InstId, State) ->
|
||||||
emqx_bridge_impl_kafka_producer:on_get_status(InstId, State).
|
emqx_bridge_impl_kafka_producer:on_get_status(InstId, State).
|
||||||
|
|
|
@ -11,6 +11,7 @@
|
||||||
on_start/2,
|
on_start/2,
|
||||||
on_stop/2,
|
on_stop/2,
|
||||||
on_query/3,
|
on_query/3,
|
||||||
|
on_query_async/4,
|
||||||
on_get_status/2
|
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)
|
%% @doc The callback API for rule-engine (or bridge without rules)
|
||||||
%% The input argument `Message' is an enriched format (as a map())
|
%% The input argument `Message' is an enriched format (as a map())
|
||||||
%% of the original #message{} record.
|
%% of the original #message{} record.
|
||||||
%% The enrichment is done by rule-engine or by the data bridge framework.
|
%% The enrichment is done by rule-engine or by the data bridge framework.
|
||||||
%% E.g. the output of rule-engine process chain
|
%% E.g. the output of rule-engine process chain
|
||||||
%% or the direct mapping from an MQTT message.
|
%% 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),
|
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.
|
%% The retuned information is discarded here.
|
||||||
%% If the producer process is down when sending, this function would
|
%% 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
|
%% 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, [#{}]}),
|
{_Partition, Pid} = wolff:send(Producers, Batch, {fun ?MODULE:on_kafka_ack/3, [AsyncReplyFn]}),
|
||||||
{async_return, ok}.
|
%% this Pid is so far never used because Kafka producer is by-passing the buffer worker
|
||||||
|
{ok, Pid}.
|
||||||
|
|
||||||
compile_message_template(T) ->
|
compile_message_template(T) ->
|
||||||
KeyTemplate = maps:get(key, T, <<"${.clientid}">>),
|
KeyTemplate = maps:get(key, T, <<"${.clientid}">>),
|
||||||
|
@ -194,9 +224,14 @@ render_timestamp(Template, Message) ->
|
||||||
erlang:system_time(millisecond)
|
erlang:system_time(millisecond)
|
||||||
end.
|
end.
|
||||||
|
|
||||||
on_kafka_ack(_Partition, _Offset, _Extra) ->
|
%% Wolff producer never gives up retrying
|
||||||
%% Do nothing so far.
|
%% so there can only be 'ok' results.
|
||||||
%% Maybe need to bump some counters?
|
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.
|
ok.
|
||||||
|
|
||||||
on_get_status(_InstId, _State) ->
|
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
|
%% we *must* match the bridge id in the event metadata with that in
|
||||||
%% the handler config; otherwise, multiple kafka producer bridges will
|
%% the handler config; otherwise, multiple kafka producer bridges will
|
||||||
%% install multiple handlers to the same wolff events, multiplying the
|
%% 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(
|
handle_telemetry_event(
|
||||||
[wolff, dropped_queue_full],
|
[wolff, dropped_queue_full],
|
||||||
#{counter_inc := Val},
|
#{counter_inc := Val},
|
||||||
#{bridge_id := ID},
|
#{bridge_id := ID},
|
||||||
#{bridge_id := ID}
|
#{bridge_id := ID}
|
||||||
) when is_integer(Val) ->
|
) when is_integer(Val) ->
|
||||||
%% When wolff emits a `dropped_queue_full' event due to replayq
|
emqx_resource_metrics:dropped_queue_full_inc(ID, Val);
|
||||||
%% 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);
|
|
||||||
handle_telemetry_event(
|
handle_telemetry_event(
|
||||||
[wolff, queuing],
|
[wolff, queuing],
|
||||||
#{gauge_set := Val},
|
#{gauge_set := Val},
|
||||||
|
@ -380,13 +401,6 @@ handle_telemetry_event(
|
||||||
#{bridge_id := ID}
|
#{bridge_id := ID}
|
||||||
) when is_integer(Val) ->
|
) when is_integer(Val) ->
|
||||||
emqx_resource_metrics:retried_inc(ID, 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(
|
handle_telemetry_event(
|
||||||
[wolff, inflight],
|
[wolff, inflight],
|
||||||
#{gauge_set := Val},
|
#{gauge_set := Val},
|
||||||
|
@ -394,27 +408,6 @@ handle_telemetry_event(
|
||||||
#{bridge_id := ID}
|
#{bridge_id := ID}
|
||||||
) when is_integer(Val) ->
|
) when is_integer(Val) ->
|
||||||
emqx_resource_metrics:inflight_set(ID, PartitionID, 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) ->
|
handle_telemetry_event(_EventId, _Metrics, _MetaData, _HandlerConfig) ->
|
||||||
%% Event that we do not handle
|
%% Event that we do not handle
|
||||||
ok.
|
ok.
|
||||||
|
@ -437,15 +430,10 @@ maybe_install_wolff_telemetry_handlers(ResourceID) ->
|
||||||
%% unique handler id
|
%% unique handler id
|
||||||
telemetry_handler_id(ResourceID),
|
telemetry_handler_id(ResourceID),
|
||||||
[
|
[
|
||||||
[wolff, dropped],
|
|
||||||
[wolff, dropped_queue_full],
|
[wolff, dropped_queue_full],
|
||||||
[wolff, queuing],
|
[wolff, queuing],
|
||||||
[wolff, retried],
|
[wolff, retried],
|
||||||
[wolff, failed],
|
[wolff, inflight]
|
||||||
[wolff, inflight],
|
|
||||||
[wolff, retried_failed],
|
|
||||||
[wolff, retried_success],
|
|
||||||
[wolff, success]
|
|
||||||
],
|
],
|
||||||
fun ?MODULE:handle_telemetry_event/4,
|
fun ?MODULE:handle_telemetry_event/4,
|
||||||
%% we *must* keep track of the same id that is handed down to
|
%% we *must* keep track of the same id that is handed down to
|
||||||
|
|
|
@ -46,7 +46,14 @@
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
|
|
||||||
all() ->
|
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() ->
|
||||||
wait_until_kafka_is_up(0).
|
wait_until_kafka_is_up(0).
|
||||||
|
@ -89,6 +96,12 @@ end_per_suite(_Config) ->
|
||||||
_ = application:stop(emqx_connector),
|
_ = application:stop(emqx_connector),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
|
init_per_group(GroupName, Config) ->
|
||||||
|
[{query_api, GroupName} | Config].
|
||||||
|
|
||||||
|
end_per_group(_, _) ->
|
||||||
|
ok.
|
||||||
|
|
||||||
set_special_configs(emqx_management) ->
|
set_special_configs(emqx_management) ->
|
||||||
Listeners = #{http => #{port => 8081}},
|
Listeners = #{http => #{port => 8081}},
|
||||||
Config = #{
|
Config = #{
|
||||||
|
@ -106,23 +119,23 @@ set_special_configs(_) ->
|
||||||
%% Test cases for all combinations of SSL, no SSL and authentication types
|
%% Test cases for all combinations of SSL, no SSL and authentication types
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
|
|
||||||
t_publish_no_auth(_CtConfig) ->
|
t_publish_no_auth(CtConfig) ->
|
||||||
publish_with_and_without_ssl("none").
|
publish_with_and_without_ssl(CtConfig, "none").
|
||||||
|
|
||||||
t_publish_no_auth_key_dispatch(_CtConfig) ->
|
t_publish_no_auth_key_dispatch(CtConfig) ->
|
||||||
publish_with_and_without_ssl("none", #{"partition_strategy" => "key_dispatch"}).
|
publish_with_and_without_ssl(CtConfig, "none", #{"partition_strategy" => "key_dispatch"}).
|
||||||
|
|
||||||
t_publish_sasl_plain(_CtConfig) ->
|
t_publish_sasl_plain(CtConfig) ->
|
||||||
publish_with_and_without_ssl(valid_sasl_plain_settings()).
|
publish_with_and_without_ssl(CtConfig, valid_sasl_plain_settings()).
|
||||||
|
|
||||||
t_publish_sasl_scram256(_CtConfig) ->
|
t_publish_sasl_scram256(CtConfig) ->
|
||||||
publish_with_and_without_ssl(valid_sasl_scram256_settings()).
|
publish_with_and_without_ssl(CtConfig, valid_sasl_scram256_settings()).
|
||||||
|
|
||||||
t_publish_sasl_scram512(_CtConfig) ->
|
t_publish_sasl_scram512(CtConfig) ->
|
||||||
publish_with_and_without_ssl(valid_sasl_scram512_settings()).
|
publish_with_and_without_ssl(CtConfig, valid_sasl_scram512_settings()).
|
||||||
|
|
||||||
t_publish_sasl_kerberos(_CtConfig) ->
|
t_publish_sasl_kerberos(CtConfig) ->
|
||||||
publish_with_and_without_ssl(valid_sasl_kerberos_settings()).
|
publish_with_and_without_ssl(CtConfig, valid_sasl_kerberos_settings()).
|
||||||
|
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
%% Test cases for REST api
|
%% 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
|
%% exists and it will. This is specially bad if the
|
||||||
%% original crash was due to misconfiguration and we are
|
%% original crash was due to misconfiguration and we are
|
||||||
%% trying to fix it...
|
%% trying to fix it...
|
||||||
t_failed_creation_then_fix(_Config) ->
|
t_failed_creation_then_fix(Config) ->
|
||||||
HostsString = kafka_hosts_string_sasl(),
|
HostsString = kafka_hosts_string_sasl(),
|
||||||
ValidAuthSettings = valid_sasl_plain_settings(),
|
ValidAuthSettings = valid_sasl_plain_settings(),
|
||||||
WrongAuthSettings = ValidAuthSettings#{"password" := "wrong"},
|
WrongAuthSettings = ValidAuthSettings#{"password" := "wrong"},
|
||||||
|
@ -394,7 +407,7 @@ t_failed_creation_then_fix(_Config) ->
|
||||||
},
|
},
|
||||||
{ok, Offset} = resolve_kafka_offset(kafka_hosts(), KafkaTopic, 0),
|
{ok, Offset} = resolve_kafka_offset(kafka_hosts(), KafkaTopic, 0),
|
||||||
ct:pal("base offset before testing ~p", [Offset]),
|
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),
|
{ok, {_, [KafkaMsg]}} = brod:fetch(kafka_hosts(), KafkaTopic, 0, Offset),
|
||||||
?assertMatch(#kafka_message{key = BinTime}, KafkaMsg),
|
?assertMatch(#kafka_message{key = BinTime}, KafkaMsg),
|
||||||
%% TODO: refactor those into init/end per testcase
|
%% TODO: refactor those into init/end per testcase
|
||||||
|
@ -406,11 +419,37 @@ t_failed_creation_then_fix(_Config) ->
|
||||||
%% Helper functions
|
%% Helper functions
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
|
|
||||||
publish_with_and_without_ssl(AuthSettings) ->
|
send(Config, ResourceId, Msg, State) when is_list(Config) ->
|
||||||
publish_with_and_without_ssl(AuthSettings, #{}).
|
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(
|
publish_helper(
|
||||||
|
CtConfig,
|
||||||
#{
|
#{
|
||||||
auth_settings => AuthSettings,
|
auth_settings => AuthSettings,
|
||||||
ssl_settings => #{}
|
ssl_settings => #{}
|
||||||
|
@ -418,6 +457,7 @@ publish_with_and_without_ssl(AuthSettings, Config) ->
|
||||||
Config
|
Config
|
||||||
),
|
),
|
||||||
publish_helper(
|
publish_helper(
|
||||||
|
CtConfig,
|
||||||
#{
|
#{
|
||||||
auth_settings => AuthSettings,
|
auth_settings => AuthSettings,
|
||||||
ssl_settings => valid_ssl_settings()
|
ssl_settings => valid_ssl_settings()
|
||||||
|
@ -426,10 +466,11 @@ publish_with_and_without_ssl(AuthSettings, Config) ->
|
||||||
),
|
),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
publish_helper(AuthSettings) ->
|
publish_helper(CtConfig, AuthSettings) ->
|
||||||
publish_helper(AuthSettings, #{}).
|
publish_helper(CtConfig, AuthSettings, #{}).
|
||||||
|
|
||||||
publish_helper(
|
publish_helper(
|
||||||
|
CtConfig,
|
||||||
#{
|
#{
|
||||||
auth_settings := AuthSettings,
|
auth_settings := AuthSettings,
|
||||||
ssl_settings := SSLSettings
|
ssl_settings := SSLSettings
|
||||||
|
@ -477,8 +518,7 @@ publish_helper(
|
||||||
ct:pal("base offset before testing ~p", [Offset]),
|
ct:pal("base offset before testing ~p", [Offset]),
|
||||||
StartRes = ?PRODUCER:on_start(InstId, Conf),
|
StartRes = ?PRODUCER:on_start(InstId, Conf),
|
||||||
{ok, State} = StartRes,
|
{ok, State} = StartRes,
|
||||||
OnQueryRes = ?PRODUCER:on_query(InstId, {send_message, Msg}, State),
|
ok = send(CtConfig, InstId, Msg, State),
|
||||||
{async_return, ok} = OnQueryRes,
|
|
||||||
{ok, {_, [KafkaMsg]}} = brod:fetch(kafka_hosts(), KafkaTopic, 0, Offset),
|
{ok, {_, [KafkaMsg]}} = brod:fetch(kafka_hosts(), KafkaTopic, 0, Offset),
|
||||||
?assertMatch(#kafka_message{key = BinTime}, KafkaMsg),
|
?assertMatch(#kafka_message{key = BinTime}, KafkaMsg),
|
||||||
ok = ?PRODUCER:on_stop(InstId, State),
|
ok = ?PRODUCER:on_stop(InstId, State),
|
||||||
|
|
|
@ -195,7 +195,7 @@ fi
|
||||||
|
|
||||||
echo "Fixing file owners and permissions for $UID_GID"
|
echo "Fixing file owners and permissions for $UID_GID"
|
||||||
# rebar and hex cache directory need to be writable by $UID
|
# 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
|
# 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"
|
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"
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue