Move most of shell scripts to Erlang code

A large part of the rabbitmq-server(8) and CLI scripts, both
Bourne-shell and Windows Batch versions, was moved to Erlang code and
the RabbitMQ startup procedure was reorganized to be closer to a regular
Erlang application.

A new application called `rabbitmq_prelaunch` is responsible for:

  1. Querying the environment variables to initialize important
     variables (using the new `rabbit_env` module in rabbitmq-common).
  2. Checking the compatibility with the Erlang/OTP runtime.
  3. Configuring Erlang distribution.
  5. Writing the PID file.

The application is started early (i.e. it is started before `rabbit`).

The `rabbit` application runs the second half of the prelaunch sequence
at the beginning of the application `start()` function. This second
phase is responsible for the following steps:

  1. Preparing the feature flags registry.
  2. Reading and validating the configuration.
  3. Configuring logging.
  4. Running the various cluster checks.

In addition to this prelaunch sequence, the `rabbit` application start
procedure ends with a "postlaunch" sequence which takes care of
starting enabled plugins.

Thanks to this, RabbitMQ can be started with `application:start(rabbit)`
as any other Erlang application. The only caveats are:

  * Mnesia must be stopped at the time `rabbit_prelaunch` is started,
    and must remain stopped when `rabbit` is started, to allow the
    Erlang distribution setup and cluster checks. `rabbit` takes care of
    starting Mnesia.

  * Likewise for Ra, because it relies on the `ra` application
    environment to be configured.

Transitioning from scripts to Erlang code has the following benefits:

  * RabbitMQ start behavior should be identical between Unix and
    Windows. Also, features should be on par now. For instance, RabbitMQ
    now writes a PID file on Windows, like it always did on Unix-based
    systems.

  * The difference between published packages and a development
    environment are greatly reduced. In fact, we removed all the "if
    this is a dev working copy, then ..." blocks.

    As part of that, the `rabbit` application is now treated like its
    plugins: it is packaged as an `.ez` archive and written to the
    `plugins` directory (even though it is not technically a plugin).

    Also in a development copy, the CLI is copied to the top-level
    project. So when testing a plugin for instance, the CLI to use is
    `sbin/rabbitmqctl` in the current directory, not the master copy in
    `rabbit/scripts`.

  * As a consequence of the previous two points, maintaining and testing
    on Windows is now made easy. It should even be possible to setup CI
    on Windows.

  * There are less issues with paths containing non-US-ASCII characters,
    which can happen on Windows because RabbitMQ stores its data in user
    directories by default.

This process brings at least one more benefit: we now have early logging
during this prelaunch phase, which eases diagnostics and debugging.

There are also behavior changes:

  * The new format configuration files used to be converted to an
    Erlang-term-based file by the Cuttlefish CLI. To do that,
    configuration schemas were copied to a temporary directory and the
    generated configuration file was written to RabbitMQ data directory.

    Now, Cuttlefish is used as a library: everything happens in memory.
    No schemas are copied, no generated configuration is written to
    disk.

  * The PID file is removed when the Erlang VM exits.

  * The `rabbit_config` module was trimmed significantly because most of
    the configuration handling is done in `rabbit_prelaunch_conf` now.

  * The RabbitMQ nodename does not appear on the command line, therefore
    it is missing from ps(1) and top(1) output.

  * The `rabbit:start()` function will probably behave differently in
    some ways because it defers everything to the Erlang application
    controller (instead of reimplementing it).
This commit is contained in:
Jean-Sébastien Pédron 2019-05-15 16:27:51 +02:00
parent 5e6043ac59
commit 68c30553cc
No known key found for this signature in database
GPG Key ID: 39E99761A5FD94CC
41 changed files with 1969 additions and 2295 deletions

6
.gitignore vendored
View File

@ -11,13 +11,17 @@ MnesiaCore.*
/debug/
/doc/
/ebin/
/escript
/escript.lock
/etc/
/logs/
/plugins/
/plugins.lock
/test/ct.cover.spec
/test/config_schema_SUITE_data/schema/**
/xrefr
/escript
/sbin/
/sbin.lock
rabbit.d
# Generated documentation.

View File

@ -133,11 +133,19 @@ define PROJECT_ENV
]
endef
LOCAL_DEPS = sasl mnesia os_mon inets compiler public_key crypto ssl syntax_tools xmerl
# With Erlang.mk default behavior, the value of `$(APPS_DIR)` is always
# relative to the top-level executed Makefile. In our case, it could be
# a plugin for instance. However, the rabbitmq_prelaunch application is
# in this repository, not the plugin's. That's why we need to override
# this value here.
APPS_DIR := $(CURDIR)/apps
LOCAL_DEPS = sasl rabbitmq_prelaunch os_mon inets compiler public_key crypto ssl syntax_tools xmerl
BUILD_DEPS = rabbitmq_cli syslog
DEPS = ranch lager rabbit_common ra sysmon_handler stdout_formatter recon observer_cli
DEPS = cuttlefish ranch lager rabbit_common ra sysmon_handler stdout_formatter recon observer_cli
TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper
dep_cuttlefish = hex 2.2.0
dep_syslog = git https://github.com/schlagert/syslog 3.4.5
define usage_xml_to_erl
@ -164,6 +172,9 @@ ERLANG_MK_COMMIT = rabbitmq-tmp
include rabbitmq-components.mk
include erlang.mk
# See above why we mess with `$(APPS_DIR)`.
unexport APPS_DIR
ifeq ($(strip $(BATS)),)
BATS := $(ERLANG_MK_TMP)/bats/bin/bats
endif
@ -240,20 +251,6 @@ USE_PROPER_QC := $(shell $(ERL) -eval 'io:format({module, proper} =:= code:ensur
RMQ_ERLC_OPTS += $(if $(filter true,$(USE_PROPER_QC)),-Duse_proper_qc)
endif
.PHONY: copy-escripts clean-extra-sources clean-escripts
CLI_ESCRIPTS_DIR = escript
copy-escripts:
$(gen_verbose) $(MAKE) -C $(DEPS_DIR)/rabbitmq_cli install \
PREFIX="$(abspath $(CLI_ESCRIPTS_DIR))" \
DESTDIR=
clean:: clean-escripts
clean-escripts:
$(gen_verbose) rm -rf "$(CLI_ESCRIPTS_DIR)"
# --------------------------------------------------------------------
# Documentation.
# --------------------------------------------------------------------
@ -297,5 +294,3 @@ distclean:: distclean-manpages
distclean-manpages::
$(gen_verbose) rm -f $(WEB_MANPAGES)
app-build: copy-escripts

12
apps/rabbitmq_prelaunch/.gitignore vendored Normal file
View File

@ -0,0 +1,12 @@
*~
.sw?
.*.sw?
*.beam
*.coverdata
/ebin/
/.erlang.mk/
/rabbitmq_prelaunch.d
/xrefr
# Dialyzer
*.plt

View File

@ -0,0 +1,11 @@
PROJECT = rabbitmq_prelaunch
PROJECT_DESCRIPTION = RabbitMQ prelaunch setup
PROJECT_VERSION = 1.0.0
PROJECT_MOD = rabbit_prelaunch_app
DEPS = rabbit_common lager
DEP_PLUGINS = rabbit_common/mk/rabbitmq-build.mk
include ../../rabbitmq-components.mk
include ../../erlang.mk

View File

@ -0,0 +1,274 @@
-module(rabbit_prelaunch).
-include_lib("eunit/include/eunit.hrl").
-export([run_prelaunch_first_phase/0,
assert_mnesia_is_stopped/0,
get_context/0,
get_boot_state/0,
set_boot_state/1,
is_boot_state_reached/1,
wait_for_boot_state/1,
wait_for_boot_state/2,
get_stop_reason/0,
set_stop_reason/1,
clear_stop_reason/0,
is_initial_pass/0,
initial_pass_finished/0,
shutdown_func/1]).
-define(PT_KEY_CONTEXT, {?MODULE, context}).
-define(PT_KEY_BOOT_STATE, {?MODULE, boot_state}).
-define(PT_KEY_INITIAL_PASS, {?MODULE, initial_pass_finished}).
-define(PT_KEY_SHUTDOWN_FUNC, {?MODULE, chained_shutdown_func}).
-define(PT_KEY_STOP_REASON, {?MODULE, stop_reason}).
run_prelaunch_first_phase() ->
try
do_run()
catch
throw:{error, _} = Error ->
rabbit_prelaunch_errors:log_error(Error),
set_stop_reason(Error),
set_boot_state(stopped),
Error;
Class:Exception:Stacktrace ->
rabbit_prelaunch_errors:log_exception(
Class, Exception, Stacktrace),
Error = {error, Exception},
set_stop_reason(Error),
set_boot_state(stopped),
Error
end.
do_run() ->
%% Indicate RabbitMQ is booting.
clear_stop_reason(),
set_boot_state(booting),
%% Configure dbg if requested.
rabbit_prelaunch_early_logging:enable_quick_dbg(rabbit_env:dbg_config()),
%% We assert Mnesia is stopped before we run the prelaunch
%% phases.
%%
%% We need this because our cluster consistency check (in the second
%% phase) depends on Mnesia not being started before it has a chance
%% to run.
%%
%% Also, in the initial pass, we don't want Mnesia to run before
%% Erlang distribution is configured.
assert_mnesia_is_stopped(),
%% Get informations to setup logging.
Context0 = rabbit_env:get_context_before_logging_init(),
?assertMatch(#{}, Context0),
%% Setup logging for the prelaunch phase.
ok = rabbit_prelaunch_early_logging:setup_early_logging(Context0, true),
IsInitialPass = is_initial_pass(),
case IsInitialPass of
true ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug(
"== Prelaunch phase [1/2] (initial pass) =="),
rabbit_log_prelaunch:debug("");
false ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Prelaunch phase [1/2] =="),
rabbit_log_prelaunch:debug("")
end,
rabbit_env:log_process_env(),
%% Load rabbitmq-env.conf, redo logging setup and continue.
Context1 = rabbit_env:get_context_after_logging_init(Context0),
?assertMatch(#{}, Context1),
ok = rabbit_prelaunch_early_logging:setup_early_logging(Context1, true),
rabbit_env:log_process_env(),
%% Complete context now that we have the final environment loaded.
Context2 = rabbit_env:get_context_after_reloading_env(Context1),
?assertMatch(#{}, Context2),
store_context(Context2),
rabbit_env:log_context(Context2),
ok = setup_shutdown_func(),
Context = Context2#{initial_pass => IsInitialPass},
rabbit_env:context_to_code_path(Context),
rabbit_env:context_to_app_env_vars(Context),
%% 1. Erlang/OTP compatibility check.
ok = rabbit_prelaunch_erlang_compat:check(Context),
%% 2. Erlang distribution check + start.
ok = rabbit_prelaunch_dist:setup(Context),
%% 3. Write PID file.
rabbit_log_prelaunch:debug(""),
_ = write_pid_file(Context),
ignore.
assert_mnesia_is_stopped() ->
?assertNot(lists:keymember(mnesia, 1, application:which_applications())).
store_context(Context) when is_map(Context) ->
persistent_term:put(?PT_KEY_CONTEXT, Context).
get_context() ->
case persistent_term:get(?PT_KEY_CONTEXT, undefined) of
undefined -> undefined;
Context -> Context#{initial_pass => is_initial_pass()}
end.
get_boot_state() ->
persistent_term:get(?PT_KEY_BOOT_STATE, stopped).
set_boot_state(stopped) ->
rabbit_log_prelaunch:debug("Change boot state to `stopped`"),
persistent_term:erase(?PT_KEY_BOOT_STATE);
set_boot_state(BootState) ->
rabbit_log_prelaunch:debug("Change boot state to `~s`", [BootState]),
?assert(is_boot_state_valid(BootState)),
persistent_term:put(?PT_KEY_BOOT_STATE, BootState).
wait_for_boot_state(BootState) ->
wait_for_boot_state(BootState, infinity).
wait_for_boot_state(BootState, Timeout) ->
?assert(is_boot_state_valid(BootState)),
wait_for_boot_state1(BootState, Timeout).
wait_for_boot_state1(BootState, infinity = Timeout) ->
case is_boot_state_reached(BootState) of
true -> ok;
false -> wait_for_boot_state1(BootState, Timeout)
end;
wait_for_boot_state1(BootState, Timeout)
when is_integer(Timeout) andalso Timeout >= 0 ->
case is_boot_state_reached(BootState) of
true -> ok;
false -> Wait = 200,
timer:sleep(Wait),
wait_for_boot_state1(BootState, Timeout - Wait)
end;
wait_for_boot_state1(_, _) ->
{error, timeout}.
boot_state_idx(stopped) -> 0;
boot_state_idx(booting) -> 1;
boot_state_idx(ready) -> 2;
boot_state_idx(stopping) -> 3;
boot_state_idx(_) -> undefined.
is_boot_state_valid(BootState) ->
is_integer(boot_state_idx(BootState)).
is_boot_state_reached(TargetBootState) ->
is_boot_state_reached(get_boot_state(), TargetBootState).
is_boot_state_reached(CurrentBootState, CurrentBootState) ->
true;
is_boot_state_reached(stopping, stopped) ->
false;
is_boot_state_reached(_CurrentBootState, stopped) ->
true;
is_boot_state_reached(stopped, _TargetBootState) ->
true;
is_boot_state_reached(CurrentBootState, TargetBootState) ->
boot_state_idx(TargetBootState) =< boot_state_idx(CurrentBootState).
get_stop_reason() ->
persistent_term:get(?PT_KEY_STOP_REASON, undefined).
set_stop_reason(Reason) ->
case get_stop_reason() of
undefined ->
rabbit_log_prelaunch:debug("Set stop reason to: ~p", [Reason]),
persistent_term:put(?PT_KEY_STOP_REASON, Reason);
_ ->
ok
end.
clear_stop_reason() ->
persistent_term:erase(?PT_KEY_STOP_REASON).
is_initial_pass() ->
not persistent_term:get(?PT_KEY_INITIAL_PASS, false).
initial_pass_finished() ->
persistent_term:put(?PT_KEY_INITIAL_PASS, true).
setup_shutdown_func() ->
ThisMod = ?MODULE,
ThisFunc = shutdown_func,
ExistingShutdownFunc = application:get_env(kernel, shutdown_func),
case ExistingShutdownFunc of
{ok, {ThisMod, ThisFunc}} ->
ok;
{ok, {ExistingMod, ExistingFunc}} ->
rabbit_log_prelaunch:debug(
"Setting up kernel shutdown function: ~s:~s/1 "
"(chained with ~s:~s/1)",
[ThisMod, ThisFunc, ExistingMod, ExistingFunc]),
ok = persistent_term:put(
?PT_KEY_SHUTDOWN_FUNC,
ExistingShutdownFunc),
ok = record_kernel_shutdown_func(ThisMod, ThisFunc);
_ ->
rabbit_log_prelaunch:debug(
"Setting up kernel shutdown function: ~s:~s/1",
[ThisMod, ThisFunc]),
ok = record_kernel_shutdown_func(ThisMod, ThisFunc)
end.
record_kernel_shutdown_func(Mod, Func) ->
application:set_env(
kernel, shutdown_func, {Mod, Func},
[{persistent, true}]).
shutdown_func(Reason) ->
rabbit_log_prelaunch:debug(
"Running ~s:shutdown_func() as part of `kernel` shutdown", [?MODULE]),
Context = get_context(),
remove_pid_file(Context),
ChainedShutdownFunc = persistent_term:get(
?PT_KEY_SHUTDOWN_FUNC,
undefined),
case ChainedShutdownFunc of
{ChainedMod, ChainedFunc} -> ChainedMod:ChainedFunc(Reason);
_ -> ok
end.
write_pid_file(#{pid_file := PidFile}) ->
rabbit_log_prelaunch:debug("Writing PID file: ~s", [PidFile]),
case filelib:ensure_dir(PidFile) of
ok ->
OSPid = os:getpid(),
case file:write_file(PidFile, OSPid) of
ok ->
ok;
{error, Reason} = Error ->
rabbit_log_prelaunch:warning(
"Failed to write PID file \"~s\": ~s",
[PidFile, file:format_error(Reason)]),
Error
end;
{error, Reason} = Error ->
rabbit_log_prelaunch:warning(
"Failed to create PID file \"~s\" directory: ~s",
[PidFile, file:format_error(Reason)]),
Error
end;
write_pid_file(_) ->
ok.
remove_pid_file(#{pid_file := PidFile, keep_pid_file_on_exit := true}) ->
rabbit_log_prelaunch:debug("Keeping PID file: ~s", [PidFile]),
ok;
remove_pid_file(#{pid_file := PidFile}) ->
rabbit_log_prelaunch:debug("Deleting PID file: ~s", [PidFile]),
_ = file:delete(PidFile);
remove_pid_file(_) ->
ok.

View File

@ -0,0 +1,11 @@
-module(rabbit_prelaunch_app).
-behaviour(application).
-export([start/2]).
-export([stop/1]).
start(_Type, _Args) ->
rabbit_prelaunch_sup:start_link().
stop(_State) ->
ok.

View File

@ -0,0 +1,102 @@
-module(rabbit_prelaunch_dist).
-export([setup/1]).
setup(#{nodename := Node, nodename_type := NameType} = Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Erlang distribution =="),
rabbit_log_prelaunch:debug("Rqeuested node name: ~s (type: ~s)",
[Node, NameType]),
case node() of
nonode@nohost ->
ok = rabbit_nodes_common:ensure_epmd(),
ok = dist_port_range_check(Context),
ok = dist_port_use_check(Context),
ok = duplicate_node_check(Context),
ok = do_setup(Context);
Node ->
rabbit_log_prelaunch:debug(
"Erlang distribution already running", []),
ok;
Unexpected ->
throw({error, {erlang_dist_running_with_unexpected_nodename,
Unexpected, Node}})
end,
ok.
do_setup(#{nodename := Node, nodename_type := NameType}) ->
rabbit_log_prelaunch:debug("Starting Erlang distribution", []),
case application:get_env(kernel, net_ticktime) of
{ok, Ticktime} when is_integer(Ticktime) andalso Ticktime >= 1 ->
%% The value passed to net_kernel:start/1 is the
%% "minimum transition traffic interval" as defined in
%% net_kernel:set_net_ticktime/1.
MTTI = Ticktime * 1000 div 4,
{ok, _} = net_kernel:start([Node, NameType, MTTI]);
_ ->
{ok, _} = net_kernel:start([Node, NameType])
end,
ok.
%% Check whether a node with the same name is already running
duplicate_node_check(#{split_nodename := {NodeName, NodeHost}}) ->
rabbit_log_prelaunch:debug(
"Checking if node name ~s is already used", [NodeName]),
PrelaunchName = rabbit_nodes:make(
{NodeName ++ "_prelaunch_" ++ os:getpid(),
"localhost"}),
{ok, _} = net_kernel:start([PrelaunchName, shortnames]),
case rabbit_nodes:names(NodeHost) of
{ok, NamePorts} ->
case proplists:is_defined(NodeName, NamePorts) of
true ->
throw({error, {duplicate_node_name, NodeName, NodeHost}});
false ->
net_kernel:stop(),
ok
end;
{error, EpmdReason} ->
throw({error, {epmd_error, NodeHost, EpmdReason}})
end.
dist_port_range_check(#{erlang_dist_tcp_port := DistTcpPort}) ->
rabbit_log_prelaunch:debug(
"Checking if TCP port ~b is valid", [DistTcpPort]),
case DistTcpPort of
_ when DistTcpPort < 1 orelse DistTcpPort > 65535 ->
throw({error, {invalid_dist_port_range, DistTcpPort}});
_ ->
ok
end.
dist_port_use_check(#{split_nodename := {_, NodeHost},
erlang_dist_tcp_port := DistTcpPort}) ->
rabbit_log_prelaunch:debug(
"Checking if TCP port ~b is available", [DistTcpPort]),
dist_port_use_check_ipv4(NodeHost, DistTcpPort).
dist_port_use_check_ipv4(NodeHost, Port) ->
case gen_tcp:listen(Port, [inet, {reuseaddr, true}]) of
{ok, Sock} -> gen_tcp:close(Sock);
{error, einval} -> dist_port_use_check_ipv6(NodeHost, Port);
{error, _} -> dist_port_use_check_fail(Port, NodeHost)
end.
dist_port_use_check_ipv6(NodeHost, Port) ->
case gen_tcp:listen(Port, [inet6, {reuseaddr, true}]) of
{ok, Sock} -> gen_tcp:close(Sock);
{error, _} -> dist_port_use_check_fail(Port, NodeHost)
end.
-spec dist_port_use_check_fail(non_neg_integer(), string()) ->
no_return().
dist_port_use_check_fail(Port, Host) ->
{ok, Names} = rabbit_nodes:names(Host),
case [N || {N, P} <- Names, P =:= Port] of
[] ->
throw({error, {dist_port_already_used, Port, not_erlang, Host}});
[Name] ->
throw({error, {dist_port_already_used, Port, Name, Host}})
end.

View File

@ -0,0 +1,61 @@
-module(rabbit_prelaunch_early_logging).
-export([setup_early_logging/2,
enable_quick_dbg/1,
use_colored_logging/0,
use_colored_logging/1]).
-define(SINK, rabbit_log_prelaunch_lager_event).
setup_early_logging(#{log_levels := undefined} = Context,
LagerEventToStdout) ->
setup_early_logging(Context#{log_levels => get_default_log_level()},
LagerEventToStdout);
setup_early_logging(Context, LagerEventToStdout) ->
case lists:member(?SINK, lager:list_all_sinks()) of
true -> ok;
false -> do_setup_early_logging(Context, LagerEventToStdout)
end.
get_default_log_level() ->
#{"prelaunch" => warning}.
do_setup_early_logging(#{log_levels := LogLevels} = Context,
LagerEventToStdout) ->
LogLevel = case LogLevels of
#{"prelaunch" := Level} -> Level;
#{global := Level} -> Level;
_ -> warning
end,
Colored = use_colored_logging(Context),
ConsoleBackend = lager_console_backend,
ConsoleOptions = [{level, LogLevel}],
application:set_env(lager, colored, Colored),
case LagerEventToStdout of
true ->
lager_app:start_handler(
lager_event, ConsoleBackend, ConsoleOptions);
false ->
ok
end,
lager_app:configure_sink(
?SINK,
[{handlers, [{ConsoleBackend, ConsoleOptions}]}]),
ok.
use_colored_logging() ->
use_colored_logging(rabbit_prelaunch:get_context()).
use_colored_logging(#{log_levels := #{color := true},
output_supports_colors := true}) ->
true;
use_colored_logging(_) ->
false.
enable_quick_dbg(#{dbg_output := Output, dbg_mods := Mods}) ->
case Output of
stdout -> {ok, _} = dbg:tracer();
_ -> {ok, _} = dbg:tracer(port, dbg:trace_port(file, Output))
end,
{ok, _} = dbg:p(all, c),
lists:foreach(fun(M) -> {ok, _} = dbg:tp(M, cx) end, Mods).

View File

@ -0,0 +1,47 @@
-module(rabbit_prelaunch_erlang_compat).
-export([check/1]).
-define(OTP_MINIMUM, "21.3").
-define(ERTS_MINIMUM, "10.3").
check(_Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Erlang/OTP compatibility check =="),
ERTSVer = erlang:system_info(version),
OTPRel = rabbit_misc:otp_release(),
rabbit_log_prelaunch:debug(
"Requiring: Erlang/OTP ~s (ERTS ~s)", [?OTP_MINIMUM, ?ERTS_MINIMUM]),
rabbit_log_prelaunch:debug(
"Running: Erlang/OTP ~s (ERTS ~s)", [OTPRel, ERTSVer]),
case rabbit_misc:version_compare(?ERTS_MINIMUM, ERTSVer, lte) of
true when ?ERTS_MINIMUM =/= ERTSVer ->
rabbit_log_prelaunch:debug(
"Erlang/OTP version requirement satisfied"),
ok;
true when ?ERTS_MINIMUM =:= ERTSVer andalso ?OTP_MINIMUM =< OTPRel ->
%% When a critical regression or bug is found, a new OTP
%% release can be published without changing the ERTS
%% version. For instance, this is the case with R16B03 and
%% R16B03-1.
%%
%% In this case, we compare the release versions
%% alphabetically.
ok;
_ ->
Msg =
"This RabbitMQ version cannot run on Erlang ~s (erts ~s): "
"minimum required version is ~s (erts ~s)",
Args = [OTPRel, ERTSVer, ?OTP_MINIMUM, ?ERTS_MINIMUM],
rabbit_log_prelaunch:error(Msg, Args),
%% Also print to stderr to make this more visible
io:format(standard_error, "Error: " ++ Msg ++ "~n", Args),
Msg2 = rabbit_misc:format(
"Erlang ~s or later is required, started on ~s",
[?OTP_MINIMUM, OTPRel]),
throw({error, {erlang_version_too_old, Msg2}})
end.

View File

@ -0,0 +1,109 @@
-module(rabbit_prelaunch_errors).
-export([format_error/1,
format_exception/3,
log_error/1,
log_exception/3]).
-define(BOOT_FAILED_HEADER,
"\n"
"BOOT FAILED\n"
"===========\n").
-define(BOOT_FAILED_FOOTER,
"\n").
log_error(Error) ->
Message = format_error(Error),
log_message(Message).
format_error({error, {duplicate_node_name, NodeName, NodeHost}}) ->
rabbit_misc:format(
"ERROR: node with name ~p already running on ~p",
[NodeName, NodeHost]);
format_error({error, {epmd_error, NodeHost, EpmdReason}}) ->
rabbit_misc:format(
"ERROR: epmd error for host ~s: ~s",
[NodeHost, rabbit_misc:format_inet_error(EpmdReason)]);
format_error({error, {invalid_dist_port_range, DistTcpPort}}) ->
rabbit_misc:format(
"Invalid Erlang distribution TCP port: ~b", [DistTcpPort]);
format_error({error, {dist_port_already_used, Port, not_erlang, Host}}) ->
rabbit_misc:format(
"ERROR: distribution port ~b in use on ~s "
"(by non-Erlang process?)", [Port, Host]);
format_error({error, {dist_port_already_used, Port, Name, Host}}) ->
rabbit_misc:format(
"ERROR: distribution port ~b in use by ~s@~s", [Port, Name, Host]);
format_error({error, {erlang_dist_running_with_unexpected_nodename,
Unexpected, Node}}) ->
rabbit_misc:format(
"Erlang distribution running with another node name (~s) "
"than the configured one (~s)",
[Unexpected, Node]);
format_error({bad_config_entry_decoder, missing_passphrase}) ->
rabbit_misc:format(
"Missing passphrase or missing passphrase read method in "
"`config_entry_decoder`");
format_error({config_decryption_error, {key, Key}, _Msg}) ->
rabbit_misc:format(
"Error while decrypting key '~p'. Please check encrypted value, "
"passphrase, and encryption configuration~n",
[Key]);
format_error({error, {timeout_waiting_for_tables, AllNodes, _}}) ->
Suffix =
"~nBACKGROUND~n==========~n~n"
"This cluster node was shut down while other nodes were still running.~n"
"To avoid losing data, you should start the other nodes first, then~n"
"start this one. To force this node to start, first invoke~n"
"\"rabbitmqctl force_boot\". If you do so, any changes made on other~n"
"cluster nodes after this one was shut down may be lost.",
{Message, Nodes} =
case AllNodes -- [node()] of
[] -> {rabbit_misc:format(
"Timeout contacting cluster nodes. Since RabbitMQ was"
" shut down forcefully~nit cannot determine which nodes"
" are timing out.~n" ++ Suffix, []),
[]};
Ns -> {rabbit_misc:format(
"Timeout contacting cluster nodes: ~p.~n" ++ Suffix,
[Ns]),
Ns}
end,
Message ++ "\n" ++ rabbit_nodes_common:diagnostics(Nodes);
format_error({error, {cannot_log_to_file, unknown, Reason}}) ->
rabbit_misc:format(
"failed to initialised logger: ~p~n",
[Reason]);
format_error({error, {cannot_log_to_file, LogFile,
{cannot_create_parent_dirs, _, Reason}}}) ->
rabbit_misc:format(
"failed to create parent directory for log file at '~s', reason: ~s~n",
[LogFile, file:format_error(Reason)]);
format_error({error, {cannot_log_to_file, LogFile, Reason}}) ->
rabbit_misc:format(
"failed to open log file at '~s', reason: ~s",
[LogFile, file:format_error(Reason)]);
format_error(Error) ->
rabbit_misc:format("Error during startup: ~p", [Error]).
log_exception(Class, Exception, Stacktrace) ->
Message = format_exception(Class, Exception, Stacktrace),
log_message(Message).
format_exception(Class, Exception, Stacktrace) ->
rabbit_misc:format(
"Exception during startup:~n~s",
[lager:pr_stacktrace(Stacktrace, {Class, Exception})]).
log_message(Message) ->
Lines = string:split(
?BOOT_FAILED_HEADER ++
Message ++
?BOOT_FAILED_FOOTER,
[$\n],
all),
[rabbit_log_prelaunch:error("~s", [Line]) || Line <- Lines],
[io:format(standard_error, "~s~n", [Line]) || Line <- Lines],
timer:sleep(1000),
ok.

View File

@ -0,0 +1,17 @@
-module(rabbit_prelaunch_sup).
-behaviour(supervisor).
-export([start_link/0]).
-export([init/1]).
start_link() ->
supervisor:start_link({local, ?MODULE}, ?MODULE, []).
init([]) ->
%% `rabbit_prelaunch` does not start a process, it only configures
%% the node.
Prelaunch = #{id => prelaunch,
start => {rabbit_prelaunch, run_prelaunch_first_phase, []},
restart => transient},
Procs = [Prelaunch],
{ok, {{one_for_one, 1, 5}, Procs}}.

Binary file not shown.

View File

@ -18,41 +18,10 @@
### next line potentially updated in package install steps
SYS_PREFIX=
### next line will be updated when generating a standalone release
ERL_DIR=
CLEAN_BOOT_FILE=start_clean
SASL_BOOT_FILE=start_sasl
if [ -f "${RABBITMQ_HOME}/erlang.mk" ]; then
# RabbitMQ is executed from its source directory. The plugins
# directory and ERL_LIBS are tuned based on this.
RABBITMQ_DEV_ENV=1
fi
## Set default values
BOOT_MODULE="rabbit"
CONFIG_FILE=${SYS_PREFIX}/etc/rabbitmq/rabbitmq
LOG_BASE=${SYS_PREFIX}/var/log/rabbitmq
MNESIA_BASE=${SYS_PREFIX}/var/lib/rabbitmq/mnesia
ENABLED_PLUGINS_FILE=${SYS_PREFIX}/etc/rabbitmq/enabled_plugins
GENERATED_CONFIG_DIR=${SYS_PREFIX}/var/lib/rabbitmq/config
ADVANCED_CONFIG_FILE=${SYS_PREFIX}/etc/rabbitmq/advanced.config
SCHEMA_DIR=${SYS_PREFIX}/var/lib/rabbitmq/schema
PLUGINS_DIR="${RABBITMQ_HOME}/plugins"
# RABBIT_HOME can contain a version number, so default plugins
# directory can be hard to find if we want to package some plugin
# separately. When RABBITMQ_HOME points to a standard location where
# it's usually being installed by package managers, we add
# "/usr/lib/rabbitmq/plugins" to plugin search path.
case "$RABBITMQ_HOME" in
/usr/lib/rabbitmq/*)
PLUGINS_DIR="/usr/lib/rabbitmq/plugins:$PLUGINS_DIR"
;;
esac
CONF_ENV_FILE=${SYS_PREFIX}/etc/rabbitmq/rabbitmq-env.conf
if test -z "$CONF_ENV_FILE" && test -z "$RABBITMQ_CONF_ENV_FILE"; then
CONF_ENV_FILE=${SYS_PREFIX}/etc/rabbitmq/rabbitmq-env.conf
fi

View File

@ -1,23 +1,8 @@
@echo off
REM ### next line potentially updated in package install steps
REM set SYS_PREFIX=
REM ### next line will be updated when generating a standalone release
REM ERL_DIR=
set ERL_DIR=
REM This boot files isn't referenced in the batch scripts
REM set SASL_BOOT_FILE=start_sasl
set SASL_BOOT_FILE=start_sasl
set CLEAN_BOOT_FILE=start_clean
if exist "%RABBITMQ_HOME%\erlang.mk" (
REM RabbitMQ is executed from its source directory. The plugins
REM directory and ERL_LIBS are tuned based on this.
set RABBITMQ_DEV_ENV=1
)
REM ## Set default values
set BOOT_MODULE=rabbit
if "!RABBITMQ_BASE!"=="" (
set RABBITMQ_BASE=!APPDATA!\RabbitMQ
@ -29,22 +14,4 @@ if not exist "!RABBITMQ_BASE!" (
mkdir "!RABBITMQ_BASE!"
)
REM BOOT_MODULE="rabbit"
REM CONFIG_FILE=${SYS_PREFIX}/etc/rabbitmq/rabbitmq
REM LOG_BASE=${SYS_PREFIX}/var/log/rabbitmq
REM MNESIA_BASE=${SYS_PREFIX}/var/lib/rabbitmq/mnesia
REM ENABLED_PLUGINS_FILE=${SYS_PREFIX}/etc/rabbitmq/enabled_plugins
set BOOT_MODULE=rabbit
set CONFIG_FILE=!RABBITMQ_BASE!\rabbitmq
set LOG_BASE=!RABBITMQ_BASE!\log
set MNESIA_BASE=!RABBITMQ_BASE!\db
set ENABLED_PLUGINS_FILE=!RABBITMQ_BASE!\enabled_plugins
set GENERATED_CONFIG_DIR=!RABBITMQ_BASE!\config
set ADVANCED_CONFIG_FILE=!RABBITMQ_BASE!\advanced.config
set SCHEMA_DIR=!RABBITMQ_BASE!\schema
REM PLUGINS_DIR="${RABBITMQ_HOME}/plugins"
for /f "delims=" %%F in ("!TDP0!..\plugins") do set PLUGINS_DIR=%%~dpF%%~nF%%~xF
REM CONF_ENV_FILE=${SYS_PREFIX}/etc/rabbitmq/rabbitmq-env.conf
set CONF_ENV_FILE=!RABBITMQ_BASE!\rabbitmq-env-conf.bat

View File

@ -49,10 +49,6 @@ if not defined ERL_CRASH_DUMP_SECONDS (
-boot !CLEAN_BOOT_FILE! ^
-noinput -noshell -hidden -smp enable ^
!RABBITMQ_CTL_ERL_ARGS! ^
-kernel inet_dist_listen_min !RABBITMQ_CTL_DIST_PORT_MIN! ^
-kernel inet_dist_listen_max !RABBITMQ_CTL_DIST_PORT_MAX! ^
-sasl errlog_type error ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-run escript start ^
-escript main rabbitmqctl_escript ^
-extra "%RABBITMQ_HOME%\escript\rabbitmq-diagnostics" !STAR!
@ -61,4 +57,6 @@ if ERRORLEVEL 1 (
exit /B %ERRORLEVEL%
)
EXIT /B 0
endlocal

View File

@ -91,20 +91,6 @@ rmq_realpath() {
fi
}
path_contains_existing_directory() {
local path="${1:?}"
local dir
local rc
local IFS="
"
for dir in $(echo "$path" | tr ':' '\n'); do
if [ -d "$dir" ]; then
return 0
fi
done
return 1
}
RABBITMQ_HOME="$(rmq_realpath "${RABBITMQ_SCRIPTS_DIR}/..")"
ESCRIPT_DIR="${RABBITMQ_HOME}/escript"
@ -118,7 +104,10 @@ saved_RABBITMQ_PID_FILE="$RABBITMQ_PID_FILE"
## Get configuration variables from the configure environment file
[ "x" = "x$RABBITMQ_CONF_ENV_FILE" ] && RABBITMQ_CONF_ENV_FILE=${CONF_ENV_FILE}
[ -f ${RABBITMQ_CONF_ENV_FILE} ] && . ${RABBITMQ_CONF_ENV_FILE} || true
if [ -f "${RABBITMQ_CONF_ENV_FILE}" ]; then
CONF_ENV_FILE_PHASE=rabbitmq-env
. ${RABBITMQ_CONF_ENV_FILE} || true
fi
[ -n "$ERL_EPMD_PORT" ] && export ERL_EPMD_PORT
[ -n "$ERL_EPMD_ADDRESS" ] && export ERL_EPMD_ADDRESS
@ -142,125 +131,17 @@ DEFAULT_MAX_NUMBER_OF_ATOMS=5000000
## Common server defaults
SERVER_ERL_ARGS=" +P $RABBITMQ_MAX_NUMBER_OF_PROCESSES +t $RABBITMQ_MAX_NUMBER_OF_ATOMS +stbt $RABBITMQ_SCHEDULER_BIND_TYPE +zdbbl $RABBITMQ_DISTRIBUTION_BUFFER_SIZE "
[ "x" = "x$RABBITMQ_USE_LONGNAME" ] && RABBITMQ_USE_LONGNAME=${USE_LONGNAME}
if [ "xtrue" = "x$RABBITMQ_USE_LONGNAME" ] ; then
RABBITMQ_NAME_TYPE=-name
[ "x" = "x$HOSTNAME" ] && HOSTNAME=`env hostname -f`
[ "x" = "x$NODENAME" ] && NODENAME=rabbit@${HOSTNAME}
else
RABBITMQ_NAME_TYPE=-sname
[ "x" = "x$HOSTNAME" ] && HOSTNAME=`env hostname`
[ "x" = "x$NODENAME" ] && NODENAME=rabbit@${HOSTNAME%%.*}
fi
##--- Set environment vars RABBITMQ_<var_name> to defaults if not set
rmq_normalize_path() {
local path=$1
# Remove redundant slashes and strip a trailing slash for a
# PATH-like vars - ':' is the delimiter
echo "$path" | sed -e 's#/\{2,\}#/#g' -e 's#/$##' -e 's#/:#:#g'
}
rmq_normalize_path_var() {
local var warning
local prefix="WARNING:"
for var in "$@"; do
local path=$(eval "echo \"\$$var\"")
case "$path" in
*/)
warning=1
echo "$prefix Removing trailing slash from $var" 1>&2
;;
esac
eval "$var=$(rmq_normalize_path "$path")"
if [ "x$warning" = "x1" ]; then
prefix=" "
fi
done
}
rmq_check_if_shared_with_mnesia() {
local var warning
local mnesia_dir=$(rmq_realpath "${RABBITMQ_MNESIA_DIR}")
local prefix="WARNING:"
for var in "$@"; do
local dir=$(eval "echo \"\$$var\"")
case $(rmq_realpath "$dir") in
${mnesia_dir})
warning=1
echo "$prefix $var is equal to RABBITMQ_MNESIA_DIR" 1>&2
;;
${mnesia_dir}/*)
warning=1
echo "$prefix $var is located inside RABBITMQ_MNESIA_DIR" 1>&2
;;
esac
if [ "x$warning" = "x1" ]; then
prefix=" "
fi
done
if [ "x$warning" = "x1" ]; then
echo "$prefix => Auto-clustering will not work ('cluster_nodes' in rabbitmq.config)" 1>&2
fi
}
DEFAULT_NODE_IP_ADDRESS=auto
DEFAULT_NODE_PORT=5672
[ "x" = "x$RABBITMQ_NODE_IP_ADDRESS" ] && RABBITMQ_NODE_IP_ADDRESS=${NODE_IP_ADDRESS}
[ "x" = "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_NODE_PORT=${NODE_PORT}
[ "x" = "x$RABBITMQ_NODE_IP_ADDRESS" ] && [ "x" != "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_NODE_IP_ADDRESS=${DEFAULT_NODE_IP_ADDRESS}
[ "x" != "x$RABBITMQ_NODE_IP_ADDRESS" ] && [ "x" = "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_NODE_PORT=${DEFAULT_NODE_PORT}
[ "x" = "x$RABBITMQ_DIST_PORT" ] && RABBITMQ_DIST_PORT=${DIST_PORT}
[ "x" = "x$RABBITMQ_DIST_PORT" ] && [ "x" = "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_DIST_PORT=$((${DEFAULT_NODE_PORT} + 20000))
[ "x" = "x$RABBITMQ_DIST_PORT" ] && [ "x" != "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_DIST_PORT=$((${RABBITMQ_NODE_PORT} + 20000))
[ "x" = "x$RABBITMQ_CTL_ERL_ARGS" ] && RABBITMQ_CTL_ERL_ARGS=${CTL_ERL_ARGS}
[ "x" = "x$RABBITMQ_CTL_DIST_PORT_MIN" ] && RABBITMQ_CTL_DIST_PORT_MIN=${CTL_DIST_PORT_MIN}
[ "x" = "x$RABBITMQ_CTL_DIST_PORT_MAX" ] && RABBITMQ_CTL_DIST_PORT_MAX=${CTL_DIST_PORT_MAX}
[ "x" = "x$RABBITMQ_CTL_DIST_PORT_MIN" ] && RABBITMQ_CTL_DIST_PORT_MIN=$((${RABBITMQ_DIST_PORT} + 10000))
[ "x" = "x$RABBITMQ_CTL_DIST_PORT_MAX" ] && RABBITMQ_CTL_DIST_PORT_MAX=$((${RABBITMQ_DIST_PORT} + 10010))
[ "x" = "x$RABBITMQ_NODENAME" ] && RABBITMQ_NODENAME=${NODENAME}
# FIXME
[ "x" = "x$RABBITMQ_IO_THREAD_POOL_SIZE" ] && RABBITMQ_IO_THREAD_POOL_SIZE=${IO_THREAD_POOL_SIZE}
[ "x" = "x$RABBITMQ_SERVER_ERL_ARGS" ] && RABBITMQ_SERVER_ERL_ARGS=${SERVER_ERL_ARGS}
[ "x" = "x$RABBITMQ_CONFIG_FILE" ] && RABBITMQ_CONFIG_FILE=${CONFIG_FILE}
[ "x" = "x$RABBITMQ_LOG_BASE" ] && RABBITMQ_LOG_BASE=${LOG_BASE}
[ "x" = "x$RABBITMQ_MNESIA_BASE" ] && RABBITMQ_MNESIA_BASE=${MNESIA_BASE}
[ "x" = "x$RABBITMQ_SERVER_START_ARGS" ] && RABBITMQ_SERVER_START_ARGS=${SERVER_START_ARGS}
[ "x" = "x$RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS" ] && RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS=${SERVER_ADDITIONAL_ERL_ARGS}
[ "x" = "x$RABBITMQ_SERVER_CODE_PATH" ] && RABBITMQ_SERVER_CODE_PATH=${SERVER_CODE_PATH}
[ "x" = "x$RABBITMQ_MNESIA_DIR" ] && RABBITMQ_MNESIA_DIR=${MNESIA_DIR}
[ "x" = "x$RABBITMQ_MNESIA_DIR" ] && RABBITMQ_MNESIA_DIR=${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}
[ "x" = "x$RABBITMQ_QUORUM_DIR" ] && RABBITMQ_QUORUM_DIR=${RABBITMQ_MNESIA_DIR}/quorum
[ "x" = "x$RABBITMQ_GENERATED_CONFIG_DIR" ] && RABBITMQ_GENERATED_CONFIG_DIR=${GENERATED_CONFIG_DIR}
[ "x" = "x$RABBITMQ_ADVANCED_CONFIG_FILE" ] && RABBITMQ_ADVANCED_CONFIG_FILE=${ADVANCED_CONFIG_FILE}
[ "x" = "x$RABBITMQ_SCHEMA_DIR" ] && RABBITMQ_SCHEMA_DIR=${SCHEMA_DIR}
[ "x" = "x$RABBITMQ_IGNORE_SIGINT" ] && RABBITMQ_IGNORE_SIGINT="true"
[ "xtrue" = "x$RABBITMQ_IGNORE_SIGINT" ] && RABBITMQ_IGNORE_SIGINT_FLAG="+B i"
rmq_normalize_path_var \
RABBITMQ_CONFIG_FILE \
RABBITMQ_LOG_BASE \
RABBITMQ_MNESIA_BASE \
RABBITMQ_MNESIA_DIR \
RABBITMQ_QUORUM_DIR
[ "x" = "x$RABBITMQ_PID_FILE" ] && RABBITMQ_PID_FILE="$PID_FILE"
if [ -n "$saved_RABBITMQ_PID_FILE" ] && \
[ "$saved_RABBITMQ_PID_FILE" != "$RABBITMQ_PID_FILE" ]
then
@ -271,155 +152,19 @@ then
RABBITMQ_PID_FILE="$saved_RABBITMQ_PID_FILE"
fi
# Note: at this point, no RABBITMQ_PID_FILE is set so we use the mnesia dir value
[ "x" = "x$RABBITMQ_PID_FILE" ] && RABBITMQ_PID_FILE="${RABBITMQ_MNESIA_DIR}.pid"
rmq_normalize_path_var RABBITMQ_PID_FILE
[ "x" = "x$RABBITMQ_BOOT_MODULE" ] && RABBITMQ_BOOT_MODULE=${BOOT_MODULE}
[ "x" != "x$RABBITMQ_FEATURE_FLAGS_FILE" ] && RABBITMQ_FEATURE_FLAGS_FILE_source=environment
[ "x" = "x$RABBITMQ_FEATURE_FLAGS_FILE" ] && RABBITMQ_FEATURE_FLAGS_FILE=${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}-feature_flags
rmq_normalize_path_var RABBITMQ_FEATURE_FLAGS_FILE
[ "x" = "x$RABBITMQ_PLUGINS_EXPAND_DIR" ] && RABBITMQ_PLUGINS_EXPAND_DIR=${PLUGINS_EXPAND_DIR}
[ "x" = "x$RABBITMQ_PLUGINS_EXPAND_DIR" ] && RABBITMQ_PLUGINS_EXPAND_DIR=${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}-plugins-expand
rmq_normalize_path_var RABBITMQ_PLUGINS_EXPAND_DIR
[ "x" != "x$RABBITMQ_ENABLED_PLUGINS_FILE" ] && RABBITMQ_ENABLED_PLUGINS_FILE_source=environment
[ "x" = "x$RABBITMQ_ENABLED_PLUGINS_FILE" ] && RABBITMQ_ENABLED_PLUGINS_FILE=${ENABLED_PLUGINS_FILE}
rmq_normalize_path_var RABBITMQ_ENABLED_PLUGINS_FILE
[ "x" != "x$RABBITMQ_PLUGINS_DIR" ] && RABBITMQ_PLUGINS_DIR_source=environment
[ "x" = "x$RABBITMQ_PLUGINS_DIR" ] && RABBITMQ_PLUGINS_DIR=${PLUGINS_DIR}
rmq_normalize_path_var RABBITMQ_PLUGINS_DIR
## Log rotation
[ "x" = "x$RABBITMQ_LOGS" ] && RABBITMQ_LOGS=${LOGS}
[ "x" != "x$RABBITMQ_LOGS" ] && export RABBITMQ_LOGS_source=environment
[ "x" = "x$RABBITMQ_LOGS" ] && RABBITMQ_LOGS="${RABBITMQ_LOG_BASE}/${RABBITMQ_NODENAME}.log"
[ "x" = "x$RABBITMQ_UPGRADE_LOG" ] && RABBITMQ_UPGRADE_LOG="${RABBITMQ_LOG_BASE}/${RABBITMQ_NODENAME}_upgrade.log"
[ "x" = "x$ERL_CRASH_DUMP" ] && ERL_CRASH_DUMP="${RABBITMQ_LOG_BASE}/erl_crash.dump"
rmq_normalize_path_var RABBITMQ_LOGS
rmq_normalize_path_var RABBITMQ_UPGRADE_LOG
# Check if files and directories non-related to Mnesia are configured
# to be in $RABBITMQ_MNESIA_DIR. If this is the case, issue a warning
# because it will prevent auto-clustering from working (the node will be
# considered non-virgin).
rmq_check_if_shared_with_mnesia \
RABBITMQ_CONFIG_FILE \
RABBITMQ_LOG_BASE \
RABBITMQ_PID_FILE \
RABBITMQ_FEATURE_FLAGS_FILE \
RABBITMQ_PLUGINS_EXPAND_DIR \
RABBITMQ_ENABLED_PLUGINS_FILE \
RABBITMQ_PLUGINS_DIR \
RABBITMQ_LOGS \
RABBITMQ_UPGRADE_LOG
##--- End of overridden <var_name> variables
## Development-specific environment.
if [ "${RABBITMQ_DEV_ENV}" ]; then
if [ "$RABBITMQ_FEATURE_FLAGS_FILE_source" != 'environment' -o \
"$RABBITMQ_PLUGINS_DIR_source" != 'environment' -o \
"$RABBITMQ_ENABLED_PLUGINS_FILE_source" != 'environment' ]; then
# We need to query the running node for the plugins directory
# and the "enabled plugins" file.
for arg in "$@"; do
case "$arg" in
-n)
next_is_node=1
;;
*)
if test "$next_is_node"; then
# If the executed script is being passed a remote node
# name, use it here to query the remote node.
node_arg="-n $arg"
break
fi
;;
esac
done
eval $( (${RABBITMQ_SCRIPTS_DIR}/rabbitmqctl $node_arg eval \
'{ok, F} = application:get_env(rabbit, feature_flags_file),
{ok, P} = application:get_env(rabbit, plugins_dir),
{ok, E} = application:get_env(rabbit, enabled_plugins_file),
B = os:getenv("RABBITMQ_MNESIA_BASE"),
M = os:getenv("RABBITMQ_MNESIA_DIR"),
io:format(
"feature_flags_file=\"~s\"~n"
"plugins_dir=\"~s\"~n"
"enabled_plugins_file=\"~s\"~n"
"mnesia_base=\"~s\"~n"
"mnesia_dir=\"~s\"~n", [F, P, E, B, M]).' \
2>/dev/null | grep -E '^(feature_flags_file|plugins_dir|enabled_plugins_file|mnesia_base|mnesia_dir)=') || :)
if [ "${feature_flags_file}" -a \
"$RABBITMQ_FEATURE_FLAGS_FILE_source" != 'environment' ]; then
RABBITMQ_FEATURE_FLAGS_FILE="${feature_flags_file}"
fi
if [ "${plugins_dir}" -a \
"$RABBITMQ_PLUGINS_DIR_source" != 'environment' ]; then
RABBITMQ_PLUGINS_DIR="${plugins_dir}"
fi
if [ "${enabled_plugins_file}" -a \
"$RABBITMQ_ENABLED_PLUGINS_FILE_source" != 'environment' ]; then
RABBITMQ_ENABLED_PLUGINS_FILE="${enabled_plugins_file}"
fi
if [ "${mnesia_base}" -a \
"$RABBITMQ_MNESIA_BASE_source" != 'environment' ]; then
RABBITMQ_MNESIA_BASE="${mnesia_base}"
fi
if [ "${mnesia_dir}" -a \
"$RABBITMQ_MNESIA_DIR_source" != 'environment' ]; then
RABBITMQ_MNESIA_DIR="${mnesia_dir}"
fi
if [ "${mnesia_dir}" -a \
"$RABBITMQ_QUORUM_DIR_source" != 'environment' ]; then
RABBITMQ_QUORUM_DIR="${mnesia_dir}/quorum"
fi
fi
if path_contains_existing_directory "${RABBITMQ_PLUGINS_DIR}" ; then
# RabbitMQ was started with "make run-broker" from its own
# source tree. Take rabbit_common from the plugins directory.
ERL_LIBS="${RABBITMQ_PLUGINS_DIR}:${ERL_LIBS}"
_rmq_env_set_erl_libs()
{
if [ -n "$ERL_LIBS" ]
then
export ERL_LIBS="$RABBITMQ_HOME/plugins:$ERL_LIBS"
else
# RabbitMQ runs from a testsuite or a plugin. The .ez files are
# not available under RabbitMQ source tree. We need to look at
# $DEPS_DIR and default locations.
if [ "${DEPS_DIR}" -a -d "${DEPS_DIR}/rabbit_common/ebin" ]; then
# $DEPS_DIR is set, and it contains rabbitmq-common, use
# this.
DEPS_DIR_norm="${DEPS_DIR}"
elif [ -f "${RABBITMQ_SCRIPTS_DIR}/../../../erlang.mk" -a \
-d "${RABBITMQ_SCRIPTS_DIR}/../../rabbit_common/ebin" ]; then
# Look at default locations: "deps" subdirectory inside a
# plugin or the Umbrella.
DEPS_DIR_norm="${RABBITMQ_SCRIPTS_DIR}/../.."
fi
DEPS_DIR_norm=$(rmq_realpath "${DEPS_DIR_norm}")
ERL_LIBS="${DEPS_DIR_norm}:${ERL_LIBS}"
export ERL_LIBS="$RABBITMQ_HOME/plugins"
fi
else
if path_contains_existing_directory "${RABBITMQ_PLUGINS_DIR}" ; then
# RabbitMQ was started from its install directory. Take
# rabbit_common from the plugins directory.
ERL_LIBS="${RABBITMQ_PLUGINS_DIR}:${ERL_LIBS}"
fi
fi
ERL_LIBS=${ERL_LIBS%:}
if [ "$ERL_LIBS" ]; then
export ERL_LIBS
fi
}
run_escript()
{
@ -428,17 +173,15 @@ run_escript()
escript="${1:?escript must be defined}"
shift
_rmq_env_set_erl_libs
# Important: do not quote RABBITMQ_CTL_ERL_ARGS as they must be
# word-split
# shellcheck disable=SC2086
exec "${ERL_DIR}erl" +B \
exec erl +B \
-boot "$CLEAN_BOOT_FILE" \
-noinput -noshell -hidden -smp enable \
$RABBITMQ_CTL_ERL_ARGS \
-kernel inet_dist_listen_min "$RABBITMQ_CTL_DIST_PORT_MIN" \
-kernel inet_dist_listen_max "$RABBITMQ_CTL_DIST_PORT_MAX" \
-sasl errlog_type error \
-mnesia dir "\"$RABBITMQ_MNESIA_DIR\"" \
-run escript start \
-escript main "$escript_main" \
-extra "$escript" "$@"

View File

@ -14,6 +14,12 @@ set SCRIPT_DIR=%TDP0%
set SCRIPT_NAME=%1
for /f "delims=" %%F in ("%SCRIPT_DIR%..") do set RABBITMQ_HOME=%%~dpF%%~nF%%~xF
if defined ERL_LIBS (
set ERL_LIBS=%RABBITMQ_HOME%\plugins;%ERL_LIBS%
) else (
set ERL_LIBS=%RABBITMQ_HOME%\plugins
)
REM If ERLANG_HOME is not defined, check if "erl.exe" is available in
REM the path and use that.
if not defined ERLANG_HOME (
@ -82,163 +88,16 @@ if "!RABBITMQ_MAX_NUMBER_OF_ATOMS!"=="" (
set RABBITMQ_MAX_NUMBER_OF_ATOMS=!DEFAULT_MAX_NUMBER_OF_ATOMS!
)
REM Common defaults
REM Common server defaults
set SERVER_ERL_ARGS=+P !RABBITMQ_MAX_NUMBER_OF_PROCESSES! +t !RABBITMQ_MAX_NUMBER_OF_ATOMS! +stbt !RABBITMQ_SCHEDULER_BIND_TYPE! +zdbbl !RABBITMQ_DISTRIBUTION_BUFFER_SIZE!
REM Check for the short names here too
if "!RABBITMQ_USE_LONGNAME!"=="true" (
set RABBITMQ_NAME_TYPE=-name
set NAMETYPE=longnames
) else (
if "!USE_LONGNAME!"=="true" (
set RABBITMQ_USE_LONGNAME=true
set RABBITMQ_NAME_TYPE=-name
set NAMETYPE=longnames
) else (
set RABBITMQ_USE_LONGNAME=false
set RABBITMQ_NAME_TYPE=-sname
set NAMETYPE=shortnames
)
)
REM [ "x" = "x$RABBITMQ_NODENAME" ] && RABBITMQ_NODENAME=${NODENAME}
if "!RABBITMQ_NODENAME!"=="" (
if "!NODENAME!"=="" (
REM We use Erlang to query the local hostname because
REM !COMPUTERNAME! and Erlang may return different results.
REM Start erl with -sname to make sure epmd is started.
call "%ERLANG_HOME%\bin\erl.exe" -A0 -noinput -boot start_clean -sname rabbit-prelaunch-epmd -eval "init:stop()." >nul 2>&1
for /f "delims=" %%F in ('call "%ERLANG_HOME%\bin\erl.exe" -A0 -noinput -boot start_clean -eval "net_kernel:start([list_to_atom(""rabbit-gethostname-"" ++ os:getpid()), %NAMETYPE%]), [_, H] = string:tokens(atom_to_list(node()), ""@""), io:format(""~s~n"", [H]), init:stop()."') do @set HOSTNAME=%%F
set RABBITMQ_NODENAME=rabbit@!HOSTNAME!
set HOSTNAME=
) else (
set RABBITMQ_NODENAME=!NODENAME!
)
)
set NAMETYPE=
REM
REM ##--- Set environment vars RABBITMQ_<var_name> to defaults if not set
REM
REM DEFAULT_NODE_IP_ADDRESS=auto
REM DEFAULT_NODE_PORT=5672
REM [ "x" = "x$RABBITMQ_NODE_IP_ADDRESS" ] && RABBITMQ_NODE_IP_ADDRESS=${NODE_IP_ADDRESS}
REM [ "x" = "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_NODE_PORT=${NODE_PORT}
REM [ "x" = "x$RABBITMQ_NODE_IP_ADDRESS" ] && [ "x" != "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_NODE_IP_ADDRESS=${DEFAULT_NODE_IP_ADDRESS}
REM [ "x" != "x$RABBITMQ_NODE_IP_ADDRESS" ] && [ "x" = "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_NODE_PORT=${DEFAULT_NODE_PORT}
if "!RABBITMQ_NODE_IP_ADDRESS!"=="" (
if not "!NODE_IP_ADDRESS!"=="" (
set RABBITMQ_NODE_IP_ADDRESS=!NODE_IP_ADDRESS!
)
)
if "!RABBITMQ_NODE_PORT!"=="" (
if not "!NODE_PORT!"=="" (
set RABBITMQ_NODE_PORT=!NODE_PORT!
)
)
if "!RABBITMQ_NODE_IP_ADDRESS!"=="" (
if not "!RABBITMQ_NODE_PORT!"=="" (
set RABBITMQ_NODE_IP_ADDRESS=auto
)
) else (
if "!RABBITMQ_NODE_PORT!"=="" (
set RABBITMQ_NODE_PORT=5672
)
)
REM [ "x" = "x$RABBITMQ_DIST_PORT" ] && RABBITMQ_DIST_PORT=${DIST_PORT}
REM [ "x" = "x$RABBITMQ_DIST_PORT" ] && [ "x" = "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_DIST_PORT=$((${DEFAULT_NODE_PORT} + 20000))
REM [ "x" = "x$RABBITMQ_DIST_PORT" ] && [ "x" != "x$RABBITMQ_NODE_PORT" ] && RABBITMQ_DIST_PORT=$((${RABBITMQ_NODE_PORT} + 20000))
if "!RABBITMQ_DIST_PORT!"=="" (
if "!DIST_PORT!"=="" (
if "!RABBITMQ_NODE_PORT!"=="" (
set RABBITMQ_DIST_PORT=25672
) else (
set /a RABBITMQ_DIST_PORT=20000+!RABBITMQ_NODE_PORT!
)
) else (
set RABBITMQ_DIST_PORT=!DIST_PORT!
)
)
REM [ "x" = "x$RABBITMQ_SERVER_ERL_ARGS" ] && RABBITMQ_SERVER_ERL_ARGS=${SERVER_ERL_ARGS}
if "!RABBITMQ_SERVER_ERL_ARGS!"=="" (
set RABBITMQ_SERVER_ERL_ARGS=!SERVER_ERL_ARGS!
)
REM [ "x" = "x$RABBITMQ_CONFIG_FILE" ] && RABBITMQ_CONFIG_FILE=${CONFIG_FILE}
if "!RABBITMQ_CONFIG_FILE!"=="" (
if "!CONFIG_FILE!"=="" (
set RABBITMQ_CONFIG_FILE=!RABBITMQ_BASE!\rabbitmq
) else (
set RABBITMQ_CONFIG_FILE=!CONFIG_FILE:"=!
)
) else (
set RABBITMQ_CONFIG_FILE=!RABBITMQ_CONFIG_FILE:"=!
)
if "!RABBITMQ_GENERATED_CONFIG_DIR!"=="" (
if "!GENERATED_CONFIG_DIR!"=="" (
set RABBITMQ_GENERATED_CONFIG_DIR=!RABBITMQ_BASE!\config
) else (
set RABBITMQ_GENERATED_CONFIG_DIR=!GENERATED_CONFIG_DIR:"=!
)
) else (
set RABBITMQ_GENERATED_CONFIG_DIR=!RABBITMQ_GENERATED_CONFIG_DIR:"=!
)
if "!RABBITMQ_ADVANCED_CONFIG_FILE!"=="" (
if "!ADVANCED_CONFIG_FILE!"=="" (
set RABBITMQ_ADVANCED_CONFIG_FILE=!RABBITMQ_BASE!\advanced.config
) else (
set RABBITMQ_ADVANCED_CONFIG_FILE=!ADVANCED_CONFIG_FILE:"=!
)
) else (
set RABBITMQ_ADVANCED_CONFIG_FILE=!RABBITMQ_ADVANCED_CONFIG_FILE:"=!
)
if "!RABBITMQ_SCHEMA_DIR!" == "" (
if "!SCHEMA_DIR!"=="" (
set RABBITMQ_SCHEMA_DIR=!RABBITMQ_HOME!\priv\schema
) else (
set RABBITMQ_SCHEMA_DIR=!SCHEMA_DIR:"=!
)
) else (
set RABBITMQ_SCHEMA_DIR=!RABBITMQ_SCHEMA_DIR:"=!
)
REM [ "x" = "x$RABBITMQ_LOG_BASE" ] && RABBITMQ_LOG_BASE=${LOG_BASE}
if "!RABBITMQ_LOG_BASE!"=="" (
if "!LOG_BASE!"=="" (
set RABBITMQ_LOG_BASE=!RABBITMQ_BASE!\log
) else (
set RABBITMQ_LOG_BASE=!LOG_BASE:"=!
)
) else (
set RABBITMQ_LOG_BASE=!RABBITMQ_LOG_BASE:"=!
)
if not exist "!RABBITMQ_LOG_BASE!" (
mkdir "!RABBITMQ_LOG_BASE!"
)
REM [ "x" = "x$RABBITMQ_MNESIA_BASE" ] && RABBITMQ_MNESIA_BASE=${MNESIA_BASE}
if "!RABBITMQ_MNESIA_BASE!"=="" (
if "!MNESIA_BASE!"=="" (
set RABBITMQ_MNESIA_BASE=!RABBITMQ_BASE!\db
) else (
set RABBITMQ_MNESIA_BASE=!MNESIA_BASE:"=!
)
) else (
set RABBITMQ_MNESIA_BASE=!RABBITMQ_MNESIA_BASE:"=!
)
if not exist "!RABBITMQ_MNESIA_BASE!" (
mkdir "!RABBITMQ_MNESIA_BASE!"
)
REM [ "x" = "x$RABBITMQ_SERVER_START_ARGS" ] && RABBITMQ_SERVER_START_ARGS=${SERVER_START_ARGS}
if "!RABBITMQ_SERVER_START_ARGS!"=="" (
if not "!SERVER_START_ARGS!"=="" (
@ -253,34 +112,6 @@ if "!RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS!"=="" (
)
)
REM [ "x" = "x$RABBITMQ_MNESIA_DIR" ] && RABBITMQ_MNESIA_DIR=${MNESIA_DIR}
REM [ "x" = "x$RABBITMQ_MNESIA_DIR" ] && RABBITMQ_MNESIA_DIR=${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}
if "!RABBITMQ_MNESIA_DIR!"=="" (
if "!MNESIA_DIR!"=="" (
set RABBITMQ_MNESIA_DIR=!RABBITMQ_MNESIA_BASE!\!RABBITMQ_NODENAME!-mnesia
) else (
set RABBITMQ_MNESIA_DIR=!MNESIA_DIR:"=!
)
) else (
set RABBITMQ_MNESIA_DIR=!RABBITMQ_MNESIA_DIR:"=!
)
if not exist "!RABBITMQ_MNESIA_DIR!" (
mkdir "!RABBITMQ_MNESIA_DIR!"
)
REM [ "x" = "x$RABBITMQ_QUORUM_DIR" ] && RABBITMQ_QUORUM_DIR=${RABBITMQ_MNESIA_DIR}/quorum
if "!RABBITMQ_QUORUM_DIR!"=="" (
set RABBITMQ_QUORUM_DIR=!RABBITMQ_MNESIA_DIR!\quorum
)
if not exist "!RABBITMQ_QUORUM_DIR!" (
mkdir "!RABBITMQ_QUORUM_DIR!"
)
for /f "delims=" %%F in ("!RABBITMQ_QUORUM_DIR!") do set RABBITMQ_QUORUM_DIR=%%~sF
REM [ "x" = "x$RABBITMQ_PID_FILE" ] && RABBITMQ_PID_FILE=${PID_FILE}
REM [ "x" = "x$RABBITMQ_PID_FILE" ] && RABBITMQ_PID_FILE=${RABBITMQ_MNESIA_DIR}.pid
REM No Windows equivalent
REM [ "x" = "x$RABBITMQ_BOOT_MODULE" ] && RABBITMQ_BOOT_MODULE=${BOOT_MODULE}
if "!RABBITMQ_BOOT_MODULE!"=="" (
if "!BOOT_MODULE!"=="" (
@ -290,114 +121,12 @@ if "!RABBITMQ_BOOT_MODULE!"=="" (
)
)
REM [ "x" = "x$RABBITMQ_FEATURE_FLAGS_FILE" ] && RABBITMQ_FEATURE_FLAGS_FILE=${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}-feature_flags
if "!RABBITMQ_FEATURE_FLAGS_FILE!"=="" (
if "!FEATURE_FLAGS_FILE!"=="" (
set RABBITMQ_FEATURE_FLAGS_FILE=!RABBITMQ_MNESIA_BASE!\!RABBITMQ_NODENAME!-feature_flags
) else (
set RABBITMQ_FEATURE_FLAGS_FILE=!FEATURE_FLAGS_FILE:"=!
)
) else (
set RABBITMQ_FEATURE_FLAGS_FILE=!RABBITMQ_FEATURE_FLAGS_FILE:"=!
set RABBITMQ_FEATURE_FLAGS_FILE_source=environment
)
REM [ "x" = "x$RABBITMQ_PLUGINS_EXPAND_DIR" ] && RABBITMQ_PLUGINS_EXPAND_DIR=${PLUGINS_EXPAND_DIR}
REM [ "x" = "x$RABBITMQ_PLUGINS_EXPAND_DIR" ] && RABBITMQ_PLUGINS_EXPAND_DIR=${RABBITMQ_MNESIA_BASE}/${RABBITMQ_NODENAME}-plugins-expand
if "!RABBITMQ_PLUGINS_EXPAND_DIR!"=="" (
if "!PLUGINS_EXPAND_DIR!"=="" (
set RABBITMQ_PLUGINS_EXPAND_DIR=!RABBITMQ_MNESIA_BASE!\!RABBITMQ_NODENAME!-plugins-expand
) else (
set RABBITMQ_PLUGINS_EXPAND_DIR=!PLUGINS_EXPAND_DIR:"=!
)
) else (
set RABBITMQ_PLUGINS_EXPAND_DIR=!RABBITMQ_PLUGINS_EXPAND_DIR:"=!
)
REM [ "x" = "x$RABBITMQ_ENABLED_PLUGINS_FILE" ] && RABBITMQ_ENABLED_PLUGINS_FILE=${ENABLED_PLUGINS_FILE}
if "!RABBITMQ_ENABLED_PLUGINS_FILE!"=="" (
if "!ENABLED_PLUGINS_FILE!"=="" (
set RABBITMQ_ENABLED_PLUGINS_FILE=!RABBITMQ_BASE!\enabled_plugins
) else (
set RABBITMQ_ENABLED_PLUGINS_FILE=!ENABLED_PLUGINS_FILE:"=!
)
) else (
set RABBITMQ_ENABLED_PLUGINS_FILE=!RABBITMQ_ENABLED_PLUGINS_FILE:"=!
set RABBITMQ_ENABLED_PLUGINS_FILE_source=environment
)
if not exist "!RABBITMQ_ENABLED_PLUGINS_FILE!" (
for /f "delims=" %%F in ("!RABBITMQ_ENABLED_PLUGINS_FILE!") do mkdir "%%~dpF" 2>NUL
copy /y NUL "!RABBITMQ_ENABLED_PLUGINS_FILE!" >NUL
)
REM [ "x" = "x$RABBITMQ_PLUGINS_DIR" ] && RABBITMQ_PLUGINS_DIR=${PLUGINS_DIR}
if "!RABBITMQ_PLUGINS_DIR!"=="" (
if "!PLUGINS_DIR!"=="" (
set RABBITMQ_PLUGINS_DIR=!RABBITMQ_HOME!\plugins
) else (
set RABBITMQ_PLUGINS_DIR=!PLUGINS_DIR:"=!
)
) else (
set RABBITMQ_PLUGINS_DIR=!RABBITMQ_PLUGINS_DIR:"=!
set RABBITMQ_PLUGINS_DIR_source=environment
)
if not exist "!RABBITMQ_PLUGINS_DIR!" (
mkdir "!RABBITMQ_PLUGINS_DIR!"
)
REM ## Log rotation
REM [ "x" = "x$RABBITMQ_LOGS" ] && RABBITMQ_LOGS=${LOGS}
REM [ "x" = "x$RABBITMQ_LOGS" ] && RABBITMQ_LOGS="${RABBITMQ_LOG_BASE}/${RABBITMQ_NODENAME}.log"
if "!RABBITMQ_LOGS!"=="" (
if "!LOGS!"=="" (
set RABBITMQ_LOGS=!RABBITMQ_LOG_BASE!\!RABBITMQ_NODENAME!.log
) else (
set RABBITMQ_LOGS=!LOGS:"=!
)
) else (
set RABBITMQ_LOGS=!RABBITMQ_LOGS:"=!
)
if not "!RABBITMQ_LOGS!" == "-" (
if not exist "!RABBITMQ_LOGS!" (
for /f "delims=" %%F in ("!RABBITMQ_LOGS!") do mkdir "%%~dpF" 2>NUL
copy /y NUL "!RABBITMQ_LOGS!" >NUL
)
)
rem [ "x" = "x$RABBITMQ_UPGRADE_LOG" ] && RABBITMQ_UPGRADE_LOG="${RABBITMQ_LOG_BASE}/${RABBITMQ_NODENAME}_upgrade.log"
if "!RABBITMQ_UPGRADE_LOG!" == "" (
set RABBITMQ_UPGRADE_LOG=!RABBITMQ_LOG_BASE!\!RABBITMQ_NODENAME!_upgrade.log
) else (
set RABBITMQ_UPGRADE_LOG=!RABBITMQ_UPGRADE_LOG:"=!
)
REM [ "x" = "x$ERL_CRASH_DUMP"] && ERL_CRASH_DUMP="${RABBITMQ_LOG_BASE}/erl_crash.dump"
if "!ERL_CRASH_DUMP!"=="" (
set ERL_CRASH_DUMP=!RABBITMQ_LOG_BASE!\erl_crash.dump
) else (
set ERL_CRASH_DUMP=!ERL_CRASH_DUMP:"=!
)
REM [ "x" = "x$RABBITMQ_CTL_ERL_ARGS" ] && RABBITMQ_CTL_ERL_ARGS=${CTL_ERL_ARGS}
if "!RABBITMQ_CTL_ERL_ARGS!"=="" (
if not "!CTL_ERL_ARGS!"=="" (
set RABBITMQ_CTL_ERL_ARGS=!CTL_ERL_ARGS!
)
)
if "!RABBITMQ_CTL_DIST_PORT_MIN!"=="" (
if not "!CTL_DIST_PORT_MIN!"=="" (
set RABBITMQ_CTL_DIST_PORT_MIN=!CTL_DIST_PORT_MIN!
)
)
if "!RABBITMQ_CTL_DIST_PORT_MAX!"=="" (
if not "!CTL_DIST_PORT_MAX!"=="" (
set RABBITMQ_CTL_DIST_PORT_MAX=!CTL_DIST_PORT_MAX!
)
)
if "!RABBITMQ_CTL_DIST_PORT_MIN!"=="" (
set /a RABBITMQ_CTL_DIST_PORT_MIN=10000+!RABBITMQ_DIST_PORT!
)
if "!RABBITMQ_CTL_DIST_PORT_MAX!"=="" (
set /a RABBITMQ_CTL_DIST_PORT_MAX=10010+!RABBITMQ_DIST_PORT!
)
REM ADDITIONAL WINDOWS ONLY CONFIG ITEMS
@ -409,93 +138,6 @@ if "!RABBITMQ_SERVICENAME!"=="" (
)
)
REM Development-specific environment.
if defined RABBITMQ_DEV_ENV (
if "!SCRIPT_NAME!" == "rabbitmq-plugins" (
REM We may need to query the running node for the plugins directory
REM and the "enabled plugins" file.
if not "%RABBITMQ_FEATURE_FLAGS_FILE_source%" == "environment" (
for /f "delims=" %%F in ('!SCRIPT_DIR!\rabbitmqctl.bat eval "{ok, P} = application:get_env(rabbit, feature_flags_file), io:format(""~s~n"", [P])."') do @set feature_flags_file=%%F
if exist "!feature_flags_file!" (
set RABBITMQ_FEATURE_FLAGS_FILE=!feature_flags_file:"=!
)
REM set feature_flags_file=
)
if not "%RABBITMQ_PLUGINS_DIR_source%" == "environment" (
for /f "delims=" %%F in ('!SCRIPT_DIR!\rabbitmqctl.bat eval "{ok, P} = application:get_env(rabbit, plugins_dir), io:format(""~s~n"", [P])."') do @set plugins_dir=%%F
if exist "!plugins_dir!" (
set RABBITMQ_PLUGINS_DIR=!plugins_dir:"=!
)
REM set plugins_dir=
)
if not "%RABBITMQ_ENABLED_PLUGINS_FILE_source%" == "environment" (
for /f "delims=" %%F in ('!SCRIPT_DIR!\rabbitmqctl.bat eval "{ok, P} = application:get_env(rabbit, enabled_plugins_file), io:format(""~s~n"", [P])."') do @set enabled_plugins_file=%%F
if exist "!enabled_plugins_file!" (
set RABBITMQ_ENABLED_PLUGINS_FILE=!enabled_plugins_file:"=!
)
REM set enabled_plugins_file=
)
)
if exist "!RABBITMQ_PLUGINS_DIR!" (
REM RabbitMQ was started with "make run-broker" from its own
REM source tree. Take rabbit_common from the plugins directory.
set ERL_LIBS=!RABBITMQ_PLUGINS_DIR!;!ERL_LIBS!
) else (
REM RabbitMQ runs from a testsuite or a plugin. The .ez files are
REM not available under RabbitMQ source tree. We need to look at
REM $DEPS_DIR and default locations.
if "!DEPS_DIR!" == "" (
if exist "!RABBITMQ_HOME!\..\..\deps\rabbit_common\erlang.mk" (
REM Dependencies in the Umbrella or a plugin.
set DEPS_DIR_norm=!RABBITMQ_HOME!\..\..\deps
) else (
if exist "!RABBITMQ_HOME!\deps\rabbit_common\erlang.mk" (
REM Dependencies in the broker.
set DEPS_DIR_norm=!RABBITMQ_HOME!\deps
)
)
) else (
for /f "delims=" %%F in ("!DEPS_DIR!") do @set DEPS_DIR_norm=%%~dpF%%~nF%%~xF
)
set ERL_LIBS=!DEPS_DIR_norm!;!ERL_LIBS!
)
) else (
if exist "!RABBITMQ_PLUGINS_DIR!" (
REM RabbitMQ was started from its install directory. Take
REM rabbit_common from the plugins directory.
set ERL_LIBS=!RABBITMQ_PLUGINS_DIR:"=!;!ERL_LIBS:"=!
)
)
REM Ensure ERL_LIBS begins with valid path
set ERL_LIBS_orig=%ERL_LIBS:"=%
set ERL_LIBS=
call :filter_paths "%ERL_LIBS_orig%"
goto :filter_paths_done
:filter_paths
set paths=%1
set paths=%paths:"=%
for /f "tokens=1* delims=;" %%a in ("%paths%") do (
if not "%%a" == "" call :filter_path "%%a"
if not "%%b" == "" call :filter_paths "%%b"
)
set paths=
goto :eof
:filter_path
IF "%ERL_LIBS%"=="" (
set "ERL_LIBS=%~dp1%~n1%~x1"
) else (
set "ERL_LIBS=!ERL_LIBS!;%~dp1%~n1%~x1"
)
goto :eof
:filter_paths_done
REM Environment cleanup
set BOOT_MODULE=
set CONFIG_FILE=
@ -509,6 +151,6 @@ set SCRIPT_NAME=
set TDP0=
REM ##--- End of overridden <var_name> variables
REM
REM # Since we source this elsewhere, don't accidentally stop execution
REM true

View File

@ -49,10 +49,6 @@ if not defined ERL_CRASH_DUMP_SECONDS (
-boot !CLEAN_BOOT_FILE! ^
-noinput -noshell -hidden -smp enable ^
!RABBITMQ_CTL_ERL_ARGS! ^
-kernel inet_dist_listen_min !RABBITMQ_CTL_DIST_PORT_MIN! ^
-kernel inet_dist_listen_max !RABBITMQ_CTL_DIST_PORT_MAX! ^
-sasl errlog_type error ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-run escript start ^
-escript main rabbitmqctl_escript ^
-extra "%RABBITMQ_HOME%\escript\rabbitmq-plugins" !STAR!
@ -61,5 +57,7 @@ if ERRORLEVEL 1 (
exit /B %ERRORLEVEL%
)
EXIT /B 0
endlocal
endlocal

View File

@ -49,11 +49,6 @@ if not defined ERL_CRASH_DUMP_SECONDS (
-boot !CLEAN_BOOT_FILE! ^
-noinput -noshell -hidden -smp enable ^
!RABBITMQ_CTL_ERL_ARGS! ^
-kernel inet_dist_listen_min !RABBITMQ_CTL_DIST_PORT_MIN! ^
-kernel inet_dist_listen_max !RABBITMQ_CTL_DIST_PORT_MAX! ^
-sasl errlog_type error ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-nodename !RABBITMQ_NODENAME! ^
-run escript start ^
-escript main rabbitmqctl_escript ^
-extra "%RABBITMQ_HOME%\escript\rabbitmq-queues" !STAR!
@ -62,5 +57,7 @@ if ERRORLEVEL 1 (
exit /B %ERRORLEVEL%
)
EXIT /B 0
endlocal
endlocal

58
scripts/rabbitmq-rel Executable file
View File

@ -0,0 +1,58 @@
#!/usr/bin/env escript
%% vim:ft=erlang:sw=2:et:
main(["show-rel"]) ->
Rel = get_rel(),
io:format("~p.~n", [Rel]);
main(["gen-boot"]) ->
generate_rel(),
generate_boot().
get_rel() ->
ok = application:load(rabbit),
Apps0 = get_apps(rabbit),
Apps1 = lists:sort(
fun
(_, rabbitmq_prelaunch) -> false;
(rabbitmq_prelaunch, _) -> true;
(_, mnesia) -> true;
(mnesia, _) -> false;
(A, B) -> A =< B
end, Apps0),
Apps = [{App, get_vsn(App)} || App <- Apps1],
ERTSVersion = erlang:system_info(version),
RabbitVersion = get_vsn(rabbit),
{release,
{"RabbitMQ", RabbitVersion},
{erts, ERTSVersion},
Apps}.
get_apps(App) ->
ok = load_app(App),
{ok, DirectDeps} = application:get_key(App, applications),
lists:umerge(
[lists:usort(get_apps(Dep)) || Dep <- DirectDeps] ++
[lists:usort([kernel, stdlib, sasl, App, mnesia])]).
load_app(App) ->
case application:load(App) of
ok -> ok;
{error, {already_loaded, App}} -> ok
end.
generate_rel() ->
Rel = get_rel(),
io:format("~p.~n", [Rel]),
Output = io_lib:format("~p.~n", [Rel]),
ok = file:write_file("rabbit.rel", Output).
generate_boot() ->
Options = [local, {path, code:get_path()}],
ok = systools:make_script("rabbit", Options).
get_vsn(App) ->
load_app(App),
{ok, Vsn} = application:get_key(App, vsn),
Vsn.

View File

@ -1,4 +1,5 @@
#!/bin/sh
# vim:sw=4:et:
## 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
@ -19,247 +20,18 @@ set -e
# Get default settings with user overrides for (RABBITMQ_)<var_name>
# Non-empty defaults should be set in rabbitmq-env
. `dirname $0`/rabbitmq-env
RABBITMQ_START_RABBIT=
[ "x" = "x$RABBITMQ_ALLOW_INPUT" ] && RABBITMQ_START_RABBIT=" -noinput"
[ "x" = "x$RABBITMQ_NODE_ONLY" ] && RABBITMQ_START_RABBIT="$RABBITMQ_START_RABBIT -s $RABBITMQ_BOOT_MODULE boot "
case "$(uname -s)" in
CYGWIN*) # we make no attempt to record the cygwin pid; rabbitmqctl wait
# will not be able to make sense of it anyway
;;
*) # When -detached is passed, we don't write the pid, since it'd be the
# wrong one
detached=""
for opt in "$@"; do
if [ "$opt" = "-detached" ]; then
detached="true"
fi
done
if [ $detached ]; then
echo "Warning: PID file not written; -detached was passed." 1>&2
else
RABBITMQ_PID_DIR="$(dirname ${RABBITMQ_PID_FILE})"
EX_CANTCREAT=73 # Standard exit code from sysexits(2)
if ! mkdir -p "$RABBITMQ_PID_DIR"; then
# Better diagnostics - 'mkdir -p' reports only the first directory in chain that
# it fails to create
echo "Failed to create directory: $RABBITMQ_PID_DIR"
exit $EX_CANTCREAT
fi
if ! echo $$ > ${RABBITMQ_PID_FILE}; then
# Better diagnostics - otherwise the only report in logs is about failed 'echo'
# command, but without any other details: neither what script has failed nor what
# file output was redirected to.
echo "Failed to write pid file: ${RABBITMQ_PID_FILE}"
exit $EX_CANTCREAT
fi
fi
esac
RABBITMQ_EBIN_ROOT="${RABBITMQ_HOME}/ebin"
SCRIPTS_DIR=$(dirname "$0")
. "$SCRIPTS_DIR/rabbitmq-env"
[ "$NOTIFY_SOCKET" ] && RUNNING_UNDER_SYSTEMD=true
get_noex() {
if [ "x" = "x${1}" ]; then
echo ""
else
local BASENAME=$(basename $1)
local DIRNAME=$(dirname $1)
if [ "x." = "x${DIRNAME}" ]; then
echo "${BASENAME%.*}"
else
echo "${DIRNAME}/${BASENAME%.*}"
fi
fi
}
# Check that advanced config file has the .config extension
# Add .config extension if it's empty
RABBITMQ_ADVANCED_CONFIG_FILE_NOEX=$(get_noex ${RABBITMQ_ADVANCED_CONFIG_FILE})
if [ "${RABBITMQ_ADVANCED_CONFIG_FILE_NOEX}.config" = "${RABBITMQ_ADVANCED_CONFIG_FILE}" \
-o "${RABBITMQ_ADVANCED_CONFIG_FILE_NOEX}" = "${RABBITMQ_ADVANCED_CONFIG_FILE}" ]; then
RABBITMQ_ADVANCED_CONFIG_FILE="${RABBITMQ_ADVANCED_CONFIG_FILE_NOEX}.config"
fi
RABBITMQ_CONFIG_FILE_NOEX=$(get_noex ${RABBITMQ_CONFIG_FILE})
# Extension is not specified.
# Determine config type from file
if [ "${RABBITMQ_CONFIG_FILE_NOEX}" = "${RABBITMQ_CONFIG_FILE}" ]; then
if [ -f "${RABBITMQ_CONFIG_FILE_NOEX}.config" ]; then
if [ -f "${RABBITMQ_CONFIG_FILE_NOEX}.conf" ]; then
# Both files exist. Print a warning.
_rmq_env_pwarn 'Both old (.config) and new (.conf) format config files exist.' \
"Using the old format config file: ${RABBITMQ_CONFIG_FILE_NOEX}.config" \
'Please update your config files to the new format and remove the old file.'
fi
RABBITMQ_CONFIG_FILE="${RABBITMQ_CONFIG_FILE_NOEX}.config"
elif [ -f "${RABBITMQ_CONFIG_FILE_NOEX}.conf" ]; then
RABBITMQ_CONFIG_FILE="${RABBITMQ_CONFIG_FILE_NOEX}.conf"
else
if [ -f ${RABBITMQ_ADVANCED_CONFIG_FILE} ]; then
_rmq_env_pwarn "Using RABBITMQ_ADVANCED_CONFIG_FILE: ${RABBITMQ_ADVANCED_CONFIG_FILE}"
fi
# No config file exist. Use advanced config for -config arg.
RABBITMQ_CONFIG_ARG_FILE="${RABBITMQ_ADVANCED_CONFIG_FILE}"
RABBITMQ_CONFIG_FILE=""
fi
fi
# Set the -config argument.
# The -config argument should not have extension.
# the file should exist
# the file should be a valid erlang term file
# Config file extension is .config
if [ "${RABBITMQ_CONFIG_FILE_NOEX}.config" = "${RABBITMQ_CONFIG_FILE}" ]; then
RABBITMQ_CONFIG_ARG_FILE="${RABBITMQ_CONFIG_FILE}"
# Config file extension is .conf
elif [ "${RABBITMQ_CONFIG_FILE_NOEX}.conf" = "${RABBITMQ_CONFIG_FILE}" ]; then
RABBITMQ_CONFIG_ARG_FILE="${RABBITMQ_ADVANCED_CONFIG_FILE}"
elif [ "x" != "x${RABBITMQ_CONFIG_FILE}" \
-a "${RABBITMQ_CONFIG_FILE_NOEX}" != "${RABBITMQ_CONFIG_FILE}" ]; then
# Config file has an extension, but it's neither .conf or .config
_rmq_env_perr "Wrong extension for RABBITMQ_CONFIG_FILE: ${RABBITMQ_CONFIG_FILE}" \
'The extension should be either .conf or .config'
exit 64 # EX_USAGE
fi
RABBITMQ_CONFIG_ARG_FILE_NOEX=$(get_noex ${RABBITMQ_CONFIG_ARG_FILE})
if [ "${RABBITMQ_CONFIG_ARG_FILE_NOEX}.config" != "${RABBITMQ_CONFIG_ARG_FILE}" ]; then
if [ "${RABBITMQ_CONFIG_ARG_FILE}" = "${RABBITMQ_ADVANCED_CONFIG_FILE}" ]; then
_rmq_env_perr "Wrong extension for RABBITMQ_ADVANCED_CONFIG_FILE: ${RABBITMQ_ADVANCED_CONFIG_FILE}" \
'The extension should be .config'
exit 64 # EX_USAGE
else
# We should never got here, but still there should be some explanation
_rmq_env_perr "Wrong extension for ${RABBITMQ_CONFIG_ARG_FILE}"
'The extension should be .config'
exit 64 # EX_USAGE
fi
fi
# Set -config if the file exists
if [ -f "${RABBITMQ_CONFIG_ARG_FILE}" ]; then
RABBITMQ_CONFIG_ARG="-config ${RABBITMQ_CONFIG_ARG_FILE_NOEX}"
fi
# Set -conf and other generated config parameters
if [ "${RABBITMQ_CONFIG_FILE_NOEX}.conf" = "${RABBITMQ_CONFIG_FILE}" ]; then
if [ ! -d ${RABBITMQ_SCHEMA_DIR} ]; then
mkdir -p "${RABBITMQ_SCHEMA_DIR}"
fi
if [ ! -d ${RABBITMQ_GENERATED_CONFIG_DIR} ]; then
mkdir -p "${RABBITMQ_GENERATED_CONFIG_DIR}"
fi
cp -f "${RABBITMQ_HOME}/priv/schema/rabbit.schema" "${RABBITMQ_SCHEMA_DIR}"
RABBITMQ_GENERATED_CONFIG_ARG="-conf ${RABBITMQ_CONFIG_FILE} \
-conf_dir ${RABBITMQ_GENERATED_CONFIG_DIR} \
-conf_script_dir `dirname $0` \
-conf_schema_dir ${RABBITMQ_SCHEMA_DIR}
-conf_advanced ${RABBITMQ_ADVANCED_CONFIG_FILE}"
fi
set +e
# `net_kernel:start/1` will fail in `longnames` mode when erlang is
# unable to determine FQDN of a node (with a dot in it). But `erl`
# itself has some magic that still allow it to start when you
# explicitly specify host (a.la `erl -name test@localhost`).
#
# It's not possible to communicate with this node, unless it's a
# connection initiator. But as prelaunch IS an initiator, it doesn't
# matter what we actually put here. But `localhost` sounds good
# enough.
RABBITMQ_PRELAUNCH_NODENAME="rabbitmqprelaunch${$}@localhost"
# NOTIFY_SOCKET is needed here to prevent epmd from impersonating the
# success of our startup sequence to systemd.
NOTIFY_SOCKET= \
RABBITMQ_CONFIG_FILE=$RABBITMQ_CONFIG_FILE \
ERL_CRASH_DUMP=$ERL_CRASH_DUMP \
RABBITMQ_CONFIG_ARG_FILE=$RABBITMQ_CONFIG_ARG_FILE \
RABBITMQ_DIST_PORT=$RABBITMQ_DIST_PORT \
${ERL_DIR}erl -pa "$RABBITMQ_EBIN_ROOT" \
-boot "${CLEAN_BOOT_FILE}" \
-noinput \
-hidden \
-s rabbit_prelaunch \
${RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS} \
${RABBITMQ_NAME_TYPE} ${RABBITMQ_PRELAUNCH_NODENAME} \
-conf_advanced "${RABBITMQ_ADVANCED_CONFIG_FILE}" \
-rabbit feature_flags_file "\"$RABBITMQ_FEATURE_FLAGS_FILE\"" \
-rabbit enabled_plugins_file "\"$RABBITMQ_ENABLED_PLUGINS_FILE\"" \
-rabbit plugins_dir "\"$RABBITMQ_PLUGINS_DIR\"" \
-extra "${RABBITMQ_NODENAME}"
PRELAUNCH_RESULT=$?
if [ ${PRELAUNCH_RESULT} = 2 ] ; then
# dist port is mentioned in config, so do not set it
true
elif [ ${PRELAUNCH_RESULT} = 0 ] ; then
# dist port is not mentioned in the config file, we can set it
RABBITMQ_DIST_ARG="-kernel inet_dist_listen_min ${RABBITMQ_DIST_PORT} -kernel inet_dist_listen_max ${RABBITMQ_DIST_PORT}"
else
exit ${PRELAUNCH_RESULT}
fi
# The default allocation strategy RabbitMQ is using was introduced
# in Erlang/OTP 20.2.3. Earlier Erlang versions fail to start with
# this configuration. We therefore need to ensure that erl accepts
# these values before we can use them.
#
# The defaults are meant to reduce RabbitMQ's memory usage and help
# it reclaim memory at the cost of a slight decrease in performance
# (due to an increase in memory operations). These defaults can be
# overridden using the RABBITMQ_SERVER_ERL_ARGS variable.
RABBITMQ_DEFAULT_ALLOC_ARGS="+MBas ageffcbf +MHas ageffcbf +MBlmbcs 512 +MHlmbcs 512 +MMmcs 30"
${ERL_DIR}erl ${RABBITMQ_DEFAULT_ALLOC_ARGS} \
-boot "${CLEAN_BOOT_FILE}" \
-noinput -eval 'halt(0)' 2>/dev/null
if [ $? != 0 ] ; then
RABBITMQ_DEFAULT_ALLOC_ARGS=
fi
set -e
RABBITMQ_LISTEN_ARG=
[ "x" != "x$RABBITMQ_NODE_PORT" ] && [ "x" != "x$RABBITMQ_NODE_IP_ADDRESS" ] && RABBITMQ_LISTEN_ARG="-rabbit tcp_listeners [{\""${RABBITMQ_NODE_IP_ADDRESS}"\","${RABBITMQ_NODE_PORT}"}]"
# If $RABBITMQ_LOGS is '-', send all log messages to stdout. This is
# particularly useful for Docker images.
if [ "$RABBITMQ_LOGS" = '-' ]; then
SASL_ERROR_LOGGER=tty
RABBIT_LAGER_HANDLER=tty
RABBITMQ_LAGER_HANDLER_UPGRADE=tty
else
SASL_ERROR_LOGGER=false
RABBIT_LAGER_HANDLER='"'${RABBITMQ_LOGS}'"'
RABBITMQ_LAGER_HANDLER_UPGRADE='"'${RABBITMQ_UPGRADE_LOG}'"'
fi
# Bump ETS table limit to 50000
if [ "x" = "x$ERL_MAX_ETS_TABLES" ]; then
ERL_MAX_ETS_TABLES=50000
fi
# we need to turn off path expansion because some of the vars, notably
# RABBITMQ_SERVER_ERL_ARGS, contain terms that look like globs and
# there is no other way of preventing their expansion.
set -f
# Lazy initialization of threed pool size - if it wasn't set
# explicitly. This parameter is only needed when server is starting,
# so it makes no sense to do this calculations in rabbitmq-env or
@ -267,70 +39,16 @@ set -f
ensure_thread_pool_size() {
if [ -z "${RABBITMQ_IO_THREAD_POOL_SIZE}" ]; then
RABBITMQ_IO_THREAD_POOL_SIZE=$(
${ERL_DIR}erl -pa "$RABBITMQ_EBIN_ROOT" \
-boot "${CLEAN_BOOT_FILE}" \
-noinput \
-s rabbit_misc report_default_thread_pool_size
erl \
-noinput \
-boot "${CLEAN_BOOT_FILE}" \
-s rabbit_misc report_default_thread_pool_size
)
fi
}
start_rabbitmq_server() {
# The arguments to -pa are in this order because they are *pre*-pended
# to the code path. Since we want RABBITMQ_SERVER_CODE_PATH to precede
# RABBITMQ_EBIN_ROOT, it must come as the second argument here.
# https://github.com/rabbitmq/rabbitmq-server/issues/1777
ensure_thread_pool_size
check_start_params &&
RABBITMQ_CONFIG_FILE=$RABBITMQ_CONFIG_FILE \
ERL_MAX_ETS_TABLES=$ERL_MAX_ETS_TABLES \
ERL_CRASH_DUMP=$ERL_CRASH_DUMP \
exec ${ERL_DIR}erl \
-pa "$RABBITMQ_EBIN_ROOT" "$RABBITMQ_SERVER_CODE_PATH" \
${RABBITMQ_START_RABBIT} \
${RABBITMQ_NAME_TYPE} ${RABBITMQ_NODENAME} \
-boot "${SASL_BOOT_FILE}" \
${RABBITMQ_CONFIG_ARG} \
${RABBITMQ_GENERATED_CONFIG_ARG} \
+W w \
+A ${RABBITMQ_IO_THREAD_POOL_SIZE} \
${RABBITMQ_DEFAULT_ALLOC_ARGS} \
${RABBITMQ_SERVER_ERL_ARGS} \
+K true \
-kernel inet_default_connect_options "[{nodelay,true}]" \
${RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS} \
${RABBITMQ_LISTEN_ARG} \
-sasl errlog_type error \
-sasl sasl_error_logger "$SASL_ERROR_LOGGER" \
-rabbit lager_log_root "\"$RABBITMQ_LOG_BASE\"" \
-rabbit lager_default_file "$RABBIT_LAGER_HANDLER" \
-rabbit lager_upgrade_file "$RABBITMQ_LAGER_HANDLER_UPGRADE" \
-rabbit feature_flags_file "\"$RABBITMQ_FEATURE_FLAGS_FILE\"" \
-rabbit enabled_plugins_file "\"$RABBITMQ_ENABLED_PLUGINS_FILE\"" \
-rabbit plugins_dir "\"$RABBITMQ_PLUGINS_DIR\"" \
-rabbit plugins_expand_dir "\"$RABBITMQ_PLUGINS_EXPAND_DIR\"" \
-os_mon start_cpu_sup false \
-os_mon start_disksup false \
-os_mon start_memsup false \
-mnesia dir "\"${RABBITMQ_MNESIA_DIR}\"" \
-ra data_dir "\"${RABBITMQ_QUORUM_DIR}\"" \
${RABBITMQ_SERVER_START_ARGS} \
${RABBITMQ_DIST_ARG} \
"$@"
}
stop_rabbitmq_server() {
RABBITMQCTL="$(dirname "$0")/rabbitmqctl"
if ${RABBITMQCTL} -n ${RABBITMQ_NODENAME} status >/dev/null 2>&1; then
${RABBITMQCTL} -n ${RABBITMQ_NODENAME} stop
fi
}
check_start_params() {
check_not_empty RABBITMQ_BOOT_MODULE
check_not_empty RABBITMQ_NAME_TYPE
check_not_empty RABBITMQ_NODENAME
check_not_empty SASL_BOOT_FILE
check_not_empty RABBITMQ_IO_THREAD_POOL_SIZE
}
@ -346,6 +64,61 @@ check_not_empty() {
fi
}
start_rabbitmq_server() {
set -e
_rmq_env_set_erl_libs
ensure_thread_pool_size
RABBITMQ_START_RABBIT=
[ "x" = "x$RABBITMQ_ALLOW_INPUT" ] && RABBITMQ_START_RABBIT=" -noinput"
if test -z "$RABBITMQ_NODE_ONLY"; then
if test "$USE_RABBIT_BOOT_SCRIPT"; then
# TODO: This is experimental and undocumented at this point.
# It is here just to do simple checks while playing with how
# RabbitMQ is started.
"$SCRIPTS_DIR/rabbitmq-rel" gen-boot
SASL_BOOT_FILE=rabbit
test -f "$SASL_BOOT_FILE.boot"
RABBITMQ_START_RABBIT="$RABBITMQ_START_RABBIT -init_debug"
else
RABBITMQ_START_RABBIT="$RABBITMQ_START_RABBIT -s $RABBITMQ_BOOT_MODULE boot"
fi
fi
# We need to turn off path expansion because some of the vars,
# notably RABBITMQ_SERVER_ERL_ARGS, contain terms that look like
# globs and there is no other way of preventing their expansion.
set -f
export ERL_MAX_ETS_TABLES \
SYS_PREFIX
check_start_params
exec erl \
-pa "$RABBITMQ_SERVER_CODE_PATH" \
${RABBITMQ_START_RABBIT} \
-boot "${SASL_BOOT_FILE}" \
+W w \
+K true \
+A ${RABBITMQ_IO_THREAD_POOL_SIZE} \
${RABBITMQ_DEFAULT_ALLOC_ARGS} \
${RABBITMQ_SERVER_ERL_ARGS} \
${RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS} \
${RABBITMQ_SERVER_START_ARGS} \
-lager crash_log false \
-lager handlers '[]' \
"$@"
}
stop_rabbitmq_server() {
if test "$rabbitmq_server_pid"; then
kill -TERM "$rabbitmq_server_pid"
wait "$rabbitmq_server_pid" || true
fi
}
if [ "$RABBITMQ_ALLOW_INPUT" -o "$RUNNING_UNDER_SYSTEMD" -o "$detached" ]; then
# Run erlang VM directly, completely replacing current shell
# process - so the pid file written in the code above will be
@ -381,7 +154,7 @@ else
trap "stop_rabbitmq_server; exit 130" INT
start_rabbitmq_server "$@" &
rabbitmq_server_pid=$!
export rabbitmq_server_pid=$!
# Block until RabbitMQ exits or a signal is caught.
# Waits for last command (which is start_rabbitmq_server)
@ -399,5 +172,5 @@ else
# force that statement to succeed and the signal handler to properly
# execute. Because the statement below has an exit code of 0, the
# signal handler has to restate the expected exit code.
wait $rabbitmq_server_pid || true
wait "$rabbitmq_server_pid" || true
fi

View File

@ -46,163 +46,8 @@ if not exist "!ERLANG_HOME!\bin\erl.exe" (
exit /B 1
)
set RABBITMQ_EBIN_ROOT=!RABBITMQ_HOME!\ebin
CALL :convert_forward_slashes "!RABBITMQ_ADVANCED_CONFIG_FILE!" RABBITMQ_ADVANCED_CONFIG_FILE
CALL :get_noex "!RABBITMQ_ADVANCED_CONFIG_FILE!" RABBITMQ_ADVANCED_CONFIG_FILE_NOEX
if "!RABBITMQ_ADVANCED_CONFIG_FILE!" == "!RABBITMQ_ADVANCED_CONFIG_FILE_NOEX!" (
set RABBITMQ_ADVANCED_CONFIG_FILE=!RABBITMQ_ADVANCED_CONFIG_FILE_NOEX!.config
REM Try to create advanced config file, if it doesn't exist
REM It still can fail to be created, but at least not for default install
if not exist "!RABBITMQ_ADVANCED_CONFIG_FILE!" (
echo []. > !RABBITMQ_ADVANCED_CONFIG_FILE!
)
)
CALL :convert_forward_slashes "!RABBITMQ_CONFIG_FILE!" RABBITMQ_CONFIG_FILE
CALL :get_noex "!RABBITMQ_CONFIG_FILE!" RABBITMQ_CONFIG_FILE_NOEX
if "!RABBITMQ_CONFIG_FILE!" == "!RABBITMQ_CONFIG_FILE_NOEX!" (
if exist "!RABBITMQ_CONFIG_FILE_NOEX!.config" (
if exist "!RABBITMQ_CONFIG_FILE_NOEX!.conf" (
rem Both files exist. Print a warning
echo "WARNING: Both old (.config) and new (.conf) format config files exist."
echo "WARNING: Using the old format config file: !RABBITMQ_CONFIG_FILE_NOEX!.config"
echo "WARNING: Please update your config files to the new format and remove the old file"
)
set RABBITMQ_CONFIG_FILE=!RABBITMQ_CONFIG_FILE_NOEX!.config
) else if exist "!RABBITMQ_CONFIG_FILE_NOEX!.conf" (
set RABBITMQ_CONFIG_FILE=!RABBITMQ_CONFIG_FILE_NOEX!.conf
) else (
rem No config file exist. Use advanced config for -config arg.
if exist "!RABBITMQ_ADVANCED_CONFIG_FILE!" (
echo "WARNING: Using RABBITMQ_ADVANCED_CONFIG_FILE: !RABBITMQ_ADVANCED_CONFIG_FILE!"
)
set RABBITMQ_CONFIG_ARG_FILE=!RABBITMQ_ADVANCED_CONFIG_FILE!
)
)
rem Set the -config argument.
rem The -config argument should not have extension.
rem the file should exist
rem the file should be a valid erlang term file
rem Config file extension is .config
if "!RABBITMQ_CONFIG_FILE_NOEX!.config" == "!RABBITMQ_CONFIG_FILE!" (
set RABBITMQ_CONFIG_ARG_FILE=!RABBITMQ_CONFIG_FILE!
) else if "!RABBITMQ_CONFIG_FILE_NOEX!.conf" == "!RABBITMQ_CONFIG_FILE!" (
set RABBITMQ_CONFIG_ARG_FILE=!RABBITMQ_ADVANCED_CONFIG_FILE!
) else if not "" == "!RABBITMQ_CONFIG_FILE!" (
if not "!RABBITMQ_CONFIG_FILE_NOEX!" == "!RABBITMQ_CONFIG_FILE!" (
rem Config file has an extension, but it's neither .conf or .config
echo "ERROR: Wrong extension for RABBITMQ_CONFIG_FILE: !RABBITMQ_CONFIG_FILE!"
echo "ERROR: extension should be either .conf or .config"
exit /B 1
)
)
CALL :convert_forward_slashes "!RABBITMQ_CONFIG_ARG_FILE!" RABBITMQ_CONFIG_ARG_FILE
CALL :get_noex "!RABBITMQ_CONFIG_ARG_FILE!" RABBITMQ_CONFIG_ARG_FILE_NOEX
if not "!RABBITMQ_CONFIG_ARG_FILE_NOEX!.config" == "!RABBITMQ_CONFIG_ARG_FILE!" (
if "!RABBITMQ_CONFIG_ARG_FILE!" == "!RABBITMQ_ADVANCED_CONFIG_FILE!" (
echo "ERROR: Wrong extension for RABBITMQ_ADVANCED_CONFIG_FILE: !RABBITMQ_ADVANCED_CONFIG_FILE!"
echo "ERROR: extension should be .config"
exit /B 1
) else (
rem We should never got here, but still there should be some explanation
echo "ERROR: Wrong extension for !RABBITMQ_CONFIG_ARG_FILE!"
echo "ERROR: extension should be .config"
exit /B 1
)
)
rem Set -config if the file exists
if exist !RABBITMQ_CONFIG_ARG_FILE! (
set RABBITMQ_CONFIG_ARG=-config "!RABBITMQ_CONFIG_ARG_FILE_NOEX!"
)
rem Set -conf and other generated config parameters
if "!RABBITMQ_CONFIG_FILE_NOEX!.conf" == "!RABBITMQ_CONFIG_FILE!" (
if not exist "!RABBITMQ_SCHEMA_DIR!" (
mkdir "!RABBITMQ_SCHEMA_DIR!"
)
if not exist "!RABBITMQ_GENERATED_CONFIG_DIR!" (
mkdir "!RABBITMQ_GENERATED_CONFIG_DIR!"
)
copy /Y "!RABBITMQ_HOME!\priv\schema\rabbit.schema" "!RABBITMQ_SCHEMA_DIR!\rabbit.schema"
set RABBITMQ_GENERATED_CONFIG_ARG=-conf "!RABBITMQ_CONFIG_FILE:\=/!" ^
-conf_dir "!RABBITMQ_GENERATED_CONFIG_DIR:\=/!" ^
-conf_script_dir "!CONF_SCRIPT_DIR:\=/!" ^
-conf_schema_dir "!RABBITMQ_SCHEMA_DIR:\=/!" ^
-conf_advanced "!RABBITMQ_ADVANCED_CONFIG_FILE:\=/!"
)
"!ERLANG_HOME!\bin\erl.exe" ^
-pa "!RABBITMQ_EBIN_ROOT:\=/!" ^
-boot !CLEAN_BOOT_FILE! ^
-noinput -hidden ^
-s rabbit_prelaunch ^
!RABBITMQ_NAME_TYPE! rabbitmqprelaunch!RANDOM!!TIME:~9!@localhost ^
-conf_advanced "!RABBITMQ_ADVANCED_CONFIG_FILE!" ^
-rabbit feature_flags_file "!RABBITMQ_FEATURE_FLAGS_FILE!" ^
-rabbit enabled_plugins_file "!RABBITMQ_ENABLED_PLUGINS_FILE!" ^
-rabbit plugins_dir "!RABBITMQ_PLUGINS_DIR!" ^
-extra "!RABBITMQ_NODENAME!"
if ERRORLEVEL 2 (
rem dist port mentioned in config, do not attempt to set it
) else if ERRORLEVEL 1 (
exit /B 1
) else (
set RABBITMQ_DIST_ARG=-kernel inet_dist_listen_min !RABBITMQ_DIST_PORT! -kernel inet_dist_listen_max !RABBITMQ_DIST_PORT!
)
rem The default allocation strategy RabbitMQ is using was introduced
rem in Erlang/OTP 20.2.3. Earlier Erlang versions fail to start with
rem this configuration. We therefore need to ensure that erl accepts
rem these values before we can use them.
rem
rem The defaults are meant to reduce RabbitMQ's memory usage and help
rem it reclaim memory at the cost of a slight decrease in performance
rem (due to an increase in memory operations). These defaults can be
rem overridden using the RABBITMQ_SERVER_ERL_ARGS variable.
set RABBITMQ_DEFAULT_ALLOC_ARGS=+MBas ageffcbf +MHas ageffcbf +MBlmbcs 512 +MHlmbcs 512 +MMmcs 30
"!ERLANG_HOME!\bin\erl.exe" ^
!RABBITMQ_DEFAULT_ALLOC_ARGS! ^
-boot !CLEAN_BOOT_FILE! ^
-noinput -eval "halt(0)"
if ERRORLEVEL 1 (
set RABBITMQ_DEFAULT_ALLOC_ARGS=
)
set RABBITMQ_LISTEN_ARG=
if not "!RABBITMQ_NODE_IP_ADDRESS!"=="" (
if not "!RABBITMQ_NODE_PORT!"=="" (
set RABBITMQ_LISTEN_ARG=-rabbit tcp_listeners [{"\"!RABBITMQ_NODE_IP_ADDRESS!\"","!RABBITMQ_NODE_PORT!"}]
)
)
REM If !RABBITMQ_LOGS! is '-', send all log messages to stdout. This is
REM particularly useful for Docker images.
if "!RABBITMQ_LOGS!" == "-" (
set SASL_ERROR_LOGGER=tty
set RABBIT_LAGER_HANDLER=tty
set RABBITMQ_LAGER_HANDLER_UPGRADE=tty
) else (
set SASL_ERROR_LOGGER=false
set RABBIT_LAGER_HANDLER="\"!RABBITMQ_LOGS:\=/!\""
set RABBITMQ_LAGER_HANDLER_UPGRADE="\"!RABBITMQ_UPGRADE_LOG:\=/!\""
)
set RABBITMQ_START_RABBIT=
if "!RABBITMQ_ALLOW_INPUT!"=="" (
set RABBITMQ_START_RABBIT=!RABBITMQ_START_RABBIT! -noinput
@ -217,9 +62,6 @@ if "!RABBITMQ_IO_THREAD_POOL_SIZE!"=="" (
set ENV_OK=true
CALL :check_not_empty "RABBITMQ_BOOT_MODULE" !RABBITMQ_BOOT_MODULE!
CALL :check_not_empty "RABBITMQ_NAME_TYPE" !RABBITMQ_NAME_TYPE!
CALL :check_not_empty "RABBITMQ_NODENAME" !RABBITMQ_NODENAME!
if "!ENV_OK!"=="false" (
EXIT /b 78
@ -232,35 +74,16 @@ if "!RABBITMQ_ALLOW_INPUT!"=="" (
)
"!ERLANG_HOME!\bin\!ERL_CMD!" ^
-pa "!RABBITMQ_EBIN_ROOT:\=/!" ^
-boot start_sasl ^
!RABBITMQ_START_RABBIT! ^
!RABBITMQ_CONFIG_ARG! ^
!RABBITMQ_GENERATED_CONFIG_ARG! ^
!RABBITMQ_NAME_TYPE! !RABBITMQ_NODENAME! ^
-boot "!SASL_BOOT_FILE!" ^
+W w ^
+A "!RABBITMQ_IO_THREAD_POOL_SIZE!" ^
!RABBITMQ_DEFAULT_ALLOC_ARGS! ^
!RABBITMQ_SERVER_ERL_ARGS! ^
!RABBITMQ_LISTEN_ARG! ^
-kernel inet_default_connect_options "[{nodelay, true}]" ^
!RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS! ^
-sasl errlog_type error ^
-sasl sasl_error_logger !SASL_ERROR_LOGGER! ^
-rabbit lager_log_root "\"!RABBITMQ_LOG_BASE:\=/!\"" ^
-rabbit lager_default_file !RABBIT_LAGER_HANDLER! ^
-rabbit lager_upgrade_file !RABBITMQ_LAGER_HANDLER_UPGRADE! ^
-rabbit feature_flags_file "\"!RABBITMQ_FEATURE_FLAGS_FILE:\=/!\"" ^
-rabbit enabled_plugins_file "\"!RABBITMQ_ENABLED_PLUGINS_FILE:\=/!\"" ^
-rabbit plugins_dir "\"!RABBITMQ_PLUGINS_DIR:\=/!\"" ^
-rabbit plugins_expand_dir "\"!RABBITMQ_PLUGINS_EXPAND_DIR:\=/!\"" ^
-mnesia dir "\"!RABBITMQ_MNESIA_DIR:\=/!\"" ^
-os_mon start_cpu_sup false ^
-os_mon start_disksup false ^
-os_mon start_memsup false ^
-ra data_dir \""!RABBITMQ_QUORUM_DIR:\=/!"\" ^
!RABBITMQ_SERVER_START_ARGS! ^
!RABBITMQ_DIST_ARG! ^
-lager crash_log false ^
-lager handlers "[]" ^
!STAR!
if ERRORLEVEL 1 (
@ -277,17 +100,6 @@ if "%~2"=="" (
)
EXIT /B 0
:get_noex
set "%~2=%~dpn1"
EXIT /B 0
rem Convert unix style path separators into windows style path separators
rem needed for comparing with _NOEX variables
rem rabbitmq/rabbitmq-server#1962
:convert_forward_slashes
set "%~2=%~dpf1"
EXIT /B 0
endlocal
endlocal
endlocal

View File

@ -35,6 +35,38 @@ REM Get default settings with user overrides for (RABBITMQ_)<var_name>
REM Non-empty defaults should be set in rabbitmq-env
call "%TDP0%\rabbitmq-env.bat" %~n0
REM Check for the short names here too
if "!RABBITMQ_USE_LONGNAME!"=="true" (
set RABBITMQ_NAME_TYPE=-name
set NAMETYPE=longnames
) else (
if "!USE_LONGNAME!"=="true" (
set RABBITMQ_USE_LONGNAME=true
set RABBITMQ_NAME_TYPE=-name
set NAMETYPE=longnames
) else (
set RABBITMQ_USE_LONGNAME=false
set RABBITMQ_NAME_TYPE=-sname
set NAMETYPE=shortnames
)
)
REM [ "x" = "x$RABBITMQ_NODENAME" ] && RABBITMQ_NODENAME=${NODENAME}
if "!RABBITMQ_NODENAME!"=="" (
if "!NODENAME!"=="" (
REM We use Erlang to query the local hostname because
REM !COMPUTERNAME! and Erlang may return different results.
REM Start erl with -sname to make sure epmd is started.
call "%ERLANG_HOME%\bin\erl.exe" -A0 -noinput -boot start_clean -sname rabbit-prelaunch-epmd -eval "init:stop()." >nul 2>&1
for /f "delims=" %%F in ('call "%ERLANG_HOME%\bin\erl.exe" -A0 -noinput -boot start_clean -eval "net_kernel:start([list_to_atom(""rabbit-gethostname-"" ++ os:getpid()), %NAMETYPE%]), [_, H] = string:tokens(atom_to_list(node()), ""@""), io:format(""~s~n"", [H]), init:stop()."') do @set HOSTNAME=%%F
set RABBITMQ_NODENAME=rabbit@!HOSTNAME!
set HOSTNAME=
) else (
set RABBITMQ_NODENAME=!NODENAME!
)
)
set NAMETYPE=
set STARVAR=
shift
:loop1
@ -112,16 +144,6 @@ if not exist "!RABBITMQ_BASE!" (
echo Creating base directory !RABBITMQ_BASE! & mkdir "!RABBITMQ_BASE!"
)
set ENV_OK=true
CALL :check_not_empty "RABBITMQ_BOOT_MODULE" !RABBITMQ_BOOT_MODULE!
CALL :check_not_empty "RABBITMQ_NAME_TYPE" !RABBITMQ_NAME_TYPE!
CALL :check_not_empty "RABBITMQ_NODENAME" !RABBITMQ_NODENAME!
if "!ENV_OK!"=="false" (
EXIT /b 78
)
"!ERLANG_SERVICE_MANAGER_PATH!\erlsrv" list !RABBITMQ_SERVICENAME! 2>NUL 1>NUL
if errorlevel 1 (
"!ERLANG_SERVICE_MANAGER_PATH!\erlsrv" add !RABBITMQ_SERVICENAME! -internalservicename !RABBITMQ_SERVICENAME!
@ -129,165 +151,8 @@ if errorlevel 1 (
echo !RABBITMQ_SERVICENAME! service is already present - only updating service parameters
)
set RABBITMQ_EBIN_ROOT=!RABBITMQ_HOME!\ebin
CALL :convert_forward_slashes "!RABBITMQ_ADVANCED_CONFIG_FILE!" RABBITMQ_ADVANCED_CONFIG_FILE
CALL :get_noex "!RABBITMQ_ADVANCED_CONFIG_FILE!" RABBITMQ_ADVANCED_CONFIG_FILE_NOEX
if "!RABBITMQ_ADVANCED_CONFIG_FILE!" == "!RABBITMQ_ADVANCED_CONFIG_FILE_NOEX!" (
set RABBITMQ_ADVANCED_CONFIG_FILE=!RABBITMQ_ADVANCED_CONFIG_FILE_NOEX!.config
REM Try to create advanced config file, if it doesn't exist
REM It still can fail to be created, but at least not for default install
if not exist "!RABBITMQ_ADVANCED_CONFIG_FILE!" (
echo []. > !RABBITMQ_ADVANCED_CONFIG_FILE!
)
)
CALL :convert_forward_slashes "!RABBITMQ_CONFIG_FILE!" RABBITMQ_CONFIG_FILE
CALL :get_noex "!RABBITMQ_CONFIG_FILE!" RABBITMQ_CONFIG_FILE_NOEX
if "!RABBITMQ_CONFIG_FILE!" == "!RABBITMQ_CONFIG_FILE_NOEX!" (
if exist "!RABBITMQ_CONFIG_FILE_NOEX!.config" (
if exist "!RABBITMQ_CONFIG_FILE_NOEX!.conf" (
rem Both files exist. Print a warning
echo "WARNING: Both old (.config) and new (.conf) format config files exist."
echo "WARNING: Using the old format config file: !RABBITMQ_CONFIG_FILE_NOEX!.config"
echo "WARNING: Please update your config files to the new format and remove the old file"
)
set RABBITMQ_CONFIG_FILE=!RABBITMQ_CONFIG_FILE_NOEX!.config
) else if exist "!RABBITMQ_CONFIG_FILE_NOEX!.conf" (
set RABBITMQ_CONFIG_FILE=!RABBITMQ_CONFIG_FILE_NOEX!.conf
) else (
rem No config file exist. Use advanced config for -config arg.
if exist "!RABBITMQ_ADVANCED_CONFIG_FILE!" (
echo "WARNING: Using RABBITMQ_ADVANCED_CONFIG_FILE: !RABBITMQ_ADVANCED_CONFIG_FILE!"
)
set RABBITMQ_CONFIG_ARG_FILE=!RABBITMQ_ADVANCED_CONFIG_FILE!
)
)
rem Set the -config argument.
rem The -config argument should not have extension.
rem the file should exist
rem the file should be a valid erlang term file
rem Config file extension is .config
if "!RABBITMQ_CONFIG_FILE_NOEX!.config" == "!RABBITMQ_CONFIG_FILE!" (
set RABBITMQ_CONFIG_ARG_FILE=!RABBITMQ_CONFIG_FILE!
) else if "!RABBITMQ_CONFIG_FILE_NOEX!.conf" == "!RABBITMQ_CONFIG_FILE!" (
set RABBITMQ_CONFIG_ARG_FILE=!RABBITMQ_ADVANCED_CONFIG_FILE!
) else if not "" == "!RABBITMQ_CONFIG_FILE!" (
if not "!RABBITMQ_CONFIG_FILE_NOEX!" == "!RABBITMQ_CONFIG_FILE!" (
rem Config file has an extension, but it's neither .conf or .config
echo "ERROR: Wrong extension for RABBITMQ_CONFIG_FILE: !RABBITMQ_CONFIG_FILE!"
echo "ERROR: extension should be either .conf or .config"
exit /B 1
)
)
CALL :convert_forward_slashes "!RABBITMQ_CONFIG_ARG_FILE!" RABBITMQ_CONFIG_ARG_FILE
CALL :get_noex "!RABBITMQ_CONFIG_ARG_FILE!" RABBITMQ_CONFIG_ARG_FILE_NOEX
if not "!RABBITMQ_CONFIG_ARG_FILE_NOEX!.config" == "!RABBITMQ_CONFIG_ARG_FILE!" (
if "!RABBITMQ_CONFIG_ARG_FILE!" == "!RABBITMQ_ADVANCED_CONFIG_FILE!" (
echo "ERROR: Wrong extension for RABBITMQ_ADVANCED_CONFIG_FILE: !RABBITMQ_ADVANCED_CONFIG_FILE!"
echo "ERROR: extension should be .config"
exit /B 1
) else (
rem We should never got here, but still there should be some explanation
echo "ERROR: Wrong extension for !RABBITMQ_CONFIG_ARG_FILE!"
echo "ERROR: extension should be .config"
exit /B 1
)
)
rem Set -config if the file exists
if exist !RABBITMQ_CONFIG_ARG_FILE! (
set RABBITMQ_CONFIG_ARG=-config "!RABBITMQ_CONFIG_ARG_FILE_NOEX!"
)
rem Set -conf and other generated config parameters
if "!RABBITMQ_CONFIG_FILE_NOEX!.conf" == "!RABBITMQ_CONFIG_FILE!" (
if not exist "!RABBITMQ_SCHEMA_DIR!" (
mkdir "!RABBITMQ_SCHEMA_DIR!"
)
if not exist "!RABBITMQ_GENERATED_CONFIG_DIR!" (
mkdir "!RABBITMQ_GENERATED_CONFIG_DIR!"
)
copy /Y "!RABBITMQ_HOME!\priv\schema\rabbit.schema" "!RABBITMQ_SCHEMA_DIR!\rabbit.schema"
set RABBITMQ_GENERATED_CONFIG_ARG=-conf "!RABBITMQ_CONFIG_FILE:\=/!" ^
-conf_dir "!RABBITMQ_GENERATED_CONFIG_DIR:\=/!" ^
-conf_script_dir "!CONF_SCRIPT_DIR:\=/!" ^
-conf_schema_dir "!RABBITMQ_SCHEMA_DIR:\=/!" ^
-conf_advanced "!RABBITMQ_ADVANCED_CONFIG_FILE:\=/!"
)
"!ERLANG_HOME!\bin\erl.exe" ^
-pa "!RABBITMQ_EBIN_ROOT:\=/!" ^
-boot !CLEAN_BOOT_FILE! ^
-noinput -hidden ^
-s rabbit_prelaunch ^
!RABBITMQ_NAME_TYPE! rabbitmqprelaunch!RANDOM!!TIME:~9!@localhost ^
-conf_advanced "!RABBITMQ_ADVANCED_CONFIG_FILE!" ^
-rabbit feature_flags_file "!RABBITMQ_FEATURE_FLAGS_FILE!" ^
-rabbit enabled_plugins_file "!RABBITMQ_ENABLED_PLUGINS_FILE!" ^
-rabbit plugins_dir "!RABBITMQ_PLUGINS_DIR!" ^
-extra "!RABBITMQ_NODENAME!"
if ERRORLEVEL 3 (
rem ERRORLEVEL means (or greater) so we need to catch all other failure
rem cases here
exit /B 1
) else if ERRORLEVEL 2 (
rem dist port mentioned in config, do not attempt to set it
) else if ERRORLEVEL 1 (
exit /B 1
) else (
set RABBITMQ_DIST_ARG=-kernel inet_dist_listen_min !RABBITMQ_DIST_PORT! -kernel inet_dist_listen_max !RABBITMQ_DIST_PORT!
)
rem The default allocation strategy RabbitMQ is using was introduced
rem in Erlang/OTP 20.2.3. Earlier Erlang versions fail to start with
rem this configuration. We therefore need to ensure that erl accepts
rem these values before we can use them.
rem
rem The defaults are meant to reduce RabbitMQ's memory usage and help
rem it reclaim memory at the cost of a slight decrease in performance
rem (due to an increase in memory operations). These defaults can be
rem overridden using the RABBITMQ_SERVER_ERL_ARGS variable.
set RABBITMQ_DEFAULT_ALLOC_ARGS=+MBas ageffcbf +MHas ageffcbf +MBlmbcs 512 +MHlmbcs 512 +MMmcs 30
"!ERLANG_HOME!\bin\erl.exe" ^
!RABBITMQ_DEFAULT_ALLOC_ARGS! ^
-boot !CLEAN_BOOT_FILE! ^
-noinput -eval "halt(0)"
if ERRORLEVEL 1 (
set RABBITMQ_DEFAULT_ALLOC_ARGS=
)
set RABBITMQ_LISTEN_ARG=
if not "!RABBITMQ_NODE_IP_ADDRESS!"=="" (
if not "!RABBITMQ_NODE_PORT!"=="" (
set RABBITMQ_LISTEN_ARG=-rabbit tcp_listeners "[{\"!RABBITMQ_NODE_IP_ADDRESS!\", !RABBITMQ_NODE_PORT!}]"
)
)
if "!RABBITMQ_LOGS!" == "-" (
set SASL_ERROR_LOGGER=tty
set RABBIT_LAGER_HANDLER=tty
set RABBITMQ_LAGER_HANDLER_UPGRADE=tty
) else (
set SASL_ERROR_LOGGER=false
set RABBIT_LAGER_HANDLER="\"!RABBITMQ_LOGS:\=/!\""
set RABBITMQ_LAGER_HANDLER_UPGRADE="\"!RABBITMQ_UPGRADE_LOG:\=/!\""
)
set RABBITMQ_START_RABBIT=
if "!RABBITMQ_NODE_ONLY!"=="" (
set RABBITMQ_START_RABBIT=-s "!RABBITMQ_BOOT_MODULE!" boot
@ -301,45 +166,46 @@ if "!RABBITMQ_SERVICE_RESTART!"=="" (
set RABBITMQ_SERVICE_RESTART=restart
)
set ENV_OK=true
CALL :check_not_empty "RABBITMQ_BOOT_MODULE" !RABBITMQ_BOOT_MODULE!
CALL :check_not_empty "RABBITMQ_NAME_TYPE" !RABBITMQ_NAME_TYPE!
CALL :check_not_empty "RABBITMQ_NODENAME" !RABBITMQ_NODENAME!
if "!ENV_OK!"=="false" (
EXIT /b 78
)
set ERLANG_SERVICE_ARGUMENTS= ^
-pa "!RABBITMQ_EBIN_ROOT:\=/!" ^
-boot start_sasl ^
!RABBITMQ_START_RABBIT! ^
!RABBITMQ_CONFIG_ARG! ^
!RABBITMQ_GENERATED_CONFIG_ARG! ^
-boot "!SASL_BOOT_FILE!" ^
+W w ^
+A "!RABBITMQ_IO_THREAD_POOL_SIZE!" ^
!RABBITMQ_DEFAULT_ALLOC_ARGS! ^
!RABBITMQ_SERVER_ERL_ARGS! ^
!RABBITMQ_LISTEN_ARG! ^
-kernel inet_default_connect_options "[{nodelay,true}]" ^
!RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS! ^
-sasl errlog_type error ^
-sasl sasl_error_logger false ^
-rabbit lager_log_root "\"!RABBITMQ_LOG_BASE:\=/!\"" ^
-rabbit lager_default_file !RABBIT_LAGER_HANDLER! ^
-rabbit lager_upgrade_file !RABBITMQ_LAGER_HANDLER_UPGRADE! ^
-rabbit feature_flags_file "\"!RABBITMQ_FEATURE_FLAGS_FILE:\=/!\"" ^
-rabbit enabled_plugins_file "\"!RABBITMQ_ENABLED_PLUGINS_FILE:\=/!\"" ^
-rabbit plugins_dir "\"!RABBITMQ_PLUGINS_DIR:\=/!\"" ^
-rabbit plugins_expand_dir "\"!RABBITMQ_PLUGINS_EXPAND_DIR:\=/!\"" ^
-rabbit windows_service_config "\"!RABBITMQ_CONFIG_FILE:\=/!\"" ^
-mnesia dir "\"!RABBITMQ_MNESIA_DIR:\=/!\"" ^
-os_mon start_cpu_sup false ^
-os_mon start_disksup false ^
-os_mon start_memsup false ^
-ra data_dir \""!RABBITMQ_QUORUM_DIR:\=/!"\" ^
!RABBITMQ_SERVER_START_ARGS! ^
!RABBITMQ_DIST_ARG! ^
-lager crash_log false ^
-lager handlers "[]" ^
!STARVAR!
set ERLANG_SERVICE_ARGUMENTS=!ERLANG_SERVICE_ARGUMENTS:\=\\!
set ERLANG_SERVICE_ARGUMENTS=!ERLANG_SERVICE_ARGUMENTS:"=\"!
rem We resolve %APPDATA% at install time so that the user's %APPDATA%
rem is passed to `rabbit_env` at runtime (instead of the service's
rem %APPDAT%).
rem
rem The goal is to keep the same behavior as when RabbitMQ data
rem locations were decided in `rabbitmq-env.bat` (sourced by this
rem script), even if now, we compute everything in `rabbit_env` at
rem runtime.
rem
rem We may revisit this in the future so that no data is stored in a
rem user-specific directory.
"!ERLANG_SERVICE_MANAGER_PATH!\erlsrv" set !RABBITMQ_SERVICENAME! ^
-onfail !RABBITMQ_SERVICE_RESTART! ^
-machine "!ERLANG_SERVICE_MANAGER_PATH!\erl.exe" ^
-env ERL_CRASH_DUMP="!RABBITMQ_BASE:\=/!/erl_crash.dump" ^
-env APPDATA="!APPDATA!" ^
-env ERL_LIBS="!ERL_LIBS!" ^
-env ERL_MAX_ETS_TABLES="!ERL_MAX_ETS_TABLES!" ^
-env ERL_MAX_PORTS="!ERL_MAX_PORTS!" ^
@ -386,17 +252,6 @@ if "%~2"=="" (
)
EXIT /B 0
:get_noex
set "%~2=%~dpn1"
EXIT /B 0
rem Convert unix style path separators into windows style path separators
rem needed for comparing with _NOEX variables
rem rabbitmq/rabbitmq-server#1962
:convert_forward_slashes
set "%~2=%~dpf1"
EXIT /B 0
endlocal
endlocal
endlocal

View File

@ -49,11 +49,6 @@ if not defined ERL_CRASH_DUMP_SECONDS (
-boot !CLEAN_BOOT_FILE! ^
-noinput -noshell -hidden -smp enable ^
!RABBITMQ_CTL_ERL_ARGS! ^
-kernel inet_dist_listen_min !RABBITMQ_CTL_DIST_PORT_MIN! ^
-kernel inet_dist_listen_max !RABBITMQ_CTL_DIST_PORT_MAX! ^
-sasl errlog_type error ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-nodename !RABBITMQ_NODENAME! ^
-run escript start ^
-escript main rabbitmqctl_escript ^
-extra "%RABBITMQ_HOME%\escript\rabbitmq-upgrade" !STAR!
@ -62,5 +57,6 @@ if ERRORLEVEL 1 (
exit /B %ERRORLEVEL%
)
endlocal
EXIT /B 0
endlocal

View File

@ -49,10 +49,6 @@ if not defined ERL_CRASH_DUMP_SECONDS (
-boot !CLEAN_BOOT_FILE! ^
-noinput -noshell -hidden -smp enable ^
!RABBITMQ_CTL_ERL_ARGS! ^
-kernel inet_dist_listen_min !RABBITMQ_CTL_DIST_PORT_MIN! ^
-kernel inet_dist_listen_max !RABBITMQ_CTL_DIST_PORT_MAX! ^
-sasl errlog_type error ^
-mnesia dir \""!RABBITMQ_MNESIA_DIR:\=/!"\" ^
-run escript start ^
-escript main rabbitmqctl_escript ^
-extra "%RABBITMQ_HOME%\escript\rabbitmqctl" !STAR!
@ -61,5 +57,7 @@ if ERRORLEVEL 1 (
exit /B %ERRORLEVEL%
)
EXIT /B 0
endlocal
endlocal

View File

@ -30,15 +30,9 @@
-export([start/2, stop/1, prep_stop/1]).
-export([start_apps/1, start_apps/2, stop_apps/1]).
-export([log_locations/0, config_files/0, decrypt_config/2]). %% for testing and mgmt-agent
-export([log_locations/0, config_files/0]). %% for testing and mgmt-agent
-export([is_booted/1, is_booted/0, is_booting/1, is_booting/0]).
-ifdef(TEST).
-export([start_logger/0]).
-endif.
%%---------------------------------------------------------------------------
%% Boot steps.
-export([maybe_insert_default_data/0, boot_delegate/0, recover/0]).
@ -262,7 +256,7 @@
-include("rabbit_framing.hrl").
-include("rabbit.hrl").
-define(APPS, [os_mon, mnesia, rabbit_common, ra, sysmon_handler, rabbit]).
-define(APPS, [os_mon, mnesia, rabbit_common, rabbitmq_prelaunch, ra, sysmon_handler, rabbit]).
-define(ASYNC_THREADS_WARNING_THRESHOLD, 8).
@ -282,75 +276,93 @@
%%----------------------------------------------------------------------------
ensure_application_loaded() ->
%% We end up looking at the rabbit app's env for HiPE and log
%% handling, so it needs to be loaded. But during the tests, it
%% may end up getting loaded twice, so guard against that.
case application:load(rabbit) of
ok -> ok;
{error, {already_loaded, rabbit}} -> ok
end.
-spec start() -> 'ok'.
start() ->
start_it(fun() ->
%% We do not want to upgrade mnesia after just
%% restarting the app.
ok = ensure_application_loaded(),
HipeResult = rabbit_hipe:maybe_hipe_compile(),
ok = start_logger(),
rabbit_hipe:log_hipe_result(HipeResult),
Apps = load_all_apps(),
rabbit_feature_flags:initialize_registry(),
rabbit_node_monitor:prepare_cluster_status_files(),
rabbit_mnesia:check_cluster_consistency(),
broker_start(Apps)
end).
%% start() vs. boot(): we want to throw an error in start().
start_it(temporary).
-spec boot() -> 'ok'.
boot() ->
start_it(fun() ->
ensure_config(),
ok = ensure_application_loaded(),
HipeResult = rabbit_hipe:maybe_hipe_compile(),
ok = start_logger(),
rabbit_hipe:log_hipe_result(HipeResult),
Apps = load_all_apps(),
rabbit_feature_flags:initialize_registry(),
rabbit_node_monitor:prepare_cluster_status_files(),
ok = rabbit_upgrade:maybe_upgrade_mnesia(),
%% It's important that the consistency check happens after
%% the upgrade, since if we are a secondary node the
%% primary node will have forgotten us
rabbit_mnesia:check_cluster_consistency(),
broker_start(Apps)
end).
%% start() vs. boot(): we want the node to exit in boot(). Because
%% applications are started with `transient`, any error during their
%% startup will abort the node.
start_it(transient).
ensure_config() ->
case rabbit_config:validate_config_files() of
ok -> ok;
{error, {ErrFmt, ErrArgs}} ->
throw({error, {check_config_file, ErrFmt, ErrArgs}})
run_prelaunch_second_phase() ->
%% Finish the prelaunch phase started by the `rabbitmq_prelaunch`
%% application.
%%
%% The first phase was handled by the `rabbitmq_prelaunch`
%% application. It was started in one of the following way:
%% - from an Erlang release boot script;
%% - from the rabbit:boot/0 or rabbit:start/0 functions.
%%
%% The `rabbitmq_prelaunch` application creates the context map from
%% the environment and the configuration files early during Erlang
%% VM startup. Once it is done, all application environments are
%% configured (in particular `mnesia` and `ra`).
%%
%% This second phase depends on other modules & facilities of
%% RabbitMQ core. That's why we need to run it now, from the
%% `rabbit` application start function.
%% We assert Mnesia is stopped before we run the prelaunch
%% phases. See `rabbit_prelaunch` for an explanation.
%%
%% This is the second assertion, just in case Mnesia is started
%% between the two prelaunch phases.
rabbit_prelaunch:assert_mnesia_is_stopped(),
%% Get the context created by `rabbitmq_prelaunch` then proceed
%% with all steps in this phase.
#{initial_pass := IsInitialPass} =
Context = rabbit_prelaunch:get_context(),
case IsInitialPass of
true ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug(
"== Prelaunch phase [2/2] (initial pass) ==");
false ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Prelaunch phase [2/2] =="),
ok
end,
case rabbit_config:prepare_and_use_config() of
{error, {generation_error, Error}} ->
throw({error, {generate_config_file, Error}});
ok -> ok
end.
load_all_apps() ->
Plugins = rabbit_plugins:setup(),
ToBeLoaded = Plugins ++ ?APPS,
app_utils:load_applications(ToBeLoaded),
ToBeLoaded.
%% 1. Feature flags registry.
ok = rabbit_prelaunch_feature_flags:setup(Context),
broker_start(Apps) ->
start_loaded_apps(Apps),
maybe_sd_notify(),
ok = rabbit_lager:broker_is_started(),
ok = log_broker_started(rabbit_plugins:strictly_plugins(rabbit_plugins:active())).
%% 2. Configuration check + loading.
ok = rabbit_prelaunch_conf:setup(Context),
%% 3. Logging.
ok = rabbit_prelaunch_logging:setup(Context),
case IsInitialPass of
true ->
%% 4. HiPE compilation.
ok = rabbit_prelaunch_hipe:setup(Context);
false ->
ok
end,
%% 5. Clustering.
ok = rabbit_prelaunch_cluster:setup(Context),
%% Start Mnesia now that everything is ready.
rabbit_log_prelaunch:debug("Starting Mnesia"),
ok = mnesia:start(),
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Prelaunch DONE =="),
case IsInitialPass of
true -> rabbit_prelaunch:initial_pass_finished();
false -> ok
end,
ok.
%% Try to send systemd ready notification if it makes sense in the
%% current environment. standard_error is used intentionally in all
@ -465,41 +477,89 @@ sd_wait_activation(Port, Unit, AttemptsLeft) ->
false
end.
start_it(StartFun) ->
start_it(StartType) ->
case spawn_boot_marker() of
{ok, Marker} ->
T0 = erlang:timestamp(),
rabbit_log:info("RabbitMQ is asked to start...", []),
try
{ok, _} = application:ensure_all_started(rabbitmq_prelaunch,
StartType),
{ok, _} = application:ensure_all_started(rabbit,
StartType),
ok = wait_for_ready_or_stopped(),
T1 = erlang:timestamp(),
rabbit_log_prelaunch:debug(
"Time to start RabbitMQ: ~p µs",
[timer:now_diff(T1, T0)]),
stop_boot_marker(Marker),
ok
catch
error:{badmatch, Error}:_ ->
stop_boot_marker(Marker),
case StartType of
temporary -> throw(Error);
_ -> exit(Error)
end
end;
{already_booting, Marker} ->
stop_boot_marker(Marker),
ok
end.
wait_for_ready_or_stopped() ->
ok = rabbit_prelaunch:wait_for_boot_state(ready),
case rabbit_prelaunch:get_boot_state() of
ready ->
ok;
_ ->
ok = rabbit_prelaunch:wait_for_boot_state(stopped),
rabbit_prelaunch:get_stop_reason()
end.
spawn_boot_marker() ->
%% Compatibility with older RabbitMQ versions:
%% We register a process doing nothing to indicate that RabbitMQ is
%% booting. This is checked by `is_booting(Node)` on a remote node.
Marker = spawn_link(fun() -> receive stop -> ok end end),
case catch register(rabbit_boot, Marker) of
true -> try
case is_running() of
true -> ok;
false -> StartFun()
end
catch Class:Reason ->
boot_error(Class, Reason)
after
unlink(Marker),
Marker ! stop,
%% give the error loggers some time to catch up
timer:sleep(100)
end;
_ -> unlink(Marker),
Marker ! stop
true -> {ok, Marker};
_ -> {already_booting, Marker}
end.
stop_boot_marker(Marker) ->
unlink(Marker),
Marker ! stop,
ok.
-spec stop() -> 'ok'.
stop() ->
case whereis(rabbit_boot) of
undefined -> ok;
_ ->
rabbit_log:info("RabbitMQ hasn't finished starting yet. Waiting for startup to finish before stopping..."),
ok = wait_for_boot_to_finish(node())
end,
rabbit_log:info("RabbitMQ is asked to stop...~n", []),
Apps = ?APPS ++ rabbit_plugins:active(),
case wait_for_ready_or_stopped() of
ok ->
case rabbit_prelaunch:get_boot_state() of
ready ->
rabbit_log:info("RabbitMQ is asked to stop..."),
do_stop(),
rabbit_log:info(
"Successfully stopped RabbitMQ and its dependencies"),
ok;
stopped ->
ok
end;
_ ->
ok
end.
do_stop() ->
Apps0 = ?APPS ++ rabbit_plugins:active(),
%% We ensure that Mnesia is stopped last (or more exactly, after rabbit).
Apps1 = app_utils:app_dependency_order(Apps0, true) -- [mnesia],
Apps = [mnesia | Apps1],
%% this will also perform unregistration with the peer discovery backend
%% as needed
stop_apps(app_utils:app_dependency_order(Apps, true)),
rabbit_log:info("Successfully stopped RabbitMQ and its dependencies~n", []).
stop_apps(Apps).
-spec stop_and_halt() -> no_return().
@ -541,57 +601,8 @@ start_apps(Apps, RestartTypes) ->
ok = rabbit_feature_flags:refresh_feature_flags_after_app_load(Apps),
start_loaded_apps(Apps, RestartTypes).
start_loaded_apps(Apps) ->
start_loaded_apps(Apps, #{}).
start_loaded_apps(Apps, RestartTypes) ->
ensure_sysmon_handler_app_config(),
%% make Ra use a custom logger that dispatches to lager instead of the
%% default OTP logger
application:set_env(ra, logger_module, rabbit_log_ra_shim),
%% use a larger segments size for queues
case application:get_env(ra, segment_max_entries) of
undefined ->
application:set_env(ra, segment_max_entries, 32768);
_ ->
ok
end,
case application:get_env(ra, wal_max_size_bytes) of
undefined ->
application:set_env(ra, wal_max_size_bytes, 536870912); %% 5 * 2 ^ 20
_ ->
ok
end,
ConfigEntryDecoder = case application:get_env(rabbit, config_entry_decoder) of
undefined ->
[];
{ok, Val} ->
Val
end,
PassPhrase = case proplists:get_value(passphrase, ConfigEntryDecoder) of
prompt ->
IoDevice = get_input_iodevice(),
io:setopts(IoDevice, [{echo, false}]),
PP = lists:droplast(io:get_line(IoDevice,
"\nPlease enter the passphrase to unlock encrypted "
"configuration entries.\n\nPassphrase: ")),
io:setopts(IoDevice, [{echo, true}]),
io:format(IoDevice, "~n", []),
PP;
{file, Filename} ->
{ok, File} = file:read_file(Filename),
[PP|_] = binary:split(File, [<<"\r\n">>, <<"\n">>]),
PP;
PP ->
PP
end,
Algo = {
proplists:get_value(cipher, ConfigEntryDecoder, rabbit_pbe:default_cipher()),
proplists:get_value(hash, ConfigEntryDecoder, rabbit_pbe:default_hash()),
proplists:get_value(iterations, ConfigEntryDecoder, rabbit_pbe:default_iterations()),
PassPhrase
},
decrypt_config(Apps, Algo),
rabbit_prelaunch_conf:decrypt_config(Apps),
OrderedApps = app_utils:app_dependency_order(Apps, false),
case lists:member(rabbit, Apps) of
false -> rabbit_boot_steps:run_boot_steps(Apps); %% plugin activation
@ -601,102 +612,6 @@ start_loaded_apps(Apps, RestartTypes) ->
handle_app_error(could_not_start),
RestartTypes).
%% rabbitmq/rabbitmq-server#952
%% This function is to be called after configuration has been optionally generated
%% and the sysmon_handler application loaded, but not started. It will ensure that
%% sane defaults are used for configuration settings that haven't been set by the
%% user
ensure_sysmon_handler_app_config() ->
Defaults = [
{process_limit, 100},
{port_limit, 100},
{gc_ms_limit, 0},
{schedule_ms_limit, 0},
{heap_word_limit, 0},
{busy_port, false},
{busy_dist_port, true}
],
lists:foreach(fun({K, V}) ->
case application:get_env(sysmon_handler, K) of
undefined ->
application:set_env(sysmon_handler, K, V);
_ ->
ok
end
end, Defaults).
%% This function retrieves the correct IoDevice for requesting
%% input. The problem with using the default IoDevice is that
%% the Erlang shell prevents us from getting the input.
%%
%% Instead we therefore look for the io process used by the
%% shell and if it can't be found (because the shell is not
%% started e.g with -noshell) we use the 'user' process.
%%
%% This function will not work when either -oldshell or -noinput
%% options are passed to erl.
get_input_iodevice() ->
case whereis(user) of
undefined -> user;
User ->
case group:interfaces(User) of
[] ->
user;
[{user_drv, Drv}] ->
case user_drv:interfaces(Drv) of
[] ->
user;
[{current_group, IoDevice}] ->
IoDevice
end
end
end.
decrypt_config([], _) ->
ok;
decrypt_config([App|Apps], Algo) ->
decrypt_app(App, application:get_all_env(App), Algo),
decrypt_config(Apps, Algo).
decrypt_app(_, [], _) ->
ok;
decrypt_app(App, [{Key, Value}|Tail], Algo) ->
try begin
case decrypt(Value, Algo) of
Value ->
ok;
NewValue ->
application:set_env(App, Key, NewValue)
end
end
catch
exit:{bad_configuration, config_entry_decoder} ->
exit({bad_configuration, config_entry_decoder});
_:Msg ->
rabbit_log:info("Error while decrypting key '~p'. Please check encrypted value, passphrase, and encryption configuration~n", [Key]),
exit({decryption_error, {key, Key}, Msg})
end,
decrypt_app(App, Tail, Algo).
decrypt({encrypted, _}, {_, _, _, undefined}) ->
exit({bad_configuration, config_entry_decoder});
decrypt({encrypted, EncValue}, {Cipher, Hash, Iterations, Password}) ->
rabbit_pbe:decrypt_term(Cipher, Hash, Iterations, Password, EncValue);
decrypt(List, Algo) when is_list(List) ->
decrypt_list(List, Algo, []);
decrypt(Value, _) ->
Value.
%% We make no distinction between strings and other lists.
%% When we receive a string, we loop through each element
%% and ultimately return the string unmodified, as intended.
decrypt_list([], _, Acc) ->
lists:reverse(Acc);
decrypt_list([{Key, Value}|Tail], Algo, Acc) when Key =/= encrypted ->
decrypt_list(Tail, Algo, [{Key, decrypt(Value, Algo)}|Acc]);
decrypt_list([Value|Tail], Algo, Acc) ->
decrypt_list(Tail, Algo, [decrypt(Value, Algo)|Acc]).
-spec stop_apps([app_name()]) -> 'ok'.
stop_apps([]) ->
@ -725,11 +640,15 @@ handle_app_error(Term) ->
is_booting() -> is_booting(node()).
is_booting(Node) when Node =:= node() ->
case rabbit_prelaunch:get_boot_state() of
booting -> true;
_ -> false
end;
is_booting(Node) ->
case rpc:call(Node, erlang, whereis, [rabbit_boot]) of
case rpc:call(Node, rabbit, is_booting, []) of
{badrpc, _} = Err -> Err;
undefined -> false;
P when is_pid(P) -> true
Ret -> Ret
end.
@ -795,9 +714,6 @@ do_wait_for_boot_to_start(Node, IterationsLeft) ->
ok
end.
wait_for_boot_to_finish(Node) ->
wait_for_boot_to_finish(Node, false, ?BOOT_FINISH_TIMEOUT).
wait_for_boot_to_finish(Node, PrintProgressReports) ->
wait_for_boot_to_finish(Node, PrintProgressReports, ?BOOT_FINISH_TIMEOUT).
@ -917,17 +833,22 @@ total_queue_count() ->
end,
0, rabbit_vhost:list_names()).
%% TODO this only determines if the rabbit application has started,
%% not if it is running, never mind plugins. It would be nice to have
%% more nuance here.
-spec is_running() -> boolean().
is_running() -> is_running(node()).
-spec is_running(node()) -> boolean().
is_running(Node) -> rabbit_nodes:is_process_running(Node, rabbit).
is_running(Node) when Node =:= node() ->
case rabbit_prelaunch:get_boot_state() of
ready -> true;
_ -> false
end;
is_running(Node) ->
case rpc:call(Node, rabbit, is_running, []) of
true -> true;
_ -> false
end.
is_booted() -> is_booted(node()).
@ -983,105 +904,136 @@ rotate_logs() ->
{'ok',pid()}.
start(normal, []) ->
case erts_version_check() of
ok ->
rabbit_log:info("~n Starting RabbitMQ ~s on Erlang ~s~n ~s~n ~s~n",
[rabbit_misc:version(), rabbit_misc:otp_release(),
?COPYRIGHT_MESSAGE, ?INFORMATION_MESSAGE]),
{ok, SupPid} = rabbit_sup:start_link(),
true = register(rabbit, self()),
print_banner(),
log_banner(),
warn_if_kernel_config_dubious(),
warn_if_disc_io_options_dubious(),
rabbit_boot_steps:run_boot_steps(),
{ok, SupPid};
{error, {erlang_version_too_old,
{found, OTPRel, ERTSVer},
{required, ?OTP_MINIMUM, ?ERTS_MINIMUM}}} ->
Msg = "This RabbitMQ version cannot run on Erlang ~s (erts ~s): "
"minimum required version is ~s (erts ~s)",
Args = [OTPRel, ERTSVer, ?OTP_MINIMUM, ?ERTS_MINIMUM],
rabbit_log:error(Msg, Args),
%% also print to stderr to make this more visible
io:format(standard_error, "Error: " ++ Msg ++ "~n", Args),
{error, {erlang_version_too_old, rabbit_misc:format("Erlang ~s or later is required, started on ~s", [?OTP_MINIMUM, OTPRel])}};
Error ->
%% Reset boot state and clear the stop reason again (it was already
%% made in rabbitmq_prelaunch).
%%
%% This is important if the previous startup attempt failed after
%% rabbitmq_prelaunch was started and the application is still
%% running.
rabbit_prelaunch:set_boot_state(booting),
rabbit_prelaunch:clear_stop_reason(),
try
run_prelaunch_second_phase(),
rabbit_log:info("~n Starting RabbitMQ ~s on Erlang ~s~n ~s~n ~s~n",
[rabbit_misc:version(), rabbit_misc:otp_release(),
?COPYRIGHT_MESSAGE, ?INFORMATION_MESSAGE]),
{ok, SupPid} = rabbit_sup:start_link(),
%% Compatibility with older RabbitMQ versions + required by
%% rabbit_node_monitor:notify_node_up/0:
%%
%% We register the app process under the name `rabbit`. This is
%% checked by `is_running(Node)` on a remote node. The process
%% is also monitord by rabbit_node_monitor.
%%
%% The process name must be registered *before* running the boot
%% steps: that's when rabbit_node_monitor will set the process
%% monitor up.
%%
%% Note that plugins were not taken care of at this point
%% either.
rabbit_log_prelaunch:debug(
"Register `rabbit` process (~p) for rabbit_node_monitor",
[self()]),
true = register(rabbit, self()),
print_banner(),
log_banner(),
warn_if_kernel_config_dubious(),
warn_if_disc_io_options_dubious(),
%% We run `rabbit` boot steps only for now. Plugins boot steps
%% will be executed as part of the postlaunch phase after they
%% are started.
rabbit_boot_steps:run_boot_steps([rabbit]),
run_postlaunch_phase(),
{ok, SupPid}
catch
throw:{error, _} = Error ->
mnesia:stop(),
rabbit_prelaunch_errors:log_error(Error),
rabbit_prelaunch:set_stop_reason(Error),
rabbit_prelaunch:set_boot_state(stopped),
Error;
Class:Exception:Stacktrace ->
mnesia:stop(),
rabbit_prelaunch_errors:log_exception(
Class, Exception, Stacktrace),
Error = {error, Exception},
rabbit_prelaunch:set_stop_reason(Error),
rabbit_prelaunch:set_boot_state(stopped),
Error
end.
run_postlaunch_phase() ->
spawn(fun() -> do_run_postlaunch_phase() end).
do_run_postlaunch_phase() ->
%% Once RabbitMQ itself is started, we need to run a few more steps,
%% in particular start plugins.
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Postlaunch phase =="),
try
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Plugins =="),
rabbit_log_prelaunch:debug("Setting plugins up"),
Plugins = rabbit_plugins:setup(),
rabbit_log_prelaunch:debug(
"Starting the following plugins: ~p", [Plugins]),
app_utils:load_applications(Plugins),
ok = rabbit_feature_flags:refresh_feature_flags_after_app_load(
Plugins),
lists:foreach(
fun(Plugin) ->
case application:ensure_all_started(Plugin) of
{ok, _} -> rabbit_boot_steps:run_boot_steps([Plugin]);
Error -> throw(Error)
end
end, Plugins),
maybe_sd_notify(),
rabbit_log_prelaunch:debug("Marking RabbitMQ as running"),
rabbit_prelaunch:set_boot_state(ready),
ok = rabbit_lager:broker_is_started(),
ok = log_broker_started(
rabbit_plugins:strictly_plugins(rabbit_plugins:active()))
catch
throw:{error, _} = Error ->
rabbit_prelaunch_errors:log_error(Error),
rabbit_prelaunch:set_stop_reason(Error),
do_stop();
Class:Exception:Stacktrace ->
rabbit_prelaunch_errors:log_exception(
Class, Exception, Stacktrace),
Error = {error, Exception},
rabbit_prelaunch:set_stop_reason(Error),
do_stop()
end.
prep_stop(State) ->
rabbit_peer_discovery:maybe_unregister(),
State.
-spec stop(_) -> 'ok'.
stop(_State) ->
stop(State) ->
rabbit_prelaunch:set_boot_state(stopping),
ok = rabbit_alarm:stop(),
ok = case rabbit_mnesia:is_clustered() of
true -> ok;
false -> rabbit_table:clear_ram_only_tables()
end,
case State of
[] -> rabbit_prelaunch:set_stop_reason(normal);
_ -> rabbit_prelaunch:set_stop_reason(State)
end,
rabbit_prelaunch:set_boot_state(stopped),
ok.
-spec boot_error(term(), not_available | [tuple()]) -> no_return().
boot_error(_, {could_not_start, rabbit, {{timeout_waiting_for_tables, _}, _}}) ->
AllNodes = rabbit_mnesia:cluster_nodes(all),
Suffix = "~nBACKGROUND~n==========~n~n"
"This cluster node was shut down while other nodes were still running.~n"
"To avoid losing data, you should start the other nodes first, then~n"
"start this one. To force this node to start, first invoke~n"
"\"rabbitmqctl force_boot\". If you do so, any changes made on other~n"
"cluster nodes after this one was shut down may be lost.~n",
{Err, Nodes} =
case AllNodes -- [node()] of
[] -> {"Timeout contacting cluster nodes. Since RabbitMQ was"
" shut down forcefully~nit cannot determine which nodes"
" are timing out.~n" ++ Suffix, []};
Ns -> {rabbit_misc:format(
"Timeout contacting cluster nodes: ~p.~n" ++ Suffix, [Ns]),
Ns}
end,
log_boot_error_and_exit(
timeout_waiting_for_tables,
"~n" ++ Err ++ rabbit_nodes:diagnostics(Nodes), []);
boot_error(_, {error, {cannot_log_to_file, unknown, Reason}}) ->
log_boot_error_and_exit(could_not_initialise_logger,
"failed to initialised logger: ~p~n",
[Reason]);
boot_error(_, {error, {cannot_log_to_file, LogFile,
{cannot_create_parent_dirs, _, Reason}}}) ->
log_boot_error_and_exit(could_not_initialise_logger,
"failed to create parent directory for log file at '~s', reason: ~p~n",
[LogFile, Reason]);
boot_error(_, {error, {cannot_log_to_file, LogFile, Reason}}) ->
log_boot_error_and_exit(could_not_initialise_logger,
"failed to open log file at '~s', reason: ~p~n",
[LogFile, Reason]);
boot_error(_, {error, {generate_config_file, Error}}) ->
log_boot_error_and_exit(generate_config_file,
"~nConfig file generation failed:~n~s"
"In case the setting comes from a plugin, make sure that the plugin is enabled.~n"
"Alternatively remove the setting from the config.~n",
[Error]);
boot_error(Class, Reason) ->
LogLocations = log_locations(),
log_boot_error_and_exit(
Reason,
"~nError description:~s"
"~nLog file(s) (may contain more information):~n" ++
lists:flatten([" ~s~n" || _ <- lists:seq(1, length(LogLocations))]),
[lager:pr_stacktrace(erlang:get_stacktrace(), {Class, Reason})] ++
LogLocations).
-spec log_boot_error_and_exit(_, _, _) -> no_return().
log_boot_error_and_exit(Reason, Format, Args) ->
rabbit_log:error(Format, Args),
io:format(standard_error, "~nBOOT FAILED~n===========~n" ++ Format ++ "~n", Args),
timer:sleep(1000),
exit(Reason).
%%---------------------------------------------------------------------------
%% boot step functions
@ -1141,10 +1093,6 @@ insert_default_data() ->
%%---------------------------------------------------------------------------
%% logging
start_logger() ->
rabbit_lager:start_logger(),
ok.
-spec log_locations() -> [rabbit_lager:log_location()].
log_locations() ->
rabbit_lager:log_locations().
@ -1176,25 +1124,29 @@ log_broker_started(Plugins) ->
rabbit_log:info(Message),
io:format(" completed with ~p plugins.~n", [length(Plugins)]).
erts_version_check() ->
ERTSVer = erlang:system_info(version),
OTPRel = rabbit_misc:otp_release(),
case rabbit_misc:version_compare(?ERTS_MINIMUM, ERTSVer, lte) of
true when ?ERTS_MINIMUM =/= ERTSVer ->
ok;
true when ?ERTS_MINIMUM =:= ERTSVer andalso ?OTP_MINIMUM =< OTPRel ->
%% When a critical regression or bug is found, a new OTP
%% release can be published without changing the ERTS
%% version. For instance, this is the case with R16B03 and
%% R16B03-1.
%%
%% In this case, we compare the release versions
%% alphabetically.
ok;
_ -> {error, {erlang_version_too_old,
{found, OTPRel, ERTSVer},
{required, ?OTP_MINIMUM, ?ERTS_MINIMUM}}}
end.
-define(RABBIT_TEXT_LOGO,
"~n ## ## ~s ~s"
"~n ## ##"
"~n ########## ~s"
"~n ###### ##"
"~n ########## ~s").
-define(FG8_START, "\033[38;5;202m").
-define(BG8_START, "\033[48;5;202m").
-define(FG32_START, "\033[38;2;255;102;0m").
-define(BG32_START, "\033[48;2;255;102;0m").
-define(C_END, "\033[0m").
-define(RABBIT_8BITCOLOR_LOGO,
"~n " ?BG8_START " " ?C_END " " ?BG8_START " " ?C_END " \033[1m" ?FG8_START "~s" ?C_END " ~s"
"~n " ?BG8_START " " ?C_END " " ?BG8_START " " ?C_END
"~n " ?BG8_START " " ?C_END " ~s"
"~n " ?BG8_START " " ?C_END " " ?BG8_START " " ?C_END
"~n " ?BG8_START " " ?C_END " ~s").
-define(RABBIT_32BITCOLOR_LOGO,
"~n " ?BG32_START " " ?C_END " " ?BG32_START " " ?C_END " \033[1m" ?FG32_START "~s" ?C_END " ~s"
"~n " ?BG32_START " " ?C_END " " ?BG32_START " " ?C_END
"~n " ?BG32_START " " ?C_END " ~s"
"~n " ?BG32_START " " ?C_END " " ?BG32_START " " ?C_END
"~n " ?BG32_START " " ?C_END " ~s").
print_banner() ->
{ok, Product} = application:get_key(description),
@ -1205,14 +1157,23 @@ print_banner() ->
(_, []) ->
{"", ["(none)"]}
end,
Logo = case rabbit_prelaunch:get_context() of
%% We use the colored logo only when running the
%% interactive shell and when colors are supported.
%%
%% Basically it means it will be used on Unix when
%% running "make run-broker" and that's about it.
#{os_type := {unix, darwin},
interactive_shell := true,
output_supports_colors := true} -> ?RABBIT_8BITCOLOR_LOGO;
#{interactive_shell := true,
output_supports_colors := true} -> ?RABBIT_32BITCOLOR_LOGO;
_ -> ?RABBIT_TEXT_LOGO
end,
%% padded list lines
{LogFmt, LogLocations} = LineListFormatter("~n ~ts", log_locations()),
{CfgFmt, CfgLocations} = LineListFormatter("~n ~ts", config_locations()),
io:format("~n ## ## ~s ~s"
"~n ## ##"
"~n ########## ~s"
"~n ###### ##"
"~n ########## ~s"
io:format(Logo ++
"~n"
"~n Doc guides: https://rabbitmq.com/documentation.html"
"~n Support: https://rabbitmq.com/contact.html"

View File

@ -1,164 +1,33 @@
-module(rabbit_config).
-export([
generate_config_file/5,
prepare_and_use_config/0,
prepare_config/1,
update_app_config/1,
schema_dir/0,
config_files/0,
get_advanced_config/0,
validate_config_files/0
get_advanced_config/0
]).
-export_type([config_location/0]).
-type config_location() :: string().
prepare_and_use_config() ->
case legacy_erlang_term_config_used() of
true ->
%% Use .config file
ok;
false ->
case prepare_config(get_confs()) of
ok ->
%% No .conf to generate from
ok;
{ok, GeneratedConfigFile} ->
%% Generated config file
update_app_config(GeneratedConfigFile);
{error, Err} ->
{error, Err}
end
end.
%% we support both the classic Erlang term
%% config file (rabbitmq.config) as well as rabbitmq.conf
legacy_erlang_term_config_used() ->
case init:get_argument(config) of
error -> false;
{ok, [Config | _]} ->
ConfigFile = Config ++ ".config",
rabbit_file:is_file(ConfigFile)
andalso
get_advanced_config() == none
case get_prelaunch_config_state() of
#{config_type := erlang,
config_advanced_file := undefined} ->
true;
_ ->
false
end.
get_confs() ->
case init:get_argument(conf) of
{ok, Confs} -> [ filename:rootname(Conf, ".conf") ++ ".conf"
|| Conf <- Confs ];
_ -> []
end.
prepare_config(Confs) ->
case {init:get_argument(conf_dir), init:get_argument(conf_script_dir)} of
{{ok, ConfDir}, {ok, ScriptDir}} ->
ConfFiles = [Conf || Conf <- Confs,
rabbit_file:is_file(Conf)],
case ConfFiles of
[] -> ok;
_ ->
case generate_config_file(ConfFiles, ConfDir, ScriptDir) of
{ok, GeneratedConfigFile} ->
{ok, GeneratedConfigFile};
{error, Reason} ->
{error, Reason}
end
end;
_ -> ok
end.
update_app_config(ConfigFile) ->
RunningApps = [ App || {App, _, _} <- application:which_applications() ],
LoadedApps = [ App || {App, _, _} <- application:loaded_applications() ],
{ok, [Config]} = file:consult(ConfigFile),
%% For application config to be updated, applications should
%% be unloaded first.
%% If an application is already running, print an error.
lists:foreach(fun({App, AppConfig}) ->
case lists:member(App, RunningApps) of
true ->
maybe_print_warning_for_running_app(App, AppConfig);
false ->
case lists:member(App, LoadedApps) of
true -> application:unload(App);
false -> ok
end
end
end,
Config),
maybe_set_net_ticktime(proplists:get_value(kernel, Config)),
ok = application_controller:change_application_data([], [ConfigFile]),
%% Make sure to load all the applications we're unloaded
lists:foreach(fun(App) -> application:load(App) end, LoadedApps),
ok.
maybe_print_warning_for_running_app(kernel, Config) ->
ConfigWithoutSupportedEntry = proplists:delete(net_ticktime, Config),
case length(ConfigWithoutSupportedEntry) > 0 of
true -> io:format(standard_error,
"~nUnable to update config for app ~p from a .conf file."
" The app is already running. Use advanced.config instead.~n", [kernel]);
false -> ok
end;
maybe_print_warning_for_running_app(App, _Config) ->
io:format(standard_error,
"~nUnable to update config for app ~p from a .conf file: "
" The app is already running.~n",
[App]).
maybe_set_net_ticktime(undefined) ->
ok;
maybe_set_net_ticktime(KernelConfig) ->
case proplists:get_value(net_ticktime, KernelConfig) of
undefined ->
ok;
NetTickTime ->
case net_kernel:set_net_ticktime(NetTickTime, 0) of
unchanged ->
ok;
change_initiated ->
ok;
{ongoing_change_to, NewNetTicktime} ->
io:format(standard_error,
"~nCouldn't set net_ticktime to ~p "
"as net_kernel is busy changing net_ticktime to ~p seconds ~n",
[NetTickTime, NewNetTicktime])
end
end.
generate_config_file(ConfFiles, ConfDir, ScriptDir) ->
generate_config_file(ConfFiles, ConfDir, ScriptDir,
schema_dir(), get_advanced_config()).
generate_config_file(ConfFiles, ConfDir, ScriptDir, SchemaDir, Advanced) ->
prepare_plugin_schemas(SchemaDir),
Cuttlefish = filename:join([ScriptDir, "cuttlefish"]),
GeneratedDir = filename:join([ConfDir, "generated"]),
AdvancedConfigArg = case check_advanced_config(Advanced) of
{ok, FileName} -> [" -a ", FileName];
none -> []
end,
rabbit_file:recursive_delete([GeneratedDir]),
Command = lists:concat(["escript ", "\"", Cuttlefish, "\"",
" -f rabbitmq -s ", "\"", SchemaDir, "\"",
" -e ", "\"", ConfDir, "\"",
[[" -c \"", ConfFile, "\""] || ConfFile <- ConfFiles],
AdvancedConfigArg]),
rabbit_log:debug("Generating config file using '~s'", [Command]),
Result = rabbit_misc:os_cmd(Command),
case string:str(Result, " -config ") of
0 -> {error, {generation_error, Result}};
case get_prelaunch_config_state() of
#{config_files := Confs} ->
[ filename:rootname(Conf, ".conf") ++ ".conf"
|| Conf <- Confs ];
_ ->
[OutFile] = rabbit_file:wildcard("rabbitmq.*.config", GeneratedDir),
ResultFile = filename:join([GeneratedDir, "rabbitmq.config"]),
rabbit_file:rename(filename:join([GeneratedDir, OutFile]),
ResultFile),
{ok, ResultFile}
[]
end.
schema_dir() ->
@ -171,17 +40,10 @@ schema_dir() ->
end
end.
check_advanced_config(none) -> none;
check_advanced_config(ConfigName) ->
case rabbit_file:is_file(ConfigName) of
true -> {ok, ConfigName};
false -> none
end.
get_advanced_config() ->
case init:get_argument(conf_advanced) of
case get_prelaunch_config_state() of
%% There can be only one advanced.config
{ok, [FileName | _]} ->
#{config_advanced_file := FileName} ->
case rabbit_file:is_file(FileName) of
true -> FileName;
false -> none
@ -189,26 +51,21 @@ get_advanced_config() ->
_ -> none
end.
prepare_plugin_schemas(SchemaDir) ->
case rabbit_file:is_dir(SchemaDir) of
true -> rabbit_plugins:extract_schemas(SchemaDir);
false -> ok
end.
-spec config_files() -> [config_location()].
config_files() ->
case legacy_erlang_term_config_used() of
true ->
case init:get_argument(config) of
{ok, Files} -> [ filename:absname(filename:rootname(File) ++ ".config")
|| [File] <- Files];
error -> case config_setting() of
none -> [];
File -> [filename:absname(filename:rootname(File, ".config") ++ ".config")
++
" (not found)"]
end
case get_prelaunch_config_state() of
#{config_files := Files} ->
[ filename:absname(filename:rootname(File) ++ ".config")
|| File <- Files];
_ ->
case config_setting() of
none -> [];
File -> [filename:absname(filename:rootname(File, ".config") ++ ".config")
++
" (not found)"]
end
end;
false ->
ConfFiles = [filename:absname(File) || File <- get_confs(),
@ -221,6 +78,8 @@ config_files() ->
end.
get_prelaunch_config_state() ->
rabbit_prelaunch_conf:get_config_state().
%% This is a pain. We want to know where the config file is. But we
%% can't specify it on the command line if it is missing or the VM
@ -232,95 +91,9 @@ config_files() ->
config_setting() ->
case application:get_env(rabbit, windows_service_config) of
{ok, File1} -> File1;
undefined -> case os:getenv("RABBITMQ_CONFIG_FILE") of
false -> none;
File2 -> File2
end
undefined ->
case application:get_env(rabbitmq_prelaunch, context) of
#{main_config_file := File2} -> File2;
_ -> none
end
end.
-spec validate_config_files() -> ok | {error, {Fmt :: string(), Args :: list()}}.
validate_config_files() ->
ConfigFile = os:getenv("RABBITMQ_CONFIG_FILE"),
AdvancedConfigFile = get_advanced_config(),
AssertConfig = case filename:extension(ConfigFile) of
".config" -> assert_config(ConfigFile, "RABBITMQ_CONFIG_FILE");
".conf" -> assert_conf(ConfigFile, "RABBITMQ_CONFIG_FILE");
_ -> ok
end,
case AssertConfig of
ok ->
assert_config(AdvancedConfigFile, "RABBITMQ_ADVANCED_CONFIG_FILE");
{error, Err} ->
{error, Err}
end.
assert_config("", _) -> ok;
assert_config(none, _) -> ok;
assert_config(Filename, Env) ->
assert_config(filename:extension(Filename), Filename, Env).
-define(ERRMSG_INDENT, " ").
assert_config(".config", Filename, Env) ->
case filelib:is_regular(Filename) of
true ->
case file:consult(Filename) of
{ok, []} -> {error,
{"Config file ~s should not be empty: ~s",
[Env, Filename]}};
{ok, [_]} -> ok;
{ok, [_|_]} -> {error,
{"Config file ~s must contain ONE list ended by <dot>: ~s",
[Env, Filename]}};
{error, {1, erl_parse, Err}} ->
% Note: the sequence of spaces is to indent from the [error] prefix, like this:
%
% 2018-09-06 07:05:40.225 [error] Unable to parse erlang terms from RABBITMQ_ADVANCED_CONFIG_FILE...
% Reason: ["syntax error before: ",[]]
{error, {"Unable to parse erlang terms from ~s file: ~s~n"
?ERRMSG_INDENT
"Reason: ~p~n"
?ERRMSG_INDENT
"Check that the file is in erlang term format. " ++
case Env of
"RABBITMQ_CONFIG_FILE" ->
"If you are using the new ini-style format, the file extension should be '.conf'~n";
_ -> ""
end,
[Env, Filename, Err]}};
{error, Err} ->
{error, {"Unable to parse erlang terms from ~s file: ~s~n"
?ERRMSG_INDENT
"Error: ~p~n",
[Env, Filename, Err]}}
end;
false ->
ok
end;
assert_config(BadExt, Filename, Env) ->
{error, {"'~s': Expected extension '.config', got extension '~s' for file '~s'~n", [Env, BadExt, Filename]}}.
assert_conf("", _) -> ok;
assert_conf(Filename, Env) ->
assert_conf(filename:extension(Filename), Filename, Env).
assert_conf(".conf", Filename, Env) ->
case filelib:is_regular(Filename) of
true ->
case file:consult(Filename) of
{ok, []} -> ok;
{ok, _} ->
{error, {"Wrong format of the config file ~s: ~s~n"
?ERRMSG_INDENT
"Check that the file is in the new ini-style config format. "
"If you are using the old format the file extension should "
"be .config~n",
[Env, Filename]}};
_ ->
ok
end;
false ->
ok
end;
assert_conf(BadExt, Filename, Env) ->
{error, {"'~s': Expected extension '.config', got extension '~s' for file '~s'~n", [Env, BadExt, Filename]}}.

View File

@ -20,17 +20,18 @@ maybe_hipe_compile() ->
end.
log_hipe_result({ok, disabled}) ->
ok;
rabbit_log_prelaunch:info(
"HiPE disabled: no modules were natively recompiled.~n", []);
log_hipe_result({ok, already_compiled}) ->
rabbit_log:info(
rabbit_log_prelaunch:info(
"HiPE in use: modules already natively compiled.~n", []);
log_hipe_result({ok, Count, Duration}) ->
rabbit_log:info(
rabbit_log_prelaunch:info(
"HiPE in use: compiled ~B modules in ~Bs.~n", [Count, Duration]);
log_hipe_result(false) ->
io:format(
"~nNot HiPE compiling: HiPE not found in this Erlang installation.~n"),
rabbit_log:warning(
rabbit_log_prelaunch:warning(
"Not HiPE compiling: HiPE not found in this Erlang installation.~n").
hipe_compile() ->

View File

@ -20,10 +20,9 @@
-export([setup/0, active/0, read_enabled/1, list/1, list/2, dependencies/3, running_plugins/0]).
-export([ensure/1]).
-export([extract_schemas/1]).
-export([validate_plugins/1, format_invalid_plugins/1]).
-export([is_strictly_plugin/1, strictly_plugins/2, strictly_plugins/1]).
-export([plugins_dir/0, plugins_expand_dir/0, enabled_plugins_file/0]).
-export([plugins_dir/0, plugin_names/1, plugins_expand_dir/0, enabled_plugins_file/0]).
% Export for testing purpose.
-export([is_version_supported/2, validate_plugins/2]).
@ -49,7 +48,6 @@ ensure1(FileJustChanged0) ->
FileJustChanged ->
Enabled = read_enabled(OurFile),
Wanted = prepare_plugins(Enabled),
rabbit_config:prepare_and_use_config(),
Current = active(),
Start = Wanted -- Current,
Stop = Current -- Wanted,
@ -132,50 +130,6 @@ setup() ->
Enabled = enabled_plugins(),
prepare_plugins(Enabled).
extract_schemas(SchemaDir) ->
application:load(rabbit),
{ok, EnabledFile} = application:get_env(rabbit, enabled_plugins_file),
Enabled = read_enabled(EnabledFile),
{ok, PluginsDistDir} = application:get_env(rabbit, plugins_dir),
AllPlugins = list(PluginsDistDir),
Wanted = dependencies(false, Enabled, AllPlugins),
WantedPlugins = lookup_plugins(Wanted, AllPlugins),
[ extract_schema(Plugin, SchemaDir) || Plugin <- WantedPlugins ],
application:unload(rabbit),
ok.
extract_schema(#plugin{type = ez, location = Location}, SchemaDir) ->
{ok, Files} = zip:extract(Location,
[memory, {file_filter,
fun(#zip_file{name = Name}) ->
string:str(Name, "priv/schema") > 0
end}]),
lists:foreach(
fun({FileName, Content}) ->
ok = file:write_file(filename:join([SchemaDir,
filename:basename(FileName)]),
Content)
end,
Files),
ok;
extract_schema(#plugin{type = dir, location = Location}, SchemaDir) ->
PluginSchema = filename:join([Location,
"priv",
"schema"]),
case rabbit_file:is_dir(PluginSchema) of
false -> ok;
true ->
PluginSchemaFiles =
[ filename:join(PluginSchema, FileName)
|| FileName <- rabbit_file:wildcard(".*\\.schema",
PluginSchema) ],
[ file:copy(SchemaFile, SchemaDir)
|| SchemaFile <- PluginSchemaFiles ]
end.
%% @doc Lists the plugins which are currently running.
-spec active() -> [plugin_name()].

View File

@ -1,161 +0,0 @@
%% 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-2019 Pivotal Software, Inc. All rights reserved.
%%
-module(rabbit_prelaunch).
-export([start/0, stop/0]).
-export([config_file_check/0]).
-import(rabbit_misc, [pget/2, pget/3]).
-include("rabbit.hrl").
-define(SET_DIST_PORT, 0).
-define(ERROR_CODE, 1).
-define(DO_NOT_SET_DIST_PORT, 2).
-define(EX_USAGE, 64).
%%----------------------------------------------------------------------------
-spec start() -> no_return().
start() ->
case init:get_plain_arguments() of
[NodeStr] ->
Node = rabbit_nodes:make(NodeStr),
{NodeName, NodeHost} = rabbit_nodes:parts(Node),
ok = duplicate_node_check(NodeName, NodeHost),
ok = dist_port_set_check(),
ok = dist_port_range_check(),
ok = dist_port_use_check(NodeHost),
ok = config_file_check();
[] ->
%% Ignore running node while installing windows service
ok = dist_port_set_check(),
ok
end,
rabbit_misc:quit(?SET_DIST_PORT),
ok.
-spec stop() -> 'ok'.
stop() ->
ok.
%%----------------------------------------------------------------------------
config_file_check() ->
case rabbit_config:validate_config_files() of
ok -> ok;
{error, {ErrFmt, ErrArgs}} ->
ErrMsg = io_lib:format(ErrFmt, ErrArgs),
{{Year, Month, Day}, {Hour, Minute, Second, Milli}} = lager_util:localtime_ms(),
io:format(standard_error, "~b-~2..0b-~2..0b ~2..0b:~2..0b:~2..0b.~b [error] ~s",
[Year, Month, Day, Hour, Minute, Second, Milli, ErrMsg]),
rabbit_misc:quit(?EX_USAGE)
end.
%% Check whether a node with the same name is already running
duplicate_node_check(NodeName, NodeHost) ->
case rabbit_nodes:names(NodeHost) of
{ok, NamePorts} ->
case proplists:is_defined(NodeName, NamePorts) of
true -> io:format(
"ERROR: node with name ~p already running on ~p~n",
[NodeName, NodeHost]),
rabbit_misc:quit(?ERROR_CODE);
false -> ok
end;
{error, EpmdReason} ->
io:format("ERROR: epmd error for host ~s: ~s~n",
[NodeHost, rabbit_misc:format_inet_error(EpmdReason)]),
rabbit_misc:quit(?ERROR_CODE)
end.
dist_port_set_check() ->
case get_config(os:getenv("RABBITMQ_CONFIG_ARG_FILE")) of
{ok, [Config]} ->
Kernel = pget(kernel, Config, []),
case {pget(inet_dist_listen_min, Kernel, none),
pget(inet_dist_listen_max, Kernel, none)} of
{none, none} -> ok;
_ -> rabbit_misc:quit(?DO_NOT_SET_DIST_PORT)
end;
{ok, _} ->
ok;
{error, _} ->
ok
end.
get_config("") -> {error, nofile};
get_config(File) ->
case consult_file(File) of
{ok, Contents} -> {ok, Contents};
{error, _} = E -> E
end.
consult_file(false) -> {error, nofile};
consult_file(File) ->
FileName = case filename:extension(File) of
"" -> File ++ ".config";
".config" -> File;
_ -> ""
end,
file:consult(FileName).
dist_port_range_check() ->
case os:getenv("RABBITMQ_DIST_PORT") of
false -> ok;
PortStr -> case catch list_to_integer(PortStr) of
Port when is_integer(Port) andalso Port > 65535 ->
rabbit_misc:quit(?DO_NOT_SET_DIST_PORT);
_ ->
ok
end
end.
dist_port_use_check(NodeHost) ->
case os:getenv("RABBITMQ_DIST_PORT") of
false -> ok;
PortStr -> Port = list_to_integer(PortStr),
dist_port_use_check_ipv4(NodeHost, Port)
end.
dist_port_use_check_ipv4(NodeHost, Port) ->
case gen_tcp:listen(Port, [inet, {reuseaddr, true}]) of
{ok, Sock} -> gen_tcp:close(Sock);
{error, einval} -> dist_port_use_check_ipv6(NodeHost, Port);
{error, _} -> dist_port_use_check_fail(Port, NodeHost)
end.
dist_port_use_check_ipv6(NodeHost, Port) ->
case gen_tcp:listen(Port, [inet6, {reuseaddr, true}]) of
{ok, Sock} -> gen_tcp:close(Sock);
{error, _} -> dist_port_use_check_fail(Port, NodeHost)
end.
-spec dist_port_use_check_fail(non_neg_integer(), string()) ->
no_return().
dist_port_use_check_fail(Port, Host) ->
{ok, Names} = rabbit_nodes:names(Host),
case [N || {N, P} <- Names, P =:= Port] of
[] -> io:format("ERROR: distribution port ~b in use on ~s "
"(by non-Erlang process?)~n", [Port, Host]);
[Name] -> io:format("ERROR: distribution port ~b in use by ~s@~s~n",
[Port, Name, Host])
end,
rabbit_misc:quit(?ERROR_CODE).

View File

@ -0,0 +1,22 @@
-module(rabbit_prelaunch_cluster).
-export([setup/1]).
setup(Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Clustering =="),
rabbit_log_prelaunch:debug("Preparing cluster status files"),
rabbit_node_monitor:prepare_cluster_status_files(),
case Context of
#{initial_pass := true} ->
rabbit_log_prelaunch:debug("Upgrading Mnesia schema"),
ok = rabbit_upgrade:maybe_upgrade_mnesia();
_ ->
ok
end,
%% It's important that the consistency check happens after
%% the upgrade, since if we are a secondary node the
%% primary node will have forgotten us
rabbit_log_prelaunch:debug("Checking cluster consistency"),
rabbit_mnesia:check_cluster_consistency(),
ok.

View File

@ -0,0 +1,534 @@
-module(rabbit_prelaunch_conf).
-include_lib("kernel/include/file.hrl").
-include_lib("stdlib/include/zip.hrl").
-include_lib("rabbit_common/include/rabbit.hrl").
-export([setup/1,
get_config_state/0,
generate_config_from_cuttlefish_files/3,
decrypt_config/1]).
-ifdef(TEST).
-export([decrypt_config/2]).
-endif.
setup(Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Configuration =="),
%% TODO: Check if directories/files are inside Mnesia dir.
%% TODO: Support glob patterns & directories in RABBITMQ_CONFIG_FILE.
%% TODO: Always try parsing of both erlang and cuttlefish formats.
update_enabled_plugins_file(Context),
set_default_config(),
AdvancedConfigFile = find_actual_advanced_config_file(Context),
State = case find_actual_main_config_file(Context) of
{MainConfigFile, erlang} ->
Config = load_erlang_term_based_config_file(
MainConfigFile),
Apps = [App || {App, _} <- Config],
decrypt_config(Apps),
#{config_type => erlang,
config_files => [MainConfigFile],
config_advanced_file => undefined};
{MainConfigFile, cuttlefish} ->
ConfigFiles = [MainConfigFile],
Config = load_cuttlefish_config_file(Context,
ConfigFiles,
AdvancedConfigFile),
Apps = [App || {App, _} <- Config],
decrypt_config(Apps),
#{config_type => cuttlefish,
config_files => ConfigFiles,
config_advanced_file => AdvancedConfigFile};
undefined when AdvancedConfigFile =/= undefined ->
rabbit_log_prelaunch:warning(
"Using RABBITMQ_ADVANCED_CONFIG_FILE: ~s",
[AdvancedConfigFile]),
Config = load_erlang_term_based_config_file(
AdvancedConfigFile),
Apps = [App || {App, _} <- Config],
decrypt_config(Apps),
#{config_type => erlang,
config_files => [AdvancedConfigFile],
config_advanced_file => AdvancedConfigFile};
undefined ->
#{config_type => undefined,
config_files => [],
config_advanced_file => undefined}
end,
override_with_hard_coded_critical_config(),
rabbit_log_prelaunch:debug(
"Saving config state to application env: ~p", [State]),
store_config_state(State).
store_config_state(ConfigState) ->
persistent_term:put({rabbitmq_prelaunch, config_state}, ConfigState).
get_config_state() ->
persistent_term:get({rabbitmq_prelaunch, config_state}, undefined).
%% -------------------------------------------------------------------
%% `enabled_plugins` file content initialization.
%% -------------------------------------------------------------------
update_enabled_plugins_file(Context) ->
%% We only do this on startup, not when the configuration is
%% reloaded.
case get_config_state() of
undefined -> update_enabled_plugins_file1(Context);
_ -> ok
end.
update_enabled_plugins_file1(#{enabled_plugins := undefined}) ->
ok;
update_enabled_plugins_file1(#{enabled_plugins := all,
plugins_path := Path} = Context) ->
List = [P#plugin.name || P <- rabbit_plugins:list(Path)],
do_update_enabled_plugins_file(Context, List);
update_enabled_plugins_file1(#{enabled_plugins := List} = Context) ->
do_update_enabled_plugins_file(Context, List).
do_update_enabled_plugins_file(#{enabled_plugins_file := File}, List) ->
SortedList = lists:usort(List),
case SortedList of
[] ->
rabbit_log_prelaunch:debug("Marking all plugins as disabled");
_ ->
rabbit_log_prelaunch:debug(
"Marking the following plugins as enabled:"),
[rabbit_log_prelaunch:debug(" - ~s", [P]) || P <- SortedList]
end,
Content = io_lib:format("~p.~n", [SortedList]),
case file:write_file(File, Content) of
ok ->
ok;
{error, Reason} ->
rabbit_log_prelaunch:error(
"Failed to update enabled plugins file \"~ts\" "
"from $RABBITMQ_ENABLED_PLUGINS: ~ts",
[File, file:format_error(Reason)]),
throw({error, failed_to_update_enabled_plugins_file})
end.
%% -------------------------------------------------------------------
%% Configuration loading.
%% -------------------------------------------------------------------
set_default_config() ->
rabbit_log_prelaunch:debug("Setting default config"),
Config = [
{ra,
[
%% Use a larger segments size for queues.
{segment_max_entries, 32768},
{wal_max_size_bytes, 536870912} %% 5 * 2 ^ 20
]},
{sysmon_handler,
[{process_limit, 100},
{port_limit, 100},
{gc_ms_limit, 0},
{schedule_ms_limit, 0},
{heap_word_limit, 0},
{busy_port, false},
{busy_dist_port, true}]}
],
apply_erlang_term_based_config(Config).
find_actual_main_config_file(#{main_config_file := File}) ->
case filelib:is_regular(File) of
true ->
Format = case filename:extension(File) of
".conf" -> cuttlefish;
".config" -> erlang;
_ -> determine_config_format(File)
end,
{File, Format};
false ->
OldFormatFile = File ++ ".config",
NewFormatFile = File ++ ".conf",
case filelib:is_regular(OldFormatFile) of
true ->
case filelib:is_regular(NewFormatFile) of
true ->
rabbit_log_prelaunch:warning(
"Both old (.config) and new (.conf) format config "
"files exist."),
rabbit_log_prelaunch:warning(
"Using the old format config file: ~s",
[OldFormatFile]),
rabbit_log_prelaunch:warning(
"Please update your config files to the new format "
"and remove the old file."),
ok;
false ->
ok
end,
{OldFormatFile, erlang};
false ->
case filelib:is_regular(NewFormatFile) of
true -> {NewFormatFile, cuttlefish};
false -> undefined
end
end
end.
find_actual_advanced_config_file(#{advanced_config_file := File}) ->
case filelib:is_regular(File) of
true -> File;
false -> undefined
end.
determine_config_format(File) ->
case filelib:file_size(File) of
0 ->
cuttlefish;
_ ->
case file:consult(File) of
{ok, _} -> erlang;
_ -> cuttlefish
end
end.
load_erlang_term_based_config_file(ConfigFile) ->
rabbit_log_prelaunch:debug(
"Loading configuration file \"~ts\" (Erlang terms based)", [ConfigFile]),
case file:consult(ConfigFile) of
{ok, [Config]} when is_list(Config) ->
apply_erlang_term_based_config(Config),
Config;
{ok, OtherTerms} ->
rabbit_log_prelaunch:error(
"Failed to load configuration file \"~ts\", "
"incorrect format: ~p",
[ConfigFile, OtherTerms]),
throw({error, failed_to_parse_configuration_file});
{error, Reason} ->
rabbit_log_prelaunch:error(
"Failed to load configuration file \"~ts\": ~ts",
[ConfigFile, file:format_error(Reason)]),
throw({error, failed_to_read_configuration_file})
end.
load_cuttlefish_config_file(Context,
ConfigFiles,
AdvancedConfigFile) ->
Config = generate_config_from_cuttlefish_files(
Context, ConfigFiles, AdvancedConfigFile),
apply_erlang_term_based_config(Config),
Config.
generate_config_from_cuttlefish_files(Context,
ConfigFiles,
AdvancedConfigFile) ->
%% Load schemas.
SchemaFiles = find_cuttlefish_schemas(Context),
case SchemaFiles of
[] ->
rabbit_log_prelaunch:error(
"No configuration schema found~n", []),
throw({error, no_configuration_schema_found});
_ ->
rabbit_log_prelaunch:debug(
"Configuration schemas found:~n", []),
[rabbit_log_prelaunch:debug(" - ~ts", [SchemaFile])
|| SchemaFile <- SchemaFiles],
ok
end,
Schema = cuttlefish_schema:files(SchemaFiles),
%% Load configuration.
rabbit_log_prelaunch:debug(
"Loading configuration files (Cuttlefish based):"),
[rabbit_log_prelaunch:debug(
" - ~ts", [ConfigFile]) || ConfigFile <- ConfigFiles],
case cuttlefish_conf:files(ConfigFiles) of
{errorlist, Errors} ->
rabbit_log_prelaunch:error("Error generating configuration:", []),
[rabbit_log_prelaunch:error(
" - ~ts",
[cuttlefish_error:xlate(Error)])
|| Error <- Errors],
throw({error, failed_to_generate_configuration_file});
Config0 ->
%% Finalize configuration, based on the schema.
Config = case cuttlefish_generator:map(Schema, Config0) of
{error, Phase, {errorlist, Errors}} ->
%% TODO
rabbit_log_prelaunch:error(
"Error generating configuration in phase ~ts:",
[Phase]),
[rabbit_log_prelaunch:error(
" - ~ts",
[cuttlefish_error:xlate(Error)])
|| Error <- Errors],
throw(
{error, failed_to_generate_configuration_file});
ValidConfig ->
proplists:delete(vm_args, ValidConfig)
end,
%% Apply advanced configuration overrides, if any.
override_with_advanced_config(Config, AdvancedConfigFile)
end.
find_cuttlefish_schemas(Context) ->
Apps = list_apps(Context),
rabbit_log_prelaunch:debug(
"Looking up configuration schemas in the following applications:"),
find_cuttlefish_schemas(Apps, []).
find_cuttlefish_schemas([App | Rest], AllSchemas) ->
Schemas = list_schemas_in_app(App),
find_cuttlefish_schemas(Rest, AllSchemas ++ Schemas);
find_cuttlefish_schemas([], AllSchemas) ->
lists:sort(fun(A,B) -> A < B end, AllSchemas).
list_apps(#{os_type := {win32, _}, plugins_path := PluginsPath}) ->
PluginsDirs = string:lexemes(PluginsPath, ";"),
list_apps1(PluginsDirs, []);
list_apps(#{plugins_path := PluginsPath}) ->
PluginsDirs = string:lexemes(PluginsPath, ":"),
list_apps1(PluginsDirs, []).
list_apps1([Dir | Rest], Apps) ->
case file:list_dir(Dir) of
{ok, Filenames} ->
NewApps = [list_to_atom(
hd(
string:split(filename:basename(F, ".ex"), "-")))
|| F <- Filenames],
Apps1 = lists:umerge(Apps, lists:sort(NewApps)),
list_apps1(Rest, Apps1);
{error, Reason} ->
rabbit_log_prelaunch:debug(
"Failed to list directory \"~ts\" content: ~ts",
[Dir, file:format_error(Reason)]),
list_apps1(Rest, Apps)
end;
list_apps1([], AppInfos) ->
AppInfos.
list_schemas_in_app(App) ->
{Loaded, Unload} = case application:load(App) of
ok -> {true, true};
{error, {already_loaded, _}} -> {true, false};
{error, _} -> {false, false}
end,
List = case Loaded of
true ->
case code:priv_dir(App) of
{error, bad_name} ->
rabbit_log_prelaunch:debug(
" [ ] ~s (no readable priv dir)", [App]),
[];
PrivDir ->
SchemaDir = filename:join([PrivDir, "schema"]),
do_list_schemas_in_app(App, SchemaDir)
end;
false ->
rabbit_log_prelaunch:debug(
" [ ] ~s (failed to load application)", [App]),
[]
end,
case Unload of
true -> application:unload(App);
false -> ok
end,
List.
do_list_schemas_in_app(App, SchemaDir) ->
case erl_prim_loader:list_dir(SchemaDir) of
{ok, Files} ->
rabbit_log_prelaunch:debug(" [x] ~s", [App]),
[filename:join(SchemaDir, File)
|| [C | _] = File <- Files,
C =/= $.];
error ->
rabbit_log_prelaunch:debug(
" [ ] ~s (no readable schema dir)", [App]),
[]
end.
override_with_advanced_config(Config, undefined) ->
Config;
override_with_advanced_config(Config, AdvancedConfigFile) ->
rabbit_log_prelaunch:debug(
"Override with advanced configuration file \"~ts\"",
[AdvancedConfigFile]),
case file:consult(AdvancedConfigFile) of
{ok, [AdvancedConfig]} ->
cuttlefish_advanced:overlay(Config, AdvancedConfig);
{ok, OtherTerms} ->
rabbit_log_prelaunch:error(
"Failed to load advanced configuration file \"~ts\", "
"incorrect format: ~p",
[AdvancedConfigFile, OtherTerms]),
throw({error, failed_to_parse_advanced_configuration_file});
{error, Reason} ->
rabbit_log_prelaunch:error(
"Failed to load advanced configuration file \"~ts\": ~ts",
[AdvancedConfigFile, file:format_error(Reason)]),
throw({error, failed_to_read_advanced_configuration_file})
end.
override_with_hard_coded_critical_config() ->
rabbit_log_prelaunch:debug("Override with hard-coded critical config"),
Config = [
{ra,
%% Make Ra use a custom logger that dispatches to lager
%% instead of the default OTP logger
[{logger_module, rabbit_log_ra_shim}]}
],
apply_erlang_term_based_config(Config).
apply_erlang_term_based_config([{_, []} | Rest]) ->
apply_erlang_term_based_config(Rest);
apply_erlang_term_based_config([{App, Vars} | Rest]) ->
rabbit_log_prelaunch:debug(" Applying configuration for '~s':", [App]),
apply_app_env_vars(App, Vars),
apply_erlang_term_based_config(Rest);
apply_erlang_term_based_config([]) ->
ok.
apply_app_env_vars(App, [{Var, Value} | Rest]) ->
rabbit_log_prelaunch:debug(
" - ~s = ~p",
[Var, Value]),
ok = application:set_env(App, Var, Value, [{persistent, true}]),
apply_app_env_vars(App, Rest);
apply_app_env_vars(_, []) ->
ok.
%% -------------------------------------------------------------------
%% Config decryption.
%% -------------------------------------------------------------------
decrypt_config(Apps) ->
rabbit_log_prelaunch:debug("Decoding encrypted config values (if any)"),
ConfigEntryDecoder = application:get_env(rabbit, config_entry_decoder, []),
decrypt_config(Apps, ConfigEntryDecoder).
decrypt_config([], _) ->
ok;
decrypt_config([App | Apps], Algo) ->
Algo1 = decrypt_app(App, application:get_all_env(App), Algo),
decrypt_config(Apps, Algo1).
decrypt_app(_, [], Algo) ->
Algo;
decrypt_app(App, [{Key, Value} | Tail], Algo) ->
Algo2 = try
case decrypt(Value, Algo) of
{Value, Algo1} ->
Algo1;
{NewValue, Algo1} ->
rabbit_log_prelaunch:debug(
"Value of `~s` decrypted", [Key]),
ok = application:set_env(App, Key, NewValue,
[{persistent, true}]),
Algo1
end
catch
throw:{bad_config_entry_decoder, _} = Error ->
throw(Error);
_:Msg ->
throw({config_decryption_error, {key, Key}, Msg})
end,
decrypt_app(App, Tail, Algo2).
decrypt({encrypted, EncValue},
{Cipher, Hash, Iterations, PassPhrase} = Algo) ->
{rabbit_pbe:decrypt_term(Cipher, Hash, Iterations, PassPhrase, EncValue),
Algo};
decrypt({encrypted, _} = Value,
ConfigEntryDecoder)
when is_list(ConfigEntryDecoder) ->
Algo = config_entry_decoder_to_algo(ConfigEntryDecoder),
decrypt(Value, Algo);
decrypt(List, Algo) when is_list(List) ->
decrypt_list(List, Algo, []);
decrypt(Value, Algo) ->
{Value, Algo}.
%% We make no distinction between strings and other lists.
%% When we receive a string, we loop through each element
%% and ultimately return the string unmodified, as intended.
decrypt_list([], Algo, Acc) ->
{lists:reverse(Acc), Algo};
decrypt_list([{Key, Value} | Tail], Algo, Acc)
when Key =/= encrypted ->
{Value1, Algo1} = decrypt(Value, Algo),
decrypt_list(Tail, Algo1, [{Key, Value1} | Acc]);
decrypt_list([Value | Tail], Algo, Acc) ->
{Value1, Algo1} = decrypt(Value, Algo),
decrypt_list(Tail, Algo1, [Value1 | Acc]).
config_entry_decoder_to_algo(ConfigEntryDecoder) ->
case get_passphrase(ConfigEntryDecoder) of
undefined ->
throw({bad_config_entry_decoder, missing_passphrase});
PassPhrase ->
{
proplists:get_value(
cipher, ConfigEntryDecoder, rabbit_pbe:default_cipher()),
proplists:get_value(
hash, ConfigEntryDecoder, rabbit_pbe:default_hash()),
proplists:get_value(
iterations, ConfigEntryDecoder, rabbit_pbe:default_iterations()),
PassPhrase
}
end.
get_passphrase(ConfigEntryDecoder) ->
rabbit_log_prelaunch:debug("Getting encrypted config passphrase"),
case proplists:get_value(passphrase, ConfigEntryDecoder) of
prompt ->
IoDevice = get_input_iodevice(),
io:setopts(IoDevice, [{echo, false}]),
PP = lists:droplast(io:get_line(IoDevice,
"\nPlease enter the passphrase to unlock encrypted "
"configuration entries.\n\nPassphrase: ")),
io:setopts(IoDevice, [{echo, true}]),
io:format(IoDevice, "~n", []),
PP;
{file, Filename} ->
{ok, File} = file:read_file(Filename),
[PP|_] = binary:split(File, [<<"\r\n">>, <<"\n">>]),
PP;
PP ->
PP
end.
%% This function retrieves the correct IoDevice for requesting
%% input. The problem with using the default IoDevice is that
%% the Erlang shell prevents us from getting the input.
%%
%% Instead we therefore look for the io process used by the
%% shell and if it can't be found (because the shell is not
%% started e.g with -noshell) we use the 'user' process.
%%
%% This function will not work when either -oldshell or -noinput
%% options are passed to erl.
get_input_iodevice() ->
case whereis(user) of
undefined ->
user;
User ->
case group:interfaces(User) of
[] ->
user;
[{user_drv, Drv}] ->
case user_drv:interfaces(Drv) of
[] -> user;
[{current_group, IoDevice}] -> IoDevice
end
end
end.

View File

@ -0,0 +1,25 @@
-module(rabbit_prelaunch_feature_flags).
-export([setup/1]).
setup(#{feature_flags_file := FFFile}) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Feature flags =="),
case filelib:ensure_dir(FFFile) of
ok ->
rabbit_log_prelaunch:debug("Initializing feature flags registry"),
case rabbit_feature_flags:initialize_registry() of
ok ->
ok;
{error, Reason} ->
rabbit_log_prelaunch:error(
"Failed to initialize feature flags registry: ~p",
[Reason]),
throw({error, failed_to_initialize_feature_flags_registry})
end;
{error, Reason} ->
rabbit_log_prelaunch:error(
"Failed to create feature flags file \"~ts\" directory: ~ts",
[FFFile, file:format_error(Reason)]),
throw({error, failed_to_create_feature_flags_file_directory})
end.

View File

@ -0,0 +1,10 @@
-module(rabbit_prelaunch_hipe).
-export([setup/1]).
setup(_Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== HiPE compitation =="),
HipeResult = rabbit_hipe:maybe_hipe_compile(),
rabbit_hipe:log_hipe_result(HipeResult),
ok.

View File

@ -0,0 +1,68 @@
-module(rabbit_prelaunch_logging).
-export([setup/1]).
setup(Context) ->
rabbit_log_prelaunch:debug(""),
rabbit_log_prelaunch:debug("== Logging =="),
ok = set_ERL_CRASH_DUMP_envvar(Context),
ok = configure_lager(Context).
set_ERL_CRASH_DUMP_envvar(#{log_base_dir := LogBaseDir}) ->
case os:getenv("ERL_CRASH_DUMP") of
false ->
ErlCrashDump = filename:join(LogBaseDir, "erl_crash.dump"),
rabbit_log_prelaunch:debug(
"Setting $ERL_CRASH_DUMP environment variable to \"~ts\"",
[ErlCrashDump]),
os:putenv("ERL_CRASH_DUMP", ErlCrashDump),
ok;
ErlCrashDump ->
rabbit_log_prelaunch:debug(
"$ERL_CRASH_DUMP environment variable already set to \"~ts\"",
[ErlCrashDump]),
ok
end.
configure_lager(#{log_base_dir := LogBaseDir,
main_log_file := MainLog,
upgrade_log_file := UpgradeLog} = Context) ->
{SaslErrorLogger,
MainLagerHandler,
UpgradeLagerHandler} = case MainLog of
"-" ->
%% Log to STDOUT.
rabbit_log_prelaunch:debug(
"Logging to stdout"),
{tty,
tty,
tty};
_ ->
rabbit_log_prelaunch:debug(
"Logging to:"),
[rabbit_log_prelaunch:debug(
" - ~ts", [Log])
|| Log <- [MainLog, UpgradeLog]],
%% Log to file.
{false,
MainLog,
UpgradeLog}
end,
ok = application:set_env(lager, crash_log, "log/crash.log"),
Fun = fun({App, Var, Value}) ->
case application:get_env(App, Var) of
undefined -> ok = application:set_env(App, Var, Value);
_ -> ok
end
end,
Vars = [{sasl, sasl_error_logger, SaslErrorLogger},
{rabbit, lager_log_root, LogBaseDir},
{rabbit, lager_default_file, MainLagerHandler},
{rabbit, lager_upgrade_file, UpgradeLagerHandler}],
lists:foreach(Fun, Vars),
ok = rabbit_lager:start_logger(),
ok = rabbit_prelaunch_early_logging:setup_early_logging(Context, false).

View File

@ -106,9 +106,11 @@ wait(TableNames, Timeout, Retries) ->
ok ->
ok;
{timeout, BadTabs} ->
{error, {timeout_waiting_for_tables, BadTabs}};
AllNodes = rabbit_mnesia:cluster_nodes(all),
{error, {timeout_waiting_for_tables, AllNodes, BadTabs}};
{error, Reason} ->
{error, {failed_waiting_for_tables, Reason}}
AllNodes = rabbit_mnesia:cluster_nodes(all),
{error, {failed_waiting_for_tables, AllNodes, Reason}}
end,
case {Retries, Result} of
{_, ok} ->

View File

@ -224,8 +224,7 @@ join_cluster_bad_operations(Config) ->
ok = stop_app(Hare),
assert_failure(fun () -> start_app(Hare) end),
ok = start_app(Rabbit),
%% The Erlang VM has stopped after previous rabbit app failure
ok = rabbit_ct_broker_helpers:start_node(Config, Hare),
ok = start_app(Hare),
ok.
%% This tests that the nodes in the cluster are notified immediately of a node

View File

@ -136,14 +136,18 @@ init_per_group(clustering, Config) ->
[{rmq_nodes_count, 2},
{rmq_nodes_clustered, false},
{start_rmq_with_plugins_disabled, true}]),
build_my_plugin(Config1);
rabbit_ct_helpers:run_setup_steps(Config1, [
fun build_my_plugin/1
]);
init_per_group(activating_plugin, Config) ->
Config1 = rabbit_ct_helpers:set_config(
Config,
[{rmq_nodes_count, 2},
{rmq_nodes_clustered, true},
{start_rmq_with_plugins_disabled, true}]),
build_my_plugin(Config1);
rabbit_ct_helpers:run_setup_steps(Config1, [
fun build_my_plugin/1
]);
init_per_group(_, Config) ->
Config.
@ -910,13 +914,31 @@ build_my_plugin(Config) ->
{ok, _} ->
{_, OtherPlugins1} = list_my_plugin_plugins(PluginSrcDir),
remove_other_plugins(PluginSrcDir, OtherPlugins1),
Config1;
update_cli_path(Config1, PluginSrcDir);
{error, _} ->
{skip, "Failed to compile the `my_plugin` test plugin"}
end;
_ ->
remove_other_plugins(PluginSrcDir, OtherPlugins),
Config1
update_cli_path(Config1, PluginSrcDir)
end.
update_cli_path(Config, PluginSrcDir) ->
SbinDir = filename:join(PluginSrcDir, "sbin"),
Rabbitmqctl = filename:join(SbinDir, "rabbitmqctl"),
RabbitmqPlugins = filename:join(SbinDir, "rabbitmq-plugins"),
RabbitmqQueues = filename:join(SbinDir, "rabbitmq-queues"),
case filelib:is_regular(Rabbitmqctl) of
true ->
ct:pal(?LOW_IMPORTANCE,
"Switching to CLI in e.g. ~s", [Rabbitmqctl]),
rabbit_ct_helpers:set_config(
Config,
[{rabbitmqctl_cmd, Rabbitmqctl},
{rabbitmq_plugins_cmd, RabbitmqPlugins},
{rabbitmq_queues_cmd, RabbitmqQueues}]);
false ->
Config
end.
list_my_plugin_plugins(PluginSrcDir) ->

View File

@ -44,8 +44,6 @@ groups() ->
]},
content_framing,
content_transcoding,
decrypt_config,
listing_plugins_from_multiple_directories,
rabbitmqctl_encode,
pmerge,
plmerge,
@ -72,7 +70,9 @@ groups() ->
decrypt_start_app,
decrypt_start_app_file,
decrypt_start_app_undefined,
decrypt_start_app_wrong_passphrase
decrypt_start_app_wrong_passphrase,
decrypt_config,
listing_plugins_from_multiple_directories
]}
].
@ -81,20 +81,13 @@ end_per_group(_, Config) -> Config.
init_per_testcase(TC, Config) when TC =:= decrypt_start_app;
TC =:= decrypt_start_app_file;
TC =:= decrypt_start_app_undefined ->
application:load(rabbit),
application:set_env(rabbit, feature_flags_file, ""),
TC =:= decrypt_start_app_undefined;
TC =:= decrypt_start_app_wrong_passphrase ->
application:set_env(rabbit, feature_flags_file, "", [{persistent, true}]),
Config;
init_per_testcase(_Testcase, Config) ->
Config.
end_per_testcase(TC, _Config) when TC =:= decrypt_start_app;
TC =:= decrypt_start_app_file;
TC =:= decrypt_start_app_undefined ->
application:unload(rabbit),
application:unload(rabbit_shovel_test);
end_per_testcase(decrypt_config, _Config) ->
application:unload(rabbit);
end_per_testcase(_TC, _Config) ->
ok.
@ -177,7 +170,7 @@ decrypt_config(_Config) ->
ok.
do_decrypt_config(Algo = {C, H, I, P}) ->
application:load(rabbit),
ok = application:load(rabbit),
RabbitConfig = application:get_all_env(rabbit),
%% Encrypt a few values in configuration.
%% Common cases.
@ -205,10 +198,10 @@ do_decrypt_config(Algo = {C, H, I, P}) ->
TCPOpts2 = lists:keyreplace(linger, 1, TCPOpts1, {linger, {encrypted, EncLinger}}),
application:set_env(rabbit, tcp_listen_options, TCPOpts2),
%% Decrypt configuration.
rabbit:decrypt_config([rabbit], Algo),
rabbit_prelaunch_conf:decrypt_config([rabbit], Algo),
%% Check that configuration was decrypted properly.
RabbitConfig = application:get_all_env(rabbit),
application:unload(rabbit),
ok = application:unload(rabbit),
ok.
encrypt_value(Key, {C, H, I, P}) ->
@ -229,7 +222,7 @@ do_decrypt_start_app(Config, Passphrase) ->
{hash, sha512},
{iterations, 1000},
{passphrase, Passphrase}
]),
], [{persistent, true}]),
%% Add the path to our test application.
code:add_path(?config(data_dir, Config) ++ "/lib/rabbit_shovel_test/ebin"),
%% Attempt to start our test application.
@ -256,7 +249,7 @@ decrypt_start_app_undefined(Config) ->
{hash, sha512},
{iterations, 1000}
%% No passphrase option!
]),
], [{persistent, true}]),
%% Add the path to our test application.
code:add_path(?config(data_dir, Config) ++ "/lib/rabbit_shovel_test/ebin"),
%% Attempt to start our test application.
@ -265,7 +258,7 @@ decrypt_start_app_undefined(Config) ->
try
rabbit:start_apps([rabbit_shovel_test], #{rabbit => temporary})
catch
exit:{bad_configuration, config_entry_decoder} -> ok;
throw:{bad_config_entry_decoder, missing_passphrase} -> ok;
_:Exception -> exit({unexpected_exception, Exception})
end.
@ -276,7 +269,7 @@ decrypt_start_app_wrong_passphrase(Config) ->
{hash, sha512},
{iterations, 1000},
{passphrase, "wrong passphrase"}
]),
], [{persistent, true}]),
%% Add the path to our test application.
code:add_path(?config(data_dir, Config) ++ "/lib/rabbit_shovel_test/ebin"),
%% Attempt to start our test application.
@ -285,7 +278,7 @@ decrypt_start_app_wrong_passphrase(Config) ->
try
rabbit:start_apps([rabbit_shovel_test], #{rabbit => temporary})
catch
exit:{decryption_error,_,_} -> ok;
throw:{config_decryption_error, _, _} -> ok;
_:Exception -> exit({unexpected_exception, Exception})
end.
@ -961,6 +954,10 @@ listing_plugins_from_multiple_directories(Config) ->
end,
Path = FirstDir ++ PathSep ++ SecondDir,
Got = lists:sort([{Name, Vsn} || #plugin{name = Name, version = Vsn} <- rabbit_plugins:list(Path)]),
%% `rabbit` was loaded automatically by `rabbit_plugins:list/1`.
%% We want to unload it now so it does not interfere with other
%% testcases.
application:unload(rabbit),
Expected = [{plugin_both, "2"}, {plugin_first_dir, "3"}, {plugin_second_dir, "4"}],
case Got of
Expected ->

View File

@ -503,7 +503,7 @@ log_file_fails_to_initialise_during_startup1(_Config, NonWritableDir) ->
ok -> exit({got_success_but_expected_failure,
log_rotation_no_write_permission_dir_test})
catch
_:could_not_initialise_logger -> ok
throw:{error, {rabbit, {{cannot_log_to_file, _, _}, _}}} -> ok
end,
%% start application with logging to a subdirectory which
@ -530,7 +530,7 @@ log_file_fails_to_initialise_during_startup1(_Config, NonWritableDir) ->
ok -> exit({got_success_but_expected_failure,
log_rotation_parent_dirs_test})
catch
_:could_not_initialise_logger -> ok
throw:{error, {rabbit, {{cannot_log_to_file, _, _}, _}}} -> ok
end,
%% clean up

View File

@ -132,6 +132,9 @@ sink_rewrite_sinks() ->
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
@ -225,6 +228,9 @@ sink_handlers_merged_with_lager_extra_sinks_handlers(_) ->
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
@ -285,7 +291,7 @@ config_sinks_level(_) ->
rabbit_lager:configure_lager(),
ExpectedSinks = sort_sinks(level_sinks()),
ExpectedSinks = sort_sinks(application:get_env(lager, extra_sinks, undefined)).
?assertEqual(ExpectedSinks, sort_sinks(application:get_env(lager, extra_sinks, undefined))).
level_sinks() ->
[{error_logger_lager_event,
@ -310,6 +316,9 @@ level_sinks() ->
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,error]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,error]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
@ -409,6 +418,9 @@ file_sinks() ->
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
@ -650,6 +662,9 @@ default_expected_sinks(UpgradeFile) ->
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
@ -722,6 +737,9 @@ tty_expected_sinks() ->
{rabbit_log_mirroring_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_prelaunch_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},
{rabbit_log_queue_lager_event,
[{handlers,[{lager_forwarder_backend,[lager_event,inherit]}]},
{rabbit_handlers,[{lager_forwarder_backend,[lager_event,inherit]}]}]},