test(gcp_pubsub_consumer): add more tests and improve bridge
Fixes https://emqx.atlassian.net/browse/EMQX-10309
This commit is contained in:
parent
c4fc0e767e
commit
30e0b4be54
|
@ -417,10 +417,14 @@ t_start_stop(BridgeType, BridgeName, BridgeConfig, StopTracePoint) ->
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
t_on_get_status(Config) ->
|
t_on_get_status(Config) ->
|
||||||
|
t_on_get_status(Config, _Opts = #{}).
|
||||||
|
|
||||||
|
t_on_get_status(Config, Opts) ->
|
||||||
ProxyPort = ?config(proxy_port, Config),
|
ProxyPort = ?config(proxy_port, Config),
|
||||||
ProxyHost = ?config(proxy_host, Config),
|
ProxyHost = ?config(proxy_host, Config),
|
||||||
ProxyName = ?config(proxy_name, Config),
|
ProxyName = ?config(proxy_name, Config),
|
||||||
ResourceId = resource_id(Config),
|
ResourceId = resource_id(Config),
|
||||||
|
FailureStatus = maps:get(failure_status, Opts, disconnected),
|
||||||
?assertMatch({ok, _}, create_bridge(Config)),
|
?assertMatch({ok, _}, create_bridge(Config)),
|
||||||
%% Since the connection process is async, we give it some time to
|
%% Since the connection process is async, we give it some time to
|
||||||
%% stabilize and avoid flakiness.
|
%% stabilize and avoid flakiness.
|
||||||
|
@ -431,7 +435,7 @@ t_on_get_status(Config) ->
|
||||||
),
|
),
|
||||||
emqx_common_test_helpers:with_failure(down, ProxyName, ProxyHost, ProxyPort, fun() ->
|
emqx_common_test_helpers:with_failure(down, ProxyName, ProxyHost, ProxyPort, fun() ->
|
||||||
ct:sleep(500),
|
ct:sleep(500),
|
||||||
?assertEqual({ok, disconnected}, emqx_resource_manager:health_check(ResourceId))
|
?assertEqual({ok, FailureStatus}, emqx_resource_manager:health_check(ResourceId))
|
||||||
end),
|
end),
|
||||||
%% Check that it recovers itself.
|
%% Check that it recovers itself.
|
||||||
?retry(
|
?retry(
|
||||||
|
|
|
@ -139,6 +139,15 @@ fields(producer) ->
|
||||||
];
|
];
|
||||||
fields(consumer) ->
|
fields(consumer) ->
|
||||||
[
|
[
|
||||||
|
%% Note: The minimum deadline pubsub does is 10 s.
|
||||||
|
{ack_deadline,
|
||||||
|
mk(
|
||||||
|
emqx_schema:timeout_duration_s(),
|
||||||
|
#{
|
||||||
|
default => <<"60s">>,
|
||||||
|
importance => ?IMPORTANCE_HIDDEN
|
||||||
|
}
|
||||||
|
)},
|
||||||
{ack_retry_interval,
|
{ack_retry_interval,
|
||||||
mk(
|
mk(
|
||||||
emqx_schema:timeout_duration_ms(),
|
emqx_schema:timeout_duration_ms(),
|
||||||
|
|
|
@ -119,6 +119,7 @@ start(
|
||||||
?tp(gcp_pubsub_starting_ehttpc_pool, #{pool_name => ResourceId}),
|
?tp(gcp_pubsub_starting_ehttpc_pool, #{pool_name => ResourceId}),
|
||||||
case ehttpc_sup:start_pool(ResourceId, PoolOpts) of
|
case ehttpc_sup:start_pool(ResourceId, PoolOpts) of
|
||||||
{ok, _} ->
|
{ok, _} ->
|
||||||
|
?tp(gcp_pubsub_ehttpc_pool_started, #{pool_name => ResourceId}),
|
||||||
{ok, State};
|
{ok, State};
|
||||||
{error, {already_started, _}} ->
|
{error, {already_started, _}} ->
|
||||||
?tp(gcp_pubsub_ehttpc_pool_already_started, #{pool_name => ResourceId}),
|
?tp(gcp_pubsub_ehttpc_pool_already_started, #{pool_name => ResourceId}),
|
||||||
|
@ -166,7 +167,7 @@ query_sync({prepared_request, PreparedRequest = {_Method, _Path, _Body}}, State)
|
||||||
{prepared_request, prepared_request()},
|
{prepared_request, prepared_request()},
|
||||||
{ReplyFun :: function(), Args :: list()},
|
{ReplyFun :: function(), Args :: list()},
|
||||||
state()
|
state()
|
||||||
) -> {ok, pid()}.
|
) -> {ok, pid()} | {error, no_pool_worker_available}.
|
||||||
query_async(
|
query_async(
|
||||||
{prepared_request, PreparedRequest = {_Method, _Path, _Body}},
|
{prepared_request, PreparedRequest = {_Method, _Path, _Body}},
|
||||||
ReplyFunAndArgs,
|
ReplyFunAndArgs,
|
||||||
|
@ -306,7 +307,7 @@ do_send_requests_sync(State, {prepared_request, {Method, Path, Body}}) ->
|
||||||
state(),
|
state(),
|
||||||
{prepared_request, prepared_request()},
|
{prepared_request, prepared_request()},
|
||||||
{ReplyFun :: function(), Args :: list()}
|
{ReplyFun :: function(), Args :: list()}
|
||||||
) -> {ok, pid()}.
|
) -> {ok, pid()} | {error, no_pool_worker_available}.
|
||||||
do_send_requests_async(
|
do_send_requests_async(
|
||||||
State, {prepared_request, {Method, Path, Body}}, ReplyFunAndArgs
|
State, {prepared_request, {Method, Path, Body}}, ReplyFunAndArgs
|
||||||
) ->
|
) ->
|
||||||
|
@ -323,21 +324,27 @@ do_send_requests_async(
|
||||||
}
|
}
|
||||||
),
|
),
|
||||||
Request = to_ehttpc_request(State, Method, Path, Body),
|
Request = to_ehttpc_request(State, Method, Path, Body),
|
||||||
Worker = ehttpc_pool:pick_worker(PoolName),
|
%% `ehttpc_pool'/`gproc_pool' might return `false' if there are no workers...
|
||||||
ok = ehttpc:request_async(
|
case ehttpc_pool:pick_worker(PoolName) of
|
||||||
Worker,
|
false ->
|
||||||
Method,
|
{error, no_pool_worker_available};
|
||||||
Request,
|
Worker ->
|
||||||
RequestTTL,
|
ok = ehttpc:request_async(
|
||||||
{fun ?MODULE:reply_delegator/3, [PoolName, ReplyFunAndArgs]}
|
Worker,
|
||||||
),
|
Method,
|
||||||
{ok, Worker}.
|
Request,
|
||||||
|
RequestTTL,
|
||||||
|
{fun ?MODULE:reply_delegator/3, [PoolName, ReplyFunAndArgs]}
|
||||||
|
),
|
||||||
|
{ok, Worker}
|
||||||
|
end.
|
||||||
|
|
||||||
to_ehttpc_request(State, Method, Path, Body) ->
|
to_ehttpc_request(State, Method, Path, Body) ->
|
||||||
#{jwt_config := JWTConfig} = State,
|
#{jwt_config := JWTConfig} = State,
|
||||||
Headers = get_jwt_authorization_header(JWTConfig),
|
Headers = get_jwt_authorization_header(JWTConfig),
|
||||||
case {Method, Body} of
|
case {Method, Body} of
|
||||||
{get, <<>>} -> {Path, Headers};
|
{get, <<>>} -> {Path, Headers};
|
||||||
|
{delete, <<>>} -> {Path, Headers};
|
||||||
_ -> {Path, Headers, Body}
|
_ -> {Path, Headers, Body}
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
|
|
@ -24,44 +24,58 @@
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export([get_subscription/1]).
|
-export([get_subscription/1]).
|
||||||
-export([reply_delegator/3, pull_async/1, process_pull_response/2, ensure_subscription/1]).
|
-export([reply_delegator/4, pull_async/1, process_pull_response/2, ensure_subscription/1]).
|
||||||
|
|
||||||
-type subscription_id() :: binary().
|
-type subscription_id() :: binary().
|
||||||
-type bridge_name() :: atom() | binary().
|
-type bridge_name() :: atom() | binary().
|
||||||
-type ack_id() :: binary().
|
-type ack_id() :: binary().
|
||||||
|
-type message_id() :: binary().
|
||||||
-type config() :: #{
|
-type config() :: #{
|
||||||
|
ack_deadline := emqx_schema:timeout_duration_s(),
|
||||||
ack_retry_interval := emqx_schema:timeout_duration_ms(),
|
ack_retry_interval := emqx_schema:timeout_duration_ms(),
|
||||||
client := emqx_bridge_gcp_pubsub_client:state(),
|
client := emqx_bridge_gcp_pubsub_client:state(),
|
||||||
ecpool_worker_id => non_neg_integer(),
|
ecpool_worker_id => non_neg_integer(),
|
||||||
|
forget_interval := timer:time(),
|
||||||
hookpoint := binary(),
|
hookpoint := binary(),
|
||||||
instance_id := binary(),
|
instance_id := binary(),
|
||||||
mqtt_config => emqx_bridge_gcp_pubsub_impl_consumer:mqtt_config(),
|
mqtt_config => emqx_bridge_gcp_pubsub_impl_consumer:mqtt_config(),
|
||||||
project_id := emqx_bridge_gcp_pubsub_client:project_id(),
|
project_id := emqx_bridge_gcp_pubsub_client:project_id(),
|
||||||
pull_max_messages := non_neg_integer(),
|
pull_max_messages := non_neg_integer(),
|
||||||
|
pull_retry_interval := emqx_schema:timeout_duration_ms(),
|
||||||
subscription_id => subscription_id(),
|
subscription_id => subscription_id(),
|
||||||
topic => emqx_bridge_gcp_pubsub_client:topic()
|
topic => emqx_bridge_gcp_pubsub_client:topic()
|
||||||
}.
|
}.
|
||||||
-type state() :: #{
|
-type state() :: #{
|
||||||
|
ack_deadline := emqx_schema:timeout_duration_s(),
|
||||||
ack_retry_interval := emqx_schema:timeout_duration_ms(),
|
ack_retry_interval := emqx_schema:timeout_duration_ms(),
|
||||||
ack_timer := undefined | reference(),
|
ack_timer := undefined | reference(),
|
||||||
async_workers := #{pid() => reference()},
|
async_workers := #{pid() => reference()},
|
||||||
client := emqx_bridge_gcp_pubsub_client:state(),
|
client := emqx_bridge_gcp_pubsub_client:state(),
|
||||||
ecpool_worker_id := non_neg_integer(),
|
ecpool_worker_id := non_neg_integer(),
|
||||||
|
forget_interval := timer:time(),
|
||||||
hookpoint := binary(),
|
hookpoint := binary(),
|
||||||
instance_id := binary(),
|
instance_id := binary(),
|
||||||
mqtt_config => emqx_bridge_gcp_pubsub_impl_consumer:mqtt_config(),
|
mqtt_config := emqx_bridge_gcp_pubsub_impl_consumer:mqtt_config(),
|
||||||
pending_acks => [ack_id()],
|
pending_acks := #{message_id() => ack_id()},
|
||||||
project_id := emqx_bridge_gcp_pubsub_client:project_id(),
|
project_id := emqx_bridge_gcp_pubsub_client:project_id(),
|
||||||
pull_max_messages := non_neg_integer(),
|
pull_max_messages := non_neg_integer(),
|
||||||
|
pull_retry_interval := emqx_schema:timeout_duration_ms(),
|
||||||
pull_timer := undefined | reference(),
|
pull_timer := undefined | reference(),
|
||||||
subscription_id => subscription_id(),
|
%% In order to avoid re-processing the same message twice due to race conditions
|
||||||
topic => emqx_bridge_gcp_pubsub_client:topic()
|
%% between acknlowledging a message and receiving a duplicate pulled message, we need
|
||||||
|
%% to keep the seen message IDs for a while...
|
||||||
|
seen_message_ids := sets:set(message_id()),
|
||||||
|
subscription_id := subscription_id(),
|
||||||
|
topic := emqx_bridge_gcp_pubsub_client:topic()
|
||||||
}.
|
}.
|
||||||
-type decoded_message() :: map().
|
-type decoded_message() :: map().
|
||||||
|
|
||||||
|
%% initialization states
|
||||||
|
-define(ensure_subscription, ensure_subscription).
|
||||||
|
-define(patch_subscription, patch_subscription).
|
||||||
|
|
||||||
-define(HEALTH_CHECK_TIMEOUT, 10_000).
|
-define(HEALTH_CHECK_TIMEOUT, 10_000).
|
||||||
-define(OPTVAR_SUB_OK(PID), {?MODULE, PID}).
|
-define(OPTVAR_SUB_OK(PID), {?MODULE, subscription_ok, PID}).
|
||||||
-define(PULL_INTERVAL, 5_000).
|
|
||||||
|
|
||||||
%%-------------------------------------------------------------------------------------------------
|
%%-------------------------------------------------------------------------------------------------
|
||||||
%% API used by `reply_delegator'
|
%% API used by `reply_delegator'
|
||||||
|
@ -79,19 +93,23 @@ process_pull_response(WorkerPid, RespBody) ->
|
||||||
ensure_subscription(WorkerPid) ->
|
ensure_subscription(WorkerPid) ->
|
||||||
gen_server:cast(WorkerPid, ensure_subscription).
|
gen_server:cast(WorkerPid, ensure_subscription).
|
||||||
|
|
||||||
-spec reply_delegator(pid(), binary(), {ok, map()} | {error, timeout | term()}) -> ok.
|
-spec reply_delegator(pid(), pull_async, binary(), {ok, map()} | {error, timeout | term()}) -> ok.
|
||||||
reply_delegator(WorkerPid, InstanceId, Result) ->
|
reply_delegator(WorkerPid, pull_async = _Action, InstanceId, Result) ->
|
||||||
|
?tp(gcp_pubsub_consumer_worker_reply_delegator, #{result => Result}),
|
||||||
case Result of
|
case Result of
|
||||||
{error, timeout} ->
|
{error, timeout} ->
|
||||||
?MODULE:pull_async(WorkerPid);
|
?MODULE:pull_async(WorkerPid);
|
||||||
{error, Reason} ->
|
{error, Reason} ->
|
||||||
?SLOG(warning, #{
|
?tp(
|
||||||
msg => "gcp_pubsub_consumer_worker_pull_error",
|
warning,
|
||||||
instance_id => InstanceId,
|
"gcp_pubsub_consumer_worker_pull_error",
|
||||||
reason => Reason
|
#{
|
||||||
}),
|
instance_id => InstanceId,
|
||||||
|
reason => Reason
|
||||||
|
}
|
||||||
|
),
|
||||||
case Reason of
|
case Reason of
|
||||||
#{status_code := 409} ->
|
#{status_code := 404} ->
|
||||||
%% the subscription was not found; deleted?!
|
%% the subscription was not found; deleted?!
|
||||||
?MODULE:ensure_subscription(WorkerPid);
|
?MODULE:ensure_subscription(WorkerPid);
|
||||||
_ ->
|
_ ->
|
||||||
|
@ -113,13 +131,13 @@ get_subscription(WorkerPid) ->
|
||||||
%% `ecpool' health check
|
%% `ecpool' health check
|
||||||
%%-------------------------------------------------------------------------------------------------
|
%%-------------------------------------------------------------------------------------------------
|
||||||
|
|
||||||
-spec health_check(pid()) -> boolean().
|
-spec health_check(pid()) -> subscription_ok | topic_not_found | timeout.
|
||||||
health_check(WorkerPid) ->
|
health_check(WorkerPid) ->
|
||||||
case optvar:read(?OPTVAR_SUB_OK(WorkerPid), ?HEALTH_CHECK_TIMEOUT) of
|
case optvar:read(?OPTVAR_SUB_OK(WorkerPid), ?HEALTH_CHECK_TIMEOUT) of
|
||||||
{ok, _} ->
|
{ok, Status} ->
|
||||||
true;
|
Status;
|
||||||
timeout ->
|
timeout ->
|
||||||
false
|
timeout
|
||||||
end.
|
end.
|
||||||
|
|
||||||
%%-------------------------------------------------------------------------------------------------
|
%%-------------------------------------------------------------------------------------------------
|
||||||
|
@ -129,30 +147,36 @@ health_check(WorkerPid) ->
|
||||||
connect(Opts0) ->
|
connect(Opts0) ->
|
||||||
Opts = maps:from_list(Opts0),
|
Opts = maps:from_list(Opts0),
|
||||||
#{
|
#{
|
||||||
|
ack_deadline := AckDeadlineSeconds,
|
||||||
ack_retry_interval := AckRetryInterval,
|
ack_retry_interval := AckRetryInterval,
|
||||||
bridge_name := BridgeName,
|
bridge_name := BridgeName,
|
||||||
client := Client,
|
client := Client,
|
||||||
ecpool_worker_id := WorkerId,
|
ecpool_worker_id := WorkerId,
|
||||||
|
forget_interval := ForgetInterval,
|
||||||
hookpoint := Hookpoint,
|
hookpoint := Hookpoint,
|
||||||
instance_id := InstanceId,
|
instance_id := InstanceId,
|
||||||
project_id := ProjectId,
|
project_id := ProjectId,
|
||||||
pull_max_messages := PullMaxMessages,
|
pull_max_messages := PullMaxMessages,
|
||||||
|
pull_retry_interval := PullRetryInterval,
|
||||||
topic_mapping := TopicMapping
|
topic_mapping := TopicMapping
|
||||||
} = Opts,
|
} = Opts,
|
||||||
TopicMappingList = lists:keysort(1, maps:to_list(TopicMapping)),
|
TopicMappingList = lists:keysort(1, maps:to_list(TopicMapping)),
|
||||||
Index = 1 + (WorkerId rem map_size(TopicMapping)),
|
Index = 1 + (WorkerId rem map_size(TopicMapping)),
|
||||||
{Topic, MQTTConfig} = lists:nth(Index, TopicMappingList),
|
{Topic, MQTTConfig} = lists:nth(Index, TopicMappingList),
|
||||||
Config = #{
|
Config = #{
|
||||||
|
ack_deadline => AckDeadlineSeconds,
|
||||||
ack_retry_interval => AckRetryInterval,
|
ack_retry_interval => AckRetryInterval,
|
||||||
%% Note: the `client' value here must be immutable and not changed by the
|
%% Note: the `client' value here must be immutable and not changed by the
|
||||||
%% bridge during `on_get_status', since we have handed it over to the pull
|
%% bridge during `on_get_status', since we have handed it over to the pull
|
||||||
%% workers.
|
%% workers.
|
||||||
client => Client,
|
client => Client,
|
||||||
|
forget_interval => ForgetInterval,
|
||||||
hookpoint => Hookpoint,
|
hookpoint => Hookpoint,
|
||||||
instance_id => InstanceId,
|
instance_id => InstanceId,
|
||||||
mqtt_config => MQTTConfig,
|
mqtt_config => MQTTConfig,
|
||||||
project_id => ProjectId,
|
project_id => ProjectId,
|
||||||
pull_max_messages => PullMaxMessages,
|
pull_max_messages => PullMaxMessages,
|
||||||
|
pull_retry_interval => PullRetryInterval,
|
||||||
topic => Topic,
|
topic => Topic,
|
||||||
subscription_id => subscription_id(BridgeName, Topic)
|
subscription_id => subscription_id(BridgeName, Topic)
|
||||||
},
|
},
|
||||||
|
@ -162,33 +186,55 @@ connect(Opts0) ->
|
||||||
%% `gen_server' API
|
%% `gen_server' API
|
||||||
%%-------------------------------------------------------------------------------------------------
|
%%-------------------------------------------------------------------------------------------------
|
||||||
|
|
||||||
-spec init(config()) -> {ok, state(), {continue, ensure_subscription}}.
|
-spec init(config()) -> {ok, state(), {continue, ?ensure_subscription}}.
|
||||||
init(Config) ->
|
init(Config) ->
|
||||||
process_flag(trap_exit, true),
|
process_flag(trap_exit, true),
|
||||||
State = Config#{
|
State = Config#{
|
||||||
ack_timer => undefined,
|
ack_timer => undefined,
|
||||||
async_workers => #{},
|
async_workers => #{},
|
||||||
pending_acks => [],
|
pending_acks => #{},
|
||||||
pull_timer => undefined
|
pull_timer => undefined,
|
||||||
|
seen_message_ids => sets:new([{version, 2}])
|
||||||
},
|
},
|
||||||
{ok, State, {continue, ensure_subscription}}.
|
?tp(gcp_pubsub_consumer_worker_init, #{topic => maps:get(topic, State)}),
|
||||||
|
{ok, State, {continue, ?ensure_subscription}}.
|
||||||
|
|
||||||
handle_continue(ensure_subscription, State0) ->
|
handle_continue(?ensure_subscription, State0) ->
|
||||||
case ensure_subscription_exists(State0) of
|
case ensure_subscription_exists(State0) of
|
||||||
ok ->
|
already_exists ->
|
||||||
|
{noreply, State0, {continue, ?patch_subscription}};
|
||||||
|
continue ->
|
||||||
#{instance_id := InstanceId} = State0,
|
#{instance_id := InstanceId} = State0,
|
||||||
|
?MODULE:pull_async(self()),
|
||||||
|
optvar:set(?OPTVAR_SUB_OK(self()), subscription_ok),
|
||||||
?tp(
|
?tp(
|
||||||
debug,
|
debug,
|
||||||
"gcp_pubsub_consumer_worker_subscription_ready",
|
"gcp_pubsub_consumer_worker_subscription_ready",
|
||||||
#{instance_id => InstanceId}
|
#{instance_id => InstanceId}
|
||||||
),
|
),
|
||||||
|
{noreply, State0};
|
||||||
|
retry ->
|
||||||
|
{noreply, State0, {continue, ?ensure_subscription}};
|
||||||
|
not_found ->
|
||||||
|
%% there's nothing much to do if the topic suddenly doesn't exist anymore.
|
||||||
|
{stop, {error, topic_not_found}, State0}
|
||||||
|
end;
|
||||||
|
handle_continue(?patch_subscription, State0) ->
|
||||||
|
?tp(gcp_pubsub_consumer_worker_patch_subscription_enter, #{}),
|
||||||
|
case patch_subscription(State0) of
|
||||||
|
ok ->
|
||||||
|
#{instance_id := InstanceId} = State0,
|
||||||
?MODULE:pull_async(self()),
|
?MODULE:pull_async(self()),
|
||||||
optvar:set(?OPTVAR_SUB_OK(self()), subscription_ok),
|
optvar:set(?OPTVAR_SUB_OK(self()), subscription_ok),
|
||||||
|
?tp(
|
||||||
|
debug,
|
||||||
|
"gcp_pubsub_consumer_worker_subscription_ready",
|
||||||
|
#{instance_id => InstanceId}
|
||||||
|
),
|
||||||
{noreply, State0};
|
{noreply, State0};
|
||||||
error ->
|
error ->
|
||||||
%% FIXME: add delay if topic does not exist?!
|
|
||||||
%% retry
|
%% retry
|
||||||
{noreply, State0, {continue, ensure_subscription}}
|
{noreply, State0, {continue, ?patch_subscription}}
|
||||||
end.
|
end.
|
||||||
|
|
||||||
handle_call(get_subscription, _From, State0) ->
|
handle_call(get_subscription, _From, State0) ->
|
||||||
|
@ -201,21 +247,20 @@ handle_cast(pull_async, State0) ->
|
||||||
State = do_pull_async(State0),
|
State = do_pull_async(State0),
|
||||||
{noreply, State};
|
{noreply, State};
|
||||||
handle_cast({process_pull_response, RespBody}, State0) ->
|
handle_cast({process_pull_response, RespBody}, State0) ->
|
||||||
|
?tp(gcp_pubsub_consumer_worker_pull_response_received, #{}),
|
||||||
State = do_process_pull_response(State0, RespBody),
|
State = do_process_pull_response(State0, RespBody),
|
||||||
{noreply, State};
|
{noreply, State};
|
||||||
handle_cast(ensure_subscription, State0) ->
|
handle_cast(ensure_subscription, State0) ->
|
||||||
{noreply, State0, {continue, ensure_subscription}};
|
{noreply, State0, {continue, ?ensure_subscription}};
|
||||||
handle_cast(_Request, State0) ->
|
handle_cast(_Request, State0) ->
|
||||||
{noreply, State0}.
|
{noreply, State0}.
|
||||||
|
|
||||||
handle_info({timeout, TRef, ack}, State0 = #{ack_timer := TRef}) ->
|
handle_info({timeout, TRef, ack}, State0 = #{ack_timer := TRef}) ->
|
||||||
State1 = acknowledge(State0),
|
State = acknowledge(State0),
|
||||||
State = ensure_ack_timer(State1),
|
|
||||||
{noreply, State};
|
{noreply, State};
|
||||||
handle_info({timeout, TRef, pull}, State0 = #{pull_timer := TRef}) ->
|
handle_info({timeout, TRef, pull}, State0 = #{pull_timer := TRef}) ->
|
||||||
State1 = State0#{pull_timer := undefined},
|
State1 = State0#{pull_timer := undefined},
|
||||||
State2 = do_pull_async(State1),
|
State = do_pull_async(State1),
|
||||||
State = ensure_pull_timer(State2),
|
|
||||||
{noreply, State};
|
{noreply, State};
|
||||||
handle_info(
|
handle_info(
|
||||||
{'DOWN', _Ref, process, AsyncWorkerPid, _Reason}, State0 = #{async_workers := Workers0}
|
{'DOWN', _Ref, process, AsyncWorkerPid, _Reason}, State0 = #{async_workers := Workers0}
|
||||||
|
@ -225,6 +270,13 @@ handle_info(
|
||||||
Workers = maps:remove(AsyncWorkerPid, Workers0),
|
Workers = maps:remove(AsyncWorkerPid, Workers0),
|
||||||
State1 = State0#{async_workers := Workers},
|
State1 = State0#{async_workers := Workers},
|
||||||
State = do_pull_async(State1),
|
State = do_pull_async(State1),
|
||||||
|
?tp(gcp_pubsub_consumer_worker_handled_async_worker_down, #{async_worker_pid => AsyncWorkerPid}),
|
||||||
|
{noreply, State};
|
||||||
|
handle_info({forget_message_ids, MsgIds}, State0) ->
|
||||||
|
State = maps:update_with(
|
||||||
|
seen_message_ids, fun(Seen) -> sets:subtract(Seen, MsgIds) end, State0
|
||||||
|
),
|
||||||
|
?tp(gcp_pubsub_consumer_worker_message_ids_forgotten, #{message_ids => MsgIds}),
|
||||||
{noreply, State};
|
{noreply, State};
|
||||||
handle_info(Msg, State0) ->
|
handle_info(Msg, State0) ->
|
||||||
#{
|
#{
|
||||||
|
@ -239,8 +291,18 @@ handle_info(Msg, State0) ->
|
||||||
}),
|
}),
|
||||||
{noreply, State0}.
|
{noreply, State0}.
|
||||||
|
|
||||||
|
terminate({error, topic_not_found} = _Reason, State) ->
|
||||||
|
#{
|
||||||
|
instance_id := InstanceId,
|
||||||
|
topic := _Topic
|
||||||
|
} = State,
|
||||||
|
optvar:unset(?OPTVAR_SUB_OK(self())),
|
||||||
|
emqx_bridge_gcp_pubsub_impl_consumer:mark_topic_as_nonexistent(InstanceId),
|
||||||
|
?tp(gcp_pubsub_consumer_worker_terminate, #{reason => _Reason, topic => _Topic}),
|
||||||
|
ok;
|
||||||
terminate(_Reason, _State) ->
|
terminate(_Reason, _State) ->
|
||||||
optvar:unset(?OPTVAR_SUB_OK(self())),
|
optvar:unset(?OPTVAR_SUB_OK(self())),
|
||||||
|
?tp(gcp_pubsub_consumer_worker_terminate, #{reason => _Reason, topic => maps:get(topic, _State)}),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
%%-------------------------------------------------------------------------------------------------
|
%%-------------------------------------------------------------------------------------------------
|
||||||
|
@ -252,21 +314,24 @@ start_link(Config) ->
|
||||||
gen_server:start_link(?MODULE, Config, []).
|
gen_server:start_link(?MODULE, Config, []).
|
||||||
|
|
||||||
-spec ensure_ack_timer(state()) -> state().
|
-spec ensure_ack_timer(state()) -> state().
|
||||||
ensure_ack_timer(State = #{pending_acks := []}) ->
|
ensure_ack_timer(State = #{ack_timer := TRef, pending_acks := PendingAcks}) ->
|
||||||
State;
|
case {map_size(PendingAcks) =:= 0, is_reference(TRef)} of
|
||||||
ensure_ack_timer(State = #{ack_timer := TRef}) when is_reference(TRef) ->
|
{false, false} ->
|
||||||
State;
|
#{ack_retry_interval := AckRetryInterval} = State,
|
||||||
ensure_ack_timer(State = #{ack_retry_interval := AckRetryInterval}) ->
|
State#{ack_timer := emqx_utils:start_timer(AckRetryInterval, ack)};
|
||||||
State#{ack_timer := emqx_utils:start_timer(AckRetryInterval, ack)}.
|
{_, _} ->
|
||||||
|
State
|
||||||
|
end.
|
||||||
|
|
||||||
-spec ensure_pull_timer(state()) -> state().
|
-spec ensure_pull_timer(state()) -> state().
|
||||||
ensure_pull_timer(State = #{pull_timer := TRef}) when is_reference(TRef) ->
|
ensure_pull_timer(State = #{pull_timer := TRef}) when is_reference(TRef) ->
|
||||||
State;
|
State;
|
||||||
ensure_pull_timer(State) ->
|
ensure_pull_timer(State = #{pull_retry_interval := PullRetryInterval}) ->
|
||||||
State#{pull_timer := emqx_utils:start_timer(?PULL_INTERVAL, pull)}.
|
State#{pull_timer := emqx_utils:start_timer(PullRetryInterval, pull)}.
|
||||||
|
|
||||||
-spec ensure_subscription_exists(state()) -> ok | error.
|
-spec ensure_subscription_exists(state()) -> continue | retry | not_found | already_exists.
|
||||||
ensure_subscription_exists(State) ->
|
ensure_subscription_exists(State) ->
|
||||||
|
?tp(gcp_pubsub_consumer_worker_create_subscription_enter, #{}),
|
||||||
#{
|
#{
|
||||||
client := Client,
|
client := Client,
|
||||||
instance_id := InstanceId,
|
instance_id := InstanceId,
|
||||||
|
@ -281,60 +346,122 @@ ensure_subscription_exists(State) ->
|
||||||
case Res of
|
case Res of
|
||||||
{error, #{status_code := 409}} ->
|
{error, #{status_code := 409}} ->
|
||||||
%% already exists
|
%% already exists
|
||||||
?SLOG(debug, #{
|
?tp(
|
||||||
msg => "gcp_pubsub_consumer_worker_subscription_already_exists",
|
debug,
|
||||||
instance_id => InstanceId,
|
"gcp_pubsub_consumer_worker_subscription_already_exists",
|
||||||
topic => Topic,
|
#{
|
||||||
subscription_id => SubscriptionId
|
instance_id => InstanceId,
|
||||||
}),
|
topic => Topic,
|
||||||
Method1 = patch,
|
subscription_id => SubscriptionId
|
||||||
Path1 = path(State, create),
|
}
|
||||||
Body1 = body(State, patch_subscription),
|
),
|
||||||
PreparedRequest1 = {prepared_request, {Method1, Path1, Body1}},
|
already_exists;
|
||||||
Res1 = emqx_bridge_gcp_pubsub_client:query_sync(PreparedRequest1, Client),
|
{error, #{status_code := 404}} ->
|
||||||
?SLOG(debug, #{
|
%% nonexistent topic
|
||||||
msg => "gcp_pubsub_consumer_worker_subscription_patch",
|
?tp(
|
||||||
instance_id => InstanceId,
|
warning,
|
||||||
topic => Topic,
|
"gcp_pubsub_consumer_worker_nonexistent_topic",
|
||||||
subscription_id => SubscriptionId,
|
#{
|
||||||
result => Res1
|
instance_id => InstanceId,
|
||||||
}),
|
topic => Topic
|
||||||
ok;
|
}
|
||||||
|
),
|
||||||
|
not_found;
|
||||||
{ok, #{status_code := 200}} ->
|
{ok, #{status_code := 200}} ->
|
||||||
?SLOG(debug, #{
|
?tp(
|
||||||
msg => "gcp_pubsub_consumer_worker_subscription_created",
|
debug,
|
||||||
instance_id => InstanceId,
|
"gcp_pubsub_consumer_worker_subscription_created",
|
||||||
topic => Topic,
|
#{
|
||||||
subscription_id => SubscriptionId
|
instance_id => InstanceId,
|
||||||
}),
|
topic => Topic,
|
||||||
|
subscription_id => SubscriptionId
|
||||||
|
}
|
||||||
|
),
|
||||||
|
continue;
|
||||||
|
{error, Reason} ->
|
||||||
|
?tp(
|
||||||
|
error,
|
||||||
|
"gcp_pubsub_consumer_worker_subscription_error",
|
||||||
|
#{
|
||||||
|
instance_id => InstanceId,
|
||||||
|
topic => Topic,
|
||||||
|
reason => Reason
|
||||||
|
}
|
||||||
|
),
|
||||||
|
retry
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec patch_subscription(state()) -> ok | error.
|
||||||
|
patch_subscription(State) ->
|
||||||
|
#{
|
||||||
|
client := Client,
|
||||||
|
instance_id := InstanceId,
|
||||||
|
subscription_id := SubscriptionId,
|
||||||
|
topic := Topic
|
||||||
|
} = State,
|
||||||
|
Method1 = patch,
|
||||||
|
Path1 = path(State, create),
|
||||||
|
Body1 = body(State, patch_subscription),
|
||||||
|
PreparedRequest1 = {prepared_request, {Method1, Path1, Body1}},
|
||||||
|
Res = emqx_bridge_gcp_pubsub_client:query_sync(PreparedRequest1, Client),
|
||||||
|
case Res of
|
||||||
|
{ok, _} ->
|
||||||
|
?tp(
|
||||||
|
debug,
|
||||||
|
"gcp_pubsub_consumer_worker_subscription_patched",
|
||||||
|
#{
|
||||||
|
instance_id => InstanceId,
|
||||||
|
topic => Topic,
|
||||||
|
subscription_id => SubscriptionId,
|
||||||
|
result => Res
|
||||||
|
}
|
||||||
|
),
|
||||||
ok;
|
ok;
|
||||||
{error, Reason} ->
|
{error, Reason} ->
|
||||||
?SLOG(error, #{
|
?tp(
|
||||||
msg => "gcp_pubsub_consumer_worker_subscription_error",
|
warning,
|
||||||
instance_id => InstanceId,
|
"gcp_pubsub_consumer_worker_subscription_patch_error",
|
||||||
topic => Topic,
|
#{
|
||||||
reason => Reason
|
instance_id => InstanceId,
|
||||||
}),
|
topic => Topic,
|
||||||
|
subscription_id => SubscriptionId,
|
||||||
|
reason => Reason
|
||||||
|
}
|
||||||
|
),
|
||||||
error
|
error
|
||||||
end.
|
end.
|
||||||
|
|
||||||
%% We use async requests so that this process will be more responsive to system messages.
|
%% We use async requests so that this process will be more responsive to system messages.
|
||||||
do_pull_async(State) ->
|
-spec do_pull_async(state()) -> state().
|
||||||
#{
|
do_pull_async(State0) ->
|
||||||
client := Client,
|
?tp_span(
|
||||||
instance_id := InstanceId
|
gcp_pubsub_consumer_worker_pull_async,
|
||||||
} = State,
|
#{topic => maps:get(topic, State0), subscription_id => maps:get(subscription_id, State0)},
|
||||||
Method = post,
|
begin
|
||||||
Path = path(State, pull),
|
#{
|
||||||
Body = body(State, pull),
|
client := Client,
|
||||||
PreparedRequest = {prepared_request, {Method, Path, Body}},
|
instance_id := InstanceId
|
||||||
ReplyFunAndArgs = {fun ?MODULE:reply_delegator/3, [self(), InstanceId]},
|
} = State0,
|
||||||
{ok, AsyncWorkerPid} = emqx_bridge_gcp_pubsub_client:query_async(
|
Method = post,
|
||||||
PreparedRequest,
|
Path = path(State0, pull),
|
||||||
ReplyFunAndArgs,
|
Body = body(State0, pull),
|
||||||
Client
|
PreparedRequest = {prepared_request, {Method, Path, Body}},
|
||||||
),
|
ReplyFunAndArgs = {fun ?MODULE:reply_delegator/4, [self(), pull_async, InstanceId]},
|
||||||
ensure_async_worker_monitored(State, AsyncWorkerPid).
|
%% `ehttpc_pool'/`gproc_pool' might return `false' if there are no workers...
|
||||||
|
Res = emqx_bridge_gcp_pubsub_client:query_async(
|
||||||
|
PreparedRequest,
|
||||||
|
ReplyFunAndArgs,
|
||||||
|
Client
|
||||||
|
),
|
||||||
|
case Res of
|
||||||
|
{ok, AsyncWorkerPid} ->
|
||||||
|
State1 = ensure_pull_timer(State0),
|
||||||
|
ensure_async_worker_monitored(State1, AsyncWorkerPid);
|
||||||
|
{error, no_pool_worker_available} ->
|
||||||
|
ensure_pull_timer(State0)
|
||||||
|
end
|
||||||
|
end
|
||||||
|
).
|
||||||
|
|
||||||
-spec ensure_async_worker_monitored(state(), pid()) -> state().
|
-spec ensure_async_worker_monitored(state(), pid()) -> state().
|
||||||
ensure_async_worker_monitored(State = #{async_workers := Workers0}, AsyncWorkerPid) ->
|
ensure_async_worker_monitored(State = #{async_workers := Workers0}, AsyncWorkerPid) ->
|
||||||
|
@ -349,22 +476,58 @@ ensure_async_worker_monitored(State = #{async_workers := Workers0}, AsyncWorkerP
|
||||||
|
|
||||||
-spec do_process_pull_response(state(), binary()) -> state().
|
-spec do_process_pull_response(state(), binary()) -> state().
|
||||||
do_process_pull_response(State0, RespBody) ->
|
do_process_pull_response(State0, RespBody) ->
|
||||||
|
#{
|
||||||
|
pending_acks := PendingAcks,
|
||||||
|
seen_message_ids := SeenMsgIds
|
||||||
|
} = State0,
|
||||||
Messages = decode_response(RespBody),
|
Messages = decode_response(RespBody),
|
||||||
AckIds = lists:map(fun(Msg) -> handle_message(State0, Msg) end, Messages),
|
?tp(gcp_pubsub_consumer_worker_decoded_messages, #{messages => Messages}),
|
||||||
State1 = maps:update_with(pending_acks, fun(AckIds0) -> AckIds0 ++ AckIds end, State0),
|
{NewPendingAcks, NewSeenMsgIds} =
|
||||||
|
lists:foldl(
|
||||||
|
fun(
|
||||||
|
Msg = #{
|
||||||
|
<<"ackId">> := AckId,
|
||||||
|
<<"message">> := #{<<"messageId">> := MsgId}
|
||||||
|
},
|
||||||
|
{AccAck, AccSeen}
|
||||||
|
) ->
|
||||||
|
case is_map_key(MsgId, PendingAcks) or sets:is_element(MsgId, SeenMsgIds) of
|
||||||
|
true ->
|
||||||
|
?tp(message_redelivered, #{message => Msg}),
|
||||||
|
%% even though it was redelivered, pubsub might change the ack
|
||||||
|
%% id... we should ack this latest value.
|
||||||
|
{AccAck#{MsgId => AckId}, AccSeen};
|
||||||
|
false ->
|
||||||
|
_ = handle_message(State0, Msg),
|
||||||
|
{AccAck#{MsgId => AckId}, sets:add_element(MsgId, AccSeen)}
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
{PendingAcks, SeenMsgIds},
|
||||||
|
Messages
|
||||||
|
),
|
||||||
|
State1 = State0#{pending_acks := NewPendingAcks, seen_message_ids := NewSeenMsgIds},
|
||||||
State2 = acknowledge(State1),
|
State2 = acknowledge(State1),
|
||||||
pull_async(self()),
|
pull_async(self()),
|
||||||
ensure_ack_timer(State2).
|
State2.
|
||||||
|
|
||||||
-spec acknowledge(state()) -> state().
|
-spec acknowledge(state()) -> state().
|
||||||
acknowledge(State0 = #{pending_acks := []}) ->
|
acknowledge(State0 = #{pending_acks := PendingAcks}) ->
|
||||||
State0;
|
case map_size(PendingAcks) =:= 0 of
|
||||||
acknowledge(State0) ->
|
true ->
|
||||||
|
State0;
|
||||||
|
false ->
|
||||||
|
do_acknowledge(State0)
|
||||||
|
end.
|
||||||
|
|
||||||
|
do_acknowledge(State0) ->
|
||||||
|
?tp(gcp_pubsub_consumer_worker_acknowledge_enter, #{}),
|
||||||
State1 = State0#{ack_timer := undefined},
|
State1 = State0#{ack_timer := undefined},
|
||||||
#{
|
#{
|
||||||
client := Client,
|
client := Client,
|
||||||
pending_acks := AckIds
|
forget_interval := ForgetInterval,
|
||||||
|
pending_acks := PendingAcks
|
||||||
} = State1,
|
} = State1,
|
||||||
|
AckIds = maps:values(PendingAcks),
|
||||||
Method = post,
|
Method = post,
|
||||||
Path = path(State1, ack),
|
Path = path(State1, ack),
|
||||||
Body = body(State1, ack, #{ack_ids => AckIds}),
|
Body = body(State1, ack, #{ack_ids => AckIds}),
|
||||||
|
@ -372,16 +535,27 @@ acknowledge(State0) ->
|
||||||
Res = emqx_bridge_gcp_pubsub_client:query_sync(PreparedRequest, Client),
|
Res = emqx_bridge_gcp_pubsub_client:query_sync(PreparedRequest, Client),
|
||||||
case Res of
|
case Res of
|
||||||
{error, Reason} ->
|
{error, Reason} ->
|
||||||
?SLOG(warning, #{msg => "gcp_pubsub_consumer_worker_ack_error", reason => Reason}),
|
?tp(
|
||||||
State1;
|
warning,
|
||||||
|
"gcp_pubsub_consumer_worker_ack_error",
|
||||||
|
#{reason => Reason}
|
||||||
|
),
|
||||||
|
ensure_ack_timer(State1);
|
||||||
{ok, #{status_code := 200}} ->
|
{ok, #{status_code := 200}} ->
|
||||||
?tp(gcp_pubsub_consumer_worker_acknowledged, #{ack_ids => AckIds}),
|
?tp(gcp_pubsub_consumer_worker_acknowledged, #{acks => PendingAcks}),
|
||||||
State1#{pending_acks := []};
|
MsgIds = maps:keys(PendingAcks),
|
||||||
|
forget_message_ids_after(MsgIds, ForgetInterval),
|
||||||
|
State1#{pending_acks := #{}};
|
||||||
{ok, Details} ->
|
{ok, Details} ->
|
||||||
?SLOG(warning, #{msg => "gcp_pubsub_consumer_worker_ack_error", details => Details}),
|
?tp(
|
||||||
State1
|
warning,
|
||||||
|
"gcp_pubsub_consumer_worker_ack_error",
|
||||||
|
#{details => Details}
|
||||||
|
),
|
||||||
|
ensure_ack_timer(State1)
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
-spec do_get_subscription(state()) -> {ok, emqx_utils_json:json_term()} | {error, term()}.
|
||||||
do_get_subscription(State) ->
|
do_get_subscription(State) ->
|
||||||
#{
|
#{
|
||||||
client := Client
|
client := Client
|
||||||
|
@ -442,12 +616,11 @@ body(State, pull) ->
|
||||||
emqx_utils_json:encode(#{<<"maxMessages">> => PullMaxMessages});
|
emqx_utils_json:encode(#{<<"maxMessages">> => PullMaxMessages});
|
||||||
body(State, create) ->
|
body(State, create) ->
|
||||||
#{
|
#{
|
||||||
ack_retry_interval := AckRetryInterval,
|
ack_deadline := AckDeadlineSeconds,
|
||||||
project_id := ProjectId,
|
project_id := ProjectId,
|
||||||
topic := PubSubTopic
|
topic := PubSubTopic
|
||||||
} = State,
|
} = State,
|
||||||
TopicResource = <<"projects/", ProjectId/binary, "/topics/", PubSubTopic/binary>>,
|
TopicResource = <<"projects/", ProjectId/binary, "/topics/", PubSubTopic/binary>>,
|
||||||
AckDeadlineSeconds = 5 + erlang:convert_time_unit(AckRetryInterval, millisecond, second),
|
|
||||||
JSON = #{
|
JSON = #{
|
||||||
<<"topic">> => TopicResource,
|
<<"topic">> => TopicResource,
|
||||||
<<"ackDeadlineSeconds">> => AckDeadlineSeconds
|
<<"ackDeadlineSeconds">> => AckDeadlineSeconds
|
||||||
|
@ -455,14 +628,13 @@ body(State, create) ->
|
||||||
emqx_utils_json:encode(JSON);
|
emqx_utils_json:encode(JSON);
|
||||||
body(State, patch_subscription) ->
|
body(State, patch_subscription) ->
|
||||||
#{
|
#{
|
||||||
ack_retry_interval := AckRetryInterval,
|
ack_deadline := AckDeadlineSeconds,
|
||||||
project_id := ProjectId,
|
project_id := ProjectId,
|
||||||
topic := PubSubTopic,
|
topic := PubSubTopic,
|
||||||
subscription_id := SubscriptionId
|
subscription_id := SubscriptionId
|
||||||
} = State,
|
} = State,
|
||||||
TopicResource = <<"projects/", ProjectId/binary, "/topics/", PubSubTopic/binary>>,
|
TopicResource = <<"projects/", ProjectId/binary, "/topics/", PubSubTopic/binary>>,
|
||||||
SubscriptionResource = subscription_resource(ProjectId, SubscriptionId),
|
SubscriptionResource = subscription_resource(ProjectId, SubscriptionId),
|
||||||
AckDeadlineSeconds = 5 + erlang:convert_time_unit(AckRetryInterval, millisecond, second),
|
|
||||||
JSON = #{
|
JSON = #{
|
||||||
<<"subscription">> =>
|
<<"subscription">> =>
|
||||||
#{
|
#{
|
||||||
|
@ -505,50 +677,52 @@ decode_response(RespBody) ->
|
||||||
[]
|
[]
|
||||||
end.
|
end.
|
||||||
|
|
||||||
-spec handle_message(state(), decoded_message()) -> [ack_id()].
|
-spec handle_message(state(), decoded_message()) -> ok.
|
||||||
handle_message(State, #{<<"ackId">> := AckId, <<"message">> := InnerMsg} = _Message) ->
|
handle_message(State, #{<<"ackId">> := AckId, <<"message">> := InnerMsg} = _Message) ->
|
||||||
?tp(
|
?tp_span(
|
||||||
debug,
|
debug,
|
||||||
"gcp_pubsub_consumer_worker_handle_message",
|
"gcp_pubsub_consumer_worker_handle_message",
|
||||||
#{message_id => maps:get(<<"messageId">>, InnerMsg), message => _Message, ack_id => AckId}
|
#{message_id => maps:get(<<"messageId">>, InnerMsg), message => _Message, ack_id => AckId},
|
||||||
),
|
begin
|
||||||
#{
|
#{
|
||||||
instance_id := InstanceId,
|
instance_id := InstanceId,
|
||||||
hookpoint := Hookpoint,
|
hookpoint := Hookpoint,
|
||||||
mqtt_config := #{
|
mqtt_config := #{
|
||||||
payload_template := PayloadTemplate,
|
payload_template := PayloadTemplate,
|
||||||
qos := MQTTQoS,
|
qos := MQTTQoS,
|
||||||
mqtt_topic := MQTTTopic
|
mqtt_topic := MQTTTopic
|
||||||
},
|
},
|
||||||
topic := Topic
|
topic := Topic
|
||||||
} = State,
|
} = State,
|
||||||
#{
|
#{
|
||||||
<<"messageId">> := MessageId,
|
<<"messageId">> := MessageId,
|
||||||
<<"publishTime">> := PublishTime
|
<<"publishTime">> := PublishTime
|
||||||
} = InnerMsg,
|
} = InnerMsg,
|
||||||
FullMessage0 = #{
|
FullMessage0 = #{
|
||||||
message_id => MessageId,
|
message_id => MessageId,
|
||||||
publish_time => PublishTime,
|
publish_time => PublishTime,
|
||||||
topic => Topic
|
topic => Topic
|
||||||
},
|
},
|
||||||
FullMessage =
|
FullMessage =
|
||||||
lists:foldl(
|
lists:foldl(
|
||||||
fun({FromKey, ToKey}, Acc) ->
|
fun({FromKey, ToKey}, Acc) ->
|
||||||
add_if_present(FromKey, InnerMsg, ToKey, Acc)
|
add_if_present(FromKey, InnerMsg, ToKey, Acc)
|
||||||
end,
|
end,
|
||||||
FullMessage0,
|
FullMessage0,
|
||||||
[
|
[
|
||||||
{<<"data">>, value},
|
{<<"data">>, value},
|
||||||
{<<"attributes">>, attributes},
|
{<<"attributes">>, attributes},
|
||||||
{<<"orderingKey">>, ordering_key}
|
{<<"orderingKey">>, ordering_key}
|
||||||
]
|
]
|
||||||
),
|
),
|
||||||
Payload = render(FullMessage, PayloadTemplate),
|
Payload = render(FullMessage, PayloadTemplate),
|
||||||
MQTTMessage = emqx_message:make(InstanceId, MQTTQoS, MQTTTopic, Payload),
|
MQTTMessage = emqx_message:make(InstanceId, MQTTQoS, MQTTTopic, Payload),
|
||||||
_ = emqx:publish(MQTTMessage),
|
_ = emqx:publish(MQTTMessage),
|
||||||
emqx:run_hook(Hookpoint, [FullMessage]),
|
emqx:run_hook(Hookpoint, [FullMessage]),
|
||||||
emqx_resource_metrics:received_inc(InstanceId),
|
emqx_resource_metrics:received_inc(InstanceId),
|
||||||
AckId.
|
ok
|
||||||
|
end
|
||||||
|
).
|
||||||
|
|
||||||
-spec add_if_present(any(), map(), any(), map()) -> map().
|
-spec add_if_present(any(), map(), any(), map()) -> map().
|
||||||
add_if_present(FromKey, Message, ToKey, Map) ->
|
add_if_present(FromKey, Message, ToKey, Map) ->
|
||||||
|
@ -563,6 +737,11 @@ render(FullMessage, PayloadTemplate) ->
|
||||||
Opts = #{return => full_binary},
|
Opts = #{return => full_binary},
|
||||||
emqx_placeholder:proc_tmpl(PayloadTemplate, FullMessage, Opts).
|
emqx_placeholder:proc_tmpl(PayloadTemplate, FullMessage, Opts).
|
||||||
|
|
||||||
|
forget_message_ids_after(MsgIds0, Timeout) ->
|
||||||
|
MsgIds = sets:from_list(MsgIds0, [{version, 2}]),
|
||||||
|
_ = erlang:send_after(Timeout, self(), {forget_message_ids, MsgIds}),
|
||||||
|
ok.
|
||||||
|
|
||||||
to_bin(A) when is_atom(A) -> atom_to_binary(A);
|
to_bin(A) when is_atom(A) -> atom_to_binary(A);
|
||||||
to_bin(L) when is_list(L) -> iolist_to_binary(L);
|
to_bin(L) when is_list(L) -> iolist_to_binary(L);
|
||||||
to_bin(B) when is_binary(B) -> B.
|
to_bin(B) when is_binary(B) -> B.
|
||||||
|
|
|
@ -15,6 +15,13 @@
|
||||||
on_get_status/2
|
on_get_status/2
|
||||||
]).
|
]).
|
||||||
|
|
||||||
|
%% health check API
|
||||||
|
-export([
|
||||||
|
mark_topic_as_nonexistent/1,
|
||||||
|
unset_nonexistent_topic/1,
|
||||||
|
is_nonexistent_topic/1
|
||||||
|
]).
|
||||||
|
|
||||||
-include_lib("emqx/include/logger.hrl").
|
-include_lib("emqx/include/logger.hrl").
|
||||||
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
|
||||||
-include_lib("emqx_resource/include/emqx_resource.hrl").
|
-include_lib("emqx_resource/include/emqx_resource.hrl").
|
||||||
|
@ -39,6 +46,12 @@
|
||||||
-export_type([mqtt_config/0]).
|
-export_type([mqtt_config/0]).
|
||||||
|
|
||||||
-define(AUTO_RECONNECT_S, 2).
|
-define(AUTO_RECONNECT_S, 2).
|
||||||
|
-define(DEFAULT_FORGET_INTERVAL, timer:seconds(60)).
|
||||||
|
-define(OPTVAR_TOPIC_NOT_FOUND(INSTANCE_ID), {?MODULE, topic_not_found, INSTANCE_ID}).
|
||||||
|
-define(TOPIC_MESSAGE,
|
||||||
|
"GCP PubSub topics are invalid. Please check the logs, check if the "
|
||||||
|
"topics exist in GCP and if the service account has permissions to use them."
|
||||||
|
).
|
||||||
|
|
||||||
%%-------------------------------------------------------------------------------------------------
|
%%-------------------------------------------------------------------------------------------------
|
||||||
%% `emqx_resource' API
|
%% `emqx_resource' API
|
||||||
|
@ -61,21 +74,45 @@ on_start(InstanceId, Config) ->
|
||||||
|
|
||||||
-spec on_stop(resource_id(), state()) -> ok | {error, term()}.
|
-spec on_stop(resource_id(), state()) -> ok | {error, term()}.
|
||||||
on_stop(InstanceId, _State) ->
|
on_stop(InstanceId, _State) ->
|
||||||
|
?tp(gcp_pubsub_consumer_stop_enter, #{}),
|
||||||
|
unset_nonexistent_topic(InstanceId),
|
||||||
ok = stop_consumers(InstanceId),
|
ok = stop_consumers(InstanceId),
|
||||||
emqx_bridge_gcp_pubsub_client:stop(InstanceId).
|
emqx_bridge_gcp_pubsub_client:stop(InstanceId).
|
||||||
|
|
||||||
-spec on_get_status(resource_id(), state()) -> connected | disconnected.
|
-spec on_get_status(resource_id(), state()) -> connected | connecting | {disconnected, state(), _}.
|
||||||
on_get_status(InstanceId, _State) ->
|
on_get_status(InstanceId, State) ->
|
||||||
%% Note: do *not* alter the `client' value here. It must be immutable, since
|
%% We need to check this flag separately because the workers might be gone when we
|
||||||
%% we have handed it over to the pull workers.
|
%% check them.
|
||||||
case
|
case is_nonexistent_topic(InstanceId) of
|
||||||
emqx_resource_pool:health_check_workers(
|
true ->
|
||||||
InstanceId,
|
{disconnected, State, {unhealthy_target, ?TOPIC_MESSAGE}};
|
||||||
fun emqx_bridge_gcp_pubsub_consumer_worker:health_check/1
|
false ->
|
||||||
)
|
#{client := Client} = State,
|
||||||
of
|
check_workers(InstanceId, Client)
|
||||||
true -> connected;
|
end.
|
||||||
false -> connecting
|
|
||||||
|
%%-------------------------------------------------------------------------------------------------
|
||||||
|
%% Health check API (signalled by consumer worker)
|
||||||
|
%%-------------------------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
-spec mark_topic_as_nonexistent(resource_id()) -> ok.
|
||||||
|
mark_topic_as_nonexistent(InstanceId) ->
|
||||||
|
optvar:set(?OPTVAR_TOPIC_NOT_FOUND(InstanceId), true),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
-spec unset_nonexistent_topic(resource_id()) -> ok.
|
||||||
|
unset_nonexistent_topic(InstanceId) ->
|
||||||
|
optvar:unset(?OPTVAR_TOPIC_NOT_FOUND(InstanceId)),
|
||||||
|
?tp(gcp_pubsub_consumer_unset_nonexistent_topic, #{}),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
-spec is_nonexistent_topic(resource_id()) -> boolean().
|
||||||
|
is_nonexistent_topic(InstanceId) ->
|
||||||
|
case optvar:peek(?OPTVAR_TOPIC_NOT_FOUND(InstanceId)) of
|
||||||
|
{ok, true} ->
|
||||||
|
true;
|
||||||
|
_ ->
|
||||||
|
false
|
||||||
end.
|
end.
|
||||||
|
|
||||||
%%-------------------------------------------------------------------------------------------------
|
%%-------------------------------------------------------------------------------------------------
|
||||||
|
@ -87,6 +124,7 @@ start_consumers(InstanceId, Client, Config) ->
|
||||||
bridge_name := BridgeName,
|
bridge_name := BridgeName,
|
||||||
consumer := ConsumerConfig0,
|
consumer := ConsumerConfig0,
|
||||||
hookpoint := Hookpoint,
|
hookpoint := Hookpoint,
|
||||||
|
resource_opts := #{request_ttl := RequestTTL},
|
||||||
service_account_json := #{project_id := ProjectId}
|
service_account_json := #{project_id := ProjectId}
|
||||||
} = Config,
|
} = Config,
|
||||||
ConsumerConfig1 = maps:update_with(topic_mapping, fun convert_topic_mapping/1, ConsumerConfig0),
|
ConsumerConfig1 = maps:update_with(topic_mapping, fun convert_topic_mapping/1, ConsumerConfig0),
|
||||||
|
@ -97,22 +135,27 @@ start_consumers(InstanceId, Client, Config) ->
|
||||||
auto_reconnect => ?AUTO_RECONNECT_S,
|
auto_reconnect => ?AUTO_RECONNECT_S,
|
||||||
bridge_name => BridgeName,
|
bridge_name => BridgeName,
|
||||||
client => Client,
|
client => Client,
|
||||||
|
forget_interval => forget_interval(RequestTTL),
|
||||||
hookpoint => Hookpoint,
|
hookpoint => Hookpoint,
|
||||||
instance_id => InstanceId,
|
instance_id => InstanceId,
|
||||||
pool_size => PoolSize,
|
pool_size => PoolSize,
|
||||||
project_id => ProjectId
|
project_id => ProjectId,
|
||||||
|
pull_retry_interval => RequestTTL
|
||||||
},
|
},
|
||||||
ConsumerOpts = maps:to_list(ConsumerConfig),
|
ConsumerOpts = maps:to_list(ConsumerConfig),
|
||||||
%% FIXME: mark as unhealthy if topics do not exist!
|
|
||||||
case validate_pubsub_topics(TopicMapping, Client) of
|
case validate_pubsub_topics(TopicMapping, Client) of
|
||||||
ok ->
|
ok ->
|
||||||
ok;
|
ok;
|
||||||
error ->
|
{error, not_found} ->
|
||||||
_ = emqx_bridge_gcp_pubsub_client:stop(InstanceId),
|
_ = emqx_bridge_gcp_pubsub_client:stop(InstanceId),
|
||||||
throw(
|
throw(
|
||||||
"GCP PubSub topics are invalid. Please check the logs, check if the "
|
{unhealthy_target, ?TOPIC_MESSAGE}
|
||||||
"topic exists in GCP and if the service account has permissions to use them."
|
);
|
||||||
)
|
{error, _} ->
|
||||||
|
%% connection might be down; we'll have to check topic existence during health
|
||||||
|
%% check, or the workers will kill themselves when they realized there's no
|
||||||
|
%% topic when upserting their subscription.
|
||||||
|
ok
|
||||||
end,
|
end,
|
||||||
case
|
case
|
||||||
emqx_resource_pool:start(InstanceId, emqx_bridge_gcp_pubsub_consumer_worker, ConsumerOpts)
|
emqx_resource_pool:start(InstanceId, emqx_bridge_gcp_pubsub_consumer_worker, ConsumerOpts)
|
||||||
|
@ -170,8 +213,8 @@ do_validate_pubsub_topics(Client, [Topic | Rest]) ->
|
||||||
case check_for_topic_existence(Topic, Client) of
|
case check_for_topic_existence(Topic, Client) of
|
||||||
ok ->
|
ok ->
|
||||||
do_validate_pubsub_topics(Client, Rest);
|
do_validate_pubsub_topics(Client, Rest);
|
||||||
{error, _} ->
|
{error, _} = Err ->
|
||||||
error
|
Err
|
||||||
end;
|
end;
|
||||||
do_validate_pubsub_topics(_Client, []) ->
|
do_validate_pubsub_topics(_Client, []) ->
|
||||||
%% we already validate that the mapping is not empty in the config schema.
|
%% we already validate that the mapping is not empty in the config schema.
|
||||||
|
@ -184,9 +227,38 @@ check_for_topic_existence(Topic, Client) ->
|
||||||
ok;
|
ok;
|
||||||
{error, #{status_code := 404}} ->
|
{error, #{status_code := 404}} ->
|
||||||
{error, not_found};
|
{error, not_found};
|
||||||
{error, Details} ->
|
{error, Reason} ->
|
||||||
?tp(warning, "gcp_pubsub_consumer_check_topic_error", Details),
|
?tp(warning, "gcp_pubsub_consumer_check_topic_error", #{reason => Reason}),
|
||||||
{error, Details}
|
{error, Reason}
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec get_client_status(emqx_bridge_gcp_pubsub_client:state()) -> connected | connecting.
|
||||||
|
get_client_status(Client) ->
|
||||||
|
case emqx_bridge_gcp_pubsub_client:get_status(Client) of
|
||||||
|
disconnected -> connecting;
|
||||||
|
connected -> connected
|
||||||
|
end.
|
||||||
|
|
||||||
|
-spec check_workers(resource_id(), emqx_bridge_gcp_pubsub_client:state()) -> connected | connecting.
|
||||||
|
check_workers(InstanceId, Client) ->
|
||||||
|
case
|
||||||
|
emqx_resource_pool:health_check_workers(
|
||||||
|
InstanceId,
|
||||||
|
fun emqx_bridge_gcp_pubsub_consumer_worker:health_check/1,
|
||||||
|
emqx_resource_pool:health_check_timeout(),
|
||||||
|
#{return_values => true}
|
||||||
|
)
|
||||||
|
of
|
||||||
|
{ok, Values} ->
|
||||||
|
AllOk = lists:all(fun(S) -> S =:= subscription_ok end, Values),
|
||||||
|
case AllOk of
|
||||||
|
true ->
|
||||||
|
get_client_status(Client);
|
||||||
|
false ->
|
||||||
|
connecting
|
||||||
|
end;
|
||||||
|
{error, _} ->
|
||||||
|
connecting
|
||||||
end.
|
end.
|
||||||
|
|
||||||
log_when_error(Fun, Log) ->
|
log_when_error(Fun, Log) ->
|
||||||
|
@ -199,3 +271,6 @@ log_when_error(Fun, Log) ->
|
||||||
reason => E
|
reason => E
|
||||||
})
|
})
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
forget_interval(infinity) -> ?DEFAULT_FORGET_INTERVAL;
|
||||||
|
forget_interval(Timeout) -> 2 * Timeout.
|
||||||
|
|
|
@ -103,7 +103,7 @@ on_query(ResourceId, {send_message, Selected}, State) ->
|
||||||
{send_message, map()},
|
{send_message, map()},
|
||||||
{ReplyFun :: function(), Args :: list()},
|
{ReplyFun :: function(), Args :: list()},
|
||||||
state()
|
state()
|
||||||
) -> {ok, pid()}.
|
) -> {ok, pid()} | {error, no_pool_worker_available}.
|
||||||
on_query_async(ResourceId, {send_message, Selected}, ReplyFunAndArgs, State) ->
|
on_query_async(ResourceId, {send_message, Selected}, ReplyFunAndArgs, State) ->
|
||||||
Requests = [{send_message, Selected}],
|
Requests = [{send_message, Selected}],
|
||||||
?TRACE(
|
?TRACE(
|
||||||
|
@ -134,7 +134,7 @@ on_batch_query(ResourceId, Requests, State) ->
|
||||||
[{send_message, map()}],
|
[{send_message, map()}],
|
||||||
{ReplyFun :: function(), Args :: list()},
|
{ReplyFun :: function(), Args :: list()},
|
||||||
state()
|
state()
|
||||||
) -> {ok, pid()}.
|
) -> {ok, pid()} | {error, no_pool_worker_available}.
|
||||||
on_batch_query_async(ResourceId, Requests, ReplyFunAndArgs, State) ->
|
on_batch_query_async(ResourceId, Requests, ReplyFunAndArgs, State) ->
|
||||||
?TRACE(
|
?TRACE(
|
||||||
"QUERY_ASYNC",
|
"QUERY_ASYNC",
|
||||||
|
@ -177,7 +177,7 @@ do_send_requests_sync(State, Requests, InstanceId) ->
|
||||||
state(),
|
state(),
|
||||||
[{send_message, map()}],
|
[{send_message, map()}],
|
||||||
{ReplyFun :: function(), Args :: list()}
|
{ReplyFun :: function(), Args :: list()}
|
||||||
) -> {ok, pid()}.
|
) -> {ok, pid()} | {error, no_pool_worker_available}.
|
||||||
do_send_requests_async(State, Requests, ReplyFunAndArgs0) ->
|
do_send_requests_async(State, Requests, ReplyFunAndArgs0) ->
|
||||||
#{client := Client} = State,
|
#{client := Client} = State,
|
||||||
Payloads =
|
Payloads =
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -19,6 +19,7 @@
|
||||||
-export([
|
-export([
|
||||||
start/3,
|
start/3,
|
||||||
stop/1,
|
stop/1,
|
||||||
|
health_check_timeout/0,
|
||||||
health_check_workers/2,
|
health_check_workers/2,
|
||||||
health_check_workers/3,
|
health_check_workers/3,
|
||||||
health_check_workers/4
|
health_check_workers/4
|
||||||
|
@ -66,6 +67,9 @@ stop(Name) ->
|
||||||
error({stop_pool_failed, Name, Reason})
|
error({stop_pool_failed, Name, Reason})
|
||||||
end.
|
end.
|
||||||
|
|
||||||
|
health_check_timeout() ->
|
||||||
|
?HEALTH_CHECK_TIMEOUT.
|
||||||
|
|
||||||
health_check_workers(PoolName, CheckFunc) ->
|
health_check_workers(PoolName, CheckFunc) ->
|
||||||
health_check_workers(PoolName, CheckFunc, ?HEALTH_CHECK_TIMEOUT, _Opts = #{}).
|
health_check_workers(PoolName, CheckFunc, ?HEALTH_CHECK_TIMEOUT, _Opts = #{}).
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue