aboutsummaryrefslogtreecommitdiffstats
path: root/lib/kernel
diff options
context:
space:
mode:
authorSiri Hansen <[email protected]>2018-10-31 15:37:04 +0100
committerSiri Hansen <[email protected]>2019-01-16 09:59:24 +0100
commit2929e79806b0e8ffdd4be5c7eaed0cea04bce850 (patch)
tree05785e6d5de014d93af9a755a0e37011c18da3df /lib/kernel
parent7000e101745cc028e4656eaaae3ea5a1a4498747 (diff)
downloadotp-2929e79806b0e8ffdd4be5c7eaed0cea04bce850.tar.gz
otp-2929e79806b0e8ffdd4be5c7eaed0cea04bce850.tar.bz2
otp-2929e79806b0e8ffdd4be5c7eaed0cea04bce850.zip
[logger] Split overload protection functionality to own module
Diffstat (limited to 'lib/kernel')
-rw-r--r--lib/kernel/src/Makefile1
-rw-r--r--lib/kernel/src/logger_h_common.erl663
-rw-r--r--lib/kernel/src/logger_olp.erl558
-rw-r--r--lib/kernel/test/logger.cover4
-rw-r--r--lib/kernel/test/logger_disk_log_h_SUITE.erl83
-rw-r--r--lib/kernel/test/logger_std_h_SUITE.erl76
6 files changed, 804 insertions, 581 deletions
diff --git a/lib/kernel/src/Makefile b/lib/kernel/src/Makefile
index 57f17defc8..c076726bf4 100644
--- a/lib/kernel/src/Makefile
+++ b/lib/kernel/src/Makefile
@@ -118,6 +118,7 @@ MODULES = \
logger_h_common \
logger_filters \
logger_formatter \
+ logger_olp \
logger_server \
logger_simple_h \
logger_sup \
diff --git a/lib/kernel/src/logger_h_common.erl b/lib/kernel/src/logger_h_common.erl
index 74a2d158fc..dd8ace8249 100644
--- a/lib/kernel/src/logger_h_common.erl
+++ b/lib/kernel/src/logger_h_common.erl
@@ -24,11 +24,11 @@
-include("logger_internal.hrl").
%% API
--export([start_link/1, info/2, filesync/2, reset/2]).
+-export([info/2, filesync/2, reset/2]).
-%% gen_server and proc_lib callbacks
--export([init/1, handle_call/3, handle_cast/2, handle_info/2,
- terminate/2, code_change/3]).
+%% logger_olp callbacks
+-export([init/1, handle_load/2, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3, notify/2, reset_state/1]).
%% logger callbacks
-export([log/2, adding_handler/1, removing_handler/1, changing_config/3,
@@ -37,27 +37,25 @@
%% Library functions for handlers
-export([error_notify/1]).
-%%%-----------------------------------------------------------------
--define(CONFIG_KEYS,[sync_mode_qlen,
- drop_mode_qlen,
- flush_qlen,
- burst_limit_enable,
- burst_limit_max_count,
- burst_limit_window_time,
- overload_kill_enable,
- overload_kill_qlen,
- overload_kill_mem_size,
- overload_kill_restart_after,
- filesync_repeat_interval]).
--define(READ_ONLY_KEYS,[handler_pid,mode_tab]).
+-define(OLP_KEYS,[sync_mode_qlen,
+ drop_mode_qlen,
+ flush_qlen,
+ burst_limit_enable,
+ burst_limit_max_count,
+ burst_limit_window_time,
+ overload_kill_enable,
+ overload_kill_qlen,
+ overload_kill_mem_size,
+ overload_kill_restart_after]).
+
+-define(COMMON_KEYS,[filesync_repeat_interval]).
+
+-define(READ_ONLY_KEYS,[olp]).
%%%-----------------------------------------------------------------
%%% API
%% This function is called by the logger_sup supervisor
-start_link(Args) ->
- proc_lib:start_link(?MODULE,init,[Args]).
-
filesync(Module, Name) ->
call(Module, Name, filesync).
@@ -71,18 +69,19 @@ reset(Module, Name) ->
%%% Handler being added
adding_handler(#{id:=Name,module:=Module}=Config) ->
HConfig0 = maps:get(config, Config, #{}),
- HandlerConfig0 = maps:without(?CONFIG_KEYS,HConfig0),
+ HandlerConfig0 = maps:without(?OLP_KEYS++?COMMON_KEYS,HConfig0),
case Module:check_config(Name,set,undefined,HandlerConfig0) of
{ok,HandlerConfig} ->
- ModifiedCommon = maps:with(?CONFIG_KEYS,HandlerConfig),
- CommonConfig0 = maps:with(?CONFIG_KEYS,HConfig0),
+ ModifiedCommon = maps:with(?COMMON_KEYS,HandlerConfig),
+ CommonConfig0 = maps:with(?COMMON_KEYS,HConfig0),
CommonConfig = maps:merge(
maps:merge(get_default_config(), CommonConfig0),
ModifiedCommon),
case check_config(CommonConfig) of
ok ->
HConfig = maps:merge(CommonConfig,HandlerConfig),
- start(Config#{config => HConfig});
+ OlpOpts = maps:with(?OLP_KEYS,HConfig0),
+ start(OlpOpts, Config#{config => HConfig});
{error,Faulty} ->
{error,{invalid_config,Module,Faulty}}
end;
@@ -92,11 +91,11 @@ adding_handler(#{id:=Name,module:=Module}=Config) ->
%%%-----------------------------------------------------------------
%%% Handler being removed
-removing_handler(#{id:=Name, module:=Module}) ->
+removing_handler(#{id:=Name, module:=Module, config:=#{olp:=Olp}}) ->
case whereis(?name_to_reg_name(Module,Name)) of
undefined ->
ok;
- Pid ->
+ _Pid ->
%% We don't want to do supervisor:terminate_child here
%% since we need to distinguish this explicit stop from a
%% system termination in order to avoid circular attempts
@@ -106,7 +105,7 @@ removing_handler(#{id:=Name, module:=Module}) ->
%% the restart type is temporary, which means that the
%% child specification is automatically removed from the
%% supervisor when the process dies.
- _ = gen_server:call(Pid, stop),
+ _ = logger_olp:stop(Olp),
ok
end.
@@ -116,34 +115,52 @@ changing_config(SetOrUpdate,
#{id:=Name,config:=OldHConfig,module:=Module},
NewConfig0) ->
NewHConfig0 = maps:get(config, NewConfig0, #{}),
- OldHandlerConfig = maps:without(?CONFIG_KEYS++?READ_ONLY_KEYS,OldHConfig),
- NewHandlerConfig0 = maps:without(?CONFIG_KEYS++?READ_ONLY_KEYS,NewHConfig0),
+ NoHandlerKeys = ?OLP_KEYS++?COMMON_KEYS++?READ_ONLY_KEYS,
+ OldHandlerConfig = maps:without(NoHandlerKeys,OldHConfig),
+ NewHandlerConfig0 = maps:without(NoHandlerKeys,NewHConfig0),
case Module:check_config(Name, SetOrUpdate,
OldHandlerConfig,NewHandlerConfig0) of
{ok, NewHandlerConfig} ->
- ModifiedCommon = maps:with(?CONFIG_KEYS,NewHandlerConfig),
- NewCommonConfig0 = maps:with(?CONFIG_KEYS,NewHConfig0),
+ ModifiedCommon = maps:with(?COMMON_KEYS,NewHandlerConfig),
+ NewCommonConfig0 = maps:with(?COMMON_KEYS,NewHConfig0),
+ OldCommonConfig = maps:with(?COMMON_KEYS,OldHConfig),
CommonDefault =
case SetOrUpdate of
set ->
get_default_config();
update ->
- maps:with(?CONFIG_KEYS,OldHConfig)
+ OldCommonConfig
end,
NewCommonConfig = maps:merge(
maps:merge(CommonDefault,NewCommonConfig0),
ModifiedCommon),
case check_config(NewCommonConfig) of
ok ->
- ReadOnly = maps:with(?READ_ONLY_KEYS,OldHConfig),
- NewHConfig = maps:merge(
- maps:merge(NewCommonConfig,NewHandlerConfig),
- ReadOnly),
- NewConfig = NewConfig0#{config=>NewHConfig},
- HPid = maps:get(handler_pid,OldHConfig),
- case call(HPid, {change_config,NewConfig}) of
- ok -> {ok,NewConfig};
- Error -> Error
+ OlpDefault =
+ case SetOrUpdate of
+ set ->
+ logger_olp:get_default_opts();
+ update ->
+ maps:with(?OLP_KEYS,OldHConfig)
+ end,
+ Olp = maps:get(olp,OldHConfig),
+ NewOlpOpts = maps:merge(OlpDefault,
+ maps:with(?OLP_KEYS,NewHConfig0)),
+ case logger_olp:set_opts(Olp,NewOlpOpts) of
+ ok ->
+ maybe_set_repeated_filesync(Olp,OldCommonConfig,
+ NewCommonConfig),
+ ReadOnly = maps:with(?READ_ONLY_KEYS,OldHConfig),
+ NewHConfig =
+ maps:merge(
+ maps:merge(
+ maps:merge(NewCommonConfig,NewHandlerConfig),
+ ReadOnly),
+ NewOlpOpts),
+ NewConfig = NewConfig0#{config=>NewHConfig},
+ {ok,NewConfig};
+ Error ->
+ Error
end;
{error,Faulty} ->
{error,{invalid_config,Module,Faulty}}
@@ -158,14 +175,12 @@ changing_config(SetOrUpdate,
LogEvent :: logger:log_event(),
Config :: logger:handler_config().
-log(LogEvent, Config = #{id := Name,
- config := #{handler_pid := HPid,
- mode_tab := ModeTab}}) ->
+log(LogEvent, Config = #{config := #{olp:=Olp}}) ->
%% if the handler has crashed, we must drop this event
%% and hope the handler restarts so we can try again
- true = is_process_alive(HPid),
+ true = logger_olp:is_alive(Olp),
Bin = log_to_binary(LogEvent, Config),
- call_cast_or_drop(Name, HPid, ModeTab, Bin).
+ logger_olp:load(Olp,Bin).
%%%-----------------------------------------------------------------
%%% Remove internal fields from configuration
@@ -180,18 +195,22 @@ filter_config(#{config:=HConfig}=Config) ->
%%%
%%% The handler process is linked to logger_sup, which is part of the
%%% kernel application's supervision tree.
-start(#{id := Name} = Config0) ->
+start(OlpOpts0, #{id := Name, module:=Module, config:=HConfig} = Config0) ->
+ RegName = ?name_to_reg_name(Module,Name),
ChildSpec =
#{id => Name,
- start => {?MODULE, start_link, [Config0]},
+ start => {logger_olp, start_link, [RegName,?MODULE,
+ Config0, OlpOpts0]},
restart => temporary,
shutdown => 2000,
type => worker,
modules => [?MODULE]},
case supervisor:start_child(logger_sup, ChildSpec) of
- {ok,Pid,Config} ->
+ {ok,Pid,{Olp,OlpOpts}} ->
ok = logger_handler_watcher:register_handler(Name,Pid),
- {ok,Config};
+ {ok,Config0#{config=>(maps:merge(HConfig,OlpOpts))#{olp=>Olp}}};
+ {error,{Reason,Ch}} when is_tuple(Ch), element(1,Ch)==child ->
+ {error,Reason};
Error ->
Error
end.
@@ -200,103 +219,48 @@ start(#{id := Name} = Config0) ->
%%% gen_server callbacks
%%%===================================================================
-init(#{id := Name, module := Module,
- formatter := Formatter, config := HConfig0} = Config0) ->
- RegName = ?name_to_reg_name(Module,Name),
- register(RegName, self()),
+init(#{id := Name, module := Module, config := HConfig}) ->
process_flag(trap_exit, true),
- process_flag(message_queue_data, off_heap),
- ?init_test_hooks(),
- ?start_observation(Name),
-
- case Module:init(Name, HConfig0) of
+ case Module:init(Name, HConfig) of
{ok,HState} ->
- try ets:new(Name, [public]) of
- ModeTab ->
- ?set_mode(ModeTab, async),
- T0 = ?timestamp(),
- HConfig = HConfig0#{handler_pid => self(),
- mode_tab => ModeTab},
- Config = Config0#{config => HConfig},
- proc_lib:init_ack({ok,self(),Config}),
- %% Storing common config in state to avoid copying
- %% (sending) the config data for each log message
- CommonConfig = maps:with(?CONFIG_KEYS,HConfig),
- State =
- ?merge_with_stats(
- CommonConfig#{id => Name,
- module => Module,
- mode_tab => ModeTab,
- mode => async,
- ctrl_sync_count =>
- ?CONTROLLER_SYNC_INTERVAL,
- last_qlen => 0,
- last_log_ts => T0,
- last_op => sync,
- burst_win_ts => T0,
- burst_msg_count => 0,
- formatter => Formatter,
- handler_state => HState}),
- State1 = set_repeated_filesync(State),
- unset_restart_flag(State1),
- gen_server:enter_loop(?MODULE, [], State1)
- catch
- _:Error ->
- unregister(RegName),
- error_notify({init_handler,Name,Error}),
- proc_lib:init_ack(Error)
- end;
+ %% Storing common config in state to avoid copying
+ %% (sending) the config data for each log message
+ CommonConfig = maps:with(?COMMON_KEYS,HConfig),
+ State = CommonConfig#{id => Name,
+ module => Module,
+ ctrl_sync_count =>
+ ?CONTROLLER_SYNC_INTERVAL,
+ last_op => sync,
+ handler_state => HState},
+ State1 = set_repeated_filesync(State),
+ {ok,State1};
Error ->
- unregister(RegName),
- error_notify({init_handler,Name,Error}),
- proc_lib:init_ack(Error)
+ Error
end.
-%% This is the synchronous log event.
-handle_call({log, Bin}, _From, State) ->
- {Result,State1} = do_log(Bin, call, State),
- %% Result == ok | dropped
- {reply,Result, State1};
+%% This is the log event.
+handle_load(Bin, #{id:=Name,
+ module:=Module,
+ handler_state:=HandlerState,
+ ctrl_sync_count := CtrlSync}=State) ->
+ if CtrlSync==0 ->
+ {_,HS1} = Module:write(Name, sync, Bin, HandlerState),
+ {ok,State#{handler_state => HS1,
+ ctrl_sync_count => ?CONTROLLER_SYNC_INTERVAL,
+ last_op=>write}};
+ true ->
+ {_,HS1} = Module:write(Name, async, Bin, HandlerState),
+ {ok,State#{handler_state => HS1,
+ ctrl_sync_count => CtrlSync-1,
+ last_op=>write}}
+ end.
handle_call(filesync, _From, State = #{id := Name,
module := Module,
handler_state := HandlerState}) ->
{Result,HandlerState1} = Module:filesync(Name,sync,HandlerState),
- {reply, Result, State#{handler_state=>HandlerState1, last_op=>sync}};
-
-handle_call({change_config, #{formatter:=Formatter, config:=NewHConfig}}, _From,
- State = #{filesync_repeat_interval := FSyncInt0}) ->
- %% In the future, if handler_state must be updated due to config
- %% change, then we need to add a callback to Module here.
- CommonConfig = maps:with(?CONFIG_KEYS,NewHConfig),
- State1 = maps:merge(State, CommonConfig),
- State2 =
- case maps:get(filesync_repeat_interval, NewHConfig) of
- FSyncInt0 ->
- State1;
- _FSyncInt1 ->
- set_repeated_filesync(cancel_repeated_filesync(State1))
- end,
- {reply, ok, State2#{formatter:=Formatter}};
-
-handle_call(info, _From, State) ->
- {reply, State, State};
-
-handle_call(reset, _From,
- #{id:=Name,module:=Module,handler_state:=HandlerState}=State) ->
- State1 = ?merge_with_stats(State),
- {reply, ok, State1#{last_qlen => 0,
- last_log_ts => ?timestamp(),
- handler_state => Module:reset_state(Name,HandlerState)}};
-
-handle_call(stop, _From, State) ->
- {stop, {shutdown,stopped}, ok, State}.
-
-%% This is the asynchronous log event.
-handle_cast({log, Bin}, State) ->
- {_,State1} = do_log(Bin, cast, State),
- {noreply, State1};
+ {reply, Result, State#{handler_state=>HandlerState1, last_op=>sync}}.
%% If FILESYNC_REPEAT_INTERVAL is set to a millisec value, this
%% clause gets called repeatedly by the handler. In order to
@@ -319,168 +283,83 @@ handle_cast(repeated_filesync,
{_,HS} = Module:filesync(Name, async, HandlerState),
State#{handler_state => HS, last_op => sync}
end,
- {noreply,set_repeated_filesync(State1)}.
+ {noreply,set_repeated_filesync(State1)};
+
+handle_cast({set_repeated_filesync,FSyncInt},State) ->
+ State1 = State#{filesync_repeat_interval=>FSyncInt},
+ State2 = set_repeated_filesync(cancel_repeated_filesync(State1)),
+ {noreply, State2}.
handle_info(Info, #{id := Name, module := Module,
handler_state := HandlerState} = State) ->
{noreply,State#{handler_state => Module:handle_info(Name,Info,HandlerState)}}.
-terminate(Reason, State = #{id := Name,
- module := Module,
- handler_state := HandlerState}) ->
+terminate(overloaded=Reason, #{id:=Name}=State) ->
+ log_handler_info(Name, "Handler ~p overloaded and stopping", [Name], State),
+ do_terminate(Reason,State),
+ ConfigResult = logger:get_handler_config(Name),
+ case ConfigResult of
+ {ok,#{module:=Module}=HConfig0} ->
+ spawn(fun() -> logger:remove_handler(Name) end),
+ HConfig = try Module:filter_config(HConfig0)
+ catch _:_ -> HConfig0
+ end,
+ {ok,fun() -> logger:add_handler(Name,Module,HConfig) end};
+ Error ->
+ error_notify({Name,restart_impossible,Error}),
+ Error
+ end;
+terminate(Reason, State) ->
+ do_terminate(Reason, State).
+
+do_terminate(Reason, State = #{id := Name,
+ module := Module,
+ handler_state := HandlerState}) ->
_ = cancel_repeated_filesync(State),
_ = Module:terminate(Name, Reason, HandlerState),
- ok = stop_or_restart(Name, Reason, State),
- unregister(?name_to_reg_name(Module, Name)),
ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
+reset_state(#{id:=Name, module:=Module, handler_state:=HandlerState} = State) ->
+ State#{handler_state=>Module:reset_state(Name, HandlerState)}.
%%%-----------------------------------------------------------------
%%% Internal functions
call(Module, Name, Op) when is_atom(Name) ->
- call(?name_to_reg_name(Module,Name), Op);
+ case logger_olp:call(?name_to_reg_name(Module,Name), Op) of
+ {error,busy} -> {error,handler_busy};
+ Other -> Other
+ end;
call(_, Name, Op) ->
{error,{badarg,{Op,[Name]}}}.
-call(Server, Msg) ->
- try
- gen_server:call(Server, Msg, ?DEFAULT_CALL_TIMEOUT)
- catch
- _:{timeout,_} -> {error,handler_busy}
- end.
-
-%% check for overload between every event (and set Mode to async,
-%% sync or drop accordingly), but never flush the whole mailbox
-%% before LogWindowSize events have been handled
-do_log(Bin, CallOrCast, State = #{id:=Name, mode:=Mode0}) ->
- T1 = ?timestamp(),
-
- %% check if the handler is getting overloaded, or if it's
- %% recovering from overload (the check must be done for each
- %% event to react quickly to large bursts of events and
- %% to ensure that the handler can never end up in drop mode
- %% with an empty mailbox, which would stop operation)
- {Mode1,QLen,Mem,State1} = check_load(State),
-
- if (Mode1 == drop) andalso (Mode0 =/= drop) ->
- log_handler_info(Name, "Handler ~p switched to drop mode",
- [Name], State);
- (Mode0 == drop) andalso ((Mode1 == async) orelse (Mode1 == sync)) ->
- log_handler_info(Name, "Handler ~p switched to ~w mode",
- [Name,Mode1], State);
- true ->
- ok
- end,
-
- %% kill the handler if it can't keep up with the load
- kill_if_choked(Name, QLen, Mem, State),
-
- if Mode1 == flush ->
- flush(Name, QLen, T1, State1);
- true ->
- write(Name, Mode1, T1, Bin, CallOrCast, State1)
- end.
-
-%% this clause is called by do_log/3 after an overload check
-%% has been performed, where QLen > FlushQLen
-flush(Name, _QLen0, T1, State=#{last_log_ts := _T0, mode_tab := ModeTab}) ->
- %% flush messages in the mailbox (a limited number in
- %% order to not cause long delays)
- NewFlushed = flush_log_events(?FLUSH_MAX_N),
-
- %% write info in log about flushed messages
+notify({mode_change,Mode0,Mode1},#{id:=Name}=State) ->
+ log_handler_info(Name,"Handler ~p switched from ~p to ~p mode",
+ [Name,Mode0,Mode1], State);
+notify({flushed,Flushed},#{id:=Name}=State) ->
log_handler_info(Name, "Handler ~p flushed ~w log events",
- [Name,NewFlushed], State),
-
- %% because of the receive loop when flushing messages, the
- %% handler will be scheduled out often and the mailbox could
- %% grow very large, so we'd better check the queue again here
- {_,_QLen1} = process_info(self(), message_queue_len),
- ?observe(Name,{max_qlen,_QLen1}),
-
- %% Add 1 for the current log event
- ?observe(Name,{flushed,NewFlushed+1}),
-
- State1 = ?update_max_time(?diff_time(T1,_T0),State),
- State2 = ?update_max_qlen(_QLen1,State1),
- {dropped,?update_other(flushed,FLUSHED,NewFlushed,
- State2#{mode => ?set_mode(ModeTab,async),
- last_qlen => 0,
- last_log_ts => T1})}.
-
-%% this clause is called to write to file
-write(Name, Mode, T1, Bin, _CallOrCast,
- State = #{module := Module,
- handler_state := HandlerState,
- mode_tab := ModeTab,
- ctrl_sync_count := CtrlSync,
- last_qlen := LastQLen,
- last_log_ts := T0}) ->
- %% check if we need to limit the number of writes
- %% during a burst of log events
- {DoWrite,State1} = limit_burst(State),
-
- %% only log synhrounously every ?CONTROLLER_SYNC_INTERVAL time, to
- %% give the handler time between writes so it can keep up with
- %% incoming messages
- {Result,LastQLen1,HandlerState1} =
- if DoWrite, CtrlSync == 0 ->
- ?observe(Name,{_CallOrCast,1}),
- {_,HS1} = Module:write(Name, sync, Bin, HandlerState),
- {ok,element(2, process_info(self(), message_queue_len)),HS1};
- DoWrite ->
- ?observe(Name,{_CallOrCast,1}),
- {_,HS1} = Module:write(Name, async, Bin, HandlerState),
- {ok,LastQLen,HS1};
- not DoWrite ->
- ?observe(Name,{flushed,1}),
- {dropped,LastQLen,HandlerState}
- end,
-
- %% Check if the time since the previous log event is long enough -
- %% and the queue length small enough - to assume the mailbox has
- %% been emptied, and if so, do filesync operation and reset mode to
- %% async. Note that this is the best we can do to detect an idle
- %% handler without setting a timer after each log call/cast. If the
- %% time between two consecutive log events is fast and no new
- %% event comes in after the last one, idle state won't be detected!
- Time = ?diff_time(T1,T0),
- State2 =
- if (LastQLen1 < ?FILESYNC_OK_QLEN) andalso
- (Time > ?IDLE_DETECT_TIME_USEC) ->
- {_,HS2} = Module:filesync(Name,async,HandlerState),
- State1#{mode => ?change_mode(ModeTab, Mode, async),
- burst_msg_count => 0,
- handler_state => HS2};
- true ->
- State1#{mode => Mode, handler_state => HandlerState1}
- end,
- State3 = ?update_calls_or_casts(_CallOrCast,1,State2),
- State4 = ?update_max_qlen(LastQLen1,State3),
- State5 =
- ?update_max_time(Time,
- State4#{last_qlen := LastQLen1,
- last_log_ts => T1,
- last_op => write,
- ctrl_sync_count =>
- if CtrlSync==0 -> ?CONTROLLER_SYNC_INTERVAL;
- true -> CtrlSync-1
- end}),
- {Result,State5}.
+ [Name,Flushed], State);
+notify(restart,#{id:=Name}=State) ->
+ log_handler_info(Name, "Handler ~p restarted", [Name], State);
+notify(idle,#{id:=Name,module:=Module,handler_state:=HandlerState}=State) ->
+ {_,HS} = Module:filesync(Name,async,HandlerState),
+ State#{handler_state=>HS, last_op=>sync}.
log_handler_info(Name, Format, Args, #{module:=Module,
- formatter:=Formatter,
- handler_state:=HandlerState}) ->
- Config = #{formatter=>Formatter},
+ handler_state:=HandlerState}=State) ->
+ Config =
+ case logger:get_handler_config(Name) of
+ {ok,Conf} -> Conf;
+ _ -> #{formatter=>{?DEFAULT_FORMATTER,?DEFAULT_FORMAT_CONFIG}}
+ end,
Meta = #{time=>erlang:system_time(microsecond)},
Bin = log_to_binary(#{level => notice,
msg => {Format,Args},
meta => Meta}, Config),
- _ = Module:write(Name, async, Bin, HandlerState),
- ok.
+ {_,HS} = Module:write(Name, async, Bin, HandlerState),
+ State#{handler_state=>HS, last_op=>write}.
%%%-----------------------------------------------------------------
%%% Convert log data on any form to binary
@@ -540,42 +419,8 @@ string_to_binary(String) ->
%%%-----------------------------------------------------------------
%%% Check that the configuration term is valid
check_config(Config) when is_map(Config) ->
- case check_common_config(maps:to_list(Config)) of
- ok ->
- case overload_levels_ok(Config) of
- true ->
- ok;
- false ->
- Faulty = maps:with([sync_mode_qlen,
- drop_mode_qlen,
- flush_qlen],Config),
- {error,{invalid_levels,Faulty}}
- end;
- Error ->
- Error
- end.
+ check_common_config(maps:to_list(Config)).
-check_common_config([{sync_mode_qlen,N}|Config]) when is_integer(N) ->
- check_common_config(Config);
-check_common_config([{drop_mode_qlen,N}|Config]) when is_integer(N) ->
- check_common_config(Config);
-check_common_config([{flush_qlen,N}|Config]) when is_integer(N) ->
- check_common_config(Config);
-check_common_config([{burst_limit_enable,Bool}|Config]) when is_boolean(Bool) ->
- check_common_config(Config);
-check_common_config([{burst_limit_max_count,N}|Config]) when is_integer(N) ->
- check_common_config(Config);
-check_common_config([{burst_limit_window_time,N}|Config]) when is_integer(N) ->
- check_common_config(Config);
-check_common_config([{overload_kill_enable,Bool}|Config]) when is_boolean(Bool) ->
- check_common_config(Config);
-check_common_config([{overload_kill_qlen,N}|Config]) when is_integer(N) ->
- check_common_config(Config);
-check_common_config([{overload_kill_mem_size,N}|Config]) when is_integer(N) ->
- check_common_config(Config);
-check_common_config([{overload_kill_restart_after,NorA}|Config])
- when is_integer(NorA); NorA == infinity ->
- check_common_config(Config);
check_common_config([{filesync_repeat_interval,NorA}|Config])
when is_integer(NorA); NorA == no_repeat ->
check_common_config(Config);
@@ -585,156 +430,7 @@ check_common_config([]) ->
ok.
get_default_config() ->
- #{sync_mode_qlen => ?SYNC_MODE_QLEN,
- drop_mode_qlen => ?DROP_MODE_QLEN,
- flush_qlen => ?FLUSH_QLEN,
- burst_limit_enable => ?BURST_LIMIT_ENABLE,
- burst_limit_max_count => ?BURST_LIMIT_MAX_COUNT,
- burst_limit_window_time => ?BURST_LIMIT_WINDOW_TIME,
- overload_kill_enable => ?OVERLOAD_KILL_ENABLE,
- overload_kill_qlen => ?OVERLOAD_KILL_QLEN,
- overload_kill_mem_size => ?OVERLOAD_KILL_MEM_SIZE,
- overload_kill_restart_after => ?OVERLOAD_KILL_RESTART_AFTER,
- filesync_repeat_interval => ?FILESYNC_REPEAT_INTERVAL}.
-
-%%%-----------------------------------------------------------------
-%%% Overload Protection
-call_cast_or_drop(_Name, HandlerPid, ModeTab, Bin) ->
- %% If the handler process is getting overloaded, the log event
- %% will be synchronous instead of asynchronous (slows down the
- %% logging tempo of a process doing lots of logging. If the
- %% handler is choked, drop mode is set and no event will be sent.
- try ?get_mode(ModeTab) of
- async ->
- gen_server:cast(HandlerPid, {log,Bin});
- sync ->
- case call(HandlerPid, {log,Bin}) of
- ok ->
- ok;
- _Other ->
- %% dropped or {error,handler_busy}
- ?observe(_Name,{dropped,1}),
- ok
- end;
- drop ->
- ?observe(_Name,{dropped,1})
- catch
- %% if the ETS table doesn't exist (maybe because of a
- %% handler restart), we can only drop the event
- _:_ -> ?observe(_Name,{dropped,1})
- end,
- ok.
-
-set_restart_flag(#{id := Name, module := Module} = State) ->
- log_handler_info(Name, "Handler ~p overloaded and stopping", [Name], State),
- Flag = list_to_atom(lists:concat([Module,"_",Name,"_restarting"])),
- spawn(fun() ->
- register(Flag, self()),
- timer:sleep(infinity)
- end),
- ok.
-
-unset_restart_flag(#{id := Name, module := Module} = State) ->
- Flag = list_to_atom(lists:concat([Module,"_",Name,"_restarting"])),
- case whereis(Flag) of
- undefined ->
- ok;
- Pid ->
- exit(Pid, kill),
- log_handler_info(Name, "Handler ~p restarted", [Name], State)
- end.
-
-check_load(State = #{id:=_Name, mode_tab := ModeTab, mode := Mode,
- sync_mode_qlen := SyncModeQLen,
- drop_mode_qlen := DropModeQLen,
- flush_qlen := FlushQLen}) ->
- {_,Mem} = process_info(self(), memory),
- ?observe(_Name,{max_mem,Mem}),
- {_,QLen} = process_info(self(), message_queue_len),
- ?observe(_Name,{max_qlen,QLen}),
- %% When the handler process gets scheduled in, it's impossible
- %% to predict the QLen. We could jump "up" arbitrarily from say
- %% async to sync, async to drop, sync to flush, etc. However, when
- %% the handler process manages the log events (without flushing),
- %% one after the other, we will move "down" from drop to sync and
- %% from sync to async. This way we don't risk getting stuck in
- %% drop or sync mode with an empty mailbox.
- {Mode1,_NewDrops,_NewFlushes} =
- if
- QLen >= FlushQLen ->
- {flush, 0,1};
- QLen >= DropModeQLen ->
- %% Note that drop mode will force log events to
- %% be dropped on the client side (never sent get to
- %% the handler).
- IncDrops = if Mode == drop -> 0; true -> 1 end,
- {?change_mode(ModeTab, Mode, drop), IncDrops,0};
- QLen >= SyncModeQLen ->
- {?change_mode(ModeTab, Mode, sync), 0,0};
- true ->
- {?change_mode(ModeTab, Mode, async), 0,0}
- end,
- State1 = ?update_other(drops,DROPS,_NewDrops,State),
- {Mode1, QLen, Mem,
- ?update_other(flushes,FLUSHES,_NewFlushes,
- State1#{last_qlen => QLen})}.
-
-limit_burst(#{burst_limit_enable := false}=State) ->
- {true,State};
-limit_burst(#{burst_win_ts := BurstWinT0,
- burst_msg_count := BurstMsgCount,
- burst_limit_window_time := BurstLimitWinTime,
- burst_limit_max_count := BurstLimitMaxCnt} = State) ->
- if (BurstMsgCount >= BurstLimitMaxCnt) ->
- %% the limit for allowed messages has been reached
- BurstWinT1 = ?timestamp(),
- case ?diff_time(BurstWinT1,BurstWinT0) of
- BurstCheckTime when BurstCheckTime < (BurstLimitWinTime*1000) ->
- %% we're still within the burst time frame
- {false,?update_other(burst_drops,BURSTS,1,State)};
- _BurstCheckTime ->
- %% burst time frame passed, reset counters
- {true,State#{burst_win_ts => BurstWinT1,
- burst_msg_count => 0}}
- end;
- true ->
- %% the limit for allowed messages not yet reached
- {true,State#{burst_win_ts => BurstWinT0,
- burst_msg_count => BurstMsgCount+1}}
- end.
-
-kill_if_choked(Name, QLen, Mem, State = #{overload_kill_enable := KillIfOL,
- overload_kill_qlen := OLKillQLen,
- overload_kill_mem_size := OLKillMem}) ->
- if KillIfOL andalso
- ((QLen > OLKillQLen) orelse (Mem > OLKillMem)) ->
- set_restart_flag(State),
- exit({shutdown,{overloaded,Name,QLen,Mem}});
- true ->
- ok
- end.
-
-flush_log_events(Limit) ->
- process_flag(priority, high),
- Flushed = flush_log_events(0, Limit),
- process_flag(priority, normal),
- Flushed.
-
-flush_log_events(Limit, Limit) ->
- Limit;
-flush_log_events(N, Limit) ->
- %% flush log events but leave other events, such as
- %% filesync, info and change_config, so that these
- %% have a chance to be processed even under heavy load
- receive
- {'$gen_cast',{log,_}} ->
- flush_log_events(N+1, Limit);
- {'$gen_call',{Pid,MRef},{log,_}} ->
- Pid ! {MRef, dropped},
- flush_log_events(N+1, Limit)
- after
- 0 -> N
- end.
+ #{filesync_repeat_interval => ?FILESYNC_REPEAT_INTERVAL}.
set_repeated_filesync(#{filesync_repeat_interval:=FSyncInt} = State)
when is_integer(FSyncInt) ->
@@ -752,51 +448,12 @@ cancel_repeated_filesync(State) ->
error ->
State
end.
-
-stop_or_restart(Name, {shutdown,Reason={overloaded,_Name,_QLen,_Mem}},
- #{overload_kill_restart_after := RestartAfter}) ->
- %% If we're terminating because of an overload situation (see
- %% kill_if_choked/4), we need to remove the handler and set a
- %% restart timer. A separate process must perform this in order to
- %% avoid deadlock.
- HandlerPid = self(),
- ConfigResult = logger:get_handler_config(Name),
- RemoveAndRestart =
- fun() ->
- MRef = erlang:monitor(process, HandlerPid),
- receive
- {'DOWN',MRef,_,_,_} ->
- ok
- after 30000 ->
- error_notify(Reason),
- exit(HandlerPid, kill)
- end,
- case ConfigResult of
- {ok,#{module:=HMod}=HConfig0} when is_integer(RestartAfter) ->
- _ = logger:remove_handler(Name),
- HConfig = try HMod:filter_config(HConfig0)
- catch _:_ -> HConfig0
- end,
- _ = timer:apply_after(RestartAfter, logger, add_handler,
- [Name,HMod,HConfig]);
- {ok,_} ->
- _ = logger:remove_handler(Name);
- {error,CfgReason} when is_integer(RestartAfter) ->
- error_notify({Name,restart_impossible,CfgReason});
- {error,_} ->
- ok
- end
- end,
- spawn(RemoveAndRestart),
- ok;
-stop_or_restart(_Name, _Reason, _State) ->
- ok.
-
-overload_levels_ok(HandlerConfig) ->
- SMQL = maps:get(sync_mode_qlen, HandlerConfig, ?SYNC_MODE_QLEN),
- DMQL = maps:get(drop_mode_qlen, HandlerConfig, ?DROP_MODE_QLEN),
- FQL = maps:get(flush_qlen, HandlerConfig, ?FLUSH_QLEN),
- (DMQL > 1) andalso (SMQL =< DMQL) andalso (DMQL =< FQL).
-
error_notify(Term) ->
?internal_log(error, Term).
+
+maybe_set_repeated_filesync(_Olp,
+ #{filesync_repeat_interval:=FSyncInt},
+ #{filesync_repeat_interval:=FSyncInt}) ->
+ ok;
+maybe_set_repeated_filesync(Olp,_,#{filesync_repeat_interval:=FSyncInt}) ->
+ logger_olp:cast(Olp,{set_repeated_filesync,FSyncInt}).
diff --git a/lib/kernel/src/logger_olp.erl b/lib/kernel/src/logger_olp.erl
new file mode 100644
index 0000000000..6b76c78c73
--- /dev/null
+++ b/lib/kernel/src/logger_olp.erl
@@ -0,0 +1,558 @@
+%%
+%% %CopyrightBegin%
+%%
+%% Copyright Ericsson AB 2017-2018. 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(logger_olp).
+-behaviour(gen_server).
+
+-include("logger_h_common.hrl").
+-include("logger_internal.hrl").
+
+%% API
+-export([start_link/4, load/2, info/1, reset/1, stop/1, restart/1,
+ set_opts/2, get_opts/1, get_default_opts/0, is_alive/1,
+ call/2, cast/2]).
+
+%% gen_server and proc_lib callbacks
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3]).
+
+%%%-----------------------------------------------------------------
+%% -define(CONFIG_KEYS,[sync_mode_qlen,
+%% drop_mode_qlen,
+%% flush_qlen,
+%% burst_limit_enable,
+%% burst_limit_max_count,
+%% burst_limit_window_time,
+%% overload_kill_enable,
+%% overload_kill_qlen,
+%% overload_kill_mem_size,
+%% overload_kill_restart_after]).
+
+%%%-----------------------------------------------------------------
+%%% API
+
+%-spec start_link(Name,Module,Args,Options) -> {ok,Pid,Olp} | {error,Reason}.
+start_link(Name,Module,Args,Options0) when is_map(Options0) ->
+ Options = maps:merge(get_default_opts(),Options0),
+ case check_opts(Options) of
+ ok ->
+ case proc_lib:start_link(?MODULE,init,
+ [[Name,Module,Args,Options]]) of
+ {ok,Pid,Olp} ->
+ {ok,Pid,{Olp,Options}};
+ Error ->
+ Error
+ end;
+ Error ->
+ Error
+ end.
+
+is_alive({_Name,Pid,_ModeRef}) ->
+ is_process_alive(Pid).
+
+load({_Name,Pid,ModeRef},Msg) ->
+ %% If the process is getting overloaded, the message will be
+ %% synchronous instead of asynchronous (slows down the tempo of a
+ %% process causing much load). If the process is choked, drop mode
+ %% is set and no message is sent.
+ try ?get_mode(ModeRef) of
+ async ->
+ gen_server:cast(Pid, {'$olp_load',Msg});
+ sync ->
+ case call(Pid, {'$olp_load',Msg}) of
+ ok ->
+ ok;
+ _Other ->
+ %% dropped or {error,busy}
+ ?observe(_Name,{dropped,1}),
+ ok
+ end;
+ drop ->
+ ?observe(_Name,{dropped,1})
+ catch
+ %% if the ETS table doesn't exist (maybe because of a
+ %% process restart), we can only drop the event
+ _:_ -> ?observe(_Name,{dropped,1})
+ end,
+ ok.
+
+info(Olp) ->
+ call(Olp, info).
+
+reset(Olp) ->
+ call(Olp, reset).
+
+stop({_Name,Pid,_ModRef}) ->
+ _ = gen_server:call(Pid, stop),
+ ok.
+
+set_opts({_Name,Pid,_ModRef}, Opts) ->
+ gen_server:call(Pid, {set_opts,Opts}).
+
+get_opts({_Name,Pid,_ModRef}) ->
+ gen_server:call(Pid, get_opts).
+
+get_default_opts() ->
+ #{sync_mode_qlen => ?SYNC_MODE_QLEN,
+ drop_mode_qlen => ?DROP_MODE_QLEN,
+ flush_qlen => ?FLUSH_QLEN,
+ burst_limit_enable => ?BURST_LIMIT_ENABLE,
+ burst_limit_max_count => ?BURST_LIMIT_MAX_COUNT,
+ burst_limit_window_time => ?BURST_LIMIT_WINDOW_TIME,
+ overload_kill_enable => ?OVERLOAD_KILL_ENABLE,
+ overload_kill_qlen => ?OVERLOAD_KILL_QLEN,
+ overload_kill_mem_size => ?OVERLOAD_KILL_MEM_SIZE,
+ overload_kill_restart_after => ?OVERLOAD_KILL_RESTART_AFTER}.
+
+restart(Fun) ->
+ erlang:display(restarting),
+ erlang:display(_ = Fun()),
+ ok.
+
+%%%===================================================================
+%%% gen_server callbacks
+%%%===================================================================
+
+init([Name,Module,Args,Options]) ->
+ register(Name, self()),
+ process_flag(message_queue_data, off_heap),
+
+ ?init_test_hooks(),
+ ?start_observation(Name),
+
+ try Module:init(Args) of
+ {ok,CBState} ->
+ try ets:new(Name, [public]) of
+ ModeRef ->
+ ?set_mode(ModeRef, async),
+ T0 = ?timestamp(),
+ proc_lib:init_ack({ok,self(),{Name,self(),ModeRef}}),
+ %% Storing options in state to avoid copying
+ %% (sending) the option data with each message
+ State0 = ?merge_with_stats(
+ Options#{id => Name,
+ module => Module,
+ mode_ref => ModeRef,
+ mode => async,
+ last_qlen => 0,
+ last_load_ts => T0,
+ burst_win_ts => T0,
+ burst_msg_count => 0,
+ cb_state => CBState}),
+ State = reset_restart_flag(State0),
+ gen_server:enter_loop(?MODULE, [], State)
+ catch
+ _:Error ->
+ unregister(Name),
+ proc_lib:init_ack(Error)
+ end;
+ Error ->
+ unregister(Name),
+ proc_lib:init_ack(Error)
+ catch
+ _:Error ->
+ unregister(Name),
+ proc_lib:init_ack(Error)
+ end.
+
+%% This is the synchronous load event.
+handle_call({'$olp_load', Msg}, _From, State) ->
+ {Result,State1} = do_load(Msg, call, State),
+ %% Result == ok | dropped
+ {reply,Result, State1};
+
+handle_call({set_opts,Opts0},_From,State) ->
+ Opts = maps:merge(get_default_opts(),Opts0),
+ case check_opts(Opts) of
+ ok ->
+ {reply, ok, maps:merge(State,Opts)};
+ Error ->
+ {reply, Error, State}
+ end;
+
+handle_call(info, _From, State) ->
+ {reply, State, State};
+
+handle_call(reset, _From, #{module:=Module,cb_state:=CBState}=State) ->
+ State1 = ?merge_with_stats(State),
+ CBState1 = try_callback_call(Module,reset_state,[CBState],CBState),
+ {reply, ok, State1#{last_qlen => 0,
+ last_load_ts => ?timestamp(),
+ cb_state => CBState1}};
+
+handle_call(stop, _From, State) ->
+ {stop, {shutdown,stopped}, ok, State};
+
+handle_call(Msg, From, #{module:=Module,cb_state:=CBState}=State) ->
+ case try_callback_call(Module,handle_call,[Msg, From, CBState]) of
+ {reply,Reply,CBState1} ->
+ {reply,Reply,State#{cb_state=>CBState1}};
+ {reply,Reply,CBState1,Timeout}->
+ {reply,Reply,State#{cb_state=>CBState1},Timeout};
+ {noreply,CBState1} ->
+ {noreply,State#{cb_state=>CBState1}};
+ {noreply,CBState1,Timeout} ->
+ {noreply,State#{cb_state=>CBState1},Timeout}
+ end.
+
+%% This is the asynchronous load event.
+handle_cast({'$olp_load', Msg}, State) ->
+ {_Result,State1} = do_load(Msg, cast, State),
+ %% Result == ok | dropped
+ {noreply,State1};
+
+handle_cast(Msg, #{module:=Module, cb_state:=CBState} = State) ->
+ case try_callback_call(Module,handle_cast,[Msg, CBState]) of
+ {noreply,CBState1} ->
+ {noreply,State#{cb_state=>CBState1}};
+ {noreply,CBState1,Timeout} ->
+ {noreply,State#{cb_state=>CBState1},Timeout}
+ end.
+
+handle_info(Msg, #{module := Module, cb_state := CBState} = State) ->
+ case try_callback_call(Module,handle_info,[Msg, CBState]) of
+ {noreply,CBState1} ->
+ {noreply,State#{cb_state=>CBState1}};
+ {noreply,CBState1,Timeout} ->
+ {noreply,State#{cb_state=>CBState1},Timeout}
+ end.
+
+terminate({shutdown,{overloaded,_QLen,_Mem}},
+ #{id:=Name, module := Module, cb_state := CBState,
+ overload_kill_restart_after := RestartAfter} = State) ->
+ %% We're terminating because of an overload situation (see
+ %% kill_if_choked/3).
+ unregister(Name), %%!!!! to avoid error printout of callback crashed on stop
+ case try_callback_call(Module,terminate,[overloaded,CBState],ok) of
+ {ok,Fun} when is_function(Fun,0), is_integer(RestartAfter) ->
+ set_restart_flag(State),
+ timer:apply_after(RestartAfter,?MODULE,restart,[Fun]),
+ ok;
+ _ ->
+ ok
+ end,
+ ok;
+terminate(Reason, #{id:=Name, module:=Module, cb_state:=CBState}) ->
+ _ = try_callback_call(Module,terminate,[Reason,CBState],ok),
+ unregister(Name),
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+
+%%%-----------------------------------------------------------------
+%%% Internal functions
+call({_Name, Pid, _ModeRef},Msg) ->
+ call(Pid, Msg);
+call(Server, Msg) ->
+ try
+ gen_server:call(Server, Msg, ?DEFAULT_CALL_TIMEOUT)
+ catch
+ _:{timeout,_} -> {error,busy}
+ end.
+
+cast({_Name, Pid, _ModeRef},Msg) ->
+ gen_server:cast(Pid, Msg).
+
+%% check for overload between every event (and set Mode to async,
+%% sync or drop accordingly), but never flush the whole mailbox
+%% before LogWindowSize events have been handled
+do_load(Msg, CallOrCast, State) ->
+ T1 = ?timestamp(),
+
+ %% check if the process is getting overloaded, or if it's
+ %% recovering from overload (the check must be done for each
+ %% event to react quickly to large bursts of events and
+ %% to ensure that the handler can never end up in drop mode
+ %% with an empty mailbox, which would stop operation)
+ {Mode1,QLen,Mem,State1} = check_load(State),
+
+ %% kill the handler if it can't keep up with the load
+ kill_if_choked(QLen, Mem, State1),
+
+ if Mode1 == flush ->
+ flush(T1, State1);
+ true ->
+ handle_load(Mode1, T1, Msg, CallOrCast, State1)
+ end.
+
+%% this function is called by do_load/3 after an overload check
+%% has been performed, where QLen > FlushQLen
+flush(T1, State=#{id := _Name, last_load_ts := T0, mode_ref := ModeRef}) ->
+ %% flush load messages in the mailbox (a limited number in order
+ %% to not cause long delays)
+ NewFlushed = flush_load(?FLUSH_MAX_N),
+
+ %% write info in log about flushed messages
+ State1=notify({flushed,NewFlushed},State),
+
+ %% because of the receive loop when flushing messages, the
+ %% handler will be scheduled out often and the mailbox could
+ %% grow very large, so we'd better check the queue again here
+ {_,_QLen1} = process_info(self(), message_queue_len),
+ ?observe(_Name,{max_qlen,_QLen1}),
+
+ %% Add 1 for the current log event
+ ?observe(_Name,{flushed,NewFlushed+1}),
+
+ State2 = ?update_max_time(?diff_time(T1,T0),State1),
+ State3 = ?update_max_qlen(_QLen1,State2),
+ {dropped,?update_other(flushed,FLUSHED,NewFlushed,
+ State3#{mode => ?set_mode(ModeRef,async),
+ last_qlen => 0,
+ last_load_ts => T1})}.
+
+%% this function is called to actually handle the message
+handle_load(Mode, T1, Msg, _CallOrCast,
+ State = #{id := _Name,
+ module := Module,
+ cb_state := CBState,
+ mode_ref := ModeRef,
+ last_qlen := LastQLen,
+ last_load_ts := T0}) ->
+ %% check if we need to limit the number of writes
+ %% during a burst of log events
+ {DoWrite,State1} = limit_burst(State),
+
+ {Result,LastQLen1,CBState1} =
+ if DoWrite ->
+ ?observe(_Name,{_CallOrCast,1}),
+ {ok,CBS} = try_callback_call(Module,handle_load,[Msg,CBState]),
+ {ok,element(2, process_info(self(), message_queue_len)),CBS};
+ true ->
+ ?observe(_Name,{flushed,1}),
+ {dropped,LastQLen,CBState}
+ end,
+ State2 = State1#{cb_state=>CBState1},
+
+ %% Check if the time since the previous load message is long
+ %% enough - and the queue length small enough - to assume the
+ %% mailbox has been emptied, and if so, reset mode to async. Note
+ %% that this is the best we can do to detect an idle handler
+ %% without setting a timer after each log call/cast. If the time
+ %% between two consecutive log events is fast and no new event
+ %% comes in after the last one, idle state won't be detected!
+ Time = ?diff_time(T1,T0),
+ State3 =
+ if (LastQLen1 < ?FILESYNC_OK_QLEN) andalso
+ (Time > ?IDLE_DETECT_TIME_USEC) ->
+ S = notify(idle,State2),
+ S#{mode => ?change_mode(ModeRef, Mode, async),
+ burst_msg_count => 0};
+ true ->
+ State2#{mode => Mode}
+ end,
+ State4 = ?update_calls_or_casts(_CallOrCast,1,State3),
+ State5 = ?update_max_qlen(LastQLen1,State4),
+ State6 =
+ ?update_max_time(Time,
+ State5#{last_qlen := LastQLen1,
+ last_load_ts => T1}),
+ {Result,State6}.
+
+
+%%%-----------------------------------------------------------------
+%%% Check that the options are valid
+check_opts(Options) when is_map(Options) ->
+ case do_check_opts(maps:to_list(Options)) of
+ ok ->
+ case overload_levels_ok(Options) of
+ true ->
+ ok;
+ false ->
+ Faulty = maps:with([sync_mode_qlen,
+ drop_mode_qlen,
+ flush_qlen],Options),
+ {error,{invalid_olp_levels,Faulty}}
+ end;
+ {error,Key,Value} ->
+ {error,{invalid_olp_config,#{Key=>Value}}}
+ end.
+
+do_check_opts([{sync_mode_qlen,N}|Options]) when is_integer(N) ->
+ do_check_opts(Options);
+do_check_opts([{drop_mode_qlen,N}|Options]) when is_integer(N) ->
+ do_check_opts(Options);
+do_check_opts([{flush_qlen,N}|Options]) when is_integer(N) ->
+ do_check_opts(Options);
+do_check_opts([{burst_limit_enable,Bool}|Options]) when is_boolean(Bool) ->
+ do_check_opts(Options);
+do_check_opts([{burst_limit_max_count,N}|Options]) when is_integer(N) ->
+ do_check_opts(Options);
+do_check_opts([{burst_limit_window_time,N}|Options]) when is_integer(N) ->
+ do_check_opts(Options);
+do_check_opts([{overload_kill_enable,Bool}|Options]) when is_boolean(Bool) ->
+ do_check_opts(Options);
+do_check_opts([{overload_kill_qlen,N}|Options]) when is_integer(N) ->
+ do_check_opts(Options);
+do_check_opts([{overload_kill_mem_size,N}|Options]) when is_integer(N) ->
+ do_check_opts(Options);
+do_check_opts([{overload_kill_restart_after,NorA}|Options])
+ when is_integer(NorA); NorA == infinity ->
+ do_check_opts(Options);
+do_check_opts([{Key,Value}|_]) ->
+ {error,Key,Value};
+do_check_opts([]) ->
+ ok.
+
+set_restart_flag(#{id := Name, module := Module}) ->
+ Flag = list_to_atom(lists:concat([Module,"_",Name,"_restarting"])),
+ spawn(fun() ->
+ register(Flag, self()),
+ timer:sleep(infinity)
+ end),
+ ok.
+
+reset_restart_flag(#{id := Name, module := Module} = State) ->
+ Flag = list_to_atom(lists:concat([Module,"_",Name,"_restarting"])),
+ case whereis(Flag) of
+ undefined ->
+ State;
+ Pid ->
+ exit(Pid, kill),
+ notify(restart,State)
+ end.
+
+check_load(State = #{id:=_Name, mode_ref := ModeRef, mode := Mode,
+ sync_mode_qlen := SyncModeQLen,
+ drop_mode_qlen := DropModeQLen,
+ flush_qlen := FlushQLen}) ->
+ {_,Mem} = process_info(self(), memory),
+ ?observe(_Name,{max_mem,Mem}),
+ {_,QLen} = process_info(self(), message_queue_len),
+ ?observe(_Name,{max_qlen,QLen}),
+ %% When the handler process gets scheduled in, it's impossible
+ %% to predict the QLen. We could jump "up" arbitrarily from say
+ %% async to sync, async to drop, sync to flush, etc. However, when
+ %% the handler process manages the log events (without flushing),
+ %% one after the other, we will move "down" from drop to sync and
+ %% from sync to async. This way we don't risk getting stuck in
+ %% drop or sync mode with an empty mailbox.
+ {Mode1,_NewDrops,_NewFlushes} =
+ if
+ QLen >= FlushQLen ->
+ {flush, 0,1};
+ QLen >= DropModeQLen ->
+ %% Note that drop mode will force load messages to
+ %% be dropped on the client side (never sent to
+ %% the handler).
+ IncDrops = if Mode == drop -> 0; true -> 1 end,
+ {?change_mode(ModeRef, Mode, drop), IncDrops,0};
+ QLen >= SyncModeQLen ->
+ {?change_mode(ModeRef, Mode, sync), 0,0};
+ true ->
+ {?change_mode(ModeRef, Mode, async), 0,0}
+ end,
+ State1 = ?update_other(drops,DROPS,_NewDrops,State),
+ State2 = maybe_notify_mode_change(Mode1,State1),
+ {Mode1, QLen, Mem,
+ ?update_other(flushes,FLUSHES,_NewFlushes,
+ State2#{last_qlen => QLen})}.
+
+limit_burst(#{burst_limit_enable := false}=State) ->
+ {true,State};
+limit_burst(#{burst_win_ts := BurstWinT0,
+ burst_msg_count := BurstMsgCount,
+ burst_limit_window_time := BurstLimitWinTime,
+ burst_limit_max_count := BurstLimitMaxCnt} = State) ->
+ if (BurstMsgCount >= BurstLimitMaxCnt) ->
+ %% the limit for allowed messages has been reached
+ BurstWinT1 = ?timestamp(),
+ case ?diff_time(BurstWinT1,BurstWinT0) of
+ BurstCheckTime when BurstCheckTime < (BurstLimitWinTime*1000) ->
+ %% we're still within the burst time frame
+ {false,?update_other(burst_drops,BURSTS,1,State)};
+ _BurstCheckTime ->
+ %% burst time frame passed, reset counters
+ {true,State#{burst_win_ts => BurstWinT1,
+ burst_msg_count => 0}}
+ end;
+ true ->
+ %% the limit for allowed messages not yet reached
+ {true,State#{burst_win_ts => BurstWinT0,
+ burst_msg_count => BurstMsgCount+1}}
+ end.
+
+kill_if_choked(QLen, Mem, #{overload_kill_enable := KillIfOL,
+ overload_kill_qlen := OLKillQLen,
+ overload_kill_mem_size := OLKillMem}) ->
+ if KillIfOL andalso
+ ((QLen > OLKillQLen) orelse (Mem > OLKillMem)) ->
+ exit({shutdown,{overloaded,QLen,Mem}});
+ true ->
+ ok
+ end.
+
+flush_load(Limit) ->
+ process_flag(priority, high),
+ Flushed = flush_load(0, Limit),
+ process_flag(priority, normal),
+ Flushed.
+
+flush_load(Limit, Limit) ->
+ Limit;
+flush_load(N, Limit) ->
+ %% flush log events but leave other events, such as info, reset
+ %% and stop, so that these have a chance to be processed even
+ %% under heavy load
+ receive
+ {'$gen_cast',{'$olp_load',_}} ->
+ flush_load(N+1, Limit);
+ {'$gen_call',{Pid,MRef},{'$olp_load',_}} ->
+ Pid ! {MRef, dropped},
+ flush_load(N+1, Limit)
+ after
+ 0 -> N
+ end.
+
+overload_levels_ok(Options) ->
+ SMQL = maps:get(sync_mode_qlen, Options, ?SYNC_MODE_QLEN),
+ DMQL = maps:get(drop_mode_qlen, Options, ?DROP_MODE_QLEN),
+ FQL = maps:get(flush_qlen, Options, ?FLUSH_QLEN),
+ (DMQL > 1) andalso (SMQL =< DMQL) andalso (DMQL =< FQL).
+
+maybe_notify_mode_change(drop,#{mode:=Mode0}=State)
+ when Mode0=/=drop ->
+ notify({mode_change,Mode0,drop},State);
+maybe_notify_mode_change(Mode1,#{mode:=drop}=State)
+ when Mode1==async; Mode1==sync ->
+ notify({mode_change,drop,Mode1},State);
+maybe_notify_mode_change(_,State) ->
+ State.
+
+notify(Note,#{module:=Module,cb_state:=CBState}=State) ->
+ CBState1 = try_callback_call(Module,notify,[Note,CBState],CBState),
+ State#{cb_state=>CBState1}.
+
+try_callback_call(Module, Function, Args) ->
+ try_callback_call(Module, Function, Args, '$no_default_return').
+
+try_callback_call(Module, Function, Args, DefRet) ->
+ try apply(Module, Function, Args)
+ catch
+ throw:R -> R;
+ error:undef:S when DefRet=/='$no_default_return' ->
+ case S of
+ [{Module,Function,Args,_}|_] ->
+ DefRet;
+ _ ->
+ erlang:raise(error,undef,S)
+ end
+ end.
diff --git a/lib/kernel/test/logger.cover b/lib/kernel/test/logger.cover
index 960bc0abff..a9ef81903d 100644
--- a/lib/kernel/test/logger.cover
+++ b/lib/kernel/test/logger.cover
@@ -4,9 +4,11 @@
logger_backend,
logger_config,
logger_disk_log_h,
- logger_h_common,
logger_filters,
logger_formatter,
+ logger_handler_watcher,
+ logger_h_common,
+ logger_olp,
logger_server,
logger_simple_h,
logger_std_h,
diff --git a/lib/kernel/test/logger_disk_log_h_SUITE.erl b/lib/kernel/test/logger_disk_log_h_SUITE.erl
index 87b8250781..1fc7605914 100644
--- a/lib/kernel/test/logger_disk_log_h_SUITE.erl
+++ b/lib/kernel/test/logger_disk_log_h_SUITE.erl
@@ -306,9 +306,9 @@ logging(cleanup, _Config) ->
filter_config(_Config) ->
ok = logger:add_handler(?MODULE,logger_disk_log_h,#{}),
{ok,#{config:=HConfig}=Config} = logger:get_handler_config(?MODULE),
- HConfig = maps:without([handler_pid,mode_tab],HConfig),
+ HConfig = maps:without([olp],HConfig),
- FakeFullHConfig = HConfig#{handler_pid=>self(),mode_tab=>erlang:make_ref()},
+ FakeFullHConfig = HConfig#{olp=>{regname,self(),erlang:make_ref()}},
#{config:=HConfig} =
logger_disk_log_h:filter_config(Config#{config=>FakeFullHConfig}),
ok.
@@ -351,9 +351,7 @@ errors(Config) ->
%% Read-only fields may (accidentially) be included in the change,
%% but it won't take effect
{ok,C} = logger:get_handler_config(Name1),
- ok = logger:set_handler_config(Name1,config,
- #{handler_pid=>self(),
- mode_tab=>erlang:make_ref()}),
+ ok = logger:set_handler_config(Name1,config,#{olp=>dummyvalue}),
{ok,C} = logger:get_handler_config(Name1),
@@ -419,19 +417,16 @@ config_fail(_Config) ->
filter_default=>log,
formatter=>{?MODULE,self()}}),
- {error,{handler_not_added,{invalid_config,logger_disk_log_h,
- {invalid_levels,#{drop_mode_qlen:=1}}}}} =
+ {error,{handler_not_added,{invalid_olp_levels,#{drop_mode_qlen:=1}}}} =
logger:add_handler(?MODULE,logger_disk_log_h,
#{config => #{drop_mode_qlen=>1}}),
- {error,{handler_not_added,{invalid_config,logger_disk_log_h,
- {invalid_levels,#{sync_mode_qlen:=43,
- drop_mode_qlen:=42}}}}} =
+ {error,{handler_not_added,{invalid_olp_levels,#{sync_mode_qlen:=43,
+ drop_mode_qlen:=42}}}} =
logger:add_handler(?MODULE,logger_disk_log_h,
#{config => #{sync_mode_qlen=>43,
drop_mode_qlen=>42}}),
- {error,{handler_not_added,{invalid_config,logger_disk_log_h,
- {invalid_levels,#{drop_mode_qlen:=43,
- flush_qlen:=42}}}}} =
+ {error,{handler_not_added,{invalid_olp_levels,#{drop_mode_qlen:=43,
+ flush_qlen:=42}}}} =
logger:add_handler(?MODULE,logger_disk_log_h,
#{config => #{drop_mode_qlen=>43,
flush_qlen=>42}}),
@@ -445,7 +440,7 @@ config_fail(_Config) ->
#{max_no_files=>2}),
%% incorrect values of OP params
{ok,#{config := HConfig}} = logger:get_handler_config(?MODULE),
- {error,{invalid_config,logger_disk_log_h,{invalid_levels,_}}} =
+ {error,{invalid_olp_levels,_}} =
logger:update_handler_config(?MODULE,config,
HConfig#{sync_mode_qlen=>100,
flush_qlen=>99}),
@@ -467,7 +462,7 @@ info_and_reset(_Config) ->
ok = logger:add_handler(?MODULE,logger_disk_log_h,
#{filter_default=>log,
formatter=>{?MODULE,self()}}),
- #{id := ?MODULE} = logger_disk_log_h:info(?MODULE),
+ #{} = logger_disk_log_h:info(?MODULE),
ok = logger_disk_log_h:reset(?MODULE).
info_and_reset(cleanup,_Config) ->
logger:remove_handler(?MODULE).
@@ -479,7 +474,7 @@ reconfig(Config) ->
#{filter_default=>log,
filters=>?DEFAULT_HANDLER_FILTERS([?MODULE]),
formatter=>{?MODULE,self()}}),
- #{id := ?MODULE,
+ #{%id := ?MODULE,
sync_mode_qlen := ?SYNC_MODE_QLEN,
drop_mode_qlen := ?DROP_MODE_QLEN,
flush_qlen := ?FLUSH_QLEN,
@@ -490,12 +485,13 @@ reconfig(Config) ->
overload_kill_qlen := ?OVERLOAD_KILL_QLEN,
overload_kill_mem_size := ?OVERLOAD_KILL_MEM_SIZE,
overload_kill_restart_after := ?OVERLOAD_KILL_RESTART_AFTER,
- filesync_repeat_interval := ?FILESYNC_REPEAT_INTERVAL,
- handler_state :=
- #{log_opts := #{type := ?DISK_LOG_TYPE,
- max_no_files := ?DISK_LOG_MAX_NO_FILES,
- max_no_bytes := ?DISK_LOG_MAX_NO_BYTES,
- file := DiskLogFile}}} =
+ cb_state :=
+ #{handler_state :=
+ #{log_opts := #{type := ?DISK_LOG_TYPE,
+ max_no_files := ?DISK_LOG_MAX_NO_FILES,
+ max_no_bytes := ?DISK_LOG_MAX_NO_BYTES,
+ file := DiskLogFile}},
+ filesync_repeat_interval := ?FILESYNC_REPEAT_INTERVAL}} =
logger_disk_log_h:info(?MODULE),
{ok,#{config :=
#{sync_mode_qlen := ?SYNC_MODE_QLEN,
@@ -527,7 +523,7 @@ reconfig(Config) ->
overload_kill_restart_after => infinity,
filesync_repeat_interval => no_repeat},
ok = logger:set_handler_config(?MODULE, config, HConfig1),
- #{id := ?MODULE,
+ #{%id := ?MODULE,
sync_mode_qlen := 1,
drop_mode_qlen := 2,
flush_qlen := 3,
@@ -538,7 +534,7 @@ reconfig(Config) ->
overload_kill_qlen := 100000,
overload_kill_mem_size := 10000000,
overload_kill_restart_after := infinity,
- filesync_repeat_interval := no_repeat} =
+ cb_state := #{filesync_repeat_interval := no_repeat}} =
logger_disk_log_h:info(?MODULE),
{ok,#{config:=HConfig1}} = logger:get_handler_config(?MODULE),
@@ -577,11 +573,12 @@ reconfig(Config) ->
max_no_files => 1,
max_no_bytes => 1024,
file => File}}),
- #{handler_state :=
- #{log_opts := #{type := halt,
- max_no_files := 1,
- max_no_bytes := 1024,
- file := File}}} =
+ #{cb_state :=
+ #{handler_state :=
+ #{log_opts := #{type := halt,
+ max_no_files := 1,
+ max_no_bytes := 1024,
+ file := File}}}} =
logger_disk_log_h:info(?MODULE),
{ok,#{config :=
#{type := halt,
@@ -652,7 +649,7 @@ sync(Config) ->
ok = logger:update_handler_config(?MODULE, config, HConfig1),
no_repeat = maps:get(filesync_repeat_interval,
- logger_disk_log_h:info(?MODULE)),
+ maps:get(cb_state,logger_disk_log_h:info(?MODULE))),
%% The following timer is to make sure the time from last log
%% ("first") to next ("second") is long enough, so the a flush is
%% triggered by the idle timeout between "fourth" and "fifth".
@@ -678,14 +675,15 @@ sync(Config) ->
WaitT = 4500,
OneSync = {logger_h_common,handle_cast,repeated_filesync},
%% receive 1 repeated_filesync per sec
- start_tracer([{logger_h_common,handle_cast,2}],
+ start_tracer([{{logger_h_common,handle_cast,2},
+ [{[repeated_filesync,'_'],[],[{message,{caller}}]}]}],
[OneSync || _ <- lists:seq(1, trunc(WaitT/SyncInt))]),
HConfig2 = HConfig#{filesync_repeat_interval => SyncInt},
ok = logger:update_handler_config(?MODULE, config, HConfig2),
SyncInt = maps:get(filesync_repeat_interval,
- logger_disk_log_h:info(?MODULE)),
+ maps:get(cb_state,logger_disk_log_h:info(?MODULE))),
timer:sleep(WaitT),
HConfig3 = HConfig#{filesync_repeat_interval => no_repeat},
ok = logger:update_handler_config(?MODULE, config, HConfig3),
@@ -861,9 +859,11 @@ write_failure(Config) ->
rpc:call(Node, ?MODULE, set_internal_log, [?MODULE,internal_log]),
rpc:call(Node, ?MODULE, set_result, [disk_log_write,ok]),
HState = rpc:call(Node, logger_disk_log_h, info, [?STANDARD_HANDLER]),
- ct:pal("LogOpts = ~p", [LogOpts = maps:get(log_opts,
- maps:get(handler_state,HState))]),
-
+ LogOpts = maps:get(log_opts,
+ maps:get(handler_state,
+ maps:get(cb_state,HState))),
+ ct:pal("LogOpts = ~p", [LogOpts]),
+
%% ?check and ?check_no_log in this test only check for internal log events
ok = log_on_remote_node(Node, "Logged1"),
rpc:call(Node, logger_disk_log_h, filesync, [?STANDARD_HANDLER]),
@@ -915,14 +915,15 @@ sync_failure(Config) ->
rpc:call(Node, ?MODULE, set_internal_log, [?MODULE,internal_log]),
rpc:call(Node, ?MODULE, set_result, [disk_log_sync,ok]),
HState = rpc:call(Node, logger_disk_log_h, info, [?STANDARD_HANDLER]),
- LogOpts = maps:get(log_opts, maps:get(handler_state,HState)),
+ LogOpts = maps:get(log_opts, maps:get(handler_state,
+ maps:get(cb_state,HState))),
SyncInt = 500,
ok = rpc:call(Node, logger, update_handler_config,
[?STANDARD_HANDLER, config,
#{filesync_repeat_interval => SyncInt}]),
Info = rpc:call(Node, logger_disk_log_h, info, [?STANDARD_HANDLER]),
- SyncInt = maps:get(filesync_repeat_interval, Info),
+ SyncInt = maps:get(filesync_repeat_interval, maps:get(cb_state, Info)),
ok = log_on_remote_node(Node, "Logged1"),
?check_no_log,
@@ -1198,7 +1199,7 @@ qlen_kill_new(Config) ->
receive
{'DOWN', MRef, _, _, Info} ->
case Info of
- {shutdown,{overloaded,?MODULE,QLen,Mem}} ->
+ {shutdown,{overloaded,QLen,Mem}} ->
ct:pal("Terminated with qlen = ~w, mem = ~w", [QLen,Mem]);
killed ->
ct:pal("Slow shutdown, handler process was killed!", [])
@@ -1235,7 +1236,7 @@ mem_kill_new(Config) ->
receive
{'DOWN', MRef, _, _, Info} ->
case Info of
- {shutdown,{overloaded,?MODULE,QLen,Mem}} ->
+ {shutdown,{overloaded,QLen,Mem}} ->
ct:pal("Terminated with qlen = ~w, mem = ~w", [QLen,Mem]);
killed ->
ct:pal("Slow shutdown, handler process was killed!", [])
@@ -1607,7 +1608,9 @@ start_tracer(Trace,Expected) ->
ok.
tpl([{M,F,A}|Trace]) ->
- {ok,Match} = dbg:tpl(M,F,A,c),
+ tpl([{{M,F,A},c}|Trace]);
+tpl([{{M,F,A},MS}|Trace]) ->
+ {ok,Match} = dbg:tpl(M,F,A,MS),
case lists:keyfind(matched,1,Match) of
{_,_,1} ->
ok;
diff --git a/lib/kernel/test/logger_std_h_SUITE.erl b/lib/kernel/test/logger_std_h_SUITE.erl
index eb17a6d857..e8f1c34f44 100644
--- a/lib/kernel/test/logger_std_h_SUITE.erl
+++ b/lib/kernel/test/logger_std_h_SUITE.erl
@@ -209,9 +209,9 @@ default_formatter(_Config) ->
filter_config(_Config) ->
ok = logger:add_handler(?MODULE,logger_std_h,#{}),
{ok,#{config:=HConfig}=Config} = logger:get_handler_config(?MODULE),
- HConfig = maps:without([handler_pid,mode_tab],HConfig),
+ HConfig = maps:without([olp],HConfig),
- FakeFullHConfig = HConfig#{handler_pid=>self(),mode_tab=>erlang:make_ref()},
+ FakeFullHConfig = HConfig#{olp=>{regname,self(),erlang:make_ref()}},
#{config:=HConfig} =
logger_std_h:filter_config(Config#{config=>FakeFullHConfig}),
ok.
@@ -246,13 +246,13 @@ errors(Config) ->
_ ->
NoDir = lists:concat(["/",?MODULE,"_dir"]),
{error,
- {handler_not_added,{{open_failed,NoDir,eacces},_}}} =
+ {handler_not_added,{open_failed,NoDir,eacces}}} =
logger:add_handler(myh2,logger_std_h,
#{config=>#{type=>{file,NoDir}}})
end,
{error,
- {handler_not_added,{{open_failed,Log,_},_}}} =
+ {handler_not_added,{open_failed,Log,_}}} =
logger:add_handler(myh3,logger_std_h,
#{config=>#{type=>{file,Log,[bad_file_opt]}}}),
@@ -320,19 +320,16 @@ config_fail(_Config) ->
#{config => #{restart_type => bad},
filter_default=>log,
formatter=>{?MODULE,self()}}),
- {error,{handler_not_added,{invalid_config,logger_std_h,
- {invalid_levels,#{drop_mode_qlen:=1}}}}} =
+ {error,{handler_not_added,{invalid_olp_levels,#{drop_mode_qlen:=1}}}} =
logger:add_handler(?MODULE,logger_std_h,
#{config => #{drop_mode_qlen=>1}}),
- {error,{handler_not_added,{invalid_config,logger_std_h,
- {invalid_levels,#{sync_mode_qlen:=43,
- drop_mode_qlen:=42}}}}} =
+ {error,{handler_not_added,{invalid_olp_levels,#{sync_mode_qlen:=43,
+ drop_mode_qlen:=42}}}} =
logger:add_handler(?MODULE,logger_std_h,
#{config => #{sync_mode_qlen=>43,
drop_mode_qlen=>42}}),
- {error,{handler_not_added,{invalid_config,logger_std_h,
- {invalid_levels,#{drop_mode_qlen:=43,
- flush_qlen:=42}}}}} =
+ {error,{handler_not_added,{invalid_olp_levels,#{drop_mode_qlen:=43,
+ flush_qlen:=42}}}} =
logger:add_handler(?MODULE,logger_std_h,
#{config => #{drop_mode_qlen=>43,
flush_qlen=>42}}),
@@ -344,7 +341,7 @@ config_fail(_Config) ->
logger:set_handler_config(?MODULE,config,
#{type=>{file,"file"}}),
- {error,{invalid_config,logger_std_h,{invalid_levels,_}}} =
+ {error,{invalid_olp_levels,_}} =
logger:set_handler_config(?MODULE,config,
#{sync_mode_qlen=>100,
flush_qlen=>99}),
@@ -355,9 +352,7 @@ config_fail(_Config) ->
%% Read-only fields may (accidentially) be included in the change,
%% but it won't take effect
{ok,C} = logger:get_handler_config(?MODULE),
- ok = logger:set_handler_config(?MODULE,config,
- #{handler_pid=>self(),
- mode_tab=>erlang:make_ref()}),
+ ok = logger:set_handler_config(?MODULE,config,#{olp=>dummyvalue}),
{ok,C} = logger:get_handler_config(?MODULE),
ok.
@@ -462,7 +457,7 @@ bad_input(_Config) ->
info_and_reset(_Config) ->
- #{id := ?STANDARD_HANDLER} = logger_std_h:info(?STANDARD_HANDLER),
+ #{} = logger_std_h:info(?STANDARD_HANDLER),
ok = logger_std_h:reset(?STANDARD_HANDLER).
reconfig(Config) ->
@@ -473,9 +468,10 @@ reconfig(Config) ->
filter_default=>log,
filters=>?DEFAULT_HANDLER_FILTERS([?MODULE]),
formatter=>{?MODULE,self()}}),
- #{id := ?MODULE,
- handler_state := #{type := standard_io,
- file_ctrl_pid := FileCtrlPid},
+ #{%id := ?MODULE,
+ cb_state:=#{handler_state := #{type := standard_io,
+ file_ctrl_pid := FileCtrlPid},
+ filesync_repeat_interval := no_repeat},
sync_mode_qlen := ?SYNC_MODE_QLEN,
drop_mode_qlen := ?DROP_MODE_QLEN,
flush_qlen := ?FLUSH_QLEN,
@@ -485,8 +481,7 @@ reconfig(Config) ->
overload_kill_enable := ?OVERLOAD_KILL_ENABLE,
overload_kill_qlen := ?OVERLOAD_KILL_QLEN,
overload_kill_mem_size := ?OVERLOAD_KILL_MEM_SIZE,
- overload_kill_restart_after := ?OVERLOAD_KILL_RESTART_AFTER,
- filesync_repeat_interval := no_repeat} = DefaultInfo =
+ overload_kill_restart_after := ?OVERLOAD_KILL_RESTART_AFTER} = DefaultInfo =
logger_std_h:info(?MODULE),
{ok,
@@ -518,9 +513,10 @@ reconfig(Config) ->
overload_kill_mem_size => 10000000,
overload_kill_restart_after => infinity,
filesync_repeat_interval => 5000}),
- #{id := ?MODULE,
- handler_state := #{type := standard_io,
- file_ctrl_pid := FileCtrlPid},
+ #{%id := ?MODULE,
+ cb_state := #{handler_state := #{type := standard_io,
+ file_ctrl_pid := FileCtrlPid},
+ filesync_repeat_interval := no_repeat},
sync_mode_qlen := 1,
drop_mode_qlen := 2,
flush_qlen := 3,
@@ -530,8 +526,7 @@ reconfig(Config) ->
overload_kill_enable := true,
overload_kill_qlen := 100000,
overload_kill_mem_size := 10000000,
- overload_kill_restart_after := infinity,
- filesync_repeat_interval := no_repeat} = Info = logger_std_h:info(?MODULE),
+ overload_kill_restart_after := infinity} = Info = logger_std_h:info(?MODULE),
{ok,#{config :=
#{type := standard_io,
@@ -613,7 +608,7 @@ file_opts(Config) ->
Log = filename:join(Dir, lists:concat([?FUNCTION_NAME,".log"])),
BadFileOpts = [raw],
BadType = {file,Log,BadFileOpts},
- {error,{handler_not_added,{{open_failed,Log,enoent},_}}} =
+ {error,{handler_not_added,{open_failed,Log,enoent}}} =
logger:add_handler(?MODULE, logger_std_h,
#{config => #{type => BadType}}),
@@ -626,7 +621,8 @@ file_opts(Config) ->
filters=>?DEFAULT_HANDLER_FILTERS([?MODULE]),
formatter=>{?MODULE,self()}}),
- #{handler_state := #{type := OkType}} = logger_std_h:info(?MODULE),
+ #{cb_state := #{handler_state := #{type := OkType}}} =
+ logger_std_h:info(?MODULE),
logger:notice(M1=?msg,?domain),
?check(M1),
B1 = ?bin(M1),
@@ -675,7 +671,8 @@ sync(Config) ->
%% a filesync is still performed when handler goes idle
ok = logger:update_handler_config(?MODULE, config,
#{filesync_repeat_interval => no_repeat}),
- no_repeat = maps:get(filesync_repeat_interval, logger_std_h:info(?MODULE)),
+ no_repeat = maps:get(filesync_repeat_interval,
+ maps:get(cb_state, logger_std_h:info(?MODULE))),
%% The following timer is to make sure the time from last log
%% ("second") to next ("third") is long enough, so the a flush is
%% triggered by the idle timeout between "thrid" and "fourth".
@@ -698,12 +695,14 @@ sync(Config) ->
WaitT = 4500,
OneSync = {logger_h_common,handle_cast,repeated_filesync},
%% receive 1 repeated_filesync per sec
- start_tracer([{logger_h_common,handle_cast,2}],
+ start_tracer([{{logger_h_common,handle_cast,2},
+ [{[repeated_filesync,'_'],[],[]}]}],
[OneSync || _ <- lists:seq(1, trunc(WaitT/SyncInt))]),
ok = logger:update_handler_config(?MODULE, config,
#{filesync_repeat_interval => SyncInt}),
- SyncInt = maps:get(filesync_repeat_interval, logger_std_h:info(?MODULE)),
+ SyncInt = maps:get(filesync_repeat_interval,
+ maps:get(cb_state,logger_std_h:info(?MODULE))),
timer:sleep(WaitT),
ok = logger:update_handler_config(?MODULE, config,
#{filesync_repeat_interval => no_repeat}),
@@ -765,7 +764,7 @@ sync_failure(Config) ->
[?STANDARD_HANDLER, config,
#{filesync_repeat_interval => SyncInt}]),
Info = rpc:call(Node, logger_std_h, info, [?STANDARD_HANDLER]),
- SyncInt = maps:get(filesync_repeat_interval, Info),
+ SyncInt = maps:get(filesync_repeat_interval, maps:get(cb_state,Info)),
ok = log_on_remote_node(Node, "Logged1"),
?check_no_log,
@@ -1095,7 +1094,7 @@ qlen_kill_new(Config) ->
receive
{'DOWN', MRef, _, _, Info} ->
case Info of
- {shutdown,{overloaded,?MODULE,QLen,Mem}} ->
+ {shutdown,{overloaded,QLen,Mem}} ->
ct:pal("Terminated with qlen = ~w, mem = ~w", [QLen,Mem]);
killed ->
ct:pal("Slow shutdown, handler process was killed!", [])
@@ -1146,7 +1145,7 @@ mem_kill_new(Config) ->
receive
{'DOWN', MRef, _, _, Info} ->
case Info of
- {shutdown,{overloaded,?MODULE,QLen,Mem}} ->
+ {shutdown,{overloaded,QLen,Mem}} ->
ct:pal("Terminated with qlen = ~w, mem = ~w", [QLen,Mem]);
killed ->
ct:pal("Slow shutdown, handler process was killed!", [])
@@ -1624,7 +1623,8 @@ start_tracer(Trace,Expected) ->
Pid = self(),
FileCtrlPid = maps:get(file_ctrl_pid,
maps:get(handler_state,
- logger_std_h:info(?MODULE))),
+ maps:get(cb_state,
+ logger_std_h:info(?MODULE)))),
dbg:tracer(process,{fun tracer/2,{Pid,Expected}}),
dbg:p(whereis(h_proc_name()),[c]),
dbg:p(FileCtrlPid,[c]),
@@ -1632,7 +1632,9 @@ start_tracer(Trace,Expected) ->
ok.
tpl([{M,F,A}|Trace]) ->
- {ok,Match} = dbg:tpl(M,F,A,[]),
+ tpl([{{M,F,A},[]}|Trace]);
+tpl([{{M,F,A},MS}|Trace]) ->
+ {ok,Match} = dbg:tpl(M,F,A,MS),
case lists:keyfind(matched,1,Match) of
{_,_,1} ->
ok;