Skip to content

Commit 3417d2a

Browse files
committed
rabbitmq_peer_discovery_{common,consul}: Switch to Logger macros
... from `rabbit_log:*` calls. Many rabbitmq_peer_discovery_consul testcases are executed outside of a RabbitMQ server node. When we had Lager `parse_transform` enabled, calls to `rabbit_log` were converted to something which happened to not break when executed outside of RabbitMQ. Now that `rabbit_log` calls are kept (there is no `parse_transform`), the missing dependency to rabbit_common in the common_test node surfaces nad hilights the breakage which has always been there. Calls to `rabbit_log` are now replaced with Logger macros and this works again, even in the context of the common_test node.
1 parent 5405ca9 commit 3417d2a

File tree

6 files changed

+200
-92
lines changed

6 files changed

+200
-92
lines changed

deps/rabbitmq_peer_discovery_common/include/rabbit_peer_discovery.hrl

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,10 @@
55
%% Copyright (c) 2020-2021 VMware, Inc. or its affiliates. All rights reserved.
66
%%
77

8+
-include_lib("rabbit_common/include/logging.hrl").
9+
10+
-define(RMQLOG_DOMAIN_PEER_DIS, ?DEFINE_RMQLOG_DOMAIN(peer_discovery)).
11+
812
% rabbitmq/rabbitmq-peer-discovery-aws#25
913
% Note: this timeout must not be greater than the default
1014
% gen_server:call timeout of 5000ms. This `timeout`,

deps/rabbitmq_peer_discovery_common/src/rabbit_peer_discovery_cleanup.erl

Lines changed: 41 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010

1111
-behaviour(gen_server).
1212

13+
-include_lib("kernel/include/logger.hrl").
1314
-include("include/rabbit_peer_discovery.hrl").
1415

1516
-export([start_link/0,
@@ -84,7 +85,9 @@ init([]) ->
8485
Map = ?CONFIG_MODULE:config_map(?CONFIG_KEY),
8586
case map_size(Map) of
8687
0 ->
87-
rabbit_log:info("Peer discovery: node cleanup is disabled", []),
88+
?LOG_INFO(
89+
"Peer discovery: node cleanup is disabled",
90+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
8891
{ok, #state{}};
8992
_ ->
9093
Interval = ?CONFIG_MODULE:get(cleanup_interval, ?CONFIG_MAPPING, Map),
@@ -96,8 +99,10 @@ init([]) ->
9699
true -> "will only log warnings";
97100
false -> "will remove nodes not known to the discovery backend"
98101
end,
99-
rabbit_log:info("Peer discovery: enabling node cleanup (~s). Check interval: ~p seconds.",
100-
[WarnMsg, State#state.interval]),
102+
?LOG_INFO(
103+
"Peer discovery: enabling node cleanup (~s). Check interval: ~p seconds.",
104+
[WarnMsg, State#state.interval],
105+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
101106
{ok, State}
102107
end.
103108

@@ -118,7 +123,9 @@ init([]) ->
118123
{stop, Reason :: term(), NewState :: #state{}}).
119124

120125
handle_call(check_cluster, _From, State) ->
121-
rabbit_log:debug("Peer discovery: checking for partitioned nodes to clean up."),
126+
?LOG_DEBUG(
127+
"Peer discovery: checking for partitioned nodes to clean up.",
128+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
122129
maybe_cleanup(State),
123130
{reply, ok, State};
124131
handle_call(_Request, _From, State) ->
@@ -226,19 +233,27 @@ maybe_cleanup(State) ->
226233
-spec maybe_cleanup(State :: #state{},
227234
UnreachableNodes :: [node()]) -> ok.
228235
maybe_cleanup(_, []) ->
229-
rabbit_log:debug("Peer discovery: all known cluster nodes are up.");
236+
?LOG_DEBUG(
237+
"Peer discovery: all known cluster nodes are up.",
238+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS});
230239
maybe_cleanup(State, UnreachableNodes) ->
231-
rabbit_log:debug("Peer discovery: cleanup discovered unreachable nodes: ~p",
232-
[UnreachableNodes]),
240+
?LOG_DEBUG(
241+
"Peer discovery: cleanup discovered unreachable nodes: ~p",
242+
[UnreachableNodes],
243+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
233244
case lists:subtract(UnreachableNodes, service_discovery_nodes()) of
234245
[] ->
235-
rabbit_log:debug("Peer discovery: all unreachable nodes are still "
236-
"registered with the discovery backend ~p",
237-
[rabbit_peer_discovery:backend()]),
246+
?LOG_DEBUG(
247+
"Peer discovery: all unreachable nodes are still "
248+
"registered with the discovery backend ~p",
249+
[rabbit_peer_discovery:backend()],
250+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
238251
ok;
239252
Nodes ->
240-
rabbit_log:debug("Peer discovery: unreachable nodes are not registered "
241-
"with the discovery backend ~p", [Nodes]),
253+
?LOG_DEBUG(
254+
"Peer discovery: unreachable nodes are not registered "
255+
"with the discovery backend ~p", [Nodes],
256+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
242257
maybe_remove_nodes(Nodes, State#state.warn_only)
243258
end.
244259

@@ -254,10 +269,14 @@ maybe_cleanup(State, UnreachableNodes) ->
254269
WarnOnly :: true | false) -> ok.
255270
maybe_remove_nodes([], _) -> ok;
256271
maybe_remove_nodes([Node | Nodes], true) ->
257-
rabbit_log:warning("Peer discovery: node ~s is unreachable", [Node]),
272+
?LOG_WARNING(
273+
"Peer discovery: node ~s is unreachable", [Node],
274+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
258275
maybe_remove_nodes(Nodes, true);
259276
maybe_remove_nodes([Node | Nodes], false) ->
260-
rabbit_log:warning("Peer discovery: removing unknown node ~s from the cluster", [Node]),
277+
?LOG_WARNING(
278+
"Peer discovery: removing unknown node ~s from the cluster", [Node],
279+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
261280
rabbit_mnesia:forget_cluster_node(Node, false),
262281
maybe_remove_nodes(Nodes, false).
263282

@@ -288,11 +307,15 @@ service_discovery_nodes() ->
288307
Module = rabbit_peer_discovery:backend(),
289308
case rabbit_peer_discovery:normalize(Module:list_nodes()) of
290309
{ok, {Nodes, _Type}} ->
291-
rabbit_log:debug("Peer discovery cleanup: ~p returned ~p",
292-
[Module, Nodes]),
310+
?LOG_DEBUG(
311+
"Peer discovery cleanup: ~p returned ~p",
312+
[Module, Nodes],
313+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
293314
Nodes;
294315
{error, Reason} ->
295-
rabbit_log:debug("Peer discovery cleanup: ~p returned error ~p",
296-
[Module, Reason]),
316+
?LOG_DEBUG(
317+
"Peer discovery cleanup: ~p returned error ~p",
318+
[Module, Reason],
319+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
297320
[]
298321
end.

deps/rabbitmq_peer_discovery_common/src/rabbit_peer_discovery_config.erl

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,8 @@
99

1010
-module(rabbit_peer_discovery_config).
1111

12-
-include("rabbit_peer_discovery.hrl").
12+
-include_lib("kernel/include/logger.hrl").
13+
-include("include/rabbit_peer_discovery.hrl").
1314

1415
-export([get/3, get_integer/3, config_map/1]).
1516

@@ -24,7 +25,10 @@
2425
get(Key, Mapping, Config) ->
2526
case maps:is_key(Key, Mapping) of
2627
false ->
27-
rabbit_log:error("Key ~s is not found in peer discovery config mapping ~p!", [Key, Mapping]),
28+
?LOG_ERROR(
29+
"Key ~s is not found in peer discovery config mapping ~p!",
30+
[Key, Mapping],
31+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
2832
throw({badkey, Key});
2933
true ->
3034
get_with_entry_meta(Key, maps:get(Key, Mapping), Config)
@@ -37,7 +41,10 @@ get(Key, Mapping, Config) ->
3741
get_integer(Key, Mapping, Config) ->
3842
case maps:is_key(Key, Mapping) of
3943
false ->
40-
rabbit_log:error("Key ~s is not found in peer discovery config mapping ~p!", [Key, Mapping]),
44+
?LOG_ERROR(
45+
"Key ~s is not found in peer discovery config mapping ~p!",
46+
[Key, Mapping],
47+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
4148
throw({badkey, Key});
4249
true ->
4350
get_integer_with_entry_meta(Key, maps:get(Key, Mapping), Config)

deps/rabbitmq_peer_discovery_common/src/rabbit_peer_discovery_httpc.erl

Lines changed: 29 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99

1010
-module(rabbit_peer_discovery_httpc).
1111

12+
-include_lib("kernel/include/logger.hrl").
1213
-include("include/rabbit_peer_discovery.hrl").
1314

1415
%%
@@ -138,10 +139,10 @@ get(Scheme, Host, Port, Path, Args) ->
138139
%%
139140
get(Scheme, Host, Port, Path, Args, Headers, HttpOpts) ->
140141
URL = build_uri(Scheme, Host, Port, Path, Args),
141-
rabbit_log:debug("GET ~s", [URL]),
142+
?LOG_DEBUG("GET ~s", [URL], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
142143
HttpOpts1 = ensure_timeout(HttpOpts),
143144
Response = httpc:request(get, {URL, Headers}, HttpOpts1, []),
144-
rabbit_log:debug("Response: ~p", [Response]),
145+
?LOG_DEBUG("Response: ~p", [Response], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
145146
parse_response(Response).
146147

147148

@@ -176,10 +177,10 @@ post(Scheme, Host, Port, Path, Args, Body) ->
176177
%%
177178
post(Scheme, Host, Port, Path, Args, Headers, HttpOpts, Body) ->
178179
URL = build_uri(Scheme, Host, Port, Path, Args),
179-
rabbit_log:debug("POST ~s [~p]", [URL, Body]),
180+
?LOG_DEBUG("POST ~s [~p]", [URL, Body], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
180181
HttpOpts1 = ensure_timeout(HttpOpts),
181182
Response = httpc:request(post, {URL, Headers, ?CONTENT_JSON, Body}, HttpOpts1, []),
182-
rabbit_log:debug("Response: [~p]", [Response]),
183+
?LOG_DEBUG("Response: [~p]", [Response], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
183184
parse_response(Response).
184185

185186

@@ -205,10 +206,10 @@ post(Scheme, Host, Port, Path, Args, Headers, HttpOpts, Body) ->
205206
Body :: string() | binary() | tuple().
206207
put(Scheme, Host, Port, Path, Args, Body) ->
207208
URL = build_uri(Scheme, Host, Port, Path, Args),
208-
rabbit_log:debug("PUT ~s [~p]", [URL, Body]),
209+
?LOG_DEBUG("PUT ~s [~p]", [URL, Body], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
209210
HttpOpts = ensure_timeout(),
210211
Response = httpc:request(put, {URL, [], ?CONTENT_URLENCODED, Body}, HttpOpts, []),
211-
rabbit_log:debug("Response: [~p]", [Response]),
212+
?LOG_DEBUG("Response: [~p]", [Response], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
212213
parse_response(Response).
213214

214215

@@ -234,10 +235,10 @@ put(Scheme, Host, Port, Path, Args, Body) ->
234235
Body :: string() | binary() | tuple().
235236
put(Scheme, Host, Port, Path, Args, Headers, Body) ->
236237
URL = build_uri(Scheme, Host, Port, Path, Args),
237-
rabbit_log:debug("PUT ~s [~p] [~p]", [URL, Headers, Body]),
238+
?LOG_DEBUG("PUT ~s [~p] [~p]", [URL, Headers, Body], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
238239
HttpOpts = ensure_timeout(),
239240
Response = httpc:request(put, {URL, Headers, ?CONTENT_URLENCODED, Body}, HttpOpts, []),
240-
rabbit_log:debug("Response: [~p]", [Response]),
241+
?LOG_DEBUG("Response: [~p]", [Response], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
241242
parse_response(Response).
242243

243244

@@ -258,10 +259,10 @@ delete(Scheme, Host, Port, PathSegments, Args, Body) when is_list(PathSegments)
258259
delete(Scheme, Host, Port, Path, Args, Body);
259260
delete(Scheme, Host, Port, Path, Args, Body) ->
260261
URL = build_uri(Scheme, Host, Port, Path, Args),
261-
rabbit_log:debug("DELETE ~s [~p]", [URL, Body]),
262+
?LOG_DEBUG("DELETE ~s [~p]", [URL, Body], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
262263
HttpOpts = ensure_timeout(),
263264
Response = httpc:request(delete, {URL, [], ?CONTENT_URLENCODED, Body}, HttpOpts, []),
264-
rabbit_log:debug("Response: [~p]", [Response]),
265+
?LOG_DEBUG("Response: [~p]", [Response], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
265266
parse_response(Response).
266267

267268

@@ -275,14 +276,18 @@ maybe_configure_proxy() ->
275276
Map = ?CONFIG_MODULE:config_map(?CONFIG_KEY),
276277
case map_size(Map) of
277278
0 ->
278-
rabbit_log:debug("HTTP client proxy is not configured"),
279+
?LOG_DEBUG(
280+
"HTTP client proxy is not configured",
281+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
279282
ok;
280283
_ ->
281284
HttpProxy = ?CONFIG_MODULE:get(http_proxy, ?CONFIG_MAPPING, Map),
282285
HttpsProxy = ?CONFIG_MODULE:get(https_proxy, ?CONFIG_MAPPING, Map),
283286
ProxyExclusions = ?CONFIG_MODULE:get(proxy_exclusions, ?CONFIG_MAPPING, Map),
284-
rabbit_log:debug("Configured HTTP proxy: ~p, HTTPS proxy: ~p, exclusions: ~p",
285-
[HttpProxy, HttpsProxy, ProxyExclusions]),
287+
?LOG_DEBUG(
288+
"Configured HTTP proxy: ~p, HTTPS proxy: ~p, exclusions: ~p",
289+
[HttpProxy, HttpsProxy, ProxyExclusions],
290+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
286291
maybe_set_proxy(proxy, HttpProxy, ProxyExclusions),
287292
maybe_set_proxy(https_proxy, HttpsProxy, ProxyExclusions),
288293
ok
@@ -315,9 +320,10 @@ maybe_set_proxy(Option, ProxyUrl, ProxyExclusions) ->
315320
UriMap ->
316321
Host = maps:get(host, UriMap),
317322
Port = maps:get(port, UriMap, 80),
318-
rabbit_log:debug(
323+
?LOG_DEBUG(
319324
"Configuring HTTP client's ~s setting: ~p, exclusions: ~p",
320-
[Option, {Host, Port}, ProxyExclusions]),
325+
[Option, {Host, Port}, ProxyExclusions],
326+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
321327
httpc:set_option(Option, {{Host, Port}, ProxyExclusions})
322328
end.
323329

@@ -360,9 +366,11 @@ decode_body(?CONTENT_JSON, Body) ->
360366
{ok, Value} ->
361367
Value;
362368
{error, Err} ->
363-
rabbit_log:error("HTTP client could not decode a JSON payload "
364-
"(JSON parser returned an error): ~p.~n",
365-
[Err]),
369+
?LOG_ERROR(
370+
"HTTP client could not decode a JSON payload "
371+
"(JSON parser returned an error): ~p.~n",
372+
[Err],
373+
#{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
366374
[]
367375
end.
368376

@@ -375,14 +383,14 @@ decode_body(?CONTENT_JSON, Body) ->
375383
-spec parse_response({ok, integer(), string()} | {error, any()}) -> {ok, string()} | {error, any()}.
376384

377385
parse_response({error, Reason}) ->
378-
rabbit_log:debug("HTTP error ~p", [Reason]),
386+
?LOG_DEBUG("HTTP error ~p", [Reason], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
379387
{error, lists:flatten(io_lib:format("~p", [Reason]))};
380388

381389
parse_response({ok, 200, Body}) -> {ok, decode_body(?CONTENT_JSON, Body)};
382390
parse_response({ok, 201, Body}) -> {ok, decode_body(?CONTENT_JSON, Body)};
383391
parse_response({ok, 204, _}) -> {ok, []};
384392
parse_response({ok, Code, Body}) ->
385-
rabbit_log:debug("HTTP Response (~p) ~s", [Code, Body]),
393+
?LOG_DEBUG("HTTP Response (~p) ~s", [Code, Body], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
386394
{error, integer_to_list(Code)};
387395

388396
parse_response({ok, {{_,200,_}, Headers, Body}}) ->
@@ -391,7 +399,7 @@ parse_response({ok,{{_,201,_}, Headers, Body}}) ->
391399
{ok, decode_body(proplists:get_value("content-type", Headers, ?CONTENT_JSON), Body)};
392400
parse_response({ok,{{_,204,_}, _, _}}) -> {ok, []};
393401
parse_response({ok,{{_Vsn,Code,_Reason},_,Body}}) ->
394-
rabbit_log:debug("HTTP Response (~p) ~s", [Code, Body]),
402+
?LOG_DEBUG("HTTP Response (~p) ~s", [Code, Body], #{domain => ?RMQLOG_DOMAIN_PEER_DIS}),
395403
{error, integer_to_list(Code)}.
396404

397405
%% @private

0 commit comments

Comments
 (0)