Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
180 commits
Select commit Hold shift + click to select a range
9ff9afd
Test queue.declare method with quorum type
dcorbacho Jan 23, 2018
5e9e1d5
Cosmetics
dcorbacho Jan 24, 2018
740397d
Merge pull request #1480 from rabbitmq/rabbitmq-server-story-154472130
dcorbacho Jan 24, 2018
c088ee1
Start quorum queue
dcorbacho Jan 23, 2018
37d70cf
Update info and list operations to use quorum queues
dcorbacho Jan 25, 2018
64c5670
Stop quorum queue
dcorbacho Jan 26, 2018
d142ce1
Restart quorum queue
dcorbacho Jan 30, 2018
db69aca
Introduce UId in ra config to support newer version of ra
dcorbacho Jan 30, 2018
a02443e
Put data inside VHost specific subdirs
dcorbacho Jan 30, 2018
5df8283
Include ra in rabbit deps to support stop_app/start_app command
dcorbacho Jan 31, 2018
ab02485
Stop quorum queues in `rabbit_amqqueue:stop/1`
dcorbacho Jan 31, 2018
f4aab4c
Revert creation of fifo ets table inside rabbit
dcorbacho Jan 31, 2018
e6f4bab
Filter quorum queues
dcorbacho Jan 31, 2018
97fb029
Test restart node with quorum queues
dcorbacho Jan 31, 2018
a27f601
Publish to quorum queues
dcorbacho Feb 1, 2018
a36366d
Use `ra:restart_node/1`
dcorbacho Feb 1, 2018
9b53bfe
Wait for stats to be published when querying quorum queues
dcorbacho Feb 2, 2018
88139c4
Test publish and queue length after restart
dcorbacho Feb 2, 2018
faec336
Consume messages from quorum queues with basic.get
dcorbacho Feb 5, 2018
74346ac
Autoack messages from quorum queues on basic.get
dcorbacho Feb 6, 2018
a59f6bc
Fix no_ack meaning
dcorbacho Feb 6, 2018
063d3bb
Use data_dir as provided in the config
dcorbacho Feb 6, 2018
078d457
Remove unused code/variables
dcorbacho Feb 6, 2018
c272d87
Subscribe to a quorum queue
dcorbacho Feb 6, 2018
8c59478
Ack messages consumed from quorum queues
dcorbacho Feb 8, 2018
672676c
Nack messages consumed from quorum queues
dcorbacho Feb 8, 2018
45a7bac
Use delivery tag as consumer tag for basic.get in quorum queues
dcorbacho Feb 9, 2018
c62eed0
Support for publisher confirms in quorum queues
dcorbacho Feb 9, 2018
494c148
Integrate with ra_fifo_client
dcorbacho Feb 16, 2018
4fd4528
Clear queue state on queue.delete
dcorbacho Feb 19, 2018
dafad5f
Fix quorum nack
dcorbacho Feb 20, 2018
f340cee
Test redelivery after nack
dcorbacho Feb 20, 2018
9ae419c
Nack without requeueing
dcorbacho Feb 20, 2018
95e6e61
Test multiple acks
dcorbacho Feb 20, 2018
8838854
Test multiple nacks
dcorbacho Feb 20, 2018
64db6c0
Configure dead letter exchange with queue declare
dcorbacho Feb 22, 2018
56d3b92
Use a per-vhost process to handle dead-lettering
dcorbacho Feb 23, 2018
514c2b6
Implement dead-lettering on nack'ed messages
dcorbacho Feb 23, 2018
4ac8f54
Use queue name as a resource on message delivery
dcorbacho Feb 26, 2018
323169d
Handle ra events on dead letter process
dcorbacho Feb 26, 2018
e664418
Pass empty queue states to queue delete
dcorbacho Feb 26, 2018
dccda71
Generate quorum queue stats and events
dcorbacho Feb 28, 2018
448c934
Ensure quorum mapping entries are available before metric emission
dcorbacho Feb 28, 2018
f226121
Configure data_dir, uses new RABBITMQ_QUORUM_BASE env var
dcorbacho Mar 1, 2018
46006a3
Use untracked enqueues when sending wihtout channel
dcorbacho Mar 2, 2018
1718286
Revert "Configure data_dir, uses new RABBITMQ_QUORUM_BASE env var"
dcorbacho Mar 2, 2018
2cd935c
Configure data_dir, uses new RABBITMQ_QUORUM_DIR based on mnesia dir
dcorbacho Mar 2, 2018
2064eb3
Fix get_quorum_state
dcorbacho Mar 2, 2018
43f6010
Fix calculation of quorum pids
dcorbacho Mar 5, 2018
d8a01b7
Move all quorum queues code to its own module
dcorbacho Mar 6, 2018
79f1407
Return an error when declaring a quorum queue with an incompatible ar…
dcorbacho Mar 6, 2018
3a9918a
Cleanup of quorum queue state after queue delete
dcorbacho Mar 13, 2018
61035ce
Merge branch 'master' into quorum-queue
hairyhum Mar 15, 2018
f79fe98
Revert Revert "Declare a quorum queue using the queue.declare method"
hairyhum Mar 19, 2018
88bbab9
Remove duplicated state info
dcorbacho Mar 20, 2018
cac9583
Start/stop multi-node quorum queue
dcorbacho Mar 20, 2018
619455f
Restart nodes in a multi-node quorum cluster
dcorbacho Mar 21, 2018
b7e17e6
Test restart and leadership takeover on multiple nodes
dcorbacho Mar 21, 2018
7aa683b
Wait for leader down after deleting a quorum cluster
dcorbacho Mar 22, 2018
46bc3cc
Populate quorum_mapping from mnesia when not available
dcorbacho Mar 22, 2018
4af36bd
Cosmetics
dcorbacho Mar 22, 2018
91367f1
Do not emit core metrics if queue has just been deleted
dcorbacho Mar 26, 2018
46da265
Use rabbit_mnesia:is_process_alive
dcorbacho Mar 26, 2018
e55928a
Only try to report stats if quorum process is alive
dcorbacho Mar 26, 2018
e5efe58
Implement cancel consumer callback
dcorbacho Mar 27, 2018
15a4d81
Remove unnecessary trigger election call
dcorbacho Apr 11, 2018
12e49f9
Apply cancellation callback on node hosting the channel
dcorbacho Apr 11, 2018
0f801e9
Cosmetics
dcorbacho Apr 11, 2018
55be48e
Read new fifo metrics which store directly total, ready and unack
dcorbacho Apr 17, 2018
b597328
Implement basic.cancel for quorum queues
dcorbacho Apr 17, 2018
80ecdd7
Store leader in amqqueue record, report all in stats
dcorbacho Apr 19, 2018
6195c33
Declare quorum queue in mnesia before starting the ra cluster
dcorbacho Apr 24, 2018
13f7f3e
Revert
dcorbacho Apr 24, 2018
003a2f0
Purge quorum queues
dcorbacho Apr 24, 2018
8327cbf
Improve use of untracked_enqueue
kjnilsson Apr 24, 2018
2e816ff
Store quorum leader in the pid field of amqqueue record
dcorbacho Apr 27, 2018
2f8e6b1
Improve recovery
kjnilsson Apr 27, 2018
87b24cd
Cleanup core metrics after leader change
dcorbacho Apr 30, 2018
406529b
Merge remote-tracking branch 'origin/quorum-queue' into quorum-queue
dcorbacho Apr 30, 2018
78c7280
Return an error on sync_queue on quorum queues
dcorbacho Apr 30, 2018
8c20887
Return an error on cancel_sync_queue on quorum queues
dcorbacho Apr 30, 2018
d82bb9a
merge
kjnilsson Apr 30, 2018
21051eb
Fix basic_cancel and basic_consume return values
kjnilsson Apr 30, 2018
3d37c78
Restore arity of amqqeueu delete and purge functions.
kjnilsson May 1, 2018
05db4b3
Fix bug returning consumers.
kjnilsson May 1, 2018
d8288cb
remove rogue debug log
kjnilsson May 1, 2018
4a777a7
Integrate ingress flow control with quorum queues
dcorbacho May 2, 2018
0e1b5f2
Configure commands soft limit
dcorbacho May 3, 2018
61efed3
Support quorum pids on rabbit_mnesia:is_process_alive
dcorbacho May 3, 2018
9b4fb2b
Publish consumers metric for quorum queues
dcorbacho May 3, 2018
017d3a7
Whitelist quorum directory in is_virgin_node
kjnilsson May 4, 2018
ecb7505
Delete queue_metrics on leader change.
kjnilsson May 9, 2018
bbdece8
Report cluster status in quorum queue infos. New per node status comm…
dcorbacho May 11, 2018
4337005
Merge remote-tracking branch 'origin/quorum-queue' into quorum-queue
dcorbacho May 11, 2018
2af3b6f
Remove quorum_mapping table
kjnilsson May 11, 2018
db707f0
Fix xref issue
kjnilsson May 11, 2018
5c13645
Provide quorum members information in stats
dcorbacho May 17, 2018
cca778f
fix unused variable
kjnilsson May 17, 2018
79c9bd2
quorum queue multiple declare handling
kjnilsson May 17, 2018
a4b9e67
Return an error when declaring exclusive/auto-delete quorum queue
dcorbacho May 17, 2018
9578070
Merge remote-tracking branch 'origin/quorum-queue' into quorum-queue
dcorbacho May 17, 2018
d78ec96
Restore lost changes
kjnilsson May 18, 2018
e604e33
recover another part of commit
kjnilsson May 18, 2018
7296bf3
fixup cherry pick
kjnilsson May 17, 2018
e08ee11
merge
kjnilsson Jun 4, 2018
05d15c7
Ra io/file metrics handler and stats publishing
dcorbacho Jun 6, 2018
5923010
Revert "Ra io/file metrics handler and stats publishing"
dcorbacho Jun 7, 2018
06279c4
Merge remote-tracking branch 'origin/master' into quorum-queue
kjnilsson Jun 8, 2018
eaebab9
Do not issue confirms on node down for quorum queues.
kjnilsson Jun 11, 2018
df6182c
Ra stats publishing
dcorbacho Jun 14, 2018
bde9214
Merge branch 'file-io-metrics' into quorum-queue
dcorbacho Jun 14, 2018
86e8a2d
Pick consumer utilisation from ra data
dcorbacho Jun 25, 2018
5a122e5
Handle error when deleting a quorum queue and all nodes are already down
dcorbacho Jun 27, 2018
9c8f46d
Return an error when declaring non-durable quorum queues
dcorbacho Jun 27, 2018
05e2ec8
Rename dirty_query to committed_query
dcorbacho Jun 27, 2018
726095f
Delete stats on leader node
dcorbacho Jun 27, 2018
ad921ff
Give full list of nodes to fifo client
dcorbacho Jun 28, 2018
b512311
Handle timeout in quorum basic_get
dcorbacho Jun 29, 2018
98b5dfe
Fix unused variable error
kjnilsson Jul 2, 2018
7865807
Handle timeout in basic get
dcorbacho Jul 3, 2018
5c98bf2
Force GC after purge
dcorbacho Jul 3, 2018
3e00fa8
Merge remote-tracking branch 'origin/master' into quorum-queue
kjnilsson Jul 4, 2018
19323cf
Increase `ra:delete_cluster` timeout to 120s
dcorbacho Jul 10, 2018
0fe9675
Revert "Force GC after purge"
dcorbacho Jul 10, 2018
d12f725
Add quorum member command
dcorbacho Jul 17, 2018
db4a9a7
Delete quorum member command
dcorbacho Jul 18, 2018
128edb6
Implement basic.recover for quorum queues
dcorbacho Jul 18, 2018
4ae8936
Change concumer utilisation
kjnilsson Jul 26, 2018
d34f6b3
Set max quorum queue size limit
dcorbacho Aug 3, 2018
73cd60d
Merge remote-tracking branch 'origin/master' into quorum-queue
kjnilsson Aug 6, 2018
f020aed
Merge branch 'quorum-queue' of github.com:rabbitmq/rabbitmq-server in…
kjnilsson Aug 6, 2018
fb14d40
remove potentially unrelated changes to rabbit_networking
kjnilsson Aug 7, 2018
a654c9d
Merge remote-tracking branch 'origin/master' into quorum-queue
kjnilsson Aug 29, 2018
d804657
Move ra_fifo to rabbit
kjnilsson Aug 31, 2018
08600e2
rabbit_fifo tidy up
kjnilsson Aug 31, 2018
f7e4ac1
rabbit_fifo tidy up
kjnilsson Aug 31, 2018
0fd11ce
rabbit_fifo: customer -> consumer rename
kjnilsson Aug 31, 2018
4bd9b84
Move ra_fifo tests
kjnilsson Aug 31, 2018
d33e49d
Merge remote-tracking branch 'origin/master' into quorum-queue
kjnilsson Sep 4, 2018
cf97adc
Tweak quorum_queue defaults
kjnilsson Sep 11, 2018
e5921e1
Merge remote-tracking branch 'origin/master' into quorum-queue
kjnilsson Sep 13, 2018
e285766
quorum_queue test reliability
kjnilsson Sep 14, 2018
41a6fb1
Optimise quorum_queue test suite.
kjnilsson Sep 19, 2018
9f3c5a3
Renamings in line with ra API changes
kjnilsson Sep 20, 2018
e2a903d
Merge remote-tracking branch 'origin/master' into quorum-queue
kjnilsson Sep 21, 2018
8392d88
rabbit_fifo fixes
kjnilsson Sep 21, 2018
c873135
Update with ra API changes
kjnilsson Sep 28, 2018
da424dd
Merge branch 'master' into quorum-queue
lukebakken Sep 28, 2018
b70c966
Update rabbit_fifo with latest ra changes
kjnilsson Oct 2, 2018
094cd58
Clean up out of date comment
dcorbacho Oct 10, 2018
96b7d44
Return map of states
dcorbacho Oct 10, 2018
81ae369
Add test case for basic.get on an empty queue
dcorbacho Oct 10, 2018
f10ec8d
Clarify use of deliver tags on record_sent
dcorbacho Oct 11, 2018
b14a11a
Clean up queues after testcase
dcorbacho Oct 12, 2018
4de27b6
Remove erlang monitor of quorum queues in rabbit_channel
dcorbacho Oct 12, 2018
80c8583
Use macros to make clearer distinctions between quorum/classic queues
dcorbacho Oct 12, 2018
c75bdfb
Erase queue stats on 'eol' event
dcorbacho Oct 12, 2018
65a63dc
Update to follow Ra's cluster_id -> cluster_name rename.
kjnilsson Oct 15, 2018
1edf874
Rename qourum-cluster-size
kjnilsson Oct 16, 2018
3ce235a
Issue confirms on quorum queue eol
kjnilsson Oct 16, 2018
da9b6f7
Only classic queues should be notified on channel down
kjnilsson Oct 16, 2018
0978ea0
Quorum queues do not support global qos
kjnilsson Oct 17, 2018
77634a6
merge
kjnilsson Oct 17, 2018
cac162e
unused variable name
kjnilsson Oct 17, 2018
c8faea1
Refactoring
kjnilsson Oct 18, 2018
f3c9b4c
Refactor foreach_per_queue in the channel.
kjnilsson Oct 18, 2018
5dbffbd
rename function
kjnilsson Oct 18, 2018
5688b23
Query classic and quorum queues separately
kjnilsson Oct 19, 2018
73a22ea
Remove force_event_refresh function
kjnilsson Oct 19, 2018
9e927ab
fix errors
kjnilsson Oct 19, 2018
35a4066
Remove created_at from amqqueue record
kjnilsson Oct 19, 2018
0ed703e
rabbit_fifo: support AMQP 1.0 consumer credit
kjnilsson Oct 19, 2018
dfcc81e
Add basic.credit support for quorum queues.
kjnilsson Oct 24, 2018
2494b7e
Make quorum queue recover idempotent
kjnilsson Oct 25, 2018
06bb572
Add tests for vhost deletion
kjnilsson Oct 25, 2018
d741a8e
remove unused clause
kjnilsson Oct 26, 2018
2b34bdc
merge
kjnilsson Oct 26, 2018
5a826df
always return latest value of queue
kjnilsson Oct 26, 2018
283d774
Add rabbitmq-queues scripts. Remove ra config from .bat scripts.
hairyhum Oct 26, 2018
75bd026
Return error if trying to get quorum status of a classic queue.
hairyhum Oct 26, 2018
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 5 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,8 @@ define PROJECT_ENV
%% see rabbitmq-server#143,
%% rabbitmq-server#949, rabbitmq-server#1098
{credit_flow_default_credit, {400, 200}},
{quorum_commands_soft_limit, 256},
{quorum_cluster_size, 5},
%% see rabbitmq-server#248
%% and rabbitmq-server#667
{channel_operation_timeout, 15000},
Expand Down Expand Up @@ -127,13 +129,14 @@ define PROJECT_ENV
%% vhost had to shut down, see server#1158 and server#1280
{vhost_restart_strategy, continue},
%% {global, prefetch count}
{default_consumer_prefetch, {false, 0}}
{default_consumer_prefetch, {false, 0}},
{channel_queue_cleanup_interval, 60000}
]
endef

LOCAL_DEPS = sasl mnesia os_mon inets
BUILD_DEPS = rabbitmq_cli syslog
DEPS = ranch lager rabbit_common
DEPS = ranch syslog lager rabbit_common ra
TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper

dep_syslog = git https://github.com/schlagert/syslog 3.4.5
Expand Down
8 changes: 7 additions & 1 deletion scripts/rabbitmq-env
Original file line number Diff line number Diff line change
Expand Up @@ -245,6 +245,7 @@ DEFAULT_NODE_PORT=5672
[ "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}
Expand All @@ -255,7 +256,8 @@ rmq_normalize_path_var \
RABBITMQ_CONFIG_FILE \
RABBITMQ_LOG_BASE \
RABBITMQ_MNESIA_BASE \
RABBITMQ_MNESIA_DIR
RABBITMQ_MNESIA_DIR \
RABBITMQ_QUORUM_DIR

[ "x" = "x$RABBITMQ_PID_FILE" ] && RABBITMQ_PID_FILE="$PID_FILE"

Expand Down Expand Up @@ -349,6 +351,10 @@ if [ "${RABBITMQ_DEV_ENV}" ]; then
"$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"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we move this setting to erlang code? We don't start applications on node boot, but from rabbit:boot/start functions. It would be nice to avoid additional bash scripting.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it was done here for a reason, I'm trying to remember why.

fi
fi

if path_contains_existing_directory "${RABBITMQ_PLUGINS_DIR}" ; then
Expand Down
32 changes: 32 additions & 0 deletions scripts/rabbitmq-queues
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
#!/bin/sh
## 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 http://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-2017 Pivotal Software, Inc. All rights reserved.
##

# Exit immediately if a pipeline, which may consist of a single simple command,
# a list, or a compound command returns a non-zero status
set -e

# Each variable or function that is created or modified is given the export
# attribute and marked for export to the environment of subsequent commands.
set -a

# shellcheck source=/dev/null
#
# TODO: when shellcheck adds support for relative paths, change to
# shellcheck source=./rabbitmq-env
. "${0%/*}"/rabbitmq-env

run_escript rabbitmqctl_escript "${ESCRIPT_DIR:?must be defined}"/rabbitmq-queues "$@"
66 changes: 66 additions & 0 deletions scripts/rabbitmq-queues.bat
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
@echo off
REM The contents of this file are subject to the Mozilla Public License
REM Version 1.1 (the "License"); you may not use this file except in
REM compliance with the License. You may obtain a copy of the License
REM at http://www.mozilla.org/MPL/
REM
REM Software distributed under the License is distributed on an "AS IS"
REM basis, WITHOUT WARRANTY OF ANY KIND, either express or implied. See
REM the License for the specific language governing rights and
REM limitations under the License.
REM
REM The Original Code is RabbitMQ.
REM
REM The Initial Developer of the Original Code is GoPivotal, Inc.
REM Copyright (c) 2007-2015 Pivotal Software, Inc. All rights reserved.
REM

REM Scopes the variables to the current batch file
setlocal

rem Preserve values that might contain exclamation marks before
rem enabling delayed expansion
set TDP0=%~dp0
set STAR=%*
setlocal enabledelayedexpansion

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

if not exist "!ERLANG_HOME!\bin\erl.exe" (
echo.
echo ******************************
echo ERLANG_HOME not set correctly.
echo ******************************
echo.
echo Please either set ERLANG_HOME to point to your Erlang installation or place the
echo RabbitMQ server distribution in the Erlang lib folder.
echo.
exit /B 1
)

REM Disable erl_crash.dump by default for control scripts.
if not defined ERL_CRASH_DUMP_SECONDS (
set ERL_CRASH_DUMP_SECONDS=0
)

"!ERLANG_HOME!\bin\erl.exe" +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:\=/!"\" ^
-nodename !RABBITMQ_NODENAME! ^
-run escript start ^
-escript main rabbitmqctl_escript ^
-extra "%RABBITMQ_HOME%\escript\rabbitmq-queues" !STAR!

if ERRORLEVEL 1 (
exit /B 1
)

endlocal
endlocal
1 change: 1 addition & 0 deletions scripts/rabbitmq-server
Original file line number Diff line number Diff line change
Expand Up @@ -311,6 +311,7 @@ start_rabbitmq_server() {
-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} \
"$@"
Expand Down
1 change: 1 addition & 0 deletions scripts/rabbitmq-server.bat
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,7 @@ if "!ENV_OK!"=="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! ^
!STAR!
Expand Down
1 change: 1 addition & 0 deletions scripts/rabbitmq-service.bat
Original file line number Diff line number Diff line change
Expand Up @@ -330,6 +330,7 @@ set ERLANG_SERVICE_ARGUMENTS= ^
-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! ^
!STARVAR!
Expand Down
11 changes: 2 additions & 9 deletions src/rabbit.erl
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@
-export([start/0, boot/0, stop/0,
stop_and_halt/0, await_startup/0, await_startup/1,
status/0, is_running/0, alarms/0,
is_running/1, environment/0, rotate_logs/0, force_event_refresh/1,
is_running/1, environment/0, rotate_logs/0,
start_fhc/0]).

-export([start/2, stop/1, prep_stop/1]).
Expand Down Expand Up @@ -225,7 +225,7 @@
-include("rabbit_framing.hrl").
-include("rabbit.hrl").

-define(APPS, [os_mon, mnesia, rabbit_common, rabbit]).
-define(APPS, [os_mon, mnesia, rabbit_common, ra, rabbit]).

-define(ASYNC_THREADS_WARNING_THRESHOLD, 8).

Expand All @@ -252,7 +252,6 @@
-spec is_running(node()) -> boolean().
-spec environment() -> [{param(), term()}].
-spec rotate_logs() -> rabbit_types:ok_or_error(any()).
-spec force_event_refresh(reference()) -> 'ok'.

-spec log_locations() -> [log_location()].

Expand Down Expand Up @@ -941,12 +940,6 @@ start_logger() ->
log_locations() ->
rabbit_lager:log_locations().

force_event_refresh(Ref) ->
rabbit_direct:force_event_refresh(Ref),
rabbit_networking:force_connection_event_refresh(Ref),
rabbit_channel:force_event_refresh(Ref),
rabbit_amqqueue:force_event_refresh(Ref).

%%---------------------------------------------------------------------------
%% misc

Expand Down
Loading