Skip to content

Commit

Permalink
Revert "[Ray Core] Fix the Paths in the Generated Monitoring Configs …
Browse files Browse the repository at this point in the history
…to Consider temp-dir" (#48050)

Reverts #47871
  • Loading branch information
can-anyscale authored Oct 16, 2024
1 parent 37e2449 commit 8bd6a9a
Show file tree
Hide file tree
Showing 10 changed files with 71 additions and 173 deletions.
14 changes: 2 additions & 12 deletions doc/source/cluster/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,6 @@ 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 @@ -82,7 +76,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`. 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`
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`.

```yaml
global:
Expand All @@ -94,17 +88,13 @@ scrape_configs:
- job_name: 'ray'
file_sd_configs:
- files:
- '/tmp/ray/prom_metrics_service_discovery.json' # or '${your_temp_path}/prom_metrics_service_discovery.json' if --temp-dir is specified
- '/tmp/ray/prom_metrics_service_discovery.json'
```
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: 10 additions & 0 deletions python/ray/dashboard/modules/metrics/export/grafana/grafana.ini
Original file line number Diff line number Diff line change
@@ -0,0 +1,10 @@
[security]
allow_embedding = true

[auth.anonymous]
enabled = true
org_name = Main Org.
org_role = Viewer

[paths]
provisioning = /tmp/ray/session_latest/metrics/grafana/provisioning
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,9 @@
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
@@ -0,0 +1,10 @@
DASHBOARD_PROVISIONING_TEMPLATE = """
apiVersion: 1
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
@@ -0,0 +1,9 @@
GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1
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,13 +74,12 @@ def install_prometheus(file_path):

def start_prometheus(prometheus_dir):

# 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.

# 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.
config_file = Path(PROMETHEUS_CONFIG_INPUT_PATH)

if not config_file.exists():
Expand Down
91 changes: 28 additions & 63 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 (
PROMETHEUS_SERVICE_DISCOVERY_FILE,
SESSION_LATEST,
env_integer,
)
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
from ray.dashboard.consts import (
AVAILABLE_COMPONENT_NAMES_FOR_METRICS,
METRICS_INPUT_ROOT,
PROMETHEUS_CONFIG_INPUT_PATH,
)
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.templates import (
from ray.dashboard.modules.metrics.grafana_dashboard_provisioning_template 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,6 +54,7 @@
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 @@ -74,22 +75,13 @@ 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
)
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"
)
grafana_config_output_path = os.path.join(self._metrics_root, "grafana")
self._grafana_dashboard_output_dir = os.environ.get(
GRAFANA_DASHBOARD_OUTPUT_DIR_ENV_VAR,
os.path.join(self._grafana_config_output_path, "dashboards"),
os.path.join(grafana_config_output_path, "dashboards"),
)

self._prometheus_name = os.environ.get(
Expand Down Expand Up @@ -194,34 +186,17 @@ def _create_default_grafana_configs(self):
"""
Creates the grafana configurations that are by default provided by Ray.
"""
# 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
)
)
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)

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

# Generate the default prometheus configurations
# Copy 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)

# 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
)
)
# 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)

@dashboard_utils.async_loop_forever(METRICS_RECORD_INTERVAL_S)
async def record_dashboard_metrics(self):
Expand Down
49 changes: 0 additions & 49 deletions python/ray/dashboard/modules/metrics/templates.py

This file was deleted.

15 changes: 0 additions & 15 deletions python/ray/dashboard/modules/tests/test_metrics_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,9 +3,7 @@

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 @@ -40,18 +38,5 @@ 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__]))
30 changes: 3 additions & 27 deletions python/ray/tests/test_metrics_head.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,53 +13,29 @@
SERVE_GRAFANA_PANELS,
)
from ray.tests.conftest import _ray_start
from ray._private.ray_constants import SESSION_LATEST


logger = logging.getLogger(__name__)


@pytest.mark.parametrize(
"is_temp_dir_set, temp_dir_val", [(False, ""), (True, "/tmp/test-temp-dir")]
)
def test_metrics_folder_and_content(is_temp_dir_set, temp_dir_val):
def test_metrics_folder():
"""
Tests that the default dashboard files get created. It also verifies paths in the
dashboard config files are set correctly.
It checks both the default case and the case where the _temp_dir is specified.
Tests that the default dashboard files get created.
"""
with _ray_start(
include_dashboard=True, _temp_dir=temp_dir_val if is_temp_dir_set else None
) as context:
with _ray_start(include_dashboard=True) as context:
session_dir = context["session_dir"]
temp_dir = temp_dir_val if is_temp_dir_set else "/tmp/ray"
assert os.path.exists(
f"{session_dir}/metrics/grafana/provisioning/dashboards/default.yml"
)
with open(
f"{session_dir}/metrics/grafana/provisioning/dashboards/default.yml", "r"
) as f:
assert f"path: {session_dir}/metrics/grafana/dashboards" in f.read()

assert os.path.exists(
f"{session_dir}/metrics/grafana/dashboards"
"/default_grafana_dashboard.json"
)
assert os.path.exists(
f"{session_dir}/metrics/grafana/provisioning/datasources/default.yml"
)

assert os.path.exists(f"{session_dir}/metrics/grafana/grafana.ini")
with open(f"{session_dir}/metrics/grafana/grafana.ini", "r") as f:
assert (
"provisioning = "
f"{temp_dir}/{SESSION_LATEST}/metrics/grafana/provisioning" in f.read()
)

assert os.path.exists(f"{session_dir}/metrics/prometheus/prometheus.yml")
with open(f"{session_dir}/metrics/prometheus/prometheus.yml", "r") as f:
assert f"- '{temp_dir}/prom_metrics_service_discovery.json'" in f.read()


@pytest.fixture
Expand Down

0 comments on commit 8bd6a9a

Please sign in to comment.