Merge pull request #11445 from zhongwencool/remove-os-mon-from-windows

fix: remove os_mon application in Windows release
This commit is contained in:
zhongwencool 2023-08-16 23:30:24 +08:00 committed by GitHub
commit e6f0dead9e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 151 additions and 81 deletions

View File

@ -14,7 +14,6 @@
esockd, esockd,
cowboy, cowboy,
sasl, sasl,
os_mon,
lc, lc,
hocon, hocon,
emqx_durable_storage emqx_durable_storage

View File

@ -38,15 +38,14 @@
%% gen_server callbacks %% gen_server callbacks
-export([ -export([
init/1, init/1,
handle_continue/2,
handle_call/3, handle_call/3,
handle_cast/2, handle_cast/2,
handle_info/2, handle_info/2,
terminate/2, terminate/2,
code_change/3 code_change/3
]). ]).
-ifdef(TEST). -export([is_os_check_supported/0]).
-export([is_sysmem_check_supported/0]).
-endif.
-include("emqx.hrl"). -include("emqx.hrl").
@ -56,7 +55,7 @@ start_link() ->
gen_server:start_link({local, ?OS_MON}, ?MODULE, [], []). gen_server:start_link({local, ?OS_MON}, ?MODULE, [], []).
update(OS) -> update(OS) ->
erlang:send(?MODULE, {monitor_conf_update, OS}). gen_server:cast(?MODULE, {monitor_conf_update, OS}).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% API %% API
@ -83,12 +82,17 @@ current_sysmem_percent() ->
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
init([]) -> init([]) ->
{ok, undefined, {continue, setup}}.
handle_continue(setup, undefined) ->
%% start os_mon temporarily
{ok, _} = application:ensure_all_started(os_mon),
%% memsup is not reliable, ignore %% memsup is not reliable, ignore
memsup:set_sysmem_high_watermark(1.0), memsup:set_sysmem_high_watermark(1.0),
SysHW = init_os_monitor(), SysHW = init_os_monitor(),
MemRef = start_mem_check_timer(), MemRef = start_mem_check_timer(),
CpuRef = start_cpu_check_timer(), CpuRef = start_cpu_check_timer(),
{ok, #{sysmem_high_watermark => SysHW, mem_time_ref => MemRef, cpu_time_ref => CpuRef}}. {noreply, #{sysmem_high_watermark => SysHW, mem_time_ref => MemRef, cpu_time_ref => CpuRef}}.
init_os_monitor() -> init_os_monitor() ->
init_os_monitor(emqx:get_config([sysmon, os])). init_os_monitor(emqx:get_config([sysmon, os])).
@ -110,6 +114,12 @@ handle_call({set_sysmem_high_watermark, New}, _From, #{sysmem_high_watermark :=
handle_call(Req, _From, State) -> handle_call(Req, _From, State) ->
{reply, {error, {unexpected_call, Req}}, State}. {reply, {error, {unexpected_call, Req}}, State}.
handle_cast({monitor_conf_update, OS}, State) ->
cancel_outdated_timer(State),
SysHW = init_os_monitor(OS),
MemRef = start_mem_check_timer(),
CpuRef = start_cpu_check_timer(),
{noreply, #{sysmem_high_watermark => SysHW, mem_time_ref => MemRef, cpu_time_ref => CpuRef}};
handle_cast(Msg, State) -> handle_cast(Msg, State) ->
?SLOG(error, #{msg => "unexpected_cast", cast => Msg}), ?SLOG(error, #{msg => "unexpected_cast", cast => Msg}),
{noreply, State}. {noreply, State}.
@ -151,12 +161,6 @@ handle_info({timeout, _Timer, cpu_check}, State) ->
end, end,
Ref = start_cpu_check_timer(), Ref = start_cpu_check_timer(),
{noreply, State#{cpu_time_ref => Ref}}; {noreply, State#{cpu_time_ref => Ref}};
handle_info({monitor_conf_update, OS}, State) ->
cancel_outdated_timer(State),
SysHW = init_os_monitor(OS),
MemRef = start_mem_check_timer(),
CpuRef = start_cpu_check_timer(),
{noreply, #{sysmem_high_watermark => SysHW, mem_time_ref => MemRef, cpu_time_ref => CpuRef}};
handle_info(Info, State) -> handle_info(Info, State) ->
?SLOG(error, #{msg => "unexpected_info", info => Info}), ?SLOG(error, #{msg => "unexpected_info", info => Info}),
{noreply, State}. {noreply, State}.
@ -182,12 +186,12 @@ start_cpu_check_timer() ->
_ -> start_timer(Interval, cpu_check) _ -> start_timer(Interval, cpu_check)
end. end.
is_sysmem_check_supported() -> is_os_check_supported() ->
{unix, linux} =:= os:type(). {unix, linux} =:= os:type().
start_mem_check_timer() -> start_mem_check_timer() ->
Interval = emqx:get_config([sysmon, os, mem_check_interval]), Interval = emqx:get_config([sysmon, os, mem_check_interval]),
case is_integer(Interval) andalso is_sysmem_check_supported() of case is_integer(Interval) andalso is_os_check_supported() of
true -> true ->
start_timer(Interval, mem_check); start_timer(Interval, mem_check);
false -> false ->
@ -205,7 +209,7 @@ update_mem_alarm_status(HWM) when HWM > 1.0 orelse HWM < 0.0 ->
<<"Deactivated mem usage alarm due to out of range threshold">> <<"Deactivated mem usage alarm due to out of range threshold">>
); );
update_mem_alarm_status(HWM) -> update_mem_alarm_status(HWM) ->
is_sysmem_check_supported() andalso is_os_check_supported() andalso
do_update_mem_alarm_status(HWM), do_update_mem_alarm_status(HWM),
ok. ok.

View File

@ -1582,7 +1582,7 @@ fields("sysmon_os") ->
sc( sc(
hoconsc:union([disabled, duration()]), hoconsc:union([disabled, duration()]),
#{ #{
default => <<"60s">>, default => default_mem_check_interval(),
desc => ?DESC(sysmon_os_mem_check_interval) desc => ?DESC(sysmon_os_mem_check_interval)
} }
)}, )},
@ -3657,3 +3657,9 @@ shared_subscription_strategy() ->
desc => ?DESC(broker_shared_subscription_strategy) desc => ?DESC(broker_shared_subscription_strategy)
} }
)}. )}.
default_mem_check_interval() ->
case emqx_os_mon:is_os_check_supported() of
true -> <<"60s">>;
false -> disabled
end.

View File

@ -29,6 +29,7 @@
%% gen_server callbacks %% gen_server callbacks
-export([ -export([
init/1, init/1,
handle_continue/2,
handle_call/3, handle_call/3,
handle_cast/2, handle_cast/2,
handle_info/2, handle_info/2,
@ -70,11 +71,14 @@ update(VM) ->
init([]) -> init([]) ->
emqx_logger:set_proc_metadata(#{sysmon => true}), emqx_logger:set_proc_metadata(#{sysmon => true}),
init_system_monitor(), {ok, undefined, {continue, setup}}.
handle_continue(setup, undefined) ->
init_system_monitor(),
%% Monitor cluster partition event %% Monitor cluster partition event
ekka:monitor(partition, fun handle_partition_event/1), ekka:monitor(partition, fun handle_partition_event/1),
{ok, start_timer(#{timer => undefined, events => []})}. NewState = start_timer(#{timer => undefined, events => []}),
{noreply, NewState, hibernate}.
start_timer(State) -> start_timer(State) ->
State#{timer := emqx_utils:start_timer(timer:seconds(2), reset)}. State#{timer := emqx_utils:start_timer(timer:seconds(2), reset)}.

View File

@ -19,21 +19,25 @@
-behaviour(supervisor). -behaviour(supervisor).
-export([start_link/0]). -export([start_link/0]).
-export([init/1]). -export([init/1]).
start_link() -> start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []). supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) -> init([]) ->
Childs = [ OsMon =
case emqx_os_mon:is_os_check_supported() of
true -> [child_spec(emqx_os_mon)];
false -> []
end,
Children =
[
child_spec(emqx_sys), child_spec(emqx_sys),
child_spec(emqx_alarm), child_spec(emqx_alarm),
child_spec(emqx_sys_mon), child_spec(emqx_sys_mon),
child_spec(emqx_os_mon),
child_spec(emqx_vm_mon) child_spec(emqx_vm_mon)
], ] ++ OsMon,
{ok, {{one_for_one, 10, 100}, Childs}}. {ok, {{one_for_one, 10, 100}, Children}}.
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% Internal functions %% Internal functions

View File

@ -44,7 +44,7 @@
get_otp_version/0 get_otp_version/0
]). ]).
-export([cpu_util/0]). -export([cpu_util/0, cpu_util/1]).
-ifdef(TEST). -ifdef(TEST).
-compile(export_all). -compile(export_all).
@ -378,16 +378,25 @@ avg15() ->
cpu_util() -> cpu_util() ->
compat_windows(fun cpu_sup:util/0). compat_windows(fun cpu_sup:util/0).
cpu_util(Args) ->
compat_windows(fun cpu_sup:util/1, Args).
compat_windows(Fun) -> compat_windows(Fun) ->
case os:type() of case compat_windows(Fun, []) of
{win32, nt} ->
0.0;
_Type ->
case catch Fun() of
Val when is_float(Val) -> floor(Val * 100) / 100; Val when is_float(Val) -> floor(Val * 100) / 100;
Val when is_number(Val) -> Val; Val when is_number(Val) -> Val;
_Error -> 0.0 _ -> 0.0
end.
compat_windows(Fun, Args) ->
try
case emqx_os_mon:is_os_check_supported() of
false -> 0.0;
true when Args =:= [] -> Fun();
true -> Fun(Args)
end end
catch
_:_ -> 0.0
end. end.
load(Avg) -> load(Avg) ->

View File

@ -39,29 +39,47 @@ init_per_testcase(t_cpu_check_alarm, Config) ->
%% 200ms %% 200ms
cpu_check_interval => 200 cpu_check_interval => 200
}), }),
ok = supervisor:terminate_child(emqx_sys_sup, emqx_os_mon), restart_os_mon(),
{ok, _} = supervisor:restart_child(emqx_sys_sup, emqx_os_mon),
Config; Config;
init_per_testcase(t_sys_mem_check_alarm, Config) -> init_per_testcase(t_sys_mem_check_alarm, Config) ->
case emqx_os_mon:is_sysmem_check_supported() of case emqx_os_mon:is_os_check_supported() of
true -> true ->
SysMon = emqx_config:get([sysmon, os], #{}), SysMon = emqx_config:get([sysmon, os], #{}),
emqx_config:put([sysmon, os], SysMon#{ emqx_config:put([sysmon, os], SysMon#{
sysmem_high_watermark => 0.51, sysmem_high_watermark => 0.51,
%% 200ms %% 200ms
mem_check_interval => 200 mem_check_interval => 200
}), });
ok = supervisor:terminate_child(emqx_sys_sup, emqx_os_mon),
{ok, _} = supervisor:restart_child(emqx_sys_sup, emqx_os_mon),
Config;
false -> false ->
Config ok
end; end,
restart_os_mon(),
Config;
init_per_testcase(_, Config) -> init_per_testcase(_, Config) ->
emqx_common_test_helpers:boot_modules(all), restart_os_mon(),
emqx_common_test_helpers:start_apps([]),
Config. Config.
restart_os_mon() ->
case emqx_os_mon:is_os_check_supported() of
true ->
ok = supervisor:terminate_child(emqx_sys_sup, emqx_os_mon),
{ok, _} = supervisor:restart_child(emqx_sys_sup, emqx_os_mon);
false ->
_ = supervisor:terminate_child(emqx_sys_sup, emqx_os_mon),
_ = supervisor:delete_child(emqx_sys_sup, emqx_os_mon),
%% run test on mac/windows.
Mod = emqx_os_mon,
OsMon = #{
id => Mod,
start => {Mod, start_link, []},
restart => permanent,
shutdown => 5000,
type => worker,
modules => [Mod]
},
{ok, _} = supervisor:start_child(emqx_sys_sup, OsMon)
end.
t_api(_) -> t_api(_) ->
?assertEqual(0.7, emqx_os_mon:get_sysmem_high_watermark()), ?assertEqual(0.7, emqx_os_mon:get_sysmem_high_watermark()),
?assertEqual(ok, emqx_os_mon:set_sysmem_high_watermark(0.8)), ?assertEqual(ok, emqx_os_mon:set_sysmem_high_watermark(0.8)),
@ -81,7 +99,7 @@ t_api(_) ->
ok. ok.
t_sys_mem_check_disable(Config) -> t_sys_mem_check_disable(Config) ->
case emqx_os_mon:is_sysmem_check_supported() of case emqx_os_mon:is_os_check_supported() of
true -> do_sys_mem_check_disable(Config); true -> do_sys_mem_check_disable(Config);
false -> skip false -> skip
end. end.
@ -100,7 +118,7 @@ do_sys_mem_check_disable(_Config) ->
ok. ok.
t_sys_mem_check_alarm(Config) -> t_sys_mem_check_alarm(Config) ->
case emqx_os_mon:is_sysmem_check_supported() of case emqx_os_mon:is_os_check_supported() of
true -> do_sys_mem_check_alarm(Config); true -> do_sys_mem_check_alarm(Config);
false -> skip false -> skip
end. end.
@ -167,7 +185,7 @@ t_cpu_check_alarm(_) ->
util, util,
fun() -> CpuUtil end, fun() -> CpuUtil end,
fun() -> fun() ->
timer:sleep(500), timer:sleep(1000),
Alarms = emqx_alarm:get_alarms(activated), Alarms = emqx_alarm:get_alarms(activated),
?assert( ?assert(
emqx_vm_mon_SUITE:is_existing(high_cpu_usage, emqx_alarm:get_alarms(activated)) emqx_vm_mon_SUITE:is_existing(high_cpu_usage, emqx_alarm:get_alarms(activated))
@ -193,7 +211,7 @@ t_cpu_check_alarm(_) ->
?assert(is_binary(Msg)), ?assert(is_binary(Msg)),
emqx_config:put([sysmon, os, cpu_high_watermark], 1), emqx_config:put([sysmon, os, cpu_high_watermark], 1),
emqx_config:put([sysmon, os, cpu_low_watermark], 0.96), emqx_config:put([sysmon, os, cpu_low_watermark], 0.96),
timer:sleep(500), timer:sleep(800),
?assertNot( ?assertNot(
emqx_vm_mon_SUITE:is_existing(high_cpu_usage, emqx_alarm:get_alarms(activated)) emqx_vm_mon_SUITE:is_existing(high_cpu_usage, emqx_alarm:get_alarms(activated))
) )

View File

@ -17,7 +17,8 @@
asn1, asn1,
syntax_tools, syntax_tools,
ssl, ssl,
os_mon, %% started temporary in emqx to prevent crash vm when permanent.
{os_mon, load},
inets, inets,
compiler, compiler,
runtime_tools, runtime_tools,
@ -36,7 +37,6 @@
[ [
emqx, emqx,
emqx_conf, emqx_conf,
esasl, esasl,
observer_cli, observer_cli,
tools, tools,

View File

@ -185,25 +185,39 @@ node_info(Nodes) ->
stopped_node_info(Node) -> stopped_node_info(Node) ->
{Node, #{node => Node, node_status => 'stopped', role => core}}. {Node, #{node => Node, node_status => 'stopped', role => core}}.
%% Hide cpu stats if os_check is not supported.
vm_stats() -> vm_stats() ->
Idle = vm_stats('cpu.idle'),
{MemUsedRatio, MemTotal} = get_sys_memory(), {MemUsedRatio, MemTotal} = get_sys_memory(),
cpu_stats() ++
[ [
{run_queue, vm_stats('run.queue')}, {run_queue, vm_stats('run.queue')},
{cpu_idle, Idle},
{cpu_use, 100 - Idle},
{total_memory, MemTotal}, {total_memory, MemTotal},
{used_memory, erlang:round(MemTotal * MemUsedRatio)} {used_memory, erlang:round(MemTotal * MemUsedRatio)}
]. ].
cpu_stats() ->
case emqx_os_mon:is_os_check_supported() of
false ->
[];
true ->
Idle = vm_stats('cpu.idle'),
[
{cpu_idle, Idle},
{cpu_use, 100 - Idle}
]
end.
vm_stats('cpu.idle') -> vm_stats('cpu.idle') ->
case cpu_sup:util([detailed]) of case emqx_vm:cpu_util([detailed]) of
%% Not support for Windows {_Num, _Use, List, _} when is_list(List) -> proplists:get_value(idle, List, 0);
{_, 0, 0, _} -> 0; %% return {all, 0, 0, []} when cpu_sup is not started
{_Num, _Use, IdleList, _} -> proplists:get_value(idle, IdleList, 0) _ -> 0
end; end;
vm_stats('cpu.use') -> vm_stats('cpu.use') ->
100 - vm_stats('cpu.idle'); case vm_stats('cpu.idle') of
0 -> 0;
Idle -> 100 - Idle
end;
vm_stats('total.memory') -> vm_stats('total.memory') ->
{_, MemTotal} = get_sys_memory(), {_, MemTotal} = get_sys_memory(),
MemTotal; MemTotal;

View File

@ -0,0 +1,2 @@
Removed os_mon application monitor support on Windows platforms to prevent VM crashes.
Functionality remains on non-Windows platforms.

View File

@ -403,7 +403,8 @@ defmodule EMQXUmbrella.MixProject do
quicer: enable_quicer?(), quicer: enable_quicer?(),
bcrypt: enable_bcrypt?(), bcrypt: enable_bcrypt?(),
jq: enable_jq?(), jq: enable_jq?(),
observer: is_app?(:observer) observer: is_app?(:observer),
os_mon: enable_os_mon?()
} }
|> Enum.reject(&elem(&1, 1)) |> Enum.reject(&elem(&1, 1))
|> Enum.map(&elem(&1, 0)) |> Enum.map(&elem(&1, 0))
@ -835,6 +836,10 @@ defmodule EMQXUmbrella.MixProject do
not win32?() not win32?()
end end
defp enable_os_mon?() do
not win32?()
end
defp enable_jq?() do defp enable_jq?() do
not Enum.any?([ not Enum.any?([
build_without_jq?(), build_without_jq?(),

View File

@ -405,12 +405,13 @@ relx_apps(ReleaseType, Edition) ->
ce -> CEBusinessApps ce -> CEBusinessApps
end, end,
BusinessApps = CommonBusinessApps ++ EditionSpecificApps, BusinessApps = CommonBusinessApps ++ EditionSpecificApps,
ExcludedApps = excluded_apps(ReleaseType), Apps =
SystemApps ++ (SystemApps ++
%% EMQX starts the DB and the business applications: %% EMQX starts the DB and the business applications:
[{App, load} || App <- (DBApps -- ExcludedApps)] ++ [{App, load} || App <- DBApps] ++
[emqx_machine] ++ [emqx_machine] ++
[{App, load} || App <- (BusinessApps -- ExcludedApps)]. [{App, load} || App <- BusinessApps]),
lists:foldl(fun proplists:delete/2, Apps, excluded_apps(ReleaseType)).
excluded_apps(ReleaseType) -> excluded_apps(ReleaseType) ->
OptionalApps = [ OptionalApps = [
@ -418,7 +419,8 @@ excluded_apps(ReleaseType) ->
{bcrypt, provide_bcrypt_release(ReleaseType)}, {bcrypt, provide_bcrypt_release(ReleaseType)},
{jq, is_jq_supported()}, {jq, is_jq_supported()},
{observer, is_app(observer)}, {observer, is_app(observer)},
{mnesia_rocksdb, is_rocksdb_supported()} {mnesia_rocksdb, is_rocksdb_supported()},
{os_mon, provide_os_mon_release()}
], ],
[App || {App, false} <- OptionalApps]. [App || {App, false} <- OptionalApps].
@ -524,6 +526,9 @@ is_debug(VarName) ->
provide_bcrypt_dep() -> provide_bcrypt_dep() ->
not is_win32(). not is_win32().
provide_os_mon_release() ->
not is_win32().
provide_bcrypt_release(ReleaseType) -> provide_bcrypt_release(ReleaseType) ->
provide_bcrypt_dep() andalso ReleaseType =:= cloud. provide_bcrypt_dep() andalso ReleaseType =:= cloud.

View File

@ -156,7 +156,7 @@ persistent_session_builtin_messages_table.label:
sysmon_os_cpu_low_watermark.desc: sysmon_os_cpu_low_watermark.desc:
"""The threshold, as percentage of system CPU load, """The threshold, as percentage of system CPU load,
for how much system cpu can be used before the corresponding alarm is cleared.""" for how much system cpu can be used before the corresponding alarm is cleared. Disabled on Windows platform"""
sysmon_os_cpu_low_watermark.label: sysmon_os_cpu_low_watermark.label:
"""CPU low watermark""" """CPU low watermark"""
@ -278,7 +278,7 @@ fields_ws_opts_mqtt_path.label:
sysmon_os_procmem_high_watermark.desc: sysmon_os_procmem_high_watermark.desc:
"""The threshold, as percentage of system memory, """The threshold, as percentage of system memory,
for how much system memory can be allocated by one Erlang process before for how much system memory can be allocated by one Erlang process before
the corresponding alarm is raised.""" the corresponding alarm is raised. Disabled on Windows platform."""
sysmon_os_procmem_high_watermark.label: sysmon_os_procmem_high_watermark.label:
"""ProcMem high wartermark""" """ProcMem high wartermark"""
@ -389,7 +389,7 @@ fields_tcp_opts_sndbuf.label:
"""TCP send buffer""" """TCP send buffer"""
sysmon_os_mem_check_interval.desc: sysmon_os_mem_check_interval.desc:
"""The time interval for the periodic memory check.""" """The time interval for the periodic memory check. Disabled on Windows platform."""
sysmon_os_mem_check_interval.label: sysmon_os_mem_check_interval.label:
"""Mem check interval""" """Mem check interval"""
@ -742,7 +742,7 @@ common_ssl_opts_schema_keyfile.label:
sysmon_os_cpu_high_watermark.desc: sysmon_os_cpu_high_watermark.desc:
"""The threshold, as percentage of system CPU load, """The threshold, as percentage of system CPU load,
for how much system cpu can be used before the corresponding alarm is raised.""" for how much system cpu can be used before the corresponding alarm is raised. Disabled on Windows platform"""
sysmon_os_cpu_high_watermark.label: sysmon_os_cpu_high_watermark.label:
"""CPU high watermark""" """CPU high watermark"""
@ -798,7 +798,7 @@ fields_ws_opts_proxy_address_header.label:
sysmon_os_sysmem_high_watermark.desc: sysmon_os_sysmem_high_watermark.desc:
"""The threshold, as percentage of system memory, """The threshold, as percentage of system memory,
for how much system memory can be allocated before the corresponding alarm is raised.""" for how much system memory can be allocated before the corresponding alarm is raised. Disabled on Windows platform"""
sysmon_os_sysmem_high_watermark.label: sysmon_os_sysmem_high_watermark.label:
"""SysMem high wartermark""" """SysMem high wartermark"""
@ -1521,7 +1521,7 @@ fields_tcp_opts_send_timeout_close.label:
"""TCP send timeout close""" """TCP send timeout close"""
sysmon_os_cpu_check_interval.desc: sysmon_os_cpu_check_interval.desc:
"""The time interval for the periodic CPU check.""" """The time interval for the periodic CPU check. Disabled on Windows platform."""
sysmon_os_cpu_check_interval.label: sysmon_os_cpu_check_interval.label:
"""The time interval for the periodic CPU check.""" """The time interval for the periodic CPU check."""