Skip to content

Commit 871d65c

Browse files
Add new local random exchange type.
This exchange type will only bind classic queues and will only return routes for queues that are local to the publishing connection. If more than one queue is bound it will make a random choice of the locally bound queues. This exchange type is suitable as a component in systems that run highly available low-latency RPC workloads. Co-authored-by: Marcial Rosales <mrosales@pivotal.io>
1 parent 07ad375 commit 871d65c

File tree

2 files changed

+287
-0
lines changed

2 files changed

+287
-0
lines changed
Lines changed: 90 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,90 @@
1+
%% This Source Code Form is subject to the terms of the Mozilla Public
2+
%% License, v. 2.0. If a copy of the MPL was not distributed with this
3+
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4+
%%
5+
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
6+
%%
7+
8+
-module(rabbit_exchange_type_local_random).
9+
-behaviour(rabbit_exchange_type).
10+
-include_lib("rabbit_common/include/rabbit.hrl").
11+
12+
-rabbit_boot_step({?MODULE,
13+
[{description, "exchange type local random"},
14+
{mfa, {rabbit_registry, register,
15+
[exchange, <<"x-local-random">>, ?MODULE]}},
16+
{requires, rabbit_registry},
17+
{enables, kernel_ready}
18+
]}).
19+
20+
-export([add_binding/3,
21+
assert_args_equivalence/2,
22+
create/2,
23+
delete/2,
24+
policy_changed/2,
25+
description/0,
26+
recover/2,
27+
remove_bindings/3,
28+
validate_binding/2,
29+
route/3,
30+
serialise_events/0,
31+
validate/1,
32+
info/1,
33+
info/2
34+
]).
35+
36+
description() ->
37+
[{name, <<"x-local-random">>},
38+
{description, <<"Picks one random local binding (queue) to route via (to).">>}].
39+
40+
route(#exchange{name = Name}, _Msg, _Opts) ->
41+
Matches = rabbit_router:match_routing_key(Name, ['_']),
42+
case lists:filter(fun filter_local_queue/1, Matches) of
43+
[] ->
44+
[];
45+
[_] = One ->
46+
One;
47+
LocalMatches ->
48+
Rand = rand:uniform(length(LocalMatches)),
49+
lists:nth(Rand, LocalMatches)
50+
end.
51+
52+
info(_X) -> [].
53+
info(_X, _) -> [].
54+
serialise_events() -> false.
55+
validate(_X) -> ok.
56+
create(_Serial, _X) -> ok.
57+
recover(_X, _Bs) -> ok.
58+
delete(_Serial, _X) -> ok.
59+
policy_changed(_X1, _X2) -> ok.
60+
add_binding(_Serial, _X, _B) -> ok.
61+
remove_bindings(_Serial, _X, _Bs) -> ok.
62+
validate_binding(_X, #binding{destination = Dest}) ->
63+
%% check destination is a classic queue
64+
case rabbit_amqqueue:lookup(Dest) of
65+
{ok, Q} ->
66+
case amqqueue:get_type(Q) of
67+
rabbit_classic_queue ->
68+
ok;
69+
Type ->
70+
{error, {binding_invalid,
71+
"Queue type ~ts not valid for this exchange type",
72+
[Type]}}
73+
end;
74+
_ ->
75+
{error, {binding_invalid,
76+
"Destination not found",
77+
[]}}
78+
end.
79+
80+
assert_args_equivalence(X, Args) ->
81+
rabbit_exchange:assert_args_equivalence(X, Args).
82+
83+
filter_local_queue(QName) ->
84+
{ok, Q} = rabbit_amqqueue:lookup(QName),
85+
case amqqueue:get_pid(Q) of
86+
undefined ->
87+
false;
88+
Pid when is_pid(Pid) ->
89+
node(Pid) =:= node()
90+
end.
Lines changed: 197 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,197 @@
1+
%% This Source Code Form is subject to the terms of the Mozilla Public
2+
%% License, v. 2.0. If a copy of the MPL was not distributed with this
3+
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
4+
%%
5+
%% Copyright (c) 2007-2023 VMware, Inc. or its affiliates. All rights reserved.
6+
%%
7+
-module(rabbit_local_random_exchange_SUITE).
8+
9+
-compile(nowarn_export_all).
10+
-compile(export_all).
11+
12+
-include_lib("common_test/include/ct.hrl").
13+
-include_lib("amqp_client/include/amqp_client.hrl").
14+
-include_lib("eunit/include/eunit.hrl").
15+
16+
all() ->
17+
[
18+
{group, non_parallel_tests}
19+
].
20+
21+
groups() ->
22+
[
23+
{non_parallel_tests, [], [
24+
routed_to_one_local_queue_test,
25+
routed_to_one_nonlocal_queue_test
26+
]}
27+
].
28+
29+
%% -------------------------------------------------------------------
30+
%% Test suite setup/teardown
31+
%% -------------------------------------------------------------------
32+
33+
init_per_suite(Config) ->
34+
rabbit_ct_helpers:log_environment(),
35+
Config1 = rabbit_ct_helpers:set_config(Config, [
36+
{rmq_nodename_suffix, ?MODULE},
37+
{rmq_nodes_count, 3}
38+
]),
39+
rabbit_ct_helpers:run_setup_steps(Config1,
40+
rabbit_ct_broker_helpers:setup_steps() ++
41+
rabbit_ct_client_helpers:setup_steps()).
42+
43+
end_per_suite(Config) ->
44+
rabbit_ct_helpers:run_teardown_steps(
45+
Config,
46+
rabbit_ct_client_helpers:teardown_steps() ++
47+
rabbit_ct_broker_helpers:teardown_steps()).
48+
49+
init_per_group(_, Config) ->
50+
Config.
51+
52+
end_per_group(_, Config) ->
53+
Config.
54+
55+
init_per_testcase(Testcase, Config) ->
56+
TestCaseName = rabbit_ct_helpers:config_to_testcase_name(Config, Testcase),
57+
Config1 = rabbit_ct_helpers:set_config(Config, {test_resource_name,
58+
re:replace(TestCaseName, "/", "-", [global, {return, list}])}),
59+
rabbit_ct_helpers:testcase_started(Config1, Testcase).
60+
61+
end_per_testcase(Testcase, Config) ->
62+
rabbit_ct_helpers:testcase_finished(Config, Testcase).
63+
64+
%% -------------------------------------------------------------------
65+
%% Test cases
66+
%% -------------------------------------------------------------------
67+
68+
routed_to_one_local_queue_test(Config) ->
69+
E = make_exchange_name(Config, "0"),
70+
declare_exchange(Config, E),
71+
%% declare queue on the first two nodes: 0, 1
72+
QueueNames = declare_and_bind_queues(Config, 2, E),
73+
%% publish message on node 1
74+
publish(Config, E, 0),
75+
publish(Config, E, 1),
76+
%% message should arrive to queue on node 1
77+
run_on_node(Config, 0,
78+
fun(Chan) ->
79+
assert_queue_size(Config, Chan, 1, lists:nth(1, QueueNames))
80+
end),
81+
run_on_node(Config, 0,
82+
fun(Chan) ->
83+
assert_queue_size(Config, Chan, 1, lists:nth(2, QueueNames))
84+
end),
85+
delete_exchange_and_queues(Config, E, QueueNames),
86+
ok.
87+
88+
routed_to_one_nonlocal_queue_test(Config) ->
89+
90+
E = make_exchange_name(Config, "0"),
91+
declare_exchange(Config, E),
92+
%% declare queue on nodes 0, 1
93+
QueueNames = declare_and_bind_queues(Config, 2, E),
94+
%% publish message on node 2
95+
publish_expect_return(Config, E, 2),
96+
%% message should arrive to any of the other nodes. Total size among all queues is 1
97+
delete_exchange_and_queues(Config, E, QueueNames),
98+
ok.
99+
100+
delete_exchange_and_queues(Config, E, QueueNames) ->
101+
run_on_node(Config, 0,
102+
fun(Chan) ->
103+
amqp_channel:call(Chan, #'exchange.delete'{exchange = E }),
104+
[amqp_channel:call(Chan, #'queue.delete'{queue = Q })
105+
|| Q <- QueueNames]
106+
end).
107+
publish(Config, E, Node) ->
108+
run_on_node(Config, Node,
109+
fun(Chan) ->
110+
amqp_channel:call(Chan, #'confirm.select'{}),
111+
amqp_channel:call(Chan,
112+
#'basic.publish'{exchange = E, routing_key = rnd()},
113+
#amqp_msg{props = #'P_basic'{}, payload = <<>>}),
114+
amqp_channel:wait_for_confirms_or_die(Chan)
115+
end).
116+
117+
publish_expect_return(Config, E, Node) ->
118+
run_on_node(Config, Node,
119+
fun(Chan) ->
120+
amqp_channel:register_return_handler(Chan, self()),
121+
amqp_channel:call(Chan,
122+
#'basic.publish'{exchange = E,
123+
mandatory = true,
124+
routing_key = rnd()},
125+
#amqp_msg{props = #'P_basic'{},
126+
payload = <<>>}),
127+
receive
128+
{#'basic.return'{}, _} ->
129+
ok
130+
after 5000 ->
131+
flush(100),
132+
ct:fail("no return received")
133+
end
134+
end).
135+
136+
run_on_node(Config, Node, RunMethod) ->
137+
{Conn, Chan} = rabbit_ct_client_helpers:open_connection_and_channel(Config, Node),
138+
Return = RunMethod(Chan),
139+
rabbit_ct_client_helpers:close_connection_and_channel(Conn, Chan),
140+
Return.
141+
142+
declare_exchange(Config, ExchangeName) ->
143+
run_on_node(Config, 0,
144+
fun(Chan) ->
145+
#'exchange.declare_ok'{} =
146+
amqp_channel:call(Chan,
147+
#'exchange.declare'{exchange = ExchangeName,
148+
type = <<"x-local-random">>,
149+
auto_delete = false})
150+
end).
151+
152+
declare_and_bind_queues(Config, NodeCount, E) ->
153+
QueueNames = [make_queue_name(Config, Node) || Node <- lists:seq(0, NodeCount -1)],
154+
[run_on_node(Config, Node,
155+
fun(Chan) ->
156+
declare_and_bind_queue(Chan, E, make_queue_name(Config, Node))
157+
end) || Node <- lists:seq(0, NodeCount -1)],
158+
QueueNames.
159+
160+
declare_and_bind_queue(Ch, E, Q) ->
161+
#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = Q}),
162+
#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = Q,
163+
exchange = E,
164+
routing_key = <<"">>}),
165+
ok.
166+
167+
assert_total_queue_size(_Config, Chan, ExpectedSize, ExpectedQueues) ->
168+
Counts = [begin
169+
#'queue.declare_ok'{message_count = M} =
170+
amqp_channel:call(Chan, #'queue.declare'{queue = Q}),
171+
M
172+
end || Q <- ExpectedQueues],
173+
?assertEqual(ExpectedSize, lists:sum(Counts)).
174+
175+
assert_queue_size(_Config, Chan, ExpectedSize, ExpectedQueue) ->
176+
ct:log("assert_queue_size ~p ~p", [ExpectedSize, ExpectedQueue]),
177+
#'queue.declare_ok'{message_count = M} =
178+
amqp_channel:call(Chan, #'queue.declare'{queue = ExpectedQueue}),
179+
?assertEqual(ExpectedSize, M).
180+
181+
rnd() ->
182+
list_to_binary(integer_to_list(rand:uniform(1000000))).
183+
184+
make_exchange_name(Config, Suffix) ->
185+
B = rabbit_ct_helpers:get_config(Config, test_resource_name),
186+
erlang:list_to_binary("x-" ++ B ++ "-" ++ Suffix).
187+
make_queue_name(Config, Node) ->
188+
B = rabbit_ct_helpers:get_config(Config, test_resource_name),
189+
erlang:list_to_binary("q-" ++ B ++ "-" ++ integer_to_list(Node)).
190+
191+
flush(T) ->
192+
receive X ->
193+
ct:pal("flushed ~p", [X]),
194+
flush(T)
195+
after T ->
196+
ok
197+
end.

0 commit comments

Comments
 (0)