test(kafka_consumer): add more clusterized tests
This commit is contained in:
parent
947e014132
commit
41b8d47696
|
@ -55,6 +55,8 @@ only_once_tests() ->
|
||||||
[
|
[
|
||||||
t_bridge_rule_action_source,
|
t_bridge_rule_action_source,
|
||||||
t_cluster_group,
|
t_cluster_group,
|
||||||
|
t_node_joins_existing_cluster,
|
||||||
|
t_cluster_node_down,
|
||||||
t_multiple_topic_mappings
|
t_multiple_topic_mappings
|
||||||
].
|
].
|
||||||
|
|
||||||
|
@ -924,12 +926,16 @@ action_response(Selected, Envs, Args) ->
|
||||||
}),
|
}),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
wait_until_group_is_balanced(KafkaTopic, NPartitions, Timeout) ->
|
wait_until_group_is_balanced(KafkaTopic, NPartitions, Nodes, Timeout) ->
|
||||||
do_wait_until_group_is_balanced(KafkaTopic, NPartitions, Timeout, #{}).
|
do_wait_until_group_is_balanced(KafkaTopic, NPartitions, Nodes, Timeout, #{}).
|
||||||
|
|
||||||
do_wait_until_group_is_balanced(KafkaTopic, NPartitions, Timeout, Acc0) ->
|
do_wait_until_group_is_balanced(KafkaTopic, NPartitions, Nodes, Timeout, Acc0) ->
|
||||||
case map_size(Acc0) =:= NPartitions of
|
AllPartitionsCovered = map_size(Acc0) =:= NPartitions,
|
||||||
|
PresentNodes = lists:usort([N || {_Partition, {N, _MemberId}} <- maps:to_list(Acc0)]),
|
||||||
|
AllNodesCovered = PresentNodes =:= lists:usort(Nodes),
|
||||||
|
case AllPartitionsCovered andalso AllNodesCovered of
|
||||||
true ->
|
true ->
|
||||||
|
ct:pal("group balanced: ~p", [Acc0]),
|
||||||
{ok, Acc0};
|
{ok, Acc0};
|
||||||
false ->
|
false ->
|
||||||
receive
|
receive
|
||||||
|
@ -942,7 +948,7 @@ do_wait_until_group_is_balanced(KafkaTopic, NPartitions, Timeout, Acc0) ->
|
||||||
topic_assignments => TopicAssignments
|
topic_assignments => TopicAssignments
|
||||||
},
|
},
|
||||||
Acc = reconstruct_assignments_from_events(KafkaTopic, [Event], Acc0),
|
Acc = reconstruct_assignments_from_events(KafkaTopic, [Event], Acc0),
|
||||||
do_wait_until_group_is_balanced(KafkaTopic, NPartitions, Timeout, Acc)
|
do_wait_until_group_is_balanced(KafkaTopic, NPartitions, Nodes, Timeout, Acc)
|
||||||
after Timeout ->
|
after Timeout ->
|
||||||
{timeout, Acc0}
|
{timeout, Acc0}
|
||||||
end
|
end
|
||||||
|
@ -974,6 +980,123 @@ reconstruct_assignments_from_events(KafkaTopic, Events0, Acc0) ->
|
||||||
Assignments
|
Assignments
|
||||||
).
|
).
|
||||||
|
|
||||||
|
setup_group_subscriber_spy(Node) ->
|
||||||
|
TestPid = self(),
|
||||||
|
ok = erpc:call(
|
||||||
|
Node,
|
||||||
|
fun() ->
|
||||||
|
ok = meck:new(brod_group_subscriber_v2, [
|
||||||
|
passthrough, no_link, no_history, non_strict
|
||||||
|
]),
|
||||||
|
ok = meck:expect(
|
||||||
|
brod_group_subscriber_v2,
|
||||||
|
assignments_received,
|
||||||
|
fun(Pid, MemberId, GenerationId, TopicAssignments) ->
|
||||||
|
?tp(
|
||||||
|
kafka_assignment,
|
||||||
|
#{
|
||||||
|
node => node(),
|
||||||
|
pid => Pid,
|
||||||
|
member_id => MemberId,
|
||||||
|
generation_id => GenerationId,
|
||||||
|
topic_assignments => TopicAssignments
|
||||||
|
}
|
||||||
|
),
|
||||||
|
TestPid !
|
||||||
|
{kafka_assignment, node(), {Pid, MemberId, GenerationId, TopicAssignments}},
|
||||||
|
meck:passthrough([Pid, MemberId, GenerationId, TopicAssignments])
|
||||||
|
end
|
||||||
|
),
|
||||||
|
ok
|
||||||
|
end
|
||||||
|
).
|
||||||
|
|
||||||
|
wait_for_cluster_rpc(Node) ->
|
||||||
|
%% need to wait until the config handler is ready after
|
||||||
|
%% restarting during the cluster join.
|
||||||
|
?retry(
|
||||||
|
_Sleep0 = 100,
|
||||||
|
_Attempts0 = 50,
|
||||||
|
true = is_pid(erpc:call(Node, erlang, whereis, [emqx_config_handler]))
|
||||||
|
).
|
||||||
|
|
||||||
|
setup_and_start_listeners(Node, NodeOpts) ->
|
||||||
|
erpc:call(
|
||||||
|
Node,
|
||||||
|
fun() ->
|
||||||
|
lists:foreach(
|
||||||
|
fun(Type) ->
|
||||||
|
Port = emqx_common_test_helpers:listener_port(NodeOpts, Type),
|
||||||
|
ok = emqx_config:put(
|
||||||
|
[listeners, Type, default, bind],
|
||||||
|
{{127, 0, 0, 1}, Port}
|
||||||
|
),
|
||||||
|
ok = emqx_config:put_raw(
|
||||||
|
[listeners, Type, default, bind],
|
||||||
|
iolist_to_binary([<<"127.0.0.1:">>, integer_to_binary(Port)])
|
||||||
|
),
|
||||||
|
ok
|
||||||
|
end,
|
||||||
|
[tcp, ssl, ws, wss]
|
||||||
|
),
|
||||||
|
ok = emqx_listeners:start(),
|
||||||
|
ok
|
||||||
|
end
|
||||||
|
).
|
||||||
|
|
||||||
|
cluster(Config) ->
|
||||||
|
PrivDataDir = ?config(priv_dir, Config),
|
||||||
|
Cluster = emqx_common_test_helpers:emqx_cluster(
|
||||||
|
[core, core],
|
||||||
|
[
|
||||||
|
{apps, [emqx_conf, emqx_bridge, emqx_rule_engine]},
|
||||||
|
{listener_ports, []},
|
||||||
|
{peer_mod, slave},
|
||||||
|
{priv_data_dir, PrivDataDir},
|
||||||
|
{load_schema, true},
|
||||||
|
{start_autocluster, true},
|
||||||
|
{schema_mod, emqx_ee_conf_schema},
|
||||||
|
{env_handler, fun
|
||||||
|
(emqx) ->
|
||||||
|
application:set_env(emqx, boot_modules, [broker, router]),
|
||||||
|
ok;
|
||||||
|
(emqx_conf) ->
|
||||||
|
ok;
|
||||||
|
(_) ->
|
||||||
|
ok
|
||||||
|
end}
|
||||||
|
]
|
||||||
|
),
|
||||||
|
ct:pal("cluster: ~p", [Cluster]),
|
||||||
|
Cluster.
|
||||||
|
|
||||||
|
start_async_publisher(Config, KafkaTopic) ->
|
||||||
|
TId = ets:new(kafka_payloads, [public, ordered_set]),
|
||||||
|
Loop = fun Go() ->
|
||||||
|
receive
|
||||||
|
stop -> ok
|
||||||
|
after 0 ->
|
||||||
|
Payload = emqx_guid:to_hexstr(emqx_guid:gen()),
|
||||||
|
publish(Config, KafkaTopic, [#{key => Payload, value => Payload}]),
|
||||||
|
ets:insert(TId, {Payload}),
|
||||||
|
timer:sleep(400),
|
||||||
|
Go()
|
||||||
|
end
|
||||||
|
end,
|
||||||
|
Pid = spawn_link(Loop),
|
||||||
|
{TId, Pid}.
|
||||||
|
|
||||||
|
stop_async_publisher(Pid) ->
|
||||||
|
MRef = monitor(process, Pid),
|
||||||
|
Pid ! stop,
|
||||||
|
receive
|
||||||
|
{'DOWN', MRef, process, Pid, _} ->
|
||||||
|
ok
|
||||||
|
after 1_000 ->
|
||||||
|
ct:fail("publisher didn't die")
|
||||||
|
end,
|
||||||
|
ok.
|
||||||
|
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
%% Testcases
|
%% Testcases
|
||||||
%%------------------------------------------------------------------------------
|
%%------------------------------------------------------------------------------
|
||||||
|
@ -1500,36 +1623,17 @@ t_bridge_rule_action_source(Config) ->
|
||||||
),
|
),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
|
%% checks that an existing cluster can be configured with a kafka
|
||||||
|
%% consumer bridge and that the consumers will distribute over the two
|
||||||
|
%% nodes.
|
||||||
t_cluster_group(Config) ->
|
t_cluster_group(Config) ->
|
||||||
ct:timetrap({seconds, 180}),
|
ct:timetrap({seconds, 150}),
|
||||||
TestPid = self(),
|
|
||||||
NPartitions = ?config(num_partitions, Config),
|
NPartitions = ?config(num_partitions, Config),
|
||||||
KafkaTopic = ?config(kafka_topic, Config),
|
KafkaTopic = ?config(kafka_topic, Config),
|
||||||
KafkaName = ?config(kafka_name, Config),
|
KafkaName = ?config(kafka_name, Config),
|
||||||
ResourceId = resource_id(Config),
|
ResourceId = resource_id(Config),
|
||||||
BridgeId = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE_BIN, KafkaName),
|
BridgeId = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE_BIN, KafkaName),
|
||||||
PrivDataDir = ?config(priv_dir, Config),
|
Cluster = cluster(Config),
|
||||||
Cluster = emqx_common_test_helpers:emqx_cluster(
|
|
||||||
[core, core],
|
|
||||||
[
|
|
||||||
{apps, [emqx_conf, emqx_bridge, emqx_rule_engine]},
|
|
||||||
{listener_ports, []},
|
|
||||||
{peer_mod, slave},
|
|
||||||
{priv_data_dir, PrivDataDir},
|
|
||||||
{load_schema, true},
|
|
||||||
{schema_mod, emqx_ee_conf_schema},
|
|
||||||
{env_handler, fun
|
|
||||||
(emqx) ->
|
|
||||||
application:set_env(emqx, boot_modules, []),
|
|
||||||
ok;
|
|
||||||
(emqx_conf) ->
|
|
||||||
ok;
|
|
||||||
(_) ->
|
|
||||||
ok
|
|
||||||
end}
|
|
||||||
]
|
|
||||||
),
|
|
||||||
ct:pal("cluster: ~p", [Cluster]),
|
|
||||||
?check_trace(
|
?check_trace(
|
||||||
begin
|
begin
|
||||||
Nodes =
|
Nodes =
|
||||||
|
@ -1540,47 +1644,19 @@ t_cluster_group(Config) ->
|
||||||
on_exit(fun() ->
|
on_exit(fun() ->
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
fun(N) ->
|
fun(N) ->
|
||||||
|
ct:pal("stopping ~p", [N]),
|
||||||
ok = emqx_common_test_helpers:stop_slave(N)
|
ok = emqx_common_test_helpers:stop_slave(N)
|
||||||
end,
|
end,
|
||||||
Nodes
|
Nodes
|
||||||
)
|
)
|
||||||
end),
|
end),
|
||||||
lists:foreach(
|
lists:foreach(fun setup_group_subscriber_spy/1, Nodes),
|
||||||
fun(N) ->
|
|
||||||
erpc:call(N, fun() ->
|
|
||||||
ok = meck:new(brod_group_subscriber_v2, [
|
|
||||||
passthrough, no_link, no_history, non_strict
|
|
||||||
]),
|
|
||||||
ok = meck:expect(
|
|
||||||
brod_group_subscriber_v2,
|
|
||||||
assignments_received,
|
|
||||||
fun(Pid, MemberId, GenerationId, TopicAssignments) ->
|
|
||||||
TestPid !
|
|
||||||
{kafka_assignment, node(),
|
|
||||||
{Pid, MemberId, GenerationId, TopicAssignments}},
|
|
||||||
?tp(
|
|
||||||
kafka_assignment,
|
|
||||||
#{
|
|
||||||
node => node(),
|
|
||||||
pid => Pid,
|
|
||||||
member_id => MemberId,
|
|
||||||
generation_id => GenerationId,
|
|
||||||
topic_assignments => TopicAssignments
|
|
||||||
}
|
|
||||||
),
|
|
||||||
meck:passthrough([Pid, MemberId, GenerationId, TopicAssignments])
|
|
||||||
end
|
|
||||||
),
|
|
||||||
ok
|
|
||||||
end)
|
|
||||||
end,
|
|
||||||
Nodes
|
|
||||||
),
|
|
||||||
{ok, SRef0} = snabbkaffe:subscribe(
|
{ok, SRef0} = snabbkaffe:subscribe(
|
||||||
?match_event(#{?snk_kind := kafka_consumer_subscriber_started}),
|
?match_event(#{?snk_kind := kafka_consumer_subscriber_started}),
|
||||||
length(Nodes),
|
length(Nodes),
|
||||||
15_000
|
15_000
|
||||||
),
|
),
|
||||||
|
wait_for_cluster_rpc(N2),
|
||||||
erpc:call(N2, fun() -> {ok, _} = create_bridge(Config) end),
|
erpc:call(N2, fun() -> {ok, _} = create_bridge(Config) end),
|
||||||
{ok, _} = snabbkaffe:receive_events(SRef0),
|
{ok, _} = snabbkaffe:receive_events(SRef0),
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
|
@ -1598,8 +1674,7 @@ t_cluster_group(Config) ->
|
||||||
%% sleep so that the two nodes have time to distribute the
|
%% sleep so that the two nodes have time to distribute the
|
||||||
%% subscribers, rather than just one node containing all
|
%% subscribers, rather than just one node containing all
|
||||||
%% of them.
|
%% of them.
|
||||||
ct:sleep(10_000),
|
{ok, _} = wait_until_group_is_balanced(KafkaTopic, NPartitions, Nodes, 30_000),
|
||||||
{ok, _} = wait_until_group_is_balanced(KafkaTopic, NPartitions, 30_000),
|
|
||||||
lists:foreach(
|
lists:foreach(
|
||||||
fun(N) ->
|
fun(N) ->
|
||||||
?assertEqual(
|
?assertEqual(
|
||||||
|
@ -1630,3 +1705,207 @@ t_cluster_group(Config) ->
|
||||||
end
|
end
|
||||||
),
|
),
|
||||||
ok.
|
ok.
|
||||||
|
|
||||||
|
%% test that the kafka consumer group rebalances correctly if a bridge
|
||||||
|
%% already exists when a new EMQX node joins the cluster.
|
||||||
|
t_node_joins_existing_cluster(Config) ->
|
||||||
|
ct:timetrap({seconds, 150}),
|
||||||
|
TopicMapping = ?config(topic_mapping, Config),
|
||||||
|
[MQTTTopic] = [MQTTTopic || #{mqtt_topic := MQTTTopic} <- TopicMapping],
|
||||||
|
NPartitions = ?config(num_partitions, Config),
|
||||||
|
KafkaTopic = ?config(kafka_topic, Config),
|
||||||
|
KafkaName = ?config(kafka_name, Config),
|
||||||
|
ResourceId = resource_id(Config),
|
||||||
|
BridgeId = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE_BIN, KafkaName),
|
||||||
|
Cluster = cluster(Config),
|
||||||
|
?check_trace(
|
||||||
|
begin
|
||||||
|
[{Name1, Opts1}, {Name2, Opts2} | _] = Cluster,
|
||||||
|
N1 = emqx_common_test_helpers:start_slave(Name1, Opts1),
|
||||||
|
on_exit(fun() -> ok = emqx_common_test_helpers:stop_slave(N1) end),
|
||||||
|
setup_group_subscriber_spy(N1),
|
||||||
|
{{ok, _}, {ok, _}} =
|
||||||
|
?wait_async_action(
|
||||||
|
erpc:call(N1, fun() -> {ok, _} = create_bridge(Config) end),
|
||||||
|
#{?snk_kind := kafka_consumer_subscriber_started},
|
||||||
|
15_000
|
||||||
|
),
|
||||||
|
?assertMatch({ok, _}, erpc:call(N1, emqx_bridge, lookup, [BridgeId])),
|
||||||
|
{ok, _} = wait_until_group_is_balanced(KafkaTopic, NPartitions, [N1], 30_000),
|
||||||
|
?assertEqual(
|
||||||
|
{ok, connected},
|
||||||
|
erpc:call(N1, emqx_resource_manager, health_check, [ResourceId])
|
||||||
|
),
|
||||||
|
|
||||||
|
%% Now, we start the second node and have it join the cluster.
|
||||||
|
setup_and_start_listeners(N1, Opts1),
|
||||||
|
TCPPort1 = emqx_common_test_helpers:listener_port(Opts1, tcp),
|
||||||
|
{ok, C1} = emqtt:start_link([{port, TCPPort1}, {proto_ver, v5}]),
|
||||||
|
on_exit(fun() -> catch emqtt:stop(C1) end),
|
||||||
|
{ok, _} = emqtt:connect(C1),
|
||||||
|
{ok, _, [2]} = emqtt:subscribe(C1, MQTTTopic, 2),
|
||||||
|
|
||||||
|
{ok, SRef0} = snabbkaffe:subscribe(
|
||||||
|
?match_event(#{?snk_kind := kafka_consumer_subscriber_started}),
|
||||||
|
1,
|
||||||
|
30_000
|
||||||
|
),
|
||||||
|
N2 = emqx_common_test_helpers:start_slave(Name2, Opts2),
|
||||||
|
on_exit(fun() -> ok = emqx_common_test_helpers:stop_slave(N2) end),
|
||||||
|
setup_group_subscriber_spy(N2),
|
||||||
|
Nodes = [N1, N2],
|
||||||
|
wait_for_cluster_rpc(N2),
|
||||||
|
|
||||||
|
{ok, _} = snabbkaffe:receive_events(SRef0),
|
||||||
|
?assertMatch({ok, _}, erpc:call(N2, emqx_bridge, lookup, [BridgeId])),
|
||||||
|
%% Give some time for the consumers in both nodes to
|
||||||
|
%% rebalance.
|
||||||
|
{ok, _} = wait_until_group_is_balanced(KafkaTopic, NPartitions, Nodes, 30_000),
|
||||||
|
%% Publish some messages so we can check they came from each node.
|
||||||
|
?retry(
|
||||||
|
_Sleep1 = 100,
|
||||||
|
_Attempts1 = 50,
|
||||||
|
true = erpc:call(N2, emqx_router, has_routes, [MQTTTopic])
|
||||||
|
),
|
||||||
|
{ok, SRef1} =
|
||||||
|
snabbkaffe:subscribe(
|
||||||
|
?match_event(#{
|
||||||
|
?snk_kind := kafka_consumer_handle_message,
|
||||||
|
?snk_span := {complete, _}
|
||||||
|
}),
|
||||||
|
NPartitions,
|
||||||
|
10_000
|
||||||
|
),
|
||||||
|
lists:foreach(
|
||||||
|
fun(N) ->
|
||||||
|
Key = <<"k", (integer_to_binary(N))/binary>>,
|
||||||
|
Val = <<"v", (integer_to_binary(N))/binary>>,
|
||||||
|
publish(Config, KafkaTopic, [#{key => Key, value => Val}])
|
||||||
|
end,
|
||||||
|
lists:seq(1, NPartitions)
|
||||||
|
),
|
||||||
|
{ok, _} = snabbkaffe:receive_events(SRef1),
|
||||||
|
|
||||||
|
#{nodes => Nodes}
|
||||||
|
end,
|
||||||
|
fun(Res, Trace0) ->
|
||||||
|
#{nodes := Nodes} = Res,
|
||||||
|
Trace1 = ?of_kind(kafka_assignment, Trace0),
|
||||||
|
Assignments = reconstruct_assignments_from_events(KafkaTopic, Trace1),
|
||||||
|
NodeAssignments = lists:usort([
|
||||||
|
N
|
||||||
|
|| {_Partition, {N, _MemberId}} <-
|
||||||
|
maps:to_list(Assignments)
|
||||||
|
]),
|
||||||
|
?assertEqual(lists:usort(Nodes), NodeAssignments),
|
||||||
|
?assertEqual(NPartitions, map_size(Assignments)),
|
||||||
|
Published = receive_published(#{n => NPartitions, timeout => 3_000}),
|
||||||
|
ct:pal("published:\n ~p", [Published]),
|
||||||
|
PublishingNodesFromTrace =
|
||||||
|
[
|
||||||
|
N
|
||||||
|
|| #{
|
||||||
|
?snk_kind := kafka_consumer_handle_message,
|
||||||
|
?snk_span := start,
|
||||||
|
?snk_meta := #{node := N}
|
||||||
|
} <- Trace0
|
||||||
|
],
|
||||||
|
?assertEqual(lists:usort(Nodes), lists:usort(PublishingNodesFromTrace)),
|
||||||
|
ok
|
||||||
|
end
|
||||||
|
),
|
||||||
|
ok.
|
||||||
|
|
||||||
|
%% Checks that the consumers get rebalanced after an EMQX nodes goes
|
||||||
|
%% down.
|
||||||
|
t_cluster_node_down(Config) ->
|
||||||
|
ct:timetrap({seconds, 150}),
|
||||||
|
TopicMapping = ?config(topic_mapping, Config),
|
||||||
|
[MQTTTopic] = [MQTTTopic || #{mqtt_topic := MQTTTopic} <- TopicMapping],
|
||||||
|
NPartitions = ?config(num_partitions, Config),
|
||||||
|
KafkaTopic = ?config(kafka_topic, Config),
|
||||||
|
KafkaName = ?config(kafka_name, Config),
|
||||||
|
BridgeId = emqx_bridge_resource:bridge_id(?BRIDGE_TYPE_BIN, KafkaName),
|
||||||
|
Cluster = cluster(Config),
|
||||||
|
?check_trace(
|
||||||
|
begin
|
||||||
|
{_N2, Opts2} = lists:nth(2, Cluster),
|
||||||
|
Nodes =
|
||||||
|
[N1, N2 | _] =
|
||||||
|
lists:map(
|
||||||
|
fun({Name, Opts}) -> emqx_common_test_helpers:start_slave(Name, Opts) end,
|
||||||
|
Cluster
|
||||||
|
),
|
||||||
|
on_exit(fun() ->
|
||||||
|
lists:foreach(
|
||||||
|
fun(N) ->
|
||||||
|
ct:pal("stopping ~p", [N]),
|
||||||
|
ok = emqx_common_test_helpers:stop_slave(N)
|
||||||
|
end,
|
||||||
|
Nodes
|
||||||
|
)
|
||||||
|
end),
|
||||||
|
lists:foreach(fun setup_group_subscriber_spy/1, Nodes),
|
||||||
|
{ok, SRef0} = snabbkaffe:subscribe(
|
||||||
|
?match_event(#{?snk_kind := kafka_consumer_subscriber_started}),
|
||||||
|
length(Nodes),
|
||||||
|
15_000
|
||||||
|
),
|
||||||
|
wait_for_cluster_rpc(N2),
|
||||||
|
erpc:call(N2, fun() -> {ok, _} = create_bridge(Config) end),
|
||||||
|
{ok, _} = snabbkaffe:receive_events(SRef0),
|
||||||
|
lists:foreach(
|
||||||
|
fun(N) ->
|
||||||
|
?assertMatch(
|
||||||
|
{ok, _},
|
||||||
|
erpc:call(N, emqx_bridge, lookup, [BridgeId]),
|
||||||
|
#{node => N}
|
||||||
|
)
|
||||||
|
end,
|
||||||
|
Nodes
|
||||||
|
),
|
||||||
|
{ok, _} = wait_until_group_is_balanced(KafkaTopic, NPartitions, Nodes, 30_000),
|
||||||
|
|
||||||
|
%% Now, we stop one of the nodes and watch the group
|
||||||
|
%% rebalance.
|
||||||
|
setup_and_start_listeners(N2, Opts2),
|
||||||
|
TCPPort = emqx_common_test_helpers:listener_port(Opts2, tcp),
|
||||||
|
{ok, C} = emqtt:start_link([{port, TCPPort}, {proto_ver, v5}]),
|
||||||
|
on_exit(fun() -> catch emqtt:stop(C) end),
|
||||||
|
{ok, _} = emqtt:connect(C),
|
||||||
|
{ok, _, [2]} = emqtt:subscribe(C, MQTTTopic, 2),
|
||||||
|
{TId, Pid} = start_async_publisher(Config, KafkaTopic),
|
||||||
|
|
||||||
|
ct:pal("stopping node ~p", [N1]),
|
||||||
|
ok = emqx_common_test_helpers:stop_slave(N1),
|
||||||
|
|
||||||
|
%% Give some time for the consumers in remaining node to
|
||||||
|
%% rebalance.
|
||||||
|
{ok, _} = wait_until_group_is_balanced(KafkaTopic, NPartitions, [N2], 60_000),
|
||||||
|
|
||||||
|
ok = stop_async_publisher(Pid),
|
||||||
|
|
||||||
|
#{nodes => Nodes, payloads_tid => TId}
|
||||||
|
end,
|
||||||
|
fun(Res, Trace0) ->
|
||||||
|
#{nodes := Nodes, payloads_tid := TId} = Res,
|
||||||
|
[_N1, N2 | _] = Nodes,
|
||||||
|
Trace1 = ?of_kind(kafka_assignment, Trace0),
|
||||||
|
Assignments = reconstruct_assignments_from_events(KafkaTopic, Trace1),
|
||||||
|
NodeAssignments = lists:usort([
|
||||||
|
N
|
||||||
|
|| {_Partition, {N, _MemberId}} <-
|
||||||
|
maps:to_list(Assignments)
|
||||||
|
]),
|
||||||
|
%% The surviving node has all the partitions assigned to
|
||||||
|
%% it.
|
||||||
|
?assertEqual([N2], NodeAssignments),
|
||||||
|
?assertEqual(NPartitions, map_size(Assignments)),
|
||||||
|
NumPublished = ets:info(TId, size),
|
||||||
|
%% All published messages are eventually received.
|
||||||
|
Published = receive_published(#{n => NumPublished, timeout => 3_000}),
|
||||||
|
ct:pal("published:\n ~p", [Published]),
|
||||||
|
ok
|
||||||
|
end
|
||||||
|
),
|
||||||
|
ok.
|
||||||
|
|
Loading…
Reference in New Issue