Switch from Lager to the new Erlang Logger API for logging

The configuration remains the same for the end-user. The only exception
is the log root directory: it is now set through the `log_root`
application env. variable in `rabbit`. People using the Cuttlefish-based
configuration file are not affected by this exception.

The main change is how the logging facility is configured. It now
happens in `rabbit_prelaunch_logging`. The `rabbit_lager` module is
removed.

The supported outputs remain the same: the console, text files, the
`amq.rabbitmq.log` exchange and syslog.

The message text format slightly changed: the timestamp is more precise
(now to the microsecond) and the level can be abbreviated to always be
4-character long to align all messages and improve readability. Here is
an example:

    2021-03-03 10:22:30.377392+01:00 [dbug] <0.229.0> == Prelaunch DONE ==
    2021-03-03 10:22:30.377860+01:00 [info] <0.229.0>
    2021-03-03 10:22:30.377860+01:00 [info] <0.229.0>  Starting RabbitMQ 3.8.10+115.g071f3fb on Erlang 23.2.5
    2021-03-03 10:22:30.377860+01:00 [info] <0.229.0>  Licensed under the MPL 2.0. Website: https://rabbitmq.com

The example above also shows that multiline messages are supported and
each line is prepended with the same prefix (the timestamp, the level
and the Erlang process PID).

JSON is also supported as a message format and now for any outputs.
Indeed, it is possible to use it with e.g. syslog or the exchange. Here
is an example of a JSON-formatted message sent to syslog:

    Mar  3 11:23:06 localhost rabbitmq-server[27908] <0.229.0> - {"time":"2021-03-03T11:23:06.998466+01:00","level":"notice","msg":"Logging: configured log handlers are now ACTIVE","meta":{"domain":"rabbitmq.prelaunch","file":"src/rabbit_prelaunch_logging.erl","gl":"<0.228.0>","line":311,"mfa":["rabbit_prelaunch_logging","configure_logger",1],"pid":"<0.229.0>"}}

For quick testing, the values accepted by the `$RABBITMQ_LOGS`
environment variables were extended:
  * `-` still means stdout
  * `-stderr` means stderr
  * `syslog:` means syslog on localhost
  * `exchange:` means logging to `amq.rabbitmq.log`

`$RABBITMQ_LOG` was also extended. It now accepts a `+json` modifier (in
addition to the existing `+color` one). With that modifier, messages are
formatted as JSON intead of plain text.

The `rabbitmqctl rotate_logs` command is deprecated. The reason is
Logger does not expose a function to force log rotation. However, it
will detect when a file was rotated by an external tool.

From a developer point of view, the old `rabbit_log*` API remains
supported, though it is now deprecated. It is implemented as regular
modules: there is no `parse_transform` involved anymore.

In the code, it is recommended to use the new Logger macros. For
instance, `?LOG_INFO(Format, Args)`. If possible, messages should be
augmented with some metadata. For instance (note the map after the
message):

    ?LOG_NOTICE("Logging: switching to configured handler(s); following "
                "messages may not be visible in this log output",
                #{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),

Domains in Erlang Logger parlance are the way to categorize messages.
Some predefined domains, matching previous categories, are currently
defined in `rabbit_common/include/logging.hrl` or headers in the
relevant plugins for plugin-specific categories.

At this point, very few messages have been converted from the old
`rabbit_log*` API to the new macros. It can be done gradually when
working on a particular module or logging.

The Erlang builtin console/file handler, `logger_std_h`, has been forked
because it lacks date-based file rotation. The configuration of
date-based rotation is identical to Lager. Once the dust has settled for
this feature, the goal is to submit it upstream for inclusion in Erlang.
The forked module is calld `rabbit_logger_std_h` and is based
`logger_std_h` in Erlang 23.0.
This commit is contained in:
Jean-Sébastien Pédron 2021-01-12 17:55:27 +01:00
parent a7387b9a1f
commit cdcf602749
No known key found for this signature in database
GPG Key ID: 39E99761A5FD94CC
108 changed files with 5345 additions and 2835 deletions

View File

@ -131,14 +131,14 @@ endef
APPS_DIR := $(CURDIR)/apps
LOCAL_DEPS = sasl rabbitmq_prelaunch os_mon inets compiler public_key crypto ssl syntax_tools xmerl
BUILD_DEPS = rabbitmq_cli syslog
DEPS = cuttlefish ranch lager rabbit_common ra sysmon_handler stdout_formatter recon observer_cli osiris amqp10_common
BUILD_DEPS = rabbitmq_cli
DEPS = cuttlefish ranch rabbit_common ra sysmon_handler stdout_formatter recon observer_cli osiris amqp10_common syslog
TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper
PLT_APPS += mnesia
dep_cuttlefish = hex 2.7.0
dep_syslog = git https://github.com/schlagert/syslog 3.4.5
dep_syslog = git https://github.com/schlagert/syslog 4.0.0
dep_osiris = git https://github.com/rabbitmq/osiris master
define usage_xml_to_erl

View File

@ -5,6 +5,7 @@
*.coverdata
/ebin/
/.erlang.mk/
/logs/
/rabbitmq_prelaunch.d
/xrefr

View File

@ -3,7 +3,7 @@ PROJECT_DESCRIPTION = RabbitMQ prelaunch setup
PROJECT_VERSION = 1.0.0
PROJECT_MOD = rabbit_prelaunch_app
DEPS = rabbit_common lager
DEPS = rabbit_common jsx
DEP_PLUGINS = rabbit_common/mk/rabbitmq-build.mk

View File

@ -8,8 +8,11 @@
-module(rabbit_boot_state).
-include_lib("kernel/include/logger.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([get/0,
set/1,
wait_for/2,
@ -28,7 +31,8 @@ get() ->
-spec set(boot_state()) -> ok.
set(BootState) ->
rabbit_log_prelaunch:debug("Change boot state to `~s`", [BootState]),
?LOG_DEBUG("Change boot state to `~s`", [BootState],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
?assert(is_valid(BootState)),
case BootState of
stopped -> persistent_term:erase(?PT_KEY_BOOT_STATE);

View File

@ -9,6 +9,10 @@
-behaviour(gen_server).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([start_link/0]).
-export([init/1,
@ -65,20 +69,23 @@ code_change(_OldVsn, State, _Extra) ->
notify_boot_state(ready = BootState,
#state{mechanism = legacy, sd_notify_module = SDNotify}) ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
?LOG_PREFIX "notifying of state `~s` (via native module)",
[BootState]),
[BootState],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
sd_notify_legacy(SDNotify);
notify_boot_state(ready = BootState,
#state{mechanism = socat, socket = Socket}) ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
?LOG_PREFIX "notifying of state `~s` (via socat(1))",
[BootState]),
[BootState],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
sd_notify_socat(Socket);
notify_boot_state(BootState, _) ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
?LOG_PREFIX "ignoring state `~s`",
[BootState]),
[BootState],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok.
sd_notify_message() ->
@ -99,9 +106,10 @@ sd_notify_legacy(SDNotify) ->
sd_notify_socat(Socket) ->
case sd_current_unit() of
{ok, Unit} ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
?LOG_PREFIX "systemd unit for activation check: \"~s\"~n",
[Unit]),
[Unit],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
sd_notify_socat(Socket, Unit);
_ ->
ok
@ -116,9 +124,10 @@ sd_notify_socat(Socket, Unit) ->
Result
catch
Class:Reason ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
?LOG_PREFIX "Failed to start socat(1): ~p:~p~n",
[Class, Reason]),
[Class, Reason],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
false
end.
@ -147,8 +156,10 @@ sd_open_port(Socket) ->
sd_wait_activation(Port, Unit) ->
case os:find_executable("systemctl") of
false ->
rabbit_log_prelaunch:debug(
?LOG_PREFIX "systemctl(1) unavailable, falling back to sleep~n"),
?LOG_DEBUG(
?LOG_PREFIX "systemctl(1) unavailable, falling back to sleep~n",
[],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
timer:sleep(5000),
ok;
_ ->
@ -156,8 +167,10 @@ sd_wait_activation(Port, Unit) ->
end.
sd_wait_activation(_, _, 0) ->
rabbit_log_prelaunch:debug(
?LOG_PREFIX "service still in 'activating' state, bailing out~n"),
?LOG_DEBUG(
?LOG_PREFIX "service still in 'activating' state, bailing out~n",
[],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok;
sd_wait_activation(Port, Unit, AttemptsLeft) ->
Ret = os:cmd("systemctl show --property=ActiveState -- '" ++ Unit ++ "'"),
@ -168,7 +181,8 @@ sd_wait_activation(Port, Unit, AttemptsLeft) ->
"ActiveState=" ++ _ ->
ok;
_ = Err ->
rabbit_log_prelaunch:debug(
?LOG_PREFIX "unexpected status from systemd: ~p~n", [Err]),
?LOG_DEBUG(
?LOG_PREFIX "unexpected status from systemd: ~p~n", [Err],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok
end.

View File

@ -0,0 +1,127 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_logger_json_fmt).
-export([format/2]).
format(
#{msg := Msg,
level := Level,
meta := #{time := Timestamp} = Meta},
Config) ->
FormattedTimestamp = unicode:characters_to_binary(
format_time(Timestamp, Config)),
FormattedMsg = unicode:characters_to_binary(
format_msg(Msg, Meta, Config)),
FormattedMeta = format_meta(Meta, Config),
Json = jsx:encode(
[{time, FormattedTimestamp},
{level, Level},
{msg, FormattedMsg},
{meta, FormattedMeta}]),
[Json, $\n].
format_time(Timestamp, _) ->
Options = [{unit, microsecond}],
calendar:system_time_to_rfc3339(Timestamp, Options).
format_msg({string, Chardata}, Meta, Config) ->
format_msg({"~ts", [Chardata]}, Meta, Config);
format_msg({report, Report}, Meta, Config) ->
FormattedReport = format_report(Report, Meta, Config),
format_msg(FormattedReport, Meta, Config);
format_msg({Format, Args}, _, _) ->
io_lib:format(Format, Args).
format_report(
#{label := {application_controller, _}} = Report, Meta, Config) ->
format_application_progress(Report, Meta, Config);
format_report(
#{label := {supervisor, progress}} = Report, Meta, Config) ->
format_supervisor_progress(Report, Meta, Config);
format_report(
Report, #{report_cb := Cb} = Meta, Config) ->
try
case erlang:fun_info(Cb, arity) of
{arity, 1} -> Cb(Report);
{arity, 2} -> {"~ts", [Cb(Report, #{})]}
end
catch
_:_:_ ->
format_report(Report, maps:remove(report_cb, Meta), Config)
end;
format_report(Report, _, _) ->
logger:format_report(Report).
format_application_progress(#{label := {_, progress},
report := InternalReport}, _, _) ->
Application = proplists:get_value(application, InternalReport),
StartedAt = proplists:get_value(started_at, InternalReport),
{"Application ~w started on ~0p",
[Application, StartedAt]};
format_application_progress(#{label := {_, exit},
report := InternalReport}, _, _) ->
Application = proplists:get_value(application, InternalReport),
Exited = proplists:get_value(exited, InternalReport),
{"Application ~w exited with reason: ~0p",
[Application, Exited]}.
format_supervisor_progress(#{report := InternalReport}, _, _) ->
Supervisor = proplists:get_value(supervisor, InternalReport),
Started = proplists:get_value(started, InternalReport),
Id = proplists:get_value(id, Started),
Pid = proplists:get_value(pid, Started),
Mfa = proplists:get_value(mfargs, Started),
{"Supervisor ~w: child ~w started (~w): ~0p",
[Supervisor, Id, Pid, Mfa]}.
format_meta(Meta, _) ->
maps:fold(
fun
(time, _, Acc) ->
Acc;
(domain = Key, Components, Acc) ->
Term = unicode:characters_to_binary(
string:join(
[atom_to_list(Cmp) || Cmp <- Components],
".")),
Acc#{Key => Term};
(Key, Value, Acc) ->
case convert_to_types_accepted_by_jsx(Value) of
false -> Acc;
Term -> Acc#{Key => Term}
end
end, #{}, Meta).
convert_to_types_accepted_by_jsx(Term) when is_map(Term) ->
maps:map(
fun(_, Value) -> convert_to_types_accepted_by_jsx(Value) end,
Term);
convert_to_types_accepted_by_jsx(Term) when is_list(Term) ->
case io_lib:deep_char_list(Term) of
true ->
unicode:characters_to_binary(Term);
false ->
[convert_to_types_accepted_by_jsx(E) || E <- Term]
end;
convert_to_types_accepted_by_jsx(Term) when is_tuple(Term) ->
convert_to_types_accepted_by_jsx(erlang:tuple_to_list(Term));
convert_to_types_accepted_by_jsx(Term) when is_function(Term) ->
String = erlang:fun_to_list(Term),
unicode:characters_to_binary(String);
convert_to_types_accepted_by_jsx(Term) when is_pid(Term) ->
String = erlang:pid_to_list(Term),
unicode:characters_to_binary(String);
convert_to_types_accepted_by_jsx(Term) when is_port(Term) ->
String = erlang:port_to_list(Term),
unicode:characters_to_binary(String);
convert_to_types_accepted_by_jsx(Term) when is_reference(Term) ->
String = erlang:ref_to_list(Term),
unicode:characters_to_binary(String);
convert_to_types_accepted_by_jsx(Term) ->
Term.

View File

@ -0,0 +1,841 @@
%%
%% %CopyrightBegin%
%%
%% Copyright Ericsson AB 2017-2020. 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.
%%
%% %CopyrightEnd%
%%
-module(rabbit_logger_std_h).
%-include("logger.hrl").
%-include("logger_internal.hrl").
%-include("logger_h_common.hrl").
-ifdef(TEST).
-define(io_put_chars(DEVICE, DATA), begin
%% We log to Common Test log as well.
%% This is the file we use to check
%% the message made it to
%% stdout/stderr.
ct:log("~ts", [DATA]),
io:put_chars(DEVICE, DATA)
end).
-else.
-define(io_put_chars(DEVICE, DATA), io:put_chars(DEVICE, DATA)).
-endif.
-define(file_write(DEVICE, DATA), file:write(DEVICE, DATA)).
-define(file_datasync(DEVICE), file:datasync(DEVICE)).
-include_lib("kernel/include/file.hrl").
%% API
-export([filesync/1]).
-export([is_date_based_rotation_needed/3]).
%% logger_h_common callbacks
-export([init/2, check_config/4, config_changed/3, reset_state/2,
filesync/3, write/4, handle_info/3, terminate/3]).
%% logger callbacks
-export([log/2, adding_handler/1, removing_handler/1, changing_config/3,
filter_config/1]).
-define(DEFAULT_CALL_TIMEOUT, 5000).
%%%===================================================================
%%% API
%%%===================================================================
%%%-----------------------------------------------------------------
%%%
-spec filesync(Name) -> ok | {error,Reason} when
Name :: atom(),
Reason :: handler_busy | {badarg,term()}.
filesync(Name) ->
logger_h_common:filesync(?MODULE,Name).
%%%===================================================================
%%% logger callbacks - just forward to logger_h_common
%%%===================================================================
%%%-----------------------------------------------------------------
%%% Handler being added
-spec adding_handler(Config) -> {ok,Config} | {error,Reason} when
Config :: logger:handler_config(),
Reason :: term().
adding_handler(Config) ->
logger_h_common:adding_handler(Config).
%%%-----------------------------------------------------------------
%%% Updating handler config
-spec changing_config(SetOrUpdate, OldConfig, NewConfig) ->
{ok,Config} | {error,Reason} when
SetOrUpdate :: set | update,
OldConfig :: logger:handler_config(),
NewConfig :: logger:handler_config(),
Config :: logger:handler_config(),
Reason :: term().
changing_config(SetOrUpdate, OldConfig, NewConfig) ->
logger_h_common:changing_config(SetOrUpdate, OldConfig, NewConfig).
%%%-----------------------------------------------------------------
%%% Handler being removed
-spec removing_handler(Config) -> ok when
Config :: logger:handler_config().
removing_handler(Config) ->
logger_h_common:removing_handler(Config).
%%%-----------------------------------------------------------------
%%% Log a string or report
-spec log(LogEvent, Config) -> ok when
LogEvent :: logger:log_event(),
Config :: logger:handler_config().
log(LogEvent, Config) ->
logger_h_common:log(LogEvent, Config).
%%%-----------------------------------------------------------------
%%% Remove internal fields from configuration
-spec filter_config(Config) -> Config when
Config :: logger:handler_config().
filter_config(Config) ->
logger_h_common:filter_config(Config).
%%%===================================================================
%%% logger_h_common callbacks
%%%===================================================================
init(Name, Config) ->
MyConfig = maps:with([type,file,modes,file_check,max_no_bytes,
rotate_on_date,max_no_files,compress_on_rotate],
Config),
case file_ctrl_start(Name, MyConfig) of
{ok,FileCtrlPid} ->
{ok,MyConfig#{file_ctrl_pid=>FileCtrlPid}};
Error ->
Error
end.
check_config(Name,set,undefined,NewHConfig) ->
check_h_config(merge_default_config(Name,normalize_config(NewHConfig)));
check_config(Name,SetOrUpdate,OldHConfig,NewHConfig0) ->
WriteOnce = maps:with([type,file,modes],OldHConfig),
Default =
case SetOrUpdate of
set ->
%% Do not reset write-once fields to defaults
merge_default_config(Name,WriteOnce);
update ->
OldHConfig
end,
NewHConfig = maps:merge(Default, normalize_config(NewHConfig0)),
%% Fail if write-once fields are changed
case maps:with([type,file,modes],NewHConfig) of
WriteOnce ->
check_h_config(NewHConfig);
Other ->
{error,{illegal_config_change,?MODULE,WriteOnce,Other}}
end.
check_h_config(HConfig) ->
case check_h_config(maps:get(type,HConfig),maps:to_list(HConfig)) of
ok ->
{ok,fix_file_opts(HConfig)};
{error,{Key,Value}} ->
{error,{invalid_config,?MODULE,#{Key=>Value}}}
end.
check_h_config(Type,[{type,Type} | Config]) when Type =:= standard_io;
Type =:= standard_error;
Type =:= file ->
check_h_config(Type,Config);
check_h_config({device,Device},[{type,{device,Device}} | Config]) ->
check_h_config({device,Device},Config);
check_h_config(file,[{file,File} | Config]) when is_list(File) ->
check_h_config(file,Config);
check_h_config(file,[{modes,Modes} | Config]) when is_list(Modes) ->
check_h_config(file,Config);
check_h_config(file,[{max_no_bytes,Size} | Config])
when (is_integer(Size) andalso Size>0) orelse Size=:=infinity ->
check_h_config(file,Config);
check_h_config(file,[{rotate_on_date,DateSpec}=Param | Config])
when is_list(DateSpec) orelse DateSpec=:=false ->
case parse_date_spec(DateSpec) of
error -> {error,Param};
_ -> check_h_config(file,Config)
end;
check_h_config(file,[{max_no_files,Num} | Config]) when is_integer(Num), Num>=0 ->
check_h_config(file,Config);
check_h_config(file,[{compress_on_rotate,Bool} | Config]) when is_boolean(Bool) ->
check_h_config(file,Config);
check_h_config(file,[{file_check,FileCheck} | Config])
when is_integer(FileCheck), FileCheck>=0 ->
check_h_config(file,Config);
check_h_config(_Type,[Other | _]) ->
{error,Other};
check_h_config(_Type,[]) ->
ok.
normalize_config(#{type:={file,File}}=HConfig) ->
normalize_config(HConfig#{type=>file,file=>File});
normalize_config(#{type:={file,File,Modes}}=HConfig) ->
normalize_config(HConfig#{type=>file,file=>File,modes=>Modes});
normalize_config(#{file:=File}=HConfig) ->
HConfig#{file=>filename:absname(File)};
normalize_config(HConfig) ->
HConfig.
merge_default_config(Name,#{type:=Type}=HConfig) ->
merge_default_config(Name,Type,HConfig);
merge_default_config(Name,#{file:=_}=HConfig) ->
merge_default_config(Name,file,HConfig);
merge_default_config(Name,HConfig) ->
merge_default_config(Name,standard_io,HConfig).
merge_default_config(Name,Type,HConfig) ->
maps:merge(get_default_config(Name,Type),HConfig).
get_default_config(Name,file) ->
#{type => file,
file => filename:absname(atom_to_list(Name)),
modes => [raw,append],
file_check => 0,
max_no_bytes => infinity,
rotate_on_date => false,
max_no_files => 0,
compress_on_rotate => false};
get_default_config(_Name,Type) ->
#{type => Type}.
fix_file_opts(#{modes:=Modes}=HConfig) ->
HConfig#{modes=>fix_modes(Modes)};
fix_file_opts(HConfig) ->
HConfig#{filesync_repeat_interval=>no_repeat}.
fix_modes(Modes) ->
%% Ensure write|append|exclusive
Modes1 =
case [M || M <- Modes,
lists:member(M,[write,append,exclusive])] of
[] -> [append|Modes];
_ -> Modes
end,
%% Ensure raw
Modes2 =
case lists:member(raw,Modes) of
false -> [raw|Modes1];
true -> Modes1
end,
%% Ensure delayed_write
case lists:partition(fun(delayed_write) -> true;
({delayed_write,_,_}) -> true;
(_) -> false
end, Modes2) of
{[],_} ->
[delayed_write|Modes2];
_ ->
Modes2
end.
config_changed(_Name,
#{file_check:=FileCheck,
max_no_bytes:=Size,
rotate_on_date:=DateSpec,
max_no_files:=Count,
compress_on_rotate:=Compress},
#{file_check:=FileCheck,
max_no_bytes:=Size,
rotate_on_date:=DateSpec,
max_no_files:=Count,
compress_on_rotate:=Compress}=State) ->
State;
config_changed(_Name,
#{file_check:=FileCheck,
max_no_bytes:=Size,
rotate_on_date:=DateSpec,
max_no_files:=Count,
compress_on_rotate:=Compress},
#{file_ctrl_pid := FileCtrlPid} = State) ->
FileCtrlPid ! {update_config,#{file_check=>FileCheck,
max_no_bytes=>Size,
rotate_on_date=>DateSpec,
max_no_files=>Count,
compress_on_rotate=>Compress}},
State#{file_check:=FileCheck,
max_no_bytes:=Size,
rotate_on_date:=DateSpec,
max_no_files:=Count,
compress_on_rotate:=Compress};
config_changed(_Name,_NewHConfig,State) ->
State.
filesync(_Name, SyncAsync, #{file_ctrl_pid := FileCtrlPid} = State) ->
Result = file_ctrl_filesync(SyncAsync, FileCtrlPid),
{Result,State}.
write(_Name, SyncAsync, Bin, #{file_ctrl_pid:=FileCtrlPid} = State) ->
Result = file_write(SyncAsync, FileCtrlPid, Bin),
{Result,State}.
reset_state(_Name, State) ->
State.
handle_info(_Name, {'EXIT',Pid,Why}, #{file_ctrl_pid := Pid}=State) ->
%% file_ctrl_pid died, file error, terminate handler
exit({error,{write_failed,maps:with([type,file,modes],State),Why}});
handle_info(_, _, State) ->
State.
terminate(_Name, _Reason, #{file_ctrl_pid:=FWPid}) ->
case is_process_alive(FWPid) of
true ->
unlink(FWPid),
_ = file_ctrl_stop(FWPid),
MRef = erlang:monitor(process, FWPid),
receive
{'DOWN',MRef,_,_,_} ->
ok
after
?DEFAULT_CALL_TIMEOUT ->
exit(FWPid, kill),
ok
end;
false ->
ok
end.
%%%===================================================================
%%% Internal functions
%%%===================================================================
%%%-----------------------------------------------------------------
%%%
open_log_file(HandlerName,#{type:=file,
file:=FileName,
modes:=Modes,
file_check:=FileCheck}) ->
try
case filelib:ensure_dir(FileName) of
ok ->
case file:open(FileName, Modes) of
{ok, Fd} ->
{ok,#file_info{inode=INode}} =
file:read_file_info(FileName,[raw]),
UpdateModes = [append | Modes--[write,append,exclusive]],
{ok,#{handler_name=>HandlerName,
file_name=>FileName,
modes=>UpdateModes,
file_check=>FileCheck,
fd=>Fd,
inode=>INode,
last_check=>timestamp(),
synced=>false,
write_res=>ok,
sync_res=>ok}};
Error ->
Error
end;
Error ->
Error
end
catch
_:Reason -> {error,Reason}
end.
close_log_file(#{fd:=Fd}) ->
_ = file:datasync(Fd), %% file:datasync may return error as it will flush the delayed_write buffer
_ = file:close(Fd),
ok;
close_log_file(_) ->
ok.
%% A special close that closes the FD properly when the delayed write close failed
delayed_write_close(#{fd:=Fd}) ->
case file:close(Fd) of
%% We got an error while closing, could be a delayed write failing
%% So we close again in order to make sure the file is closed.
{error, _} ->
file:close(Fd);
Res ->
Res
end.
%%%-----------------------------------------------------------------
%%% File control process
file_ctrl_start(HandlerName, HConfig) ->
Starter = self(),
FileCtrlPid =
spawn_link(fun() ->
file_ctrl_init(HandlerName, HConfig, Starter)
end),
receive
{FileCtrlPid,ok} ->
{ok,FileCtrlPid};
{FileCtrlPid,Error} ->
Error
after
?DEFAULT_CALL_TIMEOUT ->
{error,file_ctrl_process_not_started}
end.
file_ctrl_stop(Pid) ->
Pid ! stop.
file_write(async, Pid, Bin) ->
Pid ! {log,Bin},
ok;
file_write(sync, Pid, Bin) ->
file_ctrl_call(Pid, {log,Bin}).
file_ctrl_filesync(async, Pid) ->
Pid ! filesync,
ok;
file_ctrl_filesync(sync, Pid) ->
file_ctrl_call(Pid, filesync).
file_ctrl_call(Pid, Msg) ->
MRef = monitor(process, Pid),
Pid ! {Msg,{self(),MRef}},
receive
{MRef,Result} ->
demonitor(MRef, [flush]),
Result;
{'DOWN',MRef,_Type,_Object,Reason} ->
{error,Reason}
after
?DEFAULT_CALL_TIMEOUT ->
%% If this timeout triggers we will get a stray
%% reply message in our mailbox eventually.
%% That does not really matter though as it will
%% end up in this module's handle_info and be ignored
demonitor(MRef, [flush]),
{error,{no_response,Pid}}
end.
file_ctrl_init(HandlerName,
#{type:=file,
max_no_bytes:=Size,
rotate_on_date:=DateSpec,
max_no_files:=Count,
compress_on_rotate:=Compress,
file:=FileName} = HConfig,
Starter) ->
process_flag(message_queue_data, off_heap),
case open_log_file(HandlerName,HConfig) of
{ok,State} ->
Starter ! {self(),ok},
%% Do the initial rotate (if any) after we ack the starting
%% process as otherwise startup of the system will be
%% delayed/crash
case parse_date_spec(DateSpec) of
error ->
Starter ! {self(),{error,{invalid_date_spec,DateSpec}}};
ParsedDS ->
RotState = update_rotation({Size,ParsedDS,Count,Compress},State),
file_ctrl_loop(RotState)
end;
{error,Reason} ->
Starter ! {self(),{error,{open_failed,FileName,Reason}}}
end;
file_ctrl_init(HandlerName, #{type:={device,Dev}}, Starter) ->
Starter ! {self(),ok},
file_ctrl_loop(#{handler_name=>HandlerName,dev=>Dev});
file_ctrl_init(HandlerName, #{type:=StdDev}, Starter) ->
Starter ! {self(),ok},
file_ctrl_loop(#{handler_name=>HandlerName,dev=>StdDev}).
file_ctrl_loop(State) ->
receive
%% asynchronous event
{log,Bin} ->
State1 = write_to_dev(Bin,State),
file_ctrl_loop(State1);
%% synchronous event
{{log,Bin},{From,MRef}} ->
State1 = ensure_file(State),
State2 = write_to_dev(Bin,State1),
From ! {MRef,ok},
file_ctrl_loop(State2);
filesync ->
State1 = sync_dev(State),
file_ctrl_loop(State1);
{filesync,{From,MRef}} ->
State1 = ensure_file(State),
State2 = sync_dev(State1),
From ! {MRef,ok},
file_ctrl_loop(State2);
{update_config,#{file_check:=FileCheck,
max_no_bytes:=Size,
rotate_on_date:=DateSpec,
max_no_files:=Count,
compress_on_rotate:=Compress}} ->
case parse_date_spec(DateSpec) of
error ->
%% FIXME: Report parsing error?
file_ctrl_loop(State#{file_check=>FileCheck});
ParsedDS ->
State1 = update_rotation({Size,ParsedDS,Count,Compress},State),
file_ctrl_loop(State1#{file_check=>FileCheck})
end;
stop ->
close_log_file(State),
stopped
end.
maybe_ensure_file(#{file_check:=0}=State) ->
ensure_file(State);
maybe_ensure_file(#{last_check:=T0,file_check:=CheckInt}=State)
when is_integer(CheckInt) ->
T = timestamp(),
if T-T0 > CheckInt -> ensure_file(State);
true -> State
end;
maybe_ensure_file(State) ->
State.
%% In order to play well with tools like logrotate, we need to be able
%% to re-create the file if it has disappeared (e.g. if rotated by
%% logrotate)
ensure_file(#{inode:=INode0,file_name:=FileName,modes:=Modes}=State) ->
case file:read_file_info(FileName,[raw]) of
{ok,#file_info{inode=INode0}} ->
State#{last_check=>timestamp()};
_ ->
close_log_file(State),
case file:open(FileName,Modes) of
{ok,Fd} ->
{ok,#file_info{inode=INode}} =
file:read_file_info(FileName,[raw]),
State#{fd=>Fd,inode=>INode,
last_check=>timestamp(),
synced=>true,sync_res=>ok};
Error ->
exit({could_not_reopen_file,Error})
end
end;
ensure_file(State) ->
State.
write_to_dev(Bin,#{dev:=DevName}=State) ->
?io_put_chars(DevName, Bin),
State;
write_to_dev(Bin, State) ->
State1 = #{fd:=Fd} = maybe_ensure_file(State),
Result = ?file_write(Fd, Bin),
State2 = maybe_rotate_file(Bin,State1),
maybe_notify_error(write,Result,State2),
State2#{synced=>false,write_res=>Result}.
sync_dev(#{synced:=false}=State) ->
State1 = #{fd:=Fd} = maybe_ensure_file(State),
Result = ?file_datasync(Fd),
maybe_notify_error(filesync,Result,State1),
State1#{synced=>true,sync_res=>Result};
sync_dev(State) ->
State.
update_rotation({infinity,false,_,_},State) ->
maybe_remove_archives(0,State),
maps:remove(rotation,State);
update_rotation({Size,DateSpec,Count,Compress},#{file_name:=FileName}=State) ->
maybe_remove_archives(Count,State),
{ok,#file_info{size=CurrSize}} = file:read_file_info(FileName,[raw]),
State1 = State#{rotation=>#{size=>Size,
on_date=>DateSpec,
count=>Count,
compress=>Compress,
curr_size=>CurrSize}},
maybe_update_compress(0,State1),
maybe_rotate_file(0,State1).
parse_date_spec(false) ->
false;
parse_date_spec("") ->
false;
parse_date_spec([$$,$D | DateSpec]) ->
io:format(standard_error, "parse_date_spec: ~p (hour)~n", [DateSpec]),
parse_hour(DateSpec, #{every=>day,
hour=>0});
parse_date_spec([$$,$W | DateSpec]) ->
io:format(standard_error, "parse_date_spec: ~p (week)~n", [DateSpec]),
parse_day_of_week(DateSpec, #{every=>week,
hour=>0});
parse_date_spec([$$,$M | DateSpec]) ->
io:format(standard_error, "parse_date_spec: ~p (month)~n", [DateSpec]),
parse_day_of_month(DateSpec, #{every=>month,
hour=>0});
parse_date_spec(DateSpec) ->
io:format(standard_error, "parse_date_spec: ~p (error)~n", [DateSpec]),
error.
parse_hour(Rest,Result) ->
case date_string_to_int(Rest,0,23) of
{Hour,""} -> Result#{hour=>Hour};
error -> error
end.
parse_day_of_week(Rest,Result) ->
case date_string_to_int(Rest,0,6) of
{DayOfWeek,Rest} -> parse_hour(Rest,Result#{day_of_week=>DayOfWeek});
error -> error
end.
parse_day_of_month([Last | Rest],Result)
when Last=:=$l orelse Last=:=$L ->
parse_hour(Rest,Result#{day_of_month=>last});
parse_day_of_month(Rest,Result) ->
case date_string_to_int(Rest,1,31) of
{DayOfMonth,Rest} -> parse_hour(Rest,Result#{day_of_month=>DayOfMonth});
error -> error
end.
date_string_to_int(String,Min,Max) ->
case string:to_integer(String) of
{Int,Rest} when is_integer(Int) andalso Int>=Min andalso Int=<Max ->
{Int,Rest};
_ ->
error
end.
maybe_remove_archives(Count,#{file_name:=FileName}=State) ->
Archive = rot_file_name(FileName,Count,false),
CompressedArchive = rot_file_name(FileName,Count,true),
case {file:read_file_info(Archive,[raw]),
file:read_file_info(CompressedArchive,[raw])} of
{{error,enoent},{error,enoent}} ->
ok;
_ ->
_ = file:delete(Archive),
_ = file:delete(CompressedArchive),
maybe_remove_archives(Count+1,State)
end.
maybe_update_compress(Count,#{rotation:=#{count:=Count}}) ->
ok;
maybe_update_compress(N,#{file_name:=FileName,
rotation:=#{compress:=Compress}}=State) ->
Archive = rot_file_name(FileName,N,not Compress),
case file:read_file_info(Archive,[raw]) of
{ok,_} when Compress ->
compress_file(Archive);
{ok,_} ->
decompress_file(Archive);
_ ->
ok
end,
maybe_update_compress(N+1,State).
maybe_rotate_file(Bin,#{rotation:=_}=State) when is_binary(Bin) ->
maybe_rotate_file(byte_size(Bin),State);
maybe_rotate_file(AddSize,#{rotation:=#{size:=RotSize,
curr_size:=CurrSize}=Rotation}=State) ->
{DateBasedRotNeeded, Rotation1} = is_date_based_rotation_needed(Rotation),
NewSize = CurrSize + AddSize,
if NewSize>RotSize ->
rotate_file(State#{rotation=>Rotation1#{curr_size=>NewSize}});
DateBasedRotNeeded ->
rotate_file(State#{rotation=>Rotation1#{curr_size=>NewSize}});
true ->
State#{rotation=>Rotation1#{curr_size=>NewSize}}
end;
maybe_rotate_file(_Bin,State) ->
State.
is_date_based_rotation_needed(#{last_rotation_ts:=PrevTimestamp,
on_date:=DateSpec}=Rotation) ->
CurrTimestamp = rotation_timestamp(),
case is_date_based_rotation_needed(DateSpec,PrevTimestamp,CurrTimestamp) of
true -> {true,Rotation#{last_rotation_ts=>CurrTimestamp}};
false -> {false,Rotation}
end;
is_date_based_rotation_needed(Rotation) ->
{false,Rotation#{last_rotation_ts=>rotation_timestamp()}}.
is_date_based_rotation_needed(#{every:=day,hour:=Hour},
{Date1,Time1},{Date2,Time2})
when (Date1<Date2 orelse (Date1=:=Date2 andalso Time1<{Hour,0,0})) andalso
Time2>={Hour,0,0} ->
true;
is_date_based_rotation_needed(#{every:=day,hour:=Hour},
{Date1,_}=DateTime1,{Date2,Time2}=DateTime2)
when Date1<Date2 andalso
Time2<{Hour,0,0} ->
GregDays2 = calendar:date_to_gregorian_days(Date2),
TargetDate = calendar:gregorian_days_to_date(GregDays2 - 1),
TargetDateTime = {TargetDate,{Hour,0,0}},
DateTime1<TargetDateTime andalso DateTime2>=TargetDateTime;
is_date_based_rotation_needed(#{every:=week,day_of_week:=TargetDoW,hour:=Hour},
DateTime1,{Date2,_}=DateTime2) ->
DoW2 = calendar:day_of_the_week(Date2) rem 7,
DaysSinceTargetDoW = ((DoW2 - TargetDoW) + 7) rem 7,
GregDays2 = calendar:date_to_gregorian_days(Date2),
TargetGregDays = GregDays2 - DaysSinceTargetDoW,
TargetDate = calendar:gregorian_days_to_date(TargetGregDays),
TargetDateTime = {TargetDate,{Hour,0,0}},
DateTime1<TargetDateTime andalso DateTime2>=TargetDateTime;
is_date_based_rotation_needed(#{every:=month,day_of_month:=last,hour:=Hour},
DateTime1,{{Year2,Month2,_}=Date2,_}=DateTime2) ->
DoMA = calendar:last_day_of_the_month(Year2, Month2),
DateA = {Year2,Month2,DoMA},
TargetDate = if
DateA>Date2 ->
case Month2 - 1 of
0 ->
{Year2-1,12,31};
MonthB ->
{Year2,MonthB,
calendar:last_day_of_the_month(Year2,MonthB)}
end;
true ->
DateA
end,
TargetDateTime = {TargetDate,{Hour,0,0}},
io:format(standard_error, "TargetDateTime=~p~n", [TargetDateTime]),
DateTime1<TargetDateTime andalso DateTime2>=TargetDateTime;
is_date_based_rotation_needed(#{every:=month,day_of_month:=DoM,hour:=Hour},
DateTime1,{{Year2,Month2,_}=Date2,_}=DateTime2) ->
DateA = {Year2,Month2,adapt_day_of_month(Year2,Month2,DoM)},
TargetDate = if
DateA>Date2 ->
case Month2 - 1 of
0 ->
{Year2-1,12,31};
MonthB ->
{Year2,MonthB,
adapt_day_of_month(Year2,MonthB,DoM)}
end;
true ->
DateA
end,
TargetDateTime = {TargetDate,{Hour,0,0}},
io:format(standard_error, "TargetDateTime=~p~n", [TargetDateTime]),
DateTime1<TargetDateTime andalso DateTime2>=TargetDateTime;
is_date_based_rotation_needed(_,_,_) ->
false.
adapt_day_of_month(Year,Month,Day) ->
LastDay = calendar:last_day_of_the_month(Year,Month),
erlang:min(Day,LastDay).
rotate_file(#{file_name:=FileName,modes:=Modes,rotation:=Rotation}=State) ->
State1 = sync_dev(State),
_ = delayed_write_close(State),
rotate_files(FileName,maps:get(count,Rotation),maps:get(compress,Rotation)),
case file:open(FileName,Modes) of
{ok,Fd} ->
{ok,#file_info{inode=INode}} = file:read_file_info(FileName,[raw]),
CurrTimestamp = rotation_timestamp(),
State1#{fd=>Fd,inode=>INode,
rotation=>Rotation#{curr_size=>0,
last_rotation_ts=>CurrTimestamp}};
Error ->
exit({could_not_reopen_file,Error})
end.
rotation_timestamp() ->
calendar:now_to_local_time(erlang:timestamp()).
rotate_files(FileName,0,_Compress) ->
_ = file:delete(FileName),
ok;
rotate_files(FileName,1,Compress) ->
FileName0 = FileName++".0",
_ = file:rename(FileName,FileName0),
if Compress -> compress_file(FileName0);
true -> ok
end,
ok;
rotate_files(FileName,Count,Compress) ->
_ = file:rename(rot_file_name(FileName,Count-2,Compress),
rot_file_name(FileName,Count-1,Compress)),
rotate_files(FileName,Count-1,Compress).
rot_file_name(FileName,Count,false) ->
FileName ++ "." ++ integer_to_list(Count);
rot_file_name(FileName,Count,true) ->
rot_file_name(FileName,Count,false) ++ ".gz".
compress_file(FileName) ->
{ok,In} = file:open(FileName,[read,binary]),
{ok,Out} = file:open(FileName++".gz",[write]),
Z = zlib:open(),
zlib:deflateInit(Z, default, deflated, 31, 8, default),
compress_data(Z,In,Out),
zlib:deflateEnd(Z),
zlib:close(Z),
_ = file:close(In),
_ = file:close(Out),
_ = file:delete(FileName),
ok.
compress_data(Z,In,Out) ->
case file:read(In,100000) of
{ok,Data} ->
Compressed = zlib:deflate(Z, Data),
_ = file:write(Out,Compressed),
compress_data(Z,In,Out);
eof ->
Compressed = zlib:deflate(Z, <<>>, finish),
_ = file:write(Out,Compressed),
ok
end.
decompress_file(FileName) ->
{ok,In} = file:open(FileName,[read,binary]),
{ok,Out} = file:open(filename:rootname(FileName,".gz"),[write]),
Z = zlib:open(),
zlib:inflateInit(Z, 31),
decompress_data(Z,In,Out),
zlib:inflateEnd(Z),
zlib:close(Z),
_ = file:close(In),
_ = file:close(Out),
_ = file:delete(FileName),
ok.
decompress_data(Z,In,Out) ->
case file:read(In,1000) of
{ok,Data} ->
Decompressed = zlib:inflate(Z, Data),
_ = file:write(Out,Decompressed),
decompress_data(Z,In,Out);
eof ->
ok
end.
maybe_notify_error(_Op, ok, _State) ->
ok;
maybe_notify_error(Op, Result, #{write_res:=WR,sync_res:=SR})
when (Op==write andalso Result==WR) orelse
(Op==filesync andalso Result==SR) ->
%% don't report same error twice
ok;
maybe_notify_error(Op, Error, #{handler_name:=HandlerName,file_name:=FileName}) ->
logger_h_common:error_notify({HandlerName,Op,FileName,Error}),
ok.
timestamp() ->
erlang:monotonic_time(millisecond).

View File

@ -0,0 +1,167 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_logger_text_fmt).
-export([format/2]).
format(#{msg := Msg, meta := Meta} = LogEvent, Config) ->
Prefix = format_prefix(LogEvent, Config),
Color = pick_color(LogEvent, Config),
FormattedMsg = format_msg(Msg, Meta, Config),
prepend_prefix_to_msg_and_add_color(Prefix, Color, FormattedMsg, Config).
format_prefix(_, #{prefix := false}) ->
none;
format_prefix(#{level := Level,
meta := #{time := Timestamp,
pid := Pid}},
Config) ->
Time = format_time(Timestamp, Config),
LevelName = level_name(Level, Config),
io_lib:format("~ts [~ts] ~p", [Time, LevelName, Pid]).
level_name(Level, #{level_name := full}) ->
Level;
level_name(Level, #{level_name := uc3}) ->
level_3letter_uc_name(Level);
level_name(Level, #{level_name := lc3}) ->
level_3letter_lc_name(Level);
level_name(Level, #{level_name := uc4}) ->
level_4letter_uc_name(Level);
level_name(Level, #{level_name := lc4}) ->
level_4letter_lc_name(Level);
level_name(Level, _) ->
Level.
level_3letter_lc_name(debug) -> "dbg";
level_3letter_lc_name(info) -> "inf";
level_3letter_lc_name(notice) -> "ntc";
level_3letter_lc_name(warning) -> "wrn";
level_3letter_lc_name(error) -> "err";
level_3letter_lc_name(critical) -> "crt";
level_3letter_lc_name(alert) -> "alt";
level_3letter_lc_name(emergency) -> "emg".
level_3letter_uc_name(debug) -> "DBG";
level_3letter_uc_name(info) -> "INF";
level_3letter_uc_name(notice) -> "NTC";
level_3letter_uc_name(warning) -> "WRN";
level_3letter_uc_name(error) -> "ERR";
level_3letter_uc_name(critical) -> "CRT";
level_3letter_uc_name(alert) -> "ALT";
level_3letter_uc_name(emergency) -> "EMG".
level_4letter_lc_name(debug) -> "dbug";
level_4letter_lc_name(info) -> "info";
level_4letter_lc_name(notice) -> "noti";
level_4letter_lc_name(warning) -> "warn";
level_4letter_lc_name(error) -> "erro";
level_4letter_lc_name(critical) -> "crit";
level_4letter_lc_name(alert) -> "alrt";
level_4letter_lc_name(emergency) -> "emgc".
level_4letter_uc_name(debug) -> "DBUG";
level_4letter_uc_name(info) -> "INFO";
level_4letter_uc_name(notice) -> "NOTI";
level_4letter_uc_name(warning) -> "WARN";
level_4letter_uc_name(error) -> "ERRO";
level_4letter_uc_name(critical) -> "CRIT";
level_4letter_uc_name(alert) -> "ALRT";
level_4letter_uc_name(emergency) -> "EMGC".
format_time(Timestamp, _) ->
Options = [{unit, microsecond},
{time_designator, $\s}],
calendar:system_time_to_rfc3339(Timestamp, Options).
format_msg({string, Chardata}, Meta, Config) ->
format_msg({"~ts", [Chardata]}, Meta, Config);
format_msg({report, Report}, Meta, Config) ->
FormattedReport = format_report(Report, Meta, Config),
format_msg(FormattedReport, Meta, Config);
format_msg({Format, Args}, _, _) ->
io_lib:format(Format, Args).
format_report(
#{label := {application_controller, _}} = Report, Meta, Config) ->
format_application_progress(Report, Meta, Config);
format_report(
#{label := {supervisor, progress}} = Report, Meta, Config) ->
format_supervisor_progress(Report, Meta, Config);
format_report(
Report, #{report_cb := Cb} = Meta, Config) ->
try
case erlang:fun_info(Cb, arity) of
{arity, 1} -> Cb(Report);
{arity, 2} -> {"~ts", [Cb(Report, #{})]}
end
catch
_:_:_ ->
format_report(Report, maps:remove(report_cb, Meta), Config)
end;
format_report(Report, _, _) ->
logger:format_report(Report).
format_application_progress(#{label := {_, progress},
report := InternalReport}, _, _) ->
Application = proplists:get_value(application, InternalReport),
StartedAt = proplists:get_value(started_at, InternalReport),
{"Application ~w started on ~0p",
[Application, StartedAt]};
format_application_progress(#{label := {_, exit},
report := InternalReport}, _, _) ->
Application = proplists:get_value(application, InternalReport),
Exited = proplists:get_value(exited, InternalReport),
{"Application ~w exited with reason: ~0p",
[Application, Exited]}.
format_supervisor_progress(#{report := InternalReport}, _, _) ->
Supervisor = proplists:get_value(supervisor, InternalReport),
Started = proplists:get_value(started, InternalReport),
Id = proplists:get_value(id, Started),
Pid = proplists:get_value(pid, Started),
Mfa = proplists:get_value(mfargs, Started),
{"Supervisor ~w: child ~w started (~w): ~0p",
[Supervisor, Id, Pid, Mfa]}.
pick_color(_, #{color := false}) ->
{"", ""};
pick_color(#{level := Level}, #{color := true} = Config) ->
ColorStart = level_to_color(Level, Config),
ColorEnd = "\033[0m",
{ColorStart, ColorEnd}.
level_to_color(debug, _) -> "\033[38;5;246m";
level_to_color(info, _) -> "";
level_to_color(notice, _) -> "\033[38;5;87m";
level_to_color(warning, _) -> "\033[38;5;214m";
level_to_color(error, _) -> "\033[38;5;160m";
level_to_color(critical, _) -> "\033[1;37m\033[48;5;20m";
level_to_color(alert, _) -> "\033[1;37m\033[48;5;93m";
level_to_color(emergency, _) -> "\033[1;37m\033[48;5;196m".
prepend_prefix_to_msg_and_add_color(
none, {ColorStart, ColorEnd}, FormattedMsg, Config) ->
Lines = split_lines(FormattedMsg, Config),
[case Line of
"" -> [$\n];
_ -> [ColorStart, Line, ColorEnd, $\n]
end
|| Line <- Lines];
prepend_prefix_to_msg_and_add_color(
Prefix, {ColorStart, ColorEnd}, FormattedMsg, Config) ->
Lines = split_lines(FormattedMsg, Config),
[case Line of
"" -> [ColorStart, Prefix, ColorEnd, $\n];
_ -> [ColorStart, Prefix, " ", Line, ColorEnd, $\n]
end
|| Line <- Lines].
split_lines(FormattedMsg, _) ->
FlattenMsg = lists:flatten(FormattedMsg),
string:split(FlattenMsg, [$\n], all).

View File

@ -1,7 +1,10 @@
-module(rabbit_prelaunch).
-include_lib("kernel/include/logger.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([run_prelaunch_first_phase/0,
assert_mnesia_is_stopped/0,
get_context/0,
@ -24,6 +27,8 @@
run_prelaunch_first_phase() ->
try
ok = logger:set_process_metadata(
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
do_run()
catch
throw:{error, _} = Error ->
@ -67,26 +72,25 @@ do_run() ->
?assertMatch(#{}, Context0),
%% Setup logging for the prelaunch phase.
ok = rabbit_prelaunch_early_logging:setup_early_logging(Context0, true),
ok = rabbit_prelaunch_early_logging:setup_early_logging(Context0),
IsInitialPass = is_initial_pass(),
case IsInitialPass of
true ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug(
"== Prelaunch phase [1/2] (initial pass) =="),
rabbit_log_prelaunch:debug("");
?LOG_DEBUG(""),
?LOG_DEBUG("== Prelaunch phase [1/2] (initial pass) =="),
?LOG_DEBUG("");
false ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Prelaunch phase [1/2] =="),
rabbit_log_prelaunch:debug("")
?LOG_DEBUG(""),
?LOG_DEBUG("== Prelaunch phase [1/2] =="),
?LOG_DEBUG("")
end,
rabbit_env:log_process_env(),
%% Load rabbitmq-env.conf, redo logging setup and continue.
Context1 = rabbit_env:get_context_after_logging_init(Context0),
?assertMatch(#{}, Context1),
ok = rabbit_prelaunch_early_logging:setup_early_logging(Context1, true),
ok = rabbit_prelaunch_early_logging:setup_early_logging(Context1),
rabbit_env:log_process_env(),
%% Complete context now that we have the final environment loaded.
@ -111,7 +115,7 @@ do_run() ->
ok = rabbit_prelaunch_dist:setup(Context),
%% 4. Write PID file.
rabbit_log_prelaunch:debug(""),
?LOG_DEBUG(""),
_ = write_pid_file(Context),
ignore.
@ -138,7 +142,7 @@ get_stop_reason() ->
set_stop_reason(Reason) ->
case get_stop_reason() of
undefined ->
rabbit_log_prelaunch:debug("Set stop reason to: ~p", [Reason]),
?LOG_DEBUG("Set stop reason to: ~p", [Reason]),
persistent_term:put(?PT_KEY_STOP_REASON, Reason);
_ ->
ok
@ -161,7 +165,7 @@ setup_shutdown_func() ->
{ok, {ThisMod, ThisFunc}} ->
ok;
{ok, {ExistingMod, ExistingFunc}} ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Setting up kernel shutdown function: ~s:~s/1 "
"(chained with ~s:~s/1)",
[ThisMod, ThisFunc, ExistingMod, ExistingFunc]),
@ -170,7 +174,7 @@ setup_shutdown_func() ->
ExistingShutdownFunc),
ok = record_kernel_shutdown_func(ThisMod, ThisFunc);
_ ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Setting up kernel shutdown function: ~s:~s/1",
[ThisMod, ThisFunc]),
ok = record_kernel_shutdown_func(ThisMod, ThisFunc)
@ -182,7 +186,7 @@ record_kernel_shutdown_func(Mod, Func) ->
[{persistent, true}]).
shutdown_func(Reason) ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Running ~s:shutdown_func() as part of `kernel` shutdown", [?MODULE]),
Context = get_context(),
remove_pid_file(Context),
@ -195,7 +199,7 @@ shutdown_func(Reason) ->
end.
write_pid_file(#{pid_file := PidFile}) ->
rabbit_log_prelaunch:debug("Writing PID file: ~s", [PidFile]),
?LOG_DEBUG("Writing PID file: ~s", [PidFile]),
case filelib:ensure_dir(PidFile) of
ok ->
OSPid = os:getpid(),
@ -203,13 +207,13 @@ write_pid_file(#{pid_file := PidFile}) ->
ok ->
ok;
{error, Reason} = Error ->
rabbit_log_prelaunch:warning(
?LOG_WARNING(
"Failed to write PID file \"~s\": ~s",
[PidFile, file:format_error(Reason)]),
Error
end;
{error, Reason} = Error ->
rabbit_log_prelaunch:warning(
?LOG_WARNING(
"Failed to create PID file \"~s\" directory: ~s",
[PidFile, file:format_error(Reason)]),
Error
@ -218,10 +222,10 @@ write_pid_file(_) ->
ok.
remove_pid_file(#{pid_file := PidFile, keep_pid_file_on_exit := true}) ->
rabbit_log_prelaunch:debug("Keeping PID file: ~s", [PidFile]),
?LOG_DEBUG("Keeping PID file: ~s", [PidFile]),
ok;
remove_pid_file(#{pid_file := PidFile}) ->
rabbit_log_prelaunch:debug("Deleting PID file: ~s", [PidFile]),
?LOG_DEBUG("Deleting PID file: ~s", [PidFile]),
_ = file:delete(PidFile),
ok;
remove_pid_file(_) ->

View File

@ -1,9 +1,11 @@
-module(rabbit_prelaunch_conf).
-include_lib("kernel/include/file.hrl").
-include_lib("kernel/include/logger.hrl").
-include_lib("stdlib/include/zip.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([setup/1,
get_config_state/0,
@ -15,8 +17,9 @@
-endif.
setup(Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Configuration =="),
?LOG_DEBUG(
"\n== Configuration ==",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
%% TODO: Check if directories/files are inside Mnesia dir.
@ -52,9 +55,10 @@ setup(Context) ->
#{config_files => ConfigFiles,
config_advanced_file => AdvancedConfigFile};
undefined when AdvancedConfigFile =/= undefined ->
rabbit_log_prelaunch:warning(
?LOG_WARNING(
"Using RABBITMQ_ADVANCED_CONFIG_FILE: ~s",
[AdvancedConfigFile]),
[AdvancedConfigFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Config = load_cuttlefish_config_file(Context,
AdditionalConfigFiles,
AdvancedConfigFile),
@ -66,10 +70,10 @@ setup(Context) ->
#{config_files => [],
config_advanced_file => undefined}
end,
ok = override_with_hard_coded_critical_config(),
ok = set_credentials_obfuscation_secret(),
rabbit_log_prelaunch:debug(
"Saving config state to application env: ~p", [State]),
?LOG_DEBUG(
"Saving config state to application env: ~p", [State],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
store_config_state(State).
store_config_state(ConfigState) ->
@ -83,7 +87,8 @@ get_config_state() ->
%% -------------------------------------------------------------------
set_default_config() ->
rabbit_log_prelaunch:debug("Setting default config"),
?LOG_DEBUG("Setting default config",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Config = [
{ra,
[
@ -99,6 +104,8 @@ set_default_config() ->
%% goes down it is still immediately detected
{poll_interval, 5000}
]},
{syslog,
[{app_name, "rabbitmq-server"}]},
{sysmon_handler,
[{process_limit, 100},
{port_limit, 100},
@ -126,15 +133,18 @@ find_actual_main_config_file(#{main_config_file := File}) ->
true ->
case filelib:is_regular(NewFormatFile) of
true ->
rabbit_log_prelaunch:warning(
?LOG_WARNING(
"Both old (.config) and new (.conf) format "
"config files exist."),
rabbit_log_prelaunch:warning(
"config files exist.",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
?LOG_WARNING(
"Using the old format config file: ~s",
[OldFormatFile]),
rabbit_log_prelaunch:warning(
[OldFormatFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
?LOG_WARNING(
"Please update your config files to the new "
"format and remove the old file."),
"format and remove the old file.",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok;
false ->
ok
@ -193,15 +203,18 @@ generate_config_from_cuttlefish_files(Context,
SchemaFiles = find_cuttlefish_schemas(Context),
case SchemaFiles of
[] ->
rabbit_log_prelaunch:error(
"No configuration schema found~n", []),
?LOG_ERROR(
"No configuration schema found~n", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({error, no_configuration_schema_found});
_ ->
rabbit_log_prelaunch:debug(
"Configuration schemas found:~n", []),
?LOG_DEBUG(
"Configuration schemas found:~n", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
lists:foreach(
fun(SchemaFile) ->
rabbit_log_prelaunch:debug(" - ~ts", [SchemaFile])
?LOG_DEBUG(" - ~ts", [SchemaFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})
end,
SchemaFiles),
ok
@ -209,37 +222,44 @@ generate_config_from_cuttlefish_files(Context,
Schema = cuttlefish_schema:files(SchemaFiles),
%% Load configuration.
rabbit_log_prelaunch:debug(
"Loading configuration files (Cuttlefish based):"),
?LOG_DEBUG(
"Loading configuration files (Cuttlefish based):",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
lists:foreach(
fun(ConfigFile) ->
rabbit_log_prelaunch:debug(" - ~ts", [ConfigFile])
?LOG_DEBUG(" - ~ts", [ConfigFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})
end, ConfigFiles),
case cuttlefish_conf:files(ConfigFiles) of
{errorlist, Errors} ->
rabbit_log_prelaunch:error("Error parsing configuration:"),
?LOG_ERROR("Error parsing configuration:",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
lists:foreach(
fun(Error) ->
rabbit_log_prelaunch:error(
?LOG_ERROR(
" - ~ts",
[cuttlefish_error:xlate(Error)])
[cuttlefish_error:xlate(Error)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})
end, Errors),
rabbit_log_prelaunch:error(
"Are these files using the Cuttlefish format?"),
?LOG_ERROR(
"Are these files using the Cuttlefish format?",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({error, failed_to_parse_configuration_file});
Config0 ->
%% Finalize configuration, based on the schema.
Config = case cuttlefish_generator:map(Schema, Config0) of
{error, Phase, {errorlist, Errors}} ->
%% TODO
rabbit_log_prelaunch:error(
?LOG_ERROR(
"Error preparing configuration in phase ~ts:",
[Phase]),
[Phase],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
lists:foreach(
fun(Error) ->
rabbit_log_prelaunch:error(
?LOG_ERROR(
" - ~ts",
[cuttlefish_error:xlate(Error)])
[cuttlefish_error:xlate(Error)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})
end, Errors),
throw(
{error, failed_to_prepare_configuration});
@ -253,8 +273,9 @@ generate_config_from_cuttlefish_files(Context,
find_cuttlefish_schemas(Context) ->
Apps = list_apps(Context),
rabbit_log_prelaunch:debug(
"Looking up configuration schemas in the following applications:"),
?LOG_DEBUG(
"Looking up configuration schemas in the following applications:",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
find_cuttlefish_schemas(Apps, []).
find_cuttlefish_schemas([App | Rest], AllSchemas) ->
@ -281,9 +302,10 @@ list_apps1([Dir | Rest], Apps) ->
Apps1 = lists:umerge(Apps, lists:sort(NewApps)),
list_apps1(Rest, Apps1);
{error, Reason} ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Failed to list directory \"~ts\" content: ~ts",
[Dir, file:format_error(Reason)]),
[Dir, file:format_error(Reason)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
list_apps1(Rest, Apps)
end;
list_apps1([], AppInfos) ->
@ -299,17 +321,19 @@ list_schemas_in_app(App) ->
true ->
case code:priv_dir(App) of
{error, bad_name} ->
rabbit_log_prelaunch:debug(
" [ ] ~s (no readable priv dir)", [App]),
?LOG_DEBUG(
" [ ] ~s (no readable priv dir)", [App],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
[];
PrivDir ->
SchemaDir = filename:join([PrivDir, "schema"]),
do_list_schemas_in_app(App, SchemaDir)
end;
Reason1 ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
" [ ] ~s (failed to load application: ~p)",
[App, Reason1]),
[App, Reason1],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
[]
end,
case Unload of
@ -322,74 +346,71 @@ list_schemas_in_app(App) ->
do_list_schemas_in_app(App, SchemaDir) ->
case erl_prim_loader:list_dir(SchemaDir) of
{ok, Files} ->
rabbit_log_prelaunch:debug(" [x] ~s", [App]),
?LOG_DEBUG(" [x] ~s", [App],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
[filename:join(SchemaDir, File)
|| [C | _] = File <- Files,
C =/= $.];
error ->
rabbit_log_prelaunch:debug(
" [ ] ~s (no readable schema dir)", [App]),
?LOG_DEBUG(
" [ ] ~s (no readable schema dir)", [App],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
[]
end.
override_with_advanced_config(Config, undefined) ->
Config;
override_with_advanced_config(Config, AdvancedConfigFile) ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Override with advanced configuration file \"~ts\"",
[AdvancedConfigFile]),
[AdvancedConfigFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case file:consult(AdvancedConfigFile) of
{ok, [AdvancedConfig]} ->
cuttlefish_advanced:overlay(Config, AdvancedConfig);
{ok, OtherTerms} ->
rabbit_log_prelaunch:error(
?LOG_ERROR(
"Failed to load advanced configuration file \"~ts\", "
"incorrect format: ~p",
[AdvancedConfigFile, OtherTerms]),
[AdvancedConfigFile, OtherTerms],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({error, failed_to_parse_advanced_configuration_file});
{error, Reason} ->
rabbit_log_prelaunch:error(
?LOG_ERROR(
"Failed to load advanced configuration file \"~ts\": ~ts",
[AdvancedConfigFile, file:format_error(Reason)]),
[AdvancedConfigFile, file:format_error(Reason)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({error, failed_to_read_advanced_configuration_file})
end.
override_with_hard_coded_critical_config() ->
rabbit_log_prelaunch:debug("Override with hard-coded critical config"),
Config = [
{ra,
%% Make Ra use a custom logger that dispatches to lager
%% instead of the default OTP logger
[{logger_module, rabbit_log_ra_shim}]},
{osiris,
[{logger_module, rabbit_log_osiris_shim}]}
],
apply_erlang_term_based_config(Config).
apply_erlang_term_based_config([{_, []} | Rest]) ->
apply_erlang_term_based_config(Rest);
apply_erlang_term_based_config([{App, Vars} | Rest]) ->
rabbit_log_prelaunch:debug(" Applying configuration for '~s':", [App]),
?LOG_DEBUG(" Applying configuration for '~s':", [App],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = apply_app_env_vars(App, Vars),
apply_erlang_term_based_config(Rest);
apply_erlang_term_based_config([]) ->
ok.
apply_app_env_vars(App, [{Var, Value} | Rest]) ->
rabbit_log_prelaunch:debug(" - ~s = ~p", [Var, Value]),
?LOG_DEBUG(" - ~s = ~p", [Var, Value],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = application:set_env(App, Var, Value, [{persistent, true}]),
apply_app_env_vars(App, Rest);
apply_app_env_vars(_, []) ->
ok.
set_credentials_obfuscation_secret() ->
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Refreshing credentials obfuscation configuration from env: ~p",
[application:get_all_env(credentials_obfuscation)]),
[application:get_all_env(credentials_obfuscation)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = credentials_obfuscation:refresh_config(),
CookieBin = rabbit_data_coercion:to_binary(erlang:get_cookie()),
rabbit_log_prelaunch:debug(
"Setting credentials obfuscation secret to '~s'", [CookieBin]),
?LOG_DEBUG(
"Setting credentials obfuscation secret to '~s'", [CookieBin],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = credentials_obfuscation:set_secret(CookieBin).
%% -------------------------------------------------------------------
@ -397,7 +418,8 @@ set_credentials_obfuscation_secret() ->
%% -------------------------------------------------------------------
decrypt_config(Apps) ->
rabbit_log_prelaunch:debug("Decoding encrypted config values (if any)"),
?LOG_DEBUG("Decoding encrypted config values (if any)",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ConfigEntryDecoder = application:get_env(rabbit, config_entry_decoder, []),
decrypt_config(Apps, ConfigEntryDecoder).
@ -415,8 +437,9 @@ decrypt_app(App, [{Key, Value} | Tail], Algo) ->
{Value, Algo1} ->
Algo1;
{NewValue, Algo1} ->
rabbit_log_prelaunch:debug(
"Value of `~s` decrypted", [Key]),
?LOG_DEBUG(
"Value of `~s` decrypted", [Key],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = application:set_env(App, Key, NewValue,
[{persistent, true}]),
Algo1
@ -474,7 +497,8 @@ config_entry_decoder_to_algo(ConfigEntryDecoder) ->
end.
get_passphrase(ConfigEntryDecoder) ->
rabbit_log_prelaunch:debug("Getting encrypted config passphrase"),
?LOG_DEBUG("Getting encrypted config passphrase",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case proplists:get_value(passphrase, ConfigEntryDecoder) of
prompt ->
IoDevice = get_input_iodevice(),

View File

@ -1,12 +1,19 @@
-module(rabbit_prelaunch_dist).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([setup/1]).
setup(#{nodename := Node, nodename_type := NameType} = Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Erlang distribution =="),
rabbit_log_prelaunch:debug("Rqeuested node name: ~s (type: ~s)",
[Node, NameType]),
?LOG_DEBUG(
"~n== Erlang distribution ==", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
?LOG_DEBUG(
"Rqeuested node name: ~s (type: ~s)",
[Node, NameType],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case node() of
nonode@nohost ->
ok = rabbit_nodes_common:ensure_epmd(),
@ -16,8 +23,9 @@ setup(#{nodename := Node, nodename_type := NameType} = Context) ->
ok = do_setup(Context);
Node ->
rabbit_log_prelaunch:debug(
"Erlang distribution already running", []),
?LOG_DEBUG(
"Erlang distribution already running", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok;
Unexpected ->
throw({error, {erlang_dist_running_with_unexpected_nodename,
@ -26,7 +34,9 @@ setup(#{nodename := Node, nodename_type := NameType} = Context) ->
ok.
do_setup(#{nodename := Node, nodename_type := NameType}) ->
rabbit_log_prelaunch:debug("Starting Erlang distribution", []),
?LOG_DEBUG(
"Starting Erlang distribution",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case application:get_env(kernel, net_ticktime) of
{ok, Ticktime} when is_integer(Ticktime) andalso Ticktime >= 1 ->
%% The value passed to net_kernel:start/1 is the
@ -43,8 +53,9 @@ do_setup(#{nodename := Node, nodename_type := NameType}) ->
%% Check whether a node with the same name is already running
duplicate_node_check(#{split_nodename := {NodeName, NodeHost}}) ->
rabbit_log_prelaunch:debug(
"Checking if node name ~s is already used", [NodeName]),
?LOG_DEBUG(
"Checking if node name ~s is already used", [NodeName],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
PrelaunchName = rabbit_nodes_common:make(
{NodeName ++ "_prelaunch_" ++ os:getpid(),
"localhost"}),
@ -63,8 +74,9 @@ duplicate_node_check(#{split_nodename := {NodeName, NodeHost}}) ->
end.
dist_port_range_check(#{erlang_dist_tcp_port := DistTcpPort}) ->
rabbit_log_prelaunch:debug(
"Checking if TCP port ~b is valid", [DistTcpPort]),
?LOG_DEBUG(
"Checking if TCP port ~b is valid", [DistTcpPort],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case DistTcpPort of
_ when DistTcpPort < 1 orelse DistTcpPort > 65535 ->
throw({error, {invalid_dist_port_range, DistTcpPort}});
@ -74,8 +86,9 @@ dist_port_range_check(#{erlang_dist_tcp_port := DistTcpPort}) ->
dist_port_use_check(#{split_nodename := {_, NodeHost},
erlang_dist_tcp_port := DistTcpPort}) ->
rabbit_log_prelaunch:debug(
"Checking if TCP port ~b is available", [DistTcpPort]),
?LOG_DEBUG(
"Checking if TCP port ~b is available", [DistTcpPort],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
dist_port_use_check_ipv4(NodeHost, DistTcpPort).
dist_port_use_check_ipv4(NodeHost, Port) ->

View File

@ -1,60 +1,126 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2019-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_prelaunch_early_logging).
-include_lib("rabbit_common/include/rabbit_log.hrl").
-include_lib("kernel/include/logger.hrl").
-export([setup_early_logging/2,
-include_lib("rabbit_common/include/logging.hrl").
-export([setup_early_logging/1,
default_formatter/1,
default_console_formatter/1,
default_file_formatter/1,
default_syslog_formatter/1,
enable_quick_dbg/1,
use_colored_logging/0,
use_colored_logging/1,
list_expected_sinks/0]).
use_colored_logging/1]).
-export([filter_log_event/2]).
setup_early_logging(#{log_levels := undefined} = Context,
LagerEventToStdout) ->
setup_early_logging(Context#{log_levels => get_default_log_level()},
LagerEventToStdout);
setup_early_logging(Context, LagerEventToStdout) ->
Configured = lists:member(
lager_util:make_internal_sink_name(rabbit_log_prelaunch),
lager:list_all_sinks()),
case Configured of
-define(CONFIGURED_KEY, {?MODULE, configured}).
setup_early_logging(#{log_levels := undefined} = Context) ->
setup_early_logging(Context#{log_levels => get_default_log_level()});
setup_early_logging(Context) ->
case is_configured() of
true -> ok;
false -> do_setup_early_logging(Context, LagerEventToStdout)
false -> do_setup_early_logging(Context)
end.
get_default_log_level() ->
#{"prelaunch" => warning}.
#{"prelaunch" => notice}.
do_setup_early_logging(#{log_levels := LogLevels} = Context,
LagerEventToStdout) ->
redirect_logger_messages_to_lager(),
Colored = use_colored_logging(Context),
application:set_env(lager, colored, Colored),
ConsoleBackend = lager_console_backend,
case LagerEventToStdout of
true ->
GLogLevel = case LogLevels of
#{global := Level} -> Level;
_ -> warning
end,
_ = lager_app:start_handler(
lager_event, ConsoleBackend, [{level, GLogLevel}]),
ok;
false ->
ok
end,
lists:foreach(
fun(Sink) ->
CLogLevel = get_log_level(LogLevels, Sink),
lager_app:configure_sink(
Sink,
[{handlers, [{ConsoleBackend, [{level, CLogLevel}]}]}])
end, list_expected_sinks()),
do_setup_early_logging(#{log_levels := LogLevels} = Context) ->
add_rmqlog_filter(LogLevels),
ok = logger:update_handler_config(
default, main_handler_config(Context)).
is_configured() ->
persistent_term:get(?CONFIGURED_KEY, false).
add_rmqlog_filter(LogLevels) ->
add_erlang_specific_filters(LogLevels),
FilterConfig0 = lists:foldl(
fun
({_, V}, FC) when is_boolean(V) -> FC;
({K, V}, FC) when is_atom(K) -> FC#{K => V};
({K, V}, FC) -> FC#{list_to_atom(K) => V}
end, #{}, maps:to_list(LogLevels)),
FilterConfig1 = case maps:is_key(global, FilterConfig0) of
true -> FilterConfig0;
false -> FilterConfig0#{global => ?DEFAULT_LOG_LEVEL}
end,
ok = logger:add_handler_filter(
default, ?FILTER_NAME, {fun filter_log_event/2, FilterConfig1}),
ok = logger:set_primary_config(level, all),
ok = persistent_term:put(?CONFIGURED_KEY, true).
add_erlang_specific_filters(_) ->
_ = logger:add_handler_filter(
default, progress_reports, {fun logger_filters:progress/2, stop}),
ok.
redirect_logger_messages_to_lager() ->
io:format(standard_error, "Configuring logger redirection~n", []),
ok = logger:add_handler(rabbit_log, rabbit_log, #{}),
ok = logger:set_primary_config(level, all).
filter_log_event(
#{meta := #{domain := ?RMQLOG_DOMAIN_GLOBAL}} = LogEvent,
FilterConfig) ->
MinLevel = get_min_level(global, FilterConfig),
do_filter_log_event(LogEvent, MinLevel);
filter_log_event(
#{meta := #{domain := [?RMQLOG_SUPER_DOMAIN_NAME, CatName | _]}} = LogEvent,
FilterConfig) ->
MinLevel = get_min_level(CatName, FilterConfig),
do_filter_log_event(LogEvent, MinLevel);
filter_log_event(
#{meta := #{domain := [CatName | _]}} = LogEvent,
FilterConfig) ->
MinLevel = get_min_level(CatName, FilterConfig),
do_filter_log_event(LogEvent, MinLevel);
filter_log_event(LogEvent, FilterConfig) ->
MinLevel = get_min_level(global, FilterConfig),
do_filter_log_event(LogEvent, MinLevel).
get_min_level(global, FilterConfig) ->
maps:get(global, FilterConfig, none);
get_min_level(CatName, FilterConfig) ->
case maps:is_key(CatName, FilterConfig) of
true -> maps:get(CatName, FilterConfig);
false -> get_min_level(global, FilterConfig)
end.
do_filter_log_event(_, none) ->
stop;
do_filter_log_event(#{level := Level} = LogEvent, MinLevel) ->
case logger:compare_levels(Level, MinLevel) of
lt -> stop;
_ -> LogEvent
end.
main_handler_config(Context) ->
#{filter_default => log,
formatter => default_formatter(Context)}.
default_formatter(#{log_levels := #{json := true}}) ->
{rabbit_logger_json_fmt, #{}};
default_formatter(Context) ->
Color = use_colored_logging(Context),
{rabbit_logger_text_fmt, #{color => Color}}.
default_console_formatter(Context) ->
default_formatter(Context).
default_file_formatter(Context) ->
default_formatter(Context#{output_supports_colors => false}).
default_syslog_formatter(Context) ->
{Module, Config} = default_file_formatter(Context),
case Module of
rabbit_logger_text_fmt -> {Module, Config#{prefix => false}};
rabbit_logger_json_fmt -> {Module, Config}
end.
use_colored_logging() ->
use_colored_logging(rabbit_prelaunch:get_context()).
@ -65,45 +131,6 @@ use_colored_logging(#{log_levels := #{color := true},
use_colored_logging(_) ->
false.
list_expected_sinks() ->
Key = {?MODULE, lager_extra_sinks},
case persistent_term:get(Key, undefined) of
undefined ->
CompileOptions = proplists:get_value(options,
module_info(compile),
[]),
AutoList = [lager_util:make_internal_sink_name(M)
|| M <- proplists:get_value(lager_extra_sinks,
CompileOptions, [])],
List = case lists:member(?LAGER_SINK, AutoList) of
true -> AutoList;
false -> [?LAGER_SINK | AutoList]
end,
%% Store the list in the application environment. If this
%% module is later cover-compiled, the compile option will
%% be lost, so we will be able to retrieve the list from the
%% application environment.
persistent_term:put(Key, List),
List;
List ->
List
end.
sink_to_category(Sink) when is_atom(Sink) ->
re:replace(
atom_to_list(Sink),
"^rabbit_log_(.+)_lager_event$",
"\\1",
[{return, list}]).
get_log_level(LogLevels, Sink) ->
Category = sink_to_category(Sink),
case LogLevels of
#{Category := Level} -> Level;
#{global := Level} -> Level;
_ -> warning
end.
enable_quick_dbg(#{dbg_output := Output, dbg_mods := Mods}) ->
case Output of
stdout -> {ok, _} = dbg:tracer(),

View File

@ -1,25 +1,32 @@
-module(rabbit_prelaunch_erlang_compat).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([check/1]).
-define(OTP_MINIMUM, "23.0").
-define(ERTS_MINIMUM, "11.1").
check(_Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Erlang/OTP compatibility check =="),
?LOG_DEBUG(
"~n== Erlang/OTP compatibility check ==", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ERTSVer = erlang:system_info(version),
OTPRel = rabbit_misc:otp_release(),
rabbit_log_prelaunch:debug(
"Requiring: Erlang/OTP ~s (ERTS ~s)", [?OTP_MINIMUM, ?ERTS_MINIMUM]),
rabbit_log_prelaunch:debug(
"Running: Erlang/OTP ~s (ERTS ~s)", [OTPRel, ERTSVer]),
?LOG_DEBUG(
"Requiring: Erlang/OTP ~s (ERTS ~s)~n"
"Running: Erlang/OTP ~s (ERTS ~s)",
[?OTP_MINIMUM, ?ERTS_MINIMUM, OTPRel, ERTSVer],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case rabbit_misc:version_compare(?ERTS_MINIMUM, ERTSVer, lte) of
true when ?ERTS_MINIMUM =/= ERTSVer ->
rabbit_log_prelaunch:debug(
"Erlang/OTP version requirement satisfied"),
?LOG_DEBUG(
"Erlang/OTP version requirement satisfied", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok;
true when ?ERTS_MINIMUM =:= ERTSVer andalso ?OTP_MINIMUM =< OTPRel ->
%% When a critical regression or bug is found, a new OTP
@ -35,7 +42,7 @@ check(_Context) ->
"This RabbitMQ version cannot run on Erlang ~s (erts ~s): "
"minimum required version is ~s (erts ~s)",
Args = [OTPRel, ERTSVer, ?OTP_MINIMUM, ?ERTS_MINIMUM],
rabbit_log_prelaunch:error(Msg, Args),
?LOG_ERROR(Msg, Args, #{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
%% Also print to stderr to make this more visible
io:format(standard_error, "Error: " ++ Msg ++ "~n", Args),

View File

@ -1,5 +1,9 @@
-module(rabbit_prelaunch_errors).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([format_error/1,
format_exception/3,
log_error/1,
@ -94,9 +98,19 @@ log_exception(Class, Exception, Stacktrace) ->
log_message(Message).
format_exception(Class, Exception, Stacktrace) ->
StacktraceStrs = [case proplists:get_value(line, Props) of
undefined ->
io_lib:format(" ~ts:~ts/~b",
[Mod, Fun, Arity]);
Line ->
io_lib:format(" ~ts:~ts/~b, line ~b",
[Mod, Fun, Arity, Line])
end
|| {Mod, Fun, Arity, Props} <- Stacktrace],
ExceptionStr = io_lib:format("~ts:~0p", [Class, Exception]),
rabbit_misc:format(
"Exception during startup:~n~s",
[lager:pr_stacktrace(Stacktrace, {Class, Exception})]).
"Exception during startup:~n~n~s~n~n~s",
[ExceptionStr, string:join(StacktraceStrs, "\n")]).
log_message(Message) ->
Lines = string:split(
@ -105,9 +119,11 @@ log_message(Message) ->
?BOOT_FAILED_FOOTER,
[$\n],
all),
?LOG_ERROR(
"~s", [string:join(Lines, "\n")],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
lists:foreach(
fun(Line) ->
rabbit_log_prelaunch:error("~s", [Line]),
io:format(standard_error, "~s~n", [Line])
end, Lines),
timer:sleep(1000),

View File

@ -0,0 +1,212 @@
-module(rabbit_logger_std_h_SUITE).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-export([all/0,
groups/0,
init_per_suite/2,
end_per_suite/2,
init_per_group/2,
end_per_group/2,
init_per_testcase/2,
end_per_testcase/2,
every_day_rotation_is_detected/1,
every_week_rotation_is_detected/1,
every_month_rotation_is_detected/1
]).
all() ->
[
{group, parallel_tests}
].
groups() ->
[
{parallel_tests, [parallel], [every_day_rotation_is_detected,
every_week_rotation_is_detected,
every_month_rotation_is_detected]}
].
init_per_suite(_, Config) -> Config.
end_per_suite(_, Config) -> Config.
init_per_group(_, Config) -> Config.
end_per_group(_, Config) -> Config.
init_per_testcase(_, Config) -> Config.
end_per_testcase(_, Config) -> Config.
every_day_rotation_is_detected(_) ->
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2021, 01, 15}, {10, 00, 00}},
{{2021, 01, 15}, {11, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2021, 01, 15}, {10, 00, 00}},
{{2021, 01, 15}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2021, 01, 15}, {10, 00, 00}},
{{2021, 01, 15}, {13, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2021, 01, 15}, {11, 00, 00}},
{{2021, 01, 15}, {13, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2021, 01, 15}, {12, 00, 00}},
{{2021, 01, 15}, {13, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2021, 01, 14}, {12, 00, 00}},
{{2021, 01, 15}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2021, 01, 14}, {12, 00, 00}},
{{2021, 01, 15}, {11, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2020, 11, 15}, {12, 00, 00}},
{{2021, 01, 15}, {11, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => day, hour => 12},
{{2020, 11, 15}, {12, 00, 00}},
{{2021, 01, 15}, {12, 00, 00}})).
every_week_rotation_is_detected(_) ->
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 11}, {12, 00, 00}},
{{2021, 01, 12}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 11}, {12, 00, 00}},
{{2021, 01, 13}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 11}, {12, 00, 00}},
{{2021, 01, 14}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 13}, {12, 00, 00}},
{{2021, 01, 14}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 14}, {12, 00, 00}},
{{2021, 01, 15}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 13}, {11, 00, 00}},
{{2021, 01, 13}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 06}, {12, 00, 00}},
{{2021, 01, 13}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 07}, {12, 00, 00}},
{{2021, 01, 14}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 06}, {12, 00, 00}},
{{2021, 01, 12}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 06}, {11, 00, 00}},
{{2021, 01, 12}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => week, day_of_week => 3, hour => 12},
{{2021, 01, 06}, {12, 00, 00}},
{{2021, 01, 13}, {11, 00, 00}})).
every_month_rotation_is_detected(_) ->
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 15, hour => 12},
{{2021, 01, 15}, {10, 00, 00}},
{{2021, 01, 15}, {11, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 15, hour => 12},
{{2021, 01, 15}, {10, 00, 00}},
{{2021, 01, 15}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 15, hour => 12},
{{2021, 01, 13}, {12, 00, 00}},
{{2021, 01, 14}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 15, hour => 12},
{{2021, 01, 14}, {12, 00, 00}},
{{2021, 01, 15}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 15, hour => 12},
{{2021, 01, 15}, {12, 00, 00}},
{{2021, 01, 16}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 15, hour => 12},
{{2021, 01, 14}, {12, 00, 00}},
{{2021, 02, 14}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 15, hour => 12},
{{2021, 01, 16}, {12, 00, 00}},
{{2021, 02, 16}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 30, hour => 12},
{{2021, 01, 29}, {12, 00, 00}},
{{2021, 01, 30}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 30, hour => 12},
{{2021, 01, 30}, {12, 00, 00}},
{{2021, 01, 31}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => 30, hour => 12},
{{2021, 02, 27}, {12, 00, 00}},
{{2021, 02, 28}, {12, 00, 00}})),
?assertNot(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => last, hour => 12},
{{2021, 01, 29}, {12, 00, 00}},
{{2021, 01, 30}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => last, hour => 12},
{{2021, 01, 30}, {12, 00, 00}},
{{2021, 01, 31}, {12, 00, 00}})),
?assert(
rabbit_logger_std_h:is_date_based_rotation_needed(
#{every => month, day_of_month => last, hour => 12},
{{2021, 01, 30}, {12, 00, 00}},
{{2021, 02, 01}, {12, 00, 00}})).

View File

@ -883,7 +883,7 @@
## Logging settings.
##
## See https://rabbitmq.com/logging.html and https://github.com/erlang-lager/lager for details.
## See https://rabbitmq.com/logging.html for details.
##
## Log directory, taken from the RABBITMQ_LOG_BASE env variable by default.

View File

@ -146,25 +146,6 @@ For example, to reset the RabbitMQ node:
.sp
.Dl rabbitmqctl reset
.\" ------------------------------------------------------------------
.It Cm rotate_logs
.Pp
Instructs the RabbitMQ node to perform internal log rotation.
.Pp
Log rotation is performed according to lager settings specified in
configuration file.
.Pp
Note that there is no need to call this command in case of external log
rotation (e.g. from logrotate(8)), because lager detects renames and
automatically reopens log files.
.Pp
For example, this command starts internal log rotation
process:
.sp
.Dl rabbitmqctl rotate_logs
.Pp
Rotation is performed asynchronously, so there is no guarantee that it
will be completed when this command returns.
.\" ------------------------------------------------------------------
.It Cm shutdown
.Pp
Shuts down the node, both RabbitMQ and its runtime.

View File

@ -1179,10 +1179,10 @@ end}.
]}.
% ==========================
% Lager section
% Logging section
% ==========================
{mapping, "log.dir", "lager.log_root", [
{mapping, "log.dir", "rabbit.log_root", [
{datatype, string},
{validators, ["dir_writable"]}]}.

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -79,8 +79,8 @@ start_rabbitmq_server() {
${RABBITMQ_SERVER_ERL_ARGS} \
${RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS} \
${RABBITMQ_SERVER_START_ARGS} \
-lager crash_log false \
-lager handlers '[]' \
-syslog logger '[]' \
-syslog syslog_error_logger false \
"$@"
}

View File

@ -68,8 +68,8 @@ if "!RABBITMQ_ALLOW_INPUT!"=="" (
!RABBITMQ_SERVER_ERL_ARGS! ^
!RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS! ^
!RABBITMQ_SERVER_START_ARGS! ^
-lager crash_log false ^
-lager handlers "[]" ^
-syslog logger [] ^
-syslog syslog_error_logger false ^
!STAR!
if ERRORLEVEL 1 (

View File

@ -198,8 +198,8 @@ set ERLANG_SERVICE_ARGUMENTS= ^
!RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS! ^
!RABBITMQ_SERVER_START_ARGS! ^
!RABBITMQ_DIST_ARG! ^
-lager crash_log false ^
-lager handlers "[]" ^
-syslog logger [] ^
-syslog syslog_error_logger false ^
!STARVAR!
set ERLANG_SERVICE_ARGUMENTS=!ERLANG_SERVICE_ARGUMENTS:\=\\!

View File

@ -1,233 +0,0 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2007-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc RabbitMQ backend for lager.
%% Configuration is a proplist with the following keys:
%% <ul>
%% <li>`level' - log level to use</li>
%% <li>`formatter' - the module to use when formatting log messages. Defaults to
%% `lager_default_formatter'</li>
%% <li>`formatter_config' - the format configuration string. Defaults to
%% `time [ severity ] message'</li>
%% </ul>
-module(lager_exchange_backend).
-behaviour(gen_event).
-export([init/1, terminate/2, code_change/3,
handle_call/2, handle_event/2, handle_info/2]).
-export([maybe_init_exchange/0]).
-include("rabbit.hrl").
-include("rabbit_framing.hrl").
-include_lib("lager/include/lager.hrl").
-record(state, {level :: {'mask', integer()},
formatter :: atom(),
format_config :: any(),
init_exchange_ts = undefined :: integer() | undefined,
exchange = undefined :: #resource{} | undefined}).
-ifdef(TEST).
-include_lib("eunit/include/eunit.hrl").
-compile([{parse_transform, lager_transform}]).
-endif.
-define(INIT_EXCHANGE_INTERVAL_SECS, 5).
-define(TERSE_FORMAT, [time, " [", severity, "] ", message]).
-define(DEFAULT_FORMAT_CONFIG, ?TERSE_FORMAT).
-define(FORMAT_CONFIG_OFF, []).
-ifdef(TEST).
-define(DEPRECATED(_Msg), ok).
-else.
-define(DEPRECATED(Msg),
io:format(user, "WARNING: This is a deprecated lager_exchange_backend configuration. Please use \"~w\" instead.~n", [Msg])).
-endif.
-define(LOG_EXCH_NAME, <<"amq.rabbitmq.log">>).
init([Level]) when is_atom(Level) ->
?DEPRECATED([{level, Level}]),
init([{level, Level}]);
init([Level, true]) when is_atom(Level) -> % for backwards compatibility
?DEPRECATED([{level, Level}, {formatter_config, [{eol, "\\r\\n\\"}]}]),
init([{level, Level}, {formatter_config, ?FORMAT_CONFIG_OFF}]);
init([Level, false]) when is_atom(Level) -> % for backwards compatibility
?DEPRECATED([{level, Level}]),
init([{level, Level}]);
init(Options) when is_list(Options) ->
true = validate_options(Options),
Level = get_option(level, Options, undefined),
try lager_util:config_to_mask(Level) of
L ->
DefaultOptions = [{formatter, lager_default_formatter},
{formatter_config, ?DEFAULT_FORMAT_CONFIG}],
[Formatter, Config] = [get_option(K, Options, Default) || {K, Default} <- DefaultOptions],
State0 = #state{level=L,
formatter=Formatter,
format_config=Config},
% NB: this will probably always fail since the / vhost isn't available
State1 = maybe_init_exchange(State0),
{ok, State1}
catch
_:_ ->
{error, {fatal, bad_log_level}}
end;
init(Level) when is_atom(Level) ->
?DEPRECATED([{level, Level}]),
init([{level, Level}]);
init(Other) ->
{error, {fatal, {bad_lager_exchange_backend_config, Other}}}.
% rabbitmq/rabbitmq-server#1973
% This is called immediatly after the / vhost is created
% or recovered
maybe_init_exchange() ->
case lists:member(?MODULE, gen_event:which_handlers(lager_event)) of
true ->
_ = init_exchange(true),
ok;
_ ->
ok
end.
validate_options([]) -> true;
validate_options([{level, L}|T]) when is_atom(L) ->
case lists:member(L, ?LEVELS) of
false ->
throw({error, {fatal, {bad_level, L}}});
true ->
validate_options(T)
end;
validate_options([{formatter, M}|T]) when is_atom(M) ->
validate_options(T);
validate_options([{formatter_config, C}|T]) when is_list(C) ->
validate_options(T);
validate_options([H|_]) ->
throw({error, {fatal, {bad_lager_exchange_backend_config, H}}}).
get_option(K, Options, Default) ->
case lists:keyfind(K, 1, Options) of
{K, V} -> V;
false -> Default
end.
handle_call(get_loglevel, #state{level=Level} = State) ->
{ok, Level, State};
handle_call({set_loglevel, Level}, State) ->
try lager_util:config_to_mask(Level) of
Levels ->
{ok, ok, State#state{level=Levels}}
catch
_:_ ->
{ok, {error, bad_log_level}, State}
end;
handle_call(_Request, State) ->
{ok, ok, State}.
handle_event({log, _Message} = Event, State0) ->
State1 = maybe_init_exchange(State0),
handle_log_event(Event, State1);
handle_event(_Event, State) ->
{ok, State}.
handle_info(_Info, State) ->
{ok, State}.
terminate(_Reason, _State) ->
ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%% @private
handle_log_event({log, _Message}, #state{exchange=undefined} = State) ->
% NB: tried to define the exchange but still undefined,
% so not logging this message. Note: we can't log this dropped
% message because it will start an infinite loop
{ok, State};
handle_log_event({log, Message},
#state{level=L, exchange=LogExch,
formatter=Formatter, format_config=FormatConfig} = State) ->
case lager_util:is_loggable(Message, L, ?MODULE) of
true ->
%% 0-9-1 says the timestamp is a "64 bit POSIX timestamp". That's
%% second resolution, not millisecond.
RoutingKey = rabbit_data_coercion:to_binary(lager_msg:severity(Message)),
Timestamp = os:system_time(seconds),
Node = rabbit_data_coercion:to_binary(node()),
Headers = [{<<"node">>, longstr, Node}],
AmqpMsg = #'P_basic'{content_type = <<"text/plain">>,
timestamp = Timestamp,
headers = Headers},
Body = rabbit_data_coercion:to_binary(Formatter:format(Message, FormatConfig)),
case rabbit_basic:publish(LogExch, RoutingKey, AmqpMsg, Body) of
ok -> ok;
{error, not_found} -> ok
end,
{ok, State};
false ->
{ok, State}
end.
%% @private
maybe_init_exchange(#state{exchange=undefined, init_exchange_ts=undefined} = State) ->
Now = erlang:monotonic_time(second),
handle_init_exchange(init_exchange(true), Now, State);
maybe_init_exchange(#state{exchange=undefined, init_exchange_ts=Timestamp} = State) ->
Now = erlang:monotonic_time(second),
% NB: since we may try to declare the exchange on every log message, this ensures
% that we only try once every 5 seconds
HasEnoughTimeElapsed = Now - Timestamp > ?INIT_EXCHANGE_INTERVAL_SECS,
Result = init_exchange(HasEnoughTimeElapsed),
handle_init_exchange(Result, Now, State);
maybe_init_exchange(State) ->
State.
%% @private
init_exchange(true) ->
{ok, DefaultVHost} = application:get_env(rabbit, default_vhost),
Exchange = rabbit_misc:r(DefaultVHost, exchange, ?LOG_EXCH_NAME),
try
%% durable
#exchange{} = rabbit_exchange:declare(Exchange, topic, true, false, true, [], ?INTERNAL_USER),
rabbit_log:info("Declared exchange '~s' in vhost '~s'", [?LOG_EXCH_NAME, DefaultVHost]),
{ok, Exchange}
catch
ErrType:Err ->
rabbit_log:error("Could not declare exchange '~s' in vhost '~s', reason: ~p:~p",
[?LOG_EXCH_NAME, DefaultVHost, ErrType, Err]),
{ok, undefined}
end;
init_exchange(_) ->
{ok, undefined}.
%% @private
handle_init_exchange({ok, undefined}, Now, State) ->
State#state{init_exchange_ts=Now};
handle_init_exchange({ok, Exchange}, Now, State) ->
State#state{exchange=Exchange, init_exchange_ts=Now}.
-ifdef(TEST).
console_config_validation_test_() ->
Good = [{level, info}],
Bad1 = [{level, foo}],
Bad2 = [{larval, info}],
AllGood = [{level, info}, {formatter, my_formatter},
{formatter_config, ["blort", "garbage"]}],
[
?_assertEqual(true, validate_options(Good)),
?_assertThrow({error, {fatal, {bad_level, foo}}}, validate_options(Bad1)),
?_assertThrow({error, {fatal, {bad_lager_exchange_backend_config, {larval, info}}}}, validate_options(Bad2)),
?_assertEqual(true, validate_options(AllGood))
].
-endif.

View File

@ -7,6 +7,9 @@
-module(rabbit).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
%% Transitional step until we can require Erlang/OTP 21 and
%% use the now recommended try/catch syntax for obtaining the stack trace.
-compile(nowarn_deprecated_function).
@ -28,7 +31,8 @@
base_product_version/0,
motd_file/0,
motd/0]).
-export([log_locations/0, config_files/0]). %% for testing and mgmt-agent
%% For CLI, testing and mgmt-agent.
-export([set_log_level/1, log_locations/0, config_files/0]).
-export([is_booted/1, is_booted/0, is_booting/1, is_booting/0]).
%%---------------------------------------------------------------------------
@ -261,7 +265,7 @@
-rabbit_boot_step({networking,
[{description, "TCP and TLS listeners (backwards compatibility)"},
{mfa, {rabbit_log, debug, ["'networking' boot step skipped and moved to end of startup", []]}},
{mfa, {logger, debug, ["'networking' boot step skipped and moved to end of startup", [], #{domain => ?RMQLOG_DOMAIN_GLOBAL}]}},
{requires, notify_cluster}]}).
%%---------------------------------------------------------------------------
@ -335,12 +339,12 @@ run_prelaunch_second_phase() ->
case IsInitialPass of
true ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug(
?LOG_DEBUG(""),
?LOG_DEBUG(
"== Prelaunch phase [2/2] (initial pass) ==");
false ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Prelaunch phase [2/2] =="),
?LOG_DEBUG(""),
?LOG_DEBUG("== Prelaunch phase [2/2] =="),
ok
end,
@ -357,11 +361,11 @@ run_prelaunch_second_phase() ->
ok = rabbit_prelaunch_cluster:setup(Context),
%% Start Mnesia now that everything is ready.
rabbit_log_prelaunch:debug("Starting Mnesia"),
?LOG_DEBUG("Starting Mnesia"),
ok = mnesia:start(),
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Prelaunch DONE =="),
?LOG_DEBUG(""),
?LOG_DEBUG("== Prelaunch DONE =="),
case IsInitialPass of
true -> rabbit_prelaunch:initial_pass_finished();
@ -373,7 +377,8 @@ start_it(StartType) ->
case spawn_boot_marker() of
{ok, Marker} ->
T0 = erlang:timestamp(),
rabbit_log:info("RabbitMQ is asked to start...", []),
?LOG_INFO("RabbitMQ is asked to start...", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
try
{ok, _} = application:ensure_all_started(rabbitmq_prelaunch,
StartType),
@ -382,7 +387,7 @@ start_it(StartType) ->
ok = wait_for_ready_or_stopped(),
T1 = erlang:timestamp(),
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Time to start RabbitMQ: ~p µs",
[timer:now_diff(T1, T0)]),
stop_boot_marker(Marker),
@ -433,11 +438,13 @@ stop() ->
case rabbit_boot_state:get() of
ready ->
Product = product_name(),
rabbit_log:info("~s is asked to stop...", [Product]),
?LOG_INFO("~s is asked to stop...", [Product],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
do_stop(),
rabbit_log:info(
?LOG_INFO(
"Successfully stopped ~s and its dependencies",
[Product]),
[Product],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok;
stopped ->
ok
@ -461,19 +468,22 @@ stop_and_halt() ->
try
stop()
catch Type:Reason ->
rabbit_log:error(
?LOG_ERROR(
"Error trying to stop ~s: ~p:~p",
[product_name(), Type, Reason]),
[product_name(), Type, Reason],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
error({Type, Reason})
after
%% Enclose all the logging in the try block.
%% init:stop() will be called regardless of any errors.
try
AppsLeft = [ A || {A, _, _} <- application:which_applications() ],
rabbit_log:info(
lists:flatten(["Halting Erlang VM with the following applications:~n",
[" ~p~n" || _ <- AppsLeft]]),
AppsLeft),
?LOG_ERROR(
lists:flatten(
["Halting Erlang VM with the following applications:~n",
[" ~p~n" || _ <- AppsLeft]]),
AppsLeft,
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
%% Also duplicate this information to stderr, so console where
%% foreground broker was running (or systemd journal) will
%% contain information about graceful termination.
@ -518,10 +528,12 @@ start_apps(Apps, RestartTypes) ->
stop_apps([]) ->
ok;
stop_apps(Apps) ->
rabbit_log:info(
lists:flatten(["Stopping ~s applications and their dependencies in the following order:~n",
[" ~p~n" || _ <- Apps]]),
[product_name() | lists:reverse(Apps)]),
?LOG_INFO(
lists:flatten(
["Stopping ~s applications and their dependencies in the following order:~n",
[" ~p~n" || _ <- Apps]]),
[product_name() | lists:reverse(Apps)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = app_utils:stop_applications(
Apps, handle_app_error(error_during_shutdown)),
case lists:member(rabbit, Apps) of
@ -785,28 +797,10 @@ environment(App) ->
-spec rotate_logs() -> rabbit_types:ok_or_error(any()).
rotate_logs() ->
rabbit_lager:fold_sinks(
fun
(_, [], Acc) ->
Acc;
(SinkName, FileNames, Acc) ->
lager:log(SinkName, info, self(),
"Log file rotation forced", []),
%% FIXME: We use an internal message, understood by
%% lager_file_backend. We should use a proper API, when
%% it's added to Lager.
%%
%% FIXME: This call is effectively asynchronous: at the
%% end of this function, we can't guaranty the rotation
%% is completed.
[ok = gen_event:call(SinkName,
{lager_file_backend, FileName},
rotate,
infinity) || FileName <- FileNames],
lager:log(SinkName, info, self(),
"Log file re-opened after forced rotation", []),
Acc
end, ok).
?LOG_ERROR(
"Forcing log rotation is currently unsupported",
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
{error, unsupported}.
%%--------------------------------------------------------------------
@ -835,14 +829,18 @@ start(normal, []) ->
#{product_overridden := true,
product_base_name := BaseName,
product_base_version := BaseVersion} ->
rabbit_log:info("~n Starting ~s ~s on Erlang ~s~n Based on ~s ~s~n ~s~n ~s~n",
[product_name(), product_version(), rabbit_misc:otp_release(),
BaseName, BaseVersion,
?COPYRIGHT_MESSAGE, ?INFORMATION_MESSAGE]);
?LOG_INFO(
"~n Starting ~s ~s on Erlang ~s~n Based on ~s ~s~n ~s~n ~s~n",
[product_name(), product_version(), rabbit_misc:otp_release(),
BaseName, BaseVersion,
?COPYRIGHT_MESSAGE, ?INFORMATION_MESSAGE],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH});
_ ->
rabbit_log:info("~n Starting ~s ~s on Erlang ~s~n ~s~n ~s~n",
[product_name(), product_version(), rabbit_misc:otp_release(),
?COPYRIGHT_MESSAGE, ?INFORMATION_MESSAGE])
?LOG_INFO(
"~n Starting ~s ~s on Erlang ~s~n ~s~n ~s~n",
[product_name(), product_version(), rabbit_misc:otp_release(),
?COPYRIGHT_MESSAGE, ?INFORMATION_MESSAGE],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})
end,
log_motd(),
{ok, SupPid} = rabbit_sup:start_link(),
@ -860,7 +858,7 @@ start(normal, []) ->
%%
%% Note that plugins were not taken care of at this point
%% either.
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Register `rabbit` process (~p) for rabbit_node_monitor",
[self()]),
true = register(rabbit, self()),
@ -870,15 +868,15 @@ start(normal, []) ->
warn_if_kernel_config_dubious(),
warn_if_disc_io_options_dubious(),
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Plugins (prelaunch phase) =="),
?LOG_DEBUG(""),
?LOG_DEBUG("== Plugins (prelaunch phase) =="),
rabbit_log_prelaunch:debug("Setting plugins up"),
?LOG_DEBUG("Setting plugins up"),
%% `Plugins` contains all the enabled plugins, plus their
%% dependencies. The order is important: dependencies appear
%% before plugin which depend on them.
Plugins = rabbit_plugins:setup(),
rabbit_log_prelaunch:debug(
?LOG_DEBUG(
"Loading the following plugins: ~p", [Plugins]),
%% We can load all plugins and refresh their feature flags at
%% once, because it does not involve running code from the
@ -887,8 +885,8 @@ start(normal, []) ->
ok = rabbit_feature_flags:refresh_feature_flags_after_app_load(
Plugins),
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Boot steps =="),
?LOG_DEBUG(""),
?LOG_DEBUG("== Boot steps =="),
ok = rabbit_boot_steps:run_boot_steps([rabbit | Plugins]),
run_postlaunch_phase(Plugins),
@ -917,23 +915,22 @@ run_postlaunch_phase(Plugins) ->
do_run_postlaunch_phase(Plugins) ->
%% Once RabbitMQ itself is started, we need to run a few more steps,
%% in particular start plugins.
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Postlaunch phase =="),
?LOG_DEBUG(""),
?LOG_DEBUG("== Postlaunch phase =="),
try
%% Successful boot resets node maintenance state.
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:info("Resetting node maintenance status"),
?LOG_DEBUG(""),
?LOG_INFO("Resetting node maintenance status"),
_ = rabbit_maintenance:unmark_as_being_drained(),
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Plugins (postlaunch phase) =="),
?LOG_DEBUG(""),
?LOG_DEBUG("== Plugins (postlaunch phase) =="),
%% However, we want to run their boot steps and actually start
%% them one by one, to ensure a dependency is fully started
%% before a plugin which depends on it gets a chance to start.
rabbit_log_prelaunch:debug(
"Starting the following plugins: ~p", [Plugins]),
?LOG_DEBUG("Starting the following plugins: ~p", [Plugins]),
lists:foreach(
fun(Plugin) ->
case application:ensure_all_started(Plugin) of
@ -951,18 +948,16 @@ do_run_postlaunch_phase(Plugins) ->
%% Start listeners after all plugins have been enabled,
%% see rabbitmq/rabbitmq-server#2405.
rabbit_log_prelaunch:info(
"Ready to start client connection listeners"),
?LOG_INFO("Ready to start client connection listeners"),
ok = rabbit_networking:boot(),
%% The node is ready: mark it as such and log it.
%% NOTE: PLEASE DO NOT ADD CRITICAL NODE STARTUP CODE AFTER THIS.
ok = rabbit_lager:broker_is_started(),
ActivePlugins = rabbit_plugins:active(),
StrictlyPlugins = rabbit_plugins:strictly_plugins(ActivePlugins),
ok = log_broker_started(StrictlyPlugins),
rabbit_log_prelaunch:debug("Marking ~s as running", [product_name()]),
?LOG_DEBUG("Marking ~s as running", [product_name()]),
rabbit_boot_state:set(ready)
catch
throw:{error, _} = Error ->
@ -1011,7 +1006,7 @@ boot_delegate() ->
recover() ->
ok = rabbit_policy:recover(),
ok = rabbit_vhost:recover(),
ok = lager_exchange_backend:maybe_init_exchange().
ok.
-spec maybe_insert_default_data() -> 'ok'.
@ -1019,10 +1014,12 @@ maybe_insert_default_data() ->
NoDefsToImport = not rabbit_definitions:has_configured_definitions_to_load(),
case rabbit_table:needs_default_data() andalso NoDefsToImport of
true ->
rabbit_log:info("Will seed default virtual host and user..."),
?LOG_INFO("Will seed default virtual host and user...",
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
insert_default_data();
false ->
rabbit_log:info("Will not seed default virtual host and user: have definitions to load..."),
?LOG_INFO("Will not seed default virtual host and user: have definitions to load...",
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
ok
end.
@ -1042,7 +1039,6 @@ insert_default_data() ->
DefaultReadPermBin = rabbit_data_coercion:to_binary(DefaultReadPerm),
ok = rabbit_vhost:add(DefaultVHostBin, <<"Default virtual host">>, [], ?INTERNAL_USER),
ok = lager_exchange_backend:maybe_init_exchange(),
ok = rabbit_auth_backend_internal:add_user(
DefaultUserBin,
DefaultPassBin,
@ -1061,9 +1057,13 @@ insert_default_data() ->
%%---------------------------------------------------------------------------
%% logging
-spec log_locations() -> [rabbit_lager:log_location()].
-spec set_log_level(logger:level()) -> ok.
set_log_level(Level) ->
rabbit_prelaunch_logging:set_log_level(Level).
-spec log_locations() -> [rabbit_prelaunch_logging:log_location()].
log_locations() ->
rabbit_lager:log_locations().
rabbit_prelaunch_logging:log_locations().
-spec config_locations() -> [rabbit_config:config_location()].
config_locations() ->
@ -1094,7 +1094,8 @@ log_broker_started(Plugins) ->
Message = string:strip(rabbit_misc:format(
"Server startup complete; ~b plugins started.~n~s",
[length(Plugins), PluginList]), right, $\n),
rabbit_log:info(Message),
?LOG_INFO(Message,
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
io:format(" completed with ~p plugins.~n", [length(Plugins)]).
-define(RABBIT_TEXT_LOGO,
@ -1185,7 +1186,8 @@ log_motd() ->
_ -> [" ", Line, "\n"]
end
|| Line <- Lines],
rabbit_log:info("~n~ts", [string:trim(Padded, trailing, [$\r, $\n])])
?LOG_INFO("~n~ts", [string:trim(Padded, trailing, [$\r, $\n])],
#{domain => ?RMQLOG_DOMAIN_GLOBAL})
end.
log_banner() ->
@ -1216,7 +1218,8 @@ log_banner() ->
{K, V} ->
Format(K, V)
end || S <- Settings]), right, $\n),
rabbit_log:info("~n~ts", [Banner]).
?LOG_INFO("~n~ts", [Banner],
#{domain => ?RMQLOG_DOMAIN_GLOBAL}).
warn_if_kernel_config_dubious() ->
case os:type() of
@ -1225,16 +1228,18 @@ warn_if_kernel_config_dubious() ->
_ ->
case erlang:system_info(kernel_poll) of
true -> ok;
false -> rabbit_log:warning(
"Kernel poll (epoll, kqueue, etc) is disabled. Throughput "
"and CPU utilization may worsen.~n")
false -> ?LOG_WARNING(
"Kernel poll (epoll, kqueue, etc) is disabled. "
"Throughput and CPU utilization may worsen.~n",
#{domain => ?RMQLOG_DOMAIN_GLOBAL})
end
end,
DirtyIOSchedulers = erlang:system_info(dirty_io_schedulers),
case DirtyIOSchedulers < ?DIRTY_IO_SCHEDULERS_WARNING_THRESHOLD of
true -> rabbit_log:warning(
true -> ?LOG_WARNING(
"Erlang VM is running with ~b dirty I/O schedulers, "
"file I/O performance may worsen~n", [DirtyIOSchedulers]);
"file I/O performance may worsen~n", [DirtyIOSchedulers],
#{domain => ?RMQLOG_DOMAIN_GLOBAL});
false -> ok
end,
IDCOpts = case application:get_env(kernel, inet_default_connect_options) of
@ -1242,8 +1247,9 @@ warn_if_kernel_config_dubious() ->
{ok, Val} -> Val
end,
case proplists:get_value(nodelay, IDCOpts, false) of
false -> rabbit_log:warning("Nagle's algorithm is enabled for sockets, "
"network I/O latency will be higher~n");
false -> ?LOG_WARNING("Nagle's algorithm is enabled for sockets, "
"network I/O latency will be higher~n",
#{domain => ?RMQLOG_DOMAIN_GLOBAL});
true -> ok
end.
@ -1259,7 +1265,8 @@ warn_if_disc_io_options_dubious() ->
CreditDiscBound, IoBatchSize) of
ok -> ok;
{error, {Reason, Vars}} ->
rabbit_log:warning(Reason, Vars)
?LOG_WARNING(Reason, Vars,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})
end.
validate_msg_store_io_batch_size_and_credit_disc_bound(CreditDiscBound,
@ -1498,8 +1505,10 @@ ensure_working_fhc() ->
{ok, true} -> "ON";
{ok, false} -> "OFF"
end,
rabbit_log:info("FHC read buffering: ~s~n", [ReadBuf]),
rabbit_log:info("FHC write buffering: ~s~n", [WriteBuf]),
?LOG_INFO("FHC read buffering: ~s~n", [ReadBuf],
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
?LOG_INFO("FHC write buffering: ~s~n", [WriteBuf],
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
Filename = filename:join(code:lib_dir(kernel, ebin), "kernel.app"),
{ok, Fd} = file_handle_cache:open(Filename, [raw, binary, read], []),
{ok, _} = file_handle_cache:read(Fd, 1),

View File

@ -1,719 +0,0 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2007-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_lager).
-include_lib("rabbit_common/include/rabbit_log.hrl").
%% API
-export([start_logger/0, log_locations/0, fold_sinks/2,
broker_is_started/0, set_log_level/1]).
%% For test purposes
-export([configure_lager/0]).
-export_type([log_location/0]).
-type log_location() :: string().
start_logger() ->
ok = maybe_remove_logger_handler(),
ok = app_utils:stop_applications([lager, syslog]),
ok = ensure_lager_configured(),
ok = app_utils:start_applications([lager]),
fold_sinks(
fun
(_, [], Acc) ->
Acc;
(SinkName, _, Acc) ->
lager:log(SinkName, info, self(),
"Log file opened with Lager", []),
Acc
end, ok),
ensure_log_working().
broker_is_started() ->
{ok, HwmCurrent} = application:get_env(lager, error_logger_hwm),
{ok, HwmOrig0} = application:get_env(lager, error_logger_hwm_original),
HwmOrig = case get_most_verbose_log_level() of
debug -> HwmOrig0 * 100;
_ -> HwmOrig0
end,
case HwmOrig =:= HwmCurrent of
false ->
ok = application:set_env(lager, error_logger_hwm, HwmOrig),
Handlers = gen_event:which_handlers(lager_event),
lists:foreach(fun(Handler) ->
lager:set_loghwm(Handler, HwmOrig)
end, Handlers),
ok;
_ ->
ok
end.
set_log_level(Level) ->
IsValidLevel = lists:member(Level, lager_util:levels()),
set_log_level(IsValidLevel, Level).
set_log_level(true, Level) ->
SinksAndHandlers = [{Sink, gen_event:which_handlers(Sink)} ||
Sink <- lager:list_all_sinks()],
DefaultHwm = application:get_env(lager, error_logger_hwm_original, 50),
Hwm = case Level of
debug -> DefaultHwm * 100;
_ -> DefaultHwm
end,
application:set_env(lager, error_logger_hwm, Hwm),
set_sink_log_level(SinksAndHandlers, Level, Hwm);
set_log_level(_, Level) ->
{error, {invalid_log_level, Level}}.
set_sink_log_level([], _Level, _Hwm) ->
ok;
set_sink_log_level([{Sink, Handlers}|Rest], Level, Hwm) ->
set_sink_handler_log_level(Sink, Handlers, Level, Hwm),
set_sink_log_level(Rest, Level, Hwm).
set_sink_handler_log_level(_Sink, [], _Level, _Hwm) ->
ok;
set_sink_handler_log_level(Sink, [Handler|Rest], Level, Hwm)
when is_atom(Handler) andalso is_integer(Hwm) ->
lager:set_loghwm(Sink, Handler, undefined, Hwm),
ok = lager:set_loglevel(Sink, Handler, undefined, Level),
set_sink_handler_log_level(Sink, Rest, Level, Hwm);
set_sink_handler_log_level(Sink, [{Handler, Id}|Rest], Level, Hwm) ->
lager:set_loghwm(Sink, Handler, Id, Hwm),
ok = lager:set_loglevel(Sink, Handler, Id, Level),
set_sink_handler_log_level(Sink, Rest, Level, Hwm);
set_sink_handler_log_level(Sink, [_|Rest], Level, Hwm) ->
set_sink_handler_log_level(Sink, Rest, Level, Hwm).
log_locations() ->
ensure_lager_configured(),
DefaultHandlers = application:get_env(lager, handlers, []),
Sinks = application:get_env(lager, extra_sinks, []),
ExtraHandlers = [proplists:get_value(handlers, Props, [])
|| {_, Props} <- Sinks],
lists:sort(log_locations1([DefaultHandlers | ExtraHandlers], [])).
log_locations1([Handlers | Rest], Locations) ->
Locations1 = log_locations2(Handlers, Locations),
log_locations1(Rest, Locations1);
log_locations1([], Locations) ->
Locations.
log_locations2([{lager_file_backend, Settings} | Rest], Locations) ->
FileName = lager_file_name1(Settings),
Locations1 = case lists:member(FileName, Locations) of
false -> [FileName | Locations];
true -> Locations
end,
log_locations2(Rest, Locations1);
log_locations2([{lager_console_backend, _} | Rest], Locations) ->
Locations1 = case lists:member("<stdout>", Locations) of
false -> ["<stdout>" | Locations];
true -> Locations
end,
log_locations2(Rest, Locations1);
log_locations2([_ | Rest], Locations) ->
log_locations2(Rest, Locations);
log_locations2([], Locations) ->
Locations.
fold_sinks(Fun, Acc) ->
Handlers = lager_config:global_get(handlers),
Sinks = dict:to_list(lists:foldl(
fun
({{lager_file_backend, F}, _, S}, Dict) ->
dict:append(S, F, Dict);
({_, _, S}, Dict) ->
case dict:is_key(S, Dict) of
true -> dict:store(S, [], Dict);
false -> Dict
end
end,
dict:new(), Handlers)),
fold_sinks(Sinks, Fun, Acc).
fold_sinks([{SinkName, FileNames} | Rest], Fun, Acc) ->
Acc1 = Fun(SinkName, FileNames, Acc),
fold_sinks(Rest, Fun, Acc1);
fold_sinks([], _, Acc) ->
Acc.
ensure_log_working() ->
{ok, Handlers} = application:get_env(lager, handlers),
[ ensure_lager_handler_file_exist(Handler)
|| Handler <- Handlers ],
Sinks = application:get_env(lager, extra_sinks, []),
ensure_extra_sinks_working(Sinks, list_expected_sinks()).
ensure_extra_sinks_working(Sinks, [SinkName | Rest]) ->
case proplists:get_value(SinkName, Sinks) of
undefined -> throw({error, {cannot_log_to_file, unknown,
rabbit_log_lager_event_sink_undefined}});
Sink ->
SinkHandlers = proplists:get_value(handlers, Sink, []),
[ ensure_lager_handler_file_exist(Handler)
|| Handler <- SinkHandlers ]
end,
ensure_extra_sinks_working(Sinks, Rest);
ensure_extra_sinks_working(_Sinks, []) ->
ok.
ensure_lager_handler_file_exist(Handler) ->
case lager_file_name(Handler) of
false -> ok;
FileName -> ensure_logfile_exist(FileName)
end.
lager_file_name({lager_file_backend, Settings}) ->
lager_file_name1(Settings);
lager_file_name(_) ->
false.
lager_file_name1(Settings) when is_list(Settings) ->
{file, FileName} = proplists:lookup(file, Settings),
FileName;
lager_file_name1({FileName, _}) -> FileName;
lager_file_name1({FileName, _, _, _, _}) -> FileName;
lager_file_name1(_) ->
throw({error, {cannot_log_to_file, unknown,
lager_file_backend_config_invalid}}).
ensure_logfile_exist(LogFile) ->
case rabbit_file:read_file_info(LogFile) of
{ok,_} -> ok;
{error, Err} -> throw({error, {cannot_log_to_file, LogFile, Err}})
end.
ensure_lager_configured() ->
case lager_configured() of
false -> configure_lager();
true -> ok
end.
%% Lager should have handlers and sinks
%% Error logger forwarding to syslog should be disabled
lager_configured() ->
Sinks = lager:list_all_sinks(),
ExpectedSinks = list_expected_sinks(),
application:get_env(lager, handlers) =/= undefined
andalso
lists:all(fun(S) -> lists:member(S, Sinks) end, ExpectedSinks)
andalso
application:get_env(syslog, syslog_error_logger) =/= undefined.
configure_lager() ->
ok = app_utils:load_applications([lager]),
%% Turn off reformatting for error_logger messages
case application:get_env(lager, error_logger_redirect) of
undefined -> application:set_env(lager, error_logger_redirect, true);
_ -> ok
end,
case application:get_env(lager, error_logger_format_raw) of
undefined -> application:set_env(lager, error_logger_format_raw, true);
_ -> ok
end,
%% Setting env var to 'undefined' is different from not
%% setting it at all, and lager is sensitive to this
%% difference.
case application:get_env(rabbit, lager_log_root) of
{ok, Value} ->
ok = application:set_env(lager, log_root, Value);
_ ->
ok
end,
case application:get_env(lager, colored) of
undefined ->
UseColor = rabbit_prelaunch_early_logging:use_colored_logging(),
application:set_env(lager, colored, UseColor);
_ ->
ok
end,
%% Set rabbit.log config variable based on environment.
prepare_rabbit_log_config(),
%% Configure syslog library.
ok = configure_syslog_error_logger(),
%% At this point we should have rabbit.log application variable
%% configured to generate RabbitMQ log handlers.
GeneratedHandlers = generate_lager_handlers(),
%% If there are lager handlers configured,
%% both lager and generate RabbitMQ handlers are used.
%% This is because it's hard to decide clear preference rules.
%% RabbitMQ handlers can be set to [] to use only lager handlers.
Handlers = case application:get_env(lager, handlers, undefined) of
undefined -> GeneratedHandlers;
[] -> GeneratedHandlers;
LagerHandlers ->
%% Remove handlers generated in previous starts
FormerRabbitHandlers = application:get_env(lager, rabbit_handlers, []),
GeneratedHandlers ++ remove_rabbit_handlers(LagerHandlers,
FormerRabbitHandlers)
end,
ok = application:set_env(lager, handlers, Handlers),
ok = application:set_env(lager, rabbit_handlers, GeneratedHandlers),
%% Setup extra sink/handlers. If they are not configured, redirect
%% messages to the default sink. To know the list of expected extra
%% sinks, we look at the 'lager_extra_sinks' compilation option.
LogConfig = application:get_env(rabbit, log, []),
LogLevels = application:get_env(rabbit, log_levels, []),
Categories = proplists:get_value(categories, LogConfig, []),
CategoriesConfig0 = case {Categories, LogLevels} of
{[], []} -> [];
{[], LogLevels} ->
io:format("Using deprecated config parameter 'log_levels'. "
"Please update your configuration file according to "
"https://rabbitmq.com/logging.html"),
lists:map(fun({Name, Level}) -> {Name, [{level, Level}]} end,
LogLevels);
{Categories, []} ->
Categories;
{Categories, _} ->
io:format("Using the deprecated config parameter 'rabbit.log_levels' together "
"with a new parameter for log categories."
" 'rabbit.log_levels' will be ignored. Please remove it from the config. More at "
"https://rabbitmq.com/logging.html"),
Categories
end,
LogLevelsFromContext = case rabbit_prelaunch:get_context() of
#{log_levels := LL} -> LL;
_ -> undefined
end,
Fun = fun
(global, _, CC) ->
CC;
(color, _, CC) ->
CC;
(CategoryS, LogLevel, CC) ->
Category = list_to_atom(CategoryS),
CCEntry = proplists:get_value(
Category, CC, []),
CCEntry1 = lists:ukeymerge(
1,
[{level, LogLevel}],
lists:ukeysort(1, CCEntry)),
lists:keystore(
Category, 1, CC, {Category, CCEntry1})
end,
CategoriesConfig = case LogLevelsFromContext of
undefined ->
CategoriesConfig0;
_ ->
maps:fold(Fun,
CategoriesConfig0,
LogLevelsFromContext)
end,
SinkConfigs = lists:map(
fun({Name, Config}) ->
{rabbit_log:make_internal_sink_name(Name), Config}
end,
CategoriesConfig),
LagerSinks = application:get_env(lager, extra_sinks, []),
GeneratedSinks = generate_lager_sinks(
[error_logger_lager_event | list_expected_sinks()],
SinkConfigs),
Sinks = merge_lager_sink_handlers(LagerSinks, GeneratedSinks, []),
ok = application:set_env(lager, extra_sinks, Sinks),
case application:get_env(lager, error_logger_hwm) of
undefined ->
ok = application:set_env(lager, error_logger_hwm, 1000),
% NB: 50 is the default value in lager.app.src
ok = application:set_env(lager, error_logger_hwm_original, 50);
{ok, Val} when is_integer(Val) andalso Val < 1000 ->
ok = application:set_env(lager, error_logger_hwm, 1000),
ok = application:set_env(lager, error_logger_hwm_original, Val);
{ok, Val} when is_integer(Val) ->
ok = application:set_env(lager, error_logger_hwm_original, Val),
ok
end,
ok.
configure_syslog_error_logger() ->
%% Disable error_logger forwarding to syslog if it's not configured
case application:get_env(syslog, syslog_error_logger) of
undefined ->
application:set_env(syslog, syslog_error_logger, false);
_ -> ok
end.
remove_rabbit_handlers(Handlers, FormerHandlers) ->
lists:filter(fun(Handler) ->
not lists:member(Handler, FormerHandlers)
end,
Handlers).
generate_lager_handlers() ->
LogConfig = application:get_env(rabbit, log, []),
LogHandlersConfig = lists:keydelete(categories, 1, LogConfig),
generate_lager_handlers(LogHandlersConfig).
generate_lager_handlers(LogHandlersConfig) ->
lists:flatmap(
fun
({file, HandlerConfig}) ->
case proplists:get_value(file, HandlerConfig, false) of
false -> [];
FileName when is_list(FileName) ->
Backend = lager_backend(file),
generate_handler(Backend, HandlerConfig)
end;
({Other, HandlerConfig}) when
Other =:= console; Other =:= syslog; Other =:= exchange ->
case proplists:get_value(enabled, HandlerConfig, false) of
false -> [];
true ->
Backend = lager_backend(Other),
generate_handler(Backend,
lists:keydelete(enabled, 1, HandlerConfig))
end
end,
LogHandlersConfig).
lager_backend(file) -> lager_file_backend;
lager_backend(console) -> lager_console_backend;
lager_backend(syslog) -> syslog_lager_backend;
lager_backend(exchange) -> lager_exchange_backend.
%% Syslog backend is using an old API for configuration and
%% does not support proplists.
generate_handler(syslog_lager_backend=Backend, HandlerConfig) ->
%% The default log level is set to `debug` because the actual
%% filtering is made at the sink level. We want to accept all
%% messages here.
DefaultConfigVal = debug,
Level = proplists:get_value(level, HandlerConfig, DefaultConfigVal),
ok = configure_handler_backend(Backend),
[{Backend,
[Level,
{},
{lager_default_formatter, syslog_formatter_config()}]}];
generate_handler(Backend, HandlerConfig) ->
[{Backend,
lists:ukeymerge(1, lists:ukeysort(1, HandlerConfig),
lists:ukeysort(1, default_handler_config(Backend)))}].
configure_handler_backend(syslog_lager_backend) ->
{ok, _} = application:ensure_all_started(syslog),
ok;
configure_handler_backend(_Backend) ->
ok.
default_handler_config(lager_console_backend) ->
%% The default log level is set to `debug` because the actual
%% filtering is made at the sink level. We want to accept all
%% messages here.
DefaultConfigVal = debug,
[{level, DefaultConfigVal},
{formatter_config, default_config_value({formatter_config, console})}];
default_handler_config(lager_exchange_backend) ->
%% The default log level is set to `debug` because the actual
%% filtering is made at the sink level. We want to accept all
%% messages here.
DefaultConfigVal = debug,
[{level, DefaultConfigVal},
{formatter_config, default_config_value({formatter_config, exchange})}];
default_handler_config(lager_file_backend) ->
%% The default log level is set to `debug` because the actual
%% filtering is made at the sink level. We want to accept all
%% messages here.
DefaultConfigVal = debug,
[{level, DefaultConfigVal},
{formatter_config, default_config_value({formatter_config, file})},
{date, ""},
{size, 0}].
default_config_value(level) ->
LogConfig = application:get_env(rabbit, log, []),
FoldFun = fun
({_, Cfg}, LL) when is_list(Cfg) ->
NewLL = proplists:get_value(level, Cfg, LL),
case LL of
undefined ->
NewLL;
_ ->
MoreVerbose = lager_util:level_to_num(NewLL) > lager_util:level_to_num(LL),
case MoreVerbose of
true -> NewLL;
false -> LL
end
end;
(_, LL) ->
LL
end,
FoundLL = lists:foldl(FoldFun, undefined, LogConfig),
case FoundLL of
undefined -> info;
_ -> FoundLL
end;
default_config_value({formatter_config, console}) ->
EOL = case application:get_env(lager, colored) of
{ok, true} -> "\e[0m\r\n";
_ -> "\r\n"
end,
[date, " ", time, " ", color, "[", severity, "] ",
{pid, ""},
" ", message, EOL];
default_config_value({formatter_config, _}) ->
[date, " ", time, " ", color, "[", severity, "] ",
{pid, ""},
" ", message, "\n"].
syslog_formatter_config() ->
[color, "[", severity, "] ",
{pid, ""},
" ", message, "\n"].
prepare_rabbit_log_config() ->
%% If RABBIT_LOGS is not set, we should ignore it.
DefaultFile = application:get_env(rabbit, lager_default_file, undefined),
%% If RABBIT_UPGRADE_LOGS is not set, we should ignore it.
UpgradeFile = application:get_env(rabbit, lager_upgrade_file, undefined),
case DefaultFile of
undefined -> ok;
false ->
set_env_default_log_disabled();
tty ->
set_env_default_log_console();
FileName when is_list(FileName) ->
case rabbit_prelaunch:get_context() of
%% The user explicitly sets $RABBITMQ_LOGS;
%% we should override a file location even
%% if it's set in rabbitmq.config
#{var_origins := #{main_log_file := environment}} ->
set_env_default_log_file(FileName, override);
_ ->
set_env_default_log_file(FileName, keep)
end
end,
%% Upgrade log file never overrides the value set in rabbitmq.config
case UpgradeFile of
%% No special env for upgrade logs - redirect to the default sink
undefined -> ok;
%% Redirect logs to default output.
DefaultFile -> ok;
UpgradeFileName when is_list(UpgradeFileName) ->
set_env_upgrade_log_file(UpgradeFileName)
end.
set_env_default_log_disabled() ->
%% Disabling all the logs.
ok = application:set_env(rabbit, log, []).
set_env_default_log_console() ->
LogConfig = application:get_env(rabbit, log, []),
ConsoleConfig = proplists:get_value(console, LogConfig, []),
LogConfigConsole =
lists:keystore(console, 1, LogConfig,
{console, lists:keystore(enabled, 1, ConsoleConfig,
{enabled, true})}),
%% Remove the file handler - disable logging to file
LogConfigConsoleNoFile = lists:keydelete(file, 1, LogConfigConsole),
ok = application:set_env(rabbit, log, LogConfigConsoleNoFile).
set_env_default_log_file(FileName, Override) ->
LogConfig = application:get_env(rabbit, log, []),
FileConfig = proplists:get_value(file, LogConfig, []),
NewLogConfig = case proplists:get_value(file, FileConfig, undefined) of
undefined ->
lists:keystore(file, 1, LogConfig,
{file, lists:keystore(file, 1, FileConfig,
{file, FileName})});
_ConfiguredFileName ->
case Override of
override ->
lists:keystore(
file, 1, LogConfig,
{file, lists:keystore(file, 1, FileConfig,
{file, FileName})});
keep ->
LogConfig
end
end,
ok = application:set_env(rabbit, log, NewLogConfig).
set_env_upgrade_log_file(FileName) ->
LogConfig = application:get_env(rabbit, log, []),
SinksConfig = proplists:get_value(categories, LogConfig, []),
UpgradeSinkConfig = proplists:get_value(upgrade, SinksConfig, []),
FileConfig = proplists:get_value(file, SinksConfig, []),
NewLogConfig = case proplists:get_value(file, FileConfig, undefined) of
undefined ->
lists:keystore(
categories, 1, LogConfig,
{categories,
lists:keystore(
upgrade, 1, SinksConfig,
{upgrade,
lists:keystore(file, 1, UpgradeSinkConfig,
{file, FileName})})});
%% No cahnge. We don't want to override the configured value.
_File -> LogConfig
end,
ok = application:set_env(rabbit, log, NewLogConfig).
generate_lager_sinks(SinkNames, SinkConfigs) ->
LogLevels = case rabbit_prelaunch:get_context() of
#{log_levels := LL} -> LL;
_ -> undefined
end,
DefaultLogLevel = case LogLevels of
#{global := LogLevel} ->
LogLevel;
_ ->
default_config_value(level)
end,
lists:map(fun(SinkName) ->
SinkConfig = proplists:get_value(SinkName, SinkConfigs, []),
SinkHandlers = case proplists:get_value(file, SinkConfig, false) of
%% If no file defined - forward everything to the default backend
false ->
ForwarderLevel = proplists:get_value(level,
SinkConfig,
DefaultLogLevel),
[{lager_forwarder_backend,
[lager_util:make_internal_sink_name(lager), ForwarderLevel]}];
%% If a file defined - add a file backend to handlers and remove all default file backends.
File ->
%% Use `debug` as a default handler to not override a handler level
Level = proplists:get_value(level, SinkConfig, DefaultLogLevel),
DefaultGeneratedHandlers = application:get_env(lager, rabbit_handlers, []),
SinkFileHandlers = case proplists:get_value(lager_file_backend, DefaultGeneratedHandlers, undefined) of
undefined ->
%% Create a new file handler.
%% `info` is a default level here.
FileLevel = proplists:get_value(level, SinkConfig, DefaultLogLevel),
generate_lager_handlers([{file, [{file, File}, {level, FileLevel}]}]);
FileHandler ->
%% Replace a filename in the handler
FileHandlerChanges = case handler_level_more_verbose(FileHandler, Level) of
true -> [{file, File}, {level, Level}];
false -> [{file, File}]
end,
[{lager_file_backend,
lists:ukeymerge(1, FileHandlerChanges,
lists:ukeysort(1, FileHandler))}]
end,
%% Remove all file handlers.
AllLagerHandlers = application:get_env(lager, handlers, []),
HandlersWithoutFile = lists:filter(
fun({lager_file_backend, _}) -> false;
({_, _}) -> true
end,
AllLagerHandlers),
%% Set level for handlers which are more verbose.
%% We don't increase verbosity in sinks so it works like forwarder backend.
HandlersWithoutFileWithLevel = lists:map(fun({Name, Handler}) ->
case handler_level_more_verbose(Handler, Level) of
true -> {Name, lists:keystore(level, 1, Handler, {level, Level})};
false -> {Name, Handler}
end
end,
HandlersWithoutFile),
HandlersWithoutFileWithLevel ++ SinkFileHandlers
end,
{SinkName, [{handlers, SinkHandlers}, {rabbit_handlers, SinkHandlers}]}
end,
SinkNames).
handler_level_more_verbose(Handler, Level) ->
HandlerLevel = proplists:get_value(level, Handler, default_config_value(level)),
lager_util:level_to_num(HandlerLevel) > lager_util:level_to_num(Level).
merge_lager_sink_handlers([{Name, Sink} | RestSinks], GeneratedSinks, Agg) ->
%% rabbitmq/rabbitmq-server#2044.
%% We have to take into account that a sink's
%% handler backend may need additional configuration here.
%% {rabbit_log_federation_lager_event, [
%% {handlers, [
%% {lager_forwarder_backend, [lager_event,inherit]},
%% {syslog_lager_backend, [debug]}
%% ]},
%% {rabbit_handlers, [
%% {lager_forwarder_backend, [lager_event,inherit]}
%% ]}
%% ]}
case lists:keytake(Name, 1, GeneratedSinks) of
{value, {Name, GenSink}, RestGeneratedSinks} ->
Handlers = proplists:get_value(handlers, Sink, []),
GenHandlers = proplists:get_value(handlers, GenSink, []),
FormerRabbitHandlers = proplists:get_value(rabbit_handlers, Sink, []),
%% Remove handlers defined in previous starts
ConfiguredHandlers = remove_rabbit_handlers(Handlers, FormerRabbitHandlers),
NewHandlers = GenHandlers ++ ConfiguredHandlers,
ok = maybe_configure_handler_backends(NewHandlers),
MergedSink = lists:keystore(rabbit_handlers, 1,
lists:keystore(handlers, 1, Sink,
{handlers, NewHandlers}),
{rabbit_handlers, GenHandlers}),
merge_lager_sink_handlers(
RestSinks,
RestGeneratedSinks,
[{Name, MergedSink} | Agg]);
false ->
merge_lager_sink_handlers(
RestSinks,
GeneratedSinks,
[{Name, Sink} | Agg])
end;
merge_lager_sink_handlers([], GeneratedSinks, Agg) -> GeneratedSinks ++ Agg.
maybe_configure_handler_backends([]) ->
ok;
maybe_configure_handler_backends([{Backend, _}|Backends]) ->
ok = configure_handler_backend(Backend),
maybe_configure_handler_backends(Backends).
list_expected_sinks() ->
rabbit_prelaunch_early_logging:list_expected_sinks().
maybe_remove_logger_handler() ->
M = logger,
F = remove_handler,
try
ok = erlang:apply(M, F, [default])
catch
error:undef ->
% OK since the logger module only exists in OTP 21.1 or later
ok;
error:{badmatch, {error, {not_found, default}}} ->
% OK - this error happens when running a CLI command
ok;
Err:Reason ->
error_logger:error_msg("calling ~p:~p failed: ~p:~p~n",
[M, F, Err, Reason])
end.
get_most_verbose_log_level() ->
{ok, HandlersA} = application:get_env(lager, handlers),
{ok, ExtraSinks} = application:get_env(lager, extra_sinks),
HandlersB = lists:append(
[H || {_, Keys} <- ExtraSinks,
{handlers, H} <- Keys]),
get_most_verbose_log_level(HandlersA ++ HandlersB,
lager_util:level_to_num(none)).
get_most_verbose_log_level([{_, Props} | Rest], MostVerbose) ->
LogLevel = proplists:get_value(level, Props, info),
LogLevelNum = lager_util:level_to_num(LogLevel),
case LogLevelNum > MostVerbose of
true ->
get_most_verbose_log_level(Rest, LogLevelNum);
false ->
get_most_verbose_log_level(Rest, MostVerbose)
end;
get_most_verbose_log_level([], MostVerbose) ->
lager_util:num_to_level(MostVerbose).

120
deps/rabbit/src/rabbit_log_channel.erl vendored Normal file
View File

@ -0,0 +1,120 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_channel).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include_lib("rabbit_common/include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
-spec debug(string()) -> 'ok'.
debug(Format) -> debug(Format, []).
-spec debug(string(), [any()]) -> 'ok'.
debug(Format, Args) -> debug(self(), Format, Args).
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec info(string()) -> 'ok'.
info(Format) -> info(Format, []).
-spec info(string(), [any()]) -> 'ok'.
info(Format, Args) -> info(self(), Format, Args).
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec notice(string()) -> 'ok'.
notice(Format) -> notice(Format, []).
-spec notice(string(), [any()]) -> 'ok'.
notice(Format, Args) -> notice(self(), Format, Args).
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec warning(string()) -> 'ok'.
warning(Format) -> warning(Format, []).
-spec warning(string(), [any()]) -> 'ok'.
warning(Format, Args) -> warning(self(), Format, Args).
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec error(string()) -> 'ok'.
error(Format) -> error(Format, []).
-spec error(string(), [any()]) -> 'ok'.
error(Format, Args) -> error(self(), Format, Args).
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec critical(string()) -> 'ok'.
critical(Format) -> critical(Format, []).
-spec critical(string(), [any()]) -> 'ok'.
critical(Format, Args) -> critical(self(), Format, Args).
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec alert(string()) -> 'ok'.
alert(Format) -> alert(Format, []).
-spec alert(string(), [any()]) -> 'ok'.
alert(Format, Args) -> alert(self(), Format, Args).
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec emergency(string()) -> 'ok'.
emergency(Format) -> emergency(Format, []).
-spec emergency(string(), [any()]) -> 'ok'.
emergency(Format, Args) -> emergency(self(), Format, Args).
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CHAN}).
-spec none(string()) -> 'ok'.
none(_Format) -> ok.
-spec none(string(), [any()]) -> 'ok'.
none(_Format, _Args) -> ok.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
none(_Pid, _Format, _Args) -> ok.

View File

@ -0,0 +1,120 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_connection).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include_lib("rabbit_common/include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
-spec debug(string()) -> 'ok'.
debug(Format) -> debug(Format, []).
-spec debug(string(), [any()]) -> 'ok'.
debug(Format, Args) -> debug(self(), Format, Args).
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec info(string()) -> 'ok'.
info(Format) -> info(Format, []).
-spec info(string(), [any()]) -> 'ok'.
info(Format, Args) -> info(self(), Format, Args).
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec notice(string()) -> 'ok'.
notice(Format) -> notice(Format, []).
-spec notice(string(), [any()]) -> 'ok'.
notice(Format, Args) -> notice(self(), Format, Args).
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec warning(string()) -> 'ok'.
warning(Format) -> warning(Format, []).
-spec warning(string(), [any()]) -> 'ok'.
warning(Format, Args) -> warning(self(), Format, Args).
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec error(string()) -> 'ok'.
error(Format) -> error(Format, []).
-spec error(string(), [any()]) -> 'ok'.
error(Format, Args) -> error(self(), Format, Args).
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec critical(string()) -> 'ok'.
critical(Format) -> critical(Format, []).
-spec critical(string(), [any()]) -> 'ok'.
critical(Format, Args) -> critical(self(), Format, Args).
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec alert(string()) -> 'ok'.
alert(Format) -> alert(Format, []).
-spec alert(string(), [any()]) -> 'ok'.
alert(Format, Args) -> alert(self(), Format, Args).
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec emergency(string()) -> 'ok'.
emergency(Format) -> emergency(Format, []).
-spec emergency(string(), [any()]) -> 'ok'.
emergency(Format, Args) -> emergency(self(), Format, Args).
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_CONN}).
-spec none(string()) -> 'ok'.
none(_Format) -> ok.
-spec none(string(), [any()]) -> 'ok'.
none(_Format, _Args) -> ok.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
none(_Pid, _Format, _Args) -> ok.

View File

@ -0,0 +1,120 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_feature_flags).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include_lib("rabbit_common/include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
-spec debug(string()) -> 'ok'.
debug(Format) -> debug(Format, []).
-spec debug(string(), [any()]) -> 'ok'.
debug(Format, Args) -> debug(self(), Format, Args).
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec info(string()) -> 'ok'.
info(Format) -> info(Format, []).
-spec info(string(), [any()]) -> 'ok'.
info(Format, Args) -> info(self(), Format, Args).
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec notice(string()) -> 'ok'.
notice(Format) -> notice(Format, []).
-spec notice(string(), [any()]) -> 'ok'.
notice(Format, Args) -> notice(self(), Format, Args).
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec warning(string()) -> 'ok'.
warning(Format) -> warning(Format, []).
-spec warning(string(), [any()]) -> 'ok'.
warning(Format, Args) -> warning(self(), Format, Args).
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec error(string()) -> 'ok'.
error(Format) -> error(Format, []).
-spec error(string(), [any()]) -> 'ok'.
error(Format, Args) -> error(self(), Format, Args).
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec critical(string()) -> 'ok'.
critical(Format) -> critical(Format, []).
-spec critical(string(), [any()]) -> 'ok'.
critical(Format, Args) -> critical(self(), Format, Args).
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec alert(string()) -> 'ok'.
alert(Format) -> alert(Format, []).
-spec alert(string(), [any()]) -> 'ok'.
alert(Format, Args) -> alert(self(), Format, Args).
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec emergency(string()) -> 'ok'.
emergency(Format) -> emergency(Format, []).
-spec emergency(string(), [any()]) -> 'ok'.
emergency(Format, Args) -> emergency(self(), Format, Args).
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEAT_FLAGS}).
-spec none(string()) -> 'ok'.
none(_Format) -> ok.
-spec none(string(), [any()]) -> 'ok'.
none(_Format, _Args) -> ok.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
none(_Pid, _Format, _Args) -> ok.

122
deps/rabbit/src/rabbit_log_mirroring.erl vendored Normal file
View File

@ -0,0 +1,122 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_mirroring).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include_lib("rabbit_common/include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
%%----------------------------------------------------------------------------
-spec debug(string()) -> 'ok'.
debug(Format) -> debug(Format, []).
-spec debug(string(), [any()]) -> 'ok'.
debug(Format, Args) -> debug(self(), Format, Args).
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec info(string()) -> 'ok'.
info(Format) -> info(Format, []).
-spec info(string(), [any()]) -> 'ok'.
info(Format, Args) -> info(self(), Format, Args).
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec notice(string()) -> 'ok'.
notice(Format) -> notice(Format, []).
-spec notice(string(), [any()]) -> 'ok'.
notice(Format, Args) -> notice(self(), Format, Args).
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec warning(string()) -> 'ok'.
warning(Format) -> warning(Format, []).
-spec warning(string(), [any()]) -> 'ok'.
warning(Format, Args) -> warning(self(), Format, Args).
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec error(string()) -> 'ok'.
error(Format) -> error(Format, []).
-spec error(string(), [any()]) -> 'ok'.
error(Format, Args) -> error(self(), Format, Args).
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec critical(string()) -> 'ok'.
critical(Format) -> critical(Format, []).
-spec critical(string(), [any()]) -> 'ok'.
critical(Format, Args) -> critical(self(), Format, Args).
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec alert(string()) -> 'ok'.
alert(Format) -> alert(Format, []).
-spec alert(string(), [any()]) -> 'ok'.
alert(Format, Args) -> alert(self(), Format, Args).
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec emergency(string()) -> 'ok'.
emergency(Format) -> emergency(Format, []).
-spec emergency(string(), [any()]) -> 'ok'.
emergency(Format, Args) -> emergency(self(), Format, Args).
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_MIRRORING}).
-spec none(string()) -> 'ok'.
none(_Format) -> ok.
-spec none(string(), [any()]) -> 'ok'.
none(_Format, _Args) -> ok.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
none(_Pid, _Format, _Args) -> ok.

120
deps/rabbit/src/rabbit_log_prelaunch.erl vendored Normal file
View File

@ -0,0 +1,120 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_prelaunch).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include_lib("rabbit_common/include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
-spec debug(string()) -> 'ok'.
debug(Format) -> debug(Format, []).
-spec debug(string(), [any()]) -> 'ok'.
debug(Format, Args) -> debug(self(), Format, Args).
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec info(string()) -> 'ok'.
info(Format) -> info(Format, []).
-spec info(string(), [any()]) -> 'ok'.
info(Format, Args) -> info(self(), Format, Args).
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec notice(string()) -> 'ok'.
notice(Format) -> notice(Format, []).
-spec notice(string(), [any()]) -> 'ok'.
notice(Format, Args) -> notice(self(), Format, Args).
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec warning(string()) -> 'ok'.
warning(Format) -> warning(Format, []).
-spec warning(string(), [any()]) -> 'ok'.
warning(Format, Args) -> warning(self(), Format, Args).
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec error(string()) -> 'ok'.
error(Format) -> error(Format, []).
-spec error(string(), [any()]) -> 'ok'.
error(Format, Args) -> error(self(), Format, Args).
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec critical(string()) -> 'ok'.
critical(Format) -> critical(Format, []).
-spec critical(string(), [any()]) -> 'ok'.
critical(Format, Args) -> critical(self(), Format, Args).
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec alert(string()) -> 'ok'.
alert(Format) -> alert(Format, []).
-spec alert(string(), [any()]) -> 'ok'.
alert(Format, Args) -> alert(self(), Format, Args).
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec emergency(string()) -> 'ok'.
emergency(Format) -> emergency(Format, []).
-spec emergency(string(), [any()]) -> 'ok'.
emergency(Format, Args) -> emergency(self(), Format, Args).
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_PRELAUNCH}).
-spec none(string()) -> 'ok'.
none(_Format) -> ok.
-spec none(string(), [any()]) -> 'ok'.
none(_Format, _Args) -> ok.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
none(_Pid, _Format, _Args) -> ok.

120
deps/rabbit/src/rabbit_log_queue.erl vendored Normal file
View File

@ -0,0 +1,120 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_queue).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include_lib("rabbit_common/include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
-spec debug(string()) -> 'ok'.
debug(Format) -> debug(Format, []).
-spec debug(string(), [any()]) -> 'ok'.
debug(Format, Args) -> debug(self(), Format, Args).
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec info(string()) -> 'ok'.
info(Format) -> info(Format, []).
-spec info(string(), [any()]) -> 'ok'.
info(Format, Args) -> info(self(), Format, Args).
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec notice(string()) -> 'ok'.
notice(Format) -> notice(Format, []).
-spec notice(string(), [any()]) -> 'ok'.
notice(Format, Args) -> notice(self(), Format, Args).
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec warning(string()) -> 'ok'.
warning(Format) -> warning(Format, []).
-spec warning(string(), [any()]) -> 'ok'.
warning(Format, Args) -> warning(self(), Format, Args).
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec error(string()) -> 'ok'.
error(Format) -> error(Format, []).
-spec error(string(), [any()]) -> 'ok'.
error(Format, Args) -> error(self(), Format, Args).
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec critical(string()) -> 'ok'.
critical(Format) -> critical(Format, []).
-spec critical(string(), [any()]) -> 'ok'.
critical(Format, Args) -> critical(self(), Format, Args).
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec alert(string()) -> 'ok'.
alert(Format) -> alert(Format, []).
-spec alert(string(), [any()]) -> 'ok'.
alert(Format, Args) -> alert(self(), Format, Args).
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec emergency(string()) -> 'ok'.
emergency(Format) -> emergency(Format, []).
-spec emergency(string(), [any()]) -> 'ok'.
emergency(Format, Args) -> emergency(self(), Format, Args).
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_QUEUE}).
-spec none(string()) -> 'ok'.
none(_Format) -> ok.
-spec none(string(), [any()]) -> 'ok'.
none(_Format, _Args) -> ok.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
none(_Pid, _Format, _Args) -> ok.

122
deps/rabbit/src/rabbit_log_upgrade.erl vendored Normal file
View File

@ -0,0 +1,122 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_upgrade).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include_lib("rabbit_common/include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
%%----------------------------------------------------------------------------
-spec debug(string()) -> 'ok'.
debug(Format) -> debug(Format, []).
-spec debug(string(), [any()]) -> 'ok'.
debug(Format, Args) -> debug(self(), Format, Args).
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec info(string()) -> 'ok'.
info(Format) -> info(Format, []).
-spec info(string(), [any()]) -> 'ok'.
info(Format, Args) -> info(self(), Format, Args).
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec notice(string()) -> 'ok'.
notice(Format) -> notice(Format, []).
-spec notice(string(), [any()]) -> 'ok'.
notice(Format, Args) -> notice(self(), Format, Args).
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec warning(string()) -> 'ok'.
warning(Format) -> warning(Format, []).
-spec warning(string(), [any()]) -> 'ok'.
warning(Format, Args) -> warning(self(), Format, Args).
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec error(string()) -> 'ok'.
error(Format) -> error(Format, []).
-spec error(string(), [any()]) -> 'ok'.
error(Format, Args) -> error(self(), Format, Args).
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec critical(string()) -> 'ok'.
critical(Format) -> critical(Format, []).
-spec critical(string(), [any()]) -> 'ok'.
critical(Format, Args) -> critical(self(), Format, Args).
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec alert(string()) -> 'ok'.
alert(Format) -> alert(Format, []).
-spec alert(string(), [any()]) -> 'ok'.
alert(Format, Args) -> alert(self(), Format, Args).
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec emergency(string()) -> 'ok'.
emergency(Format) -> emergency(Format, []).
-spec emergency(string(), [any()]) -> 'ok'.
emergency(Format, Args) -> emergency(self(), Format, Args).
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_UPGRADE}).
-spec none(string()) -> 'ok'.
none(_Format) -> ok.
-spec none(string(), [any()]) -> 'ok'.
none(_Format, _Args) -> ok.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
none(_Pid, _Format, _Args) -> ok.

View File

@ -0,0 +1,176 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_logger_exchange_h).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/rabbit_framing.hrl").
-include_lib("rabbit_common/include/logging.hrl").
%% logger callbacks
-export([log/2, adding_handler/1, removing_handler/1, changing_config/3,
filter_config/1]).
-define(DECL_EXCHANGE_INTERVAL_SECS, 5).
-define(LOG_EXCH_NAME, <<"amq.rabbitmq.log">>).
-define(DEFAULT_FORMATTER, logger_formatter).
-define(DEFAULT_FORMATTER_CONFIG, #{}).
%% -------------------------------------------------------------------
%% Logger handler callbacks.
%% -------------------------------------------------------------------
adding_handler(Config) ->
Config1 = start_setup_proc(Config),
{ok, Config1}.
changing_config(_SetOrUpdate, OldConfig, _NewConfig) ->
{ok, OldConfig}.
filter_config(Config) ->
Config.
log(#{meta := #{mfa := {?MODULE, _, _}}}, _) ->
ok;
log(LogEvent, Config) ->
case rabbit_boot_state:get() of
ready -> do_log(LogEvent, Config);
_ -> ok
end.
do_log(LogEvent, #{config := #{exchange := Exchange}} = Config) ->
RoutingKey = make_routing_key(LogEvent, Config),
AmqpMsg = log_event_to_amqp_msg(LogEvent, Config),
Body = try_format_body(LogEvent, Config),
case rabbit_basic:publish(Exchange, RoutingKey, AmqpMsg, Body) of
ok -> ok;
{error, not_found} -> ok
end.
removing_handler(Config) ->
unconfigure_exchange(Config),
ok.
%% -------------------------------------------------------------------
%% Internal functions.
%% -------------------------------------------------------------------
log_event_to_amqp_msg(LogEvent, Config) ->
ContentType = guess_content_type(Config),
Timestamp = make_timestamp(LogEvent, Config),
Headers = make_headers(LogEvent, Config),
#'P_basic'{
content_type = ContentType,
timestamp = Timestamp,
headers = Headers
}.
make_routing_key(#{level := Level}, _) ->
rabbit_data_coercion:to_binary(Level).
guess_content_type(#{formatter := {rabbit_logger_json_fmt, _}}) ->
<<"application/json">>;
guess_content_type(_) ->
<<"text/plain">>.
make_timestamp(#{meta := #{time := Timestamp}}, _) ->
erlang:convert_time_unit(Timestamp, microsecond, second);
make_timestamp(_, _) ->
os:system_time(second).
make_headers(_, _) ->
Node = rabbit_data_coercion:to_binary(node()),
[{<<"node">>, longstr, Node}].
try_format_body(LogEvent, #{formatter := {Formatter, FormatterConfig}}) ->
Formatted = try_format_body(LogEvent, Formatter, FormatterConfig),
erlang:iolist_to_binary(Formatted).
try_format_body(LogEvent, Formatter, FormatterConfig) ->
try
Formatter:format(LogEvent, FormatterConfig)
catch
C:R:S ->
case {?DEFAULT_FORMATTER, ?DEFAULT_FORMATTER_CONFIG} of
{Formatter, FormatterConfig} ->
"DEFAULT FORMATTER CRASHED\n";
{DefaultFormatter, DefaultFormatterConfig} ->
Msg = {"FORMATTER CRASH: ~tp -- ~p:~p:~p",
[maps:get(msg, LogEvent), C, R, S]},
LogEvent1 = LogEvent#{msg => Msg},
try_format_body(
LogEvent1,
DefaultFormatter,
DefaultFormatterConfig)
end
end.
start_setup_proc(#{config := InternalConfig} = Config) ->
{ok, DefaultVHost} = application:get_env(rabbit, default_vhost),
Exchange = rabbit_misc:r(DefaultVHost, exchange, ?LOG_EXCH_NAME),
InternalConfig1 = InternalConfig#{exchange => Exchange},
Pid = spawn(fun() -> setup_proc(Config#{config => InternalConfig1}) end),
InternalConfig2 = InternalConfig1#{setup_proc => Pid},
Config#{config => InternalConfig2}.
setup_proc(
#{config := #{exchange := #resource{name = Name,
virtual_host = VHost}}} = Config) ->
case declare_exchange(Config) of
ok ->
?LOG_INFO(
"Logging to exchange '~s' in vhost '~s' ready", [Name, VHost],
#{domain => ?RMQLOG_DOMAIN_GLOBAL});
error ->
?LOG_DEBUG(
"Logging to exchange '~s' in vhost '~s' not ready, "
"trying again in ~b second(s)",
[Name, VHost, ?DECL_EXCHANGE_INTERVAL_SECS],
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
receive
stop -> ok
after ?DECL_EXCHANGE_INTERVAL_SECS * 1000 ->
setup_proc(Config)
end
end.
declare_exchange(
#{config := #{exchange := #resource{name = Name,
virtual_host = VHost} = Exchange}}) ->
try
%% Durable.
#exchange{} = rabbit_exchange:declare(
Exchange, topic, true, false, true, [],
?INTERNAL_USER),
?LOG_DEBUG(
"Declared exchange '~s' in vhost '~s'",
[Name, VHost],
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
ok
catch
Class:Reason ->
?LOG_DEBUG(
"Could not declare exchange '~s' in vhost '~s', "
"reason: ~0p:~0p",
[Name, VHost, Class, Reason],
#{domain => ?RMQLOG_DOMAIN_GLOBAL}),
error
end.
unconfigure_exchange(
#{config := #{exchange := #resource{name = Name,
virtual_host = VHost} = Exchange,
setup_proc := Pid}}) ->
Pid ! stop,
rabbit_exchange:delete(Exchange, false, ?INTERNAL_USER),
?LOG_INFO(
"Logging to exchange '~s' in vhost '~s' disabled",
[Name, VHost],
#{domain => ?RMQLOG_DOMAIN_GLOBAL}).

View File

@ -1,15 +1,24 @@
-module(rabbit_prelaunch_cluster).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([setup/1]).
setup(Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Clustering =="),
rabbit_log_prelaunch:debug("Preparing cluster status files"),
?LOG_DEBUG(
"~n== Clustering ==", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
?LOG_DEBUG(
"Preparing cluster status files", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
rabbit_node_monitor:prepare_cluster_status_files(),
case Context of
#{initial_pass := true} ->
rabbit_log_prelaunch:debug("Upgrading Mnesia schema"),
?LOG_DEBUG(
"Upgrading Mnesia schema", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = rabbit_upgrade:maybe_upgrade_mnesia();
_ ->
ok
@ -17,6 +26,8 @@ setup(Context) ->
%% It's important that the consistency check happens after
%% the upgrade, since if we are a secondary node the
%% primary node will have forgotten us
rabbit_log_prelaunch:debug("Checking cluster consistency"),
?LOG_DEBUG(
"Checking cluster consistency", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
rabbit_mnesia:check_cluster_consistency(),
ok.

View File

@ -7,13 +7,17 @@
-module(rabbit_prelaunch_enabled_plugins_file).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([setup/1]).
setup(Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Enabled plugins file =="),
?LOG_DEBUG(
"~n== Enabled plugins file ==", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
update_enabled_plugins_file(Context).
%% -------------------------------------------------------------------
@ -33,21 +37,28 @@ do_update_enabled_plugins_file(#{enabled_plugins_file := File}, List) ->
SortedList = lists:usort(List),
case SortedList of
[] ->
rabbit_log_prelaunch:debug("Marking all plugins as disabled");
?LOG_DEBUG(
"Marking all plugins as disabled", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH});
_ ->
rabbit_log_prelaunch:debug(
"Marking the following plugins as enabled:"),
[rabbit_log_prelaunch:debug(" - ~s", [P]) || P <- SortedList]
?LOG_DEBUG(
lists:flatten(["Marking the following plugins as enabled:",
["~n - ~s" || _ <- SortedList]]),
SortedList,
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})
end,
Content = io_lib:format("~p.~n", [SortedList]),
case file:write_file(File, Content) of
ok ->
rabbit_log_prelaunch:debug("Wrote plugins file: ~ts", [File]),
?LOG_DEBUG(
"Wrote plugins file: ~ts", [File],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok;
{error, Reason} ->
rabbit_log_prelaunch:error(
?LOG_ERROR(
"Failed to update enabled plugins file \"~ts\" "
"from $RABBITMQ_ENABLED_PLUGINS: ~ts",
[File, file:format_error(Reason)]),
[File, file:format_error(Reason)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({error, failed_to_update_enabled_plugins_file})
end.

View File

@ -7,26 +7,35 @@
-module(rabbit_prelaunch_feature_flags).
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-export([setup/1]).
setup(#{feature_flags_file := FFFile}) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Feature flags =="),
?LOG_DEBUG(
"~n== Feature flags ==", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case filelib:ensure_dir(FFFile) of
ok ->
rabbit_log_prelaunch:debug("Initializing feature flags registry"),
?LOG_DEBUG(
"Initializing feature flags registry", [],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
case rabbit_feature_flags:initialize_registry() of
ok ->
ok;
{error, Reason} ->
rabbit_log_prelaunch:error(
?LOG_ERROR(
"Failed to initialize feature flags registry: ~p",
[Reason]),
[Reason],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({error, failed_to_initialize_feature_flags_registry})
end;
{error, Reason} ->
rabbit_log_prelaunch:error(
?LOG_ERROR(
"Failed to create feature flags file \"~ts\" directory: ~ts",
[FFFile, file:format_error(Reason)]),
[FFFile, file:format_error(Reason)],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({error, failed_to_create_feature_flags_file_directory})
end.

File diff suppressed because it is too large Load Diff

View File

@ -282,9 +282,6 @@ socket_error(Reason) ->
%% This is presumably a TCP healthcheck, so don't log
%% it unless specified otherwise.
{ssl_upgrade_error, closed} ->
%% Lager sinks (rabbit_log_connection)
%% are handled by the lager parse_transform.
%% Hence have to define the loglevel as a function call.
rabbit_log_connection:debug(Fmt, Args);
_ ->
rabbit_log_connection:error(Fmt, Args)

View File

@ -141,20 +141,7 @@ init_per_testcase(Testcase, Config) ->
TestNumber = rabbit_ct_helpers:testcase_number(Config, ?MODULE, Testcase),
case ?config(tc_group_properties, Config) of
[{name, registry} | _] ->
application:set_env(lager, colored, true),
application:set_env(
lager,
handlers, [{lager_console_backend, [{level, debug}]}]),
application:set_env(
lager,
extra_sinks,
[{rabbit_log_lager_event,
[{handlers, [{lager_console_backend, [{level, debug}]}]}]
},
{rabbit_log_feature_flags_lager_event,
[{handlers, [{lager_console_backend, [{level, debug}]}]}]
}]),
lager:start(),
logger:set_primary_config(level, debug),
FeatureFlagsFile = filename:join(?config(priv_dir, Config),
rabbit_misc:format(
"feature_flags-~s",

642
deps/rabbit/test/logging_SUITE.erl vendored Normal file
View File

@ -0,0 +1,642 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2016-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(logging_SUITE).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("kernel/include/logger.hrl").
-include_lib("rabbit_common/include/logging.hrl").
-include_lib("amqp_client/include/amqp_client.hrl").
-export([all/0,
init_per_suite/1,
end_per_suite/1,
init_per_group/2,
end_per_group/2,
init_per_testcase/2,
end_per_testcase/2,
logging_with_default_config_works/1,
logging_to_stdout_configured_in_env_works/1,
logging_to_stdout_configured_in_config_works/1,
logging_to_stderr_configured_in_env_works/1,
logging_to_exchange_works/1,
setting_log_levels_in_env_works/1,
setting_log_levels_in_config_works/1,
format_messages_as_json_works/1]).
all() ->
[logging_with_default_config_works,
logging_to_stdout_configured_in_env_works,
logging_to_stdout_configured_in_config_works,
logging_to_stderr_configured_in_env_works,
logging_to_exchange_works,
setting_log_levels_in_env_works,
setting_log_levels_in_config_works,
format_messages_as_json_works].
init_per_suite(Config) ->
rabbit_ct_helpers:log_environment(),
rabbit_ct_helpers:run_setup_steps(Config).
end_per_suite(Config) ->
Config.
init_per_group(_, Config) ->
Config.
end_per_group(_, Config) ->
Config.
init_per_testcase(logging_to_exchange_works = Testcase, Config) ->
rabbit_ct_helpers:testcase_started(Config, Testcase),
Config1 = rabbit_ct_helpers:set_config(
Config,
[{rmq_nodes_count, 1},
{rmq_nodename_suffix, Testcase}]),
Config2 = rabbit_ct_helpers:merge_app_env(
Config1,
{rabbit, [{log, [{exchange, [{enabled, true},
{level, info}]},
{file, [{level, info}]}]}]}),
rabbit_ct_helpers:run_steps(
Config2,
rabbit_ct_broker_helpers:setup_steps() ++
rabbit_ct_client_helpers:setup_steps());
init_per_testcase(Testcase, Config) ->
remove_all_handlers(),
application:unset_env(rabbit, log),
LogBaseDir = filename:join(
?config(priv_dir, Config),
atom_to_list(Testcase)),
Config1 = rabbit_ct_helpers:set_config(
Config, {log_base_dir, LogBaseDir}),
rabbit_ct_helpers:testcase_finished(Config1, Testcase).
end_per_testcase(logging_to_exchange_works, Config) ->
rabbit_ct_helpers:run_steps(
Config,
rabbit_ct_client_helpers:teardown_steps() ++
rabbit_ct_broker_helpers:teardown_steps());
end_per_testcase(_, Config) ->
application:unset_env(rabbit, log),
Config.
remove_all_handlers() ->
_ = [logger:remove_handler(Id)
|| #{id := Id} <- logger:get_handler_config()].
logging_with_default_config_works(Config) ->
Context = default_context(Config),
rabbit_prelaunch_logging:clear_config_run_number(),
rabbit_prelaunch_logging:setup(Context),
Handlers = logger:get_handler_config(),
MainFileHandler = get_handler_by_id(Handlers, rmq_1_file_1),
MainFile = main_log_file_in_context(Context),
?assertNotEqual(undefined, MainFileHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := log,
filters := [{progress_reports, {_, stop}},
{rmqlog_filter, {_, #{global := info,
upgrade := none}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := MainFile}},
MainFileHandler),
UpgradeFileHandler = get_handler_by_id(Handlers, rmq_1_file_2),
UpgradeFile = upgrade_log_file_in_context(Context),
?assertNotEqual(undefined, UpgradeFileHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := stop,
filters := [{rmqlog_filter, {_, #{upgrade := info}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := UpgradeFile}},
UpgradeFileHandler),
?assert(ping_log(rmq_1_file_1, info)),
?assert(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
?assert(ping_log(rmq_1_file_1, info,
#{domain => ['3rd_party']})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assert(ping_log(rmq_1_file_2, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assertNot(ping_log(rmq_1_file_2, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
ok.
logging_to_stdout_configured_in_env_works(Config) ->
#{var_origins := Origins0} = Context0 = default_context(Config),
Context = Context0#{main_log_file => "-",
var_origins => Origins0#{
main_log_file => environment}},
logging_to_stddev_works(standard_io, rmq_1_stdout, Config, Context).
logging_to_stdout_configured_in_config_works(Config) ->
Context = default_context(Config),
ok = application:set_env(
rabbit, log,
[{console, [{enabled, true}]}],
[{persistent, true}]),
logging_to_stddev_works(standard_io, rmq_1_stdout, Config, Context).
logging_to_stderr_configured_in_env_works(Config) ->
#{var_origins := Origins0} = Context0 = default_context(Config),
Context = Context0#{main_log_file => "-stderr",
var_origins => Origins0#{
main_log_file => environment}},
logging_to_stddev_works(standard_error, rmq_1_stderr, Config, Context).
logging_to_stddev_works(Stddev, Id, Config, Context) ->
rabbit_prelaunch_logging:clear_config_run_number(),
rabbit_prelaunch_logging:setup(Context),
Handlers = logger:get_handler_config(),
StddevHandler = get_handler_by_id(Handlers, Id),
?assertNotEqual(undefined, StddevHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := log,
filters := [{progress_reports, {_, stop}},
{rmqlog_filter, {_, #{global := info,
upgrade := none}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := Stddev}},
StddevHandler),
UpgradeFileHandler = get_handler_by_id(Handlers, rmq_1_file_1),
UpgradeFile = upgrade_log_file_in_context(Context),
?assertNotEqual(undefined, UpgradeFileHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := stop,
filters := [{rmqlog_filter, {_, #{upgrade := info}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := UpgradeFile}},
UpgradeFileHandler),
?assert(ping_log(Id, info, Config)),
?assert(ping_log(Id, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL}, Config)),
?assert(ping_log(Id, info,
#{domain => ['3rd_party']}, Config)),
?assertNot(ping_log(Id, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE}, Config)),
?assert(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
ok.
logging_to_exchange_works(Config) ->
Context = rabbit_ct_broker_helpers:rpc(
Config, 0,
rabbit_prelaunch, get_context, []),
Handlers = rabbit_ct_broker_helpers:rpc(
Config, 0,
logger, get_handler_config, []),
ExchangeHandler = get_handler_by_id(Handlers, rmq_1_exchange),
?assertNotEqual(undefined, ExchangeHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_exchange_h,
filter_default := log,
filters := [{progress_reports, {_, stop}},
{rmqlog_filter, {_, #{global := info,
upgrade := none}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{exchange := _}},
ExchangeHandler),
#{config :=
#{exchange := #resource{name = XName} = Exchange}} = ExchangeHandler,
UpgradeFileHandler = get_handler_by_id(Handlers, rmq_1_file_2),
UpgradeFile = upgrade_log_file_in_context(Context),
?assertNotEqual(undefined, UpgradeFileHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := stop,
filters := [{rmqlog_filter, {_, #{upgrade := info}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := UpgradeFile}},
UpgradeFileHandler),
%% Wait for the expected exchange to be automatically declared.
lists:any(
fun(_) ->
Ret = rabbit_ct_broker_helpers:rpc(
Config, 0,
rabbit_exchange, lookup, [Exchange]),
case Ret of
{ok, _} -> true;
_ -> timer:sleep(500),
false
end
end, lists:seq(1, 20)),
%% Declare a queue to collect all logged messages.
{Conn, Chan} = rabbit_ct_client_helpers:open_connection_and_channel(
Config),
QName = <<"log-messages">>,
?assertMatch(
#'queue.declare_ok'{},
amqp_channel:call(Chan, #'queue.declare'{queue = QName,
durable = false})),
?assertMatch(
#'queue.bind_ok'{},
amqp_channel:call(Chan, #'queue.bind'{queue = QName,
exchange = XName,
routing_key = <<"#">>})),
Config1 = rabbit_ct_helpers:set_config(
Config, {test_channel_and_queue, {Chan, QName}}),
?assert(ping_log(rmq_1_exchange, info, Config1)),
?assert(ping_log(rmq_1_exchange, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL}, Config1)),
?assert(ping_log(rmq_1_exchange, info,
#{domain => ['3rd_party']}, Config1)),
?assertNot(ping_log(rmq_1_exchange, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE}, Config1)),
?assert(ping_log(rmq_1_file_2, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE}, Config)),
?assertNot(ping_log(rmq_1_file_2, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL}, Config)),
amqp_channel:call(Chan, #'queue.delete'{queue = QName}),
rabbit_ct_client_helpers:close_connection_and_channel(Conn, Chan),
ok.
setting_log_levels_in_env_works(Config) ->
GlobalLevel = warning,
PrelaunchLevel = error,
MinLevel = rabbit_prelaunch_logging:get_less_severe_level(
GlobalLevel, PrelaunchLevel),
#{var_origins := Origins0} = Context0 = default_context(Config),
Context = Context0#{log_levels => #{global => GlobalLevel,
"prelaunch" => PrelaunchLevel},
var_origins => Origins0#{log_levels => environment}},
rabbit_prelaunch_logging:clear_config_run_number(),
rabbit_prelaunch_logging:setup(Context),
Handlers = logger:get_handler_config(),
MainFileHandler = get_handler_by_id(Handlers, rmq_1_file_1),
MainFile = main_log_file_in_context(Context),
?assertNotEqual(undefined, MainFileHandler),
?assertMatch(
#{level := MinLevel,
module := rabbit_logger_std_h,
filter_default := log,
filters := [{progress_reports, {_, stop}},
{rmqlog_filter, {_, #{global := GlobalLevel,
prelaunch := PrelaunchLevel,
upgrade := none}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := MainFile}},
MainFileHandler),
UpgradeFileHandler = get_handler_by_id(Handlers, rmq_1_file_2),
UpgradeFile = upgrade_log_file_in_context(Context),
?assertNotEqual(undefined, UpgradeFileHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := stop,
filters := [{rmqlog_filter, {_, #{upgrade := info}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := UpgradeFile}},
UpgradeFileHandler),
?assertNot(ping_log(rmq_1_file_1, info)),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})),
?assertNot(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ['3rd_party']})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assert(ping_log(rmq_1_file_1, GlobalLevel)),
?assert(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
?assert(ping_log(rmq_1_file_1, PrelaunchLevel,
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})),
?assert(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ['3rd_party']})),
?assertNot(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assert(ping_log(rmq_1_file_2, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assertNot(ping_log(rmq_1_file_2, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
ok.
setting_log_levels_in_config_works(Config) ->
GlobalLevel = warning,
PrelaunchLevel = error,
MinLevel = rabbit_prelaunch_logging:get_less_severe_level(
GlobalLevel, PrelaunchLevel),
Context = default_context(Config),
ok = application:set_env(
rabbit, log,
[{file, [{level, GlobalLevel}]},
{categories, [{prelaunch, [{level, PrelaunchLevel}]}]}],
[{persistent, true}]),
rabbit_prelaunch_logging:clear_config_run_number(),
rabbit_prelaunch_logging:setup(Context),
Handlers = logger:get_handler_config(),
MainFileHandler = get_handler_by_id(Handlers, rmq_1_file_1),
MainFile = main_log_file_in_context(Context),
?assertNotEqual(undefined, MainFileHandler),
?assertMatch(
#{level := MinLevel,
module := rabbit_logger_std_h,
filter_default := log,
filters := [{progress_reports, {_, stop}},
{rmqlog_filter, {_, #{global := GlobalLevel,
prelaunch := PrelaunchLevel,
upgrade := none}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := MainFile}},
MainFileHandler),
UpgradeFileHandler = get_handler_by_id(Handlers, rmq_1_file_2),
UpgradeFile = upgrade_log_file_in_context(Context),
?assertNotEqual(undefined, UpgradeFileHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := stop,
filters := [{rmqlog_filter, {_, #{upgrade := info}}}],
formatter := {rabbit_logger_text_fmt, _},
config := #{type := file,
file := UpgradeFile}},
UpgradeFileHandler),
?assertNot(ping_log(rmq_1_file_1, info)),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})),
?assertNot(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ['3rd_party']})),
?assertNot(ping_log(rmq_1_file_1, info,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assert(ping_log(rmq_1_file_1, GlobalLevel)),
?assert(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
?assert(ping_log(rmq_1_file_1, PrelaunchLevel,
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH})),
?assert(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ['3rd_party']})),
?assertNot(ping_log(rmq_1_file_1, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assert(ping_log(rmq_1_file_2, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_UPGRADE})),
?assertNot(ping_log(rmq_1_file_2, GlobalLevel,
#{domain => ?RMQLOG_DOMAIN_GLOBAL})),
ok.
format_messages_as_json_works(Config) ->
#{var_origins := Origins0} = Context0 = default_context(Config),
Context = Context0#{log_levels => #{json => true},
var_origins => Origins0#{log_levels => environment}},
rabbit_prelaunch_logging:clear_config_run_number(),
rabbit_prelaunch_logging:setup(Context),
Handlers = logger:get_handler_config(),
MainFileHandler = get_handler_by_id(Handlers, rmq_1_file_1),
MainFile = main_log_file_in_context(Context),
?assertNotEqual(undefined, MainFileHandler),
?assertMatch(
#{level := info,
module := rabbit_logger_std_h,
filter_default := log,
filters := [{progress_reports, {_, stop}},
{rmqlog_filter, {_, #{global := info,
upgrade := none}}}],
formatter := {rabbit_logger_json_fmt, _},
config := #{type := file,
file := MainFile}},
MainFileHandler),
?assertNot(ping_log(rmq_1_file_1, info)),
RandomMsg = get_random_string(
32,
"abcdefghijklmnopqrstuvwxyz"
"ABCDEFGHIJKLMNOPQRSTUVWXYZ"),
Metadata = #{atom => rabbit,
integer => 1,
float => 1.42,
string => "string",
list => ["s", a, 3],
map => #{key => "value"},
function => fun get_random_string/2,
pid => self(),
port => hd(erlang:ports()),
ref => erlang:make_ref()},
?LOG_INFO(RandomMsg, Metadata),
rabbit_logger_std_h:filesync(rmq_1_file_1),
{ok, Content} = file:read_file(MainFile),
ReOpts = [{capture, first, binary}, multiline],
{match, [Line]} = re:run(
Content,
"^.+\"" ++ RandomMsg ++ "\".+$",
ReOpts),
Term = jsx:decode(Line, [return_maps, {labels, attempt_atom}]),
RandomMsgBin = list_to_binary(RandomMsg),
?assertMatch(#{time := _}, Term),
?assertMatch(#{level := <<"info">>}, Term),
?assertMatch(#{msg := RandomMsgBin}, Term),
Meta = maps:get(meta, Term),
FunBin = list_to_binary(erlang:fun_to_list(maps:get(function, Metadata))),
PidBin = list_to_binary(erlang:pid_to_list(maps:get(pid, Metadata))),
PortBin = list_to_binary(erlang:port_to_list(maps:get(port, Metadata))),
RefBin = list_to_binary(erlang:ref_to_list(maps:get(ref, Metadata))),
?assertMatch(#{atom := <<"rabbit">>}, Meta),
?assertMatch(#{integer := 1}, Meta),
?assertMatch(#{float := 1.42}, Meta),
?assertMatch(#{string := <<"string">>}, Meta),
?assertMatch(#{list := [<<"s">>, <<"a">>, 3]}, Meta),
?assertMatch(#{map := #{key := <<"value">>}}, Meta),
?assertMatch(#{function := FunBin}, Meta),
?assertMatch(#{pid := PidBin}, Meta),
?assertMatch(#{port := PortBin}, Meta),
?assertMatch(#{ref := RefBin}, Meta).
%% -------------------------------------------------------------------
%% Internal functions.
%% -------------------------------------------------------------------
default_context(Config) ->
LogBaseDir = ?config(log_base_dir, Config),
MainFile = "rabbit.log",
UpgradeFile = "rabbit_upgrade.log",
#{log_base_dir => LogBaseDir,
main_log_file => MainFile,
upgrade_log_file => UpgradeFile,
log_levels => undefined,
var_origins => #{log_base_dir => default,
main_log_file => default,
upgrade_log_file => default,
log_levels => default}}.
main_log_file_in_context(#{log_base_dir := LogBaseDir,
main_log_file := MainLogFile}) ->
filename:join(LogBaseDir, MainLogFile).
upgrade_log_file_in_context(#{log_base_dir := LogBaseDir,
upgrade_log_file := UpgradeLogFile}) ->
filename:join(LogBaseDir, UpgradeLogFile).
get_handler_by_id([#{id := Id} = Handler | _], Id) ->
Handler;
get_handler_by_id([_ | Rest], Id) ->
get_handler_by_id(Rest, Id);
get_handler_by_id([], _) ->
undefined.
ping_log(Id, Level) ->
ping_log(Id, Level, #{}, []).
ping_log(Id, Level, Metadata) when is_map(Metadata) ->
ping_log(Id, Level, Metadata, []);
ping_log(Id, Level, Config) when is_list(Config) ->
ping_log(Id, Level, #{}, Config).
ping_log(Id, Level, Metadata, Config) ->
RandomMsg = get_random_string(
32,
"abcdefghijklmnopqrstuvwxyz"
"ABCDEFGHIJKLMNOPQRSTUVWXYZ"),
ct:log("Logging \"~ts\" at level ~ts (~p)", [RandomMsg, Level, Metadata]),
case need_rpc(Config) of
false -> logger:log(Level, RandomMsg, Metadata);
true -> rabbit_ct_broker_helpers:rpc(
Config, 0,
logger, log, [Level, RandomMsg, Metadata])
end,
check_log(Id, RandomMsg, Config).
need_rpc(Config) ->
rabbit_ct_helpers:get_config(
Config, rmq_nodes_count) =/= undefined.
check_log(Id, RandomMsg, Config) ->
{ok, Handler} = case need_rpc(Config) of
false -> logger:get_handler_config(Id);
true -> rabbit_ct_broker_helpers:rpc(
Config, 0,
logger, get_handler_config, [Id])
end,
check_log1(Handler, RandomMsg, Config).
check_log1(#{id := Id,
module := rabbit_logger_std_h,
config := #{type := file,
file := Filename}},
RandomMsg,
Config) ->
ok = case need_rpc(Config) of
false -> rabbit_logger_std_h:filesync(Id);
true -> rabbit_ct_broker_helpers:rpc(
Config, 0,
rabbit_logger_std_h, filesync, [Id])
end,
{ok, Content} = file:read_file(Filename),
ReOpts = [{capture, none}, multiline],
match =:= re:run(Content, RandomMsg ++ "$", ReOpts);
check_log1(#{module := Mod,
config := #{type := Stddev}},
RandomMsg,
Config)
when ?IS_STD_H_COMPAT(Mod) andalso ?IS_STDDEV(Stddev) ->
Filename = html_report_filename(Config),
ReOpts = [{capture, none}, multiline],
lists:any(
fun(_) ->
{ok, Content} = file:read_file(Filename),
case re:run(Content, RandomMsg ++ "$", ReOpts) of
match -> true;
_ -> timer:sleep(500),
false
end
end, lists:seq(1, 10));
check_log1(#{module := rabbit_logger_exchange_h},
RandomMsg,
Config) ->
{Chan, QName} = ?config(test_channel_and_queue, Config),
ReOpts = [{capture, none}, multiline],
lists:any(
fun(_) ->
Ret = amqp_channel:call(
Chan, #'basic.get'{queue = QName, no_ack = false}),
case Ret of
{#'basic.get_ok'{}, #amqp_msg{payload = Content}} ->
case re:run(Content, RandomMsg ++ "$", ReOpts) of
match -> true;
_ -> timer:sleep(500),
false
end;
#'basic.get_empty'{} ->
timer:sleep(500),
false;
Other ->
io:format(standard_error, "OTHER -> ~p~n", [Other]),
timer:sleep(500),
false
end
end, lists:seq(1, 10)).
get_random_string(Length, AllowedChars) ->
lists:foldl(fun(_, Acc) ->
[lists:nth(rand:uniform(length(AllowedChars)),
AllowedChars)]
++ Acc
end, [], lists:seq(1, Length)).
html_report_filename(Config) ->
?config(tc_logfile, Config).

View File

@ -1,838 +0,0 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2016-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(unit_log_config_SUITE).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
-compile(export_all).
all() ->
[
default,
env_var_tty,
config_file_handler,
config_file_handler_level,
config_file_handler_rotation,
config_console_handler,
config_exchange_handler,
config_syslog_handler,
config_syslog_handler_options,
config_multiple_handlers,
env_var_overrides_config,
env_var_disable_log,
config_sinks_level,
config_sink_file,
config_sink_file_override_config_handler_file,
config_handlers_merged_with_lager_handlers,
sink_handlers_merged_with_lager_extra_sinks_handlers,
sink_file_rewrites_file_backends
].
init_per_testcase(_, Config) ->
application:load(rabbit),
application:load(lager),
application:unset_env(rabbit, log),
application:unset_env(rabbit, lager_log_root),
application:unset_env(rabbit, lager_default_file),
application:unset_env(rabbit, lager_upgrade_file),
application:unset_env(lager, handlers),
application:unset_env(lager, rabbit_handlers),
application:unset_env(lager, extra_sinks),
unset_logs_var_origin(),
Config.
end_per_testcase(_, Config) ->
application:unset_env(rabbit, log),
application:unset_env(rabbit, lager_log_root),
application:unset_env(rabbit, lager_default_file),
application:unset_env(rabbit, lager_upgrade_file),
application:unset_env(lager, handlers),
application:unset_env(lager, rabbit_handlers),
application:unset_env(lager, extra_sinks),
unset_logs_var_origin(),
application:unload(rabbit),
application:unload(lager),
Config.
sink_file_rewrites_file_backends(_) ->
application:set_env(rabbit, log, [
%% Disable rabbit file handler
{file, [{file, false}]},
{categories, [{federation, [{file, "federation.log"}, {level, warning}]}]}
]),
LagerHandlers = [
{lager_file_backend, [{file, "lager_file.log"}, {level, error}]},
{lager_file_backend, [{file, "lager_file_1.log"}, {level, error}]},
{lager_console_backend, [{level, info}]},
{lager_exchange_backend, [{level, info}]}
],
application:set_env(lager, handlers, LagerHandlers),
rabbit_lager:configure_lager(),
ExpectedSinks = sort_sinks(sink_rewrite_sinks()),
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
sink_rewrite_sinks() ->
[{error_logger_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_channel_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_connection_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_feature_flags_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_federation_lager_event,
[{handlers,[
{lager_file_backend,
[{date, ""},
{file, "federation.log"},
{formatter_config, formatter_config(file)},
{level, warning},
{size, 0}]},
{lager_console_backend, [{level, warning}]},
{lager_exchange_backend, [{level, warning}]}
]},
{rabbit_handlers,[
{lager_file_backend,
[{date, ""},
{file, "federation.log"},
{formatter_config, formatter_config(file)},
{level, warning},
{size, 0}]},
{lager_console_backend, [{level, warning}]},
{lager_exchange_backend, [{level, warning}]}
]}]},
{rabbit_log_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ldap_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_osiris_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ra_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_shovel_lager_event,
[{handlers, [{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_upgrade_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]}
].
sink_handlers_merged_with_lager_extra_sinks_handlers(_) ->
DefaultLevel = debug,
application:set_env(rabbit, log, [
{file, [{file, "rabbit_file.log"}, {level, DefaultLevel}]},
{console, [{enabled, true}, {level, error}]},
{exchange, [{enabled, true}, {level, error}]},
{categories, [
{connection, [{level, debug}]},
{channel, [{level, warning}, {file, "channel_log.log"}]}
]}
]),
LagerSinks = [
{rabbit_log_connection_lager_event,
[{handlers,
[{lager_file_backend,
[{file, "connection_lager.log"},
{level, info}]}]}]},
{rabbit_log_channel_lager_event,
[{handlers,
[{lager_console_backend, [{level, debug}]},
{lager_exchange_backend, [{level, debug}]},
{lager_file_backend, [{level, error},
{file, "channel_lager.log"}]}]}]}],
application:set_env(lager, extra_sinks, LagerSinks),
rabbit_lager:configure_lager(),
ExpectedSinks = sort_sinks([
{error_logger_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_channel_lager_event,
[{handlers,[
{lager_console_backend, [{level, error},
{formatter_config, formatter_config(console)}]},
{lager_exchange_backend, [{level, error},
{formatter_config, formatter_config(exchange)}]},
{lager_file_backend,
[{date, ""},
{file, "channel_log.log"},
{formatter_config, formatter_config(file)},
{level, warning},
{size, 0}]},
{lager_console_backend, [{level, debug}]},
{lager_exchange_backend, [{level, debug}]},
{lager_file_backend, [{level, error},
{file, "channel_lager.log"}]}
]},
{rabbit_handlers,[
{lager_console_backend, [{level, error},
{formatter_config, formatter_config(console)}]},
{lager_exchange_backend, [{level, error},
{formatter_config, formatter_config(exchange)}]},
{lager_file_backend,
[{date, ""},
{file, "channel_log.log"},
{formatter_config, formatter_config(file)},
{level, warning},
{size, 0}]}]}
]},
{rabbit_log_connection_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,debug]},
{lager_file_backend, [{file, "connection_lager.log"}, {level, info}]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,debug]}]}]},
{rabbit_log_feature_flags_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_federation_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_ldap_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_osiris_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_ra_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_shovel_lager_event,
[{handlers, [{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_upgrade_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]}]),
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
config_handlers_merged_with_lager_handlers(_) ->
application:set_env(rabbit, log, [
{file, [{file, "rabbit_file.log"}, {level, debug}]},
{console, [{enabled, true}, {level, error}]},
{exchange, [{enabled, true}, {level, error}]},
{syslog, [{enabled, true}]}
]),
LagerHandlers = [
{lager_file_backend, [{file, "lager_file.log"}, {level, info}]},
{lager_console_backend, [{level, info}]},
{lager_exchange_backend, [{level, info}]},
{lager_exchange_backend, [{level, info}]}
],
application:set_env(lager, handlers, LagerHandlers),
rabbit_lager:configure_lager(),
FileHandlers = default_expected_handlers("rabbit_file.log", debug),
ConsoleHandlers = expected_console_handler(error),
RabbitHandlers = expected_rabbit_handler(error),
SyslogHandlers = expected_syslog_handler(),
ExpectedRabbitHandlers = sort_handlers(FileHandlers ++ ConsoleHandlers ++ RabbitHandlers ++ SyslogHandlers),
ExpectedHandlers = sort_handlers(ExpectedRabbitHandlers ++ LagerHandlers),
?assertEqual(ExpectedRabbitHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))).
config_sinks_level(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [
{categories, [
{connection, [{level, warning}]},
{channel, [{level, debug}]},
{mirroring, [{level, error}]}
]}
]),
rabbit_lager:configure_lager(),
ExpectedSinks = sort_sinks(level_sinks()),
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
level_sinks() ->
[{error_logger_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_channel_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,debug]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,debug]}]}]},
{rabbit_log_connection_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,warning]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,warning]}]}]},
{rabbit_log_feature_flags_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_federation_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ldap_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,error]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,error]}]}]},
{rabbit_log_osiris_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ra_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_shovel_lager_event,
[{handlers, [{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,
[lager_event,info]}]}]},
{rabbit_log_upgrade_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]}
].
config_sink_file(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
DefaultLevel = error,
application:set_env(rabbit, log, [
{console, [{enabled, true}]},
{exchange, [{enabled, true}]},
{file, [{level, DefaultLevel}]},
{categories, [
{connection, [{file, "connection.log"}, {level, warning}]}
]}
]),
rabbit_lager:configure_lager(),
ExpectedSinks = sort_sinks(file_sinks(DefaultLevel)),
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
config_sink_file_override_config_handler_file(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
NonDefaultLogFile = "rabbit_not_default.log",
DefaultLevel = error,
application:set_env(rabbit, log, [
{file, [{file, NonDefaultLogFile}, {level, DefaultLevel}]},
{console, [{enabled, true}]},
{exchange, [{enabled, true}]},
{categories, [
{connection, [{file, "connection.log"}, {level, warning}]}
]}
]),
rabbit_lager:configure_lager(),
ExpectedSinks = sort_sinks(file_sinks(DefaultLevel)),
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
file_sinks() ->
file_sinks(info).
file_sinks(DefaultLevel) ->
[{error_logger_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_channel_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_connection_lager_event,
[{handlers,[
{lager_console_backend, [{level, warning},
{formatter_config, formatter_config(console)}]},
{lager_exchange_backend, [{level, warning},
{formatter_config, formatter_config(exchange)}]},
{lager_file_backend,
[{date, ""},
{file, "connection.log"},
{formatter_config, formatter_config(file)},
{level, error},
{size, 0}]}]},
{rabbit_handlers,[
{lager_console_backend, [{level, warning},
{formatter_config, formatter_config(console)}]},
{lager_exchange_backend, [{level, warning},
{formatter_config, formatter_config(exchange)}]},
{lager_file_backend,
[{date, ""},
{file, "connection.log"},
{formatter_config, formatter_config(backend)},
{level, error},
{size, 0}]}]}
]},
{rabbit_log_feature_flags_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_federation_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_ldap_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_osiris_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_ra_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_shovel_lager_event,
[{handlers, [{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]},
{rabbit_log_upgrade_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,DefaultLevel]}]}]}
].
config_multiple_handlers(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [
%% Disable file output
{file, [{file, false}]},
%% Enable console output
{console, [{enabled, true}]},
%% Enable exchange output
{exchange, [{enabled, true}]},
%% Enable a syslog output
{syslog, [{enabled, true}, {level, error}]}]),
rabbit_lager:configure_lager(),
ConsoleHandlers = expected_console_handler(),
RabbitHandlers = expected_rabbit_handler(),
SyslogHandlers = expected_syslog_handler(error),
ExpectedHandlers = sort_handlers(SyslogHandlers ++ ConsoleHandlers ++ RabbitHandlers),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
config_console_handler(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [{console, [{enabled, true}]}]),
rabbit_lager:configure_lager(),
FileHandlers = default_expected_handlers(DefaultLogFile),
ConsoleHandlers = expected_console_handler(),
ExpectedHandlers = sort_handlers(FileHandlers ++ ConsoleHandlers),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
config_exchange_handler(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [{exchange, [{enabled, true}]}]),
rabbit_lager:configure_lager(),
FileHandlers = default_expected_handlers(DefaultLogFile),
ExchangeHandlers = expected_rabbit_handler(),
ExpectedHandlers = sort_handlers(FileHandlers ++ ExchangeHandlers),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
expected_console_handler() ->
expected_console_handler(debug).
expected_console_handler(Level) ->
[{lager_console_backend, [{level, Level},
{formatter_config, formatter_config(console)}]}].
expected_rabbit_handler() ->
expected_rabbit_handler(debug).
expected_rabbit_handler(Level) ->
[{lager_exchange_backend, [{level, Level},
{formatter_config, formatter_config(exchange)}]}].
config_syslog_handler(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [{syslog, [{enabled, true}]}]),
rabbit_lager:configure_lager(),
FileHandlers = default_expected_handlers(DefaultLogFile),
SyslogHandlers = expected_syslog_handler(),
ExpectedHandlers = sort_handlers(FileHandlers ++ SyslogHandlers),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
config_syslog_handler_options(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [{syslog, [{enabled, true},
{level, warning}]}]),
rabbit_lager:configure_lager(),
FileHandlers = default_expected_handlers(DefaultLogFile),
SyslogHandlers = expected_syslog_handler(warning),
ExpectedHandlers = sort_handlers(FileHandlers ++ SyslogHandlers),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
expected_syslog_handler() ->
expected_syslog_handler(debug).
expected_syslog_handler(Level) ->
[{syslog_lager_backend, [Level,
{},
{lager_default_formatter, syslog_formatter_config()}]}].
env_var_overrides_config(_) ->
EnvLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, EnvLogFile),
ConfigLogFile = "rabbit_not_default.log",
application:set_env(rabbit, log, [{file, [{file, ConfigLogFile}]}]),
set_logs_var_origin(environment),
rabbit_lager:configure_lager(),
ExpectedHandlers = default_expected_handlers(EnvLogFile),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
env_var_disable_log(_) ->
application:set_env(rabbit, lager_default_file, false),
ConfigLogFile = "rabbit_not_default.log",
application:set_env(rabbit, log, [{file, [{file, ConfigLogFile}]}]),
set_logs_var_origin(environment),
rabbit_lager:configure_lager(),
ExpectedHandlers = [],
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
config_file_handler(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
NonDefaultLogFile = "rabbit_not_default.log",
application:set_env(rabbit, log, [{file, [{file, NonDefaultLogFile}]}]),
rabbit_lager:configure_lager(),
ExpectedHandlers = default_expected_handlers(NonDefaultLogFile),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
config_file_handler_level(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [{file, [{level, warning}]}]),
rabbit_lager:configure_lager(),
ExpectedHandlers = default_expected_handlers(DefaultLogFile, warning),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
config_file_handler_rotation(_) ->
DefaultLogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, DefaultLogFile),
application:set_env(rabbit, log, [{file, [{date, "$D0"}, {size, 5000}, {count, 10}]}]),
rabbit_lager:configure_lager(),
ExpectedHandlers = sort_handlers(default_expected_handlers(DefaultLogFile, debug, 5000, "$D0", [{count, 10}])),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))).
default(_) ->
LogRoot = "/tmp/log_base",
application:set_env(rabbit, lager_log_root, LogRoot),
LogFile = "rabbit_default.log",
application:set_env(rabbit, lager_default_file, LogFile),
LogUpgradeFile = "rabbit_default_upgrade.log",
application:set_env(rabbit, lager_upgrade_file, LogUpgradeFile),
?assertEqual(LogRoot, application:get_env(rabbit, lager_log_root, undefined)),
rabbit_lager:configure_lager(),
ExpectedHandlers = default_expected_handlers(LogFile),
?assertEqual(LogRoot, application:get_env(lager, log_root, undefined)),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))),
ExpectedSinks = default_expected_sinks(LogUpgradeFile),
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
default_expected_handlers(File) ->
default_expected_handlers(File, debug, 0, "").
default_expected_handlers(File, Level) ->
default_expected_handlers(File, Level, 0, "").
default_expected_handlers(File, Level, RotSize, RotDate) ->
default_expected_handlers(File, Level, RotSize, RotDate, []).
default_expected_handlers(File, Level, RotSize, RotDate, Extra) ->
[{lager_file_backend,
[{date, RotDate},
{file, File},
{formatter_config, formatter_config(file)},
{level, Level},
{size, RotSize}] ++ Extra}].
default_expected_sinks(UpgradeFile) ->
[{error_logger_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_channel_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_connection_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_feature_flags_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_federation_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ldap_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_osiris_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ra_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_shovel_lager_event,
[{handlers, [{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_upgrade_lager_event,
[{handlers,
[{lager_file_backend,
[{date,[]},
{file, UpgradeFile},
{formatter_config, formatter_config(file)},
{level,info},
{size,0}]}]},
{rabbit_handlers,
[{lager_file_backend,
[{date,[]},
{file, UpgradeFile},
{formatter_config, formatter_config(file)},
{level,info},
{size,0}]}]}]}].
env_var_tty(_) ->
application:set_env(rabbit, lager_log_root, "/tmp/log_base"),
application:set_env(rabbit, lager_default_file, tty),
application:set_env(rabbit, lager_upgrade_file, tty),
%% tty can only be set explicitly
set_logs_var_origin(environment),
rabbit_lager:configure_lager(),
ExpectedHandlers = tty_expected_handlers(),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, handlers, undefined))),
?assertEqual(ExpectedHandlers, sort_handlers(application:get_env(lager, rabbit_handlers, undefined))),
%% Upgrade sink will be different.
ExpectedSinks = tty_expected_sinks(),
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
set_logs_var_origin(Origin) ->
Context = #{var_origins => #{main_log_file => Origin}},
rabbit_prelaunch:store_context(Context),
ok.
unset_logs_var_origin() ->
rabbit_prelaunch:clear_context_cache(),
ok.
tty_expected_handlers() ->
[{lager_console_backend,
[{formatter_config, formatter_config(console)},
{level, debug}]}].
tty_expected_sinks() ->
[{error_logger_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_channel_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_connection_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_feature_flags_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_federation_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_lager_event,
[{handlers, [{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers, [{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ldap_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_osiris_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_ra_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_shovel_lager_event,
[{handlers, [{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,
[{lager_forwarder_backend,[lager_event,info]}]}]},
{rabbit_log_upgrade_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,info]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,info]}]}]}].
sort_sinks(Sinks) ->
lists:ukeysort(1,
lists:map(
fun({Name, Config}) ->
Handlers = proplists:get_value(handlers, Config),
RabbitHandlers = proplists:get_value(rabbit_handlers, Config),
{Name, lists:ukeymerge(1,
[{handlers, sort_handlers(Handlers)},
{rabbit_handlers, sort_handlers(RabbitHandlers)}],
lists:ukeysort(1, Config))}
end,
Sinks)).
sort_handlers(Handlers) ->
lists:keysort(1,
lists:map(
fun
({Name, [{Atom, _}|_] = Config}) when is_atom(Atom) ->
{Name, lists:ukeysort(1, Config)};
%% Non-proplist configuration. forwarder backend
(Other) ->
Other
end,
Handlers)).
formatter_config(console) ->
[date," ",time," ",color,"[",severity, "] ", {pid,[]}, " ",message,"\r\n"];
formatter_config(_) ->
[date," ",time," ",color,"[",severity, "] ", {pid,[]}, " ",message,"\n"].
syslog_formatter_config() ->
[color,"[",severity, "] ", {pid,[]}, " ",message,"\n"].

View File

@ -25,7 +25,6 @@ all() ->
groups() ->
[
{non_parallel_tests, [], [
log_management,
log_file_initialised_during_startup,
log_file_fails_to_initialise_during_startup,
externally_rotated_logs_are_automatically_reopened
@ -113,94 +112,6 @@ wait_for_application(Application, Time) ->
%% Log management.
%% -------------------------------------------------------------------
log_management(Config) ->
passed = rabbit_ct_broker_helpers:rpc(Config, 0,
?MODULE, log_management1, [Config]).
log_management1(_Config) ->
[LogFile|_] = rabbit:log_locations(),
Suffix = ".0",
ok = test_logs_working([LogFile]),
%% prepare basic logs
file:delete(LogFile ++ Suffix),
ok = test_logs_working([LogFile]),
%% simple log rotation
ok = rabbit:rotate_logs(),
%% rabbit:rotate_logs/0 is asynchronous due to a limitation in
%% Lager. Therefore, we have no choice but to wait an arbitrary
%% amount of time.
ok = rabbit_ct_helpers:await_condition(
fun() ->
[true, true] =:=
non_empty_files([LogFile ++ Suffix, LogFile])
end, 5000),
ok = test_logs_working([LogFile]),
%% log rotation on empty files
ok = clean_logs([LogFile], Suffix),
ok = rabbit:rotate_logs(),
ok = rabbit_ct_helpers:await_condition(
fun() ->
[true, true] =:=
non_empty_files([LogFile ++ Suffix, LogFile])
end, 5000),
%% logs with suffix are not writable
ok = rabbit:rotate_logs(),
ok = rabbit_ct_helpers:await_condition(
fun() ->
ok =:= make_files_non_writable([LogFile ++ Suffix])
end, 5000),
ok = rabbit:rotate_logs(),
ok = rabbit_ct_helpers:await_condition(
fun() ->
ok =:= test_logs_working([LogFile])
end, 5000),
%% rotate when original log files are not writable
ok = make_files_non_writable([LogFile]),
ok = rabbit:rotate_logs(),
timer:sleep(2000),
%% logging directed to tty (first, remove handlers)
ok = rabbit:stop(),
ok = make_files_writable([LogFile ++ Suffix]),
ok = clean_logs([LogFile], Suffix),
ok = application:set_env(rabbit, lager_default_file, tty),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
ok = rabbit:start(),
timer:sleep(200),
rabbit_log:info("test info"),
%% rotate logs when logging is turned off
ok = rabbit:stop(),
ok = clean_logs([LogFile], Suffix),
ok = application:set_env(rabbit, lager_default_file, false),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
ok = rabbit:start(),
timer:sleep(200),
rabbit_log:error("test error"),
timer:sleep(200),
?assertEqual([{error,enoent}], empty_files([LogFile])),
%% cleanup
ok = rabbit:stop(),
ok = clean_logs([LogFile], Suffix),
ok = application:set_env(rabbit, lager_default_file, LogFile),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
ok = rabbit:start(),
ok = test_logs_working([LogFile]),
passed.
log_file_initialised_during_startup(Config) ->
passed = rabbit_ct_broker_helpers:rpc(Config, 0,
?MODULE, log_file_initialised_during_startup1, [Config]).
@ -212,10 +123,8 @@ log_file_initialised_during_startup1(_Config) ->
%% start application with simple tty logging
ok = rabbit:stop(),
ok = clean_logs([LogFile], Suffix),
ok = application:set_env(rabbit, lager_default_file, tty),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
ok = application:set_env(rabbit, log, [{console, [{enabled, true}]},
{file, [{file, false}]}]),
ok = rabbit:start(),
%% start application with logging to non-existing directory
@ -224,18 +133,14 @@ log_file_initialised_during_startup1(_Config) ->
delete_file(NonExistent),
delete_file(filename:dirname(NonExistent)),
ok = rabbit:stop(),
ct:pal("Setting lager_default_file to \"~s\"", [NonExistent]),
ok = application:set_env(rabbit, lager_default_file, NonExistent),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
io:format("Setting log file to \"~s\"~n", [NonExistent]),
ok = application:set_env(rabbit, log, [{console, [{enabled, true}]},
{file, [{file, NonExistent}]}]),
ok = rabbit:start(),
%% clean up
ok = application:set_env(rabbit, lager_default_file, LogFile),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
ok = application:set_env(rabbit, log, [{console, [{enabled, true}]},
{file, [{file, LogFile}]}]),
ok = rabbit:start(),
passed.
@ -277,13 +182,12 @@ log_file_fails_to_initialise_during_startup1(_Config, NonWritableDir) ->
delete_file(filename:dirname(NoPermission1)),
ok = rabbit:stop(),
ct:pal("Setting lager_default_file to \"~s\"", [NoPermission1]),
ok = application:set_env(rabbit, lager_default_file, NoPermission1),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
io:format("Setting log file to \"~s\"~n", [NoPermission1]),
ok = application:set_env(rabbit, log, [{console, [{enabled, true}]},
{file, [{file, NoPermission1}]}]),
ct:pal("`rabbit` application env.: ~p", [application:get_all_env(rabbit)]),
io:format("rabbit application env.: ~p~n",
[application:get_all_env(rabbit)]),
?assertThrow(
{error, {rabbit, {{cannot_log_to_file, _, _}, _}}},
rabbit:start()),
@ -296,22 +200,19 @@ log_file_fails_to_initialise_during_startup1(_Config, NonWritableDir) ->
delete_file(NoPermission2),
delete_file(filename:dirname(NoPermission2)),
ct:pal("Setting lager_default_file to \"~s\"", [NoPermission2]),
ok = application:set_env(rabbit, lager_default_file, NoPermission2),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
io:format("Setting log file to \"~s\"~n", [NoPermission2]),
ok = application:set_env(rabbit, log, [{console, [{enabled, true}]},
{file, [{file, NoPermission2}]}]),
ct:pal("`rabbit` application env.: ~p", [application:get_all_env(rabbit)]),
io:format("rabbit application env.: ~p~n",
[application:get_all_env(rabbit)]),
?assertThrow(
{error, {rabbit, {{cannot_log_to_file, _, _}, _}}},
rabbit:start()),
%% clean up
ok = application:set_env(rabbit, lager_default_file, LogFile),
application:unset_env(rabbit, log),
application:unset_env(lager, handlers),
application:unset_env(lager, extra_sinks),
ok = application:set_env(rabbit, log, [{console, [{enabled, true}]},
{file, [{file, LogFile}]}]),
ok = rabbit:start(),
passed.

View File

@ -16,7 +16,7 @@ define PROJECT_APP_EXTRA_KEYS
endef
LOCAL_DEPS = compiler crypto public_key sasl ssl syntax_tools tools xmerl
DEPS = lager jsx recon credentials_obfuscation
DEPS = jsx recon credentials_obfuscation
dep_credentials_obfuscation = git https://github.com/rabbitmq/credentials-obfuscation.git master

19
deps/rabbit_common/include/logging.hrl vendored Normal file
View File

@ -0,0 +1,19 @@
-define(RMQLOG_SUPER_DOMAIN_NAME, rabbitmq).
-define(RMQLOG_DOMAIN_GLOBAL, [?RMQLOG_SUPER_DOMAIN_NAME]).
-define(DEFINE_RMQLOG_DOMAIN(Domain), [?RMQLOG_SUPER_DOMAIN_NAME, Domain]).
-define(RMQLOG_DOMAIN_CHAN, ?DEFINE_RMQLOG_DOMAIN(channel)).
-define(RMQLOG_DOMAIN_CONN, ?DEFINE_RMQLOG_DOMAIN(connection)).
-define(RMQLOG_DOMAIN_FEAT_FLAGS, ?DEFINE_RMQLOG_DOMAIN(feature_flags)).
-define(RMQLOG_DOMAIN_MIRRORING, ?DEFINE_RMQLOG_DOMAIN(mirroring)).
-define(RMQLOG_DOMAIN_PRELAUNCH, ?DEFINE_RMQLOG_DOMAIN(prelaunch)).
-define(RMQLOG_DOMAIN_QUEUE, ?DEFINE_RMQLOG_DOMAIN(queue)).
-define(RMQLOG_DOMAIN_UPGRADE, ?DEFINE_RMQLOG_DOMAIN(upgrade)).
-define(DEFAULT_LOG_LEVEL, info).
-define(FILTER_NAME, rmqlog_filter).
-define(IS_STD_H_COMPAT(Mod),
Mod =:= logger_std_h orelse Mod =:= rabbit_logger_std_h).
-define(IS_STDDEV(DevName),
DevName =:= standard_io orelse DevName =:= standard_error).

View File

@ -1,8 +0,0 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2017-2020 VMware, Inc. or its affiliates. All rights reserved.
%%
-define(LAGER_SINK, rabbit_log_lager_event).

View File

@ -18,25 +18,6 @@ ifneq ($(filter-out rabbit_common amqp_client,$(PROJECT)),)
RMQ_ERLC_OPTS += -pa $(DEPS_DIR)/rabbitmq_cli/_build/dev/lib/rabbitmqctl/ebin
endif
# Add Lager parse_transform module and our default Lager extra sinks.
LAGER_EXTRA_SINKS += rabbit_log \
rabbit_log_channel \
rabbit_log_connection \
rabbit_log_feature_flags \
rabbit_log_federation \
rabbit_log_ldap \
rabbit_log_mirroring \
rabbit_log_osiris \
rabbit_log_prelaunch \
rabbit_log_queue \
rabbit_log_ra \
rabbit_log_shovel \
rabbit_log_upgrade
lager_extra_sinks = $(subst $(space),$(comma),$(LAGER_EXTRA_SINKS))
RMQ_ERLC_OPTS += +'{parse_transform,lager_transform}' \
+'{lager_extra_sinks,[$(lager_extra_sinks)]}'
# Push our compilation options to both the normal and test ERLC_OPTS.
ERLC_OPTS += $(RMQ_ERLC_OPTS)
TEST_ERLC_OPTS += $(RMQ_ERLC_OPTS)

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -164,9 +164,7 @@ define test_rabbitmq_config
[
{rabbit, [
$(if $(RABBITMQ_NODE_PORT), {tcp_listeners$(comma) [$(RABBITMQ_NODE_PORT)]}$(comma),)
{loopback_users, []},
{log, [{file, [{level, debug}]},
{console, [{level, debug}]}]}
{loopback_users, []}
]},
{rabbitmq_management, [
$(if $(RABBITMQ_NODE_PORT), {listener$(comma) [{port$(comma) $(shell echo "$$(($(RABBITMQ_NODE_PORT) + 10000))")}]},)
@ -184,19 +182,6 @@ $(if $(RABBITMQ_NODE_PORT), {tcp_listeners$(comma) [$(shell echo "$$((5551
{data_dir, "$(RABBITMQ_QUORUM_DIR)"},
{wal_sync_method, sync}
]},
{lager, [
{colors, [
%% https://misc.flogisoft.com/bash/tip_colors_and_formatting
{debug, "\\\e[0;34m" },
{info, "\\\e[1;37m" },
{notice, "\\\e[1;36m" },
{warning, "\\\e[1;33m" },
{error, "\\\e[1;31m" },
{critical, "\\\e[1;35m" },
{alert, "\\\e[1;44m" },
{emergency, "\\\e[1;41m" }
]}
]},
{osiris, [
{data_dir, "$(RABBITMQ_STREAM_DIR)"}
]}
@ -209,8 +194,6 @@ define test_rabbitmq_config_with_tls
[
{rabbit, [
{loopback_users, []},
{log, [{file, [{level, debug}]},
{console, [{level, debug}]}]},
{ssl_listeners, [5671]},
{ssl_options, [
{cacertfile, "$(TEST_TLS_CERTS_DIR_in_config)/testca/cacert.pem"},
@ -237,19 +220,6 @@ define test_rabbitmq_config_with_tls
{data_dir, "$(RABBITMQ_QUORUM_DIR)"},
{wal_sync_method, sync}
]},
{lager, [
{colors, [
%% https://misc.flogisoft.com/bash/tip_colors_and_formatting
{debug, "\\\e[0;34m" },
{info, "\\\e[1;37m" },
{notice, "\\\e[1;36m" },
{warning, "\\\e[1;33m" },
{error, "\\\e[1;31m" },
{critical, "\\\e[1;35m" },
{alert, "\\\e[1;44m" },
{emergency, "\\\e[1;41m" }
]}
]},
{osiris, [
{data_dir, "$(RABBITMQ_STREAM_DIR)"}
]}

View File

@ -1,120 +0,0 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2007-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(lager_forwarder_backend).
-behaviour(gen_event).
-export([init/1, handle_call/2, handle_event/2, handle_info/2, terminate/2,
code_change/3]).
-record(state, {
next_sink :: atom(),
level :: {'mask', integer()} | inherit
}).
%% @private
init(Sink) when is_atom(Sink) ->
init([Sink]);
init([Sink]) when is_atom(Sink) ->
init([Sink, inherit]);
init([Sink, inherit]) when is_atom(Sink) ->
{ok, #state{
next_sink = Sink,
level = inherit
}};
init([Sink, Level]) when is_atom(Sink) ->
try
Mask = lager_util:config_to_mask(Level),
{ok, #state{
next_sink = Sink,
level = Mask
}}
catch
_:_ ->
{error, {fatal, bad_log_level}}
end;
init(_) ->
{error, {fatal, bad_config}}.
%% @private
handle_call(get_loglevel, #state{next_sink = Sink, level = inherit} = State) ->
SinkPid = whereis(Sink),
Mask = case self() of
SinkPid ->
%% Avoid direct loops, defaults to 'info'.
127;
_ ->
try
Levels = [gen_event:call(SinkPid, Handler, get_loglevel,
infinity)
|| Handler <- gen_event:which_handlers(SinkPid)],
lists:foldl(fun
({mask, Mask}, Acc) ->
Mask bor Acc;
(Level, Acc) when is_integer(Level) ->
{mask, Mask} = lager_util:config_to_mask(
lager_util:num_to_level(Level)),
Mask bor Acc;
(_, Acc) ->
Acc
end, 0, Levels)
catch
exit:noproc ->
127
end
end,
{ok, {mask, Mask}, State};
handle_call(get_loglevel, #state{level = Mask} = State) ->
{ok, Mask, State};
handle_call({set_loglevel, inherit}, State) ->
{ok, ok, State#state{level = inherit}};
handle_call({set_loglevel, Level}, State) ->
try lager_util:config_to_mask(Level) of
Mask ->
{ok, ok, State#state{level = Mask}}
catch
_:_ ->
{ok, {error, bad_log_level}, State}
end;
handle_call(_Request, State) ->
{ok, ok, State}.
%% @private
handle_event({log, LagerMsg}, #state{next_sink = Sink, level = Mask} = State) ->
SinkPid = whereis(Sink),
case self() of
SinkPid ->
%% Avoid direct loops.
ok;
_ ->
case Mask =:= inherit orelse
lager_util:is_loggable(LagerMsg, Mask, ?MODULE) of
true ->
case lager_config:get({Sink, async}, false) of
true -> gen_event:notify(SinkPid, {log, LagerMsg});
false -> gen_event:sync_notify(SinkPid, {log, LagerMsg})
end;
false ->
ok
end
end,
{ok, State};
handle_event(_Event, State) ->
{ok, State}.
%% @private
handle_info(_Info, State) ->
{ok, State}.
%% @private
terminate(_Reason, _State) ->
ok.
%% @private
code_change(_OldVsn, State, _Extra) ->
{ok, State}.

View File

@ -8,6 +8,9 @@
-module(rabbit_env).
-include_lib("kernel/include/file.hrl").
-include_lib("kernel/include/logger.hrl").
-include("include/logging.hrl").
-export([get_context/0,
get_context/1,
@ -17,6 +20,8 @@
get_context_after_reloading_env/1,
dbg_config/0,
env_vars/0,
has_var_been_overridden/1,
has_var_been_overridden/2,
get_used_env_vars/0,
log_process_env/0,
log_context/1,
@ -75,6 +80,10 @@
"SYS_PREFIX"
]).
-export_type([context/0]).
-type context() :: map().
get_context() ->
Context0 = get_context_before_logging_init(),
Context1 = get_context_after_logging_init(Context0),
@ -225,33 +234,44 @@ env_vars() ->
false -> os:env() %% OTP >= 24
end.
has_var_been_overridden(Var) ->
has_var_been_overridden(get_context(), Var).
has_var_been_overridden(#{var_origins := Origins}, Var) ->
case maps:get(Var, Origins, default) of
default -> false;
_ -> true
end.
get_used_env_vars() ->
lists:filter(
fun({Var, _}) -> var_is_used(Var) end,
lists:sort(env_vars())).
log_process_env() ->
rabbit_log_prelaunch:debug("Process environment:"),
?LOG_DEBUG("Process environment:"),
lists:foreach(
fun({Var, Value}) ->
rabbit_log_prelaunch:debug(" - ~s = ~ts", [Var, Value])
?LOG_DEBUG(" - ~s = ~ts", [Var, Value])
end, lists:sort(env_vars())).
log_context(Context) ->
rabbit_log_prelaunch:debug("Context (based on environment variables):"),
?LOG_DEBUG("Context (based on environment variables):"),
lists:foreach(
fun(Key) ->
Value = maps:get(Key, Context),
rabbit_log_prelaunch:debug(" - ~s: ~p", [Key, Value])
?LOG_DEBUG(" - ~s: ~p", [Key, Value])
end,
lists:sort(maps:keys(Context))).
context_to_app_env_vars(Context) ->
rabbit_log_prelaunch:debug(
"Setting default application environment variables:"),
?LOG_DEBUG(
"Setting default application environment variables:",
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Fun = fun({App, Param, Value}) ->
rabbit_log_prelaunch:debug(
" - ~s:~s = ~p", [App, Param, Value]),
?LOG_DEBUG(
" - ~s:~s = ~p", [App, Param, Value],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
ok = application:set_env(
App, Param, Value, [{persistent, true}])
end,
@ -623,6 +643,12 @@ parse_log_levels([CategoryValue | Rest], Result) ->
["-color"] ->
Result1 = Result#{color => false},
parse_log_levels(Rest, Result1);
["+json"] ->
Result1 = Result#{json => true},
parse_log_levels(Rest, Result1);
["-json"] ->
Result1 = Result#{json => false},
parse_log_levels(Rest, Result1);
[CategoryOrLevel] ->
case parse_level(CategoryOrLevel) of
undefined ->
@ -677,8 +703,14 @@ main_log_file(#{nodename := Nodename,
File= filename:join(LogBaseDir,
atom_to_list(Nodename) ++ ".log"),
update_context(Context, main_log_file, File, default);
"-" ->
update_context(Context, main_log_file, "-", environment);
"-" = Value ->
update_context(Context, main_log_file, Value, environment);
"-stderr" = Value ->
update_context(Context, main_log_file, Value, environment);
"exchange:" ++ _ = Value ->
update_context(Context, main_log_file, Value, environment);
"syslog:" ++ _ = Value ->
update_context(Context, main_log_file, Value, environment);
Value ->
File = normalize_path(Value),
update_context(Context, main_log_file, File, environment)
@ -1186,9 +1218,10 @@ amqp_tcp_port(Context) ->
update_context(Context, amqp_tcp_port, TcpPort, environment)
catch
_:badarg ->
rabbit_log_prelaunch:error(
"Invalid value for $RABBITMQ_NODE_PORT: ~p",
[TcpPortStr]),
?LOG_ERROR(
"Invalid value for $RABBITMQ_NODE_PORT: ~p",
[TcpPortStr],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({exit, ex_config})
end
end.
@ -1205,9 +1238,10 @@ erlang_dist_tcp_port(#{amqp_tcp_port := AmqpTcpPort} = Context) ->
erlang_dist_tcp_port, TcpPort, environment)
catch
_:badarg ->
rabbit_log_prelaunch:error(
"Invalid value for $RABBITMQ_DIST_PORT: ~p",
[TcpPortStr]),
?LOG_ERROR(
"Invalid value for $RABBITMQ_DIST_PORT: ~p",
[TcpPortStr],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
throw({exit, ex_config})
end
end.
@ -1425,8 +1459,9 @@ load_conf_env_file(#{os_type := {unix, _},
true ->
case filelib:is_regular(ConfEnvFile) of
false ->
rabbit_log_prelaunch:debug(
"No $RABBITMQ_CONF_ENV_FILE (~ts)", [ConfEnvFile]),
?LOG_DEBUG(
"No $RABBITMQ_CONF_ENV_FILE (~ts)", [ConfEnvFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Context1;
true ->
case os:find_executable("sh") of
@ -1437,9 +1472,10 @@ load_conf_env_file(#{os_type := {unix, _},
end
end;
false ->
rabbit_log_prelaunch:debug(
"Loading of $RABBITMQ_CONF_ENV_FILE (~ts) is disabled",
[ConfEnvFile]),
?LOG_DEBUG(
"Loading of $RABBITMQ_CONF_ENV_FILE (~ts) is disabled",
[ConfEnvFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Context1
end;
load_conf_env_file(#{os_type := {win32, _},
@ -1457,8 +1493,9 @@ load_conf_env_file(#{os_type := {win32, _},
true ->
case filelib:is_regular(ConfEnvFile) of
false ->
rabbit_log_prelaunch:debug(
"No $RABBITMQ_CONF_ENV_FILE (~ts)", [ConfEnvFile]),
?LOG_DEBUG(
"No $RABBITMQ_CONF_ENV_FILE (~ts)", [ConfEnvFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Context1;
true ->
case os:find_executable("cmd.exe") of
@ -1478,9 +1515,10 @@ load_conf_env_file(#{os_type := {win32, _},
end
end;
false ->
rabbit_log_prelaunch:debug(
"Loading of $RABBITMQ_CONF_ENV_FILE (~ts) is disabled",
[ConfEnvFile]),
?LOG_DEBUG(
"Loading of $RABBITMQ_CONF_ENV_FILE (~ts) is disabled",
[ConfEnvFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Context1
end;
load_conf_env_file(Context) ->
@ -1502,8 +1540,9 @@ loading_conf_env_file_enabled(_) ->
-endif.
do_load_conf_env_file(#{os_type := {unix, _}} = Context, Sh, ConfEnvFile) ->
rabbit_log_prelaunch:debug(
"Sourcing $RABBITMQ_CONF_ENV_FILE: ~ts", [ConfEnvFile]),
?LOG_DEBUG(
"Sourcing $RABBITMQ_CONF_ENV_FILE: ~ts", [ConfEnvFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
%% The script below sources the `CONF_ENV_FILE` file, then it shows a
%% marker line and all environment variables.
@ -1545,8 +1584,9 @@ do_load_conf_env_file(#{os_type := {unix, _}} = Context, Sh, ConfEnvFile) ->
collect_conf_env_file_output(Context, Port, Marker, <<>>);
do_load_conf_env_file(#{os_type := {win32, _}} = Context, Cmd, ConfEnvFile) ->
%% rabbitmq/rabbitmq-common#392
rabbit_log_prelaunch:debug(
"Executing $RABBITMQ_CONF_ENV_FILE: ~ts", [ConfEnvFile]),
?LOG_DEBUG(
"Executing $RABBITMQ_CONF_ENV_FILE: ~ts", [ConfEnvFile],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
%% The script below executes the `CONF_ENV_FILE` file, then it shows a
%% marker line and all environment variables.
@ -1607,17 +1647,20 @@ collect_conf_env_file_output(Context, Port, Marker, Output) ->
end.
post_port_cmd_output(#{os_type := {OSType, _}}, Output, ExitStatus) ->
rabbit_log_prelaunch:debug(
"$RABBITMQ_CONF_ENV_FILE exit status: ~b",
[ExitStatus]),
?LOG_DEBUG(
"$RABBITMQ_CONF_ENV_FILE exit status: ~b",
[ExitStatus],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
DecodedOutput = unicode:characters_to_list(Output),
LineSep = case OSType of
win32 -> "\r\n";
_ -> "\n"
end,
Lines = string:split(string:trim(DecodedOutput), LineSep, all),
rabbit_log_prelaunch:debug("$RABBITMQ_CONF_ENV_FILE output:"),
[rabbit_log_prelaunch:debug(" ~ts", [Line]) || Line <- Lines],
?LOG_DEBUG(
"$RABBITMQ_CONF_ENV_FILE output:~n~s",
string:join([io_lib:format(" ~ts", [Line]) || Line <- Lines], "\n"),
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
Lines.
parse_conf_env_file_output(Context, _, []) ->
@ -1637,9 +1680,10 @@ parse_conf_env_file_output1(Context, Lines) ->
IsSet = var_is_set(Var),
case IsUsed andalso not IsSet of
true ->
rabbit_log_prelaunch:debug(
"$RABBITMQ_CONF_ENV_FILE: re-exporting variable $~s",
[Var]),
?LOG_DEBUG(
"$RABBITMQ_CONF_ENV_FILE: re-exporting variable $~s",
[Var],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
os:putenv(Var, maps:get(Var, Vars));
false ->
ok
@ -1665,9 +1709,10 @@ parse_conf_env_file_output2([Line | Lines], Vars) ->
parse_conf_env_file_output2(Lines1, Vars1);
_ ->
%% Parsing failed somehow.
rabbit_log_prelaunch:warning(
"Failed to parse $RABBITMQ_CONF_ENV_FILE output: ~p",
[Line]),
?LOG_WARNING(
"Failed to parse $RABBITMQ_CONF_ENV_FILE output: ~p",
[Line],
#{domain => ?RMQLOG_DOMAIN_PRELAUNCH}),
#{}
end
end.

View File

@ -7,7 +7,7 @@
-module(rabbit_log).
-export([log/2, log/3, log/4]).
-export([log/3, log/4]).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
@ -17,23 +17,14 @@
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-export([make_internal_sink_name/1]).
-include("rabbit_log.hrl").
-include("logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
%%----------------------------------------------------------------------------
-type category() :: channel |
connection |
federation |
feature_flags |
ldap |
mirroring |
osiris |
prelaunch |
queue |
ra |
shovel |
upgrade.
-type category() :: atom().
-spec debug(string()) -> 'ok'.
-spec debug(string(), [any()]) -> 'ok'.
@ -65,100 +56,63 @@
%%----------------------------------------------------------------------------
-spec log(category(), lager:log_level(), string()) -> 'ok'.
-spec log(category(), logger:level(), string()) -> 'ok'.
log(Category, Level, Fmt) -> log(Category, Level, Fmt, []).
-spec log(category(), lager:log_level(), string(), [any()]) -> 'ok'.
-spec log(category(), logger:level(), string(), [any()]) -> 'ok'.
log(default, Level, Fmt, Args) when is_list(Args) ->
logger:log(Level, Fmt, Args, #{domain => ?RMQLOG_DOMAIN_GLOBAL});
log(Category, Level, Fmt, Args) when is_list(Args) ->
Sink = case Category of
default -> ?LAGER_SINK;
_ -> make_internal_sink_name(Category)
end,
lager:log(Sink, Level, self(), Fmt, Args).
%% logger(3) handler.
log(#{level := Level,
msg := Msg,
meta := #{pid := Pid}} = _LogEvent,
_Config) ->
case Msg of
{report, #{label := {error_logger, _}}} ->
%% Avoid recursive loop.
ok;
{report, #{label := {application_controller, progress}}} ->
%% Already logged by Lager.
ok;
{report, #{label := {supervisor, progress}}} ->
%% Already logged by Lager.
ok;
{report, #{report := Report}} ->
%% FIXME: Is this code reached?
error_logger:info_report(Report);
{report, #{format := Format, args := Args}} when is_list(Format) ->
lager:log(?LAGER_SINK, Level, Pid, Format, Args);
{string, String} ->
lager:log(?LAGER_SINK, Level, Pid, "~ts", [String]);
{Format, Args} when is_list(Format) ->
lager:log(?LAGER_SINK, Level, Pid, Format, Args)
end.
make_internal_sink_name(channel) -> rabbit_log_channel_lager_event;
make_internal_sink_name(connection) -> rabbit_log_connection_lager_event;
make_internal_sink_name(default) -> rabbit_log_lager_event;
make_internal_sink_name(feature_flags) -> rabbit_log_feature_flags_lager_event;
make_internal_sink_name(federation) -> rabbit_log_federation_lager_event;
make_internal_sink_name(ldap) -> rabbit_log_ldap_lager_event;
make_internal_sink_name(mirroring) -> rabbit_log_mirroring_lager_event;
make_internal_sink_name(osiris) -> rabbit_log_osiris_lager_event;
make_internal_sink_name(prelaunch) -> rabbit_log_prelaunch_lager_event;
make_internal_sink_name(queue) -> rabbit_log_queue_lager_event;
make_internal_sink_name(ra) -> rabbit_log_ra_lager_event;
make_internal_sink_name(shovel) -> rabbit_log_shovel_lager_event;
make_internal_sink_name(upgrade) -> rabbit_log_upgrade_lager_event;
make_internal_sink_name(Category) ->
erlang:error({unknown_category, Category}).
logger:log(Level, Fmt, Args, #{domain => ?DEFINE_RMQLOG_DOMAIN(Category)}).
debug(Format) -> debug(Format, []).
debug(Format, Args) -> debug(self(), Format, Args).
debug(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, debug, Metadata, Format, Args).
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
info(Format) -> info(Format, []).
info(Format, Args) -> info(self(), Format, Args).
info(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, info, Metadata, Format, Args).
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
notice(Format) -> notice(Format, []).
notice(Format, Args) -> notice(self(), Format, Args).
notice(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, notice, Metadata, Format, Args).
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
warning(Format) -> warning(Format, []).
warning(Format, Args) -> warning(self(), Format, Args).
warning(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, warning, Metadata, Format, Args).
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
error(Format) -> ?MODULE:error(Format, []).
error(Format, Args) -> ?MODULE:error(self(), Format, Args).
error(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, error, Metadata, Format, Args).
error(Format) -> error(Format, []).
error(Format, Args) -> error(self(), Format, Args).
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
critical(Format) -> critical(Format, []).
critical(Format, Args) -> critical(self(), Format, Args).
critical(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, critical, Metadata, Format, Args).
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
alert(Format) -> alert(Format, []).
alert(Format, Args) -> alert(self(), Format, Args).
alert(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, alert, Metadata, Format, Args).
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
emergency(Format) -> emergency(Format, []).
emergency(Format, Args) -> emergency(self(), Format, Args).
emergency(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, emergency, Metadata, Format, Args).
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_GLOBAL}).
none(Format) -> none(Format, []).
none(Format, Args) -> none(self(), Format, Args).
none(Metadata, Format, Args) ->
lager:log(?LAGER_SINK, none, Metadata, Format, Args).
none(_Format) -> ok.
none(_Format, _Args) -> ok.
none(_Pid, _Format, _Args) -> ok.

View File

@ -1,26 +0,0 @@
%% The contents of this file are subject to the Mozilla Public License
%% Version 1.1 (the "License"); you may not use this file except in
%% compliance with the License. You may obtain a copy of the License
%% at https://www.mozilla.org/MPL/
%%
%% Software distributed under the License is distributed on an "AS IS"
%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
%% the License for the specific language governing rights and
%% limitations under the License.
%%
%% The Original Code is RabbitMQ.
%%
%% The Initial Developer of the Original Code is GoPivotal, Inc.
%% Copyright (c) 2007-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_log_osiris_shim).
%% just a shim to redirect logs from ra to rabbit_log
-export([log/4]).
-spec log(lager:log_level(), string(), [any()], _) -> ok.
log(Level, Format, Args, _Meta) ->
rabbit_log:log(osiris, Level, Format, Args),
ok.

View File

@ -1,16 +0,0 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2007-2021 VMware, Inc. or its affiliates. All rights reserved.
%%
-module(rabbit_log_ra_shim).
%% just a shim to redirect logs from ra to rabbit_log
-export([log/4]).
log(Level, Format, Args, _Meta) ->
rabbit_log:log(ra, Level, Format, Args),
ok.

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -0,0 +1,3 @@
-include_lib("rabbit_common/include/logging.hrl").
-define(RMQLOG_DOMAIN_LDAP, ?DEFINE_RMQLOG_DOMAIN(ldap)).

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -0,0 +1,107 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_ldap).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include("include/logging.hrl").
-compile({no_auto_import, [error/2]}).
%%----------------------------------------------------------------------------
-spec debug(string()) -> 'ok'.
-spec debug(string(), [any()]) -> 'ok'.
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec info(string()) -> 'ok'.
-spec info(string(), [any()]) -> 'ok'.
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec notice(string()) -> 'ok'.
-spec notice(string(), [any()]) -> 'ok'.
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec warning(string()) -> 'ok'.
-spec warning(string(), [any()]) -> 'ok'.
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec error(string()) -> 'ok'.
-spec error(string(), [any()]) -> 'ok'.
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec critical(string()) -> 'ok'.
-spec critical(string(), [any()]) -> 'ok'.
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec alert(string()) -> 'ok'.
-spec alert(string(), [any()]) -> 'ok'.
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec emergency(string()) -> 'ok'.
-spec emergency(string(), [any()]) -> 'ok'.
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec none(string()) -> 'ok'.
-spec none(string(), [any()]) -> 'ok'.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
%%----------------------------------------------------------------------------
debug(Format) -> debug(Format, []).
debug(Format, Args) -> debug(self(), Format, Args).
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
info(Format) -> info(Format, []).
info(Format, Args) -> info(self(), Format, Args).
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
notice(Format) -> notice(Format, []).
notice(Format, Args) -> notice(self(), Format, Args).
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
warning(Format) -> warning(Format, []).
warning(Format, Args) -> warning(self(), Format, Args).
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
error(Format) -> error(Format, []).
error(Format, Args) -> error(self(), Format, Args).
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
critical(Format) -> critical(Format, []).
critical(Format, Args) -> critical(self(), Format, Args).
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
alert(Format) -> alert(Format, []).
alert(Format, Args) -> alert(self(), Format, Args).
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
emergency(Format) -> emergency(Format, []).
emergency(Format, Args) -> emergency(self(), Format, Args).
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_LDAP}).
none(_Format) -> ok.
none(_Format, _Args) -> ok.
none(_Pid, _Format, _Args) -> ok.

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -8,7 +8,7 @@ defmodule RabbitMQ.CLI.Core.LogFiles do
@spec get_log_locations(atom, integer | :infinity) :: [String.t] | {:badrpc, term}
def get_log_locations(node_name, timeout) do
case :rabbit_misc.rpc_call(node_name,
:rabbit_lager, :log_locations, [],
:rabbit, :log_locations, [],
timeout) do
{:badrpc, _} = error -> error;
list -> Enum.map(list, &to_string/1)
@ -28,7 +28,6 @@ defmodule RabbitMQ.CLI.Core.LogFiles do
location ->
case Enum.member?(log_locations, location) do
true -> {:ok, to_string(location)};
## Configured location was not propagated to lager?
false -> {:ok, first_log}
end
end

View File

@ -5,7 +5,7 @@
## Copyright (c) 2007-2021 VMware, Inc. or its affiliates. All rights reserved.
defmodule RabbitMQ.CLI.Ctl.Commands.RotateLogsCommand do
alias RabbitMQ.CLI.Core.DocGuide
alias RabbitMQ.CLI.Core.{DocGuide, ExitCodes}
@behaviour RabbitMQ.CLI.CommandBehaviour
@ -13,8 +13,12 @@ defmodule RabbitMQ.CLI.Ctl.Commands.RotateLogsCommand do
use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
def run([], %{node: node_name}) do
:rabbit_misc.rpc_call(node_name, :rabbit, :rotate_logs, [])
def run([], _) do
{
:error,
ExitCodes.exit_unavailable(),
"This command does not rotate logs anymore [deprecated]"
}
end
use RabbitMQ.CLI.DefaultOutput
@ -28,7 +32,7 @@ defmodule RabbitMQ.CLI.Ctl.Commands.RotateLogsCommand do
def help_section(), do: :node_management
def description(), do: "Instructs the RabbitMQ node to perform internal log rotation"
def description(), do: "Does nothing [deprecated]"
def banner(_, %{node: node_name}), do: "Rotating logs for node #{node_name} ..."
def banner(_, _), do: nil
end

View File

@ -42,7 +42,7 @@ defmodule RabbitMQ.CLI.Ctl.Commands.SetLogLevelCommand do
def run([log_level], %{node: node_name}) do
arg = String.to_atom(log_level)
:rabbit_misc.rpc_call(node_name, :rabbit_lager, :set_log_level, [arg])
:rabbit_misc.rpc_call(node_name, :rabbit, :set_log_level, [arg])
end
def usage, do: "set_log_level <log_level>"

View File

@ -36,7 +36,6 @@ defmodule RabbitMQCtl.MixfileBase do
:rabbit_event,
:rabbit_file,
:rabbit_net,
:rabbit_lager,
:rabbit_log,
:rabbit_misc,
:rabbit_mnesia,
@ -157,18 +156,6 @@ defmodule RabbitMQCtl.MixfileBase do
compile: false,
override: true
},
{
:goldrush,
path: Path.join(deps_dir, "goldrush"),
compile: false,
override: true
},
{
:lager,
path: Path.join(deps_dir, "lager"),
compile: false,
override: true
},
{
:amqp_client,
path: Path.join(deps_dir, "amqp_client"),

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 1.7.1

View File

@ -1,40 +0,0 @@
## This Source Code Form is subject to the terms of the Mozilla Public
## License, v. 2.0. If a copy of the MPL was not distributed with this
## file, You can obtain one at https://mozilla.org/MPL/2.0/.
##
## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
defmodule RotateLogsCommandTest do
use ExUnit.Case, async: false
import TestHelper
@command RabbitMQ.CLI.Ctl.Commands.RotateLogsCommand
setup_all do
RabbitMQ.CLI.Core.Distribution.start()
:ok
end
setup do
{:ok, opts: %{node: get_rabbit_hostname()}}
end
test "validate: with extra arguments returns an arg count error", context do
assert @command.validate(["extra"], context[:opts]) == {:validation_failure, :too_many_args}
end
test "run: request to a named, active node succeeds", context do
assert @command.run([], context[:opts]) == :ok
end
test "run: request to a non-existent node returns a badrpc" do
opts = %{node: :jake@thedog, timeout: 200}
assert match?({:badrpc, _}, @command.run([], opts))
end
test "banner", context do
assert @command.banner([], context[:opts]) =~ ~r/Rotating logs for node #{get_rabbit_hostname()}/
end
end

View File

@ -50,49 +50,26 @@ defmodule LogLocationCommandTest do
end
test "run: prints default log location", context do
# Let Lager's log message rate lapse or else some messages
# we assert on might be dropped. MK.
Process.sleep(1000)
{:ok, logfile} = @command.run([], context[:opts])
log_message = "file location"
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [log_message])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist(log_message)])
wait_for_log_message(log_message, logfile)
{:ok, log_file_data} = File.read(logfile)
assert String.match?(log_file_data, Regex.compile!(log_message))
end
test "run: shows all log locations", context do
# Let Lager's log message rate lapse or else some messages
# we assert on might be dropped. MK.
Process.sleep(1000)
# This assumes default configuration
[logfile, upgrade_log_file] =
[logfile, upgrade_log_file | _] =
@command.run([], Map.merge(context[:opts], %{all: true}))
log_message = "checking the default log file when checking all"
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [log_message])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist(log_message)])
wait_for_log_message(log_message, logfile)
log_message_upgrade = "checking the upgrade log file when checking all"
:rpc.call(get_rabbit_hostname(),
:rabbit_log, :log, [:upgrade, :error, log_message_upgrade, []])
:rabbit_log, :log, [:upgrade, :error, to_charlist(log_message_upgrade), []])
wait_for_log_message(log_message_upgrade, upgrade_log_file)
end
test "run: fails if there is no log file configured", context do
{:ok, upgrade_file} = :rpc.call(get_rabbit_hostname(), :application, :get_env, [:rabbit, :lager_upgrade_file])
{:ok, default_file} = :rpc.call(get_rabbit_hostname(), :application, :get_env, [:rabbit, :lager_default_file])
on_exit([], fn ->
:rpc.call(get_rabbit_hostname(), :application, :set_env, [:rabbit, :lager_upgrade_file, upgrade_file])
:rpc.call(get_rabbit_hostname(), :application, :set_env, [:rabbit, :lager_default_file, default_file])
:rpc.call(get_rabbit_hostname(), :rabbit_lager, :configure_lager, [])
start_rabbitmq_app()
end)
stop_rabbitmq_app()
:rpc.call(get_rabbit_hostname(), :application, :unset_env, [:rabbit, :lager_upgrade_file])
:rpc.call(get_rabbit_hostname(), :application, :unset_env, [:rabbit, :lager_default_file])
:rpc.call(get_rabbit_hostname(), :application, :unset_env, [:rabbit, :log])
:rpc.call(get_rabbit_hostname(), :rabbit_lager, :configure_lager, [])
{:error, "No log files configured on the node"} = @command.run([], context[:opts])
end
end

View File

@ -50,15 +50,12 @@ defmodule LogTailCommandTest do
end
test "run: shows last 50 lines from the log by default", context do
# Let Lager's log message rate lapse or else some messages
# we assert on might be dropped. MK.
Process.sleep(1000)
clear_log_files()
log_messages =
Enum.map(:lists.seq(1, 50),
fn(n) ->
message = "Getting log tail #{n}"
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [message])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist(message)])
message
end)
wait_for_log_message("Getting log tail 50")
@ -72,15 +69,11 @@ defmodule LogTailCommandTest do
end
test "run: returns N lines", context do
# Let Lager's log message rate lapse or else some messages
# we assert on might be dropped. MK.
Process.sleep(1000)
## Log a bunch of lines
Enum.map(:lists.seq(1, 50),
fn(n) ->
message = "More lines #{n}"
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [message])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist(message)])
message
end)
wait_for_log_message("More lines 50")
@ -90,15 +83,12 @@ defmodule LogTailCommandTest do
end
test "run: may return less than N lines if N is high", context do
# Let Lager's log message rate lapse or else some messages
# we assert on might be dropped. MK.
Process.sleep(1000)
clear_log_files()
## Log a bunch of lines
Enum.map(:lists.seq(1, 100),
fn(n) ->
message = "More lines #{n}"
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [message])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist(message)])
message
end)
wait_for_log_message("More lines 50")
@ -107,9 +97,12 @@ defmodule LogTailCommandTest do
end
def clear_log_files() do
[_|_] = logs = :rpc.call(get_rabbit_hostname(), :rabbit_lager, :log_locations, [])
[_|_] = logs = :rpc.call(get_rabbit_hostname(), :rabbit, :log_locations, [])
Enum.map(logs, fn(log) ->
File.write(log, "")
case log do
'<stdout>' -> :ok
_ -> File.write(log, "")
end
end)
end
end

View File

@ -54,10 +54,10 @@ defmodule LogTailStreamCommandTest do
time_before = System.system_time(:second)
stream = @command.run([], Map.merge(context[:opts], %{duration: 15}))
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, ["Message"])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, ["Message1"])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, ["Message2"])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, ["Message3"])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist("Message")])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist("Message1")])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist("Message2")])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist("Message3")])
# This may take a long time and fail with an ExUnit timeout
data = Enum.join(stream)
@ -81,7 +81,7 @@ defmodule LogTailStreamCommandTest do
end
def ensure_log_file() do
[log|_] = :rpc.call(get_rabbit_hostname(), :rabbit_lager, :log_locations, [])
[log|_] = :rpc.call(get_rabbit_hostname(), :rabbit, :log_locations, [])
ensure_file(log, 100)
end
@ -92,14 +92,14 @@ defmodule LogTailStreamCommandTest do
case File.exists?(log) do
true -> :ok
false ->
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, ["Ping"])
:rpc.call(get_rabbit_hostname(), :rabbit_log, :error, [to_charlist("Ping")])
:timer.sleep(100)
ensure_file(log, attempts - 1)
end
end
def delete_log_files() do
[_|_] = logs = :rpc.call(get_rabbit_hostname(), :rabbit_lager, :log_locations, [])
[_|_] = logs = :rpc.call(get_rabbit_hostname(), :rabbit, :log_locations, [])
Enum.map(logs, fn(log) ->
File.rm(log)
end)

View File

@ -608,7 +608,7 @@ defmodule TestHelper do
## Assume default log is the first one
log_file = case file do
nil ->
[default_log | _] = :rpc.call(get_rabbit_hostname(), :rabbit_lager, :log_locations, [])
[default_log | _] = :rpc.call(get_rabbit_hostname(), :rabbit, :log_locations, [])
default_log
_ -> file
end

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -0,0 +1,3 @@
-include_lib("rabbit_common/include/logging.hrl").
-define(RMQLOG_DOMAIN_FEDERATION, ?DEFINE_RMQLOG_DOMAIN(federation)).

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -0,0 +1,107 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_federation).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include("include/logging.hrl").
-compile({no_auto_import, [error/2, error/3]}).
%%----------------------------------------------------------------------------
-spec debug(string()) -> 'ok'.
-spec debug(string(), [any()]) -> 'ok'.
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec info(string()) -> 'ok'.
-spec info(string(), [any()]) -> 'ok'.
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec notice(string()) -> 'ok'.
-spec notice(string(), [any()]) -> 'ok'.
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec warning(string()) -> 'ok'.
-spec warning(string(), [any()]) -> 'ok'.
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec error(string()) -> 'ok'.
-spec error(string(), [any()]) -> 'ok'.
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec critical(string()) -> 'ok'.
-spec critical(string(), [any()]) -> 'ok'.
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec alert(string()) -> 'ok'.
-spec alert(string(), [any()]) -> 'ok'.
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec emergency(string()) -> 'ok'.
-spec emergency(string(), [any()]) -> 'ok'.
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec none(string()) -> 'ok'.
-spec none(string(), [any()]) -> 'ok'.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
%%----------------------------------------------------------------------------
debug(Format) -> debug(Format, []).
debug(Format, Args) -> debug(self(), Format, Args).
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
info(Format) -> info(Format, []).
info(Format, Args) -> info(self(), Format, Args).
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
notice(Format) -> notice(Format, []).
notice(Format, Args) -> notice(self(), Format, Args).
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
warning(Format) -> warning(Format, []).
warning(Format, Args) -> warning(self(), Format, Args).
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
error(Format) -> error(Format, []).
error(Format, Args) -> error(self(), Format, Args).
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
critical(Format) -> critical(Format, []).
critical(Format, Args) -> critical(self(), Format, Args).
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
alert(Format) -> alert(Format, []).
alert(Format, Args) -> alert(self(), Format, Args).
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
emergency(Format) -> emergency(Format, []).
emergency(Format, Args) -> emergency(self(), Format, Args).
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_FEDERATION}).
none(_Format) -> ok.
none(_Format, _Args) -> ok.
none(_Pid, _Format, _Args) -> ok.

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -0,0 +1,3 @@
-include_lib("rabbit_common/include/logging.hrl").
-define(RMQLOG_DOMAIN_SHOVEL, ?DEFINE_RMQLOG_DOMAIN(shovel)).

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -0,0 +1,107 @@
%% This Source Code Form is subject to the terms of the Mozilla Public
%% License, v. 2.0. If a copy of the MPL was not distributed with this
%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
%%
%% Copyright (c) 2021 VMware, Inc. or its affiliates. All rights reserved.
%%
%% @doc Compatibility module for the old Lager-based logging API.
-module(rabbit_log_shovel).
-export([debug/1, debug/2, debug/3,
info/1, info/2, info/3,
notice/1, notice/2, notice/3,
warning/1, warning/2, warning/3,
error/1, error/2, error/3,
critical/1, critical/2, critical/3,
alert/1, alert/2, alert/3,
emergency/1, emergency/2, emergency/3,
none/1, none/2, none/3]).
-include("include/logging.hrl").
-compile({no_auto_import, [error/2]}).
%%----------------------------------------------------------------------------
-spec debug(string()) -> 'ok'.
-spec debug(string(), [any()]) -> 'ok'.
-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec info(string()) -> 'ok'.
-spec info(string(), [any()]) -> 'ok'.
-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec notice(string()) -> 'ok'.
-spec notice(string(), [any()]) -> 'ok'.
-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec warning(string()) -> 'ok'.
-spec warning(string(), [any()]) -> 'ok'.
-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec error(string()) -> 'ok'.
-spec error(string(), [any()]) -> 'ok'.
-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec critical(string()) -> 'ok'.
-spec critical(string(), [any()]) -> 'ok'.
-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec alert(string()) -> 'ok'.
-spec alert(string(), [any()]) -> 'ok'.
-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec emergency(string()) -> 'ok'.
-spec emergency(string(), [any()]) -> 'ok'.
-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
-spec none(string()) -> 'ok'.
-spec none(string(), [any()]) -> 'ok'.
-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
%%----------------------------------------------------------------------------
debug(Format) -> debug(Format, []).
debug(Format, Args) -> debug(self(), Format, Args).
debug(Pid, Format, Args) ->
logger:debug(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
info(Format) -> info(Format, []).
info(Format, Args) -> info(self(), Format, Args).
info(Pid, Format, Args) ->
logger:info(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
notice(Format) -> notice(Format, []).
notice(Format, Args) -> notice(self(), Format, Args).
notice(Pid, Format, Args) ->
logger:notice(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
warning(Format) -> warning(Format, []).
warning(Format, Args) -> warning(self(), Format, Args).
warning(Pid, Format, Args) ->
logger:warning(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
error(Format) -> error(Format, []).
error(Format, Args) -> error(self(), Format, Args).
error(Pid, Format, Args) ->
logger:error(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
critical(Format) -> critical(Format, []).
critical(Format, Args) -> critical(self(), Format, Args).
critical(Pid, Format, Args) ->
logger:critical(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
alert(Format) -> alert(Format, []).
alert(Format, Args) -> alert(self(), Format, Args).
alert(Pid, Format, Args) ->
logger:alert(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
emergency(Format) -> emergency(Format, []).
emergency(Format, Args) -> emergency(self(), Format, Args).
emergency(Pid, Format, Args) ->
logger:emergency(Format, Args, #{pid => Pid,
domain => ?RMQLOG_DOMAIN_SHOVEL}).
none(_Format) -> ok.
none(_Format, _Args) -> ok.
none(_Pid, _Format, _Args) -> ok.

View File

@ -45,9 +45,7 @@ groups() ->
init_per_suite(Config0) ->
{ok, _} = application:ensure_all_started(amqp10_client),
rabbit_ct_helpers:log_environment(),
Config = rabbit_ct_helpers:merge_app_env(Config0,
[{lager, [{error_logger_hwm, 200}]}]),
Config1 = rabbit_ct_helpers:set_config(Config, [
Config1 = rabbit_ct_helpers:set_config(Config0, [
{rmq_nodename_suffix, ?MODULE}
]),
rabbit_ct_helpers:run_setup_steps(Config1,

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

View File

@ -117,7 +117,6 @@ dep_accept = hex 0.3.5
dep_cowboy = hex 2.8.0
dep_cowlib = hex 2.9.1
dep_jsx = hex 2.11.0
dep_lager = hex 3.9.1
dep_prometheus = git https://github.com/deadtrickster/prometheus.erl.git master
dep_ra = git https://github.com/rabbitmq/ra.git master
dep_ranch = hex 2.0.0

Some files were not shown because too many files have changed in this diff Show More