summaryrefslogtreecommitdiff
path: root/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics
diff options
context:
space:
mode:
Diffstat (limited to 'deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics')
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/alarms_command.ex77
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/certificates_command.ex55
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_alarms_command.ex86
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_certificate_expiration_command.ex101
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_local_alarms_command.ex85
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_connectivity_command.ex119
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_listener_command.ex82
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_protocol_listener_command.ex90
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_running_command.ex46
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_virtual_hosts_command.ex71
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/cipher_suites_command.ex122
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/command_line_arguments_command.ex41
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/consume_event_stream_command.ex71
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/disable_auth_attempt_source_tracking_command.ex35
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/discover_peers_command.ex36
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/enable_auth_attempt_source_tracking_command.ex36
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_hash_command.ex35
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_sources_command.ex116
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_version_command.ex72
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_booting_command.ex53
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_running_command.ex45
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_network_interfaces_command.ex77
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_node_auth_attempt_stats_command.ex75
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/listeners_command.ex92
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_location_command.ex56
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_command.ex50
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_stream_command.ex73
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/maybe_stuck_command.ex29
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/memory_breakdown_command.ex103
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/observer_command.ex48
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/os_env_command.ex67
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/reset_node_auth_attempt_metrics_command.ex37
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolve_hostname_command.ex94
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolver_info_command.ex84
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/runtime_thread_stats_command.ex70
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/schema_info_command.ex73
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/server_version_command.ex37
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/tls_versions_command.ex39
-rw-r--r--deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/diagnostics_helpers.ex38
39 files changed, 2616 insertions, 0 deletions
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/alarms_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/alarms_command.ex
new file mode 100644
index 0000000000..7669a523eb
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/alarms_command.ex
@@ -0,0 +1,77 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.AlarmsCommand do
+ @moduledoc """
+ Displays all alarms reported by the target node.
+
+ Returns a code of 0 unless there were connectivity and authentication
+ errors. This command is not meant to be used in health checks.
+ """
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+ import RabbitMQ.CLI.Core.Alarms
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ # Example response when there are alarms:
+ #
+ # [
+ # file_descriptor_limit,
+ # {{resource_limit,disk,hare@warp10},[]},
+ # {{resource_limit,memory,hare@warp10},[]},
+ # {{resource_limit,disk,rabbit@warp10},[]},
+ # {{resource_limit,memory,rabbit@warp10},[]}
+ # ]
+ #
+ # The topmost file_descriptor_limit alarm is node-local.
+ :rabbit_misc.rpc_call(node_name, :rabbit_alarm, :get_alarms, [], timeout)
+ end
+
+ def output([], %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "alarms" => []}}
+ end
+
+ def output([], %{node: node_name}) do
+ {:ok, "Node #{node_name} reported no alarms, local or clusterwide"}
+ end
+
+ def output(alarms, %{node: node_name, formatter: "json"}) do
+ local = local_alarms(alarms, node_name)
+ global = clusterwide_alarms(alarms, node_name)
+
+ {:ok,
+ %{
+ "result" => "ok",
+ "local" => alarm_lines(local, node_name),
+ "global" => alarm_lines(global, node_name),
+ "message" => "Node #{node_name} reported alarms"
+ }}
+ end
+
+ def output(alarms, %{node: node_name}) do
+ lines = alarm_lines(alarms, node_name)
+
+ {:ok, Enum.join(lines, line_separator())}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Lists resource alarms (local or cluster-wide) in effect on the target node"
+
+ def usage, do: "alarms"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} to report any known resource alarms ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/certificates_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/certificates_command.ex
new file mode 100644
index 0000000000..33320d8e37
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/certificates_command.ex
@@ -0,0 +1,55 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CertificatesCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ import RabbitMQ.CLI.Core.Listeners
+
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ case :rabbit_misc.rpc_call(node_name, :rabbit_networking, :active_listeners, [], timeout) do
+ {:error, _} = err ->
+ err
+
+ {:error, _, _} = err ->
+ err
+
+ xs when is_list(xs) ->
+ listeners = listeners_with_certificates(listeners_on(xs, node_name))
+
+ case listeners do
+ [] -> %{}
+ _ -> Enum.map(listeners, &listener_certs/1)
+ end
+
+ other ->
+ other
+ end
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.Erlang
+
+ def usage, do: "certificates"
+
+ def usage_doc_guides() do
+ [
+ DocGuide.configuration(),
+ DocGuide.tls()
+ ]
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Displays certificates (public keys) for every listener on target node that is configured to use TLS"
+
+ def banner(_, %{node: node_name}), do: "Certificates of node #{node_name} ..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_alarms_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_alarms_command.ex
new file mode 100644
index 0000000000..04bb70317a
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_alarms_command.ex
@@ -0,0 +1,86 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckAlarmsCommand do
+ @moduledoc """
+ Exits with a non-zero code if the target node reports any alarms,
+ local or clusterwide.
+
+ This command is meant to be used in health checks.
+ """
+
+ import RabbitMQ.CLI.Core.Alarms
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ # Example response when there are alarms:
+ #
+ # [
+ # file_descriptor_limit,
+ # {{resource_limit,disk,hare@warp10},[]},
+ # {{resource_limit,memory,hare@warp10},[]},
+ # {{resource_limit,disk,rabbit@warp10},[]},
+ # {{resource_limit,memory,rabbit@warp10},[]}
+ # ]
+ #
+ # The topmost file_descriptor_limit alarm is node-local.
+ :rabbit_misc.rpc_call(node_name, :rabbit_alarm, :get_alarms, [], timeout)
+ end
+
+ def output([], %{formatter: "json"}) do
+ {:ok, %{"result" => "ok"}}
+ end
+
+ def output([], %{silent: true}) do
+ {:ok, :check_passed}
+ end
+
+ def output([], %{node: node_name}) do
+ {:ok, "Node #{node_name} reported no alarms, local or clusterwide"}
+ end
+
+ def output(alarms, %{node: node_name, formatter: "json"}) do
+ local = local_alarms(alarms, node_name)
+ global = clusterwide_alarms(alarms, node_name)
+
+ {:error, :check_failed,
+ %{
+ "result" => "error",
+ "local" => alarm_lines(local, node_name),
+ "global" => alarm_lines(global, node_name),
+ "message" => "Node #{node_name} reported alarms"
+ }}
+ end
+
+ def output(alarms, %{silent: true} = _opts) when is_list(alarms) do
+ {:error, :check_failed}
+ end
+
+ def output(alarms, %{node: node_name}) when is_list(alarms) do
+ lines = alarm_lines(alarms, node_name)
+
+ {:error, :check_failed, Enum.join(lines, line_separator())}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Health check that exits with a non-zero code if the target node reports any alarms, local or cluster-wide."
+
+ def usage, do: "check_alarms"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} to report any local resource alarms ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_certificate_expiration_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_certificate_expiration_command.ex
new file mode 100644
index 0000000000..d14ade59f6
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_certificate_expiration_command.ex
@@ -0,0 +1,101 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckCertificateExpirationCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+ alias RabbitMQ.CLI.TimeUnit, as: TU
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ import RabbitMQ.CLI.Core.Listeners
+
+ def switches(), do: [unit: :string, within: :integer]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{unit: "weeks", within: 4}, opts)}
+ end
+
+ def validate(args, _) when length(args) > 0 do
+ {:validation_failure, :too_many_args}
+ end
+ def validate(_, %{unit: unit}) do
+ case TU.known_unit?(unit) do
+ true ->
+ :ok
+
+ false ->
+ {:validation_failure, "unit '#{unit}' is not supported. Please use one of: days, weeks, months, years"}
+ end
+ end
+ def validate(_, _), do: :ok
+
+ def run([], %{node: node_name, unit: unit, within: within, timeout: timeout}) do
+ case :rabbit_misc.rpc_call(node_name, :rabbit_networking, :active_listeners, [], timeout) do
+ {:error, _} = err ->
+ err
+
+ {:error, _, _} = err ->
+ err
+
+ {:badrpc, _} = err ->
+ err
+
+ xs when is_list(xs) ->
+ listeners = listeners_on(xs, node_name)
+ seconds = TU.convert(within, unit)
+ Enum.reduce(listeners, [], fn (listener, acc) -> case listener_expiring_within(listener, seconds) do
+ false -> acc
+ expiring -> [expiring | acc]
+ end
+ end)
+ end
+ end
+
+ def output([], %{formatter: "json"}) do
+ {:ok, %{"result" => "ok"}}
+ end
+
+ def output([], %{unit: unit, within: within}) do
+ unit_label = unit_label(within, unit)
+ {:ok, "No certificates are expiring within #{within} #{unit_label}."}
+ end
+
+ def output(listeners, %{formatter: "json"}) do
+ {:error, :check_failed, %{"result" => "error", "expired" => Enum.map(listeners, &expired_listener_map/1)}}
+ end
+
+ def output(listeners, %{}) do
+ {:error, :check_failed, Enum.map(listeners, &expired_listener_map/1)}
+ end
+
+ def unit_label(1, unit) do
+ unit |> String.slice(0..-2)
+ end
+ def unit_label(_within, unit) do
+ unit
+ end
+
+ def usage, do: "check_certificate_expiration [--within <period>] [--unit <unit>]"
+
+ def usage_additional() do
+ [
+ ["<period>", "period of time to check. Default is four (weeks)."],
+ ["<unit>", "time unit for the period, can be days, weeks, months, years. Default is weeks."],
+ ]
+ end
+
+ def usage_doc_guides() do
+ [
+ DocGuide.tls(),
+ DocGuide.networking()
+ ]
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Checks the expiration date on the certificates for every listener configured to use TLS"
+
+ def banner(_, %{node: node_name}), do: "Checking certificate expiration on node #{node_name} ..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_local_alarms_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_local_alarms_command.ex
new file mode 100644
index 0000000000..1b11537793
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_local_alarms_command.ex
@@ -0,0 +1,85 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckLocalAlarmsCommand do
+ @moduledoc """
+ Exits with a non-zero code if the target node reports any local alarms.
+
+ This command is meant to be used in health checks.
+ """
+
+ import RabbitMQ.CLI.Core.Alarms
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ # Example response when there are alarms:
+ #
+ # [
+ # file_descriptor_limit,
+ # {{resource_limit,disk,hare@warp10},[]},
+ # {{resource_limit,memory,hare@warp10},[]},
+ # {{resource_limit,disk,rabbit@warp10},[]},
+ # {{resource_limit,memory,rabbit@warp10},[]}
+ # ]
+ #
+ # The topmost file_descriptor_limit alarm is node-local.
+ case :rabbit_misc.rpc_call(node_name, :rabbit_alarm, :get_alarms, [], timeout) do
+ [] -> []
+ xs when is_list(xs) -> local_alarms(xs, node_name)
+ other -> other
+ end
+ end
+
+ def output([], %{formatter: "json"}) do
+ {:ok, %{"result" => "ok"}}
+ end
+
+ def output([], %{silent: true}) do
+ {:ok, :check_passed}
+ end
+
+ def output([], %{node: node_name}) do
+ {:ok, "Node #{node_name} reported no local alarms"}
+ end
+
+ def output(alarms, %{node: node_name, formatter: "json"}) do
+ {:error, :check_failed,
+ %{
+ "result" => "error",
+ "local" => alarm_lines(alarms, node_name),
+ "message" => "Node #{node_name} reported local alarms"
+ }}
+ end
+
+ def output(_alarms, %{silent: true}) do
+ {:error, :check_failed}
+ end
+
+ def output(alarms, %{node: node_name}) do
+ lines = alarm_lines(alarms, node_name)
+
+ {:error, Enum.join(lines, line_separator())}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Health check that exits with a non-zero code if the target node reports any local alarms"
+
+ def usage, do: "check_local_alarms"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} to report any local resource alarms ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_connectivity_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_connectivity_command.ex
new file mode 100644
index 0000000000..1c3d86ed83
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_connectivity_command.ex
@@ -0,0 +1,119 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckPortConnectivityCommand do
+ @moduledoc """
+ Checks all listeners on the target node by opening a TCP connection to each
+ and immediately closing it.
+
+ Returns a code of 0 unless there were connectivity and authentication
+ errors. This command is meant to be used in health checks.
+ """
+
+ import RabbitMQ.CLI.Diagnostics.Helpers,
+ only: [check_listener_connectivity: 3]
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+ import RabbitMQ.CLI.Core.Listeners
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ @default_timeout 30_000
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+
+ def merge_defaults(args, opts) do
+ timeout =
+ case opts[:timeout] do
+ nil -> @default_timeout
+ :infinity -> @default_timeout
+ other -> other
+ end
+
+ {args, Map.merge(opts, %{timeout: timeout})}
+ end
+
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ case :rabbit_misc.rpc_call(node_name, :rabbit_networking, :active_listeners, [], timeout) do
+ {:error, _} = err ->
+ err
+
+ {:error, _, _} = err ->
+ err
+
+ xs when is_list(xs) ->
+ locals = listeners_on(xs, node_name)
+
+ case locals do
+ [] -> {true, locals}
+ _ -> check_connectivity_of(locals, node_name, timeout)
+ end
+
+ other ->
+ other
+ end
+ end
+
+ def output({true, listeners}, %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "listeners" => listener_maps(listeners)}}
+ end
+
+ def output({true, listeners}, %{node: node_name}) do
+ ports =
+ listeners
+ |> listener_maps
+ |> Enum.map(fn %{port: p} -> p end)
+ |> Enum.sort()
+ |> Enum.join(", ")
+
+ {:ok, "Successfully connected to ports #{ports} on node #{node_name}."}
+ end
+
+ def output({false, failures}, %{formatter: "json", node: node_name}) do
+ {:error, %{"result" => "error", "node" => node_name, "failures" => listener_maps(failures)}}
+ end
+
+ def output({false, failures}, %{node: node_name}) do
+ lines = [
+ "Connection to ports of the following listeners on node #{node_name} failed: "
+ | listener_lines(failures)
+ ]
+
+ {:error, Enum.join(lines, line_separator())}
+ end
+
+ def description(), do: "Basic TCP connectivity health check for each listener's port on the target node"
+
+ def help_section(), do: :observability_and_health_checks
+
+ def usage, do: "check_port_connectivity"
+
+ def banner([], %{node: node_name}) do
+ "Testing TCP connections to all active listeners on node #{node_name} ..."
+ end
+
+ #
+ # Implementation
+ #
+
+ defp check_connectivity_of(listeners, node_name, timeout) do
+ # per listener timeout
+ t = Kernel.trunc(timeout / (length(listeners) + 1))
+
+ failures =
+ Enum.reject(
+ listeners,
+ fn l -> check_listener_connectivity(listener_map(l), node_name, t) end
+ )
+
+ case failures do
+ [] -> {true, listeners}
+ fs -> {false, fs}
+ end
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_listener_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_listener_command.ex
new file mode 100644
index 0000000000..f321d444db
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_port_listener_command.ex
@@ -0,0 +1,82 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckPortListenerCommand do
+ @moduledoc """
+ Exits with a non-zero code if there is no active listener
+ for the given port on the target node.
+
+ This command is meant to be used in health checks.
+ """
+
+ import RabbitMQ.CLI.Core.Listeners, only: [listeners_on: 2, listener_maps: 1]
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsOnePositiveIntegerArgument
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([port], %{node: node_name, timeout: timeout}) do
+ case :rabbit_misc.rpc_call(node_name, :rabbit_networking, :active_listeners, [], timeout) do
+ {:error, _} = err ->
+ err
+
+ {:error, _, _} = err ->
+ err
+
+ xs when is_list(xs) ->
+ locals = listeners_on(xs, node_name) |> listener_maps
+
+ found =
+ Enum.any?(locals, fn %{port: p} ->
+ to_string(port) == to_string(p)
+ end)
+
+ case found do
+ true -> {true, port}
+ false -> {false, port, locals}
+ end
+
+ other ->
+ other
+ end
+ end
+
+ def output({true, port}, %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "port" => port}}
+ end
+
+ def output({true, port}, %{node: node_name}) do
+ {:ok, "A listener for port #{port} is running on node #{node_name}."}
+ end
+
+ def output({false, port, listeners}, %{formatter: "json"}) do
+ ports = Enum.map(listeners, fn %{port: p} -> p end)
+
+ {:error, :check_failed,
+ %{"result" => "error", "missing" => port, "ports" => ports, "listeners" => listeners}}
+ end
+
+ def output({false, port, listeners}, %{node: node_name}) do
+ ports = Enum.map(listeners, fn %{port: p} -> p end) |> Enum.sort() |> Enum.join(", ")
+
+ {:error, :check_failed,
+ "No listener for port #{port} is active on node #{node_name}. " <>
+ "Found listeners that use the following ports: #{ports}"}
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Health check that exits with a non-zero code if target node does not have an active listener for given port"
+
+ def usage, do: "check_port_listener <port>"
+
+ def banner([port], %{node: node_name}) do
+ "Asking node #{node_name} if there's an active listener on port #{port} ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_protocol_listener_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_protocol_listener_command.ex
new file mode 100644
index 0000000000..10c81c971e
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_protocol_listener_command.ex
@@ -0,0 +1,90 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckProtocolListenerCommand do
+ @moduledoc """
+ Exits with a non-zero code if there is no active listener
+ for the given protocol on the target node.
+
+ This command is meant to be used in health checks.
+ """
+
+ import RabbitMQ.CLI.Core.Listeners,
+ only: [listeners_on: 2, listener_maps: 1, normalize_protocol: 1]
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsOnePositionalArgument
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([proto], %{node: node_name, timeout: timeout}) do
+ proto = normalize_protocol(proto)
+
+ case :rabbit_misc.rpc_call(node_name, :rabbit_networking, :active_listeners, [], timeout) do
+ {:error, _} = err ->
+ err
+
+ {:error, _, _} = err ->
+ err
+
+ xs when is_list(xs) ->
+ locals = listeners_on(xs, node_name) |> listener_maps
+
+ found =
+ Enum.any?(locals, fn %{protocol: p} ->
+ to_string(proto) == to_string(p)
+ end)
+
+ case found do
+ true -> {true, proto}
+ false -> {false, proto, locals}
+ end
+
+ other ->
+ other
+ end
+ end
+
+ def output({true, proto}, %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "protocol" => proto}}
+ end
+
+ def output({true, proto}, %{node: node_name}) do
+ {:ok, "A listener for protocol #{proto} is running on node #{node_name}."}
+ end
+
+ def output({false, proto, listeners}, %{formatter: "json"}) do
+ protocols = Enum.map(listeners, fn %{protocol: p} -> p end)
+
+ {:error,
+ %{
+ "result" => "error",
+ "missing" => proto,
+ "protocols" => protocols,
+ "listeners" => listeners
+ }}
+ end
+
+ def output({false, proto, listeners}, %{node: node_name}) do
+ protocols = Enum.map(listeners, fn %{protocol: p} -> p end) |> Enum.sort() |> Enum.join(", ")
+
+ {:error,
+ "No listener for protocol #{proto} is active on node #{node_name}. " <>
+ "Found listeners for the following protocols: #{protocols}"}
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Health check that exits with a non-zero code if target node does not have an active listener for given protocol"
+
+ def usage, do: "check_protocol_listener <protocol>"
+
+ def banner([proto], %{node: node_name}) do
+ "Asking node #{node_name} if there's an active listener for protocol #{proto} ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_running_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_running_command.ex
new file mode 100644
index 0000000000..690f17e1e7
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_running_command.ex
@@ -0,0 +1,46 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckRunningCommand do
+ @moduledoc """
+ Exits with a non-zero code if the RabbitMQ app on the target node is not running.
+
+ This command is meant to be used in health checks.
+ """
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ # Note: we use is_booted/1 over is_running/1 to avoid
+ # returning a positive result when the node is still booting
+ :rabbit_misc.rpc_call(node_name, :rabbit, :is_booted, [node_name], timeout)
+ end
+
+ def output(true, %{node: node_name} = _options) do
+ {:ok, "RabbitMQ on node #{node_name} is fully booted and running"}
+ end
+
+ def output(false, %{node: node_name} = _options) do
+ {:error,
+ "RabbitMQ on node #{node_name} is not running or has not fully booted yet (check with is_booting)"}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Health check that exits with a non-zero code if the RabbitMQ app on the target node is not running"
+
+ def usage, do: "check_running"
+
+ def banner([], %{node: node_name}) do
+ "Checking if RabbitMQ is running on node #{node_name} ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_virtual_hosts_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_virtual_hosts_command.ex
new file mode 100644
index 0000000000..b3169b522d
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/check_virtual_hosts_command.ex
@@ -0,0 +1,71 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CheckVirtualHostsCommand do
+ @moduledoc """
+ Exits with a non-zero code if the target node reports any vhost down.
+
+ This command is meant to be used in health checks.
+ """
+
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_misc.rpc_call(node_name, :rabbit_vhost_sup_sup, :check, [], timeout)
+ end
+
+ def output([], %{formatter: "json"}) do
+ {:ok, %{"result" => "ok"}}
+ end
+
+ def output([], %{silent: true}) do
+ {:ok, :check_passed}
+ end
+
+ def output([], %{formatter: "erlang"}) do
+ {:ok, :check_passed}
+ end
+
+ def output([], %{node: node_name}) do
+ {:ok, "Node #{node_name} reported all vhosts as running"}
+ end
+
+ def output(vhosts, %{formatter: "erlang"} = _opts) when is_list(vhosts) do
+ {:error, :check_failed, {:down_vhosts, vhosts}}
+ end
+
+ def output(vhosts, %{formatter: "json"} = _opts) when is_list(vhosts) do
+ {:error, :check_failed, %{"result" => "error", "down_vhosts" => vhosts}}
+ end
+
+ def output(vhosts, %{silent: true} = _opts) when is_list(vhosts) do
+ {:error, :check_failed}
+ end
+
+ def output(vhosts, %{node: node_name}) when is_list(vhosts) do
+ lines = Enum.join(vhosts, line_separator())
+ {:error, "Some virtual hosts on node #{node_name} are down:\n#{lines}"}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def description(), do: "Health check that checks if all vhosts are running in the target node"
+
+ def help_section(), do: :observability_and_health_checks
+
+ def usage, do: "check_virtual_hosts"
+
+ def banner([], %{node: node_name}) do
+ "Checking if all vhosts are running on node #{node_name} ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/cipher_suites_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/cipher_suites_command.ex
new file mode 100644
index 0000000000..86e8eee3a4
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/cipher_suites_command.ex
@@ -0,0 +1,122 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CipherSuitesCommand do
+ alias RabbitMQ.CLI.Core.Helpers
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{all: false, format: "openssl"}, Helpers.case_insensitive_format(opts))}
+ end
+
+ def switches(), do: [timeout: :integer,
+ format: :string,
+ all: :boolean]
+ def aliases(), do: [t: :timeout]
+
+ def validate(_, %{format: format})
+ when format != "openssl" and format != "erlang" and format != "map" do
+ {:validation_failure, {:bad_argument, "Format should be either openssl, erlang or map"}}
+ end
+ def validate(args, _) when length(args) > 0 do
+ {:validation_failure, :too_many_args}
+ end
+
+ def validate(_, _), do: :ok
+
+ def run([], %{node: node_name, timeout: timeout, format: format} = opts) do
+ {mod, function} = case format do
+ "openssl" -> {:rabbit_ssl, :cipher_suites_openssl};
+ "erlang" -> {:rabbit_ssl, :cipher_suites_erlang};
+ "map" -> {:rabbit_ssl, :cipher_suites}
+ end
+ args = case opts do
+ %{all: true} -> [:all];
+ %{} -> [:default]
+ end
+ :rabbit_misc.rpc_call(node_name, mod, function, args, timeout)
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def banner([], %{format: "openssl"}), do: "Listing available cipher suites in OpenSSL format"
+ def banner([], %{format: "erlang"}), do: "Listing available cipher suites in Erlang term format"
+ def banner([], %{format: "map"}), do: "Listing available cipher suites in map format"
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Lists cipher suites enabled by default. To list all available cipher suites, add the --all argument."
+
+ def usage, do: "cipher_suites [--format <openssl | erlang | map>] [--all]"
+
+ def usage_additional() do
+ [
+ ["--format", "output format to use: openssl, erlang or map"],
+ ["--all", "list all available suites"]
+ ]
+ end
+
+ defmodule Formatter do
+ alias RabbitMQ.CLI.Formatters.FormatterHelpers
+
+ @behaviour RabbitMQ.CLI.FormatterBehaviour
+
+ def format_output(item, %{format: "erlang"}) do
+ to_string(:io_lib.format("~p", [item]))
+ end
+
+ def format_output(item, %{format: "map"}) do
+ to_string(:io_lib.format("~p", [item]))
+ end
+
+ def format_output(item, %{format: "openssl"} = opts) do
+ RabbitMQ.CLI.Formatters.String.format_output(item, opts)
+ end
+
+ def format_stream(stream, %{format: "erlang"} = opts) do
+ comma_separated(stream, opts)
+ end
+
+ def format_stream(stream, %{format: "map"} = opts) do
+ comma_separated(stream, opts)
+ end
+
+ def format_stream(stream, %{format: "openssl"} = opts) do
+ Stream.map(
+ stream,
+ FormatterHelpers.without_errors_1(fn el ->
+ format_output(el, opts)
+ end)
+ )
+ end
+
+ defp comma_separated(stream, opts) do
+ elements =
+ Stream.scan(
+ stream,
+ :empty,
+ FormatterHelpers.without_errors_2(fn element, previous ->
+ separator =
+ case previous do
+ :empty -> ""
+ _ -> ","
+ end
+
+ format_element(element, separator, opts)
+ end)
+ )
+
+ Stream.concat([["["], elements, ["]"]])
+ end
+
+ defp format_element(val, separator, opts) do
+ separator <> format_output(val, opts)
+ end
+ end
+
+ def formatter(), do: Formatter
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/command_line_arguments_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/command_line_arguments_command.ex
new file mode 100644
index 0000000000..adbf14cfc3
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/command_line_arguments_command.ex
@@ -0,0 +1,41 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.CommandLineArgumentsCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def scopes(), do: [:diagnostics]
+
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+
+ def validate(_, %{formatter: "json"}) do
+ {:validation_failure, :unsupported_formatter}
+ end
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name}) do
+ :rabbit_misc.rpc_call(node_name, :init, :get_arguments, [])
+ end
+ use RabbitMQ.CLI.DefaultOutput
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.Erlang
+
+ def usage, do: "command_line_arguments"
+
+ def usage_doc_guides() do
+ [
+ DocGuide.configuration(),
+ DocGuide.monitoring()
+ ]
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Displays target node's command-line arguments and flags as reported by the runtime"
+
+ def banner(_, %{node: node_name}), do: "Command line arguments of node #{node_name} ..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/consume_event_stream_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/consume_event_stream_command.ex
new file mode 100644
index 0000000000..e7ad171d11
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/consume_event_stream_command.ex
@@ -0,0 +1,71 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2019-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ConsumeEventStreamCommand do
+ @moduledoc """
+ Displays standard log file location on the target node
+ """
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def switches(), do: [duration: :integer, pattern: :string, timeout: :integer]
+ def aliases(), do: [d: :duration, t: :timeout]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{duration: :infinity, pattern: ".*", quiet: true}, opts)}
+ end
+
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout, duration: duration, pattern: pattern}) do
+ pid = self()
+ ref = make_ref()
+ subscribed = :rabbit_misc.rpc_call(
+ node_name,
+ :rabbit_event_consumer, :register,
+ [pid, ref, duration, pattern],
+ timeout)
+ case subscribed do
+ {:ok, ^ref} ->
+ Stream.unfold(:confinue,
+ fn(:finished) -> nil
+ (:confinue) ->
+ receive do
+ {^ref, data, :finished} ->
+ {data, :finished};
+ {^ref, data, :confinue} ->
+ {data, :confinue}
+ end
+ end)
+ error -> error
+ end
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.JsonStream
+
+ def printer(), do: RabbitMQ.CLI.Printers.StdIORaw
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Streams internal events from a running node. Output is jq-compatible."
+
+ def usage, do: "consume_event_stream [--duration|-d <seconds>] [--pattern <pattern>]"
+
+ def usage_additional() do
+ [
+ ["<duration_in_seconds>", "duration in seconds to stream log. Defaults to infinity"],
+ ["<pattern>", "regular expression to pick events"]
+ ]
+ end
+
+ def banner([], %{node: node_name, duration: :infinity}) do
+ "Streaming logs from node #{node_name} ..."
+ end
+ def banner([], %{node: node_name, duration: duration}) do
+ "Streaming logs from node #{node_name} for #{duration} seconds ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/disable_auth_attempt_source_tracking_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/disable_auth_attempt_source_tracking_command.ex
new file mode 100644
index 0000000000..df182a0c97
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/disable_auth_attempt_source_tracking_command.ex
@@ -0,0 +1,35 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.DisableAuthAttemptSourceTrackingCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name}) do
+ :rabbit_misc.rpc_call(node_name, :application, :set_env,
+ [:rabbit, :track_auth_attempt_source, :false])
+ end
+ use RabbitMQ.CLI.DefaultOutput
+
+ def usage, do: "disable_track_auth_attempt_source"
+
+ def usage_doc_guides() do
+ [
+ DocGuide.access_control(),
+ DocGuide.monitoring()
+ ]
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Disables the tracking of peer IP address and username of authentication attempts"
+
+ def banner([], _), do: "Disabling authentication attempt source tracking ..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/discover_peers_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/discover_peers_command.ex
new file mode 100644
index 0000000000..b23a13e370
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/discover_peers_command.ex
@@ -0,0 +1,36 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.DiscoverPeersCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_misc.rpc_call(node_name, :rabbit_peer_discovery, :discover_cluster_nodes, [], timeout)
+ end
+
+ def output({:ok, {[], _}}, _options) do
+ {:ok, "No peers discovered"}
+ end
+
+ def output({:ok, {nodes, _}}, _options) do
+ {:ok, nodes}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Performs peer discovery and lists discovered nodes, if any"
+
+ def usage, do: "discover_peers"
+
+ def banner(_, _), do: "Discovering peers nodes ..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/enable_auth_attempt_source_tracking_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/enable_auth_attempt_source_tracking_command.ex
new file mode 100644
index 0000000000..832891094b
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/enable_auth_attempt_source_tracking_command.ex
@@ -0,0 +1,36 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.EnableAuthAttemptSourceTrackingCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name}) do
+ :rabbit_misc.rpc_call(node_name, :application, :set_env,
+ [:rabbit, :track_auth_attempt_source, :true])
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def usage, do: "enable_auth_attempt_source_tracking"
+
+ def usage_doc_guides() do
+ [
+ DocGuide.access_control(),
+ DocGuide.monitoring()
+ ]
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Enables the tracking of peer IP address and username of authentication attempts"
+
+ def banner([], _), do: "Enabling authentication attempt source tracking ..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_hash_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_hash_command.ex
new file mode 100644
index 0000000000..b6e3186c94
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_hash_command.ex
@@ -0,0 +1,35 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ErlangCookieHashCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_data_coercion.to_binary(
+ :rabbit_misc.rpc_call(node_name, :rabbit_nodes_common, :cookie_hash, [], timeout))
+ end
+
+ def output(result, %{formatter: "json"}) do
+ {:ok, %{"result" => "ok", "value" => result}}
+ end
+ def output(result, _options) when is_bitstring(result) do
+ {:ok, result}
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Displays a hash of the Erlang cookie (shared secret) used by the target node"
+
+ def usage, do: "erlang_cookie_hash"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} its Erlang cookie hash..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_sources_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_sources_command.ex
new file mode 100644
index 0000000000..578ba31c73
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_cookie_sources_command.ex
@@ -0,0 +1,116 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ErlangCookieSourcesCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ import RabbitMQ.CLI.Core.ANSI
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def distribution(_), do: :none
+
+ def run([], opts) do
+ switch_cookie = opts[:erlang_cookie]
+ home_dir = get_home_dir()
+ cookie_file_path = Path.join(home_dir, ".erlang.cookie")
+ cookie_file_stat = case File.stat(Path.join(home_dir, ".erlang.cookie")) do
+ {:error, :enoent} -> nil
+ {:ok, value} -> value
+ end
+ cookie_file_type = case cookie_file_stat do
+ nil -> nil
+ value -> value.type
+ end
+ cookie_file_access = case cookie_file_stat do
+ nil -> nil
+ value -> value.access
+ end
+ cookie_file_size = case cookie_file_stat do
+ nil -> nil
+ value -> value.size
+ end
+
+ %{
+ os_env_cookie_set: System.get_env("RABBITMQ_ERLANG_COOKIE") != nil,
+ os_env_cookie_value_length: String.length(System.get_env("RABBITMQ_ERLANG_COOKIE") || ""),
+ switch_cookie_set: switch_cookie != nil,
+ switch_cookie_value_length: String.length(to_string(switch_cookie) || ""),
+ effective_user: System.get_env("USER"),
+ home_dir: home_dir,
+ cookie_file_path: cookie_file_path,
+ cookie_file_exists: File.exists?(cookie_file_path),
+ cookie_file_type: cookie_file_type,
+ cookie_file_access: cookie_file_access,
+ cookie_file_size: cookie_file_size
+ }
+ end
+
+ def banner([], %{}), do: "Listing Erlang cookie sources used by CLI tools..."
+
+ def output(result, %{formatter: "json"}) do
+ {:ok, result}
+ end
+
+ def output(result, _opts) do
+ cookie_file_lines = [
+ "#{bright("Cookie File")}\n",
+ "Effective user: #{result[:effective_user] || "(none)"}",
+ "Effective home directory: #{result[:home_dir] || "(none)"}",
+ "Cookie file path: #{result[:cookie_file_path]}",
+ "Cookie file exists? #{result[:cookie_file_exists]}",
+ "Cookie file type: #{result[:cookie_file_type] || "(n/a)"}",
+ "Cookie file access: #{result[:cookie_file_access] || "(n/a)"}",
+ "Cookie file size: #{result[:cookie_file_size] || "(n/a)"}",
+ ]
+
+ switch_lines = [
+ "\n#{bright("Cookie CLI Switch")}\n",
+ "--erlang-cookie value set? #{result[:switch_cookie_set]}",
+ "--erlang-cookie value length: #{result[:switch_cookie_value_length] || 0}"
+ ]
+
+ os_env_lines = [
+ "\n#{bright("Env variable ")} #{bright_red("(Deprecated)")}\n",
+ "RABBITMQ_ERLANG_COOKIE value set? #{result[:os_env_cookie_set]}",
+ "RABBITMQ_ERLANG_COOKIE value length: #{result[:os_env_cookie_value_length] || 0}"
+ ]
+
+ lines = cookie_file_lines ++ switch_lines ++ os_env_lines
+
+ {:ok, lines}
+ end
+
+ def help_section(), do: :configuration
+
+ def description() do
+ "Display Erlang cookie source (e.g. $HOME/.erlang.cookie file) information useful for troubleshooting"
+ end
+
+ def usage, do: "erlang_cookie_sources"
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.StringPerLine
+
+ #
+ # Implementation
+ #
+
+ @doc """
+ Computes HOME directory path the same way Erlang VM/ei does,
+ including taking Windows-specific env variables into account.
+ """
+ def get_home_dir() do
+ homedrive = System.get_env("HOMEDRIVE")
+ homepath = System.get_env("HOMEPATH")
+
+ case {homedrive != nil, homepath != nil} do
+ {true, true} -> "#{homedrive}#{homepath}"
+ _ -> System.get_env("HOME")
+ end
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_version_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_version_command.ex
new file mode 100644
index 0000000000..053e0d142e
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/erlang_version_command.ex
@@ -0,0 +1,72 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ErlangVersionCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def switches() do
+ [details: :boolean, offline: :boolean, timeout: :integer]
+ end
+ def aliases(), do: [t: :timeout]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{details: false, offline: false}, opts)}
+ end
+
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{details: details, offline: true}) do
+ case details do
+ true ->
+ :rabbit_data_coercion.to_binary(
+ :rabbit_misc.otp_system_version())
+
+ false ->
+ :rabbit_data_coercion.to_binary(
+ :rabbit_misc.platform_and_version())
+ end
+ end
+ def run([], %{node: node_name, timeout: timeout, details: details}) do
+ case details do
+ true ->
+ :rabbit_data_coercion.to_binary(
+ :rabbit_misc.rpc_call(node_name, :rabbit_misc, :otp_system_version, [], timeout))
+
+ false ->
+ :rabbit_data_coercion.to_binary(
+ :rabbit_misc.rpc_call(node_name, :rabbit_misc, :platform_and_version, [], timeout))
+ end
+ end
+
+ def output(result, %{formatter: "json"}) do
+ {:ok, %{"result" => "ok", "value" => result}}
+ end
+ def output(result, _opts) when is_bitstring(result) do
+ {:ok, result}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Displays Erlang/OTP version on the target node"
+
+ def usage, do: "erlang_version"
+
+ def usage_additional() do
+ [
+ ["--details", "when set, display additional Erlang/OTP system information"],
+ ["--offline", "when set, displays local Erlang/OTP version (that used by CLI tools)"]
+ ]
+ end
+
+ def banner([], %{offline: true}) do
+ "CLI Erlang/OTP version ..."
+ end
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} for its Erlang/OTP version..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_booting_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_booting_command.ex
new file mode 100644
index 0000000000..56b2253c90
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_booting_command.ex
@@ -0,0 +1,53 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.IsBootingCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_misc.rpc_call(node_name, :rabbit, :is_booting, [node_name], timeout)
+ end
+
+ def output(true, %{node: node_name, formatter: "json"}) do
+ m = %{
+ "result" => true,
+ "message" => "RabbitMQ on node #{node_name} is booting"
+ }
+ {:ok, m}
+ end
+
+ def output(false, %{node: node_name, formatter: "json"}) do
+ m = %{
+ "result" => false,
+ "message" => "RabbitMQ on node #{node_name} is fully booted (check with is_running), stopped or has not started booting yet"
+ }
+ {:ok, m}
+ end
+ def output(true, %{node: node_name}) do
+ {:ok, "RabbitMQ on node #{node_name} is booting"}
+ end
+
+ def output(false, %{node: node_name}) do
+ {:ok,
+ "RabbitMQ on node #{node_name} is fully booted (check with is_running), stopped or has not started booting yet"}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Checks if RabbitMQ is still booting on the target node"
+
+ def usage, do: "is_booting"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} for its boot status ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_running_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_running_command.ex
new file mode 100644
index 0000000000..ecf5ce9368
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/is_running_command.ex
@@ -0,0 +1,45 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.IsRunningCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ # Note: we use is_booted/1 over is_running/1 to avoid
+ # returning a positive result when the node is still booting
+ :rabbit_misc.rpc_call(node_name, :rabbit, :is_booted, [node_name], timeout)
+ end
+
+ def output(true, %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => true, "message" => "RabbitMQ on node #{node_name} is fully booted and running"}}
+ end
+ def output(false, %{node: node_name, formatter: "json"}) do
+ {:ok,
+ %{"result" => false, "message" => "RabbitMQ on node #{node_name} is not running or has not fully booted yet (check with is_booting)"}}
+ end
+ def output(true, %{node: node_name}) do
+ {:ok, "RabbitMQ on node #{node_name} is fully booted and running"}
+ end
+ def output(false, %{node: node_name}) do
+ {:ok,
+ "RabbitMQ on node #{node_name} is not running or has not fully booted yet (check with is_booting)"}
+ end
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Checks if RabbitMQ is fully booted and running on the target node"
+
+ def usage, do: "is_running"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} for its status ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_network_interfaces_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_network_interfaces_command.ex
new file mode 100644
index 0000000000..d41409b8c4
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_network_interfaces_command.ex
@@ -0,0 +1,77 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ListNetworkInterfacesCommand do
+ @moduledoc """
+ Displays all network interfaces (NICs) reported by the target node.
+ """
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+ import RabbitMQ.CLI.Core.ANSI
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def switches(), do: [timeout: :integer, offline: :boolean]
+ def aliases(), do: [t: :timeout]
+
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{offline: true}) do
+ :rabbit_net.getifaddrs()
+ end
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_misc.rpc_call(node_name, :rabbit_net, :getifaddrs, [], timeout)
+ end
+
+ def output(nic_map, %{node: node_name, formatter: "json"}) when map_size(nic_map) == 0 do
+ {:ok, %{"result" => "ok", "node" => node_name, "interfaces" => %{}}}
+ end
+ def output(nic_map, %{node: node_name}) when map_size(nic_map) == 0 do
+ {:ok, "Node #{node_name} reported no network interfaces"}
+ end
+ def output(nic_map0, %{node: node_name, formatter: "json"}) do
+ nic_map = Enum.map(nic_map0, fn ({k, v}) -> {to_string(k), v} end)
+ {:ok,
+ %{
+ "result" => "ok",
+ "interfaces" => Enum.into(nic_map, %{}),
+ "message" => "Node #{node_name} reported network interfaces"
+ }}
+ end
+ def output(nic_map, _) when is_map(nic_map) do
+ lines = nic_lines(nic_map)
+
+ {:ok, Enum.join(lines, line_separator())}
+ end
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Lists network interfaces (NICs) on the target node"
+
+ def usage, do: "list_network_interfaces"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} to report its network interfaces ..."
+ end
+
+ #
+ # Implementation
+ #
+
+ defp nic_lines(nic_map) do
+ Enum.reduce(nic_map, [],
+ fn({iface, props}, acc) ->
+ iface_lines = Enum.reduce(props, [],
+ fn({prop, val}, inner_acc) ->
+ ["#{prop}: #{val}" | inner_acc]
+ end)
+
+ header = "#{bright("Interface #{iface}")}\n"
+ acc ++ [header | iface_lines] ++ ["\n"]
+ end)
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_node_auth_attempt_stats_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_node_auth_attempt_stats_command.ex
new file mode 100644
index 0000000000..4793cf6c46
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/list_node_auth_attempt_stats_command.ex
@@ -0,0 +1,75 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ListNodeAuthAttemptStatsCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.Table
+
+ def scopes(), do: [:ctl, :diagnostics]
+
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def switches(), do: [by_source: :boolean]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{by_source: false}, opts)}
+ end
+
+ def validate([], _), do: :ok
+ def validate(_, _), do: {:validation_failure, :too_many_args}
+
+ def run([], %{node: node_name, timeout: timeout, by_source: by_source}) do
+ case by_source do
+ :true ->
+ :rabbit_misc.rpc_call(
+ node_name, :rabbit_core_metrics, :get_auth_attempts_by_source, [], timeout)
+ :false ->
+ :rabbit_misc.rpc_call(
+ node_name, :rabbit_core_metrics, :get_auth_attempts, [], timeout)
+ end
+ end
+
+ def output([], %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "attempts" => []}}
+ end
+ def output([], %{node: node_name}) do
+ {:ok, "Node #{node_name} reported no authentication attempt stats"}
+ end
+ def output(rows, %{node: node_name, formatter: "json"}) do
+ maps = Enum.map(rows, &Map.new/1)
+ {:ok,
+ %{
+ "result" => "ok",
+ "node" => node_name,
+ "attempts" => maps
+ }}
+ end
+ use RabbitMQ.CLI.DefaultOutput
+
+ def usage, do: "list_node_auth_attempts [--by-source]"
+
+ def usage_additional do
+ [
+ ["--by-source", "list authentication attempts by remote address and username"]
+ ]
+ end
+
+ def usage_doc_guides() do
+ [
+ DocGuide.access_control(),
+ DocGuide.monitoring()
+ ]
+ end
+
+ def help_section(), do: :observability_and_health_checks
+ def description(), do: "Lists authentication attempts on the target node"
+
+ def banner([], %{node: node_name}), do: "Listing authentication
+ attempts for node \"#{node_name}\" ..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/listeners_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/listeners_command.ex
new file mode 100644
index 0000000000..f54ce3775e
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/listeners_command.ex
@@ -0,0 +1,92 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ListenersCommand do
+ @moduledoc """
+ Displays all listeners on a node.
+
+ Returns a code of 0 unless there were connectivity and authentication
+ errors. This command is not meant to be used in health checks.
+ """
+
+ import RabbitMQ.CLI.Core.Listeners,
+ only: [listeners_on: 2, listener_lines: 1, listener_maps: 1, listener_rows: 1]
+
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ # Example listener list:
+ #
+ # [{listener,rabbit@warp10,clustering,"localhost",
+ # {0,0,0,0,0,0,0,0},
+ # 25672,[]},
+ # {listener,rabbit@warp10,amqp,"localhost",
+ # {0,0,0,0,0,0,0,0},
+ # 5672,
+ # [{backlog,128},
+ # {nodelay,true},
+ # {linger,{true,0}},
+ # {exit_on_close,false}]},
+ # {listener,rabbit@warp10,stomp,"localhost",
+ # {0,0,0,0,0,0,0,0},
+ # 61613,
+ # [{backlog,128},{nodelay,true}]}]
+ case :rabbit_misc.rpc_call(node_name, :rabbit_networking, :active_listeners, [], timeout) do
+ {:error, _} = err -> err
+ {:error, _, _} = err -> err
+ xs when is_list(xs) -> listeners_on(xs, node_name)
+ other -> other
+ end
+ end
+
+ def output([], %{formatter: fmt}) when fmt == "csv" or fmt == "erlang" do
+ {:ok, []}
+ end
+
+ def output([], %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "listeners" => []}}
+ end
+
+ def output([], %{node: node_name}) do
+ {:ok, "Node #{node_name} reported no enabled listeners."}
+ end
+
+ def output(listeners, %{formatter: "erlang"}) do
+ {:ok, listener_rows(listeners)}
+ end
+
+ def output(listeners, %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "listeners" => listener_maps(listeners)}}
+ end
+
+ def output(listeners, %{formatter: "csv"}) do
+ {:stream, [listener_rows(listeners)]}
+ end
+
+ def output(listeners, _opts) do
+ lines = listener_lines(listeners)
+
+ {:ok, Enum.join(lines, line_separator())}
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(),
+ do: "Lists active connection listeners (bound interface, port, protocol) on the target node"
+
+ def usage, do: "listeners"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} to report its protocol listeners ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_location_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_location_command.ex
new file mode 100644
index 0000000000..36ff562b41
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_location_command.ex
@@ -0,0 +1,56 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2019-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.LogLocationCommand do
+ @moduledoc """
+ Displays standard log file location on the target node
+ """
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ alias RabbitMQ.CLI.Core.LogFiles
+
+ def switches, do: [all: :boolean, timeout: :integer]
+ def aliases, do: [a: :all, t: :timeout]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{all: false}, opts)}
+ end
+
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout, all: all}) do
+ case all do
+ true -> LogFiles.get_log_locations(node_name, timeout);
+ false -> LogFiles.get_default_log_location(node_name, timeout)
+ end
+ end
+
+ def output({:ok, location}, %{node: node_name, formatter: "json"}) do
+ {:ok, %{
+ "result" => "ok",
+ "node_name" => node_name,
+ "paths" => [location]
+ }}
+ end
+ def output(locations, %{node: node_name, formatter: "json"}) do
+ {:ok, %{
+ "result" => "ok",
+ "node_name" => node_name,
+ "paths" => locations
+ }}
+ end
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Shows log file location(s) on target node"
+
+ def usage, do: "log_location [--all|-a]"
+
+ def banner([], %{node: node_name}) do
+ "Log file location(s) on node #{node_name} ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_command.ex
new file mode 100644
index 0000000000..9717908f60
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_command.ex
@@ -0,0 +1,50 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2019-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.LogTailCommand do
+ @moduledoc """
+ Displays standard log file location on the target node
+ """
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ alias RabbitMQ.CLI.Core.LogFiles
+
+ def switches, do: [number: :integer, timeout: :integer]
+ def aliases, do: ['N': :number, t: :timeout]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{number: 50}, opts)}
+ end
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout, number: n}) do
+ case LogFiles.get_default_log_location(node_name, timeout) do
+ {:ok, file} ->
+ :rabbit_misc.rpc_call(node_name,
+ :rabbit_log_tail, :tail_n_lines, [file, n],
+ timeout)
+ error -> error
+ end
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Prints the last N lines of the log on the node"
+
+ def usage, do: "log_tail [--number|-N <number>]"
+
+ def usage_additional do
+ [
+ ["<number>", "number of lines to print. Defaults to 50"]
+ ]
+ end
+
+ def banner([], %{node: node_name, number: n}) do
+ "Last #{n} log lines on node #{node_name} ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_stream_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_stream_command.ex
new file mode 100644
index 0000000000..5080fd0d1d
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/log_tail_stream_command.ex
@@ -0,0 +1,73 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2019-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.LogTailStreamCommand do
+ @moduledoc """
+ Displays standard log file location on the target node
+ """
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ alias RabbitMQ.CLI.Core.LogFiles
+
+
+ def switches(), do: [duration: :integer, timeout: :integer]
+ def aliases(), do: [d: :duration, t: :timeout]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{duration: :infinity}, opts)}
+ end
+
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def printer(), do: RabbitMQ.CLI.Printers.StdIORaw
+
+ def run([], %{node: node_name, timeout: timeout, duration: duration}) do
+ case LogFiles.get_default_log_location(node_name, timeout) do
+ {:ok, file} ->
+ pid = self()
+ ref = make_ref()
+ subscribed = :rabbit_misc.rpc_call(
+ node_name,
+ :rabbit_log_tail, :init_tail_stream,
+ [file, pid, ref, duration],
+ timeout)
+ case subscribed do
+ {:ok, ^ref} ->
+ Stream.unfold(:confinue,
+ fn(:finished) -> nil
+ (:confinue) ->
+ receive do
+ {^ref, data, :finished} -> {data, :finished};
+ {^ref, data, :confinue} -> {data, :confinue}
+ end
+ end)
+ error -> error
+ end
+ error -> error
+ end
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Streams logs from a running node for a period of time"
+
+ def usage, do: "log_tail_stream [--duration|-d <seconds>]"
+
+ def usage_additional() do
+ [
+ ["<duration_in_seconds>", "duration in seconds to stream log. Defaults to infinity"]
+ ]
+ end
+
+ def banner([], %{node: node_name, duration: :infinity}) do
+ "Streaming logs from node #{node_name} ..."
+ end
+ def banner([], %{node: node_name, duration: duration}) do
+ "Streaming logs from node #{node_name} for #{duration} seconds ..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/maybe_stuck_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/maybe_stuck_command.ex
new file mode 100644
index 0000000000..c241780f62
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/maybe_stuck_command.ex
@@ -0,0 +1,29 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.MaybeStuckCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+ use RabbitMQ.CLI.DefaultOutput
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_misc.rpc_call(node_name, :rabbit_diagnostics, :maybe_stuck, [], timeout)
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Detects Erlang processes (\"lightweight threads\") potentially not making progress on the target node"
+
+ def usage, do: "maybe_stuck"
+
+ def banner(_, %{node: node_name}) do
+ "Asking node #{node_name} to detect potentially stuck Erlang processes..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/memory_breakdown_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/memory_breakdown_command.ex
new file mode 100644
index 0000000000..356358b7d7
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/memory_breakdown_command.ex
@@ -0,0 +1,103 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.MemoryBreakdownCommand do
+ alias RabbitMQ.CLI.InformationUnit, as: IU
+ import RabbitMQ.CLI.Core.Memory
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def switches(), do: [unit: :string, timeout: :integer]
+ def aliases(), do: [t: :timeout]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{unit: "gb"}, opts)}
+ end
+
+ def validate(args, _) when length(args) > 0 do
+ {:validation_failure, :too_many_args}
+ end
+
+ def validate(_, %{unit: unit}) do
+ case IU.known_unit?(unit) do
+ true ->
+ :ok
+
+ false ->
+ {:validation_failure, "unit '#{unit}' is not supported. Please use one of: bytes, mb, gb"}
+ end
+ end
+
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_misc.rpc_call(node_name, :rabbit_vm, :memory, [], timeout)
+ end
+
+ def output(result, %{formatter: "json"} = _opts) do
+ {:ok, compute_relative_values(result)}
+ end
+
+ def output(result, %{formatter: "csv"} = _opts) do
+ flattened =
+ compute_relative_values(result)
+ |> Enum.flat_map(fn {k, %{bytes: b, percentage: p}} ->
+ [{"#{k}.bytes", b}, {"#{k}.percentage", p}]
+ end)
+ |> Enum.sort_by(fn {key, _val} -> key end, &>=/2)
+
+ headers = Enum.map(flattened, fn {k, _v} -> k end)
+ values = Enum.map(flattened, fn {_k, v} -> v end)
+
+ {:stream, [headers, values]}
+ end
+
+ def output(result, _opts) do
+ {:ok, compute_relative_values(result)}
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Provides a memory usage breakdown on the target node."
+
+ def usage, do: "memory_breakdown [--unit <unit>]"
+
+ def usage_additional() do
+ [
+ ["--unit <bytes | mb | gb>", "byte multiple (bytes, megabytes, gigabytes) to use"],
+ ["--formatter <json | csv | erlang>", "alternative formatter to use, JSON, CSV or Erlang terms"]
+ ]
+ end
+
+ def banner([], %{node: node_name}) do
+ "Reporting memory breakdown on node #{node_name}..."
+ end
+
+ defmodule Formatter do
+ alias RabbitMQ.CLI.Formatters.FormatterHelpers
+ alias RabbitMQ.CLI.InformationUnit, as: IU
+
+ @behaviour RabbitMQ.CLI.FormatterBehaviour
+
+ def format_output(output, %{unit: unit}) do
+ Enum.reduce(output, "", fn {key, %{bytes: bytes, percentage: percentage}}, acc ->
+ u = String.downcase(unit)
+ acc <> "#{key}: #{IU.convert(bytes, u)} #{u} (#{percentage}%)\n"
+ end)
+ end
+
+ def format_stream(stream, options) do
+ Stream.map(
+ stream,
+ FormatterHelpers.without_errors_1(fn el ->
+ format_output(el, options)
+ end)
+ )
+ end
+ end
+
+ def formatter(), do: Formatter
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/observer_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/observer_command.ex
new file mode 100644
index 0000000000..717e23e6b5
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/observer_command.ex
@@ -0,0 +1,48 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ObserverCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+ use RabbitMQ.CLI.DefaultOutput
+
+ def switches(), do: [interval: :integer]
+ def aliases(), do: [i: :interval]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{interval: 5}, opts)}
+ end
+
+
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, interval: interval}) do
+ case :observer_cli.start(node_name, [{:interval, interval * 1000}]) do
+ # See zhongwencool/observer_cli#54
+ {:badrpc, _} = err -> err
+ {:error, _} = err -> err
+ {:error, _, _} = err -> err
+ :ok -> {:ok, "Disconnected from #{node_name}."}
+ :quit -> {:ok, "Disconnected from #{node_name}."}
+ other -> other
+ end
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Starts a CLI observer interface on the target node"
+
+ def usage, do: "observer [--interval <seconds>]"
+
+ def usage_additional() do
+ [
+ ["--interval <seconds>", "Update interval to use, in seconds"]
+ ]
+ end
+
+ def banner(_, %{node: node_name}) do
+ "Starting a CLI observer interface on node #{node_name}..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/os_env_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/os_env_command.ex
new file mode 100644
index 0000000000..63e8c18beb
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/os_env_command.ex
@@ -0,0 +1,67 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.OsEnvCommand do
+ @moduledoc """
+ Lists RabbitMQ-specific environment variables defined on target node
+ """
+
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def scopes(), do: [:diagnostics]
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ case :rabbit_misc.rpc_call(node_name, :rabbit_env, :get_used_env_vars, [], timeout) do
+ {:error, _} = err -> err
+ {:error, _, _} = err -> err
+ xs when is_list(xs) ->
+ # convert keys and values to binaries (Elixir strings)
+ xs
+ |> Enum.map(fn {k, v} -> {:rabbit_data_coercion.to_binary(k), :rabbit_data_coercion.to_binary(v)} end)
+ |> :maps.from_list
+ other -> other
+ end
+ end
+
+ def output([], %{formatter: fmt}) when fmt == "csv" or fmt == "erlang" do
+ {:ok, []}
+ end
+ def output([], %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "variables" => []}}
+ end
+ def output([], %{node: node_name}) do
+ {:ok, "Node #{node_name} reported no relevant environment variables."}
+ end
+ def output(vars, %{node: node_name, formatter: "json"}) do
+ {:ok, %{"result" => "ok", "node" => node_name, "variables" => vars}}
+ end
+ def output(vars, %{formatter: "csv"}) do
+ {:stream, [Enum.map(vars, fn({k, v}) -> [variable: k, value: v] end)]}
+ end
+ def output(vars, _opts) do
+ lines = Enum.map(vars, fn({k, v}) -> "#{k}=#{v}" end) |> Enum.join(line_separator())
+ {:ok, lines}
+ end
+
+ def usage() do
+ "os_env"
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Lists RabbitMQ-specific environment variables set on target node"
+
+ def banner(_, %{node: node_name}) do
+ "Listing RabbitMQ-specific environment variables defined on node #{node_name}..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/reset_node_auth_attempt_metrics_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/reset_node_auth_attempt_metrics_command.ex
new file mode 100644
index 0000000000..e3b08c2ac8
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/reset_node_auth_attempt_metrics_command.ex
@@ -0,0 +1,37 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ResetNodeAuthAttemptMetricsCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name}) do
+ :rabbit_misc.rpc_call(node_name, :rabbit_core_metrics, :reset_auth_attempt_metrics, [])
+ end
+
+ def usage, do: "reset_node_auth_attempt_metrics"
+
+ def usage_doc_guides() do
+ [
+ DocGuide.access_control(),
+ DocGuide.monitoring()
+ ]
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Resets auth attempt metrics on the target node"
+
+ def banner([], %{node: node_name}) do
+ "Reset auth attempt metrics on node #{node_name} ..."
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolve_hostname_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolve_hostname_command.ex
new file mode 100644
index 0000000000..349dbee513
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolve_hostname_command.ex
@@ -0,0 +1,94 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ResolveHostnameCommand do
+ @moduledoc """
+ Resolves a hostname to one or more addresses of a given IP address family (IPv4 ot IPv6).
+ This command is not meant to compete with `dig` but rather provide a way
+ to perform basic resolution tests that take Erlang's inetrc file into account.
+ """
+
+ import RabbitCommon.Records
+ alias RabbitMQ.CLI.Core.Networking
+ alias RabbitMQ.CLI.Core.ExitCodes
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def scopes(), do: [:diagnostics]
+
+ def switches(), do: [address_family: :string, offline: :boolean]
+ def aliases(), do: [a: :address_family]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{address_family: "IPv4", offline: false}, opts)}
+ end
+
+ def validate(args, _) when length(args) < 1, do: {:validation_failure, :not_enough_args}
+ def validate(args, _) when length(args) > 1, do: {:validation_failure, :too_many_args}
+ def validate([_], %{address_family: family}) do
+ case Networking.valid_address_family?(family) do
+ true -> :ok
+ false -> {:validation_failure, {:bad_argument, "unsupported IP address family #{family}. Valid values are: ipv4, ipv6"}}
+ end
+ end
+ def validate([_], _), do: :ok
+
+ def run([hostname], %{address_family: family, offline: true}) do
+ :inet.gethostbyname(to_charlist(hostname), Networking.address_family(family))
+ end
+ def run([hostname], %{node: node_name, address_family: family, offline: false, timeout: timeout}) do
+ case :rabbit_misc.rpc_call(node_name, :inet, :gethostbyname,
+ [to_charlist(hostname), Networking.address_family(family)], timeout) do
+ {:error, _} = err -> err
+ {:error, _, _} = err -> err
+ {:ok, result} -> {:ok, result}
+ other -> other
+ end
+ end
+
+ def output({:error, :nxdomain}, %{node: node_name, formatter: "json"}) do
+ m = %{
+ "result" => "error",
+ "node" => node_name,
+ "message" => "Hostname does not resolve (resolution failed with an nxdomain)"
+ }
+ {:error, ExitCodes.exit_dataerr(), m}
+ end
+ def output({:error, :nxdomain}, _opts) do
+ {:error, ExitCodes.exit_dataerr(), "Hostname does not resolve (resolution failed with an nxdomain)"}
+ end
+ def output({:ok, result}, %{node: node_name, address_family: family, formatter: "json"}) do
+ hostname = hostent(result, :h_name)
+ addresses = hostent(result, :h_addr_list)
+ {:ok, %{
+ "result" => "ok",
+ "node" => node_name,
+ "hostname" => to_string(hostname),
+ "address_family" => family,
+ "addresses" => Networking.format_addresses(addresses)
+ }}
+ end
+ def output({:ok, result}, _opts) do
+ addresses = hostent(result, :h_addr_list)
+ {:ok, Enum.join(Networking.format_addresses(addresses), "\n")}
+ end
+ use RabbitMQ.CLI.DefaultOutput
+
+ def usage() do
+ "resolve_hostname <hostname> [--address-family <ipv4 | ipv6>]"
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Resolves a hostname to a set of addresses. Takes Erlang's inetrc file into account."
+
+ def banner([hostname], %{offline: false, node: node_name, address_family: family}) do
+ "Asking node #{node_name} to resolve hostname #{hostname} to #{family} addresses..."
+ end
+ def banner([hostname], %{offline: true, address_family: family}) do
+ "Resolving hostname #{hostname} to #{family} addresses..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolver_info_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolver_info_command.ex
new file mode 100644
index 0000000000..a4f3d8d7d3
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/resolver_info_command.ex
@@ -0,0 +1,84 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ResolverInfoCommand do
+ @moduledoc """
+ Displays effective hostname resolver (inetrc) configuration on target node
+ """
+
+ import RabbitMQ.CLI.Core.Platform, only: [line_separator: 0]
+ import RabbitMQ.CLI.Core.ANSI, only: [bright: 1]
+ alias RabbitMQ.CLI.Core.Networking
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def scopes(), do: [:diagnostics]
+
+ def switches(), do: [offline: :boolean]
+ def aliases(), do: []
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{offline: false}, opts)}
+ end
+
+ def validate(args, _) when length(args) > 0, do: {:validation_failure, :too_many_args}
+ def validate([], _), do: :ok
+
+ def run([], %{offline: true}) do
+ Networking.inetrc_map(:inet.get_rc())
+ end
+ def run([], %{node: node_name, timeout: timeout, offline: false}) do
+ case :rabbit_misc.rpc_call(node_name, :inet, :get_rc, [], timeout) do
+ {:error, _} = err -> err
+ {:error, _, _} = err -> err
+ xs when is_list(xs) -> Networking.inetrc_map(xs)
+ other -> other
+ end
+ end
+
+ def output(info, %{node: node_name, formatter: "json"}) do
+ {:ok, %{
+ "result" => "ok",
+ "node" => node_name,
+ "resolver" => info
+ }}
+ end
+ def output(info, _opts) do
+ main_section = [
+ "#{bright("Runtime Hostname Resolver (inetrc) Settings")}\n",
+ "Lookup order: #{info["lookup"]}",
+ "Hosts file: #{info["hosts_file"]}",
+ "Resolver conf file: #{info["resolv_conf"]}",
+ "Cache size: #{info["cache_size"]}"
+ ]
+ hosts_section = [
+ "\n#{bright("inetrc File Host Entries")}\n"
+ ] ++ case info["hosts"] do
+ [] -> ["(none)"]
+ nil -> ["(none)"]
+ hs -> Enum.reduce(hs, [], fn {k, v}, acc -> ["#{k} #{Enum.join(v, ", ")}" | acc] end)
+ end
+
+ lines = main_section ++ hosts_section
+
+ {:ok, Enum.join(lines, line_separator())}
+ end
+
+ def usage() do
+ "resolver_info"
+ end
+
+ def help_section(), do: :configuration
+
+ def description(), do: "Displays effective hostname resolver (inetrc) configuration on target node"
+
+ def banner(_, %{node: node_name, offline: false}) do
+ "Asking node #{node_name} for its effective hostname resolver (inetrc) configuration..."
+ end
+ def banner(_, %{offline: true}) do
+ "Displaying effective hostname resolver (inetrc) configuration used by CLI tools..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/runtime_thread_stats_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/runtime_thread_stats_command.ex
new file mode 100644
index 0000000000..ee5bb56566
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/runtime_thread_stats_command.ex
@@ -0,0 +1,70 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.RuntimeThreadStatsCommand do
+ alias RabbitMQ.CLI.Core.DocGuide
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ def switches(), do: [sample_interval: :integer]
+ def aliases(), do: [i: :sample_interval]
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{sample_interval: 5}, opts)}
+ end
+
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout, sample_interval: interval}) do
+ case :rabbit_misc.rpc_call(
+ node_name,
+ :rabbit_runtime,
+ :msacc_stats,
+ [interval * 1000],
+ timeout
+ ) do
+ {:ok, stats} -> stats
+ other -> other
+ end
+ end
+
+ def output(result, %{formatter: "json"}) when is_list(result) do
+ {:error, "JSON formatter is not supported by this command"}
+ end
+
+ def output(result, %{formatter: "csv"}) when is_list(result) do
+ {:error, "CSV formatter is not supported by this command"}
+ end
+
+ def output(result, _options) when is_list(result) do
+ {:ok, result}
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Provides a breakdown of runtime thread activity stats on the target node"
+
+ def usage, do: "runtime_thread_stats [--sample-interval <interval>]"
+
+ def usage_additional() do
+ [
+ ["--sample-interval <seconds>", "sampling interval to use in seconds"]
+ ]
+ end
+
+ def usage_doc_guides() do
+ [
+ DocGuide.runtime_tuning()
+ ]
+ end
+
+ def banner([], %{node: node_name, sample_interval: interval}) do
+ "Will collect runtime thread stats on #{node_name} for #{interval} seconds..."
+ end
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.Msacc
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/schema_info_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/schema_info_command.ex
new file mode 100644
index 0000000000..50b750c772
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/schema_info_command.ex
@@ -0,0 +1,73 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.SchemaInfoCommand do
+ @moduledoc """
+ Lists all tables on the mnesia schema
+ """
+
+ alias RabbitMQ.CLI.Ctl.InfoKeys
+
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ @info_keys ~w(name snmp load_order active_replicas all_nodes attributes checkpoints disc_copies
+ disc_only_copies external_copies frag_properties master_nodes ram_copies
+ storage_properties subscribers user_properties cstruct local_content
+ where_to_commit where_to_read name access_mode cookie load_by_force
+ load_node record_name size storage_type type where_to_write index arity
+ majority memory commit_work where_to_wlock load_reason record_validation
+ version wild_pattern index_info)a
+
+ def info_keys(), do: @info_keys
+
+ def scopes(), do: [:ctl, :diagnostics]
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def merge_defaults([], opts) do
+ merge_defaults(
+ ~w(name cookie active_replicas user_properties),
+ opts
+ )
+ end
+
+ def merge_defaults(args, opts) do
+ {args, Map.merge(%{table_headers: true}, opts)}
+ end
+
+ def validate(args, _) do
+ case InfoKeys.validate_info_keys(args, @info_keys) do
+ {:ok, _} -> :ok
+ err -> err
+ end
+ end
+
+ def run([_ | _] = args, %{node: node_name, timeout: timeout}) do
+ info_keys = InfoKeys.prepare_info_keys(args)
+ :rabbit_misc.rpc_call(node_name, :rabbit_mnesia, :schema_info, [info_keys], timeout)
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.Table
+
+ def usage() do
+ "schema_info [--no-table-headers] [<column> ...]"
+ end
+
+ def usage_additional() do
+ [
+ ["<column>", "must be one of " <> Enum.join(Enum.sort(@info_keys), ", ")]
+ ]
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Lists schema database tables and their properties"
+
+ def banner(_, %{node: node_name}), do: "Asking node #{node_name} to report its schema..."
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/server_version_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/server_version_command.ex
new file mode 100644
index 0000000000..9f4068e459
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/server_version_command.ex
@@ -0,0 +1,37 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.ServerVersionCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+
+ def run([], %{node: node_name, timeout: timeout}) do
+ :rabbit_data_coercion.to_binary(
+ :rabbit_misc.rpc_call(node_name, :rabbit_misc, :version, [], timeout))
+ end
+
+ def output(result, %{formatter: "json"}) do
+ {:ok, %{"result" => "ok", "value" => result}}
+ end
+ def output(result, _options) when is_bitstring(result) do
+ {:ok, result}
+ end
+
+ use RabbitMQ.CLI.DefaultOutput
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Displays server version on the target node"
+
+ def usage, do: "server_version"
+
+ def banner([], %{node: node_name}) do
+ "Asking node #{node_name} for its RabbitMQ version..."
+ end
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/tls_versions_command.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/tls_versions_command.ex
new file mode 100644
index 0000000000..2f81bad889
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/commands/tls_versions_command.ex
@@ -0,0 +1,39 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Commands.TlsVersionsCommand do
+ @behaviour RabbitMQ.CLI.CommandBehaviour
+
+ use RabbitMQ.CLI.Core.AcceptsDefaultSwitchesAndTimeout
+ use RabbitMQ.CLI.Core.MergesNoDefaults
+ use RabbitMQ.CLI.Core.AcceptsNoPositionalArguments
+ use RabbitMQ.CLI.Core.RequiresRabbitAppRunning
+
+ def run([], %{node: node_name, timeout: timeout} = _opts) do
+ :rabbit_misc.rpc_call(node_name, :ssl, :versions, [], timeout)
+ end
+
+ def banner([], %{}), do: "Listing all TLS versions supported by the runtime..."
+
+ def output(result, %{formatter: "json"}) do
+ vs = Map.new(result) |> Map.get(:available)
+
+ {:ok, %{versions: vs}}
+ end
+
+ def output(result, _opts) do
+ vs = Map.new(result) |> Map.get(:available)
+ {:ok, vs}
+ end
+
+ def help_section(), do: :observability_and_health_checks
+
+ def description(), do: "Lists TLS versions supported (but not necessarily allowed) on the target node"
+
+ def usage, do: "tls_versions"
+
+ def formatter(), do: RabbitMQ.CLI.Formatters.StringPerLine
+end
diff --git a/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/diagnostics_helpers.ex b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/diagnostics_helpers.ex
new file mode 100644
index 0000000000..601cc842cb
--- /dev/null
+++ b/deps/rabbitmq_cli/lib/rabbitmq/cli/diagnostics/diagnostics_helpers.ex
@@ -0,0 +1,38 @@
+## This Source Code Form is subject to the terms of the Mozilla Public
+## License, v. 2.0. If a copy of the MPL was not distributed with this
+## file, You can obtain one at https://mozilla.org/MPL/2.0/.
+##
+## Copyright (c) 2007-2020 VMware, Inc. or its affiliates. All rights reserved.
+
+defmodule RabbitMQ.CLI.Diagnostics.Helpers do
+ def test_connection(hostname, port, timeout) do
+ case :gen_tcp.connect(hostname, port, [], timeout) do
+ {:error, _} -> :gen_tcp.connect(hostname, port, [:inet6], timeout)
+ r -> r
+ end
+ end
+
+ def check_port_connectivity(port, node_name, timeout) do
+ regex = Regex.recompile!(~r/^(.+)@/)
+ hostname = Regex.replace(regex, to_string(node_name), "") |> to_charlist
+ try do
+ case test_connection(hostname, port, timeout) do
+ {:error, _} ->
+ false
+
+ {:ok, port} ->
+ :ok = :gen_tcp.close(port)
+ true
+ end
+
+ # `gen_tcp:connect/4` will throw if the port is outside of its
+ # expected domain
+ catch
+ :exit, _ -> false
+ end
+ end
+
+ def check_listener_connectivity(%{port: port}, node_name, timeout) do
+ check_port_connectivity(port, node_name, timeout)
+ end
+end