Merge pull request #5773 from tigercl/feat/psk-file

feat(psk): support psk
This commit is contained in:
tigercl 2021-09-29 13:37:49 +08:00 committed by GitHub
commit 55ec5bab9e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 544 additions and 12 deletions

View File

@ -1000,7 +1000,7 @@ In case PSK cipher suites are intended, make sure to configured
, {"ciphers", ciphers_schema(D("ciphers"))}
, {user_lookup_fun,
sc(typerefl:alias("string", any()),
#{ default => "emqx_psk:lookup"
#{ default => "emqx_tls_psk:lookup"
, converter => fun ?MODULE:parse_user_lookup_fun/1
})
}
@ -1287,7 +1287,7 @@ parse_user_lookup_fun(StrConf) ->
[ModStr, FunStr] = string:tokens(StrConf, ":"),
Mod = list_to_atom(ModStr),
Fun = list_to_atom(FunStr),
{fun Mod:Fun/3, <<>>}.
{fun Mod:Fun/3, undefined}.
validate_ciphers(Ciphers) ->
All = case is_tlsv13_available() of

View File

@ -14,11 +14,10 @@
%% limitations under the License.
%%--------------------------------------------------------------------
-module(emqx_psk).
-module(emqx_tls_psk).
-include("logger.hrl").
%% SSL PSK Callbacks
-export([lookup/3]).
@ -26,14 +25,24 @@
-type psk_user_state() :: term().
-spec lookup(psk, psk_identity(), psk_user_state()) -> {ok, SharedSecret :: binary()} | error.
lookup(psk, ClientPSKID, _UserState) ->
try emqx_hooks:run_fold('tls_handshake.psk_lookup', [ClientPSKID], not_found) of
SharedSecret when is_binary(SharedSecret) -> {ok, SharedSecret};
Error ->
?LOG(error, "Look PSK for PSKID ~p error: ~p", [ClientPSKID, Error]),
lookup(psk, PSKIdentity, _UserState) ->
try emqx_hooks:run_fold('tls_handshake.psk_lookup', [PSKIdentity], normal) of
{ok, SharedSecret} when is_binary(SharedSecret) ->
{ok, SharedSecret};
normal ->
?SLOG(info, #{msg => "psk_identity_not_found",
psk_identity => PSKIdentity}),
error;
{error, Reason} ->
?SLOG(warning, #{msg => "psk_identity_not_found",
psk_identity => PSKIdentity,
reason => Reason}),
error
catch
Except:Error:Stacktrace ->
?LOG(error, "Lookup PSK failed, ~0p: ~0p", [{Except,Error}, Stacktrace]),
Class:Reason:Stacktrace ->
?SLOG(error, #{msg => "lookup_psk_failed",
class => Class,
reason => Reason,
stacktrace => Stacktrace}),
error
end.

View File

@ -33,7 +33,7 @@ prop_lookup() ->
?ALL({ClientPSKID, UserState},
{client_pskid(), user_state()},
begin
case emqx_psk:lookup(psk, ClientPSKID, UserState) of
case emqx_tls_psk:lookup(psk, ClientPSKID, UserState) of
{ok, _Result} -> true;
error -> true;
_Other -> false

View File

@ -152,6 +152,7 @@ reboot_apps() ->
, emqx_exhook
, emqx_authn
, emqx_authz
, emqx_psk
].
sorted_reboot_apps() ->

View File

@ -53,6 +53,7 @@
, emqx_prometheus_schema
, emqx_rule_engine_schema
, emqx_exhook_schema
, emqx_psk_schema
]).
namespace() -> undefined.

View File

@ -0,0 +1,2 @@
myclient1:8c701116e9127c57a99d5563709af3deaca75563e2c4dd0865701ae839fb6d79
myclient2:d1e617d3b963757bfc21dad3fea169716c3a2f053f23decaea5cdfaabd04bfc4

View File

@ -0,0 +1,22 @@
##--------------------------------------------------------------------
## EMQ X PSK
##--------------------------------------------------------------------
psk {
## Whether to enable the PSK feature.
enable = false
## If init file is specified, emqx will import PSKs from the file
## into the built-in database at startup for use by the runtime.
##
## The file has to be structured line-by-line, each line must be in
## the format: <PSKIdentity>:<SharedSecret>
## init_file = "{{ platform_data_dir }}/init.psk"
## Specifies the separator for PSKIdentity and SharedSecret in the init file.
## The default is colon (:)
## separator = ":"
## The size of each chunk used to import to the built-in database from psk file
## chunk_size = 50
}

View File

@ -0,0 +1,18 @@
{deps, []}.
{edoc_opts, [{preprocess, true}]}.
{erl_opts, [warn_unused_vars,
warn_shadow_vars,
warnings_as_errors,
warn_unused_import,
warn_obsolete_guard,
debug_info,
{parse_transform}]}.
{xref_checks, [undefined_function_calls, undefined_functions,
locals_not_used, deprecated_function_calls,
warnings_as_errors, deprecated_functions]}.
{cover_enabled, true}.
{cover_opts, [verbose]}.
{cover_export_enabled, true}.

View File

@ -0,0 +1,15 @@
%% -*- mode: erlang -*-
{application, emqx_psk,
[{description, "EMQ X PSK"},
{vsn, "5.0.0"}, % strict semver, bump manually!
{modules, []},
{registered, [emqx_psk_sup]},
{applications, [kernel,stdlib]},
{mod, {emqx_psk_app,[]}},
{env, []},
{licenses, ["Apache-2.0"]},
{maintainers, ["EMQ X Team <contact@emqx.io>"]},
{links, [{"Homepage", "https://emqx.io/"},
{"Github", "https://github.com/emqx/emqx"}
]}
]}.

View File

@ -0,0 +1,251 @@
%%--------------------------------------------------------------------
%% 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_psk).
-behaviour(gen_server).
-include_lib("emqx/include/logger.hrl").
-export([ load/0
, unload/0
, on_psk_lookup/2
, import/1
]).
-export([ start_link/0
, stop/0
]).
%% gen_server callbacks
-export([ init/1
, handle_call/3
, handle_cast/2
, handle_info/2
, terminate/2
, code_change/3
]).
-record(psk_entry, {psk_id :: binary(),
shared_secret :: binary(),
extra :: term()
}).
-export([mnesia/1]).
-boot_mnesia({mnesia, [boot]}).
-copy_mnesia({mnesia, [copy]}).
-define(TAB, ?MODULE).
-define(PSK_SHARD, emqx_psk_shard).
-define(DEFAULT_DELIMITER, <<":">>).
-define(CR, 13).
-define(LF, 10).
%%------------------------------------------------------------------------------
%% Mnesia bootstrap
%%------------------------------------------------------------------------------
%% @doc Create or replicate tables.
-spec(mnesia(boot | copy) -> ok).
mnesia(boot) ->
ok = ekka_mnesia:create_table(?TAB, [
{rlog_shard, ?PSK_SHARD},
{type, ordered_set},
{disc_copies, [node()]},
{record_name, psk_entry},
{attributes, record_info(fields, psk_entry)},
{storage_properties, [{ets, [{read_concurrency, true}]}]}]);
mnesia(copy) ->
ok = ekka_mnesia:copy_table(?TAB, disc_copies).
%%------------------------------------------------------------------------------
%% APIs
%%------------------------------------------------------------------------------
load() ->
emqx:hook('tls_handshake.psk_lookup', {?MODULE, on_psk_lookup, []}).
unload() ->
emqx:unhook('tls_handshake.psk_lookup', {?MODULE, on_psk_lookup, []}).
on_psk_lookup(PSKIdentity, _UserState) ->
case mnesia:dirty_read(?TAB, PSKIdentity) of
[#psk_entry{shared_secret = SharedSecret}] ->
{stop, {ok, SharedSecret}};
_ ->
ignore
end.
import(SrcFile) ->
call({import, SrcFile}).
-spec start_link() -> {ok, pid()} | ignore | {error, term()}.
start_link() ->
gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
-spec stop() -> ok.
stop() ->
gen_server:stop(?MODULE).
%%--------------------------------------------------------------------
%% gen_server callbacks
%%--------------------------------------------------------------------
init(_Opts) ->
_ = case get_config(enable) of
true -> load();
false -> ?SLOG(info, #{msg => "emqx_psk_disabled"})
end,
_ = case get_config(init_file) of
undefined -> ok;
InitFile -> import_psks(InitFile)
end,
{ok, #{}}.
handle_call({import, SrcFile}, _From, State) ->
{reply, import_psks(SrcFile), State};
handle_call(Req, _From, State) ->
?SLOG(info, #{msg => "unexpected_call_discarded", req => Req}),
{reply, {error, unexecpted}, State}.
handle_cast(Req, State) ->
?SLOG(info, #{msg => "unexpected_cast_discarded", req => Req}),
{noreply, State}.
handle_info(Info, State) ->
?SLOG(info, #{msg => "unexpected_info_discarded", info => Info}),
{noreply, State}.
terminate(_Reason, _State) ->
unload(),
ok.
code_change(_OldVsn, State, _Extra) ->
{ok, State}.
%%------------------------------------------------------------------------------
%% Internal functions
%%------------------------------------------------------------------------------
get_config(enable) ->
emqx_config:get([psk, enable]);
get_config(init_file) ->
emqx_config:get([psk, init_file], undefined);
get_config(separator) ->
emqx_config:get([psk, separator], ?DEFAULT_DELIMITER);
get_config(chunk_size) ->
emqx_config:get([psk, chunk_size]).
import_psks(SrcFile) ->
case file:open(SrcFile, [read, raw, binary, read_ahead]) of
{error, Reason} ->
?SLOG(error, #{msg => "failed_to_open_psk_file",
file => SrcFile,
reason => Reason}),
{error, Reason};
{ok, Io} ->
try import_psks(Io, get_config(separator), get_config(chunk_size), 0) of
ok -> ok;
{error, Reason} ->
?SLOG(error, #{msg => "failed_to_import_psk_file",
file => SrcFile,
reason => Reason}),
{error, Reason}
catch
Exception:Reason:Stacktrace ->
?SLOG(error, #{msg => "failed_to_import_psk_file",
file => SrcFile,
exception => Exception,
reason => Reason,
stacktrace => Stacktrace}),
{error, Reason}
after
_ = file:close(Io)
end
end.
import_psks(Io, Delimiter, ChunkSize, NChunk) ->
case get_psks(Io, Delimiter, ChunkSize) of
{ok, Entries} ->
_ = trans(fun insert_psks/1, [Entries]),
import_psks(Io, Delimiter, ChunkSize, NChunk + 1);
{eof, Entries} ->
_ = trans(fun insert_psks/1, [Entries]),
ok;
{error, {bad_format, {line, N}}} ->
{error, {bad_format, {line, NChunk * ChunkSize + N}}};
{error, Reaosn} ->
{error, Reaosn}
end.
get_psks(Io, Delimiter, Max) ->
get_psks(Io, Delimiter, Max, []).
get_psks(_Io, _Delimiter, 0, Acc) ->
{ok, Acc};
get_psks(Io, Delimiter, Remaining, Acc) ->
case file:read_line(Io) of
{ok, Line} ->
case binary:split(Line, Delimiter) of
[PSKIdentity, SharedSecret] ->
NSharedSecret = trim_crlf(SharedSecret),
get_psks(Io, Delimiter, Remaining - 1, [{PSKIdentity, NSharedSecret} | Acc]);
_ ->
{error, {bad_format, {line, length(Acc) + 1}}}
end;
eof ->
{eof, Acc};
{error, Reason} ->
{error, Reason}
end.
insert_psks(Entries) ->
lists:foreach(fun(Entry) ->
insert_psk(Entry)
end, Entries).
insert_psk({PSKIdentity, SharedSecret}) ->
mnesia:write(?TAB, #psk_entry{psk_id = PSKIdentity, shared_secret = SharedSecret}, write).
trim_crlf(Bin) ->
Size = byte_size(Bin),
case binary:at(Bin, Size - 1) of
?LF ->
case binary:at(Bin, Size - 2) of
?CR -> binary:part(Bin, 0, Size - 2);
_ -> binary:part(Bin, 0, Size - 1)
end;
_ -> Bin
end.
trans(Fun, Args) ->
case ekka_mnesia:transaction(?PSK_SHARD, Fun, Args) of
{atomic, Res} -> Res;
{aborted, Reason} -> {error, Reason}
end.
call(Request) ->
try
gen_server:call(?MODULE, Request, 10000)
catch
exit:{timeout, _Details} ->
{error, timeout}
end.

View File

@ -0,0 +1,30 @@
%%--------------------------------------------------------------------
%% 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_psk_app).
-behaviour(application).
-export([ start/2
, stop/1
]).
start(_Type, _Args) ->
{ok, Sup} = emqx_psk_sup:start_link(),
{ok, Sup}.
stop(_State) ->
ok.

View File

@ -0,0 +1,60 @@
%%--------------------------------------------------------------------
%% 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_psk_schema).
-behaviour(hocon_schema).
-include_lib("typerefl/include/types.hrl").
-export([ roots/0
, fields/1
]).
roots() -> ["psk"].
fields("psk") ->
[ {enable, fun enable/1}
, {init_file, fun init_file/1}
, {separator, fun separator/1}
, {chunk_size, fun chunk_size/1}
].
enable(type) -> boolean();
enable(desc) -> <<"Whether to enable tls psk support">>;
enable(default) -> false;
enable(_) -> undefined.
init_file(type) -> binary();
init_file(desc) ->
<<"If init_file is specified, emqx will import PSKs from the file ",
"into the built-in database at startup for use by the runtime. ",
"The file has to be structured line-by-line, each line must be in ",
"the format: <PSKIdentity>:<SharedSecret>">>;
init_file(nullable) -> true;
init_file(_) -> undefined.
separator(type) -> binary();
separator(desc) ->
<<"The separator between PSKIdentity and SharedSecret in the psk file">>;
separator(default) -> <<":">>;
separator(_) -> undefined.
chunk_size(type) -> integer();
chunk_size(desc) ->
<<"The size of each chunk used to import to the built-in database from psk file">>;
chunk_size(default) -> 50;
chunk_size(_) -> undefined.

View File

@ -0,0 +1,35 @@
%%--------------------------------------------------------------------
%% 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_psk_sup).
-behaviour(supervisor).
-export([start_link/0]).
-export([init/1]).
start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
{ok, {{one_for_one, 10, 3600},
[#{id => emqx_psk,
start => {emqx_psk, start_link, []},
restart => permanent,
shutdown => 5000,
type => worker,
modules => [emqx_psk]}]}}.

View File

@ -0,0 +1,2 @@
myclient1:8c701116e9127c57a99d5563709af3deaca75563e2c4dd0865701ae839fb6d79
myclient2:d1e617d3b963757bfc21dad3fea169716c3a2f053f23decaea5cdfaabd04bfc4

View File

@ -0,0 +1,85 @@
%%--------------------------------------------------------------------
%% 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_psk_SUITE).
-compile(nowarn_export_all).
-compile(export_all).
-include_lib("common_test/include/ct.hrl").
-include_lib("eunit/include/eunit.hrl").
all() ->
emqx_ct:all(?MODULE).
init_per_suite(Config) ->
meck:new(emqx_config, [non_strict, passthrough, no_history, no_link]),
meck:expect(emqx_config, get, fun([psk, enable]) -> true;
([psk, chunk_size]) -> 50;
(KeyPath) -> meck:passthrough([KeyPath])
end),
meck:expect(emqx_config, get, fun([psk, init_file], _) ->
filename:join([code:lib_dir(emqx_psk, test), "data/init.psk"]);
([psk, separator], _) -> <<":">>;
(KeyPath, Default) -> meck:passthrough([KeyPath, Default])
end),
emqx_ct_helpers:start_apps([emqx_psk]),
Config.
end_per_suite(_) ->
meck:unload(emqx_config),
emqx_ct_helpers:stop_apps([emqx_psk]),
ok.
t_psk_lookup(_) ->
PSKIdentity1 = <<"myclient1">>,
SharedSecret1 = <<"8c701116e9127c57a99d5563709af3deaca75563e2c4dd0865701ae839fb6d79">>,
?assertEqual({stop, {ok, SharedSecret1}}, emqx_psk:on_psk_lookup(PSKIdentity1, any)),
PSKIdentity2 = <<"myclient2">>,
SharedSecret2 = <<"d1e617d3b963757bfc21dad3fea169716c3a2f053f23decaea5cdfaabd04bfc4">>,
?assertEqual({stop, {ok, SharedSecret2}}, emqx_psk:on_psk_lookup(PSKIdentity2, any)),
?assertEqual(ignore, emqx_psk:on_psk_lookup(<<"myclient3">>, any)),
ClientLookup = fun(psk, undefined, _) -> {ok, SharedSecret1};
(psk, _, _) -> error
end,
ClientTLSOpts = #{ versions => ['tlsv1.2']
, ciphers => ["PSK-AES256-CBC-SHA"]
, psk_identity => "myclient1"
, verify => verify_none
, user_lookup_fun => {ClientLookup, undefined}
},
ServerTLSOpts = #{ versions => ['tlsv1.2']
, ciphers => ["PSK-AES256-CBC-SHA"]
, verify => verify_none
, reuseaddr => true
, user_lookup_fun => {fun emqx_tls_psk:lookup/3, undefined}
},
emqx_config:put([listeners, ssl ,default, ssl], ServerTLSOpts),
emqx_listeners:restart_listener('ssl:default'),
{ok, Socket} = ssl:connect("127.0.0.1", 8883, maps:to_list(ClientTLSOpts)),
ssl:close(Socket),
ClientTLSOpts1 = ClientTLSOpts#{psk_identity => "myclient2"},
?assertMatch({error, _}, ssl:connect("127.0.0.1", 8883, maps:to_list(ClientTLSOpts1))),
ok.

View File

@ -278,6 +278,7 @@ relx_apps(ReleaseType) ->
, emqx_retainer
, emqx_statsd
, emqx_prometheus
, emqx_psk
]
++ [quicer || is_quicer_supported()]
++ [emqx_license || is_enterprise()]