fix: create pulsar producer when on_add_channel

This commit is contained in:
zhongwencool 2024-02-27 13:32:30 +08:00
parent 7f1b4cef27
commit 650f9659a4
6 changed files with 183 additions and 169 deletions

View File

@ -33,7 +33,6 @@ roots() ->
fields(pulsar_producer) -> fields(pulsar_producer) ->
fields(config) ++ fields(config) ++
emqx_bridge_pulsar_pubsub_schema:fields(action_parameters) ++ emqx_bridge_pulsar_pubsub_schema:fields(action_parameters) ++
fields(producer_opts) ++
[ [
{local_topic, {local_topic,
mk(binary(), #{required => false, desc => ?DESC("producer_local_topic")})}, mk(binary(), #{required => false, desc => ?DESC("producer_local_topic")})},
@ -85,6 +84,7 @@ fields(config) ->
] ++ emqx_connector_schema_lib:ssl_fields(); ] ++ emqx_connector_schema_lib:ssl_fields();
fields(producer_opts) -> fields(producer_opts) ->
[ [
{pulsar_topic, mk(string(), #{required => true, desc => ?DESC("producer_pulsar_topic")})},
{batch_size, {batch_size,
mk( mk(
pos_integer(), pos_integer(),
@ -110,7 +110,6 @@ fields(producer_opts) ->
emqx_schema:bytesize(), emqx_schema:bytesize(),
#{default => <<"900KB">>, desc => ?DESC("producer_max_batch_bytes")} #{default => <<"900KB">>, desc => ?DESC("producer_max_batch_bytes")}
)}, )},
{pulsar_topic, mk(binary(), #{required => true, desc => ?DESC("producer_pulsar_topic")})},
{strategy, {strategy,
mk( mk(
hoconsc:enum([random, roundrobin, key_dispatch]), hoconsc:enum([random, roundrobin, key_dispatch]),
@ -202,7 +201,37 @@ conn_bridge_examples(_Method) ->
#{ #{
<<"pulsar_producer">> => #{ <<"pulsar_producer">> => #{
summary => <<"Pulsar Producer Bridge">>, summary => <<"Pulsar Producer Bridge">>,
value => #{todo => true} value => #{
<<"authentication">> => <<"none">>,
<<"batch_size">> => 1,
<<"buffer">> =>
#{
<<"memory_overload_protection">> => true,
<<"mode">> => <<"memory">>,
<<"per_partition_limit">> => <<"10MB">>,
<<"segment_bytes">> => <<"5MB">>
},
<<"compression">> => <<"no_compression">>,
<<"enable">> => true,
<<"local_topic">> => <<"mqtt/topic/-576460752303423482">>,
<<"max_batch_bytes">> => <<"900KB">>,
<<"message">> =>
#{<<"key">> => <<"${.clientid}">>, <<"value">> => <<"${.}">>},
<<"name">> => <<"pulsar_example_name">>,
<<"pulsar_topic">> => <<"pulsar_example_topic">>,
<<"retention_period">> => <<"infinity">>,
<<"send_buffer">> => <<"1MB">>,
<<"servers">> => <<"pulsar://127.0.0.1:6650">>,
<<"ssl">> =>
#{
<<"enable">> => false,
<<"server_name_indication">> => <<"auto">>,
<<"verify">> => <<"verify_none">>
},
<<"strategy">> => <<"key_dispatch">>,
<<"sync_timeout">> => <<"5s">>,
<<"type">> => <<"pulsar_producer">>
}
} }
} }
]. ].

View File

@ -11,8 +11,7 @@
action_type_name/0, action_type_name/0,
connector_type_name/0, connector_type_name/0,
schema_module/0, schema_module/0,
is_action/1, is_action/1
action_convert_from_connector/2
]). ]).
is_action(_) -> true. is_action(_) -> true.
@ -24,31 +23,3 @@ action_type_name() -> pulsar.
connector_type_name() -> pulsar. connector_type_name() -> pulsar.
schema_module() -> emqx_bridge_pulsar_pubsub_schema. schema_module() -> emqx_bridge_pulsar_pubsub_schema.
action_convert_from_connector(ConnectorConfig, ActionConfig) ->
Dispatch = emqx_utils_conv:bin(maps:get(<<"strategy">>, ConnectorConfig, <<>>)),
case Dispatch of
<<"key_dispatch">> ->
case emqx_utils_maps:deep_find([<<"parameters">>, <<"message">>], ActionConfig) of
{ok, Message} ->
Validator =
#{
<<"strategy">> => key_dispatch,
<<"message">> => emqx_utils_maps:binary_key_map(Message)
},
case emqx_bridge_pulsar:producer_strategy_key_validator(Validator) of
ok ->
ActionConfig;
{error, Reason} ->
throw(#{
reason => Reason,
kind => validation_error
})
end;
{not_found, _, _} ->
%% no message field, use the default message template
ActionConfig
end;
_ ->
ActionConfig
end.

View File

@ -25,13 +25,11 @@
-type pulsar_client_id() :: atom(). -type pulsar_client_id() :: atom().
-type state() :: #{ -type state() :: #{
pulsar_client_id := pulsar_client_id(), client_id := pulsar_client_id(),
producers := pulsar_producers:producers(), channels := map(),
channels := map() client_opts := map()
}. }.
-type buffer_mode() :: memory | disk | hybrid.
-type compression_mode() :: no_compression | snappy | zlib.
-type partition_strategy() :: random | roundrobin | key_dispatch.
-type message_template_raw() :: #{ -type message_template_raw() :: #{
key := binary(), key := binary(),
value := binary() value := binary()
@ -42,25 +40,9 @@
}. }.
-type config() :: #{ -type config() :: #{
authentication := _, authentication := _,
batch_size := pos_integer(),
bridge_name := atom(), bridge_name := atom(),
buffer := #{
mode := buffer_mode(),
per_partition_limit := emqx_schema:bytesize(),
segment_bytes := emqx_schema:bytesize(),
memory_overload_protection := boolean()
},
compression := compression_mode(),
connect_timeout := emqx_schema:duration_ms(),
max_batch_bytes := emqx_schema:bytesize(),
message := message_template_raw(),
pulsar_topic := binary(),
retention_period := infinity | emqx_schema:duration_ms(),
send_buffer := emqx_schema:bytesize(),
servers := binary(), servers := binary(),
ssl := _, ssl := _
strategy := partition_strategy(),
sync_timeout := emqx_schema:duration_ms()
}. }.
%% Allocatable resources %% Allocatable resources
@ -116,54 +98,57 @@ on_start(InstanceId, Config) ->
end, end,
throw(Message) throw(Message)
end, end,
start_producer(Config, InstanceId, ClientId, ClientOpts). {ok, #{channels => #{}, client_id => ClientId, client_opts => ClientOpts}}.
on_add_channel( on_add_channel(
_InstanceId, InstanceId,
#{channels := Channels} = State, #{channels := Channels, client_id := ClientId, client_opts := ClientOpts} = State,
ChannelId, ChannelId,
#{parameters := #{message := Message, sync_timeout := SyncTimeout}} #{parameters := #{message := Message, sync_timeout := SyncTimeout} = Params}
) -> ) ->
case maps:is_key(ChannelId, Channels) of case maps:is_key(ChannelId, Channels) of
true -> true ->
{error, already_exists}; {error, channel_already_exists};
false -> false ->
{ok, Producers} = start_producer(InstanceId, ChannelId, ClientId, ClientOpts, Params),
Parameters = #{ Parameters = #{
message => compile_message_template(Message), message => compile_message_template(Message),
sync_timeout => SyncTimeout sync_timeout => SyncTimeout,
producers => Producers
}, },
NewChannels = maps:put(ChannelId, Parameters, Channels), NewChannels = maps:put(ChannelId, Parameters, Channels),
{ok, State#{channels => NewChannels}} {ok, State#{channels => NewChannels}}
end. end.
on_remove_channel(_InstanceId, #{channels := Channels} = State, ChannelId) -> on_remove_channel(InstanceId, State, ChannelId) ->
{ok, State#{channels => maps:remove(ChannelId, Channels)}}. #{channels := Channels, client_id := ClientId} = State,
case maps:find(ChannelId, Channels) of
{ok, #{producers := Producers}} ->
stop_producers(ClientId, Producers),
emqx_resource:deallocate_resource(InstanceId, {?pulsar_producers, ChannelId}),
{ok, State#{channels => maps:remove(ChannelId, Channels)}};
error ->
{ok, State}
end.
on_get_channels(InstanceId) -> on_get_channels(InstanceId) ->
emqx_bridge_v2:get_channels_for_connector(InstanceId). emqx_bridge_v2:get_channels_for_connector(InstanceId).
-spec on_stop(resource_id(), state()) -> ok. -spec on_stop(resource_id(), state()) -> ok.
on_stop(InstanceId, _State) -> on_stop(InstanceId, _State) ->
case emqx_resource:get_allocated_resources(InstanceId) of Resources0 = emqx_resource:get_allocated_resources(InstanceId),
#{?pulsar_client_id := ClientId, ?pulsar_producers := Producers} -> case maps:take(?pulsar_client_id, Resources0) of
stop_producers(ClientId, Producers), {ClientId, Resources} ->
maps:foreach(
fun({?pulsar_producers, _BridgeV2Id}, Producers) ->
stop_producers(ClientId, Producers)
end,
Resources
),
stop_client(ClientId), stop_client(ClientId),
?tp(pulsar_bridge_stopped, #{
instance_id => InstanceId,
pulsar_client_id => ClientId,
pulsar_producers => Producers
}),
ok;
#{?pulsar_client_id := ClientId} ->
stop_client(ClientId),
?tp(pulsar_bridge_stopped, #{
instance_id => InstanceId,
pulsar_client_id => ClientId,
pulsar_producers => undefined
}),
ok;
_ ->
?tp(pulsar_bridge_stopped, #{instance_id => InstanceId}), ?tp(pulsar_bridge_stopped, #{instance_id => InstanceId}),
ok;
error ->
ok ok
end. end.
@ -172,35 +157,32 @@ on_stop(InstanceId, _State) ->
%% `emqx_resource_manager' will kill the Pulsar producers and messages might be lost. %% `emqx_resource_manager' will kill the Pulsar producers and messages might be lost.
-spec on_get_status(resource_id(), state()) -> connected | connecting. -spec on_get_status(resource_id(), state()) -> connected | connecting.
on_get_status(_InstanceId, State = #{}) -> on_get_status(_InstanceId, State = #{}) ->
#{ #{client_id := ClientId} = State,
pulsar_client_id := ClientId,
producers := Producers
} = State,
case pulsar_client_sup:find_client(ClientId) of case pulsar_client_sup:find_client(ClientId) of
{ok, Pid} -> {ok, Pid} ->
try pulsar_client:get_status(Pid) of try pulsar_client:get_status(Pid) of
true -> true -> ?status_connected;
get_producer_status(Producers); false -> ?status_connecting
false ->
connecting
catch catch
error:timeout -> error:timeout ->
connecting; ?status_connecting;
exit:{noproc, _} -> exit:{noproc, _} ->
connecting ?status_connecting
end; end;
{error, _} -> {error, _} ->
connecting ?status_connecting
end; end;
on_get_status(_InstanceId, _State) -> on_get_status(_InstanceId, _State) ->
%% If a health check happens just after a concurrent request to %% If a health check happens just after a concurrent request to
%% create the bridge is not quite finished, `State = undefined'. %% create the bridge is not quite finished, `State = undefined'.
connecting. ?status_connecting.
on_get_channel_status(_InstanceId, ChannelId, #{channels := Channels}) -> on_get_channel_status(_InstanceId, ChannelId, #{channels := Channels}) ->
case maps:is_key(ChannelId, Channels) of case maps:find(ChannelId, Channels) of
true -> connected; {ok, #{producers := Producers}} ->
false -> {error, channel_not_exists} get_producer_status(Producers);
error ->
{error, channel_not_exists}
end. end.
-spec on_query(resource_id(), tuple(), state()) -> -spec on_query(resource_id(), tuple(), state()) ->
@ -208,11 +190,11 @@ on_get_channel_status(_InstanceId, ChannelId, #{channels := Channels}) ->
| {error, timeout} | {error, timeout}
| {error, term()}. | {error, term()}.
on_query(_InstanceId, {ChannelId, Message}, State) -> on_query(_InstanceId, {ChannelId, Message}, State) ->
#{producers := Producers, channels := Channels} = State, #{channels := Channels} = State,
case maps:find(ChannelId, Channels) of case maps:find(ChannelId, Channels) of
error -> error ->
{error, channel_not_exists}; {error, channel_not_exists};
{ok, #{message := MessageTmpl, sync_timeout := SyncTimeout}} -> {ok, #{message := MessageTmpl, sync_timeout := SyncTimeout, producers := Producers}} ->
PulsarMessage = render_message(Message, MessageTmpl), PulsarMessage = render_message(Message, MessageTmpl),
try try
pulsar:send_sync(Producers, [PulsarMessage], SyncTimeout) pulsar:send_sync(Producers, [PulsarMessage], SyncTimeout)
@ -227,11 +209,11 @@ on_query(_InstanceId, {ChannelId, Message}, State) ->
) -> ) ->
{ok, pid()}. {ok, pid()}.
on_query_async(_InstanceId, {ChannelId, Message}, AsyncReplyFn, State) -> on_query_async(_InstanceId, {ChannelId, Message}, AsyncReplyFn, State) ->
#{producers := Producers, channels := Channels} = State, #{channels := Channels} = State,
case maps:find(ChannelId, Channels) of case maps:find(ChannelId, Channels) of
error -> error ->
{error, channel_not_exists}; {error, channel_not_exists};
{ok, #{message := MessageTmpl}} -> {ok, #{message := MessageTmpl, producers := Producers}} ->
?tp_span( ?tp_span(
pulsar_producer_on_query_async, pulsar_producer_on_query_async,
#{instance_id => _InstanceId, message => Message}, #{instance_id => _InstanceId, message => Message},
@ -299,18 +281,22 @@ conn_opts(#{authentication := #{jwt := JWT}}) ->
replayq_dir(ClientId) -> replayq_dir(ClientId) ->
filename:join([emqx:data_dir(), "pulsar", emqx_utils_conv:bin(ClientId)]). filename:join([emqx:data_dir(), "pulsar", emqx_utils_conv:bin(ClientId)]).
-spec producer_name(pulsar_client_id()) -> atom(). producer_name(InstanceId, ChannelId) ->
producer_name(ClientId) -> case is_dry_run(InstanceId) of
ClientIdBin = emqx_utils_conv:bin(ClientId), %% do not create more atom
binary_to_atom( true ->
iolist_to_binary([ pulsar_producer_probe_worker;
<<"producer-">>, false ->
ClientIdBin ChannelIdBin = emqx_utils_conv:bin(ChannelId),
]) binary_to_atom(
). iolist_to_binary([
<<"producer-">>,
ChannelIdBin
])
)
end.
-spec start_producer(config(), resource_id(), pulsar_client_id(), map()) -> {ok, state()}. start_producer(InstanceId, ChannelId, ClientId, ClientOpts, Params) ->
start_producer(Config, InstanceId, ClientId, ClientOpts) ->
#{ #{
conn_opts := ConnOpts, conn_opts := ConnOpts,
ssl_opts := SSLOpts ssl_opts := SSLOpts
@ -325,16 +311,16 @@ start_producer(Config, InstanceId, ClientId, ClientOpts) ->
}, },
compression := Compression, compression := Compression,
max_batch_bytes := MaxBatchBytes, max_batch_bytes := MaxBatchBytes,
pulsar_topic := PulsarTopic0, pulsar_topic := PulsarTopic,
retention_period := RetentionPeriod, retention_period := RetentionPeriod,
send_buffer := SendBuffer, send_buffer := SendBuffer,
strategy := Strategy strategy := Strategy
} = Config, } = Params,
{OffloadMode, ReplayQDir} = {OffloadMode, ReplayQDir} =
case BufferMode of case BufferMode of
memory -> {false, false}; memory -> {false, false};
disk -> {false, replayq_dir(ClientId)}; disk -> {false, replayq_dir(ChannelId)};
hybrid -> {true, replayq_dir(ClientId)} hybrid -> {true, replayq_dir(ChannelId)}
end, end,
MemOLP = MemOLP =
case os:type() of case os:type() of
@ -348,7 +334,7 @@ start_producer(Config, InstanceId, ClientId, ClientOpts) ->
replayq_seg_bytes => SegmentBytes, replayq_seg_bytes => SegmentBytes,
drop_if_highmem => MemOLP drop_if_highmem => MemOLP
}, },
ProducerName = producer_name(ClientId), ProducerName = producer_name(InstanceId, ChannelId),
?tp(pulsar_producer_capture_name, #{producer_name => ProducerName}), ?tp(pulsar_producer_capture_name, #{producer_name => ProducerName}),
ProducerOpts0 = ProducerOpts0 =
#{ #{
@ -363,19 +349,17 @@ start_producer(Config, InstanceId, ClientId, ClientOpts) ->
tcp_opts => [{sndbuf, SendBuffer}] tcp_opts => [{sndbuf, SendBuffer}]
}, },
ProducerOpts = maps:merge(ReplayQOpts, ProducerOpts0), ProducerOpts = maps:merge(ReplayQOpts, ProducerOpts0),
PulsarTopic = binary_to_list(PulsarTopic0),
?tp(pulsar_producer_about_to_start_producers, #{producer_name => ProducerName}), ?tp(pulsar_producer_about_to_start_producers, #{producer_name => ProducerName}),
try pulsar:ensure_supervised_producers(ClientId, PulsarTopic, ProducerOpts) of try pulsar:ensure_supervised_producers(ClientId, PulsarTopic, ProducerOpts) of
{ok, Producers} -> {ok, Producers} ->
ok = emqx_resource:allocate_resource(InstanceId, ?pulsar_producers, Producers), ok = emqx_resource:allocate_resource(
InstanceId,
{?pulsar_producers, ChannelId},
Producers
),
?tp(pulsar_producer_producers_allocated, #{}), ?tp(pulsar_producer_producers_allocated, #{}),
State = #{
pulsar_client_id => ClientId,
producers => Producers,
channels => #{}
},
?tp(pulsar_producer_bridge_started, #{}), ?tp(pulsar_producer_bridge_started, #{}),
{ok, State} {ok, Producers}
catch catch
Kind:Error:Stacktrace -> Kind:Error:Stacktrace ->
?tp( ?tp(
@ -388,7 +372,10 @@ start_producer(Config, InstanceId, ClientId, ClientOpts) ->
stacktrace => Stacktrace stacktrace => Stacktrace
} }
), ),
stop_client(ClientId), ?tp(pulsar_bridge_producer_stopped, #{
pulsar_client_id => ClientId,
producers => undefined
}),
throw(failed_to_start_pulsar_producer) throw(failed_to_start_pulsar_producer)
end. end.
@ -412,7 +399,10 @@ stop_producers(ClientId, Producers) ->
_ = log_when_error( _ = log_when_error(
fun() -> fun() ->
ok = pulsar:stop_and_delete_supervised_producers(Producers), ok = pulsar:stop_and_delete_supervised_producers(Producers),
?tp(pulsar_bridge_producer_stopped, #{pulsar_client_id => ClientId}), ?tp(pulsar_bridge_producer_stopped, #{
pulsar_client_id => ClientId,
producers => Producers
}),
ok ok
end, end,
#{ #{
@ -467,15 +457,19 @@ get_producer_status(Producers) ->
do_get_producer_status(Producers, 0). do_get_producer_status(Producers, 0).
do_get_producer_status(_Producers, TimeSpent) when TimeSpent > ?HEALTH_CHECK_RETRY_TIMEOUT -> do_get_producer_status(_Producers, TimeSpent) when TimeSpent > ?HEALTH_CHECK_RETRY_TIMEOUT ->
connecting; ?status_connecting;
do_get_producer_status(Producers, TimeSpent) -> do_get_producer_status(Producers, TimeSpent) ->
case pulsar_producers:all_connected(Producers) of try pulsar_producers:all_connected(Producers) of
true -> true ->
connected; ?status_connected;
false -> false ->
Sleep = 200, Sleep = 200,
timer:sleep(Sleep), timer:sleep(Sleep),
do_get_producer_status(Producers, TimeSpent + Sleep) do_get_producer_status(Producers, TimeSpent + Sleep)
%% producer crashed with badarg. will recover later
catch
error:badarg ->
?status_connecting
end. end.
partition_strategy(key_dispatch) -> first_key_dispatch; partition_strategy(key_dispatch) -> first_key_dispatch;
@ -485,17 +479,17 @@ is_sensitive_key(auth_data) -> true;
is_sensitive_key(_) -> false. is_sensitive_key(_) -> false.
get_error_message({BrokerErrorMap, _}) when is_map(BrokerErrorMap) -> get_error_message({BrokerErrorMap, _}) when is_map(BrokerErrorMap) ->
Iter = maps:iterator(BrokerErrorMap), Iterator = maps:iterator(BrokerErrorMap),
do_get_error_message(Iter); do_get_error_message(Iterator);
get_error_message(_Error) -> get_error_message(_Error) ->
error. error.
do_get_error_message(Iter) -> do_get_error_message(Iterator) ->
case maps:next(Iter) of case maps:next(Iterator) of
{{_Broker, _Port}, #{message := Message}, _NIter} -> {{_Broker, _Port}, #{message := Message}, _NIterator} ->
{ok, Message}; {ok, Message};
{_K, _V, NIter} -> {_K, _V, NIterator} ->
do_get_error_message(NIter); do_get_error_message(NIterator);
none -> none ->
error error
end. end.

View File

@ -18,9 +18,8 @@ namespace() -> ?TYPE.
roots() -> []. roots() -> [].
fields("config_connector") -> fields("config_connector") ->
lists:keydelete(enable, 1, emqx_bridge_schema:common_bridge_fields()) ++ emqx_bridge_schema:common_bridge_fields() ++
emqx_bridge_pulsar:fields(config) ++ lists:keydelete(enable, 1, emqx_bridge_pulsar:fields(config)) ++
emqx_bridge_pulsar:fields(producer_opts) ++
emqx_connector_schema:resource_opts_ref(?MODULE, connector_resource_opts); emqx_connector_schema:resource_opts_ref(?MODULE, connector_resource_opts);
fields(connector_resource_opts) -> fields(connector_resource_opts) ->
emqx_connector_schema:resource_opts_fields(); emqx_connector_schema:resource_opts_fields();
@ -59,13 +58,5 @@ connector_example_values() ->
servers => <<"pulsar://127.0.0.1:6650">>, servers => <<"pulsar://127.0.0.1:6650">>,
authentication => none, authentication => none,
connect_timeout => <<"5s">>, connect_timeout => <<"5s">>,
batch_size => 10,
compression => no_compression,
send_buffer => <<"1MB">>,
retention_period => <<"100s">>,
max_batch_bytes => <<"32MB">>,
pulsar_topic => <<"test_topic">>,
strategy => random,
buffer => #{mode => memory},
ssl => #{enable => false} ssl => #{enable => false}
}. }.

View File

@ -32,22 +32,23 @@ fields(publisher_action) ->
?R_REF(action_parameters), ?R_REF(action_parameters),
#{ #{
required => true, required => true,
desc => ?DESC(action_parameters) desc => ?DESC(action_parameters),
validator => fun emqx_bridge_pulsar:producer_strategy_key_validator/1
} }
), ),
#{resource_opts_ref => ?R_REF(action_resource_opts)} #{resource_opts_ref => ?R_REF(action_resource_opts)}
); );
fields(action_parameters) -> fields(action_parameters) ->
[ [
{sync_timeout,
?HOCON(emqx_schema:timeout_duration_ms(), #{
default => <<"3s">>, desc => ?DESC("producer_sync_timeout")
})},
{message, {message,
?HOCON(?R_REF(producer_pulsar_message), #{ ?HOCON(?R_REF(producer_pulsar_message), #{
required => false, desc => ?DESC("producer_message_opts") required => false, desc => ?DESC("producer_message_opts")
})},
{sync_timeout,
?HOCON(emqx_schema:timeout_duration_ms(), #{
default => <<"3s">>, desc => ?DESC("producer_sync_timeout")
})} })}
]; ] ++ emqx_bridge_pulsar:fields(producer_opts);
fields(producer_pulsar_message) -> fields(producer_pulsar_message) ->
[ [
{key, {key,
@ -114,7 +115,8 @@ bridge_v2_examples(Method) ->
message => #{ message => #{
key => <<"${.clientid}">>, key => <<"${.clientid}">>,
value => <<"${.}">> value => <<"${.}">>
} },
pulsar_topic => <<"test_topic">>
} }
} }
) )

View File

@ -599,13 +599,7 @@ t_start_and_produce_ok(Config) ->
_Sleep = 100, _Sleep = 100,
_Attempts0 = 20, _Attempts0 = 20,
begin begin
BridgeId = emqx_bridge_resource:bridge_id( Id = get_channel_id(Config),
<<"pulsar">>, ?config(pulsar_name, Config)
),
ConnectorId = emqx_bridge_resource:resource_id(
<<"pulsar">>, ?config(pulsar_name, Config)
),
Id = <<"action:", BridgeId/binary, ":", ConnectorId/binary>>,
?assertMatch( ?assertMatch(
#{ #{
counters := #{ counters := #{
@ -634,6 +628,15 @@ t_start_and_produce_ok(Config) ->
), ),
ok. ok.
get_channel_id(Config) ->
BridgeId = emqx_bridge_resource:bridge_id(
<<"pulsar">>, ?config(pulsar_name, Config)
),
ConnectorId = emqx_bridge_resource:resource_id(
<<"pulsar">>, ?config(pulsar_name, Config)
),
<<"action:", BridgeId/binary, ":", ConnectorId/binary>>.
%% Under normal operations, the bridge will be called async via %% Under normal operations, the bridge will be called async via
%% `simple_async_query'. %% `simple_async_query'.
t_sync_query(Config) -> t_sync_query(Config) ->
@ -900,7 +903,7 @@ t_failure_to_start_producer(Config) ->
{{ok, _}, {ok, _}} = {{ok, _}, {ok, _}} =
?wait_async_action( ?wait_async_action(
create_bridge(Config), create_bridge(Config),
#{?snk_kind := pulsar_bridge_client_stopped}, #{?snk_kind := pulsar_bridge_producer_stopped},
20_000 20_000
), ),
ok ok
@ -928,6 +931,8 @@ t_producer_process_crash(Config) ->
#{?snk_kind := pulsar_producer_bridge_started}, #{?snk_kind := pulsar_producer_bridge_started},
10_000 10_000
), ),
ResourceId = resource_id(Config),
ChannelId = get_channel_id(Config),
[ProducerPid | _] = [ [ProducerPid | _] = [
Pid Pid
|| {_Name, PS, _Type, _Mods} <- supervisor:which_children(pulsar_producers_sup), || {_Name, PS, _Type, _Mods} <- supervisor:which_children(pulsar_producers_sup),
@ -944,17 +949,23 @@ t_producer_process_crash(Config) ->
ok ok
after 1_000 -> ct:fail("pid didn't die") after 1_000 -> ct:fail("pid didn't die")
end, end,
ResourceId = resource_id(Config),
?retry( ?retry(
_Sleep0 = 50, _Sleep0 = 50,
_Attempts0 = 50, _Attempts0 = 50,
?assertEqual({ok, connecting}, emqx_resource_manager:health_check(ResourceId)) ?assertEqual(
#{error => <<"Not connected for unknown reason">>, status => connecting},
emqx_resource_manager:channel_health_check(ResourceId, ChannelId)
)
), ),
?assertMatch({ok, connected}, emqx_resource_manager:health_check(ResourceId)),
%% Should recover given enough time. %% Should recover given enough time.
?retry( ?retry(
_Sleep = 1_000, _Sleep = 1_000,
_Attempts = 20, _Attempts = 20,
?assertEqual({ok, connected}, emqx_resource_manager:health_check(ResourceId)) ?assertEqual(
#{error => undefined, status => connected},
emqx_resource_manager:channel_health_check(ResourceId, ChannelId)
)
), ),
{_, {ok, _}} = {_, {ok, _}} =
?wait_async_action( ?wait_async_action(
@ -1002,8 +1013,8 @@ t_resource_manager_crash_after_producers_started(Config) ->
{{error, {config_update_crashed, {killed, _}}}, {ok, _}} = {{error, {config_update_crashed, {killed, _}}}, {ok, _}} =
?wait_async_action( ?wait_async_action(
create_bridge(Config), create_bridge(Config),
#{?snk_kind := pulsar_bridge_stopped, pulsar_producers := Producers} when #{?snk_kind := pulsar_bridge_stopped, instance_id := InstanceId} when
Producers =/= undefined, InstanceId =/= undefined,
10_000 10_000
), ),
?assertEqual([], get_pulsar_producers()), ?assertEqual([], get_pulsar_producers()),
@ -1036,7 +1047,7 @@ t_resource_manager_crash_before_producers_started(Config) ->
{{error, {config_update_crashed, _}}, {ok, _}} = {{error, {config_update_crashed, _}}, {ok, _}} =
?wait_async_action( ?wait_async_action(
create_bridge(Config), create_bridge(Config),
#{?snk_kind := pulsar_bridge_stopped, pulsar_producers := undefined}, #{?snk_kind := pulsar_bridge_stopped},
10_000 10_000
), ),
?assertEqual([], get_pulsar_producers()), ?assertEqual([], get_pulsar_producers()),
@ -1236,3 +1247,19 @@ t_resilience(Config) ->
[] []
), ),
ok. ok.
get_producers_config(ConnectorId, ChannelId) ->
[
#{
state :=
#{
channels :=
#{ChannelId := #{producers := Producers}}
}
}
] =
lists:filter(
fun(#{id := Id}) -> Id =:= ConnectorId end,
emqx_resource_manager:list_all()
),
Producers.