Merge pull request #12768 from keynslug/fix/EMQX-12068/routing-schema-conflict

fix(routing): add heuristic for routing schema in empty cluster
This commit is contained in:
Zaiming (Stone) Shi 2024-03-25 19:50:51 +01:00 committed by GitHub
commit b06d3932af
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
5 changed files with 243 additions and 29 deletions

View File

@ -66,6 +66,7 @@
{emqx_resource,2}.
{emqx_retainer,1}.
{emqx_retainer,2}.
{emqx_router,1}.
{emqx_rule_engine,1}.
{emqx_shared_sub,1}.
{emqx_slow_subs,1}.

View File

@ -92,9 +92,11 @@
]).
-export_type([dest/0]).
-export_type([schemavsn/0]).
-type group() :: binary().
-type dest() :: node() | {group(), node()}.
-type schemavsn() :: v1 | v2.
%% Operation :: {add, ...} | {delete, ...}.
-type batch() :: #{batch_route() => _Operation :: tuple()}.
@ -643,8 +645,8 @@ match_to_route(M) ->
-define(PT_SCHEMA_VSN, {?MODULE, schemavsn}).
-type schemavsn() :: v1 | v2.
%% @doc Get the schema version in use.
%% BPAPI RPC Target @ emqx_router_proto
-spec get_schema_vsn() -> schemavsn().
get_schema_vsn() ->
persistent_term:get(?PT_SCHEMA_VSN).
@ -654,23 +656,23 @@ init_schema() ->
ok = mria:wait_for_tables([?ROUTE_TAB, ?ROUTE_TAB_FILTERS]),
ok = emqx_trie:wait_for_tables(),
ConfSchema = emqx_config:get([broker, routing, storage_schema]),
Schema = choose_schema_vsn(ConfSchema),
{ClusterSchema, ClusterState} = discover_cluster_schema_vsn(),
Schema = choose_schema_vsn(ConfSchema, ClusterSchema, ClusterState),
ok = persistent_term:put(?PT_SCHEMA_VSN, Schema),
case Schema of
ConfSchema ->
case Schema =:= ConfSchema of
true ->
?SLOG(info, #{
msg => "routing_schema_used",
schema => Schema
});
_ ->
false ->
?SLOG(notice, #{
msg => "configured_routing_schema_ignored",
schema_in_use => Schema,
configured => ConfSchema,
reason =>
"Could not use configured routing storage schema because "
"there are already non-empty routing tables pertaining to "
"another schema."
"cluster is already running with a different schema."
})
end.
@ -679,34 +681,147 @@ deinit_schema() ->
_ = persistent_term:erase(?PT_SCHEMA_VSN),
ok.
-spec choose_schema_vsn(schemavsn()) -> schemavsn().
choose_schema_vsn(ConfType) ->
IsEmptyIndex = emqx_trie:empty(),
IsEmptyFilters = is_empty(?ROUTE_TAB_FILTERS),
case {IsEmptyIndex, IsEmptyFilters} of
{true, true} ->
ConfType;
{false, true} ->
v1;
{true, false} ->
v2;
{false, false} ->
?SLOG(critical, #{
msg => "conflicting_routing_schemas_detected_in_cluster",
configured => ConfType,
-spec discover_cluster_schema_vsn() ->
{schemavsn() | undefined, _State :: [{node(), schemavsn() | undefined, _Details}]}.
discover_cluster_schema_vsn() ->
discover_cluster_schema_vsn(emqx:running_nodes() -- [node()]).
-spec discover_cluster_schema_vsn([node()]) ->
{schemavsn() | undefined, _State :: [{node(), schemavsn() | undefined, _Details}]}.
discover_cluster_schema_vsn([]) ->
%% single node
{undefined, []};
discover_cluster_schema_vsn(Nodes) ->
Responses = lists:zipwith(
fun
(Node, {ok, Schema}) ->
{Node, Schema, configured};
(Node, {error, {exception, undef, _Stacktrace}}) ->
%% No such function on the remote node, assuming it doesn't know about v2 routing.
{Node, v1, legacy};
(Node, {error, {exception, badarg, _Stacktrace}}) ->
%% Likely, persistent term is not defined yet.
{Node, unknown, starting};
(Node, Error) ->
{Node, unknown, Error}
end,
Nodes,
emqx_router_proto_v1:get_routing_schema_vsn(Nodes)
),
case lists:usort([Vsn || {_Node, Vsn, _} <- Responses, Vsn /= unknown]) of
[Vsn] when Vsn =:= v1; Vsn =:= v2 ->
{Vsn, Responses};
[] ->
?SLOG(warning, #{
msg => "cluster_routing_schema_discovery_failed",
responses => Responses,
reason =>
"There are records in the routing tables related to both v1 "
"and v2 storage schemas. This probably means that some nodes "
"in the cluster use v1 schema and some use v2, independently "
"of each other. The routing is likely broken. Manual intervention "
"and full cluster restart is required. This node will shut down."
"Could not determine configured routing storage schema in peer nodes."
}),
{undefined, Responses};
[_ | _] ->
Desc = schema_conflict_reason(config, Responses),
io:format(standard_error, "Error: ~ts~n", [Desc]),
?SLOG(critical, #{
msg => "conflicting_routing_schemas_in_cluster",
responses => Responses,
description => Desc
}),
error(conflicting_routing_schemas_configured_in_cluster)
end.
-spec choose_schema_vsn(
schemavsn(),
_ClusterSchema :: schemavsn() | undefined,
_ClusterState :: [{node(), schemavsn() | undefined, _Details}]
) -> schemavsn().
choose_schema_vsn(ConfSchema, ClusterSchema, State) ->
case detect_table_schema_vsn() of
[] ->
%% No records in the tables, use schema configured in the cluster if any,
%% otherwise use configured.
emqx_maybe:define(ClusterSchema, ConfSchema);
[Schema] when Schema =:= ClusterSchema ->
%% Table contents match configured schema in the cluster.
Schema;
[Schema] when ClusterSchema =:= undefined ->
%% There are existing records following some schema, we have to use it.
Schema;
_Conflicting when ClusterSchema =/= undefined ->
%% There are existing records in both v1 and v2 schema,
%% we have to use what the peer nodes agreed on.
%% because it could be THIS node which caused the cnoflict.
%%
%% The stale records will be left-over, but harmless
Desc =
"Conflicting schema version detected for routing records, but "
"all the peer nodes are running the same version, so this node "
"will use the same schema but discard the harmless stale records. "
"This warning will go away after the next full cluster (non-rolling) restart.",
?SLOG(warning, #{
msg => "conflicting_routing_storage_detected",
resolved => ClusterSchema,
description => Desc
}),
ClusterSchema;
_Conflicting ->
Desc = schema_conflict_reason(records, State),
io:format(standard_error, "Error: ~ts~n", [Desc]),
?SLOG(critical, #{
msg => "conflicting_routing_storage_in_cluster",
description => Desc
}),
error(conflicting_routing_schemas_detected_in_cluster)
end.
schema_conflict_reason(Type, State) ->
Observe =
case Type of
config ->
"Peer nodes have route storage schema resolved into conflicting versions.\n";
records ->
"There are conflicting routing records found.\n"
end,
Cause =
"\nThis was caused by a race-condition when the cluster was rolling upgraded "
"from an older version to 5.4.0, 5.4.1, 5.5.0 or 5.5.1."
"\nThis node cannot boot before the conflicts are resolved.\n",
Observe ++ Cause ++ mk_conflict_resolution_action(State).
detect_table_schema_vsn() ->
lists:flatten([
[v1 || _NonEmptyTrieIndex = not emqx_trie:empty()],
[v2 || _NonEmptyFilterTab = not is_empty(?ROUTE_TAB_FILTERS)]
]).
is_empty(Tab) ->
ets:first(Tab) =:= '$end_of_table'.
mk_conflict_resolution_action(State) ->
NodesV1 = [Node || {Node, v1, _} <- State],
NodesUnknown = [Node || {Node, unknown, _} <- State],
Format =
"There are two ways to resolve the conflict:"
"\n"
"\nA: Full cluster restart: stop ALL running nodes one by one "
"and restart them in the reversed order."
"\n"
"\nB: Force v1 nodes to clean up their routes."
"\n Following EMQX nodes are running with v1 schema: ~0p."
"\n 1. Stop listeners with command \"emqx eval 'emqx_listener:stop()'\" in all v1 nodes"
"\n 2. Wait until they are safe to restart."
"\n This could take some time, depending on the number of clients and their subscriptions."
"\n Below conditions should be true for each of the nodes in order to proceed:"
"\n a) Command 'ets:info(emqx_subscriber, size)' prints `0`."
"\n b) Command 'emqx ctl topics list' prints No topics.`"
"\n 3. Upgrade the nodes to 5.6.0 or newer.",
FormatUnkown =
"Additionally, the following nodes were unreachable during startup: ~0p."
"It is strongly advised to include them in the manual resolution procedure as well.",
Message = io_lib:format(Format, [NodesV1]),
MessageUnknown = [io_lib:format(FormatUnkown, [NodesUnknown]) || NodesUnknown =/= []],
unicode:characters_to_list([Message, "\n", MessageUnknown]).
%%--------------------------------------------------------------------
%% gen_server callbacks
%%--------------------------------------------------------------------

View File

@ -0,0 +1,37 @@
%%--------------------------------------------------------------------
%% 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_router_proto_v1).
-behaviour(emqx_bpapi).
-export([introduced_in/0]).
-export([
get_routing_schema_vsn/1
]).
-include_lib("emqx/include/bpapi.hrl").
-define(TIMEOUT, 3_000).
introduced_in() ->
"5.6.0".
-spec get_routing_schema_vsn([node()]) ->
[emqx_rpc:erpc(emqx_router:schemavsn())].
get_routing_schema_vsn(Nodes) ->
erpc:multicall(Nodes, emqx_router, get_schema_vsn, [], ?TIMEOUT).

View File

@ -30,7 +30,8 @@ all() ->
{group, routing_schema_v1},
{group, routing_schema_v2},
t_routing_schema_switch_v1,
t_routing_schema_switch_v2
t_routing_schema_switch_v2,
t_routing_schema_consistent_clean_cluster
].
groups() ->
@ -477,6 +478,60 @@ t_routing_schema_switch(VFrom, VTo, WorkDir) ->
ok = emqx_cth_cluster:stop(Nodes)
end.
t_routing_schema_consistent_clean_cluster(Config) ->
WorkDir = emqx_cth_suite:work_dir(?FUNCTION_NAME, Config),
% Start first node with routing schema v1
[Node1] = emqx_cth_cluster:start(
[
{routing_schema_consistent1, #{
apps => [mk_genrpc_appspec(), mk_emqx_appspec(1, v1)]
}}
],
#{work_dir => WorkDir}
),
% Start rest of nodes with routing schema v2
NodesRest = emqx_cth_cluster:start(
[
{routing_schema_consistent2, #{
apps => [mk_genrpc_appspec(), mk_emqx_appspec(2, v2)],
base_port => 20000,
join_to => Node1
}},
{routing_schema_consistent3, #{
apps => [mk_genrpc_appspec(), mk_emqx_appspec(3, v2)],
base_port => 20100,
join_to => Node1
}}
],
#{work_dir => WorkDir}
),
Nodes = [Node1 | NodesRest],
try
% Verify that cluser is still on v1
?assertEqual(
[{ok, v1} || _ <- Nodes],
erpc:multicall(Nodes, emqx_router, get_schema_vsn, [])
),
% Wait for all nodes to agree on cluster state
?retry(
500,
10,
?assertEqual(
[{ok, Nodes} || _ <- Nodes],
erpc:multicall(Nodes, emqx, running_nodes, [])
)
),
C1 = start_client(Node1),
C2 = start_client(hd(NodesRest)),
ok = subscribe(C2, <<"t/#">>),
{ok, _} = publish(C1, <<"t/a/b/c">>, <<"yayconsistency">>),
?assertReceive({pub, C2, #{topic := <<"t/a/b/c">>, payload := <<"yayconsistency">>}}),
ok = emqtt:stop(C1),
ok = emqtt:stop(C2)
after
ok = emqx_cth_cluster:stop(Nodes)
end.
t_slow_rlog_routing_consistency(init, Config) ->
[Core1, _Core2, _Replicant] = ?config(cluster, Config),
MnesiaHook = rpc:call(Core1, persistent_term, get, [{mnesia_hook, post_commit}]),

View File

@ -0,0 +1,6 @@
Fixed an issue which may occur when performing rolling upgrade, especially when upgrading from a version earlier than 5.4.0.
When the cluster is empty (more precisely, routing tables are empty), try to additionally ask the cluster nodes for the routing schema in use, to make more informed decision about routing storage schema upon startup. This should make routing storage schema less likely to diverge across cluster nodes, especially when the cluster is composed of different versions of EMQX.
The version also logs instructions for how to manually resolve if conflict is detected in a running cluster.