Skip to content

Commit

Permalink
master merge
Browse files Browse the repository at this point in the history
  • Loading branch information
ujjawal-khare committed Oct 15, 2024
2 parents 328c7e0 + f04ddd6 commit 29e0864
Show file tree
Hide file tree
Showing 11 changed files with 175 additions and 73 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
4 changes: 2 additions & 2 deletions doc/source/ray-contribute/development.rst
Original file line number Diff line number Diff line change
Expand Up @@ -92,8 +92,8 @@ RLlib, Tune, Autoscaler, and most Python files do not require you to build and c

.. code-block:: shell
# For example, for Python 3.8:
pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-3.0.0.dev0-cp38-cp38-manylinux2014_x86_64.whl
# For example, for Python 3.9:
pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-3.0.0.dev0-cp39-cp39-manylinux2014_x86_64.whl
4. Replace Python files in the installed package with your local editable copy. We provide a simple script to help you do this: ``python python/ray/setup-dev.py``. Running the script will remove the ``ray/tune``, ``ray/rllib``, ``ray/autoscaler`` dir (among other directories) bundled with the ``ray`` pip package, and replace them with links to your local code. This way, changing files in your git clone will directly affect the behavior of your installed Ray.

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

This file was deleted.

This file was deleted.

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 29e0864

Please sign in to comment.