chore(auth): improve emqx_config_handler to replace adhoc config updates

This commit is contained in:
Ilya Averyanov 2023-08-03 10:43:33 +03:00
parent 8213aa42c9
commit 128aa19d47
15 changed files with 451 additions and 191 deletions

View File

@ -53,11 +53,17 @@
-optional_callbacks([ -optional_callbacks([
pre_config_update/3, pre_config_update/3,
post_config_update/5 propagated_pre_config_update/3,
post_config_update/5,
propagated_post_config_update/5
]). ]).
-callback pre_config_update([atom()], emqx_config:update_request(), emqx_config:raw_config()) -> -callback pre_config_update([atom()], emqx_config:update_request(), emqx_config:raw_config()) ->
{ok, emqx_config:update_request()} | {error, term()}. ok | {ok, emqx_config:update_request()} | {error, term()}.
-callback propagated_pre_config_update(
[atom()], emqx_config:update_request(), emqx_config:raw_config()
) ->
ok | {error, term()}.
-callback post_config_update( -callback post_config_update(
[atom()], [atom()],
@ -68,6 +74,15 @@
) -> ) ->
ok | {ok, Result :: any()} | {error, Reason :: term()}. ok | {ok, Result :: any()} | {error, Reason :: term()}.
-callback propagated_post_config_update(
[atom()],
emqx_config:update_request(),
emqx_config:config(),
emqx_config:config(),
emqx_config:app_envs()
) ->
ok | {ok, Result :: any()} | {error, Reason :: term()}.
-type state() :: #{handlers := any()}. -type state() :: #{handlers := any()}.
start_link() -> start_link() ->
@ -244,7 +259,13 @@ do_update_config(ConfKeyPath, Handlers, OldRawConf, UpdateReq) ->
do_update_config(ConfKeyPath, Handlers, OldRawConf, UpdateReq, []). do_update_config(ConfKeyPath, Handlers, OldRawConf, UpdateReq, []).
do_update_config([], Handlers, OldRawConf, UpdateReq, ConfKeyPath) -> do_update_config([], Handlers, OldRawConf, UpdateReq, ConfKeyPath) ->
call_pre_config_update(Handlers, OldRawConf, UpdateReq, ConfKeyPath); call_pre_config_update(#{
handlers => Handlers,
old_raw_conf => OldRawConf,
update_req => UpdateReq,
conf_key_path => ConfKeyPath,
callback => pre_config_update
});
do_update_config( do_update_config(
[ConfKey | SubConfKeyPath], [ConfKey | SubConfKeyPath],
Handlers, Handlers,
@ -331,15 +352,16 @@ do_post_config_update(
Result, Result,
ConfKeyPath ConfKeyPath
) -> ) ->
call_post_config_update( call_post_config_update(#{
Handlers, handlers => Handlers,
OldConf, old_conf => OldConf,
NewConf, new_conf => NewConf,
AppEnvs, app_envs => AppEnvs,
up_req(UpdateArgs), update_req => up_req(UpdateArgs),
Result, result => Result,
ConfKeyPath conf_key_path => ConfKeyPath,
); callback => post_config_update
});
do_post_config_update( do_post_config_update(
[ConfKey | SubConfKeyPath], [ConfKey | SubConfKeyPath],
Handlers, Handlers,
@ -365,7 +387,7 @@ do_post_config_update(
ConfKeyPath ConfKeyPath
). ).
get_sub_handlers(ConfKey, Handlers) -> get_sub_handlers(ConfKey, Handlers) when is_atom(ConfKey) ->
case maps:find(ConfKey, Handlers) of case maps:find(ConfKey, Handlers) of
error -> maps:get(?WKEY, Handlers, #{}); error -> maps:get(?WKEY, Handlers, #{});
{ok, SubHandlers} -> SubHandlers {ok, SubHandlers} -> SubHandlers
@ -377,56 +399,230 @@ get_sub_config(ConfKey, Conf) when is_map(Conf) ->
get_sub_config(_, _Conf) -> get_sub_config(_, _Conf) ->
undefined. undefined.
call_pre_config_update(#{?MOD := HandlerName}, OldRawConf, UpdateReq, ConfKeyPath) -> call_pre_config_update(Ctx) ->
case erlang:function_exported(HandlerName, pre_config_update, 3) of case call_proper_pre_config_update(Ctx) of
{ok, NewUpdateReq} ->
case
propagate_pre_config_updates_to_subconf(Ctx#{
update_req => NewUpdateReq
})
of
{ok, _} ->
{ok, NewUpdateReq};
{error, _} = Error ->
Error
end;
{error, _} = Error ->
Error
end.
call_proper_pre_config_update(
#{
handlers := #{?MOD := Module},
callback := Callback,
update_req := UpdateReq,
old_raw_conf := OldRawConf
} = Ctx
) ->
case erlang:function_exported(Module, Callback, 3) of
true -> true ->
case HandlerName:pre_config_update(ConfKeyPath, UpdateReq, OldRawConf) of case apply_pre_config_update(Module, Ctx) of
{ok, NewUpdateReq} -> {ok, NewUpdateReq}; {ok, NewUpdateReq} ->
{error, Reason} -> {error, {pre_config_update, HandlerName, Reason}} {ok, NewUpdateReq};
ok ->
{ok, UpdateReq};
{error, Reason} ->
{error, {pre_config_update, Module, Reason}}
end; end;
false -> false ->
merge_to_old_config(UpdateReq, OldRawConf) merge_to_old_config(UpdateReq, OldRawConf)
end; end;
call_pre_config_update(_Handlers, OldRawConf, UpdateReq, _ConfKeyPath) -> call_proper_pre_config_update(
merge_to_old_config(UpdateReq, OldRawConf). #{update_req := UpdateReq}
call_post_config_update(
#{?MOD := HandlerName},
OldConf,
NewConf,
AppEnvs,
UpdateReq,
Result,
ConfKeyPath
) -> ) ->
case erlang:function_exported(HandlerName, post_config_update, 5) of {ok, UpdateReq}.
true ->
apply_pre_config_update(Module, #{
conf_key_path := ConfKeyPath,
update_req := UpdateReq,
old_raw_conf := OldRawConf,
callback := Callback
}) ->
Module:Callback(
ConfKeyPath, UpdateReq, OldRawConf
).
propagate_pre_config_updates_to_subconf(
#{handlers := #{?WKEY := _}} = Ctx
) ->
propagate_pre_config_updates_to_subconf_wkey(Ctx);
propagate_pre_config_updates_to_subconf(
#{handlers := Handlers} = Ctx
) ->
Keys = maps:keys(maps:without([?MOD], Handlers)),
propagate_pre_config_updates_to_subconf_keys(Keys, Ctx).
propagate_pre_config_updates_to_subconf_wkey(
#{
update_req := UpdateReq,
old_raw_conf := OldRawConf
} = Ctx
) ->
Keys = propagate_keys(UpdateReq, OldRawConf),
propagate_pre_config_updates_to_subconf_keys(Keys, Ctx).
propagate_pre_config_updates_to_subconf_keys([], #{update_req := UpdateReq}) ->
{ok, UpdateReq};
propagate_pre_config_updates_to_subconf_keys([Key | Keys], Ctx) ->
case propagate_pre_config_updates_to_subconf_key(Key, Ctx) of
ok ->
propagate_pre_config_updates_to_subconf_keys(Keys, Ctx);
{error, _} = Error ->
Error
end.
propagate_pre_config_updates_to_subconf_key(
Key,
#{
handlers := Handlers,
old_raw_conf := OldRawConf,
update_req := UpdateReq,
conf_key_path := ConfKeyPath
} = Ctx
) ->
AtomKey = atom(Key),
SubHandlers = get_sub_handlers(AtomKey, Handlers),
BinKey = bin(Key),
SubUpdateReq = get_sub_config(BinKey, UpdateReq),
SubOldConf = get_sub_config(BinKey, OldRawConf),
SubConfKeyPath = ConfKeyPath ++ [AtomKey],
case {SubOldConf, SubUpdateReq} of
{undefined, undefined} ->
ok;
{_, _} ->
case case
HandlerName:post_config_update( call_pre_config_update(Ctx#{
handlers := SubHandlers,
old_raw_conf := SubOldConf,
update_req := SubUpdateReq,
conf_key_path := SubConfKeyPath,
callback := propagated_pre_config_update
})
of
{ok, _SubNewConf1} ->
ok;
{error, _} = Error ->
Error
end
end.
call_post_config_update(#{handlers := Handlers} = Ctx) ->
case call_proper_post_config_update(Ctx) of
{ok, Result} ->
SubHandlers = maps:without([?MOD], Handlers),
propagate_post_config_updates_to_subconf(Ctx#{
handlers := SubHandlers,
callback := propagated_post_config_update,
result := Result
});
{error, _} = Error ->
Error
end.
call_proper_post_config_update(
#{
handlers := #{?MOD := Module},
callback := Callback,
result := Result
} = Ctx
) ->
case erlang:function_exported(Module, Callback, 5) of
true ->
case apply_post_config_update(Module, Ctx) of
ok -> {ok, Result};
{ok, Result1} -> {ok, Result#{Module => Result1}};
{error, Reason} -> {error, {post_config_update, Module, Reason}}
end;
false ->
{ok, Result}
end;
call_proper_post_config_update(
#{result := Result} = _Ctx
) ->
{ok, Result}.
apply_post_config_update(Module, #{
conf_key_path := ConfKeyPath,
update_req := UpdateReq,
new_conf := NewConf,
old_conf := OldConf,
app_envs := AppEnvs,
callback := Callback
}) ->
Module:Callback(
ConfKeyPath, ConfKeyPath,
UpdateReq, UpdateReq,
NewConf, NewConf,
OldConf, OldConf,
AppEnvs AppEnvs
) ).
of
ok -> {ok, Result}; propagate_post_config_updates_to_subconf(
{ok, Result1} -> {ok, Result#{HandlerName => Result1}}; #{handlers := #{?WKEY := _}} = Ctx
{error, Reason} -> {error, {post_config_update, HandlerName, Reason}}
end;
false ->
{ok, Result}
end;
call_post_config_update(
_Handlers,
_OldConf,
_NewConf,
_AppEnvs,
_UpdateReq,
Result,
_ConfKeyPath
) -> ) ->
{ok, Result}. propagate_post_config_updates_to_subconf_wkey(Ctx);
propagate_post_config_updates_to_subconf(
#{handlers := Handlers} = Ctx
) ->
Keys = maps:keys(Handlers),
propagate_post_config_updates_to_subconf_keys(Keys, Ctx).
propagate_post_config_updates_to_subconf_wkey(
#{
old_conf := OldConf,
new_conf := NewConf
} = Ctx
) ->
Keys = propagate_keys(OldConf, NewConf),
propagate_post_config_updates_to_subconf_keys(Keys, Ctx).
propagate_post_config_updates_to_subconf_keys([], #{result := Result}) ->
{ok, Result};
propagate_post_config_updates_to_subconf_keys([Key | Keys], Ctx) ->
case propagate_post_config_updates_to_subconf_key(Key, Ctx) of
{ok, Result1} ->
propagate_post_config_updates_to_subconf_keys(Keys, Ctx#{result := Result1});
Error ->
Error
end.
propagate_keys(OldConf, NewConf) ->
sets:to_list(sets:union(propagate_keys(OldConf), propagate_keys(NewConf))).
propagate_keys(Conf) when is_map(Conf) -> sets:from_list(maps:keys(Conf), [{version, 2}]);
propagate_keys(_) -> sets:new([{version, 2}]).
propagate_post_config_updates_to_subconf_key(
Key,
#{
handlers := Handlers,
new_conf := NewConf,
old_conf := OldConf,
result := Result,
conf_key_path := ConfKeyPath
} = Ctx
) ->
SubHandlers = maps:get(Key, Handlers, maps:get(?WKEY, Handlers, undefined)),
SubNewConf = get_sub_config(Key, NewConf),
SubOldConf = get_sub_config(Key, OldConf),
SubConfKeyPath = ConfKeyPath ++ [Key],
call_post_config_update(Ctx#{
handlers := SubHandlers,
new_conf := SubNewConf,
old_conf := SubOldConf,
result := Result,
conf_key_path := SubConfKeyPath,
callback := propagated_post_config_update
}).
%% The default callback of config handlers %% The default callback of config handlers
%% the behaviour is overwriting the old config if: %% the behaviour is overwriting the old config if:
@ -517,6 +713,7 @@ remove_empty_leaf(KeyPath, Handlers) ->
end. end.
assert_callback_function(Mod) -> assert_callback_function(Mod) ->
_ = Mod:module_info(),
case case
erlang:function_exported(Mod, pre_config_update, 3) orelse erlang:function_exported(Mod, pre_config_update, 3) orelse
erlang:function_exported(Mod, post_config_update, 5) erlang:function_exported(Mod, post_config_update, 5)

View File

@ -531,18 +531,15 @@ post_config_update(_Path, _Request, _NewConf, _OldConf, _AppEnvs) ->
ok. ok.
create_listener(Type, Name, NewConf) -> create_listener(Type, Name, NewConf) ->
StartRes = start_listener(Type, Name, NewConf), start_listener(Type, Name, NewConf).
emqx_hooks:run_fold('listener.started', [Type, Name, NewConf], StartRes).
remove_listener(Type, Name, OldConf) -> remove_listener(Type, Name, OldConf) ->
ok = unregister_ocsp_stapling_refresh(Type, Name), ok = unregister_ocsp_stapling_refresh(Type, Name),
StopRes = stop_listener(Type, Name, OldConf), stop_listener(Type, Name, OldConf).
emqx_hooks:run_fold('listener.stopped', [Type, Name, OldConf], StopRes).
update_listener(Type, Name, {OldConf, NewConf}) -> update_listener(Type, Name, {OldConf, NewConf}) ->
ok = maybe_unregister_ocsp_stapling_refresh(Type, Name, NewConf), ok = maybe_unregister_ocsp_stapling_refresh(Type, Name, NewConf),
RestartRes = restart_listener(Type, Name, {OldConf, NewConf}), restart_listener(Type, Name, {OldConf, NewConf}).
emqx_hooks:run_fold('listener.restarted', [Type, Name, {OldConf, NewConf}], RestartRes).
perform_listener_changes([]) -> perform_listener_changes([]) ->
ok; ok;
@ -824,10 +821,9 @@ convert_certs(ListenerConf) ->
fun(Type, Listeners0, Acc) -> fun(Type, Listeners0, Acc) ->
Listeners1 = Listeners1 =
maps:fold( maps:fold(
fun(Name, Conf, Acc1) -> fun(Name, Conf0, Acc1) ->
Conf1 = convert_certs(Type, Name, Conf), Conf1 = convert_certs(Type, Name, Conf0),
Conf2 = convert_authn_certs(Type, Name, Conf1), Acc1#{Name => Conf1}
Acc1#{Name => Conf2}
end, end,
#{}, #{},
Listeners0 Listeners0
@ -850,19 +846,6 @@ convert_certs(Type, Name, Conf) ->
throw({bad_ssl_config, Reason}) throw({bad_ssl_config, Reason})
end. end.
convert_authn_certs(Type, Name, #{<<"authentication">> := AuthNList} = Conf) ->
ChainName = listener_id(Type, Name),
AuthNList1 = lists:map(
fun(AuthN) ->
CertsDir = emqx_authentication_config:certs_dir(ChainName, AuthN),
emqx_authentication_config:convert_certs(CertsDir, AuthN)
end,
AuthNList
),
Conf#{<<"authentication">> => AuthNList1};
convert_authn_certs(_Type, _Name, Conf) ->
Conf.
filter_stacktrace({Reason, _Stacktrace}) -> Reason; filter_stacktrace({Reason, _Stacktrace}) -> Reason;
filter_stacktrace(Reason) -> Reason. filter_stacktrace(Reason) -> Reason.

View File

@ -34,6 +34,12 @@
inject_fields_from_mod/1 inject_fields_from_mod/1
]). ]).
%% for tests
-export([
erase_injections/0,
any_injections/0
]).
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% API %% API
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
@ -60,6 +66,36 @@ inject_fields_from_mod(Module) ->
ok ok
end. end.
erase_injections() ->
lists:foreach(
fun
({?HOOKPOINT_PT_KEY(_) = Key, _}) ->
persistent_term:erase(Key);
({?MODULE_PT_KEY(_) = Key, _}) ->
persistent_term:erase(Key);
(_) ->
ok
end,
persistent_term:get()
).
any_injections() ->
lists:any(
fun
({?HOOKPOINT_PT_KEY(_), _}) ->
true;
({?MODULE_PT_KEY(_), _}) ->
true;
(_) ->
false
end,
persistent_term:get()
).
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
do_inject_fields_from_mod(Module) -> do_inject_fields_from_mod(Module) ->
_ = Module:module_info(), _ = Module:module_info(),
case erlang:function_exported(Module, injected_fields, 0) of case erlang:function_exported(Module, injected_fields, 0) of

View File

@ -16,8 +16,6 @@
-module(emqx_common_test_helpers). -module(emqx_common_test_helpers).
-include_lib("emqx_authn/include/emqx_authentication.hrl").
-type special_config_handler() :: fun(). -type special_config_handler() :: fun().
-type apps() :: list(atom()). -type apps() :: list(atom()).
@ -351,7 +349,7 @@ stop_apps(Apps, Opts) ->
%% to avoid inter-suite flakiness %% to avoid inter-suite flakiness
application:unset_env(emqx, config_loader), application:unset_env(emqx, config_loader),
application:unset_env(emqx, boot_modules), application:unset_env(emqx, boot_modules),
persistent_term:erase(?EMQX_AUTHENTICATION_SCHEMA_MODULE_PT_KEY), emqx_schema_hooks:erase_injections(),
case Opts of case Opts of
#{erase_all_configs := false} -> #{erase_all_configs := false} ->
%% FIXME: this means inter-suite or inter-test dependencies %% FIXME: this means inter-suite or inter-test dependencies

View File

@ -26,7 +26,8 @@
-include_lib("eunit/include/eunit.hrl"). -include_lib("eunit/include/eunit.hrl").
-include_lib("common_test/include/ct.hrl"). -include_lib("common_test/include/ct.hrl").
all() -> emqx_common_test_helpers:all(?MODULE). all() ->
emqx_common_test_helpers:all(?MODULE).
init_per_suite(Config) -> init_per_suite(Config) ->
emqx_common_test_helpers:boot_modules(all), emqx_common_test_helpers:boot_modules(all),
@ -239,6 +240,62 @@ t_post_update_error(_Config) ->
), ),
ok. ok.
t_post_update_propagate_error_wkey(_Config) ->
Conf0 = emqx_config:get_raw([sysmon]),
Conf1 = emqx_utils_maps:deep_put([<<"os">>, <<"sysmem_high_watermark">>], Conf0, <<"60%">>),
callback_error(
[
[sysmon, '?', sysmem_high_watermark],
[sysmon]
],
[sysmon],
Conf1,
{error, {post_config_update, ?MODULE, post_config_update_error}}
),
ok.
t_post_update_propagate_error_key(_Config) ->
Conf0 = emqx_config:get_raw([sysmon]),
Conf1 = emqx_utils_maps:deep_put([<<"os">>, <<"sysmem_high_watermark">>], Conf0, <<"60%">>),
callback_error(
[
[sysmon, os, sysmem_high_watermark],
[sysmon]
],
[sysmon],
Conf1,
{error, {post_config_update, ?MODULE, post_config_update_error}}
),
ok.
t_pre_update_propagate_error_wkey(_Config) ->
Conf0 = emqx_config:get_raw([sysmon]),
Conf1 = emqx_utils_maps:deep_put([<<"os">>, <<"mem_check_interval">>], Conf0, <<"70s">>),
callback_error(
[
[sysmon, '?', mem_check_interval],
[sysmon]
],
[sysmon],
Conf1,
{error, {pre_config_update, ?MODULE, pre_config_update_error}}
),
ok.
t_pre_update_propagate_error_key(_Config) ->
Conf0 = emqx_config:get_raw([sysmon]),
Conf1 = emqx_utils_maps:deep_put([<<"os">>, <<"mem_check_interval">>], Conf0, <<"70s">>),
callback_error(
[
[sysmon, os, mem_check_interval],
[sysmon]
],
[sysmon],
Conf1,
{error, {pre_config_update, ?MODULE, pre_config_update_error}}
),
ok.
t_handler_root() -> t_handler_root() ->
%% Don't rely on default emqx_config_handler's merge behaviour. %% Don't rely on default emqx_config_handler's merge behaviour.
RootKey = [], RootKey = [],
@ -295,6 +352,11 @@ pre_config_update([sysmon, os, sysmem_high_watermark], UpdateReq, _RawConf) ->
pre_config_update([sysmon, os, mem_check_interval], _UpdateReq, _RawConf) -> pre_config_update([sysmon, os, mem_check_interval], _UpdateReq, _RawConf) ->
{error, pre_config_update_error}. {error, pre_config_update_error}.
propagated_pre_config_update([sysmon, os, mem_check_interval], _UpdateReq, _RawConf) ->
{error, pre_config_update_error};
propagated_pre_config_update(_ConfKeyPath, _UpdateReq, _RawConf) ->
ok.
post_config_update([sysmon], _UpdateReq, _NewConf, _OldConf, _AppEnvs) -> post_config_update([sysmon], _UpdateReq, _NewConf, _OldConf, _AppEnvs) ->
{ok, ok}; {ok, ok};
post_config_update([sysmon, os], _UpdateReq, _NewConf, _OldConf, _AppEnvs) -> post_config_update([sysmon, os], _UpdateReq, _NewConf, _OldConf, _AppEnvs) ->
@ -308,6 +370,13 @@ post_config_update([sysmon, os, cpu_high_watermark], _UpdateReq, _NewConf, _OldC
post_config_update([sysmon, os, sysmem_high_watermark], _UpdateReq, _NewConf, _OldConf, _AppEnvs) -> post_config_update([sysmon, os, sysmem_high_watermark], _UpdateReq, _NewConf, _OldConf, _AppEnvs) ->
{error, post_config_update_error}. {error, post_config_update_error}.
propagated_post_config_update(
[sysmon, os, sysmem_high_watermark], _UpdateReq, _NewConf, _OldConf, _AppEnvs
) ->
{error, post_config_update_error};
propagated_post_config_update(_ConfKeyPath, _UpdateReq, _NewConf, _OldConf, _AppEnvs) ->
ok.
wait_for_new_pid() -> wait_for_new_pid() ->
case erlang:whereis(emqx_config_handler) of case erlang:whereis(emqx_config_handler) of
undefined -> undefined ->
@ -318,10 +387,16 @@ wait_for_new_pid() ->
end. end.
callback_error(FailedPath, Update, ExpectError) -> callback_error(FailedPath, Update, ExpectError) ->
callback_error([FailedPath], FailedPath, Update, ExpectError).
callback_error(Paths, UpdatePath, Update, ExpectError) ->
ok = lists:foreach(
fun(Path) -> emqx_config_handler:add_handler(Path, ?MODULE) end,
Paths
),
Opts = #{rawconf_with_defaults => true}, Opts = #{rawconf_with_defaults => true},
ok = emqx_config_handler:add_handler(FailedPath, ?MODULE), Old = emqx:get_raw_config(UpdatePath, undefined),
Old = emqx:get_raw_config(FailedPath, undefined), Error = emqx:update_config(UpdatePath, Update, Opts),
Error = emqx:update_config(FailedPath, Update, Opts),
case ExpectError of case ExpectError of
{error, {post_config_update, ?MODULE, post_config_update_error}} -> {error, {post_config_update, ?MODULE, post_config_update_error}} ->
?assertMatch( ?assertMatch(
@ -330,7 +405,10 @@ callback_error(FailedPath, Update, ExpectError) ->
_ -> _ ->
?assertEqual(ExpectError, Error) ?assertEqual(ExpectError, Error)
end, end,
New = emqx:get_raw_config(FailedPath, undefined), New = emqx:get_raw_config(UpdatePath, undefined),
?assertEqual(Old, New), ?assertEqual(Old, New),
ok = emqx_config_handler:remove_handler(FailedPath), ok = lists:foreach(
fun(Path) -> emqx_config_handler:remove_handler(Path) end,
Paths
),
ok. ok.

View File

@ -347,12 +347,12 @@ stop_apps(Apps) ->
verify_clean_suite_state(#{work_dir := WorkDir}) -> verify_clean_suite_state(#{work_dir := WorkDir}) ->
{ok, []} = file:list_dir(WorkDir), {ok, []} = file:list_dir(WorkDir),
none = persistent_term:get(?EMQX_AUTHENTICATION_SCHEMA_MODULE_PT_KEY, none), false = emqx_schema_hooks:any_injections(),
[] = emqx_config:get_root_names(), [] = emqx_config:get_root_names(),
ok. ok.
clean_suite_state() -> clean_suite_state() ->
_ = persistent_term:erase(?EMQX_AUTHENTICATION_SCHEMA_MODULE_PT_KEY), _ = emqx_schema_hooks:erase_injections(),
_ = emqx_config:erase_all(), _ = emqx_config:erase_all(),
ok. ok.

View File

@ -34,4 +34,6 @@
{cover_opts, [verbose]}. {cover_opts, [verbose]}.
{cover_export_enabled, true}. {cover_export_enabled, true}.
{erl_first_files, ["src/emqx_authentication.erl"]}.
{project_plugins, [erlfmt]}. {project_plugins, [erlfmt]}.

View File

@ -60,6 +60,7 @@
register_providers/1, register_providers/1,
deregister_provider/1, deregister_provider/1,
deregister_providers/1, deregister_providers/1,
providers/0,
delete_chain/1, delete_chain/1,
lookup_chain/1, lookup_chain/1,
list_chains/0, list_chains/0,
@ -331,6 +332,10 @@ deregister_providers(AuthNTypes) when is_list(AuthNTypes) ->
deregister_provider(AuthNType) -> deregister_provider(AuthNType) ->
deregister_providers([AuthNType]). deregister_providers([AuthNType]).
-spec providers() -> [{authn_type(), module()}].
providers() ->
call(providers).
-spec delete_chain(chain_name()) -> ok | {error, term()}. -spec delete_chain(chain_name()) -> ok | {error, term()}.
delete_chain(Name) -> delete_chain(Name) ->
call({delete_chain, Name}). call({delete_chain, Name}).
@ -463,6 +468,8 @@ handle_call(
end; end;
handle_call({deregister_providers, AuthNTypes}, _From, #{providers := Providers} = State) -> handle_call({deregister_providers, AuthNTypes}, _From, #{providers := Providers} = State) ->
reply(ok, State#{providers := maps:without(AuthNTypes, Providers)}); reply(ok, State#{providers := maps:without(AuthNTypes, Providers)});
handle_call(providers, _From, #{providers := Providers} = State) ->
reply(maps:to_list(Providers), State);
handle_call({delete_chain, ChainName}, _From, State) -> handle_call({delete_chain, ChainName}, _From, State) ->
UpdateFun = fun(Chain) -> UpdateFun = fun(Chain) ->
{_MatchedIDs, NewChain} = do_delete_authenticators(fun(_) -> true end, Chain), {_MatchedIDs, NewChain} = do_delete_authenticators(fun(_) -> true end, Chain),

View File

@ -21,7 +21,9 @@
-export([ -export([
pre_config_update/3, pre_config_update/3,
post_config_update/5 post_config_update/5,
propagated_pre_config_update/3,
propagated_post_config_update/5
]). ]).
-export([ -export([
@ -37,8 +39,8 @@
-export_type([config/0]). -export_type([config/0]).
-include("logger.hrl").
-include("emqx_authentication.hrl"). -include("emqx_authentication.hrl").
-include_lib("emqx/include/logger.hrl").
-type parsed_config() :: #{ -type parsed_config() :: #{
mechanism := atom(), mechanism := atom(),
@ -145,6 +147,12 @@ do_pre_config_update(Paths, NewConfig, _OldConfig) ->
|| New <- to_list(NewConfig) || New <- to_list(NewConfig)
]}. ]}.
-spec propagated_pre_config_update(list(atom()), update_request(), emqx_config:raw_config()) ->
ok | {error, term()}.
propagated_pre_config_update(Paths, NewConfig, OldConfig) ->
{ok, _} = do_pre_config_update(Paths, NewConfig, OldConfig),
ok.
-spec post_config_update( -spec post_config_update(
list(atom()), list(atom()),
update_request(), update_request(),
@ -203,6 +211,26 @@ do_post_config_update(Paths, _UpdateReq, NewConfig0, OldConfig0, _AppEnvs) ->
ok = emqx_authentication:reorder_authenticator(ChainName, NewIds), ok = emqx_authentication:reorder_authenticator(ChainName, NewIds),
ok. ok.
-spec propagated_post_config_update(
list(atom()),
update_request(),
map() | list() | undefined,
emqx_config:raw_config(),
emqx_config:app_envs()
) ->
ok | {ok, map()} | {error, term()}.
propagated_post_config_update(Paths, UpdateReq, NewConfig, OldConfig, AppEnvs) ->
ok = post_config_update(Paths, UpdateReq, NewConfig, OldConfig, AppEnvs),
ChainName = chain_name(Paths),
ok = maybe_delete_chain(ChainName, NewConfig),
ok.
maybe_delete_chain(ChainName, undefined) ->
ok = emqx_authentication:delete_chain(ChainName);
maybe_delete_chain(_ChainName, _NewConfig) ->
ok.
%% create new authenticators and update existing ones %% create new authenticators and update existing ones
create_or_update_authenticators(OldIds, ChainName, NewConfig) -> create_or_update_authenticators(OldIds, ChainName, NewConfig) ->
lists:foreach( lists:foreach(

View File

@ -1,86 +0,0 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2017-2023 EMQ Technologies Co., Ltd. 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.
%%--------------------------------------------------------------------
-module(emqx_authentication_listener_hooks).
-include_lib("emqx/include/emqx_hooks.hrl").
-export([
on_listener_started/4,
on_listener_stopped/4,
on_listener_updated/4
]).
-export([
load/0,
unload/0
]).
%%--------------------------------------------------------------------
%% API
%%--------------------------------------------------------------------
load() ->
ok = emqx_hook:put('listener.started', {?MODULE, on_listener_started, []}, ?HP_AUTHN),
ok = emqx_hook:put('listener.stopped', {?MODULE, on_listener_stopped, []}, ?HP_AUTHN),
ok = emqx_hook:put('listener.updated', {?MODULE, on_listener_updated, []}, ?HP_AUTHN),
ok.
unload() ->
ok = emqx_hooks:del('listener.started', {?MODULE, authenticate, []}),
ok = emqx_hooks:del('listener.stopped', {?MODULE, authenticate, []}),
ok = emqx_hooks:del('listener.updated', {?MODULE, authenticate, []}),
ok.
%%--------------------------------------------------------------------
%% Hooks
%%--------------------------------------------------------------------
on_listener_started(Type, Name, Conf, ok) ->
recreate_authenticators(Type, Name, Conf);
on_listener_started(_Type, _Name, _Conf, _Error) ->
ok.
on_listener_updated(Type, Name, {_OldConf, NewConf}, ok) ->
recreate_authenticators(Type, Name, NewConf);
on_listener_updated(_Type, _Name, _Conf, _Error) ->
ok.
on_listener_stopped(Type, Name, _OldConf, ok) ->
_ = emqx_authentication:delete_chain(emqx_listeners:listener_id(Type, Name)),
ok;
on_listener_stopped(_Type, _Name, _Conf, _Error) ->
ok.
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
recreate_authenticators(Type, Name, Conf) ->
Chain = emqx_listeners:listener_id(Type, Name),
_ = emqx_authentication:delete_chain(Chain),
do_create_authneticators(Chain, maps:get(authentication, Conf, [])).
do_create_authneticators(Chain, [AuthN | T]) ->
case emqx_authentication:create_authenticator(Chain, AuthN) of
{ok, _} ->
do_create_authneticators(Chain, T);
Error ->
_ = emqx_authentication:delete_chain(Chain),
{ok, Error}
end;
do_create_authneticators(_Chain, []) ->
ok.

View File

@ -35,8 +35,7 @@
%%------------------------------------------------------------------------------ %%------------------------------------------------------------------------------
start(_StartType, _StartArgs) -> start(_StartType, _StartArgs) ->
%% required by test cases, ensure the injection of %% required by test cases, ensure the injection of schema
%% EMQX_AUTHENTICATION_SCHEMA_MODULE_PT_KEY
_ = emqx_conf_schema:roots(), _ = emqx_conf_schema:roots(),
ok = mria_rlog:wait_for_shards([?AUTH_SHARD], infinity), ok = mria_rlog:wait_for_shards([?AUTH_SHARD], infinity),
{ok, Sup} = emqx_authn_sup:start_link(), {ok, Sup} = emqx_authn_sup:start_link(),

View File

@ -94,19 +94,19 @@ all() ->
emqx_common_test_helpers:all(?MODULE). emqx_common_test_helpers:all(?MODULE).
init_per_suite(Config) -> init_per_suite(Config) ->
LogLevel = emqx_logger:get_primary_log_level(), Apps = emqx_cth_suite:start(
ok = emqx_logger:set_log_level(debug), [
application:set_env(ekka, strict_mode, true), emqx,
emqx_config:erase_all(), emqx_conf,
emqx_common_test_helpers:stop_apps([]), emqx_authn
emqx_common_test_helpers:boot_modules(all), ],
emqx_common_test_helpers:start_apps([]), #{work_dir => ?config(priv_dir)}
[{log_level, LogLevel} | Config]. ),
ok = deregister_providers(),
[{apps, Apps} | Config].
end_per_suite(Config) -> end_per_suite(Config) ->
emqx_common_test_helpers:stop_apps([]), emqx_cth_suite:stop(?config(apps)),
LogLevel = ?config(log_level),
emqx_logger:set_log_level(LogLevel),
ok. ok.
init_per_testcase(Case, Config) -> init_per_testcase(Case, Config) ->
@ -302,15 +302,20 @@ t_update_config(Config) when is_list(Config) ->
ok = register_provider(?config("auth1"), ?MODULE), ok = register_provider(?config("auth1"), ?MODULE),
ok = register_provider(?config("auth2"), ?MODULE), ok = register_provider(?config("auth2"), ?MODULE),
Global = ?config(global), Global = ?config(global),
%% We mocked provider implementation, but did't mock the schema
%% so we should provide full config
AuthenticatorConfig1 = #{ AuthenticatorConfig1 = #{
mechanism => password_based, <<"mechanism">> => <<"password_based">>,
backend => built_in_database, <<"backend">> => <<"built_in_database">>,
enable => true <<"enable">> => true
}, },
AuthenticatorConfig2 = #{ AuthenticatorConfig2 = #{
mechanism => password_based, <<"mechanism">> => <<"password_based">>,
backend => mysql, <<"backend">> => <<"mysql">>,
enable => true <<"query">> => <<"SELECT password_hash, salt FROM users WHERE username = ?">>,
<<"server">> => <<"127.0.0.1:5432">>,
<<"database">> => <<"emqx">>,
<<"enable">> => true
}, },
ID1 = <<"password_based:built_in_database">>, ID1 = <<"password_based:built_in_database">>,
ID2 = <<"password_based:mysql">>, ID2 = <<"password_based:mysql">>,
@ -580,3 +585,11 @@ certs(Certs) ->
register_provider(Type, Module) -> register_provider(Type, Module) ->
ok = ?AUTHN:register_providers([{Type, Module}]). ok = ?AUTHN:register_providers([{Type, Module}]).
deregister_providers() ->
lists:foreach(
fun({Type, _Module}) ->
ok = ?AUTHN:deregister_provider(Type)
end,
lists:flatten([?AUTHN:providers()])
).

View File

@ -1,7 +1,10 @@
%% -*- mode: erlang -*- %% -*- mode: erlang -*-
{erl_opts, [debug_info]}. {erl_opts, [debug_info]}.
{deps, [{emqx, {path, "../emqx"}}]}. {deps, [
{emqx, {path, "../emqx"}},
{emqx_authn, {path, "../emqx_authn"}}
]}.
{shell, [ {shell, [
% {config, "config/sys.config"}, % {config, "config/sys.config"},

View File

@ -2,5 +2,6 @@
{erl_opts, [debug_info]}. {erl_opts, [debug_info]}.
{deps, [ {deps, [
{emqx, {path, "../emqx"}}, {emqx, {path, "../emqx"}},
{emqx_utils, {path, "../emqx_utils"}} {emqx_utils, {path, "../emqx_utils"}},
{emqx_authn, {path, "../emqx_authn"}}
]}. ]}.

View File

@ -3,6 +3,7 @@
{deps, [ {deps, [
{emqx, {path, "../emqx"}}, {emqx, {path, "../emqx"}},
{emqx_dashboard, {path, "../emqx_dashboard"}}, {emqx_dashboard, {path, "../emqx_dashboard"}},
{emqx_conf, {path, "../emqx_conf"}},
{emqx_utils, {path, "../emqx_utils"}} {emqx_utils, {path, "../emqx_utils"}}
]}. ]}.