Merge pull request #6142 from HJianBo/put-message-headers

Fill the message headers
This commit is contained in:
Zaiming (Stone) Shi 2021-11-12 12:51:48 +01:00 committed by GitHub
commit 5f8d9db64b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 170 additions and 63 deletions

View File

@ -1,6 +1,6 @@
{application, emqx_coap, {application, emqx_coap,
[{description, "EMQ X CoAP Gateway"}, [{description, "EMQ X CoAP Gateway"},
{vsn, "4.3.0"}, % strict semver, bump manually! {vsn, "4.3.1"}, % strict semver, bump manually!
{modules, []}, {modules, []},
{registered, []}, {registered, []},
{applications, [kernel,stdlib,gen_coap]}, {applications, [kernel,stdlib,gen_coap]},

View File

@ -0,0 +1,9 @@
%% -*-: erlang -*-
{VSN,
[{"4.3.0",[
{load_module, emqx_coap_mqtt_adapter, brutal_purge, soft_purge, []}]},
{<<".*">>, []}],
[{"4.3.0",[
{load_module, emqx_coap_mqtt_adapter, brutal_purge, soft_purge, []}]},
{<<".*">>, []}]
}.

View File

@ -58,6 +58,8 @@
-define(SUBOPTS, #{rh => 0, rap => 0, nl => 0, qos => ?QOS_0, is_new => false}). -define(SUBOPTS, #{rh => 0, rap => 0, nl => 0, qos => ?QOS_0, is_new => false}).
-define(PROTO_VER, 1).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% API %% API
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
@ -244,15 +246,26 @@ chann_publish(Topic, Payload, State = #state{clientid = ClientId}) ->
case emqx_access_control:check_acl(clientinfo(State), publish, Topic) of case emqx_access_control:check_acl(clientinfo(State), publish, Topic) of
allow -> allow ->
_ = emqx_broker:publish( _ = emqx_broker:publish(
emqx_message:set_flag(retain, false, packet_to_message(Topic, Payload, State)), ok;
emqx_message:make(ClientId, ?QOS_0, Topic, Payload))),
ok;
deny -> deny ->
?LOG(warning, "publish to ~p by clientid ~p failed due to acl check.", ?LOG(warning, "publish to ~p by clientid ~p failed due to acl check.",
[Topic, ClientId]), [Topic, ClientId]),
{error, forbidden} {error, forbidden}
end. end.
packet_to_message(Topic, Payload,
#state{clientid = ClientId,
username = Username,
peername = {PeerHost, _}}) ->
Message = emqx_message:set_flag(
retain, false,
emqx_message:make(ClientId, ?QOS_0, Topic, Payload)
),
emqx_message:set_headers(
#{ proto_ver => ?PROTO_VER
, protocol => coap
, username => Username
, peerhost => PeerHost}, Message).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Deliver %% Deliver
@ -324,7 +337,7 @@ conninfo(#state{peername = Peername,
peercert => nossl, %% TODO: dtls peercert => nossl, %% TODO: dtls
conn_mod => ?MODULE, conn_mod => ?MODULE,
proto_name => <<"CoAP">>, proto_name => <<"CoAP">>,
proto_ver => 1, proto_ver => ?PROTO_VER,
clean_start => true, clean_start => true,
clientid => ClientId, clientid => ClientId,
username => undefined, username => undefined,
@ -384,4 +397,3 @@ clientinfo(#state{peername = {PeerHost, _},
mountpoint => undefined, mountpoint => undefined,
ws_cookie => undefined ws_cookie => undefined
}. }.

View File

@ -1,6 +1,6 @@
{application, emqx_exproto, {application, emqx_exproto,
[{description, "EMQ X Extension for Protocol"}, [{description, "EMQ X Extension for Protocol"},
{vsn, "4.3.4"}, %% 4.3.3 is used by ee {vsn, "4.3.5"}, %% 4.3.3 is used by ee
{modules, []}, {modules, []},
{registered, []}, {registered, []},
{mod, {emqx_exproto_app, []}}, {mod, {emqx_exproto_app, []}},

View File

@ -1,6 +1,8 @@
%% -*- mode: erlang -*- %% -*- mode: erlang -*-
{VSN, {VSN,
[{"4.3.3", [{"4.3.4",
[{load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]},
{"4.3.3",
[{load_module,emqx_exproto_conn,brutal_purge,soft_purge,[]}, [{load_module,emqx_exproto_conn,brutal_purge,soft_purge,[]},
{load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]}, {load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]},
{"4.3.2", {"4.3.2",
@ -12,7 +14,9 @@
{load_module,emqx_exproto_conn,brutal_purge,soft_purge,[]}, {load_module,emqx_exproto_conn,brutal_purge,soft_purge,[]},
{load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]}, {load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]},
{<<".*">>,[]}], {<<".*">>,[]}],
[{"4.3.3", [{"4.3.4",
[{load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]},
{"4.3.3",
[{load_module,emqx_exproto_conn,brutal_purge,soft_purge,[]}, [{load_module,emqx_exproto_conn,brutal_purge,soft_purge,[]},
{load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]}, {load_module,emqx_exproto_channel,brutal_purge,soft_purge,[]}]},
{"4.3.2", {"4.3.2",

View File

@ -340,17 +340,14 @@ handle_call({unsubscribe, TopicFilter},
handle_call({publish, Topic, Qos, Payload}, handle_call({publish, Topic, Qos, Payload},
Channel = #channel{ Channel = #channel{
conn_state = connected, conn_state = connected,
clientinfo = ClientInfo clientinfo = ClientInfo}) ->
= #{clientid := From,
mountpoint := Mountpoint}}) ->
case is_acl_enabled(ClientInfo) andalso case is_acl_enabled(ClientInfo) andalso
emqx_access_control:check_acl(ClientInfo, publish, Topic) of emqx_access_control:check_acl(ClientInfo, publish, Topic) of
deny -> deny ->
{reply, {error, ?RESP_PERMISSION_DENY, <<"ACL deny">>}, Channel}; {reply, {error, ?RESP_PERMISSION_DENY, <<"ACL deny">>}, Channel};
_ -> _ ->
Msg = emqx_message:make(From, Qos, Topic, Payload), Msg = packet_to_message(Topic, Qos, Payload, Channel),
NMsg = emqx_mountpoint:mount(Mountpoint, Msg), _ = emqx:publish(Msg),
_ = emqx:publish(NMsg),
{reply, ok, Channel} {reply, ok, Channel}
end; end;
@ -419,6 +416,24 @@ is_anonymous(_AuthResult) -> false.
clean_anonymous_clients() -> clean_anonymous_clients() ->
ets:delete(?CHAN_CONN_TAB, ?CHANMOCK(self())). ets:delete(?CHAN_CONN_TAB, ?CHANMOCK(self())).
packet_to_message(Topic, Qos, Payload,
#channel{
conninfo = #{proto_ver := ProtoVer},
clientinfo = #{
protocol := Protocol,
clientid := ClientId,
username := Username,
peerhost := PeerHost,
mountpoint := Mountpoint}}) ->
Msg = emqx_message:make(
ClientId, Qos,
Topic, Payload, #{},
#{proto_ver => ProtoVer,
protocol => Protocol,
username => Username,
peerhost => PeerHost}),
emqx_mountpoint:mount(Mountpoint, Msg).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Sub/UnSub %% Sub/UnSub
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------

View File

@ -1,6 +1,6 @@
{application,emqx_lwm2m, {application,emqx_lwm2m,
[{description,"EMQ X LwM2M Gateway"}, [{description,"EMQ X LwM2M Gateway"},
{vsn, "4.3.4"}, % strict semver, bump manually! {vsn, "4.3.5"}, % strict semver, bump manually!
{modules,[]}, {modules,[]},
{registered,[emqx_lwm2m_sup]}, {registered,[emqx_lwm2m_sup]},
{applications,[kernel,stdlib,lwm2m_coap]}, {applications,[kernel,stdlib,lwm2m_coap]},

View File

@ -1,5 +1,5 @@
%% -*-: erlang -*- %% -*-: erlang -*-
{"4.3.4", {VSN,
[ [
{<<"4\\.3\\.[0-1]">>, [ {<<"4\\.3\\.[0-1]">>, [
{restart_application, emqx_lwm2m} {restart_application, emqx_lwm2m}
@ -7,7 +7,10 @@
{"4.3.2", [ {"4.3.2", [
{load_module, emqx_lwm2m_message, brutal_purge, soft_purge, []} {load_module, emqx_lwm2m_message, brutal_purge, soft_purge, []}
]}, ]},
{"4.3.3", []} %% only config change {"4.3.3", []}, %% only config change
{"4.3.4", [
{load_module, emqx_lwm2m_protocol, brutal_purge, soft_purge, []}
]}
], ],
[ [
{<<"4\\.3\\.[0-1]">>, [ {<<"4\\.3\\.[0-1]">>, [
@ -16,6 +19,9 @@
{"4.3.2", [ {"4.3.2", [
{load_module, emqx_lwm2m_message, brutal_purge, soft_purge, []} {load_module, emqx_lwm2m_message, brutal_purge, soft_purge, []}
]}, ]},
{"4.3.3", []} %% only config change {"4.3.3", []}, %% only config change
{"4.3.4", [
{load_module, emqx_lwm2m_protocol, brutal_purge, soft_purge, []}
]}
] ]
}. }.

View File

@ -74,7 +74,8 @@ call(Pid, Msg, Timeout) ->
Error -> {error, Error} Error -> {error, Error}
end. end.
init(CoapPid, EndpointName, Peername = {_Peerhost, _Port}, RegInfo = #{<<"lt">> := LifeTime, <<"lwm2m">> := Ver}) -> init(CoapPid, EndpointName, Peername = {_Peerhost, _Port},
RegInfo = #{<<"lt">> := LifeTime, <<"lwm2m">> := Ver}) ->
Mountpoint = proplists:get_value(mountpoint, lwm2m_coap_responder:options(), ""), Mountpoint = proplists:get_value(mountpoint, lwm2m_coap_responder:options(), ""),
Lwm2mState = #lwm2m_state{peername = Peername, Lwm2mState = #lwm2m_state{peername = Peername,
endpoint_name = EndpointName, endpoint_name = EndpointName,
@ -105,7 +106,8 @@ init(CoapPid, EndpointName, Peername = {_Peerhost, _Port}, RegInfo = #{<<"lt">>
emqx_cm:insert_channel_info(EndpointName, info(Lwm2mState1), stats(Lwm2mState1)), emqx_cm:insert_channel_info(EndpointName, info(Lwm2mState1), stats(Lwm2mState1)),
emqx_lwm2m_cm:register_channel(EndpointName, RegInfo, LifeTime, Ver, Peername), emqx_lwm2m_cm:register_channel(EndpointName, RegInfo, LifeTime, Ver, Peername),
{ok, Lwm2mState1#lwm2m_state{life_timer = emqx_lwm2m_timer:start_timer(LifeTime, {life_timer, expired})}}; NTimer = emqx_lwm2m_timer:start_timer(LifeTime, {life_timer, expired}),
{ok, Lwm2mState1#lwm2m_state{life_timer = NTimer}};
{error, Error} -> {error, Error} ->
_ = run_hooks('client.connack', [conninfo(Lwm2mState), not_authorized], undefined), _ = run_hooks('client.connack', [conninfo(Lwm2mState), not_authorized], undefined),
{error, Error} {error, Error}
@ -186,7 +188,8 @@ deliver(#message{topic = Topic, payload = Payload},
started_at = StartedAt, started_at = StartedAt,
endpoint_name = EndpointName}) -> endpoint_name = EndpointName}) ->
IsCacheMode = is_cache_mode(RegInfo, StartedAt), IsCacheMode = is_cache_mode(RegInfo, StartedAt),
?LOG(debug, "Get MQTT message from broker, IsCacheModeNow?: ~p, Topic: ~p, Payload: ~p", [IsCacheMode, Topic, Payload]), ?LOG(debug, "Get MQTT message from broker, IsCacheModeNow?: ~p, "
"Topic: ~p, Payload: ~p", [IsCacheMode, Topic, Payload]),
AlternatePath = maps:get(<<"alternatePath">>, RegInfo, <<"/">>), AlternatePath = maps:get(<<"alternatePath">>, RegInfo, <<"/">>),
deliver_to_coap(AlternatePath, Payload, CoapPid, IsCacheMode, EndpointName), deliver_to_coap(AlternatePath, Payload, CoapPid, IsCacheMode, EndpointName),
Lwm2mState. Lwm2mState.
@ -235,8 +238,20 @@ unsubscribe(Topic, Lwm2mState = #lwm2m_state{endpoint_name = _EndpointName}) ->
emqx_broker:unsubscribe(Topic), emqx_broker:unsubscribe(Topic),
emqx_hooks:run('session.unsubscribed', [clientinfo(Lwm2mState), Topic, Opts]). emqx_hooks:run('session.unsubscribed', [clientinfo(Lwm2mState), Topic, Opts]).
publish(Topic, Payload, Qos, EndpointName) -> publish(Topic, Payload, Qos,
emqx_broker:publish(emqx_message:set_flag(retain, false, emqx_message:make(EndpointName, Qos, Topic, Payload))). #lwm2m_state{
version = ProtoVer,
peername = {PeerHost, _},
endpoint_name = EndpointName}) ->
Message = emqx_message:set_flag(
retain, false,
emqx_message:make(EndpointName, Qos, Topic, Payload)
),
NMessage = emqx_message:set_headers(
#{proto_ver => ProtoVer,
protocol => lwm2m,
peerhost => PeerHost}, Message),
emqx_broker:publish(NMessage).
time_now() -> erlang:system_time(millisecond). time_now() -> erlang:system_time(millisecond).
@ -244,7 +259,8 @@ time_now() -> erlang:system_time(millisecond).
%% Deliver downlink message to coap %% Deliver downlink message to coap
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
deliver_to_coap(AlternatePath, JsonData, CoapPid, CacheMode, EndpointName) when is_binary(JsonData)-> deliver_to_coap(AlternatePath, JsonData,
CoapPid, CacheMode, EndpointName) when is_binary(JsonData)->
try try
TermData = emqx_json:decode(JsonData, [return_maps]), TermData = emqx_json:decode(JsonData, [return_maps]),
deliver_to_coap(AlternatePath, TermData, CoapPid, CacheMode, EndpointName) deliver_to_coap(AlternatePath, TermData, CoapPid, CacheMode, EndpointName)
@ -273,7 +289,8 @@ deliver_to_coap(AlternatePath, TermData, CoapPid, CacheMode, EndpointName) when
send_to_broker(EventType, Payload = #{}, Lwm2mState) -> send_to_broker(EventType, Payload = #{}, Lwm2mState) ->
do_send_to_broker(EventType, Payload, Lwm2mState). do_send_to_broker(EventType, Payload, Lwm2mState).
do_send_to_broker(EventType, #{<<"data">> := Data} = Payload, #lwm2m_state{endpoint_name = EndpointName} = Lwm2mState) -> do_send_to_broker(EventType, #{<<"data">> := Data} = Payload,
#lwm2m_state{endpoint_name = EndpointName} = Lwm2mState) ->
ReqPath = maps:get(<<"reqPath">>, Data, undefined), ReqPath = maps:get(<<"reqPath">>, Data, undefined),
Code = maps:get(<<"code">>, Data, undefined), Code = maps:get(<<"code">>, Data, undefined),
CodeMsg = maps:get(<<"codeMsg">>, Data, undefined), CodeMsg = maps:get(<<"codeMsg">>, Data, undefined),
@ -281,7 +298,7 @@ do_send_to_broker(EventType, #{<<"data">> := Data} = Payload, #lwm2m_state{endpo
emqx_lwm2m_cm:register_cmd(EndpointName, ReqPath, EventType, {Code, CodeMsg, Content}), emqx_lwm2m_cm:register_cmd(EndpointName, ReqPath, EventType, {Code, CodeMsg, Content}),
NewPayload = maps:put(<<"msgType">>, EventType, Payload), NewPayload = maps:put(<<"msgType">>, EventType, Payload),
Topic = uplink_topic(EventType, Lwm2mState), Topic = uplink_topic(EventType, Lwm2mState),
publish(Topic, emqx_json:encode(NewPayload), _Qos = 0, Lwm2mState#lwm2m_state.endpoint_name). publish(Topic, emqx_json:encode(NewPayload), _Qos = 0, Lwm2mState).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Auto Observe %% Auto Observe
@ -321,12 +338,21 @@ observe_object_list(AlternatePath, ObjectList, CoapPid, EndpointName) ->
[ObjInsId | _LastPath1] = LastPath, [ObjInsId | _LastPath1] = LastPath,
case ObjInsId of case ObjInsId of
<<"0">> -> <<"0">> ->
observe_object_slowly(AlternatePath, <<"/19/0/0">>, CoapPid, 100, EndpointName); observe_object_slowly(
AlternatePath, <<"/19/0/0">>,
CoapPid, 100, EndpointName
);
_ -> _ ->
observe_object_slowly(AlternatePath, ObjectPath, CoapPid, 100, EndpointName) observe_object_slowly(
AlternatePath, ObjectPath,
CoapPid, 100, EndpointName
)
end; end;
_ -> _ ->
observe_object_slowly(AlternatePath, ObjectPath, CoapPid, 100, EndpointName) observe_object_slowly(
AlternatePath, ObjectPath,
CoapPid, 100, EndpointName
)
end end
end, ObjectList). end, ObjectList).
@ -380,11 +406,12 @@ get_cached_downlink_messages() ->
is_cache_mode(RegInfo, StartedAt) -> is_cache_mode(RegInfo, StartedAt) ->
case is_psm(RegInfo) orelse is_qmode(RegInfo) of case is_psm(RegInfo) orelse is_qmode(RegInfo) of
true -> true ->
QModeTimeWind = proplists:get_value(qmode_time_window, lwm2m_coap_responder:options(), 22), QModeTimeWind = proplists:get_value(
Now = time_now(), qmode_time_window,
if (Now - StartedAt) >= QModeTimeWind -> true; lwm2m_coap_responder:options(),
true -> false 22
end; ),
(time_now() - StartedAt) >= QModeTimeWind;
false -> false false -> false
end. end.

View File

@ -1,6 +1,6 @@
{application, emqx_stomp, {application, emqx_stomp,
[{description, "EMQ X Stomp Protocol Plugin"}, [{description, "EMQ X Stomp Protocol Plugin"},
{vsn, "4.3.2"}, % strict semver, bump manually! {vsn, "4.3.3"}, % strict semver, bump manually!
{modules, []}, {modules, []},
{registered, [emqx_stomp_sup]}, {registered, [emqx_stomp_sup]},
{applications, [kernel,stdlib]}, {applications, [kernel,stdlib]},

View File

@ -1,11 +1,17 @@
%% -*- mode: erlang -*- %% -*- mode: erlang -*-
{VSN, {VSN,
[{"4.3.1",[{load_module,emqx_stomp_connection,brutal_purge,soft_purge,[]}]}, [{"4.3.2",[{load_module,emqx_stomp_protocol,brutal_purge,soft_purge,[]}]},
{"4.3.1",[
{load_module,emqx_stomp_protocol,brutal_purge,soft_purge,[]},
{load_module,emqx_stomp_connection,brutal_purge,soft_purge,[]}]},
{"4.3.0", {"4.3.0",
[{restart_application,emqx_stomp}, [{restart_application,emqx_stomp},
{apply,{emqx_stomp,force_clear_after_app_stoped,[]}}]}, {apply,{emqx_stomp,force_clear_after_app_stoped,[]}}]},
{<<".*">>,[]}], {<<".*">>,[]}],
[{"4.3.1",[{load_module,emqx_stomp_connection,brutal_purge,soft_purge,[]}]}, [{"4.3.2",[{load_module,emqx_stomp_protocol,brutal_purge,soft_purge,[]}]},
{"4.3.1",[
{load_module,emqx_stomp_protocol,brutal_purge,soft_purge,[]},
{load_module,emqx_stomp_connection,brutal_purge,soft_purge,[]}]},
{"4.3.0", {"4.3.0",
[{restart_application,emqx_stomp}]}, [{restart_application,emqx_stomp}]},
{<<".*">>,[]}]}. {<<".*">>,[]}]}.

View File

@ -108,6 +108,8 @@
, init/2 , init/2
]}). ]}).
-elvis([{elvis_style, dont_repeat_yourself, disable}]).
-type(pstate() :: #pstate{}). -type(pstate() :: #pstate{}).
%% @doc Init protocol %% @doc Init protocol
@ -132,7 +134,6 @@ init(ConnInfo = #{peername := {PeerHost, _Port},
AllowAnonymous = get_value(allow_anonymous, Opts, false), AllowAnonymous = get_value(allow_anonymous, Opts, false),
DefaultUser = get_value(default_user, Opts), DefaultUser = get_value(default_user, Opts),
#pstate{ #pstate{
conninfo = NConnInfo, conninfo = NConnInfo,
clientinfo = ClientInfo, clientinfo = ClientInfo,
@ -288,7 +289,12 @@ received(#stomp_frame{command = <<"CONNECT">>}, State = #pstate{connected = true
received(Frame = #stomp_frame{command = <<"SEND">>, headers = Headers}, State) -> received(Frame = #stomp_frame{command = <<"SEND">>, headers = Headers}, State) ->
case header(<<"transaction">>, Headers) of case header(<<"transaction">>, Headers) of
undefined -> {ok, handle_recv_send_frame(Frame, State)}; undefined -> {ok, handle_recv_send_frame(Frame, State)};
TransactionId -> add_action(TransactionId, {fun ?MODULE:handle_recv_send_frame/2, [Frame]}, receipt_id(Headers), State) TransactionId ->
add_action(TransactionId,
{fun ?MODULE:handle_recv_send_frame/2, [Frame]},
receipt_id(Headers),
State
)
end; end;
received(#stomp_frame{command = <<"SUBSCRIBE">>, headers = Headers}, received(#stomp_frame{command = <<"SUBSCRIBE">>, headers = Headers},
@ -346,7 +352,11 @@ received(#stomp_frame{command = <<"UNSUBSCRIBE">>, headers = Headers},
received(Frame = #stomp_frame{command = <<"ACK">>, headers = Headers}, State) -> received(Frame = #stomp_frame{command = <<"ACK">>, headers = Headers}, State) ->
case header(<<"transaction">>, Headers) of case header(<<"transaction">>, Headers) of
undefined -> {ok, handle_recv_ack_frame(Frame, State)}; undefined -> {ok, handle_recv_ack_frame(Frame, State)};
TransactionId -> add_action(TransactionId, {fun ?MODULE:handle_recv_ack_frame/2, [Frame]}, receipt_id(Headers), State) TransactionId ->
add_action(TransactionId,
{fun ?MODULE:handle_recv_ack_frame/2, [Frame]},
receipt_id(Headers),
State)
end; end;
%% NACK %% NACK
@ -357,7 +367,11 @@ received(Frame = #stomp_frame{command = <<"ACK">>, headers = Headers}, State) ->
received(Frame = #stomp_frame{command = <<"NACK">>, headers = Headers}, State) -> received(Frame = #stomp_frame{command = <<"NACK">>, headers = Headers}, State) ->
case header(<<"transaction">>, Headers) of case header(<<"transaction">>, Headers) of
undefined -> {ok, handle_recv_nack_frame(Frame, State)}; undefined -> {ok, handle_recv_nack_frame(Frame, State)};
TransactionId -> add_action(TransactionId, {fun ?MODULE:handle_recv_nack_frame/2, [Frame]}, receipt_id(Headers), State) TransactionId ->
add_action(TransactionId,
{fun ?MODULE:handle_recv_nack_frame/2, [Frame]},
receipt_id(Headers),
State)
end; end;
%% BEGIN %% BEGIN
@ -588,9 +602,23 @@ next_ackid() ->
put(ackid, AckId + 1), put(ackid, AckId + 1),
AckId. AckId.
make_mqtt_message(Topic, Headers, Body) -> make_mqtt_message(Topic, Headers, Body,
Msg = emqx_message:make(stomp, Topic, Body), #pstate{
Headers1 = lists:foldl(fun(Key, Headers0) -> conninfo = #{proto_ver := ProtoVer},
clientinfo = #{
protocol := Protocol,
clientid := ClientId,
username := Username,
peerhost := PeerHost}}) ->
Msg = emqx_message:make(
ClientId, ?QOS_0,
Topic, Body, #{},
#{proto_ver => ProtoVer,
protocol => Protocol,
username => Username,
peerhost => PeerHost}),
Headers1 = lists:foldl(
fun(Key, Headers0) ->
proplists:delete(Key, Headers0) proplists:delete(Key, Headers0)
end, Headers, [<<"destination">>, end, Headers, [<<"destination">>,
<<"content-length">>, <<"content-length">>,
@ -611,7 +639,7 @@ handle_recv_send_frame(#stomp_frame{command = <<"SEND">>, headers = Headers, bod
allow -> allow ->
_ = maybe_send_receipt(receipt_id(Headers), State), _ = maybe_send_receipt(receipt_id(Headers), State),
_ = emqx_broker:publish( _ = emqx_broker:publish(
make_mqtt_message(Topic, Headers, iolist_to_binary(Body)) make_mqtt_message(Topic, Headers, iolist_to_binary(Body), State)
), ),
State; State;
deny -> deny ->