summaryrefslogtreecommitdiff
path: root/deps/rabbit_common/src
diff options
context:
space:
mode:
Diffstat (limited to 'deps/rabbit_common/src')
-rw-r--r--deps/rabbit_common/src/app_utils.erl167
-rw-r--r--deps/rabbit_common/src/code_version.erl348
-rw-r--r--deps/rabbit_common/src/credit_flow.erl210
-rw-r--r--deps/rabbit_common/src/delegate.erl277
-rw-r--r--deps/rabbit_common/src/delegate_sup.erl55
-rw-r--r--deps/rabbit_common/src/file_handle_cache.erl1564
-rw-r--r--deps/rabbit_common/src/file_handle_cache_stats.erl57
-rw-r--r--deps/rabbit_common/src/gen_server2.erl1419
-rw-r--r--deps/rabbit_common/src/lager_forwarder_backend.erl120
-rw-r--r--deps/rabbit_common/src/mirrored_supervisor.erl513
-rw-r--r--deps/rabbit_common/src/mnesia_sync.erl64
-rw-r--r--deps/rabbit_common/src/pmon.erl96
-rw-r--r--deps/rabbit_common/src/priority_queue.erl234
-rw-r--r--deps/rabbit_common/src/rabbit_amqp_connection.erl34
-rw-r--r--deps/rabbit_common/src/rabbit_amqqueue_common.erl39
-rw-r--r--deps/rabbit_common/src/rabbit_auth_backend_dummy.erl39
-rw-r--r--deps/rabbit_common/src/rabbit_auth_mechanism.erl41
-rw-r--r--deps/rabbit_common/src/rabbit_authn_backend.erl27
-rw-r--r--deps/rabbit_common/src/rabbit_authz_backend.erl88
-rw-r--r--deps/rabbit_common/src/rabbit_basic_common.erl41
-rw-r--r--deps/rabbit_common/src/rabbit_binary_generator.erl235
-rw-r--r--deps/rabbit_common/src/rabbit_binary_parser.erl172
-rw-r--r--deps/rabbit_common/src/rabbit_cert_info.erl270
-rw-r--r--deps/rabbit_common/src/rabbit_channel_common.erl25
-rw-r--r--deps/rabbit_common/src/rabbit_command_assembler.erl124
-rw-r--r--deps/rabbit_common/src/rabbit_control_misc.erl179
-rw-r--r--deps/rabbit_common/src/rabbit_core_metrics.erl437
-rw-r--r--deps/rabbit_common/src/rabbit_data_coercion.erl47
-rw-r--r--deps/rabbit_common/src/rabbit_env.erl1850
-rw-r--r--deps/rabbit_common/src/rabbit_error_logger_handler.erl169
-rw-r--r--deps/rabbit_common/src/rabbit_event.erl164
-rw-r--r--deps/rabbit_common/src/rabbit_exchange_type.erl68
-rw-r--r--deps/rabbit_common/src/rabbit_heartbeat.erl184
-rw-r--r--deps/rabbit_common/src/rabbit_http_util.erl967
-rw-r--r--deps/rabbit_common/src/rabbit_json.erl63
-rw-r--r--deps/rabbit_common/src/rabbit_log.erl164
-rw-r--r--deps/rabbit_common/src/rabbit_log_osiris_shim.erl26
-rw-r--r--deps/rabbit_common/src/rabbit_log_ra_shim.erl16
-rw-r--r--deps/rabbit_common/src/rabbit_misc.erl1434
-rw-r--r--deps/rabbit_common/src/rabbit_msg_store_index.erl89
-rw-r--r--deps/rabbit_common/src/rabbit_net.erl321
-rw-r--r--deps/rabbit_common/src/rabbit_nodes_common.erl227
-rw-r--r--deps/rabbit_common/src/rabbit_numerical.erl358
-rw-r--r--deps/rabbit_common/src/rabbit_password_hashing.erl11
-rw-r--r--deps/rabbit_common/src/rabbit_pbe.erl54
-rw-r--r--deps/rabbit_common/src/rabbit_peer_discovery_backend.erl59
-rw-r--r--deps/rabbit_common/src/rabbit_policy_validator.erl22
-rw-r--r--deps/rabbit_common/src/rabbit_queue_collector.erl80
-rw-r--r--deps/rabbit_common/src/rabbit_registry.erl165
-rw-r--r--deps/rabbit_common/src/rabbit_registry_class.erl12
-rw-r--r--deps/rabbit_common/src/rabbit_resource_monitor_misc.erl39
-rw-r--r--deps/rabbit_common/src/rabbit_runtime.erl66
-rw-r--r--deps/rabbit_common/src/rabbit_runtime_parameter.erl25
-rw-r--r--deps/rabbit_common/src/rabbit_semver.erl730
-rw-r--r--deps/rabbit_common/src/rabbit_semver_parser.erl306
-rw-r--r--deps/rabbit_common/src/rabbit_ssl_options.erl86
-rw-r--r--deps/rabbit_common/src/rabbit_types.erl196
-rw-r--r--deps/rabbit_common/src/rabbit_writer.erl437
-rw-r--r--deps/rabbit_common/src/supervisor2.erl1651
-rw-r--r--deps/rabbit_common/src/vm_memory_monitor.erl576
-rw-r--r--deps/rabbit_common/src/worker_pool.erl172
-rw-r--r--deps/rabbit_common/src/worker_pool_sup.erl69
-rw-r--r--deps/rabbit_common/src/worker_pool_worker.erl192
63 files changed, 17970 insertions, 0 deletions
diff --git a/deps/rabbit_common/src/app_utils.erl b/deps/rabbit_common/src/app_utils.erl
new file mode 100644
index 0000000000..df965575be
--- /dev/null
+++ b/deps/rabbit_common/src/app_utils.erl
@@ -0,0 +1,167 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(app_utils).
+
+-export([load_applications/1,
+ start_applications/1, start_applications/2, start_applications/3,
+ stop_applications/1, stop_applications/2, app_dependency_order/2,
+ app_dependencies/1]).
+
+-type error_handler() :: fun((atom(), any()) -> 'ok' | no_return()).
+-type restart_type() :: 'permanent' | 'transient' | 'temporary'.
+
+-spec load_applications([atom()]) -> 'ok'.
+-spec start_applications([atom()]) -> 'ok'.
+-spec stop_applications([atom()]) -> 'ok'.
+-spec start_applications([atom()], error_handler()) -> 'ok'.
+-spec start_applications([atom()], error_handler(), #{atom() => restart_type()}) -> 'ok'.
+-spec stop_applications([atom()], error_handler()) -> 'ok'.
+-spec app_dependency_order([atom()], boolean()) -> [digraph:vertex()].
+-spec app_dependencies(atom()) -> [atom()].
+-spec failed_to_start_app(atom(), any()) -> no_return().
+-spec failed_to_stop_app(atom(), any()) -> no_return().
+
+%%---------------------------------------------------------------------------
+%% Public API
+
+load_applications(Apps) ->
+ load_applications(queue:from_list(Apps), sets:new()),
+ ok.
+
+start_applications(Apps) ->
+ start_applications(
+ Apps, fun failed_to_start_app/2).
+
+stop_applications(Apps) ->
+ stop_applications(
+ Apps, fun failed_to_stop_app/2).
+
+failed_to_start_app(App, Reason) ->
+ throw({error, {cannot_start_application, App, Reason}}).
+
+failed_to_stop_app(App, Reason) ->
+ throw({error, {cannot_stop_application, App, Reason}}).
+
+start_applications(Apps, ErrorHandler) ->
+ start_applications(Apps, ErrorHandler, #{}).
+
+start_applications(Apps, ErrorHandler, RestartTypes) ->
+ manage_applications(fun lists:foldl/3,
+ fun(App) -> ensure_all_started(App, RestartTypes) end,
+ fun application:stop/1,
+ already_started,
+ ErrorHandler,
+ Apps).
+
+stop_applications(Apps, ErrorHandler) ->
+ manage_applications(fun lists:foldr/3,
+ fun(App) ->
+ rabbit_log:info("Stopping application '~s'", [App]),
+ application:stop(App)
+ end,
+ fun(App) -> ensure_all_started(App, #{}) end,
+ not_started,
+ ErrorHandler,
+ Apps).
+
+app_dependency_order(RootApps, StripUnreachable) ->
+ {ok, G} = rabbit_misc:build_acyclic_graph(
+ fun ({App, _Deps}) -> [{App, App}] end,
+ fun ({App, Deps}) -> [{Dep, App} || Dep <- Deps] end,
+ [{App, app_dependencies(App)} ||
+ {App, _Desc, _Vsn} <- application:loaded_applications()]),
+ try
+ case StripUnreachable of
+ true -> digraph:del_vertices(G, digraph:vertices(G) --
+ digraph_utils:reachable(RootApps, G));
+ false -> ok
+ end,
+ digraph_utils:topsort(G)
+ after
+ true = digraph:delete(G)
+ end.
+
+%%---------------------------------------------------------------------------
+%% Private API
+
+load_applications(Worklist, Loaded) ->
+ case queue:out(Worklist) of
+ {empty, _WorkList} ->
+ ok;
+ {{value, App}, Worklist1} ->
+ case sets:is_element(App, Loaded) of
+ true -> load_applications(Worklist1, Loaded);
+ false -> case application:load(App) of
+ ok -> ok;
+ {error, {already_loaded, App}} -> ok;
+ Error -> throw(Error)
+ end,
+ load_applications(
+ queue:join(Worklist1,
+ queue:from_list(app_dependencies(App))),
+ sets:add_element(App, Loaded))
+ end
+ end.
+
+app_dependencies(App) ->
+ case application:get_key(App, applications) of
+ undefined -> [];
+ {ok, Lst} -> Lst
+ end.
+
+manage_applications(Iterate, Do, Undo, SkipError, ErrorHandler, Apps) ->
+ Iterate(fun (App, Acc) ->
+ case Do(App) of
+ ok -> [App | Acc];
+ {ok, []} -> Acc;
+ {ok, [App]} -> [App | Acc];
+ {ok, StartedApps} -> StartedApps ++ Acc;
+ {error, {SkipError, _}} -> Acc;
+ {error, Reason} ->
+ lists:foreach(Undo, Acc),
+ ErrorHandler(App, Reason)
+ end
+ end, [], Apps),
+ ok.
+
+%% Stops the Erlang VM when the rabbit application stops abnormally
+%% i.e. message store reaches its restart limit
+default_restart_type(rabbit) -> transient;
+default_restart_type(_) -> temporary.
+
+%% Copyright Ericsson AB 1996-2016. All Rights Reserved.
+%%
+%% Code originally from Erlang/OTP source lib/kernel/src/application.erl
+%% and modified to use RestartTypes map
+%%
+ensure_all_started(Application, RestartTypes) ->
+ case ensure_all_started(Application, RestartTypes, []) of
+ {ok, Started} ->
+ {ok, lists:reverse(Started)};
+ {error, Reason, Started} ->
+ _ = [application:stop(App) || App <- Started],
+ {error, Reason}
+ end.
+
+ensure_all_started(Application, RestartTypes, Started) ->
+ RestartType = maps:get(Application, RestartTypes, default_restart_type(Application)),
+ case application:start(Application, RestartType) of
+ ok ->
+ {ok, [Application | Started]};
+ {error, {already_started, Application}} ->
+ {ok, Started};
+ {error, {not_started, Dependency}} ->
+ case ensure_all_started(Dependency, RestartTypes, Started) of
+ {ok, NewStarted} ->
+ ensure_all_started(Application, RestartTypes, NewStarted);
+ Error ->
+ Error
+ end;
+ {error, Reason} ->
+ {error, {Application, Reason}, Started}
+ end.
diff --git a/deps/rabbit_common/src/code_version.erl b/deps/rabbit_common/src/code_version.erl
new file mode 100644
index 0000000000..76e9c75c7f
--- /dev/null
+++ b/deps/rabbit_common/src/code_version.erl
@@ -0,0 +1,348 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+-module(code_version).
+
+-export([update/1, get_otp_version/0]).
+
+%%----------------------------------------------------------------------------
+%% API
+%%----------------------------------------------------------------------------
+
+%%----------------------------------------------------------------------------
+%% @doc Reads the abstract code of the given `Module`, modifies it to adapt to
+%% the current Erlang version, compiles and loads the result.
+%% This function finds the current Erlang version and then selects the function
+%% call for that version, removing all other versions declared in the original
+%% beam file. `code_version:update/1` is triggered by the module itself the
+%% first time an affected function is called.
+%%
+%% The purpose of this functionality is to support the new time API introduced
+%% in ERTS 7.0, while providing compatibility with previous versions.
+%%
+%% `Module` must contain an attribute `erlang_version_support` containing a list of
+%% tuples:
+%%
+%% {ErlangVersion, [{OriginalFunction, Arity, PreErlangVersionFunction,
+%% PostErlangVersionFunction}]}
+%%
+%% All these new functions may be exported, and implemented as follows:
+%%
+%% OriginalFunction() ->
+%% code_version:update(?MODULE),
+%% ?MODULE:OriginalFunction().
+%%
+%% PostErlangVersionFunction() ->
+%% %% implementation using new time API
+%% ..
+%%
+%% PreErlangVersionFunction() ->
+%% %% implementation using fallback solution
+%% ..
+%%
+%% CAUTION: Make sure that all functions in the module are patched this
+%% way! If you have "regular" functions, you might hit a race condition
+%% between the unload of the old module and the load of the patched
+%% module. If all functions are patched, loading will be serialized,
+%% thanks to a lock acquired by `code_version`. However, if you have
+%% regular functions, any call to them will bypass that lock and the old
+%% code will be reloaded from disk. This will kill the process trying to
+%% patch the module.
+%%
+%% end
+%%----------------------------------------------------------------------------
+-spec update(atom()) -> ok | no_return().
+update(Module) ->
+ AbsCode = get_abs_code(Module),
+ Forms = replace_forms(Module, get_otp_version(), AbsCode),
+ Code = compile_forms(Forms),
+ load_code(Module, Code).
+
+%%----------------------------------------------------------------------------
+%% Internal functions
+%%----------------------------------------------------------------------------
+load_code(Module, Code) ->
+ LockId = {{?MODULE, Module}, self()},
+ FakeFilename = "Loaded by rabbit_common",
+ global:set_lock(LockId, [node()]),
+ case code:which(Module) of
+ FakeFilename ->
+ ok;
+ _ ->
+ unload(Module),
+ case code:load_binary(Module, FakeFilename, Code) of
+ {module, _} -> ok;
+ {error, Reason} -> throw({cannot_load, Module, Reason})
+ end
+ end,
+ global:del_lock(LockId, [node()]),
+ ok.
+
+unload(Module) ->
+ code:soft_purge(Module),
+ code:delete(Module).
+
+compile_forms(Forms) ->
+ case compile:forms(Forms, [debug_info, return_errors]) of
+ {ok, _ModName, Code} ->
+ Code;
+ {ok, _ModName, Code, _Warnings} ->
+ Code;
+ Error ->
+ throw({cannot_compile_forms, Error})
+ end.
+
+get_abs_code(Module) ->
+ get_forms(get_object_code(Module)).
+
+get_object_code(Module) ->
+ case code:get_object_code(Module) of
+ {_Mod, Code, _File} ->
+ Code;
+ error ->
+ throw({not_found, Module})
+ end.
+
+get_forms(Code) ->
+ case beam_lib:chunks(Code, [abstract_code]) of
+ {ok, {_, [{abstract_code, {raw_abstract_v1, Forms}}]}} ->
+ Forms;
+ {ok, {Module, [{abstract_code, no_abstract_code}]}} ->
+ throw({no_abstract_code, Module});
+ {error, beam_lib, Reason} ->
+ throw({no_abstract_code, Reason})
+ end.
+
+get_otp_version() ->
+ Version = erlang:system_info(otp_release),
+ case re:run(Version, "^[0-9][0-9]", [{capture, first, list}]) of
+ {match, [V]} ->
+ list_to_integer(V);
+ _ ->
+ %% Could be anything below R17, we are not interested
+ 0
+ end.
+
+get_original_pairs(VersionSupport) ->
+ [{Orig, Arity} || {Orig, Arity, _Pre, _Post} <- VersionSupport].
+
+get_delete_pairs(true, VersionSupport) ->
+ [{Pre, Arity} || {_Orig, Arity, Pre, _Post} <- VersionSupport];
+get_delete_pairs(false, VersionSupport) ->
+ [{Post, Arity} || {_Orig, Arity, _Pre, Post} <- VersionSupport].
+
+get_rename_pairs(true, VersionSupport) ->
+ [{Post, Arity} || {_Orig, Arity, _Pre, Post} <- VersionSupport];
+get_rename_pairs(false, VersionSupport) ->
+ [{Pre, Arity} || {_Orig, Arity, Pre, _Post} <- VersionSupport].
+
+%% Pairs of {Renamed, OriginalName} functions
+get_name_pairs(true, VersionSupport) ->
+ [{{Post, Arity}, Orig} || {Orig, Arity, _Pre, Post} <- VersionSupport];
+get_name_pairs(false, VersionSupport) ->
+ [{{Pre, Arity}, Orig} || {Orig, Arity, Pre, _Post} <- VersionSupport].
+
+delete_abstract_functions(ToDelete) ->
+ fun(Tree, Function) ->
+ case lists:member(Function, ToDelete) of
+ true ->
+ erl_syntax:comment(["Deleted unused function"]);
+ false ->
+ Tree
+ end
+ end.
+
+rename_abstract_functions(ToRename, ToName) ->
+ fun(Tree, Function) ->
+ case lists:member(Function, ToRename) of
+ true ->
+ FunctionName = proplists:get_value(Function, ToName),
+ erl_syntax:function(
+ erl_syntax:atom(FunctionName),
+ erl_syntax:function_clauses(Tree));
+ false ->
+ Tree
+ end
+ end.
+
+replace_forms(Module, ErlangVersion, AbsCode) ->
+ %% Obtain attribute containing the list of functions that must be updated
+ Attr = Module:module_info(attributes),
+ VersionSupport = proplists:get_value(erlang_version_support, Attr),
+ {Pre, Post} = lists:splitwith(fun({Version, _Pairs}) ->
+ Version > ErlangVersion
+ end, VersionSupport),
+ %% Replace functions in two passes: replace for Erlang versions > current
+ %% first, Erlang versions =< current afterwards.
+ replace_version_forms(
+ true, replace_version_forms(false, AbsCode, get_version_functions(Pre)),
+ get_version_functions(Post)).
+
+get_version_functions(List) ->
+ lists:append([Pairs || {_Version, Pairs} <- List]).
+
+replace_version_forms(IsPost, AbsCode, VersionSupport) ->
+ %% Get pairs of {Function, Arity} for the triggering functions, which
+ %% are also the final function names.
+ Original = get_original_pairs(VersionSupport),
+ %% Get pairs of {Function, Arity} for the unused version
+ ToDelete = get_delete_pairs(IsPost, VersionSupport),
+ %% Delete original functions (those that trigger the code update) and
+ %% the unused version ones
+ DeleteFun = delete_abstract_functions(ToDelete ++ Original),
+ AbsCode0 = replace_function_forms(AbsCode, DeleteFun),
+ %% Get pairs of {Function, Arity} for the current version which must be
+ %% renamed
+ ToRename = get_rename_pairs(IsPost, VersionSupport),
+ %% Get paris of {Renamed, OriginalName} functions
+ ToName = get_name_pairs(IsPost, VersionSupport),
+ %% Rename versioned functions with their final name
+ RenameFun = rename_abstract_functions(ToRename, ToName),
+ AbsCode1 = replace_function_forms(AbsCode0, RenameFun),
+ %% Adjust `-dialyzer` attribute.
+ AbsCode2 = fix_dialyzer_attribute(AbsCode1, ToDelete, ToName),
+ %% Remove exports of all versioned functions
+ remove_exports(AbsCode2, ToDelete ++ ToRename).
+
+replace_function_forms(AbsCode, Fun) ->
+ ReplaceFunction =
+ fun(Tree) ->
+ Function = erl_syntax_lib:analyze_function(Tree),
+ Fun(Tree, Function)
+ end,
+ Filter = fun(Tree) ->
+ case erl_syntax:type(Tree) of
+ function -> ReplaceFunction(Tree);
+ _Other -> Tree
+ end
+ end,
+ fold_syntax_tree(Filter, AbsCode).
+
+fix_dialyzer_attribute(AbsCode, ToDelete, ToName) ->
+ FixDialyzer =
+ fun(Tree) ->
+ case erl_syntax_lib:analyze_attribute(Tree) of
+ {dialyzer, {_, Value}} ->
+ FixedValue = fix_dialyzer_attribute_value(Value,
+ ToDelete,
+ ToName),
+ rebuild_dialyzer({dialyzer, FixedValue});
+ _ ->
+ Tree
+ end
+ end,
+ Filter = fun(Tree) ->
+ case erl_syntax:type(Tree) of
+ attribute -> FixDialyzer(Tree);
+ _ -> Tree
+ end
+ end,
+ fold_syntax_tree(Filter, AbsCode).
+
+fix_dialyzer_attribute_value(Info, ToDelete, ToName)
+ when is_list(Info) ->
+ lists:map(
+ fun(I) ->
+ fix_dialyzer_attribute_value(I, ToDelete, ToName)
+ end,
+ Info);
+fix_dialyzer_attribute_value({Warn, FunList}, ToDelete, ToName) ->
+ FixedFunList = fix_dialyzer_attribute_funlist(FunList, ToDelete, ToName),
+ {Warn, FixedFunList};
+fix_dialyzer_attribute_value(Info, _, _)
+ when is_atom(Info) ->
+ Info.
+
+fix_dialyzer_attribute_funlist(FunList, ToDelete, ToName)
+ when is_list(FunList) ->
+ lists:filtermap(
+ fun(I) ->
+ case fix_dialyzer_attribute_funlist(I, ToDelete, ToName) of
+ [] -> false;
+ R -> {true, R}
+ end
+ end,
+ FunList);
+fix_dialyzer_attribute_funlist({FunName, Arity} = Fun,
+ ToDelete, ToName)
+ when is_atom(FunName) andalso is_integer(Arity) andalso Arity >= 0 ->
+ remove_or_rename(Fun, ToDelete, ToName);
+fix_dialyzer_attribute_funlist(FunList, _, _) ->
+ FunList.
+
+remove_or_rename(Fun, ToDelete, ToName) ->
+ case lists:member(Fun, ToDelete) of
+ true ->
+ [];
+ false ->
+ case proplists:get_value(Fun, ToName) of
+ undefined -> Fun;
+ NewName -> setelement(1, Fun, NewName)
+ end
+ end.
+
+rebuild_dialyzer({dialyzer, Value}) ->
+ erl_syntax:attribute(
+ erl_syntax:atom(dialyzer),
+ [rebuild_dialyzer_value(Value)]).
+
+rebuild_dialyzer_value(Value) when is_list(Value) ->
+ erl_syntax:list(
+ [rebuild_dialyzer_value(V) || V <- Value]);
+rebuild_dialyzer_value({Warn, FunList}) ->
+ erl_syntax:tuple(
+ [rebuild_dialyzer_warn(Warn),
+ rebuild_dialyzer_funlist(FunList)]);
+rebuild_dialyzer_value(Warn) when is_atom(Warn) ->
+ rebuild_dialyzer_warn(Warn).
+
+rebuild_dialyzer_warn(Warn) when is_list(Warn) ->
+ erl_syntax:list(
+ [rebuild_dialyzer_warn(W) || W <- Warn]);
+rebuild_dialyzer_warn(Warn) when is_atom(Warn) ->
+ erl_syntax:atom(Warn).
+
+rebuild_dialyzer_funlist(FunList) when is_list(FunList) ->
+ erl_syntax:list(
+ [rebuild_dialyzer_funlist({N, A}) || {N, A} <- FunList]);
+rebuild_dialyzer_funlist({FunName, Arity}) ->
+ erl_syntax:tuple([erl_syntax:atom(FunName), erl_syntax:integer(Arity)]).
+
+filter_export_pairs(Info, ToDelete) ->
+ lists:filter(fun(Pair) ->
+ not lists:member(Pair, ToDelete)
+ end, Info).
+
+remove_exports(AbsCode, ToDelete) ->
+ RemoveExports =
+ fun(Tree) ->
+ case erl_syntax_lib:analyze_attribute(Tree) of
+ {export, Info} ->
+ Remaining = filter_export_pairs(Info, ToDelete),
+ rebuild_export(Remaining);
+ _Other -> Tree
+ end
+ end,
+ Filter = fun(Tree) ->
+ case erl_syntax:type(Tree) of
+ attribute -> RemoveExports(Tree);
+ _Other -> Tree
+ end
+ end,
+ fold_syntax_tree(Filter, AbsCode).
+
+rebuild_export(Args) ->
+ erl_syntax:attribute(
+ erl_syntax:atom(export),
+ [erl_syntax:list(
+ [erl_syntax:arity_qualifier(erl_syntax:atom(N),
+ erl_syntax:integer(A))
+ || {N, A} <- Args])]).
+
+fold_syntax_tree(Filter, Forms) ->
+ Tree = erl_syntax:form_list(Forms),
+ NewTree = erl_syntax_lib:map(Filter, Tree),
+ erl_syntax:revert_forms(NewTree).
diff --git a/deps/rabbit_common/src/credit_flow.erl b/deps/rabbit_common/src/credit_flow.erl
new file mode 100644
index 0000000000..da1d9606c1
--- /dev/null
+++ b/deps/rabbit_common/src/credit_flow.erl
@@ -0,0 +1,210 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(credit_flow).
+
+%% Credit flow is controlled by a credit specification - a
+%% {InitialCredit, MoreCreditAfter} tuple. For the message sender,
+%% credit starts at InitialCredit and is decremented with every
+%% message sent. The message receiver grants more credit to the sender
+%% by sending it a {bump_credit, ...} control message after receiving
+%% MoreCreditAfter messages. The sender should pass this message in to
+%% handle_bump_msg/1. The sender should block when it goes below 0
+%% (check by invoking blocked/0). If a process is both a sender and a
+%% receiver it will not grant any more credit to its senders when it
+%% is itself blocked - thus the only processes that need to check
+%% blocked/0 are ones that read from network sockets.
+%%
+%% Credit flows left to right when process send messages down the
+%% chain, starting at the rabbit_reader, ending at the msg_store:
+%% reader -> channel -> queue_process -> msg_store.
+%%
+%% If the message store has a back log, then it will block the
+%% queue_process, which will block the channel, and finally the reader
+%% will be blocked, throttling down publishers.
+%%
+%% Once a process is unblocked, it will grant credits up the chain,
+%% possibly unblocking other processes:
+%% reader <--grant channel <--grant queue_process <--grant msg_store.
+%%
+%% Grepping the project files for `credit_flow` will reveal the places
+%% where this module is currently used, with extra comments on what's
+%% going on at each instance. Note that credit flow between mirrors
+%% synchronization has not been documented, since this doesn't affect
+%% client publishes.
+
+-define(DEFAULT_INITIAL_CREDIT, 200).
+-define(DEFAULT_MORE_CREDIT_AFTER, 100).
+
+-define(DEFAULT_CREDIT,
+ case get(credit_flow_default_credit) of
+ undefined ->
+ Val = rabbit_misc:get_env(rabbit, credit_flow_default_credit,
+ {?DEFAULT_INITIAL_CREDIT,
+ ?DEFAULT_MORE_CREDIT_AFTER}),
+ put(credit_flow_default_credit, Val),
+ Val;
+ Val -> Val
+ end).
+
+-export([send/1, send/2, ack/1, ack/2, handle_bump_msg/1, blocked/0, state/0]).
+-export([peer_down/1]).
+-export([block/1, unblock/1]).
+
+%%----------------------------------------------------------------------------
+
+-export_type([bump_msg/0]).
+
+-opaque(bump_msg() :: {pid(), non_neg_integer()}).
+-type(credit_spec() :: {non_neg_integer(), non_neg_integer()}).
+
+-spec send
+ (pid()) -> 'ok';
+ (credit_spec()) -> 'ok'.
+-spec ack(pid()) -> 'ok'.
+-spec ack(pid(), credit_spec()) -> 'ok'.
+-spec handle_bump_msg(bump_msg()) -> 'ok'.
+-spec blocked() -> boolean().
+-spec peer_down(pid()) -> 'ok'.
+
+%%----------------------------------------------------------------------------
+
+%% process dict update macro - eliminates the performance-hurting
+%% closure creation a HOF would introduce
+-define(UPDATE(Key, Default, Var, Expr),
+ begin
+ %% We deliberately allow Var to escape from the case here
+ %% to be used in Expr. Any temporary var we introduced
+ %% would also escape, and might conflict.
+ Var = case get(Key) of
+ undefined -> Default;
+ V -> V
+ end,
+ put(Key, Expr)
+ end).
+
+%% If current process was blocked by credit flow in the last
+%% STATE_CHANGE_INTERVAL milliseconds, state/0 will report it as "in
+%% flow".
+-define(STATE_CHANGE_INTERVAL, 1000000).
+
+-ifdef(CREDIT_FLOW_TRACING).
+-define(TRACE_BLOCKED(SELF, FROM), rabbit_event:notify(credit_flow_blocked,
+ [{process, SELF},
+ {process_info, erlang:process_info(SELF)},
+ {from, FROM},
+ {from_info, erlang:process_info(FROM)},
+ {timestamp,
+ os:system_time(
+ milliseconds)}])).
+-define(TRACE_UNBLOCKED(SELF, FROM), rabbit_event:notify(credit_flow_unblocked,
+ [{process, SELF},
+ {from, FROM},
+ {timestamp,
+ os:system_time(
+ milliseconds)}])).
+-else.
+-define(TRACE_BLOCKED(SELF, FROM), ok).
+-define(TRACE_UNBLOCKED(SELF, FROM), ok).
+-endif.
+
+%%----------------------------------------------------------------------------
+
+%% There are two "flows" here; of messages and of credit, going in
+%% opposite directions. The variable names "From" and "To" refer to
+%% the flow of credit, but the function names refer to the flow of
+%% messages. This is the clearest I can make it (since the function
+%% names form the API and want to make sense externally, while the
+%% variable names are used in credit bookkeeping and want to make
+%% sense internally).
+
+%% For any given pair of processes, ack/2 and send/2 must always be
+%% called with the same credit_spec().
+
+send(From) -> send(From, ?DEFAULT_CREDIT).
+
+send(From, {InitialCredit, _MoreCreditAfter}) ->
+ ?UPDATE({credit_from, From}, InitialCredit, C,
+ if C == 1 -> block(From),
+ 0;
+ true -> C - 1
+ end).
+
+ack(To) -> ack(To, ?DEFAULT_CREDIT).
+
+ack(To, {_InitialCredit, MoreCreditAfter}) ->
+ ?UPDATE({credit_to, To}, MoreCreditAfter, C,
+ if C == 1 -> grant(To, MoreCreditAfter),
+ MoreCreditAfter;
+ true -> C - 1
+ end).
+
+handle_bump_msg({From, MoreCredit}) ->
+ ?UPDATE({credit_from, From}, 0, C,
+ if C =< 0 andalso C + MoreCredit > 0 -> unblock(From),
+ C + MoreCredit;
+ true -> C + MoreCredit
+ end).
+
+blocked() -> case get(credit_blocked) of
+ undefined -> false;
+ [] -> false;
+ _ -> true
+ end.
+
+state() -> case blocked() of
+ true -> flow;
+ false -> case get(credit_blocked_at) of
+ undefined -> running;
+ B -> Now = erlang:monotonic_time(),
+ Diff = erlang:convert_time_unit(Now - B,
+ native,
+ micro_seconds),
+ case Diff < ?STATE_CHANGE_INTERVAL of
+ true -> flow;
+ false -> running
+ end
+ end
+ end.
+
+peer_down(Peer) ->
+ %% In theory we could also remove it from credit_deferred here, but it
+ %% doesn't really matter; at some point later we will drain
+ %% credit_deferred and thus send messages into the void...
+ unblock(Peer),
+ erase({credit_from, Peer}),
+ erase({credit_to, Peer}),
+ ok.
+
+%% --------------------------------------------------------------------------
+
+grant(To, Quantity) ->
+ Msg = {bump_credit, {self(), Quantity}},
+ case blocked() of
+ false -> To ! Msg;
+ true -> ?UPDATE(credit_deferred, [], Deferred, [{To, Msg} | Deferred])
+ end.
+
+block(From) ->
+ ?TRACE_BLOCKED(self(), From),
+ case blocked() of
+ false -> put(credit_blocked_at, erlang:monotonic_time());
+ true -> ok
+ end,
+ ?UPDATE(credit_blocked, [], Blocks, [From | Blocks]).
+
+unblock(From) ->
+ ?TRACE_UNBLOCKED(self(), From),
+ ?UPDATE(credit_blocked, [], Blocks, Blocks -- [From]),
+ case blocked() of
+ false -> case erase(credit_deferred) of
+ undefined -> ok;
+ Credits -> _ = [To ! Msg || {To, Msg} <- Credits],
+ ok
+ end;
+ true -> ok
+ end.
diff --git a/deps/rabbit_common/src/delegate.erl b/deps/rabbit_common/src/delegate.erl
new file mode 100644
index 0000000000..a73d5e64b1
--- /dev/null
+++ b/deps/rabbit_common/src/delegate.erl
@@ -0,0 +1,277 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(delegate).
+
+%% delegate is an alternative way of doing remote calls. Compared to
+%% the rpc module, it reduces inter-node communication. For example,
+%% if a message is routed to 1,000 queues on node A and needs to be
+%% propagated to nodes B and C, it would be nice to avoid doing 2,000
+%% remote casts to queue processes.
+%%
+%% An important issue here is preserving order - we need to make sure
+%% that messages from a certain channel to a certain queue take a
+%% consistent route, to prevent them being reordered. In fact all
+%% AMQP-ish things (such as queue declaration results and basic.get)
+%% must take the same route as well, to ensure that clients see causal
+%% ordering correctly. Therefore we have a rather generic mechanism
+%% here rather than just a message-reflector. That's also why we pick
+%% the delegate process to use based on a hash of the source pid.
+%%
+%% When a function is invoked using delegate:invoke/2,
+%% or delegate:invoke_no_result/2 on a group of pids, the pids are first split
+%% into local and remote ones. Remote processes are then grouped by
+%% node. The function is then invoked locally and on every node (using
+%% gen_server2:multi/4) as many times as there are processes on that
+%% node, sequentially.
+%%
+%% Errors returned when executing functions on remote nodes are re-raised
+%% in the caller.
+%%
+%% RabbitMQ starts a pool of delegate processes on boot. The size of
+%% the pool is configurable, the aim is to make sure we don't have too
+%% few delegates and thus limit performance on many-CPU machines.
+
+-behaviour(gen_server2).
+
+-export([start_link/1, start_link/2, invoke_no_result/2,
+ invoke/2, invoke/3, monitor/2, monitor/3, demonitor/1]).
+
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3]).
+
+-record(state, {node, monitors, name}).
+
+%%----------------------------------------------------------------------------
+
+-export_type([monitor_ref/0]).
+
+-type monitor_ref() :: reference() | {atom(), pid()}.
+-type fun_or_mfa(A) :: fun ((pid()) -> A) | {atom(), atom(), [any()]}.
+
+-spec start_link
+ (non_neg_integer()) -> {'ok', pid()} | ignore | {'error', any()}.
+-spec invoke
+ ( pid(), fun_or_mfa(A)) -> A;
+ ([pid()], fun_or_mfa(A)) -> {[{pid(), A}], [{pid(), term()}]}.
+-spec invoke_no_result(pid() | [pid()], fun_or_mfa(any())) -> 'ok'.
+-spec monitor('process', pid()) -> monitor_ref().
+-spec demonitor(monitor_ref()) -> 'true'.
+
+%%----------------------------------------------------------------------------
+
+-define(HIBERNATE_AFTER_MIN, 1000).
+-define(DESIRED_HIBERNATE, 10000).
+-define(DEFAULT_NAME, "delegate_").
+
+%%----------------------------------------------------------------------------
+
+start_link(Num) ->
+ start_link(?DEFAULT_NAME, Num).
+
+start_link(Name, Num) ->
+ Name1 = delegate_name(Name, Num),
+ gen_server2:start_link({local, Name1}, ?MODULE, [Name1], []).
+
+invoke(Pid, FunOrMFA) ->
+ invoke(Pid, ?DEFAULT_NAME, FunOrMFA).
+
+invoke(Pid, _Name, FunOrMFA) when is_pid(Pid) andalso node(Pid) =:= node() ->
+ apply1(FunOrMFA, Pid);
+invoke(Pid, Name, FunOrMFA) when is_pid(Pid) ->
+ case invoke([Pid], Name, FunOrMFA) of
+ {[{Pid, Result}], []} ->
+ Result;
+ {[], [{Pid, {Class, Reason, StackTrace}}]} ->
+ erlang:raise(Class, Reason, StackTrace)
+ end;
+
+invoke([], _Name, _FunOrMFA) -> %% optimisation
+ {[], []};
+invoke([Pid], _Name, FunOrMFA) when node(Pid) =:= node() -> %% optimisation
+ case safe_invoke(Pid, FunOrMFA) of
+ {ok, _, Result} -> {[{Pid, Result}], []};
+ {error, _, Error} -> {[], [{Pid, Error}]}
+ end;
+invoke(Pids, Name, FunOrMFA) when is_list(Pids) ->
+ {LocalPids, Grouped} = group_pids_by_node(Pids),
+ %% The use of multi_call is only safe because the timeout is
+ %% infinity, and thus there is no process spawned in order to do
+ %% the sending. Thus calls can't overtake preceding calls/casts.
+ {Replies, BadNodes} =
+ case maps:keys(Grouped) of
+ [] -> {[], []};
+ RemoteNodes -> gen_server2:multi_call(
+ RemoteNodes, delegate(self(), Name, RemoteNodes),
+ {invoke, FunOrMFA, Grouped}, infinity)
+ end,
+ BadPids = [{Pid, {exit, {nodedown, BadNode}, []}} ||
+ BadNode <- BadNodes,
+ Pid <- maps:get(BadNode, Grouped)],
+ ResultsNoNode = lists:append([safe_invoke(LocalPids, FunOrMFA) |
+ [Results || {_Node, Results} <- Replies]]),
+ lists:foldl(
+ fun ({ok, Pid, Result}, {Good, Bad}) -> {[{Pid, Result} | Good], Bad};
+ ({error, Pid, Error}, {Good, Bad}) -> {Good, [{Pid, Error} | Bad]}
+ end, {[], BadPids}, ResultsNoNode).
+
+monitor(process, Pid) ->
+ ?MODULE:monitor(process, Pid, ?DEFAULT_NAME).
+
+monitor(process, Pid, _Prefix) when node(Pid) =:= node() ->
+ erlang:monitor(process, Pid);
+monitor(process, Pid, Prefix) ->
+ Name = delegate(Pid, Prefix, [node(Pid)]),
+ gen_server2:cast(Name, {monitor, self(), Pid}),
+ {Name, Pid}.
+
+demonitor(Ref) when is_reference(Ref) ->
+ erlang:demonitor(Ref);
+demonitor({Name, Pid}) ->
+ gen_server2:cast(Name, {demonitor, self(), Pid}).
+
+invoke_no_result(Pid, FunOrMFA) when is_pid(Pid) andalso node(Pid) =:= node() ->
+ %% Optimization, avoids calling invoke_no_result/3.
+ %%
+ %% This may seem like a cosmetic change at first but it actually massively reduces the memory usage in mirrored
+ %% queues when ack/nack are sent to the node that hosts a mirror.
+ %% This way binary references are not kept around unnecessarily.
+ %%
+ %% See https://github.com/rabbitmq/rabbitmq-common/issues/208#issuecomment-311308583 for a before/after
+ %% comparison.
+ _ = safe_invoke(Pid, FunOrMFA), %% we don't care about any error
+ ok;
+invoke_no_result(Pid, FunOrMFA) when is_pid(Pid) ->
+ %% Optimization, avoids calling invoke_no_result/3
+ RemoteNode = node(Pid),
+ gen_server2:abcast([RemoteNode], delegate(self(), ?DEFAULT_NAME, [RemoteNode]),
+ {invoke, FunOrMFA,
+ maps:from_list([{RemoteNode, [Pid]}])}),
+ ok;
+invoke_no_result([], _FunOrMFA) -> %% optimisation
+ ok;
+invoke_no_result([Pid], FunOrMFA) when node(Pid) =:= node() -> %% optimisation
+ _ = safe_invoke(Pid, FunOrMFA), %% must not die
+ ok;
+invoke_no_result([Pid], FunOrMFA) ->
+ RemoteNode = node(Pid),
+ gen_server2:abcast([RemoteNode], delegate(self(), ?DEFAULT_NAME, [RemoteNode]),
+ {invoke, FunOrMFA,
+ maps:from_list([{RemoteNode, [Pid]}])}),
+ ok;
+invoke_no_result(Pids, FunOrMFA) when is_list(Pids) ->
+ {LocalPids, Grouped} = group_pids_by_node(Pids),
+ case maps:keys(Grouped) of
+ [] -> ok;
+ RemoteNodes -> gen_server2:abcast(
+ RemoteNodes, delegate(self(), ?DEFAULT_NAME, RemoteNodes),
+ {invoke, FunOrMFA, Grouped})
+ end,
+ _ = safe_invoke(LocalPids, FunOrMFA), %% must not die
+ ok.
+
+%%----------------------------------------------------------------------------
+
+group_pids_by_node(Pids) ->
+ LocalNode = node(),
+ lists:foldl(
+ fun (Pid, {Local, Remote}) when node(Pid) =:= LocalNode ->
+ {[Pid | Local], Remote};
+ (Pid, {Local, Remote}) ->
+ {Local,
+ maps:update_with(
+ node(Pid), fun (List) -> [Pid | List] end, [Pid], Remote)}
+ end, {[], maps:new()}, Pids).
+
+delegate_name(Name, Hash) ->
+ list_to_atom(Name ++ integer_to_list(Hash)).
+
+delegate(Pid, Prefix, RemoteNodes) ->
+ case get(delegate) of
+ undefined -> Name = delegate_name(Prefix,
+ erlang:phash2(Pid,
+ delegate_sup:count(RemoteNodes, Prefix))),
+ put(delegate, Name),
+ Name;
+ Name -> Name
+ end.
+
+safe_invoke(Pids, FunOrMFA) when is_list(Pids) ->
+ [safe_invoke(Pid, FunOrMFA) || Pid <- Pids];
+safe_invoke(Pid, FunOrMFA) when is_pid(Pid) ->
+ try
+ {ok, Pid, apply1(FunOrMFA, Pid)}
+ catch Class:Reason:Stacktrace ->
+ {error, Pid, {Class, Reason, Stacktrace}}
+ end.
+
+apply1({M, F, A}, Arg) -> apply(M, F, [Arg | A]);
+apply1(Fun, Arg) -> Fun(Arg).
+
+%%----------------------------------------------------------------------------
+
+init([Name]) ->
+ {ok, #state{node = node(), monitors = dict:new(), name = Name}, hibernate,
+ {backoff, ?HIBERNATE_AFTER_MIN, ?HIBERNATE_AFTER_MIN, ?DESIRED_HIBERNATE}}.
+
+handle_call({invoke, FunOrMFA, Grouped}, _From, State = #state{node = Node}) ->
+ {reply, safe_invoke(maps:get(Node, Grouped), FunOrMFA), State,
+ hibernate}.
+
+handle_cast({monitor, MonitoringPid, Pid},
+ State = #state{monitors = Monitors}) ->
+ Monitors1 = case dict:find(Pid, Monitors) of
+ {ok, {Ref, Pids}} ->
+ Pids1 = gb_sets:add_element(MonitoringPid, Pids),
+ dict:store(Pid, {Ref, Pids1}, Monitors);
+ error ->
+ Ref = erlang:monitor(process, Pid),
+ Pids = gb_sets:singleton(MonitoringPid),
+ dict:store(Pid, {Ref, Pids}, Monitors)
+ end,
+ {noreply, State#state{monitors = Monitors1}, hibernate};
+
+handle_cast({demonitor, MonitoringPid, Pid},
+ State = #state{monitors = Monitors}) ->
+ Monitors1 = case dict:find(Pid, Monitors) of
+ {ok, {Ref, Pids}} ->
+ Pids1 = gb_sets:del_element(MonitoringPid, Pids),
+ case gb_sets:is_empty(Pids1) of
+ true -> erlang:demonitor(Ref),
+ dict:erase(Pid, Monitors);
+ false -> dict:store(Pid, {Ref, Pids1}, Monitors)
+ end;
+ error ->
+ Monitors
+ end,
+ {noreply, State#state{monitors = Monitors1}, hibernate};
+
+handle_cast({invoke, FunOrMFA, Grouped}, State = #state{node = Node}) ->
+ _ = safe_invoke(maps:get(Node, Grouped), FunOrMFA),
+ {noreply, State, hibernate}.
+
+handle_info({'DOWN', Ref, process, Pid, Info},
+ State = #state{monitors = Monitors, name = Name}) ->
+ {noreply,
+ case dict:find(Pid, Monitors) of
+ {ok, {Ref, Pids}} ->
+ Msg = {'DOWN', {Name, Pid}, process, Pid, Info},
+ gb_sets:fold(fun (MonitoringPid, _) -> MonitoringPid ! Msg end,
+ none, Pids),
+ State#state{monitors = dict:erase(Pid, Monitors)};
+ error ->
+ State
+ end, hibernate};
+
+handle_info(_Info, State) ->
+ {noreply, State, hibernate}.
+
+terminate(_Reason, _State) ->
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
diff --git a/deps/rabbit_common/src/delegate_sup.erl b/deps/rabbit_common/src/delegate_sup.erl
new file mode 100644
index 0000000000..b92e1eaa46
--- /dev/null
+++ b/deps/rabbit_common/src/delegate_sup.erl
@@ -0,0 +1,55 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(delegate_sup).
+
+-behaviour(supervisor).
+
+-export([start_link/1, start_link/2, count/1, count/2, sup_name/1]).
+
+-export([init/1]).
+
+-define(SERVER, "delegate_").
+
+%%----------------------------------------------------------------------------
+
+-spec start_link(integer()) -> rabbit_types:ok_pid_or_error().
+-spec start_link(integer(), string()) -> rabbit_types:ok_pid_or_error().
+-spec count([node()]) -> integer().
+
+%%----------------------------------------------------------------------------
+
+sup_name(Prefix) ->
+ list_to_atom(Prefix ++ "sup").
+
+start_link(Count, Prefix) ->
+ supervisor:start_link({local, sup_name(Prefix)}, ?MODULE, [Count, Prefix]).
+start_link(Count) ->
+ start_link(Count, ?SERVER).
+
+count(Nodes) ->
+ count(Nodes, ?SERVER).
+
+count([], _) ->
+ 1;
+count([Node | Nodes], Prefix) ->
+ try
+ length(supervisor:which_children({sup_name(Prefix), Node}))
+ catch exit:{{R, _}, _} when R =:= nodedown; R =:= shutdown ->
+ count(Nodes, Prefix);
+ exit:{R, _} when R =:= noproc; R =:= normal; R =:= shutdown;
+ R =:= nodedown ->
+ count(Nodes, Prefix)
+ end.
+
+%%----------------------------------------------------------------------------
+
+init([Count, Name]) ->
+ {ok, {{one_for_one, 10, 10},
+ [{Num, {delegate, start_link, [Name, Num]},
+ transient, 16#ffffffff, worker, [delegate]} ||
+ Num <- lists:seq(0, Count - 1)]}}.
diff --git a/deps/rabbit_common/src/file_handle_cache.erl b/deps/rabbit_common/src/file_handle_cache.erl
new file mode 100644
index 0000000000..9220f40ce4
--- /dev/null
+++ b/deps/rabbit_common/src/file_handle_cache.erl
@@ -0,0 +1,1564 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(file_handle_cache).
+
+%% A File Handle Cache
+%%
+%% This extends a subset of the functionality of the Erlang file
+%% module. In the below, we use "file handle" to specifically refer to
+%% file handles, and "file descriptor" to refer to descriptors which
+%% are not file handles, e.g. sockets.
+%%
+%% Some constraints
+%% 1) This supports one writer, multiple readers per file. Nothing
+%% else.
+%% 2) Do not open the same file from different processes. Bad things
+%% may happen, especially for writes.
+%% 3) Writes are all appends. You cannot write to the middle of a
+%% file, although you can truncate and then append if you want.
+%% 4) There are read and write buffers. Feel free to use the read_ahead
+%% mode, but beware of the interaction between that buffer and the write
+%% buffer.
+%%
+%% Some benefits
+%% 1) You do not have to remember to call sync before close
+%% 2) Buffering is much more flexible than with the plain file module,
+%% and you can control when the buffer gets flushed out. This means
+%% that you can rely on reads-after-writes working, without having to
+%% call the expensive sync.
+%% 3) Unnecessary calls to position and sync get optimised out.
+%% 4) You can find out what your 'real' offset is, and what your
+%% 'virtual' offset is (i.e. where the hdl really is, and where it
+%% would be after the write buffer is written out).
+%%
+%% There is also a server component which serves to limit the number
+%% of open file descriptors. This is a hard limit: the server
+%% component will ensure that clients do not have more file
+%% descriptors open than it's configured to allow.
+%%
+%% On open, the client requests permission from the server to open the
+%% required number of file handles. The server may ask the client to
+%% close other file handles that it has open, or it may queue the
+%% request and ask other clients to close file handles they have open
+%% in order to satisfy the request. Requests are always satisfied in
+%% the order they arrive, even if a latter request (for a small number
+%% of file handles) can be satisfied before an earlier request (for a
+%% larger number of file handles). On close, the client sends a
+%% message to the server. These messages allow the server to keep
+%% track of the number of open handles. The client also keeps a
+%% gb_tree which is updated on every use of a file handle, mapping the
+%% time at which the file handle was last used (timestamp) to the
+%% handle. Thus the smallest key in this tree maps to the file handle
+%% that has not been used for the longest amount of time. This
+%% smallest key is included in the messages to the server. As such,
+%% the server keeps track of when the least recently used file handle
+%% was used *at the point of the most recent open or close* by each
+%% client.
+%%
+%% Note that this data can go very out of date, by the client using
+%% the least recently used handle.
+%%
+%% When the limit is exceeded (i.e. the number of open file handles is
+%% at the limit and there are pending 'open' requests), the server
+%% calculates the average age of the last reported least recently used
+%% file handle of all the clients. It then tells all the clients to
+%% close any handles not used for longer than this average, by
+%% invoking the callback the client registered. The client should
+%% receive this message and pass it into
+%% set_maximum_since_use/1. However, it is highly possible this age
+%% will be greater than the ages of all the handles the client knows
+%% of because the client has used its file handles in the mean
+%% time. Thus at this point the client reports to the server the
+%% current timestamp at which its least recently used file handle was
+%% last used. The server will check two seconds later that either it
+%% is back under the limit, in which case all is well again, or if
+%% not, it will calculate a new average age. Its data will be much
+%% more recent now, and so it is very likely that when this is
+%% communicated to the clients, the clients will close file handles.
+%% (In extreme cases, where it's very likely that all clients have
+%% used their open handles since they last sent in an update, which
+%% would mean that the average will never cause any file handles to
+%% be closed, the server can send out an average age of 0, resulting
+%% in all available clients closing all their file handles.)
+%%
+%% Care is taken to ensure that (a) processes which are blocked
+%% waiting for file descriptors to become available are not sent
+%% requests to close file handles; and (b) given it is known how many
+%% file handles a process has open, when the average age is forced to
+%% 0, close messages are only sent to enough processes to release the
+%% correct number of file handles and the list of processes is
+%% randomly shuffled. This ensures we don't cause processes to
+%% needlessly close file handles, and ensures that we don't always
+%% make such requests of the same processes.
+%%
+%% The advantage of this scheme is that there is only communication
+%% from the client to the server on open, close, and when in the
+%% process of trying to reduce file handle usage. There is no
+%% communication from the client to the server on normal file handle
+%% operations. This scheme forms a feed-back loop - the server does
+%% not care which file handles are closed, just that some are, and it
+%% checks this repeatedly when over the limit.
+%%
+%% Handles which are closed as a result of the server are put into a
+%% "soft-closed" state in which the handle is closed (data flushed out
+%% and sync'd first) but the state is maintained. The handle will be
+%% fully reopened again as soon as needed, thus users of this library
+%% do not need to worry about their handles being closed by the server
+%% - reopening them when necessary is handled transparently.
+%%
+%% The server also supports obtain, release and transfer. obtain/{0,1}
+%% blocks until a file descriptor is available, at which point the
+%% requesting process is considered to 'own' more descriptor(s).
+%% release/{0,1} is the inverse operation and releases previously obtained
+%% descriptor(s). transfer/{1,2} transfers ownership of file descriptor(s)
+%% between processes. It is non-blocking. Obtain has a
+%% lower limit, set by the ?OBTAIN_LIMIT/1 macro. File handles can use
+%% the entire limit, but will be evicted by obtain calls up to the
+%% point at which no more obtain calls can be satisfied by the obtains
+%% limit. Thus there will always be some capacity available for file
+%% handles. Processes that use obtain are never asked to return them,
+%% and they are not managed in any way by the server. It is simply a
+%% mechanism to ensure that processes that need file descriptors such
+%% as sockets can do so in such a way that the overall number of open
+%% file descriptors is managed.
+%%
+%% The callers of register_callback/3, obtain, and the argument of
+%% transfer are monitored, reducing the count of handles in use
+%% appropriately when the processes terminate.
+
+-behaviour(gen_server2).
+
+-export([register_callback/3]).
+-export([open/3, close/1, read/2, append/2, needs_sync/1, sync/1, position/2,
+ truncate/1, current_virtual_offset/1, current_raw_offset/1, flush/1,
+ copy/3, set_maximum_since_use/1, delete/1, clear/1,
+ open_with_absolute_path/3]).
+-export([obtain/0, obtain/1, release/0, release/1, transfer/1, transfer/2,
+ set_limit/1, get_limit/0, info_keys/0, with_handle/1, with_handle/2,
+ info/0, info/1, clear_read_cache/0, clear_process_read_cache/0]).
+-export([set_reservation/0, set_reservation/1, release_reservation/0]).
+-export([ulimit/0]).
+
+-export([start_link/0, start_link/2, init/1, handle_call/3, handle_cast/2,
+ handle_info/2, terminate/2, code_change/3, prioritise_cast/3]).
+
+-define(SERVER, ?MODULE).
+%% Reserve 3 handles for ra usage: wal, segment writer and a dets table
+-define(RESERVED_FOR_OTHERS, 100 + 3).
+
+-define(FILE_HANDLES_LIMIT_OTHER, 1024).
+-define(FILE_HANDLES_CHECK_INTERVAL, 2000).
+
+-define(OBTAIN_LIMIT(LIMIT), trunc((LIMIT * 0.9) - 2)).
+-define(CLIENT_ETS_TABLE, file_handle_cache_client).
+-define(ELDERS_ETS_TABLE, file_handle_cache_elders).
+
+%%----------------------------------------------------------------------------
+
+-record(file,
+ { reader_count,
+ has_writer
+ }).
+
+-record(handle,
+ { hdl,
+ ref,
+ offset,
+ is_dirty,
+ write_buffer_size,
+ write_buffer_size_limit,
+ write_buffer,
+ read_buffer,
+ read_buffer_pos,
+ read_buffer_rem, %% Num of bytes from pos to end
+ read_buffer_size, %% Next size of read buffer to use
+ read_buffer_size_limit, %% Max size of read buffer to use
+ read_buffer_usage, %% Bytes we have read from it, for tuning
+ at_eof,
+ path,
+ mode,
+ options,
+ is_write,
+ is_read,
+ last_used_at
+ }).
+
+-record(fhc_state,
+ { elders,
+ limit,
+ open_count,
+ open_pending,
+ obtain_limit, %%socket
+ obtain_count_socket,
+ obtain_count_file,
+ obtain_pending_socket,
+ obtain_pending_file,
+ clients,
+ timer_ref,
+ alarm_set,
+ alarm_clear,
+ reserve_count_socket,
+ reserve_count_file
+ }).
+
+-record(cstate,
+ { pid,
+ callback,
+ opened,
+ obtained_socket,
+ obtained_file,
+ blocked,
+ pending_closes,
+ reserved_socket,
+ reserved_file
+ }).
+
+-record(pending,
+ { kind,
+ pid,
+ requested,
+ from
+ }).
+
+%%----------------------------------------------------------------------------
+%% Specs
+%%----------------------------------------------------------------------------
+
+-type ref() :: any().
+-type ok_or_error() :: 'ok' | {'error', any()}.
+-type val_or_error(T) :: {'ok', T} | {'error', any()}.
+-type position() :: ('bof' | 'eof' | non_neg_integer() |
+ {('bof' |'eof'), non_neg_integer()} |
+ {'cur', integer()}).
+-type offset() :: non_neg_integer().
+
+-spec register_callback(atom(), atom(), [any()]) -> 'ok'.
+-spec open
+ (file:filename(), [any()],
+ [{'write_buffer', (non_neg_integer() | 'infinity' | 'unbuffered')} |
+ {'read_buffer', (non_neg_integer() | 'unbuffered')}]) ->
+ val_or_error(ref()).
+-spec open_with_absolute_path
+ (file:filename(), [any()],
+ [{'write_buffer', (non_neg_integer() | 'infinity' | 'unbuffered')} |
+ {'read_buffer', (non_neg_integer() | 'unbuffered')}]) ->
+ val_or_error(ref()).
+-spec close(ref()) -> ok_or_error().
+-spec read
+ (ref(), non_neg_integer()) -> val_or_error([char()] | binary()) | 'eof'.
+-spec append(ref(), iodata()) -> ok_or_error().
+-spec sync(ref()) -> ok_or_error().
+-spec position(ref(), position()) -> val_or_error(offset()).
+-spec truncate(ref()) -> ok_or_error().
+-spec current_virtual_offset(ref()) -> val_or_error(offset()).
+-spec current_raw_offset(ref()) -> val_or_error(offset()).
+-spec flush(ref()) -> ok_or_error().
+-spec copy(ref(), ref(), non_neg_integer()) -> val_or_error(non_neg_integer()).
+-spec delete(ref()) -> ok_or_error().
+-spec clear(ref()) -> ok_or_error().
+-spec set_maximum_since_use(non_neg_integer()) -> 'ok'.
+-spec obtain() -> 'ok'.
+-spec obtain(non_neg_integer()) -> 'ok'.
+-spec release() -> 'ok'.
+-spec release(non_neg_integer()) -> 'ok'.
+-spec transfer(pid()) -> 'ok'.
+-spec transfer(pid(), non_neg_integer()) -> 'ok'.
+-spec with_handle(fun(() -> A)) -> A.
+-spec with_handle(non_neg_integer(), fun(() -> A)) -> A.
+-spec set_limit(non_neg_integer()) -> 'ok'.
+-spec get_limit() -> non_neg_integer().
+-spec info_keys() -> rabbit_types:info_keys().
+-spec info() -> rabbit_types:infos().
+-spec info([atom()]) -> rabbit_types:infos().
+-spec ulimit() -> 'unknown' | non_neg_integer().
+
+%%----------------------------------------------------------------------------
+-define(INFO_KEYS, [total_limit, total_used, sockets_limit, sockets_used]).
+
+%%----------------------------------------------------------------------------
+%% Public API
+%%----------------------------------------------------------------------------
+
+start_link() ->
+ start_link(fun alarm_handler:set_alarm/1, fun alarm_handler:clear_alarm/1).
+
+start_link(AlarmSet, AlarmClear) ->
+ gen_server2:start_link({local, ?SERVER}, ?MODULE, [AlarmSet, AlarmClear],
+ [{timeout, infinity}]).
+
+register_callback(M, F, A)
+ when is_atom(M) andalso is_atom(F) andalso is_list(A) ->
+ gen_server2:cast(?SERVER, {register_callback, self(), {M, F, A}}).
+
+open(Path, Mode, Options) ->
+ open_with_absolute_path(filename:absname(Path), Mode, Options).
+
+open_with_absolute_path(Path, Mode, Options) ->
+ File1 = #file { reader_count = RCount, has_writer = HasWriter } =
+ case get({Path, fhc_file}) of
+ File = #file {} -> File;
+ undefined -> #file { reader_count = 0,
+ has_writer = false }
+ end,
+ Mode1 = append_to_write(Mode),
+ IsWriter = is_writer(Mode1),
+ case IsWriter andalso HasWriter of
+ true -> {error, writer_exists};
+ false -> {ok, Ref} = new_closed_handle(Path, Mode1, Options),
+ case get_or_reopen_timed([{Ref, new}]) of
+ {ok, [_Handle1]} ->
+ RCount1 = case is_reader(Mode1) of
+ true -> RCount + 1;
+ false -> RCount
+ end,
+ HasWriter1 = HasWriter orelse IsWriter,
+ put({Path, fhc_file},
+ File1 #file { reader_count = RCount1,
+ has_writer = HasWriter1 }),
+ {ok, Ref};
+ Error ->
+ erase({Ref, fhc_handle}),
+ Error
+ end
+ end.
+
+close(Ref) ->
+ case erase({Ref, fhc_handle}) of
+ undefined -> ok;
+ Handle -> case hard_close(Handle) of
+ ok -> ok;
+ {Error, Handle1} -> put_handle(Ref, Handle1),
+ Error
+ end
+ end.
+
+read(Ref, Count) ->
+ with_flushed_handles(
+ [Ref], keep,
+ fun ([#handle { is_read = false }]) ->
+ {error, not_open_for_reading};
+ ([#handle{read_buffer_size_limit = 0,
+ hdl = Hdl, offset = Offset} = Handle]) ->
+ %% The read buffer is disabled. This is just an
+ %% optimization: the clauses below can handle this case.
+ case prim_file_read(Hdl, Count) of
+ {ok, Data} -> {{ok, Data},
+ [Handle#handle{offset = Offset+size(Data)}]};
+ eof -> {eof, [Handle #handle { at_eof = true }]};
+ Error -> {Error, Handle}
+ end;
+ ([Handle = #handle{read_buffer = Buf,
+ read_buffer_pos = BufPos,
+ read_buffer_rem = BufRem,
+ read_buffer_usage = BufUsg,
+ offset = Offset}])
+ when BufRem >= Count ->
+ <<_:BufPos/binary, Res:Count/binary, _/binary>> = Buf,
+ {{ok, Res}, [Handle#handle{offset = Offset + Count,
+ read_buffer_pos = BufPos + Count,
+ read_buffer_rem = BufRem - Count,
+ read_buffer_usage = BufUsg + Count }]};
+ ([Handle0]) ->
+ maybe_reduce_read_cache([Ref]),
+ Handle = #handle{read_buffer = Buf,
+ read_buffer_pos = BufPos,
+ read_buffer_rem = BufRem,
+ read_buffer_size = BufSz,
+ hdl = Hdl,
+ offset = Offset}
+ = tune_read_buffer_limit(Handle0, Count),
+ WantedCount = Count - BufRem,
+ case prim_file_read(Hdl, max(BufSz, WantedCount)) of
+ {ok, Data} ->
+ <<_:BufPos/binary, BufTl/binary>> = Buf,
+ ReadCount = size(Data),
+ case ReadCount < WantedCount of
+ true ->
+ OffSet1 = Offset + BufRem + ReadCount,
+ {{ok, <<BufTl/binary, Data/binary>>},
+ [reset_read_buffer(
+ Handle#handle{offset = OffSet1})]};
+ false ->
+ <<Hd:WantedCount/binary, _/binary>> = Data,
+ OffSet1 = Offset + BufRem + WantedCount,
+ BufRem1 = ReadCount - WantedCount,
+ {{ok, <<BufTl/binary, Hd/binary>>},
+ [Handle#handle{offset = OffSet1,
+ read_buffer = Data,
+ read_buffer_pos = WantedCount,
+ read_buffer_rem = BufRem1,
+ read_buffer_usage = WantedCount}]}
+ end;
+ eof ->
+ {eof, [Handle #handle { at_eof = true }]};
+ Error ->
+ {Error, [reset_read_buffer(Handle)]}
+ end
+ end).
+
+append(Ref, Data) ->
+ with_handles(
+ [Ref],
+ fun ([#handle { is_write = false }]) ->
+ {error, not_open_for_writing};
+ ([Handle]) ->
+ case maybe_seek(eof, Handle) of
+ {{ok, _Offset}, #handle { hdl = Hdl, offset = Offset,
+ write_buffer_size_limit = 0,
+ at_eof = true } = Handle1} ->
+ Offset1 = Offset + iolist_size(Data),
+ {prim_file_write(Hdl, Data),
+ [Handle1 #handle { is_dirty = true, offset = Offset1 }]};
+ {{ok, _Offset}, #handle { write_buffer = WriteBuffer,
+ write_buffer_size = Size,
+ write_buffer_size_limit = Limit,
+ at_eof = true } = Handle1} ->
+ WriteBuffer1 = [Data | WriteBuffer],
+ Size1 = Size + iolist_size(Data),
+ Handle2 = Handle1 #handle { write_buffer = WriteBuffer1,
+ write_buffer_size = Size1 },
+ case Limit =/= infinity andalso Size1 > Limit of
+ true -> {Result, Handle3} = write_buffer(Handle2),
+ {Result, [Handle3]};
+ false -> {ok, [Handle2]}
+ end;
+ {{error, _} = Error, Handle1} ->
+ {Error, [Handle1]}
+ end
+ end).
+
+sync(Ref) ->
+ with_flushed_handles(
+ [Ref], keep,
+ fun ([#handle { is_dirty = false, write_buffer = [] }]) ->
+ ok;
+ ([Handle = #handle { hdl = Hdl,
+ is_dirty = true, write_buffer = [] }]) ->
+ case prim_file_sync(Hdl) of
+ ok -> {ok, [Handle #handle { is_dirty = false }]};
+ Error -> {Error, [Handle]}
+ end
+ end).
+
+needs_sync(Ref) ->
+ %% This must *not* use with_handles/2; see bug 25052
+ case get({Ref, fhc_handle}) of
+ #handle { is_dirty = false, write_buffer = [] } -> false;
+ #handle {} -> true
+ end.
+
+position(Ref, NewOffset) ->
+ with_flushed_handles(
+ [Ref], keep,
+ fun ([Handle]) -> {Result, Handle1} = maybe_seek(NewOffset, Handle),
+ {Result, [Handle1]}
+ end).
+
+truncate(Ref) ->
+ with_flushed_handles(
+ [Ref],
+ fun ([Handle1 = #handle { hdl = Hdl }]) ->
+ case prim_file:truncate(Hdl) of
+ ok -> {ok, [Handle1 #handle { at_eof = true }]};
+ Error -> {Error, [Handle1]}
+ end
+ end).
+
+current_virtual_offset(Ref) ->
+ with_handles([Ref], fun ([#handle { at_eof = true, is_write = true,
+ offset = Offset,
+ write_buffer_size = Size }]) ->
+ {ok, Offset + Size};
+ ([#handle { offset = Offset }]) ->
+ {ok, Offset}
+ end).
+
+current_raw_offset(Ref) ->
+ with_handles([Ref], fun ([Handle]) -> {ok, Handle #handle.offset} end).
+
+flush(Ref) ->
+ with_flushed_handles([Ref], fun ([Handle]) -> {ok, [Handle]} end).
+
+copy(Src, Dest, Count) ->
+ with_flushed_handles(
+ [Src, Dest],
+ fun ([SHandle = #handle { is_read = true, hdl = SHdl, offset = SOffset },
+ DHandle = #handle { is_write = true, hdl = DHdl, offset = DOffset }]
+ ) ->
+ case prim_file:copy(SHdl, DHdl, Count) of
+ {ok, Count1} = Result1 ->
+ {Result1,
+ [SHandle #handle { offset = SOffset + Count1 },
+ DHandle #handle { offset = DOffset + Count1,
+ is_dirty = true }]};
+ Error ->
+ {Error, [SHandle, DHandle]}
+ end;
+ (_Handles) ->
+ {error, incorrect_handle_modes}
+ end).
+
+delete(Ref) ->
+ case erase({Ref, fhc_handle}) of
+ undefined ->
+ ok;
+ Handle = #handle { path = Path } ->
+ case hard_close(Handle #handle { is_dirty = false,
+ write_buffer = [] }) of
+ ok -> prim_file:delete(Path);
+ {Error, Handle1} -> put_handle(Ref, Handle1),
+ Error
+ end
+ end.
+
+clear(Ref) ->
+ with_handles(
+ [Ref],
+ fun ([#handle { at_eof = true, write_buffer_size = 0, offset = 0 }]) ->
+ ok;
+ ([Handle]) ->
+ case maybe_seek(bof, Handle#handle{write_buffer = [],
+ write_buffer_size = 0}) of
+ {{ok, 0}, Handle1 = #handle { hdl = Hdl }} ->
+ case prim_file:truncate(Hdl) of
+ ok -> {ok, [Handle1 #handle { at_eof = true }]};
+ Error -> {Error, [Handle1]}
+ end;
+ {{error, _} = Error, Handle1} ->
+ {Error, [Handle1]}
+ end
+ end).
+
+set_maximum_since_use(MaximumAge) ->
+ Now = erlang:monotonic_time(),
+ case lists:foldl(
+ fun ({{Ref, fhc_handle},
+ Handle = #handle { hdl = Hdl, last_used_at = Then }}, Rep) ->
+ case Hdl =/= closed andalso
+ erlang:convert_time_unit(Now - Then,
+ native,
+ micro_seconds)
+ >= MaximumAge of
+ true -> soft_close(Ref, Handle) orelse Rep;
+ false -> Rep
+ end;
+ (_KeyValuePair, Rep) ->
+ Rep
+ end, false, get()) of
+ false -> age_tree_change(), ok;
+ true -> ok
+ end.
+
+obtain() -> obtain(1).
+set_reservation() -> set_reservation(1).
+release() -> release(1).
+release_reservation() -> release_reservation(file).
+transfer(Pid) -> transfer(Pid, 1).
+
+obtain(Count) -> obtain(Count, socket).
+set_reservation(Count) -> set_reservation(Count, file).
+release(Count) -> release(Count, socket).
+
+with_handle(Fun) ->
+ with_handle(1, Fun).
+
+with_handle(N, Fun) ->
+ ok = obtain(N, file),
+ try Fun()
+ after ok = release(N, file)
+ end.
+
+obtain(Count, Type) when Count > 0 ->
+ %% If the FHC isn't running, obtains succeed immediately.
+ case whereis(?SERVER) of
+ undefined -> ok;
+ _ -> gen_server2:call(
+ ?SERVER, {obtain, Count, Type, self()}, infinity)
+ end.
+
+set_reservation(Count, Type) when Count > 0 ->
+ %% If the FHC isn't running, reserve succeed immediately.
+ case whereis(?SERVER) of
+ undefined -> ok;
+ _ -> gen_server2:cast(?SERVER, {set_reservation, Count, Type, self()})
+ end.
+
+release(Count, Type) when Count > 0 ->
+ gen_server2:cast(?SERVER, {release, Count, Type, self()}).
+
+release_reservation(Type) ->
+ gen_server2:cast(?SERVER, {release_reservation, Type, self()}).
+
+transfer(Pid, Count) when Count > 0 ->
+ gen_server2:cast(?SERVER, {transfer, Count, self(), Pid}).
+
+set_limit(Limit) ->
+ gen_server2:call(?SERVER, {set_limit, Limit}, infinity).
+
+get_limit() ->
+ gen_server2:call(?SERVER, get_limit, infinity).
+
+info_keys() -> ?INFO_KEYS.
+
+info() -> info(?INFO_KEYS).
+info(Items) -> gen_server2:call(?SERVER, {info, Items}, infinity).
+
+clear_read_cache() ->
+ gen_server2:cast(?SERVER, clear_read_cache).
+
+clear_process_read_cache() ->
+ [
+ begin
+ Handle1 = reset_read_buffer(Handle),
+ put({Ref, fhc_handle}, Handle1)
+ end ||
+ {{Ref, fhc_handle}, Handle} <- get(),
+ size(Handle#handle.read_buffer) > 0
+ ].
+
+%%----------------------------------------------------------------------------
+%% Internal functions
+%%----------------------------------------------------------------------------
+
+prim_file_read(Hdl, Size) ->
+ file_handle_cache_stats:update(
+ io_read, Size, fun() -> prim_file:read(Hdl, Size) end).
+
+prim_file_write(Hdl, Bytes) ->
+ file_handle_cache_stats:update(
+ io_write, iolist_size(Bytes), fun() -> prim_file:write(Hdl, Bytes) end).
+
+prim_file_sync(Hdl) ->
+ file_handle_cache_stats:update(io_sync, fun() -> prim_file:sync(Hdl) end).
+
+prim_file_position(Hdl, NewOffset) ->
+ file_handle_cache_stats:update(
+ io_seek, fun() -> prim_file:position(Hdl, NewOffset) end).
+
+is_reader(Mode) -> lists:member(read, Mode).
+
+is_writer(Mode) -> lists:member(write, Mode).
+
+append_to_write(Mode) ->
+ case lists:member(append, Mode) of
+ true -> [write | Mode -- [append, write]];
+ false -> Mode
+ end.
+
+with_handles(Refs, Fun) ->
+ with_handles(Refs, reset, Fun).
+
+with_handles(Refs, ReadBuffer, Fun) ->
+ case get_or_reopen_timed([{Ref, reopen} || Ref <- Refs]) of
+ {ok, Handles0} ->
+ Handles = case ReadBuffer of
+ reset -> [reset_read_buffer(H) || H <- Handles0];
+ keep -> Handles0
+ end,
+ case Fun(Handles) of
+ {Result, Handles1} when is_list(Handles1) ->
+ _ = lists:zipwith(fun put_handle/2, Refs, Handles1),
+ Result;
+ Result ->
+ Result
+ end;
+ Error ->
+ Error
+ end.
+
+with_flushed_handles(Refs, Fun) ->
+ with_flushed_handles(Refs, reset, Fun).
+
+with_flushed_handles(Refs, ReadBuffer, Fun) ->
+ with_handles(
+ Refs, ReadBuffer,
+ fun (Handles) ->
+ case lists:foldl(
+ fun (Handle, {ok, HandlesAcc}) ->
+ {Res, Handle1} = write_buffer(Handle),
+ {Res, [Handle1 | HandlesAcc]};
+ (Handle, {Error, HandlesAcc}) ->
+ {Error, [Handle | HandlesAcc]}
+ end, {ok, []}, Handles) of
+ {ok, Handles1} ->
+ Fun(lists:reverse(Handles1));
+ {Error, Handles1} ->
+ {Error, lists:reverse(Handles1)}
+ end
+ end).
+
+get_or_reopen_timed(RefNewOrReopens) ->
+ file_handle_cache_stats:update(
+ io_file_handle_open_attempt, fun() -> get_or_reopen(RefNewOrReopens) end).
+
+get_or_reopen(RefNewOrReopens) ->
+ case partition_handles(RefNewOrReopens) of
+ {OpenHdls, []} ->
+ {ok, [Handle || {_Ref, Handle} <- OpenHdls]};
+ {OpenHdls, ClosedHdls} ->
+ Oldest = oldest(get_age_tree(),
+ fun () -> erlang:monotonic_time() end),
+ case gen_server2:call(?SERVER, {open, self(), length(ClosedHdls),
+ Oldest}, infinity) of
+ ok ->
+ case reopen(ClosedHdls) of
+ {ok, RefHdls} -> sort_handles(RefNewOrReopens,
+ OpenHdls, RefHdls, []);
+ Error -> Error
+ end;
+ close ->
+ [soft_close(Ref, Handle) ||
+ {{Ref, fhc_handle}, Handle = #handle { hdl = Hdl }} <-
+ get(),
+ Hdl =/= closed],
+ get_or_reopen(RefNewOrReopens)
+ end
+ end.
+
+reopen(ClosedHdls) -> reopen(ClosedHdls, get_age_tree(), []).
+
+reopen([], Tree, RefHdls) ->
+ put_age_tree(Tree),
+ {ok, lists:reverse(RefHdls)};
+reopen([{Ref, NewOrReopen, Handle = #handle { hdl = closed,
+ path = Path,
+ mode = Mode0,
+ offset = Offset,
+ last_used_at = undefined }} |
+ RefNewOrReopenHdls] = ToOpen, Tree, RefHdls) ->
+ Mode = case NewOrReopen of
+ new -> Mode0;
+ reopen -> file_handle_cache_stats:update(io_reopen),
+ [read | Mode0]
+ end,
+ case prim_file:open(Path, Mode) of
+ {ok, Hdl} ->
+ Now = erlang:monotonic_time(),
+ {{ok, _Offset}, Handle1} =
+ maybe_seek(Offset, reset_read_buffer(
+ Handle#handle{hdl = Hdl,
+ offset = 0,
+ last_used_at = Now})),
+ put({Ref, fhc_handle}, Handle1),
+ reopen(RefNewOrReopenHdls, gb_trees:insert({Now, Ref}, true, Tree),
+ [{Ref, Handle1} | RefHdls]);
+ Error ->
+ %% NB: none of the handles in ToOpen are in the age tree
+ Oldest = oldest(Tree, fun () -> undefined end),
+ [gen_server2:cast(?SERVER, {close, self(), Oldest}) || _ <- ToOpen],
+ put_age_tree(Tree),
+ Error
+ end.
+
+partition_handles(RefNewOrReopens) ->
+ lists:foldr(
+ fun ({Ref, NewOrReopen}, {Open, Closed}) ->
+ case get({Ref, fhc_handle}) of
+ #handle { hdl = closed } = Handle ->
+ {Open, [{Ref, NewOrReopen, Handle} | Closed]};
+ #handle {} = Handle ->
+ {[{Ref, Handle} | Open], Closed}
+ end
+ end, {[], []}, RefNewOrReopens).
+
+sort_handles([], [], [], Acc) ->
+ {ok, lists:reverse(Acc)};
+sort_handles([{Ref, _} | RefHdls], [{Ref, Handle} | RefHdlsA], RefHdlsB, Acc) ->
+ sort_handles(RefHdls, RefHdlsA, RefHdlsB, [Handle | Acc]);
+sort_handles([{Ref, _} | RefHdls], RefHdlsA, [{Ref, Handle} | RefHdlsB], Acc) ->
+ sort_handles(RefHdls, RefHdlsA, RefHdlsB, [Handle | Acc]).
+
+put_handle(Ref, Handle = #handle { last_used_at = Then }) ->
+ Now = erlang:monotonic_time(),
+ age_tree_update(Then, Now, Ref),
+ put({Ref, fhc_handle}, Handle #handle { last_used_at = Now }).
+
+with_age_tree(Fun) -> put_age_tree(Fun(get_age_tree())).
+
+get_age_tree() ->
+ case get(fhc_age_tree) of
+ undefined -> gb_trees:empty();
+ AgeTree -> AgeTree
+ end.
+
+put_age_tree(Tree) -> put(fhc_age_tree, Tree).
+
+age_tree_update(Then, Now, Ref) ->
+ with_age_tree(
+ fun (Tree) ->
+ gb_trees:insert({Now, Ref}, true,
+ gb_trees:delete_any({Then, Ref}, Tree))
+ end).
+
+age_tree_delete(Then, Ref) ->
+ with_age_tree(
+ fun (Tree) ->
+ Tree1 = gb_trees:delete_any({Then, Ref}, Tree),
+ Oldest = oldest(Tree1, fun () -> undefined end),
+ gen_server2:cast(?SERVER, {close, self(), Oldest}),
+ Tree1
+ end).
+
+age_tree_change() ->
+ with_age_tree(
+ fun (Tree) ->
+ case gb_trees:is_empty(Tree) of
+ true -> Tree;
+ false -> {{Oldest, _Ref}, _} = gb_trees:smallest(Tree),
+ gen_server2:cast(?SERVER, {update, self(), Oldest}),
+ Tree
+ end
+ end).
+
+oldest(Tree, DefaultFun) ->
+ case gb_trees:is_empty(Tree) of
+ true -> DefaultFun();
+ false -> {{Oldest, _Ref}, _} = gb_trees:smallest(Tree),
+ Oldest
+ end.
+
+new_closed_handle(Path, Mode, Options) ->
+ WriteBufferSize =
+ case application:get_env(rabbit, fhc_write_buffering) of
+ {ok, false} -> 0;
+ {ok, true} ->
+ case proplists:get_value(write_buffer, Options, unbuffered) of
+ unbuffered -> 0;
+ infinity -> infinity;
+ N when is_integer(N) -> N
+ end
+ end,
+ ReadBufferSize =
+ case application:get_env(rabbit, fhc_read_buffering) of
+ {ok, false} -> 0;
+ {ok, true} ->
+ case proplists:get_value(read_buffer, Options, unbuffered) of
+ unbuffered -> 0;
+ N2 when is_integer(N2) -> N2
+ end
+ end,
+ Ref = make_ref(),
+ put({Ref, fhc_handle}, #handle { hdl = closed,
+ ref = Ref,
+ offset = 0,
+ is_dirty = false,
+ write_buffer_size = 0,
+ write_buffer_size_limit = WriteBufferSize,
+ write_buffer = [],
+ read_buffer = <<>>,
+ read_buffer_pos = 0,
+ read_buffer_rem = 0,
+ read_buffer_size = ReadBufferSize,
+ read_buffer_size_limit = ReadBufferSize,
+ read_buffer_usage = 0,
+ at_eof = false,
+ path = Path,
+ mode = Mode,
+ options = Options,
+ is_write = is_writer(Mode),
+ is_read = is_reader(Mode),
+ last_used_at = undefined }),
+ {ok, Ref}.
+
+soft_close(Ref, Handle) ->
+ {Res, Handle1} = soft_close(Handle),
+ case Res of
+ ok -> put({Ref, fhc_handle}, Handle1),
+ true;
+ _ -> put_handle(Ref, Handle1),
+ false
+ end.
+
+soft_close(Handle = #handle { hdl = closed }) ->
+ {ok, Handle};
+soft_close(Handle) ->
+ case write_buffer(Handle) of
+ {ok, #handle { hdl = Hdl,
+ ref = Ref,
+ is_dirty = IsDirty,
+ last_used_at = Then } = Handle1 } ->
+ ok = case IsDirty of
+ true -> prim_file_sync(Hdl);
+ false -> ok
+ end,
+ ok = prim_file:close(Hdl),
+ age_tree_delete(Then, Ref),
+ {ok, Handle1 #handle { hdl = closed,
+ is_dirty = false,
+ last_used_at = undefined }};
+ {_Error, _Handle} = Result ->
+ Result
+ end.
+
+hard_close(Handle) ->
+ case soft_close(Handle) of
+ {ok, #handle { path = Path,
+ is_read = IsReader, is_write = IsWriter }} ->
+ #file { reader_count = RCount, has_writer = HasWriter } = File =
+ get({Path, fhc_file}),
+ RCount1 = case IsReader of
+ true -> RCount - 1;
+ false -> RCount
+ end,
+ HasWriter1 = HasWriter andalso not IsWriter,
+ case RCount1 =:= 0 andalso not HasWriter1 of
+ true -> erase({Path, fhc_file});
+ false -> put({Path, fhc_file},
+ File #file { reader_count = RCount1,
+ has_writer = HasWriter1 })
+ end,
+ ok;
+ {_Error, _Handle} = Result ->
+ Result
+ end.
+
+maybe_seek(New, Handle = #handle{hdl = Hdl,
+ offset = Old,
+ read_buffer_pos = BufPos,
+ read_buffer_rem = BufRem,
+ at_eof = AtEoF}) ->
+ {AtEoF1, NeedsSeek} = needs_seek(AtEoF, Old, New),
+ case NeedsSeek of
+ true when is_number(New) andalso
+ ((New >= Old andalso New =< BufRem + Old)
+ orelse (New < Old andalso Old - New =< BufPos)) ->
+ Diff = New - Old,
+ {{ok, New}, Handle#handle{offset = New,
+ at_eof = AtEoF1,
+ read_buffer_pos = BufPos + Diff,
+ read_buffer_rem = BufRem - Diff}};
+ true ->
+ case prim_file_position(Hdl, New) of
+ {ok, Offset1} = Result ->
+ {Result, reset_read_buffer(Handle#handle{offset = Offset1,
+ at_eof = AtEoF1})};
+ {error, _} = Error ->
+ {Error, Handle}
+ end;
+ false ->
+ {{ok, Old}, Handle}
+ end.
+
+needs_seek( AtEoF, _CurOffset, cur ) -> {AtEoF, false};
+needs_seek( AtEoF, _CurOffset, {cur, 0}) -> {AtEoF, false};
+needs_seek( true, _CurOffset, eof ) -> {true , false};
+needs_seek( true, _CurOffset, {eof, 0}) -> {true , false};
+needs_seek( false, _CurOffset, eof ) -> {true , true };
+needs_seek( false, _CurOffset, {eof, 0}) -> {true , true };
+needs_seek( AtEoF, 0, bof ) -> {AtEoF, false};
+needs_seek( AtEoF, 0, {bof, 0}) -> {AtEoF, false};
+needs_seek( AtEoF, CurOffset, CurOffset) -> {AtEoF, false};
+needs_seek( true, CurOffset, {bof, DesiredOffset})
+ when DesiredOffset >= CurOffset ->
+ {true, true};
+needs_seek( true, _CurOffset, {cur, DesiredOffset})
+ when DesiredOffset > 0 ->
+ {true, true};
+needs_seek( true, CurOffset, DesiredOffset) %% same as {bof, DO}
+ when is_integer(DesiredOffset) andalso DesiredOffset >= CurOffset ->
+ {true, true};
+%% because we can't really track size, we could well end up at EoF and not know
+needs_seek(_AtEoF, _CurOffset, _DesiredOffset) ->
+ {false, true}.
+
+write_buffer(Handle = #handle { write_buffer = [] }) ->
+ {ok, Handle};
+write_buffer(Handle = #handle { hdl = Hdl, offset = Offset,
+ write_buffer = WriteBuffer,
+ write_buffer_size = DataSize,
+ at_eof = true }) ->
+ case prim_file_write(Hdl, lists:reverse(WriteBuffer)) of
+ ok ->
+ Offset1 = Offset + DataSize,
+ {ok, Handle #handle { offset = Offset1, is_dirty = true,
+ write_buffer = [], write_buffer_size = 0 }};
+ {error, _} = Error ->
+ {Error, Handle}
+ end.
+
+reset_read_buffer(Handle) ->
+ Handle#handle{read_buffer = <<>>,
+ read_buffer_pos = 0,
+ read_buffer_rem = 0}.
+
+%% We come into this function whenever there's been a miss while
+%% reading from the buffer - but note that when we first start with a
+%% new handle the usage will be 0. Therefore in that case don't take
+%% it as meaning the buffer was useless, we just haven't done anything
+%% yet!
+tune_read_buffer_limit(Handle = #handle{read_buffer_usage = 0}, _Count) ->
+ Handle;
+%% In this head we have been using the buffer but now tried to read
+%% outside it. So how did we do? If we used less than the size of the
+%% buffer, make the new buffer the size of what we used before, but
+%% add one byte (so that next time we can distinguish between getting
+%% the buffer size exactly right and actually wanting more). If we
+%% read 100% of what we had, then double it for next time, up to the
+%% limit that was set when we were created.
+tune_read_buffer_limit(Handle = #handle{read_buffer = Buf,
+ read_buffer_usage = Usg,
+ read_buffer_size = Sz,
+ read_buffer_size_limit = Lim}, Count) ->
+ %% If the buffer is <<>> then we are in the first read after a
+ %% reset, the read_buffer_usage is the total usage from before the
+ %% reset. But otherwise we are in a read which read off the end of
+ %% the buffer, so really the size of this read should be included
+ %% in the usage.
+ TotalUsg = case Buf of
+ <<>> -> Usg;
+ _ -> Usg + Count
+ end,
+ Handle#handle{read_buffer_usage = 0,
+ read_buffer_size = erlang:min(case TotalUsg < Sz of
+ true -> Usg + 1;
+ false -> Usg * 2
+ end, Lim)}.
+
+maybe_reduce_read_cache(SparedRefs) ->
+ case vm_memory_monitor:get_memory_use(bytes) of
+ {_, infinity} -> ok;
+ {MemUse, MemLimit} when MemUse < MemLimit -> ok;
+ {MemUse, MemLimit} -> reduce_read_cache(
+ (MemUse - MemLimit) * 2,
+ SparedRefs)
+ end.
+
+reduce_read_cache(MemToFree, SparedRefs) ->
+ Handles = lists:sort(
+ fun({_, H1}, {_, H2}) -> H1 < H2 end,
+ [{R, H} || {{R, fhc_handle}, H} <- get(),
+ not lists:member(R, SparedRefs)
+ andalso size(H#handle.read_buffer) > 0]),
+ FreedMem = lists:foldl(
+ fun
+ (_, Freed) when Freed >= MemToFree ->
+ Freed;
+ ({Ref, #handle{read_buffer = Buf} = Handle}, Freed) ->
+ Handle1 = reset_read_buffer(Handle),
+ put({Ref, fhc_handle}, Handle1),
+ Freed + size(Buf)
+ end, 0, Handles),
+ if
+ FreedMem < MemToFree andalso SparedRefs =/= [] ->
+ reduce_read_cache(MemToFree - FreedMem, []);
+ true ->
+ ok
+ end.
+
+infos(Items, State) -> [{Item, i(Item, State)} || Item <- Items].
+
+i(total_limit, #fhc_state{limit = Limit}) -> Limit;
+i(total_used, State) -> used(State);
+i(sockets_limit, #fhc_state{obtain_limit = Limit}) -> Limit;
+i(sockets_used, #fhc_state{obtain_count_socket = Count,
+ reserve_count_socket = RCount}) -> Count + RCount;
+i(files_reserved, #fhc_state{reserve_count_file = RCount}) -> RCount;
+i(Item, _) -> throw({bad_argument, Item}).
+
+used(#fhc_state{open_count = C1,
+ obtain_count_socket = C2,
+ obtain_count_file = C3,
+ reserve_count_socket = C4,
+ reserve_count_file = C5}) -> C1 + C2 + C3 + C4 + C5.
+
+%%----------------------------------------------------------------------------
+%% gen_server2 callbacks
+%%----------------------------------------------------------------------------
+
+init([AlarmSet, AlarmClear]) ->
+ _ = file_handle_cache_stats:init(),
+ Limit = case application:get_env(file_handles_high_watermark) of
+ {ok, Watermark} when (is_integer(Watermark) andalso
+ Watermark > 0) ->
+ Watermark;
+ _ ->
+ case ulimit() of
+ unknown -> ?FILE_HANDLES_LIMIT_OTHER;
+ Lim -> lists:max([2, Lim - ?RESERVED_FOR_OTHERS])
+ end
+ end,
+ ObtainLimit = obtain_limit(Limit),
+ error_logger:info_msg("Limiting to approx ~p file handles (~p sockets)~n",
+ [Limit, ObtainLimit]),
+ Clients = ets:new(?CLIENT_ETS_TABLE, [set, private, {keypos, #cstate.pid}]),
+ Elders = ets:new(?ELDERS_ETS_TABLE, [set, private]),
+ {ok, #fhc_state { elders = Elders,
+ limit = Limit,
+ open_count = 0,
+ open_pending = pending_new(),
+ obtain_limit = ObtainLimit,
+ obtain_count_file = 0,
+ obtain_pending_file = pending_new(),
+ obtain_count_socket = 0,
+ obtain_pending_socket = pending_new(),
+ clients = Clients,
+ timer_ref = undefined,
+ alarm_set = AlarmSet,
+ alarm_clear = AlarmClear,
+ reserve_count_file = 0,
+ reserve_count_socket = 0 }}.
+
+prioritise_cast(Msg, _Len, _State) ->
+ case Msg of
+ {release, _, _, _} -> 5;
+ {release_reservation, _, _, _} -> 5;
+ _ -> 0
+ end.
+
+handle_call({open, Pid, Requested, EldestUnusedSince}, From,
+ State = #fhc_state { open_count = Count,
+ open_pending = Pending,
+ elders = Elders,
+ clients = Clients })
+ when EldestUnusedSince =/= undefined ->
+ true = ets:insert(Elders, {Pid, EldestUnusedSince}),
+ Item = #pending { kind = open,
+ pid = Pid,
+ requested = Requested,
+ from = From },
+ ok = track_client(Pid, Clients),
+ case needs_reduce(State #fhc_state { open_count = Count + Requested }) of
+ true -> case ets:lookup(Clients, Pid) of
+ [#cstate { opened = 0 }] ->
+ true = ets:update_element(
+ Clients, Pid, {#cstate.blocked, true}),
+ {noreply,
+ reduce(State #fhc_state {
+ open_pending = pending_in(Item, Pending) })};
+ [#cstate { opened = Opened }] ->
+ true = ets:update_element(
+ Clients, Pid,
+ {#cstate.pending_closes, Opened}),
+ {reply, close, State}
+ end;
+ false -> {noreply, run_pending_item(Item, State)}
+ end;
+
+handle_call({obtain, N, Type, Pid}, From,
+ State = #fhc_state { clients = Clients }) ->
+ Count = obtain_state(Type, count, State),
+ Pending = obtain_state(Type, pending, State),
+ ok = track_client(Pid, Clients),
+ Item = #pending { kind = {obtain, Type}, pid = Pid,
+ requested = N, from = From },
+ Enqueue = fun () ->
+ true = ets:update_element(Clients, Pid,
+ {#cstate.blocked, true}),
+ set_obtain_state(Type, pending,
+ pending_in(Item, Pending), State)
+ end,
+ {noreply,
+ case obtain_limit_reached(Type, State) of
+ true -> Enqueue();
+ false -> case needs_reduce(
+ set_obtain_state(Type, count, Count + 1, State)) of
+ true -> reduce(Enqueue());
+ false -> adjust_alarm(
+ State, run_pending_item(Item, State))
+ end
+ end};
+
+handle_call({set_limit, Limit}, _From, State) ->
+ {reply, ok, adjust_alarm(
+ State, maybe_reduce(
+ process_pending(
+ State #fhc_state {
+ limit = Limit,
+ obtain_limit = obtain_limit(Limit) })))};
+
+handle_call(get_limit, _From, State = #fhc_state { limit = Limit }) ->
+ {reply, Limit, State};
+
+handle_call({info, Items}, _From, State) ->
+ {reply, infos(Items, State), State}.
+
+handle_cast({register_callback, Pid, MFA},
+ State = #fhc_state { clients = Clients }) ->
+ ok = track_client(Pid, Clients),
+ true = ets:update_element(Clients, Pid, {#cstate.callback, MFA}),
+ {noreply, State};
+
+handle_cast({update, Pid, EldestUnusedSince},
+ State = #fhc_state { elders = Elders })
+ when EldestUnusedSince =/= undefined ->
+ true = ets:insert(Elders, {Pid, EldestUnusedSince}),
+ %% don't call maybe_reduce from here otherwise we can create a
+ %% storm of messages
+ {noreply, State};
+
+handle_cast({release, N, Type, Pid}, State) ->
+ State1 = process_pending(update_counts({obtain, Type}, Pid, -N, State)),
+ {noreply, adjust_alarm(State, State1)};
+
+handle_cast({close, Pid, EldestUnusedSince},
+ State = #fhc_state { elders = Elders, clients = Clients }) ->
+ true = case EldestUnusedSince of
+ undefined -> ets:delete(Elders, Pid);
+ _ -> ets:insert(Elders, {Pid, EldestUnusedSince})
+ end,
+ ets:update_counter(Clients, Pid, {#cstate.pending_closes, -1, 0, 0}),
+ {noreply, adjust_alarm(State, process_pending(
+ update_counts(open, Pid, -1, State)))};
+
+handle_cast({transfer, N, FromPid, ToPid}, State) ->
+ ok = track_client(ToPid, State#fhc_state.clients),
+ {noreply, process_pending(
+ update_counts({obtain, socket}, ToPid, +N,
+ update_counts({obtain, socket}, FromPid, -N,
+ State)))};
+
+handle_cast(clear_read_cache, State) ->
+ _ = clear_process_read_cache(),
+ {noreply, State};
+
+handle_cast({release_reservation, Type, Pid}, State) ->
+ State1 = process_pending(update_counts({reserve, Type}, Pid, 0, State)),
+ {noreply, adjust_alarm(State, State1)};
+
+handle_cast({set_reservation, N, Type, Pid},
+ State = #fhc_state { clients = Clients }) ->
+ ok = track_client(Pid, Clients),
+ NewState = process_pending(update_counts({reserve, Type}, Pid, N, State)),
+ {noreply, case needs_reduce(NewState) of
+ true -> reduce(NewState);
+ false -> adjust_alarm(State, NewState)
+ end}.
+
+handle_info(check_counts, State) ->
+ {noreply, maybe_reduce(State #fhc_state { timer_ref = undefined })};
+
+handle_info({'DOWN', _MRef, process, Pid, _Reason},
+ State = #fhc_state { elders = Elders,
+ open_count = OpenCount,
+ open_pending = OpenPending,
+ obtain_count_file = ObtainCountF,
+ obtain_count_socket = ObtainCountS,
+ obtain_pending_file = ObtainPendingF,
+ obtain_pending_socket = ObtainPendingS,
+ reserve_count_file = ReserveCountF,
+ reserve_count_socket = ReserveCountS,
+ clients = Clients }) ->
+ [#cstate { opened = Opened,
+ obtained_file = ObtainedFile,
+ obtained_socket = ObtainedSocket,
+ reserved_file = ReservedFile,
+ reserved_socket = ReservedSocket }] =
+ ets:lookup(Clients, Pid),
+ true = ets:delete(Clients, Pid),
+ true = ets:delete(Elders, Pid),
+ Fun = fun (#pending { pid = Pid1 }) -> Pid1 =/= Pid end,
+ State1 = process_pending(
+ State #fhc_state {
+ open_count = OpenCount - Opened,
+ open_pending = filter_pending(Fun, OpenPending),
+ obtain_count_file = ObtainCountF - ObtainedFile,
+ obtain_count_socket = ObtainCountS - ObtainedSocket,
+ obtain_pending_file = filter_pending(Fun, ObtainPendingF),
+ obtain_pending_socket = filter_pending(Fun, ObtainPendingS),
+ reserve_count_file = ReserveCountF - ReservedFile,
+ reserve_count_socket = ReserveCountS - ReservedSocket}),
+ {noreply, adjust_alarm(State, State1)}.
+
+terminate(_Reason, State = #fhc_state { clients = Clients,
+ elders = Elders }) ->
+ ets:delete(Clients),
+ ets:delete(Elders),
+ State.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+%%----------------------------------------------------------------------------
+%% pending queue abstraction helpers
+%%----------------------------------------------------------------------------
+
+queue_fold(Fun, Init, Q) ->
+ case queue:out(Q) of
+ {empty, _Q} -> Init;
+ {{value, V}, Q1} -> queue_fold(Fun, Fun(V, Init), Q1)
+ end.
+
+filter_pending(Fun, {Count, Queue}) ->
+ {Delta, Queue1} =
+ queue_fold(
+ fun (Item = #pending { requested = Requested }, {DeltaN, QueueN}) ->
+ case Fun(Item) of
+ true -> {DeltaN, queue:in(Item, QueueN)};
+ false -> {DeltaN - Requested, QueueN}
+ end
+ end, {0, queue:new()}, Queue),
+ {Count + Delta, Queue1}.
+
+pending_new() ->
+ {0, queue:new()}.
+
+pending_in(Item = #pending { requested = Requested }, {Count, Queue}) ->
+ {Count + Requested, queue:in(Item, Queue)}.
+
+pending_out({0, _Queue} = Pending) ->
+ {empty, Pending};
+pending_out({N, Queue}) ->
+ {{value, #pending { requested = Requested }} = Result, Queue1} =
+ queue:out(Queue),
+ {Result, {N - Requested, Queue1}}.
+
+pending_count({Count, _Queue}) ->
+ Count.
+
+%%----------------------------------------------------------------------------
+%% server helpers
+%%----------------------------------------------------------------------------
+
+obtain_limit(infinity) -> infinity;
+obtain_limit(Limit) -> case ?OBTAIN_LIMIT(Limit) of
+ OLimit when OLimit < 0 -> 0;
+ OLimit -> OLimit
+ end.
+
+obtain_limit_reached(socket, State) -> obtain_limit_reached(State);
+obtain_limit_reached(file, State) -> needs_reduce(State).
+
+obtain_limit_reached(#fhc_state{obtain_limit = Limit,
+ obtain_count_socket = Count,
+ reserve_count_socket = RCount}) ->
+ Limit =/= infinity andalso (RCount + Count) >= Limit.
+
+obtain_state(file, count, #fhc_state{obtain_count_file = N}) -> N;
+obtain_state(socket, count, #fhc_state{obtain_count_socket = N}) -> N;
+obtain_state(file, pending, #fhc_state{obtain_pending_file = N}) -> N;
+obtain_state(socket, pending, #fhc_state{obtain_pending_socket = N}) -> N.
+
+set_obtain_state(file, count, N, S) -> S#fhc_state{obtain_count_file = N};
+set_obtain_state(socket, count, N, S) -> S#fhc_state{obtain_count_socket = N};
+set_obtain_state(file, pending, N, S) -> S#fhc_state{obtain_pending_file = N};
+set_obtain_state(socket, pending, N, S) -> S#fhc_state{obtain_pending_socket = N}.
+
+adjust_alarm(OldState = #fhc_state { alarm_set = AlarmSet,
+ alarm_clear = AlarmClear }, NewState) ->
+ case {obtain_limit_reached(OldState), obtain_limit_reached(NewState)} of
+ {false, true} -> AlarmSet({file_descriptor_limit, []});
+ {true, false} -> AlarmClear(file_descriptor_limit);
+ _ -> ok
+ end,
+ NewState.
+
+process_pending(State = #fhc_state { limit = infinity }) ->
+ State;
+process_pending(State) ->
+ process_open(process_obtain(socket, process_obtain(file, State))).
+
+process_open(State = #fhc_state { limit = Limit,
+ open_pending = Pending}) ->
+ {Pending1, State1} = process_pending(Pending, Limit - used(State), State),
+ State1 #fhc_state { open_pending = Pending1 }.
+
+process_obtain(socket, State = #fhc_state { limit = Limit,
+ obtain_limit = ObtainLimit,
+ open_count = OpenCount,
+ obtain_count_socket = ObtainCount,
+ obtain_pending_socket = Pending,
+ obtain_count_file = ObtainCountF,
+ reserve_count_file = ReserveCountF,
+ reserve_count_socket = ReserveCount}) ->
+ Quota = min(ObtainLimit - ObtainCount,
+ Limit - (OpenCount + ObtainCount + ObtainCountF + ReserveCount + ReserveCountF)),
+ {Pending1, State1} = process_pending(Pending, Quota, State),
+ State1#fhc_state{obtain_pending_socket = Pending1};
+process_obtain(file, State = #fhc_state { limit = Limit,
+ open_count = OpenCount,
+ obtain_count_socket = ObtainCountS,
+ obtain_count_file = ObtainCountF,
+ obtain_pending_file = Pending,
+ reserve_count_file = ReserveCountF,
+ reserve_count_socket = ReserveCountS}) ->
+ Quota = Limit - (OpenCount + ObtainCountS + ObtainCountF + ReserveCountF + ReserveCountS),
+ {Pending1, State1} = process_pending(Pending, Quota, State),
+ State1#fhc_state{obtain_pending_file = Pending1}.
+
+process_pending(Pending, Quota, State) when Quota =< 0 ->
+ {Pending, State};
+process_pending(Pending, Quota, State) ->
+ case pending_out(Pending) of
+ {empty, _Pending} ->
+ {Pending, State};
+ {{value, #pending { requested = Requested }}, _Pending1}
+ when Requested > Quota ->
+ {Pending, State};
+ {{value, #pending { requested = Requested } = Item}, Pending1} ->
+ process_pending(Pending1, Quota - Requested,
+ run_pending_item(Item, State))
+ end.
+
+run_pending_item(#pending { kind = Kind,
+ pid = Pid,
+ requested = Requested,
+ from = From },
+ State = #fhc_state { clients = Clients }) ->
+ gen_server2:reply(From, ok),
+ true = ets:update_element(Clients, Pid, {#cstate.blocked, false}),
+ update_counts(Kind, Pid, Requested, State).
+
+update_counts(open, Pid, Delta,
+ State = #fhc_state { open_count = OpenCount,
+ clients = Clients }) ->
+ ets:update_counter(Clients, Pid, {#cstate.opened, Delta}),
+ State #fhc_state { open_count = OpenCount + Delta};
+update_counts({obtain, file}, Pid, Delta,
+ State = #fhc_state {obtain_count_file = ObtainCountF,
+ clients = Clients }) ->
+ ets:update_counter(Clients, Pid, {#cstate.obtained_file, Delta}),
+ State #fhc_state { obtain_count_file = ObtainCountF + Delta};
+update_counts({obtain, socket}, Pid, Delta,
+ State = #fhc_state {obtain_count_socket = ObtainCountS,
+ clients = Clients }) ->
+ ets:update_counter(Clients, Pid, {#cstate.obtained_socket, Delta}),
+ State #fhc_state { obtain_count_socket = ObtainCountS + Delta};
+update_counts({reserve, file}, Pid, NewReservation,
+ State = #fhc_state {reserve_count_file = ReserveCountF,
+ clients = Clients }) ->
+ [#cstate{reserved_file = R}] = ets:lookup(Clients, Pid),
+ Delta = NewReservation - R,
+ ets:update_counter(Clients, Pid, {#cstate.reserved_file, Delta}),
+ State #fhc_state { reserve_count_file = ReserveCountF + Delta};
+update_counts({reserve, socket}, Pid, NewReservation,
+ State = #fhc_state {reserve_count_socket = ReserveCountS,
+ clients = Clients }) ->
+ [#cstate{reserved_file = R}] = ets:lookup(Clients, Pid),
+ Delta = NewReservation - R,
+ ets:update_counter(Clients, Pid, {#cstate.reserved_socket, Delta}),
+ State #fhc_state { reserve_count_socket = ReserveCountS + Delta}.
+
+maybe_reduce(State) ->
+ case needs_reduce(State) of
+ true -> reduce(State);
+ false -> State
+ end.
+
+needs_reduce(#fhc_state { limit = Limit,
+ open_count = OpenCount,
+ open_pending = {OpenPending, _},
+ obtain_limit = ObtainLimit,
+ obtain_count_socket = ObtainCountS,
+ obtain_count_file = ObtainCountF,
+ obtain_pending_file = {ObtainPendingF, _},
+ obtain_pending_socket = {ObtainPendingS, _},
+ reserve_count_socket = ReserveCountS,
+ reserve_count_file = ReserveCountF}) ->
+ Limit =/= infinity
+ andalso (((OpenCount + ObtainCountS + ObtainCountF + ReserveCountS + ReserveCountF) > Limit)
+ orelse (OpenPending =/= 0)
+ orelse (ObtainPendingF =/= 0)
+ orelse (ObtainCountS < ObtainLimit
+ andalso (ObtainPendingS =/= 0))).
+
+reduce(State = #fhc_state { open_pending = OpenPending,
+ obtain_pending_file = ObtainPendingFile,
+ obtain_pending_socket = ObtainPendingSocket,
+ elders = Elders,
+ clients = Clients,
+ timer_ref = TRef }) ->
+ Now = erlang:monotonic_time(),
+ {CStates, Sum, ClientCount} =
+ ets:foldl(fun ({Pid, Eldest}, {CStatesAcc, SumAcc, CountAcc} = Accs) ->
+ [#cstate { pending_closes = PendingCloses,
+ opened = Opened,
+ blocked = Blocked } = CState] =
+ ets:lookup(Clients, Pid),
+ TimeDiff = erlang:convert_time_unit(
+ Now - Eldest, native, micro_seconds),
+ case Blocked orelse PendingCloses =:= Opened of
+ true -> Accs;
+ false -> {[CState | CStatesAcc],
+ SumAcc + TimeDiff,
+ CountAcc + 1}
+ end
+ end, {[], 0, 0}, Elders),
+ case CStates of
+ [] -> ok;
+ _ -> case (Sum / ClientCount) -
+ (1000 * ?FILE_HANDLES_CHECK_INTERVAL) of
+ AverageAge when AverageAge > 0 ->
+ notify_age(CStates, AverageAge);
+ _ ->
+ notify_age0(Clients, CStates,
+ pending_count(OpenPending) +
+ pending_count(ObtainPendingFile) +
+ pending_count(ObtainPendingSocket))
+ end
+ end,
+ case TRef of
+ undefined -> TRef1 = erlang:send_after(
+ ?FILE_HANDLES_CHECK_INTERVAL, ?SERVER,
+ check_counts),
+ State #fhc_state { timer_ref = TRef1 };
+ _ -> State
+ end.
+
+notify_age(CStates, AverageAge) ->
+ lists:foreach(
+ fun (#cstate { callback = undefined }) -> ok;
+ (#cstate { callback = {M, F, A} }) -> apply(M, F, A ++ [AverageAge])
+ end, CStates).
+
+notify_age0(Clients, CStates, Required) ->
+ case [CState || CState <- CStates, CState#cstate.callback =/= undefined] of
+ [] -> ok;
+ Notifications -> S = rand:uniform(length(Notifications)),
+ {L1, L2} = lists:split(S, Notifications),
+ notify(Clients, Required, L2 ++ L1)
+ end.
+
+notify(_Clients, _Required, []) ->
+ ok;
+notify(_Clients, Required, _Notifications) when Required =< 0 ->
+ ok;
+notify(Clients, Required, [#cstate{ pid = Pid,
+ callback = {M, F, A},
+ opened = Opened } | Notifications]) ->
+ apply(M, F, A ++ [0]),
+ ets:update_element(Clients, Pid, {#cstate.pending_closes, Opened}),
+ notify(Clients, Required - Opened, Notifications).
+
+track_client(Pid, Clients) ->
+ case ets:insert_new(Clients, #cstate { pid = Pid,
+ callback = undefined,
+ opened = 0,
+ obtained_file = 0,
+ obtained_socket = 0,
+ blocked = false,
+ pending_closes = 0,
+ reserved_file = 0,
+ reserved_socket = 0 }) of
+ true -> _MRef = erlang:monitor(process, Pid),
+ ok;
+ false -> ok
+ end.
+
+
+%% To increase the number of file descriptors: on Windows set ERL_MAX_PORTS
+%% environment variable, on Linux set `ulimit -n`.
+ulimit() ->
+ IOStats = case erlang:system_info(check_io) of
+ [Val | _] when is_list(Val) -> Val;
+ Val when is_list(Val) -> Val;
+ _Other -> []
+ end,
+ case proplists:get_value(max_fds, IOStats) of
+ MaxFds when is_integer(MaxFds) andalso MaxFds > 1 ->
+ case os:type() of
+ {win32, _OsName} ->
+ %% On Windows max_fds is twice the number of open files:
+ %% https://github.com/yrashk/erlang/blob/e1282325ed75e52a98d5/erts/emulator/sys/win32/sys.c#L2459-2466
+ MaxFds div 2;
+ _Any ->
+ %% For other operating systems trust Erlang.
+ MaxFds
+ end;
+ _ ->
+ unknown
+ end.
diff --git a/deps/rabbit_common/src/file_handle_cache_stats.erl b/deps/rabbit_common/src/file_handle_cache_stats.erl
new file mode 100644
index 0000000000..e36a4b38dc
--- /dev/null
+++ b/deps/rabbit_common/src/file_handle_cache_stats.erl
@@ -0,0 +1,57 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(file_handle_cache_stats).
+
+%% stats about read / write operations that go through the fhc.
+
+-export([init/0, update/3, update/2, update/1, get/0]).
+
+-define(TABLE, ?MODULE).
+
+-define(COUNT,
+ [io_reopen, mnesia_ram_tx, mnesia_disk_tx,
+ msg_store_read, msg_store_write,
+ queue_index_journal_write, queue_index_write, queue_index_read]).
+-define(COUNT_TIME, [io_sync, io_seek, io_file_handle_open_attempt]).
+-define(COUNT_TIME_BYTES, [io_read, io_write]).
+
+init() ->
+ _ = ets:new(?TABLE, [public, named_table]),
+ [ets:insert(?TABLE, {{Op, Counter}, 0}) || Op <- ?COUNT_TIME_BYTES,
+ Counter <- [count, bytes, time]],
+ [ets:insert(?TABLE, {{Op, Counter}, 0}) || Op <- ?COUNT_TIME,
+ Counter <- [count, time]],
+ [ets:insert(?TABLE, {{Op, Counter}, 0}) || Op <- ?COUNT,
+ Counter <- [count]].
+
+update(Op, Bytes, Thunk) ->
+ {Time, Res} = timer_tc(Thunk),
+ _ = ets:update_counter(?TABLE, {Op, count}, 1),
+ _ = ets:update_counter(?TABLE, {Op, bytes}, Bytes),
+ _ = ets:update_counter(?TABLE, {Op, time}, Time),
+ Res.
+
+update(Op, Thunk) ->
+ {Time, Res} = timer_tc(Thunk),
+ _ = ets:update_counter(?TABLE, {Op, count}, 1),
+ _ = ets:update_counter(?TABLE, {Op, time}, Time),
+ Res.
+
+update(Op) ->
+ ets:update_counter(?TABLE, {Op, count}, 1),
+ ok.
+
+get() ->
+ lists:sort(ets:tab2list(?TABLE)).
+
+timer_tc(Thunk) ->
+ T1 = erlang:monotonic_time(),
+ Res = Thunk(),
+ T2 = erlang:monotonic_time(),
+ Diff = erlang:convert_time_unit(T2 - T1, native, micro_seconds),
+ {Diff, Res}.
diff --git a/deps/rabbit_common/src/gen_server2.erl b/deps/rabbit_common/src/gen_server2.erl
new file mode 100644
index 0000000000..b80e921a89
--- /dev/null
+++ b/deps/rabbit_common/src/gen_server2.erl
@@ -0,0 +1,1419 @@
+%% This file is a copy of gen_server.erl from the R13B-1 Erlang/OTP
+%% distribution, with the following modifications:
+%%
+%% 1) the module name is gen_server2
+%%
+%% 2) more efficient handling of selective receives in callbacks
+%% gen_server2 processes drain their message queue into an internal
+%% buffer before invoking any callback module functions. Messages are
+%% dequeued from the buffer for processing. Thus the effective message
+%% queue of a gen_server2 process is the concatenation of the internal
+%% buffer and the real message queue.
+%% As a result of the draining, any selective receive invoked inside a
+%% callback is less likely to have to scan a large message queue.
+%%
+%% 3) gen_server2:cast is guaranteed to be order-preserving
+%% The original code could reorder messages when communicating with a
+%% process on a remote node that was not currently connected.
+%%
+%% 4) The callback module can optionally implement prioritise_call/4,
+%% prioritise_cast/3 and prioritise_info/3. These functions take
+%% Message, From, Length and State or just Message, Length and State
+%% (where Length is the current number of messages waiting to be
+%% processed) and return a single integer representing the priority
+%% attached to the message, or 'drop' to ignore it (for
+%% prioritise_cast/3 and prioritise_info/3 only). Messages with
+%% higher priorities are processed before requests with lower
+%% priorities. The default priority is 0.
+%%
+%% 5) The callback module can optionally implement
+%% handle_pre_hibernate/1 and handle_post_hibernate/1. These will be
+%% called immediately prior to and post hibernation, respectively. If
+%% handle_pre_hibernate returns {hibernate, NewState} then the process
+%% will hibernate. If the module does not implement
+%% handle_pre_hibernate/1 then the default action is to hibernate.
+%%
+%% 6) init can return a 4th arg, {backoff, InitialTimeout,
+%% MinimumTimeout, DesiredHibernatePeriod} (all in milliseconds,
+%% 'infinity' does not make sense here). Then, on all callbacks which
+%% can return a timeout (including init), timeout can be
+%% 'hibernate'. When this is the case, the current timeout value will
+%% be used (initially, the InitialTimeout supplied from init). After
+%% this timeout has occurred, hibernation will occur as normal. Upon
+%% awaking, a new current timeout value will be calculated.
+%%
+%% The purpose is that the gen_server2 takes care of adjusting the
+%% current timeout value such that the process will increase the
+%% timeout value repeatedly if it is unable to sleep for the
+%% DesiredHibernatePeriod. If it is able to sleep for the
+%% DesiredHibernatePeriod it will decrease the current timeout down to
+%% the MinimumTimeout, so that the process is put to sleep sooner (and
+%% hopefully stays asleep for longer). In short, should a process
+%% using this receive a burst of messages, it should not hibernate
+%% between those messages, but as the messages become less frequent,
+%% the process will not only hibernate, it will do so sooner after
+%% each message.
+%%
+%% When using this backoff mechanism, normal timeout values (i.e. not
+%% 'hibernate') can still be used, and if they are used then the
+%% handle_info(timeout, State) will be called as normal. In this case,
+%% returning 'hibernate' from handle_info(timeout, State) will not
+%% hibernate the process immediately, as it would if backoff wasn't
+%% being used. Instead it'll wait for the current timeout as described
+%% above.
+%%
+%% 7) The callback module can return from any of the handle_*
+%% functions, a {become, Module, State} triple, or a {become, Module,
+%% State, Timeout} quadruple. This allows the gen_server to
+%% dynamically change the callback module. The State is the new state
+%% which will be passed into any of the callback functions in the new
+%% module. Note there is no form also encompassing a reply, thus if
+%% you wish to reply in handle_call/3 and change the callback module,
+%% you need to use gen_server2:reply/2 to issue the reply
+%% manually. The init function can similarly return a 5th argument,
+%% Module, in order to dynamically decide the callback module on init.
+%%
+%% 8) The callback module can optionally implement
+%% format_message_queue/2 which is the equivalent of format_status/2
+%% but where the second argument is specifically the priority_queue
+%% which contains the prioritised message_queue.
+%%
+%% 9) The function with_state/2 can be used to debug a process with
+%% heavyweight state (without needing to copy the entire state out of
+%% process as sys:get_status/1 would). Pass through a function which
+%% can be invoked on the state, get back the result. The state is not
+%% modified.
+%%
+%% 10) an mcall/1 function has been added for performing multiple
+%% call/3 in parallel. Unlike multi_call, which sends the same request
+%% to same-named processes residing on a supplied list of nodes, it
+%% operates on name/request pairs, where name is anything accepted by
+%% call/3, i.e. a pid, global name, local name, or local name on a
+%% particular node.
+%%
+%% 11) Internal buffer length is emitted as a core [RabbitMQ] metric.
+
+%% All modifications are (C) 2009-2020 VMware, Inc. or its affiliates.
+
+%% ``The contents of this file are subject to the Erlang Public License,
+%% Version 1.1, (the "License"); you may not use this file except in
+%% compliance with the License. You should have received a copy of the
+%% Erlang Public License along with this software. If not, it can be
+%% retrieved via the world wide web at https://www.erlang.org/.
+%%
+%% Software distributed under the License is distributed on an "AS IS"
+%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
+%% the License for the specific language governing rights and limitations
+%% under the License.
+%%
+%% The Initial Developer of the Original Code is Ericsson Utvecklings AB.
+%% Portions created by Ericsson are Copyright 1999, Ericsson Utvecklings
+%% AB. All Rights Reserved.''
+%%
+%% $Id$
+%%
+-module(gen_server2).
+
+-ifdef(OTP_RELEASE).
+-if(?OTP_RELEASE >= 22).
+-compile(nowarn_deprecated_function).
+-endif.
+-endif.
+
+%%% ---------------------------------------------------
+%%%
+%%% The idea behind THIS server is that the user module
+%%% provides (different) functions to handle different
+%%% kind of inputs.
+%%% If the Parent process terminates the Module:terminate/2
+%%% function is called.
+%%%
+%%% The user module should export:
+%%%
+%%% init(Args)
+%%% ==> {ok, State}
+%%% {ok, State, Timeout}
+%%% {ok, State, Timeout, Backoff}
+%%% {ok, State, Timeout, Backoff, Module}
+%%% ignore
+%%% {stop, Reason}
+%%%
+%%% handle_call(Msg, {From, Tag}, State)
+%%%
+%%% ==> {reply, Reply, State}
+%%% {reply, Reply, State, Timeout}
+%%% {noreply, State}
+%%% {noreply, State, Timeout}
+%%% {stop, Reason, Reply, State}
+%%% Reason = normal | shutdown | Term terminate(State) is called
+%%%
+%%% handle_cast(Msg, State)
+%%%
+%%% ==> {noreply, State}
+%%% {noreply, State, Timeout}
+%%% {stop, Reason, State}
+%%% Reason = normal | shutdown | Term terminate(State) is called
+%%%
+%%% handle_info(Info, State) Info is e.g. {'EXIT', P, R}, {nodedown, N}, ...
+%%%
+%%% ==> {noreply, State}
+%%% {noreply, State, Timeout}
+%%% {stop, Reason, State}
+%%% Reason = normal | shutdown | Term, terminate(State) is called
+%%%
+%%% terminate(Reason, State) Let the user module clean up
+%%% Reason = normal | shutdown | {shutdown, Term} | Term
+%%% always called when server terminates
+%%%
+%%% ==> ok | Term
+%%%
+%%% handle_pre_hibernate(State)
+%%%
+%%% ==> {hibernate, State}
+%%% {stop, Reason, State}
+%%% Reason = normal | shutdown | Term, terminate(State) is called
+%%%
+%%% handle_post_hibernate(State)
+%%%
+%%% ==> {noreply, State}
+%%% {stop, Reason, State}
+%%% Reason = normal | shutdown | Term, terminate(State) is called
+%%%
+%%% The work flow (of the server) can be described as follows:
+%%%
+%%% User module Generic
+%%% ----------- -------
+%%% start -----> start
+%%% init <----- .
+%%%
+%%% loop
+%%% handle_call <----- .
+%%% -----> reply
+%%%
+%%% handle_cast <----- .
+%%%
+%%% handle_info <----- .
+%%%
+%%% terminate <----- .
+%%%
+%%% -----> reply
+%%%
+%%%
+%%% ---------------------------------------------------
+
+%% API
+-export([start/3, start/4,
+ start_link/3, start_link/4,
+ stop/1, stop/3,
+ call/2, call/3,
+ cast/2, reply/2,
+ abcast/2, abcast/3,
+ multi_call/2, multi_call/3, multi_call/4,
+ mcall/1,
+ with_state/2,
+ enter_loop/3, enter_loop/4, enter_loop/5, enter_loop/6, wake_hib/1]).
+
+%% System exports
+-export([system_continue/3,
+ system_terminate/4,
+ system_code_change/4,
+ format_status/2]).
+
+%% Internal exports
+-export([init_it/6]).
+
+-import(error_logger, [format/2]).
+
+%% State record
+-record(gs2_state, {parent, name, state, mod, time,
+ timeout_state, queue, debug, prioritisers,
+ timer, emit_stats_fun, stop_stats_fun}).
+
+%%%=========================================================================
+%%% Specs. These exist only to shut up dialyzer's warnings
+%%%=========================================================================
+
+-type gs2_state() :: #gs2_state{}.
+
+-spec handle_common_termination(any(), atom(), gs2_state()) -> no_return().
+-spec hibernate(gs2_state()) -> no_return().
+-spec pre_hibernate(gs2_state()) -> no_return().
+-spec system_terminate(_, _, _, gs2_state()) -> no_return().
+
+-type millis() :: non_neg_integer().
+
+-dialyzer({nowarn_function, do_multi_call/4}).
+
+%%%=========================================================================
+%%% API
+%%%=========================================================================
+
+-callback init(Args :: term()) ->
+ {ok, State :: term()} |
+ {ok, State :: term(), timeout() | hibernate} |
+ {ok, State :: term(), timeout() | hibernate,
+ {backoff, millis(), millis(), millis()}} |
+ {ok, State :: term(), timeout() | hibernate,
+ {backoff, millis(), millis(), millis()}, atom()} |
+ ignore |
+ {stop, Reason :: term()}.
+-callback handle_call(Request :: term(), From :: {pid(), Tag :: term()},
+ State :: term()) ->
+ {reply, Reply :: term(), NewState :: term()} |
+ {reply, Reply :: term(), NewState :: term(), timeout() | hibernate} |
+ {noreply, NewState :: term()} |
+ {noreply, NewState :: term(), timeout() | hibernate} |
+ {stop, Reason :: term(),
+ Reply :: term(), NewState :: term()}.
+-callback handle_cast(Request :: term(), State :: term()) ->
+ {noreply, NewState :: term()} |
+ {noreply, NewState :: term(), timeout() | hibernate} |
+ {stop, Reason :: term(), NewState :: term()}.
+-callback handle_info(Info :: term(), State :: term()) ->
+ {noreply, NewState :: term()} |
+ {noreply, NewState :: term(), timeout() | hibernate} |
+ {stop, Reason :: term(), NewState :: term()}.
+-callback terminate(Reason :: (normal | shutdown | {shutdown, term()} | term()),
+ State :: term()) ->
+ ok | term().
+-callback code_change(OldVsn :: (term() | {down, term()}), State :: term(),
+ Extra :: term()) ->
+ {ok, NewState :: term()} | {error, Reason :: term()}.
+
+%% It's not possible to define "optional" -callbacks, so putting specs
+%% for handle_pre_hibernate/1 and handle_post_hibernate/1 will result
+%% in warnings (the same applied for the behaviour_info before).
+
+%%% -----------------------------------------------------------------
+%%% Starts a generic server.
+%%% start(Mod, Args, Options)
+%%% start(Name, Mod, Args, Options)
+%%% start_link(Mod, Args, Options)
+%%% start_link(Name, Mod, Args, Options) where:
+%%% Name ::= {local, atom()} | {global, atom()}
+%%% Mod ::= atom(), callback module implementing the 'real' server
+%%% Args ::= term(), init arguments (to Mod:init/1)
+%%% Options ::= [{timeout, Timeout} | {debug, [Flag]}]
+%%% Flag ::= trace | log | {logfile, File} | statistics | debug
+%%% (debug == log && statistics)
+%%% Returns: {ok, Pid} |
+%%% {error, {already_started, Pid}} |
+%%% {error, Reason}
+%%% -----------------------------------------------------------------
+start(Mod, Args, Options) ->
+ gen:start(?MODULE, nolink, Mod, Args, Options).
+
+start(Name, Mod, Args, Options) ->
+ gen:start(?MODULE, nolink, Name, Mod, Args, Options).
+
+start_link(Mod, Args, Options) ->
+ gen:start(?MODULE, link, Mod, Args, Options).
+
+start_link(Name, Mod, Args, Options) ->
+ gen:start(?MODULE, link, Name, Mod, Args, Options).
+
+%% -----------------------------------------------------------------
+%% Stop a generic server and wait for it to terminate.
+%% If the server is located at another node, that node will
+%% be monitored.
+%% -----------------------------------------------------------------
+stop(Name) ->
+ gen:stop(Name).
+
+stop(Name, Reason, Timeout) ->
+ gen:stop(Name, Reason, Timeout).
+
+%% -----------------------------------------------------------------
+%% Make a call to a generic server.
+%% If the server is located at another node, that node will
+%% be monitored.
+%% If the client is trapping exits and is linked server termination
+%% is handled here (? Shall we do that here (or rely on timeouts) ?).
+%% -----------------------------------------------------------------
+call(Name, Request) ->
+ case catch gen:call(Name, '$gen_call', Request) of
+ {ok,Res} ->
+ Res;
+ {'EXIT',Reason} ->
+ exit({Reason, {?MODULE, call, [Name, Request]}})
+ end.
+
+call(Name, Request, Timeout) ->
+ case catch gen:call(Name, '$gen_call', Request, Timeout) of
+ {ok,Res} ->
+ Res;
+ {'EXIT',Reason} ->
+ exit({Reason, {?MODULE, call, [Name, Request, Timeout]}})
+ end.
+
+%% -----------------------------------------------------------------
+%% Make a cast to a generic server.
+%% -----------------------------------------------------------------
+cast({global,Name}, Request) ->
+ catch global:send(Name, {'$gen_cast', Request}),
+ ok;
+cast({Name,Node}=Dest, Request) when is_atom(Name), is_atom(Node) ->
+ catch (Dest ! {'$gen_cast', Request}),
+ ok;
+cast(Dest, Request) when is_atom(Dest); is_pid(Dest) ->
+ catch (Dest ! {'$gen_cast', Request}),
+ ok.
+
+%% -----------------------------------------------------------------
+%% Send a reply to the client.
+%% -----------------------------------------------------------------
+reply({To, Tag}, Reply) ->
+ catch To ! {Tag, Reply}.
+
+%% -----------------------------------------------------------------
+%% Asynchronous broadcast, returns nothing, it's just send'n pray
+%% -----------------------------------------------------------------
+abcast(Name, Request) when is_atom(Name) ->
+ do_abcast([node() | nodes()], Name, {'$gen_cast', Request}).
+
+abcast(Nodes, Name, Request) when is_list(Nodes), is_atom(Name) ->
+ do_abcast(Nodes, Name, {'$gen_cast', Request}).
+
+do_abcast([Node|Nodes], Name, Msg) when is_atom(Node) ->
+ catch ({Name, Node} ! Msg),
+ do_abcast(Nodes, Name, Msg);
+do_abcast([], _,_) -> abcast.
+
+%%% -----------------------------------------------------------------
+%%% Make a call to servers at several nodes.
+%%% Returns: {[Replies],[BadNodes]}
+%%% A Timeout can be given
+%%%
+%%% A middleman process is used in case late answers arrives after
+%%% the timeout. If they would be allowed to glog the callers message
+%%% queue, it would probably become confused. Late answers will
+%%% now arrive to the terminated middleman and so be discarded.
+%%% -----------------------------------------------------------------
+multi_call(Name, Req)
+ when is_atom(Name) ->
+ do_multi_call([node() | nodes()], Name, Req, infinity).
+
+multi_call(Nodes, Name, Req)
+ when is_list(Nodes), is_atom(Name) ->
+ do_multi_call(Nodes, Name, Req, infinity).
+
+multi_call(Nodes, Name, Req, infinity) ->
+ do_multi_call(Nodes, Name, Req, infinity);
+multi_call(Nodes, Name, Req, Timeout)
+ when is_list(Nodes), is_atom(Name), is_integer(Timeout), Timeout >= 0 ->
+ do_multi_call(Nodes, Name, Req, Timeout).
+
+%%% -----------------------------------------------------------------
+%%% Make multiple calls to multiple servers, given pairs of servers
+%%% and messages.
+%%% Returns: {[{Dest, Reply}], [{Dest, Error}]}
+%%%
+%%% Dest can be pid() | RegName :: atom() |
+%%% {Name :: atom(), Node :: atom()} | {global, Name :: atom()}
+%%%
+%%% A middleman process is used to avoid clogging up the callers
+%%% message queue.
+%%% -----------------------------------------------------------------
+mcall(CallSpecs) ->
+ Tag = make_ref(),
+ {_, MRef} = spawn_monitor(
+ fun() ->
+ Refs = lists:foldl(
+ fun ({Dest, _Request}=S, Dict) ->
+ dict:store(do_mcall(S), Dest, Dict)
+ end, dict:new(), CallSpecs),
+ collect_replies(Tag, Refs, [], [])
+ end),
+ receive
+ {'DOWN', MRef, _, _, {Tag, Result}} -> Result;
+ {'DOWN', MRef, _, _, Reason} -> exit(Reason)
+ end.
+
+do_mcall({{global,Name}=Dest, Request}) ->
+ %% whereis_name is simply an ets lookup, and is precisely what
+ %% global:send/2 does, yet we need a Ref to put in the call to the
+ %% server, so invoking whereis_name makes a lot more sense here.
+ case global:whereis_name(Name) of
+ Pid when is_pid(Pid) ->
+ MRef = erlang:monitor(process, Pid),
+ catch msend(Pid, MRef, Request),
+ MRef;
+ undefined ->
+ Ref = make_ref(),
+ self() ! {'DOWN', Ref, process, Dest, noproc},
+ Ref
+ end;
+do_mcall({{Name,Node}=Dest, Request}) when is_atom(Name), is_atom(Node) ->
+ {_Node, MRef} = start_monitor(Node, Name), %% NB: we don't handle R6
+ catch msend(Dest, MRef, Request),
+ MRef;
+do_mcall({Dest, Request}) when is_atom(Dest); is_pid(Dest) ->
+ MRef = erlang:monitor(process, Dest),
+ catch msend(Dest, MRef, Request),
+ MRef.
+
+msend(Dest, MRef, Request) ->
+ erlang:send(Dest, {'$gen_call', {self(), MRef}, Request}, [noconnect]).
+
+collect_replies(Tag, Refs, Replies, Errors) ->
+ case dict:size(Refs) of
+ 0 -> exit({Tag, {Replies, Errors}});
+ _ -> receive
+ {MRef, Reply} ->
+ {Refs1, Replies1} = handle_call_result(MRef, Reply,
+ Refs, Replies),
+ collect_replies(Tag, Refs1, Replies1, Errors);
+ {'DOWN', MRef, _, _, Reason} ->
+ Reason1 = case Reason of
+ noconnection -> nodedown;
+ _ -> Reason
+ end,
+ {Refs1, Errors1} = handle_call_result(MRef, Reason1,
+ Refs, Errors),
+ collect_replies(Tag, Refs1, Replies, Errors1)
+ end
+ end.
+
+handle_call_result(MRef, Result, Refs, AccList) ->
+ %% we avoid the mailbox scanning cost of a call to erlang:demonitor/{1,2}
+ %% here, so we must cope with MRefs that we've already seen and erased
+ case dict:find(MRef, Refs) of
+ {ok, Pid} -> {dict:erase(MRef, Refs), [{Pid, Result}|AccList]};
+ _ -> {Refs, AccList}
+ end.
+
+%% -----------------------------------------------------------------
+%% Apply a function to a generic server's state.
+%% -----------------------------------------------------------------
+with_state(Name, Fun) ->
+ case catch gen:call(Name, '$with_state', Fun, infinity) of
+ {ok,Res} ->
+ Res;
+ {'EXIT',Reason} ->
+ exit({Reason, {?MODULE, with_state, [Name, Fun]}})
+ end.
+
+%%-----------------------------------------------------------------
+%% enter_loop(Mod, Options, State, <ServerName>, <TimeOut>, <Backoff>) ->_
+%%
+%% Description: Makes an existing process into a gen_server.
+%% The calling process will enter the gen_server receive
+%% loop and become a gen_server process.
+%% The process *must* have been started using one of the
+%% start functions in proc_lib, see proc_lib(3).
+%% The user is responsible for any initialization of the
+%% process, including registering a name for it.
+%%-----------------------------------------------------------------
+enter_loop(Mod, Options, State) ->
+ enter_loop(Mod, Options, State, self(), infinity, undefined).
+
+enter_loop(Mod, Options, State, Backoff = {backoff, _, _ , _}) ->
+ enter_loop(Mod, Options, State, self(), infinity, Backoff);
+
+enter_loop(Mod, Options, State, ServerName = {_, _}) ->
+ enter_loop(Mod, Options, State, ServerName, infinity, undefined);
+
+enter_loop(Mod, Options, State, Timeout) ->
+ enter_loop(Mod, Options, State, self(), Timeout, undefined).
+
+enter_loop(Mod, Options, State, ServerName, Backoff = {backoff, _, _, _}) ->
+ enter_loop(Mod, Options, State, ServerName, infinity, Backoff);
+
+enter_loop(Mod, Options, State, ServerName, Timeout) ->
+ enter_loop(Mod, Options, State, ServerName, Timeout, undefined).
+
+enter_loop(Mod, Options, State, ServerName, Timeout, Backoff) ->
+ Name = get_proc_name(ServerName),
+ Parent = get_parent(),
+ Debug = debug_options(Name, Options),
+ Queue = priority_queue:new(),
+ Backoff1 = extend_backoff(Backoff),
+ {EmitStatsFun, StopStatsFun} = stats_funs(),
+ loop(init_stats(find_prioritisers(
+ #gs2_state { parent = Parent, name = Name, state = State,
+ mod = Mod, time = Timeout, timeout_state = Backoff1,
+ queue = Queue, debug = Debug,
+ emit_stats_fun = EmitStatsFun,
+ stop_stats_fun = StopStatsFun }))).
+
+%%%========================================================================
+%%% Gen-callback functions
+%%%========================================================================
+
+%%% ---------------------------------------------------
+%%% Initiate the new process.
+%%% Register the name using the Rfunc function
+%%% Calls the Mod:init/Args function.
+%%% Finally an acknowledge is sent to Parent and the main
+%%% loop is entered.
+%%% ---------------------------------------------------
+init_it(Starter, self, Name, Mod, Args, Options) ->
+ init_it(Starter, self(), Name, Mod, Args, Options);
+init_it(Starter, Parent, Name0, Mod, Args, Options) ->
+ Name = name(Name0),
+ Debug = debug_options(Name, Options),
+ Queue = priority_queue:new(),
+ {EmitStatsFun, StopStatsFun} = stats_funs(),
+ GS2State = find_prioritisers(
+ #gs2_state { parent = Parent,
+ name = Name,
+ mod = Mod,
+ queue = Queue,
+ debug = Debug,
+ emit_stats_fun = EmitStatsFun,
+ stop_stats_fun = StopStatsFun }),
+ case catch Mod:init(Args) of
+ {ok, State} ->
+ proc_lib:init_ack(Starter, {ok, self()}),
+ loop(init_stats(GS2State#gs2_state { state = State,
+ time = infinity,
+ timeout_state = undefined }));
+ {ok, State, Timeout} ->
+ proc_lib:init_ack(Starter, {ok, self()}),
+ loop(init_stats(
+ GS2State#gs2_state { state = State,
+ time = Timeout,
+ timeout_state = undefined }));
+ {ok, State, Timeout, Backoff = {backoff, _, _, _}} ->
+ Backoff1 = extend_backoff(Backoff),
+ proc_lib:init_ack(Starter, {ok, self()}),
+ loop(init_stats(GS2State#gs2_state { state = State,
+ time = Timeout,
+ timeout_state = Backoff1 }));
+ {ok, State, Timeout, Backoff = {backoff, _, _, _}, Mod1} ->
+ Backoff1 = extend_backoff(Backoff),
+ proc_lib:init_ack(Starter, {ok, self()}),
+ loop(init_stats(find_prioritisers(
+ GS2State#gs2_state { mod = Mod1,
+ state = State,
+ time = Timeout,
+ timeout_state = Backoff1 })));
+ {stop, Reason} ->
+ %% For consistency, we must make sure that the
+ %% registered name (if any) is unregistered before
+ %% the parent process is notified about the failure.
+ %% (Otherwise, the parent process could get
+ %% an 'already_started' error if it immediately
+ %% tried starting the process again.)
+ unregister_name(Name0),
+ proc_lib:init_ack(Starter, {error, Reason}),
+ exit(Reason);
+ ignore ->
+ unregister_name(Name0),
+ proc_lib:init_ack(Starter, ignore),
+ exit(normal);
+ {'EXIT', Reason} ->
+ unregister_name(Name0),
+ proc_lib:init_ack(Starter, {error, Reason}),
+ exit(Reason);
+ Else ->
+ Error = {bad_return_value, Else},
+ proc_lib:init_ack(Starter, {error, Error}),
+ exit(Error)
+ end.
+
+name({local,Name}) -> Name;
+name({global,Name}) -> Name;
+%% name(Pid) when is_pid(Pid) -> Pid;
+%% when R12 goes away, drop the line beneath and uncomment the line above
+name(Name) -> Name.
+
+unregister_name({local,Name}) ->
+ _ = (catch unregister(Name));
+unregister_name({global,Name}) ->
+ _ = global:unregister_name(Name);
+unregister_name(Pid) when is_pid(Pid) ->
+ Pid;
+%% Under R12 let's just ignore it, as we have a single term as Name.
+%% On R13 it will never get here, as we get tuple with 'local/global' atom.
+unregister_name(_Name) -> ok.
+
+extend_backoff(undefined) ->
+ undefined;
+extend_backoff({backoff, InitialTimeout, MinimumTimeout, DesiredHibPeriod}) ->
+ {backoff, InitialTimeout, MinimumTimeout, DesiredHibPeriod,
+ rand:seed(exsplus)}.
+
+%%%========================================================================
+%%% Internal functions
+%%%========================================================================
+%%% ---------------------------------------------------
+%%% The MAIN loop.
+%%% ---------------------------------------------------
+loop(GS2State = #gs2_state { time = hibernate,
+ timeout_state = undefined,
+ queue = Queue }) ->
+ case priority_queue:is_empty(Queue) of
+ true ->
+ pre_hibernate(GS2State);
+ false ->
+ process_next_msg(GS2State)
+ end;
+
+loop(GS2State) ->
+ process_next_msg(drain(GS2State)).
+
+drain(GS2State) ->
+ receive
+ Input -> drain(in(Input, GS2State))
+ after 0 -> GS2State
+ end.
+
+process_next_msg(GS2State0 = #gs2_state { time = Time,
+ timeout_state = TimeoutState,
+ queue = Queue }) ->
+ case priority_queue:out(Queue) of
+ {{value, Msg}, Queue1} ->
+ GS2State = ensure_stats_timer(GS2State0),
+ process_msg(Msg, GS2State#gs2_state { queue = Queue1 });
+ {empty, Queue1} ->
+ {Time1, HibOnTimeout, GS2State}
+ = case {Time, TimeoutState} of
+ {hibernate, {backoff, Current, _Min, _Desired, _RSt}} ->
+ {Current, true, stop_stats_timer(GS2State0)};
+ {hibernate, _} ->
+ %% wake_hib/7 will set Time to hibernate. If
+ %% we were woken and didn't receive a msg
+ %% then we will get here and need a sensible
+ %% value for Time1, otherwise we crash.
+ %% R13B1 always waits infinitely when waking
+ %% from hibernation, so that's what we do
+ %% here too.
+ {infinity, false, GS2State0};
+ _ -> {Time, false, GS2State0}
+ end,
+ receive
+ Input ->
+ %% Time could be 'hibernate' here, so *don't* call loop
+ process_next_msg(
+ drain(in(Input, GS2State #gs2_state { queue = Queue1 })))
+ after Time1 ->
+ case HibOnTimeout of
+ true ->
+ pre_hibernate(
+ GS2State #gs2_state { queue = Queue1 });
+ false ->
+ process_msg(timeout,
+ GS2State #gs2_state { queue = Queue1 })
+ end
+ end
+ end.
+
+wake_hib(GS2State = #gs2_state { timeout_state = TS }) ->
+ TimeoutState1 = case TS of
+ undefined ->
+ undefined;
+ {SleptAt, TimeoutState} ->
+ adjust_timeout_state(SleptAt,
+ erlang:monotonic_time(),
+ TimeoutState)
+ end,
+ post_hibernate(
+ drain(GS2State #gs2_state { timeout_state = TimeoutState1 })).
+
+hibernate(GS2State = #gs2_state { timeout_state = TimeoutState }) ->
+ TS = case TimeoutState of
+ undefined -> undefined;
+ {backoff, _, _, _, _} -> {erlang:monotonic_time(),
+ TimeoutState}
+ end,
+ proc_lib:hibernate(?MODULE, wake_hib,
+ [GS2State #gs2_state { timeout_state = TS }]).
+
+pre_hibernate(GS2State0 = #gs2_state { state = State,
+ mod = Mod,
+ emit_stats_fun = EmitStatsFun }) ->
+ GS2State = EmitStatsFun(stop_stats_timer(GS2State0)),
+ case erlang:function_exported(Mod, handle_pre_hibernate, 1) of
+ true ->
+ case catch Mod:handle_pre_hibernate(State) of
+ {hibernate, NState} ->
+ hibernate(GS2State #gs2_state { state = NState } );
+ Reply ->
+ handle_common_termination(Reply, pre_hibernate, GS2State)
+ end;
+ false ->
+ hibernate(GS2State)
+ end.
+
+post_hibernate(GS2State0 = #gs2_state { state = State,
+ mod = Mod }) ->
+ GS2State = ensure_stats_timer(GS2State0),
+ case erlang:function_exported(Mod, handle_post_hibernate, 1) of
+ true ->
+ case catch Mod:handle_post_hibernate(State) of
+ {noreply, NState} ->
+ process_next_msg(GS2State #gs2_state { state = NState,
+ time = infinity });
+ {noreply, NState, Time} ->
+ process_next_msg(GS2State #gs2_state { state = NState,
+ time = Time });
+ Reply ->
+ handle_common_termination(Reply, post_hibernate, GS2State)
+ end;
+ false ->
+ %% use hibernate here, not infinity. This matches
+ %% R13B. The key is that we should be able to get through
+ %% to process_msg calling sys:handle_system_msg with Time
+ %% still set to hibernate, iff that msg is the very msg
+ %% that woke us up (or the first msg we receive after
+ %% waking up).
+ process_next_msg(GS2State #gs2_state { time = hibernate })
+ end.
+
+adjust_timeout_state(SleptAt, AwokeAt, {backoff, CurrentTO, MinimumTO,
+ DesiredHibPeriod, RandomState}) ->
+ NapLengthMicros = erlang:convert_time_unit(AwokeAt - SleptAt,
+ native, micro_seconds),
+ CurrentMicros = CurrentTO * 1000,
+ MinimumMicros = MinimumTO * 1000,
+ DesiredHibMicros = DesiredHibPeriod * 1000,
+ GapBetweenMessagesMicros = NapLengthMicros + CurrentMicros,
+ Base =
+ %% If enough time has passed between the last two messages then we
+ %% should consider sleeping sooner. Otherwise stay awake longer.
+ case GapBetweenMessagesMicros > (MinimumMicros + DesiredHibMicros) of
+ true -> lists:max([MinimumTO, CurrentTO div 2]);
+ false -> CurrentTO
+ end,
+ {Extra, RandomState1} = rand:uniform_s(Base, RandomState),
+ CurrentTO1 = Base + Extra,
+ {backoff, CurrentTO1, MinimumTO, DesiredHibPeriod, RandomState1}.
+
+in({'$gen_cast', Msg} = Input,
+ GS2State = #gs2_state { prioritisers = {_, F, _} }) ->
+ in(Input, F(Msg, GS2State), GS2State);
+in({'$gen_call', From, Msg} = Input,
+ GS2State = #gs2_state { prioritisers = {F, _, _} }) ->
+ in(Input, F(Msg, From, GS2State), GS2State);
+in({'$with_state', _From, _Fun} = Input, GS2State) ->
+ in(Input, 0, GS2State);
+in({'EXIT', Parent, _R} = Input, GS2State = #gs2_state { parent = Parent }) ->
+ in(Input, infinity, GS2State);
+in({system, _From, _Req} = Input, GS2State) ->
+ in(Input, infinity, GS2State);
+in(emit_gen_server2_stats, GS2State = #gs2_state{ emit_stats_fun = EmitStatsFun}) ->
+ next_stats_timer(EmitStatsFun(GS2State));
+in(Input, GS2State = #gs2_state { prioritisers = {_, _, F} }) ->
+ in(Input, F(Input, GS2State), GS2State).
+
+in(_Input, drop, GS2State) ->
+ GS2State;
+
+in(Input, Priority, GS2State = #gs2_state { queue = Queue }) ->
+ GS2State # gs2_state { queue = priority_queue:in(Input, Priority, Queue) }.
+
+process_msg({system, From, Req},
+ GS2State = #gs2_state { parent = Parent, debug = Debug }) ->
+ case Req of
+ %% This clause will match only in R16B03.
+ %% Since 17.0 replace_state is not a system message.
+ {replace_state, StateFun} ->
+ GS2State1 = StateFun(GS2State),
+ _ = gen:reply(From, GS2State1),
+ system_continue(Parent, Debug, GS2State1);
+ _ ->
+ %% gen_server puts Hib on the end as the 7th arg, but that version
+ %% of the fun seems not to be documented so leaving out for now.
+ sys:handle_system_msg(Req, From, Parent, ?MODULE, Debug, GS2State)
+ end;
+process_msg({'$with_state', From, Fun},
+ GS2State = #gs2_state{state = State}) ->
+ reply(From, catch Fun(State)),
+ loop(GS2State);
+process_msg({'EXIT', Parent, Reason} = Msg,
+ GS2State = #gs2_state { parent = Parent }) ->
+ terminate(Reason, Msg, GS2State);
+process_msg(Msg, GS2State = #gs2_state { debug = [] }) ->
+ handle_msg(Msg, GS2State);
+process_msg(Msg, GS2State = #gs2_state { name = Name, debug = Debug }) ->
+ Debug1 = sys:handle_debug(Debug, fun print_event/3, Name, {in, Msg}),
+ handle_msg(Msg, GS2State #gs2_state { debug = Debug1 }).
+
+%%% ---------------------------------------------------
+%%% Send/recive functions
+%%% ---------------------------------------------------
+
+do_multi_call(Nodes, Name, Req, infinity) ->
+ Tag = make_ref(),
+ Monitors = send_nodes(Nodes, Name, Tag, Req),
+ rec_nodes(Tag, Monitors, Name, undefined);
+do_multi_call(Nodes, Name, Req, Timeout) ->
+ Tag = make_ref(),
+ Caller = self(),
+ Receiver =
+ spawn(
+ fun () ->
+ %% Middleman process. Should be unsensitive to regular
+ %% exit signals. The synchronization is needed in case
+ %% the receiver would exit before the caller started
+ %% the monitor.
+ process_flag(trap_exit, true),
+ Mref = erlang:monitor(process, Caller),
+ receive
+ {Caller,Tag} ->
+ Monitors = send_nodes(Nodes, Name, Tag, Req),
+ TimerId = erlang:start_timer(Timeout, self(), ok),
+ Result = rec_nodes(Tag, Monitors, Name, TimerId),
+ exit({self(),Tag,Result});
+ {'DOWN',Mref,_,_,_} ->
+ %% Caller died before sending us the go-ahead.
+ %% Give up silently.
+ exit(normal)
+ end
+ end),
+ Mref = erlang:monitor(process, Receiver),
+ Receiver ! {self(),Tag},
+ receive
+ {'DOWN',Mref,_,_,{Receiver,Tag,Result}} ->
+ Result;
+ {'DOWN',Mref,_,_,Reason} ->
+ %% The middleman code failed. Or someone did
+ %% exit(_, kill) on the middleman process => Reason==killed
+ exit(Reason)
+ end.
+
+send_nodes(Nodes, Name, Tag, Req) ->
+ send_nodes(Nodes, Name, Tag, Req, []).
+
+send_nodes([Node|Tail], Name, Tag, Req, Monitors)
+ when is_atom(Node) ->
+ Monitor = start_monitor(Node, Name),
+ %% Handle non-existing names in rec_nodes.
+ catch {Name, Node} ! {'$gen_call', {self(), {Tag, Node}}, Req},
+ send_nodes(Tail, Name, Tag, Req, [Monitor | Monitors]);
+send_nodes([_Node|Tail], Name, Tag, Req, Monitors) ->
+ %% Skip non-atom Node
+ send_nodes(Tail, Name, Tag, Req, Monitors);
+send_nodes([], _Name, _Tag, _Req, Monitors) ->
+ Monitors.
+
+%% Against old nodes:
+%% If no reply has been delivered within 2 secs. (per node) check that
+%% the server really exists and wait for ever for the answer.
+%%
+%% Against contemporary nodes:
+%% Wait for reply, server 'DOWN', or timeout from TimerId.
+
+rec_nodes(Tag, Nodes, Name, TimerId) ->
+ rec_nodes(Tag, Nodes, Name, [], [], 2000, TimerId).
+
+rec_nodes(Tag, [{N,R}|Tail], Name, Badnodes, Replies, Time, TimerId ) ->
+ receive
+ {'DOWN', R, _, _, _} ->
+ rec_nodes(Tag, Tail, Name, [N|Badnodes], Replies, Time, TimerId);
+ {{Tag, N}, Reply} -> %% Tag is bound !!!
+ unmonitor(R),
+ rec_nodes(Tag, Tail, Name, Badnodes,
+ [{N,Reply}|Replies], Time, TimerId);
+ {timeout, TimerId, _} ->
+ unmonitor(R),
+ %% Collect all replies that already have arrived
+ rec_nodes_rest(Tag, Tail, Name, [N|Badnodes], Replies)
+ end;
+rec_nodes(Tag, [N|Tail], Name, Badnodes, Replies, Time, TimerId) ->
+ %% R6 node
+ receive
+ {nodedown, N} ->
+ monitor_node(N, false),
+ rec_nodes(Tag, Tail, Name, [N|Badnodes], Replies, 2000, TimerId);
+ {{Tag, N}, Reply} -> %% Tag is bound !!!
+ receive {nodedown, N} -> ok after 0 -> ok end,
+ monitor_node(N, false),
+ rec_nodes(Tag, Tail, Name, Badnodes,
+ [{N,Reply}|Replies], 2000, TimerId);
+ {timeout, TimerId, _} ->
+ receive {nodedown, N} -> ok after 0 -> ok end,
+ monitor_node(N, false),
+ %% Collect all replies that already have arrived
+ rec_nodes_rest(Tag, Tail, Name, [N | Badnodes], Replies)
+ after Time ->
+ case rpc:call(N, erlang, whereis, [Name]) of
+ Pid when is_pid(Pid) -> % It exists try again.
+ rec_nodes(Tag, [N|Tail], Name, Badnodes,
+ Replies, infinity, TimerId);
+ _ -> % badnode
+ receive {nodedown, N} -> ok after 0 -> ok end,
+ monitor_node(N, false),
+ rec_nodes(Tag, Tail, Name, [N|Badnodes],
+ Replies, 2000, TimerId)
+ end
+ end;
+rec_nodes(_, [], _, Badnodes, Replies, _, TimerId) ->
+ case catch erlang:cancel_timer(TimerId) of
+ false -> % It has already sent it's message
+ receive
+ {timeout, TimerId, _} -> ok
+ after 0 ->
+ ok
+ end;
+ _ -> % Timer was cancelled, or TimerId was 'undefined'
+ ok
+ end,
+ {Replies, Badnodes}.
+
+%% Collect all replies that already have arrived
+rec_nodes_rest(Tag, [{N,R}|Tail], Name, Badnodes, Replies) ->
+ receive
+ {'DOWN', R, _, _, _} ->
+ rec_nodes_rest(Tag, Tail, Name, [N|Badnodes], Replies);
+ {{Tag, N}, Reply} -> %% Tag is bound !!!
+ unmonitor(R),
+ rec_nodes_rest(Tag, Tail, Name, Badnodes, [{N,Reply}|Replies])
+ after 0 ->
+ unmonitor(R),
+ rec_nodes_rest(Tag, Tail, Name, [N|Badnodes], Replies)
+ end;
+rec_nodes_rest(Tag, [N|Tail], Name, Badnodes, Replies) ->
+ %% R6 node
+ receive
+ {nodedown, N} ->
+ monitor_node(N, false),
+ rec_nodes_rest(Tag, Tail, Name, [N|Badnodes], Replies);
+ {{Tag, N}, Reply} -> %% Tag is bound !!!
+ receive {nodedown, N} -> ok after 0 -> ok end,
+ monitor_node(N, false),
+ rec_nodes_rest(Tag, Tail, Name, Badnodes, [{N,Reply}|Replies])
+ after 0 ->
+ receive {nodedown, N} -> ok after 0 -> ok end,
+ monitor_node(N, false),
+ rec_nodes_rest(Tag, Tail, Name, [N|Badnodes], Replies)
+ end;
+rec_nodes_rest(_Tag, [], _Name, Badnodes, Replies) ->
+ {Replies, Badnodes}.
+
+
+%%% ---------------------------------------------------
+%%% Monitor functions
+%%% ---------------------------------------------------
+
+start_monitor(Node, Name) when is_atom(Node), is_atom(Name) ->
+ if node() =:= nonode@nohost, Node =/= nonode@nohost ->
+ Ref = make_ref(),
+ self() ! {'DOWN', Ref, process, {Name, Node}, noconnection},
+ {Node, Ref};
+ true ->
+ case catch erlang:monitor(process, {Name, Node}) of
+ {'EXIT', _} ->
+ %% Remote node is R6
+ monitor_node(Node, true),
+ Node;
+ Ref when is_reference(Ref) ->
+ {Node, Ref}
+ end
+ end.
+
+%% Cancels a monitor started with Ref=erlang:monitor(_, _).
+unmonitor(Ref) when is_reference(Ref) ->
+ erlang:demonitor(Ref),
+ receive
+ {'DOWN', Ref, _, _, _} ->
+ true
+ after 0 ->
+ true
+ end.
+
+%%% ---------------------------------------------------
+%%% Message handling functions
+%%% ---------------------------------------------------
+
+dispatch({'$gen_cast', Msg}, Mod, State) ->
+ Mod:handle_cast(Msg, State);
+dispatch(Info, Mod, State) ->
+ Mod:handle_info(Info, State).
+
+common_reply(_Name, From, Reply, _NState, [] = _Debug) ->
+ reply(From, Reply),
+ [];
+common_reply(Name, {To, _Tag} = From, Reply, NState, Debug) ->
+ reply(From, Reply),
+ sys:handle_debug(Debug, fun print_event/3, Name, {out, Reply, To, NState}).
+
+common_noreply(_Name, _NState, [] = _Debug) ->
+ [];
+common_noreply(Name, NState, Debug) ->
+ sys:handle_debug(Debug, fun print_event/3, Name, {noreply, NState}).
+
+common_become(_Name, _Mod, _NState, [] = _Debug) ->
+ [];
+common_become(Name, Mod, NState, Debug) ->
+ sys:handle_debug(Debug, fun print_event/3, Name, {become, Mod, NState}).
+
+handle_msg({'$gen_call', From, Msg}, GS2State = #gs2_state { mod = Mod,
+ state = State,
+ name = Name,
+ debug = Debug }) ->
+ case catch Mod:handle_call(Msg, From, State) of
+ {reply, Reply, NState} ->
+ Debug1 = common_reply(Name, From, Reply, NState, Debug),
+ loop(GS2State #gs2_state { state = NState,
+ time = infinity,
+ debug = Debug1 });
+ {reply, Reply, NState, Time1} ->
+ Debug1 = common_reply(Name, From, Reply, NState, Debug),
+ loop(GS2State #gs2_state { state = NState,
+ time = Time1,
+ debug = Debug1});
+ {stop, Reason, Reply, NState} ->
+ {'EXIT', R} =
+ (catch terminate(Reason, Msg,
+ GS2State #gs2_state { state = NState })),
+ _ = common_reply(Name, From, Reply, NState, Debug),
+ exit(R);
+ Other ->
+ handle_common_reply(Other, Msg, GS2State)
+ end;
+handle_msg(Msg, GS2State = #gs2_state { mod = Mod, state = State }) ->
+ Reply = (catch dispatch(Msg, Mod, State)),
+ handle_common_reply(Reply, Msg, GS2State).
+
+handle_common_reply(Reply, Msg, GS2State = #gs2_state { name = Name,
+ debug = Debug}) ->
+ case Reply of
+ {noreply, NState} ->
+ Debug1 = common_noreply(Name, NState, Debug),
+ loop(GS2State #gs2_state {state = NState,
+ time = infinity,
+ debug = Debug1});
+ {noreply, NState, Time1} ->
+ Debug1 = common_noreply(Name, NState, Debug),
+ loop(GS2State #gs2_state {state = NState,
+ time = Time1,
+ debug = Debug1});
+ {become, Mod, NState} ->
+ Debug1 = common_become(Name, Mod, NState, Debug),
+ loop(find_prioritisers(
+ GS2State #gs2_state { mod = Mod,
+ state = NState,
+ time = infinity,
+ debug = Debug1 }));
+ {become, Mod, NState, Time1} ->
+ Debug1 = common_become(Name, Mod, NState, Debug),
+ loop(find_prioritisers(
+ GS2State #gs2_state { mod = Mod,
+ state = NState,
+ time = Time1,
+ debug = Debug1 }));
+ _ ->
+ handle_common_termination(Reply, Msg, GS2State)
+ end.
+
+handle_common_termination(Reply, Msg, GS2State) ->
+ case Reply of
+ {stop, Reason, NState} ->
+ terminate(Reason, Msg, GS2State #gs2_state { state = NState });
+ {'EXIT', What} ->
+ terminate(What, Msg, GS2State);
+ _ ->
+ terminate({bad_return_value, Reply}, Msg, GS2State)
+ end.
+
+%%-----------------------------------------------------------------
+%% Callback functions for system messages handling.
+%%-----------------------------------------------------------------
+system_continue(Parent, Debug, GS2State) ->
+ loop(GS2State #gs2_state { parent = Parent, debug = Debug }).
+
+system_terminate(Reason, _Parent, Debug, GS2State) ->
+ terminate(Reason, [], GS2State #gs2_state { debug = Debug }).
+
+system_code_change(GS2State = #gs2_state { mod = Mod,
+ state = State },
+ _Module, OldVsn, Extra) ->
+ case catch Mod:code_change(OldVsn, State, Extra) of
+ {ok, NewState} ->
+ NewGS2State = find_prioritisers(
+ GS2State #gs2_state { state = NewState }),
+ {ok, [NewGS2State]};
+ Else ->
+ Else
+ end.
+
+%%-----------------------------------------------------------------
+%% Format debug messages. Print them as the call-back module sees
+%% them, not as the real erlang messages. Use trace for that.
+%%-----------------------------------------------------------------
+print_event(Dev, {in, Msg}, Name) ->
+ case Msg of
+ {'$gen_call', {From, _Tag}, Call} ->
+ io:format(Dev, "*DBG* ~p got call ~p from ~w~n",
+ [Name, Call, From]);
+ {'$gen_cast', Cast} ->
+ io:format(Dev, "*DBG* ~p got cast ~p~n",
+ [Name, Cast]);
+ _ ->
+ io:format(Dev, "*DBG* ~p got ~p~n", [Name, Msg])
+ end;
+print_event(Dev, {out, Msg, To, State}, Name) ->
+ io:format(Dev, "*DBG* ~p sent ~p to ~w, new state ~w~n",
+ [Name, Msg, To, State]);
+print_event(Dev, {noreply, State}, Name) ->
+ io:format(Dev, "*DBG* ~p new state ~w~n", [Name, State]);
+print_event(Dev, Event, Name) ->
+ io:format(Dev, "*DBG* ~p dbg ~p~n", [Name, Event]).
+
+
+%%% ---------------------------------------------------
+%%% Terminate the server.
+%%% ---------------------------------------------------
+
+-spec terminate(_, _, _) -> no_return().
+
+terminate(Reason, Msg, #gs2_state { name = Name,
+ mod = Mod,
+ state = State,
+ debug = Debug,
+ stop_stats_fun = StopStatsFun
+ } = GS2State) ->
+ StopStatsFun(stop_stats_timer(GS2State)),
+ case catch Mod:terminate(Reason, State) of
+ {'EXIT', R} ->
+ error_info(R, Reason, Name, Msg, State, Debug),
+ exit(R);
+ _ ->
+ case Reason of
+ normal ->
+ exit(normal);
+ shutdown ->
+ exit(shutdown);
+ {shutdown,_}=Shutdown ->
+ exit(Shutdown);
+ _ ->
+ error_info(Reason, undefined, Name, Msg, State, Debug),
+ exit(Reason)
+ end
+ end.
+
+error_info(_Reason, _RootCause, application_controller, _Msg, _State, _Debug) ->
+ %% OTP-5811 Don't send an error report if it's the system process
+ %% application_controller which is terminating - let init take care
+ %% of it instead
+ ok;
+error_info(Reason, RootCause, Name, Msg, State, Debug) ->
+ Reason1 = error_reason(Reason),
+ Fmt =
+ "** Generic server ~p terminating~n"
+ "** Last message in was ~p~n"
+ "** When Server state == ~p~n"
+ "** Reason for termination == ~n** ~p~n",
+ case RootCause of
+ undefined -> format(Fmt, [Name, Msg, State, Reason1]);
+ _ -> format(Fmt ++ "** In 'terminate' callback "
+ "with reason ==~n** ~p~n",
+ [Name, Msg, State, Reason1,
+ error_reason(RootCause)])
+ end,
+ sys:print_log(Debug),
+ ok.
+
+error_reason({undef,[{M,F,A}|MFAs]} = Reason) ->
+ case code:is_loaded(M) of
+ false -> {'module could not be loaded',[{M,F,A}|MFAs]};
+ _ -> case erlang:function_exported(M, F, length(A)) of
+ true -> Reason;
+ false -> {'function not exported',[{M,F,A}|MFAs]}
+ end
+ end;
+error_reason(Reason) ->
+ Reason.
+
+%%% ---------------------------------------------------
+%%% Misc. functions.
+%%% ---------------------------------------------------
+
+opt(Op, [{Op, Value}|_]) ->
+ {ok, Value};
+opt(Op, [_|Options]) ->
+ opt(Op, Options);
+opt(_, []) ->
+ false.
+
+debug_options(Name, Opts) ->
+ case opt(debug, Opts) of
+ {ok, Options} -> dbg_options(Name, Options);
+ _ -> dbg_options(Name, [])
+ end.
+
+dbg_options(Name, []) ->
+ Opts =
+ case init:get_argument(generic_debug) of
+ error ->
+ [];
+ _ ->
+ [log, statistics]
+ end,
+ dbg_opts(Name, Opts);
+dbg_options(Name, Opts) ->
+ dbg_opts(Name, Opts).
+
+dbg_opts(Name, Opts) ->
+ case catch sys:debug_options(Opts) of
+ {'EXIT',_} ->
+ format("~p: ignoring erroneous debug options - ~p~n",
+ [Name, Opts]),
+ [];
+ Dbg ->
+ Dbg
+ end.
+
+get_proc_name(Pid) when is_pid(Pid) ->
+ Pid;
+get_proc_name({local, Name}) ->
+ case process_info(self(), registered_name) of
+ {registered_name, Name} ->
+ Name;
+ {registered_name, _Name} ->
+ exit(process_not_registered);
+ [] ->
+ exit(process_not_registered)
+ end;
+get_proc_name({global, Name}) ->
+ case whereis_name(Name) of
+ undefined ->
+ exit(process_not_registered_globally);
+ Pid when Pid =:= self() ->
+ Name;
+ _Pid ->
+ exit(process_not_registered_globally)
+ end.
+
+get_parent() ->
+ case get('$ancestors') of
+ [Parent | _] when is_pid(Parent)->
+ Parent;
+ [Parent | _] when is_atom(Parent)->
+ name_to_pid(Parent);
+ _ ->
+ exit(process_was_not_started_by_proc_lib)
+ end.
+
+name_to_pid(Name) ->
+ case whereis(Name) of
+ undefined ->
+ case whereis_name(Name) of
+ undefined ->
+ exit(could_not_find_registered_name);
+ Pid ->
+ Pid
+ end;
+ Pid ->
+ Pid
+ end.
+
+whereis_name(Name) ->
+ case ets:lookup(global_names, Name) of
+ [{_Name, Pid, _Method, _RPid, _Ref}] ->
+ if node(Pid) == node() ->
+ case is_process_alive(Pid) of
+ true -> Pid;
+ false -> undefined
+ end;
+ true ->
+ Pid
+ end;
+ [] -> undefined
+ end.
+
+find_prioritisers(GS2State = #gs2_state { mod = Mod }) ->
+ PCall = function_exported_or_default(Mod, 'prioritise_call', 4,
+ fun (_Msg, _From, _State) -> 0 end),
+ PCast = function_exported_or_default(Mod, 'prioritise_cast', 3,
+ fun (_Msg, _State) -> 0 end),
+ PInfo = function_exported_or_default(Mod, 'prioritise_info', 3,
+ fun (_Msg, _State) -> 0 end),
+ GS2State #gs2_state { prioritisers = {PCall, PCast, PInfo} }.
+
+function_exported_or_default(Mod, Fun, Arity, Default) ->
+ case erlang:function_exported(Mod, Fun, Arity) of
+ true -> case Arity of
+ 3 -> fun (Msg, GS2State = #gs2_state { queue = Queue,
+ state = State }) ->
+ Length = priority_queue:len(Queue),
+ case catch Mod:Fun(Msg, Length, State) of
+ drop ->
+ drop;
+ Res when is_integer(Res) ->
+ Res;
+ Err ->
+ handle_common_termination(Err, Msg, GS2State)
+ end
+ end;
+ 4 -> fun (Msg, From, GS2State = #gs2_state { queue = Queue,
+ state = State }) ->
+ Length = priority_queue:len(Queue),
+ case catch Mod:Fun(Msg, From, Length, State) of
+ Res when is_integer(Res) ->
+ Res;
+ Err ->
+ handle_common_termination(Err, Msg, GS2State)
+ end
+ end
+ end;
+ false -> Default
+ end.
+
+%%-----------------------------------------------------------------
+%% Status information
+%%-----------------------------------------------------------------
+format_status(Opt, StatusData) ->
+ [PDict, SysState, Parent, Debug,
+ #gs2_state{name = Name, state = State, mod = Mod, queue = Queue}] =
+ StatusData,
+ NameTag = if is_pid(Name) ->
+ pid_to_list(Name);
+ is_atom(Name) ->
+ Name
+ end,
+ Header = lists:concat(["Status for generic server ", NameTag]),
+ Log = sys:get_debug(log, Debug, []),
+ Specfic = callback(Mod, format_status, [Opt, [PDict, State]],
+ fun () -> [{data, [{"State", State}]}] end),
+ Messages = callback(Mod, format_message_queue, [Opt, Queue],
+ fun () -> priority_queue:to_list(Queue) end),
+ [{header, Header},
+ {data, [{"Status", SysState},
+ {"Parent", Parent},
+ {"Logged events", Log},
+ {"Queued messages", Messages}]} |
+ Specfic].
+
+callback(Mod, FunName, Args, DefaultThunk) ->
+ case erlang:function_exported(Mod, FunName, length(Args)) of
+ true -> case catch apply(Mod, FunName, Args) of
+ {'EXIT', _} -> DefaultThunk();
+ Success -> Success
+ end;
+ false -> DefaultThunk()
+ end.
+
+stats_funs() ->
+ case ets:info(gen_server2_metrics) of
+ undefined ->
+ {fun(GS2State) -> GS2State end,
+ fun(GS2State) -> GS2State end};
+ _ ->
+ {fun emit_stats/1, fun stop_stats/1}
+ end.
+
+init_stats(State = #gs2_state{ emit_stats_fun = EmitStatsFun }) ->
+ StateWithInitTimer = rabbit_event:init_stats_timer(State, #gs2_state.timer),
+ next_stats_timer(EmitStatsFun(StateWithInitTimer)).
+
+next_stats_timer(State) ->
+ ensure_stats_timer(rabbit_event:reset_stats_timer(State, #gs2_state.timer)).
+
+ensure_stats_timer(State) ->
+ rabbit_event:ensure_stats_timer(State,
+ #gs2_state.timer,
+ emit_gen_server2_stats).
+
+stop_stats_timer(State) ->
+ rabbit_event:stop_stats_timer(State, #gs2_state.timer).
+
+emit_stats(State = #gs2_state{queue = Queue}) ->
+ rabbit_core_metrics:gen_server2_stats(self(), priority_queue:len(Queue)),
+ State.
+
+stop_stats(State) ->
+ rabbit_core_metrics:gen_server2_deleted(self()),
+ State.
diff --git a/deps/rabbit_common/src/lager_forwarder_backend.erl b/deps/rabbit_common/src/lager_forwarder_backend.erl
new file mode 100644
index 0000000000..936a1259ce
--- /dev/null
+++ b/deps/rabbit_common/src/lager_forwarder_backend.erl
@@ -0,0 +1,120 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(lager_forwarder_backend).
+
+-behaviour(gen_event).
+
+-export([init/1, handle_call/2, handle_event/2, handle_info/2, terminate/2,
+ code_change/3]).
+
+-record(state, {
+ next_sink :: atom(),
+ level :: {'mask', integer()} | inherit
+ }).
+
+%% @private
+init(Sink) when is_atom(Sink) ->
+ init([Sink]);
+init([Sink]) when is_atom(Sink) ->
+ init([Sink, inherit]);
+init([Sink, inherit]) when is_atom(Sink) ->
+ {ok, #state{
+ next_sink = Sink,
+ level = inherit
+ }};
+init([Sink, Level]) when is_atom(Sink) ->
+ try
+ Mask = lager_util:config_to_mask(Level),
+ {ok, #state{
+ next_sink = Sink,
+ level = Mask
+ }}
+ catch
+ _:_ ->
+ {error, {fatal, bad_log_level}}
+ end;
+init(_) ->
+ {error, {fatal, bad_config}}.
+
+%% @private
+handle_call(get_loglevel, #state{next_sink = Sink, level = inherit} = State) ->
+ SinkPid = whereis(Sink),
+ Mask = case self() of
+ SinkPid ->
+ %% Avoid direct loops, defaults to 'info'.
+ 127;
+ _ ->
+ try
+ Levels = [gen_event:call(SinkPid, Handler, get_loglevel,
+ infinity)
+ || Handler <- gen_event:which_handlers(SinkPid)],
+ lists:foldl(fun
+ ({mask, Mask}, Acc) ->
+ Mask bor Acc;
+ (Level, Acc) when is_integer(Level) ->
+ {mask, Mask} = lager_util:config_to_mask(
+ lager_util:num_to_level(Level)),
+ Mask bor Acc;
+ (_, Acc) ->
+ Acc
+ end, 0, Levels)
+ catch
+ exit:noproc ->
+ 127
+ end
+ end,
+ {ok, {mask, Mask}, State};
+handle_call(get_loglevel, #state{level = Mask} = State) ->
+ {ok, Mask, State};
+handle_call({set_loglevel, inherit}, State) ->
+ {ok, ok, State#state{level = inherit}};
+handle_call({set_loglevel, Level}, State) ->
+ try lager_util:config_to_mask(Level) of
+ Mask ->
+ {ok, ok, State#state{level = Mask}}
+ catch
+ _:_ ->
+ {ok, {error, bad_log_level}, State}
+ end;
+handle_call(_Request, State) ->
+ {ok, ok, State}.
+
+%% @private
+handle_event({log, LagerMsg}, #state{next_sink = Sink, level = Mask} = State) ->
+ SinkPid = whereis(Sink),
+ case self() of
+ SinkPid ->
+ %% Avoid direct loops.
+ ok;
+ _ ->
+ case Mask =:= inherit orelse
+ lager_util:is_loggable(LagerMsg, Mask, ?MODULE) of
+ true ->
+ case lager_config:get({Sink, async}, false) of
+ true -> gen_event:notify(SinkPid, {log, LagerMsg});
+ false -> gen_event:sync_notify(SinkPid, {log, LagerMsg})
+ end;
+ false ->
+ ok
+ end
+ end,
+ {ok, State};
+handle_event(_Event, State) ->
+ {ok, State}.
+
+%% @private
+handle_info(_Info, State) ->
+ {ok, State}.
+
+%% @private
+terminate(_Reason, _State) ->
+ ok.
+
+%% @private
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
diff --git a/deps/rabbit_common/src/mirrored_supervisor.erl b/deps/rabbit_common/src/mirrored_supervisor.erl
new file mode 100644
index 0000000000..61ddc068b6
--- /dev/null
+++ b/deps/rabbit_common/src/mirrored_supervisor.erl
@@ -0,0 +1,513 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2011-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(mirrored_supervisor).
+
+%% pg2 is deprecated in OTP 23.
+-compile(nowarn_deprecated_function).
+
+%% Mirrored Supervisor
+%% ===================
+%%
+%% This module implements a new type of supervisor. It acts like a
+%% normal supervisor, but at creation time you also provide the name
+%% of a process group to join. All the supervisors within the
+%% process group act like a single large distributed supervisor:
+%%
+%% * A process with a given child_id will only exist on one
+%% supervisor within the group.
+%%
+%% * If one supervisor fails, children may migrate to surviving
+%% supervisors within the group.
+%%
+%% In almost all cases you will want to use the module name for the
+%% process group. Using multiple process groups with the same module
+%% name is supported. Having multiple module names for the same
+%% process group will lead to undefined behaviour.
+%%
+%% Motivation
+%% ----------
+%%
+%% Sometimes you have processes which:
+%%
+%% * Only need to exist once per cluster.
+%%
+%% * Does not contain much state (or can reconstruct its state easily).
+%%
+%% * Needs to be restarted elsewhere should it be running on a node
+%% which fails.
+%%
+%% By creating a mirrored supervisor group with one supervisor on
+%% each node, that's what you get.
+%%
+%%
+%% API use
+%% -------
+%%
+%% This is basically the same as for supervisor, except that:
+%%
+%% 1) start_link(Module, Args) becomes
+%% start_link(Group, TxFun, Module, Args).
+%%
+%% 2) start_link({local, Name}, Module, Args) becomes
+%% start_link({local, Name}, Group, TxFun, Module, Args).
+%%
+%% 3) start_link({global, Name}, Module, Args) is not available.
+%%
+%% 4) The restart strategy simple_one_for_one is not available.
+%%
+%% 5) Mnesia is used to hold global state. At some point your
+%% application should invoke create_tables() (or table_definitions()
+%% if it wants to manage table creation itself).
+%%
+%% The TxFun parameter to start_link/{4,5} is a function which the
+%% mirrored supervisor can use to execute Mnesia transactions. In the
+%% RabbitMQ server this goes via a worker pool; in other cases a
+%% function like:
+%%
+%% tx_fun(Fun) ->
+%% case mnesia:sync_transaction(Fun) of
+%% {atomic, Result} -> Result;
+%% {aborted, Reason} -> throw({error, Reason})
+%% end.
+%%
+%% could be used.
+%%
+%% Internals
+%% ---------
+%%
+%% Each mirrored_supervisor consists of three processes - the overall
+%% supervisor, the delegate supervisor and the mirroring server. The
+%% overall supervisor supervises the other two processes. Its pid is
+%% the one returned from start_link; the pids of the other two
+%% processes are effectively hidden in the API.
+%%
+%% The delegate supervisor is in charge of supervising all the child
+%% processes that are added to the supervisor as usual.
+%%
+%% The mirroring server intercepts calls to the supervisor API
+%% (directed at the overall supervisor), does any special handling,
+%% and forwards everything to the delegate supervisor.
+%%
+%% This module implements all three, hence init/1 is somewhat overloaded.
+%%
+%% The mirroring server creates and joins a process group on
+%% startup. It monitors all the existing members of this group, and
+%% broadcasts a "hello" message to them so that they can monitor it in
+%% turn. When it receives a 'DOWN' message, it checks to see if it's
+%% the "first" server in the group and restarts all the child
+%% processes from the dead supervisor if so.
+%%
+%% In the future we might load balance this.
+%%
+%% Startup is slightly fiddly. The mirroring server needs to know the
+%% Pid of the overall supervisor, but we don't have that until it has
+%% started. Therefore we set this after the fact. We also start any
+%% children we found in Module:init() at this point, since starting
+%% children requires knowing the overall supervisor pid.
+
+-define(SUPERVISOR, supervisor2).
+-define(GEN_SERVER, gen_server2).
+-define(SUP_MODULE, mirrored_supervisor_sups).
+
+-define(TABLE, mirrored_sup_childspec).
+-define(TABLE_DEF,
+ {?TABLE,
+ [{record_name, mirrored_sup_childspec},
+ {type, ordered_set},
+ {attributes, record_info(fields, mirrored_sup_childspec)}]}).
+-define(TABLE_MATCH, {match, #mirrored_sup_childspec{ _ = '_' }}).
+
+-export([start_link/4, start_link/5,
+ start_child/2, restart_child/2,
+ delete_child/2, terminate_child/2,
+ which_children/1, count_children/1, check_childspecs/1]).
+
+-behaviour(?GEN_SERVER).
+
+-export([init/1, handle_call/3, handle_info/2, terminate/2, code_change/3,
+ handle_cast/2]).
+
+-export([start_internal/3]).
+-export([create_tables/0, table_definitions/0]).
+
+-record(mirrored_sup_childspec, {key, mirroring_pid, childspec}).
+
+-record(state, {overall,
+ delegate,
+ group,
+ tx_fun,
+ initial_childspecs,
+ child_order}).
+
+%%--------------------------------------------------------------------------
+%% Callback behaviour
+%%--------------------------------------------------------------------------
+
+-callback init(Args :: term()) ->
+ {ok, {{RestartStrategy :: ?SUPERVISOR:strategy(),
+ MaxR :: non_neg_integer(),
+ MaxT :: non_neg_integer()},
+ [ChildSpec :: ?SUPERVISOR:child_spec()]}}
+ | ignore.
+
+%%--------------------------------------------------------------------------
+%% Specs
+%%--------------------------------------------------------------------------
+
+-type startlink_err() :: {'already_started', pid()} | 'shutdown' | term().
+-type startlink_ret() :: {'ok', pid()} | 'ignore' | {'error', startlink_err()}.
+
+-type group_name() :: any().
+
+-type(tx_fun() :: fun((fun(() -> A)) -> A)).
+
+-spec start_link(GroupName, TxFun, Module, Args) -> startlink_ret() when
+ GroupName :: group_name(),
+ TxFun :: tx_fun(),
+ Module :: module(),
+ Args :: term().
+
+-spec start_link(SupName, GroupName, TxFun, Module, Args) ->
+ startlink_ret() when
+ SupName :: ?SUPERVISOR:sup_name(),
+ GroupName :: group_name(),
+ TxFun :: tx_fun(),
+ Module :: module(),
+ Args :: term().
+
+-spec start_internal(Group, TxFun, ChildSpecs) -> Result when
+ Group :: group_name(),
+ TxFun :: tx_fun(),
+ ChildSpecs :: [?SUPERVISOR:child_spec()],
+ Result :: {'ok', pid()} | {'error', term()}.
+
+-spec create_tables() -> Result when
+ Result :: 'ok'.
+
+%%----------------------------------------------------------------------------
+
+start_link(Group, TxFun, Mod, Args) ->
+ start_link0([], Group, TxFun, init(Mod, Args)).
+
+start_link({local, SupName}, Group, TxFun, Mod, Args) ->
+ start_link0([{local, SupName}], Group, TxFun, init(Mod, Args));
+
+start_link({global, _SupName}, _Group, _TxFun, _Mod, _Args) ->
+ erlang:error(badarg).
+
+start_link0(Prefix, Group, TxFun, Init) ->
+ case apply(?SUPERVISOR, start_link,
+ Prefix ++ [?SUP_MODULE, {overall, Group, TxFun, Init}]) of
+ {ok, Pid} -> case catch call(Pid, {init, Pid}) of
+ ok -> {ok, Pid};
+ E -> E
+ end;
+ Other -> Other
+ end.
+
+init(Mod, Args) ->
+ case Mod:init(Args) of
+ {ok, {{Bad, _, _}, _ChildSpecs}} when
+ Bad =:= simple_one_for_one -> erlang:error(badarg);
+ Init -> Init
+ end.
+
+start_child(Sup, ChildSpec) -> call(Sup, {start_child, ChildSpec}).
+delete_child(Sup, Id) -> find_call(Sup, Id, {delete_child, Id}).
+restart_child(Sup, Id) -> find_call(Sup, Id, {msg, restart_child, [Id]}).
+terminate_child(Sup, Id) -> find_call(Sup, Id, {msg, terminate_child, [Id]}).
+which_children(Sup) -> fold(which_children, Sup, fun lists:append/2).
+count_children(Sup) -> fold(count_children, Sup, fun add_proplists/2).
+check_childspecs(Specs) -> ?SUPERVISOR:check_childspecs(Specs).
+
+call(Sup, Msg) -> ?GEN_SERVER:call(mirroring(Sup), Msg, infinity).
+cast(Sup, Msg) -> with_exit_handler(
+ fun() -> ok end,
+ fun() -> ?GEN_SERVER:cast(mirroring(Sup), Msg) end).
+
+find_call(Sup, Id, Msg) ->
+ Group = call(Sup, group),
+ MatchHead = #mirrored_sup_childspec{mirroring_pid = '$1',
+ key = {Group, Id},
+ _ = '_'},
+ %% If we did this inside a tx we could still have failover
+ %% immediately after the tx - we can't be 100% here. So we may as
+ %% well dirty_select.
+ case mnesia:dirty_select(?TABLE, [{MatchHead, [], ['$1']}]) of
+ [Mirror] -> call(Mirror, Msg);
+ [] -> {error, not_found}
+ end.
+
+fold(FunAtom, Sup, AggFun) ->
+ Group = call(Sup, group),
+ lists:foldl(AggFun, [],
+ [apply(?SUPERVISOR, FunAtom, [D]) ||
+ M <- pg2:get_members(Group),
+ D <- [delegate(M)]]).
+
+child(Sup, Id) ->
+ [Pid] = [Pid || {Id1, Pid, _, _} <- ?SUPERVISOR:which_children(Sup),
+ Id1 =:= Id],
+ Pid.
+
+delegate(Sup) -> child(Sup, delegate).
+mirroring(Sup) -> child(Sup, mirroring).
+
+%%----------------------------------------------------------------------------
+
+start_internal(Group, TxFun, ChildSpecs) ->
+ ?GEN_SERVER:start_link(?MODULE, {Group, TxFun, ChildSpecs},
+ [{timeout, infinity}]).
+
+%%----------------------------------------------------------------------------
+
+init({Group, TxFun, ChildSpecs}) ->
+ {ok, #state{group = Group,
+ tx_fun = TxFun,
+ initial_childspecs = ChildSpecs,
+ child_order = child_order_from(ChildSpecs)}}.
+
+handle_call({init, Overall}, _From,
+ State = #state{overall = undefined,
+ delegate = undefined,
+ group = Group,
+ tx_fun = TxFun,
+ initial_childspecs = ChildSpecs}) ->
+ process_flag(trap_exit, true),
+ pg2:create(Group),
+ ok = pg2:join(Group, Overall),
+ Rest = pg2:get_members(Group) -- [Overall],
+ case Rest of
+ [] -> TxFun(fun() -> delete_all(Group) end);
+ _ -> ok
+ end,
+ [begin
+ ?GEN_SERVER:cast(mirroring(Pid), {ensure_monitoring, Overall}),
+ erlang:monitor(process, Pid)
+ end || Pid <- Rest],
+ Delegate = delegate(Overall),
+ erlang:monitor(process, Delegate),
+ State1 = State#state{overall = Overall, delegate = Delegate},
+ case errors([maybe_start(Group, TxFun, Overall, Delegate, S)
+ || S <- ChildSpecs]) of
+ [] -> {reply, ok, State1};
+ Errors -> {stop, {shutdown, Errors}, State1}
+ end;
+
+handle_call({start_child, ChildSpec}, _From,
+ State = #state{overall = Overall,
+ delegate = Delegate,
+ group = Group,
+ tx_fun = TxFun}) ->
+ {reply, case maybe_start(Group, TxFun, Overall, Delegate, ChildSpec) of
+ already_in_mnesia -> {error, already_present};
+ {already_in_mnesia, Pid} -> {error, {already_started, Pid}};
+ Else -> Else
+ end, State};
+
+handle_call({delete_child, Id}, _From, State = #state{delegate = Delegate,
+ group = Group,
+ tx_fun = TxFun}) ->
+ {reply, stop(Group, TxFun, Delegate, Id), State};
+
+handle_call({msg, F, A}, _From, State = #state{delegate = Delegate}) ->
+ {reply, apply(?SUPERVISOR, F, [Delegate | A]), State};
+
+handle_call(group, _From, State = #state{group = Group}) ->
+ {reply, Group, State};
+
+handle_call(Msg, _From, State) ->
+ {stop, {unexpected_call, Msg}, State}.
+
+handle_cast({ensure_monitoring, Pid}, State) ->
+ erlang:monitor(process, Pid),
+ {noreply, State};
+
+handle_cast({die, Reason}, State = #state{group = Group}) ->
+ _ = tell_all_peers_to_die(Group, Reason),
+ {stop, Reason, State};
+
+handle_cast(Msg, State) ->
+ {stop, {unexpected_cast, Msg}, State}.
+
+handle_info({'DOWN', _Ref, process, Pid, Reason},
+ State = #state{delegate = Pid, group = Group}) ->
+ %% Since the delegate is temporary, its death won't cause us to
+ %% die. Since the overall supervisor kills processes in reverse
+ %% order when shutting down "from above" and we started after the
+ %% delegate, if we see the delegate die then that means it died
+ %% "from below" i.e. due to the behaviour of its children, not
+ %% because the whole app was being torn down.
+ %%
+ %% Therefore if we get here we know we need to cause the entire
+ %% mirrored sup to shut down, not just fail over.
+ _ = tell_all_peers_to_die(Group, Reason),
+ {stop, Reason, State};
+
+handle_info({'DOWN', _Ref, process, Pid, _Reason},
+ State = #state{delegate = Delegate,
+ group = Group,
+ tx_fun = TxFun,
+ overall = O,
+ child_order = ChildOrder}) ->
+ %% TODO load balance this
+ %% No guarantee pg2 will have received the DOWN before us.
+ R = case lists:sort(pg2:get_members(Group)) -- [Pid] of
+ [O | _] -> ChildSpecs =
+ TxFun(fun() -> update_all(O, Pid) end),
+ [start(Delegate, ChildSpec)
+ || ChildSpec <- restore_child_order(ChildSpecs,
+ ChildOrder)];
+ _ -> []
+ end,
+ case errors(R) of
+ [] -> {noreply, State};
+ Errors -> {stop, {shutdown, Errors}, State}
+ end;
+
+handle_info(Info, State) ->
+ {stop, {unexpected_info, Info}, State}.
+
+terminate(_Reason, _State) ->
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+%%----------------------------------------------------------------------------
+
+tell_all_peers_to_die(Group, Reason) ->
+ [cast(P, {die, Reason}) || P <- pg2:get_members(Group) -- [self()]].
+
+maybe_start(Group, TxFun, Overall, Delegate, ChildSpec) ->
+ try TxFun(fun() -> check_start(Group, Overall, Delegate, ChildSpec) end) of
+ start -> start(Delegate, ChildSpec);
+ undefined -> already_in_mnesia;
+ Pid -> {already_in_mnesia, Pid}
+ catch
+ %% If we are torn down while in the transaction...
+ {error, E} -> {error, E}
+ end.
+
+check_start(Group, Overall, Delegate, ChildSpec) ->
+ case mnesia:wread({?TABLE, {Group, id(ChildSpec)}}) of
+ [] -> _ = write(Group, Overall, ChildSpec),
+ start;
+ [S] -> #mirrored_sup_childspec{key = {Group, Id},
+ mirroring_pid = Pid} = S,
+ case Overall of
+ Pid -> child(Delegate, Id);
+ _ -> case supervisor(Pid) of
+ dead -> _ = write(Group, Overall, ChildSpec),
+ start;
+ Delegate0 -> child(Delegate0, Id)
+ end
+ end
+ end.
+
+supervisor(Pid) -> with_exit_handler(fun() -> dead end,
+ fun() -> delegate(Pid) end).
+
+write(Group, Overall, ChildSpec) ->
+ S = #mirrored_sup_childspec{key = {Group, id(ChildSpec)},
+ mirroring_pid = Overall,
+ childspec = ChildSpec},
+ ok = mnesia:write(?TABLE, S, write),
+ ChildSpec.
+
+delete(Group, Id) ->
+ ok = mnesia:delete({?TABLE, {Group, Id}}).
+
+start(Delegate, ChildSpec) ->
+ apply(?SUPERVISOR, start_child, [Delegate, ChildSpec]).
+
+stop(Group, TxFun, Delegate, Id) ->
+ try TxFun(fun() -> check_stop(Group, Delegate, Id) end) of
+ deleted -> apply(?SUPERVISOR, delete_child, [Delegate, Id]);
+ running -> {error, running}
+ catch
+ {error, E} -> {error, E}
+ end.
+
+check_stop(Group, Delegate, Id) ->
+ case child(Delegate, Id) of
+ undefined -> delete(Group, Id),
+ deleted;
+ _ -> running
+ end.
+
+id({Id, _, _, _, _, _}) -> Id.
+
+update_all(Overall, OldOverall) ->
+ MatchHead = #mirrored_sup_childspec{mirroring_pid = OldOverall,
+ key = '$1',
+ childspec = '$2',
+ _ = '_'},
+ [write(Group, Overall, C) ||
+ [{Group, _Id}, C] <- mnesia:select(?TABLE, [{MatchHead, [], ['$$']}])].
+
+delete_all(Group) ->
+ MatchHead = #mirrored_sup_childspec{key = {Group, '_'},
+ childspec = '$1',
+ _ = '_'},
+ [delete(Group, id(C)) ||
+ C <- mnesia:select(?TABLE, [{MatchHead, [], ['$1']}])].
+
+errors(Results) -> [E || {error, E} <- Results].
+
+%%----------------------------------------------------------------------------
+
+create_tables() -> create_tables([?TABLE_DEF]).
+
+create_tables([]) ->
+ ok;
+create_tables([{Table, Attributes} | Ts]) ->
+ case mnesia:create_table(Table, Attributes) of
+ {atomic, ok} -> create_tables(Ts);
+ {aborted, {already_exists, ?TABLE}} -> create_tables(Ts);
+ Err -> Err
+ end.
+
+table_definitions() ->
+ {Name, Attributes} = ?TABLE_DEF,
+ [{Name, [?TABLE_MATCH | Attributes]}].
+
+%%----------------------------------------------------------------------------
+
+with_exit_handler(Handler, Thunk) ->
+ try
+ Thunk()
+ catch
+ exit:{R, _} when R =:= noproc; R =:= nodedown;
+ R =:= normal; R =:= shutdown ->
+ Handler();
+ exit:{{R, _}, _} when R =:= nodedown; R =:= shutdown ->
+ Handler()
+ end.
+
+add_proplists(P1, P2) ->
+ add_proplists(lists:keysort(1, P1), lists:keysort(1, P2), []).
+add_proplists([], P2, Acc) -> P2 ++ Acc;
+add_proplists(P1, [], Acc) -> P1 ++ Acc;
+add_proplists([{K, V1} | P1], [{K, V2} | P2], Acc) ->
+ add_proplists(P1, P2, [{K, V1 + V2} | Acc]);
+add_proplists([{K1, _} = KV | P1], [{K2, _} | _] = P2, Acc) when K1 < K2 ->
+ add_proplists(P1, P2, [KV | Acc]);
+add_proplists(P1, [KV | P2], Acc) ->
+ add_proplists(P1, P2, [KV | Acc]).
+
+child_order_from(ChildSpecs) ->
+ lists:zipwith(fun(C, N) ->
+ {id(C), N}
+ end, ChildSpecs, lists:seq(1, length(ChildSpecs))).
+
+restore_child_order(ChildSpecs, ChildOrder) ->
+ lists:sort(fun(A, B) ->
+ proplists:get_value(id(A), ChildOrder)
+ < proplists:get_value(id(B), ChildOrder)
+ end, ChildSpecs).
diff --git a/deps/rabbit_common/src/mnesia_sync.erl b/deps/rabbit_common/src/mnesia_sync.erl
new file mode 100644
index 0000000000..2287436849
--- /dev/null
+++ b/deps/rabbit_common/src/mnesia_sync.erl
@@ -0,0 +1,64 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(mnesia_sync).
+
+%% mnesia:sync_transaction/3 fails to guarantee that the log is flushed to disk
+%% at commit. This module is an attempt to minimise the risk of data loss by
+%% performing a coalesced log fsync. Unfortunately this is performed regardless
+%% of whether or not the log was appended to.
+
+-behaviour(gen_server).
+
+-export([sync/0]).
+
+-export([start_link/0, init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3]).
+
+-define(SERVER, ?MODULE).
+
+-record(state, {waiting, disc_node}).
+
+%%----------------------------------------------------------------------------
+
+-spec sync() -> 'ok'.
+
+%%----------------------------------------------------------------------------
+
+start_link() ->
+ gen_server:start_link({local, ?SERVER}, ?MODULE, [], []).
+
+sync() ->
+ gen_server:call(?SERVER, sync, infinity).
+
+%%----------------------------------------------------------------------------
+
+init([]) ->
+ {ok, #state{disc_node = mnesia:system_info(use_dir), waiting = []}}.
+
+handle_call(sync, _From, #state{disc_node = false} = State) ->
+ {reply, ok, State};
+handle_call(sync, From, #state{waiting = Waiting} = State) ->
+ {noreply, State#state{waiting = [From | Waiting]}, 0};
+handle_call(Request, _From, State) ->
+ {stop, {unhandled_call, Request}, State}.
+
+handle_cast(Request, State) ->
+ {stop, {unhandled_cast, Request}, State}.
+
+handle_info(timeout, #state{waiting = Waiting} = State) ->
+ ok = disk_log:sync(latest_log),
+ _ = [gen_server:reply(From, ok) || From <- Waiting],
+ {noreply, State#state{waiting = []}};
+handle_info(Message, State) ->
+ {stop, {unhandled_info, Message}, State}.
+
+terminate(_Reason, _State) ->
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
diff --git a/deps/rabbit_common/src/pmon.erl b/deps/rabbit_common/src/pmon.erl
new file mode 100644
index 0000000000..f44168dfcf
--- /dev/null
+++ b/deps/rabbit_common/src/pmon.erl
@@ -0,0 +1,96 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2011-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(pmon).
+
+%% Process Monitor
+%% ================
+%%
+%% This module monitors processes so that every process has at most
+%% 1 monitor.
+%% Processes monitored can be dynamically added and removed.
+%%
+%% Unlike erlang:[de]monitor* functions, this module
+%% provides basic querying capability and avoids contacting down nodes.
+%%
+%% It is used to monitor nodes, queue mirrors, and by
+%% the queue collector, among other things.
+
+-export([new/0, new/1, monitor/2, monitor_all/2, demonitor/2,
+ is_monitored/2, erase/2, monitored/1, is_empty/1]).
+
+-compile({no_auto_import, [monitor/2]}).
+
+-record(state, {monitors = #{} :: #{item() => reference()},
+ module = erlang :: module()}).
+
+%%----------------------------------------------------------------------------
+
+-export_type([?MODULE/0]).
+
+-opaque(?MODULE() :: #state{}).
+
+-type(item() :: pid() | {atom(), node()}).
+
+
+-spec new() -> ?MODULE().
+new() -> new(erlang).
+
+-spec new('erlang' | 'delegate') -> ?MODULE().
+new(Module) -> #state{module = Module}.
+
+-spec monitor(item(), ?MODULE()) -> ?MODULE().
+monitor(Item, S = #state{monitors = M, module = Module}) ->
+ case maps:is_key(Item, M) of
+ true -> S;
+ false -> case node_alive_shortcut(Item) of
+ true -> Ref = Module:monitor(process, Item),
+ S#state{monitors = maps:put(Item, Ref, M)};
+ false -> self() ! {'DOWN', fake_ref, process, Item,
+ nodedown},
+ S
+ end
+ end.
+
+-spec monitor_all([item()], ?MODULE()) -> ?MODULE().
+monitor_all([], S) -> S; %% optimisation
+monitor_all([Item], S) -> monitor(Item, S); %% optimisation
+monitor_all(Items, S) -> lists:foldl(fun monitor/2, S, Items).
+
+-spec demonitor(item(), ?MODULE()) -> ?MODULE().
+demonitor(Item, S = #state{monitors = M0, module = Module}) ->
+ case maps:take(Item, M0) of
+ {MRef, M} -> Module:demonitor(MRef),
+ S#state{monitors = M};
+ error -> S
+ end.
+
+-spec is_monitored(item(), ?MODULE()) -> boolean().
+is_monitored(Item, #state{monitors = M}) -> maps:is_key(Item, M).
+
+-spec erase(item(), ?MODULE()) -> ?MODULE().
+erase(Item, S = #state{monitors = M}) ->
+ S#state{monitors = maps:remove(Item, M)}.
+
+-spec monitored(?MODULE()) -> [item()].
+monitored(#state{monitors = M}) -> maps:keys(M).
+
+-spec is_empty(?MODULE()) -> boolean().
+is_empty(#state{monitors = M}) -> maps:size(M) == 0.
+
+%%----------------------------------------------------------------------------
+
+%% We check here to see if the node is alive in order to avoid trying
+%% to connect to it if it isn't - this can cause substantial
+%% slowdowns. We can't perform this shortcut if passed {Name, Node}
+%% since we would need to convert that into a pid for the fake 'DOWN'
+%% message, so we always return true here - but that's OK, it's just
+%% an optimisation.
+node_alive_shortcut(P) when is_pid(P) ->
+ lists:member(node(P), [node() | nodes()]);
+node_alive_shortcut({_Name, _Node}) ->
+ true.
diff --git a/deps/rabbit_common/src/priority_queue.erl b/deps/rabbit_common/src/priority_queue.erl
new file mode 100644
index 0000000000..4a7867129d
--- /dev/null
+++ b/deps/rabbit_common/src/priority_queue.erl
@@ -0,0 +1,234 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+%% Priority queues have essentially the same interface as ordinary
+%% queues, except that a) there is an in/3 that takes a priority, and
+%% b) we have only implemented the core API we need.
+%%
+%% Priorities should be integers - the higher the value the higher the
+%% priority - but we don't actually check that.
+%%
+%% in/2 inserts items with priority 0.
+%%
+%% We optimise the case where a priority queue is being used just like
+%% an ordinary queue. When that is the case we represent the priority
+%% queue as an ordinary queue. We could just call into the 'queue'
+%% module for that, but for efficiency we implement the relevant
+%% functions directly in here, thus saving on inter-module calls and
+%% eliminating a level of boxing.
+%%
+%% When the queue contains items with non-zero priorities, it is
+%% represented as a sorted kv list with the inverted Priority as the
+%% key and an ordinary queue as the value. Here again we use our own
+%% ordinary queue implementation for efficiency, often making recursive
+%% calls into the same function knowing that ordinary queues represent
+%% a base case.
+
+
+-module(priority_queue).
+
+-export([new/0, is_queue/1, is_empty/1, len/1, to_list/1, from_list/1,
+ in/2, in/3, out/1, out_p/1, join/2, filter/2, fold/3, highest/1,
+ member/2]).
+
+%%----------------------------------------------------------------------------
+
+-export_type([q/0]).
+
+-type(q() :: pqueue()).
+-type(priority() :: integer() | 'infinity').
+-type(squeue() :: {queue, [any()], [any()], non_neg_integer()}).
+-type(pqueue() :: squeue() | {pqueue, [{priority(), squeue()}]}).
+
+-spec new() -> pqueue().
+-spec is_queue(any()) -> boolean().
+-spec is_empty(pqueue()) -> boolean().
+-spec len(pqueue()) -> non_neg_integer().
+-spec to_list(pqueue()) -> [{priority(), any()}].
+-spec from_list([{priority(), any()}]) -> pqueue().
+-spec in(any(), pqueue()) -> pqueue().
+-spec in(any(), priority(), pqueue()) -> pqueue().
+-spec out(pqueue()) -> {empty | {value, any()}, pqueue()}.
+-spec out_p(pqueue()) -> {empty | {value, any(), priority()}, pqueue()}.
+-spec join(pqueue(), pqueue()) -> pqueue().
+-spec filter(fun ((any()) -> boolean()), pqueue()) -> pqueue().
+-spec fold
+ (fun ((any(), priority(), A) -> A), A, pqueue()) -> A.
+-spec highest(pqueue()) -> priority() | 'empty'.
+-spec member(any(), pqueue()) -> boolean().
+
+%%----------------------------------------------------------------------------
+
+new() ->
+ {queue, [], [], 0}.
+
+is_queue({queue, R, F, L}) when is_list(R), is_list(F), is_integer(L) ->
+ true;
+is_queue({pqueue, Queues}) when is_list(Queues) ->
+ lists:all(fun ({infinity, Q}) -> is_queue(Q);
+ ({P, Q}) -> is_integer(P) andalso is_queue(Q)
+ end, Queues);
+is_queue(_) ->
+ false.
+
+is_empty({queue, [], [], 0}) ->
+ true;
+is_empty(_) ->
+ false.
+
+len({queue, _R, _F, L}) ->
+ L;
+len({pqueue, Queues}) ->
+ lists:sum([len(Q) || {_, Q} <- Queues]).
+
+to_list({queue, In, Out, _Len}) when is_list(In), is_list(Out) ->
+ [{0, V} || V <- Out ++ lists:reverse(In, [])];
+to_list({pqueue, Queues}) ->
+ [{maybe_negate_priority(P), V} || {P, Q} <- Queues,
+ {0, V} <- to_list(Q)].
+
+from_list(L) ->
+ lists:foldl(fun ({P, E}, Q) -> in(E, P, Q) end, new(), L).
+
+in(Item, Q) ->
+ in(Item, 0, Q).
+
+in(X, 0, {queue, [_] = In, [], 1}) ->
+ {queue, [X], In, 2};
+in(X, 0, {queue, In, Out, Len}) when is_list(In), is_list(Out) ->
+ {queue, [X|In], Out, Len + 1};
+in(X, Priority, _Q = {queue, [], [], 0}) ->
+ in(X, Priority, {pqueue, []});
+in(X, Priority, Q = {queue, _, _, _}) ->
+ in(X, Priority, {pqueue, [{0, Q}]});
+in(X, Priority, {pqueue, Queues}) ->
+ P = maybe_negate_priority(Priority),
+ {pqueue, case lists:keysearch(P, 1, Queues) of
+ {value, {_, Q}} ->
+ lists:keyreplace(P, 1, Queues, {P, in(X, Q)});
+ false when P == infinity ->
+ [{P, {queue, [X], [], 1}} | Queues];
+ false ->
+ case Queues of
+ [{infinity, InfQueue} | Queues1] ->
+ [{infinity, InfQueue} |
+ lists:keysort(1, [{P, {queue, [X], [], 1}} | Queues1])];
+ _ ->
+ lists:keysort(1, [{P, {queue, [X], [], 1}} | Queues])
+ end
+ end}.
+
+out({queue, [], [], 0} = Q) ->
+ {empty, Q};
+out({queue, [V], [], 1}) ->
+ {{value, V}, {queue, [], [], 0}};
+out({queue, [Y|In], [], Len}) ->
+ [V|Out] = lists:reverse(In, []),
+ {{value, V}, {queue, [Y], Out, Len - 1}};
+out({queue, In, [V], Len}) when is_list(In) ->
+ {{value,V}, r2f(In, Len - 1)};
+out({queue, In,[V|Out], Len}) when is_list(In) ->
+ {{value, V}, {queue, In, Out, Len - 1}};
+out({pqueue, [{P, Q} | Queues]}) ->
+ {R, Q1} = out(Q),
+ NewQ = case is_empty(Q1) of
+ true -> case Queues of
+ [] -> {queue, [], [], 0};
+ [{0, OnlyQ}] -> OnlyQ;
+ [_|_] -> {pqueue, Queues}
+ end;
+ false -> {pqueue, [{P, Q1} | Queues]}
+ end,
+ {R, NewQ}.
+
+out_p({queue, _, _, _} = Q) -> add_p(out(Q), 0);
+out_p({pqueue, [{P, _} | _]} = Q) -> add_p(out(Q), maybe_negate_priority(P)).
+
+add_p(R, P) -> case R of
+ {empty, Q} -> {empty, Q};
+ {{value, V}, Q} -> {{value, V, P}, Q}
+ end.
+
+join(A, {queue, [], [], 0}) ->
+ A;
+join({queue, [], [], 0}, B) ->
+ B;
+join({queue, AIn, AOut, ALen}, {queue, BIn, BOut, BLen}) ->
+ {queue, BIn, AOut ++ lists:reverse(AIn, BOut), ALen + BLen};
+join(A = {queue, _, _, _}, {pqueue, BPQ}) ->
+ {Pre, Post} =
+ lists:splitwith(fun ({P, _}) -> P < 0 orelse P == infinity end, BPQ),
+ Post1 = case Post of
+ [] -> [ {0, A} ];
+ [ {0, ZeroQueue} | Rest ] -> [ {0, join(A, ZeroQueue)} | Rest ];
+ _ -> [ {0, A} | Post ]
+ end,
+ {pqueue, Pre ++ Post1};
+join({pqueue, APQ}, B = {queue, _, _, _}) ->
+ {Pre, Post} =
+ lists:splitwith(fun ({P, _}) -> P < 0 orelse P == infinity end, APQ),
+ Post1 = case Post of
+ [] -> [ {0, B} ];
+ [ {0, ZeroQueue} | Rest ] -> [ {0, join(ZeroQueue, B)} | Rest ];
+ _ -> [ {0, B} | Post ]
+ end,
+ {pqueue, Pre ++ Post1};
+join({pqueue, APQ}, {pqueue, BPQ}) ->
+ {pqueue, merge(APQ, BPQ, [])}.
+
+merge([], BPQ, Acc) ->
+ lists:reverse(Acc, BPQ);
+merge(APQ, [], Acc) ->
+ lists:reverse(Acc, APQ);
+merge([{P, A}|As], [{P, B}|Bs], Acc) ->
+ merge(As, Bs, [ {P, join(A, B)} | Acc ]);
+merge([{PA, A}|As], Bs = [{PB, _}|_], Acc) when PA < PB orelse PA == infinity ->
+ merge(As, Bs, [ {PA, A} | Acc ]);
+merge(As = [{_, _}|_], [{PB, B}|Bs], Acc) ->
+ merge(As, Bs, [ {PB, B} | Acc ]).
+
+filter(Pred, Q) -> fold(fun(V, P, Acc) ->
+ case Pred(V) of
+ true -> in(V, P, Acc);
+ false -> Acc
+ end
+ end, new(), Q).
+
+fold(Fun, Init, Q) -> case out_p(Q) of
+ {empty, _Q} -> Init;
+ {{value, V, P}, Q1} -> fold(Fun, Fun(V, P, Init), Q1)
+ end.
+
+highest({queue, [], [], 0}) -> empty;
+highest({queue, _, _, _}) -> 0;
+highest({pqueue, [{P, _} | _]}) -> maybe_negate_priority(P).
+
+member(_X, {queue, [], [], 0}) ->
+ false;
+member(X, {queue, R, F, _Size}) ->
+ lists:member(X, R) orelse lists:member(X, F);
+member(_X, {pqueue, []}) ->
+ false;
+member(X, {pqueue, [{_P, Q}]}) ->
+ member(X, Q);
+member(X, {pqueue, [{_P, Q} | T]}) ->
+ case member(X, Q) of
+ true ->
+ true;
+ false ->
+ member(X, {pqueue, T})
+ end;
+member(X, Q) ->
+ erlang:error(badarg, [X,Q]).
+
+r2f([], 0) -> {queue, [], [], 0};
+r2f([_] = R, 1) -> {queue, [], R, 1};
+r2f([X,Y], 2) -> {queue, [X], [Y], 2};
+r2f([X,Y|R], L) -> {queue, [X,Y], lists:reverse(R, []), L}.
+
+maybe_negate_priority(infinity) -> infinity;
+maybe_negate_priority(P) -> -P.
diff --git a/deps/rabbit_common/src/rabbit_amqp_connection.erl b/deps/rabbit_common/src/rabbit_amqp_connection.erl
new file mode 100644
index 0000000000..58486bd239
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_amqp_connection.erl
@@ -0,0 +1,34 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_amqp_connection).
+
+-export([amqp_params/2]).
+
+-spec amqp_params(pid(), timeout()) -> [{atom(), term()}].
+amqp_params(ConnPid, Timeout) ->
+ P = try
+ gen_server:call(ConnPid, {info, [amqp_params]}, Timeout)
+ catch exit:{noproc, Error} ->
+ rabbit_log:debug("file ~p, line ~p - connection process ~p not alive: ~p~n",
+ [?FILE, ?LINE, ConnPid, Error]),
+ [];
+ _:Error ->
+ rabbit_log:debug("file ~p, line ~p - failed to get amqp_params from connection process ~p: ~p~n",
+ [?FILE, ?LINE, ConnPid, Error]),
+ []
+ end,
+ process_amqp_params_result(P).
+
+process_amqp_params_result({error, {bad_argument, amqp_params}}) ->
+ %% Some connection process modules do not handle the {info, [amqp_params]}
+ %% message (like rabbit_reader) and throw a bad_argument error
+ [];
+process_amqp_params_result({ok, AmqpParams}) ->
+ AmqpParams;
+process_amqp_params_result(AmqpParams) ->
+ AmqpParams.
diff --git a/deps/rabbit_common/src/rabbit_amqqueue_common.erl b/deps/rabbit_common/src/rabbit_amqqueue_common.erl
new file mode 100644
index 0000000000..a45356de78
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_amqqueue_common.erl
@@ -0,0 +1,39 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_amqqueue_common).
+
+-export([notify_sent/2, notify_sent_queue_down/1, delete_exclusive/2]).
+
+-define(MORE_CONSUMER_CREDIT_AFTER, 50).
+
+-spec notify_sent(pid(), pid()) -> 'ok'.
+
+notify_sent(QPid, ChPid) ->
+ Key = {consumer_credit_to, QPid},
+ put(Key, case get(Key) of
+ 1 -> gen_server2:cast(
+ QPid, {notify_sent, ChPid,
+ ?MORE_CONSUMER_CREDIT_AFTER}),
+ ?MORE_CONSUMER_CREDIT_AFTER;
+ undefined -> erlang:monitor(process, QPid),
+ ?MORE_CONSUMER_CREDIT_AFTER - 1;
+ C -> C - 1
+ end),
+ ok.
+
+-spec notify_sent_queue_down(pid()) -> 'ok'.
+
+notify_sent_queue_down(QPid) ->
+ erase({consumer_credit_to, QPid}),
+ ok.
+
+-spec delete_exclusive([pid()], pid()) -> 'ok'.
+
+delete_exclusive(QPids, ConnId) ->
+ [gen_server2:cast(QPid, {delete_exclusive, ConnId}) || QPid <- QPids],
+ ok.
diff --git a/deps/rabbit_common/src/rabbit_auth_backend_dummy.erl b/deps/rabbit_common/src/rabbit_auth_backend_dummy.erl
new file mode 100644
index 0000000000..8d30fdca1b
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_auth_backend_dummy.erl
@@ -0,0 +1,39 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_auth_backend_dummy).
+-include("rabbit.hrl").
+
+-behaviour(rabbit_authn_backend).
+-behaviour(rabbit_authz_backend).
+
+-export([user/0]).
+-export([user_login_authentication/2, user_login_authorization/2,
+ check_vhost_access/3, check_resource_access/4, check_topic_access/4]).
+-export([state_can_expire/0]).
+
+-spec user() -> rabbit_types:user().
+
+%% A user to be used by the direct client when permission checks are
+%% not needed. This user can do anything AMQPish.
+user() -> #user{username = <<"none">>,
+ tags = [],
+ authz_backends = [{?MODULE, none}]}.
+
+%% Implementation of rabbit_auth_backend
+
+user_login_authentication(_, _) ->
+ {refused, "cannot log in conventionally as dummy user", []}.
+
+user_login_authorization(_, _) ->
+ {refused, "cannot log in conventionally as dummy user", []}.
+
+check_vhost_access(#auth_user{}, _VHostPath, _AuthzData) -> true.
+check_resource_access(#auth_user{}, #resource{}, _Permission, _Context) -> true.
+check_topic_access(#auth_user{}, #resource{}, _Permission, _Context) -> true.
+
+state_can_expire() -> false.
diff --git a/deps/rabbit_common/src/rabbit_auth_mechanism.erl b/deps/rabbit_common/src/rabbit_auth_mechanism.erl
new file mode 100644
index 0000000000..38d21f3a5a
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_auth_mechanism.erl
@@ -0,0 +1,41 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_auth_mechanism).
+
+-behaviour(rabbit_registry_class).
+
+-export([added_to_rabbit_registry/2, removed_from_rabbit_registry/1]).
+
+%% A description.
+-callback description() -> [proplists:property()].
+
+%% If this mechanism is enabled, should it be offered for a given socket?
+%% (primarily so EXTERNAL can be TLS-only)
+-callback should_offer(rabbit_net:socket()) -> boolean().
+
+%% Called before authentication starts. Should create a state
+%% object to be passed through all the stages of authentication.
+-callback init(rabbit_net:socket()) -> any().
+
+%% Handle a stage of authentication. Possible responses:
+%% {ok, User}
+%% Authentication succeeded, and here's the user record.
+%% {challenge, Challenge, NextState}
+%% Another round is needed. Here's the state I want next time.
+%% {protocol_error, Msg, Args}
+%% Client got the protocol wrong. Log and die.
+%% {refused, Username, Msg, Args}
+%% Client failed authentication. Log and die.
+-callback handle_response(binary(), any()) ->
+ {'ok', rabbit_types:user()} |
+ {'challenge', binary(), any()} |
+ {'protocol_error', string(), [any()]} |
+ {'refused', rabbit_types:username() | none, string(), [any()]}.
+
+added_to_rabbit_registry(_Type, _ModuleName) -> ok.
+removed_from_rabbit_registry(_Type) -> ok.
diff --git a/deps/rabbit_common/src/rabbit_authn_backend.erl b/deps/rabbit_common/src/rabbit_authn_backend.erl
new file mode 100644
index 0000000000..e600ec884f
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_authn_backend.erl
@@ -0,0 +1,27 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_authn_backend).
+
+-include("rabbit.hrl").
+
+%% Check a user can log in, given a username and a proplist of
+%% authentication information (e.g. [{password, Password}]). If your
+%% backend is not to be used for authentication, this should always
+%% refuse access.
+%%
+%% Possible responses:
+%% {ok, User}
+%% Authentication succeeded, and here's the user record.
+%% {error, Error}
+%% Something went wrong. Log and die.
+%% {refused, Msg, Args}
+%% Client failed authentication. Log and die.
+-callback user_login_authentication(rabbit_types:username(), [term()] | map()) ->
+ {'ok', rabbit_types:auth_user()} |
+ {'refused', string(), [any()]} |
+ {'error', any()}.
diff --git a/deps/rabbit_common/src/rabbit_authz_backend.erl b/deps/rabbit_common/src/rabbit_authz_backend.erl
new file mode 100644
index 0000000000..367aa8d1ef
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_authz_backend.erl
@@ -0,0 +1,88 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_authz_backend).
+
+-include("rabbit.hrl").
+
+%% Check that a user can log in, when this backend is being used for
+%% authorisation only. Authentication has already taken place
+%% successfully, but we need to check that the user exists in this
+%% backend, and initialise any impl field we will want to have passed
+%% back in future calls to check_vhost_access/3 and
+%% check_resource_access/3.
+%%
+%% Possible responses:
+%% {ok, Impl}
+%% {ok, Impl, Tags}
+%% User authorisation succeeded, and here's the impl and potential extra tags fields.
+%% {error, Error}
+%% Something went wrong. Log and die.
+%% {refused, Msg, Args}
+%% User authorisation failed. Log and die.
+-callback user_login_authorization(rabbit_types:username(), [term()] | map()) ->
+ {'ok', any()} |
+ {'ok', any(), any()} |
+ {'refused', string(), [any()]} |
+ {'error', any()}.
+
+%% Given #auth_user, vhost and data (client IP for now), can a user log in to a vhost?
+%% Possible responses:
+%% true
+%% false
+%% {error, Error}
+%% Something went wrong. Log and die.
+-callback check_vhost_access(AuthUser :: rabbit_types:auth_user(),
+ VHost :: rabbit_types:vhost(),
+ AuthzData :: rabbit_types:authz_data()) ->
+ boolean() | {'error', any()}.
+
+%% Given #auth_user, resource and permission, can a user access a resource?
+%%
+%% Possible responses:
+%% true
+%% false
+%% {error, Error}
+%% Something went wrong. Log and die.
+-callback check_resource_access(rabbit_types:auth_user(),
+ rabbit_types:r(atom()),
+ rabbit_access_control:permission_atom(),
+ rabbit_types:authz_context()) ->
+ boolean() | {'error', any()}.
+
+%% Given #auth_user, topic as resource, permission, and context, can a user access the topic?
+%%
+%% Possible responses:
+%% true
+%% false
+%% {error, Error}
+%% Something went wrong. Log and die.
+-callback check_topic_access(rabbit_types:auth_user(),
+ rabbit_types:r(atom()),
+ rabbit_access_control:permission_atom(),
+ rabbit_types:topic_access_context()) ->
+ boolean() | {'error', any()}.
+
+%% Returns true for backends that support state or credential expiration (e.g. use JWTs).
+-callback state_can_expire() -> boolean().
+
+%% Updates backend state that has expired.
+%%
+%% Possible responses:
+%% {ok, User}
+%% Secret updated successfully, and here's the user record.
+%% {error, Error}
+%% Something went wrong.
+%% {refused, Msg, Args}
+%% New secret is not valid or the user cannot authenticate with it.
+-callback update_state(AuthUser :: rabbit_types:auth_user(),
+ NewState :: term()) ->
+ {'ok', rabbit_types:auth_user()} |
+ {'refused', string(), [any()]} |
+ {'error', any()}.
+
+-optional_callbacks([update_state/2]).
diff --git a/deps/rabbit_common/src/rabbit_basic_common.erl b/deps/rabbit_common/src/rabbit_basic_common.erl
new file mode 100644
index 0000000000..e88f1172af
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_basic_common.erl
@@ -0,0 +1,41 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_basic_common).
+-include("rabbit.hrl").
+
+-export([build_content/2, from_content/1]).
+
+-spec build_content
+ (rabbit_framing:amqp_property_record(), binary() | [binary()]) ->
+ rabbit_types:content().
+-spec from_content
+ (rabbit_types:content()) ->
+ {rabbit_framing:amqp_property_record(), binary()}.
+
+build_content(Properties, BodyBin) when is_binary(BodyBin) ->
+ build_content(Properties, [BodyBin]);
+
+build_content(Properties, PFR) ->
+ %% basic.publish hasn't changed so we can just hard-code amqp_0_9_1
+ {ClassId, _MethodId} =
+ rabbit_framing_amqp_0_9_1:method_id('basic.publish'),
+ #content{class_id = ClassId,
+ properties = Properties,
+ properties_bin = none,
+ protocol = none,
+ payload_fragments_rev = PFR}.
+
+from_content(Content) ->
+ #content{class_id = ClassId,
+ properties = Props,
+ payload_fragments_rev = FragmentsRev} =
+ rabbit_binary_parser:ensure_content_decoded(Content),
+ %% basic.publish hasn't changed so we can just hard-code amqp_0_9_1
+ {ClassId, _MethodId} =
+ rabbit_framing_amqp_0_9_1:method_id('basic.publish'),
+ {Props, list_to_binary(lists:reverse(FragmentsRev))}.
diff --git a/deps/rabbit_common/src/rabbit_binary_generator.erl b/deps/rabbit_common/src/rabbit_binary_generator.erl
new file mode 100644
index 0000000000..7a56cb92b6
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_binary_generator.erl
@@ -0,0 +1,235 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_binary_generator).
+-include("rabbit_framing.hrl").
+-include("rabbit.hrl").
+
+-export([build_simple_method_frame/3,
+ build_simple_content_frames/4,
+ build_heartbeat_frame/0]).
+-export([generate_table/1]).
+-export([check_empty_frame_size/0]).
+-export([ensure_content_encoded/2, clear_encoded_content/1]).
+-export([map_exception/3]).
+
+%%----------------------------------------------------------------------------
+
+-type frame() :: [binary()].
+
+-spec build_simple_method_frame
+ (rabbit_channel:channel_number(), rabbit_framing:amqp_method_record(),
+ rabbit_types:protocol()) ->
+ frame().
+-spec build_simple_content_frames
+ (rabbit_channel:channel_number(), rabbit_types:content(),
+ non_neg_integer(), rabbit_types:protocol()) ->
+ [frame()].
+-spec build_heartbeat_frame() -> frame().
+-spec generate_table(rabbit_framing:amqp_table()) -> binary().
+-spec check_empty_frame_size() -> 'ok'.
+-spec ensure_content_encoded
+ (rabbit_types:content(), rabbit_types:protocol()) ->
+ rabbit_types:encoded_content().
+-spec clear_encoded_content
+ (rabbit_types:content()) ->
+ rabbit_types:unencoded_content().
+-spec map_exception
+ (rabbit_channel:channel_number(), rabbit_types:amqp_error() | any(),
+ rabbit_types:protocol()) ->
+ {rabbit_channel:channel_number(),
+ rabbit_framing:amqp_method_record()}.
+
+%%----------------------------------------------------------------------------
+
+build_simple_method_frame(ChannelInt, MethodRecord, Protocol) ->
+ MethodFields = Protocol:encode_method_fields(MethodRecord),
+ MethodName = rabbit_misc:method_record_type(MethodRecord),
+ {ClassId, MethodId} = Protocol:method_id(MethodName),
+ create_frame(1, ChannelInt, [<<ClassId:16, MethodId:16>>, MethodFields]).
+
+build_simple_content_frames(ChannelInt, Content, FrameMax, Protocol) ->
+ #content{class_id = ClassId,
+ properties_bin = ContentPropertiesBin,
+ payload_fragments_rev = PayloadFragmentsRev} =
+ ensure_content_encoded(Content, Protocol),
+ {BodySize, ContentFrames} =
+ build_content_frames(PayloadFragmentsRev, FrameMax, ChannelInt),
+ HeaderFrame = create_frame(2, ChannelInt,
+ [<<ClassId:16, 0:16, BodySize:64>>,
+ ContentPropertiesBin]),
+ [HeaderFrame | ContentFrames].
+
+build_content_frames(FragsRev, FrameMax, ChannelInt) ->
+ BodyPayloadMax = if FrameMax == 0 -> iolist_size(FragsRev);
+ true -> FrameMax - ?EMPTY_FRAME_SIZE
+ end,
+ build_content_frames(0, [], BodyPayloadMax, [],
+ lists:reverse(FragsRev), BodyPayloadMax, ChannelInt).
+
+build_content_frames(SizeAcc, FramesAcc, _FragSizeRem, [],
+ [], _BodyPayloadMax, _ChannelInt) ->
+ {SizeAcc, lists:reverse(FramesAcc)};
+build_content_frames(SizeAcc, FramesAcc, _FragSizeRem, [],
+ [<<>>], _BodyPayloadMax, _ChannelInt) ->
+ {SizeAcc, lists:reverse(FramesAcc)};
+build_content_frames(SizeAcc, FramesAcc, FragSizeRem, FragAcc,
+ Frags, BodyPayloadMax, ChannelInt)
+ when FragSizeRem == 0 orelse Frags == [] ->
+ Frame = create_frame(3, ChannelInt, lists:reverse(FragAcc)),
+ FrameSize = BodyPayloadMax - FragSizeRem,
+ build_content_frames(SizeAcc + FrameSize, [Frame | FramesAcc],
+ BodyPayloadMax, [], Frags, BodyPayloadMax, ChannelInt);
+build_content_frames(SizeAcc, FramesAcc, FragSizeRem, FragAcc,
+ [Frag | Frags], BodyPayloadMax, ChannelInt) ->
+ Size = size(Frag),
+ {NewFragSizeRem, NewFragAcc, NewFrags} =
+ if Size == 0 -> {FragSizeRem, FragAcc, Frags};
+ Size =< FragSizeRem -> {FragSizeRem - Size, [Frag | FragAcc], Frags};
+ true -> <<Head:FragSizeRem/binary, Tail/binary>> =
+ Frag,
+ {0, [Head | FragAcc], [Tail | Frags]}
+ end,
+ build_content_frames(SizeAcc, FramesAcc, NewFragSizeRem, NewFragAcc,
+ NewFrags, BodyPayloadMax, ChannelInt).
+
+build_heartbeat_frame() ->
+ create_frame(?FRAME_HEARTBEAT, 0, <<>>).
+
+create_frame(TypeInt, ChannelInt, Payload) ->
+ [<<TypeInt:8, ChannelInt:16, (iolist_size(Payload)):32>>, Payload,
+ ?FRAME_END].
+
+%% table_field_to_binary supports the AMQP 0-8/0-9 standard types, S,
+%% I, D, T and F, as well as the QPid extensions b, d, f, l, s, t, x,
+%% and V.
+table_field_to_binary({FName, T, V}) ->
+ [short_string_to_binary(FName) | field_value_to_binary(T, V)].
+
+field_value_to_binary(longstr, V) -> [$S | long_string_to_binary(V)];
+field_value_to_binary(signedint, V) -> [$I, <<V:32/signed>>];
+field_value_to_binary(decimal, V) -> {Before, After} = V,
+ [$D, Before, <<After:32>>];
+field_value_to_binary(timestamp, V) -> [$T, <<V:64>>];
+field_value_to_binary(table, V) -> [$F | table_to_binary(V)];
+field_value_to_binary(array, V) -> [$A | array_to_binary(V)];
+field_value_to_binary(byte, V) -> [$b, <<V:8/signed>>];
+field_value_to_binary(double, V) -> [$d, <<V:64/float>>];
+field_value_to_binary(float, V) -> [$f, <<V:32/float>>];
+field_value_to_binary(long, V) -> [$l, <<V:64/signed>>];
+field_value_to_binary(short, V) -> [$s, <<V:16/signed>>];
+field_value_to_binary(bool, V) -> [$t, if V -> 1; true -> 0 end];
+field_value_to_binary(binary, V) -> [$x | long_string_to_binary(V)];
+field_value_to_binary(unsignedbyte, V) -> [$B, <<V:8/unsigned>>];
+field_value_to_binary(unsignedshort, V) -> [$u, <<V:16/unsigned>>];
+field_value_to_binary(unsignedint, V) -> [$i, <<V:32/unsigned>>];
+field_value_to_binary(void, _V) -> [$V].
+
+table_to_binary(Table) when is_list(Table) ->
+ BinTable = generate_table_iolist(Table),
+ [<<(iolist_size(BinTable)):32>> | BinTable].
+
+array_to_binary(Array) when is_list(Array) ->
+ BinArray = generate_array_iolist(Array),
+ [<<(iolist_size(BinArray)):32>> | BinArray].
+
+generate_table(Table) when is_list(Table) ->
+ list_to_binary(generate_table_iolist(Table)).
+
+generate_table_iolist(Table) ->
+ lists:map(fun table_field_to_binary/1, Table).
+
+generate_array_iolist(Array) ->
+ lists:map(fun ({T, V}) -> field_value_to_binary(T, V) end, Array).
+
+short_string_to_binary(String) ->
+ Len = string_length(String),
+ if Len < 256 -> [<<Len:8>>, String];
+ true -> exit(content_properties_shortstr_overflow)
+ end.
+
+long_string_to_binary(String) ->
+ Len = string_length(String),
+ [<<Len:32>>, String].
+
+string_length(String) when is_binary(String) -> size(String);
+string_length(String) -> length(String).
+
+check_empty_frame_size() ->
+ %% Intended to ensure that EMPTY_FRAME_SIZE is defined correctly.
+ case iolist_size(create_frame(?FRAME_BODY, 0, <<>>)) of
+ ?EMPTY_FRAME_SIZE -> ok;
+ ComputedSize -> exit({incorrect_empty_frame_size,
+ ComputedSize, ?EMPTY_FRAME_SIZE})
+ end.
+
+ensure_content_encoded(Content = #content{properties_bin = PropBin,
+ protocol = Protocol}, Protocol)
+ when PropBin =/= none ->
+ Content;
+ensure_content_encoded(Content = #content{properties = none,
+ properties_bin = PropBin,
+ protocol = Protocol}, Protocol1)
+ when PropBin =/= none ->
+ Props = Protocol:decode_properties(Content#content.class_id, PropBin),
+ Content#content{properties = Props,
+ properties_bin = Protocol1:encode_properties(Props),
+ protocol = Protocol1};
+ensure_content_encoded(Content = #content{properties = Props}, Protocol)
+ when Props =/= none ->
+ Content#content{properties_bin = Protocol:encode_properties(Props),
+ protocol = Protocol}.
+
+clear_encoded_content(Content = #content{properties_bin = none,
+ protocol = none}) ->
+ Content;
+clear_encoded_content(Content = #content{properties = none}) ->
+ %% Only clear when we can rebuild the properties_bin later in
+ %% accordance to the content record definition comment - maximum
+ %% one of properties and properties_bin can be 'none'
+ Content;
+clear_encoded_content(Content = #content{}) ->
+ Content#content{properties_bin = none, protocol = none}.
+
+%% NB: this function is also used by the Erlang client
+map_exception(Channel, Reason, Protocol) ->
+ {SuggestedClose, ReplyCode, ReplyText, FailedMethod} =
+ lookup_amqp_exception(Reason, Protocol),
+ {ClassId, MethodId} = case FailedMethod of
+ {_, _} -> FailedMethod;
+ none -> {0, 0};
+ _ -> Protocol:method_id(FailedMethod)
+ end,
+ case SuggestedClose orelse (Channel == 0) of
+ true -> {0, #'connection.close'{reply_code = ReplyCode,
+ reply_text = ReplyText,
+ class_id = ClassId,
+ method_id = MethodId}};
+ false -> {Channel, #'channel.close'{reply_code = ReplyCode,
+ reply_text = ReplyText,
+ class_id = ClassId,
+ method_id = MethodId}}
+ end.
+
+lookup_amqp_exception(#amqp_error{name = Name,
+ explanation = Expl,
+ method = Method},
+ Protocol) ->
+ {ShouldClose, Code, Text} = Protocol:lookup_amqp_exception(Name),
+ ExplBin = amqp_exception_explanation(Text, Expl),
+ {ShouldClose, Code, ExplBin, Method};
+lookup_amqp_exception(Other, Protocol) ->
+ rabbit_log:warning("Non-AMQP exit reason '~p'~n", [Other]),
+ {ShouldClose, Code, Text} = Protocol:lookup_amqp_exception(internal_error),
+ {ShouldClose, Code, Text, none}.
+
+amqp_exception_explanation(Text, Expl) ->
+ ExplBin = list_to_binary(Expl),
+ CompleteTextBin = <<Text/binary, " - ", ExplBin/binary>>,
+ if size(CompleteTextBin) > 255 -> <<CompleteTextBin:252/binary, "...">>;
+ true -> CompleteTextBin
+ end.
diff --git a/deps/rabbit_common/src/rabbit_binary_parser.erl b/deps/rabbit_common/src/rabbit_binary_parser.erl
new file mode 100644
index 0000000000..478b0f0cd2
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_binary_parser.erl
@@ -0,0 +1,172 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_binary_parser).
+
+-include("rabbit.hrl").
+
+-export([parse_table/1]).
+-export([ensure_content_decoded/1, clear_decoded_content/1]).
+-export([validate_utf8/1, assert_utf8/1]).
+
+%%----------------------------------------------------------------------------
+
+-spec parse_table(binary()) -> rabbit_framing:amqp_table().
+-spec ensure_content_decoded
+ (rabbit_types:content()) ->
+ rabbit_types:decoded_content().
+-spec clear_decoded_content
+ (rabbit_types:content()) ->
+ rabbit_types:undecoded_content().
+-spec validate_utf8(binary()) -> 'ok' | 'error'.
+-spec assert_utf8(binary()) -> 'ok'.
+
+%%----------------------------------------------------------------------------
+
+%% parse_table supports the AMQP 0-8/0-9 standard types, S, I, D, T
+%% and F, as well as the QPid extensions b, d, f, l, s, t, x, and V.
+
+-define(SIMPLE_PARSE_TABLE(BType, Pattern, RType),
+ parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ BType, Pattern, Rest/binary>>) ->
+ [{NameString, RType, Value} | parse_table(Rest)]).
+
+%% Note that we try to put these in approximately the order we expect
+%% to hit them, that's why the empty binary is half way through.
+
+parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ $S, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{NameString, longstr, Value} | parse_table(Rest)];
+
+?SIMPLE_PARSE_TABLE($T, Value:64/unsigned, timestamp);
+
+parse_table(<<>>) ->
+ [];
+
+?SIMPLE_PARSE_TABLE($b, Value:8/signed, byte);
+?SIMPLE_PARSE_TABLE($B, Value:8/unsigned, unsignedbyte);
+
+?SIMPLE_PARSE_TABLE($s, Value:16/signed, short);
+?SIMPLE_PARSE_TABLE($u, Value:16/unsigned, unsignedshort);
+
+?SIMPLE_PARSE_TABLE($I, Value:32/signed, signedint);
+?SIMPLE_PARSE_TABLE($i, Value:32/unsigned, unsignedint);
+
+?SIMPLE_PARSE_TABLE($d, Value:64/float, double);
+?SIMPLE_PARSE_TABLE($f, Value:32/float, float);
+
+%% yes, both 'l' and 'L' fields are decoded to 64-bit signed values;
+%% see https://github.com/rabbitmq/rabbitmq-server/issues/1093#issuecomment-276351183,
+%% https://www.rabbitmq.com/amqp-0-9-1-errata.html, and section
+%% 4.2.1 of the spec for details.
+?SIMPLE_PARSE_TABLE($l, Value:64/signed, long);
+?SIMPLE_PARSE_TABLE($L, Value:64/signed, long);
+
+
+parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ $t, Value:8/unsigned, Rest/binary>>) ->
+ [{NameString, bool, (Value /= 0)} | parse_table(Rest)];
+
+parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ $D, Before:8/unsigned, After:32/unsigned, Rest/binary>>) ->
+ [{NameString, decimal, {Before, After}} | parse_table(Rest)];
+
+parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ $F, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{NameString, table, parse_table(Value)} | parse_table(Rest)];
+
+parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ $A, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{NameString, array, parse_array(Value)} | parse_table(Rest)];
+
+parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ $x, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{NameString, binary, Value} | parse_table(Rest)];
+
+parse_table(<<NLen:8/unsigned, NameString:NLen/binary,
+ $V, Rest/binary>>) ->
+ [{NameString, void, undefined} | parse_table(Rest)].
+
+-define(SIMPLE_PARSE_ARRAY(BType, Pattern, RType),
+ parse_array(<<BType, Pattern, Rest/binary>>) ->
+ [{RType, Value} | parse_array(Rest)]).
+
+parse_array(<<$S, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{longstr, Value} | parse_array(Rest)];
+
+?SIMPLE_PARSE_ARRAY($T, Value:64/unsigned, timestamp);
+
+parse_array(<<>>) ->
+ [];
+
+?SIMPLE_PARSE_ARRAY($b, Value:8/signed, byte);
+?SIMPLE_PARSE_ARRAY($B, Value:8/unsigned, unsignedbyte);
+
+?SIMPLE_PARSE_ARRAY($s, Value:16/signed, short);
+?SIMPLE_PARSE_ARRAY($u, Value:16/unsigned, unsignedshort);
+
+?SIMPLE_PARSE_ARRAY($I, Value:32/signed, signedint);
+?SIMPLE_PARSE_ARRAY($i, Value:32/unsigned, unsignedint);
+
+?SIMPLE_PARSE_ARRAY($d, Value:64/float, double);
+?SIMPLE_PARSE_ARRAY($f, Value:32/float, float);
+
+?SIMPLE_PARSE_ARRAY($l, Value:64/signed, long);
+?SIMPLE_PARSE_ARRAY($L, Value:64/signed, long);
+
+
+parse_array(<<$t, Value:8/unsigned, Rest/binary>>) ->
+ [{bool, (Value /= 0)} | parse_array(Rest)];
+
+parse_array(<<$D, Before:8/unsigned, After:32/unsigned, Rest/binary>>) ->
+ [{decimal, {Before, After}} | parse_array(Rest)];
+
+parse_array(<<$F, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{table, parse_table(Value)} | parse_array(Rest)];
+
+parse_array(<<$A, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{array, parse_array(Value)} | parse_array(Rest)];
+
+parse_array(<<$x, VLen:32/unsigned, Value:VLen/binary, Rest/binary>>) ->
+ [{binary, Value} | parse_array(Rest)];
+
+parse_array(<<$V, Rest/binary>>) ->
+ [{void, undefined} | parse_array(Rest)].
+
+ensure_content_decoded(Content = #content{properties = Props})
+ when Props =/= none ->
+ Content;
+ensure_content_decoded(Content = #content{properties_bin = PropBin,
+ protocol = Protocol})
+ when PropBin =/= none ->
+ Content#content{properties = Protocol:decode_properties(
+ Content#content.class_id, PropBin)}.
+
+clear_decoded_content(Content = #content{properties = none}) ->
+ Content;
+clear_decoded_content(Content = #content{properties_bin = none}) ->
+ %% Only clear when we can rebuild the properties later in
+ %% accordance to the content record definition comment - maximum
+ %% one of properties and properties_bin can be 'none'
+ Content;
+clear_decoded_content(Content = #content{}) ->
+ Content#content{properties = none}.
+
+assert_utf8(B) ->
+ case validate_utf8(B) of
+ ok -> ok;
+ error -> rabbit_misc:protocol_error(
+ frame_error, "Malformed UTF-8 in shortstr", [])
+ end.
+
+validate_utf8(Bin) ->
+ try
+ _ = xmerl_ucs:from_utf8(Bin),
+ ok
+ catch exit:{ucs, _} ->
+ error
+ end.
diff --git a/deps/rabbit_common/src/rabbit_cert_info.erl b/deps/rabbit_common/src/rabbit_cert_info.erl
new file mode 100644
index 0000000000..08e6f03c6c
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_cert_info.erl
@@ -0,0 +1,270 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_cert_info).
+
+-include_lib("public_key/include/public_key.hrl").
+
+-export([issuer/1,
+ subject/1,
+ subject_alternative_names/1,
+ validity/1,
+ subject_items/2,
+ extensions/1
+]).
+
+%%--------------------------------------------------------------------------
+
+-export_type([certificate/0]).
+
+-type certificate() :: binary().
+
+%%--------------------------------------------------------------------------
+%% High-level functions used by reader
+%%--------------------------------------------------------------------------
+
+%% Return a string describing the certificate's issuer.
+-spec issuer(certificate()) -> string().
+
+issuer(Cert) ->
+ cert_info(fun(#'OTPCertificate' {
+ tbsCertificate = #'OTPTBSCertificate' {
+ issuer = Issuer }}) ->
+ format_rdn_sequence(Issuer)
+ end, Cert).
+
+%% Return a string describing the certificate's subject, as per RFC4514.
+-spec subject(certificate()) -> string().
+
+subject(Cert) ->
+ cert_info(fun(#'OTPCertificate' {
+ tbsCertificate = #'OTPTBSCertificate' {
+ subject = Subject }}) ->
+ format_rdn_sequence(Subject)
+ end, Cert).
+
+%% Return the parts of the certificate's subject.
+-spec subject_items
+ (certificate(), tuple()) -> [string()] | 'not_found'.
+
+subject_items(Cert, Type) ->
+ cert_info(fun(#'OTPCertificate' {
+ tbsCertificate = #'OTPTBSCertificate' {
+ subject = Subject }}) ->
+ find_by_type(Type, Subject)
+ end, Cert).
+
+-spec extensions(certificate()) -> [#'Extension'{}].
+extensions(Cert) ->
+ cert_info(fun(#'OTPCertificate' {
+ tbsCertificate = #'OTPTBSCertificate' {
+ extensions = Extensions }}) ->
+ Extensions
+ end, Cert).
+
+-spec subject_alternative_names(certificate()) -> [{atom(), string()}].
+subject_alternative_names(Cert) ->
+ Extensions = extensions(Cert),
+ try lists:keyfind(?'id-ce-subjectAltName', #'Extension'.extnID, Extensions) of
+ false -> [];
+ #'Extension'{extnValue = Val} -> Val
+ catch _:_ -> []
+ end.
+
+%% Return a string describing the certificate's validity.
+-spec validity(certificate()) -> string().
+
+validity(Cert) ->
+ cert_info(fun(#'OTPCertificate' {
+ tbsCertificate = #'OTPTBSCertificate' {
+ validity = {'Validity', Start, End} }}) ->
+ rabbit_misc:format("~s - ~s", [format_asn1_value(Start),
+ format_asn1_value(End)])
+ end, Cert).
+
+%%--------------------------------------------------------------------------
+
+cert_info(F, Cert) ->
+ F(public_key:pkix_decode_cert(Cert, otp)).
+
+find_by_type(Type, {rdnSequence, RDNs}) ->
+ case [V || #'AttributeTypeAndValue'{type = T, value = V}
+ <- lists:flatten(RDNs),
+ T == Type] of
+ [] -> not_found;
+ L -> [format_asn1_value(V) || V <- L]
+ end.
+
+%%--------------------------------------------------------------------------
+%% Formatting functions
+%%--------------------------------------------------------------------------
+
+%% Format and rdnSequence as a RFC4514 subject string.
+format_rdn_sequence({rdnSequence, Seq}) ->
+ string:join(lists:reverse([format_complex_rdn(RDN) || RDN <- Seq]), ",").
+
+%% Format an RDN set.
+format_complex_rdn(RDNs) ->
+ string:join([format_rdn(RDN) || RDN <- RDNs], "+").
+
+%% Format an RDN. If the type name is unknown, use the dotted decimal
+%% representation. See RFC4514, section 2.3.
+format_rdn(#'AttributeTypeAndValue'{type = T, value = V}) ->
+ FV = escape_rdn_value(format_asn1_value(V)),
+ Fmts = [{?'id-at-surname' , "SN"},
+ {?'id-at-givenName' , "GIVENNAME"},
+ {?'id-at-initials' , "INITIALS"},
+ {?'id-at-generationQualifier' , "GENERATIONQUALIFIER"},
+ {?'id-at-commonName' , "CN"},
+ {?'id-at-localityName' , "L"},
+ {?'id-at-stateOrProvinceName' , "ST"},
+ {?'id-at-organizationName' , "O"},
+ {?'id-at-organizationalUnitName' , "OU"},
+ {?'id-at-title' , "TITLE"},
+ {?'id-at-countryName' , "C"},
+ {?'id-at-serialNumber' , "SERIALNUMBER"},
+ {?'id-at-pseudonym' , "PSEUDONYM"},
+ {?'id-domainComponent' , "DC"},
+ {?'id-emailAddress' , "EMAILADDRESS"},
+ {?'street-address' , "STREET"},
+ {{0,9,2342,19200300,100,1,1} , "UID"}], %% Not in public_key.hrl
+ case proplists:lookup(T, Fmts) of
+ {_, Fmt} ->
+ rabbit_misc:format(Fmt ++ "=~s", [FV]);
+ none when is_tuple(T) ->
+ TypeL = [rabbit_misc:format("~w", [X]) || X <- tuple_to_list(T)],
+ rabbit_misc:format("~s=~s", [string:join(TypeL, "."), FV]);
+ none ->
+ rabbit_misc:format("~p=~s", [T, FV])
+ end.
+
+%% Escape a string as per RFC4514.
+escape_rdn_value(V) ->
+ escape_rdn_value(V, start).
+
+escape_rdn_value([], _) ->
+ [];
+escape_rdn_value([C | S], start) when C =:= $ ; C =:= $# ->
+ [$\\, C | escape_rdn_value(S, middle)];
+escape_rdn_value(S, start) ->
+ escape_rdn_value(S, middle);
+escape_rdn_value([$ ], middle) ->
+ [$\\, $ ];
+escape_rdn_value([C | S], middle) when C =:= $"; C =:= $+; C =:= $,; C =:= $;;
+ C =:= $<; C =:= $>; C =:= $\\ ->
+ [$\\, C | escape_rdn_value(S, middle)];
+escape_rdn_value([C | S], middle) when C < 32 ; C >= 126 ->
+ %% Of ASCII characters only U+0000 needs escaping, but for display
+ %% purposes it's handy to escape all non-printable chars. All non-ASCII
+ %% characters get converted to UTF-8 sequences and then escaped. We've
+ %% already got a UTF-8 sequence here, so just escape it.
+ rabbit_misc:format("\\~2.16.0B", [C]) ++ escape_rdn_value(S, middle);
+escape_rdn_value([C | S], middle) ->
+ [C | escape_rdn_value(S, middle)].
+
+%% Get the string representation of an OTPCertificate field.
+format_asn1_value({ST, S}) when ST =:= teletexString; ST =:= printableString;
+ ST =:= universalString; ST =:= utf8String;
+ ST =:= bmpString ->
+ format_directory_string(ST, S);
+format_asn1_value({utcTime, [Y1, Y2, M1, M2, D1, D2, H1, H2,
+ Min1, Min2, S1, S2, $Z]}) ->
+ rabbit_misc:format("20~c~c-~c~c-~c~cT~c~c:~c~c:~c~cZ",
+ [Y1, Y2, M1, M2, D1, D2, H1, H2, Min1, Min2, S1, S2]);
+%% We appear to get an untagged value back for an ia5string
+%% (e.g. domainComponent).
+format_asn1_value(V) when is_list(V) ->
+ V;
+format_asn1_value(V) when is_binary(V) ->
+ %% OTP does not decode some values when combined with an unknown
+ %% type. That's probably wrong, so as a last ditch effort let's
+ %% try manually decoding. 'DirectoryString' is semi-arbitrary -
+ %% but it is the type which covers the various string types we
+ %% handle below.
+ try
+ {ST, S} = public_key:der_decode('DirectoryString', V),
+ format_directory_string(ST, S)
+ catch _:_ ->
+ rabbit_misc:format("~p", [V])
+ end;
+format_asn1_value(V) ->
+ rabbit_misc:format("~p", [V]).
+
+%% DirectoryString { INTEGER : maxSize } ::= CHOICE {
+%% teletexString TeletexString (SIZE (1..maxSize)),
+%% printableString PrintableString (SIZE (1..maxSize)),
+%% bmpString BMPString (SIZE (1..maxSize)),
+%% universalString UniversalString (SIZE (1..maxSize)),
+%% uTF8String UTF8String (SIZE (1..maxSize)) }
+%%
+%% Precise definitions of printable / teletexString are hard to come
+%% by. This is what I reconstructed:
+%%
+%% printableString:
+%% "intended to represent the limited character sets available to
+%% mainframe input terminals"
+%% A-Z a-z 0-9 ' ( ) + , - . / : = ? [space]
+%% https://msdn.microsoft.com/en-us/library/bb540814(v=vs.85).aspx
+%%
+%% teletexString:
+%% "a sizable volume of software in the world treats TeletexString
+%% (T61String) as a simple 8-bit string with mostly Windows Latin 1
+%% (superset of iso-8859-1) encoding"
+%% https://www.mail-archive.com/asn1@asn1.org/msg00460.html
+%%
+%% (However according to that link X.680 actually defines
+%% TeletexString in some much more involved and crazy way. I suggest
+%% we treat it as ISO-8859-1 since Erlang does not support Windows
+%% Latin 1).
+%%
+%% bmpString:
+%% UCS-2 according to RFC 3641. Hence cannot represent Unicode
+%% characters above 65535 (outside the "Basic Multilingual Plane").
+%%
+%% universalString:
+%% UCS-4 according to RFC 3641.
+%%
+%% utf8String:
+%% UTF-8 according to RFC 3641.
+%%
+%% Within Rabbit we assume UTF-8 encoding. Since printableString is a
+%% subset of ASCII it is also a subset of UTF-8. The others need
+%% converting. Fortunately since the Erlang SSL library does the
+%% decoding for us (albeit into a weird format, see below), we just
+%% need to handle encoding into UTF-8. Note also that utf8Strings come
+%% back as binary.
+%%
+%% Note for testing: the default Ubuntu configuration for openssl will
+%% only create printableString or teletexString types no matter what
+%% you do. Edit string_mask in the [req] section of
+%% /etc/ssl/openssl.cnf to change this (see comments there). You
+%% probably also need to set utf8 = yes to get it to accept UTF-8 on
+%% the command line. Also note I could not get openssl to generate a
+%% universalString.
+
+format_directory_string(printableString, S) -> S;
+format_directory_string(teletexString, S) -> utf8_list_from(S);
+format_directory_string(bmpString, S) -> utf8_list_from(S);
+format_directory_string(universalString, S) -> utf8_list_from(S);
+format_directory_string(utf8String, S) -> binary_to_list(S).
+
+utf8_list_from(S) ->
+ binary_to_list(
+ unicode:characters_to_binary(flatten_ssl_list(S), utf32, utf8)).
+
+%% The Erlang SSL implementation invents its own representation for
+%% non-ascii strings - looking like [97,{0,0,3,187}] (that's LATIN
+%% SMALL LETTER A followed by GREEK SMALL LETTER LAMDA). We convert
+%% this into a list of unicode characters, which we can tell
+%% unicode:characters_to_binary is utf32.
+
+flatten_ssl_list(L) -> [flatten_ssl_list_item(I) || I <- L].
+
+flatten_ssl_list_item({A, B, C, D}) ->
+ A * (1 bsl 24) + B * (1 bsl 16) + C * (1 bsl 8) + D;
+flatten_ssl_list_item(N) when is_number (N) ->
+ N.
diff --git a/deps/rabbit_common/src/rabbit_channel_common.erl b/deps/rabbit_common/src/rabbit_channel_common.erl
new file mode 100644
index 0000000000..a21e17b2e7
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_channel_common.erl
@@ -0,0 +1,25 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_channel_common).
+
+-export([do/2, do/3, do_flow/3, ready_for_close/1]).
+
+do(Pid, Method) ->
+ do(Pid, Method, none).
+
+do(Pid, Method, Content) ->
+ gen_server2:cast(Pid, {method, Method, Content, noflow}).
+
+do_flow(Pid, Method, Content) ->
+ %% Here we are tracking messages sent by the rabbit_reader
+ %% process. We are accessing the rabbit_reader process dictionary.
+ credit_flow:send(Pid),
+ gen_server2:cast(Pid, {method, Method, Content, flow}).
+
+ready_for_close(Pid) ->
+ gen_server2:cast(Pid, ready_for_close).
diff --git a/deps/rabbit_common/src/rabbit_command_assembler.erl b/deps/rabbit_common/src/rabbit_command_assembler.erl
new file mode 100644
index 0000000000..ea6b19d083
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_command_assembler.erl
@@ -0,0 +1,124 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_command_assembler).
+-include("rabbit_framing.hrl").
+-include("rabbit.hrl").
+
+-export([analyze_frame/3, init/1, process/2]).
+
+%%----------------------------------------------------------------------------
+
+%%----------------------------------------------------------------------------
+
+-export_type([frame/0]).
+
+-type frame_type() :: ?FRAME_METHOD | ?FRAME_HEADER | ?FRAME_BODY |
+ ?FRAME_OOB_METHOD | ?FRAME_OOB_HEADER | ?FRAME_OOB_BODY |
+ ?FRAME_TRACE | ?FRAME_HEARTBEAT.
+-type protocol() :: rabbit_framing:protocol().
+-type method() :: rabbit_framing:amqp_method_record().
+-type class_id() :: rabbit_framing:amqp_class_id().
+-type weight() :: non_neg_integer().
+-type body_size() :: non_neg_integer().
+-type content() :: rabbit_types:undecoded_content().
+
+-type frame() ::
+ {'method', rabbit_framing:amqp_method_name(), binary()} |
+ {'content_header', class_id(), weight(), body_size(), binary()} |
+ {'content_body', binary()}.
+
+-type state() ::
+ {'method', protocol()} |
+ {'content_header', method(), class_id(), protocol()} |
+ {'content_body', method(), body_size(), class_id(), protocol()}.
+
+-spec analyze_frame(frame_type(), binary(), protocol()) ->
+ frame() | 'heartbeat' | 'error'.
+
+-spec init(protocol()) -> {ok, state()}.
+-spec process(frame(), state()) ->
+ {ok, state()} |
+ {ok, method(), state()} |
+ {ok, method(), content(), state()} |
+ {error, rabbit_types:amqp_error()}.
+
+%%--------------------------------------------------------------------
+
+analyze_frame(?FRAME_METHOD,
+ <<ClassId:16, MethodId:16, MethodFields/binary>>,
+ Protocol) ->
+ MethodName = Protocol:lookup_method_name({ClassId, MethodId}),
+ {method, MethodName, MethodFields};
+analyze_frame(?FRAME_HEADER,
+ <<ClassId:16, Weight:16, BodySize:64, Properties/binary>>,
+ _Protocol) ->
+ {content_header, ClassId, Weight, BodySize, Properties};
+analyze_frame(?FRAME_BODY, Body, _Protocol) ->
+ {content_body, Body};
+analyze_frame(?FRAME_HEARTBEAT, <<>>, _Protocol) ->
+ heartbeat;
+analyze_frame(_Type, _Body, _Protocol) ->
+ error.
+
+init(Protocol) -> {ok, {method, Protocol}}.
+
+process({method, MethodName, FieldsBin}, {method, Protocol}) ->
+ try
+ Method = Protocol:decode_method_fields(MethodName, FieldsBin),
+ case Protocol:method_has_content(MethodName) of
+ true -> {ClassId, _MethodId} = Protocol:method_id(MethodName),
+ {ok, {content_header, Method, ClassId, Protocol}};
+ false -> {ok, Method, {method, Protocol}}
+ end
+ catch exit:#amqp_error{} = Reason -> {error, Reason}
+ end;
+process(_Frame, {method, _Protocol}) ->
+ unexpected_frame("expected method frame, "
+ "got non method frame instead", [], none);
+process({content_header, ClassId, 0, 0, PropertiesBin},
+ {content_header, Method, ClassId, Protocol}) ->
+ Content = empty_content(ClassId, PropertiesBin, Protocol),
+ {ok, Method, Content, {method, Protocol}};
+process({content_header, ClassId, 0, BodySize, PropertiesBin},
+ {content_header, Method, ClassId, Protocol}) ->
+ Content = empty_content(ClassId, PropertiesBin, Protocol),
+ {ok, {content_body, Method, BodySize, Content, Protocol}};
+process({content_header, HeaderClassId, 0, _BodySize, _PropertiesBin},
+ {content_header, Method, ClassId, _Protocol}) ->
+ unexpected_frame("expected content header for class ~w, "
+ "got one for class ~w instead",
+ [ClassId, HeaderClassId], Method);
+process(_Frame, {content_header, Method, ClassId, _Protocol}) ->
+ unexpected_frame("expected content header for class ~w, "
+ "got non content header frame instead", [ClassId], Method);
+process({content_body, FragmentBin},
+ {content_body, Method, RemainingSize,
+ Content = #content{payload_fragments_rev = Fragments}, Protocol}) ->
+ NewContent = Content#content{
+ payload_fragments_rev = [FragmentBin | Fragments]},
+ case RemainingSize - size(FragmentBin) of
+ 0 -> {ok, Method, NewContent, {method, Protocol}};
+ Sz -> {ok, {content_body, Method, Sz, NewContent, Protocol}}
+ end;
+process(_Frame, {content_body, Method, _RemainingSize, _Content, _Protocol}) ->
+ unexpected_frame("expected content body, "
+ "got non content body frame instead", [], Method).
+
+%%--------------------------------------------------------------------
+
+empty_content(ClassId, PropertiesBin, Protocol) ->
+ #content{class_id = ClassId,
+ properties = none,
+ properties_bin = PropertiesBin,
+ protocol = Protocol,
+ payload_fragments_rev = []}.
+
+unexpected_frame(Format, Params, Method) when is_atom(Method) ->
+ {error, rabbit_misc:amqp_error(unexpected_frame, Format, Params, Method)};
+unexpected_frame(Format, Params, Method) ->
+ unexpected_frame(Format, Params, rabbit_misc:method_record_type(Method)).
diff --git a/deps/rabbit_common/src/rabbit_control_misc.erl b/deps/rabbit_common/src/rabbit_control_misc.erl
new file mode 100644
index 0000000000..0fff88a2cd
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_control_misc.erl
@@ -0,0 +1,179 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_control_misc).
+
+-export([emitting_map/4, emitting_map/5, emitting_map_with_exit_handler/4,
+ emitting_map_with_exit_handler/5, wait_for_info_messages/6,
+ spawn_emitter_caller/7, await_emitters_termination/1,
+ print_cmd_result/2]).
+
+-spec emitting_map(pid(), reference(), fun(), list()) -> 'ok'.
+-spec emitting_map(pid(), reference(), fun(), list(), atom()) -> 'ok'.
+-spec emitting_map_with_exit_handler
+ (pid(), reference(), fun(), list()) -> 'ok'.
+-spec emitting_map_with_exit_handler
+ (pid(), reference(), fun(), list(), 'continue') -> 'ok'.
+
+-type fold_fun() :: fun((Item :: term(), AccIn :: term()) -> AccOut :: term()).
+
+-spec wait_for_info_messages(pid(), reference(), fold_fun(), InitialAcc, timeout(), non_neg_integer()) -> OK | Err when
+ InitialAcc :: term(), Acc :: term(), OK :: {ok, Acc}, Err :: {error, term()}.
+-spec spawn_emitter_caller(node(), module(), atom(), [term()], reference(), pid(), timeout()) -> 'ok'.
+-spec await_emitters_termination([pid()]) -> 'ok'.
+
+-spec print_cmd_result(atom(), term()) -> 'ok'.
+
+emitting_map(AggregatorPid, Ref, Fun, List) ->
+ emitting_map(AggregatorPid, Ref, Fun, List, continue),
+ AggregatorPid ! {Ref, finished},
+ ok.
+
+emitting_map(AggregatorPid, Ref, Fun, List, continue) ->
+ _ = emitting_map0(AggregatorPid, Ref, Fun, List, fun step/4),
+ ok.
+
+emitting_map_with_exit_handler(AggregatorPid, Ref, Fun, List) ->
+ emitting_map_with_exit_handler(AggregatorPid, Ref, Fun, List, continue),
+ AggregatorPid ! {Ref, finished},
+ ok.
+
+emitting_map_with_exit_handler(AggregatorPid, Ref, Fun, List, continue) ->
+ _ = emitting_map0(AggregatorPid, Ref, Fun, List, fun step_with_exit_handler/4),
+ ok.
+
+emitting_map0(AggregatorPid, Ref, Fun, List, StepFun) ->
+ [StepFun(AggregatorPid, Ref, Fun, Item) || Item <- List].
+
+step(AggregatorPid, Ref, Fun, Item) ->
+ AggregatorPid ! {Ref, Fun(Item), continue},
+ ok.
+
+step_with_exit_handler(AggregatorPid, Ref, Fun, Item) ->
+ Noop = make_ref(),
+ case rabbit_misc:with_exit_handler(
+ fun () -> Noop end,
+ fun () -> Fun(Item) end) of
+ Noop ->
+ ok;
+ Res ->
+ AggregatorPid ! {Ref, Res, continue},
+ ok
+ end.
+
+%% Invokes RPC for async info collection in separate (but linked to
+%% the caller) process. Separate process waits for RPC to finish and
+%% in case of errors sends them in wait_for_info_messages/5-compatible
+%% form to aggregator process. Calling process is then expected to
+%% do blocking call of wait_for_info_messages/5.
+%%
+%% Remote function MUST use calls to emitting_map/4 (and other
+%% emitting_map's) to properly deliver requested information to an
+%% aggregator process.
+%%
+%% If for performance reasons several parallel emitting_map's need to
+%% be run, remote function MUST NOT return until all this
+%% emitting_map's are done. And during all this time remote RPC
+%% process MUST be linked to emitting
+%% processes. await_emitters_termination/1 helper can be used as a
+%% last statement of remote function to ensure this behaviour.
+spawn_emitter_caller(Node, Mod, Fun, Args, Ref, Pid, Timeout) ->
+ _ = spawn_monitor(
+ fun () ->
+ case rpc_call_emitter(Node, Mod, Fun, Args, Ref, Pid, Timeout) of
+ {error, _} = Error ->
+ Pid ! {Ref, error, Error};
+ {bad_argument, _} = Error ->
+ Pid ! {Ref, error, Error};
+ {badrpc, _} = Error ->
+ Pid ! {Ref, error, Error};
+ _ ->
+ ok
+ end
+ end),
+ ok.
+
+rpc_call_emitter(Node, Mod, Fun, Args, Ref, Pid, Timeout) ->
+ rabbit_misc:rpc_call(Node, Mod, Fun, Args++[Ref, Pid], Timeout).
+
+%% Aggregator process expects correct numbers of explicits ACKs about
+%% finished emission process. While everything is linked, we still
+%% need somehow to wait for termination of all emitters before
+%% returning from RPC call - otherwise links will be just broken with
+%% reason 'normal' and we can miss some errors, and subsequently
+%% hang.
+await_emitters_termination(Pids) ->
+ Monitors = [erlang:monitor(process, Pid) || Pid <- Pids],
+ collect_monitors(Monitors).
+
+collect_monitors([]) ->
+ ok;
+collect_monitors([Monitor|Rest]) ->
+ receive
+ {'DOWN', Monitor, process, _Pid, normal} ->
+ collect_monitors(Rest);
+ {'DOWN', Monitor, process, _Pid, noproc} ->
+ %% There is a link and a monitor to a process. Matching
+ %% this clause means that process has gracefully
+ %% terminated even before we've started monitoring.
+ collect_monitors(Rest);
+ {'DOWN', _, process, Pid, Reason} when Reason =/= normal,
+ Reason =/= noproc ->
+ exit({emitter_exit, Pid, Reason})
+ end.
+
+%% Wait for result of one or more calls to emitting_map-family
+%% functions.
+%%
+%% Number of expected acknowledgments is specified by ChunkCount
+%% argument. Most common usage will be with ChunkCount equals to
+%% number of live nodes, but it's not mandatory - thus more generic
+%% name of 'ChunkCount' was chosen.
+wait_for_info_messages(Pid, Ref, Fun, Acc0, Timeout, ChunkCount) ->
+ _ = notify_if_timeout(Pid, Ref, Timeout),
+ wait_for_info_messages(Ref, Fun, Acc0, ChunkCount).
+
+wait_for_info_messages(Ref, Fun, Acc0, ChunksLeft) ->
+ receive
+ {Ref, finished} when ChunksLeft =:= 1 ->
+ {ok, Acc0};
+ {Ref, finished} ->
+ wait_for_info_messages(Ref, Fun, Acc0, ChunksLeft - 1);
+ {Ref, {timeout, T}} ->
+ exit({error, {timeout, (T / 1000)}});
+ {Ref, []} ->
+ wait_for_info_messages(Ref, Fun, Acc0, ChunksLeft);
+ {Ref, Result, continue} ->
+ wait_for_info_messages(Ref, Fun, Fun(Result, Acc0), ChunksLeft);
+ {Ref, error, Error} ->
+ {error, simplify_emission_error(Error)};
+ {'DOWN', _MRef, process, _Pid, normal} ->
+ wait_for_info_messages(Ref, Fun, Acc0, ChunksLeft);
+ {'DOWN', _MRef, process, _Pid, Reason} ->
+ {error, simplify_emission_error(Reason)};
+ _Msg ->
+ wait_for_info_messages(Ref, Fun, Acc0, ChunksLeft)
+ end.
+
+simplify_emission_error({badrpc, {'EXIT', {{nocatch, EmissionError}, _Stacktrace}}}) ->
+ EmissionError;
+simplify_emission_error({{nocatch, EmissionError}, _Stacktrace}) ->
+ EmissionError;
+simplify_emission_error({error, _} = Error) ->
+ Error;
+simplify_emission_error({bad_argument, _} = Error) ->
+ Error;
+simplify_emission_error(Anything) ->
+ {error, Anything}.
+
+notify_if_timeout(_, _, infinity) ->
+ ok;
+notify_if_timeout(Pid, Ref, Timeout) ->
+ erlang:send_after(Timeout, Pid, {Ref, {timeout, Timeout}}).
+
+print_cmd_result(authenticate_user, _Result) -> io:format("Success~n");
+print_cmd_result(join_cluster, already_member) -> io:format("The node is already a member of this cluster~n").
diff --git a/deps/rabbit_common/src/rabbit_core_metrics.erl b/deps/rabbit_common/src/rabbit_core_metrics.erl
new file mode 100644
index 0000000000..3a6732c0d2
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_core_metrics.erl
@@ -0,0 +1,437 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_core_metrics).
+
+-include("rabbit_core_metrics.hrl").
+
+-export([init/0]).
+-export([terminate/0]).
+
+-export([connection_created/2,
+ connection_closed/1,
+ connection_stats/2,
+ connection_stats/4]).
+
+-export([channel_created/2,
+ channel_closed/1,
+ channel_stats/2,
+ channel_stats/3,
+ channel_stats/4,
+ channel_queue_down/1,
+ channel_queue_exchange_down/1,
+ channel_exchange_down/1]).
+
+-export([consumer_created/9,
+ consumer_updated/9,
+ consumer_deleted/3]).
+
+-export([queue_stats/2,
+ queue_stats/5,
+ queue_declared/1,
+ queue_created/1,
+ queue_deleted/1,
+ queues_deleted/1]).
+
+-export([node_stats/2]).
+
+-export([node_node_stats/2]).
+
+-export([gen_server2_stats/2,
+ gen_server2_deleted/1,
+ get_gen_server2_stats/1]).
+
+-export([delete/2]).
+
+-export([auth_attempt_failed/3,
+ auth_attempt_succeeded/3,
+ reset_auth_attempt_metrics/0,
+ get_auth_attempts/0,
+ get_auth_attempts_by_source/0]).
+
+%%----------------------------------------------------------------------------
+%% Types
+%%----------------------------------------------------------------------------
+-type(channel_stats_id() :: pid() |
+ {pid(),
+ {rabbit_amqqueue:name(), rabbit_exchange:name()}} |
+ {pid(), rabbit_amqqueue:name()} |
+ {pid(), rabbit_exchange:name()}).
+
+-type(channel_stats_type() :: queue_exchange_stats | queue_stats |
+ exchange_stats | reductions).
+
+-type(activity_status() :: up | single_active | waiting | suspected_down).
+%%----------------------------------------------------------------------------
+%% Specs
+%%----------------------------------------------------------------------------
+-spec init() -> ok.
+-spec connection_created(pid(), rabbit_types:infos()) -> ok.
+-spec connection_closed(pid()) -> ok.
+-spec connection_stats(pid(), rabbit_types:infos()) -> ok.
+-spec connection_stats(pid(), integer(), integer(), integer()) -> ok.
+-spec channel_created(pid(), rabbit_types:infos()) -> ok.
+-spec channel_closed(pid()) -> ok.
+-spec channel_stats(pid(), rabbit_types:infos()) -> ok.
+-spec channel_stats(channel_stats_type(), channel_stats_id(),
+ rabbit_types:infos() | integer()) -> ok.
+-spec channel_queue_down({pid(), rabbit_amqqueue:name()}) -> ok.
+-spec channel_queue_exchange_down({pid(), {rabbit_amqqueue:name(),
+ rabbit_exchange:name()}}) -> ok.
+-spec channel_exchange_down({pid(), rabbit_exchange:name()}) -> ok.
+-spec consumer_created(pid(), binary(), boolean(), boolean(),
+ rabbit_amqqueue:name(), integer(), boolean(), activity_status(), list()) -> ok.
+-spec consumer_updated(pid(), binary(), boolean(), boolean(),
+ rabbit_amqqueue:name(), integer(), boolean(), activity_status(), list()) -> ok.
+-spec consumer_deleted(pid(), binary(), rabbit_amqqueue:name()) -> ok.
+-spec queue_stats(rabbit_amqqueue:name(), rabbit_types:infos()) -> ok.
+-spec queue_stats(rabbit_amqqueue:name(), integer(), integer(), integer(),
+ integer()) -> ok.
+-spec node_stats(atom(), rabbit_types:infos()) -> ok.
+-spec node_node_stats({node(), node()}, rabbit_types:infos()) -> ok.
+-spec gen_server2_stats(pid(), integer()) -> ok.
+-spec gen_server2_deleted(pid()) -> ok.
+-spec get_gen_server2_stats(pid()) -> integer() | 'not_found'.
+-spec delete(atom(), any()) -> ok.
+%%----------------------------------------------------------------------------
+%% Storage of the raw metrics in RabbitMQ core. All the processing of stats
+%% is done by the management plugin.
+%%----------------------------------------------------------------------------
+%%----------------------------------------------------------------------------
+%% API
+%%----------------------------------------------------------------------------
+init() ->
+ _ = [ets:new(Table, [Type, public, named_table, {write_concurrency, true},
+ {read_concurrency, true}])
+ || {Table, Type} <- ?CORE_TABLES ++ ?CORE_EXTRA_TABLES],
+ ok.
+
+terminate() ->
+ [ets:delete(Table)
+ || {Table, _Type} <- ?CORE_TABLES ++ ?CORE_EXTRA_TABLES],
+ ok.
+
+connection_created(Pid, Infos) ->
+ ets:insert(connection_created, {Pid, Infos}),
+ ets:update_counter(connection_churn_metrics, node(), {2, 1},
+ ?CONNECTION_CHURN_METRICS),
+ ok.
+
+connection_closed(Pid) ->
+ ets:delete(connection_created, Pid),
+ ets:delete(connection_metrics, Pid),
+ %% Delete marker
+ ets:update_element(connection_coarse_metrics, Pid, {5, 1}),
+ ets:update_counter(connection_churn_metrics, node(), {3, 1},
+ ?CONNECTION_CHURN_METRICS),
+ ok.
+
+connection_stats(Pid, Infos) ->
+ ets:insert(connection_metrics, {Pid, Infos}),
+ ok.
+
+connection_stats(Pid, Recv_oct, Send_oct, Reductions) ->
+ %% Includes delete marker
+ ets:insert(connection_coarse_metrics, {Pid, Recv_oct, Send_oct, Reductions, 0}),
+ ok.
+
+channel_created(Pid, Infos) ->
+ ets:insert(channel_created, {Pid, Infos}),
+ ets:update_counter(connection_churn_metrics, node(), {4, 1},
+ ?CONNECTION_CHURN_METRICS),
+ ok.
+
+channel_closed(Pid) ->
+ ets:delete(channel_created, Pid),
+ ets:delete(channel_metrics, Pid),
+ ets:delete(channel_process_metrics, Pid),
+ ets:update_counter(connection_churn_metrics, node(), {5, 1},
+ ?CONNECTION_CHURN_METRICS),
+ ok.
+
+channel_stats(Pid, Infos) ->
+ ets:insert(channel_metrics, {Pid, Infos}),
+ ok.
+
+channel_stats(reductions, Id, Value) ->
+ ets:insert(channel_process_metrics, {Id, Value}),
+ ok.
+
+channel_stats(exchange_stats, publish, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_exchange_metrics, Id, {2, Value}, {Id, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(exchange_stats, confirm, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_exchange_metrics, Id, {3, Value}, {Id, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(exchange_stats, return_unroutable, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_exchange_metrics, Id, {4, Value}, {Id, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(exchange_stats, drop_unroutable, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_exchange_metrics, Id, {5, Value}, {Id, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(queue_exchange_stats, publish, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_exchange_metrics, Id, Value, {Id, 0, 0}),
+ ok;
+channel_stats(queue_stats, get, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_metrics, Id, {2, Value}, {Id, 0, 0, 0, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(queue_stats, get_no_ack, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_metrics, Id, {3, Value}, {Id, 0, 0, 0, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(queue_stats, deliver, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_metrics, Id, {4, Value}, {Id, 0, 0, 0, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(queue_stats, deliver_no_ack, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_metrics, Id, {5, Value}, {Id, 0, 0, 0, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(queue_stats, redeliver, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_metrics, Id, {6, Value}, {Id, 0, 0, 0, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(queue_stats, ack, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_metrics, Id, {7, Value}, {Id, 0, 0, 0, 0, 0, 0, 0, 0}),
+ ok;
+channel_stats(queue_stats, get_empty, Id, Value) ->
+ %% Includes delete marker
+ _ = ets:update_counter(channel_queue_metrics, Id, {8, Value}, {Id, 0, 0, 0, 0, 0, 0, 0, 0}),
+ ok.
+
+delete(Table, Key) ->
+ ets:delete(Table, Key),
+ ok.
+
+channel_queue_down(Id) ->
+ %% Delete marker
+ ets:update_element(channel_queue_metrics, Id, {9, 1}),
+ ok.
+
+channel_queue_exchange_down(Id) ->
+ %% Delete marker
+ ets:update_element(channel_queue_exchange_metrics, Id, {3, 1}),
+ ok.
+
+channel_exchange_down(Id) ->
+ %% Delete marker
+ ets:update_element(channel_exchange_metrics, Id, {6, 1}),
+ ok.
+
+consumer_created(ChPid, ConsumerTag, ExclusiveConsume, AckRequired, QName,
+ PrefetchCount, Active, ActivityStatus, Args) ->
+ ets:insert(consumer_created, {{QName, ChPid, ConsumerTag}, ExclusiveConsume,
+ AckRequired, PrefetchCount, Active, ActivityStatus, Args}),
+ ok.
+
+consumer_updated(ChPid, ConsumerTag, ExclusiveConsume, AckRequired, QName,
+ PrefetchCount, Active, ActivityStatus, Args) ->
+ ets:insert(consumer_created, {{QName, ChPid, ConsumerTag}, ExclusiveConsume,
+ AckRequired, PrefetchCount, Active, ActivityStatus, Args}),
+ ok.
+
+consumer_deleted(ChPid, ConsumerTag, QName) ->
+ ets:delete(consumer_created, {QName, ChPid, ConsumerTag}),
+ ok.
+
+queue_stats(Name, Infos) ->
+ %% Includes delete marker
+ ets:insert(queue_metrics, {Name, Infos, 0}),
+ ok.
+
+queue_stats(Name, MessagesReady, MessagesUnacknowledge, Messages, Reductions) ->
+ ets:insert(queue_coarse_metrics, {Name, MessagesReady, MessagesUnacknowledge,
+ Messages, Reductions}),
+ ok.
+
+queue_declared(_Name) ->
+ %% Name is not needed, but might be useful in the future.
+ ets:update_counter(connection_churn_metrics, node(), {6, 1},
+ ?CONNECTION_CHURN_METRICS),
+ ok.
+
+queue_created(_Name) ->
+ %% Name is not needed, but might be useful in the future.
+ ets:update_counter(connection_churn_metrics, node(), {7, 1},
+ ?CONNECTION_CHURN_METRICS),
+ ok.
+
+queue_deleted(Name) ->
+ ets:delete(queue_coarse_metrics, Name),
+ ets:update_counter(connection_churn_metrics, node(), {8, 1},
+ ?CONNECTION_CHURN_METRICS),
+ %% Delete markers
+ ets:update_element(queue_metrics, Name, {3, 1}),
+ CQX = ets:select(channel_queue_exchange_metrics, match_spec_cqx(Name)),
+ lists:foreach(fun(Key) ->
+ ets:update_element(channel_queue_exchange_metrics, Key, {3, 1})
+ end, CQX),
+ CQ = ets:select(channel_queue_metrics, match_spec_cq(Name)),
+ lists:foreach(fun(Key) ->
+ ets:update_element(channel_queue_metrics, Key, {9, 1})
+ end, CQ).
+
+queues_deleted(Queues) ->
+ ets:update_counter(connection_churn_metrics, node(), {8, length(Queues)},
+ ?CONNECTION_CHURN_METRICS),
+ [ delete_queue_metrics(Queue) || Queue <- Queues ],
+ [
+ begin
+ MatchSpecCondition = build_match_spec_conditions_to_delete_all_queues(QueuesPartition),
+ delete_channel_queue_exchange_metrics(MatchSpecCondition),
+ delete_channel_queue_metrics(MatchSpecCondition)
+ end || QueuesPartition <- partition_queues(Queues)
+ ],
+ ok.
+
+partition_queues(Queues) when length(Queues) >= 1000 ->
+ {Partition, Rest} = lists:split(1000, Queues),
+ [Partition | partition_queues(Rest)];
+partition_queues(Queues) ->
+ [Queues].
+
+delete_queue_metrics(Queue) ->
+ ets:delete(queue_coarse_metrics, Queue),
+ ets:update_element(queue_metrics, Queue, {3, 1}),
+ ok.
+
+delete_channel_queue_exchange_metrics(MatchSpecCondition) ->
+ ChannelQueueExchangeMetricsToUpdate = ets:select(
+ channel_queue_exchange_metrics,
+ [
+ {
+ {{'$2', {'$1', '$3'}}, '_', '_'},
+ [MatchSpecCondition],
+ [{{'$2', {{'$1', '$3'}}}}]
+ }
+ ]
+ ),
+ lists:foreach(fun(Key) ->
+ ets:update_element(channel_queue_exchange_metrics, Key, {3, 1})
+ end, ChannelQueueExchangeMetricsToUpdate).
+
+delete_channel_queue_metrics(MatchSpecCondition) ->
+ ChannelQueueMetricsToUpdate = ets:select(
+ channel_queue_metrics,
+ [
+ {
+ {{'$2', '$1'}, '_', '_', '_', '_', '_', '_', '_', '_'},
+ [MatchSpecCondition],
+ [{{'$2', '$1'}}]
+ }
+ ]
+ ),
+ lists:foreach(fun(Key) ->
+ ets:update_element(channel_queue_metrics, Key, {9, 1})
+ end, ChannelQueueMetricsToUpdate).
+
+% [{'orelse',
+% {'==', {Queue}, '$1'},
+% {'orelse',
+% {'==', {Queue}, '$1'},
+% % ...
+% {'orelse',
+% {'==', {Queue}, '$1'},
+% {'==', true, true}
+% }
+% }
+% }],
+build_match_spec_conditions_to_delete_all_queues([Queue|Queues]) ->
+ {'orelse',
+ {'==', {Queue}, '$1'},
+ build_match_spec_conditions_to_delete_all_queues(Queues)
+ };
+build_match_spec_conditions_to_delete_all_queues([]) ->
+ true.
+
+node_stats(persister_metrics, Infos) ->
+ ets:insert(node_persister_metrics, {node(), Infos}),
+ ok;
+node_stats(coarse_metrics, Infos) ->
+ ets:insert(node_coarse_metrics, {node(), Infos}),
+ ok;
+node_stats(node_metrics, Infos) ->
+ ets:insert(node_metrics, {node(), Infos}),
+ ok.
+
+node_node_stats(Id, Infos) ->
+ ets:insert(node_node_metrics, {Id, Infos}),
+ ok.
+
+match_spec_cqx(Id) ->
+ [{{{'$2', {'$1', '$3'}}, '_', '_'}, [{'==', {Id}, '$1'}], [{{'$2', {{'$1', '$3'}}}}]}].
+
+match_spec_cq(Id) ->
+ [{{{'$2', '$1'}, '_', '_', '_', '_', '_', '_', '_', '_'}, [{'==', {Id}, '$1'}], [{{'$2', '$1'}}]}].
+
+gen_server2_stats(Pid, BufferLength) ->
+ ets:insert(gen_server2_metrics, {Pid, BufferLength}),
+ ok.
+
+gen_server2_deleted(Pid) ->
+ ets:delete(gen_server2_metrics, Pid),
+ ok.
+
+get_gen_server2_stats(Pid) ->
+ case ets:lookup(gen_server2_metrics, Pid) of
+ [{Pid, BufferLength}] ->
+ BufferLength;
+ [] ->
+ not_found
+ end.
+
+auth_attempt_succeeded(RemoteAddress, Username, Protocol) ->
+ %% ETS entry is {Key = {RemoteAddress, Username}, Total, Succeeded, Failed}
+ update_auth_attempt(RemoteAddress, Username, Protocol, [{2, 1}, {3, 1}]).
+
+auth_attempt_failed(RemoteAddress, Username, Protocol) ->
+ %% ETS entry is {Key = {RemoteAddress, Username}, Total, Succeeded, Failed}
+ update_auth_attempt(RemoteAddress, Username, Protocol, [{2, 1}, {4, 1}]).
+
+update_auth_attempt(RemoteAddress, Username, Protocol, Incr) ->
+ %% It should default to false as per ip/user metrics could keep growing indefinitely
+ %% It's up to the operator to enable them, and reset it required
+ case application:get_env(rabbit, track_auth_attempt_source) of
+ {ok, true} ->
+ case {RemoteAddress, Username} of
+ {<<>>, <<>>} ->
+ ok;
+ _ ->
+ Key = {RemoteAddress, Username, Protocol},
+ _ = ets:update_counter(auth_attempt_detailed_metrics, Key, Incr, {Key, 0, 0, 0})
+ end;
+ {ok, false} ->
+ ok
+ end,
+ _ = ets:update_counter(auth_attempt_metrics, Protocol, Incr, {Protocol, 0, 0, 0}),
+ ok.
+
+reset_auth_attempt_metrics() ->
+ ets:delete_all_objects(auth_attempt_metrics),
+ ets:delete_all_objects(auth_attempt_detailed_metrics),
+ ok.
+
+get_auth_attempts() ->
+ [format_auth_attempt(A) || A <- ets:tab2list(auth_attempt_metrics)].
+
+get_auth_attempts_by_source() ->
+ [format_auth_attempt(A) || A <- ets:tab2list(auth_attempt_detailed_metrics)].
+
+format_auth_attempt({{RemoteAddress, Username, Protocol}, Total, Succeeded, Failed}) ->
+ [{remote_address, RemoteAddress}, {username, Username},
+ {protocol, atom_to_binary(Protocol, utf8)}, {auth_attempts, Total},
+ {auth_attempts_failed, Failed}, {auth_attempts_succeeded, Succeeded}];
+format_auth_attempt({Protocol, Total, Succeeded, Failed}) ->
+ [{protocol, atom_to_binary(Protocol, utf8)}, {auth_attempts, Total},
+ {auth_attempts_failed, Failed}, {auth_attempts_succeeded, Succeeded}].
diff --git a/deps/rabbit_common/src/rabbit_data_coercion.erl b/deps/rabbit_common/src/rabbit_data_coercion.erl
new file mode 100644
index 0000000000..9d2b39da94
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_data_coercion.erl
@@ -0,0 +1,47 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_data_coercion).
+
+-export([to_binary/1, to_list/1, to_atom/1, to_integer/1, to_proplist/1, to_map/1]).
+-export([to_atom/2]).
+
+-spec to_binary(Val :: binary() | list() | atom() | integer()) -> binary().
+to_binary(Val) when is_list(Val) -> list_to_binary(Val);
+to_binary(Val) when is_atom(Val) -> atom_to_binary(Val, utf8);
+to_binary(Val) when is_integer(Val) -> integer_to_binary(Val);
+to_binary(Val) -> Val.
+
+-spec to_list(Val :: integer() | list() | binary() | atom() | map()) -> list().
+to_list(Val) when is_list(Val) -> Val;
+to_list(Val) when is_map(Val) -> maps:to_list(Val);
+to_list(Val) when is_atom(Val) -> atom_to_list(Val);
+to_list(Val) when is_binary(Val) -> binary_to_list(Val);
+to_list(Val) when is_integer(Val) -> integer_to_list(Val).
+
+-spec to_atom(Val :: atom() | list() | binary()) -> atom().
+to_atom(Val) when is_atom(Val) -> Val;
+to_atom(Val) when is_list(Val) -> list_to_atom(Val);
+to_atom(Val) when is_binary(Val) -> binary_to_atom(Val, utf8).
+
+-spec to_atom(Val :: atom() | list() | binary(), Encoding :: atom()) -> atom().
+to_atom(Val, _Encoding) when is_atom(Val) -> Val;
+to_atom(Val, _Encoding) when is_list(Val) -> list_to_atom(Val);
+to_atom(Val, Encoding) when is_binary(Val) -> binary_to_atom(Val, Encoding).
+
+-spec to_integer(Val :: integer() | list() | binary()) -> integer().
+to_integer(Val) when is_integer(Val) -> Val;
+to_integer(Val) when is_list(Val) -> list_to_integer(Val);
+to_integer(Val) when is_binary(Val) -> binary_to_integer(Val).
+
+-spec to_proplist(Val :: map() | list()) -> list().
+to_proplist(Val) when is_list(Val) -> Val;
+to_proplist(Val) when is_map(Val) -> maps:to_list(Val).
+
+-spec to_map(Val :: map() | list()) -> map().
+to_map(Val) when is_map(Val) -> Val;
+to_map(Val) when is_list(Val) -> maps:from_list(Val).
diff --git a/deps/rabbit_common/src/rabbit_env.erl b/deps/rabbit_common/src/rabbit_env.erl
new file mode 100644
index 0000000000..8817103e81
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_env.erl
@@ -0,0 +1,1850 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2019-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_env).
+
+-include_lib("kernel/include/file.hrl").
+
+-export([get_context/0,
+ get_context/1,
+ get_context_before_logging_init/0,
+ get_context_before_logging_init/1,
+ get_context_after_logging_init/1,
+ get_context_after_reloading_env/1,
+ dbg_config/0,
+ get_used_env_vars/0,
+ log_process_env/0,
+ log_context/1,
+ context_to_app_env_vars/1,
+ context_to_app_env_vars_no_logging/1,
+ context_to_code_path/1]).
+
+-ifdef(TEST).
+-export([parse_conf_env_file_output2/2,
+ value_is_yes/1]).
+-endif.
+
+-define(USED_ENV_VARS,
+ [
+ "RABBITMQ_ALLOW_INPUT",
+ "RABBITMQ_ADVANCED_CONFIG_FILE",
+ "RABBITMQ_BASE",
+ "RABBITMQ_CONF_ENV_FILE",
+ "RABBITMQ_CONFIG_FILE",
+ "RABBITMQ_CONFIG_FILES",
+ "RABBITMQ_DBG",
+ "RABBITMQ_DIST_PORT",
+ "RABBITMQ_ENABLED_PLUGINS",
+ "RABBITMQ_ENABLED_PLUGINS_FILE",
+ "RABBITMQ_FEATURE_FLAGS",
+ "RABBITMQ_FEATURE_FLAGS_FILE",
+ "RABBITMQ_HOME",
+ "RABBITMQ_KEEP_PID_FILE_ON_EXIT",
+ "RABBITMQ_LOG",
+ "RABBITMQ_LOG_BASE",
+ "RABBITMQ_LOG_FF_REGISTRY",
+ "RABBITMQ_LOGS",
+ "RABBITMQ_MNESIA_BASE",
+ "RABBITMQ_MNESIA_DIR",
+ "RABBITMQ_MOTD_FILE",
+ "RABBITMQ_NODE_IP_ADDRESS",
+ "RABBITMQ_NODE_PORT",
+ "RABBITMQ_NODENAME",
+ "RABBITMQ_PID_FILE",
+ "RABBITMQ_PLUGINS_DIR",
+ "RABBITMQ_PLUGINS_EXPAND_DIR",
+ "RABBITMQ_PRODUCT_NAME",
+ "RABBITMQ_PRODUCT_VERSION",
+ "RABBITMQ_QUORUM_DIR",
+ "RABBITMQ_STREAM_DIR",
+ "RABBITMQ_UPGRADE_LOG",
+ "RABBITMQ_USE_LONGNAME",
+ "SYS_PREFIX"
+ ]).
+
+get_context() ->
+ Context0 = get_context_before_logging_init(),
+ Context1 = get_context_after_logging_init(Context0),
+ get_context_after_reloading_env(Context1).
+
+get_context(TakeFromRemoteNode) ->
+ Context0 = get_context_before_logging_init(TakeFromRemoteNode),
+ Context1 = get_context_after_logging_init(Context0),
+ get_context_after_reloading_env(Context1).
+
+get_context_before_logging_init() ->
+ get_context_before_logging_init(false).
+
+get_context_before_logging_init(TakeFromRemoteNode) ->
+ %% The order of steps below is important because some of them
+ %% depends on previous steps.
+ Steps = [
+ fun os_type/1,
+ fun log_levels/1,
+ fun interactive_shell/1,
+ fun output_supports_colors/1
+ ],
+
+ run_context_steps(context_base(TakeFromRemoteNode), Steps).
+
+get_context_after_logging_init(Context) ->
+ %% The order of steps below is important because some of them
+ %% depends on previous steps.
+ Steps = [
+ fun sys_prefix/1,
+ fun rabbitmq_base/1,
+ fun data_dir/1,
+ fun rabbitmq_home/1,
+ fun config_base_dir/1,
+ fun load_conf_env_file/1,
+ fun log_levels/1
+ ],
+
+ run_context_steps(Context, Steps).
+
+get_context_after_reloading_env(Context) ->
+ %% The order of steps below is important because some of them
+ %% depends on previous steps.
+ Steps = [
+ fun nodename_type/1,
+ fun nodename/1,
+ fun split_nodename/1,
+ fun maybe_setup_dist_for_remote_query/1,
+ fun dbg_config/1,
+ fun main_config_file/1,
+ fun additional_config_files/1,
+ fun advanced_config_file/1,
+ fun log_base_dir/1,
+ fun main_log_file/1,
+ fun upgrade_log_file/1,
+ fun mnesia_base_dir/1,
+ fun mnesia_dir/1,
+ fun quorum_queue_dir/1,
+ fun stream_queue_dir/1,
+ fun pid_file/1,
+ fun keep_pid_file_on_exit/1,
+ fun feature_flags_file/1,
+ fun forced_feature_flags_on_init/1,
+ fun log_feature_flags_registry/1,
+ fun plugins_path/1,
+ fun plugins_expand_dir/1,
+ fun enabled_plugins_file/1,
+ fun enabled_plugins/1,
+ fun maybe_stop_dist_for_remote_query/1,
+ fun amqp_ipaddr/1,
+ fun amqp_tcp_port/1,
+ fun erlang_dist_tcp_port/1,
+ fun product_name/1,
+ fun product_version/1,
+ fun motd_file/1
+ ],
+
+ run_context_steps(Context, Steps).
+
+context_base(TakeFromRemoteNode) ->
+ Context = #{},
+ case TakeFromRemoteNode of
+ false ->
+ Context;
+ offline ->
+ update_context(Context,
+ from_remote_node,
+ offline);
+ _ when is_atom(TakeFromRemoteNode) ->
+ update_context(Context,
+ from_remote_node,
+ {TakeFromRemoteNode, 10000});
+ {RemoteNode, infinity}
+ when is_atom(RemoteNode) ->
+ update_context(Context,
+ from_remote_node,
+ TakeFromRemoteNode);
+ {RemoteNode, Timeout}
+ when is_atom(RemoteNode) andalso
+ is_integer(Timeout) andalso
+ Timeout >= 0 ->
+ update_context(Context,
+ from_remote_node,
+ {TakeFromRemoteNode, Timeout})
+ end.
+
+-ifdef(TEST).
+os_type(Context) ->
+ {OSType, Origin} =
+ try
+ {persistent_term:get({?MODULE, os_type}), environment}
+ catch
+ _:badarg ->
+ {os:type(), default}
+ end,
+ update_context(Context, os_type, OSType, Origin).
+-else.
+os_type(Context) ->
+ update_context(Context, os_type, os:type(), default).
+-endif.
+
+run_context_steps(Context, Steps) ->
+ lists:foldl(
+ fun(Step, Context1) -> Step(Context1) end,
+ Context,
+ Steps).
+
+update_context(Context, Key, Value) ->
+ Context#{Key => Value}.
+
+-define(origin_is_valid(O),
+ O =:= default orelse
+ O =:= environment orelse
+ O =:= remote_node).
+
+update_context(#{var_origins := Origins} = Context, Key, Value, Origin)
+ when ?origin_is_valid(Origin) ->
+ Context#{Key => Value,
+ var_origins => Origins#{Key => Origin}};
+update_context(Context, Key, Value, Origin)
+ when ?origin_is_valid(Origin) ->
+ Context#{Key => Value,
+ var_origins => #{Key => Origin}}.
+
+get_used_env_vars() ->
+ lists:filter(
+ fun({Var, _}) -> var_is_used(Var) end,
+ lists:sort(os:list_env_vars())).
+
+log_process_env() ->
+ rabbit_log_prelaunch:debug("Process environment:"),
+ lists:foreach(
+ fun({Var, Value}) ->
+ rabbit_log_prelaunch:debug(" - ~s = ~ts", [Var, Value])
+ end, lists:sort(os:list_env_vars())).
+
+log_context(Context) ->
+ rabbit_log_prelaunch:debug("Context (based on environment variables):"),
+ lists:foreach(
+ fun(Key) ->
+ Value = maps:get(Key, Context),
+ rabbit_log_prelaunch:debug(" - ~s: ~p", [Key, Value])
+ end,
+ lists:sort(maps:keys(Context))).
+
+context_to_app_env_vars(Context) ->
+ rabbit_log_prelaunch:debug(
+ "Setting default application environment variables:"),
+ Fun = fun({App, Param, Value}) ->
+ rabbit_log_prelaunch:debug(
+ " - ~s:~s = ~p", [App, Param, Value]),
+ ok = application:set_env(
+ App, Param, Value, [{persistent, true}])
+ end,
+ context_to_app_env_vars1(Context, Fun).
+
+context_to_app_env_vars_no_logging(Context) ->
+ Fun = fun({App, Param, Value}) ->
+ ok = application:set_env(
+ App, Param, Value, [{persistent, true}])
+ end,
+ context_to_app_env_vars1(Context, Fun).
+
+context_to_app_env_vars1(
+ #{mnesia_dir := MnesiaDir,
+ feature_flags_file := FFFile,
+ quorum_queue_dir := QuorumQueueDir,
+ stream_queue_dir := StreamQueueDir,
+ plugins_path := PluginsPath,
+ plugins_expand_dir := PluginsExpandDir,
+ enabled_plugins_file := EnabledPluginsFile} = Context,
+ Fun) ->
+ lists:foreach(
+ Fun,
+ %% Those are all the application environment variables which
+ %% were historically set on the erl(1) command line in
+ %% rabbitmq-server(8).
+ [{kernel, inet_default_connect_options, [{nodelay, true}]},
+ {sasl, errlog_type, error},
+ {os_mon, start_cpu_sup, false},
+ {os_mon, start_disksup, false},
+ {os_mon, start_memsup, false},
+ {mnesia, dir, MnesiaDir},
+ {ra, data_dir, QuorumQueueDir},
+ {osiris, data_dir, StreamQueueDir},
+ {rabbit, feature_flags_file, FFFile},
+ {rabbit, plugins_dir, PluginsPath},
+ {rabbit, plugins_expand_dir, PluginsExpandDir},
+ {rabbit, enabled_plugins_file, EnabledPluginsFile}]),
+
+ case Context of
+ #{erlang_dist_tcp_port := DistTcpPort} ->
+ lists:foreach(
+ Fun,
+ [{kernel, inet_dist_listen_min, DistTcpPort},
+ {kernel, inet_dist_listen_max, DistTcpPort}]);
+ _ ->
+ ok
+ end,
+ case Context of
+ #{amqp_ipaddr := IpAddr,
+ amqp_tcp_port := TcpPort}
+ when IpAddr /= undefined andalso TcpPort /= undefined ->
+ Fun({rabbit, tcp_listeners, [{IpAddr, TcpPort}]});
+ _ ->
+ ok
+ end,
+ ok.
+
+context_to_code_path(#{os_type := OSType, plugins_path := PluginsPath}) ->
+ Dirs = get_user_lib_dirs(OSType, PluginsPath),
+ code:add_pathsa(lists:reverse(Dirs)).
+
+%% -------------------------------------------------------------------
+%% Code copied from `kernel/src/code_server.erl`.
+%%
+%% The goal is to mimic the behavior of the `$ERL_LIBS` environment
+%% variable.
+
+get_user_lib_dirs(OSType, Path) ->
+ Sep = case OSType of
+ {win32, _} -> ";";
+ _ -> ":"
+ end,
+ SplitPath = string:lexemes(Path, Sep),
+ get_user_lib_dirs_1(SplitPath).
+
+get_user_lib_dirs_1([Dir|DirList]) ->
+ case erl_prim_loader:list_dir(Dir) of
+ {ok, Dirs} ->
+ Paths = make_path(Dir, Dirs),
+ %% Only add paths trailing with ./ebin.
+ [P || P <- Paths, filename:basename(P) =:= "ebin"] ++
+ get_user_lib_dirs_1(DirList);
+ error ->
+ get_user_lib_dirs_1(DirList)
+ end;
+get_user_lib_dirs_1([]) -> [].
+
+%%
+%% Create the initial path.
+%%
+make_path(BundleDir, Bundles0) ->
+ Bundles = choose_bundles(Bundles0),
+ make_path(BundleDir, Bundles, []).
+
+choose_bundles(Bundles) ->
+ ArchiveExt = archive_extension(),
+ Bs = lists:sort([create_bundle(B, ArchiveExt) || B <- Bundles]),
+ [FullName || {_Name,_NumVsn,FullName} <-
+ choose(lists:reverse(Bs), [], ArchiveExt)].
+
+create_bundle(FullName, ArchiveExt) ->
+ BaseName = filename:basename(FullName, ArchiveExt),
+ case split_base(BaseName) of
+ {Name, VsnStr} ->
+ case vsn_to_num(VsnStr) of
+ {ok, VsnNum} ->
+ {Name,VsnNum,FullName};
+ false ->
+ {FullName,[0],FullName}
+ end;
+ _ ->
+ {FullName,[0],FullName}
+ end.
+
+%% Convert "X.Y.Z. ..." to [K, L, M| ...]
+vsn_to_num(Vsn) ->
+ case is_vsn(Vsn) of
+ true ->
+ {ok, [list_to_integer(S) || S <- string:lexemes(Vsn, ".")]};
+ _ ->
+ false
+ end.
+
+is_vsn(Str) when is_list(Str) ->
+ Vsns = string:lexemes(Str, "."),
+ lists:all(fun is_numstr/1, Vsns).
+
+is_numstr(Cs) ->
+ lists:all(fun (C) when $0 =< C, C =< $9 -> true;
+ (_) -> false
+ end, Cs).
+
+choose([{Name,NumVsn,NewFullName}=New|Bs], Acc, ArchiveExt) ->
+ case lists:keyfind(Name, 1, Acc) of
+ {_, NV, OldFullName} when NV =:= NumVsn ->
+ case filename:extension(OldFullName) =:= ArchiveExt of
+ false ->
+ choose(Bs,Acc, ArchiveExt);
+ true ->
+ Acc2 = lists:keystore(Name, 1, Acc, New),
+ choose(Bs,Acc2, ArchiveExt)
+ end;
+ {_, _, _} ->
+ choose(Bs,Acc, ArchiveExt);
+ false ->
+ choose(Bs,[{Name,NumVsn,NewFullName}|Acc], ArchiveExt)
+ end;
+choose([],Acc, _ArchiveExt) ->
+ Acc.
+
+make_path(_, [], Res) ->
+ Res;
+make_path(BundleDir, [Bundle|Tail], Res) ->
+ Dir = filename:append(BundleDir, Bundle),
+ Ebin = filename:append(Dir, "ebin"),
+ %% First try with /ebin
+ case is_dir(Ebin) of
+ true ->
+ make_path(BundleDir, Tail, [Ebin|Res]);
+ false ->
+ %% Second try with archive
+ Ext = archive_extension(),
+ Base = filename:basename(Bundle, Ext),
+ Ebin2 = filename:join([BundleDir, Base ++ Ext, Base, "ebin"]),
+ Ebins =
+ case split_base(Base) of
+ {AppName,_} ->
+ Ebin3 = filename:join([BundleDir, Base ++ Ext,
+ AppName, "ebin"]),
+ [Ebin3, Ebin2, Dir];
+ _ ->
+ [Ebin2, Dir]
+ end,
+ case try_ebin_dirs(Ebins) of
+ {ok,FoundEbin} ->
+ make_path(BundleDir, Tail, [FoundEbin|Res]);
+ error ->
+ make_path(BundleDir, Tail, Res)
+ end
+ end.
+
+try_ebin_dirs([Ebin|Ebins]) ->
+ case is_dir(Ebin) of
+ true -> {ok,Ebin};
+ false -> try_ebin_dirs(Ebins)
+ end;
+try_ebin_dirs([]) ->
+ error.
+
+split_base(BaseName) ->
+ case string:lexemes(BaseName, "-") of
+ [_, _|_] = Toks ->
+ Vsn = lists:last(Toks),
+ AllButLast = lists:droplast(Toks),
+ {string:join(AllButLast, "-"),Vsn};
+ [_|_] ->
+ BaseName
+ end.
+
+is_dir(Path) ->
+ case erl_prim_loader:read_file_info(Path) of
+ {ok,#file_info{type=directory}} -> true;
+ _ -> false
+ end.
+
+archive_extension() ->
+ init:archive_extension().
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_NODENAME
+%% Erlang node name.
+%% Default: rabbit@<hostname>
+%%
+%% RABBITMQ_USE_LONGNAME
+%% Flag indicating if long Erlang node names should be used instead
+%% of short ones.
+%% Default: unset (use short names)
+
+nodename_type(Context) ->
+ case get_prefixed_env_var("RABBITMQ_USE_LONGNAME") of
+ false ->
+ update_context(Context, nodename_type, shortnames, default);
+ Value ->
+ NameType = case value_is_yes(Value) of
+ true -> longnames;
+ false -> shortnames
+ end,
+ update_context(Context, nodename_type, NameType, environment)
+ end.
+
+nodename(#{nodename_type := NameType} = Context) ->
+ LongHostname = net_adm:localhost(),
+ ShortHostname = re:replace(LongHostname, "\\..*$", "", [{return, list}]),
+ case get_prefixed_env_var("RABBITMQ_NODENAME") of
+ false when NameType =:= shortnames ->
+ Nodename = rabbit_nodes_common:make({"rabbit", ShortHostname}),
+ update_context(Context, nodename, Nodename, default);
+ false when NameType =:= longnames ->
+ Nodename = rabbit_nodes_common:make({"rabbit", LongHostname}),
+ update_context(Context, nodename, Nodename, default);
+ Value ->
+ Nodename = case string:find(Value, "@") of
+ nomatch when NameType =:= shortnames ->
+ rabbit_nodes_common:make({Value, ShortHostname});
+ nomatch when NameType =:= longnames ->
+ rabbit_nodes_common:make({Value, LongHostname});
+ _ ->
+ rabbit_nodes_common:make(Value)
+ end,
+ update_context(Context, nodename, Nodename, environment)
+ end.
+
+split_nodename(#{nodename := Nodename} = Context) ->
+ update_context(Context,
+ split_nodename, rabbit_nodes_common:parts(Nodename)).
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_CONFIG_FILE
+%% Main configuration file.
+%% Extension is optional. `.config` for the old erlang-term-based
+%% format, `.conf` for the new Cuttlefish-based format.
+%% Default: (Unix) ${SYS_PREFIX}/etc/rabbitmq/rabbitmq
+%% (Windows) ${RABBITMQ_BASE}\rabbitmq
+%%
+%% RABBITMQ_CONFIG_FILES
+%% Additional configuration files.
+%% If a directory, all files directly inside it are loaded.
+%% If a glob pattern, all matching file are loaded.
+%% Only considered if the main configuration file is Cuttlefish-based.
+%% Default: (Unix) ${SYS_PREFIX}/etc/rabbitmq/conf.d/*.conf
+%% (Windows) ${RABBITMQ_BASE}\conf.d\*.conf
+%%
+%% RABBITMQ_ADVANCED_CONFIG_FILE
+%% Advanced configuration file.
+%% Erlang-term-based format with a `.config` extension.
+%% Default: (Unix) ${SYS_PREFIX}/etc/rabbitmq/advanced.config
+%% (Windows) ${RABBITMQ_BASE}\advanced.config
+
+config_base_dir(#{os_type := {unix, _},
+ sys_prefix := SysPrefix} = Context) ->
+ Dir = filename:join([SysPrefix, "etc", "rabbitmq"]),
+ update_context(Context, config_base_dir, Dir);
+config_base_dir(#{os_type := {win32, _},
+ rabbitmq_base := Dir} = Context) ->
+ update_context(Context, config_base_dir, Dir).
+
+main_config_file(Context) ->
+ case get_prefixed_env_var("RABBITMQ_CONFIG_FILE") of
+ false ->
+ File = get_default_main_config_file(Context),
+ update_context(Context, main_config_file, File, default);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, main_config_file, File, environment)
+ end.
+
+get_default_main_config_file(#{config_base_dir := ConfigBaseDir}) ->
+ filename:join(ConfigBaseDir, "rabbitmq").
+
+additional_config_files(Context) ->
+ case get_prefixed_env_var("RABBITMQ_CONFIG_FILES") of
+ false ->
+ Pattern = get_default_additional_config_files(Context),
+ update_context(
+ Context, additional_config_files, Pattern, default);
+ Value ->
+ Pattern = normalize_path(Value),
+ update_context(
+ Context, additional_config_files, Pattern, environment)
+ end.
+
+get_default_additional_config_files(#{config_base_dir := ConfigBaseDir}) ->
+ filename:join([ConfigBaseDir, "conf.d", "*.conf"]).
+
+advanced_config_file(Context) ->
+ case get_prefixed_env_var("RABBITMQ_ADVANCED_CONFIG_FILE") of
+ false ->
+ File = get_default_advanced_config_file(Context),
+ update_context(Context, advanced_config_file, File, default);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, advanced_config_file, File, environment)
+ end.
+
+get_default_advanced_config_file(#{config_base_dir := ConfigBaseDir}) ->
+ filename:join(ConfigBaseDir, "advanced.config").
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_LOG_BASE
+%% Directory to write log files
+%% Default: (Unix) ${SYS_PREFIX}/var/log/rabbitmq
+%% (Windows) ${RABBITMQ_BASE}\log
+%%
+%% RABBITMQ_LOGS
+%% Main log file
+%% Default: ${RABBITMQ_LOG_BASE}/${RABBITMQ_NODENAME}.log
+%%
+%% RABBITMQ_UPDATE_LOG
+%% Upgrade-procesure-specific log file
+%% Default: ${RABBITMQ_LOG_BASE}/${RABBITMQ_NODENAME}_upgrade.log
+%%
+%% RABBITMQ_LOG
+%% Log level; overrides the configuration file value
+%% Default: (undefined)
+%%
+%% RABBITMQ_DBG
+%% List of `module`, `module:function` or `module:function/arity`
+%% to watch with dbg.
+%% Default: (undefined)
+
+log_levels(Context) ->
+ case get_prefixed_env_var("RABBITMQ_LOG") of
+ false ->
+ update_context(Context, log_levels, undefined, default);
+ Value ->
+ LogLevels = parse_log_levels(string:lexemes(Value, ","), #{}),
+ update_context(Context, log_levels, LogLevels, environment)
+ end.
+
+parse_log_levels([CategoryValue | Rest], Result) ->
+ case string:lexemes(CategoryValue, "=") of
+ ["+color"] ->
+ Result1 = Result#{color => true},
+ parse_log_levels(Rest, Result1);
+ ["-color"] ->
+ Result1 = Result#{color => false},
+ parse_log_levels(Rest, Result1);
+ [CategoryOrLevel] ->
+ case parse_level(CategoryOrLevel) of
+ undefined ->
+ Result1 = Result#{CategoryOrLevel => info},
+ parse_log_levels(Rest, Result1);
+ Level ->
+ Result1 = Result#{global => Level},
+ parse_log_levels(Rest, Result1)
+ end;
+ [Category, Level0] ->
+ case parse_level(Level0) of
+ undefined ->
+ parse_log_levels(Rest, Result);
+ Level ->
+ Result1 = Result#{Category => Level},
+ parse_log_levels(Rest, Result1)
+ end
+ end;
+parse_log_levels([], Result) ->
+ Result.
+
+parse_level("debug") -> debug;
+parse_level("info") -> info;
+parse_level("notice") -> notice;
+parse_level("warning") -> warning;
+parse_level("error") -> error;
+parse_level("critical") -> critical;
+parse_level("alert") -> alert;
+parse_level("emergency") -> emergency;
+parse_level("none") -> none;
+parse_level(_) -> undefined.
+
+log_base_dir(#{os_type := OSType} = Context) ->
+ case {get_prefixed_env_var("RABBITMQ_LOG_BASE"), OSType} of
+ {false, {unix, _}} ->
+ #{sys_prefix := SysPrefix} = Context,
+ Dir = filename:join([SysPrefix, "var", "log", "rabbitmq"]),
+ update_context(Context, log_base_dir, Dir, default);
+ {false, {win32, _}} ->
+ #{rabbitmq_base := RabbitmqBase} = Context,
+ Dir = filename:join([RabbitmqBase, "log"]),
+ update_context(Context, log_base_dir, Dir, default);
+ {Value, _} ->
+ Dir = normalize_path(Value),
+ update_context(Context, log_base_dir, Dir, environment)
+ end.
+
+main_log_file(#{nodename := Nodename,
+ log_base_dir := LogBaseDir} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_LOGS") of
+ false ->
+ File= filename:join(LogBaseDir,
+ atom_to_list(Nodename) ++ ".log"),
+ update_context(Context, main_log_file, File, default);
+ "-" ->
+ update_context(Context, main_log_file, "-", environment);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, main_log_file, File, environment)
+ end.
+
+upgrade_log_file(#{nodename := Nodename,
+ log_base_dir := LogBaseDir} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_UPGRADE_LOG") of
+ false ->
+ File = filename:join(LogBaseDir,
+ atom_to_list(Nodename) ++ "_upgrade.log"),
+ update_context(Context, upgrade_log_file, File, default);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, upgrade_log_file, File, environment)
+ end.
+
+dbg_config() ->
+ {Mods, Output} = get_dbg_config(),
+ #{dbg_output => Output,
+ dbg_mods => Mods}.
+
+dbg_config(Context) ->
+ DbgContext = dbg_config(),
+ maps:merge(Context, DbgContext).
+
+get_dbg_config() ->
+ Output = stdout,
+ DbgValue = get_prefixed_env_var("RABBITMQ_DBG"),
+ case DbgValue of
+ false -> {[], Output};
+ _ -> get_dbg_config1(string:lexemes(DbgValue, ","), [], Output)
+ end.
+
+get_dbg_config1(["=" ++ Filename | Rest], Mods, _) ->
+ get_dbg_config1(Rest, Mods, Filename);
+get_dbg_config1([SpecValue | Rest], Mods, Output) ->
+ Pattern = "([^:]+)(?::([^/]+)(?:/([0-9]+))?)?",
+ Options = [{capture, all_but_first, list}],
+ Mods1 = case re:run(SpecValue, Pattern, Options) of
+ {match, [M, F, A]} ->
+ Entry = {list_to_atom(M),
+ list_to_atom(F),
+ list_to_integer(A)},
+ [Entry | Mods];
+ {match, [M, F]} ->
+ Entry = {list_to_atom(M),
+ list_to_atom(F),
+ '_'},
+ [Entry | Mods];
+ {match, [M]} ->
+ Entry = {list_to_atom(M),
+ '_',
+ '_'},
+ [Entry | Mods];
+ nomatch ->
+ Mods
+ end,
+ get_dbg_config1(Rest, Mods1, Output);
+get_dbg_config1([], Mods, Output) ->
+ {lists:reverse(Mods), Output}.
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_MNESIA_BASE
+%% Directory where to create Mnesia directory.
+%% Default: (Unix) ${SYS_PREFIX}/var/lib/rabbitmq/mnesia
+%% (Windows) ${RABBITMQ_BASE}/db
+%%
+%% RABBITMQ_MNESIA_DIR
+%% Directory where to put Mnesia data.
+%% Default: (Unix) ${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}
+%% (Windows) ${RABBITMQ_MNESIA_BASE}\${RABBITMQ_NODENAME}-mnesia
+
+mnesia_base_dir(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_MNESIA_BASE") of
+ false when Remote =:= offline ->
+ update_context(Context, mnesia_base_dir, undefined, default);
+ false ->
+ mnesia_base_dir_from_node(Context);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, mnesia_base_dir, Dir, environment)
+ end;
+mnesia_base_dir(Context) ->
+ mnesia_base_dir_from_env(Context).
+
+mnesia_base_dir_from_env(Context) ->
+ case get_prefixed_env_var("RABBITMQ_MNESIA_BASE") of
+ false ->
+ Dir = get_default_mnesia_base_dir(Context),
+ update_context(Context, mnesia_base_dir, Dir, default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, mnesia_base_dir, Dir, environment)
+ end.
+
+mnesia_base_dir_from_node(Context) ->
+ %% This variable is used to compute other variables only, we
+ %% don't need to know what a remote node used initially. Only the
+ %% variables based on it are relevant.
+ update_context(Context, mnesia_base_dir, undefined, default).
+
+get_default_mnesia_base_dir(#{data_dir := DataDir} = Context) ->
+ Basename = case Context of
+ #{os_type := {unix, _}} -> "mnesia";
+ #{os_type := {win32, _}} -> "db"
+ end,
+ filename:join(DataDir, Basename).
+
+mnesia_dir(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_MNESIA_DIR") of
+ false when Remote =:= offline ->
+ update_context(Context, mnesia_dir, undefined, default);
+ false ->
+ mnesia_dir_from_node(Context);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, mnesia_dir, Dir, environment)
+ end;
+mnesia_dir(Context) ->
+ mnesia_dir_from_env(Context).
+
+mnesia_dir_from_env(Context) ->
+ case get_prefixed_env_var("RABBITMQ_MNESIA_DIR") of
+ false ->
+ Dir = get_default_mnesia_dir(Context),
+ update_context(Context, mnesia_dir, Dir, default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, mnesia_dir, Dir, environment)
+ end.
+
+mnesia_dir_from_node(#{from_remote_node := Remote} = Context) ->
+ Ret = query_remote(Remote, application, get_env, [mnesia, dir]),
+ case Ret of
+ {ok, undefined} ->
+ throw({query, Remote, {mnesia, dir, undefined}});
+ {ok, {ok, Value}} ->
+ Dir = normalize_path(Value),
+ update_context(Context, mnesia_dir, Dir, remote_node);
+ {badrpc, nodedown} ->
+ update_context(Context, mnesia_dir, undefined, default)
+ end.
+
+get_default_mnesia_dir(#{os_type := {unix, _},
+ nodename := Nodename,
+ mnesia_base_dir := MnesiaBaseDir})
+ when MnesiaBaseDir =/= undefined ->
+ filename:join(MnesiaBaseDir, atom_to_list(Nodename));
+get_default_mnesia_dir(#{os_type := {win32, _},
+ nodename := Nodename,
+ mnesia_base_dir := MnesiaBaseDir})
+ when MnesiaBaseDir =/= undefined ->
+ filename:join(MnesiaBaseDir, atom_to_list(Nodename) ++ "-mnesia").
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_QUORUM_DIR
+%% Directory where to store Ra state for quorum queues.
+%% Default: ${RABBITMQ_MNESIA_DIR}/quorum
+
+quorum_queue_dir(#{mnesia_dir := MnesiaDir} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_QUORUM_DIR") of
+ false when MnesiaDir =/= undefined ->
+ Dir = filename:join(MnesiaDir, "quorum"),
+ update_context(Context, quorum_queue_dir, Dir, default);
+ false when MnesiaDir =:= undefined ->
+ update_context(Context, quorum_queue_dir, undefined, default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, quorum_queue_dir, Dir, environment)
+ end.
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_STREAM_DIR
+%% Directory where to store Ra state for stream queues.
+%% Default: ${RABBITMQ_MNESIA_DIR}/stream
+
+stream_queue_dir(#{mnesia_dir := MnesiaDir} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_STREAM_DIR") of
+ false when MnesiaDir =/= undefined ->
+ Dir = filename:join(MnesiaDir, "stream"),
+ update_context(Context, stream_queue_dir, Dir, default);
+ false when MnesiaDir =:= undefined ->
+ update_context(Context, stream_queue_dir, undefined, default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, stream_queue_dir, Dir, environment)
+ end.
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_PID_FILE
+%% File used to write the Erlang VM OS PID.
+%% Default: ${RABBITMQ_MNESIA_DIR}.pid
+%%
+%% RABBITMQ_KEEP_PID_FILE_ON_EXIT
+%% Whether to keep or remove the PID file on Erlang VM exit.
+%% Default: true
+
+pid_file(#{mnesia_base_dir := MnesiaBaseDir,
+ nodename := Nodename} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_PID_FILE") of
+ false when MnesiaBaseDir =/= undefined ->
+ File = filename:join(MnesiaBaseDir,
+ atom_to_list(Nodename) ++ ".pid"),
+ update_context(Context, pid_file, File, default);
+ false when MnesiaBaseDir =:= undefined ->
+ update_context(Context, pid_file, undefined, default);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, pid_file, File, environment)
+ end.
+
+keep_pid_file_on_exit(Context) ->
+ case get_prefixed_env_var("RABBITMQ_KEEP_PID_FILE_ON_EXIT") of
+ false ->
+ update_context(Context, keep_pid_file_on_exit, false, default);
+ Value ->
+ Keep = value_is_yes(Value),
+ update_context(Context, keep_pid_file_on_exit, Keep, environment)
+ end.
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_FEATURE_FLAGS_FILE
+%% File used to store enabled feature flags.
+%% Default: ${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}-feature_flags
+
+feature_flags_file(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_FEATURE_FLAGS_FILE") of
+ false when Remote =:= offline ->
+ update_context(Context, feature_flags_file, undefined, default);
+ false ->
+ feature_flags_file_from_node(Context);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, feature_flags_file, File, environment)
+ end;
+feature_flags_file(Context) ->
+ feature_flags_file_from_env(Context).
+
+feature_flags_file_from_env(#{mnesia_base_dir := MnesiaBaseDir,
+ nodename := Nodename} = Context) ->
+ case get_env_var("RABBITMQ_FEATURE_FLAGS_FILE") of
+ false ->
+ File = filename:join(MnesiaBaseDir,
+ atom_to_list(Nodename) ++ "-feature_flags"),
+ update_context(Context, feature_flags_file, File, default);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, feature_flags_file, File, environment)
+ end.
+
+feature_flags_file_from_node(#{from_remote_node := Remote} = Context) ->
+ Ret = query_remote(Remote,
+ application, get_env, [rabbit, feature_flags_file]),
+ case Ret of
+ {ok, undefined} ->
+ throw({query, Remote, {rabbit, feature_flags_file, undefined}});
+ {ok, {ok, Value}} ->
+ File = normalize_path(Value),
+ update_context(Context, feature_flags_file, File, remote_node);
+ {badrpc, nodedown} ->
+ update_context(Context, feature_flags_file, undefined, default)
+ end.
+
+forced_feature_flags_on_init(Context) ->
+ Value = get_prefixed_env_var("RABBITMQ_FEATURE_FLAGS",
+ [keep_empty_string_as_is]),
+ case Value of
+ false ->
+ %% get_prefixed_env_var() considers an empty string
+ %% is the same as an undefined environment variable.
+ update_context(Context,
+ forced_feature_flags_on_init, undefined, default);
+ _ ->
+ Flags = [list_to_atom(V) || V <- string:lexemes(Value, ",")],
+ update_context(Context,
+ forced_feature_flags_on_init, Flags, environment)
+ end.
+
+log_feature_flags_registry(Context) ->
+ case get_prefixed_env_var("RABBITMQ_LOG_FF_REGISTRY") of
+ false ->
+ update_context(Context,
+ log_feature_flags_registry, false, default);
+ Value ->
+ Log = value_is_yes(Value),
+ update_context(Context,
+ log_feature_flags_registry, Log, environment)
+ end.
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_PLUGINS_DIR
+%% List of directories where to look for plugins.
+%% Directories are separated by:
+%% ':' on Unix
+%% ';' on Windows
+%% Default: ${RABBITMQ_HOME}/plugins
+%%
+%% RABBITMQ_PLUGINS_EXPAND_DIR
+%% Directory where to expand plugin archives.
+%% Default: ${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}-plugins-expand
+%%
+%% RABBITMQ_ENABLED_PLUGINS_FILE
+%% File where the list of enabled plugins is stored.
+%% Default: (Unix) ${SYS_PREFIX}/etc/rabbitmq/enabled_plugins
+%% (Windows) ${RABBITMQ_BASE}\enabled_plugins
+%%
+%% RABBITMQ_ENABLED_PLUGINS
+%% List of plugins to enable on startup.
+%% Values are:
+%% "ALL" to enable all plugins
+%% "" to enable no plugin
+%% a list of plugin names, separated by a coma (',')
+%% Default: Empty (i.e. use ${RABBITMQ_ENABLED_PLUGINS_FILE})
+
+plugins_path(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_PLUGINS_DIR") of
+ false when Remote =:= offline ->
+ update_context(Context, plugins_path, undefined, default);
+ false ->
+ plugins_path_from_node(Context);
+ Path ->
+ update_context(Context, plugins_path, Path, environment)
+ end;
+plugins_path(Context) ->
+ plugins_path_from_env(Context).
+
+plugins_path_from_env(Context) ->
+ case get_prefixed_env_var("RABBITMQ_PLUGINS_DIR") of
+ false ->
+ Path = get_default_plugins_path_from_env(Context),
+ update_context(Context, plugins_path, Path, default);
+ Path ->
+ update_context(Context, plugins_path, Path, environment)
+ end.
+
+plugins_path_from_node(#{from_remote_node := Remote} = Context) ->
+ Ret = query_remote(Remote, application, get_env, [rabbit, plugins_dir]),
+ case Ret of
+ {ok, undefined} ->
+ throw({query, Remote, {rabbit, plugins_dir, undefined}});
+ {ok, {ok, Path}} ->
+ update_context(Context, plugins_path, Path, remote_node);
+ {badrpc, nodedown} ->
+ update_context(Context, plugins_path, undefined, default)
+ end.
+
+get_default_plugins_path(#{from_remote_node := offline}) ->
+ undefined;
+get_default_plugins_path(#{from_remote_node := Remote}) ->
+ get_default_plugins_path_from_node(Remote);
+get_default_plugins_path(Context) ->
+ get_default_plugins_path_from_env(Context).
+
+get_default_plugins_path_from_env(#{os_type := OSType}) ->
+ ThisModDir = this_module_dir(),
+ PluginsDir = rabbit_common_mod_location_to_plugins_dir(ThisModDir),
+ case {OSType, PluginsDir} of
+ {{unix, _}, "/usr/lib/rabbitmq/" ++ _} ->
+ UserPluginsDir = filename:join(
+ ["/", "usr", "lib", "rabbitmq", "plugins"]),
+ UserPluginsDir ++ ":" ++ PluginsDir;
+ _ ->
+ PluginsDir
+ end.
+
+get_default_plugins_path_from_node(Remote) ->
+ Ret = query_remote(Remote, code, where_is_file, ["rabbit_common.app"]),
+ case Ret of
+ {ok, non_existing = Error} ->
+ throw({query, Remote, {code, where_is_file, Error}});
+ {ok, Path} ->
+ rabbit_common_mod_location_to_plugins_dir(filename:dirname(Path));
+ {badrpc, nodedown} ->
+ undefined
+ end.
+
+rabbit_common_mod_location_to_plugins_dir(ModDir) ->
+ case filename:basename(ModDir) of
+ "ebin" ->
+ case filelib:is_dir(ModDir) of
+ false ->
+ %% rabbit_common in the plugin's .ez archive.
+ filename:dirname(
+ filename:dirname(
+ filename:dirname(ModDir)));
+ true ->
+ %% rabbit_common in the plugin's directory.
+ filename:dirname(
+ filename:dirname(ModDir))
+ end;
+ _ ->
+ %% rabbit_common in the CLI escript.
+ filename:join(
+ filename:dirname(
+ filename:dirname(ModDir)),
+ "plugins")
+ end.
+
+plugins_expand_dir(#{mnesia_base_dir := MnesiaBaseDir,
+ nodename := Nodename} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_PLUGINS_EXPAND_DIR") of
+ false when MnesiaBaseDir =/= undefined ->
+ Dir = filename:join(
+ MnesiaBaseDir,
+ atom_to_list(Nodename) ++ "-plugins-expand"),
+ update_context(Context, plugins_expand_dir, Dir, default);
+ false when MnesiaBaseDir =:= undefined ->
+ update_context(Context, plugins_expand_dir, undefined, default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, plugins_expand_dir, Dir, environment)
+ end.
+
+enabled_plugins_file(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_ENABLED_PLUGINS_FILE") of
+ false when Remote =:= offline ->
+ update_context(Context, enabled_plugins_file, undefined, default);
+ false ->
+ enabled_plugins_file_from_node(Context);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, enabled_plugins_file, File, environment)
+ end;
+enabled_plugins_file(Context) ->
+ enabled_plugins_file_from_env(Context).
+
+enabled_plugins_file_from_env(Context) ->
+ case get_prefixed_env_var("RABBITMQ_ENABLED_PLUGINS_FILE") of
+ false ->
+ File = get_default_enabled_plugins_file(Context),
+ update_context(Context, enabled_plugins_file, File, default);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, enabled_plugins_file, File, environment)
+ end.
+
+get_default_enabled_plugins_file(#{config_base_dir := ConfigBaseDir}) ->
+ filename:join(ConfigBaseDir, "enabled_plugins").
+
+enabled_plugins_file_from_node(#{from_remote_node := Remote} = Context) ->
+ Ret = query_remote(Remote,
+ application, get_env, [rabbit, enabled_plugins_file]),
+ case Ret of
+ {ok, undefined} ->
+ throw({query, Remote, {rabbit, enabled_plugins_file, undefined}});
+ {ok, {ok, Value}} ->
+ File = normalize_path(Value),
+ update_context(Context, enabled_plugins_file, File, remote_node);
+ {badrpc, nodedown} ->
+ update_context(Context, enabled_plugins_file, undefined, default)
+ end.
+
+enabled_plugins(Context) ->
+ Value = get_prefixed_env_var(
+ "RABBITMQ_ENABLED_PLUGINS",
+ [keep_empty_string_as_is]),
+ case Value of
+ false ->
+ update_context(Context, enabled_plugins, undefined, default);
+ "ALL" ->
+ update_context(Context, enabled_plugins, all, environment);
+ "" ->
+ update_context(Context, enabled_plugins, [], environment);
+ _ ->
+ Plugins = [list_to_atom(P) || P <- string:lexemes(Value, ",")],
+ update_context(Context, enabled_plugins, Plugins, environment)
+ end.
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_NODE_IP_ADDRESS
+%% AMQP TCP IP address to listen on
+%% Default: unset (i.e. listen on all interfaces)
+%%
+%% RABBITMQ_NODE_PORT
+%% AMQP TCP port.
+%% Default: 5672
+%%
+%% RABBITMQ_DIST_PORT
+%% Erlang distribution TCP port.
+%% Default: ${RABBITMQ_NODE_PORT} + 20000
+
+amqp_ipaddr(Context) ->
+ case get_prefixed_env_var("RABBITMQ_NODE_IP_ADDRESS") of
+ false ->
+ update_context(Context, amqp_ipaddr, "auto", default);
+ Value ->
+ update_context(Context, amqp_ipaddr, Value, environment)
+ end.
+
+amqp_tcp_port(Context) ->
+ case get_prefixed_env_var("RABBITMQ_NODE_PORT") of
+ false ->
+ update_context(Context, amqp_tcp_port, 5672, default);
+ TcpPortStr ->
+ try
+ TcpPort = erlang:list_to_integer(TcpPortStr),
+ update_context(Context, amqp_tcp_port, TcpPort, environment)
+ catch
+ _:badarg ->
+ rabbit_log_prelaunch:error(
+ "Invalid value for $RABBITMQ_NODE_PORT: ~p",
+ [TcpPortStr]),
+ throw({exit, ex_config})
+ end
+ end.
+
+erlang_dist_tcp_port(#{amqp_tcp_port := AmqpTcpPort} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_DIST_PORT") of
+ false ->
+ TcpPort = AmqpTcpPort + 20000,
+ update_context(Context, erlang_dist_tcp_port, TcpPort, default);
+ TcpPortStr ->
+ try
+ TcpPort = erlang:list_to_integer(TcpPortStr),
+ update_context(Context,
+ erlang_dist_tcp_port, TcpPort, environment)
+ catch
+ _:badarg ->
+ rabbit_log_prelaunch:error(
+ "Invalid value for $RABBITMQ_DIST_PORT: ~p",
+ [TcpPortStr]),
+ throw({exit, ex_config})
+ end
+ end.
+
+%% -------------------------------------------------------------------
+%%
+%% SYS_PREFIX [Unix only]
+%% Default: ""
+%%
+%% RABBITMQ_BASE [Windows only]
+%% Directory where to put RabbitMQ data.
+%% Default: !APPDATA!\RabbitMQ
+
+sys_prefix(#{os_type := {unix, _}} = Context) ->
+ case get_env_var("SYS_PREFIX") of
+ false ->
+ update_context(Context, sys_prefix, "", default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, sys_prefix, Dir, environment)
+ end;
+sys_prefix(Context) ->
+ Context.
+
+rabbitmq_base(#{os_type := {win32, _}} = Context) ->
+ case get_env_var("RABBITMQ_BASE") of
+ false ->
+ AppData = normalize_path(get_env_var("APPDATA")),
+ Dir = filename:join(AppData, "RabbitMQ"),
+ update_context(Context, rabbitmq_base, Dir, default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, rabbitmq_base, Dir, environment)
+ end;
+rabbitmq_base(Context) ->
+ Context.
+
+data_dir(#{os_type := {unix, _},
+ sys_prefix := SysPrefix} = Context) ->
+ Dir = filename:join([SysPrefix, "var", "lib", "rabbitmq"]),
+ update_context(Context, data_dir, Dir);
+data_dir(#{os_type := {win32, _},
+ rabbitmq_base := RabbitmqBase} = Context) ->
+ update_context(Context, data_dir, RabbitmqBase).
+
+rabbitmq_home(Context) ->
+ case get_env_var("RABBITMQ_HOME") of
+ false ->
+ Dir = filename:dirname(get_default_plugins_path(Context)),
+ update_context(Context, rabbitmq_home, Dir, default);
+ Value ->
+ Dir = normalize_path(Value),
+ update_context(Context, rabbitmq_home, Dir, environment)
+ end.
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_ALLOW_INPUT
+%% Indicate if an Erlang shell is started or not.
+%% Default: false
+
+interactive_shell(Context) ->
+ case get_env_var("RABBITMQ_ALLOW_INPUT") of
+ false ->
+ update_context(Context,
+ interactive_shell, false, default);
+ Value ->
+ update_context(Context,
+ interactive_shell, value_is_yes(Value), environment)
+ end.
+
+%% FIXME: We would need a way to call isatty(3) to make sure the output
+%% is a terminal.
+output_supports_colors(#{os_type := {unix, _}} = Context) ->
+ update_context(Context, output_supports_colors, true, default);
+output_supports_colors(#{os_type := {win32, _}} = Context) ->
+ update_context(Context, output_supports_colors, false, default).
+
+%% -------------------------------------------------------------------
+%%
+%% RABBITMQ_PRODUCT_NAME
+%% Override the product name
+%% Default: unset (i.e. "RabbitMQ")
+%%
+%% RABBITMQ_PRODUCT_VERSION
+%% Override the product version
+%% Default: unset (i.e. `rabbit` application version).
+%%
+%% RABBITMQ_MOTD_FILE
+%% Indicate a filename containing a "message of the day" to add to
+%% the banners, both the logged and the printed ones.
+%% Default: (Unix) ${SYS_PREFIX}/etc/rabbitmq/motd
+%% (Windows) ${RABBITMQ_BASE}\motd.txt
+
+product_name(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_PRODUCT_NAME") of
+ false when Remote =:= offline ->
+ update_context(Context, product_name, undefined, default);
+ false ->
+ product_name_from_node(Context);
+ Value ->
+ update_context(Context, product_name, Value, environment)
+ end;
+product_name(Context) ->
+ product_name_from_env(Context).
+
+product_name_from_env(Context) ->
+ case get_prefixed_env_var("RABBITMQ_PRODUCT_NAME") of
+ false ->
+ update_context(Context, product_name, undefined, default);
+ Value ->
+ update_context(Context, product_name, Value, environment)
+ end.
+
+product_name_from_node(#{from_remote_node := Remote} = Context) ->
+ Ret = (catch query_remote(Remote, rabbit, product_name, [])),
+ case Ret of
+ {badrpc, nodedown} ->
+ update_context(Context, product_name, undefined, default);
+ {query, _, _} ->
+ update_context(Context, product_name, undefined, default);
+ Value ->
+ update_context(Context, product_name, Value, remote_node)
+ end.
+
+product_version(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_PRODUCT_VERSION") of
+ false when Remote =:= offline ->
+ update_context(Context, product_version, undefined, default);
+ false ->
+ product_version_from_node(Context);
+ Value ->
+ update_context(Context, product_version, Value, environment)
+ end;
+product_version(Context) ->
+ product_version_from_env(Context).
+
+product_version_from_env(Context) ->
+ case get_prefixed_env_var("RABBITMQ_PRODUCT_VERSION") of
+ false ->
+ update_context(Context, product_version, undefined, default);
+ Value ->
+ update_context(Context, product_version, Value, environment)
+ end.
+
+product_version_from_node(#{from_remote_node := Remote} = Context) ->
+ Ret = (catch query_remote(Remote, rabbit, product_version, [])),
+ case Ret of
+ {badrpc, _} ->
+ update_context(Context, product_version, undefined, default);
+ {query, _, _} ->
+ update_context(Context, product_version, undefined, default);
+ Value ->
+ update_context(Context, product_version, Value, remote_node)
+ end.
+
+motd_file(#{from_remote_node := Remote} = Context) ->
+ case get_prefixed_env_var("RABBITMQ_MOTD_FILE") of
+ false when Remote =:= offline ->
+ update_context(Context, motd_file, undefined, default);
+ false ->
+ motd_file_from_node(Context);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, motd_file, File, environment)
+ end;
+motd_file(Context) ->
+ motd_file_from_env(Context).
+
+motd_file_from_env(Context) ->
+ case get_prefixed_env_var("RABBITMQ_MOTD_FILE") of
+ false ->
+ File = get_default_motd_file(Context),
+ update_context(Context, motd_file, File, default);
+ Value ->
+ File = normalize_path(Value),
+ update_context(Context, motd_file, File, environment)
+ end.
+
+get_default_motd_file(#{os_type := {unix, _},
+ config_base_dir := ConfigBaseDir}) ->
+ filename:join(ConfigBaseDir, "motd");
+get_default_motd_file(#{os_type := {win32, _},
+ config_base_dir := ConfigBaseDir}) ->
+ filename:join(ConfigBaseDir, "motd.txt").
+
+motd_file_from_node(#{from_remote_node := Remote} = Context) ->
+ Ret = (catch query_remote(Remote, rabbit, motd_file, [])),
+ case Ret of
+ {badrpc, _} ->
+ update_context(Context, motd_file, undefined, default);
+ {query, _, _} ->
+ update_context(Context, motd_file, undefined, default);
+ File ->
+ update_context(Context, motd_file, File, remote_node)
+ end.
+
+%% -------------------------------------------------------------------
+%% Loading of rabbitmq-env.conf.
+%% -------------------------------------------------------------------
+
+load_conf_env_file(#{os_type := {unix, _},
+ sys_prefix := SysPrefix} = Context) ->
+ {ConfEnvFile, Origin} =
+ case get_prefixed_env_var("RABBITMQ_CONF_ENV_FILE") of
+ false ->
+ File = filename:join(
+ [SysPrefix, "etc", "rabbitmq", "rabbitmq-env.conf"]),
+ {File, default};
+ Value ->
+ {normalize_path(Value), environment}
+ end,
+ Context1 = update_context(Context, conf_env_file, ConfEnvFile, Origin),
+ case loading_conf_env_file_enabled(Context1) of
+ true ->
+ case filelib:is_regular(ConfEnvFile) of
+ false ->
+ rabbit_log_prelaunch:debug(
+ "No $RABBITMQ_CONF_ENV_FILE (~ts)", [ConfEnvFile]),
+ Context1;
+ true ->
+ case os:find_executable("sh") of
+ false -> Context1;
+ Sh -> do_load_conf_env_file(Context1,
+ Sh,
+ ConfEnvFile)
+ end
+ end;
+ false ->
+ rabbit_log_prelaunch:debug(
+ "Loading of $RABBITMQ_CONF_ENV_FILE (~ts) is disabled",
+ [ConfEnvFile]),
+ Context1
+ end;
+load_conf_env_file(#{os_type := {win32, _},
+ rabbitmq_base := RabbitmqBase} = Context) ->
+ {ConfEnvFile, Origin} =
+ case get_prefixed_env_var("RABBITMQ_CONF_ENV_FILE") of
+ false ->
+ File = filename:join([RabbitmqBase, "rabbitmq-env-conf.bat"]),
+ {File, default};
+ Value ->
+ {normalize_path(Value), environment}
+ end,
+ Context1 = update_context(Context, conf_env_file, ConfEnvFile, Origin),
+ case loading_conf_env_file_enabled(Context1) of
+ true ->
+ case filelib:is_regular(ConfEnvFile) of
+ false ->
+ rabbit_log_prelaunch:debug(
+ "No $RABBITMQ_CONF_ENV_FILE (~ts)", [ConfEnvFile]),
+ Context1;
+ true ->
+ case os:find_executable("cmd.exe") of
+ false ->
+ Cmd = os:getenv("ComSpec"),
+ CmdExists =
+ Cmd =/= false andalso
+ filelib:is_regular(Cmd),
+ case CmdExists of
+ false -> Context1;
+ true -> do_load_conf_env_file(Context1,
+ Cmd,
+ ConfEnvFile)
+ end;
+ Cmd ->
+ do_load_conf_env_file(Context1, Cmd, ConfEnvFile)
+ end
+ end;
+ false ->
+ rabbit_log_prelaunch:debug(
+ "Loading of $RABBITMQ_CONF_ENV_FILE (~ts) is disabled",
+ [ConfEnvFile]),
+ Context1
+ end;
+load_conf_env_file(Context) ->
+ Context.
+
+-spec loading_conf_env_file_enabled(map()) -> boolean().
+
+-ifdef(TEST).
+loading_conf_env_file_enabled(_) ->
+ persistent_term:get({?MODULE, load_conf_env_file}, true).
+-else.
+loading_conf_env_file_enabled(_) ->
+ %% When this module is built without `TEST` defined, we want this
+ %% function to always return true. However, this makes Dialyzer
+ %% think it can only return true: this is not the case when the
+ %% module is compiled with `TEST` defined. The following line is
+ %% here to trick Dialyzer.
+ erlang:get({?MODULE, always_undefined}) =:= undefined.
+-endif.
+
+do_load_conf_env_file(#{os_type := {unix, _}} = Context, Sh, ConfEnvFile) ->
+ rabbit_log_prelaunch:debug(
+ "Sourcing $RABBITMQ_CONF_ENV_FILE: ~ts", [ConfEnvFile]),
+
+ %% The script below sources the `CONF_ENV_FILE` file, then it shows a
+ %% marker line and all environment variables.
+ %%
+ %% The marker line is useful to distinguish any output from the sourced
+ %% script from the variables we are interested in.
+ Marker = vars_list_marker(),
+ Script = rabbit_misc:format(
+ ". \"~ts\" && "
+ "echo \"~s\" && "
+ "set", [ConfEnvFile, Marker]),
+
+ #{sys_prefix := SysPrefix,
+ rabbitmq_home := RabbitmqHome} = Context,
+ MainConfigFile = re:replace(
+ get_default_main_config_file(Context),
+ "\\.(conf|config)$", "", [{return, list}]),
+
+ %% The variables below are those the `CONF_ENV_FILE` file can expect.
+ Env = [
+ {"SYS_PREFIX", SysPrefix},
+ {"RABBITMQ_HOME", RabbitmqHome},
+ {"CONFIG_FILE", MainConfigFile},
+ {"ADVANCED_CONFIG_FILE", get_default_advanced_config_file(Context)},
+ {"MNESIA_BASE", get_default_mnesia_base_dir(Context)},
+ {"ENABLED_PLUGINS_FILE", get_default_enabled_plugins_file(Context)},
+ {"PLUGINS_DIR", get_default_plugins_path_from_env(Context)},
+ {"CONF_ENV_FILE_PHASE", "rabbtimq-prelaunch"}
+ ],
+
+ Args = ["-ex", "-c", Script],
+ Opts = [{args, Args},
+ {env, Env},
+ binary,
+ use_stdio,
+ stderr_to_stdout,
+ exit_status],
+ Port = erlang:open_port({spawn_executable, Sh}, Opts),
+ collect_conf_env_file_output(Context, Port, Marker, <<>>);
+do_load_conf_env_file(#{os_type := {win32, _}} = Context, Cmd, ConfEnvFile) ->
+ %% rabbitmq/rabbitmq-common#392
+ rabbit_log_prelaunch:debug(
+ "Executing $RABBITMQ_CONF_ENV_FILE: ~ts", [ConfEnvFile]),
+
+ %% The script below executes the `CONF_ENV_FILE` file, then it shows a
+ %% marker line and all environment variables.
+ %%
+ %% The marker line is useful to distinguish any output from the sourced
+ %% script from the variables we are interested in.
+ %%
+ %% Arguments are split into a list of strings to support a filename with
+ %% whitespaces in the path.
+ Marker = vars_list_marker(),
+ Script = [ConfEnvFile, "&&",
+ "echo", Marker, "&&",
+ "set"],
+
+ #{rabbitmq_base := RabbitmqBase,
+ rabbitmq_home := RabbitmqHome} = Context,
+ MainConfigFile = re:replace(
+ get_default_main_config_file(Context),
+ "\\.(conf|config)$", "", [{return, list}]),
+
+ %% The variables below are those the `CONF_ENV_FILE` file can expect.
+ Env = [
+ {"RABBITMQ_BASE", RabbitmqBase},
+ {"RABBITMQ_HOME", RabbitmqHome},
+ {"CONFIG_FILE", MainConfigFile},
+ {"ADVANCED_CONFIG_FILE", get_default_advanced_config_file(Context)},
+ {"MNESIA_BASE", get_default_mnesia_base_dir(Context)},
+ {"ENABLED_PLUGINS_FILE", get_default_enabled_plugins_file(Context)},
+ {"PLUGINS_DIR", get_default_plugins_path_from_env(Context)},
+ {"CONF_ENV_FILE_PHASE", "rabbtimq-prelaunch"}
+ ],
+
+ Args = ["/Q", "/C" | Script],
+ Opts = [{args, Args},
+ {env, Env},
+ hide,
+ binary,
+ stderr_to_stdout,
+ exit_status],
+ Port = erlang:open_port({spawn_executable, Cmd}, Opts),
+ collect_conf_env_file_output(Context, Port, "\"" ++ Marker ++ "\" ", <<>>).
+
+vars_list_marker() ->
+ rabbit_misc:format(
+ "-----BEGIN VARS LIST FOR PID ~s-----", [os:getpid()]).
+
+collect_conf_env_file_output(Context, Port, Marker, Output) ->
+ receive
+ {Port, {exit_status, ExitStatus}} ->
+ Lines = post_port_cmd_output(Context, Output, ExitStatus),
+ case ExitStatus of
+ 0 -> parse_conf_env_file_output(Context, Marker, Lines);
+ _ -> Context
+ end;
+ {Port, {data, Chunk}} ->
+ collect_conf_env_file_output(
+ Context, Port, Marker, [Output, Chunk])
+ end.
+
+post_port_cmd_output(#{os_type := {OSType, _}}, Output, ExitStatus) ->
+ rabbit_log_prelaunch:debug(
+ "$RABBITMQ_CONF_ENV_FILE exit status: ~b",
+ [ExitStatus]),
+ DecodedOutput = unicode:characters_to_list(Output),
+ LineSep = case OSType of
+ win32 -> "\r\n";
+ _ -> "\n"
+ end,
+ Lines = string:split(string:trim(DecodedOutput), LineSep, all),
+ rabbit_log_prelaunch:debug("$RABBITMQ_CONF_ENV_FILE output:"),
+ [rabbit_log_prelaunch:debug(" ~ts", [Line]) || Line <- Lines],
+ Lines.
+
+parse_conf_env_file_output(Context, _, []) ->
+ Context;
+parse_conf_env_file_output(Context, Marker, [Marker | Lines]) ->
+ %% Found our marker, let's parse variables.
+ parse_conf_env_file_output1(Context, Lines);
+parse_conf_env_file_output(Context, Marker, [_ | Lines]) ->
+ parse_conf_env_file_output(Context, Marker, Lines).
+
+parse_conf_env_file_output1(Context, Lines) ->
+ Vars = parse_conf_env_file_output2(Lines, #{}),
+ %% Re-export variables.
+ lists:foreach(
+ fun(Var) ->
+ IsUsed = var_is_used(Var),
+ IsSet = var_is_set(Var),
+ case IsUsed andalso not IsSet of
+ true ->
+ rabbit_log_prelaunch:debug(
+ "$RABBITMQ_CONF_ENV_FILE: re-exporting variable $~s",
+ [Var]),
+ os:putenv(Var, maps:get(Var, Vars));
+ false ->
+ ok
+ end
+ end, lists:sort(maps:keys(Vars))),
+ Context.
+
+parse_conf_env_file_output2([], Vars) ->
+ Vars;
+parse_conf_env_file_output2([Line | Lines], Vars) ->
+ SetXOutput = is_sh_set_x_output(Line),
+ ShFunction = is_sh_function(Line, Lines),
+ if
+ SetXOutput ->
+ parse_conf_env_file_output2(Lines, Vars);
+ ShFunction ->
+ skip_sh_function(Lines, Vars);
+ true ->
+ case string:split(Line, "=") of
+ [Var, IncompleteValue] ->
+ {Value, Lines1} = parse_sh_literal(IncompleteValue, Lines, ""),
+ Vars1 = Vars#{Var => Value},
+ parse_conf_env_file_output2(Lines1, Vars1);
+ _ ->
+ %% Parsing failed somehow.
+ rabbit_log_prelaunch:warning(
+ "Failed to parse $RABBITMQ_CONF_ENV_FILE output: ~p",
+ [Line]),
+ #{}
+ end
+ end.
+
+is_sh_set_x_output(Line) ->
+ re:run(Line, "^\\++ ", [{capture, none}]) =:= match.
+
+is_sh_function(_, []) ->
+ false;
+is_sh_function(Line, Lines) ->
+ re:run(Line, "\\s\\(\\)\\s*$", [{capture, none}]) =:= match
+ andalso
+ re:run(hd(Lines), "^\\s*\\{\\s*$", [{capture, none}]) =:= match.
+
+parse_sh_literal("'" ++ SingleQuoted, Lines, Literal) ->
+ parse_single_quoted_literal(SingleQuoted, Lines, Literal);
+parse_sh_literal("\"" ++ DoubleQuoted, Lines, Literal) ->
+ parse_double_quoted_literal(DoubleQuoted, Lines, Literal);
+parse_sh_literal("$'" ++ DollarSingleQuoted, Lines, Literal) ->
+ parse_dollar_single_quoted_literal(DollarSingleQuoted, Lines, Literal);
+parse_sh_literal(Unquoted, Lines, Literal) ->
+ {lists:reverse(Literal) ++ Unquoted, Lines}.
+
+parse_single_quoted_literal([$' | Rest], Lines, Literal) ->
+ %% We reached the closing single quote.
+ parse_sh_literal(Rest, Lines, Literal);
+parse_single_quoted_literal([], [Line | Lines], Literal) ->
+ %% We reached the end of line before finding the closing single
+ %% quote. The literal continues on the next line and includes that
+ %% newline character.
+ parse_single_quoted_literal(Line, Lines, [$\n | Literal]);
+parse_single_quoted_literal([C | Rest], Lines, Literal) ->
+ parse_single_quoted_literal(Rest, Lines, [C | Literal]).
+
+parse_double_quoted_literal([$" | Rest], Lines, Literal) ->
+ %% We reached the closing double quote.
+ parse_sh_literal(Rest, Lines, Literal);
+parse_double_quoted_literal([], [Line | Lines], Literal) ->
+ %% We reached the end of line before finding the closing double
+ %% quote. The literal continues on the next line and includes that
+ %% newline character.
+ parse_double_quoted_literal(Line, Lines, [$\n | Literal]);
+parse_double_quoted_literal([C | Rest], Lines, Literal) ->
+ parse_double_quoted_literal(Rest, Lines, [C | Literal]).
+
+parse_dollar_single_quoted_literal([$'], Lines, Literal) ->
+ %% We reached the closing single quote.
+ {lists:reverse(Literal), Lines};
+parse_dollar_single_quoted_literal([], [Line | Lines], Literal) ->
+ %% We reached the end of line before finding the closing single
+ %% quote. The literal continues on the next line and includes that
+ %% newline character.
+ parse_dollar_single_quoted_literal(Line, Lines, [$\n | Literal]);
+parse_dollar_single_quoted_literal([C | Rest], Lines, Literal) ->
+ parse_dollar_single_quoted_literal(Rest, Lines, [C | Literal]).
+
+skip_sh_function(["}" | Lines], Vars) ->
+ parse_conf_env_file_output2(Lines, Vars);
+skip_sh_function([_ | Lines], Vars) ->
+ skip_sh_function(Lines, Vars).
+
+%% -------------------------------------------------------------------
+%% Helpers.
+%% -------------------------------------------------------------------
+
+get_env_var(VarName) ->
+ get_env_var(VarName, []).
+
+get_env_var(VarName, Options) ->
+ KeepEmptyString = lists:member(keep_empty_string_as_is, Options),
+ case os:getenv(VarName) of
+ false -> false;
+ "" when not KeepEmptyString -> false;
+ Value -> Value
+ end.
+
+get_prefixed_env_var(VarName) ->
+ get_prefixed_env_var(VarName, []).
+
+get_prefixed_env_var("RABBITMQ_" ++ Suffix = VarName,
+ Options) ->
+ case get_env_var(VarName, Options) of
+ false -> get_env_var(Suffix, Options);
+ Value -> Value
+ end.
+
+var_is_used("RABBITMQ_" ++ _ = PrefixedVar) ->
+ lists:member(PrefixedVar, ?USED_ENV_VARS);
+var_is_used("HOME") ->
+ false;
+var_is_used(Var) ->
+ lists:member("RABBITMQ_" ++ Var, ?USED_ENV_VARS).
+
+%% The $RABBITMQ_* variables have precedence over their un-prefixed equivalent.
+%% Therefore, when we check if $RABBITMQ_* is set, we only look at this
+%% variable. However, when we check if an un-prefixed variable is set, we first
+%% look at its $RABBITMQ_* variant.
+var_is_set("RABBITMQ_" ++ _ = PrefixedVar) ->
+ os:getenv(PrefixedVar) /= false;
+var_is_set(Var) ->
+ os:getenv("RABBITMQ_" ++ Var) /= false orelse
+ os:getenv(Var) /= false.
+
+value_is_yes(Value) when is_list(Value) orelse is_binary(Value) ->
+ Options = [{capture, none}, caseless],
+ re:run(string:trim(Value), "^(1|yes|true)$", Options) =:= match;
+value_is_yes(_) ->
+ false.
+
+normalize_path("" = Path) ->
+ Path;
+normalize_path(Path) ->
+ filename:join(filename:split(Path)).
+
+this_module_dir() ->
+ File = code:which(?MODULE),
+ %% Possible locations:
+ %% - the rabbit_common plugin (as an .ez archive):
+ %% .../plugins/rabbit_common-$version.ez/rabbit_common-$version/ebin
+ %% - the rabbit_common plugin (as a directory):
+ %% .../plugins/rabbit_common-$version/ebin
+ %% - the CLI:
+ %% .../escript/$cli
+ filename:dirname(File).
+
+maybe_setup_dist_for_remote_query(
+ #{from_remote_node := offline} = Context) ->
+ Context;
+maybe_setup_dist_for_remote_query(
+ #{from_remote_node := {RemoteNode, _}} = Context) ->
+ {NamePart, HostPart} = rabbit_nodes_common:parts(RemoteNode),
+ NameType = rabbit_nodes_common:name_type(RemoteNode),
+ ok = rabbit_nodes_common:ensure_epmd(),
+ Context1 = setup_dist_for_remote_query(
+ Context, NamePart, HostPart, NameType, 50),
+ case is_rabbitmq_loaded_on_remote_node(Context1) of
+ true -> Context1;
+ false -> maybe_stop_dist_for_remote_query(
+ update_context(Context, from_remote_node, offline))
+ end;
+maybe_setup_dist_for_remote_query(Context) ->
+ Context.
+
+setup_dist_for_remote_query(
+ #{dist_started_for_remote_query := true} = Context,
+ _, _, _, _) ->
+ Context;
+setup_dist_for_remote_query(Context, _, _, _, 0) ->
+ Context;
+setup_dist_for_remote_query(#{from_remote_node := {Remote, _}} = Context,
+ NamePart, HostPart, NameType,
+ Attempts) ->
+ RndNamePart = NamePart ++ "_ctl_" ++ integer_to_list(rand:uniform(100)),
+ Nodename = rabbit_nodes_common:make({RndNamePart, HostPart}),
+ case net_kernel:start([Nodename, NameType]) of
+ {ok, _} ->
+ update_context(Context, dist_started_for_remote_query, true);
+ {error, {already_started, _}} ->
+ Context;
+ {error, {{already_started, _}, _}} ->
+ Context;
+ Error ->
+ logger:error(
+ "rabbit_env: Failed to setup distribution (as ~s) to "
+ "query node ~s: ~p",
+ [Nodename, Remote, Error]),
+ setup_dist_for_remote_query(Context,
+ NamePart, HostPart, NameType,
+ Attempts - 1)
+ end.
+
+is_rabbitmq_loaded_on_remote_node(
+ #{from_remote_node := Remote}) ->
+ case query_remote(Remote, application, loaded_applications, []) of
+ {ok, Apps} ->
+ lists:keymember(mnesia, 1, Apps) andalso
+ lists:keymember(rabbit, 1, Apps);
+ _ ->
+ false
+ end.
+
+maybe_stop_dist_for_remote_query(
+ #{dist_started_for_remote_query := true} = Context) ->
+ net_kernel:stop(),
+ maps:remove(dist_started_for_remote_query, Context);
+maybe_stop_dist_for_remote_query(Context) ->
+ Context.
+
+query_remote({RemoteNode, Timeout}, Mod, Func, Args) ->
+ Ret = rpc:call(RemoteNode, Mod, Func, Args, Timeout),
+ case Ret of
+ {badrpc, nodedown} = Error -> Error;
+ {badrpc, _} = Error -> throw({query, RemoteNode, Error});
+ _ -> {ok, Ret}
+ end.
diff --git a/deps/rabbit_common/src/rabbit_error_logger_handler.erl b/deps/rabbit_common/src/rabbit_error_logger_handler.erl
new file mode 100644
index 0000000000..714790a449
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_error_logger_handler.erl
@@ -0,0 +1,169 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_error_logger_handler).
+
+-behaviour(gen_event).
+
+%% API
+-export([start_link/0, add_handler/0]).
+
+%% gen_event callbacks
+-export([init/1, handle_event/2, handle_call/2,
+ handle_info/2, terminate/2, code_change/3]).
+
+-define(SERVER, ?MODULE).
+
+-record(state, {report = []}).
+
+%%%===================================================================
+%%% API
+%%%===================================================================
+
+%%--------------------------------------------------------------------
+%% @doc
+%% Creates an event manager
+%%
+%% @spec start_link() -> {ok, Pid} | {error, Error}
+%% @end
+%%--------------------------------------------------------------------
+start_link() ->
+ gen_event:start_link({local, ?SERVER}).
+
+%%--------------------------------------------------------------------
+%% @doc
+%% Adds an event handler
+%%
+%% @spec add_handler() -> ok | {'EXIT', Reason} | term()
+%% @end
+%%--------------------------------------------------------------------
+add_handler() ->
+ gen_event:add_handler(?SERVER, ?MODULE, []).
+
+%%%===================================================================
+%%% gen_event callbacks
+%%%===================================================================
+
+%%--------------------------------------------------------------------
+%% @private
+%% @doc
+%% Whenever a new event handler is added to an event manager,
+%% this function is called to initialize the event handler.
+%%
+%% @spec init(Args) -> {ok, State}
+%% @end
+%%--------------------------------------------------------------------
+init([]) ->
+ {ok, #state{}}.
+
+%%--------------------------------------------------------------------
+%% @private
+%% @doc
+%% Whenever an event manager receives an event sent using
+%% gen_event:notify/2 or gen_event:sync_notify/2, this function is
+%% called for each installed event handler to handle the event.
+%%
+%% @spec handle_event(Event, State) ->
+%% {ok, State} |
+%% {swap_handler, Args1, State1, Mod2, Args2} |
+%% remove_handler
+%% @end
+%%--------------------------------------------------------------------
+
+handle_event({info_report, _Gleader, {_Pid, _Type,
+ {net_kernel, {'EXIT', _, Reason}}}},
+ #state{report = Report} = State) ->
+ NewReport = case format(Reason) of
+ [] -> Report;
+ Formatted -> [Formatted | Report]
+ end,
+ {ok, State#state{report = NewReport}};
+handle_event(_Event, State) ->
+ {ok, State}.
+
+%%--------------------------------------------------------------------
+%% @private
+%% @doc
+%% Whenever an event manager receives a request sent using
+%% gen_event:call/3,4, this function is called for the specified
+%% event handler to handle the request.
+%%
+%% @spec handle_call(Request, State) ->
+%% {ok, Reply, State} |
+%% {swap_handler, Reply, Args1, State1, Mod2, Args2} |
+%% {remove_handler, Reply}
+%% @end
+%%--------------------------------------------------------------------
+handle_call(get_connection_report, State) ->
+ {ok, lists:reverse(State#state.report), State#state{report = []}};
+handle_call(_Request, State) ->
+ Reply = ok,
+ {ok, Reply, State}.
+
+%%--------------------------------------------------------------------
+%% @private
+%% @doc
+%% This function is called for each installed event handler when
+%% an event manager receives any other message than an event or a
+%% synchronous request (or a system message).
+%%
+%% @spec handle_info(Info, State) ->
+%% {ok, State} |
+%% {swap_handler, Args1, State1, Mod2, Args2} |
+%% remove_handler
+%% @end
+%%--------------------------------------------------------------------
+handle_info(_Info, State) ->
+ {ok, State}.
+
+%%--------------------------------------------------------------------
+%% @private
+%% @doc
+%% Whenever an event handler is deleted from an event manager, this
+%% function is called. It should be the opposite of Module:init/1 and
+%% do any necessary cleaning up.
+%%
+%% @spec terminate(Reason, State) -> void()
+%% @end
+%%--------------------------------------------------------------------
+terminate(_Reason, _State) ->
+ ok.
+
+%%--------------------------------------------------------------------
+%% @private
+%% @doc
+%% Convert process state when code is changed
+%%
+%% @spec code_change(OldVsn, State, Extra) -> {ok, NewState}
+%% @end
+%%--------------------------------------------------------------------
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+%%%===================================================================
+%%% Internal functions
+%%%===================================================================
+format({check_dflag_xnc_failed, _What}) ->
+ {" * Remote node uses an incompatible Erlang version ~n", []};
+format({recv_challenge_failed, no_node, Node}) ->
+ {" * Node name (or hostname) mismatch: node ~p believes its node name is not ~p but something else.~n"
+ " All nodes and CLI tools must refer to node ~p using the same name the node itself uses (see its logs to find out what it is)~n",
+ [Node, Node, Node]};
+format({recv_challenge_failed, Error}) ->
+ {" * Distribution failed unexpectedly while waiting for challenge: ~p~n", [Error]};
+format({recv_challenge_ack_failed, bad_cookie}) ->
+ {" * Authentication failed (rejected by the local node), please check the Erlang cookie~n", []};
+format({recv_challenge_ack_failed, {error, closed}}) ->
+ {" * Authentication failed (rejected by the remote node), please check the Erlang cookie~n", []};
+format({recv_status_failed, not_allowed}) ->
+ {" * This node is not on the list of nodes authorised by remote node (see net_kernel:allow/1)~n", []};
+format({recv_status_failed, {error, closed}}) ->
+ {" * Remote host closed TCP connection before completing authentication. Is the Erlang distribution using TLS?~n", []};
+format(setup_timer_timeout) ->
+ {" * TCP connection to remote host has timed out. Is the Erlang distribution using TLS?~n", []};
+format(_) ->
+ [].
diff --git a/deps/rabbit_common/src/rabbit_event.erl b/deps/rabbit_common/src/rabbit_event.erl
new file mode 100644
index 0000000000..152335958a
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_event.erl
@@ -0,0 +1,164 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_event).
+
+-include("rabbit.hrl").
+
+-export([start_link/0]).
+-export([init_stats_timer/2, init_disabled_stats_timer/2,
+ ensure_stats_timer/3, stop_stats_timer/2, reset_stats_timer/2]).
+-export([stats_level/2, if_enabled/3]).
+-export([notify/2, notify/3, notify_if/3]).
+-export([sync_notify/2, sync_notify/3]).
+
+-ignore_xref([{gen_event, start_link, 2}]).
+-dialyzer([{no_missing_calls, start_link/0}]).
+
+%%----------------------------------------------------------------------------
+
+-record(state, {level, interval, timer}).
+
+%%----------------------------------------------------------------------------
+
+-export_type([event_type/0, event_props/0, event_timestamp/0, event/0]).
+
+-type event_type() :: atom().
+-type event_props() :: term().
+-type event_timestamp() :: non_neg_integer().
+
+-type event() :: #event { type :: event_type(),
+ props :: event_props(),
+ reference :: 'none' | reference(),
+ timestamp :: event_timestamp() }.
+
+-type level() :: 'none' | 'coarse' | 'fine'.
+
+-type timer_fun() :: fun (() -> 'ok').
+-type container() :: tuple().
+-type pos() :: non_neg_integer().
+
+-spec start_link() -> rabbit_types:ok_pid_or_error().
+-spec init_stats_timer(container(), pos()) -> container().
+-spec init_disabled_stats_timer(container(), pos()) -> container().
+-spec ensure_stats_timer(container(), pos(), term()) -> container().
+-spec stop_stats_timer(container(), pos()) -> container().
+-spec reset_stats_timer(container(), pos()) -> container().
+-spec stats_level(container(), pos()) -> level().
+-spec if_enabled(container(), pos(), timer_fun()) -> 'ok'.
+-spec notify(event_type(), event_props()) -> 'ok'.
+-spec notify(event_type(), event_props(), reference() | 'none') -> 'ok'.
+-spec notify_if(boolean(), event_type(), event_props()) -> 'ok'.
+-spec sync_notify(event_type(), event_props()) -> 'ok'.
+-spec sync_notify(event_type(), event_props(), reference() | 'none') -> 'ok'.
+
+%%----------------------------------------------------------------------------
+
+start_link() ->
+ %% gen_event:start_link/2 is not available before OTP 20
+ %% RabbitMQ 3.7 supports OTP >= 19.3
+ case erlang:function_exported(gen_event, start_link, 2) of
+ true ->
+ gen_event:start_link(
+ {local, ?MODULE},
+ [{spawn_opt, [{fullsweep_after, 0}]}]
+ );
+ false ->
+ gen_event:start_link({local, ?MODULE})
+ end.
+
+%% The idea is, for each stat-emitting object:
+%%
+%% On startup:
+%% init_stats_timer(State)
+%% notify(created event)
+%% if_enabled(internal_emit_stats) - so we immediately send something
+%%
+%% On wakeup:
+%% ensure_stats_timer(State, emit_stats)
+%% (Note we can't emit stats immediately, the timer may have fired 1ms ago.)
+%%
+%% emit_stats:
+%% if_enabled(internal_emit_stats)
+%% reset_stats_timer(State) - just bookkeeping
+%%
+%% Pre-hibernation:
+%% if_enabled(internal_emit_stats)
+%% stop_stats_timer(State)
+%%
+%% internal_emit_stats:
+%% notify(stats)
+
+init_stats_timer(C, P) ->
+ %% If the rabbit app is not loaded - use default none:5000
+ StatsLevel = application:get_env(rabbit, collect_statistics, none),
+ Interval = application:get_env(rabbit, collect_statistics_interval, 5000),
+ setelement(P, C, #state{level = StatsLevel, interval = Interval,
+ timer = undefined}).
+
+init_disabled_stats_timer(C, P) ->
+ setelement(P, C, #state{level = none, interval = 0, timer = undefined}).
+
+ensure_stats_timer(C, P, Msg) ->
+ case element(P, C) of
+ #state{level = Level, interval = Interval, timer = undefined} = State
+ when Level =/= none ->
+ TRef = erlang:send_after(Interval, self(), Msg),
+ setelement(P, C, State#state{timer = TRef});
+ #state{} ->
+ C
+ end.
+
+stop_stats_timer(C, P) ->
+ case element(P, C) of
+ #state{timer = TRef} = State when TRef =/= undefined ->
+ case erlang:cancel_timer(TRef) of
+ false -> C;
+ _ -> setelement(P, C, State#state{timer = undefined})
+ end;
+ #state{} ->
+ C
+ end.
+
+reset_stats_timer(C, P) ->
+ case element(P, C) of
+ #state{timer = TRef} = State when TRef =/= undefined ->
+ setelement(P, C, State#state{timer = undefined});
+ #state{} ->
+ C
+ end.
+
+stats_level(C, P) ->
+ #state{level = Level} = element(P, C),
+ Level.
+
+if_enabled(C, P, Fun) ->
+ case element(P, C) of
+ #state{level = none} -> ok;
+ #state{} -> Fun(), ok
+ end.
+
+notify_if(true, Type, Props) -> notify(Type, Props);
+notify_if(false, _Type, _Props) -> ok.
+
+notify(Type, Props) -> notify(Type, rabbit_data_coercion:to_proplist(Props), none).
+
+notify(Type, Props, Ref) ->
+ %% Using {Name, node()} here to not fail if the event handler is not started
+ gen_event:notify({?MODULE, node()}, event_cons(Type, rabbit_data_coercion:to_proplist(Props), Ref)).
+
+sync_notify(Type, Props) -> sync_notify(Type, Props, none).
+
+sync_notify(Type, Props, Ref) ->
+ gen_event:sync_notify(?MODULE, event_cons(Type, Props, Ref)).
+
+event_cons(Type, Props, Ref) ->
+ #event{type = Type,
+ props = Props,
+ reference = Ref,
+ timestamp = os:system_time(milli_seconds)}.
+
diff --git a/deps/rabbit_common/src/rabbit_exchange_type.erl b/deps/rabbit_common/src/rabbit_exchange_type.erl
new file mode 100644
index 0000000000..ebd5cadbdb
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_exchange_type.erl
@@ -0,0 +1,68 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_exchange_type).
+
+-behaviour(rabbit_registry_class).
+
+-export([added_to_rabbit_registry/2, removed_from_rabbit_registry/1]).
+
+-type(tx() :: 'transaction' | 'none').
+-type(serial() :: pos_integer() | tx()).
+
+-callback description() -> [proplists:property()].
+
+%% Should Rabbit ensure that all binding events that are
+%% delivered to an individual exchange can be serialised? (they
+%% might still be delivered out of order, but there'll be a
+%% serial number).
+-callback serialise_events() -> boolean().
+
+%% The no_return is there so that we can have an "invalid" exchange
+%% type (see rabbit_exchange_type_invalid).
+-callback route(rabbit_types:exchange(), rabbit_types:delivery()) ->
+ rabbit_router:match_result().
+
+%% called BEFORE declaration, to check args etc; may exit with #amqp_error{}
+-callback validate(rabbit_types:exchange()) -> 'ok'.
+
+%% called BEFORE declaration, to check args etc
+-callback validate_binding(rabbit_types:exchange(), rabbit_types:binding()) ->
+ rabbit_types:ok_or_error({'binding_invalid', string(), [any()]}).
+
+%% called after declaration and recovery
+-callback create(tx(), rabbit_types:exchange()) -> 'ok'.
+
+%% called after exchange (auto)deletion.
+-callback delete(tx(), rabbit_types:exchange(), [rabbit_types:binding()]) ->
+ 'ok'.
+
+%% called when the policy attached to this exchange changes.
+-callback policy_changed(rabbit_types:exchange(), rabbit_types:exchange()) ->
+ 'ok'.
+
+%% called after a binding has been added or recovered
+-callback add_binding(serial(), rabbit_types:exchange(),
+ rabbit_types:binding()) -> 'ok'.
+
+%% called after bindings have been deleted.
+-callback remove_bindings(serial(), rabbit_types:exchange(),
+ [rabbit_types:binding()]) -> 'ok'.
+
+%% called when comparing exchanges for equivalence - should return ok or
+%% exit with #amqp_error{}
+-callback assert_args_equivalence(rabbit_types:exchange(),
+ rabbit_framing:amqp_table()) ->
+ 'ok' | rabbit_types:connection_exit().
+
+%% Exchange type specific info keys
+-callback info(rabbit_types:exchange()) -> [{atom(), term()}].
+
+-callback info(rabbit_types:exchange(), [atom()]) -> [{atom(), term()}].
+
+added_to_rabbit_registry(_Type, _ModuleName) -> ok.
+removed_from_rabbit_registry(_Type) -> ok.
diff --git a/deps/rabbit_common/src/rabbit_heartbeat.erl b/deps/rabbit_common/src/rabbit_heartbeat.erl
new file mode 100644
index 0000000000..8dbc7f3887
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_heartbeat.erl
@@ -0,0 +1,184 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_heartbeat).
+
+-export([start/6, start/7]).
+-export([start_heartbeat_sender/4, start_heartbeat_receiver/4,
+ pause_monitor/1, resume_monitor/1]).
+
+-export([system_continue/3, system_terminate/4, system_code_change/4]).
+
+-include("rabbit.hrl").
+
+%%----------------------------------------------------------------------------
+
+-export_type([heartbeaters/0]).
+
+-type heartbeaters() :: {rabbit_types:maybe(pid()), rabbit_types:maybe(pid())}.
+
+-type heartbeat_callback() :: fun (() -> any()).
+
+-export_type([heartbeat_timeout/0]).
+-type heartbeat_timeout() :: non_neg_integer().
+
+-spec start
+ (pid(), rabbit_net:socket(), heartbeat_timeout(), heartbeat_callback(),
+ heartbeat_timeout(), heartbeat_callback()) ->
+ heartbeaters().
+
+-spec start
+ (pid(), rabbit_net:socket(), rabbit_types:proc_name(),
+ heartbeat_timeout(), heartbeat_callback(), heartbeat_timeout(),
+ heartbeat_callback()) ->
+ heartbeaters().
+
+-spec start_heartbeat_sender
+ (rabbit_net:socket(), heartbeat_timeout(), heartbeat_callback(),
+ rabbit_types:proc_type_and_name()) ->
+ rabbit_types:ok(pid()).
+-spec start_heartbeat_receiver
+ (rabbit_net:socket(), heartbeat_timeout(), heartbeat_callback(),
+ rabbit_types:proc_type_and_name()) ->
+ rabbit_types:ok(pid()).
+
+-spec pause_monitor(heartbeaters()) -> 'ok'.
+-spec resume_monitor(heartbeaters()) -> 'ok'.
+
+-spec system_code_change(_,_,_,_) -> {'ok',_}.
+-spec system_continue(_,_,{_, _}) -> any().
+-spec system_terminate(_,_,_,_) -> no_return().
+
+%%----------------------------------------------------------------------------
+start(SupPid, Sock, SendTimeoutSec, SendFun, ReceiveTimeoutSec, ReceiveFun) ->
+ start(SupPid, Sock, unknown,
+ SendTimeoutSec, SendFun, ReceiveTimeoutSec, ReceiveFun).
+
+start(SupPid, Sock, Identity,
+ SendTimeoutSec, SendFun, ReceiveTimeoutSec, ReceiveFun) ->
+ {ok, Sender} =
+ start_heartbeater(SendTimeoutSec, SupPid, Sock,
+ SendFun, heartbeat_sender,
+ start_heartbeat_sender, Identity),
+ {ok, Receiver} =
+ start_heartbeater(ReceiveTimeoutSec, SupPid, Sock,
+ ReceiveFun, heartbeat_receiver,
+ start_heartbeat_receiver, Identity),
+ {Sender, Receiver}.
+
+start_heartbeat_sender(Sock, TimeoutSec, SendFun, Identity) ->
+ %% the 'div 2' is there so that we don't end up waiting for nearly
+ %% 2 * TimeoutSec before sending a heartbeat in the boundary case
+ %% where the last message was sent just after a heartbeat.
+ heartbeater({Sock, TimeoutSec * 1000 div 2, send_oct, 0,
+ fun () -> SendFun(), continue end}, Identity).
+
+start_heartbeat_receiver(Sock, TimeoutSec, ReceiveFun, Identity) ->
+ %% we check for incoming data every interval, and time out after
+ %% two checks with no change. As a result we will time out between
+ %% 2 and 3 intervals after the last data has been received.
+ heartbeater({Sock, TimeoutSec * 1000, recv_oct, 1,
+ fun () -> ReceiveFun(), stop end}, Identity).
+
+pause_monitor({_Sender, none}) -> ok;
+pause_monitor({_Sender, Receiver}) -> Receiver ! pause, ok.
+
+resume_monitor({_Sender, none}) -> ok;
+resume_monitor({_Sender, Receiver}) -> Receiver ! resume, ok.
+
+system_continue(_Parent, Deb, {Params, State}) ->
+ heartbeater(Params, Deb, State).
+
+system_terminate(Reason, _Parent, _Deb, _State) ->
+ exit(Reason).
+
+system_code_change(Misc, _Module, _OldVsn, _Extra) ->
+ {ok, Misc}.
+
+%%----------------------------------------------------------------------------
+start_heartbeater(0, _SupPid, _Sock, _TimeoutFun, _Name, _Callback,
+ _Identity) ->
+ {ok, none};
+start_heartbeater(TimeoutSec, SupPid, Sock, TimeoutFun, Name, Callback,
+ Identity) ->
+ supervisor2:start_child(
+ SupPid, {Name,
+ {rabbit_heartbeat, Callback,
+ [Sock, TimeoutSec, TimeoutFun, {Name, Identity}]},
+ transient, ?WORKER_WAIT, worker, [rabbit_heartbeat]}).
+
+heartbeater(Params, Identity) ->
+ Deb = sys:debug_options([]),
+ {ok, proc_lib:spawn_link(fun () ->
+ rabbit_misc:store_proc_name(Identity),
+ heartbeater(Params, Deb, {0, 0})
+ end)}.
+
+heartbeater({Sock, TimeoutMillisec, StatName, Threshold, Handler} = Params,
+ Deb, {StatVal0, SameCount} = State) ->
+ Recurse = fun (State1) -> heartbeater(Params, Deb, State1) end,
+ System = fun (From, Req) ->
+ sys:handle_system_msg(
+ Req, From, self(), ?MODULE, Deb, {Params, State})
+ end,
+ receive
+ pause ->
+ receive
+ resume -> Recurse({0, 0});
+ {system, From, Req} -> System(From, Req);
+ Other -> exit({unexpected_message, Other})
+ end;
+ {system, From, Req} ->
+ System(From, Req);
+ Other ->
+ exit({unexpected_message, Other})
+ after TimeoutMillisec ->
+ OkFun = fun(StatVal1) ->
+ if StatVal0 =:= 0 andalso StatName =:= send_oct ->
+ % Note: this clause is necessary to ensure the
+ % first RMQ -> client heartbeat is sent at the
+ % first interval, instead of waiting the first
+ % two intervals
+ {run_handler, {StatVal1, 0}};
+ StatVal1 =/= StatVal0 ->
+ {recurse, {StatVal1, 0}};
+ SameCount < Threshold ->
+ {recurse, {StatVal1, SameCount +1}};
+ true ->
+ {run_handler, {StatVal1, 0}}
+ end
+ end,
+ SSResult = get_sock_stats(Sock, StatName, OkFun),
+ handle_get_sock_stats(SSResult, Sock, StatName, Recurse, Handler)
+ end.
+
+handle_get_sock_stats(stop, _Sock, _StatName, _Recurse, _Handler) ->
+ ok;
+handle_get_sock_stats({recurse, RecurseArg}, _Sock, _StatName, Recurse, _Handler) ->
+ Recurse(RecurseArg);
+handle_get_sock_stats({run_handler, {_, SameCount}}, Sock, StatName, Recurse, Handler) ->
+ case Handler() of
+ stop -> ok;
+ continue ->
+ OkFun = fun(StatVal) ->
+ {recurse, {StatVal, SameCount}}
+ end,
+ SSResult = get_sock_stats(Sock, StatName, OkFun),
+ handle_get_sock_stats(SSResult, Sock, StatName, Recurse, Handler)
+ end.
+
+get_sock_stats(Sock, StatName, OkFun) ->
+ case rabbit_net:getstat(Sock, [StatName]) of
+ {ok, [{StatName, StatVal}]} ->
+ OkFun(StatVal);
+ {error, einval} ->
+ %% the socket is dead, most likely because the
+ %% connection is being shut down -> terminate
+ stop;
+ {error, Reason} ->
+ exit({cannot_get_socket_stats, Reason})
+ end.
diff --git a/deps/rabbit_common/src/rabbit_http_util.erl b/deps/rabbit_common/src/rabbit_http_util.erl
new file mode 100644
index 0000000000..d0ff498110
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_http_util.erl
@@ -0,0 +1,967 @@
+%% @author Bob Ippolito <bob@mochimedia.com>
+%% @copyright 2007 Mochi Media, Inc.
+
+%% @doc Utilities for parsing and quoting.
+
+-module(rabbit_http_util).
+-author('bob@mochimedia.com').
+-export([join/2, quote_plus/1, urlencode/1, parse_qs/1, unquote/1]).
+-export([path_split/1]).
+-export([urlsplit/1, urlsplit_path/1, urlunsplit/1, urlunsplit_path/1]).
+-export([parse_header/1]).
+-export([shell_quote/1, cmd/1, cmd_string/1, cmd_port/2, cmd_status/1, cmd_status/2]).
+-export([record_to_proplist/2, record_to_proplist/3]).
+-export([safe_relative_path/1, partition/2]).
+-export([parse_qvalues/1, pick_accepted_encodings/3]).
+-export([make_io/1]).
+
+-define(PERCENT, 37). % $\%
+-define(FULLSTOP, 46). % $\.
+-define(IS_HEX(C), ((C >= $0 andalso C =< $9) orelse
+ (C >= $a andalso C =< $f) orelse
+ (C >= $A andalso C =< $F))).
+-define(QS_SAFE(C), ((C >= $a andalso C =< $z) orelse
+ (C >= $A andalso C =< $Z) orelse
+ (C >= $0 andalso C =< $9) orelse
+ (C =:= ?FULLSTOP orelse C =:= $- orelse C =:= $~ orelse
+ C =:= $_))).
+
+hexdigit(C) when C < 10 -> $0 + C;
+hexdigit(C) when C < 16 -> $A + (C - 10).
+
+unhexdigit(C) when C >= $0, C =< $9 -> C - $0;
+unhexdigit(C) when C >= $a, C =< $f -> C - $a + 10;
+unhexdigit(C) when C >= $A, C =< $F -> C - $A + 10.
+
+%% @spec partition(String, Sep) -> {String, [], []} | {Prefix, Sep, Postfix}
+%% @doc Inspired by Python 2.5's str.partition:
+%% partition("foo/bar", "/") = {"foo", "/", "bar"},
+%% partition("foo", "/") = {"foo", "", ""}.
+partition(String, Sep) ->
+ case partition(String, Sep, []) of
+ undefined ->
+ {String, "", ""};
+ Result ->
+ Result
+ end.
+
+partition("", _Sep, _Acc) ->
+ undefined;
+partition(S, Sep, Acc) ->
+ case partition2(S, Sep) of
+ undefined ->
+ [C | Rest] = S,
+ partition(Rest, Sep, [C | Acc]);
+ Rest ->
+ {lists:reverse(Acc), Sep, Rest}
+ end.
+
+partition2(Rest, "") ->
+ Rest;
+partition2([C | R1], [C | R2]) ->
+ partition2(R1, R2);
+partition2(_S, _Sep) ->
+ undefined.
+
+
+
+%% @spec safe_relative_path(string()) -> string() | undefined
+%% @doc Return the reduced version of a relative path or undefined if it
+%% is not safe. safe relative paths can be joined with an absolute path
+%% and will result in a subdirectory of the absolute path. Safe paths
+%% never contain a backslash character.
+safe_relative_path("/" ++ _) ->
+ undefined;
+safe_relative_path(P) ->
+ case string:chr(P, $\\) of
+ 0 ->
+ safe_relative_path(P, []);
+ _ ->
+ undefined
+ end.
+
+safe_relative_path("", Acc) ->
+ case Acc of
+ [] ->
+ "";
+ _ ->
+ string:join(lists:reverse(Acc), "/")
+ end;
+safe_relative_path(P, Acc) ->
+ case partition(P, "/") of
+ {"", "/", _} ->
+ %% /foo or foo//bar
+ undefined;
+ {"..", _, _} when Acc =:= [] ->
+ undefined;
+ {"..", _, Rest} ->
+ safe_relative_path(Rest, tl(Acc));
+ {Part, "/", ""} ->
+ safe_relative_path("", ["", Part | Acc]);
+ {Part, _, Rest} ->
+ safe_relative_path(Rest, [Part | Acc])
+ end.
+
+%% @spec shell_quote(string()) -> string()
+%% @doc Quote a string according to UNIX shell quoting rules, returns a string
+%% surrounded by double quotes.
+shell_quote(L) ->
+ shell_quote(L, [$\"]).
+
+%% @spec cmd_port([string()], Options) -> port()
+%% @doc open_port({spawn, mochiweb_util:cmd_string(Argv)}, Options).
+cmd_port(Argv, Options) ->
+ open_port({spawn, cmd_string(Argv)}, Options).
+
+%% @spec cmd([string()]) -> string()
+%% @doc os:cmd(cmd_string(Argv)).
+cmd(Argv) ->
+ os:cmd(cmd_string(Argv)).
+
+%% @spec cmd_string([string()]) -> string()
+%% @doc Create a shell quoted command string from a list of arguments.
+cmd_string(Argv) ->
+ string:join([shell_quote(X) || X <- Argv], " ").
+
+%% @spec cmd_status([string()]) -> {ExitStatus::integer(), Stdout::binary()}
+%% @doc Accumulate the output and exit status from the given application,
+%% will be spawned with cmd_port/2.
+cmd_status(Argv) ->
+ cmd_status(Argv, []).
+
+%% @spec cmd_status([string()], [atom()]) -> {ExitStatus::integer(), Stdout::binary()}
+%% @doc Accumulate the output and exit status from the given application,
+%% will be spawned with cmd_port/2.
+cmd_status(Argv, Options) ->
+ Port = cmd_port(Argv, [exit_status, stderr_to_stdout,
+ use_stdio, binary | Options]),
+ try cmd_loop(Port, [])
+ after catch port_close(Port)
+ end.
+
+%% @spec cmd_loop(port(), list()) -> {ExitStatus::integer(), Stdout::binary()}
+%% @doc Accumulate the output and exit status from a port.
+cmd_loop(Port, Acc) ->
+ receive
+ {Port, {exit_status, Status}} ->
+ {Status, iolist_to_binary(lists:reverse(Acc))};
+ {Port, {data, Data}} ->
+ cmd_loop(Port, [Data | Acc])
+ end.
+
+%% @spec join([iolist()], iolist()) -> iolist()
+%% @doc Join a list of strings or binaries together with the given separator
+%% string or char or binary. The output is flattened, but may be an
+%% iolist() instead of a string() if any of the inputs are binary().
+join([], _Separator) ->
+ [];
+join([S], _Separator) ->
+ lists:flatten(S);
+join(Strings, Separator) ->
+ lists:flatten(revjoin(lists:reverse(Strings), Separator, [])).
+
+revjoin([], _Separator, Acc) ->
+ Acc;
+revjoin([S | Rest], Separator, []) ->
+ revjoin(Rest, Separator, [S]);
+revjoin([S | Rest], Separator, Acc) ->
+ revjoin(Rest, Separator, [S, Separator | Acc]).
+
+%% @spec quote_plus(atom() | integer() | float() | string() | binary()) -> string()
+%% @doc URL safe encoding of the given term.
+quote_plus(Atom) when is_atom(Atom) ->
+ quote_plus(atom_to_list(Atom));
+quote_plus(Int) when is_integer(Int) ->
+ quote_plus(integer_to_list(Int));
+quote_plus(Binary) when is_binary(Binary) ->
+ quote_plus(binary_to_list(Binary));
+quote_plus(Float) when is_float(Float) ->
+ quote_plus(rabbit_numerical:digits(Float));
+quote_plus(String) ->
+ quote_plus(String, []).
+
+quote_plus([], Acc) ->
+ lists:reverse(Acc);
+quote_plus([C | Rest], Acc) when ?QS_SAFE(C) ->
+ quote_plus(Rest, [C | Acc]);
+quote_plus([$\s | Rest], Acc) ->
+ quote_plus(Rest, [$+ | Acc]);
+quote_plus([C | Rest], Acc) ->
+ <<Hi:4, Lo:4>> = <<C>>,
+ quote_plus(Rest, [hexdigit(Lo), hexdigit(Hi), ?PERCENT | Acc]).
+
+%% @spec urlencode([{Key, Value}]) -> string()
+%% @doc URL encode the property list.
+urlencode(Props) ->
+ Pairs = lists:foldr(
+ fun ({K, V}, Acc) ->
+ [quote_plus(K) ++ "=" ++ quote_plus(V) | Acc]
+ end, [], Props),
+ string:join(Pairs, "&").
+
+%% @spec parse_qs(string() | binary()) -> [{Key, Value}]
+%% @doc Parse a query string or application/x-www-form-urlencoded.
+parse_qs(Binary) when is_binary(Binary) ->
+ parse_qs(binary_to_list(Binary));
+parse_qs(String) ->
+ parse_qs(String, []).
+
+parse_qs([], Acc) ->
+ lists:reverse(Acc);
+parse_qs(String, Acc) ->
+ {Key, Rest} = parse_qs_key(String),
+ {Value, Rest1} = parse_qs_value(Rest),
+ parse_qs(Rest1, [{Key, Value} | Acc]).
+
+parse_qs_key(String) ->
+ parse_qs_key(String, []).
+
+parse_qs_key([], Acc) ->
+ {qs_revdecode(Acc), ""};
+parse_qs_key([$= | Rest], Acc) ->
+ {qs_revdecode(Acc), Rest};
+parse_qs_key(Rest=[$; | _], Acc) ->
+ {qs_revdecode(Acc), Rest};
+parse_qs_key(Rest=[$& | _], Acc) ->
+ {qs_revdecode(Acc), Rest};
+parse_qs_key([C | Rest], Acc) ->
+ parse_qs_key(Rest, [C | Acc]).
+
+parse_qs_value(String) ->
+ parse_qs_value(String, []).
+
+parse_qs_value([], Acc) ->
+ {qs_revdecode(Acc), ""};
+parse_qs_value([$; | Rest], Acc) ->
+ {qs_revdecode(Acc), Rest};
+parse_qs_value([$& | Rest], Acc) ->
+ {qs_revdecode(Acc), Rest};
+parse_qs_value([C | Rest], Acc) ->
+ parse_qs_value(Rest, [C | Acc]).
+
+%% @spec unquote(string() | binary()) -> string()
+%% @doc Unquote a URL encoded string.
+unquote(Binary) when is_binary(Binary) ->
+ unquote(binary_to_list(Binary));
+unquote(String) ->
+ qs_revdecode(lists:reverse(String)).
+
+qs_revdecode(S) ->
+ qs_revdecode(S, []).
+
+qs_revdecode([], Acc) ->
+ Acc;
+qs_revdecode([$+ | Rest], Acc) ->
+ qs_revdecode(Rest, [$\s | Acc]);
+qs_revdecode([Lo, Hi, ?PERCENT | Rest], Acc) when ?IS_HEX(Lo), ?IS_HEX(Hi) ->
+ qs_revdecode(Rest, [(unhexdigit(Lo) bor (unhexdigit(Hi) bsl 4)) | Acc]);
+qs_revdecode([C | Rest], Acc) ->
+ qs_revdecode(Rest, [C | Acc]).
+
+%% @spec urlsplit(Url) -> {Scheme, Netloc, Path, Query, Fragment}
+%% @doc Return a 5-tuple, does not expand % escapes. Only supports HTTP style
+%% URLs.
+urlsplit(Url) ->
+ {Scheme, Url1} = urlsplit_scheme(Url),
+ {Netloc, Url2} = urlsplit_netloc(Url1),
+ {Path, Query, Fragment} = urlsplit_path(Url2),
+ {Scheme, Netloc, Path, Query, Fragment}.
+
+urlsplit_scheme(Url) ->
+ case urlsplit_scheme(Url, []) of
+ no_scheme ->
+ {"", Url};
+ Res ->
+ Res
+ end.
+
+urlsplit_scheme([C | Rest], Acc) when ((C >= $a andalso C =< $z) orelse
+ (C >= $A andalso C =< $Z) orelse
+ (C >= $0 andalso C =< $9) orelse
+ C =:= $+ orelse C =:= $- orelse
+ C =:= $.) ->
+ urlsplit_scheme(Rest, [C | Acc]);
+urlsplit_scheme([$: | Rest], Acc=[_ | _]) ->
+ {string:to_lower(lists:reverse(Acc)), Rest};
+urlsplit_scheme(_Rest, _Acc) ->
+ no_scheme.
+
+urlsplit_netloc("//" ++ Rest) ->
+ urlsplit_netloc(Rest, []);
+urlsplit_netloc(Path) ->
+ {"", Path}.
+
+urlsplit_netloc("", Acc) ->
+ {lists:reverse(Acc), ""};
+urlsplit_netloc(Rest=[C | _], Acc) when C =:= $/; C =:= $?; C =:= $# ->
+ {lists:reverse(Acc), Rest};
+urlsplit_netloc([C | Rest], Acc) ->
+ urlsplit_netloc(Rest, [C | Acc]).
+
+
+%% @spec path_split(string()) -> {Part, Rest}
+%% @doc Split a path starting from the left, as in URL traversal.
+%% path_split("foo/bar") = {"foo", "bar"},
+%% path_split("/foo/bar") = {"", "foo/bar"}.
+path_split(S) ->
+ path_split(S, []).
+
+path_split("", Acc) ->
+ {lists:reverse(Acc), ""};
+path_split("/" ++ Rest, Acc) ->
+ {lists:reverse(Acc), Rest};
+path_split([C | Rest], Acc) ->
+ path_split(Rest, [C | Acc]).
+
+
+%% @spec urlunsplit({Scheme, Netloc, Path, Query, Fragment}) -> string()
+%% @doc Assemble a URL from the 5-tuple. Path must be absolute.
+urlunsplit({Scheme, Netloc, Path, Query, Fragment}) ->
+ lists:flatten([case Scheme of "" -> ""; _ -> [Scheme, "://"] end,
+ Netloc,
+ urlunsplit_path({Path, Query, Fragment})]).
+
+%% @spec urlunsplit_path({Path, Query, Fragment}) -> string()
+%% @doc Assemble a URL path from the 3-tuple.
+urlunsplit_path({Path, Query, Fragment}) ->
+ lists:flatten([Path,
+ case Query of "" -> ""; _ -> [$? | Query] end,
+ case Fragment of "" -> ""; _ -> [$# | Fragment] end]).
+
+%% @spec urlsplit_path(Url) -> {Path, Query, Fragment}
+%% @doc Return a 3-tuple, does not expand % escapes. Only supports HTTP style
+%% paths.
+urlsplit_path(Path) ->
+ urlsplit_path(Path, []).
+
+urlsplit_path("", Acc) ->
+ {lists:reverse(Acc), "", ""};
+urlsplit_path("?" ++ Rest, Acc) ->
+ {Query, Fragment} = urlsplit_query(Rest),
+ {lists:reverse(Acc), Query, Fragment};
+urlsplit_path("#" ++ Rest, Acc) ->
+ {lists:reverse(Acc), "", Rest};
+urlsplit_path([C | Rest], Acc) ->
+ urlsplit_path(Rest, [C | Acc]).
+
+urlsplit_query(Query) ->
+ urlsplit_query(Query, []).
+
+urlsplit_query("", Acc) ->
+ {lists:reverse(Acc), ""};
+urlsplit_query("#" ++ Rest, Acc) ->
+ {lists:reverse(Acc), Rest};
+urlsplit_query([C | Rest], Acc) ->
+ urlsplit_query(Rest, [C | Acc]).
+
+%% @spec parse_header(string()) -> {Type, [{K, V}]}
+%% @doc Parse a Content-Type like header, return the main Content-Type
+%% and a property list of options.
+parse_header(String) ->
+ %% TODO: This is exactly as broken as Python's cgi module.
+ %% Should parse properly like mochiweb_cookies.
+ [Type | Parts] = [string:strip(S) || S <- string:tokens(String, ";")],
+ F = fun (S, Acc) ->
+ case lists:splitwith(fun (C) -> C =/= $= end, S) of
+ {"", _} ->
+ %% Skip anything with no name
+ Acc;
+ {_, ""} ->
+ %% Skip anything with no value
+ Acc;
+ {Name, [$\= | Value]} ->
+ [{string:to_lower(string:strip(Name)),
+ unquote_header(string:strip(Value))} | Acc]
+ end
+ end,
+ {string:to_lower(Type),
+ lists:foldr(F, [], Parts)}.
+
+unquote_header("\"" ++ Rest) ->
+ unquote_header(Rest, []);
+unquote_header(S) ->
+ S.
+
+unquote_header("", Acc) ->
+ lists:reverse(Acc);
+unquote_header("\"", Acc) ->
+ lists:reverse(Acc);
+unquote_header([$\\, C | Rest], Acc) ->
+ unquote_header(Rest, [C | Acc]);
+unquote_header([C | Rest], Acc) ->
+ unquote_header(Rest, [C | Acc]).
+
+%% @spec record_to_proplist(Record, Fields) -> proplist()
+%% @doc calls record_to_proplist/3 with a default TypeKey of '__record'
+record_to_proplist(Record, Fields) ->
+ record_to_proplist(Record, Fields, '__record').
+
+%% @spec record_to_proplist(Record, Fields, TypeKey) -> proplist()
+%% @doc Return a proplist of the given Record with each field in the
+%% Fields list set as a key with the corresponding value in the Record.
+%% TypeKey is the key that is used to store the record type
+%% Fields should be obtained by calling record_info(fields, record_type)
+%% where record_type is the record type of Record
+record_to_proplist(Record, Fields, TypeKey)
+ when tuple_size(Record) - 1 =:= length(Fields) ->
+ lists:zip([TypeKey | Fields], tuple_to_list(Record)).
+
+
+shell_quote([], Acc) ->
+ lists:reverse([$\" | Acc]);
+shell_quote([C | Rest], Acc) when C =:= $\" orelse C =:= $\` orelse
+ C =:= $\\ orelse C =:= $\$ ->
+ shell_quote(Rest, [C, $\\ | Acc]);
+shell_quote([C | Rest], Acc) ->
+ shell_quote(Rest, [C | Acc]).
+
+%% @spec parse_qvalues(string()) -> [qvalue()] | invalid_qvalue_string
+%% @type qvalue() = {media_type() | encoding() , float()}.
+%% @type media_type() = string().
+%% @type encoding() = string().
+%%
+%% @doc Parses a list (given as a string) of elements with Q values associated
+%% to them. Elements are separated by commas and each element is separated
+%% from its Q value by a semicolon. Q values are optional but when missing
+%% the value of an element is considered as 1.0. A Q value is always in the
+%% range [0.0, 1.0]. A Q value list is used for example as the value of the
+%% HTTP "Accept" and "Accept-Encoding" headers.
+%%
+%% Q values are described in section 2.9 of the RFC 2616 (HTTP 1.1).
+%%
+%% Example:
+%%
+%% parse_qvalues("gzip; q=0.5, deflate, identity;q=0.0") ->
+%% [{"gzip", 0.5}, {"deflate", 1.0}, {"identity", 0.0}]
+%%
+parse_qvalues(QValuesStr) ->
+ try
+ lists:map(
+ fun(Pair) ->
+ [Type | Params] = string:tokens(Pair, ";"),
+ NormParams = normalize_media_params(Params),
+ {Q, NonQParams} = extract_q(NormParams),
+ {string:join([string:strip(Type) | NonQParams], ";"), Q}
+ end,
+ string:tokens(string:to_lower(QValuesStr), ",")
+ )
+ catch
+ _Type:_Error ->
+ invalid_qvalue_string
+ end.
+
+normalize_media_params(Params) ->
+ {ok, Re} = re:compile("\\s"),
+ normalize_media_params(Re, Params, []).
+
+normalize_media_params(_Re, [], Acc) ->
+ lists:reverse(Acc);
+normalize_media_params(Re, [Param | Rest], Acc) ->
+ NormParam = re:replace(Param, Re, "", [global, {return, list}]),
+ normalize_media_params(Re, Rest, [NormParam | Acc]).
+
+extract_q(NormParams) ->
+ {ok, KVRe} = re:compile("^([^=]+)=([^=]+)$"),
+ {ok, QRe} = re:compile("^((?:0|1)(?:\\.\\d{1,3})?)$"),
+ extract_q(KVRe, QRe, NormParams, []).
+
+extract_q(_KVRe, _QRe, [], Acc) ->
+ {1.0, lists:reverse(Acc)};
+extract_q(KVRe, QRe, [Param | Rest], Acc) ->
+ case re:run(Param, KVRe, [{capture, [1, 2], list}]) of
+ {match, [Name, Value]} ->
+ case Name of
+ "q" ->
+ {match, [Q]} = re:run(Value, QRe, [{capture, [1], list}]),
+ QVal = case Q of
+ "0" ->
+ 0.0;
+ "1" ->
+ 1.0;
+ Else ->
+ list_to_float(Else)
+ end,
+ case QVal < 0.0 orelse QVal > 1.0 of
+ false ->
+ {QVal, lists:reverse(Acc) ++ Rest}
+ end;
+ _ ->
+ extract_q(KVRe, QRe, Rest, [Param | Acc])
+ end
+ end.
+
+%% @spec pick_accepted_encodings([qvalue()], [encoding()], encoding()) ->
+%% [encoding()]
+%%
+%% @doc Determines which encodings specified in the given Q values list are
+%% valid according to a list of supported encodings and a default encoding.
+%%
+%% The returned list of encodings is sorted, descendingly, according to the
+%% Q values of the given list. The last element of this list is the given
+%% default encoding unless this encoding is explicitly or implicitly
+%% marked with a Q value of 0.0 in the given Q values list.
+%% Note: encodings with the same Q value are kept in the same order as
+%% found in the input Q values list.
+%%
+%% This encoding picking process is described in section 14.3 of the
+%% RFC 2616 (HTTP 1.1).
+%%
+%% Example:
+%%
+%% pick_accepted_encodings(
+%% [{"gzip", 0.5}, {"deflate", 1.0}],
+%% ["gzip", "identity"],
+%% "identity"
+%% ) ->
+%% ["gzip", "identity"]
+%%
+pick_accepted_encodings(AcceptedEncs, SupportedEncs, DefaultEnc) ->
+ SortedQList = lists:reverse(
+ lists:sort(fun({_, Q1}, {_, Q2}) -> Q1 < Q2 end, AcceptedEncs)
+ ),
+ {Accepted, Refused} = lists:foldr(
+ fun({E, Q}, {A, R}) ->
+ case Q > 0.0 of
+ true ->
+ {[E | A], R};
+ false ->
+ {A, [E | R]}
+ end
+ end,
+ {[], []},
+ SortedQList
+ ),
+ Refused1 = lists:foldr(
+ fun(Enc, Acc) ->
+ case Enc of
+ "*" ->
+ lists:subtract(SupportedEncs, Accepted) ++ Acc;
+ _ ->
+ [Enc | Acc]
+ end
+ end,
+ [],
+ Refused
+ ),
+ Accepted1 = lists:foldr(
+ fun(Enc, Acc) ->
+ case Enc of
+ "*" ->
+ lists:subtract(SupportedEncs, Accepted ++ Refused1) ++ Acc;
+ _ ->
+ [Enc | Acc]
+ end
+ end,
+ [],
+ Accepted
+ ),
+ Accepted2 = case lists:member(DefaultEnc, Accepted1) of
+ true ->
+ Accepted1;
+ false ->
+ Accepted1 ++ [DefaultEnc]
+ end,
+ [E || E <- Accepted2, lists:member(E, SupportedEncs),
+ not lists:member(E, Refused1)].
+
+make_io(Atom) when is_atom(Atom) ->
+ atom_to_list(Atom);
+make_io(Integer) when is_integer(Integer) ->
+ integer_to_list(Integer);
+make_io(Io) when is_list(Io); is_binary(Io) ->
+ Io.
+
+%%
+%% Tests
+%%
+-ifdef(TEST).
+-include_lib("eunit/include/eunit.hrl").
+
+make_io_test() ->
+ ?assertEqual(
+ <<"atom">>,
+ iolist_to_binary(make_io(atom))),
+ ?assertEqual(
+ <<"20">>,
+ iolist_to_binary(make_io(20))),
+ ?assertEqual(
+ <<"list">>,
+ iolist_to_binary(make_io("list"))),
+ ?assertEqual(
+ <<"binary">>,
+ iolist_to_binary(make_io(<<"binary">>))),
+ ok.
+
+-record(test_record, {field1=f1, field2=f2}).
+record_to_proplist_test() ->
+ ?assertEqual(
+ [{'__record', test_record},
+ {field1, f1},
+ {field2, f2}],
+ record_to_proplist(#test_record{}, record_info(fields, test_record))),
+ ?assertEqual(
+ [{'typekey', test_record},
+ {field1, f1},
+ {field2, f2}],
+ record_to_proplist(#test_record{},
+ record_info(fields, test_record),
+ typekey)),
+ ok.
+
+shell_quote_test() ->
+ ?assertEqual(
+ "\"foo \\$bar\\\"\\`' baz\"",
+ shell_quote("foo $bar\"`' baz")),
+ ok.
+
+cmd_port_test_spool(Port, Acc) ->
+ receive
+ {Port, eof} ->
+ Acc;
+ {Port, {data, {eol, Data}}} ->
+ cmd_port_test_spool(Port, ["\n", Data | Acc]);
+ {Port, Unknown} ->
+ throw({unknown, Unknown})
+ after 1000 ->
+ throw(timeout)
+ end.
+
+cmd_port_test() ->
+ Port = cmd_port(["echo", "$bling$ `word`!"],
+ [eof, stream, {line, 4096}]),
+ Res = try lists:append(lists:reverse(cmd_port_test_spool(Port, [])))
+ after catch port_close(Port)
+ end,
+ self() ! {Port, wtf},
+ try cmd_port_test_spool(Port, [])
+ catch throw:{unknown, wtf} -> ok
+ end,
+ try cmd_port_test_spool(Port, [])
+ catch throw:timeout -> ok
+ end,
+ ?assertEqual(
+ "$bling$ `word`!\n",
+ Res).
+
+cmd_test() ->
+ ?assertEqual(
+ "$bling$ `word`!\n",
+ cmd(["echo", "$bling$ `word`!"])),
+ ok.
+
+cmd_string_test() ->
+ ?assertEqual(
+ "\"echo\" \"\\$bling\\$ \\`word\\`!\"",
+ cmd_string(["echo", "$bling$ `word`!"])),
+ ok.
+
+cmd_status_test() ->
+ ?assertEqual(
+ {0, <<"$bling$ `word`!\n">>},
+ cmd_status(["echo", "$bling$ `word`!"])),
+ ok.
+
+
+parse_header_test() ->
+ ?assertEqual(
+ {"multipart/form-data", [{"boundary", "AaB03x"}]},
+ parse_header("multipart/form-data; boundary=AaB03x")),
+ %% This tests (currently) intentionally broken behavior
+ ?assertEqual(
+ {"multipart/form-data",
+ [{"b", ""},
+ {"cgi", "is"},
+ {"broken", "true\"e"}]},
+ parse_header("multipart/form-data;b=;cgi=\"i\\s;broken=true\"e;=z;z")),
+ ok.
+
+path_split_test() ->
+ {"", "foo/bar"} = path_split("/foo/bar"),
+ {"foo", "bar"} = path_split("foo/bar"),
+ {"bar", ""} = path_split("bar"),
+ ok.
+
+urlsplit_test() ->
+ {"", "", "/foo", "", "bar?baz"} = urlsplit("/foo#bar?baz"),
+ {"https", "host:port", "/foo", "", "bar?baz"} =
+ urlsplit("https://host:port/foo#bar?baz"),
+ {"https", "host", "", "", ""} = urlsplit("https://host"),
+ {"", "", "/wiki/Category:Fruit", "", ""} =
+ urlsplit("/wiki/Category:Fruit"),
+ ok.
+
+urlsplit_path_test() ->
+ {"/foo/bar", "", ""} = urlsplit_path("/foo/bar"),
+ {"/foo", "baz", ""} = urlsplit_path("/foo?baz"),
+ {"/foo", "", "bar?baz"} = urlsplit_path("/foo#bar?baz"),
+ {"/foo", "", "bar?baz#wibble"} = urlsplit_path("/foo#bar?baz#wibble"),
+ {"/foo", "bar", "baz"} = urlsplit_path("/foo?bar#baz"),
+ {"/foo", "bar?baz", "baz"} = urlsplit_path("/foo?bar?baz#baz"),
+ ok.
+
+urlunsplit_test() ->
+ "/foo#bar?baz" = urlunsplit({"", "", "/foo", "", "bar?baz"}),
+ "https://host:port/foo#bar?baz" =
+ urlunsplit({"https", "host:port", "/foo", "", "bar?baz"}),
+ ok.
+
+urlunsplit_path_test() ->
+ "/foo/bar" = urlunsplit_path({"/foo/bar", "", ""}),
+ "/foo?baz" = urlunsplit_path({"/foo", "baz", ""}),
+ "/foo#bar?baz" = urlunsplit_path({"/foo", "", "bar?baz"}),
+ "/foo#bar?baz#wibble" = urlunsplit_path({"/foo", "", "bar?baz#wibble"}),
+ "/foo?bar#baz" = urlunsplit_path({"/foo", "bar", "baz"}),
+ "/foo?bar?baz#baz" = urlunsplit_path({"/foo", "bar?baz", "baz"}),
+ ok.
+
+join_test() ->
+ ?assertEqual("foo,bar,baz",
+ join(["foo", "bar", "baz"], $,)),
+ ?assertEqual("foo,bar,baz",
+ join(["foo", "bar", "baz"], ",")),
+ ?assertEqual("foo bar",
+ join([["foo", " bar"]], ",")),
+ ?assertEqual("foo bar,baz",
+ join([["foo", " bar"], "baz"], ",")),
+ ?assertEqual("foo",
+ join(["foo"], ",")),
+ ?assertEqual("foobarbaz",
+ join(["foo", "bar", "baz"], "")),
+ ?assertEqual("foo" ++ [<<>>] ++ "bar" ++ [<<>>] ++ "baz",
+ join(["foo", "bar", "baz"], <<>>)),
+ ?assertEqual("foobar" ++ [<<"baz">>],
+ join(["foo", "bar", <<"baz">>], "")),
+ ?assertEqual("",
+ join([], "any")),
+ ok.
+
+quote_plus_test() ->
+ "foo" = quote_plus(foo),
+ "1" = quote_plus(1),
+ "1.1" = quote_plus(1.1),
+ "foo" = quote_plus("foo"),
+ "foo+bar" = quote_plus("foo bar"),
+ "foo%0A" = quote_plus("foo\n"),
+ "foo%0A" = quote_plus("foo\n"),
+ "foo%3B%26%3D" = quote_plus("foo;&="),
+ "foo%3B%26%3D" = quote_plus(<<"foo;&=">>),
+ ok.
+
+unquote_test() ->
+ ?assertEqual("foo bar",
+ unquote("foo+bar")),
+ ?assertEqual("foo bar",
+ unquote("foo%20bar")),
+ ?assertEqual("foo\r\n",
+ unquote("foo%0D%0A")),
+ ?assertEqual("foo\r\n",
+ unquote(<<"foo%0D%0A">>)),
+ ok.
+
+urlencode_test() ->
+ "foo=bar&baz=wibble+%0D%0A&z=1" = urlencode([{foo, "bar"},
+ {"baz", "wibble \r\n"},
+ {z, 1}]),
+ ok.
+
+parse_qs_test() ->
+ ?assertEqual(
+ [{"foo", "bar"}, {"baz", "wibble \r\n"}, {"z", "1"}],
+ parse_qs("foo=bar&baz=wibble+%0D%0a&z=1")),
+ ?assertEqual(
+ [{"", "bar"}, {"baz", "wibble \r\n"}, {"z", ""}],
+ parse_qs("=bar&baz=wibble+%0D%0a&z=")),
+ ?assertEqual(
+ [{"foo", "bar"}, {"baz", "wibble \r\n"}, {"z", "1"}],
+ parse_qs(<<"foo=bar&baz=wibble+%0D%0a&z=1">>)),
+ ?assertEqual(
+ [],
+ parse_qs("")),
+ ?assertEqual(
+ [{"foo", ""}, {"bar", ""}, {"baz", ""}],
+ parse_qs("foo;bar&baz")),
+ ok.
+
+partition_test() ->
+ {"foo", "", ""} = partition("foo", "/"),
+ {"foo", "/", "bar"} = partition("foo/bar", "/"),
+ {"foo", "/", ""} = partition("foo/", "/"),
+ {"", "/", "bar"} = partition("/bar", "/"),
+ {"f", "oo/ba", "r"} = partition("foo/bar", "oo/ba"),
+ ok.
+
+safe_relative_path_test() ->
+ "foo" = safe_relative_path("foo"),
+ "foo/" = safe_relative_path("foo/"),
+ "foo" = safe_relative_path("foo/bar/.."),
+ "bar" = safe_relative_path("foo/../bar"),
+ "bar/" = safe_relative_path("foo/../bar/"),
+ "" = safe_relative_path("foo/.."),
+ "" = safe_relative_path("foo/../"),
+ undefined = safe_relative_path("/foo"),
+ undefined = safe_relative_path("../foo"),
+ undefined = safe_relative_path("foo/../.."),
+ undefined = safe_relative_path("foo//"),
+ undefined = safe_relative_path("foo\\bar"),
+ ok.
+
+parse_qvalues_test() ->
+ [] = parse_qvalues(""),
+ [{"identity", 0.0}] = parse_qvalues("identity;q=0"),
+ [{"identity", 0.0}] = parse_qvalues("identity ;q=0"),
+ [{"identity", 0.0}] = parse_qvalues(" identity; q =0 "),
+ [{"identity", 0.0}] = parse_qvalues("identity ; q = 0"),
+ [{"identity", 0.0}] = parse_qvalues("identity ; q= 0.0"),
+ [{"gzip", 1.0}, {"deflate", 1.0}, {"identity", 0.0}] = parse_qvalues(
+ "gzip,deflate,identity;q=0.0"
+ ),
+ [{"deflate", 1.0}, {"gzip", 1.0}, {"identity", 0.0}] = parse_qvalues(
+ "deflate,gzip,identity;q=0.0"
+ ),
+ [{"gzip", 1.0}, {"deflate", 1.0}, {"gzip", 1.0}, {"identity", 0.0}] =
+ parse_qvalues("gzip,deflate,gzip,identity;q=0"),
+ [{"gzip", 1.0}, {"deflate", 1.0}, {"identity", 0.0}] = parse_qvalues(
+ "gzip, deflate , identity; q=0.0"
+ ),
+ [{"gzip", 1.0}, {"deflate", 1.0}, {"identity", 0.0}] = parse_qvalues(
+ "gzip; q=1, deflate;q=1.0, identity;q=0.0"
+ ),
+ [{"gzip", 0.5}, {"deflate", 1.0}, {"identity", 0.0}] = parse_qvalues(
+ "gzip; q=0.5, deflate;q=1.0, identity;q=0"
+ ),
+ [{"gzip", 0.5}, {"deflate", 1.0}, {"identity", 0.0}] = parse_qvalues(
+ "gzip; q=0.5, deflate , identity;q=0.0"
+ ),
+ [{"gzip", 0.5}, {"deflate", 0.8}, {"identity", 0.0}] = parse_qvalues(
+ "gzip; q=0.5, deflate;q=0.8, identity;q=0.0"
+ ),
+ [{"gzip", 0.5}, {"deflate", 1.0}, {"identity", 1.0}] = parse_qvalues(
+ "gzip; q=0.5,deflate,identity"
+ ),
+ [{"gzip", 0.5}, {"deflate", 1.0}, {"identity", 1.0}, {"identity", 1.0}] =
+ parse_qvalues("gzip; q=0.5,deflate,identity, identity "),
+ [{"text/html;level=1", 1.0}, {"text/plain", 0.5}] =
+ parse_qvalues("text/html;level=1, text/plain;q=0.5"),
+ [{"text/html;level=1", 0.3}, {"text/plain", 1.0}] =
+ parse_qvalues("text/html;level=1;q=0.3, text/plain"),
+ [{"text/html;level=1", 0.3}, {"text/plain", 1.0}] =
+ parse_qvalues("text/html; level = 1; q = 0.3, text/plain"),
+ [{"text/html;level=1", 0.3}, {"text/plain", 1.0}] =
+ parse_qvalues("text/html;q=0.3;level=1, text/plain"),
+ invalid_qvalue_string = parse_qvalues("gzip; q=1.1, deflate"),
+ invalid_qvalue_string = parse_qvalues("gzip; q=0.5, deflate;q=2"),
+ invalid_qvalue_string = parse_qvalues("gzip, deflate;q=AB"),
+ invalid_qvalue_string = parse_qvalues("gzip; q=2.1, deflate"),
+ invalid_qvalue_string = parse_qvalues("gzip; q=0.1234, deflate"),
+ invalid_qvalue_string = parse_qvalues("text/html;level=1;q=0.3, text/html;level"),
+ ok.
+
+pick_accepted_encodings_test() ->
+ ["identity"] = pick_accepted_encodings(
+ [],
+ ["gzip", "identity"],
+ "identity"
+ ),
+ ["gzip", "identity"] = pick_accepted_encodings(
+ [{"gzip", 1.0}],
+ ["gzip", "identity"],
+ "identity"
+ ),
+ ["identity"] = pick_accepted_encodings(
+ [{"gzip", 0.0}],
+ ["gzip", "identity"],
+ "identity"
+ ),
+ ["gzip", "identity"] = pick_accepted_encodings(
+ [{"gzip", 1.0}, {"deflate", 1.0}],
+ ["gzip", "identity"],
+ "identity"
+ ),
+ ["gzip", "identity"] = pick_accepted_encodings(
+ [{"gzip", 0.5}, {"deflate", 1.0}],
+ ["gzip", "identity"],
+ "identity"
+ ),
+ ["identity"] = pick_accepted_encodings(
+ [{"gzip", 0.0}, {"deflate", 0.0}],
+ ["gzip", "identity"],
+ "identity"
+ ),
+ ["gzip"] = pick_accepted_encodings(
+ [{"gzip", 1.0}, {"deflate", 1.0}, {"identity", 0.0}],
+ ["gzip", "identity"],
+ "identity"
+ ),
+ ["gzip", "deflate", "identity"] = pick_accepted_encodings(
+ [{"gzip", 1.0}, {"deflate", 1.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip", "deflate"] = pick_accepted_encodings(
+ [{"gzip", 1.0}, {"deflate", 1.0}, {"identity", 0.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["deflate", "gzip", "identity"] = pick_accepted_encodings(
+ [{"gzip", 0.2}, {"deflate", 1.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["deflate", "deflate", "gzip", "identity"] = pick_accepted_encodings(
+ [{"gzip", 0.2}, {"deflate", 1.0}, {"deflate", 1.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["deflate", "gzip", "gzip", "identity"] = pick_accepted_encodings(
+ [{"gzip", 0.2}, {"deflate", 1.0}, {"gzip", 1.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip", "deflate", "gzip", "identity"] = pick_accepted_encodings(
+ [{"gzip", 0.2}, {"deflate", 0.9}, {"gzip", 1.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ [] = pick_accepted_encodings(
+ [{"*", 0.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip", "deflate", "identity"] = pick_accepted_encodings(
+ [{"*", 1.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip", "deflate", "identity"] = pick_accepted_encodings(
+ [{"*", 0.6}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip"] = pick_accepted_encodings(
+ [{"gzip", 1.0}, {"*", 0.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip", "deflate"] = pick_accepted_encodings(
+ [{"gzip", 1.0}, {"deflate", 0.6}, {"*", 0.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["deflate", "gzip"] = pick_accepted_encodings(
+ [{"gzip", 0.5}, {"deflate", 1.0}, {"*", 0.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip", "identity"] = pick_accepted_encodings(
+ [{"deflate", 0.0}, {"*", 1.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ["gzip", "identity"] = pick_accepted_encodings(
+ [{"*", 1.0}, {"deflate", 0.0}],
+ ["gzip", "deflate", "identity"],
+ "identity"
+ ),
+ ok.
+
+-endif.
diff --git a/deps/rabbit_common/src/rabbit_json.erl b/deps/rabbit_common/src/rabbit_json.erl
new file mode 100644
index 0000000000..a10569135b
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_json.erl
@@ -0,0 +1,63 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_json).
+
+-export([decode/1, decode/2, try_decode/1, try_decode/2,
+ encode/1, encode/2, try_encode/1, try_encode/2]).
+
+-define(DEFAULT_DECODE_OPTIONS, [return_maps]).
+
+
+-spec decode(jsx:json_text()) -> jsx:json_term().
+decode(JSON) ->
+ decode(JSON, ?DEFAULT_DECODE_OPTIONS).
+
+
+-spec decode(jsx:json_text(), jsx_to_term:config()) -> jsx:json_term().
+decode(JSON, Opts) ->
+ jsx:decode(JSON, Opts).
+
+
+-spec try_decode(jsx:json_text()) -> {ok, jsx:json_term()} |
+ {error, Reason :: term()}.
+try_decode(JSON) ->
+ try_decode(JSON, ?DEFAULT_DECODE_OPTIONS).
+
+
+-spec try_decode(jsx:json_text(), jsx_to_term:config()) ->
+ {ok, jsx:json_term()} | {error, Reason :: term()}.
+try_decode(JSON, Opts) ->
+ try
+ {ok, decode(JSON, Opts)}
+ catch error: Reason ->
+ {error, Reason}
+ end.
+
+-spec encode(jsx:json_term()) -> jsx:json_text().
+encode(Term) ->
+ encode(Term, []).
+
+-spec encode(jsx:json_term(), jsx_to_json:config()) -> jsx:json_text().
+encode(Term, Opts) ->
+ jsx:encode(Term, Opts).
+
+
+-spec try_encode(jsx:json_term()) -> {ok, jsx:json_text()} |
+ {error, Reason :: term()}.
+try_encode(Term) ->
+ try_encode(Term, []).
+
+
+-spec try_encode(jsx:json_term(), jsx_to_term:config()) ->
+ {ok, jsx:json_text()} | {error, Reason :: term()}.
+try_encode(Term, Opts) ->
+ try
+ {ok, encode(Term, Opts)}
+ catch error: Reason ->
+ {error, Reason}
+ end.
diff --git a/deps/rabbit_common/src/rabbit_log.erl b/deps/rabbit_common/src/rabbit_log.erl
new file mode 100644
index 0000000000..22b4619d1c
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_log.erl
@@ -0,0 +1,164 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_log).
+
+-export([log/2, log/3, log/4]).
+-export([debug/1, debug/2, debug/3,
+ info/1, info/2, info/3,
+ notice/1, notice/2, notice/3,
+ warning/1, warning/2, warning/3,
+ error/1, error/2, error/3,
+ critical/1, critical/2, critical/3,
+ alert/1, alert/2, alert/3,
+ emergency/1, emergency/2, emergency/3,
+ none/1, none/2, none/3]).
+-export([make_internal_sink_name/1]).
+
+-include("rabbit_log.hrl").
+%%----------------------------------------------------------------------------
+
+-type category() :: channel |
+ connection |
+ federation |
+ feature_flags |
+ ldap |
+ mirroring |
+ osiris |
+ prelaunch |
+ queue |
+ ra |
+ shovel |
+ upgrade.
+
+-spec debug(string()) -> 'ok'.
+-spec debug(string(), [any()]) -> 'ok'.
+-spec debug(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec info(string()) -> 'ok'.
+-spec info(string(), [any()]) -> 'ok'.
+-spec info(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec notice(string()) -> 'ok'.
+-spec notice(string(), [any()]) -> 'ok'.
+-spec notice(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec warning(string()) -> 'ok'.
+-spec warning(string(), [any()]) -> 'ok'.
+-spec warning(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec error(string()) -> 'ok'.
+-spec error(string(), [any()]) -> 'ok'.
+-spec error(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec critical(string()) -> 'ok'.
+-spec critical(string(), [any()]) -> 'ok'.
+-spec critical(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec alert(string()) -> 'ok'.
+-spec alert(string(), [any()]) -> 'ok'.
+-spec alert(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec emergency(string()) -> 'ok'.
+-spec emergency(string(), [any()]) -> 'ok'.
+-spec emergency(pid() | [tuple()], string(), [any()]) -> 'ok'.
+-spec none(string()) -> 'ok'.
+-spec none(string(), [any()]) -> 'ok'.
+-spec none(pid() | [tuple()], string(), [any()]) -> 'ok'.
+
+%%----------------------------------------------------------------------------
+
+-spec log(category(), lager:log_level(), string()) -> 'ok'.
+log(Category, Level, Fmt) -> log(Category, Level, Fmt, []).
+
+-spec log(category(), lager:log_level(), string(), [any()]) -> 'ok'.
+log(Category, Level, Fmt, Args) when is_list(Args) ->
+ Sink = case Category of
+ default -> ?LAGER_SINK;
+ _ -> make_internal_sink_name(Category)
+ end,
+ lager:log(Sink, Level, self(), Fmt, Args).
+
+%% logger(3) handler.
+log(#{level := Level,
+ msg := Msg,
+ meta := #{pid := Pid}} = _LogEvent,
+ _Config) ->
+ case Msg of
+ {report, #{label := {error_logger, _}}} ->
+ %% Avoid recursive loop.
+ ok;
+ {report, #{label := {application_controller, progress}}} ->
+ %% Already logged by Lager.
+ ok;
+ {report, #{label := {supervisor, progress}}} ->
+ %% Already logged by Lager.
+ ok;
+ {report, #{report := Report}} ->
+ %% FIXME: Is this code reached?
+ error_logger:info_report(Report);
+ {report, #{format := Format, args := Args}} when is_list(Format) ->
+ lager:log(?LAGER_SINK, Level, Pid, Format, Args);
+ {string, String} ->
+ lager:log(?LAGER_SINK, Level, Pid, "~ts", [String]);
+ {Format, Args} when is_list(Format) ->
+ lager:log(?LAGER_SINK, Level, Pid, Format, Args)
+ end.
+
+make_internal_sink_name(channel) -> rabbit_log_channel_lager_event;
+make_internal_sink_name(connection) -> rabbit_log_connection_lager_event;
+make_internal_sink_name(default) -> rabbit_log_lager_event;
+make_internal_sink_name(feature_flags) -> rabbit_log_feature_flags_lager_event;
+make_internal_sink_name(federation) -> rabbit_log_federation_lager_event;
+make_internal_sink_name(ldap) -> rabbit_log_ldap_lager_event;
+make_internal_sink_name(mirroring) -> rabbit_log_mirroring_lager_event;
+make_internal_sink_name(osiris) -> rabbit_log_osiris_lager_event;
+make_internal_sink_name(prelaunch) -> rabbit_log_prelaunch_lager_event;
+make_internal_sink_name(queue) -> rabbit_log_queue_lager_event;
+make_internal_sink_name(ra) -> rabbit_log_ra_lager_event;
+make_internal_sink_name(shovel) -> rabbit_log_shovel_lager_event;
+make_internal_sink_name(upgrade) -> rabbit_log_upgrade_lager_event;
+make_internal_sink_name(Category) ->
+ erlang:error({unknown_category, Category}).
+
+debug(Format) -> debug(Format, []).
+debug(Format, Args) -> debug(self(), Format, Args).
+debug(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, debug, Metadata, Format, Args).
+
+info(Format) -> info(Format, []).
+info(Format, Args) -> info(self(), Format, Args).
+info(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, info, Metadata, Format, Args).
+
+notice(Format) -> notice(Format, []).
+notice(Format, Args) -> notice(self(), Format, Args).
+notice(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, notice, Metadata, Format, Args).
+
+warning(Format) -> warning(Format, []).
+warning(Format, Args) -> warning(self(), Format, Args).
+warning(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, warning, Metadata, Format, Args).
+
+error(Format) -> ?MODULE:error(Format, []).
+error(Format, Args) -> ?MODULE:error(self(), Format, Args).
+error(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, error, Metadata, Format, Args).
+
+critical(Format) -> critical(Format, []).
+critical(Format, Args) -> critical(self(), Format, Args).
+critical(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, critical, Metadata, Format, Args).
+
+alert(Format) -> alert(Format, []).
+alert(Format, Args) -> alert(self(), Format, Args).
+alert(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, alert, Metadata, Format, Args).
+
+emergency(Format) -> emergency(Format, []).
+emergency(Format, Args) -> emergency(self(), Format, Args).
+emergency(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, emergency, Metadata, Format, Args).
+
+none(Format) -> none(Format, []).
+none(Format, Args) -> none(self(), Format, Args).
+none(Metadata, Format, Args) ->
+ lager:log(?LAGER_SINK, none, Metadata, Format, Args).
diff --git a/deps/rabbit_common/src/rabbit_log_osiris_shim.erl b/deps/rabbit_common/src/rabbit_log_osiris_shim.erl
new file mode 100644
index 0000000000..09d6a63431
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_log_osiris_shim.erl
@@ -0,0 +1,26 @@
+%% The contents of this file are subject to the Mozilla Public License
+%% Version 1.1 (the "License"); you may not use this file except in
+%% compliance with the License. You may obtain a copy of the License
+%% at https://www.mozilla.org/MPL/
+%%
+%% Software distributed under the License is distributed on an "AS IS"
+%% basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
+%% the License for the specific language governing rights and
+%% limitations under the License.
+%%
+%% The Original Code is RabbitMQ.
+%%
+%% The Initial Developer of the Original Code is GoPivotal, Inc.
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_log_osiris_shim).
+
+%% just a shim to redirect logs from ra to rabbit_log
+
+-export([log/4]).
+
+-spec log(lager:log_level(), string(), [any()], _) -> ok.
+log(Level, Format, Args, _Meta) ->
+ rabbit_log:log(osiris, Level, Format, Args),
+ ok.
diff --git a/deps/rabbit_common/src/rabbit_log_ra_shim.erl b/deps/rabbit_common/src/rabbit_log_ra_shim.erl
new file mode 100644
index 0000000000..3d35ff6a07
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_log_ra_shim.erl
@@ -0,0 +1,16 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_log_ra_shim).
+
+%% just a shim to redirect logs from ra to rabbit_log
+
+-export([log/4]).
+
+log(Level, Format, Args, _Meta) ->
+ rabbit_log:log(ra, Level, Format, Args),
+ ok.
diff --git a/deps/rabbit_common/src/rabbit_misc.erl b/deps/rabbit_common/src/rabbit_misc.erl
new file mode 100644
index 0000000000..c5fd86dcbb
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_misc.erl
@@ -0,0 +1,1434 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_misc).
+
+-ignore_xref([{maps, get, 2}]).
+
+-include("rabbit.hrl").
+-include("rabbit_framing.hrl").
+-include("rabbit_misc.hrl").
+
+-ifdef(TEST).
+-export([decompose_pid/1, compose_pid/4]).
+-endif.
+
+-export([method_record_type/1, polite_pause/0, polite_pause/1]).
+-export([die/1, frame_error/2, amqp_error/4, quit/1,
+ protocol_error/3, protocol_error/4, protocol_error/1]).
+-export([type_class/1, assert_args_equivalence/4, assert_field_equivalence/4]).
+-export([dirty_read/1]).
+-export([table_lookup/2, set_table_value/4, amqp_table/1, to_amqp_table/1]).
+-export([r/3, r/2, r_arg/4, rs/1]).
+-export([enable_cover/0, report_cover/0]).
+-export([enable_cover/1, report_cover/1]).
+-export([start_cover/1]).
+-export([throw_on_error/2, with_exit_handler/2, is_abnormal_exit/1,
+ filter_exit_map/2]).
+-export([with_user/2]).
+-export([execute_mnesia_transaction/1]).
+-export([execute_mnesia_transaction/2]).
+-export([execute_mnesia_tx_with_tail/1]).
+-export([ensure_ok/2]).
+-export([tcp_name/3, format_inet_error/1]).
+-export([upmap/2, map_in_order/2, utf8_safe/1]).
+-export([table_filter/3]).
+-export([dirty_read_all/1, dirty_foreach_key/2, dirty_dump_log/1]).
+-export([format/2, format_many/1, format_stderr/2]).
+-export([unfold/2, ceil/1, queue_fold/3]).
+-export([sort_field_table/1]).
+-export([atom_to_binary/1, parse_bool/1, parse_int/1]).
+-export([pid_to_string/1, string_to_pid/1,
+ pid_change_node/2, node_to_fake_pid/1]).
+-export([version_compare/2, version_compare/3]).
+-export([version_minor_equivalent/2, strict_version_minor_equivalent/2]).
+-export([dict_cons/3, orddict_cons/3, maps_cons/3, gb_trees_cons/3]).
+-export([gb_trees_fold/3, gb_trees_foreach/2]).
+-export([all_module_attributes/1,
+ rabbitmq_related_module_attributes/1,
+ module_attributes_from_apps/2,
+ build_acyclic_graph/3]).
+-export([const/1]).
+-export([ntoa/1, ntoab/1]).
+-export([is_process_alive/1]).
+-export([pget/2, pget/3, pupdate/3, pget_or_die/2, pmerge/3, pset/3, plmerge/2]).
+-export([format_message_queue/2]).
+-export([append_rpc_all_nodes/4, append_rpc_all_nodes/5]).
+-export([os_cmd/1]).
+-export([is_os_process_alive/1]).
+-export([gb_sets_difference/2]).
+-export([version/0, otp_release/0, platform_and_version/0, otp_system_version/0,
+ rabbitmq_and_erlang_versions/0, which_applications/0]).
+-export([sequence_error/1]).
+-export([check_expiry/1]).
+-export([base64url/1]).
+-export([interval_operation/5]).
+-export([ensure_timer/4, stop_timer/2, send_after/3, cancel_timer/1]).
+-export([get_parent/0]).
+-export([store_proc_name/1, store_proc_name/2, get_proc_name/0]).
+-export([moving_average/4]).
+-export([escape_html_tags/1, b64decode_or_throw/1]).
+-export([get_env/3]).
+-export([get_channel_operation_timeout/0]).
+-export([random/1]).
+-export([rpc_call/4, rpc_call/5]).
+-export([get_gc_info/1]).
+-export([group_proplists_by/2]).
+
+%% Horrible macro to use in guards
+-define(IS_BENIGN_EXIT(R),
+ R =:= noproc; R =:= noconnection; R =:= nodedown; R =:= normal;
+ R =:= shutdown).
+
+%%----------------------------------------------------------------------------
+
+-export_type([resource_name/0, thunk/1, channel_or_connection_exit/0]).
+
+-type ok_or_error() :: rabbit_types:ok_or_error(any()).
+-type thunk(T) :: fun(() -> T).
+-type resource_name() :: binary().
+-type channel_or_connection_exit()
+ :: rabbit_types:channel_exit() | rabbit_types:connection_exit().
+-type digraph_label() :: term().
+-type graph_vertex_fun() ::
+ fun (({atom(), [term()]}) -> [{digraph:vertex(), digraph_label()}]).
+-type graph_edge_fun() ::
+ fun (({atom(), [term()]}) -> [{digraph:vertex(), digraph:vertex()}]).
+-type tref() :: {'erlang', reference()} | {timer, timer:tref()}.
+
+-spec method_record_type(rabbit_framing:amqp_method_record()) ->
+ rabbit_framing:amqp_method_name().
+-spec polite_pause() -> 'done'.
+-spec polite_pause(non_neg_integer()) -> 'done'.
+-spec die(rabbit_framing:amqp_exception()) -> channel_or_connection_exit().
+
+-spec quit(integer()) -> no_return().
+
+-spec frame_error(rabbit_framing:amqp_method_name(), binary()) ->
+ rabbit_types:connection_exit().
+-spec amqp_error
+ (rabbit_framing:amqp_exception(), string(), [any()],
+ rabbit_framing:amqp_method_name()) ->
+ rabbit_types:amqp_error().
+-spec protocol_error(rabbit_framing:amqp_exception(), string(), [any()]) ->
+ channel_or_connection_exit().
+-spec protocol_error
+ (rabbit_framing:amqp_exception(), string(), [any()],
+ rabbit_framing:amqp_method_name()) ->
+ channel_or_connection_exit().
+-spec protocol_error(rabbit_types:amqp_error()) ->
+ channel_or_connection_exit().
+-spec type_class(rabbit_framing:amqp_field_type()) -> atom().
+-spec assert_args_equivalence
+ (rabbit_framing:amqp_table(), rabbit_framing:amqp_table(),
+ rabbit_types:r(any()), [binary()]) ->
+ 'ok' | rabbit_types:connection_exit().
+-spec assert_field_equivalence
+ (any(), any(), rabbit_types:r(any()), atom() | binary()) ->
+ 'ok' | rabbit_types:connection_exit().
+-spec equivalence_fail
+ (any(), any(), rabbit_types:r(any()), atom() | binary()) ->
+ rabbit_types:connection_exit().
+-spec dirty_read({atom(), any()}) ->
+ rabbit_types:ok_or_error2(any(), 'not_found').
+-spec table_lookup(rabbit_framing:amqp_table(), binary()) ->
+ 'undefined' | {rabbit_framing:amqp_field_type(), any()}.
+-spec set_table_value
+ (rabbit_framing:amqp_table(), binary(), rabbit_framing:amqp_field_type(),
+ rabbit_framing:amqp_value()) ->
+ rabbit_framing:amqp_table().
+-spec r(rabbit_types:vhost(), K) ->
+ rabbit_types:r3(rabbit_types:vhost(), K, '_')
+ when is_subtype(K, atom()).
+-spec r(rabbit_types:vhost() | rabbit_types:r(atom()), K, resource_name()) ->
+ rabbit_types:r3(rabbit_types:vhost(), K, resource_name())
+ when is_subtype(K, atom()).
+-spec r_arg
+ (rabbit_types:vhost() | rabbit_types:r(atom()), K,
+ rabbit_framing:amqp_table(), binary()) ->
+ undefined |
+ rabbit_types:error(
+ {invalid_type, rabbit_framing:amqp_field_type()}) |
+ rabbit_types:r(K) when is_subtype(K, atom()).
+-spec rs(rabbit_types:r(atom())) -> string().
+-spec enable_cover() -> ok_or_error().
+-spec start_cover([{string(), string()} | string()]) -> 'ok'.
+-spec report_cover() -> 'ok'.
+-spec enable_cover([file:filename() | atom()]) -> ok_or_error().
+-spec report_cover([file:filename() | atom()]) -> 'ok'.
+-spec throw_on_error
+ (atom(), thunk(rabbit_types:error(any()) | {ok, A} | A)) -> A.
+-spec with_exit_handler(thunk(A), thunk(A)) -> A.
+-spec is_abnormal_exit(any()) -> boolean().
+-spec filter_exit_map(fun ((A) -> B), [A]) -> [B].
+-spec with_user(rabbit_types:username(), thunk(A)) -> A.
+-spec execute_mnesia_transaction(thunk(A)) -> A.
+-spec execute_mnesia_transaction(thunk(A), fun ((A, boolean()) -> B)) -> B.
+-spec execute_mnesia_tx_with_tail
+ (thunk(fun ((boolean()) -> B))) -> B | (fun ((boolean()) -> B)).
+-spec ensure_ok(ok_or_error(), atom()) -> 'ok'.
+-spec tcp_name(atom(), inet:ip_address(), rabbit_net:ip_port()) ->
+ atom().
+-spec format_inet_error(atom()) -> string().
+-spec upmap(fun ((A) -> B), [A]) -> [B].
+-spec map_in_order(fun ((A) -> B), [A]) -> [B].
+-spec table_filter
+ (fun ((A) -> boolean()), fun ((A, boolean()) -> 'ok'), atom()) -> [A].
+-spec dirty_read_all(atom()) -> [any()].
+-spec dirty_foreach_key(fun ((any()) -> any()), atom()) ->
+ 'ok' | 'aborted'.
+-spec dirty_dump_log(file:filename()) -> ok_or_error().
+-spec format(string(), [any()]) -> string().
+-spec format_many([{string(), [any()]}]) -> string().
+-spec format_stderr(string(), [any()]) -> 'ok'.
+-spec unfold (fun ((A) -> ({'true', B, A} | 'false')), A) -> {[B], A}.
+-spec ceil(number()) -> integer().
+-spec queue_fold(fun ((any(), B) -> B), B, queue:queue()) -> B.
+-spec sort_field_table(rabbit_framing:amqp_table()) ->
+ rabbit_framing:amqp_table().
+-spec pid_to_string(pid()) -> string().
+-spec string_to_pid(string()) -> pid().
+-spec pid_change_node(pid(), node()) -> pid().
+-spec node_to_fake_pid(atom()) -> pid().
+-spec version_compare(string(), string()) -> 'lt' | 'eq' | 'gt'.
+-spec version_compare
+ (rabbit_semver:version_string(), rabbit_semver:version_string(),
+ ('lt' | 'lte' | 'eq' | 'gte' | 'gt')) -> boolean().
+-spec version_minor_equivalent(rabbit_semver:version_string(), rabbit_semver:version_string()) -> boolean().
+-spec dict_cons(any(), any(), dict:dict()) -> dict:dict().
+-spec orddict_cons(any(), any(), orddict:orddict()) -> orddict:orddict().
+-spec gb_trees_cons(any(), any(), gb_trees:tree()) -> gb_trees:tree().
+-spec gb_trees_fold(fun ((any(), any(), A) -> A), A, gb_trees:tree()) -> A.
+-spec gb_trees_foreach(fun ((any(), any()) -> any()), gb_trees:tree()) ->
+ 'ok'.
+-spec all_module_attributes(atom()) -> [{atom(), atom(), [term()]}].
+-spec build_acyclic_graph
+ (graph_vertex_fun(), graph_edge_fun(), [{atom(), [term()]}]) ->
+ rabbit_types:ok_or_error2(
+ digraph:graph(),
+ {'vertex', 'duplicate', digraph:vertex()} |
+ {'edge',
+ ({bad_vertex, digraph:vertex()} |
+ {bad_edge, [digraph:vertex()]}),
+ digraph:vertex(), digraph:vertex()}).
+-spec const(A) -> thunk(A).
+-spec ntoa(inet:ip_address()) -> string().
+-spec ntoab(inet:ip_address()) -> string().
+-spec is_process_alive(pid()) -> boolean().
+
+-spec pmerge(term(), term(), [term()]) -> [term()].
+-spec plmerge([term()], [term()]) -> [term()].
+-spec pset(term(), term(), [term()]) -> [term()].
+-spec format_message_queue(any(), priority_queue:q()) -> term().
+-spec os_cmd(string()) -> string().
+-spec is_os_process_alive(non_neg_integer()) -> boolean().
+-spec gb_sets_difference(gb_sets:set(), gb_sets:set()) -> gb_sets:set().
+-spec version() -> string().
+-spec otp_release() -> string().
+-spec otp_system_version() -> string().
+-spec platform_and_version() -> string().
+-spec rabbitmq_and_erlang_versions() -> {string(), string()}.
+-spec which_applications() -> [{atom(), string(), string()}].
+-spec sequence_error([({'error', any()} | any())]) ->
+ {'error', any()} | any().
+-spec check_expiry(integer()) -> rabbit_types:ok_or_error(any()).
+-spec base64url(binary()) -> string().
+-spec interval_operation
+ ({atom(), atom(), any()}, float(), non_neg_integer(), non_neg_integer(),
+ non_neg_integer()) ->
+ {any(), non_neg_integer()}.
+-spec ensure_timer(A, non_neg_integer(), non_neg_integer(), any()) -> A.
+-spec stop_timer(A, non_neg_integer()) -> A.
+-spec send_after(non_neg_integer(), pid(), any()) -> tref().
+-spec cancel_timer(tref()) -> 'ok'.
+-spec get_parent() -> pid().
+-spec store_proc_name(atom(), rabbit_types:proc_name()) -> ok.
+-spec store_proc_name(rabbit_types:proc_type_and_name()) -> ok.
+-spec get_proc_name() -> rabbit_types:proc_name().
+-spec moving_average(float(), float(), float(), float() | 'undefined') ->
+ float().
+-spec get_env(atom(), atom(), term()) -> term().
+-spec get_channel_operation_timeout() -> non_neg_integer().
+-spec random(non_neg_integer()) -> non_neg_integer().
+-spec get_gc_info(pid()) -> [any()].
+-spec group_proplists_by(fun((proplists:proplist()) -> any()),
+ list(proplists:proplist())) -> list(list(proplists:proplist())).
+
+
+%%----------------------------------------------------------------------------
+
+method_record_type(Record) ->
+ element(1, Record).
+
+polite_pause() ->
+ polite_pause(3000).
+
+polite_pause(N) ->
+ receive
+ after N -> done
+ end.
+
+die(Error) ->
+ protocol_error(Error, "~w", [Error]).
+
+frame_error(MethodName, BinaryFields) ->
+ protocol_error(frame_error, "cannot decode ~w", [BinaryFields], MethodName).
+
+amqp_error(Name, ExplanationFormat, Params, Method) ->
+ Explanation = format(ExplanationFormat, Params),
+ #amqp_error{name = Name, explanation = Explanation, method = Method}.
+
+protocol_error(Name, ExplanationFormat, Params) ->
+ protocol_error(Name, ExplanationFormat, Params, none).
+
+protocol_error(Name, ExplanationFormat, Params, Method) ->
+ protocol_error(amqp_error(Name, ExplanationFormat, Params, Method)).
+
+protocol_error(#amqp_error{} = Error) ->
+ exit(Error).
+
+type_class(byte) -> int;
+type_class(short) -> int;
+type_class(signedint) -> int;
+type_class(long) -> int;
+type_class(decimal) -> int;
+type_class(unsignedbyte) -> int;
+type_class(unsignedshort) -> int;
+type_class(unsignedint) -> int;
+type_class(float) -> float;
+type_class(double) -> float;
+type_class(Other) -> Other.
+
+assert_args_equivalence(Orig, New, Name, Keys) ->
+ [assert_args_equivalence1(Orig, New, Name, Key) || Key <- Keys],
+ ok.
+
+assert_args_equivalence1(Orig, New, Name, Key) ->
+ {Orig1, New1} = {table_lookup(Orig, Key), table_lookup(New, Key)},
+ case {Orig1, New1} of
+ {Same, Same} ->
+ ok;
+ {{OrigType, OrigVal}, {NewType, NewVal}} ->
+ case type_class(OrigType) == type_class(NewType) andalso
+ OrigVal == NewVal of
+ true -> ok;
+ false -> assert_field_equivalence(OrigVal, NewVal, Name, Key)
+ end;
+ {OrigTypeVal, NewTypeVal} ->
+ assert_field_equivalence(OrigTypeVal, NewTypeVal, Name, Key)
+ end.
+
+%% Classic queues do not necessarily have an x-queue-type field associated with them
+%% so we special-case that scenario here
+%%
+%% Fixes rabbitmq/rabbitmq-common#341
+%%
+assert_field_equivalence(_Orig, _Orig, _Name, _Key) ->
+ ok;
+assert_field_equivalence(undefined, {longstr, <<"classic">>}, _Name, <<"x-queue-type">>) ->
+ ok;
+assert_field_equivalence({longstr, <<"classic">>}, undefined, _Name, <<"x-queue-type">>) ->
+ ok;
+assert_field_equivalence(Orig, New, Name, Key) ->
+ equivalence_fail(Orig, New, Name, Key).
+
+equivalence_fail(Orig, New, Name, Key) ->
+ protocol_error(precondition_failed, "inequivalent arg '~s' "
+ "for ~s: received ~s but current is ~s",
+ [Key, rs(Name), val(New), val(Orig)]).
+
+val(undefined) ->
+ "none";
+val({Type, Value}) ->
+ ValFmt = case is_binary(Value) of
+ true -> "~s";
+ false -> "~p"
+ end,
+ format("the value '" ++ ValFmt ++ "' of type '~s'", [Value, Type]);
+val(Value) ->
+ format(case is_binary(Value) of
+ true -> "'~s'";
+ false -> "'~p'"
+ end, [Value]).
+
+%% Normally we'd call mnesia:dirty_read/1 here, but that is quite
+%% expensive due to general mnesia overheads (figuring out table types
+%% and locations, etc). We get away with bypassing these because we
+%% know that the tables we are looking at here
+%% - are not the schema table
+%% - have a local ram copy
+%% - do not have any indices
+dirty_read({Table, Key}) ->
+ case ets:lookup(Table, Key) of
+ [Result] -> {ok, Result};
+ [] -> {error, not_found}
+ end.
+
+%%
+%% Attribute Tables
+%%
+
+table_lookup(Table, Key) ->
+ case lists:keysearch(Key, 1, Table) of
+ {value, {_, TypeBin, ValueBin}} -> {TypeBin, ValueBin};
+ false -> undefined
+ end.
+
+set_table_value(Table, Key, Type, Value) ->
+ sort_field_table(
+ lists:keystore(Key, 1, Table, {Key, Type, Value})).
+
+to_amqp_table(M) when is_map(M) ->
+ lists:reverse(maps:fold(fun(K, V, Acc) -> [to_amqp_table_row(K, V)|Acc] end,
+ [], M));
+to_amqp_table(L) when is_list(L) ->
+ L.
+
+to_amqp_table_row(K, V) ->
+ {T, V2} = type_val(V),
+ {K, T, V2}.
+
+to_amqp_array(L) ->
+ [type_val(I) || I <- L].
+
+type_val(M) when is_map(M) -> {table, to_amqp_table(M)};
+type_val(L) when is_list(L) -> {array, to_amqp_array(L)};
+type_val(X) when is_binary(X) -> {longstr, X};
+type_val(X) when is_integer(X) -> {long, X};
+type_val(X) when is_number(X) -> {double, X};
+type_val(true) -> {bool, true};
+type_val(false) -> {bool, false};
+type_val(null) -> throw({error, null_not_allowed});
+type_val(X) -> throw({error, {unhandled_type, X}}).
+
+amqp_table(unknown) -> unknown;
+amqp_table(undefined) -> amqp_table([]);
+amqp_table([]) -> #{};
+amqp_table(#{}) -> #{};
+amqp_table(Table) -> maps:from_list([{Name, amqp_value(Type, Value)} ||
+ {Name, Type, Value} <- Table]).
+
+amqp_value(array, Vs) -> [amqp_value(T, V) || {T, V} <- Vs];
+amqp_value(table, V) -> amqp_table(V);
+amqp_value(decimal, {Before, After}) ->
+ erlang:list_to_float(
+ lists:flatten(io_lib:format("~p.~p", [Before, After])));
+amqp_value(_Type, V) when is_binary(V) -> utf8_safe(V);
+amqp_value(_Type, V) -> V.
+
+
+%%
+%% Resources
+%%
+
+r(#resource{virtual_host = VHostPath}, Kind, Name) ->
+ #resource{virtual_host = VHostPath, kind = Kind, name = Name};
+r(VHostPath, Kind, Name) ->
+ #resource{virtual_host = VHostPath, kind = Kind, name = Name}.
+
+r(VHostPath, Kind) ->
+ #resource{virtual_host = VHostPath, kind = Kind, name = '_'}.
+
+r_arg(#resource{virtual_host = VHostPath}, Kind, Table, Key) ->
+ r_arg(VHostPath, Kind, Table, Key);
+r_arg(VHostPath, Kind, Table, Key) ->
+ case table_lookup(Table, Key) of
+ {longstr, NameBin} -> r(VHostPath, Kind, NameBin);
+ undefined -> undefined;
+ {Type, _} -> {error, {invalid_type, Type}}
+ end.
+
+rs(#resource{virtual_host = VHostPath, kind = topic, name = Name}) ->
+ format("'~s' in vhost '~s'", [Name, VHostPath]);
+rs(#resource{virtual_host = VHostPath, kind = Kind, name = Name}) ->
+ format("~s '~s' in vhost '~s'", [Kind, Name, VHostPath]).
+
+enable_cover() -> enable_cover(["."]).
+
+enable_cover(Dirs) ->
+ lists:foldl(fun (Dir, ok) ->
+ case cover:compile_beam_directory(
+ filename:join(lists:concat([Dir]),"ebin")) of
+ {error, _} = Err -> Err;
+ _ -> ok
+ end;
+ (_Dir, Err) ->
+ Err
+ end, ok, Dirs).
+
+start_cover(NodesS) ->
+ {ok, _} = cover:start([rabbit_nodes_common:make(N) || N <- NodesS]),
+ ok.
+
+report_cover() -> report_cover(["."]).
+
+report_cover(Dirs) -> [report_cover1(lists:concat([Dir])) || Dir <- Dirs], ok.
+
+report_cover1(Root) ->
+ Dir = filename:join(Root, "cover"),
+ ok = filelib:ensure_dir(filename:join(Dir, "junk")),
+ lists:foreach(fun (F) -> file:delete(F) end,
+ filelib:wildcard(filename:join(Dir, "*.html"))),
+ {ok, SummaryFile} = file:open(filename:join(Dir, "summary.txt"), [write]),
+ {CT, NCT} =
+ lists:foldl(
+ fun (M,{CovTot, NotCovTot}) ->
+ {ok, {M, {Cov, NotCov}}} = cover:analyze(M, module),
+ ok = report_coverage_percentage(SummaryFile,
+ Cov, NotCov, M),
+ {ok,_} = cover:analyze_to_file(
+ M,
+ filename:join(Dir, atom_to_list(M) ++ ".html"),
+ [html]),
+ {CovTot+Cov, NotCovTot+NotCov}
+ end,
+ {0, 0},
+ lists:sort(cover:modules())),
+ ok = report_coverage_percentage(SummaryFile, CT, NCT, 'TOTAL'),
+ ok = file:close(SummaryFile),
+ ok.
+
+report_coverage_percentage(File, Cov, NotCov, Mod) ->
+ io:fwrite(File, "~6.2f ~p~n",
+ [if
+ Cov+NotCov > 0 -> 100.0*Cov/(Cov+NotCov);
+ true -> 100.0
+ end,
+ Mod]).
+
+%% @doc Halts the emulator returning the given status code to the os.
+%% On Windows this function will block indefinitely so as to give the io
+%% subsystem time to flush stdout completely.
+quit(Status) ->
+ case os:type() of
+ {unix, _} -> halt(Status);
+ {win32, _} -> init:stop(Status),
+ receive
+ after infinity -> ok
+ end
+ end.
+
+throw_on_error(E, Thunk) ->
+ case Thunk() of
+ {error, Reason} -> throw({E, Reason});
+ {ok, Res} -> Res;
+ Res -> Res
+ end.
+
+with_exit_handler(Handler, Thunk) ->
+ try
+ Thunk()
+ catch
+ exit:{R, _} when ?IS_BENIGN_EXIT(R) -> Handler();
+ exit:{{R, _}, _} when ?IS_BENIGN_EXIT(R) -> Handler()
+ end.
+
+is_abnormal_exit(R) when ?IS_BENIGN_EXIT(R) -> false;
+is_abnormal_exit({R, _}) when ?IS_BENIGN_EXIT(R) -> false;
+is_abnormal_exit(_) -> true.
+
+filter_exit_map(F, L) ->
+ Ref = make_ref(),
+ lists:filter(fun (R) -> R =/= Ref end,
+ [with_exit_handler(
+ fun () -> Ref end,
+ fun () -> F(I) end) || I <- L]).
+
+
+with_user(Username, Thunk) ->
+ fun () ->
+ case mnesia:read({rabbit_user, Username}) of
+ [] ->
+ mnesia:abort({no_such_user, Username});
+ [_U] ->
+ Thunk()
+ end
+ end.
+
+execute_mnesia_transaction(TxFun) ->
+ %% Making this a sync_transaction allows us to use dirty_read
+ %% elsewhere and get a consistent result even when that read
+ %% executes on a different node.
+ case worker_pool:submit(
+ fun () ->
+ case mnesia:is_transaction() of
+ false -> DiskLogBefore = mnesia_dumper:get_log_writes(),
+ Res = mnesia:sync_transaction(TxFun),
+ DiskLogAfter = mnesia_dumper:get_log_writes(),
+ case DiskLogAfter == DiskLogBefore of
+ true -> file_handle_cache_stats:update(
+ mnesia_ram_tx),
+ Res;
+ false -> file_handle_cache_stats:update(
+ mnesia_disk_tx),
+ {sync, Res}
+ end;
+ true -> mnesia:sync_transaction(TxFun)
+ end
+ end, single) of
+ {sync, {atomic, Result}} -> mnesia_sync:sync(), Result;
+ {sync, {aborted, Reason}} -> throw({error, Reason});
+ {atomic, Result} -> Result;
+ {aborted, Reason} -> throw({error, Reason})
+ end.
+
+%% Like execute_mnesia_transaction/1 with additional Pre- and Post-
+%% commit function
+execute_mnesia_transaction(TxFun, PrePostCommitFun) ->
+ case mnesia:is_transaction() of
+ true -> throw(unexpected_transaction);
+ false -> ok
+ end,
+ PrePostCommitFun(execute_mnesia_transaction(
+ fun () ->
+ Result = TxFun(),
+ PrePostCommitFun(Result, true),
+ Result
+ end), false).
+
+%% Like execute_mnesia_transaction/2, but TxFun is expected to return a
+%% TailFun which gets called (only) immediately after the tx commit
+execute_mnesia_tx_with_tail(TxFun) ->
+ case mnesia:is_transaction() of
+ true -> execute_mnesia_transaction(TxFun);
+ false -> TailFun = execute_mnesia_transaction(TxFun),
+ TailFun()
+ end.
+
+ensure_ok(ok, _) -> ok;
+ensure_ok({error, Reason}, ErrorTag) -> throw({error, {ErrorTag, Reason}}).
+
+tcp_name(Prefix, IPAddress, Port)
+ when is_atom(Prefix) andalso is_number(Port) ->
+ list_to_atom(
+ format("~w_~s:~w", [Prefix, inet_parse:ntoa(IPAddress), Port])).
+
+format_inet_error(E) -> format("~w (~s)", [E, format_inet_error0(E)]).
+
+format_inet_error0(address) -> "cannot connect to host/port";
+format_inet_error0(timeout) -> "timed out";
+format_inet_error0(Error) -> inet:format_error(Error).
+
+%% base64:decode throws lots of weird errors. Catch and convert to one
+%% that will cause a bad_request.
+b64decode_or_throw(B64) ->
+ try
+ base64:decode(B64)
+ catch error:_ ->
+ throw({error, {not_base64, B64}})
+ end.
+
+utf8_safe(V) ->
+ try
+ _ = xmerl_ucs:from_utf8(V),
+ V
+ catch exit:{ucs, _} ->
+ Enc = split_lines(base64:encode(V)),
+ <<"Not UTF-8, base64 is: ", Enc/binary>>
+ end.
+
+%% MIME enforces a limit on line length of base 64-encoded data to 76 characters.
+split_lines(<<Text:76/binary, Rest/binary>>) ->
+ <<Text/binary, $\n, (split_lines(Rest))/binary>>;
+split_lines(Text) ->
+ Text.
+
+
+%% This is a modified version of Luke Gorrie's pmap -
+%% https://lukego.livejournal.com/6753.html - that doesn't care about
+%% the order in which results are received.
+%%
+%% WARNING: This is is deliberately lightweight rather than robust -- if F
+%% throws, upmap will hang forever, so make sure F doesn't throw!
+upmap(F, L) ->
+ Parent = self(),
+ Ref = make_ref(),
+ [receive {Ref, Result} -> Result end
+ || _ <- [spawn(fun () -> Parent ! {Ref, F(X)} end) || X <- L]].
+
+map_in_order(F, L) ->
+ lists:reverse(
+ lists:foldl(fun (E, Acc) -> [F(E) | Acc] end, [], L)).
+
+%% Apply a pre-post-commit function to all entries in a table that
+%% satisfy a predicate, and return those entries.
+%%
+%% We ignore entries that have been modified or removed.
+table_filter(Pred, PrePostCommitFun, TableName) ->
+ lists:foldl(
+ fun (E, Acc) ->
+ case execute_mnesia_transaction(
+ fun () -> mnesia:match_object(TableName, E, read) =/= []
+ andalso Pred(E) end,
+ fun (false, _Tx) -> false;
+ (true, Tx) -> PrePostCommitFun(E, Tx), true
+ end) of
+ false -> Acc;
+ true -> [E | Acc]
+ end
+ end, [], dirty_read_all(TableName)).
+
+dirty_read_all(TableName) ->
+ mnesia:dirty_select(TableName, [{'$1',[],['$1']}]).
+
+dirty_foreach_key(F, TableName) ->
+ dirty_foreach_key1(F, TableName, mnesia:dirty_first(TableName)).
+
+dirty_foreach_key1(_F, _TableName, '$end_of_table') ->
+ ok;
+dirty_foreach_key1(F, TableName, K) ->
+ case catch mnesia:dirty_next(TableName, K) of
+ {'EXIT', _} ->
+ aborted;
+ NextKey ->
+ F(K),
+ dirty_foreach_key1(F, TableName, NextKey)
+ end.
+
+dirty_dump_log(FileName) ->
+ {ok, LH} = disk_log:open([{name, dirty_dump_log},
+ {mode, read_only},
+ {file, FileName}]),
+ dirty_dump_log1(LH, disk_log:chunk(LH, start)),
+ disk_log:close(LH).
+
+dirty_dump_log1(_LH, eof) ->
+ io:format("Done.~n");
+dirty_dump_log1(LH, {K, Terms}) ->
+ io:format("Chunk: ~p~n", [Terms]),
+ dirty_dump_log1(LH, disk_log:chunk(LH, K));
+dirty_dump_log1(LH, {K, Terms, BadBytes}) ->
+ io:format("Bad Chunk, ~p: ~p~n", [BadBytes, Terms]),
+ dirty_dump_log1(LH, disk_log:chunk(LH, K)).
+
+format(Fmt, Args) -> lists:flatten(io_lib:format(Fmt, Args)).
+
+format_many(List) ->
+ lists:flatten([io_lib:format(F ++ "~n", A) || {F, A} <- List]).
+
+format_stderr(Fmt, Args) ->
+ io:format(standard_error, Fmt, Args),
+ ok.
+
+unfold(Fun, Init) ->
+ unfold(Fun, [], Init).
+
+unfold(Fun, Acc, Init) ->
+ case Fun(Init) of
+ {true, E, I} -> unfold(Fun, [E|Acc], I);
+ false -> {Acc, Init}
+ end.
+
+ceil(N) ->
+ T = trunc(N),
+ case N == T of
+ true -> T;
+ false -> 1 + T
+ end.
+
+parse_bool(<<"true">>) -> true;
+parse_bool(<<"false">>) -> false;
+parse_bool(true) -> true;
+parse_bool(false) -> false;
+parse_bool(undefined) -> undefined;
+parse_bool(V) -> throw({error, {not_boolean, V}}).
+
+parse_int(I) when is_integer(I) -> I;
+parse_int(F) when is_number(F) -> trunc(F);
+parse_int(S) -> try
+ list_to_integer(binary_to_list(S))
+ catch error:badarg ->
+ throw({error, {not_integer, S}})
+ end.
+
+
+queue_fold(Fun, Init, Q) ->
+ case queue:out(Q) of
+ {empty, _Q} -> Init;
+ {{value, V}, Q1} -> queue_fold(Fun, Fun(V, Init), Q1)
+ end.
+
+%% Sorts a list of AMQP 0-9-1 table fields as per the AMQP 0-9-1 spec
+sort_field_table([]) ->
+ [];
+sort_field_table(M) when is_map(M) andalso map_size(M) =:= 0 ->
+ [];
+sort_field_table(Arguments) when is_map(Arguments) ->
+ sort_field_table(maps:to_list(Arguments));
+sort_field_table(Arguments) ->
+ lists:keysort(1, Arguments).
+
+atom_to_binary(A) ->
+ list_to_binary(atom_to_list(A)).
+
+%% This provides a string representation of a pid that is the same
+%% regardless of what node we are running on. The representation also
+%% permits easy identification of the pid's node.
+pid_to_string(Pid) when is_pid(Pid) ->
+ {Node, Cre, Id, Ser} = decompose_pid(Pid),
+ format("<~s.~B.~B.~B>", [Node, Cre, Id, Ser]).
+
+%% inverse of above
+string_to_pid(Str) ->
+ Err = {error, {invalid_pid_syntax, Str}},
+ %% The \ before the trailing $ is only there to keep emacs
+ %% font-lock from getting confused.
+ case re:run(Str, "^<(.*)\\.(\\d+)\\.(\\d+)\\.(\\d+)>\$",
+ [{capture,all_but_first,list}]) of
+ {match, [NodeStr, CreStr, IdStr, SerStr]} ->
+ [Cre, Id, Ser] = lists:map(fun list_to_integer/1,
+ [CreStr, IdStr, SerStr]),
+ compose_pid(list_to_atom(NodeStr), Cre, Id, Ser);
+ nomatch ->
+ throw(Err)
+ end.
+
+pid_change_node(Pid, NewNode) ->
+ {_OldNode, Cre, Id, Ser} = decompose_pid(Pid),
+ compose_pid(NewNode, Cre, Id, Ser).
+
+%% node(node_to_fake_pid(Node)) =:= Node.
+node_to_fake_pid(Node) ->
+ compose_pid(Node, 0, 0, 0).
+
+decompose_pid(Pid) when is_pid(Pid) ->
+ %% see http://erlang.org/doc/apps/erts/erl_ext_dist.html (8.10 and
+ %% 8.7)
+ Node = node(Pid),
+ BinPid0 = term_to_binary(Pid),
+ case BinPid0 of
+ %% NEW_PID_EXT
+ <<131, 88, BinPid/bits>> ->
+ NodeByteSize = byte_size(BinPid0) - 14,
+ <<_NodePrefix:NodeByteSize/binary, Id:32, Ser:32, Cre:32>> = BinPid,
+ {Node, Cre, Id, Ser};
+ %% PID_EXT
+ <<131, 103, BinPid/bits>> ->
+ NodeByteSize = byte_size(BinPid0) - 11,
+ <<_NodePrefix:NodeByteSize/binary, Id:32, Ser:32, Cre:8>> = BinPid,
+ {Node, Cre, Id, Ser}
+ end.
+
+compose_pid(Node, Cre, Id, Ser) ->
+ <<131,NodeEnc/binary>> = term_to_binary(Node),
+ binary_to_term(<<131,88,NodeEnc/binary,Id:32,Ser:32,Cre:32>>).
+
+version_compare(A, B, eq) -> rabbit_semver:eql(A, B);
+version_compare(A, B, lt) -> rabbit_semver:lt(A, B);
+version_compare(A, B, lte) -> rabbit_semver:lte(A, B);
+version_compare(A, B, gt) -> rabbit_semver:gt(A, B);
+version_compare(A, B, gte) -> rabbit_semver:gte(A, B).
+
+version_compare(A, B) ->
+ case version_compare(A, B, lt) of
+ true -> lt;
+ false -> case version_compare(A, B, gt) of
+ true -> gt;
+ false -> eq
+ end
+ end.
+
+%% For versions starting from 3.7.x:
+%% Versions are considered compatible (except for special cases; see
+%% below). The feature flags will determine if they are actually
+%% compatible.
+%%
+%% For versions up-to 3.7.x:
+%% a.b.c and a.b.d match, but a.b.c and a.d.e don't. If
+%% versions do not match that pattern, just compare them.
+%%
+%% Special case for 3.6.6 because it introduced a change to the schema.
+%% e.g. 3.6.6 is not compatible with 3.6.5
+%% This special case can be removed once 3.6.x reaches EOL
+version_minor_equivalent(A, B) ->
+ {{MajA, MinA, PatchA, _}, _} = rabbit_semver:normalize(rabbit_semver:parse(A)),
+ {{MajB, MinB, PatchB, _}, _} = rabbit_semver:normalize(rabbit_semver:parse(B)),
+
+ case {MajA, MinA, MajB, MinB} of
+ {3, 6, 3, 6} ->
+ if
+ PatchA >= 6 -> PatchB >= 6;
+ PatchA < 6 -> PatchB < 6;
+ true -> false
+ end;
+ _
+ when (MajA < 3 orelse (MajA =:= 3 andalso MinA =< 6))
+ orelse
+ (MajB < 3 orelse (MajB =:= 3 andalso MinB =< 6)) ->
+ MajA =:= MajB andalso MinA =:= MinB;
+ _ ->
+ %% Starting with RabbitMQ 3.7.x, we consider this
+ %% minor release series and all subsequent series to
+ %% be possibly compatible, based on just the version.
+ %% The real compatibility check is deferred to the
+ %% rabbit_feature_flags module in rabbitmq-server.
+ true
+ end.
+
+%% This is the same as above except that e.g. 3.7.x and 3.8.x are
+%% considered incompatible (as if there were no feature flags). This is
+%% useful to check plugin compatibility (`broker_versions_requirement`
+%% field in plugins).
+
+strict_version_minor_equivalent(A, B) ->
+ {{MajA, MinA, PatchA, _}, _} = rabbit_semver:normalize(rabbit_semver:parse(A)),
+ {{MajB, MinB, PatchB, _}, _} = rabbit_semver:normalize(rabbit_semver:parse(B)),
+
+ case {MajA, MinA, MajB, MinB} of
+ {3, 6, 3, 6} -> if
+ PatchA >= 6 -> PatchB >= 6;
+ PatchA < 6 -> PatchB < 6;
+ true -> false
+ end;
+ _ -> MajA =:= MajB andalso MinA =:= MinB
+ end.
+
+dict_cons(Key, Value, Dict) ->
+ dict:update(Key, fun (List) -> [Value | List] end, [Value], Dict).
+
+orddict_cons(Key, Value, Dict) ->
+ orddict:update(Key, fun (List) -> [Value | List] end, [Value], Dict).
+
+maps_cons(Key, Value, Map) ->
+ maps:update_with(Key, fun (List) -> [Value | List] end, [Value], Map).
+
+gb_trees_cons(Key, Value, Tree) ->
+ case gb_trees:lookup(Key, Tree) of
+ {value, Values} -> gb_trees:update(Key, [Value | Values], Tree);
+ none -> gb_trees:insert(Key, [Value], Tree)
+ end.
+
+gb_trees_fold(Fun, Acc, Tree) ->
+ gb_trees_fold1(Fun, Acc, gb_trees:next(gb_trees:iterator(Tree))).
+
+gb_trees_fold1(_Fun, Acc, none) ->
+ Acc;
+gb_trees_fold1(Fun, Acc, {Key, Val, It}) ->
+ gb_trees_fold1(Fun, Fun(Key, Val, Acc), gb_trees:next(It)).
+
+gb_trees_foreach(Fun, Tree) ->
+ gb_trees_fold(fun (Key, Val, Acc) -> Fun(Key, Val), Acc end, ok, Tree).
+
+module_attributes(Module) ->
+ try
+ Module:module_info(attributes)
+ catch
+ _:undef ->
+ io:format("WARNING: module ~p not found, so not scanned for boot steps.~n",
+ [Module]),
+ []
+ end.
+
+all_module_attributes(Name) ->
+ Apps = [App || {App, _, _} <- application:loaded_applications()],
+ module_attributes_from_apps(Name, Apps).
+
+rabbitmq_related_module_attributes(Name) ->
+ Apps = rabbitmq_related_apps(),
+ module_attributes_from_apps(Name, Apps).
+
+rabbitmq_related_apps() ->
+ [App
+ || {App, _, _} <- application:loaded_applications(),
+ %% Only select RabbitMQ-related applications.
+ App =:= rabbit_common orelse
+ App =:= rabbitmq_prelaunch orelse
+ App =:= rabbit orelse
+ lists:member(
+ rabbit,
+ element(2, application:get_key(App, applications)))].
+
+module_attributes_from_apps(Name, Apps) ->
+ Targets =
+ lists:usort(
+ lists:append(
+ [[{App, Module} || Module <- Modules] ||
+ App <- Apps,
+ {ok, Modules} <- [application:get_key(App, modules)]])),
+ lists:foldl(
+ fun ({App, Module}, Acc) ->
+ case lists:append([Atts || {N, Atts} <- module_attributes(Module),
+ N =:= Name]) of
+ [] -> Acc;
+ Atts -> [{App, Module, Atts} | Acc]
+ end
+ end, [], Targets).
+
+build_acyclic_graph(VertexFun, EdgeFun, Graph) ->
+ G = digraph:new([acyclic]),
+ try
+ _ = [case digraph:vertex(G, Vertex) of
+ false -> digraph:add_vertex(G, Vertex, Label);
+ _ -> ok = throw({graph_error, {vertex, duplicate, Vertex}})
+ end || GraphElem <- Graph,
+ {Vertex, Label} <- VertexFun(GraphElem)],
+ [case digraph:add_edge(G, From, To) of
+ {error, E} -> throw({graph_error, {edge, E, From, To}});
+ _ -> ok
+ end || GraphElem <- Graph,
+ {From, To} <- EdgeFun(GraphElem)],
+ {ok, G}
+ catch {graph_error, Reason} ->
+ true = digraph:delete(G),
+ {error, Reason}
+ end.
+
+const(X) -> fun () -> X end.
+
+%% Format IPv4-mapped IPv6 addresses as IPv4, since they're what we see
+%% when IPv6 is enabled but not used (i.e. 99% of the time).
+ntoa({0,0,0,0,0,16#ffff,AB,CD}) ->
+ inet_parse:ntoa({AB bsr 8, AB rem 256, CD bsr 8, CD rem 256});
+ntoa(IP) ->
+ inet_parse:ntoa(IP).
+
+ntoab(IP) ->
+ Str = ntoa(IP),
+ case string:str(Str, ":") of
+ 0 -> Str;
+ _ -> "[" ++ Str ++ "]"
+ end.
+
+%% We try to avoid reconnecting to down nodes here; this is used in a
+%% loop in rabbit_amqqueue:on_node_down/1 and any delays we incur
+%% would be bad news.
+%%
+%% See also rabbit_mnesia:is_process_alive/1 which also requires the
+%% process be in the same running cluster as us (i.e. not partitioned
+%% or some random node).
+is_process_alive(Pid) when node(Pid) =:= node() ->
+ erlang:is_process_alive(Pid);
+is_process_alive(Pid) ->
+ Node = node(Pid),
+ lists:member(Node, [node() | nodes(connected)]) andalso
+ rpc:call(Node, erlang, is_process_alive, [Pid]) =:= true.
+
+-spec pget(term(), list() | map()) -> term().
+pget(K, M) when is_map(M) ->
+ case maps:find(K, M) of
+ {ok, V} ->
+ V;
+ _ ->
+ undefined
+ end;
+
+pget(K, P) ->
+ case lists:keyfind(K, 1, P) of
+ {K, V} ->
+ V;
+ _ ->
+ undefined
+ end.
+
+-spec pget(term(), list() | map(), term()) -> term().
+pget(K, M, D) when is_map(M) ->
+ case maps:find(K, M) of
+ {ok, V} ->
+ V;
+ _ ->
+ D
+ end;
+
+pget(K, P, D) ->
+ case lists:keyfind(K, 1, P) of
+ {K, V} ->
+ V;
+ _ ->
+ D
+ end.
+
+-spec pget_or_die(term(), list() | map()) -> term() | no_return().
+pget_or_die(K, M) when is_map(M) ->
+ case maps:find(K, M) of
+ error -> exit({error, key_missing, K});
+ {ok, V} -> V
+ end;
+
+pget_or_die(K, P) ->
+ case proplists:get_value(K, P) of
+ undefined -> exit({error, key_missing, K});
+ V -> V
+ end.
+
+pupdate(K, UpdateFun, P) ->
+ case lists:keyfind(K, 1, P) of
+ {K, V} ->
+ pset(K, UpdateFun(V), P);
+ _ ->
+ undefined
+ end.
+
+%% property merge
+pmerge(Key, Val, List) ->
+ case proplists:is_defined(Key, List) of
+ true -> List;
+ _ -> [{Key, Val} | List]
+ end.
+
+%% proplists merge
+plmerge(P1, P2) ->
+ %% Value from P1 suppresses value from P2
+ maps:to_list(maps:merge(maps:from_list(P2),
+ maps:from_list(P1))).
+
+%% groups a list of proplists by a key function
+group_proplists_by(KeyFun, ListOfPropLists) ->
+ Res = lists:foldl(fun(P, Agg) ->
+ Key = KeyFun(P),
+ Val = case maps:find(Key, Agg) of
+ {ok, O} -> [P|O];
+ error -> [P]
+ end,
+ maps:put(Key, Val, Agg)
+ end, #{}, ListOfPropLists),
+ [ X || {_, X} <- maps:to_list(Res)].
+
+pset(Key, Value, List) -> [{Key, Value} | proplists:delete(Key, List)].
+
+format_message_queue(_Opt, MQ) ->
+ Len = priority_queue:len(MQ),
+ {Len,
+ case Len > 100 of
+ false -> priority_queue:to_list(MQ);
+ true -> {summary,
+ maps:to_list(
+ lists:foldl(
+ fun ({P, V}, Counts) ->
+ maps:update_with(
+ {P, format_message_queue_entry(V)},
+ fun(Old) -> Old + 1 end, 1, Counts)
+ end, maps:new(), priority_queue:to_list(MQ)))}
+ end}.
+
+format_message_queue_entry(V) when is_atom(V) ->
+ V;
+format_message_queue_entry(V) when is_tuple(V) ->
+ list_to_tuple([format_message_queue_entry(E) || E <- tuple_to_list(V)]);
+format_message_queue_entry(_V) ->
+ '_'.
+
+%% Same as rpc:multicall/4 but concatenates all results.
+%% M, F, A is expected to return a list. If it does not,
+%% its return value will be wrapped in a list.
+-spec append_rpc_all_nodes([node()], atom(), atom(), [any()]) -> [any()].
+append_rpc_all_nodes(Nodes, M, F, A) ->
+ do_append_rpc_all_nodes(Nodes, M, F, A, ?RPC_INFINITE_TIMEOUT).
+
+-spec append_rpc_all_nodes([node()], atom(), atom(), [any()], timeout()) -> [any()].
+append_rpc_all_nodes(Nodes, M, F, A, Timeout) ->
+ do_append_rpc_all_nodes(Nodes, M, F, A, Timeout).
+
+do_append_rpc_all_nodes(Nodes, M, F, A, ?RPC_INFINITE_TIMEOUT) ->
+ {ResL, _} = rpc:multicall(Nodes, M, F, A, ?RPC_INFINITE_TIMEOUT),
+ process_rpc_multicall_result(ResL);
+do_append_rpc_all_nodes(Nodes, M, F, A, Timeout) ->
+ {ResL, _} = try
+ rpc:multicall(Nodes, M, F, A, Timeout)
+ catch
+ error:internal_error -> {[], Nodes}
+ end,
+ process_rpc_multicall_result(ResL).
+
+process_rpc_multicall_result(ResL) ->
+ lists:append([case Res of
+ {badrpc, _} -> [];
+ Xs when is_list(Xs) -> Xs;
+ %% wrap it in a list
+ Other -> [Other]
+ end || Res <- ResL]).
+
+os_cmd(Command) ->
+ case os:type() of
+ {win32, _} ->
+ %% Clink workaround; see
+ %% https://code.google.com/p/clink/issues/detail?id=141
+ os:cmd(" " ++ Command);
+ _ ->
+ %% Don't just return "/bin/sh: <cmd>: not found" if not found
+ Exec = hd(string:tokens(Command, " ")),
+ case os:find_executable(Exec) of
+ false -> throw({command_not_found, Exec});
+ _ -> os:cmd(Command)
+ end
+ end.
+
+is_os_process_alive(Pid) ->
+ with_os([{unix, fun () ->
+ run_ps(Pid) =:= 0
+ end},
+ {win32, fun () ->
+ PidS = rabbit_data_coercion:to_list(Pid),
+ case os:find_executable("tasklist.exe") of
+ false ->
+ Cmd =
+ format(
+ "PowerShell -Command "
+ "\"(Get-Process -Id ~s).ProcessName\"",
+ [PidS]),
+ Res =
+ os_cmd(Cmd ++ " 2>&1") -- [$\r, $\n],
+ case Res of
+ "erl" -> true;
+ "werl" -> true;
+ _ -> false
+ end;
+ _ ->
+ Cmd =
+ "tasklist /nh /fi "
+ "\"pid eq " ++ PidS ++ "\"",
+ Res = os_cmd(Cmd ++ " 2>&1"),
+ match =:= re:run(Res,
+ "erl\\.exe",
+ [{capture, none}])
+ end
+ end}]).
+
+with_os(Handlers) ->
+ {OsFamily, _} = os:type(),
+ case proplists:get_value(OsFamily, Handlers) of
+ undefined -> throw({unsupported_os, OsFamily});
+ Handler -> Handler()
+ end.
+
+run_ps(Pid) ->
+ Cmd = "ps -p " ++ rabbit_data_coercion:to_list(Pid),
+ Port = erlang:open_port({spawn, Cmd},
+ [exit_status, {line, 16384},
+ use_stdio, stderr_to_stdout]),
+ exit_loop(Port).
+
+exit_loop(Port) ->
+ receive
+ {Port, {exit_status, Rc}} -> Rc;
+ {Port, _} -> exit_loop(Port)
+ end.
+
+gb_sets_difference(S1, S2) ->
+ gb_sets:fold(fun gb_sets:delete_any/2, S1, S2).
+
+version() ->
+ {ok, VSN} = application:get_key(rabbit, vsn),
+ VSN.
+
+%% See https://www.erlang.org/doc/system_principles/versions.html
+otp_release() ->
+ File = filename:join([code:root_dir(), "releases",
+ erlang:system_info(otp_release), "OTP_VERSION"]),
+ case file:read_file(File) of
+ {ok, VerBin} ->
+ %% 17.0 or later, we need the file for the minor version
+ string:strip(binary_to_list(VerBin), both, $\n);
+ {error, _} ->
+ %% R16B03 or earlier (no file, otp_release is correct)
+ %% or we couldn't read the file (so this is best we can do)
+ erlang:system_info(otp_release)
+ end.
+
+platform_and_version() ->
+ string:join(["Erlang/OTP", otp_release()], " ").
+
+otp_system_version() ->
+ string:strip(erlang:system_info(system_version), both, $\n).
+
+rabbitmq_and_erlang_versions() ->
+ {version(), otp_release()}.
+
+%% application:which_applications(infinity) is dangerous, since it can
+%% cause deadlocks on shutdown. So we have to use a timeout variant,
+%% but w/o creating spurious timeout errors. The timeout value is twice
+%% that of gen_server:call/2.
+which_applications() ->
+ try
+ application:which_applications(10000)
+ catch
+ exit:{timeout, _} -> []
+ end.
+
+sequence_error([T]) -> T;
+sequence_error([{error, _} = Error | _]) -> Error;
+sequence_error([_ | Rest]) -> sequence_error(Rest).
+
+check_expiry(N) when N < 0 -> {error, {value_negative, N}};
+check_expiry(_N) -> ok.
+
+base64url(In) ->
+ lists:reverse(lists:foldl(fun ($\+, Acc) -> [$\- | Acc];
+ ($\/, Acc) -> [$\_ | Acc];
+ ($\=, Acc) -> Acc;
+ (Chr, Acc) -> [Chr | Acc]
+ end, [], base64:encode_to_string(In))).
+
+%% Ideally, you'd want Fun to run every IdealInterval. but you don't
+%% want it to take more than MaxRatio of IdealInterval. So if it takes
+%% more then you want to run it less often. So we time how long it
+%% takes to run, and then suggest how long you should wait before
+%% running it again with a user specified max interval. Times are in millis.
+interval_operation({M, F, A}, MaxRatio, MaxInterval, IdealInterval, LastInterval) ->
+ {Micros, Res} = timer:tc(M, F, A),
+ {Res, case {Micros > 1000 * (MaxRatio * IdealInterval),
+ Micros > 1000 * (MaxRatio * LastInterval)} of
+ {true, true} -> lists:min([MaxInterval,
+ round(LastInterval * 1.5)]);
+ {true, false} -> LastInterval;
+ {false, false} -> lists:max([IdealInterval,
+ round(LastInterval / 1.5)])
+ end}.
+
+ensure_timer(State, Idx, After, Msg) ->
+ case element(Idx, State) of
+ undefined -> TRef = send_after(After, self(), Msg),
+ setelement(Idx, State, TRef);
+ _ -> State
+ end.
+
+stop_timer(State, Idx) ->
+ case element(Idx, State) of
+ undefined -> State;
+ TRef -> cancel_timer(TRef),
+ setelement(Idx, State, undefined)
+ end.
+
+%% timer:send_after/3 goes through a single timer process but allows
+%% long delays. erlang:send_after/3 does not have a bottleneck but
+%% only allows max 2^32-1 millis.
+-define(MAX_ERLANG_SEND_AFTER, 4294967295).
+send_after(Millis, Pid, Msg) when Millis > ?MAX_ERLANG_SEND_AFTER ->
+ {ok, Ref} = timer:send_after(Millis, Pid, Msg),
+ {timer, Ref};
+send_after(Millis, Pid, Msg) ->
+ {erlang, erlang:send_after(Millis, Pid, Msg)}.
+
+cancel_timer({erlang, Ref}) -> _ = erlang:cancel_timer(Ref),
+ ok;
+cancel_timer({timer, Ref}) -> {ok, cancel} = timer:cancel(Ref),
+ ok.
+
+store_proc_name(Type, ProcName) -> store_proc_name({Type, ProcName}).
+store_proc_name(TypeProcName) -> put(process_name, TypeProcName).
+
+get_proc_name() ->
+ case get(process_name) of
+ undefined ->
+ undefined;
+ {_Type, Name} ->
+ {ok, Name}
+ end.
+
+%% application:get_env/3 is only available in R16B01 or later.
+get_env(Application, Key, Def) ->
+ case application:get_env(Application, Key) of
+ {ok, Val} -> Val;
+ undefined -> Def
+ end.
+
+get_channel_operation_timeout() ->
+ %% Default channel_operation_timeout set to net_ticktime + 10s to
+ %% give allowance for any down messages to be received first,
+ %% whenever it is used for cross-node calls with timeouts.
+ Default = (net_kernel:get_net_ticktime() + 10) * 1000,
+ application:get_env(rabbit, channel_operation_timeout, Default).
+
+moving_average(_Time, _HalfLife, Next, undefined) ->
+ Next;
+%% We want the Weight to decrease as Time goes up (since Weight is the
+%% weight for the current sample, not the new one), so that the moving
+%% average decays at the same speed regardless of how long the time is
+%% between samplings. So we want Weight = math:exp(Something), where
+%% Something turns out to be negative.
+%%
+%% We want to determine Something here in terms of the Time taken
+%% since the last measurement, and a HalfLife. So we want Weight =
+%% math:exp(Time * Constant / HalfLife). What should Constant be? We
+%% want Weight to be 0.5 when Time = HalfLife.
+%%
+%% Plug those numbers in and you get 0.5 = math:exp(Constant). Take
+%% the log of each side and you get math:log(0.5) = Constant.
+moving_average(Time, HalfLife, Next, Current) ->
+ Weight = math:exp(Time * math:log(0.5) / HalfLife),
+ Next * (1 - Weight) + Current * Weight.
+
+random(N) ->
+ rand:uniform(N).
+
+-spec escape_html_tags(string()) -> binary().
+
+escape_html_tags(S) ->
+ escape_html_tags(rabbit_data_coercion:to_list(S), []).
+
+
+-spec escape_html_tags(string(), string()) -> binary().
+
+escape_html_tags([], Acc) ->
+ rabbit_data_coercion:to_binary(lists:reverse(Acc));
+escape_html_tags("<" ++ Rest, Acc) ->
+ escape_html_tags(Rest, lists:reverse("&lt;", Acc));
+escape_html_tags(">" ++ Rest, Acc) ->
+ escape_html_tags(Rest, lists:reverse("&gt;", Acc));
+escape_html_tags("&" ++ Rest, Acc) ->
+ escape_html_tags(Rest, lists:reverse("&amp;", Acc));
+escape_html_tags([C | Rest], Acc) ->
+ escape_html_tags(Rest, [C | Acc]).
+
+%% If the server we are talking to has non-standard net_ticktime, and
+%% our connection lasts a while, we could get disconnected because of
+%% a timeout unless we set our ticktime to be the same. So let's do
+%% that.
+%% TODO: do not use an infinite timeout!
+-spec rpc_call(node(), atom(), atom(), [any()]) -> any() | {badrpc, term()}.
+rpc_call(Node, Mod, Fun, Args) ->
+ rpc_call(Node, Mod, Fun, Args, ?RPC_INFINITE_TIMEOUT).
+
+-spec rpc_call(node(), atom(), atom(), [any()], infinity | non_neg_integer()) -> any() | {badrpc, term()}.
+rpc_call(Node, Mod, Fun, Args, Timeout) ->
+ case rpc:call(Node, net_kernel, get_net_ticktime, [], Timeout) of
+ {badrpc, _} = E -> E;
+ ignored ->
+ rpc:call(Node, Mod, Fun, Args, Timeout);
+ {ongoing_change_to, NewValue} ->
+ _ = net_kernel:set_net_ticktime(NewValue, 0),
+ rpc:call(Node, Mod, Fun, Args, Timeout);
+ Time ->
+ _ = net_kernel:set_net_ticktime(Time, 0),
+ rpc:call(Node, Mod, Fun, Args, Timeout)
+ end.
+
+get_gc_info(Pid) ->
+ rabbit_runtime:get_gc_info(Pid).
+
+%% -------------------------------------------------------------------------
+%% Begin copypasta from gen_server2.erl
+
+get_parent() ->
+ case get('$ancestors') of
+ [Parent | _] when is_pid (Parent) -> Parent;
+ [Parent | _] when is_atom(Parent) -> name_to_pid(Parent);
+ _ -> exit(process_was_not_started_by_proc_lib)
+ end.
+
+name_to_pid(Name) ->
+ case whereis(Name) of
+ undefined -> case whereis_name(Name) of
+ undefined -> exit(could_not_find_registered_name);
+ Pid -> Pid
+ end;
+ Pid -> Pid
+ end.
+
+whereis_name(Name) ->
+ case ets:lookup(global_names, Name) of
+ [{_Name, Pid, _Method, _RPid, _Ref}] ->
+ if node(Pid) == node() -> case erlang:is_process_alive(Pid) of
+ true -> Pid;
+ false -> undefined
+ end;
+ true -> Pid
+ end;
+ [] -> undefined
+ end.
+
+%% End copypasta from gen_server2.erl
+%% -------------------------------------------------------------------------
diff --git a/deps/rabbit_common/src/rabbit_msg_store_index.erl b/deps/rabbit_common/src/rabbit_msg_store_index.erl
new file mode 100644
index 0000000000..ce9abe97a6
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_msg_store_index.erl
@@ -0,0 +1,89 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_msg_store_index).
+
+-include("rabbit_msg_store.hrl").
+
+%% Behaviour module to provide pluggable message store index.
+%% The index is used to locate message on disk and for reference-counting.
+
+%% Message store have several additional assumptions about performance and
+%% atomicity of some operations. See comments for each callback.
+
+-type(dir() :: string()).
+-type(index_state() :: any()).
+-type(fieldpos() :: non_neg_integer()).
+-type(fieldvalue() :: any()).
+-type(msg_location() :: #msg_location{}).
+
+
+%% There are two ways of starting an index:
+%% - `new` - starts a clean index
+%% - `recover` - attempts to read a saved index
+%% In both cases the old saved state should be deleted from directory.
+
+%% Initialize a fresh index state for msg store directory.
+-callback new(dir()) -> index_state().
+%% Try to recover gracefully stopped index state.
+-callback recover(dir()) -> rabbit_types:ok_or_error2(index_state(), any()).
+%% Gracefully shutdown the index.
+%% Should save the index state, which will be loaded by the 'recover' function.
+-callback terminate(index_state()) -> any().
+
+%% Lookup an entry in the index.
+%% Is called concurrently by msg_store, it's clients and GC processes.
+%% This function is called multiple times for each message store operation.
+%% Message store tries to avoid writing messages on disk if consumers can
+%% process them fast, so there will be a lot of lookups for non-existent
+%% entries, which should be as fast as possible.
+-callback lookup(rabbit_types:msg_id(), index_state()) -> ('not_found' | msg_location()).
+
+%% Insert an entry into the index.
+%% Is called by a msg_store process only.
+%% This function can exit if there is already an entry with the same ID
+-callback insert(msg_location(), index_state()) -> 'ok'.
+
+%% Update an entry in the index.
+%% Is called by a msg_store process only.
+%% The function is called during message store recovery after crash.
+%% The difference between update and insert functions, is that update
+%% should not fail if entry already exist, and should be atomic.
+-callback update(msg_location(), index_state()) -> 'ok'.
+
+%% Update positional fields in the entry tuple.
+%% Is called by msg_store and GC processes concurrently.
+%% This function can exit if there is no entry with specified ID
+%% This function is called to update reference-counters and file locations.
+%% File locations are updated from a GC process, reference-counters are
+%% updated from a message store process.
+%% This function should be atomic.
+-callback update_fields(rabbit_types:msg_id(), ({fieldpos(), fieldvalue()} |
+ [{fieldpos(), fieldvalue()}]),
+ index_state()) -> 'ok'.
+
+%% Delete an entry from the index by ID.
+%% Is called from a msg_store process only.
+%% This function should be atomic.
+-callback delete(rabbit_types:msg_id(), index_state()) -> 'ok'.
+
+%% Delete an exactly matching entry from the index.
+%% Is called by GC process only.
+%% This function should match exact object to avoid deleting a zero-reference
+%% object, which reference-counter is being concurrently updated.
+%% This function should be atomic.
+-callback delete_object(msg_location(), index_state()) -> 'ok'.
+
+%% Delete temporary reference count entries with the 'file' record field equal to 'undefined'.
+%% Is called during index rebuild from scratch (e.g. after non-clean stop)
+%% During recovery after non-clean stop or file corruption, reference-counters
+%% are added to the index with `undefined` value for the `file` field.
+%% If message is found in a message store file, it's file field is updated.
+%% If some reference-counters miss the message location after recovery - they
+%% should be deleted.
+-callback clean_up_temporary_reference_count_entries_without_file(index_state()) -> 'ok'.
+
diff --git a/deps/rabbit_common/src/rabbit_net.erl b/deps/rabbit_common/src/rabbit_net.erl
new file mode 100644
index 0000000000..7685687ff0
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_net.erl
@@ -0,0 +1,321 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_net).
+-include("rabbit.hrl").
+
+-include_lib("kernel/include/inet.hrl").
+
+-export([is_ssl/1, ssl_info/1, controlling_process/2, getstat/2,
+ recv/1, sync_recv/2, async_recv/3, port_command/2, getopts/2,
+ setopts/2, send/2, close/1, fast_close/1, sockname/1, peername/1,
+ peercert/1, connection_string/2, socket_ends/2, is_loopback/1,
+ tcp_host/1, unwrap_socket/1, maybe_get_proxy_socket/1,
+ hostname/0, getifaddrs/0]).
+
+%%---------------------------------------------------------------------------
+
+-export_type([socket/0, ip_port/0, hostname/0]).
+
+-type stat_option() ::
+ 'recv_cnt' | 'recv_max' | 'recv_avg' | 'recv_oct' | 'recv_dvi' |
+ 'send_cnt' | 'send_max' | 'send_avg' | 'send_oct' | 'send_pend'.
+-type ok_val_or_error(A) :: rabbit_types:ok_or_error2(A, any()).
+-type ok_or_any_error() :: rabbit_types:ok_or_error(any()).
+-type socket() :: port() | ssl:sslsocket().
+-type opts() :: [{atom(), any()} |
+ {raw, non_neg_integer(), non_neg_integer(), binary()}].
+-type hostname() :: inet:hostname().
+-type ip_port() :: inet:port_number().
+% -type host_or_ip() :: binary() | inet:ip_address().
+-spec is_ssl(socket()) -> boolean().
+-spec ssl_info(socket()) -> 'nossl' | ok_val_or_error([{atom(), any()}]).
+-spec controlling_process(socket(), pid()) -> ok_or_any_error().
+-spec getstat(socket(), [stat_option()]) ->
+ ok_val_or_error([{stat_option(), integer()}]).
+-spec recv(socket()) ->
+ {'data', [char()] | binary()} |
+ 'closed' |
+ rabbit_types:error(any()) |
+ {'other', any()}.
+-spec sync_recv(socket(), integer()) ->
+ rabbit_types:ok(binary()) |
+ rabbit_types:error(any()).
+-spec async_recv(socket(), integer(), timeout()) ->
+ rabbit_types:ok(any()).
+-spec port_command(socket(), iolist()) -> 'true'.
+-spec getopts
+ (socket(),
+ [atom() |
+ {raw, non_neg_integer(), non_neg_integer(),
+ non_neg_integer() | binary()}]) ->
+ ok_val_or_error(opts()).
+-spec setopts(socket(), opts()) -> ok_or_any_error().
+-spec send(socket(), binary() | iolist()) -> ok_or_any_error().
+-spec close(socket()) -> ok_or_any_error().
+-spec fast_close(socket()) -> ok_or_any_error().
+-spec sockname(socket()) ->
+ ok_val_or_error({inet:ip_address(), ip_port()}).
+-spec peername(socket()) ->
+ ok_val_or_error({inet:ip_address(), ip_port()}).
+-spec peercert(socket()) ->
+ 'nossl' | ok_val_or_error(rabbit_ssl:certificate()).
+-spec connection_string(socket(), 'inbound' | 'outbound') ->
+ ok_val_or_error(string()).
+% -spec socket_ends(socket() | ranch_proxy:proxy_socket() | ranch_proxy_ssl:ssl_socket(),
+% 'inbound' | 'outbound') ->
+% ok_val_or_error({host_or_ip(), ip_port(),
+% host_or_ip(), ip_port()}).
+-spec is_loopback(socket() | inet:ip_address()) -> boolean().
+% -spec unwrap_socket(socket() | ranch_proxy:proxy_socket() | ranch_proxy_ssl:ssl_socket()) -> socket().
+
+-dialyzer({nowarn_function, [socket_ends/2, unwrap_socket/1]}).
+
+%%---------------------------------------------------------------------------
+
+-define(SSL_CLOSE_TIMEOUT, 5000).
+
+-define(IS_SSL(Sock), is_tuple(Sock)
+ andalso (tuple_size(Sock) =:= 3)
+ andalso (element(1, Sock) =:= sslsocket)).
+
+is_ssl(Sock) -> ?IS_SSL(Sock).
+
+%% Seems hackish. Is hackish. But the structure is stable and
+%% kept this way for backward compatibility reasons. We need
+%% it for two reasons: there are no ssl:getstat(Sock) function,
+%% and no ssl:close(Timeout) function. Both of them are being
+%% worked on as we speak.
+ssl_get_socket(Sock) ->
+ element(2, element(2, Sock)).
+
+ssl_info(Sock) when ?IS_SSL(Sock) ->
+ ssl:connection_information(Sock);
+ssl_info(_Sock) ->
+ nossl.
+
+controlling_process(Sock, Pid) when ?IS_SSL(Sock) ->
+ ssl:controlling_process(Sock, Pid);
+controlling_process(Sock, Pid) when is_port(Sock) ->
+ gen_tcp:controlling_process(Sock, Pid).
+
+getstat(Sock, Stats) when ?IS_SSL(Sock) ->
+ inet:getstat(ssl_get_socket(Sock), Stats);
+getstat(Sock, Stats) when is_port(Sock) ->
+ inet:getstat(Sock, Stats);
+%% Used by Proxy protocol support in plugins
+getstat({rabbit_proxy_socket, Sock, _}, Stats) when ?IS_SSL(Sock) ->
+ inet:getstat(ssl_get_socket(Sock), Stats);
+getstat({rabbit_proxy_socket, Sock, _}, Stats) when is_port(Sock) ->
+ inet:getstat(Sock, Stats).
+
+recv(Sock) when ?IS_SSL(Sock) ->
+ recv(Sock, {ssl, ssl_closed, ssl_error});
+recv(Sock) when is_port(Sock) ->
+ recv(Sock, {tcp, tcp_closed, tcp_error}).
+
+recv(S, {DataTag, ClosedTag, ErrorTag}) ->
+ receive
+ {DataTag, S, Data} -> {data, Data};
+ {ClosedTag, S} -> closed;
+ {ErrorTag, S, Reason} -> {error, Reason};
+ Other -> {other, Other}
+ end.
+
+sync_recv(Sock, Length) when ?IS_SSL(Sock) ->
+ ssl:recv(Sock, Length);
+sync_recv(Sock, Length) ->
+ gen_tcp:recv(Sock, Length).
+
+async_recv(Sock, Length, Timeout) when ?IS_SSL(Sock) ->
+ Pid = self(),
+ Ref = make_ref(),
+
+ spawn(fun () -> Pid ! {inet_async, Sock, Ref,
+ ssl:recv(Sock, Length, Timeout)}
+ end),
+
+ {ok, Ref};
+async_recv(Sock, Length, infinity) when is_port(Sock) ->
+ prim_inet:async_recv(Sock, Length, -1);
+async_recv(Sock, Length, Timeout) when is_port(Sock) ->
+ prim_inet:async_recv(Sock, Length, Timeout).
+
+port_command(Sock, Data) when ?IS_SSL(Sock) ->
+ case ssl:send(Sock, Data) of
+ ok -> self() ! {inet_reply, Sock, ok},
+ true;
+ {error, Reason} -> erlang:error(Reason)
+ end;
+port_command(Sock, Data) when is_port(Sock) ->
+ erlang:port_command(Sock, Data).
+
+getopts(Sock, Options) when ?IS_SSL(Sock) ->
+ ssl:getopts(Sock, Options);
+getopts(Sock, Options) when is_port(Sock) ->
+ inet:getopts(Sock, Options).
+
+setopts(Sock, Options) when ?IS_SSL(Sock) ->
+ ssl:setopts(Sock, Options);
+setopts(Sock, Options) when is_port(Sock) ->
+ inet:setopts(Sock, Options).
+
+send(Sock, Data) when ?IS_SSL(Sock) -> ssl:send(Sock, Data);
+send(Sock, Data) when is_port(Sock) -> gen_tcp:send(Sock, Data).
+
+close(Sock) when ?IS_SSL(Sock) -> ssl:close(Sock);
+close(Sock) when is_port(Sock) -> gen_tcp:close(Sock).
+
+fast_close(Sock) when ?IS_SSL(Sock) ->
+ %% We cannot simply port_close the underlying tcp socket since the
+ %% TLS protocol is quite insistent that a proper closing handshake
+ %% should take place (see RFC 5245 s7.2.1). So we call ssl:close
+ %% instead, but that can block for a very long time, e.g. when
+ %% there is lots of pending output and there is tcp backpressure,
+ %% or the ssl_connection process has entered the the
+ %% workaround_transport_delivery_problems function during
+ %% termination, which, inexplicably, does a gen_tcp:recv(Socket,
+ %% 0), which may never return if the client doesn't send a FIN or
+ %% that gets swallowed by the network. Since there is no timeout
+ %% variant of ssl:close, we construct our own.
+ {Pid, MRef} = spawn_monitor(fun () -> ssl:close(Sock) end),
+ erlang:send_after(?SSL_CLOSE_TIMEOUT, self(), {Pid, ssl_close_timeout}),
+ receive
+ {Pid, ssl_close_timeout} ->
+ erlang:demonitor(MRef, [flush]),
+ exit(Pid, kill);
+ {'DOWN', MRef, process, Pid, _Reason} ->
+ ok
+ end,
+ catch port_close(ssl_get_socket(Sock)),
+ ok;
+fast_close(Sock) when is_port(Sock) ->
+ catch port_close(Sock), ok.
+
+sockname(Sock) when ?IS_SSL(Sock) -> ssl:sockname(Sock);
+sockname(Sock) when is_port(Sock) -> inet:sockname(Sock).
+
+peername(Sock) when ?IS_SSL(Sock) -> ssl:peername(Sock);
+peername(Sock) when is_port(Sock) -> inet:peername(Sock).
+
+peercert(Sock) when ?IS_SSL(Sock) -> ssl:peercert(Sock);
+peercert(Sock) when is_port(Sock) -> nossl.
+
+connection_string(Sock, Direction) ->
+ case socket_ends(Sock, Direction) of
+ {ok, {FromAddress, FromPort, ToAddress, ToPort}} ->
+ {ok, rabbit_misc:format(
+ "~s:~p -> ~s:~p",
+ [maybe_ntoab(FromAddress), FromPort,
+ maybe_ntoab(ToAddress), ToPort])};
+ Error ->
+ Error
+ end.
+
+socket_ends(Sock, Direction) when ?IS_SSL(Sock);
+ is_port(Sock) ->
+ {From, To} = sock_funs(Direction),
+ case {From(Sock), To(Sock)} of
+ {{ok, {FromAddress, FromPort}}, {ok, {ToAddress, ToPort}}} ->
+ {ok, {rdns(FromAddress), FromPort,
+ rdns(ToAddress), ToPort}};
+ {{error, _Reason} = Error, _} ->
+ Error;
+ {_, {error, _Reason} = Error} ->
+ Error
+ end;
+socket_ends({rabbit_proxy_socket, CSocket, ProxyInfo}, Direction = inbound) ->
+ #{
+ src_address := FromAddress,
+ src_port := FromPort
+ } = ProxyInfo,
+ {_From, To} = sock_funs(Direction),
+ case To(CSocket) of
+ {ok, {ToAddress, ToPort}} ->
+ {ok, {rdns(FromAddress), FromPort,
+ rdns(ToAddress), ToPort}};
+ {error, _Reason} = Error ->
+ Error
+ end.
+
+maybe_ntoab(Addr) when is_tuple(Addr) -> rabbit_misc:ntoab(Addr);
+maybe_ntoab(Host) -> Host.
+
+tcp_host({0,0,0,0}) ->
+ hostname();
+
+tcp_host({0,0,0,0,0,0,0,0}) ->
+ hostname();
+
+tcp_host(IPAddress) ->
+ case inet:gethostbyaddr(IPAddress) of
+ {ok, #hostent{h_name = Name}} -> Name;
+ {error, _Reason} -> rabbit_misc:ntoa(IPAddress)
+ end.
+
+hostname() ->
+ {ok, Hostname} = inet:gethostname(),
+ case inet:gethostbyname(Hostname) of
+ {ok, #hostent{h_name = Name}} -> Name;
+ {error, _Reason} -> Hostname
+ end.
+
+format_nic_attribute({Key, undefined}) ->
+ {Key, undefined};
+format_nic_attribute({Key = flags, List}) when is_list(List) ->
+ Val = string:join(lists:map(fun rabbit_data_coercion:to_list/1, List), ", "),
+ {Key, rabbit_data_coercion:to_binary(Val)};
+format_nic_attribute({Key, Tuple}) when is_tuple(Tuple) and (Key =:= addr orelse
+ Key =:= broadaddr orelse
+ Key =:= netmask orelse
+ Key =:= dstaddr) ->
+ Val = inet_parse:ntoa(Tuple),
+ {Key, rabbit_data_coercion:to_binary(Val)};
+format_nic_attribute({Key = hwaddr, List}) when is_list(List) ->
+ %% [140, 133, 144, 28, 241, 121] => 8C:85:90:1C:F1:79
+ Val = string:join(lists:map(fun(N) -> integer_to_list(N, 16) end, List), ":"),
+ {Key, rabbit_data_coercion:to_binary(Val)}.
+
+getifaddrs() ->
+ {ok, AddrList} = inet:getifaddrs(),
+ Addrs0 = maps:from_list(AddrList),
+ maps:map(fun (_Key, Proplist) ->
+ lists:map(fun format_nic_attribute/1, Proplist)
+ end, Addrs0).
+
+rdns(Addr) ->
+ case application:get_env(rabbit, reverse_dns_lookups) of
+ {ok, true} -> list_to_binary(tcp_host(Addr));
+ _ -> Addr
+ end.
+
+sock_funs(inbound) -> {fun peername/1, fun sockname/1};
+sock_funs(outbound) -> {fun sockname/1, fun peername/1}.
+
+is_loopback(Sock) when is_port(Sock) ; ?IS_SSL(Sock) ->
+ case sockname(Sock) of
+ {ok, {Addr, _Port}} -> is_loopback(Addr);
+ {error, _} -> false
+ end;
+%% We could parse the results of inet:getifaddrs() instead. But that
+%% would be more complex and less maybe Windows-compatible...
+is_loopback({127,_,_,_}) -> true;
+is_loopback({0,0,0,0,0,0,0,1}) -> true;
+is_loopback({0,0,0,0,0,65535,AB,CD}) -> is_loopback(ipv4(AB, CD));
+is_loopback(_) -> false.
+
+ipv4(AB, CD) -> {AB bsr 8, AB band 255, CD bsr 8, CD band 255}.
+
+unwrap_socket({rabbit_proxy_socket, Sock, _}) ->
+ Sock;
+unwrap_socket(Sock) ->
+ Sock.
+
+maybe_get_proxy_socket(Sock={rabbit_proxy_socket, _, _}) ->
+ Sock;
+maybe_get_proxy_socket(_Sock) ->
+ undefined.
diff --git a/deps/rabbit_common/src/rabbit_nodes_common.erl b/deps/rabbit_common/src/rabbit_nodes_common.erl
new file mode 100644
index 0000000000..7e87ce2ea4
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_nodes_common.erl
@@ -0,0 +1,227 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_nodes_common).
+
+-define(EPMD_TIMEOUT, 30000).
+-define(TCP_DIAGNOSTIC_TIMEOUT, 5000).
+-define(ERROR_LOGGER_HANDLER, rabbit_error_logger_handler).
+
+-include_lib("kernel/include/inet.hrl").
+
+%%
+%% API
+%%
+
+-export([make/1, parts/1, names/1, name_type/1, ensure_epmd/0, is_running/2, is_process_running/2]).
+-export([cookie_hash/0, epmd_port/0, diagnostics/1]).
+
+-spec make({string(), string()} | string()) -> node().
+-spec parts(node() | string()) -> {string(), string()}.
+-spec ensure_epmd() -> 'ok'.
+-spec epmd_port() -> string().
+
+-spec names(string()) ->
+ rabbit_types:ok_or_error2([{string(), integer()}], term()).
+-spec diagnostics([node()]) -> string().
+-spec cookie_hash() -> string().
+
+%% net_adm:name/1 returns a new value, 'noport', in Erlang 24. This value being
+%% absent in the function spec in previous versions of Erlang, we get a warning
+%% from Dialyzer until we start to the yet-to-be-release Erlang 24 in CI.
+%% Therefore we disable this specific warning.
+-dialyzer({nowarn_function, diagnostics_node/1}).
+
+names(Hostname) ->
+ Self = self(),
+ Ref = make_ref(),
+ {Pid, MRef} = spawn_monitor(
+ fun () -> Self ! {Ref, net_adm:names(Hostname)} end),
+ _ = timer:exit_after(?EPMD_TIMEOUT, Pid, timeout),
+ receive
+ {Ref, Names} -> erlang:demonitor(MRef, [flush]),
+ Names;
+ {'DOWN', MRef, process, Pid, Reason} -> {error, Reason}
+ end.
+
+make({Prefix, Suffix}) -> rabbit_data_coercion:to_atom(
+ lists:append([rabbit_data_coercion:to_list(Prefix),
+ "@",
+ rabbit_data_coercion:to_list(Suffix)]));
+make(NodeStr) -> make(parts(NodeStr)).
+
+parts(Node) when is_atom(Node) ->
+ parts(atom_to_list(Node));
+parts(NodeStr) ->
+ case lists:splitwith(fun (E) -> E =/= $@ end, NodeStr) of
+ {Prefix, []} -> {_, Suffix} = parts(node()),
+ {Prefix, Suffix};
+ {Prefix, Suffix} -> {Prefix, tl(Suffix)}
+ end.
+
+name_type(Node) ->
+ {_, HostPart} = parts(Node),
+ case lists:member($., HostPart) of
+ false -> shortnames;
+ true -> longnames
+ end.
+
+epmd_port() ->
+ case init:get_argument(epmd_port) of
+ {ok, [[Port | _] | _]} when is_list(Port) -> Port;
+ error -> "4369"
+ end.
+
+ensure_epmd() ->
+ Exe = rabbit_runtime:get_erl_path(),
+ ID = rabbit_misc:random(1000000000),
+ Port = open_port(
+ {spawn_executable, Exe},
+ [{args, ["-boot", "no_dot_erlang",
+ "-sname", rabbit_misc:format("epmd-starter-~b", [ID]),
+ "-noinput", "-s", "erlang", "halt"]},
+ exit_status, stderr_to_stdout, use_stdio]),
+ port_shutdown_loop(Port).
+
+port_shutdown_loop(Port) ->
+ receive
+ {Port, {exit_status, _Rc}} -> ok;
+ {Port, _} -> port_shutdown_loop(Port)
+ end.
+
+cookie_hash() ->
+ base64:encode_to_string(erlang:md5(atom_to_list(erlang:get_cookie()))).
+
+diagnostics(Nodes) ->
+ verbose_erlang_distribution(true),
+ NodeDiags = [{"~nDIAGNOSTICS~n===========~n~n"
+ "attempted to contact: ~p~n", [Nodes]}] ++
+ [diagnostics_node(Node) || Node <- Nodes] ++
+ current_node_details(),
+ verbose_erlang_distribution(false),
+ rabbit_misc:format_many(lists:flatten(NodeDiags)).
+
+verbose_erlang_distribution(true) ->
+ net_kernel:verbose(1),
+ error_logger:add_report_handler(?ERROR_LOGGER_HANDLER);
+verbose_erlang_distribution(false) ->
+ net_kernel:verbose(0),
+ error_logger:delete_report_handler(?ERROR_LOGGER_HANDLER).
+
+current_node_details() ->
+ [{"~nCurrent node details:~n * node name: ~w", [node()]},
+ case init:get_argument(home) of
+ {ok, [[Home]]} -> {" * effective user's home directory: ~s", [Home]};
+ Other -> {" * effective user has no home directory: ~p", [Other]}
+ end,
+ {" * Erlang cookie hash: ~s", [cookie_hash()]}].
+
+diagnostics_node(Node) ->
+ {Name, Host} = parts(Node),
+ [{"~s:", [Node]} |
+ case names(Host) of
+ {error, Reason} ->
+ [{" * unable to connect to epmd (port ~s) on ~s: ~s~n",
+ [epmd_port(), Host, rabbit_misc:format_inet_error(Reason)]}];
+ noport ->
+ [{" * unable to connect to epmd (port ~s) on ~s: "
+ "couldn't resolve hostname~n",
+ [epmd_port(), Host]}];
+ {ok, NamePorts} ->
+ [{" * connected to epmd (port ~s) on ~s",
+ [epmd_port(), Host]}] ++
+ case net_adm:ping(Node) of
+ pong -> dist_working_diagnostics(Node);
+ pang -> dist_broken_diagnostics(Name, Host, NamePorts)
+ end
+ end].
+
+dist_working_diagnostics(Node) ->
+ case is_process_running(Node, rabbit) of
+ true -> [{" * node ~s up, 'rabbit' application running", [Node]}];
+ false -> [{" * node ~s up, 'rabbit' application not running~n"
+ " * running applications on ~s: ~p~n"
+ " * suggestion: use rabbitmqctl start_app on ~s",
+ [Node, Node, remote_apps(Node), Node]}]
+ end.
+
+remote_apps(Node) ->
+ %% We want a timeout here because really, we don't trust the node,
+ %% the last thing we want to do is hang.
+ case rpc:call(Node, application, which_applications, [5000]) of
+ {badrpc, _} = E -> E;
+ Apps -> [App || {App, _, _} <- Apps]
+ end.
+
+dist_broken_diagnostics(Name, Host, NamePorts) ->
+ case [{N, P} || {N, P} <- NamePorts, N =:= Name] of
+ [] ->
+ {SelfName, SelfHost} = parts(node()),
+ Others = [list_to_atom(N) || {N, _} <- NamePorts,
+ N =/= case SelfHost of
+ Host -> SelfName;
+ _ -> never_matches
+ end],
+ OthersDiag = case Others of
+ [] -> [{" no other nodes on ~s",
+ [Host]}];
+ _ -> [{" other nodes on ~s: ~p",
+ [Host, Others]}]
+ end,
+ [{" * epmd reports: node '~s' not running at all", [Name]},
+ OthersDiag, {" * suggestion: start the node", []}];
+ [{Name, Port}] ->
+ [{" * epmd reports node '~s' uses port ~b for inter-node and CLI tool traffic ", [Name, Port]} |
+ case diagnose_connect(Host, Port) of
+ ok ->
+ connection_succeeded_diagnostics();
+ {error, Reason} ->
+ [{" * can't establish TCP connection to the target node, reason: ~s~n"
+ " * suggestion: check if host '~s' resolves, is reachable and ports ~b, 4369 are not blocked by firewall",
+ [rabbit_misc:format_inet_error(Reason), Host, Port]}]
+ end]
+ end.
+
+connection_succeeded_diagnostics() ->
+ case gen_event:call(error_logger, ?ERROR_LOGGER_HANDLER, get_connection_report) of
+ [] ->
+ [{" * TCP connection succeeded but Erlang distribution failed ~n"
+ " * suggestion: check if the Erlang cookie identical for all server nodes and CLI tools~n"
+ " * suggestion: check if all server nodes and CLI tools use consistent hostnames when addressing each other~n"
+ " * suggestion: check if inter-node connections may be configured to use TLS. If so, all nodes and CLI tools must do that~n"
+ " * suggestion: see the CLI, clustering and networking guides on https://rabbitmq.com/documentation.html to learn more~n", []}];
+ Report ->
+ [{" * TCP connection succeeded but Erlang distribution "
+ "failed ~n", []}]
+ ++ Report
+ end.
+
+diagnose_connect(Host, Port) ->
+ case inet:gethostbyname(Host) of
+ {ok, #hostent{h_addrtype = Family}} ->
+ case gen_tcp:connect(Host, Port, [Family],
+ ?TCP_DIAGNOSTIC_TIMEOUT) of
+ {ok, Socket} -> gen_tcp:close(Socket),
+ ok;
+ {error, _} = E -> E
+ end;
+ {error, _} = E ->
+ E
+ end.
+
+is_running(Node, Application) ->
+ case rpc:call(Node, rabbit_misc, which_applications, []) of
+ {badrpc, _} -> false;
+ Apps -> proplists:is_defined(Application, Apps)
+ end.
+
+is_process_running(Node, Process) ->
+ case rpc:call(Node, erlang, whereis, [Process]) of
+ {badrpc, _} -> false;
+ undefined -> false;
+ P when is_pid(P) -> true
+ end.
diff --git a/deps/rabbit_common/src/rabbit_numerical.erl b/deps/rabbit_common/src/rabbit_numerical.erl
new file mode 100644
index 0000000000..45cc67fda6
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_numerical.erl
@@ -0,0 +1,358 @@
+%% This file is a copy of `mochijson2.erl' from mochiweb, revision
+%% d541e9a0f36c00dcadc2e589f20e47fbf46fc76f. For the license, see
+%% `LICENSE-MIT-Mochi'.
+
+%% @copyright 2007 Mochi Media, Inc.
+%% @author Bob Ippolito <bob@mochimedia.com>
+
+%% @doc Useful numeric algorithms for floats that cover some deficiencies
+%% in the math module. More interesting is digits/1, which implements
+%% the algorithm from:
+%% https://cs.indiana.edu/~burger/fp/index.html
+%% See also "Printing Floating-Point Numbers Quickly and Accurately"
+%% in Proceedings of the SIGPLAN '96 Conference on Programming Language
+%% Design and Implementation.
+
+-module(rabbit_numerical).
+-author("Bob Ippolito <bob@mochimedia.com>").
+-export([digits/1, frexp/1, int_pow/2, int_ceil/1]).
+
+%% IEEE 754 Float exponent bias
+-define(FLOAT_BIAS, 1022).
+-define(MIN_EXP, -1074).
+-define(BIG_POW, 4503599627370496).
+
+%% External API
+
+%% @spec digits(number()) -> string()
+%% @doc Returns a string that accurately represents the given integer or float
+%% using a conservative amount of digits. Great for generating
+%% human-readable output, or compact ASCII serializations for floats.
+digits(N) when is_integer(N) ->
+ integer_to_list(N);
+digits(0.0) ->
+ "0.0";
+digits(Float) ->
+ {Frac1, Exp1} = frexp_int(Float),
+ [Place0 | Digits0] = digits1(Float, Exp1, Frac1),
+ {Place, Digits} = transform_digits(Place0, Digits0),
+ R = insert_decimal(Place, Digits),
+ case Float < 0 of
+ true ->
+ [$- | R];
+ _ ->
+ R
+ end.
+
+%% @spec frexp(F::float()) -> {Frac::float(), Exp::float()}
+%% @doc Return the fractional and exponent part of an IEEE 754 double,
+%% equivalent to the libc function of the same name.
+%% F = Frac * pow(2, Exp).
+frexp(F) ->
+ frexp1(unpack(F)).
+
+%% @spec int_pow(X::integer(), N::integer()) -> Y::integer()
+%% @doc Moderately efficient way to exponentiate integers.
+%% int_pow(10, 2) = 100.
+int_pow(_X, 0) ->
+ 1;
+int_pow(X, N) when N > 0 ->
+ int_pow(X, N, 1).
+
+%% @spec int_ceil(F::float()) -> integer()
+%% @doc Return the ceiling of F as an integer. The ceiling is defined as
+%% F when F == trunc(F);
+%% trunc(F) when F &lt; 0;
+%% trunc(F) + 1 when F &gt; 0.
+int_ceil(X) ->
+ T = trunc(X),
+ case (X - T) of
+ Pos when Pos > 0 -> T + 1;
+ _ -> T
+ end.
+
+
+%% Internal API
+
+int_pow(X, N, R) when N < 2 ->
+ R * X;
+int_pow(X, N, R) ->
+ int_pow(X * X, N bsr 1, case N band 1 of 1 -> R * X; 0 -> R end).
+
+insert_decimal(0, S) ->
+ "0." ++ S;
+insert_decimal(Place, S) when Place > 0 ->
+ L = length(S),
+ case Place - L of
+ 0 ->
+ S ++ ".0";
+ N when N < 0 ->
+ {S0, S1} = lists:split(L + N, S),
+ S0 ++ "." ++ S1;
+ N when N < 6 ->
+ %% More places than digits
+ S ++ lists:duplicate(N, $0) ++ ".0";
+ _ ->
+ insert_decimal_exp(Place, S)
+ end;
+insert_decimal(Place, S) when Place > -6 ->
+ "0." ++ lists:duplicate(abs(Place), $0) ++ S;
+insert_decimal(Place, S) ->
+ insert_decimal_exp(Place, S).
+
+insert_decimal_exp(Place, S) ->
+ [C | S0] = S,
+ S1 = case S0 of
+ [] ->
+ "0";
+ _ ->
+ S0
+ end,
+ Exp = case Place < 0 of
+ true ->
+ "e-";
+ false ->
+ "e+"
+ end,
+ [C] ++ "." ++ S1 ++ Exp ++ integer_to_list(abs(Place - 1)).
+
+
+digits1(Float, Exp, Frac) ->
+ Round = ((Frac band 1) =:= 0),
+ case Exp >= 0 of
+ true ->
+ BExp = 1 bsl Exp,
+ case (Frac =/= ?BIG_POW) of
+ true ->
+ scale((Frac * BExp * 2), 2, BExp, BExp,
+ Round, Round, Float);
+ false ->
+ scale((Frac * BExp * 4), 4, (BExp * 2), BExp,
+ Round, Round, Float)
+ end;
+ false ->
+ case (Exp =:= ?MIN_EXP) orelse (Frac =/= ?BIG_POW) of
+ true ->
+ scale((Frac * 2), 1 bsl (1 - Exp), 1, 1,
+ Round, Round, Float);
+ false ->
+ scale((Frac * 4), 1 bsl (2 - Exp), 2, 1,
+ Round, Round, Float)
+ end
+ end.
+
+scale(R, S, MPlus, MMinus, LowOk, HighOk, Float) ->
+ Est = int_ceil(math:log10(abs(Float)) - 1.0e-10),
+ %% Note that the scheme implementation uses a 326 element look-up table
+ %% for int_pow(10, N) where we do not.
+ case Est >= 0 of
+ true ->
+ fixup(R, S * int_pow(10, Est), MPlus, MMinus, Est,
+ LowOk, HighOk);
+ false ->
+ Scale = int_pow(10, -Est),
+ fixup(R * Scale, S, MPlus * Scale, MMinus * Scale, Est,
+ LowOk, HighOk)
+ end.
+
+fixup(R, S, MPlus, MMinus, K, LowOk, HighOk) ->
+ TooLow = case HighOk of
+ true ->
+ (R + MPlus) >= S;
+ false ->
+ (R + MPlus) > S
+ end,
+ case TooLow of
+ true ->
+ [(K + 1) | generate(R, S, MPlus, MMinus, LowOk, HighOk)];
+ false ->
+ [K | generate(R * 10, S, MPlus * 10, MMinus * 10, LowOk, HighOk)]
+ end.
+
+generate(R0, S, MPlus, MMinus, LowOk, HighOk) ->
+ D = R0 div S,
+ R = R0 rem S,
+ TC1 = case LowOk of
+ true ->
+ R =< MMinus;
+ false ->
+ R < MMinus
+ end,
+ TC2 = case HighOk of
+ true ->
+ (R + MPlus) >= S;
+ false ->
+ (R + MPlus) > S
+ end,
+ case TC1 of
+ false ->
+ case TC2 of
+ false ->
+ [D | generate(R * 10, S, MPlus * 10, MMinus * 10,
+ LowOk, HighOk)];
+ true ->
+ [D + 1]
+ end;
+ true ->
+ case TC2 of
+ false ->
+ [D];
+ true ->
+ case R * 2 < S of
+ true ->
+ [D];
+ false ->
+ [D + 1]
+ end
+ end
+ end.
+
+unpack(Float) ->
+ <<Sign:1, Exp:11, Frac:52>> = <<Float:64/float>>,
+ {Sign, Exp, Frac}.
+
+frexp1({_Sign, 0, 0}) ->
+ {0.0, 0};
+frexp1({Sign, 0, Frac}) ->
+ Exp = log2floor(Frac),
+ <<Frac1:64/float>> = <<Sign:1, ?FLOAT_BIAS:11, (Frac-1):52>>,
+ {Frac1, -(?FLOAT_BIAS) - 52 + Exp};
+frexp1({Sign, Exp, Frac}) ->
+ <<Frac1:64/float>> = <<Sign:1, ?FLOAT_BIAS:11, Frac:52>>,
+ {Frac1, Exp - ?FLOAT_BIAS}.
+
+log2floor(Int) ->
+ log2floor(Int, 0).
+
+log2floor(0, N) ->
+ N;
+log2floor(Int, N) ->
+ log2floor(Int bsr 1, 1 + N).
+
+
+transform_digits(Place, [0 | Rest]) ->
+ transform_digits(Place, Rest);
+transform_digits(Place, Digits) ->
+ {Place, [$0 + D || D <- Digits]}.
+
+
+frexp_int(F) ->
+ case unpack(F) of
+ {_Sign, 0, Frac} ->
+ {Frac, ?MIN_EXP};
+ {_Sign, Exp, Frac} ->
+ {Frac + (1 bsl 52), Exp - 53 - ?FLOAT_BIAS}
+ end.
+
+%%
+%% Tests
+%%
+-ifdef(TEST).
+-include_lib("eunit/include/eunit.hrl").
+
+int_ceil_test() ->
+ ?assertEqual(1, int_ceil(0.0001)),
+ ?assertEqual(0, int_ceil(0.0)),
+ ?assertEqual(1, int_ceil(0.99)),
+ ?assertEqual(1, int_ceil(1.0)),
+ ?assertEqual(-1, int_ceil(-1.5)),
+ ?assertEqual(-2, int_ceil(-2.0)),
+ ok.
+
+int_pow_test() ->
+ ?assertEqual(1, int_pow(1, 1)),
+ ?assertEqual(1, int_pow(1, 0)),
+ ?assertEqual(1, int_pow(10, 0)),
+ ?assertEqual(10, int_pow(10, 1)),
+ ?assertEqual(100, int_pow(10, 2)),
+ ?assertEqual(1000, int_pow(10, 3)),
+ ok.
+
+digits_test() ->
+ ?assertEqual("0",
+ digits(0)),
+ ?assertEqual("0.0",
+ digits(0.0)),
+ ?assertEqual("1.0",
+ digits(1.0)),
+ ?assertEqual("-1.0",
+ digits(-1.0)),
+ ?assertEqual("0.1",
+ digits(0.1)),
+ ?assertEqual("0.01",
+ digits(0.01)),
+ ?assertEqual("0.001",
+ digits(0.001)),
+ ?assertEqual("1.0e+6",
+ digits(1000000.0)),
+ ?assertEqual("0.5",
+ digits(0.5)),
+ ?assertEqual("4503599627370496.0",
+ digits(4503599627370496.0)),
+ %% small denormalized number
+ %% 4.94065645841246544177e-324 =:= 5.0e-324
+ <<SmallDenorm/float>> = <<0,0,0,0,0,0,0,1>>,
+ ?assertEqual("5.0e-324",
+ digits(SmallDenorm)),
+ ?assertEqual(SmallDenorm,
+ list_to_float(digits(SmallDenorm))),
+ %% large denormalized number
+ %% 2.22507385850720088902e-308
+ <<BigDenorm/float>> = <<0,15,255,255,255,255,255,255>>,
+ ?assertEqual("2.225073858507201e-308",
+ digits(BigDenorm)),
+ ?assertEqual(BigDenorm,
+ list_to_float(digits(BigDenorm))),
+ %% small normalized number
+ %% 2.22507385850720138309e-308
+ <<SmallNorm/float>> = <<0,16,0,0,0,0,0,0>>,
+ ?assertEqual("2.2250738585072014e-308",
+ digits(SmallNorm)),
+ ?assertEqual(SmallNorm,
+ list_to_float(digits(SmallNorm))),
+ %% large normalized number
+ %% 1.79769313486231570815e+308
+ <<LargeNorm/float>> = <<127,239,255,255,255,255,255,255>>,
+ ?assertEqual("1.7976931348623157e+308",
+ digits(LargeNorm)),
+ ?assertEqual(LargeNorm,
+ list_to_float(digits(LargeNorm))),
+ %% issue #10 - mochinum:frexp(math:pow(2, -1074)).
+ ?assertEqual("5.0e-324",
+ digits(math:pow(2, -1074))),
+ ok.
+
+frexp_test() ->
+ %% zero
+ ?assertEqual({0.0, 0}, frexp(0.0)),
+ %% one
+ ?assertEqual({0.5, 1}, frexp(1.0)),
+ %% negative one
+ ?assertEqual({-0.5, 1}, frexp(-1.0)),
+ %% small denormalized number
+ %% 4.94065645841246544177e-324
+ <<SmallDenorm/float>> = <<0,0,0,0,0,0,0,1>>,
+ ?assertEqual({0.5, -1073}, frexp(SmallDenorm)),
+ %% large denormalized number
+ %% 2.22507385850720088902e-308
+ <<BigDenorm/float>> = <<0,15,255,255,255,255,255,255>>,
+ ?assertEqual(
+ {0.99999999999999978, -1022},
+ frexp(BigDenorm)),
+ %% small normalized number
+ %% 2.22507385850720138309e-308
+ <<SmallNorm/float>> = <<0,16,0,0,0,0,0,0>>,
+ ?assertEqual({0.5, -1021}, frexp(SmallNorm)),
+ %% large normalized number
+ %% 1.79769313486231570815e+308
+ <<LargeNorm/float>> = <<127,239,255,255,255,255,255,255>>,
+ ?assertEqual(
+ {0.99999999999999989, 1024},
+ frexp(LargeNorm)),
+ %% issue #10 - mochinum:frexp(math:pow(2, -1074)).
+ ?assertEqual(
+ {0.5, -1073},
+ frexp(math:pow(2, -1074))),
+ ok.
+
+-endif.
diff --git a/deps/rabbit_common/src/rabbit_password_hashing.erl b/deps/rabbit_common/src/rabbit_password_hashing.erl
new file mode 100644
index 0000000000..53d4d04e10
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_password_hashing.erl
@@ -0,0 +1,11 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_password_hashing).
+-include("rabbit.hrl").
+
+-callback hash(rabbit_types:password()) -> rabbit_types:password_hash().
diff --git a/deps/rabbit_common/src/rabbit_pbe.erl b/deps/rabbit_common/src/rabbit_pbe.erl
new file mode 100644
index 0000000000..d999d520a4
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_pbe.erl
@@ -0,0 +1,54 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_pbe).
+
+-export([supported_ciphers/0, supported_hashes/0, default_cipher/0, default_hash/0, default_iterations/0]).
+-export([encrypt_term/5, decrypt_term/5]).
+-export([encrypt/5, decrypt/5]).
+
+-export_type([encryption_result/0]).
+
+supported_ciphers() ->
+ credentials_obfuscation_pbe:supported_ciphers().
+
+supported_hashes() ->
+ credentials_obfuscation_pbe:supported_hashes().
+
+%% Default encryption parameters.
+default_cipher() ->
+ credentials_obfuscation_pbe:default_cipher().
+
+default_hash() ->
+ credentials_obfuscation_pbe:default_hash().
+
+default_iterations() ->
+ credentials_obfuscation_pbe:default_iterations().
+
+%% Encryption/decryption of arbitrary Erlang terms.
+
+encrypt_term(Cipher, Hash, Iterations, PassPhrase, Term) ->
+ credentials_obfuscation_pbe:encrypt_term(Cipher, Hash, Iterations, PassPhrase, Term).
+
+decrypt_term(_Cipher, _Hash, _Iterations, _PassPhrase, {plaintext, Term}) ->
+ Term;
+decrypt_term(Cipher, Hash, Iterations, PassPhrase, {encrypted, _Base64Binary}=Encrypted) ->
+ credentials_obfuscation_pbe:decrypt_term(Cipher, Hash, Iterations, PassPhrase, Encrypted).
+
+-type encryption_result() :: {'encrypted', binary()} | {'plaintext', binary()}.
+
+-spec encrypt(crypto:block_cipher(), crypto:hash_algorithms(),
+ pos_integer(), iodata() | '$pending-secret', binary()) -> encryption_result().
+encrypt(Cipher, Hash, Iterations, PassPhrase, ClearText) ->
+ credentials_obfuscation_pbe:encrypt(Cipher, Hash, Iterations, PassPhrase, ClearText).
+
+-spec decrypt(crypto:block_cipher(), crypto:hash_algorithms(),
+ pos_integer(), iodata(), encryption_result()) -> any().
+decrypt(_Cipher, _Hash, _Iterations, _PassPhrase, {plaintext, Term}) ->
+ Term;
+decrypt(Cipher, Hash, Iterations, PassPhrase, {encrypted, _Base64Binary}=Encrypted) ->
+ credentials_obfuscation_pbe:decrypt(Cipher, Hash, Iterations, PassPhrase, Encrypted).
diff --git a/deps/rabbit_common/src/rabbit_peer_discovery_backend.erl b/deps/rabbit_common/src/rabbit_peer_discovery_backend.erl
new file mode 100644
index 0000000000..af3683e72b
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_peer_discovery_backend.erl
@@ -0,0 +1,59 @@
+%% This module is based on the autocluster_backend module
+%% from rabbitmq-autocluster by Gavin Roy.
+%%
+%% Copyright (c) 2014-2015 AWeber Communications
+%% Copyright (c) 2016-2020 VMware, Inc. or its affiliates
+%% All rights reserved.
+%%
+%% Redistribution and use in source and binary forms, with or without modification,
+%% are permitted provided that the following conditions are met:
+%%
+%% * Redistributions of source code must retain the above copyright notice, this
+%% list of conditions and the following disclaimer.
+%% * Redistributions in binary form must reproduce the above copyright notice,
+%% this list of conditions and the following disclaimer in the documentation
+%% and/or other materials provided with the distribution.
+%% * Neither the name of the project nor the names of its
+%% contributors may be used to endorse or promote products derived from this
+%% software without specific prior written permission.
+%%
+%% THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+%% ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+%% WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED.
+%% IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT,
+%% INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+%% BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+%% DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF
+%% LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE
+%% OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF
+%% ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+%%
+%% The Original Code is rabbitmq-autocluster.
+%%
+%% The Initial Developer of the Original Code is AWeber Communications.
+%% Copyright (c) 2014-2015 AWeber Communications
+%% Copyright (c) 2016-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_peer_discovery_backend).
+
+-include("rabbit.hrl").
+
+-callback init() -> ok | {error, Reason :: string()}.
+
+-callback list_nodes() -> {ok, {Nodes :: list(), NodeType :: rabbit_types:node_type()}} |
+ {error, Reason :: string()}.
+
+-callback supports_registration() -> boolean().
+
+-callback register() -> ok | {error, Reason :: string()}.
+
+-callback unregister() -> ok | {error, Reason :: string()}.
+
+-callback post_registration() -> ok | {error, Reason :: string()}.
+
+-callback lock(Node :: atom()) -> {ok, Data :: term()} | not_supported | {error, Reason :: string()}.
+
+-callback unlock(Data :: term()) -> ok | {error, Reason :: string()}.
+
+-optional_callbacks([init/0]).
diff --git a/deps/rabbit_common/src/rabbit_policy_validator.erl b/deps/rabbit_common/src/rabbit_policy_validator.erl
new file mode 100644
index 0000000000..32b7a44fd9
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_policy_validator.erl
@@ -0,0 +1,22 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_policy_validator).
+
+-behaviour(rabbit_registry_class).
+
+-export([added_to_rabbit_registry/2, removed_from_rabbit_registry/1]).
+
+-export_type([validate_results/0]).
+
+-type(validate_results() ::
+ 'ok' | {error, string(), [term()]} | [validate_results()]).
+
+-callback validate_policy([{binary(), term()}]) -> validate_results().
+
+added_to_rabbit_registry(_Type, _ModuleName) -> ok.
+removed_from_rabbit_registry(_Type) -> ok.
diff --git a/deps/rabbit_common/src/rabbit_queue_collector.erl b/deps/rabbit_common/src/rabbit_queue_collector.erl
new file mode 100644
index 0000000000..ffc94ba6fb
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_queue_collector.erl
@@ -0,0 +1,80 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_queue_collector).
+
+%% Queue collector keeps track of exclusive queues and cleans them
+%% up e.g. when their connection is closed.
+
+-behaviour(gen_server).
+
+-export([start_link/1, register/2, delete_all/1]).
+
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3]).
+
+-record(state, {monitors, delete_from}).
+
+-include("rabbit.hrl").
+
+%%----------------------------------------------------------------------------
+
+-spec start_link(rabbit_types:proc_name()) -> rabbit_types:ok_pid_or_error().
+
+start_link(ProcName) ->
+ gen_server:start_link(?MODULE, [ProcName], []).
+
+-spec register(pid(), pid()) -> 'ok'.
+
+register(CollectorPid, Q) ->
+ gen_server:call(CollectorPid, {register, Q}, infinity).
+
+delete_all(CollectorPid) ->
+ gen_server:call(CollectorPid, delete_all, infinity).
+
+%%----------------------------------------------------------------------------
+
+init([ProcName]) ->
+ ?LG_PROCESS_TYPE(queue_collector),
+ ?store_proc_name(ProcName),
+ {ok, #state{monitors = pmon:new(), delete_from = undefined}}.
+
+%%--------------------------------------------------------------------------
+
+handle_call({register, QPid}, _From,
+ State = #state{monitors = QMons, delete_from = Deleting}) ->
+ case Deleting of
+ undefined -> ok;
+ _ -> ok = rabbit_amqqueue_common:delete_exclusive([QPid], Deleting)
+ end,
+ {reply, ok, State#state{monitors = pmon:monitor(QPid, QMons)}};
+
+handle_call(delete_all, From, State = #state{monitors = QMons,
+ delete_from = undefined}) ->
+ case pmon:monitored(QMons) of
+ [] -> {reply, ok, State#state{delete_from = From}};
+ QPids -> ok = rabbit_amqqueue_common:delete_exclusive(QPids, From),
+ {noreply, State#state{delete_from = From}}
+ end.
+
+handle_cast(Msg, State) ->
+ {stop, {unhandled_cast, Msg}, State}.
+
+handle_info({'DOWN', _MRef, process, DownPid, _Reason},
+ State = #state{monitors = QMons, delete_from = Deleting}) ->
+ QMons1 = pmon:erase(DownPid, QMons),
+ case Deleting =/= undefined andalso pmon:is_empty(QMons1) of
+ true -> gen_server:reply(Deleting, ok);
+ false -> ok
+ end,
+ {noreply, State#state{monitors = QMons1}}.
+
+terminate(_Reason, _State) ->
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
diff --git a/deps/rabbit_common/src/rabbit_registry.erl b/deps/rabbit_common/src/rabbit_registry.erl
new file mode 100644
index 0000000000..e68574828c
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_registry.erl
@@ -0,0 +1,165 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_registry).
+
+-behaviour(gen_server).
+
+-export([start_link/0]).
+
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,
+ code_change/3]).
+
+-export([register/3, unregister/2,
+ binary_to_type/1, lookup_module/2, lookup_all/1]).
+
+-define(SERVER, ?MODULE).
+-define(ETS_NAME, ?MODULE).
+
+-spec start_link() -> rabbit_types:ok_pid_or_error().
+-spec register(atom(), binary(), atom()) -> 'ok'.
+-spec unregister(atom(), binary()) -> 'ok'.
+-spec binary_to_type(binary()) -> atom() | rabbit_types:error('not_found').
+-spec lookup_module(atom(), atom()) ->
+ rabbit_types:ok_or_error2(atom(), 'not_found').
+-spec lookup_all(atom()) -> [{atom(), atom()}].
+
+%%---------------------------------------------------------------------------
+
+start_link() ->
+ gen_server:start_link({local, ?SERVER}, ?MODULE, [], []).
+
+%%---------------------------------------------------------------------------
+
+register(Class, TypeName, ModuleName) ->
+ gen_server:call(?SERVER, {register, Class, TypeName, ModuleName}, infinity).
+
+unregister(Class, TypeName) ->
+ gen_server:call(?SERVER, {unregister, Class, TypeName}, infinity).
+
+%% This is used with user-supplied arguments (e.g., on exchange
+%% declare), so we restrict it to existing atoms only. This means it
+%% can throw a badarg, indicating that the type cannot have been
+%% registered.
+binary_to_type(TypeBin) when is_binary(TypeBin) ->
+ case catch list_to_existing_atom(binary_to_list(TypeBin)) of
+ {'EXIT', {badarg, _}} -> {error, not_found};
+ TypeAtom -> TypeAtom
+ end.
+
+lookup_module(Class, T) when is_atom(T) ->
+ case ets:lookup(?ETS_NAME, {Class, T}) of
+ [{_, Module}] ->
+ {ok, Module};
+ [] ->
+ {error, not_found}
+ end.
+
+lookup_all(Class) ->
+ [{K, V} || [K, V] <- ets:match(?ETS_NAME, {{Class, '$1'}, '$2'})].
+
+%%---------------------------------------------------------------------------
+
+internal_binary_to_type(TypeBin) when is_binary(TypeBin) ->
+ list_to_atom(binary_to_list(TypeBin)).
+
+internal_register(Class, TypeName, ModuleName)
+ when is_atom(Class), is_binary(TypeName), is_atom(ModuleName) ->
+ ClassModule = class_module(Class),
+ Type = internal_binary_to_type(TypeName),
+ RegArg = {{Class, Type}, ModuleName},
+ ok = sanity_check_module(ClassModule, ModuleName),
+ true = ets:insert(?ETS_NAME, RegArg),
+ conditional_register(RegArg),
+ ok = ClassModule:added_to_rabbit_registry(Type, ModuleName),
+ ok.
+
+internal_unregister(Class, TypeName) ->
+ ClassModule = class_module(Class),
+ Type = internal_binary_to_type(TypeName),
+ UnregArg = {Class, Type},
+ conditional_unregister(UnregArg),
+ true = ets:delete(?ETS_NAME, UnregArg),
+ ok = ClassModule:removed_from_rabbit_registry(Type),
+ ok.
+
+%% register exchange decorator route callback only when implemented,
+%% in order to avoid unnecessary decorator calls on the fast
+%% publishing path
+conditional_register({{exchange_decorator, Type}, ModuleName}) ->
+ case erlang:function_exported(ModuleName, route, 2) of
+ true -> true = ets:insert(?ETS_NAME,
+ {{exchange_decorator_route, Type},
+ ModuleName});
+ false -> ok
+ end;
+conditional_register(_) ->
+ ok.
+
+conditional_unregister({exchange_decorator, Type}) ->
+ true = ets:delete(?ETS_NAME, {exchange_decorator_route, Type}),
+ ok;
+conditional_unregister(_) ->
+ ok.
+
+sanity_check_module(ClassModule, Module) ->
+ case catch lists:member(ClassModule,
+ lists:flatten(
+ [Bs || {Attr, Bs} <-
+ Module:module_info(attributes),
+ Attr =:= behavior orelse
+ Attr =:= behaviour])) of
+ {'EXIT', {undef, _}} -> {error, not_module};
+ false -> {error, {not_type, ClassModule}};
+ true -> ok
+ end.
+
+
+% Registry class modules. There should exist module for each registry class.
+% Class module should be behaviour (export behaviour_info/1) and implement
+% rabbit_registry_class behaviour itself: export added_to_rabbit_registry/2
+% and removed_from_rabbit_registry/1 functions.
+class_module(exchange) -> rabbit_exchange_type;
+class_module(auth_mechanism) -> rabbit_auth_mechanism;
+class_module(runtime_parameter) -> rabbit_runtime_parameter;
+class_module(exchange_decorator) -> rabbit_exchange_decorator;
+class_module(queue_decorator) -> rabbit_queue_decorator;
+class_module(policy_validator) -> rabbit_policy_validator;
+class_module(operator_policy_validator) -> rabbit_policy_validator;
+class_module(policy_merge_strategy) -> rabbit_policy_merge_strategy;
+class_module(ha_mode) -> rabbit_mirror_queue_mode;
+class_module(channel_interceptor) -> rabbit_channel_interceptor;
+class_module(queue_master_locator) -> rabbit_queue_master_locator.
+
+%%---------------------------------------------------------------------------
+
+init([]) ->
+ ?ETS_NAME = ets:new(?ETS_NAME, [protected, set, named_table]),
+ {ok, none}.
+
+handle_call({register, Class, TypeName, ModuleName}, _From, State) ->
+ ok = internal_register(Class, TypeName, ModuleName),
+ {reply, ok, State};
+
+handle_call({unregister, Class, TypeName}, _From, State) ->
+ ok = internal_unregister(Class, TypeName),
+ {reply, ok, State};
+
+handle_call(Request, _From, State) ->
+ {stop, {unhandled_call, Request}, State}.
+
+handle_cast(Request, State) ->
+ {stop, {unhandled_cast, Request}, State}.
+
+handle_info(Message, State) ->
+ {stop, {unhandled_info, Message}, State}.
+
+terminate(_Reason, _State) ->
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
diff --git a/deps/rabbit_common/src/rabbit_registry_class.erl b/deps/rabbit_common/src/rabbit_registry_class.erl
new file mode 100644
index 0000000000..c302dc2311
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_registry_class.erl
@@ -0,0 +1,12 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2016-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_registry_class).
+
+-callback added_to_rabbit_registry(atom(), atom()) -> ok.
+
+-callback removed_from_rabbit_registry(atom()) -> ok.
diff --git a/deps/rabbit_common/src/rabbit_resource_monitor_misc.erl b/deps/rabbit_common/src/rabbit_resource_monitor_misc.erl
new file mode 100644
index 0000000000..6661706998
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_resource_monitor_misc.erl
@@ -0,0 +1,39 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+
+-module(rabbit_resource_monitor_misc).
+
+-export([parse_information_unit/1]).
+
+-spec parse_information_unit(integer() | string()) ->
+ {ok, integer()} | {error, parse_error}.
+
+parse_information_unit(Value) when is_integer(Value) -> {ok, Value};
+parse_information_unit(Value0) ->
+ Value = rabbit_data_coercion:to_list(Value0),
+ case re:run(Value,
+ "^(?<VAL>[0-9]+)(?<UNIT>kB|KB|MB|GB|kb|mb|gb|Kb|Mb|Gb|kiB|KiB|MiB|GiB|kib|mib|gib|KIB|MIB|GIB|k|K|m|M|g|G)?$",
+ [{capture, all_but_first, list}]) of
+ {match, [[], _]} ->
+ {ok, list_to_integer(Value)};
+ {match, [Num]} ->
+ {ok, list_to_integer(Num)};
+ {match, [Num, Unit]} ->
+ Multiplier = case Unit of
+ KiB when KiB =:= "k"; KiB =:= "kiB"; KiB =:= "K"; KiB =:= "KIB"; KiB =:= "kib" -> 1024;
+ MiB when MiB =:= "m"; MiB =:= "MiB"; MiB =:= "M"; MiB =:= "MIB"; MiB =:= "mib" -> 1024*1024;
+ GiB when GiB =:= "g"; GiB =:= "GiB"; GiB =:= "G"; GiB =:= "GIB"; GiB =:= "gib" -> 1024*1024*1024;
+ KB when KB =:= "KB"; KB =:= "kB"; KB =:= "kb"; KB =:= "Kb" -> 1000;
+ MB when MB =:= "MB"; MB =:= "mB"; MB =:= "mb"; MB =:= "Mb" -> 1000000;
+ GB when GB =:= "GB"; GB =:= "gB"; GB =:= "gb"; GB =:= "Gb" -> 1000000000
+ end,
+ {ok, list_to_integer(Num) * Multiplier};
+ nomatch ->
+ % log error
+ {error, parse_error}
+ end.
diff --git a/deps/rabbit_common/src/rabbit_runtime.erl b/deps/rabbit_common/src/rabbit_runtime.erl
new file mode 100644
index 0000000000..94a5a5fcfe
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_runtime.erl
@@ -0,0 +1,66 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+%% This module provides access to runtime metrics that are exposed
+%% via CLI tools, management UI or otherwise used by the broker.
+
+-module(rabbit_runtime).
+
+%%
+%% API
+%%
+
+-export([guess_number_of_cpu_cores/0, msacc_stats/1]).
+-export([get_gc_info/1, gc_all_processes/0]).
+-export([get_erl_path/0]).
+
+-spec guess_number_of_cpu_cores() -> pos_integer().
+guess_number_of_cpu_cores() ->
+ case erlang:system_info(logical_processors_available) of
+ unknown -> % Happens on Mac OS X.
+ erlang:system_info(schedulers);
+ N -> N
+ end.
+
+-spec gc_all_processes() -> ok.
+gc_all_processes() ->
+ %% Run GC asynchronously. We don't care for completion notifications, so
+ %% don't use the asynchonous execution option.
+ spawn(fun() -> [erlang:garbage_collect(P, []) || P <- erlang:processes()] end),
+ ok.
+
+-spec get_gc_info(pid()) -> nonempty_list(tuple()).
+get_gc_info(Pid) ->
+ {garbage_collection, GC} = erlang:process_info(Pid, garbage_collection),
+ case proplists:get_value(max_heap_size, GC) of
+ I when is_integer(I) ->
+ GC;
+ undefined ->
+ GC;
+ Map ->
+ lists:keyreplace(max_heap_size, 1, GC,
+ {max_heap_size, maps:get(size, Map)})
+ end.
+
+-spec msacc_stats(integer()) -> nonempty_list(#{atom() => any()}).
+msacc_stats(TimeInMs) ->
+ msacc:start(TimeInMs),
+ S = msacc:stats(),
+ msacc:stop(),
+ S.
+
+% get the full path to the erl executable used to start this VM
+-spec get_erl_path() -> file:filename_all().
+get_erl_path() ->
+ {ok, [[Root]]} = init:get_argument(root),
+ Bin = filename:join(Root, "bin"),
+ case os:type() of
+ {win32, _} ->
+ filename:join(Bin, "erl.exe");
+ _ ->
+ filename:join(Bin, "erl")
+ end.
diff --git a/deps/rabbit_common/src/rabbit_runtime_parameter.erl b/deps/rabbit_common/src/rabbit_runtime_parameter.erl
new file mode 100644
index 0000000000..5f9970d25d
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_runtime_parameter.erl
@@ -0,0 +1,25 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_runtime_parameter).
+
+-behaviour(rabbit_registry_class).
+
+-export([added_to_rabbit_registry/2, removed_from_rabbit_registry/1]).
+
+-type(validate_results() ::
+ 'ok' | {error, string(), [term()]} | [validate_results()]).
+
+-callback validate(rabbit_types:vhost(), binary(), binary(),
+ term(), rabbit_types:user()) -> validate_results().
+-callback notify(rabbit_types:vhost(), binary(), binary(), term(),
+ rabbit_types:username()) -> 'ok'.
+-callback notify_clear(rabbit_types:vhost(), binary(), binary(),
+ rabbit_types:username()) -> 'ok'.
+
+added_to_rabbit_registry(_Type, _ModuleName) -> ok.
+removed_from_rabbit_registry(_Type) -> ok.
diff --git a/deps/rabbit_common/src/rabbit_semver.erl b/deps/rabbit_common/src/rabbit_semver.erl
new file mode 100644
index 0000000000..c80db0c27a
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_semver.erl
@@ -0,0 +1,730 @@
+%%% vi:ts=4 sw=4 et
+
+%%% Imported from https://github.com/erlware/erlware_commons.git
+%%% Commit 09168347525916e291c8aa6e3073e260e5f4a116
+%%% - We export normalize/1.
+%%% - We add a few more testcases around string/binary comparison.
+
+%%%-------------------------------------------------------------------
+%%% @copyright (C) 2011, Erlware LLC
+%%% @doc
+%%% Helper functions for working with semver versioning strings.
+%%% See https://semver.org/ for the spec.
+%%% @end
+%%%-------------------------------------------------------------------
+-module(rabbit_semver).
+
+-export([parse/1,
+ format/1,
+ eql/2,
+ gt/2,
+ gte/2,
+ lt/2,
+ lte/2,
+ pes/2,
+ normalize/1,
+ between/3]).
+
+%% For internal use by the rabbit_semver_parser peg
+-export([internal_parse_version/1]).
+
+-export_type([semver/0,
+ version_string/0,
+ any_version/0]).
+
+%%%===================================================================
+%%% Public Types
+%%%===================================================================
+
+-type version_element() :: non_neg_integer() | binary().
+
+-type major_minor_patch_minpatch() ::
+ version_element()
+ | {version_element(), version_element()}
+ | {version_element(), version_element(), version_element()}
+ | {version_element(), version_element(),
+ version_element(), version_element()}.
+
+-type alpha_part() :: integer() | binary() | string().
+-type alpha_info() :: {PreRelease::[alpha_part()],
+ BuildVersion::[alpha_part()]}.
+
+-type semver() :: {major_minor_patch_minpatch(), alpha_info()}.
+
+-type version_string() :: string() | binary().
+
+-type any_version() :: version_string() | semver().
+
+%%%===================================================================
+%%% API
+%%%===================================================================
+
+%% @doc parse a string or binary into a valid semver representation
+-spec parse(any_version()) -> semver().
+parse(Version) when erlang:is_list(Version) ->
+ case rabbit_semver_parser:parse(Version) of
+ {fail, _} ->
+ {erlang:iolist_to_binary(Version), {[],[]}};
+ Good ->
+ Good
+ end;
+parse(Version) when erlang:is_binary(Version) ->
+ case rabbit_semver_parser:parse(Version) of
+ {fail, _} ->
+ {Version, {[],[]}};
+ Good ->
+ Good
+ end;
+parse(Version) ->
+ Version.
+
+-spec format(semver()) -> iolist().
+format({Maj, {AlphaPart, BuildPart}})
+ when erlang:is_integer(Maj);
+ erlang:is_binary(Maj) ->
+ [format_version_part(Maj),
+ format_vsn_rest(<<"-">>, AlphaPart),
+ format_vsn_rest(<<"+">>, BuildPart)];
+format({{Maj, Min}, {AlphaPart, BuildPart}}) ->
+ [format_version_part(Maj), ".",
+ format_version_part(Min),
+ format_vsn_rest(<<"-">>, AlphaPart),
+ format_vsn_rest(<<"+">>, BuildPart)];
+format({{Maj, Min, Patch}, {AlphaPart, BuildPart}}) ->
+ [format_version_part(Maj), ".",
+ format_version_part(Min), ".",
+ format_version_part(Patch),
+ format_vsn_rest(<<"-">>, AlphaPart),
+ format_vsn_rest(<<"+">>, BuildPart)];
+format({{Maj, Min, Patch, MinPatch}, {AlphaPart, BuildPart}}) ->
+ [format_version_part(Maj), ".",
+ format_version_part(Min), ".",
+ format_version_part(Patch), ".",
+ format_version_part(MinPatch),
+ format_vsn_rest(<<"-">>, AlphaPart),
+ format_vsn_rest(<<"+">>, BuildPart)].
+
+-spec format_version_part(integer() | binary()) -> iolist().
+format_version_part(Vsn)
+ when erlang:is_integer(Vsn) ->
+ erlang:integer_to_list(Vsn);
+format_version_part(Vsn)
+ when erlang:is_binary(Vsn) ->
+ Vsn.
+
+
+
+%% @doc test for quality between semver versions
+-spec eql(any_version(), any_version()) -> boolean().
+eql(VsnA, VsnB) ->
+ NVsnA = normalize(parse(VsnA)),
+ NVsnB = normalize(parse(VsnB)),
+ NVsnA =:= NVsnB.
+
+%% @doc Test that VsnA is greater than VsnB
+-spec gt(any_version(), any_version()) -> boolean().
+gt(VsnA, VsnB) ->
+ {MMPA, {AlphaA, PatchA}} = normalize(parse(VsnA)),
+ {MMPB, {AlphaB, PatchB}} = normalize(parse(VsnB)),
+ ((MMPA > MMPB)
+ orelse
+ ((MMPA =:= MMPB)
+ andalso
+ ((AlphaA =:= [] andalso AlphaB =/= [])
+ orelse
+ ((not (AlphaB =:= [] andalso AlphaA =/= []))
+ andalso
+ (AlphaA > AlphaB))))
+ orelse
+ ((MMPA =:= MMPB)
+ andalso
+ (AlphaA =:= AlphaB)
+ andalso
+ ((PatchB =:= [] andalso PatchA =/= [])
+ orelse
+ PatchA > PatchB))).
+
+%% @doc Test that VsnA is greater than or equal to VsnB
+-spec gte(any_version(), any_version()) -> boolean().
+gte(VsnA, VsnB) ->
+ NVsnA = normalize(parse(VsnA)),
+ NVsnB = normalize(parse(VsnB)),
+ gt(NVsnA, NVsnB) orelse eql(NVsnA, NVsnB).
+
+%% @doc Test that VsnA is less than VsnB
+-spec lt(any_version(), any_version()) -> boolean().
+lt(VsnA, VsnB) ->
+ {MMPA, {AlphaA, PatchA}} = normalize(parse(VsnA)),
+ {MMPB, {AlphaB, PatchB}} = normalize(parse(VsnB)),
+ ((MMPA < MMPB)
+ orelse
+ ((MMPA =:= MMPB)
+ andalso
+ ((AlphaB =:= [] andalso AlphaA =/= [])
+ orelse
+ ((not (AlphaA =:= [] andalso AlphaB =/= []))
+ andalso
+ (AlphaA < AlphaB))))
+ orelse
+ ((MMPA =:= MMPB)
+ andalso
+ (AlphaA =:= AlphaB)
+ andalso
+ ((PatchA =:= [] andalso PatchB =/= [])
+ orelse
+ PatchA < PatchB))).
+
+%% @doc Test that VsnA is less than or equal to VsnB
+-spec lte(any_version(), any_version()) -> boolean().
+lte(VsnA, VsnB) ->
+ NVsnA = normalize(parse(VsnA)),
+ NVsnB = normalize(parse(VsnB)),
+ lt(NVsnA, NVsnB) orelse eql(NVsnA, NVsnB).
+
+%% @doc Test that VsnMatch is greater than or equal to Vsn1 and
+%% less than or equal to Vsn2
+-spec between(any_version(), any_version(), any_version()) -> boolean().
+between(Vsn1, Vsn2, VsnMatch) ->
+ NVsnA = normalize(parse(Vsn1)),
+ NVsnB = normalize(parse(Vsn2)),
+ NVsnMatch = normalize(parse(VsnMatch)),
+ gte(NVsnMatch, NVsnA) andalso
+ lte(NVsnMatch, NVsnB).
+
+%% @doc check that VsnA is Approximately greater than VsnB
+%%
+%% Specifying ">= 2.6.5" is an optimistic version constraint. All
+%% versions greater than the one specified, including major releases
+%% (e.g. 3.0.0) are allowed.
+%%
+%% Conversely, specifying "~> 2.6" is pessimistic about future major
+%% revisions and "~> 2.6.5" is pessimistic about future minor
+%% revisions.
+%%
+%% "~> 2.6" matches cookbooks >= 2.6.0 AND &lt; 3.0.0
+%% "~> 2.6.5" matches cookbooks >= 2.6.5 AND &lt; 2.7.0
+pes(VsnA, VsnB) ->
+ internal_pes(parse(VsnA), parse(VsnB)).
+
+%%%===================================================================
+%%% Friend Functions
+%%%===================================================================
+%% @doc helper function for the peg grammar to parse the iolist into a semver
+-spec internal_parse_version(iolist()) -> semver().
+internal_parse_version([MMP, AlphaPart, BuildPart, _]) ->
+ {parse_major_minor_patch_minpatch(MMP), {parse_alpha_part(AlphaPart),
+ parse_alpha_part(BuildPart)}}.
+
+%% @doc helper function for the peg grammar to parse the iolist into a major_minor_patch
+-spec parse_major_minor_patch_minpatch(iolist()) -> major_minor_patch_minpatch().
+parse_major_minor_patch_minpatch([MajVsn, [], [], []]) ->
+ strip_maj_version(MajVsn);
+parse_major_minor_patch_minpatch([MajVsn, [<<".">>, MinVsn], [], []]) ->
+ {strip_maj_version(MajVsn), MinVsn};
+parse_major_minor_patch_minpatch([MajVsn,
+ [<<".">>, MinVsn],
+ [<<".">>, PatchVsn], []]) ->
+ {strip_maj_version(MajVsn), MinVsn, PatchVsn};
+parse_major_minor_patch_minpatch([MajVsn,
+ [<<".">>, MinVsn],
+ [<<".">>, PatchVsn],
+ [<<".">>, MinPatch]]) ->
+ {strip_maj_version(MajVsn), MinVsn, PatchVsn, MinPatch}.
+
+%% @doc helper function for the peg grammar to parse the iolist into an alpha part
+-spec parse_alpha_part(iolist()) -> [alpha_part()].
+parse_alpha_part([]) ->
+ [];
+parse_alpha_part([_, AV1, Rest]) ->
+ [erlang:iolist_to_binary(AV1) |
+ [format_alpha_part(Part) || Part <- Rest]].
+
+%% @doc according to semver alpha parts that can be treated like
+%% numbers must be. We implement that here by taking the alpha part
+%% and trying to convert it to a number, if it succeeds we use
+%% it. Otherwise we do not.
+-spec format_alpha_part(iolist()) -> integer() | binary().
+format_alpha_part([<<".">>, AlphaPart]) ->
+ Bin = erlang:iolist_to_binary(AlphaPart),
+ try
+ erlang:list_to_integer(erlang:binary_to_list(Bin))
+ catch
+ error:badarg ->
+ Bin
+ end.
+
+%%%===================================================================
+%%% Internal Functions
+%%%===================================================================
+-spec strip_maj_version(iolist()) -> version_element().
+strip_maj_version([<<"v">>, MajVsn]) ->
+ MajVsn;
+strip_maj_version([[], MajVsn]) ->
+ MajVsn;
+strip_maj_version(MajVsn) ->
+ MajVsn.
+
+-spec to_list(integer() | binary() | string()) -> string() | binary().
+to_list(Detail) when erlang:is_integer(Detail) ->
+ erlang:integer_to_list(Detail);
+to_list(Detail) when erlang:is_list(Detail); erlang:is_binary(Detail) ->
+ Detail.
+
+-spec format_vsn_rest(binary() | string(), [integer() | binary()]) -> iolist().
+format_vsn_rest(_TypeMark, []) ->
+ [];
+format_vsn_rest(TypeMark, [Head | Rest]) ->
+ [TypeMark, Head |
+ [[".", to_list(Detail)] || Detail <- Rest]].
+
+%% @doc normalize the semver so they can be compared
+-spec normalize(semver()) -> semver().
+normalize({Vsn, Rest})
+ when erlang:is_binary(Vsn);
+ erlang:is_integer(Vsn) ->
+ {{Vsn, 0, 0, 0}, Rest};
+normalize({{Maj, Min}, Rest}) ->
+ {{Maj, Min, 0, 0}, Rest};
+normalize({{Maj, Min, Patch}, Rest}) ->
+ {{Maj, Min, Patch, 0}, Rest};
+normalize(Other = {{_, _, _, _}, {_,_}}) ->
+ Other.
+
+%% @doc to do the pessimistic compare we need a parsed semver. This is
+%% the internal implementation of the of the pessimistic run. The
+%% external just ensures that versions are parsed.
+-spec internal_pes(semver(), semver()) -> boolean().
+internal_pes(VsnA, {{LM, LMI}, _})
+ when erlang:is_integer(LM),
+ erlang:is_integer(LMI) ->
+ gte(VsnA, {{LM, LMI, 0}, {[], []}}) andalso
+ lt(VsnA, {{LM + 1, 0, 0, 0}, {[], []}});
+internal_pes(VsnA, {{LM, LMI, LP}, _})
+ when erlang:is_integer(LM),
+ erlang:is_integer(LMI),
+ erlang:is_integer(LP) ->
+ gte(VsnA, {{LM, LMI, LP}, {[], []}})
+ andalso
+ lt(VsnA, {{LM, LMI + 1, 0, 0}, {[], []}});
+internal_pes(VsnA, {{LM, LMI, LP, LMP}, _})
+ when erlang:is_integer(LM),
+ erlang:is_integer(LMI),
+ erlang:is_integer(LP),
+ erlang:is_integer(LMP) ->
+ gte(VsnA, {{LM, LMI, LP, LMP}, {[], []}})
+ andalso
+ lt(VsnA, {{LM, LMI, LP + 1, 0}, {[], []}});
+internal_pes(Vsn, LVsn) ->
+ gte(Vsn, LVsn).
+
+%%%===================================================================
+%%% Test Functions
+%%%===================================================================
+
+-ifdef(TEST).
+-include_lib("eunit/include/eunit.hrl").
+
+eql_test() ->
+ ?assertMatch(true, eql("1.0.0-alpha",
+ "1.0.0-alpha")),
+ ?assertMatch(true, eql(<<"1.0.0-alpha">>,
+ "1.0.0-alpha")),
+ ?assertMatch(true, eql("1.0.0-alpha",
+ <<"1.0.0-alpha">>)),
+ ?assertMatch(true, eql(<<"1.0.0-alpha">>,
+ <<"1.0.0-alpha">>)),
+ ?assertMatch(true, eql("v1.0.0-alpha",
+ "1.0.0-alpha")),
+ ?assertMatch(true, eql("1",
+ "1.0.0")),
+ ?assertMatch(true, eql("v1",
+ "v1.0.0")),
+ ?assertMatch(true, eql("1.0",
+ "1.0.0")),
+ ?assertMatch(true, eql("1.0.0",
+ "1")),
+ ?assertMatch(true, eql("1.0.0.0",
+ "1")),
+ ?assertMatch(true, eql("1.0+alpha.1",
+ "1.0.0+alpha.1")),
+ ?assertMatch(true, eql("1.0-alpha.1+build.1",
+ "1.0.0-alpha.1+build.1")),
+ ?assertMatch(true, eql("1.0-alpha.1+build.1",
+ "1.0.0.0-alpha.1+build.1")),
+ ?assertMatch(true, eql("1.0-alpha.1+build.1",
+ "v1.0.0.0-alpha.1+build.1")),
+ ?assertMatch(true, eql("aa", "aa")),
+ ?assertMatch(true, eql("AA.BB", "AA.BB")),
+ ?assertMatch(true, eql("BBB-super", "BBB-super")),
+ ?assertMatch(true, not eql("1.0.0",
+ "1.0.1")),
+ ?assertMatch(true, not eql(<<"1.0.0">>,
+ "1.0.1")),
+ ?assertMatch(true, not eql("1.0.0",
+ <<"1.0.1">>)),
+ ?assertMatch(true, not eql(<<"1.0.0">>,
+ <<"1.0.1">>)),
+ ?assertMatch(true, not eql("1.0.0-alpha",
+ "1.0.1+alpha")),
+ ?assertMatch(true, not eql("1.0.0+build.1",
+ "1.0.1+build.2")),
+ ?assertMatch(true, not eql("1.0.0.0+build.1",
+ "1.0.0.1+build.2")),
+ ?assertMatch(true, not eql("FFF", "BBB")),
+ ?assertMatch(true, not eql("1", "1BBBB")).
+
+gt_test() ->
+ ?assertMatch(true, gt("1.0.0-alpha.1",
+ "1.0.0-alpha")),
+ ?assertMatch(true, gt("1.0.0.1-alpha.1",
+ "1.0.0.1-alpha")),
+ ?assertMatch(true, gt("1.0.0.4-alpha.1",
+ "1.0.0.2-alpha")),
+ ?assertMatch(true, gt("1.0.0.0-alpha.1",
+ "1.0.0-alpha")),
+ ?assertMatch(true, gt("1.0.0-beta.2",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, gt("1.0.0-beta.11",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, gt("1.0.0-beta.11",
+ "1.0.0.0-beta.2")),
+ ?assertMatch(true, gt("1.0.0-rc.1", "1.0.0-beta.11")),
+ ?assertMatch(true, gt("1.0.0-rc.1+build.1", "1.0.0-rc.1")),
+ ?assertMatch(true, gt("1.0.0", "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, gt("1.0.0+0.3.7", "1.0.0")),
+ ?assertMatch(true, gt("1.3.7+build", "1.0.0+0.3.7")),
+ ?assertMatch(true, gt("1.3.7+build.2.b8f12d7",
+ "1.3.7+build")),
+ ?assertMatch(true, gt("1.3.7+build.2.b8f12d7",
+ "1.3.7.0+build")),
+ ?assertMatch(true, gt("1.3.7+build.11.e0f985a",
+ "1.3.7+build.2.b8f12d7")),
+ ?assertMatch(true, gt("aa.cc",
+ "aa.bb")),
+ ?assertMatch(true, not gt("1.0.0-alpha",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, not gt("1.0.0-alpha",
+ "1.0.0.0-alpha.1")),
+ ?assertMatch(true, not gt("1.0.0-alpha.1",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, not gt("1.0.0-beta.2",
+ "1.0.0-beta.11")),
+ ?assertMatch(true, not gt("1.0.0-beta.11",
+ "1.0.0-rc.1")),
+ ?assertMatch(true, not gt("1.0.0-rc.1",
+ "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, not gt("1.0.0-rc.1+build.1",
+ "1.0.0")),
+ ?assertMatch(true, not gt("1.0.0",
+ "1.0.0+0.3.7")),
+ ?assertMatch(true, not gt("1.0.0+0.3.7",
+ "1.3.7+build")),
+ ?assertMatch(true, not gt("1.3.7+build",
+ "1.3.7+build.2.b8f12d7")),
+ ?assertMatch(true, not gt("1.3.7+build.2.b8f12d7",
+ "1.3.7+build.11.e0f985a")),
+ ?assertMatch(true, not gt("1.0.0-alpha",
+ "1.0.0-alpha")),
+ ?assertMatch(true, not gt("1",
+ "1.0.0")),
+ ?assertMatch(true, not gt("aa.bb",
+ "aa.bb")),
+ ?assertMatch(true, not gt("aa.cc",
+ "aa.dd")),
+ ?assertMatch(true, not gt("1.0",
+ "1.0.0")),
+ ?assertMatch(true, not gt("1.0.0",
+ "1")),
+ ?assertMatch(true, not gt("1.0+alpha.1",
+ "1.0.0+alpha.1")),
+ ?assertMatch(true, not gt("1.0-alpha.1+build.1",
+ "1.0.0-alpha.1+build.1")).
+
+lt_test() ->
+ ?assertMatch(true, lt("1.0.0-alpha",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, lt("1.0.0-alpha",
+ "1.0.0.0-alpha.1")),
+ ?assertMatch(true, lt("1.0.0-alpha.1",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, lt("1.0.0-beta.2",
+ "1.0.0-beta.11")),
+ ?assertMatch(true, lt("1.0.0-beta.11",
+ "1.0.0-rc.1")),
+ ?assertMatch(true, lt("1.0.0.1-beta.11",
+ "1.0.0.1-rc.1")),
+ ?assertMatch(true, lt("1.0.0-rc.1",
+ "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, lt("1.0.0-rc.1+build.1",
+ "1.0.0")),
+ ?assertMatch(true, lt("1.0.0",
+ "1.0.0+0.3.7")),
+ ?assertMatch(true, lt("1.0.0+0.3.7",
+ "1.3.7+build")),
+ ?assertMatch(true, lt("1.3.7+build",
+ "1.3.7+build.2.b8f12d7")),
+ ?assertMatch(true, lt("1.3.7+build.2.b8f12d7",
+ "1.3.7+build.11.e0f985a")),
+ ?assertMatch(true, not lt("1.0.0-alpha",
+ "1.0.0-alpha")),
+ ?assertMatch(true, not lt("1",
+ "1.0.0")),
+ ?assertMatch(true, lt("1",
+ "1.0.0.1")),
+ ?assertMatch(true, lt("AA.DD",
+ "AA.EE")),
+ ?assertMatch(true, not lt("1.0",
+ "1.0.0")),
+ ?assertMatch(true, not lt("1.0.0.0",
+ "1")),
+ ?assertMatch(true, not lt("1.0+alpha.1",
+ "1.0.0+alpha.1")),
+ ?assertMatch(true, not lt("AA.DD", "AA.CC")),
+ ?assertMatch(true, not lt("1.0-alpha.1+build.1",
+ "1.0.0-alpha.1+build.1")),
+ ?assertMatch(true, not lt("1.0.0-alpha.1",
+ "1.0.0-alpha")),
+ ?assertMatch(true, not lt("1.0.0-beta.2",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, not lt("1.0.0-beta.11",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, not lt("1.0.0-rc.1", "1.0.0-beta.11")),
+ ?assertMatch(true, not lt("1.0.0-rc.1+build.1", "1.0.0-rc.1")),
+ ?assertMatch(true, not lt("1.0.0", "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, not lt("1.0.0+0.3.7", "1.0.0")),
+ ?assertMatch(true, not lt("1.3.7+build", "1.0.0+0.3.7")),
+ ?assertMatch(true, not lt("1.3.7+build.2.b8f12d7",
+ "1.3.7+build")),
+ ?assertMatch(true, not lt("1.3.7+build.11.e0f985a",
+ "1.3.7+build.2.b8f12d7")).
+
+gte_test() ->
+ ?assertMatch(true, gte("1.0.0-alpha",
+ "1.0.0-alpha")),
+
+ ?assertMatch(true, gte("1",
+ "1.0.0")),
+
+ ?assertMatch(true, gte("1.0",
+ "1.0.0")),
+
+ ?assertMatch(true, gte("1.0.0",
+ "1")),
+
+ ?assertMatch(true, gte("1.0.0.0",
+ "1")),
+
+ ?assertMatch(true, gte("1.0+alpha.1",
+ "1.0.0+alpha.1")),
+
+ ?assertMatch(true, gte("1.0-alpha.1+build.1",
+ "1.0.0-alpha.1+build.1")),
+
+ ?assertMatch(true, gte("1.0.0-alpha.1+build.1",
+ "1.0.0.0-alpha.1+build.1")),
+ ?assertMatch(true, gte("1.0.0-alpha.1",
+ "1.0.0-alpha")),
+ ?assertMatch(true, gte("1.0.0-beta.2",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, gte("1.0.0-beta.11",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, gte("aa.bb", "aa.bb")),
+ ?assertMatch(true, gte("dd", "aa")),
+ ?assertMatch(true, gte("1.0.0-rc.1", "1.0.0-beta.11")),
+ ?assertMatch(true, gte("1.0.0-rc.1+build.1", "1.0.0-rc.1")),
+ ?assertMatch(true, gte("1.0.0", "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, gte("1.0.0+0.3.7", "1.0.0")),
+ ?assertMatch(true, gte("1.3.7+build", "1.0.0+0.3.7")),
+ ?assertMatch(true, gte("1.3.7+build.2.b8f12d7",
+ "1.3.7+build")),
+ ?assertMatch(true, gte("1.3.7+build.11.e0f985a",
+ "1.3.7+build.2.b8f12d7")),
+ ?assertMatch(true, not gte("1.0.0-alpha",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, not gte("CC", "DD")),
+ ?assertMatch(true, not gte("1.0.0-alpha.1",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, not gte("1.0.0-beta.2",
+ "1.0.0-beta.11")),
+ ?assertMatch(true, not gte("1.0.0-beta.11",
+ "1.0.0-rc.1")),
+ ?assertMatch(true, not gte("1.0.0-rc.1",
+ "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, not gte("1.0.0-rc.1+build.1",
+ "1.0.0")),
+ ?assertMatch(true, not gte("1.0.0",
+ "1.0.0+0.3.7")),
+ ?assertMatch(true, not gte("1.0.0+0.3.7",
+ "1.3.7+build")),
+ ?assertMatch(true, not gte("1.0.0",
+ "1.0.0+build.1")),
+ ?assertMatch(true, not gte("1.3.7+build",
+ "1.3.7+build.2.b8f12d7")),
+ ?assertMatch(true, not gte("1.3.7+build.2.b8f12d7",
+ "1.3.7+build.11.e0f985a")).
+lte_test() ->
+ ?assertMatch(true, lte("1.0.0-alpha",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, lte("1.0.0-alpha.1",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, lte("1.0.0-beta.2",
+ "1.0.0-beta.11")),
+ ?assertMatch(true, lte("1.0.0-beta.11",
+ "1.0.0-rc.1")),
+ ?assertMatch(true, lte("1.0.0-rc.1",
+ "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, lte("1.0.0-rc.1+build.1",
+ "1.0.0")),
+ ?assertMatch(true, lte("1.0.0",
+ "1.0.0+0.3.7")),
+ ?assertMatch(true, lte("1.0.0+0.3.7",
+ "1.3.7+build")),
+ ?assertMatch(true, lte("1.3.7+build",
+ "1.3.7+build.2.b8f12d7")),
+ ?assertMatch(true, lte("1.3.7+build.2.b8f12d7",
+ "1.3.7+build.11.e0f985a")),
+ ?assertMatch(true, lte("1.0.0-alpha",
+ "1.0.0-alpha")),
+ ?assertMatch(true, lte("1",
+ "1.0.0")),
+ ?assertMatch(true, lte("1.0",
+ "1.0.0")),
+ ?assertMatch(true, lte("1.0.0",
+ "1")),
+ ?assertMatch(true, lte("1.0+alpha.1",
+ "1.0.0+alpha.1")),
+ ?assertMatch(true, lte("1.0.0.0+alpha.1",
+ "1.0.0+alpha.1")),
+ ?assertMatch(true, lte("1.0-alpha.1+build.1",
+ "1.0.0-alpha.1+build.1")),
+ ?assertMatch(true, lte("aa","cc")),
+ ?assertMatch(true, lte("cc","cc")),
+ ?assertMatch(true, not lte("1.0.0-alpha.1",
+ "1.0.0-alpha")),
+ ?assertMatch(true, not lte("cc", "aa")),
+ ?assertMatch(true, not lte("1.0.0-beta.2",
+ "1.0.0-alpha.1")),
+ ?assertMatch(true, not lte("1.0.0-beta.11",
+ "1.0.0-beta.2")),
+ ?assertMatch(true, not lte("1.0.0-rc.1", "1.0.0-beta.11")),
+ ?assertMatch(true, not lte("1.0.0-rc.1+build.1", "1.0.0-rc.1")),
+ ?assertMatch(true, not lte("1.0.0", "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, not lte("1.0.0+0.3.7", "1.0.0")),
+ ?assertMatch(true, not lte("1.3.7+build", "1.0.0+0.3.7")),
+ ?assertMatch(true, not lte("1.3.7+build.2.b8f12d7",
+ "1.3.7+build")),
+ ?assertMatch(true, not lte("1.3.7+build.11.e0f985a",
+ "1.3.7+build.2.b8f12d7")).
+
+between_test() ->
+ ?assertMatch(true, between("1.0.0-alpha",
+ "1.0.0-alpha.3",
+ "1.0.0-alpha.2")),
+ ?assertMatch(true, between("1.0.0-alpha.1",
+ "1.0.0-beta.2",
+ "1.0.0-alpha.25")),
+ ?assertMatch(true, between("1.0.0-beta.2",
+ "1.0.0-beta.11",
+ "1.0.0-beta.7")),
+ ?assertMatch(true, between("1.0.0-beta.11",
+ "1.0.0-rc.3",
+ "1.0.0-rc.1")),
+ ?assertMatch(true, between("1.0.0-rc.1",
+ "1.0.0-rc.1+build.3",
+ "1.0.0-rc.1+build.1")),
+
+ ?assertMatch(true, between("1.0.0.0-rc.1",
+ "1.0.0-rc.1+build.3",
+ "1.0.0-rc.1+build.1")),
+ ?assertMatch(true, between("1.0.0-rc.1+build.1",
+ "1.0.0",
+ "1.0.0-rc.33")),
+ ?assertMatch(true, between("1.0.0",
+ "1.0.0+0.3.7",
+ "1.0.0+0.2")),
+ ?assertMatch(true, between("1.0.0+0.3.7",
+ "1.3.7+build",
+ "1.2")),
+ ?assertMatch(true, between("1.3.7+build",
+ "1.3.7+build.2.b8f12d7",
+ "1.3.7+build.1")),
+ ?assertMatch(true, between("1.3.7+build.2.b8f12d7",
+ "1.3.7+build.11.e0f985a",
+ "1.3.7+build.10.a36faa")),
+ ?assertMatch(true, between("1.0.0-alpha",
+ "1.0.0-alpha",
+ "1.0.0-alpha")),
+ ?assertMatch(true, between("1",
+ "1.0.0",
+ "1.0.0")),
+ ?assertMatch(true, between("1.0",
+ "1.0.0",
+ "1.0.0")),
+
+ ?assertMatch(true, between("1.0",
+ "1.0.0.0",
+ "1.0.0.0")),
+ ?assertMatch(true, between("1.0.0",
+ "1",
+ "1")),
+ ?assertMatch(true, between("1.0+alpha.1",
+ "1.0.0+alpha.1",
+ "1.0.0+alpha.1")),
+ ?assertMatch(true, between("1.0-alpha.1+build.1",
+ "1.0.0-alpha.1+build.1",
+ "1.0.0-alpha.1+build.1")),
+ ?assertMatch(true, between("aaa",
+ "ddd",
+ "cc")),
+ ?assertMatch(true, not between("1.0.0-alpha.1",
+ "1.0.0-alpha.22",
+ "1.0.0")),
+ ?assertMatch(true, not between("1.0.0",
+ "1.0.0-alpha.1",
+ "2.0")),
+ ?assertMatch(true, not between("1.0.0-beta.1",
+ "1.0.0-beta.11",
+ "1.0.0-alpha")),
+ ?assertMatch(true, not between("1.0.0-beta.11", "1.0.0-rc.1",
+ "1.0.0-rc.22")),
+ ?assertMatch(true, not between("aaa", "ddd", "zzz")).
+
+pes_test() ->
+ ?assertMatch(true, pes("2.6.0", "2.6")),
+ ?assertMatch(true, pes("2.7", "2.6")),
+ ?assertMatch(true, pes("2.8", "2.6")),
+ ?assertMatch(true, pes("2.9", "2.6")),
+ ?assertMatch(true, pes("A.B", "A.A")),
+ ?assertMatch(true, not pes("3.0.0", "2.6")),
+ ?assertMatch(true, not pes("2.5", "2.6")),
+ ?assertMatch(true, pes("2.6.5", "2.6.5")),
+ ?assertMatch(true, pes("2.6.6", "2.6.5")),
+ ?assertMatch(true, pes("2.6.7", "2.6.5")),
+ ?assertMatch(true, pes("2.6.8", "2.6.5")),
+ ?assertMatch(true, pes("2.6.9", "2.6.5")),
+ ?assertMatch(true, pes("2.6.0.9", "2.6.0.5")),
+ ?assertMatch(true, not pes("2.7", "2.6.5")),
+ ?assertMatch(true, not pes("2.1.7", "2.1.6.5")),
+ ?assertMatch(true, not pes("A.A", "A.B")),
+ ?assertMatch(true, not pes("2.5", "2.6.5")).
+
+version_format_test() ->
+ ?assertEqual(["1", [], []], format({1, {[],[]}})),
+ ?assertEqual(["1", ".", "2", ".", "34", [], []], format({{1,2,34},{[],[]}})),
+ ?assertEqual(<<"a">>, erlang:iolist_to_binary(format({<<"a">>, {[],[]}}))),
+ ?assertEqual(<<"a.b">>, erlang:iolist_to_binary(format({{<<"a">>,<<"b">>}, {[],[]}}))),
+ ?assertEqual(<<"1">>, erlang:iolist_to_binary(format({1, {[],[]}}))),
+ ?assertEqual(<<"1.2">>, erlang:iolist_to_binary(format({{1,2}, {[],[]}}))),
+ ?assertEqual(<<"1.2.2">>, erlang:iolist_to_binary(format({{1,2,2}, {[],[]}}))),
+ ?assertEqual(<<"1.99.2">>, erlang:iolist_to_binary(format({{1,99,2}, {[],[]}}))),
+ ?assertEqual(<<"1.99.2-alpha">>, erlang:iolist_to_binary(format({{1,99,2}, {[<<"alpha">>],[]}}))),
+ ?assertEqual(<<"1.99.2-alpha.1">>, erlang:iolist_to_binary(format({{1,99,2}, {[<<"alpha">>,1], []}}))),
+ ?assertEqual(<<"1.99.2+build.1.a36">>,
+ erlang:iolist_to_binary(format({{1,99,2}, {[], [<<"build">>, 1, <<"a36">>]}}))),
+ ?assertEqual(<<"1.99.2.44+build.1.a36">>,
+ erlang:iolist_to_binary(format({{1,99,2,44}, {[], [<<"build">>, 1, <<"a36">>]}}))),
+ ?assertEqual(<<"1.99.2-alpha.1+build.1.a36">>,
+ erlang:iolist_to_binary(format({{1,99,2}, {[<<"alpha">>, 1], [<<"build">>, 1, <<"a36">>]}}))),
+ ?assertEqual(<<"1">>, erlang:iolist_to_binary(format({1, {[],[]}}))).
+
+-endif.
diff --git a/deps/rabbit_common/src/rabbit_semver_parser.erl b/deps/rabbit_common/src/rabbit_semver_parser.erl
new file mode 100644
index 0000000000..3a036021f7
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_semver_parser.erl
@@ -0,0 +1,306 @@
+%%% Imported from https://github.com/erlware/erlware_commons.git
+%%% Commit 09168347525916e291c8aa6e3073e260e5f4a116
+
+-module(rabbit_semver_parser).
+-export([parse/1,file/1]).
+-define(p_anything,true).
+-define(p_charclass,true).
+-define(p_choose,true).
+-define(p_not,true).
+-define(p_one_or_more,true).
+-define(p_optional,true).
+-define(p_scan,true).
+-define(p_seq,true).
+-define(p_string,true).
+-define(p_zero_or_more,true).
+
+
+
+-spec file(file:name()) -> any().
+file(Filename) -> case file:read_file(Filename) of {ok,Bin} -> parse(Bin); Err -> Err end.
+
+-spec parse(binary() | list()) -> any().
+parse(List) when is_list(List) -> parse(unicode:characters_to_binary(List));
+parse(Input) when is_binary(Input) ->
+ _ = setup_memo(),
+ Result = case 'semver'(Input,{{line,1},{column,1}}) of
+ {AST, <<>>, _Index} -> AST;
+ Any -> Any
+ end,
+ release_memo(), Result.
+
+-spec 'semver'(input(), index()) -> parse_result().
+'semver'(Input, Index) ->
+ p(Input, Index, 'semver', fun(I,D) -> (p_seq([fun 'major_minor_patch_min_patch'/2, p_optional(p_seq([p_string(<<"-">>), fun 'alpha_part'/2, p_zero_or_more(p_seq([p_string(<<".">>), fun 'alpha_part'/2]))])), p_optional(p_seq([p_string(<<"+">>), fun 'alpha_part'/2, p_zero_or_more(p_seq([p_string(<<".">>), fun 'alpha_part'/2]))])), p_not(p_anything())]))(I,D) end, fun(Node, _Idx) -> rabbit_semver:internal_parse_version(Node) end).
+
+-spec 'major_minor_patch_min_patch'(input(), index()) -> parse_result().
+'major_minor_patch_min_patch'(Input, Index) ->
+ p(Input, Index, 'major_minor_patch_min_patch', fun(I,D) -> (p_seq([p_choose([p_seq([p_optional(p_string(<<"v">>)), fun 'numeric_part'/2]), fun 'alpha_part'/2]), p_optional(p_seq([p_string(<<".">>), fun 'version_part'/2])), p_optional(p_seq([p_string(<<".">>), fun 'version_part'/2])), p_optional(p_seq([p_string(<<".">>), fun 'version_part'/2]))]))(I,D) end, fun(Node, Idx) ->transform('major_minor_patch_min_patch', Node, Idx) end).
+
+-spec 'version_part'(input(), index()) -> parse_result().
+'version_part'(Input, Index) ->
+ p(Input, Index, 'version_part', fun(I,D) -> (p_choose([fun 'numeric_part'/2, fun 'alpha_part'/2]))(I,D) end, fun(Node, Idx) ->transform('version_part', Node, Idx) end).
+
+-spec 'numeric_part'(input(), index()) -> parse_result().
+'numeric_part'(Input, Index) ->
+ p(Input, Index, 'numeric_part', fun(I,D) -> (p_one_or_more(p_charclass(<<"[0-9]">>)))(I,D) end, fun(Node, _Idx) ->erlang:list_to_integer(erlang:binary_to_list(erlang:iolist_to_binary(Node))) end).
+
+-spec 'alpha_part'(input(), index()) -> parse_result().
+'alpha_part'(Input, Index) ->
+ p(Input, Index, 'alpha_part', fun(I,D) -> (p_one_or_more(p_charclass(<<"[A-Za-z0-9]">>)))(I,D) end, fun(Node, _Idx) ->erlang:iolist_to_binary(Node) end).
+
+
+transform(_,Node,_Index) -> Node.
+-file("peg_includes.hrl", 1).
+-type index() :: {{line, pos_integer()}, {column, pos_integer()}}.
+-type input() :: binary().
+-type parse_failure() :: {fail, term()}.
+-type parse_success() :: {term(), input(), index()}.
+-type parse_result() :: parse_failure() | parse_success().
+-type parse_fun() :: fun((input(), index()) -> parse_result()).
+-type xform_fun() :: fun((input(), index()) -> term()).
+
+-spec p(input(), index(), atom(), parse_fun(), xform_fun()) -> parse_result().
+p(Inp, StartIndex, Name, ParseFun, TransformFun) ->
+ case get_memo(StartIndex, Name) of % See if the current reduction is memoized
+ {ok, Memo} -> %Memo; % If it is, return the stored result
+ Memo;
+ _ -> % If not, attempt to parse
+ Result = case ParseFun(Inp, StartIndex) of
+ {fail,_} = Failure -> % If it fails, memoize the failure
+ Failure;
+ {Match, InpRem, NewIndex} -> % If it passes, transform and memoize the result.
+ Transformed = TransformFun(Match, StartIndex),
+ {Transformed, InpRem, NewIndex}
+ end,
+ memoize(StartIndex, Name, Result),
+ Result
+ end.
+
+-spec setup_memo() -> ets:tid().
+setup_memo() ->
+ put({parse_memo_table, ?MODULE}, ets:new(?MODULE, [set])).
+
+-spec release_memo() -> true.
+release_memo() ->
+ ets:delete(memo_table_name()).
+
+-spec memoize(index(), atom(), parse_result()) -> true.
+memoize(Index, Name, Result) ->
+ Memo = case ets:lookup(memo_table_name(), Index) of
+ [] -> [];
+ [{Index, Plist}] -> Plist
+ end,
+ ets:insert(memo_table_name(), {Index, [{Name, Result}|Memo]}).
+
+-spec get_memo(index(), atom()) -> {ok, term()} | {error, not_found}.
+get_memo(Index, Name) ->
+ case ets:lookup(memo_table_name(), Index) of
+ [] -> {error, not_found};
+ [{Index, Plist}] ->
+ case proplists:lookup(Name, Plist) of
+ {Name, Result} -> {ok, Result};
+ _ -> {error, not_found}
+ end
+ end.
+
+-spec memo_table_name() -> ets:tid().
+memo_table_name() ->
+ get({parse_memo_table, ?MODULE}).
+
+-ifdef(p_eof).
+-spec p_eof() -> parse_fun().
+p_eof() ->
+ fun(<<>>, Index) -> {eof, [], Index};
+ (_, Index) -> {fail, {expected, eof, Index}} end.
+-endif.
+
+-ifdef(p_optional).
+-spec p_optional(parse_fun()) -> parse_fun().
+p_optional(P) ->
+ fun(Input, Index) ->
+ case P(Input, Index) of
+ {fail,_} -> {[], Input, Index};
+ {_, _, _} = Success -> Success
+ end
+ end.
+-endif.
+
+-ifdef(p_not).
+-spec p_not(parse_fun()) -> parse_fun().
+p_not(P) ->
+ fun(Input, Index)->
+ case P(Input,Index) of
+ {fail,_} ->
+ {[], Input, Index};
+ {Result, _, _} -> {fail, {expected, {no_match, Result},Index}}
+ end
+ end.
+-endif.
+
+-ifdef(p_assert).
+-spec p_assert(parse_fun()) -> parse_fun().
+p_assert(P) ->
+ fun(Input,Index) ->
+ case P(Input,Index) of
+ {fail,_} = Failure-> Failure;
+ _ -> {[], Input, Index}
+ end
+ end.
+-endif.
+
+-ifdef(p_seq).
+-spec p_seq([parse_fun()]) -> parse_fun().
+p_seq(P) ->
+ fun(Input, Index) ->
+ p_all(P, Input, Index, [])
+ end.
+
+-spec p_all([parse_fun()], input(), index(), [term()]) -> parse_result().
+p_all([], Inp, Index, Accum ) -> {lists:reverse( Accum ), Inp, Index};
+p_all([P|Parsers], Inp, Index, Accum) ->
+ case P(Inp, Index) of
+ {fail, _} = Failure -> Failure;
+ {Result, InpRem, NewIndex} -> p_all(Parsers, InpRem, NewIndex, [Result|Accum])
+ end.
+-endif.
+
+-ifdef(p_choose).
+-spec p_choose([parse_fun()]) -> parse_fun().
+p_choose(Parsers) ->
+ fun(Input, Index) ->
+ p_attempt(Parsers, Input, Index, none)
+ end.
+
+-spec p_attempt([parse_fun()], input(), index(), none | parse_failure()) -> parse_result().
+p_attempt([], _Input, _Index, Failure) -> Failure;
+p_attempt([P|Parsers], Input, Index, FirstFailure)->
+ case P(Input, Index) of
+ {fail, _} = Failure ->
+ case FirstFailure of
+ none -> p_attempt(Parsers, Input, Index, Failure);
+ _ -> p_attempt(Parsers, Input, Index, FirstFailure)
+ end;
+ Result -> Result
+ end.
+-endif.
+
+-ifdef(p_zero_or_more).
+-spec p_zero_or_more(parse_fun()) -> parse_fun().
+p_zero_or_more(P) ->
+ fun(Input, Index) ->
+ p_scan(P, Input, Index, [])
+ end.
+-endif.
+
+-ifdef(p_one_or_more).
+-spec p_one_or_more(parse_fun()) -> parse_fun().
+p_one_or_more(P) ->
+ fun(Input, Index)->
+ Result = p_scan(P, Input, Index, []),
+ case Result of
+ {[_|_], _, _} ->
+ Result;
+ _ ->
+ {fail, {expected, Failure, _}} = P(Input,Index),
+ {fail, {expected, {at_least_one, Failure}, Index}}
+ end
+ end.
+-endif.
+
+-ifdef(p_label).
+-spec p_label(atom(), parse_fun()) -> parse_fun().
+p_label(Tag, P) ->
+ fun(Input, Index) ->
+ case P(Input, Index) of
+ {fail,_} = Failure ->
+ Failure;
+ {Result, InpRem, NewIndex} ->
+ {{Tag, Result}, InpRem, NewIndex}
+ end
+ end.
+-endif.
+
+-ifdef(p_scan).
+-spec p_scan(parse_fun(), input(), index(), [term()]) -> {[term()], input(), index()}.
+p_scan(_, <<>>, Index, Accum) -> {lists:reverse(Accum), <<>>, Index};
+p_scan(P, Inp, Index, Accum) ->
+ case P(Inp, Index) of
+ {fail,_} -> {lists:reverse(Accum), Inp, Index};
+ {Result, InpRem, NewIndex} -> p_scan(P, InpRem, NewIndex, [Result | Accum])
+ end.
+-endif.
+
+-ifdef(p_string).
+-spec p_string(binary()) -> parse_fun().
+p_string(S) ->
+ Length = erlang:byte_size(S),
+ fun(Input, Index) ->
+ try
+ <<S:Length/binary, Rest/binary>> = Input,
+ {S, Rest, p_advance_index(S, Index)}
+ catch
+ error:{badmatch,_} -> {fail, {expected, {string, S}, Index}}
+ end
+ end.
+-endif.
+
+-ifdef(p_anything).
+-spec p_anything() -> parse_fun().
+p_anything() ->
+ fun(<<>>, Index) -> {fail, {expected, any_character, Index}};
+ (Input, Index) when is_binary(Input) ->
+ <<C/utf8, Rest/binary>> = Input,
+ {<<C/utf8>>, Rest, p_advance_index(<<C/utf8>>, Index)}
+ end.
+-endif.
+
+-ifdef(p_charclass).
+-spec p_charclass(string() | binary()) -> parse_fun().
+p_charclass(Class) ->
+ {ok, RE} = re:compile(Class, [unicode, dotall]),
+ fun(Inp, Index) ->
+ case re:run(Inp, RE, [anchored]) of
+ {match, [{0, Length}|_]} ->
+ {Head, Tail} = erlang:split_binary(Inp, Length),
+ {Head, Tail, p_advance_index(Head, Index)};
+ _ -> {fail, {expected, {character_class, binary_to_list(Class)}, Index}}
+ end
+ end.
+-endif.
+
+-ifdef(p_regexp).
+-spec p_regexp(binary()) -> parse_fun().
+p_regexp(Regexp) ->
+ {ok, RE} = re:compile(Regexp, [unicode, dotall, anchored]),
+ fun(Inp, Index) ->
+ case re:run(Inp, RE) of
+ {match, [{0, Length}|_]} ->
+ {Head, Tail} = erlang:split_binary(Inp, Length),
+ {Head, Tail, p_advance_index(Head, Index)};
+ _ -> {fail, {expected, {regexp, binary_to_list(Regexp)}, Index}}
+ end
+ end.
+-endif.
+
+-ifdef(line).
+-spec line(index() | term()) -> pos_integer() | undefined.
+line({{line,L},_}) -> L;
+line(_) -> undefined.
+-endif.
+
+-ifdef(column).
+-spec column(index() | term()) -> pos_integer() | undefined.
+column({_,{column,C}}) -> C;
+column(_) -> undefined.
+-endif.
+
+-spec p_advance_index(input() | unicode:charlist() | pos_integer(), index()) -> index().
+p_advance_index(MatchedInput, Index) when is_list(MatchedInput) orelse is_binary(MatchedInput)-> % strings
+ lists:foldl(fun p_advance_index/2, Index, unicode:characters_to_list(MatchedInput));
+p_advance_index(MatchedInput, Index) when is_integer(MatchedInput) -> % single characters
+ {{line, Line}, {column, Col}} = Index,
+ case MatchedInput of
+ $\n -> {{line, Line+1}, {column, 1}};
+ _ -> {{line, Line}, {column, Col+1}}
+ end.
diff --git a/deps/rabbit_common/src/rabbit_ssl_options.erl b/deps/rabbit_common/src/rabbit_ssl_options.erl
new file mode 100644
index 0000000000..4c2967df97
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_ssl_options.erl
@@ -0,0 +1,86 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_ssl_options).
+
+-export([fix/1]).
+
+
+-define(BAD_SSL_PROTOCOL_VERSIONS, [
+ %% POODLE
+ sslv3
+ ]).
+
+-spec fix(rabbit_types:infos()) -> rabbit_types:infos().
+
+fix(Config) ->
+ fix_verify_fun(fix_ssl_protocol_versions(Config)).
+
+fix_verify_fun(SslOptsConfig) ->
+ %% Starting with ssl 4.0.1 in Erlang R14B, the verify_fun function
+ %% takes 3 arguments and returns a tuple.
+ case rabbit_misc:pget(verify_fun, SslOptsConfig) of
+ {Module, Function, InitialUserState} ->
+ Fun = make_verify_fun(Module, Function, InitialUserState),
+ rabbit_misc:pset(verify_fun, Fun, SslOptsConfig);
+ {Module, Function} when is_atom(Module) ->
+ Fun = make_verify_fun(Module, Function, none),
+ rabbit_misc:pset(verify_fun, Fun, SslOptsConfig);
+ {Verifyfun, _InitialUserState} when is_function(Verifyfun, 3) ->
+ SslOptsConfig;
+ undefined ->
+ SslOptsConfig
+ end.
+
+make_verify_fun(Module, Function, InitialUserState) ->
+ try
+ %% Preload the module: it is required to use
+ %% erlang:function_exported/3.
+ Module:module_info()
+ catch
+ _:Exception ->
+ rabbit_log:error("SSL verify_fun: module ~s missing: ~p~n",
+ [Module, Exception]),
+ throw({error, {invalid_verify_fun, missing_module}})
+ end,
+ NewForm = erlang:function_exported(Module, Function, 3),
+ OldForm = erlang:function_exported(Module, Function, 1),
+ case {NewForm, OldForm} of
+ {true, _} ->
+ %% This verify_fun is supported by Erlang R14B+ (ssl
+ %% 4.0.1 and later).
+ Fun = fun(OtpCert, Event, UserState) ->
+ Module:Function(OtpCert, Event, UserState)
+ end,
+ {Fun, InitialUserState};
+ {_, true} ->
+ %% This verify_fun is supported by Erlang R14B+ for
+ %% undocumented backward compatibility.
+ %%
+ %% InitialUserState is ignored in this case.
+ fun(Args) ->
+ Module:Function(Args)
+ end;
+ _ ->
+ rabbit_log:error("SSL verify_fun: no ~s:~s/3 exported~n",
+ [Module, Function]),
+ throw({error, {invalid_verify_fun, function_not_exported}})
+ end.
+
+fix_ssl_protocol_versions(Config) ->
+ case application:get_env(rabbit, ssl_allow_poodle_attack) of
+ {ok, true} ->
+ Config;
+ _ ->
+ Configured = case rabbit_misc:pget(versions, Config) of
+ undefined -> rabbit_misc:pget(available,
+ ssl:versions(),
+ []);
+ Vs -> Vs
+ end,
+ rabbit_misc:pset(versions, Configured -- ?BAD_SSL_PROTOCOL_VERSIONS, Config)
+ end.
diff --git a/deps/rabbit_common/src/rabbit_types.erl b/deps/rabbit_common/src/rabbit_types.erl
new file mode 100644
index 0000000000..c11004fdf4
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_types.erl
@@ -0,0 +1,196 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_types).
+
+-include("rabbit.hrl").
+
+-export_type([maybe/1, info/0, infos/0, info_key/0, info_keys/0,
+ message/0, msg_id/0, basic_message/0,
+ delivery/0, content/0, decoded_content/0, undecoded_content/0,
+ unencoded_content/0, encoded_content/0, message_properties/0,
+ vhost/0, ctag/0, amqp_error/0, r/1, r2/2, r3/3, listener/0,
+ binding/0, binding_source/0, binding_destination/0,
+ exchange/0,
+ connection/0, connection_name/0, channel/0, channel_name/0,
+ protocol/0, auth_user/0, user/0,
+ username/0, password/0, password_hash/0,
+ ok/1, error/1, error/2, ok_or_error/1, ok_or_error2/2, ok_pid_or_error/0,
+ channel_exit/0, connection_exit/0, mfargs/0, proc_name/0,
+ proc_type_and_name/0, timestamp/0, tracked_connection_id/0,
+ tracked_connection/0, tracked_channel_id/0, tracked_channel/0,
+ node_type/0, topic_access_context/0,
+ authz_data/0, authz_context/0]).
+
+-type(maybe(T) :: T | 'none').
+-type(timestamp() :: {non_neg_integer(), non_neg_integer(), non_neg_integer()}).
+
+-type(vhost() :: vhost:name()).
+-type(ctag() :: binary()).
+
+%% TODO: make this more precise by tying specific class_ids to
+%% specific properties
+-type(undecoded_content() ::
+ #content{class_id :: rabbit_framing:amqp_class_id(),
+ properties :: 'none',
+ properties_bin :: binary(),
+ payload_fragments_rev :: [binary()]} |
+ #content{class_id :: rabbit_framing:amqp_class_id(),
+ properties :: rabbit_framing:amqp_property_record(),
+ properties_bin :: 'none',
+ payload_fragments_rev :: [binary()]}).
+-type(unencoded_content() :: undecoded_content()).
+-type(decoded_content() ::
+ #content{class_id :: rabbit_framing:amqp_class_id(),
+ properties :: rabbit_framing:amqp_property_record(),
+ properties_bin :: maybe(binary()),
+ payload_fragments_rev :: [binary()]}).
+-type(encoded_content() ::
+ #content{class_id :: rabbit_framing:amqp_class_id(),
+ properties :: maybe(rabbit_framing:amqp_property_record()),
+ properties_bin :: binary(),
+ payload_fragments_rev :: [binary()]}).
+-type(content() :: undecoded_content() | decoded_content()).
+-type(msg_id() :: rabbit_guid:guid()).
+-type(basic_message() ::
+ #basic_message{exchange_name :: rabbit_exchange:name(),
+ routing_keys :: [rabbit_router:routing_key()],
+ content :: content(),
+ id :: msg_id(),
+ is_persistent :: boolean()}).
+-type(message() :: basic_message()).
+-type(delivery() ::
+ #delivery{mandatory :: boolean(),
+ sender :: pid(),
+ message :: message()}).
+-type(message_properties() ::
+ #message_properties{expiry :: pos_integer() | 'undefined',
+ needs_confirming :: boolean()}).
+
+-type(info_key() :: atom()).
+-type(info_keys() :: [info_key()]).
+
+-type(info() :: {info_key(), any()}).
+-type(infos() :: [info()]).
+
+-type(amqp_error() ::
+ #amqp_error{name :: rabbit_framing:amqp_exception(),
+ explanation :: string(),
+ method :: rabbit_framing:amqp_method_name()}).
+
+-type(r(Kind) ::
+ r2(vhost(), Kind)).
+-type(r2(VirtualHost, Kind) ::
+ r3(VirtualHost, Kind, rabbit_misc:resource_name())).
+-type(r3(VirtualHost, Kind, Name) ::
+ #resource{virtual_host :: VirtualHost,
+ kind :: Kind,
+ name :: Name}).
+
+-type(listener() ::
+ #listener{node :: node(),
+ protocol :: atom(),
+ host :: rabbit_net:hostname(),
+ port :: rabbit_net:ip_port()}).
+
+-type(binding_source() :: rabbit_exchange:name()).
+-type(binding_destination() :: rabbit_amqqueue:name() | rabbit_exchange:name()).
+
+-type(binding() ::
+ #binding{source :: rabbit_exchange:name(),
+ destination :: binding_destination(),
+ key :: rabbit_binding:key(),
+ args :: rabbit_framing:amqp_table()}).
+
+-type(exchange() ::
+ #exchange{name :: rabbit_exchange:name(),
+ type :: rabbit_exchange:type(),
+ durable :: boolean(),
+ auto_delete :: boolean(),
+ arguments :: rabbit_framing:amqp_table()}).
+
+-type(connection_name() :: binary()).
+
+%% used e.g. by rabbit_networking
+-type(connection() :: pid()).
+
+%% used e.g. by rabbit_connection_tracking
+-type(tracked_connection_id() :: {node(), connection_name()}).
+
+-type(tracked_connection() ::
+ #tracked_connection{id :: tracked_connection_id(),
+ node :: node(),
+ vhost :: vhost(),
+ name :: connection_name(),
+ pid :: connection(),
+ protocol :: protocol_name(),
+ peer_host :: rabbit_networking:hostname(),
+ peer_port :: rabbit_networking:ip_port(),
+ username :: username(),
+ connected_at :: integer()}).
+
+-type(channel_name() :: binary()).
+
+-type(channel() :: pid()).
+
+%% used e.g. by rabbit_channel_tracking
+-type(tracked_channel_id() :: {node(), channel_name()}).
+
+-type(tracked_channel() ::
+ #tracked_channel{ id :: tracked_channel_id(),
+ node :: node(),
+ vhost :: vhost(),
+ name :: channel_name(),
+ pid :: channel(),
+ username :: username(),
+ connection :: connection()}).
+
+%% old AMQP 0-9-1-centric type, avoid when possible
+-type(protocol() :: rabbit_framing:protocol()).
+
+-type(protocol_name() :: 'amqp0_8' | 'amqp0_9_1' | 'amqp1_0' | 'mqtt' | 'stomp' | any()).
+
+-type(node_type() :: 'disc' | 'ram').
+
+-type(auth_user() ::
+ #auth_user{username :: username(),
+ tags :: [atom()],
+ impl :: any()}).
+
+-type(authz_data() ::
+ #{peeraddr := inet:ip_address() | binary(),
+ _ => _ } | undefined).
+
+-type(user() ::
+ #user{username :: username(),
+ tags :: [atom()],
+ authz_backends :: [{atom(), any()}]}).
+
+-type(username() :: binary()).
+-type(password() :: binary()).
+-type(password_hash() :: binary()).
+
+-type(ok(A) :: {'ok', A}).
+-type(error(A) :: {'error', A}).
+-type(error(A, B) :: {'error', A, B}).
+-type(ok_or_error(A) :: 'ok' | error(A)).
+-type(ok_or_error2(A, B) :: ok(A) | error(B)).
+-type(ok_pid_or_error() :: ok_or_error2(pid(), any())).
+
+-type(channel_exit() :: no_return()).
+-type(connection_exit() :: no_return()).
+
+-type(mfargs() :: {atom(), atom(), [any()]}).
+
+-type(proc_name() :: term()).
+-type(proc_type_and_name() :: {atom(), proc_name()}).
+
+-type(topic_access_context() :: #{routing_key => rabbit_router:routing_key(),
+ variable_map => map(),
+ _ => _}).
+
+-type(authz_context() :: map()).
diff --git a/deps/rabbit_common/src/rabbit_writer.erl b/deps/rabbit_common/src/rabbit_writer.erl
new file mode 100644
index 0000000000..5bce50c87a
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_writer.erl
@@ -0,0 +1,437 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(rabbit_writer).
+
+%% This module backs writer processes ("writers"). The responsibility of
+%% a writer is to serialise protocol methods and write them to the socket.
+%% Every writer is associated with a channel and normally it's the channel
+%% that delegates method delivery to it. However, rabbit_reader
+%% (connection process) can use this module's functions to send data
+%% on channel 0, which is only used for connection negotiation and
+%% other "special" purposes.
+%%
+%% This module provides multiple functions that send protocol commands,
+%% including some that are credit flow-aware.
+%%
+%% Writers perform internal buffering. When the amount of data
+%% buffered exceeds a threshold, a socket flush is performed.
+%% See FLUSH_THRESHOLD for details.
+%%
+%% When a socket write fails, writer will exit.
+
+-include("rabbit.hrl").
+-include("rabbit_framing.hrl").
+
+-export([start/6, start_link/6, start/7, start_link/7, start/8, start_link/8]).
+
+-export([system_continue/3, system_terminate/4, system_code_change/4]).
+
+-export([send_command/2, send_command/3,
+ send_command_sync/2, send_command_sync/3,
+ send_command_and_notify/4, send_command_and_notify/5,
+ send_command_flow/2, send_command_flow/3,
+ flush/1]).
+-export([internal_send_command/4, internal_send_command/6]).
+-export([msg_size/1, maybe_gc_large_msg/1, maybe_gc_large_msg/2]).
+
+%% internal
+-export([enter_mainloop/2, mainloop/2, mainloop1/2]).
+
+-record(wstate, {
+ %% socket (port)
+ sock,
+ %% channel number
+ channel,
+ %% connection-negotiated frame_max setting
+ frame_max,
+ %% see #connection.protocol in rabbit_reader
+ protocol,
+ %% connection (rabbit_reader) process
+ reader,
+ %% statistics emission timer
+ stats_timer,
+ %% data pending delivery (between socket
+ %% flushes)
+ pending,
+ %% defines how ofter gc will be executed
+ writer_gc_threshold
+}).
+
+-define(HIBERNATE_AFTER, 5000).
+%% 1GB
+-define(DEFAULT_GC_THRESHOLD, 1000000000).
+
+%%---------------------------------------------------------------------------
+
+-spec start
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ non_neg_integer(), rabbit_types:protocol(), pid(),
+ rabbit_types:proc_name()) ->
+ rabbit_types:ok(pid()).
+-spec start_link
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ non_neg_integer(), rabbit_types:protocol(), pid(),
+ rabbit_types:proc_name()) ->
+ rabbit_types:ok(pid()).
+-spec start
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ non_neg_integer(), rabbit_types:protocol(), pid(),
+ rabbit_types:proc_name(), boolean()) ->
+ rabbit_types:ok(pid()).
+-spec start_link
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ non_neg_integer(), rabbit_types:protocol(), pid(),
+ rabbit_types:proc_name(), boolean()) ->
+ rabbit_types:ok(pid()).
+-spec start
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ non_neg_integer(), rabbit_types:protocol(), pid(),
+ rabbit_types:proc_name(), boolean(), undefined|non_neg_integer()) ->
+ rabbit_types:ok(pid()).
+-spec start_link
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ non_neg_integer(), rabbit_types:protocol(), pid(),
+ rabbit_types:proc_name(), boolean(), undefined|non_neg_integer()) ->
+ rabbit_types:ok(pid()).
+
+-spec system_code_change(_,_,_,_) -> {'ok',_}.
+-spec system_continue(_,_,#wstate{}) -> any().
+-spec system_terminate(_,_,_,_) -> no_return().
+
+-spec send_command(pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
+-spec send_command
+ (pid(), rabbit_framing:amqp_method_record(), rabbit_types:content()) ->
+ 'ok'.
+-spec send_command_sync(pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
+-spec send_command_sync
+ (pid(), rabbit_framing:amqp_method_record(), rabbit_types:content()) ->
+ 'ok'.
+-spec send_command_and_notify
+ (pid(), pid(), pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
+-spec send_command_and_notify
+ (pid(), pid(), pid(), rabbit_framing:amqp_method_record(),
+ rabbit_types:content()) ->
+ 'ok'.
+-spec send_command_flow(pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
+-spec send_command_flow
+ (pid(), rabbit_framing:amqp_method_record(), rabbit_types:content()) ->
+ 'ok'.
+-spec flush(pid()) -> 'ok'.
+-spec internal_send_command
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ rabbit_framing:amqp_method_record(), rabbit_types:protocol()) ->
+ 'ok'.
+-spec internal_send_command
+ (rabbit_net:socket(), rabbit_channel:channel_number(),
+ rabbit_framing:amqp_method_record(), rabbit_types:content(),
+ non_neg_integer(), rabbit_types:protocol()) ->
+ 'ok'.
+
+-spec msg_size
+ (rabbit_types:content() | rabbit_types:message()) -> non_neg_integer().
+
+-spec maybe_gc_large_msg
+ (rabbit_types:content() | rabbit_types:message()) -> non_neg_integer().
+-spec maybe_gc_large_msg
+ (rabbit_types:content() | rabbit_types:message(),
+ undefined | non_neg_integer()) -> undefined | non_neg_integer().
+
+%%---------------------------------------------------------------------------
+
+start(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity) ->
+ start(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity, false).
+
+start_link(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity) ->
+ start_link(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity, false).
+
+start(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity,
+ ReaderWantsStats) ->
+ start(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity,
+ ReaderWantsStats, ?DEFAULT_GC_THRESHOLD).
+
+start_link(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity,
+ ReaderWantsStats) ->
+ start_link(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity,
+ ReaderWantsStats, ?DEFAULT_GC_THRESHOLD).
+
+start(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity,
+ ReaderWantsStats, GCThreshold) ->
+ State = initial_state(Sock, Channel, FrameMax, Protocol, ReaderPid,
+ ReaderWantsStats, GCThreshold),
+ {ok, proc_lib:spawn(?MODULE, enter_mainloop, [Identity, State])}.
+
+start_link(Sock, Channel, FrameMax, Protocol, ReaderPid, Identity,
+ ReaderWantsStats, GCThreshold) ->
+ State = initial_state(Sock, Channel, FrameMax, Protocol, ReaderPid,
+ ReaderWantsStats, GCThreshold),
+ {ok, proc_lib:spawn_link(?MODULE, enter_mainloop, [Identity, State])}.
+
+initial_state(Sock, Channel, FrameMax, Protocol, ReaderPid, ReaderWantsStats, GCThreshold) ->
+ (case ReaderWantsStats of
+ true -> fun rabbit_event:init_stats_timer/2;
+ false -> fun rabbit_event:init_disabled_stats_timer/2
+ end)(#wstate{sock = Sock,
+ channel = Channel,
+ frame_max = FrameMax,
+ protocol = Protocol,
+ reader = ReaderPid,
+ pending = [],
+ writer_gc_threshold = GCThreshold},
+ #wstate.stats_timer).
+
+system_continue(Parent, Deb, State) ->
+ mainloop(Deb, State#wstate{reader = Parent}).
+
+system_terminate(Reason, _Parent, _Deb, _State) ->
+ exit(Reason).
+
+system_code_change(Misc, _Module, _OldVsn, _Extra) ->
+ {ok, Misc}.
+
+enter_mainloop(Identity, State) ->
+ ?LG_PROCESS_TYPE(writer),
+ Deb = sys:debug_options([]),
+ ?store_proc_name(Identity),
+ mainloop(Deb, State).
+
+mainloop(Deb, State) ->
+ try
+ mainloop1(Deb, State)
+ catch
+ exit:Error -> #wstate{reader = ReaderPid, channel = Channel} = State,
+ ReaderPid ! {channel_exit, Channel, Error}
+ end,
+ done.
+
+mainloop1(Deb, State = #wstate{pending = []}) ->
+ receive
+ Message -> {Deb1, State1} = handle_message(Deb, Message, State),
+ ?MODULE:mainloop1(Deb1, State1)
+ after ?HIBERNATE_AFTER ->
+ erlang:hibernate(?MODULE, mainloop, [Deb, State])
+ end;
+mainloop1(Deb, State) ->
+ receive
+ Message -> {Deb1, State1} = handle_message(Deb, Message, State),
+ ?MODULE:mainloop1(Deb1, State1)
+ after 0 ->
+ ?MODULE:mainloop1(Deb, internal_flush(State))
+ end.
+
+handle_message(Deb, {system, From, Req}, State = #wstate{reader = Parent}) ->
+ sys:handle_system_msg(Req, From, Parent, ?MODULE, Deb, State);
+handle_message(Deb, Message, State) ->
+ {Deb, handle_message(Message, State)}.
+
+handle_message({send_command, MethodRecord}, State) ->
+ internal_send_command_async(MethodRecord, State);
+handle_message({send_command, MethodRecord, Content}, State) ->
+ internal_send_command_async(MethodRecord, Content, State);
+handle_message({send_command_flow, MethodRecord, Sender}, State) ->
+ credit_flow:ack(Sender),
+ internal_send_command_async(MethodRecord, State);
+handle_message({send_command_flow, MethodRecord, Content, Sender}, State) ->
+ credit_flow:ack(Sender),
+ internal_send_command_async(MethodRecord, Content, State);
+handle_message({'$gen_call', From, {send_command_sync, MethodRecord}}, State) ->
+ State1 = internal_flush(
+ internal_send_command_async(MethodRecord, State)),
+ gen_server:reply(From, ok),
+ State1;
+handle_message({'$gen_call', From, {send_command_sync, MethodRecord, Content}},
+ State) ->
+ State1 = internal_flush(
+ internal_send_command_async(MethodRecord, Content, State)),
+ gen_server:reply(From, ok),
+ State1;
+handle_message({'$gen_call', From, flush}, State) ->
+ State1 = internal_flush(State),
+ gen_server:reply(From, ok),
+ State1;
+handle_message({send_command_and_notify, QPid, ChPid, MethodRecord}, State) ->
+ State1 = internal_send_command_async(MethodRecord, State),
+ rabbit_amqqueue_common:notify_sent(QPid, ChPid),
+ State1;
+handle_message({send_command_and_notify, QPid, ChPid, MethodRecord, Content},
+ State) ->
+ State1 = internal_send_command_async(MethodRecord, Content, State),
+ rabbit_amqqueue_common:notify_sent(QPid, ChPid),
+ State1;
+handle_message({'DOWN', _MRef, process, QPid, _Reason}, State) ->
+ rabbit_amqqueue_common:notify_sent_queue_down(QPid),
+ State;
+handle_message({inet_reply, _, ok}, State) ->
+ rabbit_event:ensure_stats_timer(State, #wstate.stats_timer, emit_stats);
+handle_message({inet_reply, _, Status}, _State) ->
+ exit({writer, send_failed, Status});
+handle_message(emit_stats, State = #wstate{reader = ReaderPid}) ->
+ ReaderPid ! ensure_stats,
+ rabbit_event:reset_stats_timer(State, #wstate.stats_timer);
+handle_message(Message, _State) ->
+ exit({writer, message_not_understood, Message}).
+
+%%---------------------------------------------------------------------------
+
+send_command(W, MethodRecord) ->
+ W ! {send_command, MethodRecord},
+ ok.
+
+send_command(W, MethodRecord, Content) ->
+ W ! {send_command, MethodRecord, Content},
+ ok.
+
+send_command_flow(W, MethodRecord) ->
+ credit_flow:send(W),
+ W ! {send_command_flow, MethodRecord, self()},
+ ok.
+
+send_command_flow(W, MethodRecord, Content) ->
+ credit_flow:send(W),
+ W ! {send_command_flow, MethodRecord, Content, self()},
+ ok.
+
+send_command_sync(W, MethodRecord) ->
+ call(W, {send_command_sync, MethodRecord}).
+
+send_command_sync(W, MethodRecord, Content) ->
+ call(W, {send_command_sync, MethodRecord, Content}).
+
+send_command_and_notify(W, Q, ChPid, MethodRecord) ->
+ W ! {send_command_and_notify, Q, ChPid, MethodRecord},
+ ok.
+
+send_command_and_notify(W, Q, ChPid, MethodRecord, Content) ->
+ W ! {send_command_and_notify, Q, ChPid, MethodRecord, Content},
+ ok.
+
+flush(W) -> call(W, flush).
+
+%%---------------------------------------------------------------------------
+
+call(Pid, Msg) ->
+ {ok, Res} = gen:call(Pid, '$gen_call', Msg, infinity),
+ Res.
+
+%%---------------------------------------------------------------------------
+
+assemble_frame(Channel, MethodRecord, Protocol) ->
+ rabbit_binary_generator:build_simple_method_frame(
+ Channel, MethodRecord, Protocol).
+
+assemble_frames(Channel, MethodRecord, Content, FrameMax, Protocol) ->
+ MethodName = rabbit_misc:method_record_type(MethodRecord),
+ true = Protocol:method_has_content(MethodName), % assertion
+ MethodFrame = rabbit_binary_generator:build_simple_method_frame(
+ Channel, MethodRecord, Protocol),
+ ContentFrames = rabbit_binary_generator:build_simple_content_frames(
+ Channel, Content, FrameMax, Protocol),
+ [MethodFrame | ContentFrames].
+
+tcp_send(Sock, Data) ->
+ rabbit_misc:throw_on_error(inet_error,
+ fun () -> rabbit_net:send(Sock, Data) end).
+
+internal_send_command(Sock, Channel, MethodRecord, Protocol) ->
+ ok = tcp_send(Sock, assemble_frame(Channel, MethodRecord, Protocol)).
+
+internal_send_command(Sock, Channel, MethodRecord, Content, FrameMax,
+ Protocol) ->
+ ok = lists:foldl(fun (Frame, ok) -> tcp_send(Sock, Frame);
+ (_Frame, Other) -> Other
+ end, ok, assemble_frames(Channel, MethodRecord,
+ Content, FrameMax, Protocol)).
+
+internal_send_command_async(MethodRecord,
+ State = #wstate{channel = Channel,
+ protocol = Protocol,
+ pending = Pending}) ->
+ Frame = assemble_frame(Channel, MethodRecord, Protocol),
+ maybe_flush(State#wstate{pending = [Frame | Pending]}).
+
+internal_send_command_async(MethodRecord, Content,
+ State = #wstate{channel = Channel,
+ frame_max = FrameMax,
+ protocol = Protocol,
+ pending = Pending,
+ writer_gc_threshold = GCThreshold}) ->
+ Frames = assemble_frames(Channel, MethodRecord, Content, FrameMax,
+ Protocol),
+ maybe_gc_large_msg(Content, GCThreshold),
+ maybe_flush(State#wstate{pending = [Frames | Pending]}).
+
+%% When the amount of protocol method data buffered exceeds
+%% this threshold, a socket flush is performed.
+%%
+%% This magic number is the tcp-over-ethernet MSS (1460) minus the
+%% minimum size of a AMQP 0-9-1 basic.deliver method frame (24) plus basic
+%% content header (22). The idea is that we want to flush just before
+%% exceeding the MSS.
+-define(FLUSH_THRESHOLD, 1414).
+
+maybe_flush(State = #wstate{pending = Pending}) ->
+ case iolist_size(Pending) >= ?FLUSH_THRESHOLD of
+ true -> internal_flush(State);
+ false -> State
+ end.
+
+internal_flush(State = #wstate{pending = []}) ->
+ State;
+internal_flush(State = #wstate{sock = Sock, pending = Pending}) ->
+ ok = port_cmd(Sock, lists:reverse(Pending)),
+ State#wstate{pending = []}.
+
+%% gen_tcp:send/2 does a selective receive of {inet_reply, Sock,
+%% Status} to obtain the result. That is bad when it is called from
+%% the writer since it requires scanning of the writers possibly quite
+%% large message queue.
+%%
+%% So instead we lift the code from prim_inet:send/2, which is what
+%% gen_tcp:send/2 calls, do the first half here and then just process
+%% the result code in handle_message/2 as and when it arrives.
+%%
+%% This means we may end up happily sending data down a closed/broken
+%% socket, but that's ok since a) data in the buffers will be lost in
+%% any case (so qualitatively we are no worse off than if we used
+%% gen_tcp:send/2), and b) we do detect the changed socket status
+%% eventually, i.e. when we get round to handling the result code.
+%%
+%% Also note that the port has bounded buffers and port_command blocks
+%% when these are full. So the fact that we process the result
+%% asynchronously does not impact flow control.
+port_cmd(Sock, Data) ->
+ true = try rabbit_net:port_command(Sock, Data)
+ catch error:Error -> exit({writer, send_failed, Error})
+ end,
+ ok.
+
+%% Some processes (channel, writer) can get huge amounts of binary
+%% garbage when processing huge messages at high speed (since we only
+%% do enough reductions to GC every few hundred messages, and if each
+%% message is 1MB then that's ugly). So count how many bytes of
+%% message we have processed, and force a GC every so often.
+maybe_gc_large_msg(Content) ->
+ maybe_gc_large_msg(Content, ?DEFAULT_GC_THRESHOLD).
+
+maybe_gc_large_msg(_Content, undefined) ->
+ undefined;
+maybe_gc_large_msg(Content, GCThreshold) ->
+ Size = msg_size(Content),
+ Current = case get(msg_size_for_gc) of
+ undefined -> 0;
+ C -> C
+ end,
+ New = Current + Size,
+ put(msg_size_for_gc, case New > GCThreshold of
+ true -> erlang:garbage_collect(),
+ 0;
+ false -> New
+ end),
+ Size.
+
+msg_size(#content{payload_fragments_rev = PFR}) -> iolist_size(PFR);
+msg_size(#basic_message{content = Content}) -> msg_size(Content).
diff --git a/deps/rabbit_common/src/supervisor2.erl b/deps/rabbit_common/src/supervisor2.erl
new file mode 100644
index 0000000000..08c764d0d8
--- /dev/null
+++ b/deps/rabbit_common/src/supervisor2.erl
@@ -0,0 +1,1651 @@
+%% This file is a copy of supervisor.erl from the Erlang/OTP
+%% distribution, with the following modifications:
+%%
+%% 1) the module name is supervisor2
+%%
+%% 2) a find_child/2 utility function has been added
+%%
+%% 3) Added an 'intrinsic' restart type. Like the transient type, this
+%% type means the child should only be restarted if the child exits
+%% abnormally. Unlike the transient type, if the child exits
+%% normally, the supervisor itself also exits normally. If the
+%% child is a supervisor and it exits normally (i.e. with reason of
+%% 'shutdown') then the child's parent also exits normally.
+%%
+%% 4) child specifications can contain, as the restart type, a tuple
+%% {permanent, Delay} | {transient, Delay} | {intrinsic, Delay}
+%% where Delay >= 0 (see point (4) below for intrinsic). The delay,
+%% in seconds, indicates what should happen if a child, upon being
+%% restarted, exceeds the MaxT and MaxR parameters. Thus, if a
+%% child exits, it is restarted as normal. If it exits sufficiently
+%% quickly and often to exceed the boundaries set by the MaxT and
+%% MaxR parameters, and a Delay is specified, then rather than
+%% stopping the supervisor, the supervisor instead continues and
+%% tries to start up the child again, Delay seconds later.
+%%
+%% Note that if a child is delay-restarted this will reset the
+%% count of restarts towrds MaxR and MaxT. This matters if MaxT >
+%% Delay, since otherwise we would fail to restart after the delay.
+%%
+%% Sometimes, you may wish for a transient or intrinsic child to
+%% exit abnormally so that it gets restarted, but still log
+%% nothing. gen_server will log any exit reason other than
+%% 'normal', 'shutdown' or {'shutdown', _}. Thus the exit reason of
+%% {'shutdown', 'restart'} is interpreted to mean you wish the
+%% child to be restarted according to the delay parameters, but
+%% gen_server will not log the error. Thus from gen_server's
+%% perspective it's a normal exit, whilst from supervisor's
+%% perspective, it's an abnormal exit.
+%%
+%% 5) normal, and {shutdown, _} exit reasons are all treated the same
+%% (i.e. are regarded as normal exits)
+%%
+%% All modifications are (C) 2010-2020 VMware, Inc. or its affiliates.
+%%
+%% %CopyrightBegin%
+%%
+%% Copyright Ericsson AB 1996-2018. 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.
+%%
+%% %CopyrightEnd%
+%%
+-module(supervisor2).
+
+-behaviour(gen_server).
+
+%% External exports
+-export([start_link/2, start_link/3,
+ start_child/2, restart_child/2,
+ delete_child/2, terminate_child/2,
+ which_children/1, count_children/1,
+ check_childspecs/1, get_childspec/2,
+ find_child/2]).
+
+%% Internal exports
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3, format_status/2]).
+
+%% For release_handler only
+-export([get_callback_module/1]).
+
+-include_lib("kernel/include/logger.hrl").
+
+-define(report_error(Error, Reason, Child, SupName),
+ ?LOG_ERROR(#{label=>{supervisor,Error},
+ report=>[{supervisor,SupName},
+ {errorContext,Error},
+ {reason,Reason},
+ {offender,extract_child(Child)}]},
+ #{domain=>[otp,sasl],
+ report_cb=>fun logger:format_otp_report/1,
+ logger_formatter=>#{title=>"SUPERVISOR REPORT"},
+ error_logger=>#{tag=>error_report,
+ type=>supervisor_report}})).
+
+%%--------------------------------------------------------------------------
+
+-export_type([sup_flags/0, child_spec/0, startchild_ret/0, strategy/0]).
+
+%%--------------------------------------------------------------------------
+
+-type child() :: 'undefined' | pid().
+-type child_id() :: term().
+-type mfargs() :: {M :: module(), F :: atom(), A :: [term()] | undefined}.
+-type modules() :: [module()] | 'dynamic'.
+-type delay() :: non_neg_integer().
+-type restart() :: 'permanent' | 'transient' | 'temporary' | 'intrinsic' | {'permanent', delay()} | {'transient', delay()} | {'intrinsic', delay()}.
+-type shutdown() :: 'brutal_kill' | timeout().
+-type worker() :: 'worker' | 'supervisor'.
+-type sup_name() :: {'local', Name :: atom()}
+ | {'global', Name :: atom()}
+ | {'via', Module :: module(), Name :: any()}.
+-type sup_ref() :: (Name :: atom())
+ | {Name :: atom(), Node :: node()}
+ | {'global', Name :: atom()}
+ | {'via', Module :: module(), Name :: any()}
+ | pid().
+-type child_spec() :: #{id := child_id(), % mandatory
+ start := mfargs(), % mandatory
+ restart => restart(), % optional
+ shutdown => shutdown(), % optional
+ type => worker(), % optional
+ modules => modules()} % optional
+ | {Id :: child_id(),
+ StartFunc :: mfargs(),
+ Restart :: restart(),
+ Shutdown :: shutdown(),
+ Type :: worker(),
+ Modules :: modules()}.
+
+-type strategy() :: 'one_for_all' | 'one_for_one'
+ | 'rest_for_one' | 'simple_one_for_one'.
+
+-type sup_flags() :: #{strategy => strategy(), % optional
+ intensity => non_neg_integer(), % optional
+ period => pos_integer()} % optional
+ | {RestartStrategy :: strategy(),
+ Intensity :: non_neg_integer(),
+ Period :: pos_integer()}.
+-type children() :: {Ids :: [child_id()], Db :: #{child_id() => child_rec()}}.
+
+%%--------------------------------------------------------------------------
+%% Defaults
+-define(default_flags, #{strategy => one_for_one,
+ intensity => 1,
+ period => 5}).
+-define(default_child_spec, #{restart => permanent,
+ type => worker}).
+%% Default 'shutdown' is 5000 for workers and infinity for supervisors.
+%% Default 'modules' is [M], where M comes from the child's start {M,F,A}.
+
+%%--------------------------------------------------------------------------
+
+-record(child, {% pid is undefined when child is not running
+ pid = undefined :: child()
+ | {restarting, pid() | undefined}
+ | [pid()],
+ id :: child_id(),
+ mfargs :: mfargs(),
+ restart_type :: restart(),
+ shutdown :: shutdown(),
+ child_type :: worker(),
+ modules = [] :: modules()}).
+-type child_rec() :: #child{}.
+
+-record(state, {name,
+ strategy :: strategy() | 'undefined',
+ children = {[],#{}} :: children(), % Ids in start order
+ dynamics :: {'maps', #{pid() => list()}}
+ | {'sets', sets:set(pid())}
+ | 'undefined',
+ intensity :: non_neg_integer() | 'undefined',
+ period :: pos_integer() | 'undefined',
+ restarts = [],
+ dynamic_restarts = 0 :: non_neg_integer(),
+ module,
+ args}).
+-type state() :: #state{}.
+
+-define(is_simple(State), State#state.strategy=:=simple_one_for_one).
+-define(is_temporary(_Child_), _Child_#child.restart_type=:=temporary).
+-define(is_permanent(_Child_), ((_Child_#child.restart_type=:=permanent) orelse
+ (is_tuple(_Child_#child.restart_type) andalso
+ tuple_size(_Child_#child.restart_type) =:= 2 andalso
+ element(1, _Child_#child.restart_type) =:= permanent))).
+
+-define(is_explicit_restart(R),
+ R == {shutdown, restart}).
+
+-callback init(Args :: term()) ->
+ {ok, {SupFlags :: sup_flags(), [ChildSpec :: child_spec()]}}
+ | ignore.
+
+-define(restarting(_Pid_), {restarting,_Pid_}).
+
+%%% ---------------------------------------------------
+%%% This is a general process supervisor built upon gen_server.erl.
+%%% Servers/processes should/could also be built using gen_server.erl.
+%%% SupName = {local, atom()} | {global, atom()}.
+%%% ---------------------------------------------------
+
+-type startlink_err() :: {'already_started', pid()}
+ | {'shutdown', term()}
+ | term().
+-type startlink_ret() :: {'ok', pid()} | 'ignore' | {'error', startlink_err()}.
+
+-spec start_link(Module, Args) -> startlink_ret() when
+ Module :: module(),
+ Args :: term().
+start_link(Mod, Args) ->
+ gen_server:start_link(?MODULE, {self, Mod, Args}, []).
+
+-spec start_link(SupName, Module, Args) -> startlink_ret() when
+ SupName :: sup_name(),
+ Module :: module(),
+ Args :: term().
+start_link(SupName, Mod, Args) ->
+ gen_server:start_link(SupName, ?MODULE, {SupName, Mod, Args}, []).
+
+%%% ---------------------------------------------------
+%%% Interface functions.
+%%% ---------------------------------------------------
+
+-type startchild_err() :: 'already_present' | {'already_started', Child :: child()} | term().
+-type startchild_ret() :: {'ok', Child :: child()} | {'ok', Child :: child(), Info :: term()} | {'error', startchild_err()}.
+
+-spec start_child(SupRef, ChildSpec) -> startchild_ret() when
+ SupRef :: sup_ref(),
+ ChildSpec :: child_spec() | (List :: [term()]).
+start_child(Supervisor, ChildSpec) ->
+ call(Supervisor, {start_child, ChildSpec}).
+
+-spec restart_child(SupRef, Id) -> Result when
+ SupRef :: sup_ref(),
+ Id :: child_id(),
+ Result :: {'ok', Child :: child()}
+ | {'ok', Child :: child(), Info :: term()}
+ | {'error', Error},
+ Error :: 'running' | 'restarting' | 'not_found' | 'simple_one_for_one' | term().
+restart_child(Supervisor, Id) ->
+ call(Supervisor, {restart_child, Id}).
+
+-spec delete_child(SupRef, Id) -> Result when
+ SupRef :: sup_ref(),
+ Id :: child_id(),
+ Result :: 'ok' | {'error', Error},
+ Error :: 'running' | 'restarting' | 'not_found' | 'simple_one_for_one'.
+delete_child(Supervisor, Id) ->
+ call(Supervisor, {delete_child, Id}).
+
+%%-----------------------------------------------------------------
+%% Func: terminate_child/2
+%% Returns: ok | {error, Reason}
+%% Note that the child is *always* terminated in some
+%% way (maybe killed).
+%%-----------------------------------------------------------------
+
+-spec terminate_child(SupRef, Id) -> Result when
+ SupRef :: sup_ref(),
+ Id :: pid() | child_id(),
+ Result :: 'ok' | {'error', Error},
+ Error :: 'not_found' | 'simple_one_for_one'.
+terminate_child(Supervisor, Id) ->
+ call(Supervisor, {terminate_child, Id}).
+
+-spec get_childspec(SupRef, Id) -> Result when
+ SupRef :: sup_ref(),
+ Id :: pid() | child_id(),
+ Result :: {'ok', child_spec()} | {'error', Error},
+ Error :: 'not_found'.
+get_childspec(Supervisor, Id) ->
+ call(Supervisor, {get_childspec, Id}).
+
+-spec which_children(SupRef) -> [{Id,Child,Type,Modules}] when
+ SupRef :: sup_ref(),
+ Id :: child_id() | undefined,
+ Child :: child() | 'restarting',
+ Type :: worker(),
+ Modules :: modules().
+which_children(Supervisor) ->
+ call(Supervisor, which_children).
+
+-spec count_children(SupRef) -> PropListOfCounts when
+ SupRef :: sup_ref(),
+ PropListOfCounts :: [Count],
+ Count :: {specs, ChildSpecCount :: non_neg_integer()}
+ | {active, ActiveProcessCount :: non_neg_integer()}
+ | {supervisors, ChildSupervisorCount :: non_neg_integer()}
+ |{workers, ChildWorkerCount :: non_neg_integer()}.
+count_children(Supervisor) ->
+ call(Supervisor, count_children).
+
+-spec find_child(Supervisor, Name) -> [pid()] when
+ Supervisor :: sup_ref(),
+ Name :: child_id().
+find_child(Supervisor, Name) ->
+ [Pid || {Name1, Pid, _Type, _Modules} <- which_children(Supervisor),
+ Name1 =:= Name].
+
+call(Supervisor, Req) ->
+ gen_server:call(Supervisor, Req, infinity).
+
+-spec check_childspecs(ChildSpecs) -> Result when
+ ChildSpecs :: [child_spec()],
+ Result :: 'ok' | {'error', Error :: term()}.
+check_childspecs(ChildSpecs) when is_list(ChildSpecs) ->
+ case check_startspec(ChildSpecs) of
+ {ok, _} -> ok;
+ Error -> {error, Error}
+ end;
+check_childspecs(X) -> {error, {badarg, X}}.
+
+%%%-----------------------------------------------------------------
+%%% Called by release_handler during upgrade
+-spec get_callback_module(Pid) -> Module when
+ Pid :: pid(),
+ Module :: atom().
+get_callback_module(Pid) ->
+ {status, _Pid, {module, _Mod},
+ [_PDict, _SysState, _Parent, _Dbg, Misc]} = sys:get_status(Pid),
+ case lists:keyfind(?MODULE, 1, Misc) of
+ {?MODULE, [{"Callback", Mod}]} ->
+ Mod;
+ _ ->
+ [_Header, _Data, {data, [{"State", State}]} | _] = Misc,
+ State#state.module
+ end.
+
+%%% ---------------------------------------------------
+%%%
+%%% Initialize the supervisor.
+%%%
+%%% ---------------------------------------------------
+
+-type init_sup_name() :: sup_name() | 'self'.
+
+-type stop_rsn() :: {'shutdown', term()}
+ | {'bad_return', {module(),'init', term()}}
+ | {'bad_start_spec', term()}
+ | {'start_spec', term()}
+ | {'supervisor_data', term()}.
+
+-spec init({init_sup_name(), module(), [term()]}) ->
+ {'ok', state()} | 'ignore' | {'stop', stop_rsn()}.
+
+init({SupName, Mod, Args}) ->
+ process_flag(trap_exit, true),
+ case Mod:init(Args) of
+ {ok, {SupFlags, StartSpec}} ->
+ case init_state(SupName, SupFlags, Mod, Args) of
+ {ok, State} when ?is_simple(State) ->
+ init_dynamic(State, StartSpec);
+ {ok, State} ->
+ init_children(State, StartSpec);
+ Error ->
+ {stop, {supervisor_data, Error}}
+ end;
+ ignore ->
+ ignore;
+ Error ->
+ {stop, {bad_return, {Mod, init, Error}}}
+ end.
+
+init_children(State, StartSpec) ->
+ SupName = State#state.name,
+ case check_startspec(StartSpec) of
+ {ok, Children} ->
+ case start_children(Children, SupName) of
+ {ok, NChildren} ->
+ {ok, State#state{children = NChildren}};
+ {error, NChildren, Reason} ->
+ _ = terminate_children(NChildren, SupName),
+ {stop, {shutdown, Reason}}
+ end;
+ Error ->
+ {stop, {start_spec, Error}}
+ end.
+
+init_dynamic(State, [StartSpec]) ->
+ case check_startspec([StartSpec]) of
+ {ok, Children} ->
+ {ok, dyn_init(State#state{children = Children})};
+ Error ->
+ {stop, {start_spec, Error}}
+ end;
+init_dynamic(_State, StartSpec) ->
+ {stop, {bad_start_spec, StartSpec}}.
+
+%%-----------------------------------------------------------------
+%% Func: start_children/2
+%% Args: Children = children() % Ids in start order
+%% SupName = {local, atom()} | {global, atom()} | {pid(), Mod}
+%% Purpose: Start all children. The new map contains #child's
+%% with pids.
+%% Returns: {ok, NChildren} | {error, NChildren, Reason}
+%% NChildren = children() % Ids in termination order
+%% (reversed start order)
+%%-----------------------------------------------------------------
+start_children(Children, SupName) ->
+ Start =
+ fun(Id,Child) ->
+ case do_start_child(SupName, Child) of
+ {ok, undefined} when ?is_temporary(Child) ->
+ remove;
+ {ok, Pid} ->
+ {update,Child#child{pid = Pid}};
+ {ok, Pid, _Extra} ->
+ {update,Child#child{pid = Pid}};
+ {error, Reason} ->
+ ?report_error(start_error, Reason, Child, SupName),
+ {abort,{failed_to_start_child,Id,Reason}}
+ end
+ end,
+ children_map(Start,Children).
+
+do_start_child(SupName, Child) ->
+ #child{mfargs = {M, F, Args}} = Child,
+ case do_start_child_i(M, F, Args) of
+ {ok, Pid} when is_pid(Pid) ->
+ NChild = Child#child{pid = Pid},
+ report_progress(NChild, SupName),
+ {ok, Pid};
+ {ok, Pid, Extra} when is_pid(Pid) ->
+ NChild = Child#child{pid = Pid},
+ report_progress(NChild, SupName),
+ {ok, Pid, Extra};
+ Other ->
+ Other
+ end.
+
+do_start_child_i(M, F, A) ->
+ case catch apply(M, F, A) of
+ {ok, Pid} when is_pid(Pid) ->
+ {ok, Pid};
+ {ok, Pid, Extra} when is_pid(Pid) ->
+ {ok, Pid, Extra};
+ ignore ->
+ {ok, undefined};
+ {error, Error} ->
+ {error, Error};
+ What ->
+ {error, What}
+ end.
+
+%%% ---------------------------------------------------
+%%%
+%%% Callback functions.
+%%%
+%%% ---------------------------------------------------
+-type call() :: 'which_children' | 'count_children' | {_, _}. % XXX: refine
+-spec handle_call(call(), term(), state()) -> {'reply', term(), state()}.
+
+handle_call({start_child, EArgs}, _From, State) when ?is_simple(State) ->
+ Child = get_dynamic_child(State),
+ #child{mfargs = {M, F, A}} = Child,
+ Args = A ++ EArgs,
+ case do_start_child_i(M, F, Args) of
+ {ok, undefined} ->
+ {reply, {ok, undefined}, State};
+ {ok, Pid} ->
+ NState = dyn_store(Pid, Args, State),
+ {reply, {ok, Pid}, NState};
+ {ok, Pid, Extra} ->
+ NState = dyn_store(Pid, Args, State),
+ {reply, {ok, Pid, Extra}, NState};
+ What ->
+ {reply, What, State}
+ end;
+
+handle_call({start_child, ChildSpec}, _From, State) ->
+ case check_childspec(ChildSpec) of
+ {ok, Child} ->
+ {Resp, NState} = handle_start_child(Child, State),
+ {reply, Resp, NState};
+ What ->
+ {reply, {error, What}, State}
+ end;
+
+%% terminate_child for simple_one_for_one can only be done with pid
+handle_call({terminate_child, Id}, _From, State) when not is_pid(Id),
+ ?is_simple(State) ->
+ {reply, {error, simple_one_for_one}, State};
+
+handle_call({terminate_child, Id}, _From, State) ->
+ case internal_find_child(Id, State) of
+ {ok, Child} ->
+ do_terminate(Child, State#state.name),
+ {reply, ok, del_child(Child, State)};
+ error ->
+ {reply, {error, not_found}, State}
+ end;
+
+%% restart_child request is invalid for simple_one_for_one supervisors
+handle_call({restart_child, _Id}, _From, State) when ?is_simple(State) ->
+ {reply, {error, simple_one_for_one}, State};
+
+handle_call({restart_child, Id}, _From, State) ->
+ case internal_find_child(Id, State) of
+ {ok, Child} when Child#child.pid =:= undefined ->
+ case do_start_child(State#state.name, Child) of
+ {ok, Pid} ->
+ NState = set_pid(Pid, Id, State),
+ {reply, {ok, Pid}, NState};
+ {ok, Pid, Extra} ->
+ NState = set_pid(Pid, Id, State),
+ {reply, {ok, Pid, Extra}, NState};
+ Error ->
+ {reply, Error, State}
+ end;
+ {ok, #child{pid=?restarting(_)}} ->
+ {reply, {error, restarting}, State};
+ {ok, _} ->
+ {reply, {error, running}, State};
+ _ ->
+ {reply, {error, not_found}, State}
+ end;
+
+%% delete_child request is invalid for simple_one_for_one supervisors
+handle_call({delete_child, _Id}, _From, State) when ?is_simple(State) ->
+ {reply, {error, simple_one_for_one}, State};
+
+handle_call({delete_child, Id}, _From, State) ->
+ case internal_find_child(Id, State) of
+ {ok, Child} when Child#child.pid =:= undefined ->
+ NState = remove_child(Id, State),
+ {reply, ok, NState};
+ {ok, #child{pid=?restarting(_)}} ->
+ {reply, {error, restarting}, State};
+ {ok, _} ->
+ {reply, {error, running}, State};
+ _ ->
+ {reply, {error, not_found}, State}
+ end;
+
+handle_call({get_childspec, Id}, _From, State) ->
+ case internal_find_child(Id, State) of
+ {ok, Child} ->
+ {reply, {ok, child_to_spec(Child)}, State};
+ error ->
+ {reply, {error, not_found}, State}
+ end;
+
+handle_call(which_children, _From, State) when ?is_simple(State) ->
+ #child{child_type = CT,modules = Mods} = get_dynamic_child(State),
+ Reply = dyn_map(fun(?restarting(_)) -> {undefined, restarting, CT, Mods};
+ (Pid) -> {undefined, Pid, CT, Mods}
+ end, State),
+ {reply, Reply, State};
+
+handle_call(which_children, _From, State) ->
+ Resp =
+ children_to_list(
+ fun(Id,#child{pid = ?restarting(_),
+ child_type = ChildType, modules = Mods}) ->
+ {Id, restarting, ChildType, Mods};
+ (Id,#child{pid = Pid,
+ child_type = ChildType, modules = Mods}) ->
+ {Id, Pid, ChildType, Mods}
+ end,
+ State#state.children),
+ {reply, Resp, State};
+
+handle_call(count_children, _From, #state{dynamic_restarts = Restarts} = State)
+ when ?is_simple(State) ->
+ #child{child_type = CT} = get_dynamic_child(State),
+ Sz = dyn_size(State),
+ Active = Sz - Restarts, % Restarts is always 0 for temporary children
+ Reply = case CT of
+ supervisor -> [{specs, 1}, {active, Active},
+ {supervisors, Sz}, {workers, 0}];
+ worker -> [{specs, 1}, {active, Active},
+ {supervisors, 0}, {workers, Sz}]
+ end,
+ {reply, Reply, State};
+
+handle_call(count_children, _From, State) ->
+ %% Specs and children are together on the children list...
+ {Specs, Active, Supers, Workers} =
+ children_fold(fun(_Id, Child, Counts) ->
+ count_child(Child, Counts)
+ end, {0,0,0,0}, State#state.children),
+
+ %% Reformat counts to a property list.
+ Reply = [{specs, Specs}, {active, Active},
+ {supervisors, Supers}, {workers, Workers}],
+ {reply, Reply, State}.
+
+count_child(#child{pid = Pid, child_type = worker},
+ {Specs, Active, Supers, Workers}) ->
+ case is_pid(Pid) andalso is_process_alive(Pid) of
+ true -> {Specs+1, Active+1, Supers, Workers+1};
+ false -> {Specs+1, Active, Supers, Workers+1}
+ end;
+count_child(#child{pid = Pid, child_type = supervisor},
+ {Specs, Active, Supers, Workers}) ->
+ case is_pid(Pid) andalso is_process_alive(Pid) of
+ true -> {Specs+1, Active+1, Supers+1, Workers};
+ false -> {Specs+1, Active, Supers+1, Workers}
+ end.
+
+%%% If a restart attempt failed, this message is cast
+%%% from restart/2 in order to give gen_server the chance to
+%%% check it's inbox before trying again.
+-spec handle_cast({try_again_restart, child_id() | {'restarting',pid()}}, state()) ->
+ {'noreply', state()} | {stop, shutdown, state()}.
+
+handle_cast({try_again_restart,TryAgainId}, State) ->
+ case find_child_and_args(TryAgainId, State) of
+ {ok, Child = #child{pid=?restarting(_)}} ->
+ case restart(Child,State) of
+ {ok, State1} ->
+ {noreply, State1};
+ {shutdown, State1} ->
+ {stop, shutdown, State1}
+ end;
+ _ ->
+ {noreply,State}
+ end.
+
+%%
+%% Take care of terminated children.
+%%
+-spec handle_info(term(), state()) ->
+ {'noreply', state()} | {'stop', 'shutdown', state()}.
+
+handle_info({'EXIT', Pid, Reason}, State) ->
+ case restart_child(Pid, Reason, State) of
+ {ok, State1} ->
+ {noreply, State1};
+ {shutdown, State1} ->
+ {stop, shutdown, State1}
+ end;
+
+handle_info({delayed_restart, {Reason, Child}}, State) when ?is_simple(State) ->
+ try_restart(Reason, Child, State#state{restarts = []}); %% [1]
+handle_info({delayed_restart, {Reason, Child}}, State) ->
+ ChildId = Child#child.id,
+ case internal_find_child(ChildId, State) of
+ {ok, Child1} ->
+ try_restart(Reason, Child1, State#state{restarts = []}); %% [1]
+ _What ->
+ {noreply, State}
+ end;
+%% [1] When we receive a delayed_restart message we want to reset the
+%% restarts field since otherwise the MaxT might not have elapsed and
+%% we would just delay again and again. Since a common use of the
+%% delayed restart feature is for MaxR = 1, MaxT = some huge number
+%% (so that we don't end up bouncing around in non-delayed restarts)
+%% this is important.
+
+handle_info(Msg, State) ->
+ ?LOG_ERROR("Supervisor received unexpected message: ~tp~n",[Msg],
+ #{domain=>[otp],
+ error_logger=>#{tag=>error}}),
+ {noreply, State}.
+
+%%
+%% Terminate this server.
+%%
+-spec terminate(term(), state()) -> 'ok'.
+
+terminate(_Reason, State) when ?is_simple(State) ->
+ terminate_dynamic_children(State);
+terminate(_Reason, State) ->
+ terminate_children(State#state.children, State#state.name).
+
+%%
+%% Change code for the supervisor.
+%% Call the new call-back module and fetch the new start specification.
+%% Combine the new spec. with the old. If the new start spec. is
+%% not valid the code change will not succeed.
+%% Use the old Args as argument to Module:init/1.
+%% NOTE: This requires that the init function of the call-back module
+%% does not have any side effects.
+%%
+-spec code_change(term(), state(), term()) ->
+ {'ok', state()} | {'error', term()}.
+
+code_change(_, State, _) ->
+ case (State#state.module):init(State#state.args) of
+ {ok, {SupFlags, StartSpec}} ->
+ case set_flags(SupFlags, State) of
+ {ok, State1} ->
+ update_childspec(State1, StartSpec);
+ {invalid_type, SupFlags} ->
+ {error, {bad_flags, SupFlags}}; % backwards compatibility
+ Error ->
+ {error, Error}
+ end;
+ ignore ->
+ {ok, State};
+ Error ->
+ Error
+ end.
+
+update_childspec(State, StartSpec) when ?is_simple(State) ->
+ case check_startspec(StartSpec) of
+ {ok, {[_],_}=Children} ->
+ {ok, State#state{children = Children}};
+ Error ->
+ {error, Error}
+ end;
+update_childspec(State, StartSpec) ->
+ case check_startspec(StartSpec) of
+ {ok, Children} ->
+ OldC = State#state.children, % In reverse start order !
+ NewC = update_childspec1(OldC, Children, []),
+ {ok, State#state{children = NewC}};
+ Error ->
+ {error, Error}
+ end.
+
+update_childspec1({[Id|OldIds], OldDb}, {Ids,Db}, KeepOld) ->
+ case update_chsp(maps:get(Id,OldDb), Db) of
+ {ok,NewDb} ->
+ update_childspec1({OldIds,OldDb}, {Ids,NewDb}, KeepOld);
+ false ->
+ update_childspec1({OldIds,OldDb}, {Ids,Db}, [Id|KeepOld])
+ end;
+update_childspec1({[],OldDb}, {Ids,Db}, KeepOld) ->
+ KeepOldDb = maps:with(KeepOld,OldDb),
+ %% Return them in (kept) reverse start order.
+ {lists:reverse(Ids ++ KeepOld),maps:merge(KeepOldDb,Db)}.
+
+update_chsp(#child{id=Id}=OldChild, NewDb) ->
+ case maps:find(Id, NewDb) of
+ {ok,Child} ->
+ {ok,NewDb#{Id => Child#child{pid = OldChild#child.pid}}};
+ error -> % Id not found in new spec.
+ false
+ end.
+
+
+%%% ---------------------------------------------------
+%%% Start a new child.
+%%% ---------------------------------------------------
+
+handle_start_child(Child, State) ->
+ case internal_find_child(Child#child.id, State) of
+ error ->
+ case do_start_child(State#state.name, Child) of
+ {ok, undefined} when ?is_temporary(Child) ->
+ {{ok, undefined}, State};
+ {ok, Pid} ->
+ {{ok, Pid}, save_child(Child#child{pid = Pid}, State)};
+ {ok, Pid, Extra} ->
+ {{ok, Pid, Extra}, save_child(Child#child{pid = Pid}, State)};
+ {error, What} ->
+ {{error, {What, Child}}, State}
+ end;
+ {ok, OldChild} when is_pid(OldChild#child.pid) ->
+ {{error, {already_started, OldChild#child.pid}}, State};
+ {ok, _OldChild} ->
+ {{error, already_present}, State}
+ end.
+
+%%% ---------------------------------------------------
+%%% Restart. A process has terminated.
+%%% Returns: {ok, state()} | {shutdown, state()}
+%%% ---------------------------------------------------
+
+restart_child(Pid, Reason, State) ->
+ case find_child_and_args(Pid, State) of
+ {ok, Child} ->
+ do_restart(Reason, Child, State);
+ error ->
+ {ok, State}
+ end.
+
+try_restart(Reason, Child, State) ->
+ case do_restart(Reason, Child, State) of
+ {ok, NState} -> {noreply, NState};
+ {shutdown, State2} -> {stop, shutdown, State2}
+ end.
+
+do_restart(Reason, Child=#child{restart_type=permanent}, State) -> % is_permanent
+ ?report_error(child_terminated, Reason, Child, State#state.name),
+ restart(Child, State);
+do_restart(Reason, Child=#child{restart_type={permanent,_Delay}}, State) -> % is_permanent_delay
+ ?report_error(child_terminated, Reason, Child, State#state.name),
+ do_restart_delay(Reason, Child, State);
+do_restart(Reason, Child=#child{restart_type=transient}, State) -> % is_transient
+ maybe_report_error(Reason, Child, State),
+ restart_if_explicit_or_abnormal(fun restart/2,
+ fun delete_child_and_continue/2,
+ Reason, Child, State);
+do_restart(Reason, Child=#child{restart_type={transient,_Delay}}, State) -> % is_transient_delay
+ maybe_report_error(Reason, Child, State),
+ restart_if_explicit_or_abnormal(defer_to_restart_delay(Reason),
+ fun delete_child_and_continue/2,
+ Reason, Child, State);
+do_restart(Reason, Child=#child{restart_type=intrinsic}, State) -> % is_intrinsic
+ maybe_report_error(Reason, Child, State),
+ restart_if_explicit_or_abnormal(fun restart/2,
+ fun delete_child_and_stop/2,
+ Reason, Child, State);
+do_restart(Reason, Child=#child{restart_type={intrinsic,_Delay}}, State) -> % is_intrinsic_delay
+ maybe_report_error(Reason, Child, State),
+ restart_if_explicit_or_abnormal(defer_to_restart_delay(Reason),
+ fun delete_child_and_stop/2,
+ Reason, Child, State);
+do_restart(normal, Child, State) ->
+ NState = del_child(Child, State),
+ {ok, NState};
+do_restart(shutdown, Child, State) ->
+ NState = del_child(Child, State),
+ {ok, NState};
+do_restart({shutdown, _Term}, Child, State) ->
+ NState = del_child(Child, State),
+ {ok, NState};
+do_restart(Reason, Child, State) when ?is_temporary(Child) ->
+ ?report_error(child_terminated, Reason, Child, State#state.name),
+ NState = del_child(Child, State),
+ {ok, NState}.
+
+maybe_report_error(Reason, Child, State) ->
+ case is_abnormal_termination(Reason) of
+ true ->
+ ?report_error(child_terminated, Reason, Child, State#state.name);
+ false ->
+ ok
+ end.
+
+restart_if_explicit_or_abnormal(RestartHow, Otherwise, Reason, Child, State) ->
+ case ?is_explicit_restart(Reason) orelse is_abnormal_termination(Reason) of
+ true -> RestartHow(Child, State);
+ false -> Otherwise(Child, State)
+ end.
+
+defer_to_restart_delay(Reason) ->
+ fun(Child, State) -> do_restart_delay(Reason, Child, State) end.
+
+delete_child_and_continue(Child, State) ->
+ {ok, del_child(Child, State)}.
+
+delete_child_and_stop(Child, State) ->
+ NState = del_child(Child, State),
+ {shutdown, NState}.
+
+is_abnormal_termination(normal) -> false;
+is_abnormal_termination(shutdown) -> false;
+is_abnormal_termination({shutdown, _}) -> false;
+is_abnormal_termination(_Other) -> true.
+
+do_restart_delay(Reason,
+ Child = #child{id = ChildId,
+ pid = ChildPid0,
+ restart_type = {_RestartType, Delay}},
+ State0) ->
+ case add_restart(State0) of
+ {ok, State1} ->
+ Strategy = State1#state.strategy,
+ maybe_restart(Strategy, Child, State1);
+ {terminate, State1} ->
+ %% we've reached the max restart intensity, but the
+ %% add_restart will have added to the restarts
+ %% field. Given we don't want to die here, we need to go
+ %% back to the old restarts field otherwise we'll never
+ %% attempt to restart later, which is why we ignore
+ %% NState for this clause.
+ Msg = {delayed_restart, {Reason, Child}},
+ _TRef = erlang:send_after(trunc(Delay*1000), self(), Msg),
+ ChildPid1 = restarting(ChildPid0),
+ % Note: State0 is intentionally used here
+ % TODO LRB
+ State2 = set_pid(ChildPid1, ChildId, State1),
+ {ok, State2}
+ end.
+
+maybe_restart(Strategy, Child, State) ->
+ case restart(Strategy, Child, State) of
+ {{try_again, Reason}, NState2} ->
+ %% Leaving control back to gen_server before
+ %% trying again. This way other incoming requests
+ %% for the supervisor can be handled - e.g. a
+ %% shutdown request for the supervisor or the
+ %% child.
+ Id = if ?is_simple(State) -> Child#child.pid;
+ true -> Child#child.id
+ end,
+ Args = [self(), Id, Reason],
+ {ok, _TRef} = timer:apply_after(0, ?MODULE, try_again_restart, Args),
+ {ok, NState2};
+ Other ->
+ Other
+ end.
+
+restart(Child, State) ->
+ case add_restart(State) of
+ {ok, NState} ->
+ case restart(NState#state.strategy, Child, NState) of
+ {{try_again, TryAgainId}, NState2} ->
+ %% Leaving control back to gen_server before
+ %% trying again. This way other incoming requsts
+ %% for the supervisor can be handled - e.g. a
+ %% shutdown request for the supervisor or the
+ %% child.
+ try_again_restart(TryAgainId),
+ {ok,NState2};
+ Other ->
+ Other
+ end;
+ {terminate, NState} ->
+ ?report_error(shutdown, reached_max_restart_intensity,
+ Child, State#state.name),
+ {shutdown, del_child(Child, NState)}
+ end.
+
+restart(simple_one_for_one, Child, State0) ->
+ #child{pid = OldPid, mfargs = {M, F, A}} = Child,
+ State1 = case OldPid of
+ ?restarting(_) ->
+ NRes = State0#state.dynamic_restarts - 1,
+ State0#state{dynamic_restarts = NRes};
+ _ ->
+ State0
+ end,
+ State2 = dyn_erase(OldPid, State1),
+ case do_start_child_i(M, F, A) of
+ {ok, Pid} ->
+ NState = dyn_store(Pid, A, State2),
+ {ok, NState};
+ {ok, Pid, _Extra} ->
+ NState = dyn_store(Pid, A, State2),
+ {ok, NState};
+ {error, Error} ->
+ ROldPid = restarting(OldPid),
+ NRestarts = State2#state.dynamic_restarts + 1,
+ State3 = State2#state{dynamic_restarts = NRestarts},
+ NState = dyn_store(ROldPid, A, State3),
+ ?report_error(start_error, Error, Child, NState#state.name),
+ {{try_again, ROldPid}, NState}
+ end;
+restart(one_for_one, #child{id=Id} = Child, State) ->
+ OldPid = Child#child.pid,
+ case do_start_child(State#state.name, Child) of
+ {ok, Pid} ->
+ NState = set_pid(Pid, Id, State),
+ {ok, NState};
+ {ok, Pid, _Extra} ->
+ NState = set_pid(Pid, Id, State),
+ {ok, NState};
+ {error, Reason} ->
+ NState = set_pid(restarting(OldPid), Id, State),
+ ?report_error(start_error, Reason, Child, State#state.name),
+ {{try_again,Id}, NState}
+ end;
+restart(rest_for_one, #child{id=Id} = Child, #state{name=SupName} = State) ->
+ {ChAfter, ChBefore} = split_child(Id, State#state.children),
+ {Return, ChAfter2} = restart_multiple_children(Child, ChAfter, SupName),
+ {Return, State#state{children = append(ChAfter2,ChBefore)}};
+restart(one_for_all, Child, #state{name=SupName} = State) ->
+ Children1 = del_child(Child#child.id, State#state.children),
+ {Return, NChildren} = restart_multiple_children(Child, Children1, SupName),
+ {Return, State#state{children = NChildren}}.
+
+restart_multiple_children(Child, Children, SupName) ->
+ Children1 = terminate_children(Children, SupName),
+ case start_children(Children1, SupName) of
+ {ok, NChildren} ->
+ {ok, NChildren};
+ {error, NChildren, {failed_to_start_child, FailedId, _Reason}} ->
+ NewPid = if FailedId =:= Child#child.id ->
+ restarting(Child#child.pid);
+ true ->
+ ?restarting(undefined)
+ end,
+ {{try_again, FailedId}, set_pid(NewPid,FailedId,NChildren)}
+ end.
+
+restarting(Pid) when is_pid(Pid) -> ?restarting(Pid);
+restarting(RPid) -> RPid.
+
+-spec try_again_restart(child_id() | {'restarting',pid()}) -> 'ok'.
+try_again_restart(TryAgainId) ->
+ gen_server:cast(self(), {try_again_restart, TryAgainId}).
+
+%%-----------------------------------------------------------------
+%% Func: terminate_children/2
+%% Args: Children = children() % Ids in termination order
+%% SupName = {local, atom()} | {global, atom()} | {pid(),Mod}
+%% Returns: NChildren = children() % Ids in startup order
+%% % (reversed termination order)
+%%-----------------------------------------------------------------
+terminate_children(Children, SupName) ->
+ Terminate =
+ fun(_Id,Child) when ?is_temporary(Child) ->
+ %% Temporary children should not be restarted and thus should
+ %% be skipped when building the list of terminated children.
+ do_terminate(Child, SupName),
+ remove;
+ (_Id,Child) ->
+ do_terminate(Child, SupName),
+ {update,Child#child{pid=undefined}}
+ end,
+ {ok,NChildren} = children_map(Terminate, Children),
+ NChildren.
+
+do_terminate(Child, SupName) when is_pid(Child#child.pid) ->
+ case shutdown(Child#child.pid, Child#child.shutdown) of
+ ok ->
+ ok;
+ {error, normal} when not (?is_permanent(Child)) ->
+ ok;
+ {error, OtherReason} ->
+ ?report_error(shutdown_error, OtherReason, Child, SupName)
+ end,
+ ok;
+do_terminate(_Child, _SupName) ->
+ ok.
+
+%%-----------------------------------------------------------------
+%% Shutdowns a child. We must check the EXIT value
+%% of the child, because it might have died with another reason than
+%% the wanted. In that case we want to report the error. We put a
+%% monitor on the child an check for the 'DOWN' message instead of
+%% checking for the 'EXIT' message, because if we check the 'EXIT'
+%% message a "naughty" child, who does unlink(Sup), could hang the
+%% supervisor.
+%% Returns: ok | {error, OtherReason} (this should be reported)
+%%-----------------------------------------------------------------
+shutdown(Pid, brutal_kill) ->
+ case monitor_child(Pid) of
+ ok ->
+ exit(Pid, kill),
+ receive
+ {'DOWN', _MRef, process, Pid, killed} ->
+ ok;
+ {'DOWN', _MRef, process, Pid, OtherReason} ->
+ {error, OtherReason}
+ end;
+ {error, Reason} ->
+ {error, Reason}
+ end;
+shutdown(Pid, Time) ->
+ case monitor_child(Pid) of
+ ok ->
+ exit(Pid, shutdown), %% Try to shutdown gracefully
+ receive
+ {'DOWN', _MRef, process, Pid, shutdown} ->
+ ok;
+ {'DOWN', _MRef, process, Pid, OtherReason} ->
+ {error, OtherReason}
+ after Time ->
+ exit(Pid, kill), %% Force termination.
+ receive
+ {'DOWN', _MRef, process, Pid, OtherReason} ->
+ {error, OtherReason}
+ end
+ end;
+ {error, Reason} ->
+ {error, Reason}
+ end.
+
+%% Help function to shutdown/2 switches from link to monitor approach
+monitor_child(Pid) ->
+
+ %% Do the monitor operation first so that if the child dies
+ %% before the monitoring is done causing a 'DOWN'-message with
+ %% reason noproc, we will get the real reason in the 'EXIT'-message
+ %% unless a naughty child has already done unlink...
+ erlang:monitor(process, Pid),
+ unlink(Pid),
+
+ receive
+ %% If the child dies before the unlik we must empty
+ %% the mail-box of the 'EXIT'-message and the 'DOWN'-message.
+ {'EXIT', Pid, Reason} ->
+ receive
+ {'DOWN', _, process, Pid, _} ->
+ {error, Reason}
+ end
+ after 0 ->
+ %% If a naughty child did unlink and the child dies before
+ %% monitor the result will be that shutdown/2 receives a
+ %% 'DOWN'-message with reason noproc.
+ %% If the child should die after the unlink there
+ %% will be a 'DOWN'-message with a correct reason
+ %% that will be handled in shutdown/2.
+ ok
+ end.
+
+%%-----------------------------------------------------------------
+%% Func: terminate_dynamic_children/1
+%% Args: State
+%% Returns: ok
+%%
+%% Shutdown all dynamic children. This happens when the supervisor is
+%% stopped. Because the supervisor can have millions of dynamic children, we
+%% can have a significative overhead here.
+%%-----------------------------------------------------------------
+terminate_dynamic_children(State) ->
+ Child = get_dynamic_child(State),
+ {Pids, EStack0} = monitor_dynamic_children(Child,State),
+ Sz = sets:size(Pids),
+ EStack = case Child#child.shutdown of
+ brutal_kill ->
+ sets:fold(fun(P, _) -> exit(P, kill) end, ok, Pids),
+ wait_dynamic_children(Child, Pids, Sz, undefined, EStack0);
+ infinity ->
+ sets:fold(fun(P, _) -> exit(P, shutdown) end, ok, Pids),
+ wait_dynamic_children(Child, Pids, Sz, undefined, EStack0);
+ Time ->
+ sets:fold(fun(P, _) -> exit(P, shutdown) end, ok, Pids),
+ TRef = erlang:start_timer(Time, self(), kill),
+ wait_dynamic_children(Child, Pids, Sz, TRef, EStack0)
+ end,
+ %% Unroll stacked errors and report them
+ dict:fold(fun(Reason, Ls, _) ->
+ ?report_error(shutdown_error, Reason,
+ Child#child{pid=Ls}, State#state.name)
+ end, ok, EStack).
+
+monitor_dynamic_children(Child,State) ->
+ dyn_fold(fun(P,{Pids, EStack}) when is_pid(P) ->
+ case monitor_child(P) of
+ ok ->
+ {sets:add_element(P, Pids), EStack};
+ {error, normal} when not (?is_permanent(Child)) ->
+ {Pids, EStack};
+ {error, Reason} ->
+ {Pids, dict:append(Reason, P, EStack)}
+ end;
+ (?restarting(_), {Pids, EStack}) ->
+ {Pids, EStack}
+ end, {sets:new(), dict:new()}, State).
+
+wait_dynamic_children(_Child, _Pids, 0, undefined, EStack) ->
+ EStack;
+wait_dynamic_children(_Child, _Pids, 0, TRef, EStack) ->
+ %% If the timer has expired before its cancellation, we must empty the
+ %% mail-box of the 'timeout'-message.
+ _ = erlang:cancel_timer(TRef),
+ receive
+ {timeout, TRef, kill} ->
+ EStack
+ after 0 ->
+ EStack
+ end;
+wait_dynamic_children(#child{shutdown=brutal_kill} = Child, Pids, Sz,
+ TRef, EStack) ->
+ receive
+ {'DOWN', _MRef, process, Pid, killed} ->
+ wait_dynamic_children(Child, sets:del_element(Pid, Pids), Sz-1,
+ TRef, EStack);
+
+ {'DOWN', _MRef, process, Pid, Reason} ->
+ wait_dynamic_children(Child, sets:del_element(Pid, Pids), Sz-1,
+ TRef, dict:append(Reason, Pid, EStack))
+ end;
+wait_dynamic_children(Child, Pids, Sz, TRef, EStack) ->
+ receive
+ {'DOWN', _MRef, process, Pid, shutdown} ->
+ wait_dynamic_children(Child, sets:del_element(Pid, Pids), Sz-1,
+ TRef, EStack);
+
+ {'DOWN', _MRef, process, Pid, {shutdown, _}} ->
+ wait_dynamic_children(Child, sets:del_element(Pid, Pids), Sz-1,
+ TRef, EStack);
+
+ {'DOWN', _MRef, process, Pid, normal} when not (?is_permanent(Child)) ->
+ wait_dynamic_children(Child, sets:del_element(Pid, Pids), Sz-1,
+ TRef, EStack);
+
+ {'DOWN', _MRef, process, Pid, Reason} ->
+ wait_dynamic_children(Child, sets:del_element(Pid, Pids), Sz-1,
+ TRef, dict:append(Reason, Pid, EStack));
+
+ {timeout, TRef, kill} ->
+ sets:fold(fun(P, _) -> exit(P, kill) end, ok, Pids),
+ wait_dynamic_children(Child, Pids, Sz, undefined, EStack)
+ end.
+
+%%-----------------------------------------------------------------
+%% Access #state.children
+%%-----------------------------------------------------------------
+
+%% Note we do not want to save the parameter list for temporary processes as
+%% they will not be restarted, and hence we do not need this information.
+%% Especially for dynamic children to simple_one_for_one supervisors
+%% it could become very costly as it is not uncommon to spawn
+%% very many such processes.
+-spec save_child(child_rec(), state()) -> state().
+save_child(#child{mfargs = {M, F, _}} = Child, State) when ?is_temporary(Child) ->
+ do_save_child(Child#child{mfargs = {M, F, undefined}}, State);
+save_child(Child, State) ->
+ do_save_child(Child, State).
+
+-spec do_save_child(child_rec(), state()) -> state().
+do_save_child(#child{id = Id} = Child, #state{children = {Ids,Db}} = State) ->
+ State#state{children = {[Id|Ids],Db#{Id => Child}}}.
+
+-spec del_child(child_rec(), state()) -> state();
+ (child_id(), children()) -> children().
+del_child(#child{pid = Pid}, State) when ?is_simple(State) ->
+ dyn_erase(Pid,State);
+del_child(Child, State) when is_record(Child,child), is_record(State,state) ->
+ NChildren = del_child(Child#child.id, State#state.children),
+ State#state{children = NChildren};
+del_child(Id, {Ids,Db}) ->
+ case maps:get(Id, Db) of
+ Child when Child#child.restart_type =:= temporary ->
+ {lists:delete(Id, Ids), maps:remove(Id, Db)};
+ Child ->
+ {Ids, Db#{Id=>Child#child{pid=undefined}}}
+ end.
+
+%% In: {[S4, S3, Ch, S1, S0],Db}
+%% Ret: {{[S4, S3, Ch],Db1}, {[S1, S0],Db2}}
+%% Db1 and Db2 contain the keys in the lists they are associated with.
+-spec split_child(child_id(), children()) -> {children(), children()}.
+split_child(Id, {Ids,Db}) ->
+ {IdsAfter,IdsBefore} = split_ids(Id, Ids, []),
+ DbBefore = maps:with(IdsBefore,Db),
+ #{Id:=Ch} = DbAfter = maps:with(IdsAfter,Db),
+ {{IdsAfter,DbAfter#{Id=>Ch#child{pid=undefined}}},{IdsBefore,DbBefore}}.
+
+split_ids(Id, [Id|Ids], After) ->
+ {lists:reverse([Id|After]), Ids};
+split_ids(Id, [Other|Ids], After) ->
+ split_ids(Id, Ids, [Other | After]).
+
+%% Find the child record for a given Pid (dynamic child) or Id
+%% (non-dynamic child). This is called from the API functions.
+-spec internal_find_child(pid() | child_id(), state()) -> {ok,child_rec()} | error.
+internal_find_child(Pid, State) when is_pid(Pid), ?is_simple(State) ->
+ case find_dynamic_child(Pid, State) of
+ error ->
+ case find_dynamic_child(restarting(Pid), State) of
+ error ->
+ case erlang:is_process_alive(Pid) of
+ true -> error;
+ false -> {ok, get_dynamic_child(State)}
+ end;
+ Other ->
+ Other
+ end;
+ Other ->
+ Other
+ end;
+internal_find_child(Id, #state{children = {_Ids,Db}}) ->
+ maps:find(Id, Db).
+
+%% Get the child record - either by child id or by pid. If
+%% simple_one_for_one, then insert the pid and args into the returned
+%% child record. This is called when trying to restart the child.
+-spec find_child_and_args(IdOrPid, state()) -> {ok, child_rec()} | error when
+ IdOrPid :: pid() | {restarting,pid()} | child_id().
+find_child_and_args(Pid, State) when ?is_simple(State) ->
+ case find_dynamic_child(Pid, State) of
+ {ok,#child{mfargs={M,F,_}} = Child} ->
+ {ok, Args} = dyn_args(Pid, State),
+ {ok, Child#child{mfargs = {M, F, Args}}};
+ error ->
+ error
+ end;
+find_child_and_args(Pid, State) when is_pid(Pid) ->
+ find_child_by_pid(Pid, State);
+find_child_and_args(Id, #state{children={_Ids,Db}}) ->
+ maps:find(Id, Db).
+
+%% Given the pid, find the child record for a dynamic child, and
+%% include the pid in the returned record.
+-spec find_dynamic_child(IdOrPid, state()) -> {ok, child_rec()} | error when
+ IdOrPid :: pid() | {restarting,pid()} | child_id().
+find_dynamic_child(Pid, State) ->
+ case dyn_exists(Pid, State) of
+ true ->
+ Child = get_dynamic_child(State),
+ {ok, Child#child{pid=Pid}};
+ false ->
+ error
+ end.
+
+%% Given the pid, find the child record for a non-dyanamic child.
+-spec find_child_by_pid(IdOrPid, state()) -> {ok,child_rec()} | error when
+ IdOrPid :: pid() | {restarting,pid()}.
+find_child_by_pid(Pid,#state{children={_Ids,Db}}) ->
+ Fun = fun(_Id,#child{pid=P}=Ch,_) when P =:= Pid ->
+ throw(Ch);
+ (_,_,error) ->
+ error
+ end,
+ try maps:fold(Fun,error,Db)
+ catch throw:Child -> {ok,Child}
+ end.
+
+%% Get the child record from a simple_one_for_one supervisor - no pid
+%% It is assumed that the child can always be found
+-spec get_dynamic_child(state()) -> child_rec().
+get_dynamic_child(#state{children={[Id],Db}}) ->
+ #{Id := Child} = Db,
+ Child.
+
+%% Update pid in the given child record and store it in the process state
+-spec set_pid(term(), child_id(), state()) -> state();
+ (term(), child_id(), children()) -> children().
+set_pid(Pid, Id, #state{children=Children} = State) ->
+ State#state{children = set_pid(Pid, Id, Children)};
+set_pid(Pid, Id, {Ids, Db}) ->
+ NewDb = maps:update_with(Id, fun(Child) -> Child#child{pid=Pid} end, Db),
+ {Ids,NewDb}.
+
+%% Remove the Id and the child record from the process state
+-spec remove_child(child_id(), state()) -> state().
+remove_child(Id, #state{children={Ids,Db}} = State) ->
+ NewIds = lists:delete(Id,Ids),
+ NewDb = maps:remove(Id,Db),
+ State#state{children = {NewIds,NewDb}}.
+
+%% In the order of Ids, traverse the children and update each child
+%% according to the return value of the Fun.
+%% On error, abort and return the merge of the old and the updated map.
+%% NOTE: The returned list of Ids is reverted compared to the input.
+-spec children_map(Fun, children()) -> {ok, children()} |
+ {error,children(),Reason} when
+ Fun :: fun((child_id(),child_rec()) -> {update,child_rec()} |
+ remove |
+ {abort, Reason}),
+ Reason :: term().
+children_map(Fun,{Ids,Db}) ->
+ children_map(Fun, Ids, Db, []).
+
+children_map(Fun,[Id|Ids],Db,Acc) ->
+ case Fun(Id,maps:get(Id,Db)) of
+ {update,Child} ->
+ children_map(Fun,Ids,Db#{Id => Child},[Id|Acc]);
+ remove ->
+ children_map(Fun,Ids,maps:remove(Id,Db),Acc);
+ {abort,Reason} ->
+ {error,{lists:reverse(Ids)++[Id|Acc],Db},Reason}
+ end;
+children_map(_Fun,[],Db,Acc) ->
+ {ok,{Acc,Db}}.
+
+%% In the order of Ids, map over all children and return the list
+-spec children_to_list(Fun, children()) -> List when
+ Fun :: fun((child_id(), child_rec()) -> Elem),
+ List :: list(Elem),
+ Elem :: term().
+children_to_list(Fun,{Ids,Db}) ->
+ children_to_list(Fun, Ids, Db, []).
+children_to_list(Fun,[Id|Ids],Db,Acc) ->
+ children_to_list(Fun,Ids,Db,[Fun(Id,maps:get(Id,Db))|Acc]);
+children_to_list(_Fun,[],_Db,Acc) ->
+ lists:reverse(Acc).
+
+%% The order is not important - so ignore Ids
+-spec children_fold(Fun, Acc0, children()) -> Acc1 when
+ Fun :: fun((child_id(), child_rec(), AccIn) -> AccOut),
+ Acc0 :: term(),
+ Acc1 :: term(),
+ AccIn :: term(),
+ AccOut :: term().
+children_fold(Fun,Init,{_Ids,Db}) ->
+ maps:fold(Fun, Init, Db).
+
+-spec append(children(), children()) -> children().
+append({Ids1,Db1},{Ids2,Db2}) ->
+ {Ids1++Ids2,maps:merge(Db1,Db2)}.
+
+%%-----------------------------------------------------------------
+%% Func: init_state/4
+%% Args: SupName = {local, atom()} | {global, atom()} | self
+%% Type = {Strategy, MaxIntensity, Period}
+%% Strategy = one_for_one | one_for_all | simple_one_for_one |
+%% rest_for_one
+%% MaxIntensity = integer() >= 0
+%% Period = integer() > 0
+%% Mod :== atom()
+%% Args :== term()
+%% Purpose: Check that Type is of correct type (!)
+%% Returns: {ok, state()} | Error
+%%-----------------------------------------------------------------
+init_state(SupName, Type, Mod, Args) ->
+ set_flags(Type, #state{name = supname(SupName,Mod),
+ module = Mod,
+ args = Args}).
+
+set_flags(Flags, State) ->
+ try check_flags(Flags) of
+ #{strategy := Strategy, intensity := MaxIntensity, period := Period} ->
+ {ok, State#state{strategy = Strategy,
+ intensity = MaxIntensity,
+ period = Period}}
+ catch
+ Thrown -> Thrown
+ end.
+
+check_flags(SupFlags) when is_map(SupFlags) ->
+ do_check_flags(maps:merge(?default_flags,SupFlags));
+check_flags({Strategy, MaxIntensity, Period}) ->
+ check_flags(#{strategy => Strategy,
+ intensity => MaxIntensity,
+ period => Period});
+check_flags(What) ->
+ throw({invalid_type, What}).
+
+do_check_flags(#{strategy := Strategy,
+ intensity := MaxIntensity,
+ period := Period} = Flags) ->
+ validStrategy(Strategy),
+ validIntensity(MaxIntensity),
+ validPeriod(Period),
+ Flags.
+
+validStrategy(simple_one_for_one) -> true;
+validStrategy(one_for_one) -> true;
+validStrategy(one_for_all) -> true;
+validStrategy(rest_for_one) -> true;
+validStrategy(What) -> throw({invalid_strategy, What}).
+
+validIntensity(Max) when is_integer(Max),
+ Max >= 0 -> true;
+validIntensity(What) -> throw({invalid_intensity, What}).
+
+validPeriod(Period) when is_integer(Period),
+ Period > 0 -> true;
+validPeriod(What) -> throw({invalid_period, What}).
+
+supname(self, Mod) -> {self(), Mod};
+supname(N, _) -> N.
+
+%%% ------------------------------------------------------
+%%% Check that the children start specification is valid.
+%%% Input: [child_spec()]
+%%% Returns: {ok, [child_rec()]} | Error
+%%% ------------------------------------------------------
+
+check_startspec(Children) -> check_startspec(Children, [], #{}).
+
+check_startspec([ChildSpec|T], Ids, Db) ->
+ case check_childspec(ChildSpec) of
+ {ok, #child{id=Id}=Child} ->
+ case maps:is_key(Id, Db) of
+ %% The error message duplicate_child_name is kept for
+ %% backwards compatibility, although
+ %% duplicate_child_id would be more correct.
+ true -> {duplicate_child_name, Id};
+ false -> check_startspec(T, [Id | Ids], Db#{Id=>Child})
+ end;
+ Error -> Error
+ end;
+check_startspec([], Ids, Db) ->
+ {ok, {lists:reverse(Ids),Db}}.
+
+check_childspec(ChildSpec) when is_map(ChildSpec) ->
+ catch do_check_childspec(maps:merge(?default_child_spec,ChildSpec));
+check_childspec({Id, Func, RestartType, Shutdown, ChildType, Mods}) ->
+ check_childspec(#{id => Id,
+ start => Func,
+ restart => RestartType,
+ shutdown => Shutdown,
+ type => ChildType,
+ modules => Mods});
+check_childspec(X) -> {invalid_child_spec, X}.
+
+do_check_childspec(#{restart := RestartType,
+ type := ChildType} = ChildSpec)->
+ Id = case ChildSpec of
+ #{id := I} -> I;
+ _ -> throw(missing_id)
+ end,
+ Func = case ChildSpec of
+ #{start := F} -> F;
+ _ -> throw(missing_start)
+ end,
+ validId(Id),
+ validFunc(Func),
+ validRestartType(RestartType),
+ validChildType(ChildType),
+ Shutdown = case ChildSpec of
+ #{shutdown := S} -> S;
+ #{type := worker} -> 5000;
+ #{type := supervisor} -> infinity
+ end,
+ validShutdown(Shutdown),
+ Mods = case ChildSpec of
+ #{modules := Ms} -> Ms;
+ _ -> {M,_,_} = Func, [M]
+ end,
+ validMods(Mods),
+ {ok, #child{id = Id, mfargs = Func, restart_type = RestartType,
+ shutdown = Shutdown, child_type = ChildType, modules = Mods}}.
+
+validChildType(supervisor) -> true;
+validChildType(worker) -> true;
+validChildType(What) -> throw({invalid_child_type, What}).
+
+validId(_Id) -> true.
+
+validFunc({M, F, A}) when is_atom(M),
+ is_atom(F),
+ is_list(A) -> true;
+validFunc(Func) -> throw({invalid_mfa, Func}).
+
+validRestartType(permanent) -> true;
+validRestartType({permanent, Delay}) -> validDelay(Delay);
+validRestartType(temporary) -> true;
+validRestartType(transient) -> true;
+validRestartType({transient, Delay}) -> validDelay(Delay);
+validRestartType(intrinsic) -> true;
+validRestartType({intrinsic, Delay}) -> validDelay(Delay);
+validRestartType(RestartType) -> throw({invalid_restart_type, RestartType}).
+
+validDelay(Delay) when is_number(Delay), Delay >= 0 ->
+ true;
+validDelay(What) ->
+ throw({invalid_delay, What}).
+
+validShutdown(Shutdown)
+ when is_integer(Shutdown), Shutdown > 0 -> true;
+validShutdown(infinity) -> true;
+validShutdown(brutal_kill) -> true;
+validShutdown(Shutdown) -> throw({invalid_shutdown, Shutdown}).
+
+validMods(dynamic) -> true;
+validMods(Mods) when is_list(Mods) ->
+ lists:foreach(fun(Mod) ->
+ if
+ is_atom(Mod) -> ok;
+ true -> throw({invalid_module, Mod})
+ end
+ end,
+ Mods);
+validMods(Mods) -> throw({invalid_modules, Mods}).
+
+child_to_spec(#child{id = Id,
+ mfargs = Func,
+ restart_type = RestartType,
+ shutdown = Shutdown,
+ child_type = ChildType,
+ modules = Mods}) ->
+ #{id => Id,
+ start => Func,
+ restart => RestartType,
+ shutdown => Shutdown,
+ type => ChildType,
+ modules => Mods}.
+
+%%% ------------------------------------------------------
+%%% Add a new restart and calculate if the max restart
+%%% intensity has been reached (in that case the supervisor
+%%% shall terminate).
+%%% All restarts accured inside the period amount of seconds
+%%% are kept in the #state.restarts list.
+%%% Returns: {ok, State'} | {terminate, State'}
+%%% ------------------------------------------------------
+
+add_restart(State) ->
+ I = State#state.intensity,
+ P = State#state.period,
+ R = State#state.restarts,
+ Now = erlang:monotonic_time(1),
+ R1 = add_restart([Now|R], Now, P),
+ State1 = State#state{restarts = R1},
+ case length(R1) of
+ CurI when CurI =< I ->
+ {ok, State1};
+ _ ->
+ {terminate, State1}
+ end.
+
+add_restart([R|Restarts], Now, Period) ->
+ case inPeriod(R, Now, Period) of
+ true ->
+ [R|add_restart(Restarts, Now, Period)];
+ _ ->
+ []
+ end;
+add_restart([], _, _) ->
+ [].
+
+inPeriod(Then, Now, Period) ->
+ Now =< Then + Period.
+
+%%% ------------------------------------------------------
+%%% Error and progress reporting.
+%%% ------------------------------------------------------
+extract_child(Child) when is_list(Child#child.pid) ->
+ [{nb_children, length(Child#child.pid)},
+ {id, Child#child.id},
+ {mfargs, Child#child.mfargs},
+ {restart_type, Child#child.restart_type},
+ {shutdown, Child#child.shutdown},
+ {child_type, Child#child.child_type}];
+extract_child(Child) ->
+ [{pid, Child#child.pid},
+ {id, Child#child.id},
+ {mfargs, Child#child.mfargs},
+ {restart_type, Child#child.restart_type},
+ {shutdown, Child#child.shutdown},
+ {child_type, Child#child.child_type}].
+
+report_progress(Child, SupName) ->
+ ?LOG_INFO(#{label=>{supervisor,progress},
+ report=>[{supervisor,SupName},
+ {started,extract_child(Child)}]},
+ #{domain=>[otp,sasl],
+ report_cb=>fun logger:format_otp_report/1,
+ logger_formatter=>#{title=>"PROGRESS REPORT"},
+ error_logger=>#{tag=>info_report,type=>progress}}).
+
+format_status(terminate, [_PDict, State]) ->
+ State;
+format_status(_, [_PDict, State]) ->
+ [{data, [{"State", State}]},
+ {supervisor, [{"Callback", State#state.module}]}].
+
+%%%-----------------------------------------------------------------
+%%% Dynamics database access
+dyn_size(#state{dynamics = {Mod,Db}}) ->
+ Mod:size(Db).
+
+dyn_erase(Pid,#state{dynamics={sets,Db}}=State) ->
+ State#state{dynamics={sets,sets:del_element(Pid,Db)}};
+dyn_erase(Pid,#state{dynamics={maps,Db}}=State) ->
+ State#state{dynamics={maps,maps:remove(Pid,Db)}}.
+
+dyn_store(Pid,_,#state{dynamics={sets,Db}}=State) ->
+ State#state{dynamics={sets,sets:add_element(Pid,Db)}};
+dyn_store(Pid,Args,#state{dynamics={maps,Db}}=State) ->
+ State#state{dynamics={maps,Db#{Pid => Args}}}.
+
+dyn_fold(Fun,Init,#state{dynamics={sets,Db}}) ->
+ sets:fold(Fun,Init,Db);
+dyn_fold(Fun,Init,#state{dynamics={maps,Db}}) ->
+ maps:fold(fun(Pid,_,Acc) -> Fun(Pid,Acc) end, Init, Db).
+
+dyn_map(Fun, #state{dynamics={sets,Db}}) ->
+ lists:map(Fun, sets:to_list(Db));
+dyn_map(Fun, #state{dynamics={maps,Db}}) ->
+ lists:map(Fun, maps:keys(Db)).
+
+dyn_exists(Pid, #state{dynamics={sets, Db}}) ->
+ sets:is_element(Pid, Db);
+dyn_exists(Pid, #state{dynamics={maps, Db}}) ->
+ maps:is_key(Pid, Db).
+
+dyn_args(_Pid, #state{dynamics={sets, _Db}}) ->
+ {ok,undefined};
+dyn_args(Pid, #state{dynamics={maps, Db}}) ->
+ maps:find(Pid, Db).
+
+dyn_init(State) ->
+ dyn_init(get_dynamic_child(State),State).
+
+dyn_init(Child,State) when ?is_temporary(Child) ->
+ State#state{dynamics={sets,sets:new()}};
+dyn_init(_Child,State) ->
+ State#state{dynamics={maps,maps:new()}}.
diff --git a/deps/rabbit_common/src/vm_memory_monitor.erl b/deps/rabbit_common/src/vm_memory_monitor.erl
new file mode 100644
index 0000000000..73b5a23b78
--- /dev/null
+++ b/deps/rabbit_common/src/vm_memory_monitor.erl
@@ -0,0 +1,576 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+%% In practice Erlang shouldn't be allowed to grow to more than a half
+%% of available memory. The pessimistic scenario is when the Erlang VM
+%% has a single process that's consuming all memory. In such a case,
+%% during garbage collection, Erlang tries to allocate a huge chunk of
+%% continuous memory, which can result in a crash or heavy swapping.
+%%
+%% This module tries to warn Rabbit before such situations occur, so
+%% that it has a higher chance to avoid running out of memory.
+
+-module(vm_memory_monitor).
+
+-behaviour(gen_server).
+
+-export([start_link/1, start_link/3]).
+
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3]).
+
+-export([get_total_memory/0, get_vm_limit/0,
+ get_check_interval/0, set_check_interval/1,
+ get_vm_memory_high_watermark/0, set_vm_memory_high_watermark/1,
+ get_memory_limit/0,
+ %% TODO: refactor in master
+ get_memory_use/1,
+ get_process_memory/0,
+ get_process_memory/1,
+ get_memory_calculation_strategy/0,
+ get_rss_memory/0]).
+
+%% for tests
+-export([parse_line_linux/1, parse_mem_limit/1]).
+
+-define(SERVER, ?MODULE).
+
+-record(state, {total_memory,
+ memory_limit,
+ process_memory,
+ memory_config_limit,
+ timeout,
+ timer,
+ alarmed,
+ alarm_funs,
+ os_type = undefined,
+ os_pid = undefined,
+ page_size = undefined,
+ proc_file = undefined}).
+
+-include("rabbit_memory.hrl").
+
+%%----------------------------------------------------------------------------
+
+-type memory_calculation_strategy() :: rss | erlang | allocated.
+-type vm_memory_high_watermark() :: (float() | {'absolute', integer() | string()}).
+-spec start_link(float()) -> rabbit_types:ok_pid_or_error().
+-spec start_link(float(), fun ((any()) -> 'ok'),
+ fun ((any()) -> 'ok')) -> rabbit_types:ok_pid_or_error().
+-spec get_total_memory() -> (non_neg_integer() | 'unknown').
+-spec get_vm_limit() -> non_neg_integer().
+-spec get_check_interval() -> non_neg_integer().
+-spec set_check_interval(non_neg_integer()) -> 'ok'.
+-spec get_vm_memory_high_watermark() -> vm_memory_high_watermark().
+-spec set_vm_memory_high_watermark(vm_memory_high_watermark()) -> 'ok'.
+-spec get_memory_limit() -> non_neg_integer().
+-spec get_memory_use(bytes) -> {non_neg_integer(), float() | infinity};
+ (ratio) -> float() | infinity.
+-spec get_cached_process_memory_and_limit() -> {non_neg_integer(),
+ float() | infinity}.
+-spec get_rss_memory() -> non_neg_integer().
+
+-export_type([memory_calculation_strategy/0]).
+%%----------------------------------------------------------------------------
+%% Public API
+%%----------------------------------------------------------------------------
+
+get_total_memory() ->
+ case application:get_env(rabbit, total_memory_available_override_value) of
+ {ok, Value} ->
+ case rabbit_resource_monitor_misc:parse_information_unit(Value) of
+ {ok, ParsedTotal} ->
+ ParsedTotal;
+ {error, parse_error} ->
+ rabbit_log:warning(
+ "The override value for the total memmory available is "
+ "not a valid value: ~p, getting total from the system.~n",
+ [Value]),
+ get_total_memory_from_os()
+ end;
+ undefined ->
+ get_total_memory_from_os()
+ end.
+
+get_vm_limit() -> get_vm_limit(os:type()).
+
+get_check_interval() ->
+ gen_server:call(?MODULE, get_check_interval, infinity).
+
+set_check_interval(Fraction) ->
+ gen_server:call(?MODULE, {set_check_interval, Fraction}, infinity).
+
+get_vm_memory_high_watermark() ->
+ gen_server:call(?MODULE, get_vm_memory_high_watermark, infinity).
+
+set_vm_memory_high_watermark(Fraction) ->
+ gen_server:call(?MODULE, {set_vm_memory_high_watermark, Fraction},
+ infinity).
+
+get_memory_limit() ->
+ gen_server:call(?MODULE, get_memory_limit, infinity).
+
+get_memory_use(bytes) ->
+ {ProcessMemory, MemoryLimit} = get_cached_process_memory_and_limit(),
+ {ProcessMemory, case MemoryLimit > 0.0 of
+ true -> MemoryLimit;
+ false -> infinity
+ end};
+get_memory_use(ratio) ->
+ {ProcessMemory, MemoryLimit} = get_cached_process_memory_and_limit(),
+ case MemoryLimit > 0.0 of
+ true -> ProcessMemory / MemoryLimit;
+ false -> infinity
+ end.
+
+%% Memory reported by erlang:memory(total) is not supposed to
+%% be equal to the total size of all pages mapped to the emulator,
+%% according to http://erlang.org/doc/man/erlang.html#memory-0
+%% erlang:memory(total) under-reports memory usage by around 20%
+%%
+%% Win32 Note: 3.6.12 shipped with code that used wmic.exe to get the
+%% WorkingSetSize value for the running erl.exe process. Unfortunately
+%% even with a moderate invocation rate of 1 ops/second that uses more
+%% CPU resources than some Windows users are willing to tolerate.
+%% See rabbitmq/rabbitmq-server#1343 and rabbitmq/rabbitmq-common#224
+%% for details.
+-spec get_process_memory() -> Bytes :: integer().
+get_process_memory() ->
+ {ProcMem, _} = get_memory_use(bytes),
+ ProcMem.
+
+-spec get_process_memory(cached | current) -> Bytes :: integer().
+get_process_memory(cached) ->
+ {ProcMem, _} = get_memory_use(bytes),
+ ProcMem;
+get_process_memory(current) ->
+ get_process_memory_uncached().
+
+-spec get_memory_calculation_strategy() -> memory_calculation_strategy().
+get_memory_calculation_strategy() ->
+ case rabbit_misc:get_env(rabbit, vm_memory_calculation_strategy, rss) of
+ allocated -> allocated;
+ erlang -> erlang;
+ legacy -> erlang; %% backwards compatibility
+ rss -> rss;
+ UnsupportedValue ->
+ rabbit_log:warning(
+ "Unsupported value '~p' for vm_memory_calculation_strategy. "
+ "Supported values: (allocated|erlang|legacy|rss). "
+ "Defaulting to 'rss'",
+ [UnsupportedValue]
+ ),
+ rss
+ end.
+
+%%----------------------------------------------------------------------------
+%% gen_server callbacks
+%%----------------------------------------------------------------------------
+
+start_link(MemFraction) ->
+ start_link(MemFraction,
+ fun alarm_handler:set_alarm/1, fun alarm_handler:clear_alarm/1).
+
+start_link(MemFraction, AlarmSet, AlarmClear) ->
+ gen_server:start_link({local, ?SERVER}, ?MODULE,
+ [MemFraction, {AlarmSet, AlarmClear}], []).
+
+init([MemFraction, AlarmFuns]) ->
+ TRef = erlang:send_after(?DEFAULT_MEMORY_CHECK_INTERVAL, self(), update),
+ State0 = #state{timeout = ?DEFAULT_MEMORY_CHECK_INTERVAL,
+ timer = TRef,
+ alarmed = false,
+ alarm_funs = AlarmFuns},
+ State1 = update_process_memory(init_state_by_os(State0)),
+ {ok, set_mem_limits(State1, MemFraction)}.
+
+handle_call(get_vm_memory_high_watermark, _From,
+ #state{memory_config_limit = MemLimit} = State) ->
+ {reply, MemLimit, State};
+
+handle_call({set_vm_memory_high_watermark, MemLimit}, _From, State) ->
+ {reply, ok, set_mem_limits(State, MemLimit)};
+
+handle_call(get_check_interval, _From, State) ->
+ {reply, State#state.timeout, State};
+
+handle_call({set_check_interval, Timeout}, _From, State) ->
+ State1 = case erlang:cancel_timer(State#state.timer) of
+ false ->
+ State#state{timeout = Timeout};
+ _ ->
+ State#state{timeout = Timeout,
+ timer = erlang:send_after(Timeout, self(), update)}
+ end,
+ {reply, ok, State1};
+
+handle_call(get_memory_limit, _From, State) ->
+ {reply, State#state.memory_limit, State};
+
+handle_call(get_cached_process_memory_and_limit, _From, State) ->
+ {reply, {State#state.process_memory, State#state.memory_limit}, State};
+
+handle_call(_Request, _From, State) ->
+ {noreply, State}.
+
+handle_cast(_Request, State) ->
+ {noreply, State}.
+
+handle_info(update, State) ->
+ _ = erlang:cancel_timer(State#state.timer),
+ State1 = internal_update(State),
+ TRef = erlang:send_after(State1#state.timeout, self(), update),
+ {noreply, State1#state{ timer = TRef }};
+
+handle_info(_Info, State) ->
+ {noreply, State}.
+
+terminate(_Reason, _State) ->
+ ok.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+%%----------------------------------------------------------------------------
+%% Server Internals
+%%----------------------------------------------------------------------------
+get_rss_memory() ->
+ TmpState = init_state_by_os(#state{}),
+ {ok, ProcMem} = get_process_memory_using_strategy(rss, TmpState),
+ ProcMem.
+
+get_cached_process_memory_and_limit() ->
+ try
+ gen_server:call(?MODULE, get_cached_process_memory_and_limit, infinity)
+ catch exit:{noproc, Error} ->
+ rabbit_log:warning("Memory monitor process not yet started: ~p~n", [Error]),
+ ProcessMemory = get_process_memory_uncached(),
+ {ProcessMemory, infinity}
+ end.
+
+get_process_memory_uncached() ->
+ TmpState = update_process_memory(init_state_by_os(#state{})),
+ TmpState#state.process_memory.
+
+update_process_memory(State) ->
+ Strategy = get_memory_calculation_strategy(),
+ {ok, ProcMem} = get_process_memory_using_strategy(Strategy, State),
+ State#state{process_memory = ProcMem}.
+
+init_state_by_os(State = #state{os_type = undefined}) ->
+ OsType = os:type(),
+ OsPid = os:getpid(),
+ init_state_by_os(State#state{os_type = OsType, os_pid = OsPid});
+init_state_by_os(State0 = #state{os_type = {unix, linux}, os_pid = OsPid}) ->
+ PageSize = get_linux_pagesize(),
+ ProcFile = io_lib:format("/proc/~s/statm", [OsPid]),
+ State0#state{page_size = PageSize, proc_file = ProcFile};
+init_state_by_os(State) ->
+ State.
+
+get_process_memory_using_strategy(rss, #state{os_type = {unix, linux},
+ page_size = PageSize,
+ proc_file = ProcFile}) ->
+ Data = read_proc_file(ProcFile),
+ [_|[RssPagesStr|_]] = string:tokens(Data, " "),
+ ProcMem = list_to_integer(RssPagesStr) * PageSize,
+ {ok, ProcMem};
+get_process_memory_using_strategy(rss, #state{os_type = {unix, _},
+ os_pid = OsPid}) ->
+ Cmd = "ps -p " ++ OsPid ++ " -o rss=",
+ CmdOutput = os:cmd(Cmd),
+ case re:run(CmdOutput, "[0-9]+", [{capture, first, list}]) of
+ {match, [Match]} ->
+ ProcMem = list_to_integer(Match) * 1024,
+ {ok, ProcMem};
+ _ ->
+ {error, {unexpected_output_from_command, Cmd, CmdOutput}}
+ end;
+get_process_memory_using_strategy(rss, _State) ->
+ {ok, recon_alloc:memory(allocated)};
+get_process_memory_using_strategy(allocated, _State) ->
+ {ok, recon_alloc:memory(allocated)};
+get_process_memory_using_strategy(erlang, _State) ->
+ {ok, erlang:memory(total)}.
+
+get_total_memory_from_os() ->
+ try
+ get_total_memory(os:type())
+ catch _:Error:Stacktrace ->
+ rabbit_log:warning(
+ "Failed to get total system memory: ~n~p~n~p~n",
+ [Error, Stacktrace]),
+ unknown
+ end.
+
+set_mem_limits(State, MemLimit) ->
+ case erlang:system_info(wordsize) of
+ 4 ->
+ rabbit_log:warning(
+ "You are using a 32-bit version of Erlang: you may run into "
+ "memory address~n"
+ "space exhaustion or statistic counters overflow.~n");
+ _ ->
+ ok
+ end,
+ TotalMemory =
+ case get_total_memory() of
+ unknown ->
+ case State of
+ #state { total_memory = undefined,
+ memory_limit = undefined } ->
+ rabbit_log:warning(
+ "Unknown total memory size for your OS ~p. "
+ "Assuming memory size is ~p MiB (~p bytes).~n",
+ [os:type(),
+ trunc(?MEMORY_SIZE_FOR_UNKNOWN_OS/?ONE_MiB),
+ ?MEMORY_SIZE_FOR_UNKNOWN_OS]);
+ _ ->
+ ok
+ end,
+ ?MEMORY_SIZE_FOR_UNKNOWN_OS;
+ Memory -> Memory
+ end,
+ UsableMemory =
+ case get_vm_limit() of
+ Limit when Limit < TotalMemory ->
+ rabbit_log:warning(
+ "Only ~p MiB (~p bytes) of ~p MiB (~p bytes) memory usable due to "
+ "limited address space.~n"
+ "Crashes due to memory exhaustion are possible - see~n"
+ "https://www.rabbitmq.com/memory.html#address-space~n",
+ [trunc(Limit/?ONE_MiB), Limit, trunc(TotalMemory/?ONE_MiB),
+ TotalMemory]),
+ Limit;
+ _ ->
+ TotalMemory
+ end,
+ MemLim = interpret_limit(parse_mem_limit(MemLimit), UsableMemory),
+ rabbit_log:info(
+ "Memory high watermark set to ~p MiB (~p bytes)"
+ " of ~p MiB (~p bytes) total~n",
+ [trunc(MemLim/?ONE_MiB), MemLim,
+ trunc(TotalMemory/?ONE_MiB), TotalMemory]
+ ),
+ internal_update(State #state { total_memory = TotalMemory,
+ memory_limit = MemLim,
+ memory_config_limit = MemLimit}).
+
+interpret_limit({'absolute', MemLim}, UsableMemory) ->
+ erlang:min(MemLim, UsableMemory);
+interpret_limit(MemFraction, UsableMemory) ->
+ trunc(MemFraction * UsableMemory).
+
+parse_mem_limit({absolute, Limit}) ->
+ case rabbit_resource_monitor_misc:parse_information_unit(Limit) of
+ {ok, ParsedLimit} -> {absolute, ParsedLimit};
+ {error, parse_error} ->
+ rabbit_log:error("Unable to parse vm_memory_high_watermark value ~p", [Limit]),
+ ?DEFAULT_VM_MEMORY_HIGH_WATERMARK
+ end;
+parse_mem_limit(MemLimit) when is_integer(MemLimit) ->
+ parse_mem_limit(float(MemLimit));
+parse_mem_limit(MemLimit) when is_float(MemLimit), MemLimit =< ?MAX_VM_MEMORY_HIGH_WATERMARK ->
+ MemLimit;
+parse_mem_limit(MemLimit) when is_float(MemLimit), MemLimit > ?MAX_VM_MEMORY_HIGH_WATERMARK ->
+ rabbit_log:warning(
+ "Memory high watermark of ~p is above the allowed maximum, falling back to ~p~n",
+ [MemLimit, ?MAX_VM_MEMORY_HIGH_WATERMARK]
+ ),
+ ?MAX_VM_MEMORY_HIGH_WATERMARK;
+parse_mem_limit(MemLimit) ->
+ rabbit_log:warning(
+ "Memory high watermark of ~p is invalid, defaulting to ~p~n",
+ [MemLimit, ?DEFAULT_VM_MEMORY_HIGH_WATERMARK]
+ ),
+ ?DEFAULT_VM_MEMORY_HIGH_WATERMARK.
+
+internal_update(State0 = #state{memory_limit = MemLimit,
+ alarmed = Alarmed,
+ alarm_funs = {AlarmSet, AlarmClear}}) ->
+ State1 = update_process_memory(State0),
+ ProcMem = State1#state.process_memory,
+ NewAlarmed = ProcMem > MemLimit,
+ case {Alarmed, NewAlarmed} of
+ {false, true} -> emit_update_info(set, ProcMem, MemLimit),
+ AlarmSet({{resource_limit, memory, node()}, []});
+ {true, false} -> emit_update_info(clear, ProcMem, MemLimit),
+ AlarmClear({resource_limit, memory, node()});
+ _ -> ok
+ end,
+ State1#state{alarmed = NewAlarmed}.
+
+emit_update_info(AlarmState, MemUsed, MemLimit) ->
+ rabbit_log:info(
+ "vm_memory_high_watermark ~p. Memory used:~p allowed:~p~n",
+ [AlarmState, MemUsed, MemLimit]).
+
+%% According to https://msdn.microsoft.com/en-us/library/aa366778(VS.85).aspx
+%% Windows has 2GB and 8TB of address space for 32 and 64 bit accordingly.
+get_vm_limit({win32,_OSname}) ->
+ case erlang:system_info(wordsize) of
+ 4 -> 2*1024*1024*1024; %% 2 GB for 32 bits 2^31
+ 8 -> 8*1024*1024*1024*1024 %% 8 TB for 64 bits 2^42
+ end;
+
+%% On a 32-bit machine, if you're using more than 2 gigs of RAM you're
+%% in big trouble anyway.
+get_vm_limit(_OsType) ->
+ case erlang:system_info(wordsize) of
+ 4 -> 2*1024*1024*1024; %% 2 GB for 32 bits 2^31
+ 8 -> 256*1024*1024*1024*1024 %% 256 TB for 64 bits 2^48
+ %%https://en.wikipedia.org/wiki/X86-64#Virtual_address_space_details
+ end.
+
+%%----------------------------------------------------------------------------
+%% Internal Helpers
+%%----------------------------------------------------------------------------
+cmd(Command) ->
+ cmd(Command, true).
+
+cmd(Command, ThrowIfMissing) ->
+ Exec = hd(string:tokens(Command, " ")),
+ case {ThrowIfMissing, os:find_executable(Exec)} of
+ {true, false} ->
+ throw({command_not_found, Exec});
+ {false, false} ->
+ {error, command_not_found};
+ {_, _Filename} ->
+ os:cmd(Command)
+ end.
+
+default_linux_pagesize(CmdOutput) ->
+ rabbit_log:warning(
+ "Failed to get memory page size, using 4096. Reason: ~s",
+ [CmdOutput]),
+ 4096.
+
+get_linux_pagesize() ->
+ case cmd("getconf PAGESIZE", false) of
+ {error, command_not_found} ->
+ default_linux_pagesize("getconf not found in PATH");
+ CmdOutput ->
+ case re:run(CmdOutput, "^[0-9]+", [{capture, first, list}]) of
+ {match, [Match]} -> list_to_integer(Match);
+ _ ->
+ default_linux_pagesize(CmdOutput)
+ end
+ end.
+
+%% get_total_memory(OS) -> Total
+%% Windows and Freebsd code based on: memsup:get_memory_usage/1
+%% Original code was part of OTP and released under "Erlang Public License".
+
+get_total_memory({unix, darwin}) ->
+ sysctl("hw.memsize");
+
+get_total_memory({unix, freebsd}) ->
+ PageSize = sysctl("vm.stats.vm.v_page_size"),
+ PageCount = sysctl("vm.stats.vm.v_page_count"),
+ PageCount * PageSize;
+
+get_total_memory({unix, openbsd}) ->
+ sysctl("hw.usermem");
+
+get_total_memory({win32, _OSname}) ->
+ [Result|_] = os_mon_sysinfo:get_mem_info(),
+ {ok, [_MemLoad, TotPhys, _AvailPhys, _TotPage, _AvailPage, _TotV, _AvailV],
+ _RestStr} =
+ io_lib:fread("~d~d~d~d~d~d~d", Result),
+ TotPhys;
+
+get_total_memory({unix, linux}) ->
+ File = read_proc_file("/proc/meminfo"),
+ Lines = string:tokens(File, "\n"),
+ Dict = dict:from_list(lists:map(fun parse_line_linux/1, Lines)),
+ dict:fetch('MemTotal', Dict);
+
+get_total_memory({unix, sunos}) ->
+ File = cmd("/usr/sbin/prtconf"),
+ Lines = string:tokens(File, "\n"),
+ Dict = dict:from_list(lists:map(fun parse_line_sunos/1, Lines)),
+ dict:fetch('Memory size', Dict);
+
+get_total_memory({unix, aix}) ->
+ File = cmd("/usr/bin/vmstat -v"),
+ Lines = string:tokens(File, "\n"),
+ Dict = dict:from_list(lists:map(fun parse_line_aix/1, Lines)),
+ dict:fetch('memory pages', Dict) * 4096;
+
+get_total_memory(_OsType) ->
+ unknown.
+
+%% A line looks like "MemTotal: 502968 kB"
+%% or (with broken OS/modules) "Readahead 123456 kB"
+parse_line_linux(Line) ->
+ {Name, Value, UnitRest} =
+ case string:tokens(Line, ":") of
+ %% no colon in the line
+ [S] ->
+ [K, RHS] = re:split(S, "\s", [{parts, 2}, {return, list}]),
+ [V | Unit] = string:tokens(RHS, " "),
+ {K, V, Unit};
+ [K, RHS | _Rest] ->
+ [V | Unit] = string:tokens(RHS, " "),
+ {K, V, Unit}
+ end,
+ Value1 = case UnitRest of
+ [] -> list_to_integer(Value); %% no units
+ ["kB"] -> list_to_integer(Value) * 1024;
+ ["KB"] -> list_to_integer(Value) * 1024
+ end,
+ {list_to_atom(Name), Value1}.
+
+%% A line looks like "Memory size: 1024 Megabytes"
+parse_line_sunos(Line) ->
+ case string:tokens(Line, ":") of
+ [Name, RHS | _Rest] ->
+ [Value1 | UnitsRest] = string:tokens(RHS, " "),
+ Value2 = case UnitsRest of
+ ["Gigabytes"] ->
+ list_to_integer(Value1) * ?ONE_MiB * 1024;
+ ["Megabytes"] ->
+ list_to_integer(Value1) * ?ONE_MiB;
+ ["Kilobytes"] ->
+ list_to_integer(Value1) * 1024;
+ _ ->
+ Value1 ++ UnitsRest %% no known units
+ end,
+ {list_to_atom(Name), Value2};
+ [Name] -> {list_to_atom(Name), none}
+ end.
+
+%% Lines look like " 12345 memory pages"
+%% or " 80.1 maxpin percentage"
+parse_line_aix(Line) ->
+ [Value | NameWords] = string:tokens(Line, " "),
+ Name = string:join(NameWords, " "),
+ {list_to_atom(Name),
+ case lists:member($., Value) of
+ true -> trunc(list_to_float(Value));
+ false -> list_to_integer(Value)
+ end}.
+
+sysctl(Def) ->
+ list_to_integer(cmd("/usr/bin/env sysctl -n " ++ Def) -- "\n").
+
+%% file:read_file does not work on files in /proc as it seems to get
+%% the size of the file first and then read that many bytes. But files
+%% in /proc always have length 0, we just have to read until we get
+%% eof.
+read_proc_file(File) ->
+ {ok, IoDevice} = file:open(File, [read, raw]),
+ Res = read_proc_file(IoDevice, []),
+ _ = file:close(IoDevice),
+ lists:flatten(lists:reverse(Res)).
+
+-define(BUFFER_SIZE, 1024).
+read_proc_file(IoDevice, Acc) ->
+ case file:read(IoDevice, ?BUFFER_SIZE) of
+ {ok, Res} -> read_proc_file(IoDevice, [Res | Acc]);
+ eof -> Acc
+ end.
diff --git a/deps/rabbit_common/src/worker_pool.erl b/deps/rabbit_common/src/worker_pool.erl
new file mode 100644
index 0000000000..f81e924653
--- /dev/null
+++ b/deps/rabbit_common/src/worker_pool.erl
@@ -0,0 +1,172 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(worker_pool).
+
+%% Generic worker pool manager.
+%%
+%% Submitted jobs are functions. They can be executed synchronously
+%% (using worker_pool:submit/1, worker_pool:submit/2) or asynchronously
+%% (using worker_pool:submit_async/1).
+%%
+%% We typically use the worker pool if we want to limit the maximum
+%% parallelism of some job. We are not trying to dodge the cost of
+%% creating Erlang processes.
+%%
+%% Supports nested submission of jobs and two execution modes:
+%% 'single' and 'reuse'. Jobs executed in 'single' mode are invoked in
+%% a one-off process. Those executed in 'reuse' mode are invoked in a
+%% worker process out of the pool. Nested jobs are always executed
+%% immediately in current worker process.
+%%
+%% 'single' mode is offered to work around a bug in Mnesia: after
+%% network partitions reply messages for prior failed requests can be
+%% sent to Mnesia clients - a reused worker pool process can crash on
+%% receiving one.
+%%
+%% Caller submissions are enqueued internally. When the next worker
+%% process is available, it communicates it to the pool and is
+%% assigned a job to execute. If job execution fails with an error, no
+%% response is returned to the caller.
+%%
+%% Worker processes prioritise certain command-and-control messages
+%% from the pool.
+%%
+%% Future improvement points: job prioritisation.
+
+-behaviour(gen_server2).
+
+-export([start_link/1,
+ submit/1, submit/2, submit/3,
+ submit_async/1, submit_async/2,
+ dispatch_sync/1, dispatch_sync/2,
+ ready/2,
+ idle/2,
+ default_pool/0]).
+
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3]).
+
+%%----------------------------------------------------------------------------
+
+-type mfargs() :: {atom(), atom(), [any()]}.
+
+-spec start_link(atom()) -> {'ok', pid()} | {'error', any()}.
+-spec submit(fun (() -> A) | mfargs()) -> A.
+-spec submit(fun (() -> A) | mfargs(), 'reuse' | 'single') -> A.
+-spec submit(atom(), fun (() -> A) | mfargs(), 'reuse' | 'single') -> A.
+-spec submit_async(fun (() -> any()) | mfargs()) -> 'ok'.
+-spec dispatch_sync(fun(() -> any()) | mfargs()) -> 'ok'.
+-spec ready(atom(), pid()) -> 'ok'.
+-spec idle(atom(), pid()) -> 'ok'.
+-spec default_pool() -> atom().
+
+%%----------------------------------------------------------------------------
+
+-define(DEFAULT_POOL, ?MODULE).
+-define(HIBERNATE_AFTER_MIN, 1000).
+-define(DESIRED_HIBERNATE, 10000).
+
+-record(state, { available, pending }).
+
+%%----------------------------------------------------------------------------
+
+start_link(Name) -> gen_server2:start_link({local, Name}, ?MODULE, [],
+ [{timeout, infinity}]).
+
+submit(Fun) ->
+ submit(?DEFAULT_POOL, Fun, reuse).
+
+%% ProcessModel =:= single is for working around the mnesia_locker bug.
+submit(Fun, ProcessModel) ->
+ submit(?DEFAULT_POOL, Fun, ProcessModel).
+
+submit(Server, Fun, ProcessModel) ->
+ case get(worker_pool_worker) of
+ true -> worker_pool_worker:run(Fun);
+ _ -> Pid = gen_server2:call(Server, {next_free, self()}, infinity),
+ worker_pool_worker:submit(Pid, Fun, ProcessModel)
+ end.
+
+submit_async(Fun) -> submit_async(?DEFAULT_POOL, Fun).
+
+submit_async(Server, Fun) -> gen_server2:cast(Server, {run_async, Fun}).
+
+dispatch_sync(Fun) ->
+ dispatch_sync(?DEFAULT_POOL, Fun).
+
+dispatch_sync(Server, Fun) ->
+ Pid = gen_server2:call(Server, {next_free, self()}, infinity),
+ worker_pool_worker:submit_async(Pid, Fun).
+
+ready(Server, WPid) -> gen_server2:cast(Server, {ready, WPid}).
+
+idle(Server, WPid) -> gen_server2:cast(Server, {idle, WPid}).
+
+default_pool() -> ?DEFAULT_POOL.
+
+%%----------------------------------------------------------------------------
+
+init([]) ->
+ {ok, #state { pending = queue:new(), available = ordsets:new() }, hibernate,
+ {backoff, ?HIBERNATE_AFTER_MIN, ?HIBERNATE_AFTER_MIN, ?DESIRED_HIBERNATE}}.
+
+handle_call({next_free, CPid}, From, State = #state { available = [],
+ pending = Pending }) ->
+ {noreply, State#state{pending = queue:in({next_free, From, CPid}, Pending)},
+ hibernate};
+handle_call({next_free, CPid}, _From, State = #state { available =
+ [WPid | Avail1] }) ->
+ worker_pool_worker:next_job_from(WPid, CPid),
+ {reply, WPid, State #state { available = Avail1 }, hibernate};
+
+handle_call(Msg, _From, State) ->
+ {stop, {unexpected_call, Msg}, State}.
+
+handle_cast({ready, WPid}, State) ->
+ erlang:monitor(process, WPid),
+ handle_cast({idle, WPid}, State);
+
+handle_cast({idle, WPid}, State = #state { available = Avail,
+ pending = Pending }) ->
+ {noreply,
+ case queue:out(Pending) of
+ {empty, _Pending} ->
+ State #state { available = ordsets:add_element(WPid, Avail) };
+ {{value, {next_free, From, CPid}}, Pending1} ->
+ worker_pool_worker:next_job_from(WPid, CPid),
+ gen_server2:reply(From, WPid),
+ State #state { pending = Pending1 };
+ {{value, {run_async, Fun}}, Pending1} ->
+ worker_pool_worker:submit_async(WPid, Fun),
+ State #state { pending = Pending1 }
+ end, hibernate};
+
+handle_cast({run_async, Fun}, State = #state { available = [],
+ pending = Pending }) ->
+ {noreply, State #state { pending = queue:in({run_async, Fun}, Pending)},
+ hibernate};
+handle_cast({run_async, Fun}, State = #state { available = [WPid | Avail1] }) ->
+ worker_pool_worker:submit_async(WPid, Fun),
+ {noreply, State #state { available = Avail1 }, hibernate};
+
+handle_cast(Msg, State) ->
+ {stop, {unexpected_cast, Msg}, State}.
+
+handle_info({'DOWN', _MRef, process, WPid, _Reason},
+ State = #state { available = Avail }) ->
+ {noreply, State #state { available = ordsets:del_element(WPid, Avail) },
+ hibernate};
+
+handle_info(Msg, State) ->
+ {stop, {unexpected_info, Msg}, State}.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+terminate(_Reason, State) ->
+ State.
diff --git a/deps/rabbit_common/src/worker_pool_sup.erl b/deps/rabbit_common/src/worker_pool_sup.erl
new file mode 100644
index 0000000000..96dbbb2357
--- /dev/null
+++ b/deps/rabbit_common/src/worker_pool_sup.erl
@@ -0,0 +1,69 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(worker_pool_sup).
+
+-behaviour(supervisor).
+
+-export([start_link/0, start_link/1, start_link/2]).
+
+-export([init/1]).
+
+-export([default_pool_size/0]).
+
+%%----------------------------------------------------------------------------
+
+-spec start_link() -> rabbit_types:ok_pid_or_error().
+-spec start_link(non_neg_integer()) -> rabbit_types:ok_pid_or_error().
+-spec start_link(non_neg_integer(), atom())
+ -> rabbit_types:ok_pid_or_error().
+
+%%----------------------------------------------------------------------------
+
+start_link() ->
+ Size = default_pool_size(),
+ start_link(Size).
+
+start_link(PoolSize) ->
+ rabbit_log:info("Will use ~p processes for default worker pool", [PoolSize]),
+ start_link(PoolSize, worker_pool:default_pool()).
+
+start_link(PoolSize, PoolName) ->
+ rabbit_log:info("Starting worker pool '~p' with ~p processes in it", [PoolName, PoolSize]),
+ SupName = list_to_atom(atom_to_list(PoolName) ++ "_sup"),
+ supervisor:start_link({local, SupName}, ?MODULE, [PoolSize, PoolName]).
+
+%%----------------------------------------------------------------------------
+
+init([PoolSize, PoolName]) ->
+ %% we want to survive up to 1K of worker restarts per second,
+ %% e.g. when a large worker pool used for network connections
+ %% encounters a network failure. This is the case in the LDAP authentication
+ %% backend plugin.
+ {ok, {{one_for_one, 1000, 1},
+ [{worker_pool, {worker_pool, start_link, [PoolName]}, transient,
+ 16#ffffffff, worker, [worker_pool]} |
+ [{N, {worker_pool_worker, start_link, [PoolName]}, transient,
+ 16#ffffffff, worker, [worker_pool_worker]}
+ || N <- lists:seq(1, PoolSize)]]}}.
+
+%%
+%% Implementation
+%%
+
+-spec default_pool_size() -> integer().
+
+default_pool_size() ->
+ case rabbit_misc:get_env(rabbit, default_worker_pool_size, undefined) of
+ N when is_integer(N) -> N;
+ _ -> guess_default_pool_size()
+ end.
+
+-spec guess_default_pool_size() -> integer().
+
+guess_default_pool_size() ->
+ erlang:system_info(schedulers).
diff --git a/deps/rabbit_common/src/worker_pool_worker.erl b/deps/rabbit_common/src/worker_pool_worker.erl
new file mode 100644
index 0000000000..79436e0773
--- /dev/null
+++ b/deps/rabbit_common/src/worker_pool_worker.erl
@@ -0,0 +1,192 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+%%
+
+-module(worker_pool_worker).
+
+%% Executes jobs (functions) submitted to a worker pool with worker_pool:submit/1,
+%% worker_pool:submit/2 or worker_pool:submit_async/1.
+%%
+%% See worker_pool for an overview.
+
+-behaviour(gen_server2).
+
+-export([start_link/1, next_job_from/2, submit/3, submit_async/2,
+ run/1]).
+
+-export([set_maximum_since_use/2]).
+-export([set_timeout/2, set_timeout/3, clear_timeout/1]).
+
+-export([init/1, handle_call/3, handle_cast/2, handle_info/2,
+ terminate/2, code_change/3, prioritise_cast/3]).
+
+%%----------------------------------------------------------------------------
+
+-type mfargs() :: {atom(), atom(), [any()]}.
+
+-spec start_link(atom) -> {'ok', pid()} | {'error', any()}.
+-spec next_job_from(pid(), pid()) -> 'ok'.
+-spec submit(pid(), fun (() -> A) | mfargs(), 'reuse' | 'single') -> A.
+-spec submit_async(pid(), fun (() -> any()) | mfargs()) -> 'ok'.
+-spec run(fun (() -> A)) -> A; (mfargs()) -> any().
+-spec set_maximum_since_use(pid(), non_neg_integer()) -> 'ok'.
+
+%%----------------------------------------------------------------------------
+
+-define(HIBERNATE_AFTER_MIN, 1000).
+-define(DESIRED_HIBERNATE, 10000).
+
+%%----------------------------------------------------------------------------
+
+start_link(PoolName) ->
+ gen_server2:start_link(?MODULE, [PoolName], [{timeout, infinity}]).
+
+next_job_from(Pid, CPid) ->
+ gen_server2:cast(Pid, {next_job_from, CPid}).
+
+submit(Pid, Fun, ProcessModel) ->
+ gen_server2:call(Pid, {submit, Fun, self(), ProcessModel}, infinity).
+
+submit_async(Pid, Fun) ->
+ gen_server2:cast(Pid, {submit_async, Fun, self()}).
+
+set_maximum_since_use(Pid, Age) ->
+ gen_server2:cast(Pid, {set_maximum_since_use, Age}).
+
+run({M, F, A}) -> apply(M, F, A);
+run(Fun) -> Fun().
+
+run(Fun, reuse) ->
+ run(Fun);
+run(Fun, single) ->
+ Self = self(),
+ Ref = make_ref(),
+ spawn_link(fun () ->
+ put(worker_pool_worker, true),
+ Self ! {Ref, run(Fun)},
+ unlink(Self)
+ end),
+ receive
+ {Ref, Res} -> Res
+ end.
+
+%%----------------------------------------------------------------------------
+
+init([PoolName]) ->
+ ok = file_handle_cache:register_callback(?MODULE, set_maximum_since_use,
+ [self()]),
+ ok = worker_pool:ready(PoolName, self()),
+ put(worker_pool_worker, true),
+ put(worker_pool_name, PoolName),
+ {ok, undefined, hibernate,
+ {backoff, ?HIBERNATE_AFTER_MIN, ?HIBERNATE_AFTER_MIN, ?DESIRED_HIBERNATE}}.
+
+prioritise_cast({set_maximum_since_use, _Age}, _Len, _State) -> 8;
+prioritise_cast({next_job_from, _CPid}, _Len, _State) -> 7;
+prioritise_cast(_Msg, _Len, _State) -> 0.
+
+handle_call({submit, Fun, CPid, ProcessModel}, From, undefined) ->
+ {noreply, {job, CPid, From, Fun, ProcessModel}, hibernate};
+
+handle_call({submit, Fun, CPid, ProcessModel}, From, {from, CPid, MRef}) ->
+ erlang:demonitor(MRef),
+ gen_server2:reply(From, run(Fun, ProcessModel)),
+ ok = worker_pool:idle(get(worker_pool_name), self()),
+ {noreply, undefined, hibernate};
+
+handle_call(Msg, _From, State) ->
+ {stop, {unexpected_call, Msg}, State}.
+
+handle_cast({next_job_from, CPid}, undefined) ->
+ MRef = erlang:monitor(process, CPid),
+ {noreply, {from, CPid, MRef}, hibernate};
+
+handle_cast({next_job_from, CPid}, {job, CPid, From, Fun, ProcessModel}) ->
+ gen_server2:reply(From, run(Fun, ProcessModel)),
+ ok = worker_pool:idle(get(worker_pool_name), self()),
+ {noreply, undefined, hibernate};
+
+handle_cast({submit_async, Fun, _CPid}, undefined) ->
+ run(Fun),
+ ok = worker_pool:idle(get(worker_pool_name), self()),
+ {noreply, undefined, hibernate};
+
+handle_cast({submit_async, Fun, CPid}, {from, CPid, MRef}) ->
+ erlang:demonitor(MRef),
+ run(Fun),
+ ok = worker_pool:idle(get(worker_pool_name), self()),
+ {noreply, undefined, hibernate};
+
+handle_cast({set_maximum_since_use, Age}, State) ->
+ ok = file_handle_cache:set_maximum_since_use(Age),
+ {noreply, State, hibernate};
+
+handle_cast(Msg, State) ->
+ {stop, {unexpected_cast, Msg}, State}.
+
+handle_info({'DOWN', MRef, process, CPid, _Reason}, {from, CPid, MRef}) ->
+ ok = worker_pool:idle(get(worker_pool_name), self()),
+ {noreply, undefined, hibernate};
+
+handle_info({'DOWN', _MRef, process, _Pid, _Reason}, State) ->
+ {noreply, State, hibernate};
+
+handle_info({timeout, Key, Fun}, State) ->
+ clear_timeout(Key),
+ Fun(),
+ {noreply, State, hibernate};
+
+handle_info(Msg, State) ->
+ {stop, {unexpected_info, Msg}, State}.
+
+code_change(_OldVsn, State, _Extra) ->
+ {ok, State}.
+
+terminate(_Reason, State) ->
+ State.
+
+-spec set_timeout(non_neg_integer(), fun(() -> any())) ->
+ {ok, reference()}.
+set_timeout(Time, Fun) ->
+ Key = make_ref(),
+ set_timeout(Key, Time, Fun).
+
+-spec set_timeout(Key, non_neg_integer(), fun(() -> any())) ->
+ {ok, Key} when Key :: any().
+set_timeout(Key, Time, Fun) ->
+ Timeouts = get_timeouts(),
+ set_timeout(Key, Time, Fun, Timeouts).
+
+-spec clear_timeout(any()) -> ok.
+clear_timeout(Key) ->
+ NewTimeouts = cancel_timeout(Key, get_timeouts()),
+ put(timeouts, NewTimeouts),
+ ok.
+
+get_timeouts() ->
+ case get(timeouts) of
+ undefined -> dict:new();
+ Dict -> Dict
+ end.
+
+set_timeout(Key, Time, Fun, Timeouts) ->
+ _ = cancel_timeout(Key, Timeouts),
+ TRef = erlang:send_after(Time, self(), {timeout, Key, Fun}),
+ NewTimeouts = dict:store(Key, TRef, Timeouts),
+ put(timeouts, NewTimeouts),
+ {ok, Key}.
+
+cancel_timeout(Key, Timeouts) ->
+ case dict:find(Key, Timeouts) of
+ {ok, TRef} ->
+ _ = erlang:cancel_timer(TRef),
+ receive {timeout, Key, _} -> ok
+ after 0 -> ok
+ end,
+ dict:erase(Key, Timeouts);
+ error ->
+ Timeouts
+ end.