Merge pull request #6373 from savonarola/authz-backend-behaviours

chore(authz): encapsulate authz backend logic
This commit is contained in:
Ilya Averyanov 2021-12-11 00:11:16 +03:00 committed by GitHub
commit 59f1bfeaac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 301 additions and 135 deletions

View File

@ -1,24 +1,3 @@
-type(ipaddress() :: {ipaddr, esockd_cidr:cidr_string()} |
{ipaddrs, list(esockd_cidr:cidr_string())}).
-type(username() :: {username, binary()}).
-type(clientid() :: {clientid, binary()}).
-type(who() :: ipaddress() | username() | clientid() |
{'and', [ipaddress() | username() | clientid()]} |
{'or', [ipaddress() | username() | clientid()]} |
all).
-type(action() :: subscribe | publish | all).
-type(permission() :: allow | deny).
-type(rule() :: {permission(), who(), action(), list(emqx_types:topic())}).
-type(rules() :: [rule()]).
-type(sources() :: [map()]).
-define(APP, emqx_authz). -define(APP, emqx_authz).
-define(ALLOW_DENY(A), ((A =:= allow) orelse (A =:= <<"allow">>) orelse -define(ALLOW_DENY(A), ((A =:= allow) orelse (A =:= <<"allow">>) orelse
@ -38,6 +17,9 @@
-define(ACL_TABLE_USERNAME, 1). -define(ACL_TABLE_USERNAME, 1).
-define(ACL_TABLE_CLIENTID, 2). -define(ACL_TABLE_CLIENTID, 2).
-type(action() :: subscribe | publish | all).
-type(permission() :: allow | deny).
-record(emqx_acl, { -record(emqx_acl, {
who :: ?ACL_TABLE_ALL| {?ACL_TABLE_USERNAME, binary()} | {?ACL_TABLE_CLIENTID, binary()}, who :: ?ACL_TABLE_ALL| {?ACL_TABLE_USERNAME, binary()} | {?ACL_TABLE_CLIENTID, binary()},
rules :: [ {permission(), action(), emqx_topic:topic()} ] rules :: [ {permission(), action(), emqx_topic:topic()} ]

View File

@ -27,6 +27,7 @@
-export([ register_metrics/0 -export([ register_metrics/0
, init/0 , init/0
, deinit/0
, lookup/0 , lookup/0
, lookup/1 , lookup/1
, move/2 , move/2
@ -42,6 +43,31 @@
-export([ph_to_re/1]). -export([ph_to_re/1]).
-type(source() :: map()).
-type(match_result() :: {matched, allow} | {matched, deny} | nomatch).
-type(default_result() :: allow | deny).
-type(authz_result() :: {stop, allow} | {ok, deny}).
-type(sources() :: [source()]).
-callback(init(source()) -> source()).
-callback(description() -> string()).
-callback(destroy(source()) -> ok).
-callback(dry_run(source()) -> ok | {error, term()}).
-callback(authorize(
emqx_types:clientinfo(),
emqx_types:pubsub(),
emqx_types:topic(),
source()) -> match_result()).
-spec(register_metrics() -> ok). -spec(register_metrics() -> ok).
register_metrics() -> register_metrics() ->
lists:foreach(fun emqx_metrics:ensure/1, ?AUTHZ_METRICS). lists:foreach(fun emqx_metrics:ensure/1, ?AUTHZ_METRICS).
@ -54,6 +80,11 @@ init() ->
NSources = init_sources(Sources), NSources = init_sources(Sources),
ok = emqx_hooks:add('client.authorize', {?MODULE, authorize, [NSources]}, -1). ok = emqx_hooks:add('client.authorize', {?MODULE, authorize, [NSources]}, -1).
deinit() ->
ok = emqx_hooks:del('client.authorize', {?MODULE, authorize}),
emqx_conf:remove_handler(?CONF_KEY_PATH),
emqx_authz_utils:cleanup_resources().
lookup() -> lookup() ->
{_M, _F, [A]}= find_action_in_hooks(), {_M, _F, [A]}= find_action_in_hooks(),
A. A.
@ -115,7 +146,7 @@ do_update({{?CMD_REPLACE, Type}, #{<<"enable">> := true} = Source}, Conf) when i
NConf = Front ++ [Source | Rear], NConf = Front ++ [Source | Rear],
ok = check_dup_types(NConf), ok = check_dup_types(NConf),
NConf; NConf;
Error -> Error {error, _} = Error -> Error
end; end;
do_update({{?CMD_REPLACE, Type}, Source}, Conf) when is_map(Source), is_list(Conf) -> do_update({{?CMD_REPLACE, Type}, Source}, Conf) when is_map(Source), is_list(Conf) ->
{_Old, Front, Rear} = take(Type, Conf), {_Old, Front, Rear} = take(Type, Conf),
@ -178,9 +209,9 @@ do_post_update(_, NewSources) ->
ok = emqx_authz_cache:drain_cache(). ok = emqx_authz_cache:drain_cache().
ensure_resource_deleted(#{enable := false}) -> ok; ensure_resource_deleted(#{enable := false}) -> ok;
ensure_resource_deleted(#{type := file}) -> ok; ensure_resource_deleted(#{type := Type} = Source) ->
ensure_resource_deleted(#{type := 'built-in-database'}) -> ok; Module = authz_module(Type),
ensure_resource_deleted(#{annotations := #{id := Id}}) -> ok = emqx_resource:remove(Id). Module:destroy(Source).
check_dup_types(Sources) -> check_dup_types(Sources) ->
check_dup_types(Sources, []). check_dup_types(Sources, []).
@ -204,26 +235,10 @@ check_dup_types([Source | Sources], Checked) ->
check_dup_types(Sources, [Type | Checked]) check_dup_types(Sources, [Type | Checked])
end. end.
create_dry_run(T, Source) -> create_dry_run(Type, Source) ->
case is_connector_source(T) of
true ->
[CheckedSource] = check_sources([Source]), [CheckedSource] = check_sources([Source]),
case T of Module = authz_module(Type),
http -> Module:dry_run(CheckedSource).
URIMap = maps:get(url, CheckedSource),
NSource = maps:put(base_url, maps:remove(query, URIMap), CheckedSource)
end,
emqx_resource:create_dry_run(connector_module(T), NSource);
false ->
ok
end.
is_connector_source(http) -> true;
is_connector_source(mongodb) -> true;
is_connector_source(mysql) -> true;
is_connector_source(postgresql) -> true;
is_connector_source(redis) -> true;
is_connector_source(_) -> false.
init_sources(Sources) -> init_sources(Sources) ->
{_Enabled, Disabled} = lists:partition(fun(#{enable := Enable}) -> Enable end, Sources), {_Enabled, Disabled} = lists:partition(fun(#{enable := Enable}) -> Enable end, Sources),
@ -234,54 +249,9 @@ init_sources(Sources) ->
lists:map(fun init_source/1, Sources). lists:map(fun init_source/1, Sources).
init_source(#{enable := false} = Source) -> Source; init_source(#{enable := false} = Source) -> Source;
init_source(#{type := file, init_source(#{type := Type} = Source) ->
path := Path Module = authz_module(Type),
} = Source) -> Module:init(Source).
Rules = case file:consult(Path) of
{ok, Terms} ->
[emqx_authz_rule:compile(Term) || Term <- Terms];
{error, eacces} ->
?SLOG(alert, #{msg => "insufficient_permissions_to_read_file", path => Path}),
error(eaccess);
{error, enoent} ->
?SLOG(alert, #{msg => "file_does_not_exist", path => Path}),
error(enoent);
{error, Reason} ->
?SLOG(alert, #{msg => "failed_to_read_file", path => Path, reason => Reason}),
error(Reason)
end,
Source#{annotations => #{rules => Rules}};
init_source(#{type := http,
url := Url
} = Source) ->
NSource= maps:put(base_url, maps:remove(query, Url), Source),
case create_resource(NSource) of
{error, Reason} -> error({load_config_error, Reason});
Id -> Source#{annotations => #{id => Id}}
end;
init_source(#{type := 'built-in-database'
} = Source) ->
Source;
init_source(#{type := DB
} = Source) when DB =:= redis;
DB =:= mongodb ->
case create_resource(Source) of
{error, Reason} -> error({load_config_error, Reason});
Id -> Source#{annotations => #{id => Id}}
end;
init_source(#{type := DB,
query := SQL
} = Source) when DB =:= mysql;
DB =:= postgresql ->
Mod = authz_module(DB),
case create_resource(Source) of
{error, Reason} -> error({load_config_error, Reason});
Id -> Source#{annotations =>
#{id => Id,
query => erlang:apply(Mod, parse_query, [SQL])
}
}
end.
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
%% AuthZ callbacks %% AuthZ callbacks
@ -289,11 +259,11 @@ init_source(#{type := DB,
%% @doc Check AuthZ %% @doc Check AuthZ
-spec(authorize( emqx_types:clientinfo() -spec(authorize( emqx_types:clientinfo()
, emqx_types:all() , emqx_types:pubsub()
, emqx_types:topic() , emqx_types:topic()
, allow | deny , default_result()
, sources()) , sources())
-> {stop, allow} | {ok, deny}). -> authz_result()).
authorize(#{username := Username, authorize(#{username := Username,
peerhost := IpAddress peerhost := IpAddress
} = Client, PubSub, Topic, DefaultResult, Sources) -> } = Client, PubSub, Topic, DefaultResult, Sources) ->
@ -325,16 +295,10 @@ do_authorize(_Client, _PubSub, _Topic, []) ->
nomatch; nomatch;
do_authorize(Client, PubSub, Topic, [#{enable := false} | Rest]) -> do_authorize(Client, PubSub, Topic, [#{enable := false} | Rest]) ->
do_authorize(Client, PubSub, Topic, Rest); do_authorize(Client, PubSub, Topic, Rest);
do_authorize(Client, PubSub, Topic, [#{type := file} = F | Tail]) ->
#{annotations := #{rules := Rules}} = F,
case emqx_authz_rule:matches(Client, PubSub, Topic, Rules) of
nomatch -> do_authorize(Client, PubSub, Topic, Tail);
Matched -> Matched
end;
do_authorize(Client, PubSub, Topic, do_authorize(Client, PubSub, Topic,
[Connector = #{type := Type} | Tail] ) -> [Connector = #{type := Type} | Tail] ) ->
Mod = authz_module(Type), Module = authz_module(Type),
case erlang:apply(Mod, authorize, [Client, PubSub, Topic, Connector]) of case Module:authorize(Client, PubSub, Topic, Connector) of
nomatch -> do_authorize(Client, PubSub, Topic, Tail); nomatch -> do_authorize(Client, PubSub, Topic, Tail);
Matched -> Matched Matched -> Matched
end. end.
@ -367,29 +331,11 @@ find_action_in_hooks() ->
[Action] = [Action || {callback,{?MODULE, authorize, _} = Action, _, _} <- Callbacks ], [Action] = [Action || {callback,{?MODULE, authorize, _} = Action, _, _} <- Callbacks ],
Action. Action.
gen_id(Type) ->
iolist_to_binary([io_lib:format("~ts_~ts",[?APP, Type])]).
create_resource(#{type := DB} = Source) ->
ResourceID = gen_id(DB),
case emqx_resource:create(ResourceID, connector_module(DB), Source) of
{ok, already_created} -> ResourceID;
{ok, _} -> ResourceID;
{error, Reason} -> {error, Reason}
end.
authz_module('built-in-database') -> authz_module('built-in-database') ->
emqx_authz_mnesia; emqx_authz_mnesia;
authz_module(Type) -> authz_module(Type) ->
list_to_existing_atom("emqx_authz_" ++ atom_to_list(Type)). list_to_existing_atom("emqx_authz_" ++ atom_to_list(Type)).
connector_module(mongodb) ->
emqx_connector_mongo;
connector_module(postgresql) ->
emqx_connector_pgsql;
connector_module(Type) ->
list_to_existing_atom("emqx_connector_" ++ atom_to_list(Type)).
type(#{type := Type}) -> type(Type); type(#{type := Type}) -> type(Type);
type(#{<<"type">> := Type}) -> type(Type); type(#{<<"type">> := Type}) -> type(Type);
type(file) -> file; type(file) -> file;

View File

@ -34,7 +34,7 @@ start(_StartType, _StartArgs) ->
{ok, Sup}. {ok, Sup}.
stop(_State) -> stop(_State) ->
emqx_conf:remove_handler(?CONF_KEY_PATH), ok = emqx_authz:deinit(),
ok. ok.
%% internal functions %% internal functions

View File

@ -0,0 +1,61 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2020-2021 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_authz_file).
-include("emqx_authz.hrl").
-include_lib("emqx/include/logger.hrl").
-behaviour(emqx_authz).
-ifdef(TEST).
-compile(export_all).
-compile(nowarn_export_all).
-endif.
%% APIs
-export([ description/0
, init/1
, destroy/1
, dry_run/1
, authorize/4
]).
description() ->
"AuthZ with static rules".
init(#{path := Path} = Source) ->
Rules = case file:consult(Path) of
{ok, Terms} ->
[emqx_authz_rule:compile(Term) || Term <- Terms];
{error, eacces} ->
?SLOG(alert, #{msg => "insufficient_permissions_to_read_file", path => Path}),
error(eaccess);
{error, enoent} ->
?SLOG(alert, #{msg => "file_does_not_exist", path => Path}),
error(enoent);
{error, Reason} ->
?SLOG(alert, #{msg => "failed_to_read_file", path => Path, reason => Reason}),
error(Reason)
end,
Source#{annotations => #{rules => Rules}}.
destroy(_Source) -> ok.
dry_run(_Source) -> ok.
authorize(Client, PubSub, Topic, #{annotations := #{rules := Rules}}) ->
emqx_authz_rule:matches(Client, PubSub, Topic, Rules).

View File

@ -21,9 +21,14 @@
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-include_lib("emqx/include/emqx_placeholder.hrl"). -include_lib("emqx/include/emqx_placeholder.hrl").
-behaviour(emqx_authz).
%% AuthZ Callbacks %% AuthZ Callbacks
-export([ authorize/4 -export([ description/0
, description/0 , init/1
, destroy/1
, dry_run/1
, authorize/4
, parse_url/1 , parse_url/1
]). ]).
@ -35,6 +40,21 @@
description() -> description() ->
"AuthZ with http". "AuthZ with http".
init(#{url := Url} = Source) ->
NSource= maps:put(base_url, maps:remove(query, Url), Source),
case emqx_authz_utils:create_resource(emqx_connector_http, NSource) of
{error, Reason} -> error({load_config_error, Reason});
{ok, Id} -> Source#{annotations => #{id => Id}}
end.
destroy(#{annotations := #{id := Id}}) ->
ok = emqx_resource:remove(Id).
dry_run(Source) ->
URIMap = maps:get(url, Source),
NSource = maps:put(base_url, maps:remove(query, URIMap), Source),
emqx_resource:create_dry_run(emqx_connector_http, NSource).
authorize(Client, PubSub, Topic, authorize(Client, PubSub, Topic,
#{type := http, #{type := http,
url := #{path := Path} = URL, url := #{path := Path} = URL,

View File

@ -20,10 +20,15 @@
-include_lib("emqx/include/emqx.hrl"). -include_lib("emqx/include/emqx.hrl").
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-behaviour(emqx_authz).
%% AuthZ Callbacks %% AuthZ Callbacks
-export([ mnesia/1 -export([ mnesia/1
, authorize/4
, description/0 , description/0
, init/1
, destroy/1
, dry_run/1
, authorize/4
]). ]).
-ifdef(TEST). -ifdef(TEST).
@ -45,6 +50,12 @@ mnesia(boot) ->
description() -> description() ->
"AuthZ with Mnesia". "AuthZ with Mnesia".
init(Source) -> Source.
destroy(_Source) -> ok.
dry_run(_Source) -> ok.
authorize(#{username := Username, authorize(#{username := Username,
clientid := Clientid clientid := Clientid
} = Client, PubSub, Topic, #{type := 'built-in-database'}) -> } = Client, PubSub, Topic, #{type := 'built-in-database'}) ->

View File

@ -21,9 +21,14 @@
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-include_lib("emqx/include/emqx_placeholder.hrl"). -include_lib("emqx/include/emqx_placeholder.hrl").
-behaviour(emqx_authz).
%% AuthZ Callbacks %% AuthZ Callbacks
-export([ authorize/4 -export([ description/0
, description/0 , init/1
, destroy/1
, dry_run/1
, authorize/4
]). ]).
-ifdef(TEST). -ifdef(TEST).
@ -34,6 +39,18 @@
description() -> description() ->
"AuthZ with MongoDB". "AuthZ with MongoDB".
init(Source) ->
case emqx_authz_utils:create_resource(emqx_connector_mongo, Source) of
{error, Reason} -> error({load_config_error, Reason});
{ok, Id} -> Source#{annotations => #{id => Id}}
end.
dry_run(Source) ->
emqx_resource:create_dry_run(emqx_connector_mongo, Source).
destroy(#{annotations := #{id := Id}}) ->
ok = emqx_resource:remove(Id).
authorize(Client, PubSub, Topic, authorize(Client, PubSub, Topic,
#{collection := Collection, #{collection := Collection,
selector := Selector, selector := Selector,

View File

@ -21,9 +21,13 @@
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-include_lib("emqx/include/emqx_placeholder.hrl"). -include_lib("emqx/include/emqx_placeholder.hrl").
-behaviour(emqx_authz).
%% AuthZ Callbacks %% AuthZ Callbacks
-export([ description/0 -export([ description/0
, parse_query/1 , init/1
, destroy/1
, dry_run/1
, authorize/4 , authorize/4
]). ]).
@ -35,6 +39,20 @@
description() -> description() ->
"AuthZ with Mysql". "AuthZ with Mysql".
init(#{query := SQL} = Source) ->
case emqx_authz_utils:create_resource(emqx_connector_mysql, Source) of
{error, Reason} -> error({load_config_error, Reason});
{ok, Id} -> Source#{annotations =>
#{id => Id,
query => parse_query(SQL)}}
end.
dry_run(Source) ->
emqx_resource:create_dry_run(emqx_connector_mysql, Source).
destroy(#{annotations := #{id := Id}}) ->
ok = emqx_resource:remove(Id).
parse_query(undefined) -> parse_query(undefined) ->
undefined; undefined;
parse_query(Sql) -> parse_query(Sql) ->

View File

@ -21,9 +21,13 @@
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-include_lib("emqx/include/emqx_placeholder.hrl"). -include_lib("emqx/include/emqx_placeholder.hrl").
-behaviour(emqx_authz).
%% AuthZ Callbacks %% AuthZ Callbacks
-export([ description/0 -export([ description/0
, parse_query/1 , init/1
, destroy/1
, dry_run/1
, authorize/4 , authorize/4
]). ]).
@ -33,7 +37,21 @@
-endif. -endif.
description() -> description() ->
"AuthZ with postgresql". "AuthZ with Postgresql".
init(#{query := SQL} = Source) ->
case emqx_authz_utils:create_resource(emqx_connector_pgsql, Source) of
{error, Reason} -> error({load_config_error, Reason});
{ok, Id} -> Source#{annotations =>
#{id => Id,
query => parse_query(SQL)}}
end.
destroy(#{annotations := #{id := Id}}) ->
ok = emqx_resource:remove(Id).
dry_run(Source) ->
emqx_resource:create_dry_run(emqx_connector_pgsql, Source).
parse_query(undefined) -> parse_query(undefined) ->
undefined; undefined;

View File

@ -21,9 +21,14 @@
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-include_lib("emqx/include/emqx_placeholder.hrl"). -include_lib("emqx/include/emqx_placeholder.hrl").
-behaviour(emqx_authz).
%% AuthZ Callbacks %% AuthZ Callbacks
-export([ authorize/4 -export([ description/0
, description/0 , init/1
, destroy/1
, dry_run/1
, authorize/4
]). ]).
-ifdef(TEST). -ifdef(TEST).
@ -32,7 +37,19 @@
-endif. -endif.
description() -> description() ->
"AuthZ with redis". "AuthZ with Redis".
init(Source) ->
case emqx_authz_utils:create_resource(emqx_connector_redis, Source) of
{error, Reason} -> error({load_config_error, Reason});
{ok, Id} -> Source#{annotations => #{id => Id}}
end.
destroy(#{annotations := #{id := Id}}) ->
ok = emqx_resource:remove(Id).
dry_run(Source) ->
emqx_resource:create_dry_run(emqx_connector_redis, Source).
authorize(Client, PubSub, Topic, authorize(Client, PubSub, Topic,
#{cmd := CMD, #{cmd := CMD,

View File

@ -31,6 +31,20 @@
, compile/1 , compile/1
]). ]).
-type(ipaddress() :: {ipaddr, esockd_cidr:cidr_string()} |
{ipaddrs, list(esockd_cidr:cidr_string())}).
-type(username() :: {username, binary()}).
-type(clientid() :: {clientid, binary()}).
-type(who() :: ipaddress() | username() | clientid() |
{'and', [ipaddress() | username() | clientid()]} |
{'or', [ipaddress() | username() | clientid()]} |
all).
-type(rule() :: {permission(), who(), action(), list(emqx_types:topic())}).
-export_type([rule/0]). -export_type([rule/0]).
compile({Permission, all}) compile({Permission, all})

View File

@ -0,0 +1,54 @@
%%--------------------------------------------------------------------
%% Copyright (c) 2021 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_authz_utils).
-include_lib("emqx/include/emqx_placeholder.hrl").
-export([cleanup_resources/0,
make_resource_id/1,
create_resource/2]).
-define(RESOURCE_GROUP, <<"emqx_authz">>).
%%------------------------------------------------------------------------------
%% APIs
%%------------------------------------------------------------------------------
create_resource(Module, Config) ->
ResourceID = make_resource_id(Module),
case emqx_resource:create(ResourceID, Module, Config) of
{ok, already_created} -> {ok, ResourceID};
{ok, _} -> {ok, ResourceID};
{error, Reason} -> {error, Reason}
end.
cleanup_resources() ->
lists:foreach(
fun emqx_resource:remove/1,
emqx_resource:list_group_instances(?RESOURCE_GROUP)).
make_resource_id(Name) ->
NameBin = bin(Name),
emqx_resource:generate_id(?RESOURCE_GROUP, NameBin).
%%------------------------------------------------------------------------------
%% Internal functions
%%------------------------------------------------------------------------------
bin(A) when is_atom(A) -> atom_to_binary(A, utf8);
bin(L) when is_list(L) -> list_to_binary(L);
bin(X) -> X.

View File

@ -34,6 +34,7 @@ init_per_suite(Config) ->
meck:expect(emqx_resource, create, fun(_, _, _) -> {ok, meck_data} end), meck:expect(emqx_resource, create, fun(_, _, _) -> {ok, meck_data} end),
meck:expect(emqx_resource, update, fun(_, _, _, _) -> {ok, meck_data} end), meck:expect(emqx_resource, update, fun(_, _, _, _) -> {ok, meck_data} end),
meck:expect(emqx_resource, remove, fun(_) -> ok end), meck:expect(emqx_resource, remove, fun(_) -> ok end),
meck:expect(emqx_resource, create_dry_run, fun(_, _) -> ok end),
ok = emqx_common_test_helpers:start_apps( ok = emqx_common_test_helpers:start_apps(
[emqx_connector, emqx_conf, emqx_authz], [emqx_connector, emqx_conf, emqx_authz],
@ -137,6 +138,13 @@ t_update_source(_) ->
, #{type := file, enable := true} , #{type := file, enable := true}
], emqx_conf:get([authorization, sources], [])), ], emqx_conf:get([authorization, sources], [])),
{ok, _} = emqx_authz:update({?CMD_REPLACE, http}, ?SOURCE1#{<<"enable">> := true}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, mongodb}, ?SOURCE2#{<<"enable">> := true}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, mysql}, ?SOURCE3#{<<"enable">> := true}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, postgresql}, ?SOURCE4#{<<"enable">> := true}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, redis}, ?SOURCE5#{<<"enable">> := true}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, file}, ?SOURCE6#{<<"enable">> := true}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, http}, ?SOURCE1#{<<"enable">> := false}), {ok, _} = emqx_authz:update({?CMD_REPLACE, http}, ?SOURCE1#{<<"enable">> := false}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, mongodb}, ?SOURCE2#{<<"enable">> := false}), {ok, _} = emqx_authz:update({?CMD_REPLACE, mongodb}, ?SOURCE2#{<<"enable">> := false}),
{ok, _} = emqx_authz:update({?CMD_REPLACE, mysql}, ?SOURCE3#{<<"enable">> := false}), {ok, _} = emqx_authz:update({?CMD_REPLACE, mysql}, ?SOURCE3#{<<"enable">> := false}),