|
|
@@ -11,7 +11,12 @@
|
|
|
query_mode/1,
|
|
|
on_start/2,
|
|
|
on_stop/2,
|
|
|
- on_get_status/2
|
|
|
+ on_get_status/2,
|
|
|
+
|
|
|
+ on_add_channel/4,
|
|
|
+ on_remove_channel/3,
|
|
|
+ on_get_channels/1,
|
|
|
+ on_get_channel_status/3
|
|
|
]).
|
|
|
|
|
|
%% `brod_group_consumer' API
|
|
|
@@ -30,45 +35,57 @@
|
|
|
-include_lib("brod/include/brod.hrl").
|
|
|
-include_lib("emqx_resource/include/emqx_resource.hrl").
|
|
|
|
|
|
--type config() :: #{
|
|
|
+-type connector_config() :: #{
|
|
|
authentication := term(),
|
|
|
bootstrap_hosts := binary(),
|
|
|
- bridge_name := atom(),
|
|
|
- kafka := #{
|
|
|
- max_batch_bytes := emqx_schema:bytesize(),
|
|
|
- max_rejoin_attempts := non_neg_integer(),
|
|
|
- offset_commit_interval_seconds := pos_integer(),
|
|
|
- offset_reset_policy := offset_reset_policy(),
|
|
|
- topic := binary()
|
|
|
- },
|
|
|
- topic_mapping := nonempty_list(
|
|
|
- #{
|
|
|
- kafka_topic := kafka_topic(),
|
|
|
- mqtt_topic := emqx_types:topic(),
|
|
|
- qos := emqx_types:qos(),
|
|
|
- payload_template := string()
|
|
|
- }
|
|
|
- ),
|
|
|
+ connector_name := atom() | binary(),
|
|
|
+ connector_type := atom() | binary(),
|
|
|
+ socket_opts := _,
|
|
|
ssl := _,
|
|
|
any() => term()
|
|
|
}.
|
|
|
+-type source_config() :: #{
|
|
|
+ bridge_name := atom(),
|
|
|
+ hookpoints := [binary()],
|
|
|
+ parameters := source_parameters()
|
|
|
+}.
|
|
|
+-type source_parameters() :: #{
|
|
|
+ key_encoding_mode := encoding_mode(),
|
|
|
+ max_batch_bytes := emqx_schema:bytesize(),
|
|
|
+ max_rejoin_attempts := non_neg_integer(),
|
|
|
+ offset_commit_interval_seconds := pos_integer(),
|
|
|
+ offset_reset_policy := offset_reset_policy(),
|
|
|
+ topic := kafka_topic(),
|
|
|
+ value_encoding_mode := encoding_mode(),
|
|
|
+ topic_mapping => [one_topic_mapping()]
|
|
|
+}.
|
|
|
+-type one_topic_mapping() :: #{
|
|
|
+ kafka_topic => kafka_topic(),
|
|
|
+ mqtt_topic => emqx_types:topic(),
|
|
|
+ qos => emqx_types:qos(),
|
|
|
+ payload_template => string()
|
|
|
+}.
|
|
|
-type subscriber_id() :: emqx_bridge_kafka_consumer_sup:child_id().
|
|
|
-type kafka_topic() :: brod:topic().
|
|
|
-type kafka_message() :: #kafka_message{}.
|
|
|
--type state() :: #{
|
|
|
- kafka_topics := nonempty_list(kafka_topic()),
|
|
|
+-type connector_state() :: #{
|
|
|
+ kafka_client_id := brod:client_id(),
|
|
|
+ installed_sources := #{source_resource_id() => source_state()}
|
|
|
+}.
|
|
|
+-type source_state() :: #{
|
|
|
subscriber_id := subscriber_id(),
|
|
|
- kafka_client_id := brod:client_id()
|
|
|
+ kafka_client_id := brod:client_id(),
|
|
|
+ kafka_topics := [kafka_topic()]
|
|
|
}.
|
|
|
-type offset_reset_policy() :: latest | earliest.
|
|
|
-type encoding_mode() :: none | base64.
|
|
|
-type consumer_init_data() :: #{
|
|
|
- hookpoint := binary(),
|
|
|
+ hookpoints := [binary()],
|
|
|
key_encoding_mode := encoding_mode(),
|
|
|
- resource_id := resource_id(),
|
|
|
+ resource_id := source_resource_id(),
|
|
|
topic_mapping := #{
|
|
|
kafka_topic() := #{
|
|
|
- payload_template := emqx_placeholder:tmpl_token(),
|
|
|
+ payload_template => emqx_placeholder:tmpl_token(),
|
|
|
mqtt_topic_template => emqx_placeholder:tmpl_token(),
|
|
|
qos => emqx_types:qos()
|
|
|
}
|
|
|
@@ -76,13 +93,13 @@
|
|
|
value_encoding_mode := encoding_mode()
|
|
|
}.
|
|
|
-type consumer_state() :: #{
|
|
|
- hookpoint := binary(),
|
|
|
- kafka_topic := binary(),
|
|
|
+ hookpoints := [binary()],
|
|
|
+ kafka_topic := kafka_topic(),
|
|
|
key_encoding_mode := encoding_mode(),
|
|
|
- resource_id := resource_id(),
|
|
|
+ resource_id := source_resource_id(),
|
|
|
topic_mapping := #{
|
|
|
kafka_topic() := #{
|
|
|
- payload_template := emqx_placeholder:tmpl_token(),
|
|
|
+ payload_template => emqx_placeholder:tmpl_token(),
|
|
|
mqtt_topic_template => emqx_placeholder:tmpl_token(),
|
|
|
qos => emqx_types:qos()
|
|
|
}
|
|
|
@@ -90,7 +107,7 @@
|
|
|
value_encoding_mode := encoding_mode()
|
|
|
}.
|
|
|
-type subscriber_init_info() :: #{
|
|
|
- topic => brod:topic(),
|
|
|
+ topic := brod:topic(),
|
|
|
parition => brod:partition(),
|
|
|
group_id => brod:group_id(),
|
|
|
commit_fun => brod_group_subscriber_v2:commit_fun()
|
|
|
@@ -103,7 +120,7 @@
|
|
|
|
|
|
%% Allocatable resources
|
|
|
-define(kafka_client_id, kafka_client_id).
|
|
|
--define(kafka_subscriber_id, kafka_subscriber_id).
|
|
|
+-define(kafka_subscriber_ids, kafka_subscriber_ids).
|
|
|
|
|
|
%%-------------------------------------------------------------------------------------
|
|
|
%% `emqx_resource' API
|
|
|
@@ -116,27 +133,19 @@ callback_mode() ->
|
|
|
query_mode(_Config) ->
|
|
|
no_queries.
|
|
|
|
|
|
--spec on_start(resource_id(), config()) -> {ok, state()}.
|
|
|
-on_start(ResourceId, Config) ->
|
|
|
+-spec on_start(connector_resource_id(), connector_config()) -> {ok, connector_state()}.
|
|
|
+on_start(ConnectorResId, Config) ->
|
|
|
#{
|
|
|
authentication := Auth,
|
|
|
bootstrap_hosts := BootstrapHosts0,
|
|
|
- bridge_type := BridgeType,
|
|
|
- bridge_name := BridgeName,
|
|
|
- hookpoint := _,
|
|
|
- kafka := #{
|
|
|
- max_batch_bytes := _,
|
|
|
- max_rejoin_attempts := _,
|
|
|
- offset_commit_interval_seconds := _,
|
|
|
- offset_reset_policy := _
|
|
|
- },
|
|
|
+ connector_type := ConnectorType,
|
|
|
+ connector_name := ConnectorName,
|
|
|
socket_opts := SocketOpts0,
|
|
|
- ssl := SSL,
|
|
|
- topic_mapping := _
|
|
|
+ ssl := SSL
|
|
|
} = Config,
|
|
|
BootstrapHosts = emqx_bridge_kafka_impl:hosts(BootstrapHosts0),
|
|
|
%% Note: this is distinct per node.
|
|
|
- ClientID = make_client_id(ResourceId, BridgeType, BridgeName),
|
|
|
+ ClientID = make_client_id(ConnectorResId, ConnectorType, ConnectorName),
|
|
|
ClientOpts0 =
|
|
|
case Auth of
|
|
|
none -> [];
|
|
|
@@ -145,34 +154,37 @@ on_start(ResourceId, Config) ->
|
|
|
ClientOpts = add_ssl_opts(ClientOpts0, SSL),
|
|
|
SocketOpts = emqx_bridge_kafka_impl:socket_opts(SocketOpts0),
|
|
|
ClientOpts1 = [{extra_sock_opts, SocketOpts} | ClientOpts],
|
|
|
- ok = emqx_resource:allocate_resource(ResourceId, ?kafka_client_id, ClientID),
|
|
|
+ ok = emqx_resource:allocate_resource(ConnectorResId, ?kafka_client_id, ClientID),
|
|
|
case brod:start_client(BootstrapHosts, ClientID, ClientOpts1) of
|
|
|
ok ->
|
|
|
?tp(
|
|
|
kafka_consumer_client_started,
|
|
|
- #{client_id => ClientID, resource_id => ResourceId}
|
|
|
+ #{client_id => ClientID, resource_id => ConnectorResId}
|
|
|
),
|
|
|
?SLOG(info, #{
|
|
|
msg => "kafka_consumer_client_started",
|
|
|
- resource_id => ResourceId,
|
|
|
+ resource_id => ConnectorResId,
|
|
|
kafka_hosts => BootstrapHosts
|
|
|
});
|
|
|
{error, Reason} ->
|
|
|
?SLOG(error, #{
|
|
|
msg => "failed_to_start_kafka_consumer_client",
|
|
|
- resource_id => ResourceId,
|
|
|
+ resource_id => ConnectorResId,
|
|
|
kafka_hosts => BootstrapHosts,
|
|
|
reason => emqx_utils:redact(Reason)
|
|
|
}),
|
|
|
throw(?CLIENT_DOWN_MESSAGE)
|
|
|
end,
|
|
|
- start_consumer(Config, ResourceId, ClientID).
|
|
|
-
|
|
|
--spec on_stop(resource_id(), state()) -> ok.
|
|
|
-on_stop(ResourceId, _State = undefined) ->
|
|
|
- case emqx_resource:get_allocated_resources(ResourceId) of
|
|
|
- #{?kafka_client_id := ClientID, ?kafka_subscriber_id := SubscriberId} ->
|
|
|
- stop_subscriber(SubscriberId),
|
|
|
+ {ok, #{
|
|
|
+ kafka_client_id => ClientID,
|
|
|
+ installed_sources => #{}
|
|
|
+ }}.
|
|
|
+
|
|
|
+-spec on_stop(resource_id(), connector_state()) -> ok.
|
|
|
+on_stop(ConnectorResId, _State = undefined) ->
|
|
|
+ case emqx_resource:get_allocated_resources(ConnectorResId) of
|
|
|
+ #{?kafka_client_id := ClientID, ?kafka_subscriber_ids := SubscriberIds} ->
|
|
|
+ lists:foreach(fun stop_subscriber/1, SubscriberIds),
|
|
|
stop_client(ClientID),
|
|
|
?tp(kafka_consumer_subcriber_and_client_stopped, #{}),
|
|
|
ok;
|
|
|
@@ -183,29 +195,91 @@ on_stop(ResourceId, _State = undefined) ->
|
|
|
_ ->
|
|
|
ok
|
|
|
end;
|
|
|
-on_stop(_ResourceId, State) ->
|
|
|
+on_stop(ConnectorResId, State) ->
|
|
|
#{
|
|
|
- subscriber_id := SubscriberId,
|
|
|
+ installed_sources := InstalledSources,
|
|
|
kafka_client_id := ClientID
|
|
|
} = State,
|
|
|
- stop_subscriber(SubscriberId),
|
|
|
+ maps:foreach(
|
|
|
+ fun(_SourceResId, #{subscriber_id := SubscriberId}) ->
|
|
|
+ stop_subscriber(SubscriberId)
|
|
|
+ end,
|
|
|
+ InstalledSources
|
|
|
+ ),
|
|
|
stop_client(ClientID),
|
|
|
+ ?tp(kafka_consumer_subcriber_and_client_stopped, #{instance_id => ConnectorResId}),
|
|
|
ok.
|
|
|
|
|
|
--spec on_get_status(resource_id(), state()) -> connected | disconnected.
|
|
|
-on_get_status(_ResourceID, State) ->
|
|
|
+-spec on_get_status(resource_id(), connector_state()) -> connected | disconnected.
|
|
|
+on_get_status(_ResourceID, _State) ->
|
|
|
+ ?status_connected.
|
|
|
+
|
|
|
+-spec on_add_channel(
|
|
|
+ connector_resource_id(),
|
|
|
+ connector_state(),
|
|
|
+ source_resource_id(),
|
|
|
+ source_config()
|
|
|
+) ->
|
|
|
+ {ok, connector_state()}.
|
|
|
+on_add_channel(ConnectorResId, ConnectorState0, SourceResId, SourceConfig) ->
|
|
|
#{
|
|
|
- subscriber_id := SubscriberId,
|
|
|
kafka_client_id := ClientID,
|
|
|
- kafka_topics := KafkaTopics
|
|
|
- } = State,
|
|
|
- case do_get_status(ClientID, KafkaTopics, SubscriberId) of
|
|
|
- {disconnected, Message} ->
|
|
|
- {disconnected, State, Message};
|
|
|
- Res ->
|
|
|
- Res
|
|
|
+ installed_sources := InstalledSources0
|
|
|
+ } = ConnectorState0,
|
|
|
+ case start_consumer(SourceConfig, ConnectorResId, SourceResId, ClientID) of
|
|
|
+ {ok, SourceState} ->
|
|
|
+ InstalledSources = InstalledSources0#{SourceResId => SourceState},
|
|
|
+ ConnectorState = ConnectorState0#{installed_sources := InstalledSources},
|
|
|
+ {ok, ConnectorState};
|
|
|
+ Error = {error, _} ->
|
|
|
+ Error
|
|
|
end.
|
|
|
|
|
|
+-spec on_remove_channel(
|
|
|
+ connector_resource_id(),
|
|
|
+ connector_state(),
|
|
|
+ source_resource_id()
|
|
|
+) ->
|
|
|
+ {ok, connector_state()}.
|
|
|
+on_remove_channel(ConnectorResId, ConnectorState0, SourceResId) ->
|
|
|
+ #{installed_sources := InstalledSources0} = ConnectorState0,
|
|
|
+ case maps:take(SourceResId, InstalledSources0) of
|
|
|
+ {SourceState, InstalledSources} ->
|
|
|
+ #{subscriber_id := SubscriberId} = SourceState,
|
|
|
+ stop_subscriber(SubscriberId),
|
|
|
+ deallocate_subscriber_id(ConnectorResId, SubscriberId),
|
|
|
+ ok;
|
|
|
+ error ->
|
|
|
+ InstalledSources = InstalledSources0
|
|
|
+ end,
|
|
|
+ ConnectorState = ConnectorState0#{installed_sources := InstalledSources},
|
|
|
+ {ok, ConnectorState}.
|
|
|
+
|
|
|
+-spec on_get_channels(connector_resource_id()) ->
|
|
|
+ [{action_resource_id(), source_config()}].
|
|
|
+on_get_channels(ConnectorResId) ->
|
|
|
+ emqx_bridge_v2:get_channels_for_connector(ConnectorResId).
|
|
|
+
|
|
|
+-spec on_get_channel_status(
|
|
|
+ connector_resource_id(),
|
|
|
+ source_resource_id(),
|
|
|
+ connector_state()
|
|
|
+) ->
|
|
|
+ ?status_connected | ?status_disconnected.
|
|
|
+on_get_channel_status(
|
|
|
+ _ConnectorResId,
|
|
|
+ SourceResId,
|
|
|
+ ConnectorState = #{installed_sources := InstalledSources}
|
|
|
+) when is_map_key(SourceResId, InstalledSources) ->
|
|
|
+ #{kafka_client_id := ClientID} = ConnectorState,
|
|
|
+ #{
|
|
|
+ kafka_topics := KafkaTopics,
|
|
|
+ subscriber_id := SubscriberId
|
|
|
+ } = maps:get(SourceResId, InstalledSources),
|
|
|
+ do_get_status(ClientID, KafkaTopics, SubscriberId);
|
|
|
+on_get_channel_status(_ConnectorResId, _SourceResId, _ConnectorState) ->
|
|
|
+ ?status_disconnected.
|
|
|
+
|
|
|
%%-------------------------------------------------------------------------------------
|
|
|
%% `brod_group_subscriber' API
|
|
|
%%-------------------------------------------------------------------------------------
|
|
|
@@ -227,18 +301,13 @@ handle_message(Message, State) ->
|
|
|
|
|
|
do_handle_message(Message, State) ->
|
|
|
#{
|
|
|
- hookpoint := Hookpoint,
|
|
|
+ hookpoints := Hookpoints,
|
|
|
kafka_topic := KafkaTopic,
|
|
|
key_encoding_mode := KeyEncodingMode,
|
|
|
- resource_id := ResourceId,
|
|
|
+ resource_id := SourceResId,
|
|
|
topic_mapping := TopicMapping,
|
|
|
value_encoding_mode := ValueEncodingMode
|
|
|
} = State,
|
|
|
- #{
|
|
|
- mqtt_topic_template := MQTTTopicTemplate,
|
|
|
- qos := MQTTQoS,
|
|
|
- payload_template := PayloadTemplate
|
|
|
- } = maps:get(KafkaTopic, TopicMapping),
|
|
|
FullMessage = #{
|
|
|
headers => maps:from_list(Message#kafka_message.headers),
|
|
|
key => encode(Message#kafka_message.key, KeyEncodingMode),
|
|
|
@@ -248,16 +317,31 @@ do_handle_message(Message, State) ->
|
|
|
ts_type => Message#kafka_message.ts_type,
|
|
|
value => encode(Message#kafka_message.value, ValueEncodingMode)
|
|
|
},
|
|
|
- Payload = render(FullMessage, PayloadTemplate),
|
|
|
- MQTTTopic = render(FullMessage, MQTTTopicTemplate),
|
|
|
- MQTTMessage = emqx_message:make(ResourceId, MQTTQoS, MQTTTopic, Payload),
|
|
|
- _ = emqx_broker:safe_publish(MQTTMessage),
|
|
|
- emqx_hooks:run(Hookpoint, [FullMessage]),
|
|
|
- emqx_resource_metrics:received_inc(ResourceId),
|
|
|
+ LegacyMQTTConfig = maps:get(KafkaTopic, TopicMapping, #{}),
|
|
|
+ legacy_maybe_publish_mqtt_message(LegacyMQTTConfig, SourceResId, FullMessage),
|
|
|
+ lists:foreach(fun(Hookpoint) -> emqx_hooks:run(Hookpoint, [FullMessage]) end, Hookpoints),
|
|
|
+ emqx_resource_metrics:received_inc(SourceResId),
|
|
|
%% note: just `ack' does not commit the offset to the
|
|
|
%% kafka consumer group.
|
|
|
{ok, commit, State}.
|
|
|
|
|
|
+legacy_maybe_publish_mqtt_message(
|
|
|
+ _MQTTConfig = #{
|
|
|
+ payload_template := PayloadTemplate,
|
|
|
+ qos := MQTTQoS,
|
|
|
+ mqtt_topic_template := MQTTTopicTemplate
|
|
|
+ },
|
|
|
+ SourceResId,
|
|
|
+ FullMessage
|
|
|
+) when MQTTTopicTemplate =/= <<>> ->
|
|
|
+ Payload = render(FullMessage, PayloadTemplate),
|
|
|
+ MQTTTopic = render(FullMessage, MQTTTopicTemplate),
|
|
|
+ MQTTMessage = emqx_message:make(SourceResId, MQTTQoS, MQTTTopic, Payload),
|
|
|
+ _ = emqx_broker:safe_publish(MQTTMessage),
|
|
|
+ ok;
|
|
|
+legacy_maybe_publish_mqtt_message(_MQTTConfig, _SourceResId, _FullMessage) ->
|
|
|
+ ok.
|
|
|
+
|
|
|
%%-------------------------------------------------------------------------------------
|
|
|
%% Helper fns
|
|
|
%%-------------------------------------------------------------------------------------
|
|
|
@@ -292,28 +376,33 @@ ensure_consumer_supervisor_started() ->
|
|
|
ok
|
|
|
end.
|
|
|
|
|
|
--spec start_consumer(config(), resource_id(), brod:client_id()) -> {ok, state()}.
|
|
|
-start_consumer(Config, ResourceId, ClientID) ->
|
|
|
+-spec start_consumer(
|
|
|
+ source_config(),
|
|
|
+ connector_resource_id(),
|
|
|
+ source_resource_id(),
|
|
|
+ brod:client_id()
|
|
|
+) ->
|
|
|
+ {ok, source_state()} | {error, term()}.
|
|
|
+start_consumer(Config, ConnectorResId, SourceResId, ClientID) ->
|
|
|
#{
|
|
|
- bootstrap_hosts := BootstrapHosts0,
|
|
|
bridge_name := BridgeName,
|
|
|
- hookpoint := Hookpoint,
|
|
|
- kafka := #{
|
|
|
+ hookpoints := Hookpoints,
|
|
|
+ parameters := #{
|
|
|
+ key_encoding_mode := KeyEncodingMode,
|
|
|
max_batch_bytes := MaxBatchBytes,
|
|
|
max_rejoin_attempts := MaxRejoinAttempts,
|
|
|
offset_commit_interval_seconds := OffsetCommitInterval,
|
|
|
- offset_reset_policy := OffsetResetPolicy0
|
|
|
- },
|
|
|
- key_encoding_mode := KeyEncodingMode,
|
|
|
- topic_mapping := TopicMapping0,
|
|
|
- value_encoding_mode := ValueEncodingMode
|
|
|
+ offset_reset_policy := OffsetResetPolicy0,
|
|
|
+ topic := _Topic,
|
|
|
+ value_encoding_mode := ValueEncodingMode
|
|
|
+ } = Params0
|
|
|
} = Config,
|
|
|
ok = ensure_consumer_supervisor_started(),
|
|
|
- TopicMapping = convert_topic_mapping(TopicMapping0),
|
|
|
+ TopicMapping = ensure_topic_mapping(Params0),
|
|
|
InitialState = #{
|
|
|
key_encoding_mode => KeyEncodingMode,
|
|
|
- hookpoint => Hookpoint,
|
|
|
- resource_id => ResourceId,
|
|
|
+ hookpoints => Hookpoints,
|
|
|
+ resource_id => SourceResId,
|
|
|
topic_mapping => TopicMapping,
|
|
|
value_encoding_mode => ValueEncodingMode
|
|
|
},
|
|
|
@@ -355,30 +444,38 @@ start_consumer(Config, ResourceId, ClientID) ->
|
|
|
%% automatically, so we should not spawn duplicate workers.
|
|
|
SubscriberId = make_subscriber_id(BridgeName),
|
|
|
?tp(kafka_consumer_about_to_start_subscriber, #{}),
|
|
|
- ok = emqx_resource:allocate_resource(ResourceId, ?kafka_subscriber_id, SubscriberId),
|
|
|
+ ok = allocate_subscriber_id(ConnectorResId, SubscriberId),
|
|
|
?tp(kafka_consumer_subscriber_allocated, #{}),
|
|
|
case emqx_bridge_kafka_consumer_sup:start_child(SubscriberId, GroupSubscriberConfig) of
|
|
|
{ok, _ConsumerPid} ->
|
|
|
?tp(
|
|
|
kafka_consumer_subscriber_started,
|
|
|
- #{resource_id => ResourceId, subscriber_id => SubscriberId}
|
|
|
+ #{resource_id => SourceResId, subscriber_id => SubscriberId}
|
|
|
),
|
|
|
{ok, #{
|
|
|
subscriber_id => SubscriberId,
|
|
|
kafka_client_id => ClientID,
|
|
|
kafka_topics => KafkaTopics
|
|
|
}};
|
|
|
- {error, Reason2} ->
|
|
|
+ {error, Reason} ->
|
|
|
?SLOG(error, #{
|
|
|
msg => "failed_to_start_kafka_consumer",
|
|
|
- resource_id => ResourceId,
|
|
|
- kafka_hosts => emqx_bridge_kafka_impl:hosts(BootstrapHosts0),
|
|
|
- reason => emqx_utils:redact(Reason2)
|
|
|
+ resource_id => SourceResId,
|
|
|
+ reason => emqx_utils:redact(Reason)
|
|
|
}),
|
|
|
- stop_client(ClientID),
|
|
|
- throw(failed_to_start_kafka_consumer)
|
|
|
+ {error, Reason}
|
|
|
end.
|
|
|
|
|
|
+%% This is to ensure backwards compatibility with the deprectated topic mapping.
|
|
|
+-spec ensure_topic_mapping(source_parameters()) -> #{kafka_topic() := map()}.
|
|
|
+ensure_topic_mapping(#{topic_mapping := [_ | _] = TM}) ->
|
|
|
+ %% There is an existing topic mapping: legacy config. We use it and ignore the single
|
|
|
+ %% pubsub topic so that the bridge keeps working as before.
|
|
|
+ convert_topic_mapping(TM);
|
|
|
+ensure_topic_mapping(#{topic := KafkaTopic}) ->
|
|
|
+ %% No topic mapping: generate one without MQTT templates.
|
|
|
+ #{KafkaTopic => #{}}.
|
|
|
+
|
|
|
-spec stop_subscriber(emqx_bridge_kafka_consumer_sup:child_id()) -> ok.
|
|
|
stop_subscriber(SubscriberId) ->
|
|
|
_ = log_when_error(
|
|
|
@@ -415,36 +512,38 @@ do_get_status(ClientID, [KafkaTopic | RestTopics], SubscriberId) ->
|
|
|
case brod:get_partitions_count(ClientID, KafkaTopic) of
|
|
|
{ok, NPartitions} ->
|
|
|
case do_get_topic_status(ClientID, KafkaTopic, SubscriberId, NPartitions) of
|
|
|
- connected -> do_get_status(ClientID, RestTopics, SubscriberId);
|
|
|
- disconnected -> disconnected
|
|
|
+ ?status_connected ->
|
|
|
+ do_get_status(ClientID, RestTopics, SubscriberId);
|
|
|
+ ?status_disconnected ->
|
|
|
+ ?status_disconnected
|
|
|
end;
|
|
|
{error, {client_down, Context}} ->
|
|
|
case infer_client_error(Context) of
|
|
|
auth_error ->
|
|
|
Message = "Authentication error. " ++ ?CLIENT_DOWN_MESSAGE,
|
|
|
- {disconnected, Message};
|
|
|
+ {?status_disconnected, Message};
|
|
|
{auth_error, Message0} ->
|
|
|
Message = binary_to_list(Message0) ++ "; " ++ ?CLIENT_DOWN_MESSAGE,
|
|
|
- {disconnected, Message};
|
|
|
+ {?status_disconnected, Message};
|
|
|
connection_refused ->
|
|
|
Message = "Connection refused. " ++ ?CLIENT_DOWN_MESSAGE,
|
|
|
- {disconnected, Message};
|
|
|
+ {?status_disconnected, Message};
|
|
|
_ ->
|
|
|
- {disconnected, ?CLIENT_DOWN_MESSAGE}
|
|
|
+ {?status_disconnected, ?CLIENT_DOWN_MESSAGE}
|
|
|
end;
|
|
|
{error, leader_not_available} ->
|
|
|
Message =
|
|
|
"Leader connection not available. Please check the Kafka topic used,"
|
|
|
" the connection parameters and Kafka cluster health",
|
|
|
- {disconnected, Message};
|
|
|
+ {?status_disconnected, Message};
|
|
|
_ ->
|
|
|
- disconnected
|
|
|
+ ?status_disconnected
|
|
|
end;
|
|
|
do_get_status(_ClientID, _KafkaTopics = [], _SubscriberId) ->
|
|
|
- connected.
|
|
|
+ ?status_connected.
|
|
|
|
|
|
-spec do_get_topic_status(brod:client_id(), binary(), subscriber_id(), pos_integer()) ->
|
|
|
- connected | disconnected.
|
|
|
+ ?status_connected | ?status_disconnected.
|
|
|
do_get_topic_status(ClientID, KafkaTopic, SubscriberId, NPartitions) ->
|
|
|
Results =
|
|
|
lists:map(
|
|
|
@@ -467,9 +566,9 @@ do_get_topic_status(ClientID, KafkaTopic, SubscriberId, NPartitions) ->
|
|
|
WorkersAlive = are_subscriber_workers_alive(SubscriberId),
|
|
|
case AllLeadersOk andalso WorkersAlive of
|
|
|
true ->
|
|
|
- connected;
|
|
|
+ ?status_connected;
|
|
|
false ->
|
|
|
- disconnected
|
|
|
+ ?status_disconnected
|
|
|
end.
|
|
|
|
|
|
are_subscriber_workers_alive(SubscriberId) ->
|
|
|
@@ -507,19 +606,19 @@ consumer_group_id(BridgeName0) ->
|
|
|
BridgeName = to_bin(BridgeName0),
|
|
|
<<"emqx-kafka-consumer-", BridgeName/binary>>.
|
|
|
|
|
|
--spec is_dry_run(resource_id()) -> boolean().
|
|
|
-is_dry_run(ResourceId) ->
|
|
|
- TestIdStart = string:find(ResourceId, ?TEST_ID_PREFIX),
|
|
|
+-spec is_dry_run(connector_resource_id()) -> boolean().
|
|
|
+is_dry_run(ConnectorResId) ->
|
|
|
+ TestIdStart = string:find(ConnectorResId, ?TEST_ID_PREFIX),
|
|
|
case TestIdStart of
|
|
|
nomatch ->
|
|
|
false;
|
|
|
_ ->
|
|
|
- string:equal(TestIdStart, ResourceId)
|
|
|
+ string:equal(TestIdStart, ConnectorResId)
|
|
|
end.
|
|
|
|
|
|
--spec make_client_id(resource_id(), binary(), atom() | binary()) -> atom().
|
|
|
-make_client_id(ResourceId, BridgeType, BridgeName) ->
|
|
|
- case is_dry_run(ResourceId) of
|
|
|
+-spec make_client_id(connector_resource_id(), binary(), atom() | binary()) -> atom().
|
|
|
+make_client_id(ConnectorResId, BridgeType, BridgeName) ->
|
|
|
+ case is_dry_run(ConnectorResId) of
|
|
|
false ->
|
|
|
ClientID0 = emqx_bridge_kafka_impl:make_client_id(BridgeType, BridgeName),
|
|
|
binary_to_atom(ClientID0);
|
|
|
@@ -583,3 +682,19 @@ infer_client_error(Error) ->
|
|
|
_ ->
|
|
|
undefined
|
|
|
end.
|
|
|
+
|
|
|
+allocate_subscriber_id(ConnectorResId, SubscriberId) ->
|
|
|
+ AllocatedResources = emqx_resource:get_allocated_resources(ConnectorResId),
|
|
|
+ AllocatedSubscriberIds0 = maps:get(?kafka_subscriber_ids, AllocatedResources, []),
|
|
|
+ AllocatedSubscriberIds = lists:usort([SubscriberId | AllocatedSubscriberIds0]),
|
|
|
+ ok = emqx_resource:allocate_resource(
|
|
|
+ ConnectorResId, ?kafka_subscriber_ids, AllocatedSubscriberIds
|
|
|
+ ).
|
|
|
+
|
|
|
+deallocate_subscriber_id(ConnectorResId, SubscriberId) ->
|
|
|
+ AllocatedResources = emqx_resource:get_allocated_resources(ConnectorResId),
|
|
|
+ AllocatedSubscriberIds0 = maps:get(?kafka_subscriber_ids, AllocatedResources, []),
|
|
|
+ AllocatedSubscriberIds = AllocatedSubscriberIds0 -- [SubscriberId],
|
|
|
+ ok = emqx_resource:allocate_resource(
|
|
|
+ ConnectorResId, ?kafka_subscriber_ids, AllocatedSubscriberIds
|
|
|
+ ).
|