Merge pull request #13126 from SergeTupchiy/EMQX-11967-cross-cluster-route-replication-syncer-actor

cluster linking
This commit is contained in:
SergeTupchiy 2024-06-17 18:52:46 +03:00 committed by GitHub
commit 830266b4d5
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
41 changed files with 4732 additions and 77 deletions

View File

@ -67,7 +67,7 @@
-record(route, { -record(route, {
topic :: binary(), topic :: binary(),
dest :: node() | {binary(), node()} | emqx_session:session_id() dest :: node() | {binary(), node()} | emqx_session:session_id() | emqx_external_broker:dest()
}). }).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------

View File

@ -244,11 +244,24 @@ publish(Msg) when is_record(Msg, message) ->
topic => Topic topic => Topic
}), }),
[]; [];
Msg1 = #message{topic = Topic} -> Msg1 = #message{} ->
PersistRes = persist_publish(Msg1), do_publish(Msg1);
route(aggre(emqx_router:match_routes(Topic)), delivery(Msg1), PersistRes) Msgs when is_list(Msgs) ->
do_publish_many(Msgs)
end. end.
do_publish_many([]) ->
[];
do_publish_many([Msg | T]) ->
do_publish(Msg) ++ do_publish_many(T).
do_publish(#message{topic = Topic} = Msg) ->
PersistRes = persist_publish(Msg),
Routes = aggre(emqx_router:match_routes(Topic)),
Delivery = delivery(Msg),
RouteRes = route(Routes, Delivery, PersistRes),
do_forward_external(Delivery, RouteRes).
persist_publish(Msg) -> persist_publish(Msg) ->
case emqx_persistent_message:persist(Msg) of case emqx_persistent_message:persist(Msg) of
ok -> ok ->
@ -332,6 +345,9 @@ aggre([], false, Acc) ->
aggre([], true, Acc) -> aggre([], true, Acc) ->
lists:usort(Acc). lists:usort(Acc).
do_forward_external(Delivery, RouteRes) ->
emqx_external_broker:forward(Delivery) ++ RouteRes.
%% @doc Forward message to another node. %% @doc Forward message to another node.
-spec forward( -spec forward(
node(), emqx_types:topic() | emqx_types:share(), emqx_types:delivery(), RpcMode :: sync | async node(), emqx_types:topic() | emqx_types:share(), emqx_types:delivery(), RpcMode :: sync | async
@ -643,19 +659,27 @@ maybe_delete_route(Topic) ->
sync_route(Action, Topic, ReplyTo) -> sync_route(Action, Topic, ReplyTo) ->
EnabledOn = emqx_config:get([broker, routing, batch_sync, enable_on]), EnabledOn = emqx_config:get([broker, routing, batch_sync, enable_on]),
case EnabledOn of Res =
all -> case EnabledOn of
push_sync_route(Action, Topic, ReplyTo); all ->
none -> push_sync_route(Action, Topic, ReplyTo);
regular_sync_route(Action, Topic); none ->
Role -> regular_sync_route(Action, Topic);
case Role =:= mria_config:whoami() of Role ->
true -> case Role =:= mria_config:whoami() of
push_sync_route(Action, Topic, ReplyTo); true ->
false -> push_sync_route(Action, Topic, ReplyTo);
regular_sync_route(Action, Topic) false ->
end regular_sync_route(Action, Topic)
end. end
end,
_ = external_sync_route(Action, Topic),
Res.
external_sync_route(add, Topic) ->
emqx_external_broker:add_route(Topic);
external_sync_route(delete, Topic) ->
emqx_external_broker:delete_route(Topic).
push_sync_route(Action, Topic, Opts) -> push_sync_route(Action, Topic, Opts) ->
emqx_router_syncer:push(Action, Topic, node(), Opts). emqx_router_syncer:push(Action, Topic, node(), Opts).

View File

@ -47,7 +47,7 @@ init([]) ->
router_syncer_pool, router_syncer_pool,
hash, hash,
PoolSize, PoolSize,
{emqx_router_syncer, start_link, []} {emqx_router_syncer, start_link_pooled, []}
]), ]),
%% Shared subscription %% Shared subscription

View File

@ -0,0 +1,140 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%
%% Licensed under the Apache License, Version 2.0 (the "License");
%% you may not use this file except in compliance with the License.
%% You may obtain a copy of the License at
%%
%% http://www.apache.org/licenses/LICENSE-2.0
%%
%% Unless required by applicable law or agreed to in writing, software
%% distributed under the License is distributed on an "AS IS" BASIS,
%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
%% See the License for the specific language governing permissions and
%% limitations under the License.
%%--------------------------------------------------------------------
-module(emqx_external_broker).
-callback forward(emqx_types:delivery()) ->
emqx_types:publish_result().
-callback add_route(emqx_types:topic()) -> ok.
-callback delete_route(emqx_types:topic()) -> ok.
-callback add_shared_route(emqx_types:topic(), emqx_types:group()) -> ok.
-callback delete_shared_route(emqx_types:topic(), emqx_types:group()) -> ok.
-callback add_persistent_route(emqx_types:topic(), emqx_persistent_session_ds:id()) -> ok.
-callback delete_persistent_route(emqx_types:topic(), emqx_persistent_session_ds:id()) -> ok.
-type dest() :: term().
-export([
%% Registration
provider/0,
register_provider/1,
unregister_provider/1,
%% Forwarding
forward/1,
%% Routing updates
add_route/1,
delete_route/1,
add_shared_route/2,
delete_shared_route/2,
add_persistent_route/2,
delete_persistent_route/2
]).
-export_type([dest/0]).
-include("logger.hrl").
-define(PROVIDER, {?MODULE, external_broker}).
-define(safe_with_provider(IfRegistered, IfNotRegistered),
case persistent_term:get(?PROVIDER, undefined) of
undefined ->
IfNotRegistered;
Provider ->
try
Provider:IfRegistered
catch
Err:Reason:St ->
?SLOG_THROTTLE(error, #{
msg => external_broker_crashed,
provider => Provider,
callback => ?FUNCTION_NAME,
stacktrace => St,
error => Err,
reason => Reason
}),
{error, Reason}
end
end
).
%% TODO: provider API copied from emqx_external_traces,
%% but it can be moved to a common module.
%%--------------------------------------------------------------------
%% Provider API
%%--------------------------------------------------------------------
-spec register_provider(module()) -> ok | {error, term()}.
register_provider(Module) when is_atom(Module) ->
case is_valid_provider(Module) of
true ->
persistent_term:put(?PROVIDER, Module);
false ->
{error, invalid_provider}
end.
-spec unregister_provider(module()) -> ok | {error, term()}.
unregister_provider(Module) ->
case persistent_term:get(?PROVIDER, undefined) of
Module ->
persistent_term:erase(?PROVIDER),
ok;
_ ->
{error, not_registered}
end.
-spec provider() -> module() | undefined.
provider() ->
persistent_term:get(?PROVIDER, undefined).
%%--------------------------------------------------------------------
%% Broker API
%%--------------------------------------------------------------------
forward(Delivery) ->
?safe_with_provider(?FUNCTION_NAME(Delivery), []).
add_route(Topic) ->
?safe_with_provider(?FUNCTION_NAME(Topic), ok).
delete_route(Topic) ->
?safe_with_provider(?FUNCTION_NAME(Topic), ok).
add_shared_route(Topic, Group) ->
?safe_with_provider(?FUNCTION_NAME(Topic, Group), ok).
delete_shared_route(Topic, Group) ->
?safe_with_provider(?FUNCTION_NAME(Topic, Group), ok).
add_persistent_route(Topic, ID) ->
?safe_with_provider(?FUNCTION_NAME(Topic, ID), ok).
delete_persistent_route(Topic, ID) ->
?safe_with_provider(?FUNCTION_NAME(Topic, ID), ok).
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
is_valid_provider(Module) ->
lists:all(
fun({F, A}) -> erlang:function_exported(Module, F, A) end,
?MODULE:behaviour_info(callbacks)
).

View File

@ -92,6 +92,7 @@ on_subscribe(TopicFilter, SubOpts, #{id := SessionId, s := S0, props := Props})
case emqx_persistent_session_ds_state:n_subscriptions(S0) < MaxSubscriptions of case emqx_persistent_session_ds_state:n_subscriptions(S0) < MaxSubscriptions of
true -> true ->
ok = emqx_persistent_session_ds_router:do_add_route(TopicFilter, SessionId), ok = emqx_persistent_session_ds_router:do_add_route(TopicFilter, SessionId),
_ = emqx_external_broker:add_persistent_route(TopicFilter, SessionId),
{SubId, S1} = emqx_persistent_session_ds_state:new_id(S0), {SubId, S1} = emqx_persistent_session_ds_state:new_id(S0),
{SStateId, S2} = emqx_persistent_session_ds_state:new_id(S1), {SStateId, S2} = emqx_persistent_session_ds_state:new_id(S1),
SState = #{ SState = #{
@ -154,6 +155,7 @@ on_unsubscribe(SessionId, TopicFilter, S0) ->
#{session_id => SessionId, topic_filter => TopicFilter}, #{session_id => SessionId, topic_filter => TopicFilter},
ok = emqx_persistent_session_ds_router:do_delete_route(TopicFilter, SessionId) ok = emqx_persistent_session_ds_router:do_delete_route(TopicFilter, SessionId)
), ),
_ = emqx_external_broker:delete_persistent_route(TopicFilter, SessionId),
{ok, emqx_persistent_session_ds_state:del_subscription(TopicFilter, S0), Subscription} {ok, emqx_persistent_session_ds_state:del_subscription(TopicFilter, S0), Subscription}
end. end.

View File

@ -107,7 +107,14 @@
unused = [] :: nil() unused = [] :: nil()
}). }).
-define(node_patterns(Node), [Node, {'_', Node}]). -define(dest_patterns(NodeOrExtDest),
case is_atom(NodeOrExtDest) of
%% node
true -> [NodeOrExtDest, {'_', NodeOrExtDest}];
%% external destination
false -> [NodeOrExtDest]
end
).
-define(UNSUPPORTED, unsupported). -define(UNSUPPORTED, unsupported).
@ -307,13 +314,13 @@ print_routes(Topic) ->
). ).
-spec cleanup_routes(node()) -> ok. -spec cleanup_routes(node()) -> ok.
cleanup_routes(Node) -> cleanup_routes(NodeOrExtDest) ->
cleanup_routes(get_schema_vsn(), Node). cleanup_routes(get_schema_vsn(), NodeOrExtDest).
cleanup_routes(v2, Node) -> cleanup_routes(v2, NodeOrExtDest) ->
cleanup_routes_v2(Node); cleanup_routes_v2(NodeOrExtDest);
cleanup_routes(v1, Node) -> cleanup_routes(v1, NodeOrExtDest) ->
cleanup_routes_v1(Node). cleanup_routes_v1(NodeOrExtDest).
-spec foldl_routes(fun((emqx_types:route(), Acc) -> Acc), Acc) -> Acc. -spec foldl_routes(fun((emqx_types:route(), Acc) -> Acc), Acc) -> Acc.
foldl_routes(FoldFun, AccIn) -> foldl_routes(FoldFun, AccIn) ->
@ -430,19 +437,19 @@ has_route_v1(Topic, Dest) ->
has_route_tab_entry(Topic, Dest) -> has_route_tab_entry(Topic, Dest) ->
[] =/= ets:match(?ROUTE_TAB, #route{topic = Topic, dest = Dest}). [] =/= ets:match(?ROUTE_TAB, #route{topic = Topic, dest = Dest}).
cleanup_routes_v1(Node) -> cleanup_routes_v1(NodeOrExtDest) ->
?with_fallback( ?with_fallback(
lists:foreach( lists:foreach(
fun(Pattern) -> fun(Pattern) ->
throw_unsupported(mria:match_delete(?ROUTE_TAB, make_route_rec_pat(Pattern))) throw_unsupported(mria:match_delete(?ROUTE_TAB, make_route_rec_pat(Pattern)))
end, end,
?node_patterns(Node) ?dest_patterns(NodeOrExtDest)
), ),
cleanup_routes_v1_fallback(Node) cleanup_routes_v1_fallback(NodeOrExtDest)
). ).
cleanup_routes_v1_fallback(Node) -> cleanup_routes_v1_fallback(NodeOrExtDest) ->
Patterns = [make_route_rec_pat(P) || P <- ?node_patterns(Node)], Patterns = [make_route_rec_pat(P) || P <- ?dest_patterns(NodeOrExtDest)],
mria:transaction(?ROUTE_SHARD, fun() -> mria:transaction(?ROUTE_SHARD, fun() ->
[ [
mnesia:delete_object(?ROUTE_TAB, Route, write) mnesia:delete_object(?ROUTE_TAB, Route, write)
@ -525,7 +532,7 @@ has_route_v2(Topic, Dest) ->
has_route_tab_entry(Topic, Dest) has_route_tab_entry(Topic, Dest)
end. end.
cleanup_routes_v2(Node) -> cleanup_routes_v2(NodeOrExtDest) ->
?with_fallback( ?with_fallback(
lists:foreach( lists:foreach(
fun(Pattern) -> fun(Pattern) ->
@ -537,18 +544,18 @@ cleanup_routes_v2(Node) ->
), ),
throw_unsupported(mria:match_delete(?ROUTE_TAB, make_route_rec_pat(Pattern))) throw_unsupported(mria:match_delete(?ROUTE_TAB, make_route_rec_pat(Pattern)))
end, end,
?node_patterns(Node) ?dest_patterns(NodeOrExtDest)
), ),
cleanup_routes_v2_fallback(Node) cleanup_routes_v2_fallback(NodeOrExtDest)
). ).
cleanup_routes_v2_fallback(Node) -> cleanup_routes_v2_fallback(NodeOrExtDest) ->
%% NOTE %% NOTE
%% No point in transaction here because all the operations on filters table are dirty. %% No point in transaction here because all the operations on filters table are dirty.
ok = ets:foldl( ok = ets:foldl(
fun(#routeidx{entry = K}, ok) -> fun(#routeidx{entry = K}, ok) ->
case get_dest_node(emqx_topic_index:get_id(K)) of case get_dest_node(emqx_topic_index:get_id(K)) of
Node -> NodeOrExtDest ->
mria:dirty_delete(?ROUTE_TAB_FILTERS, K); mria:dirty_delete(?ROUTE_TAB_FILTERS, K);
_ -> _ ->
ok ok
@ -560,7 +567,7 @@ cleanup_routes_v2_fallback(Node) ->
ok = ets:foldl( ok = ets:foldl(
fun(#route{dest = Dest} = Route, ok) -> fun(#route{dest = Dest} = Route, ok) ->
case get_dest_node(Dest) of case get_dest_node(Dest) of
Node -> NodeOrExtDest ->
mria:dirty_delete_object(?ROUTE_TAB, Route); mria:dirty_delete_object(?ROUTE_TAB, Route);
_ -> _ ->
ok ok
@ -570,6 +577,8 @@ cleanup_routes_v2_fallback(Node) ->
?ROUTE_TAB ?ROUTE_TAB
). ).
get_dest_node({external, _} = ExtDest) ->
ExtDest;
get_dest_node({_, Node}) -> get_dest_node({_, Node}) ->
Node; Node;
get_dest_node(Node) -> get_dest_node(Node) ->

View File

@ -21,11 +21,17 @@
-behaviour(gen_server). -behaviour(gen_server).
-export([start_link/1]).
-export([start_link/2]). -export([start_link/2]).
-export([start_link_pooled/2]).
-export([push/4]). -export([push/4]).
-export([push/5]).
-export([wait/1]). -export([wait/1]).
-export([suspend/1]).
-export([activate/1]).
-export([stats/0]). -export([stats/0]).
-export([ -export([
@ -38,6 +44,15 @@
-type action() :: add | delete. -type action() :: add | delete.
-type options() :: #{
max_batch_size => pos_integer(),
min_sync_interval => non_neg_integer(),
error_delay => non_neg_integer(),
error_retry_interval => non_neg_integer(),
initial_state => activated | suspended,
batch_handler => {module(), _Function :: atom(), _Args :: list()}
}.
-define(POOL, router_syncer_pool). -define(POOL, router_syncer_pool).
-define(MAX_BATCH_SIZE, 1000). -define(MAX_BATCH_SIZE, 1000).
@ -77,13 +92,23 @@
%% %%
-spec start_link(atom(), pos_integer()) -> -spec start_link(options()) ->
{ok, pid()} | {error, _Reason}.
start_link(Options) ->
gen_server:start_link(?MODULE, mk_state(Options), []).
-spec start_link(_Name, options()) ->
{ok, pid()} | {error, _Reason}.
start_link(Name, Options) ->
gen_server:start_link(Name, ?MODULE, mk_state(Options), []).
-spec start_link_pooled(atom(), pos_integer()) ->
{ok, pid()}. {ok, pid()}.
start_link(Pool, Id) -> start_link_pooled(Pool, Id) ->
gen_server:start_link( gen_server:start_link(
{local, emqx_utils:proc_name(?MODULE, Id)}, {local, emqx_utils:proc_name(?MODULE, Id)},
?MODULE, ?MODULE,
[Pool, Id], {Pool, Id, mk_state(#{})},
[] []
). ).
@ -93,9 +118,16 @@ when
Opts :: #{reply => pid()}. Opts :: #{reply => pid()}.
push(Action, Topic, Dest, Opts) -> push(Action, Topic, Dest, Opts) ->
Worker = gproc_pool:pick_worker(?POOL, Topic), Worker = gproc_pool:pick_worker(?POOL, Topic),
push(Worker, Action, Topic, Dest, Opts).
-spec push(_Ref, action(), emqx_types:topic(), emqx_router:dest(), Opts) ->
ok | _WaitRef :: reference()
when
Opts :: #{reply => pid()}.
push(Ref, Action, Topic, Dest, Opts) ->
Prio = designate_prio(Action, Opts), Prio = designate_prio(Action, Opts),
Context = mk_push_context(Opts), Context = mk_push_context(Opts),
_ = erlang:send(Worker, ?PUSH(Prio, {Action, Topic, Dest, Context})), _ = gproc:send(Ref, ?PUSH(Prio, {Action, Topic, Dest, Context})),
case Context of case Context of
[{MRef, _}] -> [{MRef, _}] ->
MRef; MRef;
@ -134,6 +166,16 @@ mk_push_context(_) ->
%% %%
%% Suspended syncer receives and accumulates route ops but doesn't apply them
%% until it is activated.
suspend(Ref) ->
gen_server:call(Ref, suspend, infinity).
activate(Ref) ->
gen_server:call(Ref, activate, infinity).
%%
-type stats() :: #{ -type stats() :: #{
size := non_neg_integer(), size := non_neg_integer(),
n_add := non_neg_integer(), n_add := non_neg_integer(),
@ -149,10 +191,34 @@ stats() ->
%% %%
init([Pool, Id]) -> mk_state(Options) ->
true = gproc_pool:connect_worker(Pool, {Pool, Id}), #{
{ok, #{stash => stash_new()}}. state => maps:get(initial_state, Options, active),
stash => stash_new(),
retry_timer => undefined,
max_batch_size => maps:get(max_batch_size, Options, ?MAX_BATCH_SIZE),
min_sync_interval => maps:get(min_sync_interval, Options, ?MIN_SYNC_INTERVAL),
error_delay => maps:get(error_delay, Options, ?ERROR_DELAY),
error_retry_interval => maps:get(error_retry_interval, Options, ?ERROR_RETRY_INTERVAL),
batch_handler => maps:get(batch_handler, Options, default)
}.
%%
init({Pool, Id, State}) ->
true = gproc_pool:connect_worker(Pool, {Pool, Id}),
{ok, State};
init(State) ->
{ok, State}.
handle_call(suspend, _From, State) ->
NState = State#{state := suspended},
{reply, ok, NState};
handle_call(activate, _From, State = #{state := suspended}) ->
NState = run_batch_loop([], State#{state := active}),
{reply, ok, NState};
handle_call(activate, _From, State) ->
{reply, ok, State};
handle_call(stats, _From, State = #{stash := Stash}) -> handle_call(stats, _From, State = #{stash := Stash}) ->
{reply, stash_stats(Stash), State}; {reply, stash_stats(Stash), State};
handle_call(_Call, _From, State) -> handle_call(_Call, _From, State) ->
@ -162,11 +228,11 @@ handle_cast(_Msg, State) ->
{noreply, State}. {noreply, State}.
handle_info({timeout, _TRef, retry}, State) -> handle_info({timeout, _TRef, retry}, State) ->
NState = run_batch_loop([], maps:remove(retry_timer, State)), NState = run_batch_loop([], State#{retry_timer := undefined}),
{noreply, NState}; {noreply, NState};
handle_info(Push = ?PUSH(_, _), State) -> handle_info(Push = ?PUSH(_, _), State = #{min_sync_interval := MSI}) ->
%% NOTE: Wait a bit to collect potentially overlapping operations. %% NOTE: Wait a bit to collect potentially overlapping operations.
ok = timer:sleep(?MIN_SYNC_INTERVAL), ok = timer:sleep(MSI),
NState = run_batch_loop([Push], State), NState = run_batch_loop([Push], State),
{noreply, NState}. {noreply, NState}.
@ -175,12 +241,16 @@ terminate(_Reason, _State) ->
%% %%
run_batch_loop(Incoming, State = #{stash := Stash0}) -> run_batch_loop(Incoming, State = #{stash := Stash0, state := suspended}) ->
Stash1 = stash_add(Incoming, Stash0), Stash1 = stash_add(Incoming, Stash0),
Stash2 = stash_drain(Stash1), Stash2 = stash_drain(Stash1),
{Batch, Stash3} = mk_batch(Stash2), State#{stash := Stash2};
run_batch_loop(Incoming, State = #{stash := Stash0, max_batch_size := MBS}) ->
Stash1 = stash_add(Incoming, Stash0),
Stash2 = stash_drain(Stash1),
{Batch, Stash3} = mk_batch(Stash2, MBS),
?tp_ignore_side_effects_in_prod(router_syncer_new_batch, batch_stats(Batch, Stash3)), ?tp_ignore_side_effects_in_prod(router_syncer_new_batch, batch_stats(Batch, Stash3)),
case run_batch(Batch) of case run_batch(Batch, State) of
Status = #{} -> Status = #{} ->
ok = send_replies(Status, Batch), ok = send_replies(Status, Batch),
NState = cancel_retry_timer(State#{stash := Stash3}), NState = cancel_retry_timer(State#{stash := Stash3}),
@ -203,37 +273,37 @@ run_batch_loop(Incoming, State = #{stash := Stash0}) ->
batch => batch_stats(Batch, Stash3) batch => batch_stats(Batch, Stash3)
}), }),
NState = State#{stash := Stash2}, NState = State#{stash := Stash2},
ok = timer:sleep(?ERROR_DELAY), ok = error_cooldown(NState),
ensure_retry_timer(NState) ensure_retry_timer(NState)
end. end.
error_cooldown(#{error_delay := ED}) ->
timer:sleep(ED).
ensure_retry_timer(State = #{retry_timer := undefined, error_retry_interval := ERI}) ->
TRef = emqx_utils:start_timer(ERI, retry),
State#{retry_timer := TRef};
ensure_retry_timer(State = #{retry_timer := _TRef}) -> ensure_retry_timer(State = #{retry_timer := _TRef}) ->
State; State.
ensure_retry_timer(State) ->
TRef = emqx_utils:start_timer(?ERROR_RETRY_INTERVAL, retry),
State#{retry_timer => TRef}.
cancel_retry_timer(State = #{retry_timer := TRef}) -> cancel_retry_timer(State = #{retry_timer := TRef}) ->
ok = emqx_utils:cancel_timer(TRef), ok = emqx_utils:cancel_timer(TRef),
maps:remove(retry_timer, State); State#{retry_timer := undefined};
cancel_retry_timer(State) -> cancel_retry_timer(State) ->
State. State.
%% %%
mk_batch(Stash) when map_size(Stash) =< ?MAX_BATCH_SIZE -> mk_batch(Stash, BatchSize) when map_size(Stash) =< BatchSize ->
%% This is perfect situation, we just use stash as batch w/o extra reallocations. %% This is perfect situation, we just use stash as batch w/o extra reallocations.
{Stash, stash_new()}; {Stash, stash_new()};
mk_batch(Stash) -> mk_batch(Stash, BatchSize) ->
%% Take a subset of stashed operations to form a batch. %% Take a subset of stashed operations to form a batch.
%% Note that stash is an unordered map, it's not a queue. The order of operations is %% Note that stash is an unordered map, it's not a queue. The order of operations is
%% not preserved strictly, only loosely, because of how we start from high priority %% not preserved strictly, only loosely, because of how we start from high priority
%% operations and go down to low priority ones. This might cause some operations to %% operations and go down to low priority ones. This might cause some operations to
%% stay in stash for unfairly long time, when there are many high priority operations. %% stay in stash for unfairly long time, when there are many high priority operations.
%% However, it's unclear how likely this is to happen in practice. %% However, it's unclear how likely this is to happen in practice.
mk_batch(Stash, ?MAX_BATCH_SIZE).
mk_batch(Stash, BatchSize) ->
mk_batch(?PRIO_HI, #{}, BatchSize, Stash). mk_batch(?PRIO_HI, #{}, BatchSize, Stash).
mk_batch(Prio, Batch, SizeLeft, Stash) -> mk_batch(Prio, Batch, SizeLeft, Stash) ->
@ -278,10 +348,12 @@ replyctx_send(Result, RefsPids) ->
%% %%
run_batch(Batch) when map_size(Batch) > 0 -> run_batch(Empty, _State) when Empty =:= #{} ->
#{};
run_batch(Batch, #{batch_handler := default}) ->
catch emqx_router:do_batch(Batch); catch emqx_router:do_batch(Batch);
run_batch(_Empty) -> run_batch(Batch, #{batch_handler := {Module, Function, Args}}) ->
#{}. erlang:apply(Module, Function, [Batch | Args]).
%% %%

View File

@ -421,8 +421,12 @@ init_monitors() ->
handle_call({subscribe, Group, Topic, SubPid}, _From, State = #state{pmon = PMon}) -> handle_call({subscribe, Group, Topic, SubPid}, _From, State = #state{pmon = PMon}) ->
mria:dirty_write(?SHARED_SUBSCRIPTION, record(Group, Topic, SubPid)), mria:dirty_write(?SHARED_SUBSCRIPTION, record(Group, Topic, SubPid)),
case ets:member(?SHARED_SUBSCRIBER, {Group, Topic}) of case ets:member(?SHARED_SUBSCRIBER, {Group, Topic}) of
true -> ok; true ->
false -> ok = emqx_router:do_add_route(Topic, {Group, node()}) ok;
false ->
ok = emqx_router:do_add_route(Topic, {Group, node()}),
_ = emqx_external_broker:add_shared_route(Topic, Group),
ok
end, end,
ok = maybe_insert_alive_tab(SubPid), ok = maybe_insert_alive_tab(SubPid),
ok = maybe_insert_round_robin_count({Group, Topic}), ok = maybe_insert_round_robin_count({Group, Topic}),
@ -545,7 +549,9 @@ is_alive_sub(Pid) ->
delete_route_if_needed({Group, Topic} = GroupTopic) -> delete_route_if_needed({Group, Topic} = GroupTopic) ->
if_no_more_subscribers(GroupTopic, fun() -> if_no_more_subscribers(GroupTopic, fun() ->
ok = emqx_router:do_delete_route(Topic, {Group, node()}) ok = emqx_router:do_delete_route(Topic, {Group, node()}),
_ = emqx_external_broker:delete_shared_route(Topic, Group),
ok
end). end).
get_default_shared_subscription_strategy() -> get_default_shared_subscription_strategy() ->

View File

@ -33,7 +33,8 @@
feed_var/3, feed_var/3,
systop/1, systop/1,
parse/1, parse/1,
parse/2 parse/2,
intersection/2
]). ]).
-export([ -export([
@ -52,6 +53,8 @@
((C =:= '#' orelse C =:= <<"#">>) andalso REST =/= []) ((C =:= '#' orelse C =:= <<"#">>) andalso REST =/= [])
). ).
-define(IS_WILDCARD(W), W =:= '+' orelse W =:= '#').
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% APIs %% APIs
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
@ -98,6 +101,55 @@ match(_, ['#']) ->
match(_, _) -> match(_, _) ->
false. false.
%% @doc Finds an intersection between two topics, two filters or a topic and a filter.
%% The function is commutative: reversing parameters doesn't affect the returned value.
%% Two topics intersect only when they are equal.
%% The intersection of a topic and a filter is always either the topic itself or false (no intersection).
%% The intersection of two filters is either false or a new topic filter that would match only those topics,
%% that can be matched by both input filters.
%% For example, the intersection of "t/global/#" and "t/+/1/+" is "t/global/1/+".
-spec intersection(TopicOrFilter, TopicOrFilter) -> TopicOrFilter | false when
TopicOrFilter :: emqx_types:topic().
intersection(Topic1, Topic2) when is_binary(Topic1), is_binary(Topic2) ->
case intersect_start(words(Topic1), words(Topic2)) of
false -> false;
Intersection -> join(Intersection)
end.
intersect_start([<<"$", _/bytes>> | _], [W | _]) when ?IS_WILDCARD(W) ->
false;
intersect_start([W | _], [<<"$", _/bytes>> | _]) when ?IS_WILDCARD(W) ->
false;
intersect_start(Words1, Words2) ->
intersect(Words1, Words2).
intersect(Words1, ['#']) ->
Words1;
intersect(['#'], Words2) ->
Words2;
intersect([W1], ['+']) ->
[W1];
intersect(['+'], [W2]) ->
[W2];
intersect([W1 | T1], [W2 | T2]) when ?IS_WILDCARD(W1), ?IS_WILDCARD(W2) ->
intersect_join(wildcard_intersection(W1, W2), intersect(T1, T2));
intersect([W | T1], [W | T2]) ->
intersect_join(W, intersect(T1, T2));
intersect([W1 | T1], [W2 | T2]) when ?IS_WILDCARD(W1) ->
intersect_join(W2, intersect(T1, T2));
intersect([W1 | T1], [W2 | T2]) when ?IS_WILDCARD(W2) ->
intersect_join(W1, intersect(T1, T2));
intersect([], []) ->
[];
intersect(_, _) ->
false.
intersect_join(_, false) -> false;
intersect_join(W, Words) -> [W | Words].
wildcard_intersection(W, W) -> W;
wildcard_intersection(_, _) -> '+'.
-spec match_share(Name, Filter) -> boolean() when -spec match_share(Name, Filter) -> boolean() when
Name :: share(), Name :: share(),
Filter :: topic() | share(). Filter :: topic() | share().

View File

@ -23,6 +23,7 @@
-export([delete/3]). -export([delete/3]).
-export([match/2]). -export([match/2]).
-export([matches/3]). -export([matches/3]).
-export([matches_filter/3]).
-export([make_key/2]). -export([make_key/2]).
@ -72,6 +73,12 @@ match(Topic, Tab) ->
matches(Topic, Tab, Opts) -> matches(Topic, Tab, Opts) ->
emqx_trie_search:matches(Topic, make_nextf(Tab), Opts). emqx_trie_search:matches(Topic, make_nextf(Tab), Opts).
%% @doc Match given topic filter against the index and return _all_ matches.
%% If `unique` option is given, return only unique matches by record ID.
-spec matches_filter(emqx_types:topic(), ets:table(), emqx_trie_search:opts()) -> [match(_ID)].
matches_filter(TopicFilter, Tab, Opts) ->
emqx_trie_search:matches_filter(TopicFilter, make_nextf(Tab), Opts).
%% @doc Extract record ID from the match. %% @doc Extract record ID from the match.
-spec get_id(match(ID)) -> ID. -spec get_id(match(ID)) -> ID.
get_id(Key) -> get_id(Key) ->

View File

@ -99,7 +99,7 @@
-module(emqx_trie_search). -module(emqx_trie_search).
-export([make_key/2, make_pat/2, filter/1]). -export([make_key/2, make_pat/2, filter/1]).
-export([match/2, matches/3, get_id/1, get_topic/1]). -export([match/2, matches/3, get_id/1, get_topic/1, matches_filter/3]).
-export_type([key/1, word/0, words/0, nextf/0, opts/0]). -export_type([key/1, word/0, words/0, nextf/0, opts/0]).
-define(END, '$end_of_table'). -define(END, '$end_of_table').
@ -183,9 +183,20 @@ match(Topic, NextF) ->
matches(Topic, NextF, Opts) -> matches(Topic, NextF, Opts) ->
search(Topic, NextF, Opts). search(Topic, NextF, Opts).
%% @doc Match given topic filter against the index and return _all_ matches.
-spec matches_filter(emqx_types:topic(), nextf(), opts()) -> [key(_)].
matches_filter(TopicFilter, NextF, Opts) ->
search(TopicFilter, NextF, [topic_filter | Opts]).
%% @doc Entrypoint of the search for a given topic. %% @doc Entrypoint of the search for a given topic.
search(Topic, NextF, Opts) -> search(Topic, NextF, Opts) ->
Words = topic_words(Topic), %% A private opt
IsFilter = proplists:get_bool(topic_filter, Opts),
Words =
case IsFilter of
true -> filter_words(Topic);
false -> topic_words(Topic)
end,
Base = base_init(Words), Base = base_init(Words),
ORetFirst = proplists:get_bool(return_first, Opts), ORetFirst = proplists:get_bool(return_first, Opts),
OUnique = proplists:get_bool(unique, Opts), OUnique = proplists:get_bool(unique, Opts),
@ -200,8 +211,10 @@ search(Topic, NextF, Opts) ->
end, end,
Matches = Matches =
case search_new(Words, Base, NextF, Acc0) of case search_new(Words, Base, NextF, Acc0) of
{Cursor, Acc} -> {Cursor, Acc} when not IsFilter ->
match_topics(Topic, Cursor, NextF, Acc); match_topics(Topic, Cursor, NextF, Acc);
{_Cursor, Acc} ->
Acc;
Acc -> Acc ->
Acc Acc
end, end,
@ -275,6 +288,17 @@ compare(['#'], _Words, _) ->
% Closest possible next entries that we must not miss: % Closest possible next entries that we must not miss:
% * a/+/+/d/# (same topic but a different ID) % * a/+/+/d/# (same topic but a different ID)
match_full; match_full;
%% Filter search %%
compare(_Filter, ['#'], _) ->
match_full;
compare([_ | TF], ['+' | TW], Pos) ->
case compare(TF, TW, Pos + 1) of
lower ->
lower;
Other ->
Other
end;
%% Filter search end %%
compare(['+' | TF], [HW | TW], Pos) -> compare(['+' | TF], [HW | TW], Pos) ->
case compare(TF, TW, Pos + 1) of case compare(TF, TW, Pos + 1) of
lower -> lower ->

View File

@ -267,6 +267,7 @@
[ [
{node(), topic(), deliver_result()} {node(), topic(), deliver_result()}
| {share, topic(), deliver_result()} | {share, topic(), deliver_result()}
| {emqx_external_broker:dest(), topic(), deliver_result()}
| persisted | persisted
] ]
| disconnect. | disconnect.

View File

@ -158,7 +158,7 @@ wait_clustered([Node | Nodes] = All, Check, Deadline) ->
nodes_not_running => NodesNotRunnging nodes_not_running => NodesNotRunnging
}} }}
); );
{false, Nodes} -> {false, _Nodes} ->
timer:sleep(100), timer:sleep(100),
wait_clustered(All, Check, Deadline) wait_clustered(All, Check, Deadline)
end. end.

View File

@ -28,6 +28,7 @@
[ [
wildcard/1, wildcard/1,
match/2, match/2,
intersection/2,
validate/1, validate/1,
prepend/2, prepend/2,
join/1, join/1,
@ -128,6 +129,63 @@ t_match_perf(_) ->
true = match(Name, Filter), true = match(Name, Filter),
ok = bench('match/2', fun emqx_topic:match/2, [Name, Filter]). ok = bench('match/2', fun emqx_topic:match/2, [Name, Filter]).
t_intersect(_) ->
<<"t/global/1/+">> = intersection(<<"t/global/#">>, <<"t/+/1/+">>),
<<"t/global/#">> = intersection(<<"t/global/#">>, <<"#">>),
<<"t/global/#">> = intersection(<<"t/global/#">>, <<"t/global/#">>),
<<"1/2/3/4/5">> = intersection(<<"1/+/3/+/5/#">>, <<"+/2/+/4/+">>),
<<"t/local/1">> = intersection(<<"t/local/1/#">>, <<"t/local/+">>),
false = intersection(<<"t/global/#">>, <<"t/local/+">>),
false = intersection(<<"t/local/1/+">>, <<"t/local/+">>).
t_intersect_topic_wildcard(_) ->
<<"t/test/1">> = intersection(<<"t/test/#">>, <<"t/test/1">>),
<<"t/test/1/1">> = intersection(<<"t/test/1/1">>, <<"t/test/#">>),
false = intersection(<<"t/test/1/1">>, <<"t/test/+">>),
<<"t/test/1/1">> = intersection(<<"t/test/1/1">>, <<"t/test/1/1">>),
false = intersection(<<"t/test/1">>, <<"t/test/2">>),
false = intersection(<<"t/test/1">>, <<"t/test/1/2">>).
t_intersect_commutes(_) ->
?assertEqual(
intersection(<<"t/+/1/+">>, <<"t/global/#">>),
intersection(<<"t/global/#">>, <<"t/+/1/+">>)
),
?assertEqual(
intersection(<<"#">>, <<"t/global/#">>),
intersection(<<"t/global/#">>, <<"#">>)
),
?assertEqual(
intersection(<<"+/2/+/4/+">>, <<"1/+/3/+/5/#">>),
intersection(<<"1/+/3/+/5/#">>, <<"+/2/+/4/+">>)
),
?assertEqual(
intersection(<<"t/local/+">>, <<"t/local/1/#">>),
intersection(<<"t/local/1/#">>, <<"t/local/+">>)
),
?assertEqual(
intersection(<<"t/local/+">>, <<"t/global/#">>),
intersection(<<"t/global/#">>, <<"t/local/+">>)
),
?assertEqual(
intersection(<<"t/local/+">>, <<"t/local/1/+">>),
intersection(<<"t/local/1/+">>, <<"t/local/+">>)
),
?assertEqual(
intersection(<<"t/test/#">>, <<"t/test/1/1">>),
intersection(<<"t/test/1/1">>, <<"t/test/#">>)
),
?assertEqual(
intersection(<<"t/test/+">>, <<"t/test/1/1">>),
intersection(<<"t/test/1/1">>, <<"t/test/+">>)
).
t_sys_intersect(_) ->
<<"$SYS/broker/+">> = intersection(<<"$SYS/broker/#">>, <<"$SYS/+/+">>),
<<"$SYS/broker">> = intersection(<<"$SYS/broker">>, <<"$SYS/+">>),
false = intersection(<<"$SYS/broker">>, <<"+/+">>),
false = intersection(<<"$SYS/broker">>, <<"#">>).
t_validate(_) -> t_validate(_) ->
true = validate(<<"a/+/#">>), true = validate(<<"a/+/#">>),
true = validate(<<"a/b/c/d">>), true = validate(<<"a/b/c/d">>),

View File

@ -0,0 +1,94 @@
Business Source License 1.1
Licensor: Hangzhou EMQ Technologies Co., Ltd.
Licensed Work: EMQX Enterprise Edition
The Licensed Work is (c) 2024
Hangzhou EMQ Technologies Co., Ltd.
Additional Use Grant: Students and educators are granted right to copy,
modify, and create derivative work for research
or education.
Change Date: 2028-04-17
Change License: Apache License, Version 2.0
For information about alternative licensing arrangements for the Software,
please contact Licensor: https://www.emqx.com/en/contact
Notice
The Business Source License (this document, or the “License”) is not an Open
Source license. However, the Licensed Work will eventually be made available
under an Open Source License, as stated in this License.
License text copyright (c) 2017 MariaDB Corporation Ab, All Rights Reserved.
“Business Source License” is a trademark of MariaDB Corporation Ab.
-----------------------------------------------------------------------------
Business Source License 1.1
Terms
The Licensor hereby grants you the right to copy, modify, create derivative
works, redistribute, and make non-production use of the Licensed Work. The
Licensor may make an Additional Use Grant, above, permitting limited
production use.
Effective on the Change Date, or the fourth anniversary of the first publicly
available distribution of a specific version of the Licensed Work under this
License, whichever comes first, the Licensor hereby grants you rights under
the terms of the Change License, and the rights granted in the paragraph
above terminate.
If your use of the Licensed Work does not comply with the requirements
currently in effect as described in this License, you must purchase a
commercial license from the Licensor, its affiliated entities, or authorized
resellers, or you must refrain from using the Licensed Work.
All copies of the original and modified Licensed Work, and derivative works
of the Licensed Work, are subject to this License. This License applies
separately for each version of the Licensed Work and the Change Date may vary
for each version of the Licensed Work released by Licensor.
You must conspicuously display this License on each original or modified copy
of the Licensed Work. If you receive the Licensed Work in original or
modified form from a third party, the terms and conditions set forth in this
License apply to your use of that work.
Any use of the Licensed Work in violation of this License will automatically
terminate your rights under this License for the current and all other
versions of the Licensed Work.
This License does not grant you any right in any trademark or logo of
Licensor or its affiliates (provided that you may use a trademark or logo of
Licensor as expressly required by this License).
TO THE EXTENT PERMITTED BY APPLICABLE LAW, THE LICENSED WORK IS PROVIDED ON
AN “AS IS” BASIS. LICENSOR HEREBY DISCLAIMS ALL WARRANTIES AND CONDITIONS,
EXPRESS OR IMPLIED, INCLUDING (WITHOUT LIMITATION) WARRANTIES OF
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE, NON-INFRINGEMENT, AND
TITLE.
MariaDB hereby grants you permission to use this Licenses text to license
your works, and to refer to it using the trademark “Business Source License”,
as long as you comply with the Covenants of Licensor below.
Covenants of Licensor
In consideration of the right to use this Licenses text and the “Business
Source License” name and trademark, Licensor covenants to MariaDB, and to all
other recipients of the licensed work to be provided by Licensor:
1. To specify as the Change License the GPL Version 2.0 or any later version,
or a license that is compatible with GPL Version 2.0 or a later version,
where “compatible” means that software provided under the Change License can
be included in a program with software provided under GPL Version 2.0 or a
later version. Licensor may specify additional Change Licenses without
limitation.
2. To either: (a) specify an additional grant of rights to use that does not
impose any additional restriction on the right granted in this License, as
the Additional Use Grant; or (b) insert the text “None”.
3. To specify a Change Date.
4. Not to modify this License in any other way.

View File

@ -0,0 +1,19 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-define(TOPIC_PREFIX, "$LINK/cluster/").
-define(TOPIC_PREFIX_WILDCARD, <<?TOPIC_PREFIX "#">>).
-define(ROUTE_TOPIC_PREFIX, ?TOPIC_PREFIX "route/").
-define(MSG_TOPIC_PREFIX, ?TOPIC_PREFIX "msg/").
-define(RESP_TOPIC_PREFIX, ?TOPIC_PREFIX "resp/").
-define(MY_CLUSTER_NAME, emqx_cluster_link_config:cluster()).
-define(ROUTE_TOPIC, <<?ROUTE_TOPIC_PREFIX, (?MY_CLUSTER_NAME)/binary>>).
-define(MSG_FWD_TOPIC, <<?MSG_TOPIC_PREFIX, (?MY_CLUSTER_NAME)/binary>>).
-define(RESP_TOPIC(Actor), <<?RESP_TOPIC_PREFIX, (?MY_CLUSTER_NAME)/binary, "/", Actor/binary>>).
%% Fairly compact text encoding.
-define(SHARED_ROUTE_ID(Topic, Group), <<"$s/", Group/binary, "/", Topic/binary>>).
-define(PERSISTENT_ROUTE_ID(Topic, ID), <<"$p/", ID/binary, "/", Topic/binary>>).

View File

@ -0,0 +1,8 @@
%% -*- mode: erlang; -*-
{erl_opts, [debug_info]}.
{deps, [
{emqx, {path, "../../apps/emqx"}},
{emqx_resource, {path, "../../apps/emqx_resource"}}
]}.

View File

@ -0,0 +1,23 @@
%% -*- mode: erlang -*-
{application, emqx_cluster_link, [
{description, "EMQX Cluster Linking"},
% strict semver, bump manually!
{vsn, "0.1.0"},
{modules, []},
{registered, []},
{applications, [
kernel,
stdlib,
emqtt,
emqx,
emqx_resource
]},
{mod, {emqx_cluster_link_app, []}},
{env, []},
{licenses, ["Business Source License 1.1"]},
{maintainers, ["EMQX Team <contact@emqx.io>"]},
{links, [
{"Homepage", "https://emqx.io/"},
{"Github", "https://github.com/emqx/emqx"}
]}
]}.

View File

@ -0,0 +1,255 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link).
-behaviour(emqx_external_broker).
-export([
is_registered/0,
register_external_broker/0,
unregister_external_broker/0,
add_route/1,
delete_route/1,
add_shared_route/2,
delete_shared_route/2,
add_persistent_route/2,
delete_persistent_route/2,
forward/1
]).
%% emqx hooks
-export([
put_hook/0,
delete_hook/0,
on_message_publish/1
]).
-include("emqx_cluster_link.hrl").
-include_lib("emqx/include/emqx.hrl").
-include_lib("emqx/include/emqx_hooks.hrl").
-include_lib("emqx/include/logger.hrl").
%%--------------------------------------------------------------------
%% emqx_external_broker API
%%--------------------------------------------------------------------
is_registered() ->
emqx_external_broker:provider() =:= ?MODULE.
register_external_broker() ->
case is_registered() of
true -> ok;
false -> emqx_external_broker:register_provider(?MODULE)
end.
unregister_external_broker() ->
emqx_external_broker:unregister_provider(?MODULE).
%% Using original Topic as Route ID in the most common scenario:
%% (non-shared, non-persistent routes).
%% Original Topic is used to identify the route and be able
%% to delete it on a remote cluster.
%% There is no need to push Node name as this info can be derived from
%% agent state on the remote cluster.
add_route(Topic) ->
maybe_push_route_op(add, Topic, Topic).
delete_route(Topic) ->
maybe_push_route_op(delete, Topic, Topic).
add_shared_route(Topic, Group) ->
maybe_push_route_op(add, Topic, ?SHARED_ROUTE_ID(Topic, Group)).
delete_shared_route(Topic, Group) ->
maybe_push_route_op(delete, Topic, ?SHARED_ROUTE_ID(Topic, Group)).
add_persistent_route(Topic, ID) ->
maybe_push_route_op(add, Topic, ?PERSISTENT_ROUTE_ID(Topic, ID), push_persistent_route).
delete_persistent_route(Topic, ID) ->
maybe_push_route_op(delete, Topic, ?PERSISTENT_ROUTE_ID(Topic, ID), push_persistent_route).
forward(#delivery{message = #message{extra = #{link_origin := _}}}) ->
%% Do not forward any external messages to other links.
%% Only forward locally originated messages to all the relevant links, i.e. no gossip
%% message forwarding.
[];
forward(Delivery = #delivery{message = #message{topic = Topic}}) ->
Routes = emqx_cluster_link_extrouter:match_routes(Topic),
forward(Routes, Delivery).
forward([], _Delivery) ->
[];
forward(Routes, Delivery) ->
lists:foldl(
fun(#route{topic = To, dest = Cluster}, Acc) ->
Result = emqx_cluster_link_mqtt:forward(Cluster, Delivery),
[{Cluster, To, Result} | Acc]
end,
[],
Routes
).
%%--------------------------------------------------------------------
%% EMQX Hooks
%%--------------------------------------------------------------------
on_message_publish(
#message{topic = <<?ROUTE_TOPIC_PREFIX, ClusterName/binary>>, payload = Payload} = Msg
) ->
case emqx_cluster_link_mqtt:decode_route_op(Payload) of
{actor_init, Actor, InitInfo} ->
Result = actor_init(ClusterName, Actor, InitInfo),
_ = actor_init_ack(Actor, Result, Msg),
ok;
{route_updates, #{actor := Actor}, RouteOps} ->
ok = update_routes(ClusterName, Actor, RouteOps);
{heartbeat, #{actor := Actor}} ->
ok = actor_heartbeat(ClusterName, Actor);
{error, {unknown_payload, ParsedPayload}} ->
?SLOG(warning, #{
msg => "unexpected_cluster_link_route_op_payload",
payload => ParsedPayload
})
end,
{stop, []};
on_message_publish(#message{topic = <<?MSG_TOPIC_PREFIX, ClusterName/binary>>, payload = Payload}) ->
case emqx_cluster_link_mqtt:decode_forwarded_msg(Payload) of
#message{} = ForwardedMsg ->
{stop, maybe_filter_incomming_msg(ForwardedMsg, ClusterName)};
_Err ->
%% Just ignore it. It must be already logged by the decoder
{stop, []}
end;
on_message_publish(_Msg) ->
ok.
put_hook() ->
emqx_hooks:put('message.publish', {?MODULE, on_message_publish, []}, ?HP_SYS_MSGS).
delete_hook() ->
emqx_hooks:del('message.publish', {?MODULE, on_message_publish, []}).
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
-define(PD_EXTROUTER_ACTOR, '$clink_extrouter_actor').
-define(PD_EXTROUTER_ACTOR_STATE, '$clink_extrouter_actor_state').
maybe_push_route_op(Op, Topic, RouteID) ->
maybe_push_route_op(Op, Topic, RouteID, push).
maybe_push_route_op(Op, Topic, RouteID, PushFun) ->
lists:foreach(
fun(#{name := Cluster, topics := LinkFilters}) ->
case topic_intersect_any(Topic, LinkFilters) of
false ->
ok;
TopicIntersection ->
emqx_cluster_link_router_syncer:PushFun(Cluster, Op, TopicIntersection, RouteID)
end
end,
emqx_cluster_link_config:enabled_links()
).
topic_intersect_any(Topic, [LinkFilter | T]) ->
case emqx_topic:intersection(Topic, LinkFilter) of
false -> topic_intersect_any(Topic, T);
TopicOrFilter -> TopicOrFilter
end;
topic_intersect_any(_Topic, []) ->
false.
actor_init(
ClusterName,
#{actor := Actor, incarnation := Incr},
#{
target_cluster := TargetCluster,
proto_ver := _
}
) ->
case emqx_cluster_link_config:link(ClusterName) of
undefined ->
?SLOG(error, #{
msg => "init_link_request_from_unknown_cluster",
link_name => ClusterName
}),
%% Avoid atom error reasons, since they can be sent to the remote cluster,
%% which will use safe binary_to_term decoding
%% TODO: add error details?
{error, <<"unknown_cluster">>};
#{enable := true} = _LinkConf ->
MyClusterName = emqx_cluster_link_config:cluster(),
case MyClusterName of
TargetCluster ->
Env = #{timestamp => erlang:system_time(millisecond)},
{ok, ActorSt} = emqx_cluster_link_extrouter:actor_init(
ClusterName, Actor, Incr, Env
),
undefined = set_actor_state(ClusterName, Actor, ActorSt),
ok;
_ ->
%% The remote cluster uses a different name to refer to this cluster
?SLOG(error, #{
msg => "misconfigured_cluster_link_name",
%% How this cluster names itself
local_name => MyClusterName,
%% How the remote cluster names this local cluster
remote_name => TargetCluster,
%% How the remote cluster names itself
received_from => ClusterName
}),
{error, <<"bad_remote_cluster_link_name">>}
end;
#{enable := false} ->
{error, <<"cluster_link_disabled">>}
end.
actor_init_ack(#{actor := Actor}, Res, MsgIn) ->
RespMsg = emqx_cluster_link_mqtt:actor_init_ack_resp_msg(Actor, Res, MsgIn),
emqx_broker:publish(RespMsg).
update_routes(ClusterName, Actor, RouteOps) ->
ActorSt = get_actor_state(ClusterName, Actor),
lists:foreach(
fun(RouteOp) ->
_ = emqx_cluster_link_extrouter:actor_apply_operation(RouteOp, ActorSt)
end,
RouteOps
).
actor_heartbeat(ClusterName, Actor) ->
Env = #{timestamp => erlang:system_time(millisecond)},
ActorSt0 = get_actor_state(ClusterName, Actor),
ActorSt = emqx_cluster_link_extrouter:actor_apply_operation(heartbeat, ActorSt0, Env),
_ = update_actor_state(ActorSt),
ok.
get_actor_state(ClusterName, Actor) ->
{ClusterName, Actor} = erlang:get(?PD_EXTROUTER_ACTOR),
erlang:get(?PD_EXTROUTER_ACTOR_STATE).
set_actor_state(ClusterName, Actor, ActorSt) ->
_Undefined = erlang:put(?PD_EXTROUTER_ACTOR, {ClusterName, Actor}),
update_actor_state(ActorSt).
update_actor_state(ActorSt) ->
erlang:put(?PD_EXTROUTER_ACTOR_STATE, ActorSt).
%% let it crash if extra is not a map,
%% we don't expect the message to be forwarded from an older EMQX release,
%% that doesn't set extra = #{} by default.
with_sender_name(#message{extra = Extra} = Msg, ClusterName) when is_map(Extra) ->
Msg#message{extra = Extra#{link_origin => ClusterName}}.
maybe_filter_incomming_msg(#message{topic = T} = Msg, ClusterName) ->
%% Should prevent irrelevant messages from being dispatched in case
%% the remote routing state lags behind the local config changes.
#{enable := Enable, topics := Topics} = emqx_cluster_link_config:link(ClusterName),
case Enable andalso emqx_topic:match_any(T, Topics) of
true -> with_sender_name(Msg, ClusterName);
false -> []
end.

View File

@ -0,0 +1,116 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_api).
-behaviour(minirest_api).
-include_lib("hocon/include/hoconsc.hrl").
-include_lib("emqx/include/http_api.hrl").
-export([
api_spec/0,
paths/0,
schema/1
]).
-export([config/2]).
-define(CONF_PATH, [cluster, links]).
-define(TAGS, [<<"Cluster">>]).
api_spec() ->
emqx_dashboard_swagger:spec(?MODULE, #{check_schema => true}).
paths() ->
[
"/cluster/links"
].
schema("/cluster/links") ->
#{
'operationId' => config,
get =>
#{
description => "Get cluster links configuration",
tags => ?TAGS,
responses =>
#{200 => links_config_schema()}
},
put =>
#{
description => "Update cluster links configuration",
tags => ?TAGS,
'requestBody' => links_config_schema(),
responses =>
#{
200 => links_config_schema(),
400 =>
emqx_dashboard_swagger:error_codes(
[?BAD_REQUEST], <<"Update Config Failed">>
)
}
}
}.
%%--------------------------------------------------------------------
%% API Handler funcs
%%--------------------------------------------------------------------
config(get, _Params) ->
{200, get_raw()};
config(put, #{body := Body}) ->
case emqx_cluster_link_config:update(Body) of
{ok, NewConfig} ->
{200, NewConfig};
{error, Reason} ->
Message = list_to_binary(io_lib:format("Update config failed ~p", [Reason])),
{400, ?BAD_REQUEST, Message}
end.
%%--------------------------------------------------------------------
%% Internal funcs
%%--------------------------------------------------------------------
get_raw() ->
#{<<"links">> := Conf} =
emqx_config:fill_defaults(
#{<<"links">> => emqx_conf:get_raw(?CONF_PATH)},
#{obfuscate_sensitive_values => true}
),
Conf.
links_config_schema() ->
emqx_cluster_link_schema:links_schema(
#{
examples => #{<<"example">> => links_config_example()}
}
).
links_config_example() ->
[
#{
<<"enable">> => true,
<<"pool_size">> => 10,
<<"server">> => <<"emqxcl_b.host:1883">>,
<<"ssl">> => #{<<"enable">> => false},
<<"topics">> =>
[
<<"t/topic-example">>,
<<"t/topic-filter-example/1/#">>
],
<<"name">> => <<"emqxcl_b">>
},
#{
<<"enable">> => true,
<<"pool_size">> => 10,
<<"server">> => <<"emqxcl_c.host:1883">>,
<<"ssl">> => #{<<"enable">> => false},
<<"topics">> =>
[
<<"t/topic-example">>,
<<"t/topic-filter-example/1/#">>
],
<<"name">> => <<"emqxcl_c">>
}
].

View File

@ -0,0 +1,55 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_app).
-behaviour(application).
-export([start/2, prep_stop/1, stop/1]).
-define(BROKER_MOD, emqx_cluster_link).
start(_StartType, _StartArgs) ->
ok = mria:wait_for_tables(emqx_cluster_link_extrouter:create_tables()),
emqx_cluster_link_config:add_handler(),
LinksConf = emqx_cluster_link_config:enabled_links(),
case LinksConf of
[_ | _] ->
ok = emqx_cluster_link:register_external_broker(),
ok = emqx_cluster_link:put_hook(),
ok = start_msg_fwd_resources(LinksConf);
_ ->
ok
end,
emqx_cluster_link_sup:start_link(LinksConf).
prep_stop(State) ->
emqx_cluster_link_config:remove_handler(),
State.
stop(_State) ->
_ = emqx_cluster_link:delete_hook(),
_ = emqx_cluster_link:unregister_external_broker(),
_ = remove_msg_fwd_resources(emqx_cluster_link_config:links()),
ok.
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
start_msg_fwd_resources(LinksConf) ->
lists:foreach(
fun(LinkConf) ->
{ok, _} = emqx_cluster_link_mqtt:ensure_msg_fwd_resource(LinkConf)
end,
LinksConf
).
remove_msg_fwd_resources(LinksConf) ->
lists:foreach(
fun(#{name := Name}) ->
emqx_cluster_link_mqtt:remove_msg_fwd_resource(Name)
end,
LinksConf
).

View File

@ -0,0 +1,322 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_config).
-behaviour(emqx_config_handler).
-include_lib("emqx/include/logger.hrl").
-define(LINKS_PATH, [cluster, links]).
-define(CERTS_PATH(LinkName), filename:join(["cluster", "links", LinkName])).
-define(MQTT_HOST_OPTS, #{default_port => 1883}).
-ifndef(TEST).
-define(DEFAULT_ACTOR_TTL, 30_000).
-else.
-define(DEFAULT_ACTOR_TTL, 3_000).
-endif.
-define(COMMON_FIELDS, [username, password, clientid, server, ssl]).
%% NOTE: retry_interval, max_inflight may be used for router syncer client as well,
%% but for now they are not.
-define(MSG_RES_FIELDS, [resource_opts, pool_size, retry_interval, max_inflight]).
%% Excludes a special hidden `ps_actor_incarnation` field.
-define(ACTOR_FIELDS, [topics]).
-export([
%% General
update/1,
cluster/0,
enabled_links/0,
links/0,
link/1,
topic_filters/1,
%% Connections
emqtt_options/1,
mk_emqtt_options/1,
%% Actor Lifecycle
actor_ttl/0,
actor_gc_interval/0,
actor_heartbeat_interval/0
]).
-export([
add_handler/0,
remove_handler/0
]).
-export([
pre_config_update/3,
post_config_update/5
]).
%%
update(Config) ->
case
emqx_conf:update(
?LINKS_PATH,
Config,
#{rawconf_with_defaults => true, override_to => cluster}
)
of
{ok, #{raw_config := NewConfigRows}} ->
{ok, NewConfigRows};
{error, Reason} ->
{error, Reason}
end.
cluster() ->
atom_to_binary(emqx_config:get([cluster, name])).
links() ->
emqx:get_config(?LINKS_PATH, []).
enabled_links() ->
[L || L = #{enable := true} <- links()].
link(Name) ->
case lists:dropwhile(fun(L) -> Name =/= upstream_name(L) end, links()) of
[LinkConf | _] -> LinkConf;
[] -> undefined
end.
emqtt_options(LinkName) ->
emqx_maybe:apply(fun mk_emqtt_options/1, ?MODULE:link(LinkName)).
topic_filters(LinkName) ->
maps:get(topics, ?MODULE:link(LinkName), []).
-spec actor_ttl() -> _Milliseconds :: pos_integer().
actor_ttl() ->
?DEFAULT_ACTOR_TTL.
-spec actor_gc_interval() -> _Milliseconds :: pos_integer().
actor_gc_interval() ->
actor_ttl().
-spec actor_heartbeat_interval() -> _Milliseconds :: pos_integer().
actor_heartbeat_interval() ->
actor_ttl() div 3.
%%
mk_emqtt_options(#{server := Server, ssl := #{enable := EnableSsl} = Ssl} = LinkConf) ->
ClientId = maps:get(clientid, LinkConf, cluster()),
#{hostname := Host, port := Port} = emqx_schema:parse_server(Server, ?MQTT_HOST_OPTS),
Opts = maps:with([username, retry_interval, max_inflight], LinkConf),
Opts1 = Opts#{
host => Host,
port => Port,
clientid => ClientId,
proto_ver => v5,
ssl => EnableSsl,
ssl_opts => maps:to_list(maps:remove(enable, Ssl))
},
with_password(Opts1, LinkConf).
with_password(Opts, #{password := P} = _LinkConf) ->
Opts#{password => emqx_secret:unwrap(P)};
with_password(Opts, _LinkConf) ->
Opts.
%%
add_handler() ->
ok = emqx_config_handler:add_handler(?LINKS_PATH, ?MODULE).
remove_handler() ->
ok = emqx_config_handler:remove_handler(?LINKS_PATH).
pre_config_update(?LINKS_PATH, RawConf, RawConf) ->
{ok, RawConf};
pre_config_update(?LINKS_PATH, NewRawConf, OldRawConf) ->
{ok, convert_certs(maybe_increment_ps_actor_incr(NewRawConf, OldRawConf))}.
post_config_update(?LINKS_PATH, _Req, Old, Old, _AppEnvs) ->
ok;
post_config_update(?LINKS_PATH, _Req, New, Old, _AppEnvs) ->
ok = toggle_hook_and_broker(enabled_links(New), enabled_links(Old)),
#{
removed := Removed,
added := Added,
changed := Changed
} = emqx_utils:diff_lists(New, Old, fun upstream_name/1),
RemovedRes = remove_links(Removed),
AddedRes = add_links(Added),
UpdatedRes = update_links(Changed),
IsAllOk = all_ok(RemovedRes) andalso all_ok(AddedRes) andalso all_ok(UpdatedRes),
case IsAllOk of
true ->
ok;
false ->
{error, #{added => AddedRes, removed => RemovedRes, updated => UpdatedRes}}
end.
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
toggle_hook_and_broker([_ | _] = _NewEnabledLinks, [] = _OldEnabledLinks) ->
ok = emqx_cluster_link:register_external_broker(),
ok = emqx_cluster_link:put_hook();
toggle_hook_and_broker([] = _NewEnabledLinks, _OldLinks) ->
_ = emqx_cluster_link:unregister_external_broker(),
ok = emqx_cluster_link:delete_hook();
toggle_hook_and_broker(_, _) ->
ok.
enabled_links(LinksConf) ->
[L || #{enable := true} = L <- LinksConf].
all_ok(Results) ->
lists:all(
fun
(ok) -> true;
({ok, _}) -> true;
(_) -> false
end,
Results
).
add_links(LinksConf) ->
[add_link(Link) || Link <- LinksConf].
add_link(#{enable := true} = LinkConf) ->
{ok, _Pid} = emqx_cluster_link_sup:ensure_actor(LinkConf),
{ok, _} = emqx_cluster_link_mqtt:ensure_msg_fwd_resource(LinkConf),
ok;
add_link(_DisabledLinkConf) ->
ok.
remove_links(LinksConf) ->
[remove_link(Name) || #{name := Name} <- LinksConf].
remove_link(Name) ->
_ = emqx_cluster_link_mqtt:remove_msg_fwd_resource(Name),
ensure_actor_stopped(Name).
update_links(LinksConf) ->
[update_link(Link) || Link <- LinksConf].
update_link({OldLinkConf, #{enable := true, name := Name} = NewLinkConf}) ->
case what_is_changed(OldLinkConf, NewLinkConf) of
both ->
_ = ensure_actor_stopped(Name),
{ok, _Pid} = emqx_cluster_link_sup:ensure_actor(NewLinkConf),
ok = update_msg_fwd_resource(OldLinkConf, NewLinkConf);
actor ->
_ = ensure_actor_stopped(Name),
{ok, _Pid} = emqx_cluster_link_sup:ensure_actor(NewLinkConf),
ok;
msg_resource ->
ok = update_msg_fwd_resource(OldLinkConf, NewLinkConf)
end;
update_link({_OldLinkConf, #{enable := false, name := Name} = _NewLinkConf}) ->
_ = emqx_cluster_link_mqtt:remove_msg_fwd_resource(Name),
ensure_actor_stopped(Name).
what_is_changed(OldLink, NewLink) ->
CommonChanged = are_fields_changed(?COMMON_FIELDS, OldLink, NewLink),
ActorChanged = are_fields_changed(?ACTOR_FIELDS, OldLink, NewLink),
MsgResChanged = are_fields_changed(?MSG_RES_FIELDS, OldLink, NewLink),
AllChanged = ActorChanged andalso MsgResChanged,
case CommonChanged orelse AllChanged of
true ->
both;
false ->
%% This function is only applicable when it's certain that link conf is changed,
%% so if resource fields are the same,
%% then some other actor-related fields are definitely changed.
case MsgResChanged of
true -> msg_resource;
false -> actor
end
end.
are_fields_changed(Fields, OldLink, NewLink) ->
maps:with(Fields, OldLink) =/= maps:with(Fields, NewLink).
update_msg_fwd_resource(_, #{name := Name} = NewConf) ->
_ = emqx_cluster_link_mqtt:remove_msg_fwd_resource(Name),
{ok, _} = emqx_cluster_link_mqtt:ensure_msg_fwd_resource(NewConf),
ok.
ensure_actor_stopped(ClusterName) ->
emqx_cluster_link_sup:ensure_actor_stopped(ClusterName).
upstream_name(#{name := N}) -> N;
upstream_name(#{<<"name">> := N}) -> N.
maybe_increment_ps_actor_incr(New, Old) ->
case emqx_persistent_message:is_persistence_enabled() of
true ->
%% TODO: what if a link was removed and then added again?
%% Assume that incarnation was 0 when the link was removed
%% and now it's also 0 (a default value for new actor).
%% If persistent routing state changed during this link absence
%% and remote GC has not started before ps actor restart (with the same incarnation),
%% then some old (stale) external ps routes may be never cleaned on the remote side.
%% No (permanent) message loss is expected, as new actor incrantaion will re-bootstrap.
%% Similarly, irrelevant messages will be filtered out at receiving end, so
%% the main risk is having some stale routes unreachable for GC...
#{changed := Changed} = emqx_utils:diff_lists(New, Old, fun upstream_name/1),
ChangedNames = [upstream_name(C) || {_, C} <- Changed],
lists:foldr(
fun(LConf, Acc) ->
case lists:member(upstream_name(LConf), ChangedNames) of
true -> [increment_ps_actor_incr(LConf) | Acc];
false -> [LConf | Acc]
end
end,
[],
New
);
false ->
New
end.
increment_ps_actor_incr(#{ps_actor_incarnation := Incr} = Conf) ->
Conf#{ps_actor_incarnation => Incr + 1};
increment_ps_actor_incr(#{<<"ps_actor_incarnation">> := Incr} = Conf) ->
Conf#{<<"ps_actor_incarnation">> => Incr + 1};
%% Default value set in schema is 0, so need to set it to 1 during the first update.
increment_ps_actor_incr(#{<<"name">> := _} = Conf) ->
Conf#{<<"ps_actor_incarnation">> => 1};
increment_ps_actor_incr(#{name := _} = Conf) ->
Conf#{ps_actor_incarnation => 1}.
convert_certs(LinksConf) ->
lists:map(
fun
(#{ssl := SSLOpts} = LinkConf) ->
LinkConf#{ssl => do_convert_certs(upstream_name(LinkConf), SSLOpts)};
(#{<<"ssl">> := SSLOpts} = LinkConf) ->
LinkConf#{<<"ssl">> => do_convert_certs(upstream_name(LinkConf), SSLOpts)};
(LinkConf) ->
LinkConf
end,
LinksConf
).
do_convert_certs(LinkName, SSLOpts) ->
case emqx_tls_lib:ensure_ssl_files(?CERTS_PATH(LinkName), SSLOpts) of
{ok, undefined} ->
SSLOpts;
{ok, SSLOpts1} ->
SSLOpts1;
{error, Reason} ->
?SLOG(
error,
#{
msg => "bad_ssl_config",
config_path => ?LINKS_PATH,
name => LinkName,
reason => Reason
}
),
throw({bad_ssl_config, Reason})
end.

View File

@ -0,0 +1,413 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_extrouter).
-include_lib("snabbkaffe/include/trace.hrl").
-export([create_tables/0]).
%% Router API
-export([
match_routes/1,
lookup_routes/1,
topics/0
]).
%% Actor API
-export([
actor_init/4,
actor_state/3,
actor_apply_operation/2,
actor_apply_operation/3,
actor_gc/1,
is_present_incarnation/1,
list_actors/1
]).
%% Internal API
-export([
mnesia_actor_init/4,
mnesia_actor_heartbeat/3,
mnesia_clean_incarnation/1,
apply_actor_operation/5
]).
%% Strictly monotonically increasing integer.
-type smint() :: integer().
%% Remote cluster name
-type cluster() :: binary().
%% Actor.
%% Identifies an independent route replication actor on the remote broker.
%% Usually something like `node()` or `{node(), _Shard}`.
-type actor() :: term().
%% Identifies incarnation of the actor.
%% In the event of actor restart, it's the actor's responsibility to keep track of
%% monotonicity of its incarnation number. Each time actor's incarnation increases,
%% we assume that all the state of the previous incarnations is lost.
-type incarnation() :: smint().
%% Operation.
%% RouteID should come in handy when two or more different routes on the actor side
%% are "intersected" to the same topic filter that needs to be replicated here.
-type op() :: {add | delete, {_TopicFilter :: binary(), _RouteID}} | heartbeat.
%% Basically a bit offset.
%% Each actor + incarnation pair occupies a separate lane in the multi-counter.
%% Example:
%% Actors | n1@ds n2@ds n3@ds
%% Lanes | 0 1 2
%% ---------------------------
%% Op1 | n3@ds add client/42/# MCounter += 1 bsl 2 = 4
%% Op2 | n2@ds add client/42/# MCounter += 1 bsl 1 = 6
%% Op3 | n3@ds delete client/42/# MCounter -= 1 bsl 2 = 2
%% Op4 | n2@ds delete client/42/# MCounter -= 1 bsl 1 = 0 route deleted
-type lane() :: non_neg_integer().
-include_lib("emqx/include/emqx.hrl").
-define(EXTROUTE_SHARD, ?MODULE).
-define(EXTROUTE_TAB, emqx_external_router_route).
-define(EXTROUTE_ACTOR_TAB, emqx_external_router_actor).
-define(ACTOR_ID(Cluster, Actor), {Cluster, Actor}).
-define(ROUTE_ID(Cluster, RouteID), {Cluster, RouteID}).
-record(extroute, {
entry :: emqx_topic_index:key(_RouteID),
mcounter = 0 :: non_neg_integer()
}).
-record(actor, {
id :: {cluster(), actor()},
incarnation :: incarnation(),
lane :: lane(),
until :: _Timestamp
}).
%%
create_tables() ->
%% TODO: Table per link viable?
mria_config:set_dirty_shard(?EXTROUTE_SHARD, true),
ok = mria:create_table(?EXTROUTE_ACTOR_TAB, [
{type, set},
{rlog_shard, ?EXTROUTE_SHARD},
{storage, ram_copies},
{record_name, actor},
{attributes, record_info(fields, actor)}
]),
ok = mria:create_table(?EXTROUTE_TAB, [
{type, ordered_set},
{rlog_shard, ?EXTROUTE_SHARD},
{storage, ram_copies},
{record_name, extroute},
{attributes, record_info(fields, extroute)},
{storage_properties, [
{ets, [
{read_concurrency, true},
{write_concurrency, true},
{decentralized_counters, true}
]}
]}
]),
[?EXTROUTE_ACTOR_TAB, ?EXTROUTE_TAB].
%%
-spec match_routes(emqx_types:topic()) -> [emqx_types:route()].
match_routes(Topic) ->
Matches = emqx_topic_index:matches(Topic, ?EXTROUTE_TAB, [unique]),
%% `unique` opt is not enough, since we keep the original Topic as a part of RouteID
lists:ukeysort(#route.dest, [match_to_route(M) || M <- Matches]).
-spec lookup_routes(emqx_types:topic()) -> [emqx_types:route()].
lookup_routes(Topic) ->
Pat = make_extroute_rec_pat(emqx_topic_index:make_key(Topic, '$1')),
[match_to_route(R#extroute.entry) || Records <- ets:match(?EXTROUTE_TAB, Pat), R <- Records].
-spec topics() -> [emqx_types:topic()].
topics() ->
Pat = make_extroute_rec_pat('$1'),
[emqx_topic_index:get_topic(K) || [K] <- ets:match(?EXTROUTE_TAB, Pat)].
match_to_route(M) ->
?ROUTE_ID(Cluster, _) = emqx_topic_index:get_id(M),
#route{topic = emqx_topic_index:get_topic(M), dest = Cluster}.
%% Make Dialyzer happy
make_extroute_rec_pat(Entry) ->
erlang:make_tuple(
record_info(size, extroute),
'_',
[{1, extroute}, {#extroute.entry, Entry}]
).
%%
-record(state, {
cluster :: cluster(),
actor :: actor(),
incarnation :: incarnation(),
lane :: lane() | undefined,
extra = #{} :: map()
}).
-type state() :: #state{}.
-type env() :: #{timestamp => _Milliseconds}.
-spec actor_init(cluster(), actor(), incarnation(), env()) -> {ok, state()}.
actor_init(Cluster, Actor, Incarnation, Env = #{timestamp := Now}) ->
%% TODO: Rolling upgrade safety?
case transaction(fun ?MODULE:mnesia_actor_init/4, [Cluster, Actor, Incarnation, Now]) of
{ok, State} ->
{ok, State};
{reincarnate, Rec} ->
%% TODO: Do this asynchronously.
ok = clean_incarnation(Rec),
actor_init(Cluster, Actor, Incarnation, Env)
end.
-spec is_present_incarnation(state()) -> boolean().
is_present_incarnation(#state{extra = #{is_present_incarnation := IsNew}}) ->
IsNew;
is_present_incarnation(_State) ->
false.
-spec list_actors(cluster()) -> [#{actor := actor(), incarnation := incarnation()}].
list_actors(Cluster) ->
Pat = make_actor_rec_pat([{#actor.id, {Cluster, '$1'}}, {#actor.incarnation, '$2'}]),
Matches = ets:match(emqx_external_router_actor, Pat),
[#{actor => Actor, incarnation => Incr} || [Actor, Incr] <- Matches].
mnesia_actor_init(Cluster, Actor, Incarnation, TS) ->
%% NOTE
%% We perform this heavy-weight transaction only in the case of a new route
%% replication connection. The implicit assumption is that each replication
%% channel is uniquely identified by the ClientID (reflecting the Actor), and
%% the broker will take care of ensuring that there's only one connection per
%% ClientID. There's always a chance of having stray process severely lagging
%% that applies some update out of the blue, but it seems impossible to prevent
%% it completely w/o transactions.
State = #state{cluster = Cluster, actor = Actor, incarnation = Incarnation},
ActorID = ?ACTOR_ID(Cluster, Actor),
case mnesia:read(?EXTROUTE_ACTOR_TAB, ActorID, write) of
[#actor{incarnation = Incarnation, lane = Lane} = Rec] ->
ok = mnesia:write(?EXTROUTE_ACTOR_TAB, Rec#actor{until = bump_actor_ttl(TS)}, write),
{ok, State#state{lane = Lane, extra = #{is_present_incarnation => true}}};
[] ->
Lane = mnesia_assign_lane(Cluster),
Rec = #actor{
id = ActorID,
incarnation = Incarnation,
lane = Lane,
until = bump_actor_ttl(TS)
},
ok = mnesia:write(?EXTROUTE_ACTOR_TAB, Rec, write),
{ok, State#state{lane = Lane, extra = #{is_present_incarnation => false}}};
[#actor{incarnation = Outdated} = Rec] when Incarnation > Outdated ->
{reincarnate, Rec};
[#actor{incarnation = Newer}] ->
mnesia:abort({outdated_incarnation_actor, Actor, Incarnation, Newer})
end.
-spec actor_state(cluster(), actor(), incarnation()) -> state().
actor_state(Cluster, Actor, Incarnation) ->
ActorID = ?ACTOR_ID(Cluster, Actor),
[#actor{lane = Lane}] = mnesia:dirty_read(?EXTROUTE_ACTOR_TAB, ActorID),
#state{cluster = Cluster, actor = Actor, incarnation = Incarnation, lane = Lane}.
-spec actor_apply_operation(op(), state()) -> state().
actor_apply_operation(Op, State) ->
actor_apply_operation(Op, State, #{}).
-spec actor_apply_operation(op(), state(), env()) -> state().
actor_apply_operation(
{OpName, {TopicFilter, ID}},
State = #state{cluster = Cluster, actor = Actor, incarnation = Incarnation, lane = Lane},
_Env
) ->
ActorID = ?ACTOR_ID(Cluster, Actor),
Entry = emqx_topic_index:make_key(TopicFilter, ?ROUTE_ID(Cluster, ID)),
case mria_config:whoami() of
Role when Role /= replicant ->
apply_actor_operation(ActorID, Incarnation, Entry, OpName, Lane);
replicant ->
mria:async_dirty(
?EXTROUTE_SHARD,
fun ?MODULE:apply_actor_operation/5,
[ActorID, Incarnation, Entry, OpName, Lane]
)
end,
State;
actor_apply_operation(
heartbeat,
State = #state{cluster = Cluster, actor = Actor, incarnation = Incarnation},
_Env = #{timestamp := Now}
) ->
ActorID = ?ACTOR_ID(Cluster, Actor),
ok = transaction(fun ?MODULE:mnesia_actor_heartbeat/3, [ActorID, Incarnation, Now]),
State.
apply_actor_operation(ActorID, Incarnation, Entry, OpName, Lane) ->
_ = assert_current_incarnation(ActorID, Incarnation),
apply_operation(Entry, OpName, Lane).
apply_operation(Entry, OpName, Lane) ->
%% NOTE
%% This is safe sequence of operations only on core nodes. On replicants,
%% `mria:dirty_update_counter/3` will be replicated asynchronously, which
%% means this read can be stale.
case mnesia:dirty_read(?EXTROUTE_TAB, Entry) of
[#extroute{mcounter = MCounter}] ->
apply_operation(Entry, MCounter, OpName, Lane);
[] ->
apply_operation(Entry, 0, OpName, Lane)
end.
apply_operation(Entry, MCounter, OpName, Lane) ->
%% NOTE
%% We are relying on the fact that changes to each individual lane of this
%% multi-counter are synchronized. Without this, such counter updates would
%% be unsafe. Instead, we would have to use another, more complex approach,
%% that runs `ets:lookup/2` + `ets:select_replace/2` in a loop until the
%% counter is updated accordingly.
Marker = 1 bsl Lane,
case MCounter band Marker of
0 when OpName =:= add ->
mria:dirty_update_counter(?EXTROUTE_TAB, Entry, Marker);
Marker when OpName =:= add ->
%% Already added.
MCounter;
Marker when OpName =:= delete ->
case mria:dirty_update_counter(?EXTROUTE_TAB, Entry, -Marker) of
0 ->
Record = #extroute{entry = Entry, mcounter = 0},
ok = mria:dirty_delete_object(?EXTROUTE_TAB, Record),
0;
C ->
C
end;
0 when OpName =:= delete ->
%% Already deleted.
MCounter
end.
-spec actor_gc(env()) -> _NumCleaned :: non_neg_integer().
actor_gc(#{timestamp := Now}) ->
Pat = make_actor_rec_pat([{#actor.until, '$1'}]),
MS = [{Pat, [{'<', '$1', Now}], ['$_']}],
Dead = mnesia:dirty_select(?EXTROUTE_ACTOR_TAB, MS),
try_clean_incarnation(Dead).
try_clean_incarnation([Rec | Rest]) ->
%% NOTE: One at a time.
case clean_incarnation(Rec) of
ok ->
1;
stale ->
try_clean_incarnation(Rest)
end;
try_clean_incarnation([]) ->
0.
mnesia_assign_lane(Cluster) ->
Assignment = lists:foldl(
fun(Lane, Acc) -> Acc bor (1 bsl Lane) end,
0,
select_cluster_lanes(Cluster)
),
Lane = first_zero_bit(Assignment),
Lane.
select_cluster_lanes(Cluster) ->
Pat = make_actor_rec_pat([{#actor.id, {Cluster, '_'}}, {#actor.lane, '$1'}]),
MS = [{Pat, [], ['$1']}],
mnesia:select(?EXTROUTE_ACTOR_TAB, MS, write).
%% Make Dialyzer happy
make_actor_rec_pat(PosValues) ->
erlang:make_tuple(
record_info(size, actor),
'_',
[{1, actor} | PosValues]
).
mnesia_actor_heartbeat(ActorID, Incarnation, TS) ->
case mnesia:read(?EXTROUTE_ACTOR_TAB, ActorID, write) of
[#actor{incarnation = Incarnation} = Rec] ->
ok = mnesia:write(?EXTROUTE_ACTOR_TAB, Rec#actor{until = bump_actor_ttl(TS)}, write);
[#actor{incarnation = Outdated}] ->
mnesia:abort({outdated_incarnation_actor, ActorID, Incarnation, Outdated});
[] ->
mnesia:abort({nonexistent_actor, ActorID})
end.
clean_incarnation(Rec = #actor{id = {Cluster, Actor}}) ->
case transaction(fun ?MODULE:mnesia_clean_incarnation/1, [Rec]) of
ok ->
?tp(debug, clink_extrouter_actor_cleaned, #{
cluster => Cluster,
actor => Actor
});
Result ->
Result
end.
mnesia_clean_incarnation(#actor{id = Actor, incarnation = Incarnation, lane = Lane}) ->
case mnesia:read(?EXTROUTE_ACTOR_TAB, Actor, write) of
[#actor{incarnation = Incarnation}] ->
_ = clean_lane(Lane),
mnesia:delete(?EXTROUTE_ACTOR_TAB, Actor, write);
_Renewed ->
stale
end.
clean_lane(Lane) ->
ets:foldl(
fun(#extroute{entry = Entry, mcounter = MCounter}, _) ->
apply_operation(Entry, MCounter, delete, Lane)
end,
0,
?EXTROUTE_TAB
).
assert_current_incarnation(ActorID, Incarnation) ->
%% NOTE
%% Ugly, but should not really happen anyway. This is a safety net for the case
%% when this process tries to apply some outdated operation for whatever reason
%% (e.g. heavy CPU starvation). Still, w/o transactions, it's just a best-effort
%% attempt.
[#actor{incarnation = Incarnation}] = mnesia:dirty_read(?EXTROUTE_ACTOR_TAB, ActorID),
ok.
%%
transaction(Fun, Args) ->
case mria:transaction(?EXTROUTE_SHARD, Fun, Args) of
{atomic, Result} ->
Result;
{aborted, Reason} ->
error(Reason)
end.
%%
first_zero_bit(N) ->
first_zero_bit(N, 0).
first_zero_bit(N, I) ->
case N band 1 of
0 -> I;
_ -> first_zero_bit(N bsr 1, I + 1)
end.
%%
bump_actor_ttl(TS) ->
TS + emqx_cluster_link_config:actor_ttl().

View File

@ -0,0 +1,99 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_extrouter_gc).
-include_lib("emqx/include/logger.hrl").
-export([start_link/0]).
-export([run/0]).
-behaviour(gen_server).
-export([
init/1,
handle_call/3,
handle_cast/2,
handle_info/2
]).
-define(SERVER, ?MODULE).
-ifndef(TEST).
-define(REPEAT_GC_INTERVAL, 5_000).
-else.
-define(REPEAT_GC_INTERVAL, 1_000).
-endif.
%%
start_link() ->
gen_server:start_link({local, ?SERVER}, ?MODULE, [], []).
run() ->
gen_server:call(?SERVER, run).
%%
-record(st, {
gc_timer :: undefined | reference()
}).
init(_) ->
{ok, schedule_gc(#st{})}.
handle_call(run, _From, St) ->
Result = run_gc(),
Timeout = choose_timeout(Result),
{reply, Result, reschedule_gc(Timeout, St)};
handle_call(_Call, _From, St) ->
{reply, ignored, St}.
handle_cast(Cast, State) ->
?SLOG(warning, #{msg => "unexpected_cast", cast => Cast}),
{noreply, State}.
handle_info({timeout, TRef, _GC}, St = #st{gc_timer = TRef}) ->
Result = run_gc_exclusive(),
Timeout = choose_timeout(Result),
{noreply, schedule_gc(Timeout, St#st{gc_timer = undefined})};
handle_info(Info, St) ->
?SLOG(warning, #{msg => "unexpected_info", info => Info}),
{noreply, St}.
%%
run_gc_exclusive() ->
case is_responsible() of
true -> run_gc();
false -> 0
end.
is_responsible() ->
Nodes = lists:sort(mria_membership:running_core_nodelist()),
Nodes =/= [] andalso hd(Nodes) == node().
-spec run_gc() -> _NumCleaned :: non_neg_integer().
run_gc() ->
Env = #{timestamp => erlang:system_time(millisecond)},
emqx_cluster_link_extrouter:actor_gc(Env).
choose_timeout(_NumCleaned = 0) ->
emqx_cluster_link_config:actor_gc_interval();
choose_timeout(_NumCleaned) ->
%% NOTE: There could likely be more outdated actors.
?REPEAT_GC_INTERVAL.
schedule_gc(St) ->
schedule_gc(emqx_cluster_link_config:actor_gc_interval(), St).
schedule_gc(Timeout, St = #st{gc_timer = undefined}) ->
TRef = erlang:start_timer(Timeout, self(), gc),
St#st{gc_timer = TRef}.
reschedule_gc(Timeout, St = #st{gc_timer = undefined}) ->
schedule_gc(Timeout, St);
reschedule_gc(Timeout, St = #st{gc_timer = TRef}) ->
ok = emqx_utils:cancel_timer(TRef),
schedule_gc(Timeout, St#st{gc_timer = undefined}).

View File

@ -0,0 +1,421 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_mqtt).
-include("emqx_cluster_link.hrl").
-include_lib("emqx/include/emqx.hrl").
-include_lib("emqx/include/emqx_mqtt.hrl").
-include_lib("emqx/include/logger.hrl").
-include_lib("snabbkaffe/include/trace.hrl").
-behaviour(emqx_resource).
-behaviour(ecpool_worker).
%% ecpool
-export([connect/1]).
%% callbacks of behaviour emqx_resource
-export([
callback_mode/0,
on_start/2,
on_stop/2,
on_query/3,
on_query_async/4,
on_get_status/2
]).
-export([
ensure_msg_fwd_resource/1,
remove_msg_fwd_resource/1,
decode_route_op/1,
decode_forwarded_msg/1,
decode_resp/1
]).
-export([
publish_actor_init_sync/6,
actor_init_ack_resp_msg/3,
publish_route_sync/4,
publish_heartbeat/3,
encode_field/2
]).
-export([
forward/2
]).
-define(MSG_CLIENTID_SUFFIX, ":msg:").
-define(MQTT_HOST_OPTS, #{default_port => 1883}).
-define(MSG_POOL_PREFIX, "emqx_cluster_link_mqtt:msg:").
-define(RES_NAME(Prefix, ClusterName), <<Prefix, ClusterName/binary>>).
-define(ROUTE_POOL_NAME(ClusterName), ?RES_NAME(?ROUTE_POOL_PREFIX, ClusterName)).
-define(MSG_RES_ID(ClusterName), ?RES_NAME(?MSG_POOL_PREFIX, ClusterName)).
-define(HEALTH_CHECK_TIMEOUT, 1000).
-define(RES_GROUP, <<"emqx_cluster_link">>).
-define(PROTO_VER, 1).
-define(DECODE(Payload), erlang:binary_to_term(Payload, [safe])).
-define(ENCODE(Payload), erlang:term_to_binary(Payload)).
-define(F_OPERATION, '$op').
-define(OP_ROUTE, <<"route">>).
-define(OP_HEARTBEAT, <<"heartbeat">>).
-define(OP_ACTOR_INIT, <<"actor_init">>).
-define(OP_ACTOR_INIT_ACK, <<"actor_init_ack">>).
-define(F_ACTOR, 10).
-define(F_INCARNATION, 11).
-define(F_ROUTES, 12).
-define(F_TARGET_CLUSTER, 13).
-define(F_PROTO_VER, 14).
-define(F_RESULT, 15).
-define(F_NEED_BOOTSTRAP, 16).
-define(ROUTE_DELETE, 100).
-define(PUB_TIMEOUT, 10_000).
-spec ensure_msg_fwd_resource(map()) ->
{ok, emqx_resource:resource_data() | already_started} | {error, Reason :: term()}.
ensure_msg_fwd_resource(#{name := Name, resource_opts := ResOpts} = ClusterConf) ->
ResOpts1 = ResOpts#{
query_mode => async,
start_after_created => true
},
emqx_resource:create_local(?MSG_RES_ID(Name), ?RES_GROUP, ?MODULE, ClusterConf, ResOpts1).
-spec remove_msg_fwd_resource(binary() | map()) -> ok | {error, Reason :: term()}.
remove_msg_fwd_resource(ClusterName) ->
emqx_resource:remove_local(?MSG_RES_ID(ClusterName)).
%%--------------------------------------------------------------------
%% emqx_resource callbacks (message forwarding)
%%--------------------------------------------------------------------
callback_mode() -> async_if_possible.
on_start(ResourceId, #{pool_size := PoolSize} = ClusterConf) ->
PoolName = ResourceId,
Options = [
{name, PoolName},
{pool_size, PoolSize},
{pool_type, hash},
{client_opts, emqtt_client_opts(?MSG_CLIENTID_SUFFIX, ClusterConf)}
],
ok = emqx_resource:allocate_resource(ResourceId, pool_name, PoolName),
case emqx_resource_pool:start(PoolName, ?MODULE, Options) of
ok ->
{ok, #{pool_name => PoolName, topic => ?MSG_FWD_TOPIC}};
{error, {start_pool_failed, _, Reason}} ->
{error, Reason}
end.
on_stop(ResourceId, _State) ->
#{pool_name := PoolName} = emqx_resource:get_allocated_resources(ResourceId),
emqx_resource_pool:stop(PoolName).
on_query(_ResourceId, FwdMsg, #{pool_name := PoolName, topic := LinkTopic} = _State) when
is_record(FwdMsg, message)
->
#message{topic = Topic, qos = QoS} = FwdMsg,
PubResult = ecpool:pick_and_do(
{PoolName, Topic},
fun(ConnPid) ->
emqtt:publish(ConnPid, LinkTopic, ?ENCODE(FwdMsg), QoS)
end,
no_handover
),
?tp_ignore_side_effects_in_prod(clink_message_forwarded, #{
pool => PoolName,
message => FwdMsg,
pub_result => PubResult
}),
handle_send_result(PubResult).
on_query_async(
_ResourceId, FwdMsg, CallbackIn, #{pool_name := PoolName, topic := LinkTopic} = _State
) ->
Callback = {fun on_async_result/2, [CallbackIn]},
#message{topic = Topic, qos = QoS} = FwdMsg,
%% TODO check message ordering, pick by topic,client pair?
ecpool:pick_and_do(
{PoolName, Topic},
fun(ConnPid) ->
%% #delivery{} record has no valuable data for a remote link...
Payload = ?ENCODE(FwdMsg),
%% TODO: check override QOS requirements (if any)
PubResult = emqtt:publish_async(ConnPid, LinkTopic, Payload, QoS, Callback),
?tp_ignore_side_effects_in_prod(clink_message_forwarded, #{
pool => PoolName,
message => FwdMsg,
pub_result => PubResult
}),
PubResult
end,
no_handover
).
%% copied from emqx_bridge_mqtt_connector
on_async_result(Callback, Result) ->
apply_callback_function(Callback, handle_send_result(Result)).
apply_callback_function(F, Result) when is_function(F) ->
erlang:apply(F, [Result]);
apply_callback_function({F, A}, Result) when is_function(F), is_list(A) ->
erlang:apply(F, A ++ [Result]);
apply_callback_function({M, F, A}, Result) when is_atom(M), is_atom(F), is_list(A) ->
erlang:apply(M, F, A ++ [Result]).
handle_send_result(ok) ->
ok;
handle_send_result({ok, #{reason_code := ?RC_SUCCESS}}) ->
ok;
handle_send_result({ok, #{reason_code := ?RC_NO_MATCHING_SUBSCRIBERS}}) ->
ok;
handle_send_result({ok, Reply}) ->
{error, classify_reply(Reply)};
handle_send_result({error, Reason}) ->
{error, classify_error(Reason)}.
classify_reply(Reply = #{reason_code := _}) ->
{unrecoverable_error, Reply}.
classify_error(disconnected = Reason) ->
{recoverable_error, Reason};
classify_error(ecpool_empty) ->
{recoverable_error, disconnected};
classify_error({disconnected, _RC, _} = Reason) ->
{recoverable_error, Reason};
classify_error({shutdown, _} = Reason) ->
{recoverable_error, Reason};
classify_error(shutdown = Reason) ->
{recoverable_error, Reason};
classify_error(Reason) ->
{unrecoverable_error, Reason}.
%% copied from emqx_bridge_mqtt_connector
on_get_status(_ResourceId, #{pool_name := PoolName} = _State) ->
Workers = [Worker || {_Name, Worker} <- ecpool:workers(PoolName)],
try emqx_utils:pmap(fun get_status/1, Workers, ?HEALTH_CHECK_TIMEOUT) of
Statuses ->
combine_status(Statuses)
catch
exit:timeout ->
connecting
end.
get_status(Worker) ->
case ecpool_worker:client(Worker) of
{ok, Client} -> status(Client);
{error, _} -> disconnected
end.
status(Pid) ->
try
case proplists:get_value(socket, emqtt:info(Pid)) of
Socket when Socket /= undefined ->
connected;
undefined ->
connecting
end
catch
exit:{noproc, _} ->
disconnected
end.
combine_status(Statuses) ->
%% NOTE
%% Natural order of statuses: [connected, connecting, disconnected]
%% * `disconnected` wins over any other status
%% * `connecting` wins over `connected`
case lists:reverse(lists:usort(Statuses)) of
[Status | _] ->
Status;
[] ->
disconnected
end.
%%--------------------------------------------------------------------
%% ecpool
%%--------------------------------------------------------------------
connect(Options) ->
WorkerIdBin = integer_to_binary(proplists:get_value(ecpool_worker_id, Options)),
#{clientid := ClientId} = ClientOpts = proplists:get_value(client_opts, Options),
ClientId1 = <<ClientId/binary, ":", WorkerIdBin/binary>>,
ClientOpts1 = ClientOpts#{clientid => ClientId1},
case emqtt:start_link(ClientOpts1) of
{ok, Pid} ->
case emqtt:connect(Pid) of
{ok, _Props} ->
{ok, Pid};
Error ->
Error
end;
{error, Reason} = Error ->
?SLOG(error, #{
msg => "client_start_failed",
config => emqx_utils:redact(ClientOpts),
reason => Reason
}),
Error
end.
%%--------------------------------------------------------------------
%% Protocol
%%--------------------------------------------------------------------
%%% New leader-less Syncer/Actor implementation
publish_actor_init_sync(ClientPid, ReqId, RespTopic, TargetCluster, Actor, Incarnation) ->
Payload = #{
?F_OPERATION => ?OP_ACTOR_INIT,
?F_PROTO_VER => ?PROTO_VER,
?F_TARGET_CLUSTER => TargetCluster,
?F_ACTOR => Actor,
?F_INCARNATION => Incarnation
},
Properties = #{
'Response-Topic' => RespTopic,
'Correlation-Data' => ReqId
},
emqtt:publish(ClientPid, ?ROUTE_TOPIC, Properties, ?ENCODE(Payload), [{qos, ?QOS_1}]).
actor_init_ack_resp_msg(Actor, InitRes, MsgIn) ->
Payload = #{
?F_OPERATION => ?OP_ACTOR_INIT_ACK,
?F_PROTO_VER => ?PROTO_VER,
?F_ACTOR => Actor
},
Payload1 = with_res_and_bootstrap(Payload, InitRes),
#{
'Response-Topic' := RespTopic,
'Correlation-Data' := ReqId
} = emqx_message:get_header(properties, MsgIn),
emqx_message:make(
undefined,
?QOS_1,
RespTopic,
?ENCODE(Payload1),
#{},
#{properties => #{'Correlation-Data' => ReqId}}
).
with_res_and_bootstrap(Payload, {ok, ActorState}) ->
Payload#{
?F_RESULT => ok,
?F_NEED_BOOTSTRAP => not emqx_cluster_link_extrouter:is_present_incarnation(ActorState)
};
with_res_and_bootstrap(Payload, Error) ->
Payload#{
?F_RESULT => Error,
?F_NEED_BOOTSTRAP => false
}.
publish_route_sync(ClientPid, Actor, Incarnation, Updates) ->
Payload = #{
?F_OPERATION => ?OP_ROUTE,
?F_ACTOR => Actor,
?F_INCARNATION => Incarnation,
?F_ROUTES => Updates
},
emqtt:publish(ClientPid, ?ROUTE_TOPIC, ?ENCODE(Payload), ?QOS_1).
publish_heartbeat(ClientPid, Actor, Incarnation) ->
Payload = #{
?F_OPERATION => ?OP_HEARTBEAT,
?F_ACTOR => Actor,
?F_INCARNATION => Incarnation
},
emqtt:publish_async(ClientPid, ?ROUTE_TOPIC, ?ENCODE(Payload), ?QOS_0, {fun(_) -> ok end, []}).
decode_route_op(Payload) ->
decode_route_op1(?DECODE(Payload)).
decode_resp(Payload) ->
decode_resp1(?DECODE(Payload)).
decode_route_op1(#{
?F_OPERATION := ?OP_ACTOR_INIT,
?F_PROTO_VER := ProtoVer,
?F_TARGET_CLUSTER := TargetCluster,
?F_ACTOR := Actor,
?F_INCARNATION := Incr
}) ->
Info = #{
target_cluster => TargetCluster,
proto_ver => ProtoVer
},
{actor_init, #{actor => Actor, incarnation => Incr}, Info};
decode_route_op1(#{
?F_OPERATION := ?OP_ROUTE,
?F_ACTOR := Actor,
?F_INCARNATION := Incr,
?F_ROUTES := RouteOps
}) ->
RouteOps1 = lists:map(fun(Op) -> decode_field(route, Op) end, RouteOps),
{route_updates, #{actor => Actor, incarnation => Incr}, RouteOps1};
decode_route_op1(#{
?F_OPERATION := ?OP_HEARTBEAT,
?F_ACTOR := Actor,
?F_INCARNATION := Incr
}) ->
{heartbeat, #{actor => Actor, incarnation => Incr}};
decode_route_op1(Payload) ->
{error, {unknown_payload, Payload}}.
decode_resp1(#{
?F_OPERATION := ?OP_ACTOR_INIT_ACK,
?F_ACTOR := Actor,
?F_PROTO_VER := ProtoVer,
?F_RESULT := InitResult,
?F_NEED_BOOTSTRAP := NeedBootstrap
}) ->
{actor_init_ack, #{
actor => Actor, result => InitResult, proto_ver => ProtoVer, need_bootstrap => NeedBootstrap
}}.
decode_forwarded_msg(Payload) ->
case ?DECODE(Payload) of
#message{} = Msg ->
Msg;
_ ->
?SLOG(warning, #{
msg => "unexpected_cluster_link_forwarded_msg_payload",
payload => Payload
}),
{error, Payload}
end.
encode_field(route, {add, Route = {_Topic, _ID}}) ->
Route;
encode_field(route, {delete, {Topic, ID}}) ->
{?ROUTE_DELETE, Topic, ID}.
decode_field(route, {?ROUTE_DELETE, Topic, ID}) ->
{delete, {Topic, ID}};
decode_field(route, Route = {_Topic, _ID}) ->
{add, Route}.
%%--------------------------------------------------------------------
%% emqx_external_broker
%%--------------------------------------------------------------------
forward(ClusterName, #delivery{message = #message{topic = Topic} = Msg}) ->
QueryOpts = #{pick_key => Topic},
emqx_resource:query(?MSG_RES_ID(ClusterName), Msg, QueryOpts).
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
emqtt_client_opts(ClientIdSuffix, ClusterConf) ->
#{clientid := BaseClientId} = Opts = emqx_cluster_link_config:mk_emqtt_options(ClusterConf),
ClientId = emqx_bridge_mqtt_lib:clientid_base([BaseClientId, ClientIdSuffix]),
Opts#{clientid => ClientId}.

View File

@ -0,0 +1,142 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_router_bootstrap).
-include_lib("emqx/include/emqx_router.hrl").
-include_lib("emqx/include/emqx_shared_sub.hrl").
-include_lib("emqx/src/emqx_persistent_session_ds/emqx_ps_ds_int.hrl").
-include("emqx_cluster_link.hrl").
-export([
init/3,
next_batch/1
]).
-define(MAX_BATCH_SIZE, 4000).
-record(bootstrap, {
target :: _ClusterName :: binary(),
wildcards :: [emqx_types:topic()],
topics :: [emqx_types:topic()],
stash :: [{emqx_types:topic(), _RouteID}],
max_batch_size :: non_neg_integer(),
is_persistent_route :: boolean()
}).
%%
init(TargetCluster, LinkFilters, Options) ->
{Wildcards, Topics} = lists:partition(fun emqx_topic:wildcard/1, LinkFilters),
IsPersistentRoute = maps:get(is_persistent_route, Options, false),
#bootstrap{
target = TargetCluster,
wildcards = Wildcards,
topics = Topics,
stash = [],
max_batch_size = maps:get(max_batch_size, Options, ?MAX_BATCH_SIZE),
is_persistent_route = IsPersistentRoute
}.
next_batch(B = #bootstrap{stash = S0 = [_ | _], max_batch_size = MBS}) ->
{Batch, Stash} = mk_batch(S0, MBS),
{Batch, B#bootstrap{stash = Stash}};
next_batch(B = #bootstrap{topics = Topics = [_ | _], stash = [], is_persistent_route = IsPs}) ->
next_batch(B#bootstrap{topics = [], stash = routes_by_topic(Topics, IsPs)});
next_batch(
B0 = #bootstrap{wildcards = Wildcards = [_ | _], stash = [], is_persistent_route = IsPs}
) ->
next_batch(B0#bootstrap{wildcards = [], stash = routes_by_wildcards(Wildcards, IsPs)});
next_batch(#bootstrap{topics = [], wildcards = [], stash = []}) ->
done.
mk_batch(Stash, MaxBatchSize) when length(Stash) =< MaxBatchSize ->
{Stash, []};
mk_batch(Stash, MaxBatchSize) ->
{Batch, Rest} = lists:split(MaxBatchSize, Stash),
{Batch, Rest}.
%%
routes_by_topic(Topics, _IsPersistentRoute = false) ->
Routes = select_routes_by_topics(Topics),
SharedRoutes = select_shared_sub_routes_by_topics(Topics),
Routes ++ SharedRoutes;
routes_by_topic(Topics, _IsPersistentRoute = true) ->
lists:foldl(
fun(T, Acc) ->
Routes = emqx_persistent_session_ds_router:lookup_routes(T),
[encode_route(T, ?PERSISTENT_ROUTE_ID(T, D)) || #ps_route{dest = D} <- Routes] ++ Acc
end,
[],
Topics
).
routes_by_wildcards(Wildcards, _IsPersistentRoute = false) ->
Routes = select_routes_by_wildcards(Wildcards),
SharedRoutes = select_shared_sub_routes_by_wildcards(Wildcards),
Routes ++ SharedRoutes;
routes_by_wildcards(Wildcards, _IsPersistentRoute = true) ->
emqx_persistent_session_ds_router:foldl_routes(
fun(#ps_route{dest = D, topic = T}, Acc) ->
case topic_intersect_any(T, Wildcards) of
false ->
Acc;
Intersec ->
[encode_route(Intersec, ?PERSISTENT_ROUTE_ID(T, D)) | Acc]
end
end,
[]
).
select_routes_by_topics(Topics) ->
[encode_route(Topic, Topic) || Topic <- Topics, emqx_broker:subscribers(Topic) =/= []].
select_routes_by_wildcards(Wildcards) ->
emqx_utils_ets:keyfoldl(
fun(Topic, Acc) -> intersecting_route(Topic, Wildcards) ++ Acc end,
[],
?SUBSCRIBER
).
select_shared_sub_routes_by_topics([T | Topics]) ->
select_shared_sub_routes(T) ++ select_shared_sub_routes_by_topics(Topics);
select_shared_sub_routes_by_topics([]) ->
[].
select_shared_sub_routes_by_wildcards(Wildcards) ->
emqx_utils_ets:keyfoldl(
fun({Group, Topic}, Acc) ->
RouteID = ?SHARED_ROUTE_ID(Topic, Group),
intersecting_route(Topic, RouteID, Wildcards) ++ Acc
end,
[],
?SHARED_SUBSCRIBER
).
select_shared_sub_routes(Topic) ->
LocalGroups = lists:usort(ets:select(?SHARED_SUBSCRIBER, [{{{'$1', Topic}, '_'}, [], ['$1']}])),
[encode_route(Topic, ?SHARED_ROUTE_ID(Topic, G)) || G <- LocalGroups].
intersecting_route(Topic, Wildcards) ->
intersecting_route(Topic, Topic, Wildcards).
intersecting_route(Topic, RouteID, Wildcards) ->
%% TODO: probably nice to validate cluster link topic filters
%% to have no intersections between each other?
case topic_intersect_any(Topic, Wildcards) of
false -> [];
Intersection -> [encode_route(Intersection, RouteID)]
end.
topic_intersect_any(Topic, [LinkFilter | T]) ->
case emqx_topic:intersection(Topic, LinkFilter) of
false -> topic_intersect_any(Topic, T);
TopicOrFilter -> TopicOrFilter
end;
topic_intersect_any(_Topic, []) ->
false.
encode_route(Topic, RouteID) ->
emqx_cluster_link_mqtt:encode_field(route, {add, {Topic, RouteID}}).

View File

@ -0,0 +1,603 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_router_syncer).
-include_lib("emqtt/include/emqtt.hrl").
-include_lib("emqx/include/logger.hrl").
-include_lib("snabbkaffe/include/trace.hrl").
-include("emqx_cluster_link.hrl").
%% API
-export([start_link/1]).
-export([
push/4,
push_persistent_route/4
]).
%% debug/test helpers
-export([
status/1,
where/1,
where/2
]).
-export([
start_link_actor/4,
start_link_syncer/4
]).
%% Internal API / Syncer
-export([
process_syncer_batch/4
]).
%% silence warning
%% -behaviour(supervisor).
-export([init/1]).
-behaviour(gen_server).
-export([
handle_continue/2,
handle_call/3,
handle_cast/2,
handle_info/2,
terminate/2
]).
-define(NAME(Cluster), {n, l, {?MODULE, Cluster}}).
-define(REF(Cluster), {via, gproc, ?NAME(Cluster)}).
-define(NAME(Cluster, What), {n, l, {?MODULE, Cluster, What}}).
-define(CLIENT_NAME(Cluster), ?NAME(Cluster, client)).
-define(SYNCER_NAME(Cluster), ?NAME(Cluster, syncer)).
-define(SYNCER_REF(Cluster), {via, gproc, ?SYNCER_NAME(Cluster)}).
-define(ACTOR_NAME(Cluster), ?NAME(Cluster, actor)).
-define(ACTOR_REF(Cluster), {via, gproc, ?ACTOR_NAME(Cluster)}).
-define(MAX_BATCH_SIZE, 4000).
-define(MIN_SYNC_INTERVAL, 10).
-define(ERROR_DELAY, 200).
-define(RECONNECT_TIMEOUT, 5_000).
-define(ACTOR_REINIT_TIMEOUT, 7000).
-define(CLIENT_SUFFIX, ":routesync:").
-define(PS_CLIENT_SUFFIX, ":routesync-ps:").
%% Special actor for persistent routes that has the same actor name on all nodes.
%% Node actors with the same name nay race with each other (e.g. during bootstrap),
%% but it must be tolerable, since persistent route destination is a client ID,
%% which is unique cluster-wide.
-define(PS_ACTOR, <<"ps-routes-v1">>).
-define(PS_ACTOR_REF(Cluster), {via, gproc, ?NAME(Cluster, ps_actor)}).
-define(PS_ACTOR_NAME(Cluster), ?NAME(Cluster, ps_actor)).
-define(PS_CLIENT_NAME(Cluster), ?NAME(Cluster, ps_client)).
-define(PS_SYNCER_REF(Cluster), {via, gproc, ?PS_SYNCER_NAME(Cluster)}).
-define(PS_SYNCER_NAME(Cluster), ?NAME(Cluster, ps_syncer)).
-define(SAFE_MQTT_PUB(Expr, ClientPid), ?SAFE_MQTT_PUB(Expr, ClientPid, ok)).
-define(SAFE_MQTT_PUB(Expr, ClientPid, OnSuccess),
try Expr of
{ok, #{reason_code := __RC}} when __RC < ?RC_UNSPECIFIED_ERROR ->
OnSuccess;
{ok, #{reason_code_name := __RCN}} ->
{error, {mqtt, __RCN}};
{error, __Reason} ->
{error, __Reason}
catch
exit:__Reason ->
{error, {client, ClientPid, __Reason}}
end
).
-record(st, {
target :: binary(),
actor :: binary(),
incarnation :: non_neg_integer(),
client :: undefined | pid(),
bootstrapped :: boolean(),
reconnect_timer :: undefined | reference(),
heartbeat_timer :: undefined | reference(),
actor_init_req_id :: undefined | binary(),
actor_init_timer :: undefined | reference(),
remote_actor_info :: undefined | map(),
status :: connecting | connected | disconnected,
error :: undefined | term(),
link_conf :: map()
}).
push(TargetCluster, OpName, Topic, ID) ->
do_push(?SYNCER_NAME(TargetCluster), OpName, Topic, ID).
push_persistent_route(TargetCluster, OpName, Topic, ID) ->
do_push(?PS_SYNCER_NAME(TargetCluster), OpName, Topic, ID).
do_push(SyncerName, OpName, Topic, ID) ->
case gproc:where(SyncerName) of
SyncerPid when is_pid(SyncerPid) ->
emqx_router_syncer:push(SyncerPid, OpName, Topic, ID, #{});
undefined ->
dropped
end.
%% Debug/test helpers
where(Cluster) ->
where(actor, Cluster).
where(actor, Cluster) ->
gproc:where(?ACTOR_NAME(Cluster));
where(ps_actor, Cluster) ->
gproc:where(?PS_ACTOR_NAME(Cluster)).
status(Cluster) ->
case where(actor, Cluster) of
Pid when is_pid(Pid) ->
#st{error = Err, status = Status} = sys:get_state(Pid),
#{error => Err, status => Status};
undefined ->
undefined
end.
%% Supervisor:
%% 1. Actor + MQTT Client
%% 2. Syncer
start_link(#{name := TargetCluster} = LinkConf) ->
supervisor:start_link(?REF(TargetCluster), ?MODULE, {sup, LinkConf}).
%% Actor
new_incarnation() ->
%% TODO: Subject to clock skew, need something more robust.
erlang:system_time(millisecond).
start_link_actor(ActorRef, Actor, Incarnation, LinkConf) ->
gen_server:start_link(
ActorRef,
?MODULE,
{actor, mk_state(LinkConf, Actor, Incarnation)},
[]
).
get_actor_id() ->
atom_to_binary(node()).
%% MQTT Client
start_link_client(Actor, LinkConf) ->
Options = emqx_cluster_link_config:mk_emqtt_options(LinkConf),
case emqtt:start_link(refine_client_options(Options, Actor)) of
{ok, Pid} ->
case emqtt:connect(Pid) of
{ok, _Props} ->
{ok, Pid};
Error ->
_ = flush_link_signal(Pid),
Error
end;
Error ->
Error
end.
flush_link_signal(Pid) ->
receive
{'EXIT', Pid, _} -> ok
after 1 -> timeout
end.
refine_client_options(Options = #{clientid := ClientID}, Actor) ->
Suffix =
case Actor of
?PS_ACTOR -> ?PS_CLIENT_SUFFIX;
_ -> ?CLIENT_SUFFIX
end,
%% TODO: Reconnect should help, but it looks broken right now.
Options#{
clientid => emqx_bridge_mqtt_lib:clientid_base([ClientID, Suffix]),
clean_start => false,
properties => #{'Session-Expiry-Interval' => 60},
retry_interval => 0
}.
announce_client(Actor, TargetCluster, Pid) ->
Name =
case Actor of
?PS_ACTOR -> ?PS_CLIENT_NAME(TargetCluster);
_ -> ?CLIENT_NAME(TargetCluster)
end,
true = gproc:reg_other(Name, Pid),
ok.
publish_routes(ClientPid, Actor, Incarnation, Updates) ->
?SAFE_MQTT_PUB(
emqx_cluster_link_mqtt:publish_route_sync(ClientPid, Actor, Incarnation, Updates),
ClientPid,
#{}
).
publish_heartbeat(ClientPid, Actor, Incarnation) ->
%% NOTE: Fully asynchronous, no need for error handling.
emqx_cluster_link_mqtt:publish_heartbeat(ClientPid, Actor, Incarnation).
%% Route syncer
start_syncer(TargetCluster, Actor, Incr) ->
Spec = child_spec(syncer, Actor, Incr, TargetCluster),
case supervisor:start_child(?REF(TargetCluster), Spec) of
{ok, _} ->
ok;
{error, {already_started, _}} ->
ok
end.
start_link_syncer(Actor, Incarnation, SyncerRef, ClientName) ->
emqx_router_syncer:start_link(SyncerRef, #{
max_batch_size => ?MAX_BATCH_SIZE,
min_sync_interval => ?MIN_SYNC_INTERVAL,
error_delay => ?ERROR_DELAY,
initial_state => suspended,
batch_handler => {?MODULE, process_syncer_batch, [ClientName, Actor, Incarnation]}
%% TODO: enable_replies => false
}).
suspend_syncer(TargetCluster, ?PS_ACTOR) ->
emqx_router_syncer:suspend(?PS_SYNCER_REF(TargetCluster));
suspend_syncer(TargetCluster, _Actor) ->
emqx_router_syncer:suspend(?SYNCER_REF(TargetCluster)).
activate_syncer(TargetCluster, ?PS_ACTOR) ->
emqx_router_syncer:activate(?PS_SYNCER_REF(TargetCluster));
activate_syncer(TargetCluster, _Actor) ->
emqx_router_syncer:activate(?SYNCER_REF(TargetCluster)).
process_syncer_batch(Batch, ClientName, Actor, Incarnation) ->
Updates = maps:fold(
fun(Route, Op, Acc) ->
OpName = batch_get_opname(Op),
Entry = emqx_cluster_link_mqtt:encode_field(route, {OpName, Route}),
[Entry | Acc]
end,
[],
Batch
),
Result = publish_routes(gproc:where(ClientName), Actor, Incarnation, Updates),
?tp(debug, clink_route_sync_complete, #{
actor => {Actor, Incarnation},
batch => Batch
}),
Result.
batch_get_opname(Op) ->
element(1, Op).
%%
init({sup, LinkConf}) ->
%% FIXME: Intensity.
SupFlags = #{
%% TODO: one_for_one?
strategy => one_for_all,
intensity => 10,
period => 60
},
Children = lists:append([
[child_spec(actor, LinkConf)],
[child_spec(ps_actor, LinkConf) || emqx_persistent_message:is_persistence_enabled()]
]),
{ok, {SupFlags, Children}};
init({actor, State}) ->
init_actor(State).
child_spec(actor, #{name := TargetCluster} = LinkConf) ->
%% Actor process.
%% Wraps MQTT Client process.
%% ClientID: `mycluster:emqx1@emqx.local:routesync`
%% Occasional TCP/MQTT-level disconnects are expected, and should be handled
%% gracefully.
Actor = get_actor_id(),
Incarnation = new_incarnation(),
actor_spec(actor, ?ACTOR_REF(TargetCluster), Actor, Incarnation, LinkConf);
child_spec(ps_actor, #{name := TargetCluster, ps_actor_incarnation := Incr} = LinkConf) ->
actor_spec(ps_actor, ?PS_ACTOR_REF(TargetCluster), ?PS_ACTOR, Incr, LinkConf).
child_spec(syncer, ?PS_ACTOR, Incarnation, TargetCluster) ->
SyncerRef = ?PS_SYNCER_REF(TargetCluster),
ClientName = ?PS_CLIENT_NAME(TargetCluster),
syncer_spec(ps_syncer, ?PS_ACTOR, Incarnation, SyncerRef, ClientName);
child_spec(syncer, Actor, Incarnation, TargetCluster) ->
%% Route syncer process.
%% Initially starts in a "closed" state. Actor decides when to open it, i.e.
%% when bootstrapping is done. Syncer crash means re-bootstrap is needed, so
%% we just restart the actor in this case.
SyncerRef = ?SYNCER_REF(TargetCluster),
ClientName = ?CLIENT_NAME(TargetCluster),
syncer_spec(syncer, Actor, Incarnation, SyncerRef, ClientName).
actor_spec(ChildID, ActorRef, Actor, Incarnation, LinkConf) ->
#{
id => ChildID,
start => {?MODULE, start_link_actor, [ActorRef, Actor, Incarnation, LinkConf]},
restart => permanent,
type => worker
}.
syncer_spec(ChildID, Actor, Incarnation, SyncerRef, ClientName) ->
#{
id => ChildID,
start => {?MODULE, start_link_syncer, [Actor, Incarnation, SyncerRef, ClientName]},
restart => permanent,
type => worker
}.
mk_state(#{name := TargetCluster} = LinkConf, Actor, Incarnation) ->
#st{
target = TargetCluster,
actor = Actor,
incarnation = Incarnation,
bootstrapped = false,
status = connecting,
link_conf = LinkConf
}.
init_actor(State = #st{}) ->
_ = erlang:process_flag(trap_exit, true),
{ok, State, {continue, connect}}.
handle_continue(connect, St) ->
{noreply, process_connect(St)}.
handle_call(_Request, _From, St) ->
{reply, ignored, St}.
handle_cast(_Request, St) ->
{noreply, St}.
handle_info({'EXIT', ClientPid, Reason}, St = #st{client = ClientPid}) ->
{noreply, handle_client_down(Reason, St)};
handle_info(
{publish, #{payload := Payload, properties := #{'Correlation-Data' := ReqId}}},
St = #st{actor_init_req_id = ReqId}
) ->
{actor_init_ack,
#{
result := Res,
need_bootstrap := NeedBootstrap
} = AckInfoMap} = emqx_cluster_link_mqtt:decode_resp(
Payload
),
St1 = St#st{
actor_init_req_id = undefined, actor_init_timer = undefined, remote_actor_info = AckInfoMap
},
case Res of
ok ->
_ = maybe_deactivate_alarm(St),
{noreply,
post_actor_init(St1#st{error = undefined, status = connected}, NeedBootstrap)};
Error ->
Reason = error_reason(Error),
?SLOG(error, #{
msg => "failed_to_init_link",
reason => Reason,
target_cluster => St#st.target,
actor => St#st.actor,
remote_link_proto_ver => maps:get(proto_ver, AckInfoMap, undefined)
}),
_ = maybe_alarm(Reason, St1),
?tp(
debug,
clink_handshake_error,
#{actor => {St1#st.actor, St1#st.incarnation}, reason => Reason}
),
%% TODO: retry after a timeout?
{noreply, St1#st{error = Reason, status = disconnected}}
end;
handle_info({publish, #{}}, St) ->
{noreply, St};
handle_info({timeout, TRef, reconnect}, St = #st{reconnect_timer = TRef}) ->
{noreply, process_connect(St#st{reconnect_timer = undefined})};
handle_info({timeout, TRef, actor_reinit}, St = #st{actor_init_timer = TRef}) ->
?SLOG(error, #{
msg => "remote_actor_init_timeout",
target_cluster => St#st.target,
actor => St#st.actor
}),
Reason = init_timeout,
_ = maybe_alarm(Reason, St),
{noreply,
init_remote_actor(St#st{actor_init_timer = undefined, status = disconnected, error = Reason})};
handle_info({timeout, TRef, _Heartbeat}, St = #st{heartbeat_timer = TRef}) ->
{noreply, process_heartbeat(St#st{heartbeat_timer = undefined})};
%% Stale timeout.
handle_info({timeout, _, _}, St) ->
{noreply, St};
handle_info(Info, St) ->
?SLOG(warning, #{msg => "unexpected_info", info => Info}),
{noreply, St}.
terminate(_Reason, State) ->
_ = maybe_deactivate_alarm(State),
ok.
process_connect(St = #st{target = TargetCluster, actor = Actor, link_conf = Conf}) ->
case start_link_client(Actor, Conf) of
{ok, ClientPid} ->
_ = maybe_deactivate_alarm(St),
ok = announce_client(Actor, TargetCluster, ClientPid),
init_remote_actor(St#st{client = ClientPid});
{error, Reason} ->
handle_connect_error(Reason, St)
end.
init_remote_actor(
St = #st{target = TargetCluster, client = ClientPid, actor = Actor, incarnation = Incr}
) ->
ReqId = emqx_utils_conv:bin(emqx_utils:gen_id(16)),
%% TODO: handle subscribe errors
{ok, _, _} = emqtt:subscribe(ClientPid, ?RESP_TOPIC(Actor), ?QOS_1),
Res = ?SAFE_MQTT_PUB(
emqx_cluster_link_mqtt:publish_actor_init_sync(
ClientPid, ReqId, ?RESP_TOPIC(Actor), TargetCluster, Actor, Incr
),
ClientPid
),
St1 =
case Res of
ok ->
St#st{status = connecting};
{error, Reason} ->
?SLOG(error, #{
msg => "cluster_link_init_failed",
reason => Reason,
target_cluster => TargetCluster,
actor => Actor
}),
_ = maybe_alarm(Reason, St),
St#st{error = Reason, status = disconnected}
end,
TRef = erlang:start_timer(?ACTOR_REINIT_TIMEOUT, self(), actor_reinit),
St1#st{actor_init_req_id = ReqId, actor_init_timer = TRef}.
post_actor_init(
St = #st{client = ClientPid, target = TargetCluster, actor = Actor, incarnation = Incr},
NeedBootstrap
) ->
ok = start_syncer(TargetCluster, Actor, Incr),
NSt = schedule_heartbeat(St#st{client = ClientPid}),
process_bootstrap(NSt, NeedBootstrap).
handle_connect_error(Reason, St) ->
?SLOG(error, #{
msg => "cluster_link_connection_failed",
reason => Reason,
target_cluster => St#st.target,
actor => St#st.actor
}),
TRef = erlang:start_timer(?RECONNECT_TIMEOUT, self(), reconnect),
_ = maybe_alarm(Reason, St),
St#st{reconnect_timer = TRef, error = Reason, status = disconnected}.
handle_client_down(Reason, St = #st{target = TargetCluster, actor = Actor}) ->
?SLOG(error, #{
msg => "cluster_link_connection_failed",
reason => Reason,
target_cluster => St#st.target,
actor => St#st.actor
}),
%% TODO: syncer may be already down due to one_for_all strategy
ok = suspend_syncer(TargetCluster, Actor),
_ = maybe_alarm(Reason, St),
NSt = cancel_heartbeat(St),
process_connect(NSt#st{client = undefined, error = Reason, status = connecting}).
process_bootstrap(St = #st{bootstrapped = false}, _NeedBootstrap) ->
run_bootstrap(St);
process_bootstrap(St = #st{bootstrapped = true}, NeedBootstrap) ->
case NeedBootstrap of
true ->
run_bootstrap(St);
false ->
process_bootstrapped(St)
end.
process_heartbeat(St = #st{client = ClientPid, actor = Actor, incarnation = Incarnation}) ->
ok = publish_heartbeat(ClientPid, Actor, Incarnation),
schedule_heartbeat(St).
schedule_heartbeat(St = #st{heartbeat_timer = undefined}) ->
Timeout = emqx_cluster_link_config:actor_heartbeat_interval(),
TRef = erlang:start_timer(Timeout, self(), heartbeat),
St#st{heartbeat_timer = TRef}.
cancel_heartbeat(St = #st{heartbeat_timer = undefined}) ->
St;
cancel_heartbeat(St = #st{heartbeat_timer = TRef}) ->
ok = emqx_utils:cancel_timer(TRef),
St#st{heartbeat_timer = undefined}.
%% Bootstrapping.
%% Responsible for transferring local routing table snapshot to the target
%% cluster. Does so either during the initial startup or when MQTT connection
%% is re-established with a clean session. Once bootstrapping is done, it
%% opens the syncer.
run_bootstrap(St = #st{target = TargetCluster, actor = ?PS_ACTOR, link_conf = #{topics := Topics}}) ->
case mria_config:whoami() of
Role when Role /= replicant ->
Opts = #{is_persistent_route => true},
Bootstrap = emqx_cluster_link_router_bootstrap:init(TargetCluster, Topics, Opts),
run_bootstrap(Bootstrap, St);
_ ->
process_bootstrapped(St)
end;
run_bootstrap(St = #st{target = TargetCluster, link_conf = #{topics := Topics}}) ->
Bootstrap = emqx_cluster_link_router_bootstrap:init(TargetCluster, Topics, #{}),
run_bootstrap(Bootstrap, St).
run_bootstrap(Bootstrap, St) ->
case emqx_cluster_link_router_bootstrap:next_batch(Bootstrap) of
done ->
?tp(
debug,
clink_route_bootstrap_complete,
#{actor => {St#st.actor, St#st.incarnation}, cluster => St#st.target}
),
process_bootstrapped(St);
{Batch, NBootstrap} ->
%% TODO: Better error handling.
case process_bootstrap_batch(Batch, St) of
#{} ->
NSt = ensure_bootstrap_heartbeat(St),
run_bootstrap(NBootstrap, NSt);
{error, {client, _, _}} ->
%% Client has exited, let `reconnect` codepath handle it.
St
end
end.
process_bootstrapped(
St = #st{target = TargetCluster, actor = Actor}
) ->
ok = activate_syncer(TargetCluster, Actor),
St#st{bootstrapped = true}.
process_bootstrap_batch(Batch, #st{client = ClientPid, actor = Actor, incarnation = Incarnation}) ->
publish_routes(ClientPid, Actor, Incarnation, Batch).
ensure_bootstrap_heartbeat(St = #st{heartbeat_timer = TRef}) ->
case erlang:read_timer(TRef) of
false ->
ok = emqx_utils:cancel_timer(TRef),
process_heartbeat(St#st{heartbeat_timer = undefined});
_TimeLeft ->
St
end.
%%
error_reason({error, Reason}) ->
Reason;
error_reason(OtherErr) ->
OtherErr.
%% Assume that alarm is already active
maybe_alarm(Error, #st{error = Error}) ->
ok;
maybe_alarm(Error, St) ->
HrError = emqx_utils:readable_error_msg(error_reason(Error)),
Name = link_name(St),
emqx_alarm:safe_activate(
Name,
#{custer_link => Name, reason => cluster_link_down},
<<"cluster link down: ", HrError/binary>>
).
maybe_deactivate_alarm(#st{error = undefined}) ->
ok;
maybe_deactivate_alarm(St) ->
emqx_alarm:safe_deactivate(link_name(St)).
link_name(#st{actor = ?PS_ACTOR = Actor, target = Target}) ->
<<"cluster_link:", Target/binary, ":", (get_actor_id())/binary, ":", Actor/binary>>;
link_name(#st{actor = Actor, target = Target}) ->
<<"cluster_link:", Target/binary, ":", Actor/binary>>.

View File

@ -0,0 +1,154 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_schema).
-behaviour(emqx_schema_hooks).
-include("emqx_cluster_link.hrl").
-include_lib("hocon/include/hoconsc.hrl").
-export([injected_fields/0]).
%% Used in emqx_cluster_link_api
-export([links_schema/1]).
-export([
roots/0,
fields/1,
namespace/0,
desc/1
]).
-import(emqx_schema, [mk_duration/2]).
-define(MQTT_HOST_OPTS, #{default_port => 1883}).
namespace() -> "cluster".
roots() -> [].
injected_fields() ->
#{cluster => [{links, links_schema(#{})}]}.
links_schema(Meta) ->
?HOCON(?ARRAY(?R_REF("link")), Meta#{
default => [], validator => fun links_validator/1, desc => ?DESC("links")
}).
fields("link") ->
[
{enable, ?HOCON(boolean(), #{default => true, desc => ?DESC(enable)})},
{name, ?HOCON(binary(), #{required => true, desc => ?DESC(link_name)})},
{server,
emqx_schema:servers_sc(#{required => true, desc => ?DESC(server)}, ?MQTT_HOST_OPTS)},
{clientid, ?HOCON(binary(), #{desc => ?DESC(clientid)})},
{username, ?HOCON(binary(), #{desc => ?DESC(username)})},
{password, emqx_schema_secret:mk(#{desc => ?DESC(password)})},
{ssl, #{
type => ?R_REF(emqx_schema, "ssl_client_opts"),
default => #{<<"enable">> => false},
desc => ?DESC(ssl)
}},
{topics,
?HOCON(?ARRAY(binary()), #{
desc => ?DESC(topics), required => true, validator => fun topics_validator/1
})},
{pool_size, ?HOCON(pos_integer(), #{default => 8, desc => ?DESC(pool_size)})},
{retry_interval,
mk_duration(
"MQTT Message retry interval. Delay for the link to retry sending the QoS1/QoS2 "
"messages in case of ACK not received.",
#{default => <<"15s">>}
)},
{max_inflight,
?HOCON(
non_neg_integer(),
#{
default => 32,
desc => ?DESC("max_inflight")
}
)},
{resource_opts,
?HOCON(
?R_REF(?MODULE, "creation_opts"),
#{
required => false,
default => #{},
desc => ?DESC(emqx_resource_schema, <<"resource_opts">>)
}
)},
%% Must not be configured manually. The value is incremented by cluster link config handler
%% and is used as a globally synchronized sequence to ensure persistent routes actors have
%% the same next incarnation after each config change.
{ps_actor_incarnation, ?HOCON(integer(), #{default => 0, importance => ?IMPORTANCE_HIDDEN})}
];
fields("creation_opts") ->
Opts = emqx_resource_schema:fields("creation_opts"),
[O || {Field, _} = O <- Opts, not is_hidden_res_opt(Field)].
desc("links") ->
?DESC("links");
desc("link") ->
?DESC("link");
desc("creation_opts" = Name) ->
emqx_resource_schema:desc(Name);
desc(_) ->
undefined.
is_hidden_res_opt(Field) ->
lists:member(
Field,
[start_after_created, query_mode, enable_batch, batch_size, batch_time]
).
%% TODO: check that no link name equals local cluster name,
%% but this may be tricky since the link config is injected into cluster config (emqx_conf_schema).
links_validator(Links) ->
{_, Dups} = lists:foldl(
fun(Link, {Acc, DupAcc}) ->
Name = link_name(Link),
case Acc of
#{Name := _} ->
{Acc, [Name | DupAcc]};
_ ->
{Acc#{Name => undefined}, DupAcc}
end
end,
{#{}, []},
Links
),
check_errors(Dups, duplicated_cluster_links, names).
link_name(#{name := Name}) -> Name;
link_name(#{<<"name">> := Name}) -> Name.
topics_validator(Topics) ->
Errors = lists:foldl(
fun(T, ErrAcc) ->
try
_ = emqx_topic:validate(T),
validate_sys_link_topic(T, ErrAcc)
catch
E:R ->
[{T, {E, R}} | ErrAcc]
end
end,
[],
Topics
),
check_errors(Errors, invalid_topics, topics).
validate_sys_link_topic(T, ErrAcc) ->
case emqx_topic:match(T, ?TOPIC_PREFIX_WILDCARD) of
true ->
[{T, {error, topic_not_allowed}} | ErrAcc];
false ->
ErrAcc
end.
check_errors([] = _Errors, _Reason, _ValuesField) ->
ok;
check_errors(Errors, Reason, ValuesField) ->
{error, #{reason => Reason, ValuesField => Errors}}.

View File

@ -0,0 +1,73 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_sup).
-behaviour(supervisor).
-export([start_link/1]).
-export([
ensure_actor/1,
ensure_actor_stopped/1
]).
-export([init/1]).
-define(SERVER, ?MODULE).
-define(ACTOR_MODULE, emqx_cluster_link_router_syncer).
start_link(LinksConf) ->
supervisor:start_link({local, ?SERVER}, ?SERVER, LinksConf).
init(LinksConf) ->
SupFlags = #{
strategy => one_for_one,
intensity => 10,
period => 5
},
ExtrouterGC = extrouter_gc_spec(),
RouteActors = [
sup_spec(Name, ?ACTOR_MODULE, [LinkConf])
|| #{name := Name} = LinkConf <- LinksConf
],
{ok, {SupFlags, [ExtrouterGC | RouteActors]}}.
extrouter_gc_spec() ->
%% NOTE: This one is currently global, not per-link.
#{
id => {extrouter, gc},
start => {emqx_cluster_link_extrouter_gc, start_link, []},
restart => permanent,
type => worker
}.
sup_spec(Id, Mod, Args) ->
#{
id => Id,
start => {Mod, start_link, Args},
restart => permanent,
shutdown => infinity,
type => supervisor,
modules => [Mod]
}.
ensure_actor(#{name := Name} = LinkConf) ->
case supervisor:start_child(?SERVER, sup_spec(Name, ?ACTOR_MODULE, [LinkConf])) of
{ok, Pid} ->
{ok, Pid};
{error, {already_started, Pid}} ->
{ok, Pid};
Err ->
Err
end.
ensure_actor_stopped(ClusterName) ->
case supervisor:terminate_child(?MODULE, ClusterName) of
ok ->
_ = supervisor:delete_child(?MODULE, ClusterName),
ok;
{error, not_found} ->
ok
end.

View File

@ -0,0 +1,279 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_SUITE).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("emqx/include/asserts.hrl").
-include_lib("emqx_utils/include/emqx_message.hrl").
-compile(export_all).
-compile(nowarn_export_all).
%%
all() ->
[
{group, shared_subs},
{group, non_shared_subs}
].
groups() ->
AllTCs = emqx_common_test_helpers:all(?MODULE),
[
{shared_subs, AllTCs},
{non_shared_subs, AllTCs}
].
init_per_suite(Config) ->
Config.
end_per_suite(_Config) ->
ok.
init_per_group(shared_subs, Config) ->
[{is_shared_sub, true} | Config];
init_per_group(non_shared_subs, Config) ->
[{is_shared_sub, false} | Config].
end_per_group(_Group, _Config) ->
ok.
init_per_testcase(TCName, Config) ->
emqx_common_test_helpers:init_per_testcase(?MODULE, TCName, Config).
end_per_testcase(TCName, Config) ->
emqx_common_test_helpers:end_per_testcase(?MODULE, TCName, Config).
%%
mk_source_cluster(BaseName, Config) ->
SourceConf =
"cluster {"
"\n name = cl.source"
"\n links = ["
"\n { enable = true"
"\n name = cl.target"
"\n server = \"localhost:31883\""
"\n clientid = client.source"
"\n topics = []"
"\n }"
"\n ]}",
SourceApps1 = [{emqx_conf, combine([conf_log(), SourceConf])}],
SourceApps2 = [{emqx_conf, combine([conf_log(), conf_mqtt_listener(41883), SourceConf])}],
emqx_cth_cluster:mk_nodespecs(
[
{mk_nodename(BaseName, s1), #{apps => SourceApps1}},
{mk_nodename(BaseName, s2), #{apps => SourceApps2}}
],
#{work_dir => emqx_cth_suite:work_dir(Config)}
).
mk_target_cluster(BaseName, Config) ->
TargetConf =
"cluster {"
"\n name = cl.target"
"\n links = ["
"\n { enable = true"
"\n name = cl.source"
"\n server = \"localhost:41883\""
"\n clientid = client.target"
"\n topics = [\"#\"]"
"\n }"
"\n ]}",
TargetApps1 = [{emqx_conf, combine([conf_log(), TargetConf])}],
TargetApps2 = [{emqx_conf, combine([conf_log(), conf_mqtt_listener(31883), TargetConf])}],
emqx_cth_cluster:mk_nodespecs(
[
{mk_nodename(BaseName, t1), #{apps => TargetApps1, base_port => 20100}},
{mk_nodename(BaseName, t2), #{apps => TargetApps2, base_port => 20200}}
],
#{work_dir => emqx_cth_suite:work_dir(Config)}
).
mk_nodename(BaseName, Suffix) ->
binary_to_atom(fmt("emqx_clink_~s_~s", [BaseName, Suffix])).
conf_mqtt_listener(LPort) when is_integer(LPort) ->
fmt("listeners.tcp.clink { bind = ~p }", [LPort]);
conf_mqtt_listener(_) ->
"".
conf_log() ->
"log.file { enable = true, level = debug, path = node.log, supervisor_reports = progress }".
combine([Entry | Rest]) ->
lists:foldl(fun emqx_cth_suite:merge_config/2, Entry, Rest).
start_cluster_link(Nodes, Config) ->
[{ok, Apps}] = lists:usort(
erpc:multicall(Nodes, emqx_cth_suite, start_apps, [
[emqx_cluster_link],
#{work_dir => emqx_cth_suite:work_dir(Config)}
])
),
Apps.
stop_cluster_link(Config) ->
Apps = ?config(tc_apps, Config),
Nodes = nodes_all(Config),
[{ok, ok}] = lists:usort(
erpc:multicall(Nodes, emqx_cth_suite, stop_apps, [Apps])
).
%%
nodes_all(Config) ->
nodes_source(Config) ++ nodes_target(Config).
nodes_source(Config) ->
?config(source_nodes, Config).
nodes_target(Config) ->
?config(target_nodes, Config).
%%
t_message_forwarding('init', Config) ->
SourceNodes = emqx_cth_cluster:start(mk_source_cluster(?FUNCTION_NAME, Config)),
TargetNodes = emqx_cth_cluster:start(mk_target_cluster(?FUNCTION_NAME, Config)),
_Apps = start_cluster_link(SourceNodes ++ TargetNodes, Config),
ok = snabbkaffe:start_trace(),
[
{source_nodes, SourceNodes},
{target_nodes, TargetNodes}
| Config
];
t_message_forwarding('end', Config) ->
ok = snabbkaffe:stop(),
ok = emqx_cth_cluster:stop(?config(source_nodes, Config)),
ok = emqx_cth_cluster:stop(?config(target_nodes, Config)).
t_message_forwarding(Config) ->
[SourceNode1 | _] = nodes_source(Config),
[TargetNode1, TargetNode2 | _] = nodes_target(Config),
SourceC1 = start_client("t_message_forwarding", SourceNode1),
TargetC1 = start_client("t_message_forwarding1", TargetNode1),
TargetC2 = start_client("t_message_forwarding2", TargetNode2),
IsShared = ?config(is_shared_sub, Config),
{ok, _, _} = emqtt:subscribe(TargetC1, maybe_shared_topic(IsShared, <<"t/+">>), qos1),
{ok, _, _} = emqtt:subscribe(TargetC2, maybe_shared_topic(IsShared, <<"t/#">>), qos1),
{ok, _} = ?block_until(#{?snk_kind := clink_route_sync_complete}),
{ok, _} = emqtt:publish(SourceC1, <<"t/42">>, <<"hello">>, qos1),
?assertReceive(
{publish, #{topic := <<"t/42">>, payload := <<"hello">>, client_pid := TargetC1}}
),
?assertReceive(
{publish, #{topic := <<"t/42">>, payload := <<"hello">>, client_pid := TargetC2}}
),
?assertNotReceive({publish, _Message = #{}}),
ok = emqtt:stop(SourceC1),
ok = emqtt:stop(TargetC1),
ok = emqtt:stop(TargetC2).
t_target_extrouting_gc('init', Config) ->
SourceCluster = mk_source_cluster(?FUNCTION_NAME, Config),
SourceNodes = emqx_cth_cluster:start(SourceCluster),
TargetCluster = mk_target_cluster(?FUNCTION_NAME, Config),
TargetNodes = emqx_cth_cluster:start(TargetCluster),
_Apps = start_cluster_link(SourceNodes ++ TargetNodes, Config),
ok = snabbkaffe:start_trace(),
[
{source_cluster, SourceCluster},
{source_nodes, SourceNodes},
{target_cluster, TargetCluster},
{target_nodes, TargetNodes}
| Config
];
t_target_extrouting_gc('end', Config) ->
ok = snabbkaffe:stop(),
ok = emqx_cth_cluster:stop(?config(source_nodes, Config)).
t_target_extrouting_gc(Config) ->
[SourceNode1 | _] = nodes_source(Config),
[TargetNode1, TargetNode2 | _] = nodes_target(Config),
SourceC1 = start_client("t_target_extrouting_gc", SourceNode1),
TargetC1 = start_client_unlink("t_target_extrouting_gc1", TargetNode1),
TargetC2 = start_client_unlink("t_target_extrouting_gc2", TargetNode2),
IsShared = ?config(is_shared_sub, Config),
{ok, _, _} = emqtt:subscribe(TargetC1, maybe_shared_topic(IsShared, <<"t/#">>), qos1),
{ok, _, _} = emqtt:subscribe(TargetC2, maybe_shared_topic(IsShared, <<"t/+">>), qos1),
{ok, _} = ?block_until(#{?snk_kind := clink_route_sync_complete}),
{ok, _} = emqtt:publish(SourceC1, <<"t/1">>, <<"HELLO1">>, qos1),
{ok, _} = emqtt:publish(SourceC1, <<"t/2/ext">>, <<"HELLO2">>, qos1),
{ok, _} = emqtt:publish(SourceC1, <<"t/3/ext">>, <<"HELLO3">>, qos1),
Pubs1 = [M || {publish, M} <- ?drainMailbox(1_000)],
{ok, _} = ?wait_async_action(
emqx_cth_cluster:stop_node(TargetNode1),
#{?snk_kind := clink_extrouter_actor_cleaned, cluster := <<"cl.target">>}
),
{ok, _} = emqtt:publish(SourceC1, <<"t/4/ext">>, <<"HELLO4">>, qos1),
{ok, _} = emqtt:publish(SourceC1, <<"t/5">>, <<"HELLO5">>, qos1),
Pubs2 = [M || {publish, M} <- ?drainMailbox(1_000)],
{ok, _} = ?wait_async_action(
emqx_cth_cluster:stop_node(TargetNode2),
#{?snk_kind := clink_extrouter_actor_cleaned, cluster := <<"cl.target">>}
),
ok = emqtt:stop(SourceC1),
%% Verify that extrouter table eventually becomes empty.
?assertEqual(
[],
erpc:call(SourceNode1, emqx_cluster_link_extrouter, topics, []),
{
erpc:call(SourceNode1, ets, tab2list, [emqx_external_router_actor]),
erpc:call(SourceNode1, ets, tab2list, [emqx_external_router_route])
}
),
%% Verify all relevant messages were forwarded.
?assertMatch(
[
#{topic := <<"t/1">>, payload := <<"HELLO1">>, client_pid := _C1},
#{topic := <<"t/1">>, payload := <<"HELLO1">>, client_pid := _C2},
#{topic := <<"t/2/ext">>, payload := <<"HELLO2">>},
#{topic := <<"t/3/ext">>, payload := <<"HELLO3">>},
#{topic := <<"t/5">>, payload := <<"HELLO5">>}
],
lists:sort(emqx_utils_maps:key_comparer(topic), Pubs1 ++ Pubs2)
),
%% Verify there was no unnecessary forwarding.
Trace = snabbkaffe:collect_trace(),
?assertMatch(
[
#{message := #message{topic = <<"t/1">>, payload = <<"HELLO1">>}},
#{message := #message{topic = <<"t/2/ext">>, payload = <<"HELLO2">>}},
#{message := #message{topic = <<"t/3/ext">>, payload = <<"HELLO3">>}},
#{message := #message{topic = <<"t/5">>, payload = <<"HELLO5">>}}
],
?of_kind(clink_message_forwarded, Trace),
Trace
).
%%
maybe_shared_topic(true = _IsShared, Topic) ->
<<"$share/test-group/", Topic/binary>>;
maybe_shared_topic(false = _IsShared, Topic) ->
Topic.
start_client_unlink(ClientId, Node) ->
Client = start_client(ClientId, Node),
_ = erlang:unlink(Client),
Client.
start_client(ClientId, Node) ->
Port = tcp_port(Node),
{ok, Client} = emqtt:start_link([{proto_ver, v5}, {clientid, ClientId}, {port, Port}]),
{ok, _} = emqtt:connect(Client),
Client.
tcp_port(Node) ->
{_Host, Port} = erpc:call(Node, emqx_config, get, [[listeners, tcp, default, bind]]),
Port.
fmt(Fmt, Args) ->
emqx_utils:format(Fmt, Args).

View File

@ -0,0 +1,132 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_api_SUITE).
-compile(export_all).
-compile(nowarn_export_all).
-include_lib("eunit/include/eunit.hrl").
-include_lib("common_test/include/ct.hrl").
-include_lib("snabbkaffe/include/snabbkaffe.hrl").
-define(API_PATH, emqx_mgmt_api_test_util:api_path(["cluster", "links"])).
-define(CONF_PATH, [cluster, links]).
-define(CACERT, <<
"-----BEGIN CERTIFICATE-----\n"
"MIIDUTCCAjmgAwIBAgIJAPPYCjTmxdt/MA0GCSqGSIb3DQEBCwUAMD8xCzAJBgNV\n"
"BAYTAkNOMREwDwYDVQQIDAhoYW5nemhvdTEMMAoGA1UECgwDRU1RMQ8wDQYDVQQD\n"
"DAZSb290Q0EwHhcNMjAwNTA4MDgwNjUyWhcNMzAwNTA2MDgwNjUyWjA/MQswCQYD\n"
"VQQGEwJDTjERMA8GA1UECAwIaGFuZ3pob3UxDDAKBgNVBAoMA0VNUTEPMA0GA1UE\n"
"AwwGUm9vdENBMIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAzcgVLex1\n"
"EZ9ON64EX8v+wcSjzOZpiEOsAOuSXOEN3wb8FKUxCdsGrsJYB7a5VM/Jot25Mod2\n"
"juS3OBMg6r85k2TWjdxUoUs+HiUB/pP/ARaaW6VntpAEokpij/przWMPgJnBF3Ur\n"
"MjtbLayH9hGmpQrI5c2vmHQ2reRZnSFbY+2b8SXZ+3lZZgz9+BaQYWdQWfaUWEHZ\n"
"uDaNiViVO0OT8DRjCuiDp3yYDj3iLWbTA/gDL6Tf5XuHuEwcOQUrd+h0hyIphO8D\n"
"tsrsHZ14j4AWYLk1CPA6pq1HIUvEl2rANx2lVUNv+nt64K/Mr3RnVQd9s8bK+TXQ\n"
"KGHd2Lv/PALYuwIDAQABo1AwTjAdBgNVHQ4EFgQUGBmW+iDzxctWAWxmhgdlE8Pj\n"
"EbQwHwYDVR0jBBgwFoAUGBmW+iDzxctWAWxmhgdlE8PjEbQwDAYDVR0TBAUwAwEB\n"
"/zANBgkqhkiG9w0BAQsFAAOCAQEAGbhRUjpIred4cFAFJ7bbYD9hKu/yzWPWkMRa\n"
"ErlCKHmuYsYk+5d16JQhJaFy6MGXfLgo3KV2itl0d+OWNH0U9ULXcglTxy6+njo5\n"
"CFqdUBPwN1jxhzo9yteDMKF4+AHIxbvCAJa17qcwUKR5MKNvv09C6pvQDJLzid7y\n"
"E2dkgSuggik3oa0427KvctFf8uhOV94RvEDyqvT5+pgNYZ2Yfga9pD/jjpoHEUlo\n"
"88IGU8/wJCx3Ds2yc8+oBg/ynxG8f/HmCC1ET6EHHoe2jlo8FpU/SgGtghS1YL30\n"
"IWxNsPrUP+XsZpBJy/mvOhE5QXo6Y35zDqqj8tI7AGmAWu22jg==\n"
"-----END CERTIFICATE-----"
>>).
all() ->
emqx_common_test_helpers:all(?MODULE).
init_per_suite(Config) ->
%% This is called by emqx_machine in EMQX release
emqx_otel_app:configure_otel_deps(),
Apps = emqx_cth_suite:start(
[
emqx_conf,
emqx_management,
{emqx_dashboard, "dashboard.listeners.http { enable = true, bind = 18083 }"},
emqx_cluster_link
],
#{work_dir => emqx_cth_suite:work_dir(Config)}
),
Auth = auth_header(),
[{suite_apps, Apps}, {auth, Auth} | Config].
end_per_suite(Config) ->
emqx_cth_suite:stop(?config(suite_apps, Config)),
emqx_config:delete_override_conf_files(),
ok.
auth_header() ->
{ok, API} = emqx_common_test_http:create_default_app(),
emqx_common_test_http:auth_header(API).
init_per_testcase(_TC, Config) ->
{ok, _} = emqx_cluster_link_config:update([]),
Config.
end_per_testcase(_TC, _Config) ->
ok.
t_put_get_valid(Config) ->
Auth = ?config(auth, Config),
Path = ?API_PATH,
{ok, Resp} = emqx_mgmt_api_test_util:request_api(get, Path, Auth),
?assertMatch([], emqx_utils_json:decode(Resp)),
Link1 = #{
<<"pool_size">> => 1,
<<"server">> => <<"emqxcl_2.nohost:31883">>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => <<"emqcl_1">>
},
Link2 = #{
<<"pool_size">> => 1,
<<"server">> => <<"emqxcl_2.nohost:41883">>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => <<"emqcl_2">>
},
?assertMatch({ok, _}, emqx_mgmt_api_test_util:request_api(put, Path, "", Auth, [Link1, Link2])),
{ok, Resp1} = emqx_mgmt_api_test_util:request_api(get, Path, Auth),
?assertMatch([Link1, Link2], emqx_utils_json:decode(Resp1)),
DisabledLink1 = Link1#{<<"enable">> => false},
?assertMatch(
{ok, _}, emqx_mgmt_api_test_util:request_api(put, Path, "", Auth, [DisabledLink1, Link2])
),
{ok, Resp2} = emqx_mgmt_api_test_util:request_api(get, Path, Auth),
?assertMatch([DisabledLink1, Link2], emqx_utils_json:decode(Resp2)),
SSL = #{<<"enable">> => true, <<"cacertfile">> => ?CACERT},
SSLLink1 = Link1#{<<"ssl">> => SSL},
?assertMatch(
{ok, _}, emqx_mgmt_api_test_util:request_api(put, Path, "", Auth, [Link2, SSLLink1])
),
{ok, Resp3} = emqx_mgmt_api_test_util:request_api(get, Path, Auth),
?assertMatch(
[Link2, #{<<"ssl">> := #{<<"enable">> := true, <<"cacertfile">> := _Path}}],
emqx_utils_json:decode(Resp3)
).
t_put_invalid(Config) ->
Auth = ?config(auth, Config),
Path = ?API_PATH,
Link = #{
<<"pool_size">> => 1,
<<"server">> => <<"emqxcl_2.nohost:31883">>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => <<"emqcl_1">>
},
?assertMatch(
{error, {_, 400, _}}, emqx_mgmt_api_test_util:request_api(put, Path, "", Auth, [Link, Link])
),
?assertMatch(
{error, {_, 400, _}},
emqx_mgmt_api_test_util:request_api(put, Path, "", Auth, [maps:remove(<<"name">>, Link)])
).

View File

@ -0,0 +1,647 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_config_SUITE).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("emqx/include/asserts.hrl").
-include_lib("emqx_utils/include/emqx_message.hrl").
-compile(export_all).
-compile(nowarn_export_all).
all() ->
emqx_common_test_helpers:all(?MODULE).
init_per_suite(Config) ->
Config.
end_per_suite(_Config) ->
ok.
init_per_testcase(TCName, Config) ->
emqx_common_test_helpers:init_per_testcase(?MODULE, TCName, Config).
end_per_testcase(TCName, Config) ->
emqx_common_test_helpers:end_per_testcase(?MODULE, TCName, Config).
mk_clusters(NameA, NameB, PortA, PortB, ConfA, ConfB, Config) ->
AppsA = [{emqx_conf, ConfA}, emqx_cluster_link],
AppsA1 = [
{emqx_conf, combine([ConfA, conf_mqtt_listener(PortA)])},
emqx_cluster_link
],
AppsB = [{emqx_conf, ConfB}, emqx_cluster_link],
AppsB1 = [
{emqx_conf, combine([ConfB, conf_mqtt_listener(PortB)])},
emqx_cluster_link
],
NodesA = emqx_cth_cluster:mk_nodespecs(
[
{mk_nodename(NameA, 1), #{apps => AppsA}},
{mk_nodename(NameA, 2), #{apps => AppsA}},
{mk_nodename(NameA, 3), #{apps => AppsA1, role => replicant}}
],
#{work_dir => emqx_cth_suite:work_dir(Config)}
),
NodesB = emqx_cth_cluster:mk_nodespecs(
[
{mk_nodename(NameB, 1), #{apps => AppsB, base_port => 20100}},
{mk_nodename(NameB, 2), #{apps => AppsB1, base_port => 20200}}
],
#{work_dir => emqx_cth_suite:work_dir(Config)}
),
{NodesA, NodesB}.
t_config_update('init', Config) ->
NameA = fmt("~s_~s", [?FUNCTION_NAME, "a"]),
NameB = fmt("~s_~s", [?FUNCTION_NAME, "b"]),
LPortA = 31883,
LPortB = 41883,
ConfA = combine([conf_cluster(NameA), conf_log()]),
ConfB = combine([conf_cluster(NameB), conf_log()]),
{NodesA, NodesB} = mk_clusters(NameA, NameB, LPortA, LPortB, ConfA, ConfB, Config),
ClusterA = emqx_cth_cluster:start(NodesA),
ClusterB = emqx_cth_cluster:start(NodesB),
ok = snabbkaffe:start_trace(),
[
{cluster_a, ClusterA},
{cluster_b, ClusterB},
{lport_a, LPortA},
{lport_b, LPortB},
{name_a, NameA},
{name_b, NameB}
| Config
];
t_config_update('end', Config) ->
ok = snabbkaffe:stop(),
ok = emqx_cth_cluster:stop(?config(cluster_a, Config)),
ok = emqx_cth_cluster:stop(?config(cluster_b, Config)).
t_config_update(Config) ->
[NodeA1, _, _] = ?config(cluster_a, Config),
[NodeB1, _] = ?config(cluster_b, Config),
LPortA = ?config(lport_a, Config),
LPortB = ?config(lport_b, Config),
NameA = ?config(name_a, Config),
NameB = ?config(name_b, Config),
ClientA = start_client("t_config_a", NodeA1),
ClientB = start_client("t_config_b", NodeB1),
{ok, _, _} = emqtt:subscribe(ClientA, <<"t/test/1/+">>, qos1),
{ok, _, _} = emqtt:subscribe(ClientB, <<"t/test-topic">>, qos1),
%% add link
LinkConfA = #{
<<"enable">> => true,
<<"pool_size">> => 1,
<<"server">> => <<"localhost:", (integer_to_binary(LPortB))/binary>>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => NameB
},
LinkConfB = #{
<<"enable">> => true,
<<"pool_size">> => 1,
<<"server">> => <<"localhost:", (integer_to_binary(LPortA))/binary>>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => NameA
},
{ok, SubRef} = snabbkaffe:subscribe(
?match_event(#{?snk_kind := clink_route_bootstrap_complete}),
%% 5 nodes = 5 actors (durable storage is dsabled)
5,
30_000
),
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA]])),
?assertMatch({ok, _}, erpc:call(NodeB1, emqx_cluster_link_config, update, [[LinkConfB]])),
?assertMatch(
{ok, [
#{?snk_kind := clink_route_bootstrap_complete},
#{?snk_kind := clink_route_bootstrap_complete},
#{?snk_kind := clink_route_bootstrap_complete},
#{?snk_kind := clink_route_bootstrap_complete},
#{?snk_kind := clink_route_bootstrap_complete}
]},
snabbkaffe:receive_events(SubRef)
),
{ok, _} = emqtt:publish(ClientA, <<"t/test-topic">>, <<"hello-from-a">>, qos1),
{ok, _} = emqtt:publish(ClientB, <<"t/test/1/1">>, <<"hello-from-b">>, qos1),
?assertReceive(
{publish, #{
topic := <<"t/test-topic">>, payload := <<"hello-from-a">>, client_pid := ClientB
}},
7000
),
?assertReceive(
{publish, #{
topic := <<"t/test/1/1">>, payload := <<"hello-from-b">>, client_pid := ClientA
}},
7000
),
%% no more messages expected
?assertNotReceive({publish, _Message = #{}}),
{ok, SubRef1} = snabbkaffe:subscribe(
?match_event(#{?snk_kind := clink_route_bootstrap_complete}),
%% 3 nodes in cluster a
3,
30_000
),
%% update link
LinkConfA1 = LinkConfA#{<<"pool_size">> => 2, <<"topics">> => [<<"t/new/+">>]},
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA1]])),
?assertMatch(
{ok, [
#{?snk_kind := clink_route_bootstrap_complete},
#{?snk_kind := clink_route_bootstrap_complete},
#{?snk_kind := clink_route_bootstrap_complete}
]},
snabbkaffe:receive_events(SubRef1)
),
%% wait for route sync on ClientA node
{{ok, _, _}, {ok, _}} = ?wait_async_action(
emqtt:subscribe(ClientA, <<"t/new/1">>, qos1),
#{?snk_kind := clink_route_sync_complete, ?snk_meta := #{node := NodeA1}},
10_000
),
%% not expected to be received anymore
{ok, _} = emqtt:publish(ClientB, <<"t/test/1/1">>, <<"not-expected-hello-from-b">>, qos1),
{ok, _} = emqtt:publish(ClientB, <<"t/new/1">>, <<"hello-from-b-1">>, qos1),
?assertReceive(
{publish, #{topic := <<"t/new/1">>, payload := <<"hello-from-b-1">>, client_pid := ClientA}},
7000
),
?assertNotReceive({publish, _Message = #{}}),
%% disable link
LinkConfA2 = LinkConfA1#{<<"enable">> => false},
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA2]])),
%% must be already blocked by the receiving cluster even if externak routing state is not
%% updated yet
{ok, _} = emqtt:publish(ClientB, <<"t/new/1">>, <<"not-expected-hello-from-b-1">>, qos1),
LinkConfB1 = LinkConfB#{<<"enable">> => false},
?assertMatch({ok, _}, erpc:call(NodeB1, emqx_cluster_link_config, update, [[LinkConfB1]])),
{ok, _} = emqtt:publish(ClientA, <<"t/test-topic">>, <<"not-expected-hello-from-a">>, qos1),
?assertNotReceive({publish, _Message = #{}}, 3000),
%% delete links
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[]])),
?assertMatch({ok, _}, erpc:call(NodeB1, emqx_cluster_link_config, update, [[]])),
ok = emqtt:stop(ClientA),
ok = emqtt:stop(ClientB).
t_config_validations('init', Config) ->
NameA = fmt("~s_~s", [?FUNCTION_NAME, "a"]),
NameB = fmt("~s_~s", [?FUNCTION_NAME, "b"]),
LPortA = 31883,
LPortB = 41883,
ConfA = combine([conf_cluster(NameA), conf_log()]),
ConfB = combine([conf_cluster(NameB), conf_log()]),
%% Single node clusters are enough for a basic validation test
{[NodeA, _, _], [NodeB, _]} = mk_clusters(NameA, NameB, LPortA, LPortB, ConfA, ConfB, Config),
ClusterA = emqx_cth_cluster:start([NodeA]),
ClusterB = emqx_cth_cluster:start([NodeB]),
ok = snabbkaffe:start_trace(),
[
{cluster_a, ClusterA},
{cluster_b, ClusterB},
{lport_a, LPortA},
{lport_b, LPortB},
{name_a, NameA},
{name_b, NameB}
| Config
];
t_config_validations('end', Config) ->
ok = snabbkaffe:stop(),
ok = emqx_cth_cluster:stop(?config(cluster_a, Config)),
ok = emqx_cth_cluster:stop(?config(cluster_b, Config)).
t_config_validations(Config) ->
[NodeA] = ?config(cluster_a, Config),
LPortB = ?config(lport_b, Config),
NameB = ?config(name_b, Config),
LinkConfA = #{
<<"enable">> => true,
<<"pool_size">> => 1,
<<"server">> => <<"localhost:", (integer_to_binary(LPortB))/binary>>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => NameB
},
DuplicatedLinks = [LinkConfA, LinkConfA#{<<"enable">> => false, <<"pool_size">> => 2}],
?assertMatch(
{error, #{reason := #{reason := duplicated_cluster_links, names := _}}},
erpc:call(NodeA, emqx_cluster_link_config, update, [DuplicatedLinks])
),
InvalidTopics = [<<"t/test/#">>, <<"$LINK/cluster/test/#">>],
InvalidTopics1 = [<<"t/+/#/+">>, <<>>],
?assertMatch(
{error, #{reason := #{reason := invalid_topics, topics := _}}},
erpc:call(NodeA, emqx_cluster_link_config, update, [
[LinkConfA#{<<"topics">> => InvalidTopics}]
])
),
?assertMatch(
{error, #{reason := #{reason := invalid_topics, topics := _}}},
erpc:call(NodeA, emqx_cluster_link_config, update, [
[LinkConfA#{<<"topics">> => InvalidTopics1}]
])
),
?assertMatch(
{error, #{reason := required_field}},
erpc:call(NodeA, emqx_cluster_link_config, update, [
[maps:remove(<<"name">>, LinkConfA)]
])
),
?assertMatch(
{error, #{reason := required_field}},
erpc:call(NodeA, emqx_cluster_link_config, update, [[maps:remove(<<"server">>, LinkConfA)]])
),
?assertMatch(
{error, #{reason := required_field}},
erpc:call(NodeA, emqx_cluster_link_config, update, [[maps:remove(<<"topics">>, LinkConfA)]])
),
%% Some valid changes to cover different update scenarios (msg resource changed, actor changed, both changed)
?assertMatch(
{ok, _},
erpc:call(NodeA, emqx_cluster_link_config, update, [[LinkConfA]])
),
LinkConfUnknown = LinkConfA#{
<<"name">> => <<"no-cluster">>, <<"server">> => <<"no-cluster.emqx:31883">>
},
?assertMatch(
{ok, _},
erpc:call(NodeA, emqx_cluster_link_config, update, [
[LinkConfA#{<<"pool_size">> => 5}, LinkConfUnknown]
])
),
?assertMatch(
{ok, _},
erpc:call(NodeA, emqx_cluster_link_config, update, [
[LinkConfA, LinkConfUnknown#{<<"topics">> => []}]
])
),
?assertMatch(
{ok, _},
erpc:call(
NodeA,
emqx_cluster_link_config,
update,
[
[
LinkConfA#{
<<"clientid">> => <<"new-client">>,
<<"username">> => <<"user">>
},
LinkConfUnknown#{
<<"clientid">> => <<"new-client">>,
<<"username">> => <<"user">>
}
]
]
)
).
t_config_update_ds('init', Config) ->
NameA = fmt("~s_~s", [?FUNCTION_NAME, "a"]),
NameB = fmt("~s_~s", [?FUNCTION_NAME, "b"]),
LPortA = 31883,
LPortB = 41883,
ConfA = combine([conf_cluster(NameA), conf_log(), conf_ds()]),
ConfB = combine([conf_cluster(NameB), conf_log(), conf_ds()]),
{NodesA, NodesB} = mk_clusters(NameA, NameB, LPortA, LPortB, ConfA, ConfB, Config),
ClusterA = emqx_cth_cluster:start(NodesA),
ClusterB = emqx_cth_cluster:start(NodesB),
ok = snabbkaffe:start_trace(),
[
{cluster_a, ClusterA},
{cluster_b, ClusterB},
{lport_a, LPortA},
{lport_b, LPortB},
{name_a, NameA},
{name_b, NameB}
| Config
];
t_config_update_ds('end', Config) ->
ok = snabbkaffe:stop(),
ok = emqx_cth_cluster:stop(?config(cluster_a, Config)),
ok = emqx_cth_cluster:stop(?config(cluster_b, Config)).
t_config_update_ds(Config) ->
[NodeA1, _, _] = ?config(cluster_a, Config),
[NodeB1, _] = ?config(cluster_b, Config),
LPortA = ?config(lport_a, Config),
LPortB = ?config(lport_b, Config),
NameA = ?config(name_a, Config),
NameB = ?config(name_b, Config),
ClientA = start_client("t_config_a", NodeA1, false),
ClientB = start_client("t_config_b", NodeB1, false),
{ok, _, _} = emqtt:subscribe(ClientA, <<"t/test/1/+">>, qos1),
{ok, _, _} = emqtt:subscribe(ClientB, <<"t/test-topic">>, qos1),
LinkConfA = #{
<<"enable">> => true,
<<"pool_size">> => 1,
<<"server">> => <<"localhost:", (integer_to_binary(LPortB))/binary>>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => NameB
},
LinkConfB = #{
<<"enable">> => true,
<<"pool_size">> => 1,
<<"server">> => <<"localhost:", (integer_to_binary(LPortA))/binary>>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => NameA
},
{ok, SubRef} = snabbkaffe:subscribe(
?match_event(#{?snk_kind := clink_route_bootstrap_complete}),
%% 5 nodes = 9 actors (durable storage is enabled,
%% 1 replicant node is not doing ds bootstrap)
9,
30_000
),
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA]])),
?assertMatch({ok, _}, erpc:call(NodeB1, emqx_cluster_link_config, update, [[LinkConfB]])),
?assertMatch(
[#{ps_actor_incarnation := 0}], erpc:call(NodeA1, emqx, get_config, [[cluster, links]])
),
?assertMatch(
[#{ps_actor_incarnation := 0}], erpc:call(NodeB1, emqx, get_config, [[cluster, links]])
),
{ok, Events} = snabbkaffe:receive_events(SubRef),
?assertEqual(9, length(Events)),
{ok, _} = emqtt:publish(ClientA, <<"t/test-topic">>, <<"hello-from-a">>, qos1),
{ok, _} = emqtt:publish(ClientB, <<"t/test/1/1">>, <<"hello-from-b">>, qos1),
?assertReceive(
{publish, #{
topic := <<"t/test-topic">>, payload := <<"hello-from-a">>, client_pid := ClientB
}},
30_000
),
?assertReceive(
{publish, #{
topic := <<"t/test/1/1">>, payload := <<"hello-from-b">>, client_pid := ClientA
}},
30_000
),
%% no more messages expected
?assertNotReceive({publish, _Message = #{}}),
{ok, SubRef1} = snabbkaffe:subscribe(
?match_event(#{?snk_kind := clink_route_bootstrap_complete}),
%% 3 nodes (1 replicant) in cluster a (5 actors including ds)
5,
30_000
),
%% update link
LinkConfA1 = LinkConfA#{<<"pool_size">> => 2, <<"topics">> => [<<"t/new/+">>]},
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA1]])),
{ok, Events1} = snabbkaffe:receive_events(SubRef1),
?assertEqual(5, length(Events1)),
%% wait for route sync on ClientA node
{{ok, _, _}, {ok, _}} = ?wait_async_action(
emqtt:subscribe(ClientA, <<"t/new/1">>, qos1),
#{
?snk_kind := clink_route_sync_complete,
?snk_meta := #{node := NodeA1},
actor := {<<"ps-routes-v1">>, 1}
},
10_000
),
%% not expected to be received anymore
{ok, _} = emqtt:publish(ClientB, <<"t/test/1/1">>, <<"not-expected-hello-from-b">>, qos1),
{ok, _} = emqtt:publish(ClientB, <<"t/new/1">>, <<"hello-from-b-1">>, qos1),
?assertReceive(
{publish, #{topic := <<"t/new/1">>, payload := <<"hello-from-b-1">>, client_pid := ClientA}},
30_000
),
?assertNotReceive({publish, _Message = #{}}),
?assertMatch(
[#{ps_actor_incarnation := 1}], erpc:call(NodeA1, emqx, get_config, [[cluster, links]])
),
?assertMatch(
[#{ps_actor_incarnation := 1}], erpc:call(NodeA1, emqx, get_config, [[cluster, links]])
),
ok = emqtt:stop(ClientA),
ok = emqtt:stop(ClientB).
t_misconfigured_links('init', Config) ->
NameA = fmt("~s_~s", [?FUNCTION_NAME, "a"]),
NameB = fmt("~s_~s", [?FUNCTION_NAME, "b"]),
LPortA = 31883,
LPortB = 41883,
ConfA = combine([conf_cluster(NameA), conf_log()]),
ConfB = combine([conf_cluster(NameB), conf_log()]),
{NodesA, NodesB} = mk_clusters(NameA, NameB, LPortA, LPortB, ConfA, ConfB, Config),
ClusterA = emqx_cth_cluster:start(NodesA),
ClusterB = emqx_cth_cluster:start(NodesB),
ok = snabbkaffe:start_trace(),
[
{cluster_a, ClusterA},
{cluster_b, ClusterB},
{lport_a, LPortA},
{lport_b, LPortB},
{name_a, NameA},
{name_b, NameB}
| Config
];
t_misconfigured_links('end', Config) ->
ok = snabbkaffe:stop(),
ok = emqx_cth_cluster:stop(?config(cluster_a, Config)),
ok = emqx_cth_cluster:stop(?config(cluster_b, Config)).
t_misconfigured_links(Config) ->
[NodeA1, _, _] = ?config(cluster_a, Config),
[NodeB1, _] = ?config(cluster_b, Config),
LPortA = ?config(lport_a, Config),
LPortB = ?config(lport_b, Config),
NameA = ?config(name_a, Config),
NameB = ?config(name_b, Config),
ClientA = start_client("t_config_a", NodeA1),
ClientB = start_client("t_config_b", NodeB1),
{ok, _, _} = emqtt:subscribe(ClientA, <<"t/test/1/+">>, qos1),
{ok, _, _} = emqtt:subscribe(ClientB, <<"t/test-topic">>, qos1),
LinkConfA = #{
<<"enable">> => true,
<<"pool_size">> => 1,
<<"server">> => <<"localhost:", (integer_to_binary(LPortB))/binary>>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => <<"bad-b-name">>
},
LinkConfB = #{
<<"enable">> => true,
<<"pool_size">> => 1,
<<"server">> => <<"localhost:", (integer_to_binary(LPortA))/binary>>,
<<"topics">> => [<<"t/test-topic">>, <<"t/test/#">>],
<<"name">> => NameA
},
?assertMatch({ok, _}, erpc:call(NodeB1, emqx_cluster_link_config, update, [[LinkConfB]])),
{{ok, _}, {ok, _}} = ?wait_async_action(
erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA]]),
#{
?snk_kind := clink_handshake_error,
reason := <<"bad_remote_cluster_link_name">>,
?snk_meta := #{node := NodeA1}
},
10_000
),
timer:sleep(10),
?assertMatch(
#{error := <<"bad_remote_cluster_link_name">>},
erpc:call(NodeA1, emqx_cluster_link_router_syncer, status, [<<"bad-b-name">>])
),
{{ok, _}, {ok, _}} = ?wait_async_action(
erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA#{<<"name">> => NameB}]]),
#{
?snk_kind := clink_route_bootstrap_complete,
?snk_meta := #{node := NodeA1}
},
10_000
),
?assertMatch(
#{status := connected, error := undefined},
erpc:call(NodeA1, emqx_cluster_link_router_syncer, status, [NameB])
),
?assertEqual(
undefined, erpc:call(NodeA1, emqx_cluster_link_router_syncer, status, [<<"bad-b-name">>])
),
?assertMatch(
{ok, _},
erpc:call(
NodeB1,
emqx_cluster_link_config,
update,
[
[
LinkConfB#{<<"enable">> => false},
%% An extra dummy link to keep B hook/external_broker registered and be able to
%% respond with "link disabled error" for the first disabled link
LinkConfB#{<<"name">> => <<"bad-a-name">>}
]
]
)
),
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[]])),
{{ok, _}, {ok, _}} = ?wait_async_action(
erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA#{<<"name">> => NameB}]]),
#{
?snk_kind := clink_handshake_error,
reason := <<"cluster_link_disabled">>,
?snk_meta := #{node := NodeA1}
},
10_000
),
timer:sleep(10),
?assertMatch(
#{error := <<"cluster_link_disabled">>},
erpc:call(NodeA1, emqx_cluster_link_router_syncer, status, [NameB])
),
?assertMatch(
{ok, _},
erpc:call(NodeB1, emqx_cluster_link_config, update, [
[LinkConfB#{<<"name">> => <<"bad-a-name">>}]
])
),
?assertMatch({ok, _}, erpc:call(NodeA1, emqx_cluster_link_config, update, [[]])),
{{ok, _}, {ok, _}} = ?wait_async_action(
erpc:call(NodeA1, emqx_cluster_link_config, update, [[LinkConfA#{<<"name">> => NameB}]]),
#{
?snk_kind := clink_handshake_error,
reason := <<"unknown_cluster">>,
?snk_meta := #{node := NodeA1}
},
10_000
),
timer:sleep(10),
?assertMatch(
#{error := <<"unknown_cluster">>},
erpc:call(NodeA1, emqx_cluster_link_router_syncer, status, [NameB])
),
ok = emqtt:stop(ClientA),
ok = emqtt:stop(ClientB).
start_client(ClientId, Node) ->
start_client(ClientId, Node, true).
start_client(ClientId, Node, CleanStart) ->
Port = tcp_port(Node),
{ok, Client} = emqtt:start_link(
[
{proto_ver, v5},
{clientid, ClientId},
{port, Port},
{clean_start, CleanStart}
| [{properties, #{'Session-Expiry-Interval' => 300}} || CleanStart =:= false]
]
),
{ok, _} = emqtt:connect(Client),
Client.
tcp_port(Node) ->
{_Host, Port} = erpc:call(Node, emqx_config, get, [[listeners, tcp, default, bind]]),
Port.
combine([Entry | Rest]) ->
lists:foldl(fun emqx_cth_suite:merge_config/2, Entry, Rest).
conf_mqtt_listener(LPort) when is_integer(LPort) ->
fmt("listeners.tcp.clink { bind = ~p }", [LPort]);
conf_mqtt_listener(_) ->
"".
conf_cluster(ClusterName) ->
fmt("cluster.name = ~s", [ClusterName]).
conf_log() ->
"log.file { enable = true, level = debug, path = node.log, supervisor_reports = progress }".
conf_ds() ->
"durable_sessions.enable = true".
fmt(Fmt, Args) ->
emqx_utils:format(Fmt, Args).
mk_nodename(BaseName, Idx) ->
binary_to_atom(fmt("emqx_clink_~s_~b", [BaseName, Idx])).

View File

@ -0,0 +1,322 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2024 EMQ Technologies Co., Ltd. All Rights Reserved.
%%--------------------------------------------------------------------
-module(emqx_cluster_link_extrouter_SUITE).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("emqx/include/asserts.hrl").
-include_lib("emqx/include/emqx.hrl").
-compile(export_all).
-compile(nowarn_export_all).
-define(CLUSTER, <<"link1">>).
%%
all() ->
emqx_common_test_helpers:all(?MODULE).
init_per_suite(Config) ->
Apps = emqx_cth_suite:start([], #{work_dir => emqx_cth_suite:work_dir(Config)}),
ok = init_db(),
[{apps, Apps} | Config].
end_per_suite(Config) ->
ok = emqx_cth_suite:stop(?config(apps, Config)).
init_per_testcase(TC, Config) ->
emqx_common_test_helpers:init_per_testcase(?MODULE, TC, Config).
end_per_testcase(TC, Config) ->
emqx_common_test_helpers:end_per_testcase(?MODULE, TC, Config).
init_db() ->
mria:wait_for_tables(emqx_cluster_link_extrouter:create_tables()).
init_db_nodes(Nodes) ->
ok = lists:foreach(
fun(Node) -> ok = erpc:call(Node, ?MODULE, init_db, []) end,
Nodes
).
%%
t_consistent_routing_view(_Config) ->
Actor1 = {?FUNCTION_NAME, 1},
Actor2 = {?FUNCTION_NAME, 2},
Actor3 = {?FUNCTION_NAME, 3},
{ok, AS10} = actor_init(Actor1, 1),
{ok, AS20} = actor_init(Actor2, 1),
{ok, AS30} = actor_init(Actor3, 1),
%% Add few routes originating from different actors.
%% Also test that route operations are idempotent.
AS11 = apply_operation({add, {<<"t/client/#">>, id}}, AS10),
_AS11 = apply_operation({add, {<<"t/client/#">>, id}}, AS10),
AS21 = apply_operation({add, {<<"t/client/#">>, id}}, AS20),
AS31 = apply_operation({add, {<<"t/client/+/+">>, id1}}, AS30),
AS32 = apply_operation({add, {<<"t/client/+/+">>, id2}}, AS31),
_AS22 = apply_operation({delete, {<<"t/client/#">>, id}}, AS21),
AS12 = apply_operation({add, {<<"t/client/+/+">>, id1}}, AS11),
AS33 = apply_operation({delete, {<<"t/client/+/+">>, id1}}, AS32),
_AS34 = apply_operation({delete, {<<"t/client/+/+">>, id2}}, AS33),
?assertEqual(
[<<"t/client/#">>, <<"t/client/+/+">>],
topics_sorted()
),
?assertEqual(
[#route{topic = <<"t/client/#">>, dest = ?CLUSTER}],
emqx_cluster_link_extrouter:match_routes(<<"t/client/42">>)
),
%% Remove all routes from the actors.
AS13 = apply_operation({delete, {<<"t/client/#">>, id}}, AS12),
AS14 = apply_operation({delete, {<<"t/client/+/+">>, id1}}, AS13),
AS14 = apply_operation({delete, {<<"t/client/+/+">>, id1}}, AS13),
?assertEqual(
[],
topics_sorted()
).
t_actor_reincarnation(_Config) ->
Actor1 = {?FUNCTION_NAME, 1},
Actor2 = {?FUNCTION_NAME, 2},
{ok, AS10} = actor_init(Actor1, 1),
{ok, AS20} = actor_init(Actor2, 1),
AS11 = apply_operation({add, {<<"topic/#">>, id}}, AS10),
AS12 = apply_operation({add, {<<"topic/42/+">>, id}}, AS11),
AS21 = apply_operation({add, {<<"topic/#">>, id}}, AS20),
?assertEqual(
[<<"topic/#">>, <<"topic/42/+">>],
topics_sorted()
),
{ok, _AS3} = actor_init(Actor1, 2),
?assertError(
_IncarnationMismatch,
apply_operation({add, {<<"toolate/#">>, id}}, AS12)
),
?assertEqual(
[<<"topic/#">>],
topics_sorted()
),
{ok, _AS4} = actor_init(Actor2, 2),
?assertError(
_IncarnationMismatch,
apply_operation({add, {<<"toolate/#">>, id}}, AS21)
),
?assertEqual(
[],
topics_sorted()
).
t_actor_gc(_Config) ->
Actor1 = {?FUNCTION_NAME, 1},
Actor2 = {?FUNCTION_NAME, 2},
{ok, AS10} = actor_init(Actor1, 1),
{ok, AS20} = actor_init(Actor2, 1),
AS11 = apply_operation({add, {<<"topic/#">>, id}}, AS10),
AS12 = apply_operation({add, {<<"topic/42/+">>, id}}, AS11),
AS21 = apply_operation({add, {<<"global/#">>, id}}, AS20),
?assertEqual(
[<<"global/#">>, <<"topic/#">>, <<"topic/42/+">>],
topics_sorted()
),
_AS13 = apply_operation(heartbeat, AS12, 60_000),
?assertEqual(
1,
emqx_cluster_link_extrouter:actor_gc(env(60_000))
),
?assertEqual(
[<<"topic/#">>, <<"topic/42/+">>],
topics_sorted()
),
?assertError(
_IncarnationMismatch,
apply_operation({add, {<<"toolate/#">>, id}}, AS21)
),
?assertEqual(
1,
emqx_cluster_link_extrouter:actor_gc(env(120_000))
),
?assertEqual(
[],
topics_sorted()
).
t_consistent_routing_view_concurrent_updates(_Config) ->
A1Seq = repeat(10, [
reincarnate,
{add, {<<"t/client/#">>, id}},
{add, {<<"t/client/+/+">>, id1}},
{add, {<<"t/client/+/+">>, id1}},
{delete, {<<"t/client/#">>, id}}
]),
A2Seq = repeat(10, [
{add, {<<"global/#">>, id}},
{add, {<<"t/client/+/+">>, id1}},
{add, {<<"t/client/+/+">>, id2}},
{delete, {<<"t/client/+/+">>, id1}},
heartbeat
]),
A3Seq = repeat(10, [
{add, {<<"global/#">>, id}},
{delete, {<<"global/#">>, id}},
{add, {<<"t/client/+/+">>, id1}},
{delete, {<<"t/client/+/+">>, id1}},
{add, {<<"t/client/+/+">>, id2}},
{delete, {<<"t/client/+/+">>, id2}},
reincarnate
]),
A4Seq = repeat(10, [
gc,
{sleep, 1}
]),
_ = emqx_utils:pmap(
fun run_actor/1,
[
{{?FUNCTION_NAME, 1}, A1Seq},
{{?FUNCTION_NAME, 2}, A2Seq},
{{?FUNCTION_NAME, 3}, A3Seq},
{{?FUNCTION_NAME, gc}, A4Seq}
],
infinity
),
?assertEqual(
[<<"global/#">>, <<"t/client/+/+">>, <<"t/client/+/+">>],
topics_sorted()
).
t_consistent_routing_view_concurrent_cluster_updates('init', Config) ->
Specs = [
{emqx_cluster_link_extrouter1, #{role => core}},
{emqx_cluster_link_extrouter2, #{role => core}},
{emqx_cluster_link_extrouter3, #{role => core}}
],
Cluster = emqx_cth_cluster:start(
Specs,
#{work_dir => emqx_cth_suite:work_dir(?FUNCTION_NAME, Config)}
),
ok = init_db_nodes(Cluster),
[{cluster, Cluster} | Config];
t_consistent_routing_view_concurrent_cluster_updates('end', Config) ->
ok = emqx_cth_cluster:stop(?config(cluster, Config)).
t_consistent_routing_view_concurrent_cluster_updates(Config) ->
[N1, N2, N3] = ?config(cluster, Config),
A1Seq = repeat(10, [
reincarnate,
{add, {<<"t/client/#">>, id}},
{add, {<<"t/client/+/+">>, id1}},
{add, {<<"t/client/+/+">>, id1}},
{delete, {<<"t/client/#">>, id}}
]),
A2Seq = repeat(10, [
{add, {<<"global/#">>, id}},
{add, {<<"t/client/+/+">>, id1}},
{add, {<<"t/client/+/+">>, id2}},
{delete, {<<"t/client/+/+">>, id1}},
heartbeat
]),
A3Seq = repeat(10, [
{add, {<<"global/#">>, id}},
{delete, {<<"global/#">>, id}},
{add, {<<"t/client/+/+">>, id1}},
{delete, {<<"t/client/+/+">>, id1}},
{add, {<<"t/client/+/+">>, id2}},
{delete, {<<"t/client/+/+">>, id2}},
reincarnate
]),
A4Seq = repeat(10, [
gc,
{sleep, 1}
]),
Runners = lists:map(
fun run_remote_actor/1,
[
{N1, {{?FUNCTION_NAME, 1}, A1Seq}},
{N2, {{?FUNCTION_NAME, 2}, A2Seq}},
{N3, {{?FUNCTION_NAME, 3}, A3Seq}},
{N3, {{?FUNCTION_NAME, gc}, A4Seq}}
]
),
[?assertReceive({'DOWN', MRef, _, Pid, normal}) || {Pid, MRef} <- Runners],
?assertEqual(
[<<"global/#">>, <<"t/client/+/+">>, <<"t/client/+/+">>],
erpc:call(N1, ?MODULE, topics_sorted, [])
).
t_consistent_routing_view_concurrent_cluster_replicant_updates('init', Config) ->
Specs = [
{emqx_cluster_link_extrouter_repl1, #{role => core}},
{emqx_cluster_link_extrouter_repl2, #{role => core}},
{emqx_cluster_link_extrouter_repl3, #{role => replicant}}
],
Cluster = emqx_cth_cluster:start(
Specs,
#{work_dir => emqx_cth_suite:work_dir(?FUNCTION_NAME, Config)}
),
ok = init_db_nodes(Cluster),
[{cluster, Cluster} | Config];
t_consistent_routing_view_concurrent_cluster_replicant_updates('end', Config) ->
ok = emqx_cth_cluster:stop(?config(cluster, Config)).
t_consistent_routing_view_concurrent_cluster_replicant_updates(Config) ->
t_consistent_routing_view_concurrent_cluster_updates(Config).
run_remote_actor({Node, Run}) ->
erlang:spawn_monitor(Node, ?MODULE, run_actor, [Run]).
run_actor({Actor, Seq}) ->
{ok, AS0} = actor_init(Actor, 0),
lists:foldl(
fun
({TS, {add, _} = Op}, AS) ->
apply_operation(Op, AS, TS);
({TS, {delete, _} = Op}, AS) ->
apply_operation(Op, AS, TS);
({TS, heartbeat}, AS) ->
apply_operation(heartbeat, AS, TS);
({TS, gc}, AS) ->
_NC = emqx_cluster_link_extrouter:actor_gc(env(TS)),
AS;
({_TS, {sleep, MS}}, AS) ->
ok = timer:sleep(MS),
AS;
({TS, reincarnate}, _AS) ->
{ok, AS} = actor_init(Actor, TS, TS),
AS
end,
AS0,
lists:enumerate(Seq)
).
%%
actor_init(Actor, Incarnation) ->
actor_init(Actor, Incarnation, _TS = 0).
actor_init(Actor, Incarnation, TS) ->
emqx_cluster_link_extrouter:actor_init(?CLUSTER, Actor, Incarnation, env(TS)).
apply_operation(Op, AS) ->
apply_operation(Op, AS, _TS = 42).
apply_operation(Op, AS, TS) ->
emqx_cluster_link_extrouter:actor_apply_operation(Op, AS, env(TS)).
env() ->
env(42).
env(TS) ->
#{timestamp => TS}.
topics_sorted() ->
lists:sort(emqx_cluster_link_extrouter:topics()).
%%
repeat(N, L) ->
lists:flatten(lists:duplicate(N, L)).

View File

@ -74,7 +74,9 @@
(?CE_AUTHN_PROVIDER_SCHEMA_MODS ++ ?EE_AUTHN_PROVIDER_SCHEMA_MODS) (?CE_AUTHN_PROVIDER_SCHEMA_MODS ++ ?EE_AUTHN_PROVIDER_SCHEMA_MODS)
). ).
-define(OTHER_INJECTING_CONFIGS, ?AUTH_EXT_SCHEMA_MODS). -define(CLUSTER_LINKING_SCHEMA_MODS, [emqx_cluster_link_schema]).
-define(OTHER_INJECTING_CONFIGS, ?AUTH_EXT_SCHEMA_MODS ++ ?CLUSTER_LINKING_SCHEMA_MODS).
-else. -else.

View File

@ -82,7 +82,8 @@
connection_rejected_due_to_license_limit_reached, connection_rejected_due_to_license_limit_reached,
dropped_msg_due_to_mqueue_is_full, dropped_msg_due_to_mqueue_is_full,
socket_receive_paused_by_rate_limit, socket_receive_paused_by_rate_limit,
data_bridge_buffer_overflow data_bridge_buffer_overflow,
external_broker_crashed
]). ]).
%% Callback to upgrade config after loaded from config file but before validation. %% Callback to upgrade config after loaded from config file but before validation.
@ -272,7 +273,7 @@ fields("cluster") ->
importance => ?IMPORTANCE_HIDDEN importance => ?IMPORTANCE_HIDDEN
} }
)} )}
]; ] ++ emqx_schema_hooks:injection_point(cluster);
fields(cluster_static) -> fields(cluster_static) ->
[ [
{"seeds", {"seeds",

View File

@ -133,7 +133,8 @@
emqx_bridge_syskeeper, emqx_bridge_syskeeper,
emqx_bridge_confluent, emqx_bridge_confluent,
emqx_ds_shared_sub, emqx_ds_shared_sub,
emqx_auth_ext emqx_auth_ext,
emqx_cluster_link
], ],
%% must always be of type `load' %% must always be of type `load'
ce_business_apps => ce_business_apps =>

View File

@ -65,6 +65,7 @@
flattermap/2, flattermap/2,
tcp_keepalive_opts/4, tcp_keepalive_opts/4,
format/1, format/1,
format/2,
format_mfal/2, format_mfal/2,
call_first_defined/1, call_first_defined/1,
ntoa/1, ntoa/1,
@ -564,7 +565,10 @@ tcp_keepalive_opts(OS, _Idle, _Interval, _Probes) ->
{error, {unsupported_os, OS}}. {error, {unsupported_os, OS}}.
format(Term) -> format(Term) ->
iolist_to_binary(io_lib:format("~0p", [Term])). unicode:characters_to_binary(io_lib:format("~0p", [Term])).
format(Fmt, Args) ->
unicode:characters_to_binary(io_lib:format(Fmt, Args)).
%% @doc Helper function for log formatters. %% @doc Helper function for log formatters.
-spec format_mfal(map(), map()) -> undefined | binary(). -spec format_mfal(map(), map()) -> undefined | binary().

View File

@ -26,6 +26,8 @@
lookup_value/3 lookup_value/3
]). ]).
-export([keyfoldl/3]).
-export([delete/1]). -export([delete/1]).
%% Create an ets table. %% Create an ets table.
@ -57,6 +59,24 @@ lookup_value(Tab, Key, Def) ->
error:badarg -> Def error:badarg -> Def
end. end.
-spec keyfoldl(fun((_Key :: term(), Acc) -> Acc), Acc, ets:tab()) -> Acc.
keyfoldl(F, Acc, Tab) ->
true = ets:safe_fixtable(Tab, true),
First = ets:first(Tab),
try
keyfoldl(F, Acc, First, Tab)
after
ets:safe_fixtable(Tab, false)
end.
keyfoldl(F, Acc, Key, Tab) ->
case Key of
'$end_of_table' ->
Acc;
_ ->
keyfoldl(F, F(Key, Acc), ets:next(Tab, Key), Tab)
end.
%% Delete the ets table. %% Delete the ets table.
-spec delete(ets:tab()) -> ok. -spec delete(ets:tab()) -> ok.
delete(Tab) -> delete(Tab) ->

View File

@ -204,7 +204,8 @@ defmodule EMQXUmbrella.MixProject do
:emqx_gateway_jt808, :emqx_gateway_jt808,
:emqx_bridge_syskeeper, :emqx_bridge_syskeeper,
:emqx_ds_shared_sub, :emqx_ds_shared_sub,
:emqx_auth_ext :emqx_auth_ext,
:emqx_cluster_link
]) ])
end end

View File

@ -123,6 +123,7 @@ is_community_umbrella_app("apps/emqx_eviction_agent") -> false;
is_community_umbrella_app("apps/emqx_node_rebalance") -> false; is_community_umbrella_app("apps/emqx_node_rebalance") -> false;
is_community_umbrella_app("apps/emqx_ds_shared_sub") -> false; is_community_umbrella_app("apps/emqx_ds_shared_sub") -> false;
is_community_umbrella_app("apps/emqx_auth_ext") -> false; is_community_umbrella_app("apps/emqx_auth_ext") -> false;
is_community_umbrella_app("apps/emqx_cluster_link") -> false;
is_community_umbrella_app(_) -> true. is_community_umbrella_app(_) -> true.
%% BUILD_WITHOUT_JQ %% BUILD_WITHOUT_JQ

View File

@ -0,0 +1,53 @@
emqx_cluster_link_schema {
links.desc:
"""The list of the linked EMQX clusters."""
links.label: "Cluster Links"
link.desc:
"""Cluster link configuration"""
link.label: "Cluster Link"
enable.desc:
"""Enable or disable a cluster link. The link is enabled by default, disabling it allows stopping the link without removing its configuration. The link must be enabled on both sides to be operational. Disabling the link should also be done on both clusters in order to free up all associated resources."""
enable.label: "Enable"
link_name.desc:
"""Linked (remote) cluster name. Must be exactly equal to the value of `cluster.name` configured at the remote cluster. Must not be equal to the local cluster.name. All configured cluster link names must be unique."""
link_name.label: "Linked Cluster Name"
server.desc:
"""MQTT host and port of the remote EMQX broker."""
server.label: "MQTT Server"
username.desc:
"""Optional MQTT username for connecting to the remote EMQX cluster."""
username.label: "Username"
password.desc:
"""Optional MQTT username for connecting to the remote EMQX cluster."""
password.label: "Password"
clientid.desc:
"""Optional Base MQTT client ID for connecting to the remote EMQX cluster. If omitted, local `cluster.name` is used. EMQX maintains several connections between linked clusters, so distinct suffixes are automatically appended to the base client ID."""
clientid.label: "Base Client ID"
ssl.desc: """SSL configuration for connecting to the remote EMQX cluster."""
ssl.label: "SSL Options"
topics.desc: """MQTT topics to be forwarded by the linked remote EMQX broker to the local broker. Messages are only forwarded if the local EMQX broker has matching subscriber(s).
Wildcards are supported. Setting empty topics list on one side of the link can be used to establish unidirectional links: the side with the empty topics won't receive remote messages, but it can forward relevant messages to its linked counterpart (according to the topics configured on that side of the link)."""
topics.label: "Topics"
pool_size.desc:
"""Size of the pool of MQTT clients that will publish messages to the linked EMQX broker."""
pool_size.label:
"""Connection Pool Size"""
max_inflight.desc:
"""Max inflight (sent, but un-acked) messages of the MQTT protocol"""
max_inflight.label:
"""Max Inflight Message"""
}