Merge pull request #6441 from savonarola/refactor-acl

refactor(authz): hide mnesia authz implementation details
This commit is contained in:
Ilya Averyanov 2021-12-14 11:32:25 +03:00 committed by GitHub
commit 5538cd3708
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 177 additions and 102 deletions

View File

@ -8,29 +8,6 @@
(A =:= all) orelse (A =:= <<"all">>) (A =:= all) orelse (A =:= <<"all">>)
)). )).
-define(ACL_SHARDED, emqx_acl_sharded).
-define(ACL_TABLE, emqx_acl).
%% To save some space, use an integer for label, 0 for 'all', {1, Username} and {2, ClientId}.
-define(ACL_TABLE_ALL, 0).
-define(ACL_TABLE_USERNAME, 1).
-define(ACL_TABLE_CLIENTID, 2).
-type(action() :: subscribe | publish | all).
-type(permission() :: allow | deny).
-record(emqx_acl, {
who :: ?ACL_TABLE_ALL| {?ACL_TABLE_USERNAME, binary()} | {?ACL_TABLE_CLIENTID, binary()},
rules :: [ {permission(), action(), emqx_topic:topic()} ]
}).
-record(authz_metrics, {
allow = 'client.authorize.allow',
deny = 'client.authorize.deny',
ignore = 'client.authorize.ignore'
}).
-define(CMD_REPLACE, replace). -define(CMD_REPLACE, replace).
-define(CMD_DELETE, delete). -define(CMD_DELETE, delete).
-define(CMD_PREPEND, prepend). -define(CMD_PREPEND, prepend).
@ -42,12 +19,6 @@
-define(CMD_MOVE_BEFORE(Before), {<<"before">>, Before}). -define(CMD_MOVE_BEFORE(Before), {<<"before">>, Before}).
-define(CMD_MOVE_AFTER(After), {<<"after">>, After}). -define(CMD_MOVE_AFTER(After), {<<"after">>, After}).
-define(METRICS(Type), tl(tuple_to_list(#Type{}))).
-define(METRICS(Type, K), #Type{}#Type.K).
-define(AUTHZ_METRICS, ?METRICS(authz_metrics)).
-define(AUTHZ_METRICS(K), ?METRICS(authz_metrics, K)).
-define(CONF_KEY_PATH, [authorization, sources]). -define(CONF_KEY_PATH, [authorization, sources]).
-define(RE_PLACEHOLDER, "\\$\\{[a-z0-9\\-]+\\}"). -define(RE_PLACEHOLDER, "\\$\\{[a-z0-9\\-]+\\}").

View File

@ -53,15 +53,32 @@
-type(sources() :: [source()]). -type(sources() :: [source()]).
-define(METRIC_ALLOW, 'client.authorize.allow').
-define(METRIC_DENY, 'client.authorize.deny').
-define(METRIC_NOMATCH, 'client.authorize.nomatch').
-define(METRICS, [?METRIC_ALLOW, ?METRIC_DENY, ?METRIC_NOMATCH]).
%% Initialize authz backend.
%% Populate the passed configuration map with necessary data,
%% like `ResourceID`s
-callback(init(source()) -> source()). -callback(init(source()) -> source()).
%% Get authz text description.
-callback(description() -> string()). -callback(description() -> string()).
%% Destroy authz backend.
%% Make cleanup of all allocated data.
%% An authz backend will not be used after `destroy`.
-callback(destroy(source()) -> ok). -callback(destroy(source()) -> ok).
%% Check if a configuration map is valid for further
%% authz backend initialization.
%% The callback must deallocate all resources allocated
%% during verification.
-callback(dry_run(source()) -> ok | {error, term()}). -callback(dry_run(source()) -> ok | {error, term()}).
%% Authorize client action.
-callback(authorize( -callback(authorize(
emqx_types:clientinfo(), emqx_types:clientinfo(),
emqx_types:pubsub(), emqx_types:pubsub(),
@ -70,7 +87,7 @@
-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, ?METRICS).
init() -> init() ->
ok = register_metrics(), ok = register_metrics(),
@ -273,14 +290,14 @@ authorize(#{username := Username,
username => Username, username => Username,
ipaddr => IpAddress, ipaddr => IpAddress,
topic => Topic}), topic => Topic}),
emqx_metrics:inc(?AUTHZ_METRICS(allow)), emqx_metrics:inc(?METRIC_ALLOW),
{stop, allow}; {stop, allow};
{matched, deny} -> {matched, deny} ->
?SLOG(info, #{msg => "authorization_permission_denied", ?SLOG(info, #{msg => "authorization_permission_denied",
username => Username, username => Username,
ipaddr => IpAddress, ipaddr => IpAddress,
topic => Topic}), topic => Topic}),
emqx_metrics:inc(?AUTHZ_METRICS(deny)), emqx_metrics:inc(?METRIC_DENY),
{stop, deny}; {stop, deny};
nomatch -> nomatch ->
?SLOG(info, #{msg => "authorization_failed_nomatch", ?SLOG(info, #{msg => "authorization_failed_nomatch",
@ -288,6 +305,7 @@ authorize(#{username := Username,
ipaddr => IpAddress, ipaddr => IpAddress,
topic => Topic, topic => Topic,
reason => "no-match rule"}), reason => "no-match rule"}),
emqx_metrics:inc(?METRIC_NOMATCH),
{stop, DefaultResult} {stop, DefaultResult}
end. end.

View File

@ -20,7 +20,6 @@
-include("emqx_authz.hrl"). -include("emqx_authz.hrl").
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-include_lib("stdlib/include/ms_transform.hrl").
-include_lib("typerefl/include/types.hrl"). -include_lib("typerefl/include/types.hrl").
-define(FORMAT_USERNAME_FUN, {?MODULE, format_by_username}). -define(FORMAT_USERNAME_FUN, {?MODULE, format_by_username}).
@ -269,39 +268,27 @@ fields(meta) ->
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------
users(get, #{query_string := PageParams}) -> users(get, #{query_string := PageParams}) ->
MatchSpec = ets:fun2ms( {Table, MatchSpec} = emqx_authz_mnesia:list_username_rules(),
fun({?ACL_TABLE, {?ACL_TABLE_USERNAME, Username}, Rules}) -> {200, emqx_mgmt_api:paginate(Table, MatchSpec, PageParams, ?FORMAT_USERNAME_FUN)};
[{username, Username}, {rules, Rules}]
end),
{200, emqx_mgmt_api:paginate(?ACL_TABLE, MatchSpec, PageParams, ?FORMAT_USERNAME_FUN)};
users(post, #{body := Body}) when is_list(Body) -> users(post, #{body := Body}) when is_list(Body) ->
lists:foreach(fun(#{<<"username">> := Username, <<"rules">> := Rules}) -> lists:foreach(fun(#{<<"username">> := Username, <<"rules">> := Rules}) ->
mria:dirty_write(#emqx_acl{ emqx_authz_mnesia:store_rules({username, Username}, format_rules(Rules))
who = {?ACL_TABLE_USERNAME, Username},
rules = format_rules(Rules)
})
end, Body), end, Body),
{204}. {204}.
clients(get, #{query_string := PageParams}) -> clients(get, #{query_string := PageParams}) ->
MatchSpec = ets:fun2ms( {Table, MatchSpec} = emqx_authz_mnesia:list_clientid_rules(),
fun({?ACL_TABLE, {?ACL_TABLE_CLIENTID, Clientid}, Rules}) -> {200, emqx_mgmt_api:paginate(Table, MatchSpec, PageParams, ?FORMAT_CLIENTID_FUN)};
[{clientid, Clientid}, {rules, Rules}]
end),
{200, emqx_mgmt_api:paginate(?ACL_TABLE, MatchSpec, PageParams, ?FORMAT_CLIENTID_FUN)};
clients(post, #{body := Body}) when is_list(Body) -> clients(post, #{body := Body}) when is_list(Body) ->
lists:foreach(fun(#{<<"clientid">> := Clientid, <<"rules">> := Rules}) -> lists:foreach(fun(#{<<"clientid">> := Clientid, <<"rules">> := Rules}) ->
mria:dirty_write(#emqx_acl{ emqx_authz_mnesia:store_rules({clientid, Clientid}, format_rules(Rules))
who = {?ACL_TABLE_CLIENTID, Clientid},
rules = format_rules(Rules)
})
end, Body), end, Body),
{204}. {204}.
user(get, #{bindings := #{username := Username}}) -> user(get, #{bindings := #{username := Username}}) ->
case mnesia:dirty_read(?ACL_TABLE, {?ACL_TABLE_USERNAME, Username}) of case emqx_authz_mnesia:get_rules({username, Username}) of
[] -> {404, #{code => <<"NOT_FOUND">>, message => <<"Not Found">>}}; not_found -> {404, #{code => <<"NOT_FOUND">>, message => <<"Not Found">>}};
[#emqx_acl{who = {?ACL_TABLE_USERNAME, Username}, rules = Rules}] -> {ok, Rules} ->
{200, #{username => Username, {200, #{username => Username,
rules => [ #{topic => Topic, rules => [ #{topic => Topic,
action => Action, action => Action,
@ -311,19 +298,16 @@ user(get, #{bindings := #{username := Username}}) ->
end; end;
user(put, #{bindings := #{username := Username}, user(put, #{bindings := #{username := Username},
body := #{<<"username">> := Username, <<"rules">> := Rules}}) -> body := #{<<"username">> := Username, <<"rules">> := Rules}}) ->
mria:dirty_write(#emqx_acl{ emqx_authz_mnesia:store_rules({username, Username}, format_rules(Rules)),
who = {?ACL_TABLE_USERNAME, Username},
rules = format_rules(Rules)
}),
{204}; {204};
user(delete, #{bindings := #{username := Username}}) -> user(delete, #{bindings := #{username := Username}}) ->
mria:dirty_delete({?ACL_TABLE, {?ACL_TABLE_USERNAME, Username}}), emqx_authz_mnesia:delete_rules({username, Username}),
{204}. {204}.
client(get, #{bindings := #{clientid := Clientid}}) -> client(get, #{bindings := #{clientid := Clientid}}) ->
case mnesia:dirty_read(?ACL_TABLE, {?ACL_TABLE_CLIENTID, Clientid}) of case emqx_authz_mnesia:get_rules({clientid, Clientid}) of
[] -> {404, #{code => <<"NOT_FOUND">>, message => <<"Not Found">>}}; not_found -> {404, #{code => <<"NOT_FOUND">>, message => <<"Not Found">>}};
[#emqx_acl{who = {?ACL_TABLE_CLIENTID, Clientid}, rules = Rules}] -> {ok, Rules} ->
{200, #{clientid => Clientid, {200, #{clientid => Clientid,
rules => [ #{topic => Topic, rules => [ #{topic => Topic,
action => Action, action => Action,
@ -333,20 +317,17 @@ client(get, #{bindings := #{clientid := Clientid}}) ->
end; end;
client(put, #{bindings := #{clientid := Clientid}, client(put, #{bindings := #{clientid := Clientid},
body := #{<<"clientid">> := Clientid, <<"rules">> := Rules}}) -> body := #{<<"clientid">> := Clientid, <<"rules">> := Rules}}) ->
mria:dirty_write(#emqx_acl{ emqx_authz_mnesia:store_rules({clientid, Clientid}, format_rules(Rules)),
who = {?ACL_TABLE_CLIENTID, Clientid},
rules = format_rules(Rules)
}),
{204}; {204};
client(delete, #{bindings := #{clientid := Clientid}}) -> client(delete, #{bindings := #{clientid := Clientid}}) ->
mria:dirty_delete({?ACL_TABLE, {?ACL_TABLE_CLIENTID, Clientid}}), emqx_authz_mnesia:delete_rules({clientid, Clientid}),
{204}. {204}.
all(get, _) -> all(get, _) ->
case mnesia:dirty_read(?ACL_TABLE, ?ACL_TABLE_ALL) of case emqx_authz_mnesia:get_rules(all) of
[] -> not_found ->
{200, #{rules => []}}; {200, #{rules => []}};
[#emqx_acl{who = ?ACL_TABLE_ALL, rules = Rules}] -> {ok, Rules} ->
{200, #{rules => [ #{topic => Topic, {200, #{rules => [ #{topic => Topic,
action => Action, action => Action,
permission => Permission permission => Permission
@ -354,18 +335,13 @@ all(get, _) ->
} }
end; end;
all(put, #{body := #{<<"rules">> := Rules}}) -> all(put, #{body := #{<<"rules">> := Rules}}) ->
mria:dirty_write(#emqx_acl{ emqx_authz_mnesia:store_rules(all, format_rules(Rules)),
who = ?ACL_TABLE_ALL,
rules = format_rules(Rules)
}),
{204}. {204}.
purge(delete, _) -> purge(delete, _) ->
case emqx_authz_api_sources:get_raw_source(<<"built-in-database">>) of case emqx_authz_api_sources:get_raw_source(<<"built-in-database">>) of
[#{<<"enable">> := false}] -> [#{<<"enable">> := false}] ->
ok = lists:foreach(fun(Key) -> ok = emqx_authz_mnesia:purge_rules(),
ok = mria:dirty_delete(?ACL_TABLE, Key)
end, mnesia:dirty_all_keys(?ACL_TABLE)),
{204}; {204};
[#{<<"enable">> := true}] -> [#{<<"enable">> := true}] ->
{400, #{code => <<"BAD_REQUEST">>, {400, #{code => <<"BAD_REQUEST">>,

View File

@ -23,12 +23,10 @@
-behaviour(application). -behaviour(application).
-include("emqx_authz.hrl").
-export([start/2, stop/1]). -export([start/2, stop/1]).
start(_StartType, _StartArgs) -> start(_StartType, _StartArgs) ->
ok = mria_rlog:wait_for_shards([?ACL_SHARDED], infinity), ok = emqx_authz_mnesia:init_tables(),
{ok, Sup} = emqx_authz_sup:start_link(), {ok, Sup} = emqx_authz_sup:start_link(),
ok = emqx_authz:init(), ok = emqx_authz:init(),
{ok, Sup}. {ok, Sup}.
@ -36,5 +34,3 @@ start(_StartType, _StartArgs) ->
stop(_State) -> stop(_State) ->
ok = emqx_authz:deinit(), ok = emqx_authz:deinit(),
ok. ok.
%% internal functions

View File

@ -16,21 +16,53 @@
-module(emqx_authz_mnesia). -module(emqx_authz_mnesia).
-include("emqx_authz.hrl").
-include_lib("emqx/include/emqx.hrl"). -include_lib("emqx/include/emqx.hrl").
-include_lib("stdlib/include/ms_transform.hrl").
-include_lib("emqx/include/logger.hrl"). -include_lib("emqx/include/logger.hrl").
-define(ACL_SHARDED, emqx_acl_sharded).
-define(ACL_TABLE, emqx_acl).
%% To save some space, use an integer for label, 0 for 'all', {1, Username} and {2, ClientId}.
-define(ACL_TABLE_ALL, 0).
-define(ACL_TABLE_USERNAME, 1).
-define(ACL_TABLE_CLIENTID, 2).
-type(username() :: {username, binary()}).
-type(clientid() :: {clientid, binary()}).
-type(who() :: username() | clientid() | all).
-type(rule() :: {emqx_authz_rule:permission(), emqx_authz_rule:action(), emqx_topic:topic()}).
-type(rules() :: [rule()]).
-record(emqx_acl, {
who :: ?ACL_TABLE_ALL | {?ACL_TABLE_USERNAME, binary()} | {?ACL_TABLE_CLIENTID, binary()},
rules :: rules()
}).
-behaviour(emqx_authz). -behaviour(emqx_authz).
%% AuthZ Callbacks %% AuthZ Callbacks
-export([ mnesia/1 -export([ description/0
, description/0
, init/1 , init/1
, destroy/1 , destroy/1
, dry_run/1 , dry_run/1
, authorize/4 , authorize/4
]). ]).
%% Management API
-export([ mnesia/1
, init_tables/0
, store_rules/2
, purge_rules/0
, get_rules/1
, delete_rules/1
, list_clientid_rules/0
, list_username_rules/0
, record_count/0
]).
-ifdef(TEST). -ifdef(TEST).
-compile(export_all). -compile(export_all).
-compile(nowarn_export_all). -compile(nowarn_export_all).
@ -47,6 +79,10 @@ mnesia(boot) ->
{attributes, record_info(fields, ?ACL_TABLE)}, {attributes, record_info(fields, ?ACL_TABLE)},
{storage_properties, [{ets, [{read_concurrency, true}]}]}]). {storage_properties, [{ets, [{read_concurrency, true}]}]}]).
%%--------------------------------------------------------------------
%% emqx_authz callbacks
%%--------------------------------------------------------------------
description() -> description() ->
"AuthZ with Mnesia". "AuthZ with Mnesia".
@ -74,6 +110,78 @@ authorize(#{username := Username,
end, end,
do_authorize(Client, PubSub, Topic, Rules). do_authorize(Client, PubSub, Topic, Rules).
%%--------------------------------------------------------------------
%% Management API
%%--------------------------------------------------------------------
init_tables() ->
ok = mria_rlog:wait_for_shards([?ACL_SHARDED], infinity).
-spec(store_rules(who(), rules()) -> ok).
store_rules({username, Username}, Rules) ->
Record = #emqx_acl{who = {?ACL_TABLE_USERNAME, Username}, rules = Rules},
mria:dirty_write(Record);
store_rules({clientid, Clientid}, Rules) ->
Record = #emqx_acl{who = {?ACL_TABLE_CLIENTID, Clientid}, rules = Rules},
mria:dirty_write(Record);
store_rules(all, Rules) ->
Record = #emqx_acl{who = ?ACL_TABLE_ALL, rules = Rules},
mria:dirty_write(Record).
-spec(purge_rules() -> ok).
purge_rules() ->
ok = lists:foreach(
fun(Key) ->
ok = mria:dirty_delete(?ACL_TABLE, Key)
end,
mnesia:dirty_all_keys(?ACL_TABLE)).
-spec(get_rules(who()) -> {ok, rules()} | not_found).
get_rules({username, Username}) ->
do_get_rules({?ACL_TABLE_USERNAME, Username});
get_rules({clientid, Clientid}) ->
do_get_rules({?ACL_TABLE_CLIENTID, Clientid});
get_rules(all) ->
do_get_rules(?ACL_TABLE_ALL).
-spec(delete_rules(who()) -> ok).
delete_rules({username, Username}) ->
mria:dirty_delete(?ACL_TABLE, {?ACL_TABLE_USERNAME, Username});
delete_rules({clientid, Clientid}) ->
mria:dirty_delete(?ACL_TABLE, {?ACL_TABLE_CLIENTID, Clientid});
delete_rules(all) ->
mria:dirty_delete(?ACL_TABLE, ?ACL_TABLE_ALL).
-spec(list_username_rules() -> {mria:table(), ets:match_spec()}).
list_username_rules() ->
MatchSpec = ets:fun2ms(
fun(#emqx_acl{who = {?ACL_TABLE_USERNAME, Username}, rules = Rules}) ->
[{username, Username}, {rules, Rules}]
end),
{?ACL_TABLE, MatchSpec}.
-spec(list_clientid_rules() -> {mria:table(), ets:match_spec()}).
list_clientid_rules() ->
MatchSpec = ets:fun2ms(
fun(#emqx_acl{who = {?ACL_TABLE_CLIENTID, Clientid}, rules = Rules}) ->
[{clientid, Clientid}, {rules, Rules}]
end),
{?ACL_TABLE, MatchSpec}.
-spec(record_count() -> non_neg_integer()).
record_count() ->
mnesia:table_info(?ACL_TABLE, size).
%%--------------------------------------------------------------------
%% Internal functions
%%--------------------------------------------------------------------
do_get_rules(Key) ->
case mnesia:dirty_read(?ACL_TABLE, Key) of
[#emqx_acl{rules = Rules}] -> {ok, Rules};
[] -> not_found
end.
do_authorize(_Client, _PubSub, _Topic, []) -> nomatch; do_authorize(_Client, _PubSub, _Topic, []) -> nomatch;
do_authorize(Client, PubSub, Topic, [ {Permission, Action, TopicFilter} | Tail]) -> do_authorize(Client, PubSub, Topic, [ {Permission, Action, TopicFilter} | Tail]) ->
case emqx_authz_rule:match(Client, PubSub, Topic, case emqx_authz_rule:match(Client, PubSub, Topic,

View File

@ -43,9 +43,14 @@
{'or', [ipaddress() | username() | clientid()]} | {'or', [ipaddress() | username() | clientid()]} |
all). all).
-type(action() :: subscribe | publish | all).
-type(permission() :: allow | deny).
-type(rule() :: {permission(), who(), action(), list(emqx_types:topic())}). -type(rule() :: {permission(), who(), action(), list(emqx_types:topic())}).
-export_type([rule/0]). -export_type([ action/0
, permission/0
]).
compile({Permission, all}) compile({Permission, all})
when ?ALLOW_DENY(Permission) -> {Permission, all, all, [compile_topic(<<"#">>)]}; when ?ALLOW_DENY(Permission) -> {Permission, all, all, [compile_topic(<<"#">>)]};

View File

@ -217,7 +217,7 @@ t_api(_) ->
request( delete request( delete
, uri(["authorization", "sources", "built-in-database", "purge-all"]) , uri(["authorization", "sources", "built-in-database", "purge-all"])
, []), , []),
?assertEqual([], mnesia:dirty_all_keys(?ACL_TABLE)), ?assertEqual(0, emqx_authz_mnesia:record_count()),
ok. ok.
%%-------------------------------------------------------------------- %%--------------------------------------------------------------------

View File

@ -55,24 +55,25 @@ set_special_configs(_App) ->
ok. ok.
init_per_testcase(t_authz, Config) -> init_per_testcase(t_authz, Config) ->
mria:dirty_write(#emqx_acl{who = {?ACL_TABLE_USERNAME, <<"test_username">>}, emqx_authz_mnesia:store_rules(
rules = [{allow, publish, <<"test/", ?PH_S_USERNAME>>}, {username, <<"test_username">>},
{allow, subscribe, <<"eq #">>} [{allow, publish, <<"test/", ?PH_S_USERNAME>>},
] {allow, subscribe, <<"eq #">>}]),
}),
mria:dirty_write(#emqx_acl{who = {?ACL_TABLE_CLIENTID, <<"test_clientid">>}, emqx_authz_mnesia:store_rules(
rules = [{allow, publish, <<"test/", ?PH_S_CLIENTID>>}, {clientid, <<"test_clientid">>},
{deny, subscribe, <<"eq #">>} [{allow, publish, <<"test/", ?PH_S_CLIENTID>>},
] {deny, subscribe, <<"eq #">>}]),
}),
mria:dirty_write(#emqx_acl{who = ?ACL_TABLE_ALL, emqx_authz_mnesia:store_rules(
rules = [{deny, all, <<"#">>}] all,
}), [{deny, all, <<"#">>}]),
Config; Config;
init_per_testcase(_, Config) -> Config. init_per_testcase(_, Config) -> Config.
end_per_testcase(t_authz, Config) -> end_per_testcase(t_authz, Config) ->
[ mria:dirty_delete(?ACL_TABLE, K) || K <- mnesia:dirty_all_keys(?ACL_TABLE)], ok = emqx_authz_mnesia:purge_rules(),
Config; Config;
end_per_testcase(_, Config) -> Config. end_per_testcase(_, Config) -> Config.