feat(authz mnesia): add api

Signed-off-by: zhanghongtong <rory-z@outlook.com>
This commit is contained in:
zhanghongtong 2021-09-16 18:11:43 +08:00
parent acb5c693ba
commit ed6f4895e2
8 changed files with 866 additions and 5 deletions

View File

@ -19,6 +19,15 @@
-type(sources() :: [map()]).
-define(ACL_SHARDED, emqx_acl_sharded).
-define(ACL_TABLE, emqx_acl).
-record(emqx_acl, {
who :: username() | clientid() | all,
rules :: [ {permission(), action(), emqx_topic:topic()} ]
}).
-define(APP, emqx_authz).
-define(ALLOW_DENY(A), ((A =:= allow) orelse (A =:= <<"allow">>) orelse

View File

@ -39,7 +39,7 @@
-export([post_config_update/4, pre_config_update/2]).
-define(CONF_KEY_PATH, [authorization, sources]).
-define(SOURCE_TYPES, [file, http, mongodb, mysql, postgresql, redis]).
-define(SOURCE_TYPES, [file, http, mongodb, mysql, postgresql, redis, 'built-in-database']).
-spec(register_metrics() -> ok).
register_metrics() ->
@ -297,6 +297,9 @@ init_source(#{enable := true,
{error, Reason} -> error({load_config_error, Reason});
Id -> Source#{annotations => #{id => Id}}
end;
init_source(#{enable := true,
type := 'built-in-database'
} = Source) -> Source;
init_source(#{enable := true,
type := DB
} = Source) when DB =:= redis;

View File

@ -0,0 +1,543 @@
%%--------------------------------------------------------------------
%% 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_api_mnesia).
-behavior(minirest_api).
-include("emqx_authz.hrl").
-include_lib("emqx/include/logger.hrl").
-include_lib("stdlib/include/ms_transform.hrl").
-define(EXAMPLE_USERNAME, #{type => username,
key => user1,
rules => [ #{topic => <<"test/toopic/1">>,
permission => <<"allow">>,
action => <<"publish">>
}
, #{topic => <<"test/toopic/2">>,
permission => <<"allow">>,
action => <<"subscribe">>
}
, #{topic => <<"eq test/#">>,
permission => <<"deny">>,
action => <<"all">>
}
]
}).
-define(EXAMPLE_CLIENTID, #{type => clientid,
key => client1,
rules => [ #{topic => <<"test/toopic/1">>,
permission => <<"allow">>,
action => <<"publish">>
}
, #{topic => <<"test/toopic/2">>,
permission => <<"allow">>,
action => <<"subscribe">>
}
, #{topic => <<"eq test/#">>,
permission => <<"deny">>,
action => <<"all">>
}
]
}).
-define(EXAMPLE_ALL , #{type => all,
rules => [ #{topic => <<"test/toopic/1">>,
permission => <<"allow">>,
action => <<"publish">>
}
, #{topic => <<"test/toopic/2">>,
permission => <<"allow">>,
action => <<"subscribe">>
}
, #{topic => <<"eq test/#">>,
permission => <<"deny">>,
action => <<"all">>
}
]
}).
-export([ api_spec/0
, purge/2
, tickets/2
, ticket/2
]).
api_spec() ->
{[ purge_api()
, tickets_api()
, ticket_api()
], definitions()}.
definitions() ->
Rules = #{
type => array,
items => #{
type => object,
required => [topic, permission, action],
properties => #{
topic => #{
type => string,
example => <<"test/topic/1">>
},
permission => #{
type => string,
enum => [<<"allow">>, <<"deny">>],
example => <<"allow">>
},
action => #{
type => string,
enum => [<<"publish">>, <<"subscribe">>, <<"all">>],
example => <<"publish">>
}
}
}
},
Ticket = #{
oneOf => [ #{type => object,
required => [username, rules],
properties => #{
username => #{
type => string,
example => <<"username">>
},
rules => minirest:ref(<<"rules">>)
}
}
, #{type => object,
required => [cleitnid, rules],
properties => #{
username => #{
type => string,
example => <<"clientid">>
},
rules => minirest:ref(<<"rules">>)
}
}
, #{type => object,
required => [rules],
properties => #{
rules => minirest:ref(<<"rules">>)
}
}
]
},
[ #{<<"rules">> => Rules}
, #{<<"ticket">> => Ticket}
].
purge_api() ->
Metadata = #{
delete => #{
description => "Purge all tickets",
responses => #{
<<"204">> => #{description => <<"No Content">>},
<<"400">> => emqx_mgmt_util:bad_request()
}
}
},
{"/authorization/sources/built-in-database/purge-all", Metadata, purge}.
tickets_api() ->
Metadata = #{
get => #{
description => "List tickets",
parameters => [
#{
name => type,
in => path,
schema => #{
type => string,
enum => [<<"username">>, <<"clientid">>, <<"all">>]
},
required => true
}
],
responses => #{
<<"200">> => #{
description => <<"OK">>,
content => #{
'application/json' => #{
schema => #{
type => array,
items => minirest:ref(<<"ticket">>)
},
examples => #{
username => #{
summary => <<"Username">>,
value => jsx:encode([?EXAMPLE_USERNAME])
},
clientid => #{
summary => <<"Clientid">>,
value => jsx:encode([?EXAMPLE_CLIENTID])
},
all => #{
summary => <<"All">>,
value => jsx:encode([?EXAMPLE_ALL])
}
}
}
}
}
}
},
post => #{
description => "Add new tickets",
parameters => [
#{
name => type,
in => path,
schema => #{
type => string,
enum => [<<"username">>, <<"clientid">>]
},
required => true
}
],
requestBody => #{
content => #{
'application/json' => #{
schema => minirest:ref(<<"ticket">>),
examples => #{
username => #{
summary => <<"Username">>,
value => jsx:encode(?EXAMPLE_USERNAME)
},
clientid => #{
summary => <<"Clientid">>,
value => jsx:encode(?EXAMPLE_CLIENTID)
}
}
}
}
},
responses => #{
<<"204">> => #{description => <<"Created">>},
<<"400">> => emqx_mgmt_util:bad_request()
}
},
put => #{
description => "Set the list of rules for all",
parameters => [
#{
name => type,
in => path,
schema => #{
type => string,
enum => [<<"all">>]
},
required => true
}
],
requestBody => #{
content => #{
'application/json' => #{
schema => minirest:ref(<<"ticket">>),
examples => #{
all => #{
summary => <<"All">>,
value => jsx:encode(?EXAMPLE_ALL)
}
}
}
}
},
responses => #{
<<"204">> => #{description => <<"Created">>},
<<"400">> => emqx_mgmt_util:bad_request()
}
}
},
{"/authorization/sources/built-in-database/:type", Metadata, tickets}.
ticket_api() ->
Metadata = #{
get => #{
description => "Get ticket info",
parameters => [
#{
name => type,
in => path,
schema => #{
type => string,
enum => [<<"username">>, <<"clientid">>]
},
required => true
},
#{
name => key,
in => path,
schema => #{
type => string
},
required => true
}
],
responses => #{
<<"200">> => #{
description => <<"OK">>,
content => #{
'application/json' => #{
schema => minirest:ref(<<"ticket">>),
examples => #{
username => #{
summary => <<"Username">>,
value => jsx:encode(?EXAMPLE_USERNAME)
},
clientid => #{
summary => <<"Clientid">>,
value => jsx:encode(?EXAMPLE_CLIENTID)
},
all => #{
summary => <<"All">>,
value => jsx:encode(?EXAMPLE_ALL)
}
}
}
}
},
<<"404">> => emqx_mgmt_util:bad_request(<<"Not Found">>)
}
},
put => #{
description => "Update one ticket",
parameters => [
#{
name => type,
in => path,
schema => #{
type => string,
enum => [<<"username">>, <<"clientid">>]
},
required => true
},
#{
name => key,
in => path,
schema => #{
type => string
},
required => true
}
],
requestBody => #{
content => #{
'application/json' => #{
schema => minirest:ref(<<"ticket">>),
examples => #{
username => #{
summary => <<"Username">>,
value => jsx:encode(?EXAMPLE_USERNAME)
},
clientid => #{
summary => <<"Clientid">>,
value => jsx:encode(?EXAMPLE_CLIENTID)
}
}
}
}
},
responses => #{
<<"204">> => #{description => <<"Updated">>},
<<"400">> => emqx_mgmt_util:bad_request()
}
},
delete => #{
description => "Delete one ticket",
parameters => [
#{
name => type,
in => path,
schema => #{
type => string,
enum => [<<"username">>, <<"clientid">>]
},
required => true
},
#{
name => key,
in => path,
schema => #{
type => string
},
required => true
}
],
responses => #{
<<"204">> => #{description => <<"No Content">>},
<<"400">> => emqx_mgmt_util:bad_request()
}
}
},
{"/authorization/sources/built-in-database/:type/:key", Metadata, ticket}.
purge(delete, _) ->
[ mnesia:dirty_delete(?ACL_TABLE, K) || K <- mnesia:dirty_all_keys(?ACL_TABLE)],
{204}.
tickets(get, #{bindings := #{type := <<"username">>}}) ->
MatchSpec = ets:fun2ms(
fun({?ACL_TABLE, {username, Username}, Rules}) ->
[{username, Username}, {rules, Rules}]
end),
{200, [ #{username => Username,
rules => [ #{topic => Topic,
action => Action,
permission => Permission
} || {Permission, Action, Topic} <- Rules]
} || [{username, Username}, {rules, Rules}] <- ets:select(?ACL_TABLE, MatchSpec)]};
tickets(get, #{bindings := #{type := <<"clientid">>}}) ->
MatchSpec = ets:fun2ms(
fun({?ACL_TABLE, {clientid, Clientid}, Rules}) ->
[{clientid, Clientid}, {rules, Rules}]
end),
{200, [ #{clientid => Clientid,
rules => [ #{topic => Topic,
action => Action,
permission => Permission
} || {Permission, Action, Topic} <- Rules]
} || [{clientid, Clientid}, {rules, Rules}] <- ets:select(?ACL_TABLE, MatchSpec)]};
tickets(get, #{bindings := #{type := <<"all">>}}) ->
MatchSpec = ets:fun2ms(
fun({?ACL_TABLE, all, Rules}) ->
[{rules, Rules}]
end),
{200, [ #{rules => [ #{topic => Topic,
action => Action,
permission => Permission
} || {Permission, Action, Topic} <- Rules]
} || [{rules, Rules}] <- ets:select(?ACL_TABLE, MatchSpec)]};
tickets(post, #{bindings := #{type := <<"username">>},
body := #{<<"username">> := Username, <<"rules">> := Rules}}) ->
Ticket = #emqx_acl{
who = {username, Username},
rules = format_rules(Rules)
},
case ret(mnesia:transaction(fun insert/1, [Ticket])) of
ok -> {204};
{error, Reason} ->
{400, #{code => <<"BAD_REQUEST">>,
message => atom_to_binary(Reason)}}
end;
tickets(post, #{bindings := #{type := <<"clientid">>},
body := #{<<"clientid">> := Clientid, <<"rules">> := Rules}}) ->
Ticket = #emqx_acl{
who = {clientid, Clientid},
rules = format_rules(Rules)
},
case ret(mnesia:transaction(fun insert/1, [Ticket])) of
ok -> {204};
{error, Reason} ->
{400, #{code => <<"BAD_REQUEST">>,
message => atom_to_binary(Reason)}}
end;
tickets(put, #{bindings := #{type := <<"all">>},
body := #{<<"rules">> := Rules}}) ->
Ticket = #emqx_acl{
who = all,
rules = format_rules(Rules)
},
case ret(mnesia:transaction(fun mnesia:write/1, [Ticket])) of
ok -> {204};
{error, Reason} ->
{400, #{code => <<"BAD_REQUEST">>,
message => atom_to_binary(Reason)}}
end.
ticket(get, #{bindings := #{type := <<"username">>, key := Key}}) ->
case mnesia:dirty_read(?ACL_TABLE, {username, Key}) of
[] -> {404, #{code => <<"NOT_FOUND">>, message => <<"Not Found">>}};
[#emqx_acl{who = {username, Username}, rules = Rules}] ->
{200, #{username => Username,
rules => [ #{topic => Topic,
action => Action,
permission => Permission
} || {Permission, Action, Topic} <- Rules]}
}
end;
ticket(get, #{bindings := #{type := <<"clientid">>, key := Key}}) ->
case mnesia:dirty_read(?ACL_TABLE, {clientid, Key}) of
[] -> {404, #{code => <<"NOT_FOUND">>, message => <<"Not Found">>}};
[#emqx_acl{who = {clientid, Clientid}, rules = Rules}] ->
{200, #{clientid => Clientid,
rules => [ #{topic => Topic,
action => Action,
permission => Permission
} || {Permission, Action, Topic} <- Rules]}
}
end;
ticket(put, #{bindings := #{type := <<"username">>, key := Username},
body := #{<<"username">> := Username, <<"rules">> := Rules}}) ->
case ret(mnesia:transaction(fun update/2, [{username, Username}, format_rules(Rules)])) of
ok -> {204};
{error, Reason} ->
{400, #{code => <<"BAD_REQUEST">>,
message => atom_to_binary(Reason)}}
end;
ticket(put, #{bindings := #{type := <<"clientid">>, key := Clientid},
body := #{<<"clientid">> := Clientid, <<"rules">> := Rules}}) ->
case ret(mnesia:transaction(fun update/2, [{clientid, Clientid}, format_rules(Rules)])) of
ok -> {204};
{error, Reason} ->
{400, #{code => <<"BAD_REQUEST">>,
message => atom_to_binary(Reason)}}
end;
ticket(delete, #{bindings := #{type := <<"username">>, key := Key}}) ->
case ret(mnesia:transaction(fun mnesia:delete/1, [{?ACL_TABLE, {username, Key}}])) of
ok -> {204};
{error, Reason} ->
{400, #{code => <<"BAD_REQUEST">>,
message => atom_to_binary(Reason)}}
end;
ticket(delete, #{bindings := #{type := <<"clientid">>, key := Key}}) ->
case ret(mnesia:transaction(fun mnesia:delete/1, [{?ACL_TABLE, {clientid, Key}}])) of
ok -> {204};
{error, Reason} ->
{400, #{code => <<"BAD_REQUEST">>,
message => atom_to_binary(Reason)}}
end.
format_rules(Rules) when is_list(Rules) ->
lists:foldl(fun(#{<<"topic">> := Topic,
<<"action">> := Action,
<<"permission">> := Permission
}, AccIn) when ?PUBSUB(Action)
andalso ?ALLOW_DENY(Permission) ->
AccIn ++ [{ atom(Permission), atom(Action), Topic }]
end, [], Rules).
atom(B) when is_binary(B) ->
try binary_to_existing_atom(B, utf8)
catch
_ -> binary_to_atom(B)
end;
atom(A) when is_atom(A) -> A.
insert(Ticket = #emqx_acl{who = Who}) ->
case mnesia:read(?ACL_TABLE, Who) of
[] -> mnesia:write(Ticket);
[_|_] -> mnesia:abort(existed)
end.
update(Who, Rules) ->
case mnesia:read(?ACL_TABLE, Who) of
[#emqx_acl{} = Ticket] ->
mnesia:write(Ticket#emqx_acl{rules = Rules});
[] -> mnesia:abort(noexisted)
end.
ret({atomic, ok}) -> ok;
ret({aborted, Error}) -> {error, Error}.

View File

@ -147,7 +147,15 @@ source_api() ->
name => type,
in => path,
schema => #{
type => string
type => string,
enum => [ <<"file">>
, <<"http">>
, <<"mongodb">>
, <<"mysql">>
, <<"postgresql">>
, <<"redis">>
, <<"built-in-database">>
]
},
required => true
}
@ -181,7 +189,15 @@ source_api() ->
name => type,
in => path,
schema => #{
type => string
type => string,
enum => [ <<"file">>
, <<"http">>
, <<"mongodb">>
, <<"mysql">>
, <<"postgresql">>
, <<"redis">>
, <<"built-in-database">>
]
},
required => true
}
@ -216,7 +232,15 @@ source_api() ->
name => type,
in => path,
schema => #{
type => string
type => string,
enum => [ <<"file">>
, <<"http">>
, <<"mongodb">>
, <<"mysql">>
, <<"postgresql">>
, <<"redis">>
, <<"built-in-database">>
]
},
required => true
}
@ -238,7 +262,15 @@ move_source_api() ->
name => type,
in => path,
schema => #{
type => string
type => string,
enum => [ <<"file">>
, <<"http">>
, <<"mongodb">>
, <<"mysql">>
, <<"postgresql">>
, <<"redis">>
, <<"built-in-database">>
]
},
required => true
}

View File

@ -7,9 +7,12 @@
-behaviour(application).
-include("emqx_authz.hrl").
-export([start/2, stop/1]).
start(_StartType, _StartArgs) ->
ok = ekka_rlog:wait_for_shards([?ACL_SHARDED], infinity),
{ok, Sup} = emqx_authz_sup:start_link(),
ok = emqx_authz:init(),
{ok, Sup}.

View File

@ -0,0 +1,54 @@
%%--------------------------------------------------------------------
%% 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_mnesia).
-include("emqx_authz.hrl").
-include_lib("emqx/include/emqx.hrl").
-include_lib("emqx/include/logger.hrl").
%% AuthZ Callbacks
-export([ mnesia/1
, authorize/4
, description/0
]).
-ifdef(TEST).
-compile(export_all).
-compile(nowarn_export_all).
-endif.
-boot_mnesia({mnesia, [boot]}).
-copy_mnesia({mnesia, [copy]}).
-spec(mnesia(boot | copy) -> ok).
mnesia(boot) ->
ok = ekka_mnesia:create_table(?ACL_TABLE, [
{type, ordered_set},
{rlog_shard, ?ACL_SHARDED},
{disc_copies, [node()]},
{attributes, record_info(fields, ?ACL_TABLE)},
{storage_properties, [{ets, [{read_concurrency, true}]}]}]);
mnesia(copy) ->
ok = ekka_mnesia:copy_table(?ACL_TABLE, disc_copies).
description() ->
"AuthZ with Mnesia".
authorize(#{username := _Username,
clientid := _Clientid
} = _Client, _PubSub, _Topic, #{type := mnesia}) ->
ok.

View File

@ -31,6 +31,7 @@ fields("authorization") ->
[ hoconsc:ref(?MODULE, file)
, hoconsc:ref(?MODULE, http_get)
, hoconsc:ref(?MODULE, http_post)
, hoconsc:ref(?MODULE, mnesia)
, hoconsc:ref(?MODULE, mongo_single)
, hoconsc:ref(?MODULE, mongo_rs)
, hoconsc:ref(?MODULE, mongo_sharded)
@ -115,6 +116,11 @@ fields(http_post) ->
}
}
] ++ proplists:delete(base_url, emqx_connector_http:fields(config));
fields(mnesia) ->
[ {type, #{type => 'built-in-database'}}
, {enable, #{type => boolean(),
default => true}}
];
fields(mongo_single) ->
[ {collection, #{type => atom()}}
, {selector, #{type => map()}}

View File

@ -0,0 +1,211 @@
%%--------------------------------------------------------------------
%% 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_api_mnesia_SUITE).
-compile(nowarn_export_all).
-compile(export_all).
-include("emqx_authz.hrl").
-include_lib("eunit/include/eunit.hrl").
-include_lib("common_test/include/ct.hrl").
-define(CONF_DEFAULT, <<"authorization: {sources: []}">>).
-import(emqx_ct_http, [ request_api/3
, request_api/5
, get_http_data/1
, create_default_app/0
, delete_default_app/0
, default_auth_header/0
, auth_header/2
]).
-define(HOST, "http://127.0.0.1:18083/").
-define(API_VERSION, "v5").
-define(BASE_PATH, "api").
-define(EXAMPLE_USERNAME, #{username => user1,
rules => [ #{topic => <<"test/toopic/1">>,
permission => <<"allow">>,
action => <<"publish">>
}
, #{topic => <<"test/toopic/2">>,
permission => <<"allow">>,
action => <<"subscribe">>
}
, #{topic => <<"eq test/#">>,
permission => <<"deny">>,
action => <<"all">>
}
]
}).
-define(EXAMPLE_CLIENTID, #{clientid => client1,
rules => [ #{topic => <<"test/toopic/1">>,
permission => <<"allow">>,
action => <<"publish">>
}
, #{topic => <<"test/toopic/2">>,
permission => <<"allow">>,
action => <<"subscribe">>
}
, #{topic => <<"eq test/#">>,
permission => <<"deny">>,
action => <<"all">>
}
]
}).
-define(EXAMPLE_ALL , #{rules => [ #{topic => <<"test/toopic/1">>,
permission => <<"allow">>,
action => <<"publish">>
}
, #{topic => <<"test/toopic/2">>,
permission => <<"allow">>,
action => <<"subscribe">>
}
, #{topic => <<"eq test/#">>,
permission => <<"deny">>,
action => <<"all">>
}
]
}).
all() ->
emqx_ct:all(?MODULE).
groups() ->
[].
init_per_suite(Config) ->
meck:new(emqx_schema, [non_strict, passthrough, no_history, no_link]),
meck:expect(emqx_schema, fields, fun("authorization") ->
meck:passthrough(["authorization"]) ++
emqx_authz_schema:fields("authorization");
(F) -> meck:passthrough([F])
end),
ok = emqx_config:init_load(emqx_authz_schema, ?CONF_DEFAULT),
ok = emqx_ct_helpers:start_apps([emqx_authz, emqx_dashboard], fun set_special_configs/1),
{ok, _} = emqx:update_config([authorization, cache, enable], false),
{ok, _} = emqx:update_config([authorization, no_match], deny),
Config.
end_per_suite(_Config) ->
{ok, _} = emqx_authz:update(replace, []),
emqx_ct_helpers:stop_apps([emqx_authz, emqx_dashboard]),
meck:unload(emqx_schema),
ok.
set_special_configs(emqx_dashboard) ->
Config = #{
default_username => <<"admin">>,
default_password => <<"public">>,
listeners => [#{
protocol => http,
port => 18083
}]
},
emqx_config:put([emqx_dashboard], Config),
ok;
set_special_configs(emqx_authz) ->
emqx_config:put([authorization], #{sources => [#{type => 'built-in-database',
enable => true}
]}),
ok;
set_special_configs(_App) ->
ok.
%%------------------------------------------------------------------------------
%% Testcases
%%------------------------------------------------------------------------------
t_api(_) ->
{ok, 204, _} = request(post, uri(["authorization", "sources", "built-in-database", "username"]), ?EXAMPLE_USERNAME),
{ok, 200, Request1} = request(get, uri(["authorization", "sources", "built-in-database", "username"]), []),
{ok, 200, Request2} = request(get, uri(["authorization", "sources", "built-in-database", "username", "user1"]), []),
[#{<<"username">> := <<"user1">>, <<"rules">> := Rules1}] = jsx:decode(Request1),
#{<<"username">> := <<"user1">>, <<"rules">> := Rules1} = jsx:decode(Request2),
?assertEqual(3, length(Rules1)),
{ok, 204, _} = request(put, uri(["authorization", "sources", "built-in-database", "username", "user1"]), ?EXAMPLE_USERNAME#{rules => []}),
{ok, 200, Request3} = request(get, uri(["authorization", "sources", "built-in-database", "username", "user1"]), []),
#{<<"username">> := <<"user1">>, <<"rules">> := Rules2} = jsx:decode(Request3),
?assertEqual(0, length(Rules2)),
{ok, 204, _} = request(delete, uri(["authorization", "sources", "built-in-database", "username", "user1"]), []),
{ok, 404, _} = request(get, uri(["authorization", "sources", "built-in-database", "username", "user1"]), []),
{ok, 204, _} = request(post, uri(["authorization", "sources", "built-in-database", "clientid"]), ?EXAMPLE_CLIENTID),
{ok, 200, Request4} = request(get, uri(["authorization", "sources", "built-in-database", "clientid"]), []),
{ok, 200, Request5} = request(get, uri(["authorization", "sources", "built-in-database", "clientid", "client1"]), []),
[#{<<"clientid">> := <<"client1">>, <<"rules">> := Rules3}] = jsx:decode(Request4),
#{<<"clientid">> := <<"client1">>, <<"rules">> := Rules3} = jsx:decode(Request5),
?assertEqual(3, length(Rules3)),
{ok, 204, _} = request(put, uri(["authorization", "sources", "built-in-database", "clientid", "client1"]), ?EXAMPLE_CLIENTID#{rules => []}),
{ok, 200, Request6} = request(get, uri(["authorization", "sources", "built-in-database", "clientid", "client1"]), []),
#{<<"clientid">> := <<"client1">>, <<"rules">> := Rules4} = jsx:decode(Request6),
?assertEqual(0, length(Rules4)),
{ok, 204, _} = request(delete, uri(["authorization", "sources", "built-in-database", "clientid", "client1"]), []),
{ok, 404, _} = request(get, uri(["authorization", "sources", "built-in-database", "clientid", "client1"]), []),
{ok, 204, _} = request(put, uri(["authorization", "sources", "built-in-database", "all"]), ?EXAMPLE_ALL),
{ok, 200, Request7} = request(get, uri(["authorization", "sources", "built-in-database", "all"]), []),
[#{<<"rules">> := Rules5}] = jsx:decode(Request7),
?assertEqual(3, length(Rules5)),
{ok, 204, _} = request(put, uri(["authorization", "sources", "built-in-database", "all"]), ?EXAMPLE_ALL#{rules => []}),
{ok, 200, Request8} = request(get, uri(["authorization", "sources", "built-in-database", "all"]), []),
[#{<<"rules">> := Rules6}] = jsx:decode(Request8),
?assertEqual(0, length(Rules6)),
{ok, 204, _} = request(delete, uri(["authorization", "sources", "built-in-database", "purge-all"]), []),
ok.
%%--------------------------------------------------------------------
%% HTTP Request
%%--------------------------------------------------------------------
request(Method, Url, Body) ->
Request = case Body of
[] -> {Url, [auth_header_()]};
_ -> {Url, [auth_header_()], "application/json", jsx:encode(Body)}
end,
ct:pal("Method: ~p, Request: ~p", [Method, Request]),
case httpc:request(Method, Request, [], [{body_format, binary}]) of
{error, socket_closed_remotely} ->
{error, socket_closed_remotely};
{ok, {{"HTTP/1.1", Code, _}, _Headers, Return} } ->
{ok, Code, Return};
{ok, {Reason, _, _}} ->
{error, Reason}
end.
uri() -> uri([]).
uri(Parts) when is_list(Parts) ->
NParts = [E || E <- Parts],
?HOST ++ filename:join([?BASE_PATH, ?API_VERSION | NParts]).
get_sources(Result) -> jsx:decode(Result).
auth_header_() ->
Username = <<"admin">>,
Password = <<"public">>,
{ok, Token} = emqx_dashboard_admin:sign_token(Username, Password),
{"Authorization", "Bearer " ++ binary_to_list(Token)}.