feat(bridges): start one mqtt bridge worker for each in/out channel
This commit is contained in:
parent
4f82debbe7
commit
eb8822ce41
|
@ -5,7 +5,8 @@
|
||||||
bridges.mqtt.my_mqtt_bridge {
|
bridges.mqtt.my_mqtt_bridge {
|
||||||
server = "127.0.0.1:1883"
|
server = "127.0.0.1:1883"
|
||||||
proto_ver = "v4"
|
proto_ver = "v4"
|
||||||
clientid = "client1"
|
## the clientid will be the concatenation of `clientid_prefix` and ids in `in` and `out`.
|
||||||
|
clientid_prefix = "emqx_bridge_"
|
||||||
username = "username1"
|
username = "username1"
|
||||||
password = ""
|
password = ""
|
||||||
clean_start = true
|
clean_start = true
|
||||||
|
@ -26,20 +27,24 @@ bridges.mqtt.my_mqtt_bridge {
|
||||||
certfile = "{{ platform_etc_dir }}/certs/client-cert.pem"
|
certfile = "{{ platform_etc_dir }}/certs/client-cert.pem"
|
||||||
cacertfile = "{{ platform_etc_dir }}/certs/cacert.pem"
|
cacertfile = "{{ platform_etc_dir }}/certs/cacert.pem"
|
||||||
}
|
}
|
||||||
|
## we will create one MQTT connection for each element of the `in`
|
||||||
in: [{
|
in: [{
|
||||||
subscribe_remote_topic = "msg/#"
|
id = "pull_msgs_from_aws"
|
||||||
|
subscribe_remote_topic = "aws/#"
|
||||||
subscribe_qos = 1
|
subscribe_qos = 1
|
||||||
publish_local_topic = "from_aws/${topic}"
|
local_topic = "from_aws/${topic}"
|
||||||
publish_payload = "${payload}"
|
payload = "${payload}"
|
||||||
publish_qos = "${qos}"
|
qos = "${qos}"
|
||||||
publish_retain = "${retain}"
|
retain = "${retain}"
|
||||||
}]
|
}]
|
||||||
|
## we will create one MQTT connection for each element of the `out`
|
||||||
out: [{
|
out: [{
|
||||||
subscribe_local_topic = "msg/#"
|
id = "push_msgs_to_aws"
|
||||||
publish_remote_topic = "from_emqx/${topic}"
|
subscribe_local_topic = "emqx/#"
|
||||||
publish_payload = "${payload}"
|
remote_topic = "from_emqx/${topic}"
|
||||||
publish_qos = 1
|
payload = "${payload}"
|
||||||
publish_retain = false
|
qos = 1
|
||||||
|
retain = false
|
||||||
}]
|
}]
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -50,8 +50,9 @@ start(Config) ->
|
||||||
Parent = self(),
|
Parent = self(),
|
||||||
{Host, Port} = maps:get(server, Config),
|
{Host, Port} = maps:get(server, Config),
|
||||||
Mountpoint = maps:get(receive_mountpoint, Config, undefined),
|
Mountpoint = maps:get(receive_mountpoint, Config, undefined),
|
||||||
Subscriptions = maps:get(subscriptions, Config, []),
|
Subscriptions = maps:get(subscriptions, Config),
|
||||||
Handlers = make_hdlr(Parent, Mountpoint),
|
Vars = emqx_bridge_msg:make_pub_vars(Mountpoint, Subscriptions),
|
||||||
|
Handlers = make_hdlr(Parent, Vars),
|
||||||
Config1 = Config#{
|
Config1 = Config#{
|
||||||
msg_handler => Handlers,
|
msg_handler => Handlers,
|
||||||
host => Host,
|
host => Host,
|
||||||
|
@ -59,7 +60,7 @@ start(Config) ->
|
||||||
force_ping => true,
|
force_ping => true,
|
||||||
proto_ver => maps:get(proto_ver, Config, v4)
|
proto_ver => maps:get(proto_ver, Config, v4)
|
||||||
},
|
},
|
||||||
case emqtt:start_link(without_config(Config1)) of
|
case emqtt:start_link(process_config(Config1)) of
|
||||||
{ok, Pid} ->
|
{ok, Pid} ->
|
||||||
case emqtt:connect(Pid) of
|
case emqtt:connect(Pid) of
|
||||||
{ok, _} ->
|
{ok, _} ->
|
||||||
|
@ -156,25 +157,35 @@ handle_puback(#{packet_id := PktId, reason_code := RC}, Parent)
|
||||||
handle_puback(#{packet_id := PktId, reason_code := RC}, _Parent) ->
|
handle_puback(#{packet_id := PktId, reason_code := RC}, _Parent) ->
|
||||||
?LOG(warning, "Publish ~p to remote node falied, reason_code: ~p", [PktId, RC]).
|
?LOG(warning, "Publish ~p to remote node falied, reason_code: ~p", [PktId, RC]).
|
||||||
|
|
||||||
handle_publish(Msg, Mountpoint) ->
|
handle_publish(Msg, undefined) ->
|
||||||
emqx_broker:publish(emqx_bridge_msg:to_broker_msg(Msg, Mountpoint)).
|
?LOG(error, "Cannot publish to local broker as 'bridge.mqtt.<name>.in' not configured, msg: ~p", [Msg]);
|
||||||
|
handle_publish(Msg, Vars) ->
|
||||||
|
?LOG(debug, "Publish to local broker, msg: ~p, vars: ~p", [Msg, Vars]),
|
||||||
|
emqx_broker:publish(emqx_bridge_msg:to_broker_msg(Msg, Vars)).
|
||||||
|
|
||||||
handle_disconnected(Reason, Parent) ->
|
handle_disconnected(Reason, Parent) ->
|
||||||
Parent ! {disconnected, self(), Reason}.
|
Parent ! {disconnected, self(), Reason}.
|
||||||
|
|
||||||
make_hdlr(Parent, Mountpoint) ->
|
make_hdlr(Parent, Vars) ->
|
||||||
#{puback => {fun ?MODULE:handle_puback/2, [Parent]},
|
#{puback => {fun ?MODULE:handle_puback/2, [Parent]},
|
||||||
publish => {fun ?MODULE:handle_publish/2, [Mountpoint]},
|
publish => {fun ?MODULE:handle_publish/2, [Vars]},
|
||||||
disconnected => {fun ?MODULE:handle_disconnected/2, [Parent]}
|
disconnected => {fun ?MODULE:handle_disconnected/2, [Parent]}
|
||||||
}.
|
}.
|
||||||
|
|
||||||
subscribe_remote_topics(ClientPid, Subscriptions) ->
|
subscribe_remote_topics(_ClientPid, undefined) -> ok;
|
||||||
lists:foreach(fun(#{subscribe_remote_topic := FromTopic, subscribe_qos := QoS}) ->
|
subscribe_remote_topics(ClientPid, #{subscribe_remote_topic := FromTopic, subscribe_qos := QoS}) ->
|
||||||
case emqtt:subscribe(ClientPid, FromTopic, QoS) of
|
case emqtt:subscribe(ClientPid, FromTopic, QoS) of
|
||||||
{ok, _, _} -> ok;
|
{ok, _, _} -> ok;
|
||||||
Error -> throw(Error)
|
Error -> throw(Error)
|
||||||
end
|
end.
|
||||||
end, Subscriptions).
|
|
||||||
|
|
||||||
without_config(Config) ->
|
process_config(#{name := Name, clientid_prefix := Prefix} = Config) ->
|
||||||
maps:without([conn_type, address, receive_mountpoint, subscriptions], Config).
|
Conf0 = maps:without([conn_type, address, receive_mountpoint, subscriptions, name], Config),
|
||||||
|
Conf0#{clientid => iolist_to_binary([str(Prefix), str(Name)])}.
|
||||||
|
|
||||||
|
str(A) when is_atom(A) ->
|
||||||
|
atom_to_list(A);
|
||||||
|
str(B) when is_binary(B) ->
|
||||||
|
binary_to_list(B);
|
||||||
|
str(S) when is_list(S) ->
|
||||||
|
S.
|
|
@ -65,6 +65,5 @@ drop_bridge(Name) ->
|
||||||
ok ->
|
ok ->
|
||||||
supervisor:delete_child(?MODULE, Name);
|
supervisor:delete_child(?MODULE, Name);
|
||||||
{error, Error} ->
|
{error, Error} ->
|
||||||
?LOG(error, "Delete bridge failed, error : ~p", [Error]),
|
|
||||||
{error, Error}
|
{error, Error}
|
||||||
end.
|
end.
|
||||||
|
|
|
@ -18,9 +18,8 @@
|
||||||
|
|
||||||
-export([ to_binary/1
|
-export([ to_binary/1
|
||||||
, from_binary/1
|
, from_binary/1
|
||||||
, to_export/3
|
, make_pub_vars/2
|
||||||
, to_broker_msgs/1
|
, to_remote_msg/3
|
||||||
, to_broker_msg/1
|
|
||||||
, to_broker_msg/2
|
, to_broker_msg/2
|
||||||
, estimate_size/1
|
, estimate_size/1
|
||||||
]).
|
]).
|
||||||
|
@ -44,6 +43,12 @@
|
||||||
payload := binary()
|
payload := binary()
|
||||||
}.
|
}.
|
||||||
|
|
||||||
|
make_pub_vars(_, undefined) -> undefined;
|
||||||
|
make_pub_vars(Mountpoint, #{payload := _, qos := _, retain := _, remote_topic := Topic} = Conf) ->
|
||||||
|
Conf#{topic => Topic, mountpoint => Mountpoint};
|
||||||
|
make_pub_vars(Mountpoint, #{payload := _, qos := _, retain := _, local_topic := Topic} = Conf) ->
|
||||||
|
Conf#{topic => Topic, mountpoint => Mountpoint}.
|
||||||
|
|
||||||
%% @doc Make export format:
|
%% @doc Make export format:
|
||||||
%% 1. Mount topic to a prefix
|
%% 1. Mount topic to a prefix
|
||||||
%% 2. Fix QoS to 1
|
%% 2. Fix QoS to 1
|
||||||
|
@ -51,37 +56,52 @@
|
||||||
%% Shame that we have to know the callback module here
|
%% Shame that we have to know the callback module here
|
||||||
%% would be great if we can get rid of #mqtt_msg{} record
|
%% would be great if we can get rid of #mqtt_msg{} record
|
||||||
%% and use #message{} in all places.
|
%% and use #message{} in all places.
|
||||||
-spec to_export(emqx_bridge_rpc | emqx_bridge_worker, variables(), msg())
|
-spec to_remote_msg(emqx_bridge_rpc | emqx_bridge_worker, msg(), variables())
|
||||||
-> exp_msg().
|
-> exp_msg().
|
||||||
to_export(emqx_bridge_mqtt, Vars, #message{flags = Flags0} = Msg) ->
|
to_remote_msg(emqx_bridge_mqtt, #message{flags = Flags0} = Msg, Vars) ->
|
||||||
Retain0 = maps:get(retain, Flags0, false),
|
Retain0 = maps:get(retain, Flags0, false),
|
||||||
MapMsg = maps:put(retain, Retain0, emqx_message:to_map(Msg)),
|
MapMsg = maps:put(retain, Retain0, emqx_message:to_map(Msg)),
|
||||||
to_export(emqx_bridge_mqtt, Vars, MapMsg);
|
to_remote_msg(emqx_bridge_mqtt, MapMsg, Vars);
|
||||||
to_export(emqx_bridge_mqtt, #{topic := TopicToken, payload := PayloadToken,
|
to_remote_msg(emqx_bridge_mqtt, MapMsg, #{topic := TopicToken, payload := PayloadToken,
|
||||||
qos := QoSToken, retain := RetainToken, mountpoint := Mountpoint},
|
qos := QoSToken, retain := RetainToken, mountpoint := Mountpoint}) when is_map(MapMsg) ->
|
||||||
MapMsg) when is_map(MapMsg) ->
|
|
||||||
Topic = replace_vars_in_str(TopicToken, MapMsg),
|
Topic = replace_vars_in_str(TopicToken, MapMsg),
|
||||||
Payload = replace_vars_in_str(PayloadToken, MapMsg),
|
Payload = replace_vars_in_str(PayloadToken, MapMsg),
|
||||||
QoS = replace_vars(QoSToken, MapMsg),
|
QoS = replace_simple_var(QoSToken, MapMsg),
|
||||||
Retain = replace_vars(RetainToken, MapMsg),
|
Retain = replace_simple_var(RetainToken, MapMsg),
|
||||||
#mqtt_msg{qos = QoS,
|
#mqtt_msg{qos = QoS,
|
||||||
retain = Retain,
|
retain = Retain,
|
||||||
topic = topic(Mountpoint, Topic),
|
topic = topic(Mountpoint, Topic),
|
||||||
props = #{},
|
props = #{},
|
||||||
payload = Payload};
|
payload = Payload};
|
||||||
to_export(_Module, #{mountpoint := Mountpoint},
|
to_remote_msg(_Module, #message{topic = Topic} = Msg, #{mountpoint := Mountpoint}) ->
|
||||||
#message{topic = Topic} = Msg) ->
|
|
||||||
Msg#message{topic = topic(Mountpoint, Topic)}.
|
Msg#message{topic = topic(Mountpoint, Topic)}.
|
||||||
|
|
||||||
|
%% published from remote node over a MQTT connection
|
||||||
|
to_broker_msg(#{dup := Dup, properties := Props} = MapMsg,
|
||||||
|
#{topic := TopicToken, payload := PayloadToken,
|
||||||
|
qos := QoSToken, retain := RetainToken, mountpoint := Mountpoint}) ->
|
||||||
|
Topic = replace_vars_in_str(TopicToken, MapMsg),
|
||||||
|
Payload = replace_vars_in_str(PayloadToken, MapMsg),
|
||||||
|
QoS = replace_simple_var(QoSToken, MapMsg),
|
||||||
|
Retain = replace_simple_var(RetainToken, MapMsg),
|
||||||
|
set_headers(Props,
|
||||||
|
emqx_message:set_flags(#{dup => Dup, retain => Retain},
|
||||||
|
emqx_message:make(bridge, QoS, topic(Mountpoint, Topic), Payload))).
|
||||||
|
|
||||||
|
%% Replace a string contains vars to another string in which the placeholders are replace by the
|
||||||
|
%% corresponding values. For example, given "a: ${var}", if the var=1, the result string will be:
|
||||||
|
%% "a: 1".
|
||||||
replace_vars_in_str(Tokens, Data) when is_list(Tokens) ->
|
replace_vars_in_str(Tokens, Data) when is_list(Tokens) ->
|
||||||
emqx_plugin_libs_rule:proc_tmpl(Tokens, Data, #{return => full_binary});
|
emqx_plugin_libs_rule:proc_tmpl(Tokens, Data, #{return => full_binary});
|
||||||
replace_vars_in_str(Val, _Data) ->
|
replace_vars_in_str(Val, _Data) ->
|
||||||
Val.
|
Val.
|
||||||
|
|
||||||
replace_vars(Tokens, Data) when is_list(Tokens) ->
|
%% Replace a simple var to its value. For example, given "${var}", if the var=1, then the result
|
||||||
|
%% value will be an integer 1.
|
||||||
|
replace_simple_var(Tokens, Data) when is_list(Tokens) ->
|
||||||
[Var] = emqx_plugin_libs_rule:proc_tmpl(Tokens, Data, #{return => rawlist}),
|
[Var] = emqx_plugin_libs_rule:proc_tmpl(Tokens, Data, #{return => rawlist}),
|
||||||
Var;
|
Var;
|
||||||
replace_vars(Val, _Data) ->
|
replace_simple_var(Val, _Data) ->
|
||||||
Val.
|
Val.
|
||||||
|
|
||||||
%% @doc Make `binary()' in order to make iodata to be persisted on disk.
|
%% @doc Make `binary()' in order to make iodata to be persisted on disk.
|
||||||
|
@ -98,22 +118,6 @@ from_binary(Bin) -> binary_to_term(Bin).
|
||||||
estimate_size(#message{topic = Topic, payload = Payload}) ->
|
estimate_size(#message{topic = Topic, payload = Payload}) ->
|
||||||
size(Topic) + size(Payload).
|
size(Topic) + size(Payload).
|
||||||
|
|
||||||
%% @doc By message/batch receiver, transform received batch into
|
|
||||||
%% messages to deliver to local brokers.
|
|
||||||
to_broker_msgs(Batch) -> lists:map(fun to_broker_msg/1, Batch).
|
|
||||||
|
|
||||||
to_broker_msg(#message{} = Msg) ->
|
|
||||||
%% internal format from another EMQX node via rpc
|
|
||||||
Msg;
|
|
||||||
to_broker_msg(Msg) ->
|
|
||||||
to_broker_msg(Msg, undefined).
|
|
||||||
to_broker_msg(#{qos := QoS, dup := Dup, retain := Retain, topic := Topic,
|
|
||||||
properties := Props, payload := Payload}, Mountpoint) ->
|
|
||||||
%% published from remote node over a MQTT connection
|
|
||||||
set_headers(Props,
|
|
||||||
emqx_message:set_flags(#{dup => Dup, retain => Retain},
|
|
||||||
emqx_message:make(bridge, QoS, topic(Mountpoint, Topic), Payload))).
|
|
||||||
|
|
||||||
set_headers(undefined, Msg) ->
|
set_headers(undefined, Msg) ->
|
||||||
Msg;
|
Msg;
|
||||||
set_headers(Val, Msg) ->
|
set_headers(Val, Msg) ->
|
||||||
|
|
|
@ -90,13 +90,9 @@
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export([ get_forwards/1
|
-export([ get_forwards/1
|
||||||
, ensure_forward_present/2
|
|
||||||
, ensure_forward_absent/2
|
|
||||||
]).
|
]).
|
||||||
|
|
||||||
-export([ get_subscriptions/1
|
-export([ get_subscriptions/1
|
||||||
, ensure_subscription_present/3
|
|
||||||
, ensure_subscription_absent/2
|
|
||||||
]).
|
]).
|
||||||
|
|
||||||
%% Internal
|
%% Internal
|
||||||
|
@ -183,41 +179,19 @@ get_forwards(Name) -> gen_statem:call(name(Name), get_forwards, timer:seconds(10
|
||||||
-spec get_subscriptions(id()) -> [{emqx_topic:topic(), qos()}].
|
-spec get_subscriptions(id()) -> [{emqx_topic:topic(), qos()}].
|
||||||
get_subscriptions(Name) -> gen_statem:call(name(Name), get_subscriptions).
|
get_subscriptions(Name) -> gen_statem:call(name(Name), get_subscriptions).
|
||||||
|
|
||||||
%% @doc Add a new forward (local topic subscription).
|
|
||||||
-spec ensure_forward_present(id(), topic()) -> ok.
|
|
||||||
ensure_forward_present(Name, Topic) ->
|
|
||||||
gen_statem:call(name(Name), {ensure_forward_present, topic(Topic)}).
|
|
||||||
|
|
||||||
%% @doc Ensure a forward topic is deleted.
|
|
||||||
-spec ensure_forward_absent(id(), topic()) -> ok.
|
|
||||||
ensure_forward_absent(Name, Topic) ->
|
|
||||||
gen_statem:call(name(Name), {ensure_forward_absent, topic(Topic)}).
|
|
||||||
|
|
||||||
%% @doc Ensure subscribed to remote topic.
|
|
||||||
%% NOTE: only applicable when connection module is emqx_bridge_mqtt
|
|
||||||
%% return `{error, no_remote_subscription_support}' otherwise.
|
|
||||||
-spec ensure_subscription_present(id(), topic(), qos()) -> ok | {error, any()}.
|
|
||||||
ensure_subscription_present(Name, Topic, QoS) ->
|
|
||||||
gen_statem:call(name(Name), {ensure_subscription_present, topic(Topic), QoS}).
|
|
||||||
|
|
||||||
%% @doc Ensure unsubscribed from remote topic.
|
|
||||||
%% NOTE: only applicable when connection module is emqx_bridge_mqtt
|
|
||||||
-spec ensure_subscription_absent(id(), topic()) -> ok.
|
|
||||||
ensure_subscription_absent(Name, Topic) ->
|
|
||||||
gen_statem:call(name(Name), {ensure_subscription_absent, topic(Topic)}).
|
|
||||||
|
|
||||||
callback_mode() -> [state_functions].
|
callback_mode() -> [state_functions].
|
||||||
|
|
||||||
%% @doc Config should be a map().
|
%% @doc Config should be a map().
|
||||||
init(Opts) ->
|
init(#{name := Name} = ConnectOpts) ->
|
||||||
|
?LOG(info, "starting bridge worker for ~p", [Name]),
|
||||||
erlang:process_flag(trap_exit, true),
|
erlang:process_flag(trap_exit, true),
|
||||||
ConnectOpts = maps:get(config, Opts),
|
|
||||||
ConnectModule = conn_type(maps:get(conn_type, ConnectOpts)),
|
ConnectModule = conn_type(maps:get(conn_type, ConnectOpts)),
|
||||||
Forwards = maps:get(forwards, Opts, []),
|
Forwards = maps:get(forwards, ConnectOpts, #{}),
|
||||||
Queue = open_replayq(maps:get(replayq, Opts, #{})),
|
Queue = open_replayq(Name, maps:get(replayq, ConnectOpts, #{})),
|
||||||
State = init_state(Opts),
|
State = init_state(ConnectOpts),
|
||||||
self() ! idle,
|
self() ! idle,
|
||||||
{ok, idle, State#{connect_module => ConnectModule,
|
{ok, idle, State#{
|
||||||
|
connect_module => ConnectModule,
|
||||||
connect_opts => pre_process_opts(ConnectOpts),
|
connect_opts => pre_process_opts(ConnectOpts),
|
||||||
forwards => Forwards,
|
forwards => Forwards,
|
||||||
replayq => Queue
|
replayq => Queue
|
||||||
|
@ -241,27 +215,29 @@ init_state(Opts) ->
|
||||||
if_record_metrics => IfRecordMetrics,
|
if_record_metrics => IfRecordMetrics,
|
||||||
name => Name}.
|
name => Name}.
|
||||||
|
|
||||||
open_replayq(QCfg) ->
|
open_replayq(Name, QCfg) ->
|
||||||
Dir = maps:get(dir, QCfg, undefined),
|
Dir = maps:get(dir, QCfg, undefined),
|
||||||
SegBytes = maps:get(seg_bytes, QCfg, ?DEFAULT_SEG_BYTES),
|
SegBytes = maps:get(seg_bytes, QCfg, ?DEFAULT_SEG_BYTES),
|
||||||
MaxTotalSize = maps:get(max_total_size, QCfg, ?DEFAULT_MAX_TOTAL_SIZE),
|
MaxTotalSize = maps:get(max_total_size, QCfg, ?DEFAULT_MAX_TOTAL_SIZE),
|
||||||
QueueConfig = case Dir =:= undefined orelse Dir =:= "" of
|
QueueConfig = case Dir =:= undefined orelse Dir =:= "" of
|
||||||
true -> #{mem_only => true};
|
true -> #{mem_only => true};
|
||||||
false -> #{dir => Dir, seg_bytes => SegBytes, max_total_size => MaxTotalSize}
|
false -> #{dir => filename:join([Dir, node(), Name]),
|
||||||
|
seg_bytes => SegBytes, max_total_size => MaxTotalSize}
|
||||||
end,
|
end,
|
||||||
replayq:open(QueueConfig#{sizer => fun emqx_bridge_msg:estimate_size/1,
|
replayq:open(QueueConfig#{sizer => fun emqx_bridge_msg:estimate_size/1,
|
||||||
marshaller => fun ?MODULE:msg_marshaller/1}).
|
marshaller => fun ?MODULE:msg_marshaller/1}).
|
||||||
|
|
||||||
pre_process_opts(#{subscriptions := InConf, forwards := OutConf} = ConnectOpts) ->
|
pre_process_opts(#{subscriptions := InConf, forwards := OutConf} = ConnectOpts) ->
|
||||||
ConnectOpts#{subscriptions => [pre_process_in_out(In) || In <- InConf],
|
ConnectOpts#{subscriptions => pre_process_in_out(InConf),
|
||||||
forwards => [pre_process_in_out(Out) || Out <- OutConf]}.
|
forwards => pre_process_in_out(OutConf)}.
|
||||||
|
|
||||||
pre_process_in_out(Conf) ->
|
pre_process_in_out(undefined) -> undefined;
|
||||||
Conf1 = pre_process_conf(publish_local_topic, Conf),
|
pre_process_in_out(Conf) when is_map(Conf) ->
|
||||||
Conf2 = pre_process_conf(publish_remote_topic, Conf1),
|
Conf1 = pre_process_conf(local_topic, Conf),
|
||||||
Conf3 = pre_process_conf(publish_payload, Conf2),
|
Conf2 = pre_process_conf(remote_topic, Conf1),
|
||||||
Conf4 = pre_process_conf(publish_qos, Conf3),
|
Conf3 = pre_process_conf(payload, Conf2),
|
||||||
pre_process_conf(publish_retain, Conf4).
|
Conf4 = pre_process_conf(qos, Conf3),
|
||||||
|
pre_process_conf(retain, Conf4).
|
||||||
|
|
||||||
pre_process_conf(Key, Conf) ->
|
pre_process_conf(Key, Conf) ->
|
||||||
case maps:find(Key, Conf) of
|
case maps:find(Key, Conf) of
|
||||||
|
@ -350,19 +326,7 @@ common(_StateName, {call, From}, ensure_stopped, #{connection := Conn,
|
||||||
common(_StateName, {call, From}, get_forwards, #{forwards := Forwards}) ->
|
common(_StateName, {call, From}, get_forwards, #{forwards := Forwards}) ->
|
||||||
{keep_state_and_data, [{reply, From, Forwards}]};
|
{keep_state_and_data, [{reply, From, Forwards}]};
|
||||||
common(_StateName, {call, From}, get_subscriptions, #{connection := Connection}) ->
|
common(_StateName, {call, From}, get_subscriptions, #{connection := Connection}) ->
|
||||||
{keep_state_and_data, [{reply, From, maps:get(subscriptions, Connection, [])}]};
|
{keep_state_and_data, [{reply, From, maps:get(subscriptions, Connection, #{})}]};
|
||||||
common(_StateName, {call, From}, {ensure_forward_present, Topic}, State) ->
|
|
||||||
{Result, NewState} = do_ensure_forward_present(Topic, State),
|
|
||||||
{keep_state, NewState, [{reply, From, Result}]};
|
|
||||||
common(_StateName, {call, From}, {ensure_subscription_present, Topic, QoS}, State) ->
|
|
||||||
{Result, NewState} = do_ensure_subscription_present(Topic, QoS, State),
|
|
||||||
{keep_state, NewState, [{reply, From, Result}]};
|
|
||||||
common(_StateName, {call, From}, {ensure_forward_absent, Topic}, State) ->
|
|
||||||
{Result, NewState} = do_ensure_forward_absent(Topic, State),
|
|
||||||
{keep_state, NewState, [{reply, From, Result}]};
|
|
||||||
common(_StateName, {call, From}, {ensure_subscription_absent, Topic}, State) ->
|
|
||||||
{Result, NewState} = do_ensure_subscription_absent(Topic, State),
|
|
||||||
{keep_state, NewState, [{reply, From, Result}]};
|
|
||||||
common(_StateName, info, {deliver, _, Msg},
|
common(_StateName, info, {deliver, _, Msg},
|
||||||
State = #{replayq := Q, if_record_metrics := IfRecordMetric}) ->
|
State = #{replayq := Q, if_record_metrics := IfRecordMetric}) ->
|
||||||
Msgs = collect([Msg]),
|
Msgs = collect([Msg]),
|
||||||
|
@ -379,75 +343,15 @@ common(StateName, Type, Content, #{name := Name} = State) ->
|
||||||
[Name, Type, StateName, Content]),
|
[Name, Type, StateName, Content]),
|
||||||
{keep_state, State}.
|
{keep_state, State}.
|
||||||
|
|
||||||
do_ensure_forward_present(Topic, #{forwards := Forwards, name := Name} = State) ->
|
|
||||||
case is_local_sub_present(Topic, Forwards) of
|
|
||||||
true ->
|
|
||||||
{ok, State};
|
|
||||||
false ->
|
|
||||||
R = subscribe_local_topic(Topic, Name),
|
|
||||||
{R, State#{forwards => [Topic | Forwards]}}
|
|
||||||
end.
|
|
||||||
|
|
||||||
do_ensure_subscription_present(_Topic, _QoS, #{connection := undefined} = State) ->
|
|
||||||
{{error, no_connection}, State};
|
|
||||||
do_ensure_subscription_present(_Topic, _QoS, #{connect_module := emqx_bridge_rpc} = State) ->
|
|
||||||
{{error, no_remote_subscription_support}, State};
|
|
||||||
do_ensure_subscription_present(Topic, QoS, #{connect_module := ConnectModule,
|
|
||||||
connection := Conn} = State) ->
|
|
||||||
case is_remote_sub_present(Topic, maps:get(subscriptions, Conn, [])) of
|
|
||||||
true ->
|
|
||||||
{ok, State};
|
|
||||||
false ->
|
|
||||||
case ConnectModule:ensure_subscribed(Conn, Topic, QoS) of
|
|
||||||
{error, Error} ->
|
|
||||||
{{error, Error}, State};
|
|
||||||
Conn1 ->
|
|
||||||
{ok, State#{connection => Conn1}}
|
|
||||||
end
|
|
||||||
end.
|
|
||||||
|
|
||||||
do_ensure_forward_absent(Topic, #{forwards := Forwards} = State) ->
|
|
||||||
case is_local_sub_present(Topic, Forwards) of
|
|
||||||
true ->
|
|
||||||
R = do_unsubscribe(Topic),
|
|
||||||
{R, State#{forwards => lists:delete(Topic, Forwards)}};
|
|
||||||
false ->
|
|
||||||
{ok, State}
|
|
||||||
end.
|
|
||||||
do_ensure_subscription_absent(_Topic, #{connection := undefined} = State) ->
|
|
||||||
{{error, no_connection}, State};
|
|
||||||
do_ensure_subscription_absent(_Topic, #{connect_module := emqx_bridge_rpc} = State) ->
|
|
||||||
{{error, no_remote_subscription_support}, State};
|
|
||||||
do_ensure_subscription_absent(Topic, #{connect_module := ConnectModule,
|
|
||||||
connection := Conn} = State) ->
|
|
||||||
case is_remote_sub_present(Topic, maps:get(subscriptions, Conn, [])) of
|
|
||||||
true ->
|
|
||||||
case ConnectModule:ensure_unsubscribed(Conn, Topic) of
|
|
||||||
{error, Error} ->
|
|
||||||
{{error, Error}, State};
|
|
||||||
Conn1 ->
|
|
||||||
{ok, State#{connection => Conn1}}
|
|
||||||
end;
|
|
||||||
false ->
|
|
||||||
{ok, State}
|
|
||||||
end.
|
|
||||||
|
|
||||||
is_local_sub_present(Topic, Configs) ->
|
|
||||||
is_topic_present(subscribe_local_topic, Topic, Configs).
|
|
||||||
is_remote_sub_present(Topic, Configs) ->
|
|
||||||
is_topic_present(subscribe_remote_topic, Topic, Configs).
|
|
||||||
|
|
||||||
is_topic_present(Type, Topic, Configs) ->
|
|
||||||
lists:any(fun(Conf) ->
|
|
||||||
Topic == maps:get(Type, Conf, undefined)
|
|
||||||
end, Configs).
|
|
||||||
|
|
||||||
do_connect(#{forwards := Forwards,
|
do_connect(#{forwards := Forwards,
|
||||||
connect_module := ConnectModule,
|
connect_module := ConnectModule,
|
||||||
connect_opts := ConnectOpts,
|
connect_opts := ConnectOpts,
|
||||||
inflight := Inflight,
|
inflight := Inflight,
|
||||||
name := Name} = State) ->
|
name := Name} = State) ->
|
||||||
ok = subscribe_local_topics(Forwards, Name),
|
case Forwards of
|
||||||
|
undefined -> ok;
|
||||||
|
#{subscribe_local_topic := Topic} -> subscribe_local_topic(Topic, Name)
|
||||||
|
end,
|
||||||
case ConnectModule:start(ConnectOpts) of
|
case ConnectModule:start(ConnectOpts) of
|
||||||
{ok, Conn} ->
|
{ok, Conn} ->
|
||||||
?tp(info, connected, #{name => Name, inflight => length(Inflight)}),
|
?tp(info, connected, #{name => Name, inflight => length(Inflight)}),
|
||||||
|
@ -503,16 +407,18 @@ pop_and_send_loop(#{replayq := Q, connect_module := Module} = State, N) ->
|
||||||
end.
|
end.
|
||||||
|
|
||||||
%% Assert non-empty batch because we have a is_empty check earlier.
|
%% Assert non-empty batch because we have a is_empty check earlier.
|
||||||
|
do_send(#{forwards := undefined}, _QAckRef, Batch) ->
|
||||||
|
?LOG(error, "cannot forward messages to remote broker as 'bridge.mqtt.<name>.in' not configured, msg: ~p", [Batch]);
|
||||||
do_send(#{inflight := Inflight,
|
do_send(#{inflight := Inflight,
|
||||||
connect_module := Module,
|
connect_module := Module,
|
||||||
connection := Connection,
|
connection := Connection,
|
||||||
mountpoint := Mountpoint,
|
mountpoint := Mountpoint,
|
||||||
forwards := Forwards,
|
forwards := Forwards,
|
||||||
if_record_metrics := IfRecordMetrics} = State, QAckRef, [_ | _] = Batch) ->
|
if_record_metrics := IfRecordMetrics} = State, QAckRef, [_ | _] = Batch) ->
|
||||||
Vars = make_export_variables(Mountpoint, Forwards),
|
Vars = emqx_bridge_msg:make_pub_vars(Mountpoint, Forwards),
|
||||||
ExportMsg = fun(Message) ->
|
ExportMsg = fun(Message) ->
|
||||||
bridges_metrics_inc(IfRecordMetrics, 'bridge.mqtt.message_sent'),
|
bridges_metrics_inc(IfRecordMetrics, 'bridge.mqtt.message_sent'),
|
||||||
emqx_bridge_msg:to_export(Module, Vars, Message)
|
emqx_bridge_msg:to_remote_msg(Module, Message, Vars)
|
||||||
end,
|
end,
|
||||||
case Module:send(Connection, [ExportMsg(M) || M <- Batch]) of
|
case Module:send(Connection, [ExportMsg(M) || M <- Batch]) of
|
||||||
{ok, Refs} ->
|
{ok, Refs} ->
|
||||||
|
@ -524,15 +430,6 @@ do_send(#{inflight := Inflight,
|
||||||
{error, State}
|
{error, State}
|
||||||
end.
|
end.
|
||||||
|
|
||||||
make_export_variables(Mountpoint, #{
|
|
||||||
publish_remote_topic := PubTopic,
|
|
||||||
publish_payload := PayloadTmpl,
|
|
||||||
publish_qos := PubQoS,
|
|
||||||
publish_retain := PubRetain}) ->
|
|
||||||
#{topic => PubTopic, payload => PayloadTmpl,
|
|
||||||
qos => PubQoS, retain => PubRetain,
|
|
||||||
mountpoint => Mountpoint}.
|
|
||||||
|
|
||||||
%% map as set, ack-reference -> 1
|
%% map as set, ack-reference -> 1
|
||||||
map_set(Ref) when is_reference(Ref) ->
|
map_set(Ref) when is_reference(Ref) ->
|
||||||
%% QoS-0 or RPC call returns a reference
|
%% QoS-0 or RPC call returns a reference
|
||||||
|
@ -578,9 +475,6 @@ drop_acked_batches(Q, [#{send_ack_ref := Refs,
|
||||||
All
|
All
|
||||||
end.
|
end.
|
||||||
|
|
||||||
subscribe_local_topics(Topics, Name) ->
|
|
||||||
lists:foreach(fun(Topic) -> subscribe_local_topic(Topic, Name) end, Topics).
|
|
||||||
|
|
||||||
subscribe_local_topic(Topic, Name) ->
|
subscribe_local_topic(Topic, Name) ->
|
||||||
do_subscribe(Topic, Name).
|
do_subscribe(Topic, Name).
|
||||||
|
|
||||||
|
@ -597,14 +491,9 @@ validate(RawTopic) ->
|
||||||
|
|
||||||
do_subscribe(RawTopic, Name) ->
|
do_subscribe(RawTopic, Name) ->
|
||||||
TopicFilter = validate(RawTopic),
|
TopicFilter = validate(RawTopic),
|
||||||
{Topic, SubOpts} = emqx_topic:parse(TopicFilter, #{qos => ?QOS_1}),
|
{Topic, SubOpts} = emqx_topic:parse(TopicFilter, #{qos => ?QOS_2}),
|
||||||
emqx_broker:subscribe(Topic, Name, SubOpts).
|
emqx_broker:subscribe(Topic, Name, SubOpts).
|
||||||
|
|
||||||
do_unsubscribe(RawTopic) ->
|
|
||||||
TopicFilter = validate(RawTopic),
|
|
||||||
{Topic, _SubOpts} = emqx_topic:parse(TopicFilter),
|
|
||||||
emqx_broker:unsubscribe(Topic).
|
|
||||||
|
|
||||||
disconnect(#{connection := Conn,
|
disconnect(#{connection := Conn,
|
||||||
connect_module := Module
|
connect_module := Module
|
||||||
} = State) when Conn =/= undefined ->
|
} = State) when Conn =/= undefined ->
|
||||||
|
@ -622,7 +511,7 @@ format_mountpoint(undefined) ->
|
||||||
format_mountpoint(Prefix) ->
|
format_mountpoint(Prefix) ->
|
||||||
binary:replace(iolist_to_binary(Prefix), <<"${node}">>, atom_to_binary(node(), utf8)).
|
binary:replace(iolist_to_binary(Prefix), <<"${node}">>, atom_to_binary(node(), utf8)).
|
||||||
|
|
||||||
name(Id) -> list_to_atom(lists:concat([?MODULE, "_", Id])).
|
name(Id) -> list_to_atom(str(Id)).
|
||||||
|
|
||||||
register_metrics() ->
|
register_metrics() ->
|
||||||
lists:foreach(fun emqx_metrics:ensure/1,
|
lists:foreach(fun emqx_metrics:ensure/1,
|
||||||
|
@ -657,3 +546,10 @@ conn_type(mqtt) ->
|
||||||
emqx_bridge_mqtt;
|
emqx_bridge_mqtt;
|
||||||
conn_type(Mod) when is_atom(Mod) ->
|
conn_type(Mod) when is_atom(Mod) ->
|
||||||
Mod.
|
Mod.
|
||||||
|
|
||||||
|
str(A) when is_atom(A) ->
|
||||||
|
atom_to_list(A);
|
||||||
|
str(B) when is_binary(B) ->
|
||||||
|
binary_to_list(B);
|
||||||
|
str(S) when is_list(S) ->
|
||||||
|
S.
|
|
@ -40,7 +40,7 @@ fields("config") ->
|
||||||
, {reconnect_interval, emqx_schema:t(emqx_schema:duration_ms(), undefined, "30s")}
|
, {reconnect_interval, emqx_schema:t(emqx_schema:duration_ms(), undefined, "30s")}
|
||||||
, {proto_ver, fun proto_ver/1}
|
, {proto_ver, fun proto_ver/1}
|
||||||
, {bridge_mode, emqx_schema:t(boolean(), undefined, true)}
|
, {bridge_mode, emqx_schema:t(boolean(), undefined, true)}
|
||||||
, {clientid, emqx_schema:t(string())}
|
, {clientid_prefix, emqx_schema:t(string())}
|
||||||
, {username, emqx_schema:t(string())}
|
, {username, emqx_schema:t(string())}
|
||||||
, {password, emqx_schema:t(string())}
|
, {password, emqx_schema:t(string())}
|
||||||
, {clean_start, emqx_schema:t(boolean(), undefined, true)}
|
, {clean_start, emqx_schema:t(boolean(), undefined, true)}
|
||||||
|
@ -54,14 +54,14 @@ fields("config") ->
|
||||||
|
|
||||||
fields("in") ->
|
fields("in") ->
|
||||||
[ {subscribe_remote_topic, #{type => binary(), nullable => false}}
|
[ {subscribe_remote_topic, #{type => binary(), nullable => false}}
|
||||||
, {publish_local_topic, emqx_schema:t(binary(), undefined, <<"${topic}">>)}
|
, {local_topic, emqx_schema:t(binary(), undefined, <<"${topic}">>)}
|
||||||
, {subscribe_qos, emqx_schema:t(hoconsc:union([0, 1, 2, binary()]), undefined, 1)}
|
, {subscribe_qos, emqx_schema:t(hoconsc:union([0, 1, 2, binary()]), undefined, 1)}
|
||||||
] ++ publish_confs();
|
] ++ common_inout_confs();
|
||||||
|
|
||||||
fields("out") ->
|
fields("out") ->
|
||||||
[ {subscribe_local_topic, #{type => binary(), nullable => false}}
|
[ {subscribe_local_topic, #{type => binary(), nullable => false}}
|
||||||
, {publish_remote_topic, emqx_schema:t(binary(), undefined, <<"${topic}">>)}
|
, {remote_topic, emqx_schema:t(binary(), undefined, <<"${topic}">>)}
|
||||||
] ++ publish_confs();
|
] ++ common_inout_confs();
|
||||||
|
|
||||||
fields("replayq") ->
|
fields("replayq") ->
|
||||||
[ {dir, hoconsc:union([boolean(), string()])}
|
[ {dir, hoconsc:union([boolean(), string()])}
|
||||||
|
@ -70,10 +70,13 @@ fields("replayq") ->
|
||||||
, {max_total_bytes, emqx_schema:t(emqx_schema:bytesize(), undefined, "1024MB")}
|
, {max_total_bytes, emqx_schema:t(emqx_schema:bytesize(), undefined, "1024MB")}
|
||||||
].
|
].
|
||||||
|
|
||||||
|
common_inout_confs() ->
|
||||||
|
[{id, #{type => binary(), nullable => false}}] ++ publish_confs().
|
||||||
|
|
||||||
publish_confs() ->
|
publish_confs() ->
|
||||||
[ {publish_qos, emqx_schema:t(hoconsc:union([0, 1, 2, binary()]), undefined, <<"${qos}">>)}
|
[ {qos, emqx_schema:t(hoconsc:union([0, 1, 2, binary()]), undefined, <<"${qos}">>)}
|
||||||
, {publish_retain, emqx_schema:t(hoconsc:union([boolean(), binary()]), undefined, <<"${retain}">>)}
|
, {retain, emqx_schema:t(hoconsc:union([boolean(), binary()]), undefined, <<"${retain}">>)}
|
||||||
, {publish_payload, emqx_schema:t(binary(), undefined, <<"${payload}">>)}
|
, {payload, emqx_schema:t(binary(), undefined, <<"${payload}">>)}
|
||||||
].
|
].
|
||||||
|
|
||||||
proto_ver(type) -> hoconsc:enum([v3, v4, v5]);
|
proto_ver(type) -> hoconsc:enum([v3, v4, v5]);
|
||||||
|
@ -81,11 +84,91 @@ proto_ver(default) -> v4;
|
||||||
proto_ver(_) -> undefined.
|
proto_ver(_) -> undefined.
|
||||||
|
|
||||||
%% ===================================================================
|
%% ===================================================================
|
||||||
on_start(InstId, #{server := Server,
|
on_start(InstId, Conf) ->
|
||||||
|
logger:info("starting mqtt connector: ~p, ~p", [InstId, Conf]),
|
||||||
|
NamePrefix = binary_to_list(InstId),
|
||||||
|
BasicConf = basic_config(Conf),
|
||||||
|
InitRes = {ok, #{name_prefix => NamePrefix, baisc_conf => BasicConf, sub_bridges => []}},
|
||||||
|
InOutConfigs = check_channel_id_dup(maps:get(in, Conf, []) ++ maps:get(out, Conf, [])),
|
||||||
|
lists:foldl(fun
|
||||||
|
(_InOutConf, {error, Reason}) ->
|
||||||
|
{error, Reason};
|
||||||
|
(InOutConf, {ok, #{sub_bridges := SubBridges} = Res}) ->
|
||||||
|
case create_channel(InOutConf, NamePrefix, BasicConf) of
|
||||||
|
{error, Reason} -> {error, Reason};
|
||||||
|
{ok, Name} -> {ok, Res#{sub_bridges => [Name | SubBridges]}}
|
||||||
|
end
|
||||||
|
end, InitRes, InOutConfigs).
|
||||||
|
|
||||||
|
on_stop(InstId, #{}) ->
|
||||||
|
logger:info("stopping mqtt connector: ~p", [InstId]),
|
||||||
|
case emqx_bridge_mqtt_sup:drop_bridge(InstId) of
|
||||||
|
ok -> ok;
|
||||||
|
{error, not_found} -> ok;
|
||||||
|
{error, Reason} ->
|
||||||
|
logger:error("stop bridge failed, error: ~p", [Reason])
|
||||||
|
end.
|
||||||
|
|
||||||
|
%% TODO: let the emqx_resource trigger on_query/4 automatically according to the
|
||||||
|
%% `in` and `out` config
|
||||||
|
on_query(InstId, {create_channel, Conf}, _AfterQuery, #{name_prefix := Prefix,
|
||||||
|
baisc_conf := BasicConf}) ->
|
||||||
|
logger:debug("create channel to connector: ~p, conf: ~p", [InstId, Conf]),
|
||||||
|
create_channel(Conf, Prefix, BasicConf);
|
||||||
|
on_query(InstId, {publish_to_local, Msg}, _AfterQuery, _State) ->
|
||||||
|
logger:debug("publish to local node, connector: ~p, msg: ~p", [InstId, Msg]);
|
||||||
|
on_query(InstId, {publish_to_remote, Msg}, _AfterQuery, _State) ->
|
||||||
|
logger:debug("publish to remote node, connector: ~p, msg: ~p", [InstId, Msg]).
|
||||||
|
|
||||||
|
on_health_check(_InstId, #{sub_bridges := NameList} = State) ->
|
||||||
|
Results = [{Name, emqx_bridge_worker:ping(Name)} || Name <- NameList],
|
||||||
|
case lists:all(fun({_, pong}) -> true; ({_, _}) -> false end, Results) of
|
||||||
|
true -> {ok, State};
|
||||||
|
false -> {error, {some_sub_bridge_down, Results}, State}
|
||||||
|
end.
|
||||||
|
|
||||||
|
check_channel_id_dup(Confs) ->
|
||||||
|
lists:foreach(fun(#{id := Id}) ->
|
||||||
|
case length([Id || #{id := Id0} <- Confs, Id0 == Id]) of
|
||||||
|
1 -> ok;
|
||||||
|
L when L > 1 -> error({mqtt_bridge_conf, {duplicate_id_found, Id}})
|
||||||
|
end
|
||||||
|
end, Confs),
|
||||||
|
Confs.
|
||||||
|
|
||||||
|
%% this is an `in` bridge
|
||||||
|
create_channel(#{subscribe_remote_topic := _, id := BridgeId} = InConf, NamePrefix, BasicConf) ->
|
||||||
|
logger:info("creating 'in' channel for: ~p", [BridgeId]),
|
||||||
|
create_sub_bridge(BasicConf#{name => bridge_name(NamePrefix, BridgeId),
|
||||||
|
subscriptions => InConf, forwards => undefined});
|
||||||
|
%% this is an `out` bridge
|
||||||
|
create_channel(#{subscribe_local_topic := _, id := BridgeId} = OutConf, NamePrefix, BasicConf) ->
|
||||||
|
logger:info("creating 'out' channel for: ~p", [BridgeId]),
|
||||||
|
create_sub_bridge(BasicConf#{name => bridge_name(NamePrefix, BridgeId),
|
||||||
|
subscriptions => undefined, forwards => OutConf}).
|
||||||
|
|
||||||
|
create_sub_bridge(#{name := Name} = Conf) ->
|
||||||
|
case emqx_bridge_mqtt_sup:create_bridge(Conf) of
|
||||||
|
{ok, _Pid} ->
|
||||||
|
start_sub_bridge(Name);
|
||||||
|
{error, {already_started, _Pid}} ->
|
||||||
|
ok;
|
||||||
|
{error, Reason} ->
|
||||||
|
{error, Reason}
|
||||||
|
end.
|
||||||
|
|
||||||
|
start_sub_bridge(Name) ->
|
||||||
|
case emqx_bridge_worker:ensure_started(Name) of
|
||||||
|
ok -> {ok, Name};
|
||||||
|
{error, Reason} -> {error, Reason}
|
||||||
|
end.
|
||||||
|
|
||||||
|
basic_config(#{
|
||||||
|
server := Server,
|
||||||
reconnect_interval := ReconnIntv,
|
reconnect_interval := ReconnIntv,
|
||||||
proto_ver := ProtoVer,
|
proto_ver := ProtoVer,
|
||||||
bridge_mode := BridgeMod,
|
bridge_mode := BridgeMod,
|
||||||
clientid := ClientID,
|
clientid_prefix := ClientIdPrefix,
|
||||||
username := User,
|
username := User,
|
||||||
password := Password,
|
password := Password,
|
||||||
clean_start := CleanStart,
|
clean_start := CleanStart,
|
||||||
|
@ -93,24 +176,16 @@ on_start(InstId, #{server := Server,
|
||||||
retry_interval := RetryIntv,
|
retry_interval := RetryIntv,
|
||||||
max_inflight := MaxInflight,
|
max_inflight := MaxInflight,
|
||||||
replayq := ReplayQ,
|
replayq := ReplayQ,
|
||||||
in := In,
|
ssl := #{enable := EnableSsl} = Ssl}) ->
|
||||||
out := Out,
|
#{
|
||||||
ssl := #{enable := EnableSsl} = Ssl} = Conf) ->
|
|
||||||
logger:info("starting mqtt connector: ~p, ~p", [InstId, Conf]),
|
|
||||||
BridgeName = binary_to_atom(InstId, latin1),
|
|
||||||
BridgeConf = Conf#{
|
|
||||||
name => BridgeName,
|
|
||||||
config => #{
|
|
||||||
conn_type => mqtt,
|
conn_type => mqtt,
|
||||||
subscriptions => In,
|
|
||||||
forwards => Out,
|
|
||||||
replayq => ReplayQ,
|
replayq => ReplayQ,
|
||||||
%% connection opts
|
%% connection opts
|
||||||
server => Server,
|
server => Server,
|
||||||
reconnect_interval => ReconnIntv,
|
reconnect_interval => ReconnIntv,
|
||||||
proto_ver => ProtoVer,
|
proto_ver => ProtoVer,
|
||||||
bridge_mode => BridgeMod,
|
bridge_mode => BridgeMod,
|
||||||
clientid => ClientID,
|
clientid_prefix => ClientIdPrefix,
|
||||||
username => User,
|
username => User,
|
||||||
password => Password,
|
password => Password,
|
||||||
clean_start => CleanStart,
|
clean_start => CleanStart,
|
||||||
|
@ -120,33 +195,14 @@ on_start(InstId, #{server := Server,
|
||||||
ssl => EnableSsl,
|
ssl => EnableSsl,
|
||||||
ssl_opts => maps:to_list(maps:remove(enable, Ssl)),
|
ssl_opts => maps:to_list(maps:remove(enable, Ssl)),
|
||||||
if_record_metrics => true
|
if_record_metrics => true
|
||||||
}
|
}.
|
||||||
},
|
|
||||||
case emqx_bridge_mqtt_sup:create_bridge(BridgeConf) of
|
|
||||||
{ok, _Pid} ->
|
|
||||||
start_bridge(BridgeName);
|
|
||||||
{error, {already_started, _Pid}} ->
|
|
||||||
start_bridge(BridgeName);
|
|
||||||
{error, Reason} ->
|
|
||||||
{error, Reason}
|
|
||||||
end.
|
|
||||||
|
|
||||||
on_stop(InstId, #{}) ->
|
bridge_name(Prefix, Id) ->
|
||||||
logger:info("stopping mqtt connector: ~p", [InstId]),
|
list_to_atom(str(Prefix) ++ ":" ++ str(Id)).
|
||||||
emqx_bridge_mqtt_sup:drop_bridge(InstId).
|
|
||||||
|
|
||||||
%% TODO: let the emqx_resource trigger on_query/4 automatically according to the
|
str(A) when is_atom(A) ->
|
||||||
%% `in` and `out` config
|
atom_to_list(A);
|
||||||
on_query(InstId, {publish_to_local, Msg}, _AfterQuery, _State) ->
|
str(B) when is_binary(B) ->
|
||||||
logger:debug("publish to local node, connector: ~p, msg: ~p", [InstId, Msg]);
|
binary_to_list(B);
|
||||||
on_query(InstId, {publish_to_remote, Msg}, _AfterQuery, _State) ->
|
str(S) when is_list(S) ->
|
||||||
logger:debug("publish to remote node, connector: ~p, msg: ~p", [InstId, Msg]).
|
S.
|
||||||
|
|
||||||
on_health_check(_InstId, #{bridge_name := Name}) ->
|
|
||||||
{ok, emqx_bridge_worker:ping(Name)}.
|
|
||||||
|
|
||||||
start_bridge(Name) ->
|
|
||||||
case emqx_bridge_worker:ensure_started(Name) of
|
|
||||||
ok -> {ok, #{bridge_name => Name}};
|
|
||||||
{error, Reason} -> {error, Reason}
|
|
||||||
end.
|
|
||||||
|
|
|
@ -222,6 +222,7 @@ do_compare('<=', L, R) -> L =< R;
|
||||||
do_compare('>=', L, R) -> L >= R;
|
do_compare('>=', L, R) -> L >= R;
|
||||||
do_compare('<>', L, R) -> L /= R;
|
do_compare('<>', L, R) -> L /= R;
|
||||||
do_compare('!=', L, R) -> L /= R;
|
do_compare('!=', L, R) -> L /= R;
|
||||||
|
do_compare('~=', T, F) -> emqx_topic:match(T, F);
|
||||||
do_compare('=~', T, F) -> emqx_topic:match(T, F).
|
do_compare('=~', T, F) -> emqx_topic:match(T, F).
|
||||||
|
|
||||||
number(Bin) ->
|
number(Bin) ->
|
||||||
|
|
Loading…
Reference in New Issue