Skip to content

Commit

Permalink
Resubmit: #47871 [Ray Core] Fix the Paths in the Generated Monitoring…
Browse files Browse the repository at this point in the history
… Configs to Consider temp-dir

Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
  • Loading branch information
MengjinYan committed Oct 16, 2024
1 parent 8bd6a9a commit 1740d4f
Show file tree
Hide file tree
Showing 10 changed files with 190 additions and 69 deletions.
14 changes: 12 additions & 2 deletions doc/source/cluster/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,12 @@ For a quick demo, you can run Prometheus locally on your machine. Follow the qui

### Quickstart: Running Prometheus locally

```{admonition} Note
:class: note
If you need to change the root temporary directory by using "--temp-dir" in your ray
cluster setup, please follow the [manual steps](#optional-manual-running-prometheus-locally) to setup Prometheus locally.
```

Run the following command to download and start Prometheus locally with a configuration that scrapes metrics from a local Ray Cluster.

```bash
Expand Down Expand Up @@ -76,7 +82,7 @@ tar xvfz prometheus-*.tar.gz
cd prometheus-*
```

Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`.
Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. If you specify the `--temp-dir={your_temp_path}` when starting the ray cluster, the config file will be at `{yout_temp_path}/session_latest/metrics/prometheus/prometheus.yml`

```yaml
global:
Expand All @@ -88,13 +94,17 @@ scrape_configs:
- job_name: 'ray'
file_sd_configs:
- files:
- '/tmp/ray/prom_metrics_service_discovery.json'
- '/tmp/ray/prom_metrics_service_discovery.json' # or '${your_temp_path}/prom_metrics_service_discovery.json' if --temp-dir is specified
```
Next, start Prometheus:
```shell
# With default settings
./prometheus --config.file=/tmp/ray/session_latest/metrics/prometheus/prometheus.yml

# With specified --temp-dir
./prometheus --config.file={your_temp_path}/session_latest/metrics/prometheus/prometheus.yml
```
```{admonition} Note
:class: note
Expand Down
10 changes: 0 additions & 10 deletions python/ray/dashboard/modules/metrics/export/grafana/grafana.ini

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,6 @@
serve_deployment_dashboard_config,
)

METRICS_INPUT_ROOT = os.path.join(os.path.dirname(__file__), "export")
GRAFANA_CONFIG_INPUT_PATH = os.path.join(METRICS_INPUT_ROOT, "grafana")

GRAFANA_DASHBOARD_UID_OVERRIDE_ENV_VAR_TEMPLATE = "RAY_GRAFANA_{name}_DASHBOARD_UID"
GRAFANA_DASHBOARD_GLOBAL_FILTERS_OVERRIDE_ENV_VAR_TEMPLATE = (
"RAY_GRAFANA_{name}_DASHBOARD_GLOBAL_FILTERS"
Expand Down
Original file line number Diff line number Diff line change
@@ -1,10 +1,10 @@
DASHBOARD_PROVISIONING_TEMPLATE = """
apiVersion: 1
# DASHBOARD_PROVISIONING_TEMPLATE = """
# apiVersion: 1

providers:
- name: Ray # Default dashboards provided by OSS ray
folder: Ray
type: file
options:
path: {dashboard_output_folder}
"""
# providers:
# - name: Ray # Default dashboards provided by OSS ray
# folder: Ray
# type: file
# options:
# path: {dashboard_output_folder}
# """
Original file line number Diff line number Diff line change
@@ -1,9 +1,9 @@
GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1
# GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1

datasources:
- name: {prometheus_name}
url: {prometheus_host}
type: prometheus
isDefault: true
access: proxy
"""
# datasources:
# - name: {prometheus_name}
# url: {prometheus_host}
# type: prometheus
# isDefault: true
# access: proxy
# """
Original file line number Diff line number Diff line change
Expand Up @@ -74,12 +74,13 @@ def install_prometheus(file_path):

def start_prometheus(prometheus_dir):

# Currently, Ray never modifies this config file, so we can just use the
# hardcoded path. (It just copies it to a more user-friendly location, in
# MetricsHead._create_default_prometheus_configs.)
# However, if in the future Ray ever modifies this file at runtime, we'll
# need to use the user-friendly location instead, and reload the config
# file after it's updated by Ray.
# The function assumes the Ray cluster to be monitored by Prometheus uses the
# default configuration with "/tmp/ray" as the default root temporary directory.
#
# This is to support the `ray metrics launch-prometheus` command, when a ray cluster
# is not started yet and we have no way to get a `--temp-dir` anywhere. So we choose
# to use a hardcoded default value.

config_file = Path(PROMETHEUS_CONFIG_INPUT_PATH)

if not config_file.exists():
Expand Down
91 changes: 63 additions & 28 deletions python/ray/dashboard/modules/metrics/metrics_head.py
Original file line number Diff line number Diff line change
Expand Up @@ -11,24 +11,24 @@
import ray.dashboard.optional_utils as dashboard_optional_utils
import ray.dashboard.utils as dashboard_utils
from ray._private.async_utils import enable_monitor_loop_lag
from ray._private.ray_constants import env_integer
from ray._private.utils import get_or_create_event_loop
from ray.dashboard.consts import (
AVAILABLE_COMPONENT_NAMES_FOR_METRICS,
METRICS_INPUT_ROOT,
PROMETHEUS_CONFIG_INPUT_PATH,
from ray._private.ray_constants import (
PROMETHEUS_SERVICE_DISCOVERY_FILE,
SESSION_LATEST,
env_integer,
)
from ray._private.utils import get_or_create_event_loop
from ray.dashboard.consts import AVAILABLE_COMPONENT_NAMES_FOR_METRICS
from ray.dashboard.modules.metrics.grafana_dashboard_factory import (
generate_data_grafana_dashboard,
generate_default_grafana_dashboard,
generate_serve_deployment_grafana_dashboard,
generate_serve_grafana_dashboard,
)
from ray.dashboard.modules.metrics.grafana_dashboard_provisioning_template import (
from ray.dashboard.modules.metrics.templates import (
DASHBOARD_PROVISIONING_TEMPLATE,
)
from ray.dashboard.modules.metrics.grafana_datasource_template import (
GRAFANA_DATASOURCE_TEMPLATE,
GRAFANA_INI_TEMPLATE,
PROMETHEUS_YML_TEMPLATE,
)

import psutil
Expand All @@ -54,7 +54,6 @@
GRAFANA_HOST_DISABLED_VALUE = "DISABLED"
GRAFANA_IFRAME_HOST_ENV_VAR = "RAY_GRAFANA_IFRAME_HOST"
GRAFANA_DASHBOARD_OUTPUT_DIR_ENV_VAR = "RAY_METRICS_GRAFANA_DASHBOARD_OUTPUT_DIR"
GRAFANA_CONFIG_INPUT_PATH = os.path.join(METRICS_INPUT_ROOT, "grafana")
GRAFANA_HEALTHCHECK_PATH = "api/health"


Expand All @@ -75,13 +74,22 @@ def __init__(self, dashboard_head):
PROMETHEUS_HOST_ENV_VAR, DEFAULT_PROMETHEUS_HOST
)
default_metrics_root = os.path.join(self._dashboard_head.session_dir, "metrics")
session_latest_metrics_root = os.path.join(
self._dashboard_head.temp_dir, SESSION_LATEST, "metrics"
)
self._metrics_root = os.environ.get(
METRICS_OUTPUT_ROOT_ENV_VAR, default_metrics_root
)
grafana_config_output_path = os.path.join(self._metrics_root, "grafana")
self._metrics_root_session_latest = os.environ.get(
METRICS_OUTPUT_ROOT_ENV_VAR, session_latest_metrics_root
)
self._grafana_config_output_path = os.path.join(self._metrics_root, "grafana")
self._grafana_session_latest_config_output_path = os.path.join(
self._metrics_root_session_latest, "grafana"
)
self._grafana_dashboard_output_dir = os.environ.get(
GRAFANA_DASHBOARD_OUTPUT_DIR_ENV_VAR,
os.path.join(grafana_config_output_path, "dashboards"),
os.path.join(self._grafana_config_output_path, "dashboards"),
)

self._prometheus_name = os.environ.get(
Expand Down Expand Up @@ -186,17 +194,34 @@ def _create_default_grafana_configs(self):
"""
Creates the grafana configurations that are by default provided by Ray.
"""
grafana_config_output_path = os.path.join(self._metrics_root, "grafana")

# Copy default grafana configurations
if os.path.exists(grafana_config_output_path):
shutil.rmtree(grafana_config_output_path)
os.makedirs(os.path.dirname(grafana_config_output_path), exist_ok=True)
shutil.copytree(GRAFANA_CONFIG_INPUT_PATH, grafana_config_output_path)
# Create grafana configuration folder
if os.path.exists(self._grafana_config_output_path):
shutil.rmtree(self._grafana_config_output_path)
os.makedirs(self._grafana_config_output_path, exist_ok=True)

# Overwrite grafana's configuration file
grafana_provisioning_folder = os.path.join(
self._grafana_config_output_path, "provisioning"
)
grafana_prov_folder_with_latest_session = os.path.join(
self._grafana_session_latest_config_output_path, "provisioning"
)
with open(
os.path.join(
self._grafana_config_output_path,
"grafana.ini",
),
"w",
) as f:
f.write(
GRAFANA_INI_TEMPLATE.format(
grafana_provisioning_folder=grafana_prov_folder_with_latest_session
)
)

# Overwrite grafana's dashboard provisioning directory based on env var
dashboard_provisioning_path = os.path.join(
grafana_config_output_path, "provisioning", "dashboards"
grafana_provisioning_folder, "dashboards"
)
os.makedirs(
dashboard_provisioning_path,
Expand All @@ -219,9 +244,7 @@ def _create_default_grafana_configs(self):
prometheus_host = os.environ.get(
PROMETHEUS_HOST_ENV_VAR, DEFAULT_PROMETHEUS_HOST
)
data_sources_path = os.path.join(
grafana_config_output_path, "provisioning", "datasources"
)
data_sources_path = os.path.join(grafana_provisioning_folder, "datasources")
os.makedirs(
data_sources_path,
exist_ok=True,
Expand Down Expand Up @@ -297,14 +320,26 @@ def _create_default_prometheus_configs(self):
self._metrics_root, "prometheus", "prometheus.yml"
)

# Copy default prometheus configurations
# Generate the default prometheus configurations
if os.path.exists(prometheus_config_output_path):
os.remove(prometheus_config_output_path)
os.makedirs(os.path.dirname(prometheus_config_output_path), exist_ok=True)
# Currently Ray directly copies this file without modifying it at runtime.
# If Ray ever modifies this file at runtime, please ensure start_prometheus
# in install_and_start_prometheus.py is updated to reload the config file.
shutil.copy(PROMETHEUS_CONFIG_INPUT_PATH, prometheus_config_output_path)

# This code generates the Prometheus config based on the custom temporary root
# path set by the user at Ray cluster start up (via --temp-dir). In contrast,
# start_prometheus in install_and_start_prometheus.py uses a hardcoded
# Prometheus config at PROMETHEUS_CONFIG_INPUT_PATH that always uses "/tmp/ray".
# Other than the root path, the config file generated here is identical to that
# hardcoded config file.
prom_discovery_file_path = os.path.join(
self._dashboard_head.temp_dir, PROMETHEUS_SERVICE_DISCOVERY_FILE
)
with open(prometheus_config_output_path, "w") as f:
f.write(
PROMETHEUS_YML_TEMPLATE.format(
prom_metrics_service_discovery_file_path=prom_discovery_file_path
)
)

@dashboard_utils.async_loop_forever(METRICS_RECORD_INTERVAL_S)
async def record_dashboard_metrics(self):
Expand Down
49 changes: 49 additions & 0 deletions python/ray/dashboard/modules/metrics/templates.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
GRAFANA_INI_TEMPLATE = """
[security]
allow_embedding = true
[auth.anonymous]
enabled = true
org_name = Main Org.
org_role = Viewer
[paths]
provisioning = {grafana_provisioning_folder}
"""

DASHBOARD_PROVISIONING_TEMPLATE = """
apiVersion: 1
providers:
- name: Ray # Default dashboards provided by OSS ray
folder: Ray
type: file
options:
path: {dashboard_output_folder}
"""

GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1
datasources:
- name: {prometheus_name}
url: {prometheus_host}
type: prometheus
isDefault: true
access: proxy
"""

PROMETHEUS_YML_TEMPLATE = """# my global config
global:
scrape_interval: 10s # Set the scrape interval to every 10 seconds. Default is every \
1 minute.
evaluation_interval: 10s # Evaluate rules every 10 seconds. The default is every 1 \
minute.
# scrape_timeout is set to the global default (10s).
scrape_configs:
# Scrape from each ray node as defined in the service_discovery.json provided by ray.
- job_name: 'ray'
file_sd_configs:
- files:
- '{prom_metrics_service_discovery_file_path}'
"""
15 changes: 15 additions & 0 deletions python/ray/dashboard/modules/tests/test_metrics_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,9 @@

import pytest

from ray.dashboard.consts import PROMETHEUS_CONFIG_INPUT_PATH
from ray.dashboard.modules.metrics import install_and_start_prometheus
from ray.dashboard.modules.metrics.templates import PROMETHEUS_YML_TEMPLATE


@pytest.mark.parametrize(
Expand Down Expand Up @@ -38,5 +40,18 @@ def test_e2e(capsys):
subprocess.run(["kill", str(pid)])


def test_prometheus_config_content():
# Test to make sure the content in the hardcoded file
# (python/ray/dashboard/modules/metrics/export/prometheus/prometheus.yml) will
# always be the same as the template (templates.py) used to generate prometheus
# config file when ray startup
PROM_DISCOVERY_FILE_PATH = "/tmp/ray/prom_metrics_service_discovery.json"
template_content = PROMETHEUS_YML_TEMPLATE.format(
prom_metrics_service_discovery_file_path=PROM_DISCOVERY_FILE_PATH
)
with open(PROMETHEUS_CONFIG_INPUT_PATH) as f:
assert f.read() == template_content


if __name__ == "__main__":
sys.exit(pytest.main(["-v", __file__]))
Loading

0 comments on commit 1740d4f

Please sign in to comment.