From aab97f1a01188080a70cb7816315123a8d151824 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Thu, 11 May 2023 18:45:05 -0700 Subject: [PATCH 01/29] moving log persistence content to clusters Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 17 ++++++---- .../cluster/kubernetes/user-guides/logging.md | 2 +- doc/source/ray-observability/ray-metrics.rst | 33 ------------------- 3 files changed, 11 insertions(+), 41 deletions(-) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 39c494d142cf8..acb88e19639bd 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -344,13 +344,13 @@ parts: - file: cluster/kubernetes/getting-started - file: cluster/kubernetes/user-guides sections: - - file: cluster/kubernetes/user-guides/k8s-cluster-setup.md - - file: cluster/kubernetes/user-guides/config.md - - file: cluster/kubernetes/user-guides/configuring-autoscaling.md - - file: cluster/kubernetes/user-guides/logging.md - - file: cluster/kubernetes/user-guides/gpu.md - - file: cluster/kubernetes/user-guides/experimental.md - - file: cluster/kubernetes/user-guides/static-ray-cluster-without-kuberay.md + - file: cluster/kubernetes/user-guides/k8s-cluster-setup + - file: cluster/kubernetes/user-guides/config + - file: cluster/kubernetes/user-guides/configuring-autoscaling + - file: cluster/kubernetes/user-guides/logging + - file: cluster/kubernetes/user-guides/gpu + - file: cluster/kubernetes/user-guides/experimental + - file: cluster/kubernetes/user-guides/static-ray-cluster-without-kuberay - file: cluster/kubernetes/examples sections: - file: cluster/kubernetes/examples/ml-example.md @@ -366,6 +366,7 @@ parts: - file: cluster/vms/user-guides/launching-clusters/index - file: cluster/vms/user-guides/large-cluster-best-practices - file: cluster/vms/user-guides/configuring-autoscaling + - file: cluster/vms/user-guides/logging - file: cluster/vms/user-guides/community/index title: Community-supported Cluster Managers sections: @@ -396,6 +397,8 @@ parts: - file: ray-observability/user-guides/troubleshoot-apps/optimize-performance - file: ray-observability/user-guides/troubleshoot-apps/ray-debugging - file: ray-observability/user-guides/troubleshoot-apps/ray-core-profiling + - file: ray-observability/user-guides/configure-logging + - file: ray-observability/user-guides/add-app-metrics - file: ray-observability/user-guides/ray-tracing - file: ray-references/api diff --git a/doc/source/cluster/kubernetes/user-guides/logging.md b/doc/source/cluster/kubernetes/user-guides/logging.md index a02dbdc42dbe2..23550031cd24b 100644 --- a/doc/source/cluster/kubernetes/user-guides/logging.md +++ b/doc/source/cluster/kubernetes/user-guides/logging.md @@ -1,6 +1,6 @@ (kuberay-logging)= -# Logging +# Log Persistence This page provides tips on how to collect logs from Ray clusters running on Kubernetes. diff --git a/doc/source/ray-observability/ray-metrics.rst b/doc/source/ray-observability/ray-metrics.rst index bcedba3a3f778..7c8e4b4cb05c9 100644 --- a/doc/source/ray-observability/ray-metrics.rst +++ b/doc/source/ray-observability/ray-metrics.rst @@ -198,39 +198,6 @@ Ray guarantees all its internal state metrics are *eventually* consistent even i For the `ray_tasks` and `ray_actors` metrics, you should use sum queries to plot their outputs (e.g., ``sum(ray_tasks) by (Name, State)``). The reason for this is that Ray's task metrics are emitted from multiple distributed components. Hence, there are multiple metric points, including negative metric points, emitted from different processes that must be summed to produce the correct logical view of the distributed system. For example, for a single task submitted and executed, Ray may emit ``(submitter) SUBMITTED_TO_WORKER: 1, (executor) SUBMITTED_TO_WORKER: -1, (executor) RUNNING: 1``, which reduces to ``SUBMITTED_TO_WORKER: 0, RUNNING: 1`` after summation. -.. _application-level-metrics: - -Application-level Metrics -------------------------- -Ray provides a convenient API in :ref:`ray.util.metrics ` for defining and exporting custom metrics for visibility into your applications. -There are currently three metrics supported: Counter, Gauge, and Histogram. -These metrics correspond to the same `Prometheus metric types `_. -Below is a simple example of an actor that exports metrics using these APIs: - -.. literalinclude:: doc_code/metrics_example.py - :language: python - -While the script is running, the metrics will be exported to ``localhost:8080`` (this is the endpoint that Prometheus would be configured to scrape). -If you open this in the browser, you should see the following output: - -.. code-block:: none - - # HELP ray_request_latency Latencies of requests in ms. - # TYPE ray_request_latency histogram - ray_request_latency_bucket{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor",le="0.1"} 2.0 - ray_request_latency_bucket{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor",le="1.0"} 2.0 - ray_request_latency_bucket{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor",le="+Inf"} 2.0 - ray_request_latency_count{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} 2.0 - ray_request_latency_sum{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} 0.11992454528808594 - # HELP ray_curr_count Current count held by the actor. Goes up and down. - # TYPE ray_curr_count gauge - ray_curr_count{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} -15.0 - # HELP ray_num_requests_total Number of requests processed by the actor. - # TYPE ray_num_requests_total counter - ray_num_requests_total{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} 2.0 - -Please see :ref:`ray.util.metrics ` for more details. - Configurations -------------- From 3e11e1eec3c2ae3d8707d8eb21bb510800b160b4 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Thu, 11 May 2023 19:09:58 -0700 Subject: [PATCH 02/29] move redirect stdout and stderr; add and rename some files Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 14 +- .../cluster/kubernetes/user-guides/logging.md | 155 +++++++++++ doc/source/cluster/vms/user-guides/logging.md | 253 ++++++++++++++++++ .../user-guides/add-app-metrics.rst | 33 +++ .../user-guides/configure-logging.rst | 94 +++++++ 5 files changed, 542 insertions(+), 7 deletions(-) create mode 100644 doc/source/cluster/vms/user-guides/logging.md create mode 100644 doc/source/ray-observability/user-guides/add-app-metrics.rst create mode 100644 doc/source/ray-observability/user-guides/configure-logging.rst diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index acb88e19639bd..2291374b94eca 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -344,13 +344,13 @@ parts: - file: cluster/kubernetes/getting-started - file: cluster/kubernetes/user-guides sections: - - file: cluster/kubernetes/user-guides/k8s-cluster-setup - - file: cluster/kubernetes/user-guides/config - - file: cluster/kubernetes/user-guides/configuring-autoscaling - - file: cluster/kubernetes/user-guides/logging - - file: cluster/kubernetes/user-guides/gpu - - file: cluster/kubernetes/user-guides/experimental - - file: cluster/kubernetes/user-guides/static-ray-cluster-without-kuberay + - file: cluster/kubernetes/user-guides/k8s-cluster-setup.md + - file: cluster/kubernetes/user-guides/config.md + - file: cluster/kubernetes/user-guides/configuring-autoscaling.md + - file: cluster/kubernetes/user-guides/logging.md + - file: cluster/kubernetes/user-guides/gpu.md + - file: cluster/kubernetes/user-guides/experimental.md + - file: cluster/kubernetes/user-guides/static-ray-cluster-without-kuberay.md - file: cluster/kubernetes/examples sections: - file: cluster/kubernetes/examples/ml-example.md diff --git a/doc/source/cluster/kubernetes/user-guides/logging.md b/doc/source/cluster/kubernetes/user-guides/logging.md index 23550031cd24b..79db274d4ccf5 100644 --- a/doc/source/cluster/kubernetes/user-guides/logging.md +++ b/doc/source/cluster/kubernetes/user-guides/logging.md @@ -143,3 +143,158 @@ kubectl logs raycluster-complete-logs-head-xxxxx -c fluentbit [Promtail]: https://grafana.com/docs/loki/latest/clients/promtail/ [KubDoc]: https://kubernetes.io/docs/concepts/cluster-administration/logging/ [ConfigLink]: https://raw.githubusercontent.com/ray-project/ray/releases/2.4.0/doc/source/cluster/kubernetes/configs/ray-cluster.log.yaml + +## How to set up loggers + +When using Ray, all of the tasks and actors are executed remotely in Ray's worker processes. +Since Python logger module creates a singleton logger per process, loggers should be configured on per task/actor basis. + +.. note:: + + To stream logs to a driver, they should be flushed to stdout and stderr. + +.. code-block:: python + + import ray + import logging + # Initiate a driver. + ray.init() + + @ray.remote + class Actor: + def __init__(self): + # Basic config automatically configures logs to + # be streamed to stdout and stderr. + # Set the severity to INFO so that info logs are printed to stdout. + logging.basicConfig(level=logging.INFO) + + def log(self, msg): + logging.info(msg) + + actor = Actor.remote() + ray.get(actor.log.remote("A log message for an actor.")) + + @ray.remote + def f(msg): + logging.basicConfig(level=logging.INFO) + logging.info(msg) + + ray.get(f.remote("A log message for a task")) + +.. code-block:: bash + + (pid=95193) INFO:root:A log message for a task + (pid=95192) INFO:root:A log message for an actor. + +## How to use structured logging + +The metadata of tasks or actors may be obtained by Ray's :ref:`runtime_context APIs `. +Runtime context APIs help you to add metadata to your logging messages, making your logs more structured. + +.. code-block:: python + + import ray + # Initiate a driver. + ray.init() + + @ray.remote + def task(): + print(f"task_id: {ray.get_runtime_context().task_id}") + + ray.get(task.remote()) + +.. code-block:: bash + + (pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) + +## Logging directory structure + +By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. + +.. note:: + + The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. + +A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. + +Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/stderr and ``.err`` is logs from stderr. The backward compatibility of log directories is not maintained. + +- ``dashboard.[log|err]``: A log file of a Ray dashboard. ``log.`` file contains logs generated from the dashboard's logger. ``.err`` file contains stdout and stderr printed from the dashboard. They are usually empty except when the dashboard crashes unexpectedly. +- ``dashboard_agent.log``: Every Ray node has one dashboard agent. This is a log file of the agent. +- ``gcs_server.[out|err]``: The GCS server is a stateless server that manages Ray cluster metadata. It exists only in the head node. +- ``io-worker-[worker_id]-[pid].[out|err]``: Ray creates IO workers to spill/restore objects to external storage by default from Ray 1.3+. This is a log file of IO workers. +- ``job-driver-[submission_id].log``: The stdout of a job submitted via the :ref:`Ray Jobs API `. +- ``log_monitor.[log|err]``: The log monitor is in charge of streaming logs to the driver. ``log.`` file contains logs generated from the log monitor's logger. ``.err`` file contains the stdout and stderr printed from the log monitor. They are usually empty except when the log monitor crashes unexpectedly. +- ``monitor.[out|err]``: Stdout and stderr of a cluster launcher. +- ``monitor.log``: Ray's cluster launcher is operated with a monitor process. It also manages the autoscaler. +- ``plasma_store.[out|err]``: Deprecated. +- ``python-core-driver-[worker_id]_[pid].log``: Ray drivers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. +- ``python-core-worker-[worker_id]_[pid].log``: Ray workers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. +- ``raylet.[out|err]``: A log file of raylets. +- ``redis-shard_[shard_index].[out|err]``: Redis shard log files. +- ``redis.[out|err]``: Redis log files. +- ``runtime_env_agent.log``: Every Ray node has one agent that manages :ref:`runtime environment ` creation, deletion and caching. + This is the log file of the agent containing logs of create/delete requests and cache hits and misses. + For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). +- ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. +- ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. +- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver.- + +## Redirecting Ray logs to stderr + +By default, Ray logs are written to files under the ``/tmp/ray/session_*/logs`` directory. If you wish to redirect all internal Ray logging and your own logging within tasks/actors to stderr of the host nodes, you can do so by ensuring that the ``RAY_LOG_TO_STDERR=1`` environment variable is set on the driver and on all Ray nodes. This is very useful if you are using a log aggregator that needs log records to be written to stderr in order for them to be captured. + +Redirecting logging to stderr will also cause a ``({component})`` prefix, e.g. ``(raylet)``, to be added to each of the log record messages. + +.. code-block:: bash + + [2022-01-24 19:42:02,978 I 1829336 1829336] (gcs_server) grpc_server.cc:103: GcsServer server started, listening on port 50009. + [2022-01-24 19:42:06,696 I 1829415 1829415] (raylet) grpc_server.cc:103: ObjectManager server started, listening on port 40545. + 2022-01-24 19:42:05,087 INFO (dashboard) dashboard.py:95 -- Setup static dir for dashboard: /mnt/data/workspace/ray/python/ray/dashboard/client/build + 2022-01-24 19:42:07,500 INFO (dashboard_agent) agent.py:105 -- Dashboard agent grpc address: 0.0.0.0:49228 + +This should make it easier to filter the stderr stream of logs down to the component of interest. Note that multi-line log records will **not** have this component marker at the beginning of each line. + +When running a local Ray cluster, this environment variable should be set before starting the local cluster: + +.. code-block:: python + + os.environ["RAY_LOG_TO_STDERR"] = "1" + ray.init() + +When starting a local cluster via the CLI or when starting nodes in a multi-node Ray cluster, this environment variable should be set before starting up each node: + +.. code-block:: bash + + env RAY_LOG_TO_STDERR=1 ray start + +If using the Ray cluster launcher, you would specify this environment variable in the Ray start commands: + +.. code-block:: bash + + head_start_ray_commands: + - ray stop + - env RAY_LOG_TO_STDERR=1 ray start --head --port=6379 --object-manager-port=8076 --autoscaling-config=~/ray_bootstrap_config.yaml + + worker_start_ray_commands: + - ray stop + - env RAY_LOG_TO_STDERR=1 ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076 + +When connecting to the cluster, be sure to set the environment variable before connecting: + +.. code-block:: python + + os.environ["RAY_LOG_TO_STDERR"] = "1" + ray.init(address="auto") + +## Log rotation + +Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). + +By default, logs are rotating when it reaches to 512MB (maxBytes), and there could be up to 5 backup files (backupCount). Indexes are appended to all backup files (e.g., `raylet.out.1`) +If you'd like to change the log rotation configuration, you can do it by specifying environment variables. For example, + +.. code-block:: bash + + RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. + RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. diff --git a/doc/source/cluster/vms/user-guides/logging.md b/doc/source/cluster/vms/user-guides/logging.md new file mode 100644 index 0000000000000..2052f1cf26963 --- /dev/null +++ b/doc/source/cluster/vms/user-guides/logging.md @@ -0,0 +1,253 @@ +(vms-logging)= + +# Log Persistence + +This page provides tips on how to collect logs from +Ray clusters running on Kubernetes. + +:::{tip} +Skip to {ref}`the deployment instructions ` +for a sample configuration showing how to extract logs from a Ray pod. +::: + +## The Ray log directory +Each Ray pod runs several component processes, such as the Raylet, object manager, dashboard agent, etc. +These components log to files in the directory `/tmp/ray/session_latest/logs` in the pod's file system. +Extracting and persisting these logs requires some setup. + +## Log processing tools +There are a number of log processing tools available within the Kubernetes +ecosystem. This page will shows how to extract Ray logs using [Fluent Bit][FluentBit]. +Other popular tools include [Fluentd][Fluentd], [Filebeat][Filebeat], and [Promtail][Promtail]. + +## Log collection strategies +We mention two strategies for collecting logs written to a pod's filesystem, +**sidecar containers** and **daemonsets**. You can read more about these logging +patterns in the [Kubernetes documentation][KubDoc]. + +### Sidecar containers +We will provide an {ref}`example ` of the sidecar strategy in this guide. +You can process logs by configuring a log-processing sidecar +for each Ray pod. Ray containers should be configured to share the `/tmp/ray` +directory with the logging sidecar via a volume mount. + +You can configure the sidecar to do either of the following: +* Stream Ray logs to the sidecar's stdout. +* Export logs to an external service. + +### Daemonset +Alternatively, it is possible to collect logs at the Kubernetes node level. +To do this, one deploys a log-processing daemonset onto the Kubernetes cluster's +nodes. With this strategy, it is key to mount +the Ray container's `/tmp/ray` directory to the relevant `hostPath`. + +(kuberay-fluentbit)= +# Setting up logging sidecars with Fluent Bit. +In this section, we give an example of how to set up log-emitting +[Fluent Bit][FluentBit] sidecars for Ray pods. + +See the full config for a single-pod RayCluster with a logging sidecar [here][ConfigLink]. +We now discuss this configuration and show how to deploy it. + +## Configure log processing +The first step is to create a ConfigMap with configuration +for Fluent Bit. + +Here is a minimal ConfigMap which tells a Fluent Bit sidecar to +* Tail Ray logs. +* Output the logs to the container's stdout. +```{literalinclude} ../configs/ray-cluster.log.yaml +:language: yaml +:start-after: Fluent Bit ConfigMap +:end-before: --- +``` +A few notes on the above config: +- In addition to streaming logs to stdout, you can use an [OUTPUT] clause to export logs to any + [storage backend][FluentBitStorage] supported by Fluent Bit. +- The `Path_Key true` line above ensures that file names are included in the log records + emitted by Fluent Bit. +- The `Refresh_Interval 5` line asks Fluent Bit to refresh the list of files + in the log directory once per 5 seconds, rather than the default 60. + The reason is that the directory `/tmp/ray/session_latest/logs/` does not exist + initially (Ray must create it first). Setting the `Refresh_Interval` low allows us to see logs + in the Fluent Bit container's stdout sooner. + + +## Add logging sidecars to your RayCluster CR. + +### Add log and config volumes. +For each pod template in our RayCluster CR, we +need to add two volumes: One volume for Ray's logs +and another volume to store Fluent Bit configuration from the ConfigMap +applied above. +```{literalinclude} ../configs/ray-cluster.log.yaml +:language: yaml +:start-after: Log and config volumes +``` + +### Mount the Ray log directory +Add the following volume mount to the Ray container's configuration. +```{literalinclude} ../configs/ray-cluster.log.yaml +:language: yaml +:start-after: Share logs with Fluent Bit +:end-before: Fluent Bit sidecar +``` + +### Add the Fluent Bit sidecar +Finally, add the Fluent Bit sidecar container to each Ray pod config +in your RayCluster CR. +```{literalinclude} ../configs/ray-cluster.log.yaml +:language: yaml +:start-after: Fluent Bit sidecar +:end-before: Log and config volumes +``` +Mounting the `ray-logs` volume gives the sidecar container access to Ray's logs. +The `fluentbit-config` volume gives the sidecar access to logging configuration. + +### Putting everything together +Putting all of the above elements together, we have the following yaml configuration +for a single-pod RayCluster will a log-processing sidecar. +```{literalinclude} ../configs/ray-cluster.log.yaml +:language: yaml +``` + +## Deploying a RayCluster with logging CR. +(kuberay-logging-tldr)= +Now, we will see how to deploy the configuration described above. + +Deploy the KubeRay Operator if you haven't yet. +Refer to the {ref}`Getting Started guide ` +for instructions on this step. + +Now, run the following commands to deploy the Fluent Bit ConfigMap and a single-pod RayCluster with +a Fluent Bit sidecar. +```shell +kubectl apply -f https://raw.githubusercontent.com/ray-project/ray/releases/2.4.0/doc/source/cluster/kubernetes/configs/ray-cluster.log.yaml +``` + +Determine the Ray pod's name with +```shell +kubectl get pod | grep raycluster-complete-logs +``` + +Examine the FluentBit sidecar's STDOUT to see logs for Ray's component processes. +```shell +# Substitute the name of your Ray pod. +kubectl logs raycluster-complete-logs-head-xxxxx -c fluentbit +``` + +[FluentBit]: https://docs.fluentbit.io/manual +[FluentBitStorage]: https://docs.fluentbit.io/manual +[Filebeat]: https://www.elastic.co/guide/en/beats/filebeat/7.17/index.html +[Fluentd]: https://docs.fluentd.org/ +[Promtail]: https://grafana.com/docs/loki/latest/clients/promtail/ +[KubDoc]: https://kubernetes.io/docs/concepts/cluster-administration/logging/ +[ConfigLink]: https://raw.githubusercontent.com/ray-project/ray/releases/2.4.0/doc/source/cluster/kubernetes/configs/ray-cluster.log.yaml + +## How to set up loggers + +When using Ray, all of the tasks and actors are executed remotely in Ray's worker processes. +Since Python logger module creates a singleton logger per process, loggers should be configured on per task/actor basis. + +.. note:: + + To stream logs to a driver, they should be flushed to stdout and stderr. + +.. code-block:: python + + import ray + import logging + # Initiate a driver. + ray.init() + + @ray.remote + class Actor: + def __init__(self): + # Basic config automatically configures logs to + # be streamed to stdout and stderr. + # Set the severity to INFO so that info logs are printed to stdout. + logging.basicConfig(level=logging.INFO) + + def log(self, msg): + logging.info(msg) + + actor = Actor.remote() + ray.get(actor.log.remote("A log message for an actor.")) + + @ray.remote + def f(msg): + logging.basicConfig(level=logging.INFO) + logging.info(msg) + + ray.get(f.remote("A log message for a task")) + +.. code-block:: bash + + (pid=95193) INFO:root:A log message for a task + (pid=95192) INFO:root:A log message for an actor. + +## How to use structured logging + +The metadata of tasks or actors may be obtained by Ray's :ref:`runtime_context APIs `. +Runtime context APIs help you to add metadata to your logging messages, making your logs more structured. + +.. code-block:: python + + import ray + # Initiate a driver. + ray.init() + + @ray.remote + def task(): + print(f"task_id: {ray.get_runtime_context().task_id}") + + ray.get(task.remote()) + +.. code-block:: bash + + (pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) + +## Logging directory structure + +By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. + +.. note:: + + The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. + +A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. + +Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/stderr and ``.err`` is logs from stderr. The backward compatibility of log directories is not maintained. + +- ``dashboard.[log|err]``: A log file of a Ray dashboard. ``log.`` file contains logs generated from the dashboard's logger. ``.err`` file contains stdout and stderr printed from the dashboard. They are usually empty except when the dashboard crashes unexpectedly. +- ``dashboard_agent.log``: Every Ray node has one dashboard agent. This is a log file of the agent. +- ``gcs_server.[out|err]``: The GCS server is a stateless server that manages Ray cluster metadata. It exists only in the head node. +- ``io-worker-[worker_id]-[pid].[out|err]``: Ray creates IO workers to spill/restore objects to external storage by default from Ray 1.3+. This is a log file of IO workers. +- ``job-driver-[submission_id].log``: The stdout of a job submitted via the :ref:`Ray Jobs API `. +- ``log_monitor.[log|err]``: The log monitor is in charge of streaming logs to the driver. ``log.`` file contains logs generated from the log monitor's logger. ``.err`` file contains the stdout and stderr printed from the log monitor. They are usually empty except when the log monitor crashes unexpectedly. +- ``monitor.[out|err]``: Stdout and stderr of a cluster launcher. +- ``monitor.log``: Ray's cluster launcher is operated with a monitor process. It also manages the autoscaler. +- ``plasma_store.[out|err]``: Deprecated. +- ``python-core-driver-[worker_id]_[pid].log``: Ray drivers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. +- ``python-core-worker-[worker_id]_[pid].log``: Ray workers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. +- ``raylet.[out|err]``: A log file of raylets. +- ``redis-shard_[shard_index].[out|err]``: Redis shard log files. +- ``redis.[out|err]``: Redis log files. +- ``runtime_env_agent.log``: Every Ray node has one agent that manages :ref:`runtime environment ` creation, deletion and caching. + This is the log file of the agent containing logs of create/delete requests and cache hits and misses. + For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). +- ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. +- ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. +- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver.- + +## Log rotation + +Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). + +By default, logs are rotating when it reaches to 512MB (maxBytes), and there could be up to 5 backup files (backupCount). Indexes are appended to all backup files (e.g., `raylet.out.1`) +If you'd like to change the log rotation configuration, you can do it by specifying environment variables. For example, + +.. code-block:: bash + + RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. + RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. diff --git a/doc/source/ray-observability/user-guides/add-app-metrics.rst b/doc/source/ray-observability/user-guides/add-app-metrics.rst new file mode 100644 index 0000000000000..b82946ed1544c --- /dev/null +++ b/doc/source/ray-observability/user-guides/add-app-metrics.rst @@ -0,0 +1,33 @@ +.. _application-level-metrics: + +Adding Application-Level Metrics +-------------------------------- + +Ray provides a convenient API in :ref:`ray.util.metrics ` for defining and exporting custom metrics for visibility into your applications. +There are currently three metrics supported: Counter, Gauge, and Histogram. +These metrics correspond to the same `Prometheus metric types `_. +Below is a simple example of an actor that exports metrics using these APIs: + +.. literalinclude:: doc_code/metrics_example.py + :language: python + +While the script is running, the metrics are exported to ``localhost:8080`` (this is the endpoint that Prometheus would be configured to scrape). +If you open this in the browser, you should see the following output: + +.. code-block:: none + + # HELP ray_request_latency Latencies of requests in ms. + # TYPE ray_request_latency histogram + ray_request_latency_bucket{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor",le="0.1"} 2.0 + ray_request_latency_bucket{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor",le="1.0"} 2.0 + ray_request_latency_bucket{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor",le="+Inf"} 2.0 + ray_request_latency_count{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} 2.0 + ray_request_latency_sum{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} 0.11992454528808594 + # HELP ray_curr_count Current count held by the actor. Goes up and down. + # TYPE ray_curr_count gauge + ray_curr_count{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} -15.0 + # HELP ray_num_requests_total Number of requests processed by the actor. + # TYPE ray_num_requests_total counter + ray_num_requests_total{Component="core_worker",Version="3.0.0.dev0",actor_name="my_actor"} 2.0 + +Please see :ref:`ray.util.metrics ` for more details. diff --git a/doc/source/ray-observability/user-guides/configure-logging.rst b/doc/source/ray-observability/user-guides/configure-logging.rst new file mode 100644 index 0000000000000..fb166093350d8 --- /dev/null +++ b/doc/source/ray-observability/user-guides/configure-logging.rst @@ -0,0 +1,94 @@ +.. _configure-logging: + +Configuring Logging +=================== + +This document explains Ray's logging system and best practices. + +Disabling logging to the driver +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +In large scale runs, it may be undesirable to route all worker logs to the driver. You can disable this feature by setting ``log_to_driver=False`` in Ray init: + +.. code-block:: python + + import ray + + # Task and actor logs will not be copied to the driver stdout. + ray.init(log_to_driver=False) + +Log deduplication +~~~~~~~~~~~~~~~~~ + +By default, Ray will deduplicate logs that appear redundantly across multiple processes. The first instance of each log message will always be immediately printed. However, subsequent log messages of the same pattern (ignoring words with numeric components) will be buffered for up to five seconds and printed in batch. For example, for the following code snippet: + +.. code-block:: python + + import ray + import random + + @ray.remote + def task(): + print("Hello there, I am a task", random.random()) + + ray.get([task.remote() for _ in range(100)]) + +The output will be as follows: + +.. code-block:: bash + + 2023-03-27 15:08:34,195 INFO worker.py:1603 -- Started a local Ray instance. View the dashboard at http://127.0.0.1:8265 + (task pid=534172) Hello there, I am a task 0.20583517821231412 + (task pid=534174) Hello there, I am a task 0.17536720316370757 [repeated 99x across cluster] (Ray deduplicates logs by default. Set RAY_DEDUP_LOGS=0 to disable log deduplication) + +This feature is especially useful when importing libraries such as `tensorflow` or `numpy`, which may emit many verbose warning messages when imported. You can configure this feature as follows: + +1. Set ``RAY_DEDUP_LOGS=0`` to disable this feature entirely. +2. Set ``RAY_DEDUP_LOGS_AGG_WINDOW_S=`` to change the agggregation window. +3. Set ``RAY_DEDUP_LOGS_ALLOW_REGEX=`` to specify log messages to never deduplicate. +4. Set ``RAY_DEDUP_LOGS_SKIP_REGEX=`` to specify log messages to skip printing. + + +Customizing Actor logs prefixes +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +It is often useful to distinguish between log messages from different actors. For example, suppose you have a large number of worker actors. In this case, you may want to be able to easily see the index of the actor that logged a particular message. This can be achieved by defining the `__repr__ `__ method for an actor class. When defined, the actor repr will be used in place of the actor name. For example: + +.. literalinclude:: /ray-core/doc_code/actor-repr.py + +This produces the following output: + +.. code-block:: bash + + (MyActor(index=2) pid=482120) hello there + (MyActor(index=1) pid=482119) hello there + +Coloring Actor log prefixes +~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +By default Ray prints Actor logs prefixes in light blue: +Users may instead activate multi-color prefixes by setting the environment variable ``RAY_COLOR_PREFIX=1``. +This will index into an array of colors modulo the PID of each process. + +.. image:: ./images/coloring-actor-log-prefixes.png + :align: center + +Distributed progress bars (tqdm) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +When using `tqdm `__ in Ray remote tasks or actors, you may notice that the progress bar output is corrupted. To avoid this problem, you can use the Ray distributed tqdm implementation at ``ray.experimental.tqdm_ray``: + +.. literalinclude:: /ray-core/doc_code/tqdm.py + +This tqdm implementation works as follows: + +1. The ``tqdm_ray`` module translates TQDM calls into special json log messages written to worker stdout. +2. The Ray log monitor, instead of copying these log messages directly to the driver stdout, routes these messages to a tqdm singleton. +3. The tqdm singleton determines the positions of progress bars from various Ray tasks / actors, ensuring they don't collide or conflict with each other. + +Limitations: + +- Only a subset of tqdm functionality is supported. Refer to the ray_tqdm `implementation `__ for more details. +- Performance may be poor if there are more than a couple thousand updates per second (updates are not batched). + +By default, the builtin print will also be patched to use `ray.experimental.tqdm_ray.safe_print` when `tqdm_ray` is used. +This avoids progress bar corruption on driver print statements. To disable this, set `RAY_TQDM_PATCH_PRINT=0`. From 414a314179e5e6f4e421b01a230499ff758e5a0c Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Thu, 11 May 2023 21:29:36 -0700 Subject: [PATCH 03/29] fix notes and code blocks from rst to md syntax, fix heading hierarchies, copy edits Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 1 + .../cluster/kubernetes/user-guides/logging.md | 172 +++++----- doc/source/cluster/vms/user-guides/logging.md | 122 ++++--- .../monitoring-debugging.rst | 1 - doc/source/ray-observability/ray-logging.rst | 300 ------------------ .../user-guides/configure-logging.rst | 6 +- 6 files changed, 149 insertions(+), 453 deletions(-) delete mode 100644 doc/source/ray-observability/ray-logging.rst diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 2291374b94eca..350a55f587932 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -386,6 +386,7 @@ parts: - file: ray-observability/monitoring-debugging/monitoring-debugging title: "Monitoring and Debugging" sections: + - file: ray-observability/key-concepts - file: ray-observability/user-guides/index title: User Guides sections: diff --git a/doc/source/cluster/kubernetes/user-guides/logging.md b/doc/source/cluster/kubernetes/user-guides/logging.md index 79db274d4ccf5..4007fe5736c41 100644 --- a/doc/source/cluster/kubernetes/user-guides/logging.md +++ b/doc/source/cluster/kubernetes/user-guides/logging.md @@ -42,14 +42,14 @@ nodes. With this strategy, it is key to mount the Ray container's `/tmp/ray` directory to the relevant `hostPath`. (kuberay-fluentbit)= -# Setting up logging sidecars with Fluent Bit. +## Setting up logging sidecars with Fluent Bit In this section, we give an example of how to set up log-emitting [Fluent Bit][FluentBit] sidecars for Ray pods. See the full config for a single-pod RayCluster with a logging sidecar [here][ConfigLink]. We now discuss this configuration and show how to deploy it. -## Configure log processing +### Configuring log processing The first step is to create a ConfigMap with configuration for Fluent Bit. @@ -73,9 +73,9 @@ A few notes on the above config: in the Fluent Bit container's stdout sooner. -## Add logging sidecars to your RayCluster CR. +### Adding logging sidecars to your RayCluster CR -### Add log and config volumes. +#### Adding log and config volumes For each pod template in our RayCluster CR, we need to add two volumes: One volume for Ray's logs and another volume to store Fluent Bit configuration from the ConfigMap @@ -85,7 +85,7 @@ applied above. :start-after: Log and config volumes ``` -### Mount the Ray log directory +#### Mounting the Ray log directory Add the following volume mount to the Ray container's configuration. ```{literalinclude} ../configs/ray-cluster.log.yaml :language: yaml @@ -93,7 +93,7 @@ Add the following volume mount to the Ray container's configuration. :end-before: Fluent Bit sidecar ``` -### Add the Fluent Bit sidecar +#### Adding the Fluent Bit sidecar Finally, add the Fluent Bit sidecar container to each Ray pod config in your RayCluster CR. ```{literalinclude} ../configs/ray-cluster.log.yaml @@ -104,14 +104,14 @@ in your RayCluster CR. Mounting the `ray-logs` volume gives the sidecar container access to Ray's logs. The `fluentbit-config` volume gives the sidecar access to logging configuration. -### Putting everything together +#### Putting everything together Putting all of the above elements together, we have the following yaml configuration for a single-pod RayCluster will a log-processing sidecar. ```{literalinclude} ../configs/ray-cluster.log.yaml :language: yaml ``` -## Deploying a RayCluster with logging CR. +### Deploying a RayCluster with logging CR (kuberay-logging-tldr)= Now, we will see how to deploy the configuration described above. @@ -144,76 +144,74 @@ kubectl logs raycluster-complete-logs-head-xxxxx -c fluentbit [KubDoc]: https://kubernetes.io/docs/concepts/cluster-administration/logging/ [ConfigLink]: https://raw.githubusercontent.com/ray-project/ray/releases/2.4.0/doc/source/cluster/kubernetes/configs/ray-cluster.log.yaml -## How to set up loggers +## Setting up loggers When using Ray, all of the tasks and actors are executed remotely in Ray's worker processes. Since Python logger module creates a singleton logger per process, loggers should be configured on per task/actor basis. -.. note:: - - To stream logs to a driver, they should be flushed to stdout and stderr. - -.. code-block:: python - - import ray - import logging - # Initiate a driver. - ray.init() - - @ray.remote - class Actor: - def __init__(self): - # Basic config automatically configures logs to - # be streamed to stdout and stderr. - # Set the severity to INFO so that info logs are printed to stdout. - logging.basicConfig(level=logging.INFO) - - def log(self, msg): - logging.info(msg) - - actor = Actor.remote() - ray.get(actor.log.remote("A log message for an actor.")) +:::{note} +To stream logs to a driver, they should be flushed to stdout and stderr. +::: - @ray.remote - def f(msg): +```python +import ray +import logging +# Initiate a driver. +ray.init() + +@ray.remote +class Actor: + def __init__(self): + # Basic config automatically configures logs to + # be streamed to stdout and stderr. + # Set the severity to INFO so that info logs are printed to stdout. logging.basicConfig(level=logging.INFO) + + def log(self, msg): logging.info(msg) - ray.get(f.remote("A log message for a task")) +actor = Actor.remote() +ray.get(actor.log.remote("A log message for an actor.")) -.. code-block:: bash +@ray.remote +def f(msg): + logging.basicConfig(level=logging.INFO) + logging.info(msg) - (pid=95193) INFO:root:A log message for a task - (pid=95192) INFO:root:A log message for an actor. +ray.get(f.remote("A log message for a task")) +``` -## How to use structured logging +```bash +(pid=95193) INFO:root:A log message for a task +(pid=95192) INFO:root:A log message for an actor. +``` +## Using structured logging The metadata of tasks or actors may be obtained by Ray's :ref:`runtime_context APIs `. Runtime context APIs help you to add metadata to your logging messages, making your logs more structured. -.. code-block:: python - - import ray - # Initiate a driver. - ray.init() +```python +import ray +# Initiate a driver. +ray.init() - @ray.remote - def task(): - print(f"task_id: {ray.get_runtime_context().task_id}") + @ray.remote +def task(): + print(f"task_id: {ray.get_runtime_context().task_id}") - ray.get(task.remote()) - -.. code-block:: bash - - (pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) +ray.get(task.remote()) +``` +```bash +(pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) +``` ## Logging directory structure By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. -.. note:: - - The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. +:::{note} +The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. +::: A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. @@ -238,7 +236,7 @@ Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/std For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). - ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. - ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. -- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver.- +- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver. ## Redirecting Ray logs to stderr @@ -246,55 +244,55 @@ By default, Ray logs are written to files under the ``/tmp/ray/session_*/logs`` Redirecting logging to stderr will also cause a ``({component})`` prefix, e.g. ``(raylet)``, to be added to each of the log record messages. -.. code-block:: bash - - [2022-01-24 19:42:02,978 I 1829336 1829336] (gcs_server) grpc_server.cc:103: GcsServer server started, listening on port 50009. - [2022-01-24 19:42:06,696 I 1829415 1829415] (raylet) grpc_server.cc:103: ObjectManager server started, listening on port 40545. - 2022-01-24 19:42:05,087 INFO (dashboard) dashboard.py:95 -- Setup static dir for dashboard: /mnt/data/workspace/ray/python/ray/dashboard/client/build - 2022-01-24 19:42:07,500 INFO (dashboard_agent) agent.py:105 -- Dashboard agent grpc address: 0.0.0.0:49228 +```bash +[2022-01-24 19:42:02,978 I 1829336 1829336] (gcs_server) grpc_server.cc:103: GcsServer server started, listening on port 50009. +[2022-01-24 19:42:06,696 I 1829415 1829415] (raylet) grpc_server.cc:103: ObjectManager server started, listening on port 40545. +2022-01-24 19:42:05,087 INFO (dashboard) dashboard.py:95 -- Setup static dir for dashboard: /mnt/data/workspace/ray/python/ray/dashboard/client/build +2022-01-24 19:42:07,500 INFO (dashboard_agent) agent.py:105 -- Dashboard agent grpc address: 0.0.0.0:49228 +``` This should make it easier to filter the stderr stream of logs down to the component of interest. Note that multi-line log records will **not** have this component marker at the beginning of each line. When running a local Ray cluster, this environment variable should be set before starting the local cluster: -.. code-block:: python - - os.environ["RAY_LOG_TO_STDERR"] = "1" - ray.init() +```python +os.environ["RAY_LOG_TO_STDERR"] = "1" +ray.init() +``` When starting a local cluster via the CLI or when starting nodes in a multi-node Ray cluster, this environment variable should be set before starting up each node: -.. code-block:: bash - - env RAY_LOG_TO_STDERR=1 ray start +```bash +env RAY_LOG_TO_STDERR=1 ray start +``` If using the Ray cluster launcher, you would specify this environment variable in the Ray start commands: -.. code-block:: bash +```bash +head_start_ray_commands: + - ray stop + - env RAY_LOG_TO_STDERR=1 ray start --head --port=6379 --object-manager-port=8076 --autoscaling-config=~/ray_bootstrap_config.yaml - head_start_ray_commands: - - ray stop - - env RAY_LOG_TO_STDERR=1 ray start --head --port=6379 --object-manager-port=8076 --autoscaling-config=~/ray_bootstrap_config.yaml - - worker_start_ray_commands: - - ray stop - - env RAY_LOG_TO_STDERR=1 ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076 +worker_start_ray_commands: + - ray stop + - env RAY_LOG_TO_STDERR=1 ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076 +``` When connecting to the cluster, be sure to set the environment variable before connecting: -.. code-block:: python - - os.environ["RAY_LOG_TO_STDERR"] = "1" - ray.init(address="auto") +```python +os.environ["RAY_LOG_TO_STDERR"] = "1" +ray.init(address="auto") +``` -## Log rotation +## Rotating logs Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). By default, logs are rotating when it reaches to 512MB (maxBytes), and there could be up to 5 backup files (backupCount). Indexes are appended to all backup files (e.g., `raylet.out.1`) If you'd like to change the log rotation configuration, you can do it by specifying environment variables. For example, -.. code-block:: bash - - RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. - RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. +```bash +RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. +RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. +``` \ No newline at end of file diff --git a/doc/source/cluster/vms/user-guides/logging.md b/doc/source/cluster/vms/user-guides/logging.md index 2052f1cf26963..d94bcc771192d 100644 --- a/doc/source/cluster/vms/user-guides/logging.md +++ b/doc/source/cluster/vms/user-guides/logging.md @@ -42,14 +42,14 @@ nodes. With this strategy, it is key to mount the Ray container's `/tmp/ray` directory to the relevant `hostPath`. (kuberay-fluentbit)= -# Setting up logging sidecars with Fluent Bit. +## Setting up logging sidecars with Fluent Bit In this section, we give an example of how to set up log-emitting [Fluent Bit][FluentBit] sidecars for Ray pods. See the full config for a single-pod RayCluster with a logging sidecar [here][ConfigLink]. We now discuss this configuration and show how to deploy it. -## Configure log processing +### Configure log processing The first step is to create a ConfigMap with configuration for Fluent Bit. @@ -73,9 +73,9 @@ A few notes on the above config: in the Fluent Bit container's stdout sooner. -## Add logging sidecars to your RayCluster CR. +### Add logging sidecars to your RayCluster CR -### Add log and config volumes. +#### Add log and config volumes For each pod template in our RayCluster CR, we need to add two volumes: One volume for Ray's logs and another volume to store Fluent Bit configuration from the ConfigMap @@ -85,7 +85,7 @@ applied above. :start-after: Log and config volumes ``` -### Mount the Ray log directory +#### Mount the Ray log directory Add the following volume mount to the Ray container's configuration. ```{literalinclude} ../configs/ray-cluster.log.yaml :language: yaml @@ -93,7 +93,7 @@ Add the following volume mount to the Ray container's configuration. :end-before: Fluent Bit sidecar ``` -### Add the Fluent Bit sidecar +#### Add the Fluent Bit sidecar Finally, add the Fluent Bit sidecar container to each Ray pod config in your RayCluster CR. ```{literalinclude} ../configs/ray-cluster.log.yaml @@ -104,14 +104,14 @@ in your RayCluster CR. Mounting the `ray-logs` volume gives the sidecar container access to Ray's logs. The `fluentbit-config` volume gives the sidecar access to logging configuration. -### Putting everything together +#### Putting everything together Putting all of the above elements together, we have the following yaml configuration for a single-pod RayCluster will a log-processing sidecar. ```{literalinclude} ../configs/ray-cluster.log.yaml :language: yaml ``` -## Deploying a RayCluster with logging CR. +### Deploying a RayCluster with logging CR (kuberay-logging-tldr)= Now, we will see how to deploy the configuration described above. @@ -144,76 +144,74 @@ kubectl logs raycluster-complete-logs-head-xxxxx -c fluentbit [KubDoc]: https://kubernetes.io/docs/concepts/cluster-administration/logging/ [ConfigLink]: https://raw.githubusercontent.com/ray-project/ray/releases/2.4.0/doc/source/cluster/kubernetes/configs/ray-cluster.log.yaml -## How to set up loggers +## Setting up loggers When using Ray, all of the tasks and actors are executed remotely in Ray's worker processes. Since Python logger module creates a singleton logger per process, loggers should be configured on per task/actor basis. -.. note:: - - To stream logs to a driver, they should be flushed to stdout and stderr. - -.. code-block:: python - - import ray - import logging - # Initiate a driver. - ray.init() - - @ray.remote - class Actor: - def __init__(self): - # Basic config automatically configures logs to - # be streamed to stdout and stderr. - # Set the severity to INFO so that info logs are printed to stdout. - logging.basicConfig(level=logging.INFO) - - def log(self, msg): - logging.info(msg) - - actor = Actor.remote() - ray.get(actor.log.remote("A log message for an actor.")) +:::{note} +To stream logs to a driver, they should be flushed to stdout and stderr. +::: - @ray.remote - def f(msg): +```python +import ray +import logging +# Initiate a driver. +ray.init() + +@ray.remote +class Actor: + def __init__(self): + # Basic config automatically configures logs to + # be streamed to stdout and stderr. + # Set the severity to INFO so that info logs are printed to stdout. logging.basicConfig(level=logging.INFO) + + def log(self, msg): logging.info(msg) - ray.get(f.remote("A log message for a task")) +actor = Actor.remote() +ray.get(actor.log.remote("A log message for an actor.")) -.. code-block:: bash +@ray.remote +def f(msg): + logging.basicConfig(level=logging.INFO) + logging.info(msg) - (pid=95193) INFO:root:A log message for a task - (pid=95192) INFO:root:A log message for an actor. +ray.get(f.remote("A log message for a task")) +``` -## How to use structured logging +```bash +(pid=95193) INFO:root:A log message for a task +(pid=95192) INFO:root:A log message for an actor. +``` +## Using structured logging The metadata of tasks or actors may be obtained by Ray's :ref:`runtime_context APIs `. Runtime context APIs help you to add metadata to your logging messages, making your logs more structured. -.. code-block:: python - - import ray - # Initiate a driver. - ray.init() - - @ray.remote - def task(): - print(f"task_id: {ray.get_runtime_context().task_id}") +```python +import ray +# Initiate a driver. +ray.init() - ray.get(task.remote()) + @ray.remote +def task(): + print(f"task_id: {ray.get_runtime_context().task_id}") -.. code-block:: bash - - (pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) +ray.get(task.remote()) +``` +```bash +(pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) +``` ## Logging directory structure By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. -.. note:: - - The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. +:::{note} +The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. +::: A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. @@ -238,16 +236,16 @@ Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/std For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). - ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. - ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. -- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver.- +- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver. -## Log rotation +## Rotating logs Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). By default, logs are rotating when it reaches to 512MB (maxBytes), and there could be up to 5 backup files (backupCount). Indexes are appended to all backup files (e.g., `raylet.out.1`) If you'd like to change the log rotation configuration, you can do it by specifying environment variables. For example, -.. code-block:: bash - - RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. - RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. +```bash +RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. +RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. +``` \ No newline at end of file diff --git a/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst b/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst index 6d882c34fe598..aa83829f679dd 100644 --- a/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst +++ b/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst @@ -13,7 +13,6 @@ See :ref:`Getting Help ` if your problem is not s ../overview ../../ray-core/ray-dashboard ../state/state-api - ../ray-logging ../ray-metrics profiling gotchas diff --git a/doc/source/ray-observability/ray-logging.rst b/doc/source/ray-observability/ray-logging.rst deleted file mode 100644 index ac238cc4a715e..0000000000000 --- a/doc/source/ray-observability/ray-logging.rst +++ /dev/null @@ -1,300 +0,0 @@ -.. _ray-logging: - -Logging -======= -This document will explain Ray's logging system and its best practices. - -Driver logs -~~~~~~~~~~~ -An entry point of Ray applications that calls ``ray.init()`` is called a driver. -All the driver logs are handled in the same way as normal Python programs. - -Job logs -~~~~~~~~ -Logs for jobs submitted via the :ref:`Ray Jobs API ` can be retrieved using the ``ray job logs`` :ref:`CLI command ` or using ``JobSubmissionClient.get_logs()`` or ``JobSubmissionClient.tail_job_logs()`` via the :ref:`Python SDK `. -The log file consists of the stdout of the entrypoint command of the job. For the location of the log file on disk, see :ref:`Logging directory structure `. - -.. _ray-worker-logs: - -Worker logs -~~~~~~~~~~~ -Ray's tasks or actors are executed remotely within Ray's worker processes. Ray has special support to improve the visibility of logs produced by workers. - -- By default, all of the tasks/actors stdout and stderr are redirected to the worker log files. Check out :ref:`Logging directory structure ` to learn how Ray's logging directory is structured. -- By default, all of the tasks/actors stdout and stderr that is redirected to worker log files are published to the driver. Drivers display logs generated from its tasks/actors to its stdout and stderr. - -Let's look at a code example to see how this works. - -.. code-block:: python - - import ray - # Initiate a driver. - ray.init() - - @ray.remote - def task(): - print("task") - - ray.get(task.remote()) - -You should be able to see the string `task` from your driver stdout. - -When logs are printed, the process id (pid) and an IP address of the node that executes tasks/actors are printed together. Check out the output below. - -.. code-block:: bash - - (pid=45601) task - -Actor log messages look like the following by default. - -.. code-block:: bash - - (MyActor pid=480956) actor log message - -Log deduplication -~~~~~~~~~~~~~~~~~ - -By default, Ray will deduplicate logs that appear redundantly across multiple processes. The first instance of each log message will always be immediately printed. However, subsequent log messages of the same pattern (ignoring words with numeric components) will be buffered for up to five seconds and printed in batch. For example, for the following code snippet: - -.. code-block:: python - - import ray - import random - - @ray.remote - def task(): - print("Hello there, I am a task", random.random()) - - ray.get([task.remote() for _ in range(100)]) - -The output will be as follows: - -.. code-block:: bash - - 2023-03-27 15:08:34,195 INFO worker.py:1603 -- Started a local Ray instance. View the dashboard at http://127.0.0.1:8265 - (task pid=534172) Hello there, I am a task 0.20583517821231412 - (task pid=534174) Hello there, I am a task 0.17536720316370757 [repeated 99x across cluster] (Ray deduplicates logs by default. Set RAY_DEDUP_LOGS=0 to disable log deduplication) - -This feature is especially useful when importing libraries such as `tensorflow` or `numpy`, which may emit many verbose warning messages when imported. You can configure this feature as follows: - -1. Set ``RAY_DEDUP_LOGS=0`` to disable this feature entirely. -2. Set ``RAY_DEDUP_LOGS_AGG_WINDOW_S=`` to change the agggregation window. -3. Set ``RAY_DEDUP_LOGS_ALLOW_REGEX=`` to specify log messages to never deduplicate. -4. Set ``RAY_DEDUP_LOGS_SKIP_REGEX=`` to specify log messages to skip printing. - - -Disabling logging to the driver -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -In large scale runs, it may be undesirable to route all worker logs to the driver. You can disable this feature by setting ``log_to_driver=False`` in Ray init: - -.. code-block:: python - - import ray - - # Task and actor logs will not be copied to the driver stdout. - ray.init(log_to_driver=False) - -Customizing Actor logs prefixes -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -It is often useful to distinguish between log messages from different actors. For example, suppose you have a large number of worker actors. In this case, you may want to be able to easily see the index of the actor that logged a particular message. This can be achieved by defining the `__repr__ `__ method for an actor class. When defined, the actor repr will be used in place of the actor name. For example: - -.. literalinclude:: /ray-core/doc_code/actor-repr.py - -This produces the following output: - -.. code-block:: bash - - (MyActor(index=2) pid=482120) hello there - (MyActor(index=1) pid=482119) hello there - -Coloring Actor log prefixes -~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -By default Ray prints Actor logs prefixes in light blue: -Users may instead activate multi-color prefixes by setting the environment variable ``RAY_COLOR_PREFIX=1``. -This will index into an array of colors modulo the PID of each process. - -.. image:: ./images/coloring-actor-log-prefixes.png - :align: center - -Distributed progress bars (tqdm) -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -When using `tqdm `__ in Ray remote tasks or actors, you may notice that the progress bar output is corrupted. To avoid this problem, you can use the Ray distributed tqdm implementation at ``ray.experimental.tqdm_ray``: - -.. literalinclude:: /ray-core/doc_code/tqdm.py - -This tqdm implementation works as follows: - -1. The ``tqdm_ray`` module translates TQDM calls into special json log messages written to worker stdout. -2. The Ray log monitor, instead of copying these log messages directly to the driver stdout, routes these messages to a tqdm singleton. -3. The tqdm singleton determines the positions of progress bars from various Ray tasks / actors, ensuring they don't collide or conflict with each other. - -Limitations: - -- Only a subset of tqdm functionality is supported. Refer to the ray_tqdm `implementation `__ for more details. -- Performance may be poor if there are more than a couple thousand updates per second (updates are not batched). - -By default, the builtin print will also be patched to use `ray.experimental.tqdm_ray.safe_print` when `tqdm_ray` is used. -This avoids progress bar corruption on driver print statements. To disable this, set `RAY_TQDM_PATCH_PRINT=0`. - -How to set up loggers -~~~~~~~~~~~~~~~~~~~~~ -When using ray, all of the tasks and actors are executed remotely in Ray's worker processes. -Since Python logger module creates a singleton logger per process, loggers should be configured on per task/actor basis. - -.. note:: - - To stream logs to a driver, they should be flushed to stdout and stderr. - -.. code-block:: python - - import ray - import logging - # Initiate a driver. - ray.init() - - @ray.remote - class Actor: - def __init__(self): - # Basic config automatically configures logs to - # be streamed to stdout and stderr. - # Set the severity to INFO so that info logs are printed to stdout. - logging.basicConfig(level=logging.INFO) - - def log(self, msg): - logging.info(msg) - - actor = Actor.remote() - ray.get(actor.log.remote("A log message for an actor.")) - - @ray.remote - def f(msg): - logging.basicConfig(level=logging.INFO) - logging.info(msg) - - ray.get(f.remote("A log message for a task")) - -.. code-block:: bash - - (pid=95193) INFO:root:A log message for a task - (pid=95192) INFO:root:A log message for an actor. - -How to use structured logging -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -The metadata of tasks or actors may be obtained by Ray's :ref:`runtime_context APIs `. -Runtime context APIs help you to add metadata to your logging messages, making your logs more structured. - -.. code-block:: python - - import ray - # Initiate a driver. - ray.init() - - @ray.remote - def task(): - print(f"task_id: {ray.get_runtime_context().task_id}") - - ray.get(task.remote()) - -.. code-block:: bash - - (pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) - -Logging directory structure ---------------------------- -.. _logging-directory-structure: - -By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. - -.. note:: - - The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. - -A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. - -Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/stderr and ``.err`` is logs from stderr. The backward compatibility of log directories is not maintained. - -- ``dashboard.[log|err]``: A log file of a Ray dashboard. ``log.`` file contains logs generated from the dashboard's logger. ``.err`` file contains stdout and stderr printed from the dashboard. They are usually empty except when the dashboard crashes unexpectedly. -- ``dashboard_agent.log``: Every Ray node has one dashboard agent. This is a log file of the agent. -- ``gcs_server.[out|err]``: The GCS server is a stateless server that manages Ray cluster metadata. It exists only in the head node. -- ``io-worker-[worker_id]-[pid].[out|err]``: Ray creates IO workers to spill/restore objects to external storage by default from Ray 1.3+. This is a log file of IO workers. -- ``job-driver-[submission_id].log``: The stdout of a job submitted via the :ref:`Ray Jobs API `. -- ``log_monitor.[log|err]``: The log monitor is in charge of streaming logs to the driver. ``log.`` file contains logs generated from the log monitor's logger. ``.err`` file contains the stdout and stderr printed from the log monitor. They are usually empty except when the log monitor crashes unexpectedly. -- ``monitor.[out|err]``: Stdout and stderr of a cluster launcher. -- ``monitor.log``: Ray's cluster launcher is operated with a monitor process. It also manages the autoscaler. -- ``plasma_store.[out|err]``: Deprecated. -- ``python-core-driver-[worker_id]_[pid].log``: Ray drivers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. -- ``python-core-worker-[worker_id]_[pid].log``: Ray workers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. -- ``raylet.[out|err]``: A log file of raylets. -- ``redis-shard_[shard_index].[out|err]``: Redis shard log files. -- ``redis.[out|err]``: Redis log files. -- ``runtime_env_agent.log``: Every Ray node has one agent that manages :ref:`runtime environment ` creation, deletion and caching. - This is the log file of the agent containing logs of create/delete requests and cache hits and misses. - For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). -- ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. -- ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. -- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver.- - -.. _ray-log-rotation: - -Log rotation ------------- - -Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). - -By default, logs are rotating when it reaches to 512MB (maxBytes), and there could be up to 5 backup files (backupCount). Indexes are appended to all backup files (e.g., `raylet.out.1`) -If you'd like to change the log rotation configuration, you can do it by specifying environment variables. For example, - -.. code-block:: bash - - RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. - RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. - -Redirecting Ray logs to stderr -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -By default, Ray logs are written to files under the ``/tmp/ray/session_*/logs`` directory. If you wish to redirect all internal Ray logging and your own logging within tasks/actors to stderr of the host nodes, you can do so by ensuring that the ``RAY_LOG_TO_STDERR=1`` environment variable is set on the driver and on all Ray nodes. This is very useful if you are using a log aggregator that needs log records to be written to stderr in order for them to be captured. - -Redirecting logging to stderr will also cause a ``({component})`` prefix, e.g. ``(raylet)``, to be added to each of the log record messages. - -.. code-block:: bash - - [2022-01-24 19:42:02,978 I 1829336 1829336] (gcs_server) grpc_server.cc:103: GcsServer server started, listening on port 50009. - [2022-01-24 19:42:06,696 I 1829415 1829415] (raylet) grpc_server.cc:103: ObjectManager server started, listening on port 40545. - 2022-01-24 19:42:05,087 INFO (dashboard) dashboard.py:95 -- Setup static dir for dashboard: /mnt/data/workspace/ray/python/ray/dashboard/client/build - 2022-01-24 19:42:07,500 INFO (dashboard_agent) agent.py:105 -- Dashboard agent grpc address: 0.0.0.0:49228 - -This should make it easier to filter the stderr stream of logs down to the component of interest. Note that multi-line log records will **not** have this component marker at the beginning of each line. - -When running a local Ray cluster, this environment variable should be set before starting the local cluster: - -.. code-block:: python - - os.environ["RAY_LOG_TO_STDERR"] = "1" - ray.init() - -When starting a local cluster via the CLI or when starting nodes in a multi-node Ray cluster, this environment variable should be set before starting up each node: - -.. code-block:: bash - - env RAY_LOG_TO_STDERR=1 ray start - -If using the Ray cluster launcher, you would specify this environment variable in the Ray start commands: - -.. code-block:: bash - - head_start_ray_commands: - - ray stop - - env RAY_LOG_TO_STDERR=1 ray start --head --port=6379 --object-manager-port=8076 --autoscaling-config=~/ray_bootstrap_config.yaml - - worker_start_ray_commands: - - ray stop - - env RAY_LOG_TO_STDERR=1 ray start --address=$RAY_HEAD_IP:6379 --object-manager-port=8076 - -When connecting to the cluster, be sure to set the environment variable before connecting: - -.. code-block:: python - - os.environ["RAY_LOG_TO_STDERR"] = "1" - ray.init(address="auto") diff --git a/doc/source/ray-observability/user-guides/configure-logging.rst b/doc/source/ray-observability/user-guides/configure-logging.rst index fb166093350d8..fd0338a8cabb0 100644 --- a/doc/source/ray-observability/user-guides/configure-logging.rst +++ b/doc/source/ray-observability/user-guides/configure-logging.rst @@ -3,7 +3,7 @@ Configuring Logging =================== -This document explains Ray's logging system and best practices. +This guide helps you modify the default configuration of Ray's logging system. Disabling logging to the driver ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ @@ -20,7 +20,7 @@ In large scale runs, it may be undesirable to route all worker logs to the drive Log deduplication ~~~~~~~~~~~~~~~~~ -By default, Ray will deduplicate logs that appear redundantly across multiple processes. The first instance of each log message will always be immediately printed. However, subsequent log messages of the same pattern (ignoring words with numeric components) will be buffered for up to five seconds and printed in batch. For example, for the following code snippet: +By default, Ray deduplicates logs that appear redundantly across multiple processes. The first instance of each log message is always immediately printed. However, subsequent log messages of the same pattern (ignoring words with numeric components) are buffered for up to five seconds and printed in batch. For example, for the following code snippet: .. code-block:: python @@ -33,7 +33,7 @@ By default, Ray will deduplicate logs that appear redundantly across multiple pr ray.get([task.remote() for _ in range(100)]) -The output will be as follows: +The output is as follows: .. code-block:: bash From 50f9291f0499590be01d71fbfabd22dcbaf368a2 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Fri, 12 May 2023 10:44:40 -0700 Subject: [PATCH 04/29] rename overview content to key concepts Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/ray-observability/key-concepts.rst | 201 ++++++++++++++++++ 1 file changed, 201 insertions(+) create mode 100644 doc/source/ray-observability/key-concepts.rst diff --git a/doc/source/ray-observability/key-concepts.rst b/doc/source/ray-observability/key-concepts.rst new file mode 100644 index 0000000000000..622263cc3e4f8 --- /dev/null +++ b/doc/source/ray-observability/key-concepts.rst @@ -0,0 +1,201 @@ +.. _observability-key-concepts: + +Key Concepts +============ + +This section covers a list of key concepts for monitoring and debugging tools and features in Ray. + +Dashboard (Web UI) +------------------ +Ray supports the web-based dashboard to help users monitor the cluster. When a new cluster is started, the dashboard is available +through the default address `localhost:8265` (port can be automatically incremented if port 8265 is already occupied). + +See :ref:`Ray Dashboard ` for more details. + +Application Logging +------------------- +By default, all stdout and stderr of tasks and actors are streamed to the Ray driver (the entrypoint script that calls ``ray.init``). + +.. literalinclude:: doc_code/app_logging.py + :language: python + +All stdout emitted from the ``print`` method is printed to the driver with a ``(the task or actor repr, the process ID, IP address)`` prefix. + +.. code-block:: bash + + (pid=45601) task + (Actor pid=480956) actor + +See :ref:`Logging ` for more details. + +Driver logs +~~~~~~~~~~~ +An entry point of Ray applications that calls ``ray.init()`` is called a driver. +All the driver logs are handled in the same way as normal Python programs. + +Job logs +~~~~~~~~ +Logs for jobs submitted via the :ref:`Ray Jobs API ` can be retrieved using the ``ray job logs`` :ref:`CLI command ` or using ``JobSubmissionClient.get_logs()`` or ``JobSubmissionClient.tail_job_logs()`` via the :ref:`Python SDK `. +The log file consists of the stdout of the entrypoint command of the job. For the location of the log file on disk, see :ref:`Logging directory structure `. + +.. _ray-worker-logs: + +Worker logs +~~~~~~~~~~~ +Ray's tasks or actors are executed remotely within Ray's worker processes. Ray has special support to improve the visibility of logs produced by workers. + +- By default, all of the tasks/actors stdout and stderr are redirected to the worker log files. Check out :ref:`Logging directory structure ` to learn how Ray's logging directory is structured. +- By default, all of the tasks/actors stdout and stderr that is redirected to worker log files are published to the driver. Drivers display logs generated from its tasks/actors to its stdout and stderr. + +Let's look at a code example to see how this works. + +.. code-block:: python + + import ray + # Initiate a driver. + ray.init() + + @ray.remote + def task(): + print("task") + + ray.get(task.remote()) + +You should be able to see the string `task` from your driver stdout. + +When logs are printed, the process id (pid) and an IP address of the node that executes tasks/actors are printed together. Check out the output below. + +.. code-block:: bash + + (pid=45601) task + +Actor log messages look like the following by default. + +.. code-block:: bash + + (MyActor pid=480956) actor log message + +Accessing Ray States +-------------------- +Starting from Ray 2.0, it supports CLI / Python APIs to query the state of resources (e.g., actor, task, object, etc.). + +For example, the following command will summarize the task state of the cluster. + +.. code-block:: bash + + ray summary tasks + +.. code-block:: text + + ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== + Stats: + ------------------------------------ + total_actor_scheduled: 2 + total_actor_tasks: 0 + total_tasks: 2 + + + Table (group by func_name): + ------------------------------------ + FUNC_OR_CLASS_NAME STATE_COUNTS TYPE + 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK + 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK + +The following command will list all the actors from the cluster. + +.. code-block:: bash + + ray list actors + +.. code-block:: text + + ======== List: 2022-07-23 21:29:39.323925 ======== + Stats: + ------------------------------ + Total: 2 + + Table: + ------------------------------ + ACTOR_ID CLASS_NAME NAME PID STATE + 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE + 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE + +See :ref:`Ray State API ` for more details. + +Metrics +------- +Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), +internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), +and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used +to monitor the cluster over time. + +See :ref:`Ray Metrics ` for more details. + +Exceptions +---------- +Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, +the API raises an exception if anything goes wrong with a related task, actor or object. For example, + +- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. +- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). +- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. + +See :ref:`Exceptions Reference ` for more details. + +Debugger +-------- +Ray has a built-in debugger that allows you to debug your distributed applications. +It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can +drop into a PDB session that you can then use to: + +- Inspect variables in that context +- Step within that task or actor +- Move up or down the stack + +See :ref:`Ray Debugger ` for more details. + +Monitoring Cluster State and Resource Demands +--------------------------------------------- +You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays + +- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. +- **Autoscaling Status**: The number of nodes that are autoscaling up and down. +- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. + +Here's an example output. + +.. code-block:: shell + + $ ray status + ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== + Node status + --------------------------------------------------------------- + Healthy: + 1 ray.head.default + 2 ray.worker.cpu + Pending: + (no pending nodes) + Recent failures: + (no failures) + + Resources + --------------------------------------------------------------- + Usage: + 0.0/10.0 CPU + 0.00/70.437 GiB memory + 0.00/10.306 GiB object_store_memory + + Demands: + (no resource demands) + +Profiling +--------- +Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. + +See :ref:`Profiling ` for more details. + +Tracing +------- +To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. + +See :ref:`Ray Tracing ` for more details. \ No newline at end of file From 59a0ed8fd2b97190b903e9e5ec3361f5d7acf9e5 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 10:56:20 -0700 Subject: [PATCH 05/29] new page for getting started on observability with programmatic interfaces Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../getting-started-program.rst | 201 ++++++++++++++++++ 1 file changed, 201 insertions(+) create mode 100644 doc/source/ray-observability/getting-started-program.rst diff --git a/doc/source/ray-observability/getting-started-program.rst b/doc/source/ray-observability/getting-started-program.rst new file mode 100644 index 0000000000000..622263cc3e4f8 --- /dev/null +++ b/doc/source/ray-observability/getting-started-program.rst @@ -0,0 +1,201 @@ +.. _observability-key-concepts: + +Key Concepts +============ + +This section covers a list of key concepts for monitoring and debugging tools and features in Ray. + +Dashboard (Web UI) +------------------ +Ray supports the web-based dashboard to help users monitor the cluster. When a new cluster is started, the dashboard is available +through the default address `localhost:8265` (port can be automatically incremented if port 8265 is already occupied). + +See :ref:`Ray Dashboard ` for more details. + +Application Logging +------------------- +By default, all stdout and stderr of tasks and actors are streamed to the Ray driver (the entrypoint script that calls ``ray.init``). + +.. literalinclude:: doc_code/app_logging.py + :language: python + +All stdout emitted from the ``print`` method is printed to the driver with a ``(the task or actor repr, the process ID, IP address)`` prefix. + +.. code-block:: bash + + (pid=45601) task + (Actor pid=480956) actor + +See :ref:`Logging ` for more details. + +Driver logs +~~~~~~~~~~~ +An entry point of Ray applications that calls ``ray.init()`` is called a driver. +All the driver logs are handled in the same way as normal Python programs. + +Job logs +~~~~~~~~ +Logs for jobs submitted via the :ref:`Ray Jobs API ` can be retrieved using the ``ray job logs`` :ref:`CLI command ` or using ``JobSubmissionClient.get_logs()`` or ``JobSubmissionClient.tail_job_logs()`` via the :ref:`Python SDK `. +The log file consists of the stdout of the entrypoint command of the job. For the location of the log file on disk, see :ref:`Logging directory structure `. + +.. _ray-worker-logs: + +Worker logs +~~~~~~~~~~~ +Ray's tasks or actors are executed remotely within Ray's worker processes. Ray has special support to improve the visibility of logs produced by workers. + +- By default, all of the tasks/actors stdout and stderr are redirected to the worker log files. Check out :ref:`Logging directory structure ` to learn how Ray's logging directory is structured. +- By default, all of the tasks/actors stdout and stderr that is redirected to worker log files are published to the driver. Drivers display logs generated from its tasks/actors to its stdout and stderr. + +Let's look at a code example to see how this works. + +.. code-block:: python + + import ray + # Initiate a driver. + ray.init() + + @ray.remote + def task(): + print("task") + + ray.get(task.remote()) + +You should be able to see the string `task` from your driver stdout. + +When logs are printed, the process id (pid) and an IP address of the node that executes tasks/actors are printed together. Check out the output below. + +.. code-block:: bash + + (pid=45601) task + +Actor log messages look like the following by default. + +.. code-block:: bash + + (MyActor pid=480956) actor log message + +Accessing Ray States +-------------------- +Starting from Ray 2.0, it supports CLI / Python APIs to query the state of resources (e.g., actor, task, object, etc.). + +For example, the following command will summarize the task state of the cluster. + +.. code-block:: bash + + ray summary tasks + +.. code-block:: text + + ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== + Stats: + ------------------------------------ + total_actor_scheduled: 2 + total_actor_tasks: 0 + total_tasks: 2 + + + Table (group by func_name): + ------------------------------------ + FUNC_OR_CLASS_NAME STATE_COUNTS TYPE + 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK + 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK + +The following command will list all the actors from the cluster. + +.. code-block:: bash + + ray list actors + +.. code-block:: text + + ======== List: 2022-07-23 21:29:39.323925 ======== + Stats: + ------------------------------ + Total: 2 + + Table: + ------------------------------ + ACTOR_ID CLASS_NAME NAME PID STATE + 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE + 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE + +See :ref:`Ray State API ` for more details. + +Metrics +------- +Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), +internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), +and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used +to monitor the cluster over time. + +See :ref:`Ray Metrics ` for more details. + +Exceptions +---------- +Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, +the API raises an exception if anything goes wrong with a related task, actor or object. For example, + +- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. +- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). +- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. + +See :ref:`Exceptions Reference ` for more details. + +Debugger +-------- +Ray has a built-in debugger that allows you to debug your distributed applications. +It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can +drop into a PDB session that you can then use to: + +- Inspect variables in that context +- Step within that task or actor +- Move up or down the stack + +See :ref:`Ray Debugger ` for more details. + +Monitoring Cluster State and Resource Demands +--------------------------------------------- +You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays + +- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. +- **Autoscaling Status**: The number of nodes that are autoscaling up and down. +- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. + +Here's an example output. + +.. code-block:: shell + + $ ray status + ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== + Node status + --------------------------------------------------------------- + Healthy: + 1 ray.head.default + 2 ray.worker.cpu + Pending: + (no pending nodes) + Recent failures: + (no failures) + + Resources + --------------------------------------------------------------- + Usage: + 0.0/10.0 CPU + 0.00/70.437 GiB memory + 0.00/10.306 GiB object_store_memory + + Demands: + (no resource demands) + +Profiling +--------- +Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. + +See :ref:`Profiling ` for more details. + +Tracing +------- +To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. + +See :ref:`Ray Tracing ` for more details. \ No newline at end of file From 8ea2d4475e8a9b14e39f8830dccfa993b83483c8 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 11:14:50 -0700 Subject: [PATCH 06/29] move content to new page for config and manage dashboard Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../monitoring-and-observability.rst | 38 ---- doc/source/ray-core/ray-dashboard.rst | 119 ----------- .../config-manage-dashboard.rst | 201 ++++++++++++++++++ 3 files changed, 201 insertions(+), 157 deletions(-) create mode 100644 doc/source/ray-observability/config-manage-dashboard.rst diff --git a/doc/source/cluster/running-applications/monitoring-and-observability.rst b/doc/source/cluster/running-applications/monitoring-and-observability.rst index c9dd3e39ee9a9..fcc47a746c22b 100644 --- a/doc/source/cluster/running-applications/monitoring-and-observability.rst +++ b/doc/source/cluster/running-applications/monitoring-and-observability.rst @@ -13,44 +13,6 @@ The rest of this page will focus on how to access these services when running a .. _monitor-cluster-via-dashboard: -Monitoring the cluster via the dashboard -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ - -:ref:`The dashboard ` provides detailed information about the state of the cluster, -including the running jobs, actors, workers, nodes, etc. -By default, the :ref:`cluster launcher ` and :ref:`KubeRay operator ` will launch the dashboard, but will -not publicly expose the port. - -.. tab-set:: - - .. tab-item:: If using the VM cluster launcher - - You can securely port-forward local traffic to the dashboard via the ``ray - dashboard`` command. - - .. code-block:: shell - - $ ray dashboard [-p ] - - The dashboard will now be visible at ``http://localhost:8265``. - - .. tab-item:: If using Kubernetes - - The KubeRay operator makes the dashboard available via a Service targeting - the Ray head pod, named ``-head-svc``. You can access the - dashboard from within the Kubernetes cluster at ``http://-head-svc:8265``. - - You can also view the dashboard from outside the Kubernetes cluster by - using port-forwarding: - - .. code-block:: shell - - $ kubectl port-forward service/raycluster-autoscaler-head-svc 8265:8265 - - For more information about configuring network access to a Ray cluster on - Kubernetes, see the :ref:`networking notes `. - - Using Ray Cluster CLI tools ^^^^^^^^^^^^^^^^^^^^^^^^^^^ diff --git a/doc/source/ray-core/ray-dashboard.rst b/doc/source/ray-core/ray-dashboard.rst index ef9e0553a17b6..fd5b2d81867b2 100644 --- a/doc/source/ray-core/ray-dashboard.rst +++ b/doc/source/ray-core/ray-dashboard.rst @@ -446,125 +446,6 @@ To understand the log file structure of Ray, see the :ref:`Logging directory str The logs view provides search functionality to help you find specific log messages. -Advanced Usage --------------- - -Changing Dashboard Ports -~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tab-set:: - - .. tab-item:: Single-node local cluster - - **CLI** - - To customize the port on which the dashboard runs, you can pass - the ``--dashboard-port`` argument with ``ray start`` in the command line. - - **ray.init** - - If you need to customize the port on which the dashboard will run, you can pass the - keyword argument ``dashboard_port`` in your call to ``ray.init()``. - - .. tab-item:: VM Cluster Launcher - - To disable the dashboard while using the "VM cluster launcher", include the "ray start --head --include-dashboard=False" argument - and specify the desired port number in the "head_start_ray_commands" section of the `cluster launcher's YAML file `_. - - .. tab-item:: Kuberay - - See the `Specifying non-default ports `_ page. - -Viewing Built-in Dashboard API Metrics -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -The dashboard is powered by a server that serves both the UI code and the data about the cluster via API endpoints. -There are basic Prometheus metrics that are emitted for each of these API endpoints: - -`ray_dashboard_api_requests_count_requests_total`: Collects the total count of requests. This is tagged by endpoint, method, and http_status. - -`ray_dashboard_api_requests_duration_seconds_bucket`: Collects the duration of requests. This is tagged by endpoint and method. - -For example, you can view the p95 duration of all requests with this query: - -.. code-block:: text - - histogram_quantile(0.95, sum(rate(ray_dashboard_api_requests_duration_seconds_bucket[5m])) by (le)) - -These metrics can be queried via Prometheus or Grafana UI. Instructions on how to set these tools up can be found :ref:`here `. - - -Running Behind a Reverse Proxy -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -The dashboard should work out-of-the-box when accessed via a reverse proxy. API requests don't need to be proxied individually. - -Always access the dashboard with a trailing ``/`` at the end of the URL. -For example, if your proxy is set up to handle requests to ``/ray/dashboard``, view the dashboard at ``www.my-website.com/ray/dashboard/``. - -The dashboard now sends HTTP requests with relative URL paths. Browsers will handle these requests as expected when the ``window.location.href`` ends in a trailing ``/``. - -This is a peculiarity of how many browsers handle requests with relative URLs, despite what `MDN `_ -defines as the expected behavior. - -Make your dashboard visible without a trailing ``/`` by including a rule in your reverse proxy that -redirects the user's browser to ``/``, i.e. ``/ray/dashboard`` --> ``/ray/dashboard/``. - -Below is an example with a `traefik `_ TOML file that accomplishes this: - -.. code-block:: yaml - - [http] - [http.routers] - [http.routers.to-dashboard] - rule = "PathPrefix(`/ray/dashboard`)" - middlewares = ["test-redirectregex", "strip"] - service = "dashboard" - [http.middlewares] - [http.middlewares.test-redirectregex.redirectRegex] - regex = "^(.*)/ray/dashboard$" - replacement = "${1}/ray/dashboard/" - [http.middlewares.strip.stripPrefix] - prefixes = ["/ray/dashboard"] - [http.services] - [http.services.dashboard.loadBalancer] - [[http.services.dashboard.loadBalancer.servers]] - url = "http://localhost:8265" - -Disabling the Dashboard -~~~~~~~~~~~~~~~~~~~~~~~ -Dashboard is included in the `ray[default]` installation by default and automatically started. - -To disable the dashboard, use the following arguments `--include-dashboard`. - -.. tab-set:: - - .. tab-item:: Single-node local cluster - - **CLI** - - .. code-block:: bash - - ray start --include-dashboard=False - - **ray.init** - - .. testcode:: - :hide: - - ray.shutdown() - - .. testcode:: - - ray.init(include_dashboard=False) - - .. tab-item:: VM Cluster Launcher - - To disable the dashboard while using the "VM cluster launcher", include the "ray start --head --include-dashboard=False" argument - in the "head_start_ray_commands" section of the `cluster launcher's YAML file `_. - - .. tab-item:: Kuberay - - TODO - .. _dash-reference: Page References diff --git a/doc/source/ray-observability/config-manage-dashboard.rst b/doc/source/ray-observability/config-manage-dashboard.rst new file mode 100644 index 0000000000000..622263cc3e4f8 --- /dev/null +++ b/doc/source/ray-observability/config-manage-dashboard.rst @@ -0,0 +1,201 @@ +.. _observability-key-concepts: + +Key Concepts +============ + +This section covers a list of key concepts for monitoring and debugging tools and features in Ray. + +Dashboard (Web UI) +------------------ +Ray supports the web-based dashboard to help users monitor the cluster. When a new cluster is started, the dashboard is available +through the default address `localhost:8265` (port can be automatically incremented if port 8265 is already occupied). + +See :ref:`Ray Dashboard ` for more details. + +Application Logging +------------------- +By default, all stdout and stderr of tasks and actors are streamed to the Ray driver (the entrypoint script that calls ``ray.init``). + +.. literalinclude:: doc_code/app_logging.py + :language: python + +All stdout emitted from the ``print`` method is printed to the driver with a ``(the task or actor repr, the process ID, IP address)`` prefix. + +.. code-block:: bash + + (pid=45601) task + (Actor pid=480956) actor + +See :ref:`Logging ` for more details. + +Driver logs +~~~~~~~~~~~ +An entry point of Ray applications that calls ``ray.init()`` is called a driver. +All the driver logs are handled in the same way as normal Python programs. + +Job logs +~~~~~~~~ +Logs for jobs submitted via the :ref:`Ray Jobs API ` can be retrieved using the ``ray job logs`` :ref:`CLI command ` or using ``JobSubmissionClient.get_logs()`` or ``JobSubmissionClient.tail_job_logs()`` via the :ref:`Python SDK `. +The log file consists of the stdout of the entrypoint command of the job. For the location of the log file on disk, see :ref:`Logging directory structure `. + +.. _ray-worker-logs: + +Worker logs +~~~~~~~~~~~ +Ray's tasks or actors are executed remotely within Ray's worker processes. Ray has special support to improve the visibility of logs produced by workers. + +- By default, all of the tasks/actors stdout and stderr are redirected to the worker log files. Check out :ref:`Logging directory structure ` to learn how Ray's logging directory is structured. +- By default, all of the tasks/actors stdout and stderr that is redirected to worker log files are published to the driver. Drivers display logs generated from its tasks/actors to its stdout and stderr. + +Let's look at a code example to see how this works. + +.. code-block:: python + + import ray + # Initiate a driver. + ray.init() + + @ray.remote + def task(): + print("task") + + ray.get(task.remote()) + +You should be able to see the string `task` from your driver stdout. + +When logs are printed, the process id (pid) and an IP address of the node that executes tasks/actors are printed together. Check out the output below. + +.. code-block:: bash + + (pid=45601) task + +Actor log messages look like the following by default. + +.. code-block:: bash + + (MyActor pid=480956) actor log message + +Accessing Ray States +-------------------- +Starting from Ray 2.0, it supports CLI / Python APIs to query the state of resources (e.g., actor, task, object, etc.). + +For example, the following command will summarize the task state of the cluster. + +.. code-block:: bash + + ray summary tasks + +.. code-block:: text + + ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== + Stats: + ------------------------------------ + total_actor_scheduled: 2 + total_actor_tasks: 0 + total_tasks: 2 + + + Table (group by func_name): + ------------------------------------ + FUNC_OR_CLASS_NAME STATE_COUNTS TYPE + 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK + 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK + +The following command will list all the actors from the cluster. + +.. code-block:: bash + + ray list actors + +.. code-block:: text + + ======== List: 2022-07-23 21:29:39.323925 ======== + Stats: + ------------------------------ + Total: 2 + + Table: + ------------------------------ + ACTOR_ID CLASS_NAME NAME PID STATE + 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE + 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE + +See :ref:`Ray State API ` for more details. + +Metrics +------- +Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), +internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), +and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used +to monitor the cluster over time. + +See :ref:`Ray Metrics ` for more details. + +Exceptions +---------- +Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, +the API raises an exception if anything goes wrong with a related task, actor or object. For example, + +- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. +- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). +- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. + +See :ref:`Exceptions Reference ` for more details. + +Debugger +-------- +Ray has a built-in debugger that allows you to debug your distributed applications. +It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can +drop into a PDB session that you can then use to: + +- Inspect variables in that context +- Step within that task or actor +- Move up or down the stack + +See :ref:`Ray Debugger ` for more details. + +Monitoring Cluster State and Resource Demands +--------------------------------------------- +You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays + +- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. +- **Autoscaling Status**: The number of nodes that are autoscaling up and down. +- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. + +Here's an example output. + +.. code-block:: shell + + $ ray status + ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== + Node status + --------------------------------------------------------------- + Healthy: + 1 ray.head.default + 2 ray.worker.cpu + Pending: + (no pending nodes) + Recent failures: + (no failures) + + Resources + --------------------------------------------------------------- + Usage: + 0.0/10.0 CPU + 0.00/70.437 GiB memory + 0.00/10.306 GiB object_store_memory + + Demands: + (no resource demands) + +Profiling +--------- +Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. + +See :ref:`Profiling ` for more details. + +Tracing +------- +To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. + +See :ref:`Ray Tracing ` for more details. \ No newline at end of file From 529187fbd3e19b534a25fb0c3fd05007d1beffcf Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 12:05:00 -0700 Subject: [PATCH 07/29] populating configure dashboard doc and moving to clusters section Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../cluster/configure-manage-dashboard.rst | 344 ++++++++++++++++++ doc/source/ray-core/ray-dashboard.rst | 2 +- .../config-manage-dashboard.rst | 201 ---------- doc/source/ray-observability/ray-metrics.rst | 180 --------- 4 files changed, 345 insertions(+), 382 deletions(-) create mode 100644 doc/source/cluster/configure-manage-dashboard.rst delete mode 100644 doc/source/ray-observability/config-manage-dashboard.rst diff --git a/doc/source/cluster/configure-manage-dashboard.rst b/doc/source/cluster/configure-manage-dashboard.rst new file mode 100644 index 0000000000000..fd3f2d4e0a239 --- /dev/null +++ b/doc/source/cluster/configure-manage-dashboard.rst @@ -0,0 +1,344 @@ +.. _observability-configure-manage-dashboard: + +Configure and Manage the Dashboard +================================== + +Modify the configuration + +Port forwarding +--------------- + +:ref:`The dashboard ` provides detailed information about the state of the cluster, +including the running jobs, actors, workers, nodes, etc. +By default, the :ref:`cluster launcher ` and :ref:`KubeRay operator ` will launch the dashboard, but will +not publicly expose the port. + +.. tab-set:: + + .. tab-item:: VM + + You can securely port-forward local traffic to the dashboard via the ``ray + dashboard`` command. + + .. code-block:: shell + + $ ray dashboard [-p ] + + The dashboard is now be visible at ``http://localhost:8265``. + + .. tab-item:: Kubernetes + + The KubeRay operator makes the dashboard available via a Service targeting + the Ray head pod, named ``-head-svc``. You can access the + dashboard from within the Kubernetes cluster at ``http://-head-svc:8265``. + + You can also view the dashboard from outside the Kubernetes cluster by + using port-forwarding: + + .. code-block:: shell + + $ kubectl port-forward service/raycluster-autoscaler-head-svc 8265:8265 + + For more information about configuring network access to a Ray cluster on + Kubernetes, see the :ref:`networking notes `. + +Changing Dashboard Ports +------------------------ + +.. tab-set:: + + .. tab-item:: Single-node local cluster + + **CLI** + + To customize the port on which the dashboard runs, you can pass + the ``--dashboard-port`` argument with ``ray start`` in the command line. + + **ray.init** + + If you need to customize the port on which the dashboard runs, you can pass the + keyword argument ``dashboard_port`` in your call to ``ray.init()``. + + .. tab-item:: VM Cluster Launcher + + To disable the dashboard while using the "VM cluster launcher", include the "ray start --head --include-dashboard=False" argument + and specify the desired port number in the "head_start_ray_commands" section of the `cluster launcher's YAML file `_. + + .. tab-item:: Kuberay + + See the `Specifying non-default ports `_ page. + + +Running Behind a Reverse Proxy +------------------------------ + +The dashboard should work out-of-the-box when accessed via a reverse proxy. API requests don't need to be proxied individually. + +Always access the dashboard with a trailing ``/`` at the end of the URL. +For example, if your proxy is set up to handle requests to ``/ray/dashboard``, view the dashboard at ``www.my-website.com/ray/dashboard/``. + +The dashboard now sends HTTP requests with relative URL paths. Browsers will handle these requests as expected when the ``window.location.href`` ends in a trailing ``/``. + +This is a peculiarity of how many browsers handle requests with relative URLs, despite what `MDN `_ +defines as the expected behavior. + +Make your dashboard visible without a trailing ``/`` by including a rule in your reverse proxy that +redirects the user's browser to ``/``, i.e. ``/ray/dashboard`` --> ``/ray/dashboard/``. + +Below is an example with a `traefik `_ TOML file that accomplishes this: + +.. code-block:: yaml + + [http] + [http.routers] + [http.routers.to-dashboard] + rule = "PathPrefix(`/ray/dashboard`)" + middlewares = ["test-redirectregex", "strip"] + service = "dashboard" + [http.middlewares] + [http.middlewares.test-redirectregex.redirectRegex] + regex = "^(.*)/ray/dashboard$" + replacement = "${1}/ray/dashboard/" + [http.middlewares.strip.stripPrefix] + prefixes = ["/ray/dashboard"] + [http.services] + [http.services.dashboard.loadBalancer] + [[http.services.dashboard.loadBalancer.servers]] + url = "http://localhost:8265" + +Disabling the Dashboard +----------------------- + +Dashboard is included in the `ray[default]` installation by default and automatically started. + +To disable the dashboard, use the following arguments `--include-dashboard`. + +.. tab-set:: + + .. tab-item:: Single-node local cluster + + **CLI** + + .. code-block:: bash + + ray start --include-dashboard=False + + **ray.init** + + .. testcode:: + :hide: + + ray.shutdown() + + .. testcode:: + + ray.init(include_dashboard=False) + + .. tab-item:: VM Cluster Launcher + + To disable the dashboard while using the "VM cluster launcher", include the "ray start --head --include-dashboard=False" argument + in the "head_start_ray_commands" section of the `cluster launcher's YAML file `_. + + .. tab-item:: Kuberay + + TODO + +Viewing Built-in Dashboard API Metrics +-------------------------------------- + +The dashboard is powered by a server that serves both the UI code and the data about the cluster via API endpoints. +There are basic Prometheus metrics that are emitted for each of these API endpoints: + +`ray_dashboard_api_requests_count_requests_total`: Collects the total count of requests. This is tagged by endpoint, method, and http_status. + +`ray_dashboard_api_requests_duration_seconds_bucket`: Collects the duration of requests. This is tagged by endpoint and method. + +For example, you can view the p95 duration of all requests with this query: + +.. code-block:: text + + histogram_quantile(0.95, sum(rate(ray_dashboard_api_requests_duration_seconds_bucket[5m])) by (le)) + +These metrics can be queried via Prometheus or Grafana UI. Instructions on how to set these tools up can be found :ref:`here `. + +Setting up Prometheus +--------------------- + +.. tip:: + + The below instructions for Prometheus to enable a basic workflow of running and accessing the dashboard on your local machine. + For more information about how to run Prometheus on a remote cluster, see :ref:`here `. + +Ray exposes its metrics in Prometheus format. This allows us to easily scrape them using Prometheus. + +First, `download Prometheus `_. Make sure to download the correct binary for your operating system. (Ex: darwin for mac osx) + +Then, unzip the archive into a local directory using the following command. + +.. code-block:: bash + + tar xvfz prometheus-*.tar.gz + cd prometheus-* + +Ray exports metrics only when ``ray[default]`` is installed. + +.. code-block:: bash + + pip install "ray[default]" + +Ray provides a prometheus config that works out of the box. After running ray, it can be found at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. + +.. code-block:: yaml + + global: + scrape_interval: 15s + evaluation_interval: 15s + + scrape_configs: + # Scrape from each ray node as defined in the service_discovery.json provided by ray. + - job_name: 'ray' + file_sd_configs: + - files: + - '/tmp/ray/prom_metrics_service_discovery.json' + + +Next, let's start Prometheus. + +.. code-block:: shell + + ./prometheus --config.file=/tmp/ray/session_latest/metrics/prometheus/prometheus.yml + +.. note:: + If you are using mac, you may receive an error at this point about trying to launch an application where the developer has not been verified. See :ref:`this link ` to fix the issue. + +Now, you can access Ray metrics from the default Prometheus url, `http://localhost:9090`. + +See :ref:`here ` for more information on how to set up Prometheus on a Ray Cluster. + +.. _grafana: + +Setting up Grafana +------------------ + +.. tip:: + + The below instructions for Grafana setup to enable a basic workflow of running and accessing the dashboard on your local machine. + For more information about how to run Grafana on a remote cluster, see :ref:`here `. + +Grafana is a tool that supports more advanced visualizations of prometheus metrics and +allows you to create custom dashboards with your favorite metrics. Ray exports some default +configurations which includes a default dashboard showing some of the most valuable metrics +for debugging ray applications. + + +Deploying Grafana +~~~~~~~~~~~~~~~~~ + +First, `download Grafana `_. Follow the instructions on the download page to download the right binary for your operating system. + +Then go to to the location of the binary and run grafana using the built in configuration found in `/tmp/ray/session_latest/metrics/grafana` folder. + +.. code-block:: shell + + ./bin/grafana-server --config /tmp/ray/session_latest/metrics/grafana/grafana.ini web + +Now, you can access grafana using the default grafana url, `http://localhost:3000`. +You can then see the default dashboard by going to dashboards -> manage -> Ray -> Default Dashboard. The same :ref:`metric graphs ` are also accessible via :ref:`Ray Dashboard `. + +.. tip:: + + If this is your first time using Grafana, you can login with the username: `admin` and password `admin`. + +.. image:: images/graphs.png + :align: center + + +See :ref:`here ` for more information on how to set up Grafana on a Ray Cluster. + +.. _system-metrics: + +Configurations +-------------- + +Customize prometheus export port +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Ray by default provides the service discovery file, but you can directly scrape metrics from prometheus ports. +To do that, you may want to customize the port that metrics gets exposed to a pre-defined port. + +.. code-block:: bash + + ray start --head --metrics-export-port=8080 # Assign metrics export port on a head node. + +Now, you can scrape Ray's metrics using Prometheus via ``:8080``. + +Alternate Prometheus host location +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +You can choose to run Prometheus on a non-default port or on a different machine. When doing so, you should +make sure that prometheus can scrape the metrics from your ray nodes following instructions :ref:`here `. + +In addition, both Ray and Grafana needs to know how to access this prometheus instance. This can be configured +by setting the `RAY_PROMETHEUS_HOST` env var when launching ray. The env var takes in the address to access Prometheus. More +info can be found :ref:`here `. By default, we assume Prometheus is hosted at `localhost:9090`. + +For example, if Prometheus is hosted at port 9000 on a node with ip 55.66.77.88, One should set the value to +`RAY_PROMETHEUS_HOST=http://55.66.77.88:9000`. + + +Alternate Grafana host location +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +You can choose to run Grafana on a non-default port or on a different machine. If you choose to do this, the +:ref:`Dashboard ` needs to be configured with a public address to that service so the web page +can load the graphs. This can be done with the `RAY_GRAFANA_HOST` env var when launching ray. The env var takes +in the address to access Grafana. More info can be found :ref:`here `. Instructions +to use an existing Grafana instance can be found :ref:`here `. + +For the Grafana charts to work on the Ray dashboard, the user of the dashboard's browser must be able to reach +the Grafana service. If this browser cannot reach Grafana the same way the Ray head node can, you can use a separate +env var `RAY_GRAFANA_IFRAME_HOST` to customize the host the browser users to attempt to reach Grafana. If this is not set, +we use the value of `RAY_GRAFANA_HOST` by default. + +For example, if Grafana is hosted at is 55.66.77.88 on port 3000. One should set the value +to `RAY_GRAFANA_HOST=http://55.66.77.88:3000`. + + +Troubleshooting +--------------- + +Getting Prometheus and Grafana to use the Ray configurations when installed via homebrew on macOS X +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +With homebrew, Prometheus and Grafana are installed as services that are automatically launched for you. +Therefore, to configure these services, you cannot simply pass in the config files as command line arguments. + +Instead, follow these instructions: +1. Change the --config-file line in `/usr/local/etc/prometheus.args` to read `--config.file /tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. +2. Update `/usr/local/etc/grafana/grafana.ini` file so that it matches the contents of `/tmp/ray/session_latest/metrics/grafana/grafana.ini`. + +You can then start or restart the services with `brew services start grafana` and `brew services start prometheus`. + +.. _unverified-developer: + +MacOS does not trust the developer to install Prometheus or Grafana +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +You may have received an error that looks like this: + +.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/troubleshooting/prometheus-trusted-developer.png + :align: center + +When downloading binaries from the internet, Mac requires that the binary be signed by a trusted developer ID. +Unfortunately, many developers today are not trusted by Mac and so this requirement must be overridden by the user manaully. + +See `these instructions `_ on how to override the restriction and install or run the application. + +Grafana dashboards are not embedded in the Ray dashboard +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +If you're getting an error that says `RAY_GRAFANA_HOST` is not setup despite having set it up, check that: +You've included the protocol in the URL (e.g., `http://your-grafana-url.com` instead of `your-grafana-url.com`). +The URL doesn't have a trailing slash (e.g., `http://your-grafana-url.com` instead of `http://your-grafana-url.com/`). + +Certificate Authority (CA error) +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +You may see a CA error if your Grafana instance is hosted behind HTTPS. Contact the Grafana service owner to properly enable HTTPS traffic. + diff --git a/doc/source/ray-core/ray-dashboard.rst b/doc/source/ray-core/ray-dashboard.rst index fd5b2d81867b2..aeb988bedae45 100644 --- a/doc/source/ray-core/ray-dashboard.rst +++ b/doc/source/ray-core/ray-dashboard.rst @@ -437,7 +437,7 @@ Logs View .. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard/logs.png :align: center -The logs view lets you view all the ray logs that are in your cluster. It is organized by node and log file name. Many log links in the other pages link to this view and filter the list so the relevant logs appear. +The logs view lets you view all the Ray logs in your cluster. It is organized by node and log file name. Many log links in the other pages link to this view and filter the list so the relevant logs appear. To understand the log file structure of Ray, see the :ref:`Logging directory structure page `. diff --git a/doc/source/ray-observability/config-manage-dashboard.rst b/doc/source/ray-observability/config-manage-dashboard.rst deleted file mode 100644 index 622263cc3e4f8..0000000000000 --- a/doc/source/ray-observability/config-manage-dashboard.rst +++ /dev/null @@ -1,201 +0,0 @@ -.. _observability-key-concepts: - -Key Concepts -============ - -This section covers a list of key concepts for monitoring and debugging tools and features in Ray. - -Dashboard (Web UI) ------------------- -Ray supports the web-based dashboard to help users monitor the cluster. When a new cluster is started, the dashboard is available -through the default address `localhost:8265` (port can be automatically incremented if port 8265 is already occupied). - -See :ref:`Ray Dashboard ` for more details. - -Application Logging -------------------- -By default, all stdout and stderr of tasks and actors are streamed to the Ray driver (the entrypoint script that calls ``ray.init``). - -.. literalinclude:: doc_code/app_logging.py - :language: python - -All stdout emitted from the ``print`` method is printed to the driver with a ``(the task or actor repr, the process ID, IP address)`` prefix. - -.. code-block:: bash - - (pid=45601) task - (Actor pid=480956) actor - -See :ref:`Logging ` for more details. - -Driver logs -~~~~~~~~~~~ -An entry point of Ray applications that calls ``ray.init()`` is called a driver. -All the driver logs are handled in the same way as normal Python programs. - -Job logs -~~~~~~~~ -Logs for jobs submitted via the :ref:`Ray Jobs API ` can be retrieved using the ``ray job logs`` :ref:`CLI command ` or using ``JobSubmissionClient.get_logs()`` or ``JobSubmissionClient.tail_job_logs()`` via the :ref:`Python SDK `. -The log file consists of the stdout of the entrypoint command of the job. For the location of the log file on disk, see :ref:`Logging directory structure `. - -.. _ray-worker-logs: - -Worker logs -~~~~~~~~~~~ -Ray's tasks or actors are executed remotely within Ray's worker processes. Ray has special support to improve the visibility of logs produced by workers. - -- By default, all of the tasks/actors stdout and stderr are redirected to the worker log files. Check out :ref:`Logging directory structure ` to learn how Ray's logging directory is structured. -- By default, all of the tasks/actors stdout and stderr that is redirected to worker log files are published to the driver. Drivers display logs generated from its tasks/actors to its stdout and stderr. - -Let's look at a code example to see how this works. - -.. code-block:: python - - import ray - # Initiate a driver. - ray.init() - - @ray.remote - def task(): - print("task") - - ray.get(task.remote()) - -You should be able to see the string `task` from your driver stdout. - -When logs are printed, the process id (pid) and an IP address of the node that executes tasks/actors are printed together. Check out the output below. - -.. code-block:: bash - - (pid=45601) task - -Actor log messages look like the following by default. - -.. code-block:: bash - - (MyActor pid=480956) actor log message - -Accessing Ray States --------------------- -Starting from Ray 2.0, it supports CLI / Python APIs to query the state of resources (e.g., actor, task, object, etc.). - -For example, the following command will summarize the task state of the cluster. - -.. code-block:: bash - - ray summary tasks - -.. code-block:: text - - ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== - Stats: - ------------------------------------ - total_actor_scheduled: 2 - total_actor_tasks: 0 - total_tasks: 2 - - - Table (group by func_name): - ------------------------------------ - FUNC_OR_CLASS_NAME STATE_COUNTS TYPE - 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK - 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK - -The following command will list all the actors from the cluster. - -.. code-block:: bash - - ray list actors - -.. code-block:: text - - ======== List: 2022-07-23 21:29:39.323925 ======== - Stats: - ------------------------------ - Total: 2 - - Table: - ------------------------------ - ACTOR_ID CLASS_NAME NAME PID STATE - 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE - 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE - -See :ref:`Ray State API ` for more details. - -Metrics -------- -Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), -internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), -and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used -to monitor the cluster over time. - -See :ref:`Ray Metrics ` for more details. - -Exceptions ----------- -Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, -the API raises an exception if anything goes wrong with a related task, actor or object. For example, - -- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. -- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). -- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. - -See :ref:`Exceptions Reference ` for more details. - -Debugger --------- -Ray has a built-in debugger that allows you to debug your distributed applications. -It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can -drop into a PDB session that you can then use to: - -- Inspect variables in that context -- Step within that task or actor -- Move up or down the stack - -See :ref:`Ray Debugger ` for more details. - -Monitoring Cluster State and Resource Demands ---------------------------------------------- -You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays - -- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. -- **Autoscaling Status**: The number of nodes that are autoscaling up and down. -- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. - -Here's an example output. - -.. code-block:: shell - - $ ray status - ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== - Node status - --------------------------------------------------------------- - Healthy: - 1 ray.head.default - 2 ray.worker.cpu - Pending: - (no pending nodes) - Recent failures: - (no failures) - - Resources - --------------------------------------------------------------- - Usage: - 0.0/10.0 CPU - 0.00/70.437 GiB memory - 0.00/10.306 GiB object_store_memory - - Demands: - (no resource demands) - -Profiling ---------- -Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. - -See :ref:`Profiling ` for more details. - -Tracing -------- -To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. - -See :ref:`Ray Tracing ` for more details. \ No newline at end of file diff --git a/doc/source/ray-observability/ray-metrics.rst b/doc/source/ray-observability/ray-metrics.rst index 7c8e4b4cb05c9..86770d9c1d33a 100644 --- a/doc/source/ray-observability/ray-metrics.rst +++ b/doc/source/ray-observability/ray-metrics.rst @@ -11,102 +11,6 @@ To help monitor Ray applications, Ray - Exposes metrics in a Prometheus format. We'll call the endpoint to access these metrics a Prometheus endpoint. - Supports custom metrics APIs that resemble Prometheus `metric types `_. -Getting Started ---------------- - -.. tip:: - - The below instructions for Prometheus to enable a basic workflow of running and accessing the dashboard on your local machine. - For more information about how to run Prometheus on a remote cluster, see :ref:`here `. - -Ray exposes its metrics in Prometheus format. This allows us to easily scrape them using Prometheus. - -First, `download Prometheus `_. Make sure to download the correct binary for your operating system. (Ex: darwin for mac osx) - -Then, unzip the archive into a local directory using the following command. - -.. code-block:: bash - - tar xvfz prometheus-*.tar.gz - cd prometheus-* - -Ray exports metrics only when ``ray[default]`` is installed. - -.. code-block:: bash - - pip install "ray[default]" - -Ray provides a prometheus config that works out of the box. After running ray, it can be found at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. - -.. code-block:: yaml - - global: - scrape_interval: 15s - evaluation_interval: 15s - - scrape_configs: - # Scrape from each ray node as defined in the service_discovery.json provided by ray. - - job_name: 'ray' - file_sd_configs: - - files: - - '/tmp/ray/prom_metrics_service_discovery.json' - - -Next, let's start Prometheus. - -.. code-block:: shell - - ./prometheus --config.file=/tmp/ray/session_latest/metrics/prometheus/prometheus.yml - -.. note:: - If you are using mac, you may receive an error at this point about trying to launch an application where the developer has not been verified. See :ref:`this link ` to fix the issue. - -Now, you can access Ray metrics from the default Prometheus url, `http://localhost:9090`. - -See :ref:`here ` for more information on how to set up Prometheus on a Ray Cluster. - -.. _grafana: - -Grafana -------- - -.. tip:: - - The below instructions for Grafana setup to enable a basic workflow of running and accessing the dashboard on your local machine. - For more information about how to run Grafana on a remote cluster, see :ref:`here `. - -Grafana is a tool that supports more advanced visualizations of prometheus metrics and -allows you to create custom dashboards with your favorite metrics. Ray exports some default -configurations which includes a default dashboard showing some of the most valuable metrics -for debugging ray applications. - - -Deploying Grafana -~~~~~~~~~~~~~~~~~ - -First, `download Grafana `_. Follow the instructions on the download page to download the right binary for your operating system. - -Then go to to the location of the binary and run grafana using the built in configuration found in `/tmp/ray/session_latest/metrics/grafana` folder. - -.. code-block:: shell - - ./bin/grafana-server --config /tmp/ray/session_latest/metrics/grafana/grafana.ini web - -Now, you can access grafana using the default grafana url, `http://localhost:3000`. -You can then see the default dashboard by going to dashboards -> manage -> Ray -> Default Dashboard. The same :ref:`metric graphs ` are also accessible via :ref:`Ray Dashboard `. - -.. tip:: - - If this is your first time using Grafana, you can login with the username: `admin` and password `admin`. - -.. image:: images/graphs.png - :align: center - - -See :ref:`here ` for more information on how to set up Grafana on a Ray Cluster. - -.. _system-metrics: - System Metrics -------------- Ray exports a number of system metrics, which provide introspection into the state of Ray workloads, as well as hardware utilization statistics. The following table describes the officially supported metrics: @@ -197,87 +101,3 @@ Metrics Semantics and Consistency Ray guarantees all its internal state metrics are *eventually* consistent even in the presence of failures--- should any worker fail, eventually the right state will be reflected in the Prometheus time-series output. However, any particular metrics query is not guaranteed to reflect an exact snapshot of the cluster state. For the `ray_tasks` and `ray_actors` metrics, you should use sum queries to plot their outputs (e.g., ``sum(ray_tasks) by (Name, State)``). The reason for this is that Ray's task metrics are emitted from multiple distributed components. Hence, there are multiple metric points, including negative metric points, emitted from different processes that must be summed to produce the correct logical view of the distributed system. For example, for a single task submitted and executed, Ray may emit ``(submitter) SUBMITTED_TO_WORKER: 1, (executor) SUBMITTED_TO_WORKER: -1, (executor) RUNNING: 1``, which reduces to ``SUBMITTED_TO_WORKER: 0, RUNNING: 1`` after summation. - -Configurations --------------- - -Customize prometheus export port -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Ray by default provides the service discovery file, but you can directly scrape metrics from prometheus ports. -To do that, you may want to customize the port that metrics gets exposed to a pre-defined port. - -.. code-block:: bash - - ray start --head --metrics-export-port=8080 # Assign metrics export port on a head node. - -Now, you can scrape Ray's metrics using Prometheus via ``:8080``. - -Alternate Prometheus host location -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -You can choose to run Prometheus on a non-default port or on a different machine. When doing so, you should -make sure that prometheus can scrape the metrics from your ray nodes following instructions :ref:`here `. - -In addition, both Ray and Grafana needs to know how to access this prometheus instance. This can be configured -by setting the `RAY_PROMETHEUS_HOST` env var when launching ray. The env var takes in the address to access Prometheus. More -info can be found :ref:`here `. By default, we assume Prometheus is hosted at `localhost:9090`. - -For example, if Prometheus is hosted at port 9000 on a node with ip 55.66.77.88, One should set the value to -`RAY_PROMETHEUS_HOST=http://55.66.77.88:9000`. - - -Alternate Grafana host location -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -You can choose to run Grafana on a non-default port or on a different machine. If you choose to do this, the -:ref:`Dashboard ` needs to be configured with a public address to that service so the web page -can load the graphs. This can be done with the `RAY_GRAFANA_HOST` env var when launching ray. The env var takes -in the address to access Grafana. More info can be found :ref:`here `. Instructions -to use an existing Grafana instance can be found :ref:`here `. - -For the Grafana charts to work on the Ray dashboard, the user of the dashboard's browser must be able to reach -the Grafana service. If this browser cannot reach Grafana the same way the Ray head node can, you can use a separate -env var `RAY_GRAFANA_IFRAME_HOST` to customize the host the browser users to attempt to reach Grafana. If this is not set, -we use the value of `RAY_GRAFANA_HOST` by default. - -For example, if Grafana is hosted at is 55.66.77.88 on port 3000. One should set the value -to `RAY_GRAFANA_HOST=http://55.66.77.88:3000`. - -Troubleshooting ---------------- - -Getting Prometheus and Grafana to use the Ray configurations when installed via homebrew on macOS X -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -With homebrew, Prometheus and Grafana are installed as services that are automatically launched for you. -Therefore, to configure these services, you cannot simply pass in the config files as command line arguments. - -Instead, follow these instructions: -1. Change the --config-file line in `/usr/local/etc/prometheus.args` to read `--config.file /tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. -2. Update `/usr/local/etc/grafana/grafana.ini` file so that it matches the contents of `/tmp/ray/session_latest/metrics/grafana/grafana.ini`. - -You can then start or restart the services with `brew services start grafana` and `brew services start prometheus`. - -.. _unverified-developer: - -MacOS does not trust the developer to install Prometheus or Grafana -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -You may have received an error that looks like this: - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/troubleshooting/prometheus-trusted-developer.png - :align: center - -When downloading binaries from the internet, Mac requires that the binary be signed by a trusted developer ID. -Unfortunately, many developers today are not trusted by Mac and so this requirement must be overridden by the user manaully. - -See `these instructions `_ on how to override the restriction and install or run the application. - -Grafana dashboards are not embedded in the Ray dashboard -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -If you're getting an error that says `RAY_GRAFANA_HOST` is not setup despite having set it up, check that: -You've included the protocol in the URL (e.g., `http://your-grafana-url.com` instead of `your-grafana-url.com`). -The URL doesn't have a trailing slash (e.g., `http://your-grafana-url.com` instead of `http://your-grafana-url.com/`). - -Certificate Authority (CA error) -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -You may see a CA error if your Grafana instance is hosted behind HTTPS. Contact the Grafana service owner to properly enable HTTPS traffic. From f18b2b82410b85ee0b9a0bb3e1545c7c4879fd60 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 12:31:13 -0700 Subject: [PATCH 08/29] forgot to save changes Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index c149b65eaf52d..1ddbe3f10b62f 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -379,6 +379,8 @@ parts: sections: - file: cluster/vms/examples/ml-example - file: cluster/vms/references/index + - file: cluster/configure-manage-dashboard + title: Applications Guide - file: cluster/running-applications/index title: Applications Guide - file: cluster/faq From cacd6198de31b7faec45fb5d22f8eda60ffd7214 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 12:44:12 -0700 Subject: [PATCH 09/29] change title name on side nav Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 1ddbe3f10b62f..df208eff8b3fd 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -380,7 +380,7 @@ parts: - file: cluster/vms/examples/ml-example - file: cluster/vms/references/index - file: cluster/configure-manage-dashboard - title: Applications Guide + title: Configuring and Managing the Dashboard - file: cluster/running-applications/index title: Applications Guide - file: cluster/faq From 794130fd9e590f994ae53b92fb2f7166f5825e4e Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 12:55:51 -0700 Subject: [PATCH 10/29] change title name on side nav Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index df208eff8b3fd..3f7b8bfcfa0bb 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -380,7 +380,6 @@ parts: - file: cluster/vms/examples/ml-example - file: cluster/vms/references/index - file: cluster/configure-manage-dashboard - title: Configuring and Managing the Dashboard - file: cluster/running-applications/index title: Applications Guide - file: cluster/faq From ce42aba8649d8b7ed790d7e998b5e5c9b4accffc Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 13:59:15 -0700 Subject: [PATCH 11/29] changing headers and titles to gerunds for consistency Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../cluster/configure-manage-dashboard.rst | 16 +++++++--------- doc/source/ray-observability/ray-metrics.rst | 2 ++ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/doc/source/cluster/configure-manage-dashboard.rst b/doc/source/cluster/configure-manage-dashboard.rst index fd3f2d4e0a239..7b449e7640724 100644 --- a/doc/source/cluster/configure-manage-dashboard.rst +++ b/doc/source/cluster/configure-manage-dashboard.rst @@ -1,7 +1,7 @@ .. _observability-configure-manage-dashboard: -Configure and Manage the Dashboard -================================== +Configuring and Managing the Dashboard +====================================== Modify the configuration @@ -161,6 +161,9 @@ For example, you can view the p95 duration of all requests with this query: These metrics can be queried via Prometheus or Grafana UI. Instructions on how to set these tools up can be found :ref:`here `. +Integrating with Prometheus and Grafana +--------------------------------------- + Setting up Prometheus --------------------- @@ -255,13 +258,8 @@ You can then see the default dashboard by going to dashboards -> manage -> Ray - See :ref:`here ` for more information on how to set up Grafana on a Ray Cluster. -.. _system-metrics: - -Configurations --------------- - -Customize prometheus export port -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +Customizing the Prometheus export port +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ Ray by default provides the service discovery file, but you can directly scrape metrics from prometheus ports. To do that, you may want to customize the port that metrics gets exposed to a pre-defined port. diff --git a/doc/source/ray-observability/ray-metrics.rst b/doc/source/ray-observability/ray-metrics.rst index 86770d9c1d33a..7f5bbffc140e9 100644 --- a/doc/source/ray-observability/ray-metrics.rst +++ b/doc/source/ray-observability/ray-metrics.rst @@ -11,6 +11,8 @@ To help monitor Ray applications, Ray - Exposes metrics in a Prometheus format. We'll call the endpoint to access these metrics a Prometheus endpoint. - Supports custom metrics APIs that resemble Prometheus `metric types `_. +.. _system-metrics: + System Metrics -------------- Ray exports a number of system metrics, which provide introspection into the state of Ray workloads, as well as hardware utilization statistics. The following table describes the officially supported metrics: From 32618ab8293a690faebaf4f07ddefe068a7ace43 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 14:32:52 -0700 Subject: [PATCH 12/29] remove overview, rename dashboard page to getting started Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 2 + .../cluster/configure-manage-dashboard.rst | 2 +- .../getting-started-dashboard.rst} | 0 .../monitoring-debugging.rst | 2 - doc/source/ray-observability/overview.rst | 156 ------------------ 5 files changed, 3 insertions(+), 159 deletions(-) rename doc/source/{ray-core/ray-dashboard.rst => ray-observability/monitoring-debugging/getting-started-dashboard.rst} (100%) delete mode 100644 doc/source/ray-observability/overview.rst diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 3f7b8bfcfa0bb..3da548de46035 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -389,6 +389,8 @@ parts: title: "Monitoring and Debugging" sections: - file: ray-observability/key-concepts + - file: ray-observability/getting-started-dashboard + - file: ray-observability/getting-started-program - file: ray-observability/user-guides/index title: User Guides sections: diff --git a/doc/source/cluster/configure-manage-dashboard.rst b/doc/source/cluster/configure-manage-dashboard.rst index 7b449e7640724..2cbd1976e7d37 100644 --- a/doc/source/cluster/configure-manage-dashboard.rst +++ b/doc/source/cluster/configure-manage-dashboard.rst @@ -3,7 +3,7 @@ Configuring and Managing the Dashboard ====================================== -Modify the configuration +Setting up the dashboard may require some configuration depending on your use model and cluster environment. Integrations with Prometheus and Grafana are optional for extending visualization capabilities. Port forwarding --------------- diff --git a/doc/source/ray-core/ray-dashboard.rst b/doc/source/ray-observability/monitoring-debugging/getting-started-dashboard.rst similarity index 100% rename from doc/source/ray-core/ray-dashboard.rst rename to doc/source/ray-observability/monitoring-debugging/getting-started-dashboard.rst diff --git a/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst b/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst index aa83829f679dd..d757efffe6819 100644 --- a/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst +++ b/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst @@ -10,8 +10,6 @@ See :ref:`Getting Help ` if your problem is not s .. toctree:: :maxdepth: 0 - ../overview - ../../ray-core/ray-dashboard ../state/state-api ../ray-metrics profiling diff --git a/doc/source/ray-observability/overview.rst b/doc/source/ray-observability/overview.rst deleted file mode 100644 index 8919d3f29b5a5..0000000000000 --- a/doc/source/ray-observability/overview.rst +++ /dev/null @@ -1,156 +0,0 @@ -.. _observability-overview: - -Overview -======== - -This section covers a list of available monitoring and debugging tools and features in Ray. - -This documentation only covers the high-level description of available tools and features. For more details, see :ref:`Ray Observability `. - -Dashboard (Web UI) ------------------- -Ray supports the web-based dashboard to help users monitor the cluster. When a new cluster is started, the dashboard is available -through the default address `localhost:8265` (port can be automatically incremented if port 8265 is already occupied). - -See :ref:`Ray Dashboard ` for more details. - -Application Logging -------------------- -By default, all stdout and stderr of tasks and actors are streamed to the Ray driver (the entrypoint script that calls ``ray.init``). - -.. literalinclude:: doc_code/app_logging.py - :language: python - -All stdout emitted from the ``print`` method is printed to the driver with a ``(the task or actor repr, the process ID, IP address)`` prefix. - -.. code-block:: bash - - (pid=45601) task - (Actor pid=480956) actor - -See :ref:`Logging ` for more details. - -Exceptions ----------- -Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, -the API raises an exception if anything goes wrong with a related task, actor or object. For example, - -- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. -- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). -- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. - -See :ref:`Exceptions Reference ` for more details. - -Accessing Ray States --------------------- -Starting from Ray 2.0, it supports CLI / Python APIs to query the state of resources (e.g., actor, task, object, etc.). - -For example, the following command will summarize the task state of the cluster. - -.. code-block:: bash - - ray summary tasks - -.. code-block:: text - - ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== - Stats: - ------------------------------------ - total_actor_scheduled: 2 - total_actor_tasks: 0 - total_tasks: 2 - - - Table (group by func_name): - ------------------------------------ - FUNC_OR_CLASS_NAME STATE_COUNTS TYPE - 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK - 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK - -The following command will list all the actors from the cluster. - -.. code-block:: bash - - ray list actors - -.. code-block:: text - - ======== List: 2022-07-23 21:29:39.323925 ======== - Stats: - ------------------------------ - Total: 2 - - Table: - ------------------------------ - ACTOR_ID CLASS_NAME NAME PID STATE - 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE - 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE - -See :ref:`Ray State API ` for more details. - -Debugger --------- -Ray has a built-in debugger that allows you to debug your distributed applications. -It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can -drop into a PDB session that you can then use to: - -- Inspect variables in that context -- Step within that task or actor -- Move up or down the stack - -See :ref:`Ray Debugger ` for more details. - -Monitoring Cluster State and Resource Demands ---------------------------------------------- -You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays - -- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. -- **Autoscaling Status**: The number of nodes that are autoscaling up and down. -- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. - -Here's an example output. - -.. code-block:: shell - - $ ray status - ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== - Node status - --------------------------------------------------------------- - Healthy: - 1 ray.head.default - 2 ray.worker.cpu - Pending: - (no pending nodes) - Recent failures: - (no failures) - - Resources - --------------------------------------------------------------- - Usage: - 0.0/10.0 CPU - 0.00/70.437 GiB memory - 0.00/10.306 GiB object_store_memory - - Demands: - (no resource demands) - -Metrics -------- -Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), -internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), -and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used -to monitor the cluster over time. - -See :ref:`Ray Metrics ` for more details. - -Profiling ---------- -Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. - -See :ref:`Profiling ` for more details. - -Tracing -------- -To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. - -See :ref:`Ray Tracing ` for more details. \ No newline at end of file From 1448b4d80d27b6d0fe24fca63399b7663d5bd805 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 15:19:47 -0700 Subject: [PATCH 13/29] fixed file location to move up a level Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../getting-started-dashboard.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) rename doc/source/ray-observability/{monitoring-debugging => }/getting-started-dashboard.rst (99%) diff --git a/doc/source/ray-observability/monitoring-debugging/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started-dashboard.rst similarity index 99% rename from doc/source/ray-observability/monitoring-debugging/getting-started-dashboard.rst rename to doc/source/ray-observability/getting-started-dashboard.rst index aeb988bedae45..c22816f241968 100644 --- a/doc/source/ray-observability/monitoring-debugging/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started-dashboard.rst @@ -1,7 +1,7 @@ -.. _ray-dashboard: +.. _observability-getting-started-dashboard: -Ray Dashboard -============= +Getting started with the dashboard +================================== Ray provides a web-based dashboard for monitoring and debugging Ray applications. The dashboard provides a visual representation of the system state, allowing users to track the performance of their applications and troubleshoot issues. From f19deaf8b2bbe7363ce3f3653b379864761cebf4 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 15:47:50 -0700 Subject: [PATCH 14/29] fixed titles Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 2 +- .../getting-started-dashboard.rst | 4 +- .../getting-started-program.rst | 161 +----------------- 3 files changed, 10 insertions(+), 157 deletions(-) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 3da548de46035..c240236c744f7 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -388,9 +388,9 @@ parts: - file: ray-observability/monitoring-debugging/monitoring-debugging title: "Monitoring and Debugging" sections: - - file: ray-observability/key-concepts - file: ray-observability/getting-started-dashboard - file: ray-observability/getting-started-program + - file: ray-observability/key-concepts - file: ray-observability/user-guides/index title: User Guides sections: diff --git a/doc/source/ray-observability/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started-dashboard.rst index c22816f241968..40510a02df861 100644 --- a/doc/source/ray-observability/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started-dashboard.rst @@ -1,7 +1,7 @@ .. _observability-getting-started-dashboard: -Getting started with the dashboard -================================== +Getting started (dashboard) +=========================== Ray provides a web-based dashboard for monitoring and debugging Ray applications. The dashboard provides a visual representation of the system state, allowing users to track the performance of their applications and troubleshoot issues. diff --git a/doc/source/ray-observability/getting-started-program.rst b/doc/source/ray-observability/getting-started-program.rst index 622263cc3e4f8..e852a165230ed 100644 --- a/doc/source/ray-observability/getting-started-program.rst +++ b/doc/source/ray-observability/getting-started-program.rst @@ -1,85 +1,16 @@ -.. _observability-key-concepts: +.. _observability-getting-started-program: -Key Concepts -============ +Getting started (programmatically) +================================== -This section covers a list of key concepts for monitoring and debugging tools and features in Ray. +Monitoring and debugging capabilities in Ray are available through an API, CLI, or SDK. -Dashboard (Web UI) ------------------- -Ray supports the web-based dashboard to help users monitor the cluster. When a new cluster is started, the dashboard is available -through the default address `localhost:8265` (port can be automatically incremented if port 8265 is already occupied). - -See :ref:`Ray Dashboard ` for more details. - -Application Logging -------------------- -By default, all stdout and stderr of tasks and actors are streamed to the Ray driver (the entrypoint script that calls ``ray.init``). - -.. literalinclude:: doc_code/app_logging.py - :language: python - -All stdout emitted from the ``print`` method is printed to the driver with a ``(the task or actor repr, the process ID, IP address)`` prefix. - -.. code-block:: bash - - (pid=45601) task - (Actor pid=480956) actor - -See :ref:`Logging ` for more details. - -Driver logs -~~~~~~~~~~~ -An entry point of Ray applications that calls ``ray.init()`` is called a driver. -All the driver logs are handled in the same way as normal Python programs. - -Job logs -~~~~~~~~ -Logs for jobs submitted via the :ref:`Ray Jobs API ` can be retrieved using the ``ray job logs`` :ref:`CLI command ` or using ``JobSubmissionClient.get_logs()`` or ``JobSubmissionClient.tail_job_logs()`` via the :ref:`Python SDK `. -The log file consists of the stdout of the entrypoint command of the job. For the location of the log file on disk, see :ref:`Logging directory structure `. - -.. _ray-worker-logs: - -Worker logs -~~~~~~~~~~~ -Ray's tasks or actors are executed remotely within Ray's worker processes. Ray has special support to improve the visibility of logs produced by workers. - -- By default, all of the tasks/actors stdout and stderr are redirected to the worker log files. Check out :ref:`Logging directory structure ` to learn how Ray's logging directory is structured. -- By default, all of the tasks/actors stdout and stderr that is redirected to worker log files are published to the driver. Drivers display logs generated from its tasks/actors to its stdout and stderr. - -Let's look at a code example to see how this works. - -.. code-block:: python - - import ray - # Initiate a driver. - ray.init() - - @ray.remote - def task(): - print("task") - - ray.get(task.remote()) - -You should be able to see the string `task` from your driver stdout. - -When logs are printed, the process id (pid) and an IP address of the node that executes tasks/actors are printed together. Check out the output below. - -.. code-block:: bash - - (pid=45601) task - -Actor log messages look like the following by default. - -.. code-block:: bash - - (MyActor pid=480956) actor log message Accessing Ray States -------------------- -Starting from Ray 2.0, it supports CLI / Python APIs to query the state of resources (e.g., actor, task, object, etc.). +Ray 2.0 and later versions support CLI and Python APIs for querying the state of resources (e.g., actor, task, object, etc.) -For example, the following command will summarize the task state of the cluster. +For example, the following command summarizes the task state of the cluster: .. code-block:: bash @@ -101,7 +32,7 @@ For example, the following command will summarize the task state of the cluster. 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK -The following command will list all the actors from the cluster. +The following command lists all the actors from the cluster: .. code-block:: bash @@ -121,81 +52,3 @@ The following command will list all the actors from the cluster. 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE See :ref:`Ray State API ` for more details. - -Metrics -------- -Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), -internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), -and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used -to monitor the cluster over time. - -See :ref:`Ray Metrics ` for more details. - -Exceptions ----------- -Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, -the API raises an exception if anything goes wrong with a related task, actor or object. For example, - -- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. -- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). -- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. - -See :ref:`Exceptions Reference ` for more details. - -Debugger --------- -Ray has a built-in debugger that allows you to debug your distributed applications. -It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can -drop into a PDB session that you can then use to: - -- Inspect variables in that context -- Step within that task or actor -- Move up or down the stack - -See :ref:`Ray Debugger ` for more details. - -Monitoring Cluster State and Resource Demands ---------------------------------------------- -You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays - -- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. -- **Autoscaling Status**: The number of nodes that are autoscaling up and down. -- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. - -Here's an example output. - -.. code-block:: shell - - $ ray status - ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== - Node status - --------------------------------------------------------------- - Healthy: - 1 ray.head.default - 2 ray.worker.cpu - Pending: - (no pending nodes) - Recent failures: - (no failures) - - Resources - --------------------------------------------------------------- - Usage: - 0.0/10.0 CPU - 0.00/70.437 GiB memory - 0.00/10.306 GiB object_store_memory - - Demands: - (no resource demands) - -Profiling ---------- -Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. - -See :ref:`Profiling ` for more details. - -Tracing -------- -To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. - -See :ref:`Ray Tracing ` for more details. \ No newline at end of file From 408bc0fe3eeaa123887f388e59ca2941c25078b8 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 18:17:53 -0700 Subject: [PATCH 15/29] another iteration of titles to fit side nav better Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/ray-observability/getting-started-dashboard.rst | 2 +- doc/source/ray-observability/getting-started-program.rst | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/doc/source/ray-observability/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started-dashboard.rst index 40510a02df861..ddc5fcbb28908 100644 --- a/doc/source/ray-observability/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started-dashboard.rst @@ -1,6 +1,6 @@ .. _observability-getting-started-dashboard: -Getting started (dashboard) +Getting Started (Dashboard) =========================== Ray provides a web-based dashboard for monitoring and debugging Ray applications. The dashboard provides a visual representation of the system state, allowing users to track the performance diff --git a/doc/source/ray-observability/getting-started-program.rst b/doc/source/ray-observability/getting-started-program.rst index e852a165230ed..2c6d32206198e 100644 --- a/doc/source/ray-observability/getting-started-program.rst +++ b/doc/source/ray-observability/getting-started-program.rst @@ -1,7 +1,7 @@ .. _observability-getting-started-program: -Getting started (programmatically) -================================== +Getting Started (API, CLI, SDK) +=============================== Monitoring and debugging capabilities in Ray are available through an API, CLI, or SDK. From e78de8d5a5d0c23818813d650b75c843ba1919e8 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 18:52:01 -0700 Subject: [PATCH 16/29] create reference subdirectory; moved content for cli GS and metrics Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 6 + .../getting-started-program.rst | 621 +++++++++++++++++- doc/source/ray-observability/key-concepts.rst | 129 ++-- .../{api/state => reference}/api.rst | 0 .../{api/state => reference}/cli.rst | 0 .../ray-observability/reference/index.md | 9 + .../system-metrics.rst} | 13 - .../ray-observability/state/state-api.rst | 616 ----------------- 8 files changed, 674 insertions(+), 720 deletions(-) rename doc/source/ray-observability/{api/state => reference}/api.rst (100%) rename doc/source/ray-observability/{api/state => reference}/cli.rst (100%) create mode 100644 doc/source/ray-observability/reference/index.md rename doc/source/ray-observability/{ray-metrics.rst => reference/system-metrics.rst} (93%) delete mode 100644 doc/source/ray-observability/state/state-api.rst diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 01a2bf4d06431..b06ea7a9e0fa4 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -406,6 +406,12 @@ parts: - file: ray-observability/user-guides/configure-logging - file: ray-observability/user-guides/add-app-metrics - file: ray-observability/user-guides/ray-tracing + - file: ray-observability/reference/index + title: Reference + sections: + - file: ray-observability/reference/system-metrics + - file: ray-observability/reference/api + - file: ray-observability/reference/cli - file: ray-references/api title: References diff --git a/doc/source/ray-observability/getting-started-program.rst b/doc/source/ray-observability/getting-started-program.rst index 2c6d32206198e..8157b20c072ca 100644 --- a/doc/source/ray-observability/getting-started-program.rst +++ b/doc/source/ray-observability/getting-started-program.rst @@ -6,15 +6,100 @@ Getting Started (API, CLI, SDK) Monitoring and debugging capabilities in Ray are available through an API, CLI, or SDK. -Accessing Ray States --------------------- -Ray 2.0 and later versions support CLI and Python APIs for querying the state of resources (e.g., actor, task, object, etc.) +Monitoring Cluster State and Resource Demands +--------------------------------------------- +You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays -For example, the following command summarizes the task state of the cluster: +- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. +- **Autoscaling Status**: The number of nodes that are autoscaling up and down. +- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. -.. code-block:: bash +Here's an example output. - ray summary tasks +.. code-block:: shell + + $ ray status + ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== + Node status + --------------------------------------------------------------- + Healthy: + 1 ray.head.default + 2 ray.worker.cpu + Pending: + (no pending nodes) + Recent failures: + (no failures) + + Resources + --------------------------------------------------------------- + Usage: + 0.0/10.0 CPU + 0.00/70.437 GiB memory + 0.00/10.306 GiB object_store_memory + + Demands: + (no resource demands) + +.. _state-api-overview-ref: + +Monitoring Ray States +===================== + +.. tip:: We'd love to hear your feedback on using Ray state APIs - `feedback form `_! + +Ray state APIs allow users to conveniently access the current state (snapshot) of Ray through CLI or Python SDK (developer APIs). + +.. note:: + + This feature requires a full installation of Ray using ``pip install "ray[default]"``. This feature also requires the dashboard component to be available. The dashboard component needs to be included when starting the Ray cluster, which is the default behavior for ``ray start`` and ``ray.init()``. For more in-depth debugging, check the dashboard log at ``/dashboard.log``, which is usually ``/tmp/ray/session_latest/logs/dashboard.log``. + +.. note:: + + State API CLI commands are :ref:`stable `, while python SDKs are :ref:`DeveloperAPI `. CLI usage is recommended over Python SDKs. + +Run any workload. In this example, you will use the following script that runs 2 tasks and creates 2 actors. + +.. code-block:: python + + import ray + import time + + ray.init(num_cpus=4) + + @ray.remote + def task_running_300_seconds(): + print("Start!") + time.sleep(300) + + @ray.remote + class Actor: + def __init__(self): + print("Actor created") + + # Create 2 tasks + tasks = [task_running_300_seconds.remote() for _ in range(2)] + + # Create 2 actors + actors = [Actor.remote() for _ in range(2)] + + ray.get(tasks) + +Now, let's see the summarized states of tasks. If it doesn't return the output immediately, retry the command. + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray summary tasks + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import summarize_tasks + print(summarize_tasks()) .. code-block:: text @@ -32,11 +117,22 @@ For example, the following command summarizes the task state of the cluster: 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK -The following command lists all the actors from the cluster: +Let's list all actors. + +.. tabs:: -.. code-block:: bash + .. group-tab:: CLI (Recommended) - ray list actors + .. code-block:: bash + + ray list actors + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_actors + print(list_actors()) .. code-block:: text @@ -51,4 +147,509 @@ The following command lists all the actors from the cluster: 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE -See :ref:`Ray State API ` for more details. +You can get the state of a single task using the get API. + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + # In this case, 31405554844820381c2f0f8501000000 + ray get actors + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_actor + # In this case, 31405554844820381c2f0f8501000000 + print(get_actor(id=)) + + +.. code-block:: text + + --- + actor_id: 31405554844820381c2f0f8501000000 + class_name: Actor + death_cause: null + is_detached: false + name: '' + pid: 96956 + resource_mapping: [] + serialized_runtime_env: '{}' + state: ALIVE + +You can also access logs through ``ray logs`` API. + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list actors + # In this case, ACTOR_ID is 31405554844820381c2f0f8501000000 + ray logs actor --id + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_log + + # In this case, ACTOR_ID is 31405554844820381c2f0f8501000000 + for line in get_log(actor_id=): + print(line) + +.. code-block:: text + + --- Log has been truncated to last 1000 lines. Use `--tail` flag to toggle. --- + + :actor_name:Actor + Actor created + + +Key Concepts +------------ +Ray state APIs allow you to access **states** of **resources** through **summary**, **list**, and **get** APIs. It also supports **logs** API to access logs. + +- **states**: The state of the cluster of corresponding resources. States consist of immutable metadata (e.g., actor's name) and mutable states (e.g., actor's scheduling state or pid). +- **resources**: Resources created by Ray. E.g., actors, tasks, objects, placement groups, and etc. +- **summary**: API to return the summarized view of resources. +- **list**: API to return every individual entity of resources. +- **get**: API to return a single entity of resources in detail. +- **logs**: API to access the log of actors, tasks, workers, or system log files. + +Summary +------- +Return the summarized information of the given Ray resource (objects, actors, tasks). +It is recommended to start monitoring states through summary APIs first. When you find anomalies +(e.g., actors running for a long time, tasks that are not scheduled for a long time), +you can use ``list`` or ``get`` APIs to get more details for an individual abnormal resource. + +E.g., Summarize all actors +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray summary actors + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import summarize_actors + print(summarize_actors()) + +E.g., Summarize all tasks +~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray summary tasks + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import summarize_tasks + print(summarize_tasks()) + +E.g., Summarize all objects +~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. note:: + + By default, objects are summarized by callsite. However, callsite is not recorded by Ray by default. + To get callsite info, set env variable `RAY_record_ref_creation_sites=1` when starting the ray cluster + RAY_record_ref_creation_sites=1 ray start --head + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray summary objects + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import summarize_objects + print(summarize_objects()) + +List +---- + +Get a list of resources, possible resources include: + +- :ref:`Actors `, e.g., actor id, state, pid, death_cause. (:class:`output schema `) +- :ref:`Tasks `, e.g., name, scheduling state, type, runtime env info (:class:`output schema `) +- :ref:`Objects `, e.g., object id, callsites, reference types. (:class:`output schema `) +- :ref:`Jobs `, e.g., start/end time, entrypoint, status. (:class:`output schema `) +- :ref:`Placement Groups `, e.g., name, bundles, stats. (:class:`output schema `) +- Nodes (Ray worker nodes), e.g., node id, node ip, node state. (:class:`output schema `) +- Workers (Ray worker processes), e.g., worker id, type, exit type and details. (:class:`output schema `) +- :ref:`Runtime environments `, e.g., runtime envs, creation time, nodes (:class:`output schema `) + +E.g., List all nodes +~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list nodes + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_nodes() + list_nodes() + +E.g., List all placement groups +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list placement-groups + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_placement_groups + list_placement_groups() + + +E.g., List local referenced objects created by a process +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tip:: You can list resources with one or multiple filters: using `--filter` or `-f` + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list objects -f pid= -f reference_type=LOCAL_REFERENCE + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_objects + list_objects(filters=[("pid", "=", ), ("reference_type", "=", "LOCAL_REFERENCE")]) + +E.g., List alive actors +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list actors -f state=ALIVE + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_actors + list_actors(filters=[("state", "=", "ALIVE")]) + +E.g., List running tasks +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list tasks -f state=RUNNING + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_tasks + list_tasks(filters=[("state", "=", "RUNNING")]) + +E.g., List non-running tasks +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list tasks -f state!=RUNNING + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_tasks + list_tasks(filters=[("state", "!=", "RUNNING")]) + +E.g., List running tasks that have a name func +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list tasks -f state=RUNNING -f name="task_running_300_seconds()" + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_tasks + list_tasks(filters=[("state", "=", "RUNNING"), ("name", "=", "task_running_300_seconds()")]) + +E.g., List tasks with more details +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tip:: When ``--detail`` is specified, the API can query more data sources to obtain state information in details. + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray list tasks --detail + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import list_tasks + list_tasks(detail=True) + +Get +--- + +E.g., Get a task info +~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray get tasks + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_task + get_task(id=) + +E.g., Get a node info +~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray get nodes + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_node + get_node(id=) + +Logs +---- + +.. _state-api-log-doc: + +State API also allows you to conveniently access ray logs. Note that you cannot access the logs from a dead node. +By default, the API prints log from a head node. + +E.g., Get all retrievable log file names from a head node in a cluster +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray logs cluster + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + # You could get the node id / node ip from `ray list nodes` + from ray.util.state import list_logs + # `ray logs` by default print logs from a head node. + # So in order to list the same logs, you should provide the head node id. + # You could get the node id / node ip from `ray list nodes` + list_logs(node_id=) + +E.g., Get a particular log file from a node +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + # You could get the node id / node ip from `ray list nodes` + ray logs cluster gcs_server.out --node-id + # `ray logs cluster` is alias to `ray logs` when querying with globs. + ray logs gcs_server.out --node-id + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_log + + # Node IP could be retrieved from list_nodes() or ray.nodes() + for line in get_log(filename="gcs_server.out", node_id=): + print(line) + +E.g., Stream a log file from a node +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + # You could get the node id / node ip from `ray list nodes` + ray logs raylet.out --node-ip --follow + # Or, + ray logs cluster raylet.out --node-ip --follow + + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_log + + # Node IP could be retrieved from list_nodes() or ray.nodes() + # The loop will block with `follow=True` + for line in get_log(filename="raylet.out", node_ip=, follow=True): + print(line) + +E.g., Stream log from an actor with actor id +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray logs actor --id= --follow + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_log + + # You could get the actor's ID from the output of `ray list actors`. + # The loop will block with `follow=True` + for line in get_log(actor_id=, follow=True): + print(line) + +E.g., Stream log from a pid +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. tabs:: + + .. group-tab:: CLI (Recommended) + + .. code-block:: bash + + ray logs worker --pid= --follow + + .. group-tab:: Python SDK (Internal Developer API) + + .. code-block:: python + + from ray.util.state import get_log + + # Node IP could be retrieved from list_nodes() or ray.nodes() + # You could get the pid of the worker running the actor easily when output + # of worker being directed to the driver (default) + # The loop will block with `follow=True` + for line in get_log(pid=, node_ip=, follow=True): + print(line) + +Failure Semantics +----------------- + +The state APIs don't guarantee to return a consistent/complete snapshot of the cluster all the time. By default, +all Python SDKs raise an exception when there's a missing output from the API. And CLI returns a partial result +and provides warning messages. Here are cases where there can be missing output from the API. + +Query Failures +~~~~~~~~~~~~~~ + +State APIs query "data sources" (e.g., GCS, raylets, etc.) to obtain and build the snapshot of the cluster. +However, data sources are sometimes unavailable (e.g., the source is down or overloaded). In this case, APIs +will return a partial (incomplete) snapshot of the cluster, and users are informed that the output is incomplete through a warning message. +All warnings are printed through Python's ``warnings`` library, and they can be suppressed. + +Data Truncation +~~~~~~~~~~~~~~~ + +When the returned number of entities (number of rows) is too large (> 100K), state APIs truncate the output data to ensure system stability +(when this happens, there's no way to choose truncated data). When truncation happens it will be informed through Python's +``warnings`` module. + +Garbage Collected Resources +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Depending on the lifecycle of the resources, some "finished" resources are not accessible +through the APIs because they are already garbage collected. +**It is recommended not to rely on this API to obtain correct information on finished resources**. +For example, Ray periodically garbage collects DEAD state actor data to reduce memory usage. +Or it cleans up the FINISHED state of tasks when its lineage goes out of scope. + +API Reference +------------- + +- For the CLI Reference, see :ref:`State CLI Refernece `. +- For the SDK Reference, see :ref:`State API Reference `. +- For the Log CLI Reference, see :ref:`Log CLI Reference `. \ No newline at end of file diff --git a/doc/source/ray-observability/key-concepts.rst b/doc/source/ray-observability/key-concepts.rst index 622263cc3e4f8..454cbe6d6ea22 100644 --- a/doc/source/ray-observability/key-concepts.rst +++ b/doc/source/ray-observability/key-concepts.rst @@ -12,6 +12,54 @@ through the default address `localhost:8265` (port can be automatically incremen See :ref:`Ray Dashboard ` for more details. +Accessing Ray States +-------------------- +Ray 2.0 and later versions support CLI and Python APIs for querying the state of resources (e.g., actor, task, object, etc.) + +For example, the following command summarizes the task state of the cluster: + +.. code-block:: bash + + ray summary tasks + +.. code-block:: text + + ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== + Stats: + ------------------------------------ + total_actor_scheduled: 2 + total_actor_tasks: 0 + total_tasks: 2 + + + Table (group by func_name): + ------------------------------------ + FUNC_OR_CLASS_NAME STATE_COUNTS TYPE + 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK + 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK + +The following command lists all the actors from the cluster: + +.. code-block:: bash + + ray list actors + +.. code-block:: text + + ======== List: 2022-07-23 21:29:39.323925 ======== + Stats: + ------------------------------ + Total: 2 + + Table: + ------------------------------ + ACTOR_ID CLASS_NAME NAME PID STATE + 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE + 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE + +See :ref:`Ray State API ` for more details. + + Application Logging ------------------- By default, all stdout and stderr of tasks and actors are streamed to the Ray driver (the entrypoint script that calls ``ray.init``). @@ -75,53 +123,6 @@ Actor log messages look like the following by default. (MyActor pid=480956) actor log message -Accessing Ray States --------------------- -Starting from Ray 2.0, it supports CLI / Python APIs to query the state of resources (e.g., actor, task, object, etc.). - -For example, the following command will summarize the task state of the cluster. - -.. code-block:: bash - - ray summary tasks - -.. code-block:: text - - ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== - Stats: - ------------------------------------ - total_actor_scheduled: 2 - total_actor_tasks: 0 - total_tasks: 2 - - - Table (group by func_name): - ------------------------------------ - FUNC_OR_CLASS_NAME STATE_COUNTS TYPE - 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK - 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK - -The following command will list all the actors from the cluster. - -.. code-block:: bash - - ray list actors - -.. code-block:: text - - ======== List: 2022-07-23 21:29:39.323925 ======== - Stats: - ------------------------------ - Total: 2 - - Table: - ------------------------------ - ACTOR_ID CLASS_NAME NAME PID STATE - 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE - 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE - -See :ref:`Ray State API ` for more details. - Metrics ------- Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), @@ -154,40 +155,6 @@ drop into a PDB session that you can then use to: See :ref:`Ray Debugger ` for more details. -Monitoring Cluster State and Resource Demands ---------------------------------------------- -You can monitor cluster usage and auto-scaling status by running (on the head node) a CLI command ``ray status``. It displays - -- **Cluster State**: Nodes that are up and running. Addresses of running nodes. Information about pending nodes and failed nodes. -- **Autoscaling Status**: The number of nodes that are autoscaling up and down. -- **Cluster Usage**: The resource usage of the cluster. E.g., requested CPUs from all Ray tasks and actors. Number of GPUs that are used. - -Here's an example output. - -.. code-block:: shell - - $ ray status - ======== Autoscaler status: 2021-10-12 13:10:21.035674 ======== - Node status - --------------------------------------------------------------- - Healthy: - 1 ray.head.default - 2 ray.worker.cpu - Pending: - (no pending nodes) - Recent failures: - (no failures) - - Resources - --------------------------------------------------------------- - Usage: - 0.0/10.0 CPU - 0.00/70.437 GiB memory - 0.00/10.306 GiB object_store_memory - - Demands: - (no resource demands) - Profiling --------- Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. diff --git a/doc/source/ray-observability/api/state/api.rst b/doc/source/ray-observability/reference/api.rst similarity index 100% rename from doc/source/ray-observability/api/state/api.rst rename to doc/source/ray-observability/reference/api.rst diff --git a/doc/source/ray-observability/api/state/cli.rst b/doc/source/ray-observability/reference/cli.rst similarity index 100% rename from doc/source/ray-observability/api/state/cli.rst rename to doc/source/ray-observability/reference/cli.rst diff --git a/doc/source/ray-observability/reference/index.md b/doc/source/ray-observability/reference/index.md new file mode 100644 index 0000000000000..448c3f38cac21 --- /dev/null +++ b/doc/source/ray-observability/reference/index.md @@ -0,0 +1,9 @@ +(observability-reference)= + +# Reference + +Monitor and debug your Ray applications and clusters using the API and CLI documented in these references. + +The guides include: +* {ref}`observability-state-api-ref` +* {ref}`observability-state-api-cli-ref` \ No newline at end of file diff --git a/doc/source/ray-observability/ray-metrics.rst b/doc/source/ray-observability/reference/system-metrics.rst similarity index 93% rename from doc/source/ray-observability/ray-metrics.rst rename to doc/source/ray-observability/reference/system-metrics.rst index 7f5bbffc140e9..fe0a4e458b8e6 100644 --- a/doc/source/ray-observability/ray-metrics.rst +++ b/doc/source/ray-observability/reference/system-metrics.rst @@ -1,16 +1,3 @@ -.. _ray-metrics: - -Metrics -======= - -To help monitor Ray applications, Ray - -- Collects system-level metrics. -- Provides a default configuration for prometheus. -- Provides a default Grafana dashboard. -- Exposes metrics in a Prometheus format. We'll call the endpoint to access these metrics a Prometheus endpoint. -- Supports custom metrics APIs that resemble Prometheus `metric types `_. - .. _system-metrics: System Metrics diff --git a/doc/source/ray-observability/state/state-api.rst b/doc/source/ray-observability/state/state-api.rst deleted file mode 100644 index 33f5bfa4f2d92..0000000000000 --- a/doc/source/ray-observability/state/state-api.rst +++ /dev/null @@ -1,616 +0,0 @@ -.. _state-api-overview-ref: - -Monitoring Ray States -===================== - -.. tip:: We'd love to hear your feedback on using Ray state APIs - `feedback form `_! - -Ray state APIs allow users to conveniently access the current state (snapshot) of Ray through CLI or Python SDK (developer APIs). - -.. note:: - - This feature requires a full installation of Ray using ``pip install "ray[default]"``. This feature also requires the dashboard component to be available. The dashboard component needs to be included when starting the Ray cluster, which is the default behavior for ``ray start`` and ``ray.init()``. For more in-depth debugging, check the dashboard log at ``/dashboard.log``, which is usually ``/tmp/ray/session_latest/logs/dashboard.log``. - -.. note:: - - State API CLI commands are :ref:`stable `, while python SDKs are :ref:`DeveloperAPI `. CLI usage is recommended over Python SDKs. - -Getting Started ---------------- - -Run any workload. In this example, you will use the following script that runs 2 tasks and creates 2 actors. - -.. code-block:: python - - import ray - import time - - ray.init(num_cpus=4) - - @ray.remote - def task_running_300_seconds(): - print("Start!") - time.sleep(300) - - @ray.remote - class Actor: - def __init__(self): - print("Actor created") - - # Create 2 tasks - tasks = [task_running_300_seconds.remote() for _ in range(2)] - - # Create 2 actors - actors = [Actor.remote() for _ in range(2)] - - ray.get(tasks) - -Now, let's see the summarized states of tasks. If it doesn't return the output immediately, retry the command. - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray summary tasks - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import summarize_tasks - print(summarize_tasks()) - -.. code-block:: text - - ======== Tasks Summary: 2022-07-22 08:54:38.332537 ======== - Stats: - ------------------------------------ - total_actor_scheduled: 2 - total_actor_tasks: 0 - total_tasks: 2 - - - Table (group by func_name): - ------------------------------------ - FUNC_OR_CLASS_NAME STATE_COUNTS TYPE - 0 task_running_300_seconds RUNNING: 2 NORMAL_TASK - 1 Actor.__init__ FINISHED: 2 ACTOR_CREATION_TASK - -Let's list all actors. - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list actors - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_actors - print(list_actors()) - -.. code-block:: text - - ======== List: 2022-07-23 21:29:39.323925 ======== - Stats: - ------------------------------ - Total: 2 - - Table: - ------------------------------ - ACTOR_ID CLASS_NAME NAME PID STATE - 0 31405554844820381c2f0f8501000000 Actor 96956 ALIVE - 1 f36758a9f8871a9ca993b1d201000000 Actor 96955 ALIVE - -You can get the state of a single task using the get API. - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - # In this case, 31405554844820381c2f0f8501000000 - ray get actors - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_actor - # In this case, 31405554844820381c2f0f8501000000 - print(get_actor(id=)) - - -.. code-block:: text - - --- - actor_id: 31405554844820381c2f0f8501000000 - class_name: Actor - death_cause: null - is_detached: false - name: '' - pid: 96956 - resource_mapping: [] - serialized_runtime_env: '{}' - state: ALIVE - -You can also access logs through ``ray logs`` API. - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list actors - # In this case, ACTOR_ID is 31405554844820381c2f0f8501000000 - ray logs actor --id - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_log - - # In this case, ACTOR_ID is 31405554844820381c2f0f8501000000 - for line in get_log(actor_id=): - print(line) - -.. code-block:: text - - --- Log has been truncated to last 1000 lines. Use `--tail` flag to toggle. --- - - :actor_name:Actor - Actor created - - -Key Concepts ------------- -Ray state APIs allow you to access **states** of **resources** through **summary**, **list**, and **get** APIs. It also supports **logs** API to access logs. - -- **states**: The state of the cluster of corresponding resources. States consist of immutable metadata (e.g., actor's name) and mutable states (e.g., actor's scheduling state or pid). -- **resources**: Resources created by Ray. E.g., actors, tasks, objects, placement groups, and etc. -- **summary**: API to return the summarized view of resources. -- **list**: API to return every individual entity of resources. -- **get**: API to return a single entity of resources in detail. -- **logs**: API to access the log of actors, tasks, workers, or system log files. - -Summary -------- -Return the summarized information of the given Ray resource (objects, actors, tasks). -It is recommended to start monitoring states through summary APIs first. When you find anomalies -(e.g., actors running for a long time, tasks that are not scheduled for a long time), -you can use ``list`` or ``get`` APIs to get more details for an individual abnormal resource. - -E.g., Summarize all actors -~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray summary actors - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import summarize_actors - print(summarize_actors()) - -E.g., Summarize all tasks -~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray summary tasks - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import summarize_tasks - print(summarize_tasks()) - -E.g., Summarize all objects -~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. note:: - - By default, objects are summarized by callsite. However, callsite is not recorded by Ray by default. - To get callsite info, set env variable `RAY_record_ref_creation_sites=1` when starting the ray cluster - RAY_record_ref_creation_sites=1 ray start --head - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray summary objects - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import summarize_objects - print(summarize_objects()) - -List ----- - -Get a list of resources, possible resources include: - -- :ref:`Actors `, e.g., actor id, state, pid, death_cause. (:class:`output schema `) -- :ref:`Tasks `, e.g., name, scheduling state, type, runtime env info (:class:`output schema `) -- :ref:`Objects `, e.g., object id, callsites, reference types. (:class:`output schema `) -- :ref:`Jobs `, e.g., start/end time, entrypoint, status. (:class:`output schema `) -- :ref:`Placement Groups `, e.g., name, bundles, stats. (:class:`output schema `) -- Nodes (Ray worker nodes), e.g., node id, node ip, node state. (:class:`output schema `) -- Workers (Ray worker processes), e.g., worker id, type, exit type and details. (:class:`output schema `) -- :ref:`Runtime environments `, e.g., runtime envs, creation time, nodes (:class:`output schema `) - -E.g., List all nodes -~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list nodes - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_nodes() - list_nodes() - -E.g., List all placement groups -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list placement-groups - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_placement_groups - list_placement_groups() - - -E.g., List local referenced objects created by a process -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tip:: You can list resources with one or multiple filters: using `--filter` or `-f` - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list objects -f pid= -f reference_type=LOCAL_REFERENCE - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_objects - list_objects(filters=[("pid", "=", ), ("reference_type", "=", "LOCAL_REFERENCE")]) - -E.g., List alive actors -~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list actors -f state=ALIVE - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_actors - list_actors(filters=[("state", "=", "ALIVE")]) - -E.g., List running tasks -~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list tasks -f state=RUNNING - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_tasks - list_tasks(filters=[("state", "=", "RUNNING")]) - -E.g., List non-running tasks -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list tasks -f state!=RUNNING - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_tasks - list_tasks(filters=[("state", "!=", "RUNNING")]) - -E.g., List running tasks that have a name func -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list tasks -f state=RUNNING -f name="task_running_300_seconds()" - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_tasks - list_tasks(filters=[("state", "=", "RUNNING"), ("name", "=", "task_running_300_seconds()")]) - -E.g., List tasks with more details -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tip:: When ``--detail`` is specified, the API can query more data sources to obtain state information in details. - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray list tasks --detail - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import list_tasks - list_tasks(detail=True) - -Get ---- - -E.g., Get a task info -~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray get tasks - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_task - get_task(id=) - -E.g., Get a node info -~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray get nodes - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_node - get_node(id=) - -Logs ----- - -.. _state-api-log-doc: - -State API also allows you to conveniently access ray logs. Note that you cannot access the logs from a dead node. -By default, the API prints log from a head node. - -E.g., Get all retrievable log file names from a head node in a cluster -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray logs cluster - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - # You could get the node id / node ip from `ray list nodes` - from ray.util.state import list_logs - # `ray logs` by default print logs from a head node. - # So in order to list the same logs, you should provide the head node id. - # You could get the node id / node ip from `ray list nodes` - list_logs(node_id=) - -E.g., Get a particular log file from a node -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - # You could get the node id / node ip from `ray list nodes` - ray logs cluster gcs_server.out --node-id - # `ray logs cluster` is alias to `ray logs` when querying with globs. - ray logs gcs_server.out --node-id - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_log - - # Node IP could be retrieved from list_nodes() or ray.nodes() - for line in get_log(filename="gcs_server.out", node_id=): - print(line) - -E.g., Stream a log file from a node -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - # You could get the node id / node ip from `ray list nodes` - ray logs raylet.out --node-ip --follow - # Or, - ray logs cluster raylet.out --node-ip --follow - - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_log - - # Node IP could be retrieved from list_nodes() or ray.nodes() - # The loop will block with `follow=True` - for line in get_log(filename="raylet.out", node_ip=, follow=True): - print(line) - -E.g., Stream log from an actor with actor id -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray logs actor --id= --follow - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_log - - # You could get the actor's ID from the output of `ray list actors`. - # The loop will block with `follow=True` - for line in get_log(actor_id=, follow=True): - print(line) - -E.g., Stream log from a pid -~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. tabs:: - - .. group-tab:: CLI (Recommended) - - .. code-block:: bash - - ray logs worker --pid= --follow - - .. group-tab:: Python SDK (Internal Developer API) - - .. code-block:: python - - from ray.util.state import get_log - - # Node IP could be retrieved from list_nodes() or ray.nodes() - # You could get the pid of the worker running the actor easily when output - # of worker being directed to the driver (default) - # The loop will block with `follow=True` - for line in get_log(pid=, node_ip=, follow=True): - print(line) - -Failure Semantics ------------------ - -The state APIs don't guarantee to return a consistent/complete snapshot of the cluster all the time. By default, -all Python SDKs raise an exception when there's a missing output from the API. And CLI returns a partial result -and provides warning messages. Here are cases where there can be missing output from the API. - -Query Failures -~~~~~~~~~~~~~~ - -State APIs query "data sources" (e.g., GCS, raylets, etc.) to obtain and build the snapshot of the cluster. -However, data sources are sometimes unavailable (e.g., the source is down or overloaded). In this case, APIs -will return a partial (incomplete) snapshot of the cluster, and users are informed that the output is incomplete through a warning message. -All warnings are printed through Python's ``warnings`` library, and they can be suppressed. - -Data Truncation -~~~~~~~~~~~~~~~ - -When the returned number of entities (number of rows) is too large (> 100K), state APIs truncate the output data to ensure system stability -(when this happens, there's no way to choose truncated data). When truncation happens it will be informed through Python's -``warnings`` module. - -Garbage Collected Resources -~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Depending on the lifecycle of the resources, some "finished" resources are not accessible -through the APIs because they are already garbage collected. -**It is recommended not to rely on this API to obtain correct information on finished resources**. -For example, Ray periodically garbage collects DEAD state actor data to reduce memory usage. -Or it cleans up the FINISHED state of tasks when its lineage goes out of scope. - -API Reference -------------- - -- For the CLI Reference, see :ref:`State CLI Refernece `. -- For the SDK Reference, see :ref:`State API Reference `. -- For the Log CLI Reference, see :ref:`Log CLI Reference `. \ No newline at end of file From 6827e2587bf8b5ecacd02fa7f5680dba8f7b77ca Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 19:05:42 -0700 Subject: [PATCH 17/29] add youtube links for overview and jobs videos Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../getting-started-dashboard.rst | 159 ++++++++---------- 1 file changed, 71 insertions(+), 88 deletions(-) diff --git a/doc/source/ray-observability/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started-dashboard.rst index ddc5fcbb28908..124cee918fb15 100644 --- a/doc/source/ray-observability/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started-dashboard.rst @@ -9,24 +9,9 @@ of their applications and troubleshoot issues. .. raw:: html
- +
-Common Workflows ----------------- - -Here are common workflows when using the Ray dashboard. - -- :ref:`View the metrics graphs `. -- :ref:`View the progress of your job `. -- :ref:`Find the application logs or error messages of failed tasks or actors `. -- :ref:`Profile, trace dump, and visualize the timeline of the Ray jobs, tasks, or actors `. -- :ref:`Analyze the CPU and memory usage of the cluster, tasks and actors `. -- :ref:`View the individual state of task, actor, placement group `, and :ref:`nodes (machines from a cluster) ` which is equivalent to :ref:`Ray state APIs `. -- :ref:`View the hardware utilization (CPU, GPU, memory) `. - -Getting Started ---------------- To use the dashboard, you should use the `ray[default]` installation: @@ -65,8 +50,76 @@ Ray cluster comes with the dashboard. See :ref:`Cluster Monitoring `. See :ref:`Ray Metrics ` to learn how to set up Prometheus and Grafana. -How to Guides -------------- +Jobs View +--------- + +.. raw:: html + +
+ +
+ +The Jobs View lets you monitor the different jobs that ran on your Ray cluster. + +A job is a ray workload that uses Ray APIs (e.g., ``ray.init``). It can be submitted directly (e.g., by executing a Python script within a head node) or via :ref:`Ray job API `. + +.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/job_list.png + :align: center + +The job page displays a list of active, finished, and failed jobs, and clicking on an ID allows users to view detailed information about that job. +For more information on Ray jobs, see the Ray Job Overview section. + +Job Profiling +~~~~~~~~~~~~~ + +You can profile Ray jobs by clicking on the “Stack Trace” or “CPU Flame Graph” actions. See the :ref:`Dashboard Profiling ` for more details. + +.. _dash-workflow-job-progress: + +Advanced Task and Actor Breakdown +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/advanced-progress.png + :align: left + +The job page allows you to see tasks and actors broken down by their states. +Tasks and actors are grouped and nested by default. You can see the nested entries by clicking the expand button. + +Tasks and actors are grouped and nested by the following criteria. + +- All tasks and actors are grouped together, and you can view individual entries by expanding the corresponding row. +- Tasks are grouped by their ``name`` attribute (e.g., ``task.options(name="").remote()``). +- Child tasks (nested tasks) are nested under their parent task's row. +- Actors are grouped by their class name. +- Child actors (actors created within an actor) are nested under their parent actor's row. +- Actor tasks (remote methods within an actor) are nested under the actor for the corresponding actor method. + +.. note:: + + Ray dashboard can only display or retrieve up to 10K tasks at a time. If there are more than 10K tasks from your job, + they are unaccounted. The number of unaccounted tasks is available from the task breakdown. + +Task Timeline +~~~~~~~~~~~~~ + +The :ref:`timeline API ` is available from the dashboard. + +.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/profile-button.png + :align: center + +First, you can download the chrome tracing file by clicking the download button. + +.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/profile_drag.png + :align: center + +Second, you can use tools like ``chrome://tracing`` or the `Perfetto UI `_ and drop the downloaded chrome tracing file. We will use the Perfetto as it is the recommendation way to visualize chrome tracing files. + +.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/timeline.png + :align: center + +Now, you can see the timeline visualization of Ray tasks and actors. There are Node rows (hardware) and Worker rows (processes). +Each worker rows display a list of events (e.g., task scheduled, task running, input/output deserialization, etc.) happening from that worker over time. + .. _dash-workflow-logs: @@ -190,76 +243,6 @@ Two types of events are available. .. _dash-jobs-view: -Jobs View ---------- - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/jobs.png - :align: center - -The Jobs View lets you monitor the different jobs that ran on your Ray cluster. - -A job is a ray workload that uses Ray APIs (e.g., ``ray.init``). It can be submitted directly (e.g., by executing a Python script within a head node) or via :ref:`Ray job API `. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/job_list.png - :align: center - -The job page displays a list of active, finished, and failed jobs, and clicking on an ID allows users to view detailed information about that job. -For more information on Ray jobs, see the Ray Job Overview section. - -Job Profiling -~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/profile-job.png - :align: center - -You can profile Ray jobs by clicking on the “Stack Trace” or “CPU Flame Graph” actions. See the :ref:`Dashboard Profiling ` for more details. - -.. _dash-workflow-job-progress: - -Advanced Task and Actor Breakdown -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/advanced-progress.png - :align: left - -The job page allows you to see tasks and actors broken down by their states. -Tasks and actors are grouped and nested by default. You can see the nested entries by clicking the expand button. - -Tasks and actors are grouped and nested by the following criteria. - -- All tasks and actors are grouped together, and you can view individual entries by expanding the corresponding row. -- Tasks are grouped by their ``name`` attribute (e.g., ``task.options(name="").remote()``). -- Child tasks (nested tasks) are nested under their parent task's row. -- Actors are grouped by their class name. -- Child actors (actors created within an actor) are nested under their parent actor's row. -- Actor tasks (remote methods within an actor) are nested under the actor for the corresponding actor method. - -.. note:: - - Ray dashboard can only display or retrieve up to 10K tasks at a time. If there are more than 10K tasks from your job, - they are unaccounted. The number of unaccounted tasks is available from the task breakdown. - -Task Timeline -~~~~~~~~~~~~~ - -The :ref:`timeline API ` is available from the dashboard. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/profile-button.png - :align: center - -First, you can download the chrome tracing file by clicking the download button. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/profile_drag.png - :align: center - -Second, you can use tools like ``chrome://tracing`` or the `Perfetto UI `_ and drop the downloaded chrome tracing file. We will use the Perfetto as it is the recommendation way to visualize chrome tracing files. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/timeline.png - :align: center - -Now, you can see the timeline visualization of Ray tasks and actors. There are Node rows (hardware) and Worker rows (processes). -Each worker rows display a list of events (e.g., task scheduled, task running, input/output deserialization, etc.) happening from that worker over time. - Ray Status ~~~~~~~~~~ From 0819a37cd10115cfdb2db92e4ef70a93657f8ea3 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 19:56:31 -0700 Subject: [PATCH 18/29] add five youtube links; fix anchors for api and cli refs Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../getting-started-dashboard.rst | 227 ++++++++---------- .../ray-observability/reference/api.rst | 4 +- .../ray-observability/reference/cli.rst | 4 +- .../ray-observability/reference/index.md | 4 +- 4 files changed, 105 insertions(+), 134 deletions(-) diff --git a/doc/source/ray-observability/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started-dashboard.rst index 124cee918fb15..882a71aadb69b 100644 --- a/doc/source/ray-observability/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started-dashboard.rst @@ -9,7 +9,7 @@ of their applications and troubleshoot issues. .. raw:: html
- +
@@ -56,16 +56,13 @@ Jobs View .. raw:: html
- +
The Jobs View lets you monitor the different jobs that ran on your Ray cluster. A job is a ray workload that uses Ray APIs (e.g., ``ray.init``). It can be submitted directly (e.g., by executing a Python script within a head node) or via :ref:`Ray job API `. -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/job_list.png - :align: center - The job page displays a list of active, finished, and failed jobs, and clicking on an ID allows users to view detailed information about that job. For more information on Ray jobs, see the Ray Job Overview section. @@ -79,9 +76,6 @@ You can profile Ray jobs by clicking on the “Stack Trace” or “CPU Flame Gr Advanced Task and Actor Breakdown ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/advanced-progress.png - :align: left - The job page allows you to see tasks and actors broken down by their states. Tasks and actors are grouped and nested by default. You can see the nested entries by clicking the expand button. @@ -104,23 +98,111 @@ Task Timeline The :ref:`timeline API ` is available from the dashboard. -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/profile-button.png - :align: center - First, you can download the chrome tracing file by clicking the download button. -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/profile_drag.png - :align: center - Second, you can use tools like ``chrome://tracing`` or the `Perfetto UI `_ and drop the downloaded chrome tracing file. We will use the Perfetto as it is the recommendation way to visualize chrome tracing files. -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/timeline.png - :align: center - Now, you can see the timeline visualization of Ray tasks and actors. There are Node rows (hardware) and Worker rows (processes). Each worker rows display a list of events (e.g., task scheduled, task running, input/output deserialization, etc.) happening from that worker over time. +.. _dash-serve-view: + +Serve View +---------- + +The Serve view lets you monitor the status of your :ref:`Ray Serve ` applications. + +.. raw:: html + +
+ +
+ +The initial page showcases your general Serve configurations, a list of the Serve applications, and, if you have :ref:`Grafana and Prometheus ` configured, some high-level +metrics of all your Serve applications. Click the name of a Serve application to go to the Serve Application Detail Page. + +Serve Application Detail Page +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +This page shows the Serve application's configurations and metadata. It also lists the :ref:`Serve deployments and replicas `. +Click the expand button of a deployment to see all the replicas in that deployment. + +For each deployment, there are two available actions. You can view the Deployment config and, if you configured :ref:`Grafana and Prometheus `, you can open +a Grafana dashboard with detailed metrics about that deployment. + +For each replica, there are two available actions. You can see the logs of that replica and, if you configured :ref:`Grafana and Prometheus `, you can open +a Grafana dashboard with detailed metrics about that replica. Click on the replica name to go to the Serve Replica Detail Page. + + +Serve Replica Detail Page +~~~~~~~~~~~~~~~~~~~~~~~~~ + +This page shows metadata about the Serve replica, high-level metrics about the replica if you configured :ref:`Grafana and Prometheus `, and +a history of completed :ref:`tasks ` of that replica. + + +Serve Metrics +~~~~~~~~~~~~~ + +Ray serve exports various time-series metrics to understand the status of your Serve application over time. More details of these metrics can be found :ref:`here `. +In order to store and visualize these metrics, you must set up Prometheus and Grafana by following the instructions :ref:`here `. + +These metrics are available in the Ray dashboard in the Serve page and the Serve Replica Detail page. They are also accessible as Grafana dashboards. +Within the Grafana dashboard, use the dropdown filters on the top to filter metrics by route, deployment, or replica. Exact descriptions +of each graph are available by hovering over the "info" icon on the top left of each graph. + + +.. _dash-node-view: + +Cluster View +------------ + +.. raw:: html + +
+ +
+ +The cluster view visualizes hierarchical relationship of +machines (nodes) and workers (processes). Each host consists of many workers, and +you can see them by clicking the + button. This also shows the assignment of GPU resources to specific actors or tasks. + +You can also click the node id to go into a node detail page where you can see more information. + +In addition, the machine view lets you see **logs** for a node or a worker. + +.. _dash-actors-view: + +Actors View +----------- + +.. raw:: html + +
+ +
+ + +The Actors view lets you see information about the actors that have existed on the ray cluster. + +You can view the logs for an actor and you can see which job created the actor. +The information of up to 1000 dead actors will be stored. +This value can be overridden by using the `RAY_DASHBOARD_MAX_ACTORS_TO_CACHE` environment variable +when starting Ray. + +Actor Profiling +~~~~~~~~~~~~~~~ + +You can also run the profiler on a running actor. See :ref:`Dashboard Profiling ` for more details. + +Actor Detail Page +~~~~~~~~~~~~~~~~~ + +By clicking the ID, you can also see the detail view of the actor. + +From the actor detail page, you can see the metadata, state, and the all tasks that have run from this actor. + .. _dash-workflow-logs: View the application logs and errors @@ -274,117 +356,6 @@ You get the same information from the :ref:`Ray state APIs ` applications. - -The initial page showcases your general Serve configurations, a list of the Serve applications, and, if you have :ref:`Grafana and Prometheus ` configured, some high-level -metrics of all your Serve applications. Click the name of a Serve application to go to the Serve Application Detail Page. - -Serve Application Detail Page -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/serve-application.png - :align: center - -This page shows the Serve application's configurations and metadata. It also lists the :ref:`Serve deployments and replicas `. -Click the expand button of a deployment to see all the replicas in that deployment. - -For each deployment, there are two available actions. You can view the Deployment config and, if you configured :ref:`Grafana and Prometheus `, you can open -a Grafana dashboard with detailed metrics about that deployment. - -For each replica, there are two available actions. You can see the logs of that replica and, if you configured :ref:`Grafana and Prometheus `, you can open -a Grafana dashboard with detailed metrics about that replica. Click on the replica name to go to the Serve Replica Detail Page. - - -Serve Replica Detail Page -~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/serve-replica.png - :align: center - -This page shows metadata about the Serve replica, high-level metrics about the replica if you configured :ref:`Grafana and Prometheus `, and -a history of completed :ref:`tasks ` of that replica. - - -Serve Metrics -~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/serve-metrics.png - :align: center - -Ray serve exports various time-series metrics to understand the status of your Serve application over time. More details of these metrics can be found :ref:`here `. -In order to store and visualize these metrics, you must set up Prometheus and Grafana by following the instructions :ref:`here `. - -These metrics are available in the Ray dashboard in the Serve page and the Serve Replica Detail page. They are also accessible as Grafana dashboards. -Within the Grafana dashboard, use the dropdown filters on the top to filter metrics by route, deployment, or replica. Exact descriptions -of each graph are available by hovering over the "info" icon on the top left of each graph. - -.. _dash-node-view: - -Cluster View ------------- - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard/nodes-view-expand.png - :align: center - -The cluster view visualizes hierarchical relationship of -machines (nodes) and workers (processes). Each host consists of many workers, and -you can see them by clicking the + button. This also shows the assignment of GPU resources to specific actors or tasks. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard/node-detail.png - :align: center - -You can also click the node id to go into a node detail page where you can see more information. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/machine-view-log.png - :align: center - - -In addition, the machine view lets you see **logs** for a node or a worker. - -.. _dash-actors-view: - -Actors View ------------ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/actor-page.png - :align: center - -The Actors view lets you see information about the actors that have existed on the ray cluster. - -You can view the logs for an actor and you can see which job created the actor. -The information of up to 1000 dead actors will be stored. -This value can be overridden by using the `RAY_DASHBOARD_MAX_ACTORS_TO_CACHE` environment variable -when starting Ray. - -Actor Profiling -~~~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/actor-profiling.png - :align: center - -You can also run the profiler on a running actor. See :ref:`Dashboard Profiling ` for more details. - -Actor Detail Page -~~~~~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/actor-list-id.png - :align: center - -By clicking the ID, you can also see the detail view of the actor. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/actor-detail.png - :align: center - -From the actor detail page, you can see the metadata, state, and the all tasks that have run from this actor. - .. _dash-metrics-view: Metrics View diff --git a/doc/source/ray-observability/reference/api.rst b/doc/source/ray-observability/reference/api.rst index 5056422e5bd7d..bf875b5098000 100644 --- a/doc/source/ray-observability/reference/api.rst +++ b/doc/source/ray-observability/reference/api.rst @@ -1,8 +1,8 @@ +.. _state-api-ref: + State API ========= -.. _state-api-ref: - .. note:: APIs are :ref:`alpha `. This feature requires a full installation of Ray using ``pip install "ray[default]"``. diff --git a/doc/source/ray-observability/reference/cli.rst b/doc/source/ray-observability/reference/cli.rst index e12dfc45fdb5d..c0b7f3bdb7e11 100644 --- a/doc/source/ray-observability/reference/cli.rst +++ b/doc/source/ray-observability/reference/cli.rst @@ -1,8 +1,8 @@ +.. _state-api-cli-ref: + Ray State CLI ============= -.. _state-api-cli-ref: - State ----- This section contains commands to access the :ref:`live state of Ray resources (actor, task, object, etc.) `. diff --git a/doc/source/ray-observability/reference/index.md b/doc/source/ray-observability/reference/index.md index 448c3f38cac21..2078c234a912b 100644 --- a/doc/source/ray-observability/reference/index.md +++ b/doc/source/ray-observability/reference/index.md @@ -5,5 +5,5 @@ Monitor and debug your Ray applications and clusters using the API and CLI documented in these references. The guides include: -* {ref}`observability-state-api-ref` -* {ref}`observability-state-api-cli-ref` \ No newline at end of file +* {ref}`state-api-ref` +* {ref}`state-api-cli-ref` \ No newline at end of file From d5f55589c7e5af9e2b166a3306a8e1fb3dd3d16f Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Tue, 16 May 2023 21:51:37 -0700 Subject: [PATCH 19/29] add two youtube links Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../getting-started-dashboard.rst | 291 +++++------------- 1 file changed, 80 insertions(+), 211 deletions(-) diff --git a/doc/source/ray-observability/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started-dashboard.rst index 882a71aadb69b..e49c151b2ed98 100644 --- a/doc/source/ray-observability/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started-dashboard.rst @@ -50,6 +50,8 @@ Ray cluster comes with the dashboard. See :ref:`Cluster Monitoring `. See :ref:`Ray Metrics ` to learn how to set up Prometheus and Grafana. +.. _dash-jobs-view: + Jobs View --------- @@ -105,6 +107,36 @@ Second, you can use tools like ``chrome://tracing`` or the `Perfetto UI `. + +.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/task-table.png + :align: center + +You can expand the table to see a list of each task, actor, and placement group. .. _dash-serve-view: @@ -183,7 +215,6 @@ Actors View - The Actors view lets you see information about the actors that have existed on the ray cluster. You can view the logs for an actor and you can see which job created the actor. @@ -203,15 +234,57 @@ By clicking the ID, you can also see the detail view of the actor. From the actor detail page, you can see the metadata, state, and the all tasks that have run from this actor. -.. _dash-workflow-logs: +.. _dash-metrics-view: -View the application logs and errors -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +Metrics View +------------ -**Driver Logs** +.. raw:: html -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/log_button_at_job.png - :align: center +
+ +
+ + +Ray exports default metrics which are available from the :ref:`Metrics View `. Here are some available example metrics. + +- The tasks, actors, and placement groups broken down by states. +- The :ref:`logical resource usage ` across nodes. +- The hardware resource usage across nodes. +- The autoscaler status. + +See :ref:`System Metrics Page ` for available metrics. + +.. note:: + + The metrics view required the Prometheus and Grafana setup. See :ref:`Ray Metrics ` to learn how to set up Prometheus and Grafana. + +The metrics view lets you view visualizations of the time series metrics emitted by Ray. + +You can select the time range of the metrics in the top right corner. The graphs refresh automatically every 15 seconds. + +There is also a convenient button to open the grafana UI from the dashboard. The Grafana UI provides additional customizability of the charts. + +.. _dash-logs-view: + +Logs View +--------- + +.. raw:: html + +
+ +
+ +The logs view lets you view all the Ray logs in your cluster. It is organized by node and log file name. Many log links in the other pages link to this view and filter the list so the relevant logs appear. + +To understand the log file structure of Ray, see the :ref:`Logging directory structure page `. + + +The logs view provides search functionality to help you find specific log messages. + + +**Driver Logs** If the Ray job is submitted by :ref:`Ray job API `, the job logs are available from the dashboard. The log file follows the following format; ``job-driver-.log``. @@ -221,31 +294,16 @@ If the Ray job is submitted by :ref:`Ray job API `, the job log **Task and Actor Logs** -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/actor_log.png - :align: center - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/task_log.png - :align: center - Task and actor logs are accessible from the :ref:`task and actor table view `. Click the log button. You can see the worker logs (``worker-[worker_id]-[job_id]-[pid].[out|err]``) that execute the task and actor. ``.out`` (stdout) and ``.err`` (stderr) logs contain the logs emitted from the tasks and actors. The core worker logs (``python-core-worker-[worker_id]_[pid].log``) contain the system-level logs for the corresponding worker. **Task and Actor Errors** -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/failed_task_progress-bar.png - :align: center - You can easily identify failed tasks or actors by looking at the job progress bar, which links to the table. -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/task_error_button.png - :align: center - The table displays the name of the failed tasks or actors and provides access to their corresponding log or error messages. -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/task_error_box.png - :align: center - .. _dash-workflow-cpu-memory-analysis: Analyze the CPU and memory usage of tasks and actors @@ -323,195 +381,6 @@ Two types of events are available. - Job: Events related to :ref:`Ray job submission APIs `. - Autoscaler: Events related to the :ref:`Ray autoscaler `. -.. _dash-jobs-view: - -Ray Status -~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/ray-status.png - :align: center - -The job page displays the output of the CLI tool ``ray status``, which shows the autoscaler status of the Ray cluster. - -The left page shows the autoscaling status, including pending, active, and failed nodes. -The right page displays the cluster's demands, which are resources that cannot be scheduled to the cluster at the moment. This page is useful for debugging resource deadlocks or slow scheduling. - -.. note:: - - The output shows the aggregated information across the cluster (not by job). If you run more than one job, some of the demands may come from other jobs. - -.. _dash-workflow-state-apis: - -Task Table, Actor Table, Placement Group Table -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/tables.png - :align: center - -The dashboard shows a table with the status of the job's tasks, actors, and placement groups. -You get the same information from the :ref:`Ray state APIs `. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/task-table.png - :align: center - -You can expand the table to see a list of each task, actor, and placement group. - -.. _dash-metrics-view: - -Metrics View ------------- - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard/metrics.png - :align: center - -Ray exports default metrics which are available from the :ref:`Metrics View `. Here are some available example metrics. - -- The tasks, actors, and placement groups broken down by states. -- The :ref:`logical resource usage ` across nodes. -- The hardware resource usage across nodes. -- The autoscaler status. - -See :ref:`System Metrics Page ` for available metrics. - -.. note:: - - The metrics view required the Prometheus and Grafana setup. See :ref:`Ray Metrics ` to learn how to set up Prometheus and Grafana. - -The metrics view lets you view visualizations of the time series metrics emitted by Ray. - -You can select the time range of the metrics in the top right corner. The graphs refresh automatically every 15 seconds. - -There is also a convenient button to open the grafana UI from the dashboard. The Grafana UI provides additional customizability of the charts. - -.. _dash-logs-view: - -Logs View ---------- - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard/logs.png - :align: center - -The logs view lets you view all the Ray logs in your cluster. It is organized by node and log file name. Many log links in the other pages link to this view and filter the list so the relevant logs appear. - -To understand the log file structure of Ray, see the :ref:`Logging directory structure page `. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard/logs-content.png - :align: center - -The logs view provides search functionality to help you find specific log messages. - -.. _dash-reference: - -Page References ---------------- - -Cluster View -~~~~~~~~~~~~ - -.. list-table:: Cluster View Node Table Reference - :widths: 25 75 - :header-rows: 1 - - * - Term - - Description - * - **State** - - Whether the node or worker is alive or dead. - * - **ID** - - The ID of the node or the workerId for the worker. - * - **Host / Cmd line** - - If it is a node, it shows host information. If it is a worker, it shows the name of the task that is being run. - * - **IP / PID** - - If it is a node, it shows the IP address of the node. If it's a worker, it shows the PID of the worker process. - * - **CPU Usage** - - CPU usage of each node and worker. - * - **Memory** - - RAM usage of each node and worker. - * - **GPU** - - GPU usage of the node. - * - **GRAM** - - GPU memory usage of the node. - * - **Object Store Memory** - - Amount of memory used by the object store for this node. - * - **Disk** - - Disk usage of the node. - * - **Sent** - - Network bytes sent for each node and worker. - * - **Received** - - Network bytes received for each node and worker. - * - **Log** - - Logs messages at each node and worker. You can see log files relevant to a node or worker by clicking this link. - * - **Stack Trace** - - Get the Python stack trace for the specified worker. Refer to :ref:`dashboard-profiling` for more information. - * - **CPU Flame Graph** - - Get a CPU flame graph for the specified worker. Refer to :ref:`dashboard-profiling` for more information. - - -Jobs View -~~~~~~~~~ - -.. list-table:: Jobs View Reference - :widths: 25 75 - :header-rows: 1 - - * - Term - - Description - * - **Job ID** - - The ID of the job. This is the primary id that associates tasks and actors to this job. - * - **Submission ID** - - An alternate ID that can be provided by a user or generated for all ray job submissions. - It's useful if you would like to associate your job with an ID that is provided by some external system. - * - **Status** - - Describes the state of a job. One of: - * PENDING: The job has not started yet, likely waiting for the runtime_env to be set up. - * RUNNING: The job is currently running. - * STOPPED: The job was intentionally stopped by the user. - * SUCCEEDED: The job finished successfully. - * FAILED: The job failed. - * - **Logs** - - A link to the logs for this job. - * - **StartTime** - - The time the job was started. - * - **EndTime** - - The time the job finished. - * - **DriverPid** - - The PID for the driver process that is started the job. - -Actors -~~~~~~ - -.. list-table:: Actor View Reference - :widths: 25 75 - :header-rows: 1 - - * - Term - - Description - * - **Actor ID** - - The ID of the actor. - * - **Restart Times** - - Number of times this actor has been restarted. - * - **Name** - - The name of an actor. This can be user defined. - * - **Class** - - The class of the actor. - * - **Function** - - The current function the actor is running. - * - **Job ID** - - The job in which this actor was created. - * - **Pid** - - ID of the worker process on which the actor is running. - * - **IP** - - Node IP Address where the actor is located. - * - **Port** - - The Port for the actor. - * - **State** - - Either one of "ALIVE" or "DEAD". - * - **Log** - - A link to the logs that are relevant to this actor. - * - **Stack Trace** - - Get the Python stack trace for the specified actor. Refer to :ref:`dashboard-profiling` for more information. - * - **CPU Flame Graph** - - Get a CPU flame graph for the specified actor. Refer to :ref:`dashboard-profiling` for more information. - Resources --------- - `Ray Summit observability talk `_ From 399e9b8ddc8dfaee2f998294aeedbd2104a7277d Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Wed, 17 May 2023 11:45:32 -0700 Subject: [PATCH 20/29] moving some misplaced logging content; fixed extra blank spaces after videos; moved programmatic monitoring to user guides Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 2 +- .../cluster/configure-manage-dashboard.rst | 52 ++++---- .../cluster/kubernetes/user-guides/logging.md | 35 +----- .../monitoring-and-observability.rst | 3 + doc/source/cluster/vms/user-guides/logging.md | 33 ----- .../getting-started-dashboard.rst | 99 ++++++--------- doc/source/ray-observability/key-concepts.rst | 118 +++++++++++------- .../cli-sdk.rst} | 8 +- 8 files changed, 150 insertions(+), 200 deletions(-) rename doc/source/ray-observability/{getting-started-program.rst => user-guides/cli-sdk.rst} (99%) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index b06ea7a9e0fa4..6749a34e150a4 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -390,7 +390,6 @@ parts: title: "Monitoring and Debugging" sections: - file: ray-observability/getting-started-dashboard - - file: ray-observability/getting-started-program - file: ray-observability/key-concepts - file: ray-observability/user-guides/index title: User Guides @@ -403,6 +402,7 @@ parts: - file: ray-observability/user-guides/troubleshoot-apps/optimize-performance - file: ray-observability/user-guides/troubleshoot-apps/ray-debugging - file: ray-observability/user-guides/troubleshoot-apps/ray-core-profiling + - file: ray-observability/user-guides/cli-sdk - file: ray-observability/user-guides/configure-logging - file: ray-observability/user-guides/add-app-metrics - file: ray-observability/user-guides/ray-tracing diff --git a/doc/source/cluster/configure-manage-dashboard.rst b/doc/source/cluster/configure-manage-dashboard.rst index 2cbd1976e7d37..139201d62a3bb 100644 --- a/doc/source/cluster/configure-manage-dashboard.rst +++ b/doc/source/cluster/configure-manage-dashboard.rst @@ -106,6 +106,24 @@ Below is an example with a `traefik `. + Disabling the Dashboard ----------------------- @@ -143,29 +161,13 @@ To disable the dashboard, use the following arguments `--include-dashboard`. TODO -Viewing Built-in Dashboard API Metrics --------------------------------------- - -The dashboard is powered by a server that serves both the UI code and the data about the cluster via API endpoints. -There are basic Prometheus metrics that are emitted for each of these API endpoints: - -`ray_dashboard_api_requests_count_requests_total`: Collects the total count of requests. This is tagged by endpoint, method, and http_status. - -`ray_dashboard_api_requests_duration_seconds_bucket`: Collects the duration of requests. This is tagged by endpoint and method. - -For example, you can view the p95 duration of all requests with this query: - -.. code-block:: text - - histogram_quantile(0.95, sum(rate(ray_dashboard_api_requests_duration_seconds_bucket[5m])) by (le)) - -These metrics can be queried via Prometheus or Grafana UI. Instructions on how to set these tools up can be found :ref:`here `. +.. _observability-visualization-setup: Integrating with Prometheus and Grafana --------------------------------------- Setting up Prometheus ---------------------- +~~~~~~~~~~~~~~~~~~~~~ .. tip:: @@ -221,7 +223,7 @@ See :ref:`here ` for more information on how to set up Prome .. _grafana: Setting up Grafana ------------------- +~~~~~~~~~~~~~~~~~~ .. tip:: @@ -235,7 +237,7 @@ for debugging ray applications. Deploying Grafana -~~~~~~~~~~~~~~~~~ +***************** First, `download Grafana `_. Follow the instructions on the download page to download the right binary for your operating system. @@ -301,10 +303,10 @@ to `RAY_GRAFANA_HOST=http://55.66.77.88:3000`. Troubleshooting ---------------- +~~~~~~~~~~~~~~~ Getting Prometheus and Grafana to use the Ray configurations when installed via homebrew on macOS X -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +*************************************************************************************************** With homebrew, Prometheus and Grafana are installed as services that are automatically launched for you. Therefore, to configure these services, you cannot simply pass in the config files as command line arguments. @@ -318,7 +320,7 @@ You can then start or restart the services with `brew services start grafana` an .. _unverified-developer: MacOS does not trust the developer to install Prometheus or Grafana -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +******************************************************************* You may have received an error that looks like this: @@ -331,12 +333,12 @@ Unfortunately, many developers today are not trusted by Mac and so this requirem See `these instructions `_ on how to override the restriction and install or run the application. Grafana dashboards are not embedded in the Ray dashboard -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +******************************************************** If you're getting an error that says `RAY_GRAFANA_HOST` is not setup despite having set it up, check that: You've included the protocol in the URL (e.g., `http://your-grafana-url.com` instead of `your-grafana-url.com`). The URL doesn't have a trailing slash (e.g., `http://your-grafana-url.com` instead of `http://your-grafana-url.com/`). Certificate Authority (CA error) -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ +******************************** You may see a CA error if your Grafana instance is hosted behind HTTPS. Contact the Grafana service owner to properly enable HTTPS traffic. diff --git a/doc/source/cluster/kubernetes/user-guides/logging.md b/doc/source/cluster/kubernetes/user-guides/logging.md index 4007fe5736c41..57e4241fa094d 100644 --- a/doc/source/cluster/kubernetes/user-guides/logging.md +++ b/doc/source/cluster/kubernetes/user-guides/logging.md @@ -205,42 +205,9 @@ ray.get(task.remote()) ```bash (pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) ``` -## Logging directory structure - -By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. - -:::{note} -The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. -::: - -A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. - -Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/stderr and ``.err`` is logs from stderr. The backward compatibility of log directories is not maintained. - -- ``dashboard.[log|err]``: A log file of a Ray dashboard. ``log.`` file contains logs generated from the dashboard's logger. ``.err`` file contains stdout and stderr printed from the dashboard. They are usually empty except when the dashboard crashes unexpectedly. -- ``dashboard_agent.log``: Every Ray node has one dashboard agent. This is a log file of the agent. -- ``gcs_server.[out|err]``: The GCS server is a stateless server that manages Ray cluster metadata. It exists only in the head node. -- ``io-worker-[worker_id]-[pid].[out|err]``: Ray creates IO workers to spill/restore objects to external storage by default from Ray 1.3+. This is a log file of IO workers. -- ``job-driver-[submission_id].log``: The stdout of a job submitted via the :ref:`Ray Jobs API `. -- ``log_monitor.[log|err]``: The log monitor is in charge of streaming logs to the driver. ``log.`` file contains logs generated from the log monitor's logger. ``.err`` file contains the stdout and stderr printed from the log monitor. They are usually empty except when the log monitor crashes unexpectedly. -- ``monitor.[out|err]``: Stdout and stderr of a cluster launcher. -- ``monitor.log``: Ray's cluster launcher is operated with a monitor process. It also manages the autoscaler. -- ``plasma_store.[out|err]``: Deprecated. -- ``python-core-driver-[worker_id]_[pid].log``: Ray drivers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. -- ``python-core-worker-[worker_id]_[pid].log``: Ray workers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. -- ``raylet.[out|err]``: A log file of raylets. -- ``redis-shard_[shard_index].[out|err]``: Redis shard log files. -- ``redis.[out|err]``: Redis log files. -- ``runtime_env_agent.log``: Every Ray node has one agent that manages :ref:`runtime environment ` creation, deletion and caching. - This is the log file of the agent containing logs of create/delete requests and cache hits and misses. - For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). -- ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. -- ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. -- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver. - ## Redirecting Ray logs to stderr -By default, Ray logs are written to files under the ``/tmp/ray/session_*/logs`` directory. If you wish to redirect all internal Ray logging and your own logging within tasks/actors to stderr of the host nodes, you can do so by ensuring that the ``RAY_LOG_TO_STDERR=1`` environment variable is set on the driver and on all Ray nodes. This is very useful if you are using a log aggregator that needs log records to be written to stderr in order for them to be captured. +By default, Ray logs are written to files under the ``/tmp/ray/session_*/logs`` directory. If you wish to redirect all internal Ray logging and your own logging within tasks/actors to stderr of the host nodes, you can do so by ensuring that the ``RAY_LOG_TO_STDERR=1`` environment variable is set on the driver and on all Ray nodes. This practice is not recommended but may be useful if you are using a log aggregator that needs log records to be written to stderr in order for them to be captured. Redirecting logging to stderr will also cause a ``({component})`` prefix, e.g. ``(raylet)``, to be added to each of the log record messages. diff --git a/doc/source/cluster/running-applications/monitoring-and-observability.rst b/doc/source/cluster/running-applications/monitoring-and-observability.rst index fcc47a746c22b..53f21c856ca45 100644 --- a/doc/source/cluster/running-applications/monitoring-and-observability.rst +++ b/doc/source/cluster/running-applications/monitoring-and-observability.rst @@ -1,3 +1,6 @@ +Scraping and Persisting Metrics +=============================== + Cluster Monitoring ------------------ diff --git a/doc/source/cluster/vms/user-guides/logging.md b/doc/source/cluster/vms/user-guides/logging.md index d94bcc771192d..dd97643b5e2a9 100644 --- a/doc/source/cluster/vms/user-guides/logging.md +++ b/doc/source/cluster/vms/user-guides/logging.md @@ -205,39 +205,6 @@ ray.get(task.remote()) ```bash (pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) ``` -## Logging directory structure - -By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. - -:::{note} -The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. -::: - -A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. - -Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/stderr and ``.err`` is logs from stderr. The backward compatibility of log directories is not maintained. - -- ``dashboard.[log|err]``: A log file of a Ray dashboard. ``log.`` file contains logs generated from the dashboard's logger. ``.err`` file contains stdout and stderr printed from the dashboard. They are usually empty except when the dashboard crashes unexpectedly. -- ``dashboard_agent.log``: Every Ray node has one dashboard agent. This is a log file of the agent. -- ``gcs_server.[out|err]``: The GCS server is a stateless server that manages Ray cluster metadata. It exists only in the head node. -- ``io-worker-[worker_id]-[pid].[out|err]``: Ray creates IO workers to spill/restore objects to external storage by default from Ray 1.3+. This is a log file of IO workers. -- ``job-driver-[submission_id].log``: The stdout of a job submitted via the :ref:`Ray Jobs API `. -- ``log_monitor.[log|err]``: The log monitor is in charge of streaming logs to the driver. ``log.`` file contains logs generated from the log monitor's logger. ``.err`` file contains the stdout and stderr printed from the log monitor. They are usually empty except when the log monitor crashes unexpectedly. -- ``monitor.[out|err]``: Stdout and stderr of a cluster launcher. -- ``monitor.log``: Ray's cluster launcher is operated with a monitor process. It also manages the autoscaler. -- ``plasma_store.[out|err]``: Deprecated. -- ``python-core-driver-[worker_id]_[pid].log``: Ray drivers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. -- ``python-core-worker-[worker_id]_[pid].log``: Ray workers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. -- ``raylet.[out|err]``: A log file of raylets. -- ``redis-shard_[shard_index].[out|err]``: Redis shard log files. -- ``redis.[out|err]``: Redis log files. -- ``runtime_env_agent.log``: Every Ray node has one agent that manages :ref:`runtime environment ` creation, deletion and caching. - This is the log file of the agent containing logs of create/delete requests and cache hits and misses. - For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). -- ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. -- ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. -- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver. - ## Rotating logs Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). diff --git a/doc/source/ray-observability/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started-dashboard.rst index e49c151b2ed98..9d641a49cdb71 100644 --- a/doc/source/ray-observability/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started-dashboard.rst @@ -8,7 +8,7 @@ of their applications and troubleshoot issues. .. raw:: html -
+
@@ -50,6 +50,8 @@ Ray cluster comes with the dashboard. See :ref:`Cluster Monitoring `. See :ref:`Ray Metrics ` to learn how to set up Prometheus and Grafana. + .. _dash-workflow-cpu-memory-analysis: + .. _dash-jobs-view: Jobs View @@ -57,7 +59,7 @@ Jobs View .. raw:: html -
+
@@ -110,9 +112,6 @@ Each worker rows display a list of events (e.g., task scheduled, task running, i Ray Status ~~~~~~~~~~ -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/ray-status.png - :align: center - The job page displays the output of the CLI tool ``ray status``, which shows the autoscaler status of the Ray cluster. The left page shows the autoscaling status, including pending, active, and failed nodes. @@ -127,15 +126,9 @@ The right page displays the cluster's demands, which are resources that cannot b Task Table, Actor Table, Placement Group Table ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/tables.png - :align: center - The dashboard shows a table with the status of the job's tasks, actors, and placement groups. You get the same information from the :ref:`Ray state APIs `. -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/task-table.png - :align: center - You can expand the table to see a list of each task, actor, and placement group. .. _dash-serve-view: @@ -147,7 +140,7 @@ The Serve view lets you monitor the status of your :ref:`Ray Serve ` a .. raw:: html -
+
@@ -192,7 +185,7 @@ Cluster View .. raw:: html -
+
@@ -211,7 +204,7 @@ Actors View .. raw:: html -
+
@@ -241,7 +234,7 @@ Metrics View .. raw:: html -
+
@@ -265,6 +258,35 @@ You can select the time range of the metrics in the top right corner. The graphs There is also a convenient button to open the grafana UI from the dashboard. The Grafana UI provides additional customizability of the charts. +Analyze the CPU and memory usage of tasks and actors +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +The :ref:`Metrics View ` in the Ray dashboard provides a "per-component CPU/memory usage graph" that displays CPU and memory usage over time for each task and actor in the application (as well as system components). +This allows users to identify tasks and actors that may be consuming more resources than expected and optimize the performance of the application. + +Per component CPU graph. 0.379 cores mean that it uses 40% of a single CPU core. Ray process names start with ``ray::``. ``raylet``, ``agent``, ``dashboard``, or ``gcs`` are system components. + +Per component memory graph. Ray process names start with ``ray::``. ``raylet``, ``agent``, ``dashboard``, or ``gcs`` are system components. + +Additionally, users can see a snapshot of hardware utilization from the :ref:`cluster page `, which provides an overview of resource usage across the entire Ray cluster. + +.. _dash-workflow-resource-utilization: + +View the Resource Utilization +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Ray requires users to specify the number of :ref:`resources ` their tasks and actors will use through arguments such as ``num_cpus``, ``num_gpus``, ``memory``, and ``resource``. +These values are used for scheduling, but may not always match the actual resource utilization (physical resource utilization). + +- You can see the logical and physical resource utilization over time from the :ref:`Metrics View `. +- The snapshot of physical resource utilization (CPU, GPU, memory, disk, network) is also available from the :ref:`Cluster View `. + +The :ref:`logical resources ` usage. + +The physical resources (hardware) usage. Ray provides CPU, GPU, Memory, GRAM, disk, and network usage for each machine in a cluster. + + + .. _dash-logs-view: Logs View @@ -272,7 +294,7 @@ Logs View .. raw:: html -
+
@@ -304,51 +326,6 @@ You can easily identify failed tasks or actors by looking at the job progress ba The table displays the name of the failed tasks or actors and provides access to their corresponding log or error messages. -.. _dash-workflow-cpu-memory-analysis: - -Analyze the CPU and memory usage of tasks and actors -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -The :ref:`Metrics View ` in the Ray dashboard provides a "per-component CPU/memory usage graph" that displays CPU and memory usage over time for each task and actor in the application (as well as system components). -This allows users to identify tasks and actors that may be consuming more resources than expected and optimize the performance of the application. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/node_cpu_by_comp.png - :align: center - - -Per component CPU graph. 0.379 cores mean that it uses 40% of a single CPU core. Ray process names start with ``ray::``. ``raylet``, ``agent``, ``dashboard``, or ``gcs`` are system components. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/node_memory_by_comp.png - :align: center - -Per component memory graph. Ray process names start with ``ray::``. ``raylet``, ``agent``, ``dashboard``, or ``gcs`` are system components. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/cluster_page.png - :align: center - -Additionally, users can see a snapshot of hardware utilization from the :ref:`cluster page `, which provides an overview of resource usage across the entire Ray cluster. - -.. _dash-workflow-resource-utilization: - -View the Resource Utilization -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Ray requires users to specify the number of :ref:`resources ` their tasks and actors will use through arguments such as ``num_cpus``, ``num_gpus``, ``memory``, and ``resource``. -These values are used for scheduling, but may not always match the actual resource utilization (physical resource utilization). - -- You can see the logical and physical resource utilization over time from the :ref:`Metrics View `. -- The snapshot of physical resource utilization (CPU, GPU, memory, disk, network) is also available from the :ref:`Cluster View `. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/logical_resource.png - :align: center - -The :ref:`logical resources ` usage. - -.. image:: https://raw.githubusercontent.com/ray-project/Images/master/docs/new-dashboard-v2/dashboard-pics/physical_resource.png - :align: center - -The physical resources (hardware) usage. Ray provides CPU, GPU, Memory, GRAM, disk, and network usage for each machine in a cluster. - .. _dash-overview: Overview diff --git a/doc/source/ray-observability/key-concepts.rst b/doc/source/ray-observability/key-concepts.rst index 454cbe6d6ea22..982f6e99d5f69 100644 --- a/doc/source/ray-observability/key-concepts.rst +++ b/doc/source/ray-observability/key-concepts.rst @@ -59,6 +59,49 @@ The following command lists all the actors from the cluster: See :ref:`Ray State API ` for more details. +Metrics +------- +Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), +internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), +and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used +to monitor the cluster over time. + +See :ref:`Ray Metrics ` for more details. + +Exceptions +---------- +Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, +the API raises an exception if anything goes wrong with a related task, actor or object. For example, + +- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. +- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). +- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. + +See :ref:`Exceptions Reference ` for more details. + +Debugger +-------- +Ray has a built-in debugger that allows you to debug your distributed applications. +It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can +drop into a PDB session that you can then use to: + +- Inspect variables in that context +- Step within that task or actor +- Move up or down the stack + +See :ref:`Ray Debugger ` for more details. + +Profiling +--------- +Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. + +See :ref:`Profiling ` for more details. + +Tracing +------- +To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. + +See :ref:`Ray Tracing ` for more details. Application Logging ------------------- @@ -123,46 +166,37 @@ Actor log messages look like the following by default. (MyActor pid=480956) actor log message -Metrics -------- -Ray collects and exposes the physical stats (e.g., CPU, memory, GRAM, disk, and network usage of each node), -internal stats (e.g., number of actors in the cluster, number of worker failures of the cluster), -and custom metrics (e.g., metrics defined by users). All stats can be exported as time series data (to Prometheus by default) and used -to monitor the cluster over time. - -See :ref:`Ray Metrics ` for more details. - -Exceptions ----------- -Creating a new task or submitting an actor task generates an object reference. When ``ray.get`` is called on the object reference, -the API raises an exception if anything goes wrong with a related task, actor or object. For example, - -- :class:`RayTaskError ` is raised when there's an error from user code that throws an exception. -- :class:`RayActorError ` is raised when an actor is dead (by a system failure such as node failure or user-level failure such as an exception from ``__init__`` method). -- :class:`RuntimeEnvSetupError ` is raised when the actor or task couldn't be started because :ref:`a runtime environment ` failed to be created. - -See :ref:`Exceptions Reference ` for more details. - -Debugger --------- -Ray has a built-in debugger that allows you to debug your distributed applications. -It allows you to set breakpoints in your Ray tasks and actors, and when hitting the breakpoint, you can -drop into a PDB session that you can then use to: - -- Inspect variables in that context -- Step within that task or actor -- Move up or down the stack - -See :ref:`Ray Debugger ` for more details. - -Profiling ---------- -Ray is compatible with Python profiling tools such as ``CProfile``. It also supports its built-in profiling tool such as :ref:```ray timeline`` `. - -See :ref:`Profiling ` for more details. - -Tracing -------- -To help debug and monitor Ray applications, Ray supports distributed tracing (integration with OpenTelemetry) across tasks and actors. +Logging directory structure +~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. + +:::{note} +The default temp directory is ``/tmp/ray`` (for Linux and Mac OS). If you'd like to change the temp directory, you can specify it when ``ray start`` or ``ray.init()`` is called. +::: + +A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``. + +Here's a Ray log directory structure. Note that ``.out`` is logs from stdout/stderr and ``.err`` is logs from stderr. The backward compatibility of log directories is not maintained. + +- ``dashboard.[log|err]``: A log file of a Ray dashboard. ``log.`` file contains logs generated from the dashboard's logger. ``.err`` file contains stdout and stderr printed from the dashboard. They are usually empty except when the dashboard crashes unexpectedly. +- ``dashboard_agent.log``: Every Ray node has one dashboard agent. This is a log file of the agent. +- ``gcs_server.[out|err]``: The GCS server is a stateless server that manages Ray cluster metadata. It exists only in the head node. +- ``io-worker-[worker_id]-[pid].[out|err]``: Ray creates IO workers to spill/restore objects to external storage by default from Ray 1.3+. This is a log file of IO workers. +- ``job-driver-[submission_id].log``: The stdout of a job submitted via the :ref:`Ray Jobs API `. +- ``log_monitor.[log|err]``: The log monitor is in charge of streaming logs to the driver. ``log.`` file contains logs generated from the log monitor's logger. ``.err`` file contains the stdout and stderr printed from the log monitor. They are usually empty except when the log monitor crashes unexpectedly. +- ``monitor.[out|err]``: Stdout and stderr of a cluster launcher. +- ``monitor.log``: Ray's cluster launcher is operated with a monitor process. It also manages the autoscaler. +- ``plasma_store.[out|err]``: Deprecated. +- ``python-core-driver-[worker_id]_[pid].log``: Ray drivers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. +- ``python-core-worker-[worker_id]_[pid].log``: Ray workers consist of CPP core and Python/Java frontend. This is a log file generated from CPP code. +- ``raylet.[out|err]``: A log file of raylets. +- ``redis-shard_[shard_index].[out|err]``: Redis shard log files. +- ``redis.[out|err]``: Redis log files. +- ``runtime_env_agent.log``: Every Ray node has one agent that manages :ref:`runtime environment ` creation, deletion and caching. + This is the log file of the agent containing logs of create/delete requests and cache hits and misses. + For the logs of the actual installations (including e.g. ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). +- ``runtime_env_setup-[job_id].log``: Logs from installing :ref:`runtime environments ` for a task, actor or job. This file will only be present if a runtime environment is installed. +- ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing :ref:`runtime environments ` for a job when connecting via :ref:`Ray Client `. +- ``worker-[worker_id]-[job_id]-[pid].[out|err]``: Python/Java part of Ray drivers and workers. All of stdout and stderr from tasks/actors are streamed here. Note that job_id is an id of the driver. -See :ref:`Ray Tracing ` for more details. \ No newline at end of file diff --git a/doc/source/ray-observability/getting-started-program.rst b/doc/source/ray-observability/user-guides/cli-sdk.rst similarity index 99% rename from doc/source/ray-observability/getting-started-program.rst rename to doc/source/ray-observability/user-guides/cli-sdk.rst index 8157b20c072ca..aa52cb14874df 100644 --- a/doc/source/ray-observability/getting-started-program.rst +++ b/doc/source/ray-observability/user-guides/cli-sdk.rst @@ -1,9 +1,9 @@ -.. _observability-getting-started-program: +.. _observability-programmatic: -Getting Started (API, CLI, SDK) -=============================== +Monitoring with the CLI or SDK +============================== -Monitoring and debugging capabilities in Ray are available through an API, CLI, or SDK. +Monitoring and debugging capabilities in Ray are available through a CLI or SDK. Monitoring Cluster State and Resource Demands From 4d64a1ef4cc3284ac0733413897b0c2980bdf339 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Wed, 17 May 2023 12:13:25 -0700 Subject: [PATCH 21/29] removed unnecessary file hierarchy Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 7 ++- ...rted-dashboard.rst => getting-started.rst} | 7 ++- doc/source/ray-observability/index.rst | 8 +++ .../monitoring-debugging/getting-help.rst | 60 ------------------- .../monitoring-debugging.rst | 17 ------ .../ray-observability/reference/cli.rst | 4 +- .../ray-observability/reference/index.md | 3 +- .../general-troubleshoot.rst} | 6 +- 8 files changed, 23 insertions(+), 89 deletions(-) rename doc/source/ray-observability/{getting-started-dashboard.rst => getting-started.rst} (99%) create mode 100644 doc/source/ray-observability/index.rst delete mode 100644 doc/source/ray-observability/monitoring-debugging/getting-help.rst delete mode 100644 doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst rename doc/source/ray-observability/{monitoring-debugging/gotchas.rst => user-guides/general-troubleshoot.rst} (98%) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 6749a34e150a4..b49a53539b833 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -386,14 +386,15 @@ parts: - file: cluster/faq - file: cluster/package-overview - - file: ray-observability/monitoring-debugging/monitoring-debugging + - file: ray-observability/index title: "Monitoring and Debugging" sections: - - file: ray-observability/getting-started-dashboard + - file: ray-observability/getting-started - file: ray-observability/key-concepts - file: ray-observability/user-guides/index title: User Guides sections: + - file: ray-observability/user-guides/general-troubleshoot - file: ray-observability/user-guides/troubleshoot-apps/index title: Troubleshooting Applications sections: @@ -409,9 +410,9 @@ parts: - file: ray-observability/reference/index title: Reference sections: - - file: ray-observability/reference/system-metrics - file: ray-observability/reference/api - file: ray-observability/reference/cli + - file: ray-observability/reference/system-metrics - file: ray-references/api title: References diff --git a/doc/source/ray-observability/getting-started-dashboard.rst b/doc/source/ray-observability/getting-started.rst similarity index 99% rename from doc/source/ray-observability/getting-started-dashboard.rst rename to doc/source/ray-observability/getting-started.rst index 9d641a49cdb71..c0e57bdf5f0fd 100644 --- a/doc/source/ray-observability/getting-started-dashboard.rst +++ b/doc/source/ray-observability/getting-started.rst @@ -1,7 +1,8 @@ -.. _observability-getting-started-dashboard: +.. _observability-getting-started: + +Getting Started +=============== -Getting Started (Dashboard) -=========================== Ray provides a web-based dashboard for monitoring and debugging Ray applications. The dashboard provides a visual representation of the system state, allowing users to track the performance of their applications and troubleshoot issues. diff --git a/doc/source/ray-observability/index.rst b/doc/source/ray-observability/index.rst new file mode 100644 index 0000000000000..1f18fc4885f6f --- /dev/null +++ b/doc/source/ray-observability/index.rst @@ -0,0 +1,8 @@ +.. _observability: + +Monitoring and Debugging +======================== + +This section covers how to **monitor and debug Ray applications and clusters**. + + diff --git a/doc/source/ray-observability/monitoring-debugging/getting-help.rst b/doc/source/ray-observability/monitoring-debugging/getting-help.rst deleted file mode 100644 index 8118de7f7ca8d..0000000000000 --- a/doc/source/ray-observability/monitoring-debugging/getting-help.rst +++ /dev/null @@ -1,60 +0,0 @@ -.. _ray-troubleshoot-getting-help: - -Getting Help -============ - -Ray Community -------------- - -If you stuck on a problem, there are several ways to ask the Ray community. - -.. _`Discourse Forum`: https://discuss.ray.io/ -.. _`GitHub Issues`: https://github.com/ray-project/ray/issues -.. _`StackOverflow`: https://stackoverflow.com/questions/tagged/ray -.. _`Slack`: https://forms.gle/9TSdDYUgxYs8SA9e8 - -.. list-table:: - :widths: 25 50 25 25 - :header-rows: 1 - - * - Platform - - Purpose - - Estimated Response Time - - Support Level - * - `Discourse Forum`_ - - For discussions about development and questions about usage. - - < 1 day - - Community - * - `GitHub Issues`_ - - For reporting bugs and filing feature requests. - - < 2 days - - Ray OSS Team - * - `Slack`_ - - For collaborating with other Ray users. - - < 2 days - - Community - * - `StackOverflow`_ - - For asking questions about how to use Ray. - - 3-5 days - - Community - -Discourse Forum -~~~~~~~~~~~~~~~ -`Discourse Forum` is the primary place to ask questions, where the Ray committers, contributors, and other Ray users answer questions. -Someone from the community may have already answered your question, so before you ask a new question, please make sure to search them. -The Ray contributors monitor the forum daily and expect to respond within a day. - -Bugs or Feature Requests -~~~~~~~~~~~~~~~~~~~~~~~~ -Sometimes, the question or problem you have turns out to be a real bug or requires an enhancement request. In this case, -file a new issue to the `GitHub Issues`_ page. Ray contributors will triage and -address them accordingly. - -StackOverflow -~~~~~~~~~~~~~ -You can also ask questions to `StackOverflow`_ with a Ray tag. On StackOverflow, we strive to respond to questions within 3~5 days. - -Slack -~~~~~ -Many Ray users hang out on Ray `Slack`_ (the invitation is open to everyone). You can join the slack and directly communicate to other Ray users or contributors. -For asking questions, we recommend using the discourse forum or StackOverflow for future searchability. diff --git a/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst b/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst deleted file mode 100644 index d757efffe6819..0000000000000 --- a/doc/source/ray-observability/monitoring-debugging/monitoring-debugging.rst +++ /dev/null @@ -1,17 +0,0 @@ -.. _observability: - -Monitoring and Debugging -======================== - -This section covers how to **monitor and debug Ray applications and clusters**. - -See :ref:`Getting Help ` if your problem is not solved by the troubleshooting guide. - -.. toctree:: - :maxdepth: 0 - - ../state/state-api - ../ray-metrics - profiling - gotchas - getting-help diff --git a/doc/source/ray-observability/reference/cli.rst b/doc/source/ray-observability/reference/cli.rst index c0b7f3bdb7e11..5c7ea0d57eb99 100644 --- a/doc/source/ray-observability/reference/cli.rst +++ b/doc/source/ray-observability/reference/cli.rst @@ -1,7 +1,7 @@ .. _state-api-cli-ref: -Ray State CLI -============= +State CLI +========= State ----- diff --git a/doc/source/ray-observability/reference/index.md b/doc/source/ray-observability/reference/index.md index 2078c234a912b..06ef3bfc34498 100644 --- a/doc/source/ray-observability/reference/index.md +++ b/doc/source/ray-observability/reference/index.md @@ -6,4 +6,5 @@ Monitor and debug your Ray applications and clusters using the API and CLI docum The guides include: * {ref}`state-api-ref` -* {ref}`state-api-cli-ref` \ No newline at end of file +* {ref}`state-api-cli-ref` +* {ref}`system-metrics` \ No newline at end of file diff --git a/doc/source/ray-observability/monitoring-debugging/gotchas.rst b/doc/source/ray-observability/user-guides/general-troubleshoot.rst similarity index 98% rename from doc/source/ray-observability/monitoring-debugging/gotchas.rst rename to doc/source/ray-observability/user-guides/general-troubleshoot.rst index b1d4fb36f19ca..5106b05bb3385 100644 --- a/doc/source/ray-observability/monitoring-debugging/gotchas.rst +++ b/doc/source/ray-observability/user-guides/general-troubleshoot.rst @@ -1,7 +1,7 @@ -.. _gotchas: +.. _observability-troubleshoot-general: -Ray Gotchas -=========== +General Troubleshooting +======================= Ray sometimes has some aspects of its behavior that might catch users off guard. There may be sound arguments for these design choices. From c2cc47d6c1149098363b5e8855e0b9b8f8a21c03 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Wed, 17 May 2023 13:17:53 -0700 Subject: [PATCH 22/29] list guides in index of user guides, fixed anchors Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../ray-observability/user-guides/general-troubleshoot.rst | 2 +- doc/source/ray-observability/user-guides/index.md | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/doc/source/ray-observability/user-guides/general-troubleshoot.rst b/doc/source/ray-observability/user-guides/general-troubleshoot.rst index 5106b05bb3385..96ffbab7cc021 100644 --- a/doc/source/ray-observability/user-guides/general-troubleshoot.rst +++ b/doc/source/ray-observability/user-guides/general-troubleshoot.rst @@ -1,4 +1,4 @@ -.. _observability-troubleshoot-general: +.. _observability-general-troubleshoot: General Troubleshooting ======================= diff --git a/doc/source/ray-observability/user-guides/index.md b/doc/source/ray-observability/user-guides/index.md index a8772bfe58a14..748cc34b51f90 100644 --- a/doc/source/ray-observability/user-guides/index.md +++ b/doc/source/ray-observability/user-guides/index.md @@ -5,5 +5,9 @@ These guides help you monitor and debug your Ray applications and clusters. The guides include: +* {ref}`observability-general-troubleshoot` * {ref}`observability-troubleshoot-user-guides` +* {ref}`observability-programmatic` +* {ref}`configure-logging` +* {ref}`application-level-metrics` * {ref}`ray-tracing` \ No newline at end of file From c29d4c28c4c5be22ff5e4ed49b7dedf0597d2937 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Thu, 18 May 2023 10:29:17 -0700 Subject: [PATCH 23/29] move gotchas content to troubleshooting apps Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 4 ++-- .../{ => troubleshoot-apps}/general-troubleshoot.rst | 0 .../ray-observability/user-guides/troubleshoot-apps/index.md | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) rename doc/source/ray-observability/user-guides/{ => troubleshoot-apps}/general-troubleshoot.rst (100%) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index b49a53539b833..ef5ad81d45833 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -394,10 +394,10 @@ parts: - file: ray-observability/user-guides/index title: User Guides sections: - - file: ray-observability/user-guides/general-troubleshoot - - file: ray-observability/user-guides/troubleshoot-apps/index + - file: ray-observability/user-guides/troubleshoot-apps/index title: Troubleshooting Applications sections: + - file: ray-observability/user-guides/troubleshoot-apps/general-troubleshoot - file: ray-observability/user-guides/troubleshoot-apps/troubleshoot-failures - file: ray-observability/user-guides/troubleshoot-apps/troubleshoot-hangs - file: ray-observability/user-guides/troubleshoot-apps/optimize-performance diff --git a/doc/source/ray-observability/user-guides/general-troubleshoot.rst b/doc/source/ray-observability/user-guides/troubleshoot-apps/general-troubleshoot.rst similarity index 100% rename from doc/source/ray-observability/user-guides/general-troubleshoot.rst rename to doc/source/ray-observability/user-guides/troubleshoot-apps/general-troubleshoot.rst diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/index.md b/doc/source/ray-observability/user-guides/troubleshoot-apps/index.md index cd6562375a40d..fcabd934c8bda 100644 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/index.md +++ b/doc/source/ray-observability/user-guides/troubleshoot-apps/index.md @@ -3,6 +3,7 @@ # Troubleshooting Applications These guides help you perform common debugging or optimization tasks for your distributed application on Ray: +* {ref}`observability-general-troubleshoot` * {ref}`observability-troubleshoot-failures` * {ref}`observability-troubleshoot-hangs` * {ref}`observability-optimize-performance` From 7929f38cecdb15010ee341ffb929132a7e5463ae Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Thu, 18 May 2023 10:47:08 -0700 Subject: [PATCH 24/29] revert inadvertent deletion of section Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../monitoring-and-observability.rst | 41 +++++++++++++++++-- 1 file changed, 38 insertions(+), 3 deletions(-) diff --git a/doc/source/cluster/running-applications/monitoring-and-observability.rst b/doc/source/cluster/running-applications/monitoring-and-observability.rst index 53f21c856ca45..c9dd3e39ee9a9 100644 --- a/doc/source/cluster/running-applications/monitoring-and-observability.rst +++ b/doc/source/cluster/running-applications/monitoring-and-observability.rst @@ -1,6 +1,3 @@ -Scraping and Persisting Metrics -=============================== - Cluster Monitoring ------------------ @@ -16,6 +13,44 @@ The rest of this page will focus on how to access these services when running a .. _monitor-cluster-via-dashboard: +Monitoring the cluster via the dashboard +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +:ref:`The dashboard ` provides detailed information about the state of the cluster, +including the running jobs, actors, workers, nodes, etc. +By default, the :ref:`cluster launcher ` and :ref:`KubeRay operator ` will launch the dashboard, but will +not publicly expose the port. + +.. tab-set:: + + .. tab-item:: If using the VM cluster launcher + + You can securely port-forward local traffic to the dashboard via the ``ray + dashboard`` command. + + .. code-block:: shell + + $ ray dashboard [-p ] + + The dashboard will now be visible at ``http://localhost:8265``. + + .. tab-item:: If using Kubernetes + + The KubeRay operator makes the dashboard available via a Service targeting + the Ray head pod, named ``-head-svc``. You can access the + dashboard from within the Kubernetes cluster at ``http://-head-svc:8265``. + + You can also view the dashboard from outside the Kubernetes cluster by + using port-forwarding: + + .. code-block:: shell + + $ kubectl port-forward service/raycluster-autoscaler-head-svc 8265:8265 + + For more information about configuring network access to a Ray cluster on + Kubernetes, see the :ref:`networking notes `. + + Using Ray Cluster CLI tools ^^^^^^^^^^^^^^^^^^^^^^^^^^^ From 4ae150f01ea545e017781dad57e9bbd72700211f Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Thu, 18 May 2023 10:50:19 -0700 Subject: [PATCH 25/29] change title Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- .../running-applications/monitoring-and-observability.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/doc/source/cluster/running-applications/monitoring-and-observability.rst b/doc/source/cluster/running-applications/monitoring-and-observability.rst index c9dd3e39ee9a9..c0a3e9ac9b6ab 100644 --- a/doc/source/cluster/running-applications/monitoring-and-observability.rst +++ b/doc/source/cluster/running-applications/monitoring-and-observability.rst @@ -1,5 +1,5 @@ -Cluster Monitoring ------------------- +Scraping and Persisting Metrics +=============================== Ray ships with the following observability features: From fc16269bdfb2a0803a67418630f2e69b93cd1a91 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Thu, 18 May 2023 11:35:44 -0700 Subject: [PATCH 26/29] fixed indentation Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index ef5ad81d45833..10c3f93d955a9 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -394,7 +394,7 @@ parts: - file: ray-observability/user-guides/index title: User Guides sections: - - file: ray-observability/user-guides/troubleshoot-apps/index + - file: ray-observability/user-guides/troubleshoot-apps/index title: Troubleshooting Applications sections: - file: ray-observability/user-guides/troubleshoot-apps/general-troubleshoot From 3122f7ed47d4f28f5b88966faaca5f039266a936 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Mon, 22 May 2023 10:07:08 -0700 Subject: [PATCH 27/29] fixing links Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/_toc.yml | 17 +- .../cluster/configure-manage-dashboard.rst | 6 +- .../monitoring-and-observability.rst | 8 +- doc/source/cluster/vms/user-guides/logging.md | 218 ------------ doc/source/data/performance-tips.rst | 2 +- doc/source/ray-air/getting-started.rst | 2 +- .../scheduling/ray-oom-prevention.rst | 2 +- doc/source/ray-core/walkthrough.rst | 2 +- .../ray-observability/getting-started.rst | 14 +- doc/source/ray-observability/key-concepts.rst | 2 +- .../ray-observability/user-guides/index.md | 2 +- .../general-troubleshoot.rst | 217 ------------ .../user-guides/troubleshoot-apps/index.md | 11 - .../optimize-performance.rst | 59 ---- .../troubleshoot-apps/profiling.rst | 325 ------------------ .../troubleshoot-apps/ray-debugging.rst | 325 ------------------ .../troubleshoot-failures.rst | 272 --------------- .../troubleshoot-apps/troubleshoot-hangs.rst | 16 - doc/source/rllib/rllib-training.rst | 2 +- doc/source/train/getting-started.rst | 2 +- doc/source/tune/getting-started.rst | 2 +- 21 files changed, 32 insertions(+), 1474 deletions(-) delete mode 100644 doc/source/cluster/vms/user-guides/logging.md delete mode 100644 doc/source/ray-observability/user-guides/troubleshoot-apps/general-troubleshoot.rst delete mode 100644 doc/source/ray-observability/user-guides/troubleshoot-apps/index.md delete mode 100644 doc/source/ray-observability/user-guides/troubleshoot-apps/optimize-performance.rst delete mode 100644 doc/source/ray-observability/user-guides/troubleshoot-apps/profiling.rst delete mode 100644 doc/source/ray-observability/user-guides/troubleshoot-apps/ray-debugging.rst delete mode 100644 doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-failures.rst delete mode 100644 doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-hangs.rst diff --git a/doc/source/_toc.yml b/doc/source/_toc.yml index 10c3f93d955a9..39d7d442d799b 100644 --- a/doc/source/_toc.yml +++ b/doc/source/_toc.yml @@ -394,15 +394,16 @@ parts: - file: ray-observability/user-guides/index title: User Guides sections: - - file: ray-observability/user-guides/troubleshoot-apps/index - title: Troubleshooting Applications + - file: ray-observability/user-guides/debug-apps/index + title: Debugging Applications sections: - - file: ray-observability/user-guides/troubleshoot-apps/general-troubleshoot - - file: ray-observability/user-guides/troubleshoot-apps/troubleshoot-failures - - file: ray-observability/user-guides/troubleshoot-apps/troubleshoot-hangs - - file: ray-observability/user-guides/troubleshoot-apps/optimize-performance - - file: ray-observability/user-guides/troubleshoot-apps/ray-debugging - - file: ray-observability/user-guides/troubleshoot-apps/ray-core-profiling + - file: ray-observability/user-guides/debug-apps/general-troubleshoot + - file: ray-observability/user-guides/debug-apps/debug-memory + - file: ray-observability/user-guides/debug-apps/debug-hangs + - file: ray-observability/user-guides/debug-apps/debug-failures + - file: ray-observability/user-guides/debug-apps/optimize-performance + - file: ray-observability/user-guides/debug-apps/ray-debugging + - file: ray-observability/user-guides/debug-apps/ray-core-profiling - file: ray-observability/user-guides/cli-sdk - file: ray-observability/user-guides/configure-logging - file: ray-observability/user-guides/add-app-metrics diff --git a/doc/source/cluster/configure-manage-dashboard.rst b/doc/source/cluster/configure-manage-dashboard.rst index 139201d62a3bb..2502e850d29bc 100644 --- a/doc/source/cluster/configure-manage-dashboard.rst +++ b/doc/source/cluster/configure-manage-dashboard.rst @@ -8,7 +8,7 @@ Setting up the dashboard may require some configuration depending on your use mo Port forwarding --------------- -:ref:`The dashboard ` provides detailed information about the state of the cluster, +:ref:`The dashboard ` provides detailed information about the state of the cluster, including the running jobs, actors, workers, nodes, etc. By default, the :ref:`cluster launcher ` and :ref:`KubeRay operator ` will launch the dashboard, but will not publicly expose the port. @@ -248,7 +248,7 @@ Then go to to the location of the binary and run grafana using the built in conf ./bin/grafana-server --config /tmp/ray/session_latest/metrics/grafana/grafana.ini web Now, you can access grafana using the default grafana url, `http://localhost:3000`. -You can then see the default dashboard by going to dashboards -> manage -> Ray -> Default Dashboard. The same :ref:`metric graphs ` are also accessible via :ref:`Ray Dashboard `. +You can then see the default dashboard by going to dashboards -> manage -> Ray -> Default Dashboard. The same :ref:`metric graphs ` are also accessible via :ref:`Ray Dashboard `. .. tip:: @@ -288,7 +288,7 @@ For example, if Prometheus is hosted at port 9000 on a node with ip 55.66.77.88, Alternate Grafana host location ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ You can choose to run Grafana on a non-default port or on a different machine. If you choose to do this, the -:ref:`Dashboard ` needs to be configured with a public address to that service so the web page +:ref:`Dashboard ` needs to be configured with a public address to that service so the web page can load the graphs. This can be done with the `RAY_GRAFANA_HOST` env var when launching ray. The env var takes in the address to access Grafana. More info can be found :ref:`here `. Instructions to use an existing Grafana instance can be found :ref:`here `. diff --git a/doc/source/cluster/running-applications/monitoring-and-observability.rst b/doc/source/cluster/running-applications/monitoring-and-observability.rst index c0a3e9ac9b6ab..80a556d5586a4 100644 --- a/doc/source/cluster/running-applications/monitoring-and-observability.rst +++ b/doc/source/cluster/running-applications/monitoring-and-observability.rst @@ -3,7 +3,7 @@ Scraping and Persisting Metrics Ray ships with the following observability features: -1. :ref:`The dashboard `, for viewing cluster state. +1. :ref:`The dashboard `, for viewing cluster state. 2. CLI tools such as the :ref:`Ray state APIs ` and :ref:`ray status `, for checking application and cluster status. 3. :ref:`Prometheus metrics ` for internal and custom user-defined metrics. @@ -16,7 +16,7 @@ The rest of this page will focus on how to access these services when running a Monitoring the cluster via the dashboard ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ -:ref:`The dashboard ` provides detailed information about the state of the cluster, +:ref:`The dashboard ` provides detailed information about the state of the cluster, including the running jobs, actors, workers, nodes, etc. By default, the :ref:`cluster launcher ` and :ref:`KubeRay operator ` will launch the dashboard, but will not publicly expose the port. @@ -96,14 +96,14 @@ below. Prometheus ^^^^^^^^^^ Ray supports Prometheus for emitting and recording time-series metrics. -See :ref:`metrics ` for more details of the metrics emitted. +See :ref:`metrics ` for more details of the metrics emitted. To use Prometheus in a Ray cluster, decide where to host it, then configure it so that it can scrape the metrics from Ray. Scraping metrics ################ -Ray runs a metrics agent per node to export :ref:`metrics ` about Ray core as well as +Ray runs a metrics agent per node to export :ref:`metrics ` about Ray core as well as custom user-defined metrics. Each metrics agent collects metrics from the local node and exposes these in a Prometheus format. You can then scrape each endpoint to access Ray's metrics. diff --git a/doc/source/cluster/vms/user-guides/logging.md b/doc/source/cluster/vms/user-guides/logging.md deleted file mode 100644 index dd97643b5e2a9..0000000000000 --- a/doc/source/cluster/vms/user-guides/logging.md +++ /dev/null @@ -1,218 +0,0 @@ -(vms-logging)= - -# Log Persistence - -This page provides tips on how to collect logs from -Ray clusters running on Kubernetes. - -:::{tip} -Skip to {ref}`the deployment instructions ` -for a sample configuration showing how to extract logs from a Ray pod. -::: - -## The Ray log directory -Each Ray pod runs several component processes, such as the Raylet, object manager, dashboard agent, etc. -These components log to files in the directory `/tmp/ray/session_latest/logs` in the pod's file system. -Extracting and persisting these logs requires some setup. - -## Log processing tools -There are a number of log processing tools available within the Kubernetes -ecosystem. This page will shows how to extract Ray logs using [Fluent Bit][FluentBit]. -Other popular tools include [Fluentd][Fluentd], [Filebeat][Filebeat], and [Promtail][Promtail]. - -## Log collection strategies -We mention two strategies for collecting logs written to a pod's filesystem, -**sidecar containers** and **daemonsets**. You can read more about these logging -patterns in the [Kubernetes documentation][KubDoc]. - -### Sidecar containers -We will provide an {ref}`example ` of the sidecar strategy in this guide. -You can process logs by configuring a log-processing sidecar -for each Ray pod. Ray containers should be configured to share the `/tmp/ray` -directory with the logging sidecar via a volume mount. - -You can configure the sidecar to do either of the following: -* Stream Ray logs to the sidecar's stdout. -* Export logs to an external service. - -### Daemonset -Alternatively, it is possible to collect logs at the Kubernetes node level. -To do this, one deploys a log-processing daemonset onto the Kubernetes cluster's -nodes. With this strategy, it is key to mount -the Ray container's `/tmp/ray` directory to the relevant `hostPath`. - -(kuberay-fluentbit)= -## Setting up logging sidecars with Fluent Bit -In this section, we give an example of how to set up log-emitting -[Fluent Bit][FluentBit] sidecars for Ray pods. - -See the full config for a single-pod RayCluster with a logging sidecar [here][ConfigLink]. -We now discuss this configuration and show how to deploy it. - -### Configure log processing -The first step is to create a ConfigMap with configuration -for Fluent Bit. - -Here is a minimal ConfigMap which tells a Fluent Bit sidecar to -* Tail Ray logs. -* Output the logs to the container's stdout. -```{literalinclude} ../configs/ray-cluster.log.yaml -:language: yaml -:start-after: Fluent Bit ConfigMap -:end-before: --- -``` -A few notes on the above config: -- In addition to streaming logs to stdout, you can use an [OUTPUT] clause to export logs to any - [storage backend][FluentBitStorage] supported by Fluent Bit. -- The `Path_Key true` line above ensures that file names are included in the log records - emitted by Fluent Bit. -- The `Refresh_Interval 5` line asks Fluent Bit to refresh the list of files - in the log directory once per 5 seconds, rather than the default 60. - The reason is that the directory `/tmp/ray/session_latest/logs/` does not exist - initially (Ray must create it first). Setting the `Refresh_Interval` low allows us to see logs - in the Fluent Bit container's stdout sooner. - - -### Add logging sidecars to your RayCluster CR - -#### Add log and config volumes -For each pod template in our RayCluster CR, we -need to add two volumes: One volume for Ray's logs -and another volume to store Fluent Bit configuration from the ConfigMap -applied above. -```{literalinclude} ../configs/ray-cluster.log.yaml -:language: yaml -:start-after: Log and config volumes -``` - -#### Mount the Ray log directory -Add the following volume mount to the Ray container's configuration. -```{literalinclude} ../configs/ray-cluster.log.yaml -:language: yaml -:start-after: Share logs with Fluent Bit -:end-before: Fluent Bit sidecar -``` - -#### Add the Fluent Bit sidecar -Finally, add the Fluent Bit sidecar container to each Ray pod config -in your RayCluster CR. -```{literalinclude} ../configs/ray-cluster.log.yaml -:language: yaml -:start-after: Fluent Bit sidecar -:end-before: Log and config volumes -``` -Mounting the `ray-logs` volume gives the sidecar container access to Ray's logs. -The `fluentbit-config` volume gives the sidecar access to logging configuration. - -#### Putting everything together -Putting all of the above elements together, we have the following yaml configuration -for a single-pod RayCluster will a log-processing sidecar. -```{literalinclude} ../configs/ray-cluster.log.yaml -:language: yaml -``` - -### Deploying a RayCluster with logging CR -(kuberay-logging-tldr)= -Now, we will see how to deploy the configuration described above. - -Deploy the KubeRay Operator if you haven't yet. -Refer to the {ref}`Getting Started guide ` -for instructions on this step. - -Now, run the following commands to deploy the Fluent Bit ConfigMap and a single-pod RayCluster with -a Fluent Bit sidecar. -```shell -kubectl apply -f https://raw.githubusercontent.com/ray-project/ray/releases/2.4.0/doc/source/cluster/kubernetes/configs/ray-cluster.log.yaml -``` - -Determine the Ray pod's name with -```shell -kubectl get pod | grep raycluster-complete-logs -``` - -Examine the FluentBit sidecar's STDOUT to see logs for Ray's component processes. -```shell -# Substitute the name of your Ray pod. -kubectl logs raycluster-complete-logs-head-xxxxx -c fluentbit -``` - -[FluentBit]: https://docs.fluentbit.io/manual -[FluentBitStorage]: https://docs.fluentbit.io/manual -[Filebeat]: https://www.elastic.co/guide/en/beats/filebeat/7.17/index.html -[Fluentd]: https://docs.fluentd.org/ -[Promtail]: https://grafana.com/docs/loki/latest/clients/promtail/ -[KubDoc]: https://kubernetes.io/docs/concepts/cluster-administration/logging/ -[ConfigLink]: https://raw.githubusercontent.com/ray-project/ray/releases/2.4.0/doc/source/cluster/kubernetes/configs/ray-cluster.log.yaml - -## Setting up loggers - -When using Ray, all of the tasks and actors are executed remotely in Ray's worker processes. -Since Python logger module creates a singleton logger per process, loggers should be configured on per task/actor basis. - -:::{note} -To stream logs to a driver, they should be flushed to stdout and stderr. -::: - -```python -import ray -import logging -# Initiate a driver. -ray.init() - -@ray.remote -class Actor: - def __init__(self): - # Basic config automatically configures logs to - # be streamed to stdout and stderr. - # Set the severity to INFO so that info logs are printed to stdout. - logging.basicConfig(level=logging.INFO) - - def log(self, msg): - logging.info(msg) - -actor = Actor.remote() -ray.get(actor.log.remote("A log message for an actor.")) - -@ray.remote -def f(msg): - logging.basicConfig(level=logging.INFO) - logging.info(msg) - -ray.get(f.remote("A log message for a task")) -``` - -```bash -(pid=95193) INFO:root:A log message for a task -(pid=95192) INFO:root:A log message for an actor. -``` -## Using structured logging - -The metadata of tasks or actors may be obtained by Ray's :ref:`runtime_context APIs `. -Runtime context APIs help you to add metadata to your logging messages, making your logs more structured. - -```python -import ray -# Initiate a driver. -ray.init() - - @ray.remote -def task(): - print(f"task_id: {ray.get_runtime_context().task_id}") - -ray.get(task.remote()) -``` - -```bash -(pid=47411) task_id: TaskID(a67dc375e60ddd1affffffffffffffffffffffff01000000) -``` -## Rotating logs - -Ray supports log rotation of log files. Note that not all components are currently supporting log rotation. (Raylet and Python/Java worker logs are not rotating). - -By default, logs are rotating when it reaches to 512MB (maxBytes), and there could be up to 5 backup files (backupCount). Indexes are appended to all backup files (e.g., `raylet.out.1`) -If you'd like to change the log rotation configuration, you can do it by specifying environment variables. For example, - -```bash -RAY_ROTATION_MAX_BYTES=1024; ray start --head # Start a ray instance with maxBytes 1KB. -RAY_ROTATION_BACKUP_COUNT=1; ray start --head # Start a ray instance with backupCount 1. -``` \ No newline at end of file diff --git a/doc/source/data/performance-tips.rst b/doc/source/data/performance-tips.rst index 6d9ec81d9bb5e..2ff9ef94ee1fe 100644 --- a/doc/source/data/performance-tips.rst +++ b/doc/source/data/performance-tips.rst @@ -7,7 +7,7 @@ Monitoring your application ~~~~~~~~~~~~~~~~~~~~~~~~~~~ View the Ray dashboard to monitor your application and troubleshoot issues. To learn -more about the Ray dashboard, read :ref:`Ray Dashboard `. +more about the Ray dashboard, read :ref:`Ray Dashboard `. Debugging Statistics ~~~~~~~~~~~~~~~~~~~~ diff --git a/doc/source/ray-air/getting-started.rst b/doc/source/ray-air/getting-started.rst index e35f1e7c11921..9bcdd66f18eae 100644 --- a/doc/source/ray-air/getting-started.rst +++ b/doc/source/ray-air/getting-started.rst @@ -216,4 +216,4 @@ Next Steps - :ref:`air-examples-ref` - :ref:`API reference ` - :ref:`Technical whitepaper ` -- To check how your application is doing, you can use the :ref:`Ray dashboard`. +- To check how your application is doing, you can use the :ref:`Ray dashboard`. diff --git a/doc/source/ray-core/scheduling/ray-oom-prevention.rst b/doc/source/ray-core/scheduling/ray-oom-prevention.rst index eaa0076465d72..99aa1bc215ced 100644 --- a/doc/source/ray-core/scheduling/ray-oom-prevention.rst +++ b/doc/source/ray-core/scheduling/ray-oom-prevention.rst @@ -1,7 +1,7 @@ Out-Of-Memory Prevention ======================== -If application tasks or actors consume a large amount of heap space, it can cause the node to run out of memory (OOM). When that happens, the operating system will start killing worker or raylet processes, disrupting the application. OOM may also stall metrics and if this happens on the head node, it may stall the :ref:`dashboard ` or other control processes and cause the cluster to become unusable. +If application tasks or actors consume a large amount of heap space, it can cause the node to run out of memory (OOM). When that happens, the operating system will start killing worker or raylet processes, disrupting the application. OOM may also stall metrics and if this happens on the head node, it may stall the :ref:`dashboard ` or other control processes and cause the cluster to become unusable. In this section we will go over: diff --git a/doc/source/ray-core/walkthrough.rst b/doc/source/ray-core/walkthrough.rst index e4f721a7d59ed..d1522fbed31c2 100644 --- a/doc/source/ray-core/walkthrough.rst +++ b/doc/source/ray-core/walkthrough.rst @@ -60,7 +60,7 @@ As seen above, Ray stores task and actor call results in its :ref:`distributed o Next Steps ---------- -.. tip:: To check how your application is doing, you can use the :ref:`Ray dashboard `. +.. tip:: To check how your application is doing, you can use the :ref:`Ray dashboard `. Ray's key primitives are simple, but can be composed together to express almost any kind of distributed computation. Learn more about Ray's :ref:`key concepts ` with the following user guides: diff --git a/doc/source/ray-observability/getting-started.rst b/doc/source/ray-observability/getting-started.rst index c0e57bdf5f0fd..858794dc384a3 100644 --- a/doc/source/ray-observability/getting-started.rst +++ b/doc/source/ray-observability/getting-started.rst @@ -49,7 +49,7 @@ Ray cluster comes with the dashboard. See :ref:`Cluster Monitoring `. - See :ref:`Ray Metrics ` to learn how to set up Prometheus and Grafana. + See :ref:`Configuring and Managing the Dashboard ` to learn how to set up Prometheus and Grafana. .. _dash-workflow-cpu-memory-analysis: @@ -145,7 +145,7 @@ The Serve view lets you monitor the status of your :ref:`Ray Serve ` a
-The initial page showcases your general Serve configurations, a list of the Serve applications, and, if you have :ref:`Grafana and Prometheus ` configured, some high-level +The initial page showcases your general Serve configurations, a list of the Serve applications, and, if you have :ref:`Grafana and Prometheus ` configured, some high-level metrics of all your Serve applications. Click the name of a Serve application to go to the Serve Application Detail Page. Serve Application Detail Page @@ -154,17 +154,17 @@ Serve Application Detail Page This page shows the Serve application's configurations and metadata. It also lists the :ref:`Serve deployments and replicas `. Click the expand button of a deployment to see all the replicas in that deployment. -For each deployment, there are two available actions. You can view the Deployment config and, if you configured :ref:`Grafana and Prometheus `, you can open +For each deployment, there are two available actions. You can view the Deployment config and, if you configured :ref:`Grafana and Prometheus `, you can open a Grafana dashboard with detailed metrics about that deployment. -For each replica, there are two available actions. You can see the logs of that replica and, if you configured :ref:`Grafana and Prometheus `, you can open +For each replica, there are two available actions. You can see the logs of that replica and, if you configured :ref:`Grafana and Prometheus `, you can open a Grafana dashboard with detailed metrics about that replica. Click on the replica name to go to the Serve Replica Detail Page. Serve Replica Detail Page ~~~~~~~~~~~~~~~~~~~~~~~~~ -This page shows metadata about the Serve replica, high-level metrics about the replica if you configured :ref:`Grafana and Prometheus `, and +This page shows metadata about the Serve replica, high-level metrics about the replica if you configured :ref:`Grafana and Prometheus `, and a history of completed :ref:`tasks ` of that replica. @@ -172,7 +172,7 @@ Serve Metrics ~~~~~~~~~~~~~ Ray serve exports various time-series metrics to understand the status of your Serve application over time. More details of these metrics can be found :ref:`here `. -In order to store and visualize these metrics, you must set up Prometheus and Grafana by following the instructions :ref:`here `. +In order to store and visualize these metrics, you must set up Prometheus and Grafana by following the instructions :ref:`here `. These metrics are available in the Ray dashboard in the Serve page and the Serve Replica Detail page. They are also accessible as Grafana dashboards. Within the Grafana dashboard, use the dropdown filters on the top to filter metrics by route, deployment, or replica. Exact descriptions @@ -251,7 +251,7 @@ See :ref:`System Metrics Page ` for available metrics. .. note:: - The metrics view required the Prometheus and Grafana setup. See :ref:`Ray Metrics ` to learn how to set up Prometheus and Grafana. + The metrics view required the Prometheus and Grafana setup. See :ref:`Configuring and Managing the Dashboard ` to learn how to set up Prometheus and Grafana. The metrics view lets you view visualizations of the time series metrics emitted by Ray. diff --git a/doc/source/ray-observability/key-concepts.rst b/doc/source/ray-observability/key-concepts.rst index 982f6e99d5f69..fc650f6eaa7e3 100644 --- a/doc/source/ray-observability/key-concepts.rst +++ b/doc/source/ray-observability/key-concepts.rst @@ -10,7 +10,7 @@ Dashboard (Web UI) Ray supports the web-based dashboard to help users monitor the cluster. When a new cluster is started, the dashboard is available through the default address `localhost:8265` (port can be automatically incremented if port 8265 is already occupied). -See :ref:`Ray Dashboard ` for more details. +See :ref:`Getting Started ` for more details about the dashboard. Accessing Ray States -------------------- diff --git a/doc/source/ray-observability/user-guides/index.md b/doc/source/ray-observability/user-guides/index.md index 748cc34b51f90..9b81d03a55915 100644 --- a/doc/source/ray-observability/user-guides/index.md +++ b/doc/source/ray-observability/user-guides/index.md @@ -6,7 +6,7 @@ These guides help you monitor and debug your Ray applications and clusters. The guides include: * {ref}`observability-general-troubleshoot` -* {ref}`observability-troubleshoot-user-guides` +* {ref}`observability-user-guides` * {ref}`observability-programmatic` * {ref}`configure-logging` * {ref}`application-level-metrics` diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/general-troubleshoot.rst b/doc/source/ray-observability/user-guides/troubleshoot-apps/general-troubleshoot.rst deleted file mode 100644 index 96ffbab7cc021..0000000000000 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/general-troubleshoot.rst +++ /dev/null @@ -1,217 +0,0 @@ -.. _observability-general-troubleshoot: - -General Troubleshooting -======================= - -Ray sometimes has some aspects of its behavior that might catch -users off guard. There may be sound arguments for these design choices. - -In particular, users think of Ray as running on their local machine, and -while this is mostly true, this doesn't work. - -Environment variables are not passed from the driver to workers ---------------------------------------------------------------- - -**Issue**: If you set an environment variable at the command line, it is not passed to all the workers running in the cluster -if the cluster was started previously. - -**Example**: If you have a file ``baz.py`` in the directory you are running Ray in, and you run the following command: - -.. literalinclude:: ../../ray-core/doc_code/gotchas.py - :language: python - :start-after: __env_var_start__ - :end-before: __env_var_end__ - -**Expected behavior**: Most people would expect (as if it was a single process on a single machine) that the environment variables would be the same in all workers. It won’t be. - -**Fix**: Use runtime environments to pass environment variables explicity. -If you call ``ray.init(runtime_env=...)``, -then the workers will have the environment variable set. - - -.. literalinclude:: ../../ray-core/doc_code/gotchas.py - :language: python - :start-after: __env_var_fix_start__ - :end-before: __env_var_fix_end__ - - -Filenames work sometimes and not at other times ------------------------------------------------ - -**Issue**: If you reference a file by name in a task or actor, -it will sometimes work and sometimes fail. This is -because if the task or actor runs on the head node -of the cluster, it will work, but if the task or actor -runs on another machine it won't. - -**Example**: Let's say we do the following command: - -.. code-block:: bash - - % touch /tmp/foo.txt - -And I have this code: - -.. code-block:: python - - import os - - ray.init() - @ray.remote - def check_file(): - foo_exists = os.path.exists("/tmp/foo.txt") - print(f"Foo exists? {foo_exists}") - - futures = [] - for _ in range(1000): - futures.append(check_file.remote()) - - ray.get(futures) - - -then you will get a mix of True and False. If -``check_file()`` runs on the head node, or we're running -locally it works. But if it runs on a worker node, it returns ``False``. - -**Expected behavior**: Most people would expect this to either fail or succeed consistently. -It's the same code after all. - -**Fix** - -- Use only shared paths for such applications -- e.g. if you are using a network file system you can use that, or the files can be on s3. -- Do not rely on file path consistency. - - - -Placement groups are not composable ------------------------------------ - -**Issue**: If you have a task that is called from something that runs in a placement -group, the resources are never allocated and it hangs. - -**Example**: You are using Ray Tune which creates placement groups, and you want to -apply it to an objective function, but that objective function makes use -of Ray Tasks itself, e.g. - -.. code-block:: python - - from ray import air, tune - - def create_task_that_uses_resources(): - @ray.remote(num_cpus=10) - def sample_task(): - print("Hello") - return - - return ray.get([sample_task.remote() for i in range(10)]) - - def objective(config): - create_task_that_uses_resources() - - tuner = tune.Tuner(objective, param_space={"a": 1}) - tuner.fit() - -This will error with message: -ValueError: Cannot schedule create_task_that_uses_resources..sample_task with the placement group -because the resource request {'CPU': 10} cannot fit into any bundles for the placement group, [{'CPU': 1.0}]. - -**Expected behavior**: The above executes. - -**Fix**: In the ``@ray.remote`` declaration of tasks -called by ``create_task_that_uses_resources()`` , include a -``scheduling_strategy=PlacementGroupSchedulingStrategy(placement_group=None)``. - -.. code-block:: diff - - def create_task_that_uses_resources(): - + @ray.remote(num_cpus=10, scheduling_strategy=PlacementGroupSchedulingStrategy(placement_group=None)) - - @ray.remote(num_cpus=10) - -Outdated Function Definitions ------------------------------ - -Due to subtleties of Python, if you redefine a remote function, you may not -always get the expected behavior. In this case, it may be that Ray is not -running the newest version of the function. - -Suppose you define a remote function ``f`` and then redefine it. Ray should use -the newest version. - -.. code-block:: python - - @ray.remote - def f(): - return 1 - - @ray.remote - def f(): - return 2 - - ray.get(f.remote()) # This should be 2. - -However, the following are cases where modifying the remote function will -not update Ray to the new version (at least without stopping and restarting -Ray). - -- **The function is imported from an external file:** In this case, - ``f`` is defined in some external file ``file.py``. If you ``import file``, - change the definition of ``f`` in ``file.py``, then re-``import file``, - the function ``f`` will not be updated. - - This is because the second import gets ignored as a no-op, so ``f`` is - still defined by the first import. - - A solution to this problem is to use ``reload(file)`` instead of a second - ``import file``. Reloading causes the new definition of ``f`` to be - re-executed, and exports it to the other machines. Note that in Python 3, you - need to do ``from importlib import reload``. - -- **The function relies on a helper function from an external file:** - In this case, ``f`` can be defined within your Ray application, but relies - on a helper function ``h`` defined in some external file ``file.py``. If the - definition of ``h`` gets changed in ``file.py``, redefining ``f`` will not - update Ray to use the new version of ``h``. - - This is because when ``f`` first gets defined, its definition is shipped to - all of the workers, and is unpickled. During unpickling, ``file.py`` gets - imported in the workers. Then when ``f`` gets redefined, its definition is - again shipped and unpickled in all of the workers. But since ``file.py`` - has been imported in the workers already, it is treated as a second import - and is ignored as a no-op. - - Unfortunately, reloading on the driver does not update ``h``, as the reload - needs to happen on the worker. - - A solution to this problem is to redefine ``f`` to reload ``file.py`` before - it calls ``h``. For example, if inside ``file.py`` you have - - .. code-block:: python - - def h(): - return 1 - - And you define remote function ``f`` as - - .. code-block:: python - - @ray.remote - def f(): - return file.h() - - You can redefine ``f`` as follows. - - .. code-block:: python - - @ray.remote - def f(): - reload(file) - return file.h() - - This forces the reload to happen on the workers as needed. Note that in - Python 3, you need to do ``from importlib import reload``. - -This document discusses some common problems that people run into when using Ray -as well as some known problems. If you encounter other problems, please -`let us know`_. - -.. _`let us know`: https://github.com/ray-project/ray/issues diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/index.md b/doc/source/ray-observability/user-guides/troubleshoot-apps/index.md deleted file mode 100644 index fcabd934c8bda..0000000000000 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/index.md +++ /dev/null @@ -1,11 +0,0 @@ -(observability-troubleshoot-user-guides)= - -# Troubleshooting Applications - -These guides help you perform common debugging or optimization tasks for your distributed application on Ray: -* {ref}`observability-general-troubleshoot` -* {ref}`observability-troubleshoot-failures` -* {ref}`observability-troubleshoot-hangs` -* {ref}`observability-optimize-performance` -* {ref}`ray-debugger` -* {ref}`ray-core-profiling` \ No newline at end of file diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/optimize-performance.rst b/doc/source/ray-observability/user-guides/troubleshoot-apps/optimize-performance.rst deleted file mode 100644 index 465f7b6b5c524..0000000000000 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/optimize-performance.rst +++ /dev/null @@ -1,59 +0,0 @@ -.. _observability-optimize-performance: - -Optimizing Performance -====================== - -No Speedup ----------- - -You just ran an application using Ray, but it wasn't as fast as you expected it -to be. Or worse, perhaps it was slower than the serial version of the -application! The most common reasons are the following. - -- **Number of cores:** How many cores is Ray using? When you start Ray, it will - determine the number of CPUs on each machine with ``psutil.cpu_count()``. Ray - usually will not schedule more tasks in parallel than the number of CPUs. So - if the number of CPUs is 4, the most you should expect is a 4x speedup. - -- **Physical versus logical CPUs:** Do the machines you're running on have fewer - **physical** cores than **logical** cores? You can check the number of logical - cores with ``psutil.cpu_count()`` and the number of physical cores with - ``psutil.cpu_count(logical=False)``. This is common on a lot of machines and - especially on EC2. For many workloads (especially numerical workloads), you - often cannot expect a greater speedup than the number of physical CPUs. - -- **Small tasks:** Are your tasks very small? Ray introduces some overhead for - each task (the amount of overhead depends on the arguments that are passed - in). You will be unlikely to see speedups if your tasks take less than ten - milliseconds. For many workloads, you can easily increase the sizes of your - tasks by batching them together. - -- **Variable durations:** Do your tasks have variable duration? If you run 10 - tasks with variable duration in parallel, you shouldn't expect an N-fold - speedup (because you'll end up waiting for the slowest task). In this case, - consider using ``ray.wait`` to begin processing tasks that finish first. - -- **Multi-threaded libraries:** Are all of your tasks attempting to use all of - the cores on the machine? If so, they are likely to experience contention and - prevent your application from achieving a speedup. - This is common with some versions of ``numpy``. To avoid contention, set an - environment variable like ``MKL_NUM_THREADS`` (or the equivalent depending on - your installation) to ``1``. - - For many - but not all - libraries, you can diagnose this by opening ``top`` - while your application is running. If one process is using most of the CPUs, - and the others are using a small amount, this may be the problem. The most - common exception is PyTorch, which will appear to be using all the cores - despite needing ``torch.set_num_threads(1)`` to be called to avoid contention. - -If you are still experiencing a slowdown, but none of the above problems apply, -we'd really like to know! Please create a `GitHub issue`_ and consider -submitting a minimal code example that demonstrates the problem. - -.. _`Github issue`: https://github.com/ray-project/ray/issues - -This document discusses some common problems that people run into when using Ray -as well as some known problems. If you encounter other problems, please -`let us know`_. - -.. _`let us know`: https://github.com/ray-project/ray/issues diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/profiling.rst b/doc/source/ray-observability/user-guides/troubleshoot-apps/profiling.rst deleted file mode 100644 index 756005eaec21f..0000000000000 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/profiling.rst +++ /dev/null @@ -1,325 +0,0 @@ -.. _ray-core-profiling: - -Profiling -========= - -.. _ray-core-mem-profiling: - -Memory profile Ray Actors and Tasks ------------------------------------ - -To memory profile Ray tasks or actors, use `memray `_. -Note that you can also use other memory profiling tools if it supports a similar API. - -First, install ``memray``. - -.. code-block:: bash - - pip install memray - -``memray`` supports a Python context manager to enable memory profiling. You can write the ``memray`` profiling file wherever you want. -But in this example, we will write them to `/tmp/ray/session_latest/logs` because Ray dashboard allows you to download files inside the log folder. -This will allow you to download profiling files from other nodes. - -.. tab-set:: - - .. tab-item:: Actors - - .. literalinclude:: ../doc_code/memray_profiling.py - :language: python - :start-after: __memray_profiling_start__ - :end-before: __memray_profiling_end__ - - .. tab-item:: Tasks - - Note that tasks have a shorter lifetime, so there could be lots of memory profiling files. - - .. literalinclude:: ../doc_code/memray_profiling.py - :language: python - :start-after: __memray_profiling_task_start__ - :end-before: __memray_profiling_task_end__ - -Once the task or actor runs, go to the :ref:`Logs View ` of the dashboard. Find and click the log file name. - -.. image:: ../images/memory-profiling-files.png - :align: center - -Click the download button. - -.. image:: ../images/download-memory-profiling-files.png - :align: center - -Now, you have the memory profiling file. Running - -.. code-block:: bash - - memray flamegraph - -And you can see the result of the memory profiling! - -.. _ray-core-timeline: - -Visualizing Tasks in the Ray Timeline -------------------------------------- - -The most important tool is the timeline visualization tool. To visualize tasks -in the Ray timeline, you can dump the timeline as a JSON file by running ``ray -timeline`` from the command line or ``ray.timeline`` from the Python API. - -To use the timeline, Ray profiling must be enabled by setting the -``RAY_PROFILING=1`` environment variable prior to starting Ray on every machine, and ``RAY_task_events_report_interval_ms`` must be larger than 0 (default 1000). - -.. code-block:: python - - ray.timeline(filename="/tmp/timeline.json") - -Then open `chrome://tracing`_ in the Chrome web browser, and load -``timeline.json``. - -.. _`chrome://tracing`: chrome://tracing - -.. _dashboard-profiling: - -Python CPU Profiling in the Dashboard -------------------------------------- - -The :ref:`ray-dashboard` lets you profile Ray worker processes by clicking on the "Stack Trace" or "CPU Flame Graph" -actions for active workers, actors, and jobs. - -.. image:: /images/profile.png - :align: center - :width: 80% - -Clicking "Stack Trace" will return the current stack trace sample using ``py-spy``. By default, only the Python stack -trace is shown. To show native code frames, set the URL parameter ``native=1`` (only supported on Linux). - -.. image:: /images/stack.png - :align: center - :width: 60% - -Clicking "CPU Flame Graph" will take a number of stack trace samples and combine them into a flame graph visualization. -This flame graph can be useful for understanding the CPU activity of the particular process. To adjust the duration -of the flame graph, you can change the ``duration`` parameter in the URL. Similarly, you can change the ``native`` -parameter to enable native profiling. - -.. image:: /images/flamegraph.png - :align: center - :width: 80% - -The profiling feature requires ``py-spy`` to be installed. If it is not installed, or if the ``py-spy`` binary does -not have root permissions, the dashboard will prompt with instructions on how to setup ``py-spy`` correctly: - -.. code-block:: - - This command requires `py-spy` to be installed with root permissions. You - can install `py-spy` and give it root permissions as follows: - $ pip install py-spy - $ sudo chown root:root `which py-spy` - $ sudo chmod u+s `which py-spy` - - Alternatively, you can start Ray with passwordless sudo / root permissions. - -Profiling Using Python's CProfile ---------------------------------- - -You can use Python's native cProfile `profiling module`_ to profile the performance of your Ray application. Rather than tracking -line-by-line of your application code, cProfile can give the total runtime -of each loop function, as well as list the number of calls made and -execution time of all function calls made within the profiled code. - -.. _`profiling module`: https://docs.python.org/3/library/profile.html#module-cProfile - -Unlike ``line_profiler`` above, this detailed list of profiled function calls -**includes** internal function calls and function calls made within Ray. - -However, similar to ``line_profiler``, cProfile can be enabled with minimal -changes to your application code (given that each section of the code you want -to profile is defined as its own function). To use cProfile, add an import -statement, then replace calls to the loop functions as follows: - -.. code-block:: python - - import cProfile # Added import statement - - def ex1(): - list1 = [] - for i in range(5): - list1.append(ray.get(func.remote())) - - def main(): - ray.init() - cProfile.run('ex1()') # Modified call to ex1 - cProfile.run('ex2()') - cProfile.run('ex3()') - - if __name__ == "__main__": - main() - -Now, when you execute your Python script, a cProfile list of profiled function -calls are printed on the terminal for each call made to ``cProfile.run()``. -At the very top of cProfile's output gives the total execution time for -``'ex1()'``: - -.. code-block:: bash - - 601 function calls (595 primitive calls) in 2.509 seconds - -Following is a snippet of profiled function calls for ``'ex1()'``. Most of -these calls are quick and take around 0.000 seconds, so the functions of -interest are the ones with non-zero execution times: - -.. code-block:: bash - - ncalls tottime percall cumtime percall filename:lineno(function) - ... - 1 0.000 0.000 2.509 2.509 your_script_here.py:31(ex1) - 5 0.000 0.000 0.001 0.000 remote_function.py:103(remote) - 5 0.000 0.000 0.001 0.000 remote_function.py:107(_remote) - ... - 10 0.000 0.000 0.000 0.000 worker.py:2459(__init__) - 5 0.000 0.000 2.508 0.502 worker.py:2535(get) - 5 0.000 0.000 0.000 0.000 worker.py:2695(get_global_worker) - 10 0.000 0.000 2.507 0.251 worker.py:374(retrieve_and_deserialize) - 5 0.000 0.000 2.508 0.502 worker.py:424(get_object) - 5 0.000 0.000 0.000 0.000 worker.py:514(submit_task) - ... - -The 5 separate calls to Ray's ``get``, taking the full 0.502 seconds each call, -can be noticed at ``worker.py:2535(get)``. Meanwhile, the act of calling the -remote function itself at ``remote_function.py:103(remote)`` only takes 0.001 -seconds over 5 calls, and thus is not the source of the slow performance of -``ex1()``. - - -Profiling Ray Actors with cProfile -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Considering that the detailed output of cProfile can be quite different depending -on what Ray functionalities we use, let us see what cProfile's output might look -like if our example involved Actors (for an introduction to Ray actors, see our -`Actor documentation here`_). - -.. _`Actor documentation here`: http://docs.ray.io/en/master/actors.html - -Now, instead of looping over five calls to a remote function like in ``ex1``, -let's create a new example and loop over five calls to a remote function -**inside an actor**. Our actor's remote function again just sleeps for 0.5 -seconds: - -.. code-block:: python - - # Our actor - @ray.remote - class Sleeper(object): - def __init__(self): - self.sleepValue = 0.5 - - # Equivalent to func(), but defined within an actor - def actor_func(self): - time.sleep(self.sleepValue) - -Recalling the suboptimality of ``ex1``, let's first see what happens if we -attempt to perform all five ``actor_func()`` calls within a single actor: - -.. code-block:: python - - def ex4(): - # This is suboptimal in Ray, and should only be used for the sake of this example - actor_example = Sleeper.remote() - - five_results = [] - for i in range(5): - five_results.append(actor_example.actor_func.remote()) - - # Wait until the end to call ray.get() - ray.get(five_results) - -We enable cProfile on this example as follows: - -.. code-block:: python - - def main(): - ray.init() - cProfile.run('ex4()') - - if __name__ == "__main__": - main() - -Running our new Actor example, cProfile's abbreviated output is as follows: - -.. code-block:: bash - - 12519 function calls (11956 primitive calls) in 2.525 seconds - - ncalls tottime percall cumtime percall filename:lineno(function) - ... - 1 0.000 0.000 0.015 0.015 actor.py:546(remote) - 1 0.000 0.000 0.015 0.015 actor.py:560(_remote) - 1 0.000 0.000 0.000 0.000 actor.py:697(__init__) - ... - 1 0.000 0.000 2.525 2.525 your_script_here.py:63(ex4) - ... - 9 0.000 0.000 0.000 0.000 worker.py:2459(__init__) - 1 0.000 0.000 2.509 2.509 worker.py:2535(get) - 9 0.000 0.000 0.000 0.000 worker.py:2695(get_global_worker) - 4 0.000 0.000 2.508 0.627 worker.py:374(retrieve_and_deserialize) - 1 0.000 0.000 2.509 2.509 worker.py:424(get_object) - 8 0.000 0.000 0.001 0.000 worker.py:514(submit_task) - ... - -It turns out that the entire example still took 2.5 seconds to execute, or the -time for five calls to ``actor_func()`` to run in serial. If you recall ``ex1``, -this behavior was because we did not wait until after submitting all five -remote function tasks to call ``ray.get()``, but we can verify on cProfile's -output line ``worker.py:2535(get)`` that ``ray.get()`` was only called once at -the end, for 2.509 seconds. What happened? - -It turns out Ray cannot parallelize this example, because we have only -initialized a single ``Sleeper`` actor. Because each actor is a single, -stateful worker, our entire code is submitted and ran on a single worker the -whole time. - -To better parallelize the actors in ``ex4``, we can take advantage -that each call to ``actor_func()`` is independent, and instead -create five ``Sleeper`` actors. That way, we are creating five workers -that can run in parallel, instead of creating a single worker that -can only handle one call to ``actor_func()`` at a time. - -.. code-block:: python - - def ex4(): - # Modified to create five separate Sleepers - five_actors = [Sleeper.remote() for i in range(5)] - - # Each call to actor_func now goes to a different Sleeper - five_results = [] - for actor_example in five_actors: - five_results.append(actor_example.actor_func.remote()) - - ray.get(five_results) - -Our example in total now takes only 1.5 seconds to run: - -.. code-block:: bash - - 1378 function calls (1363 primitive calls) in 1.567 seconds - - ncalls tottime percall cumtime percall filename:lineno(function) - ... - 5 0.000 0.000 0.002 0.000 actor.py:546(remote) - 5 0.000 0.000 0.002 0.000 actor.py:560(_remote) - 5 0.000 0.000 0.000 0.000 actor.py:697(__init__) - ... - 1 0.000 0.000 1.566 1.566 your_script_here.py:71(ex4) - ... - 21 0.000 0.000 0.000 0.000 worker.py:2459(__init__) - 1 0.000 0.000 1.564 1.564 worker.py:2535(get) - 25 0.000 0.000 0.000 0.000 worker.py:2695(get_global_worker) - 3 0.000 0.000 1.564 0.521 worker.py:374(retrieve_and_deserialize) - 1 0.000 0.000 1.564 1.564 worker.py:424(get_object) - 20 0.001 0.000 0.001 0.000 worker.py:514(submit_task) - ... - -Profiling (Internal) --------------------- -If you are developing Ray core or debugging some system level failures, profiling the Ray core could help. In this case, see :ref:`Profiling (Internal) `. diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/ray-debugging.rst b/doc/source/ray-observability/user-guides/troubleshoot-apps/ray-debugging.rst deleted file mode 100644 index afc98df070a61..0000000000000 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/ray-debugging.rst +++ /dev/null @@ -1,325 +0,0 @@ -.. _ray-debugger: - -Using the Ray Debugger -====================== - -Ray has a built in debugger that allows you to debug your distributed applications. It allows -to set breakpoints in your Ray tasks and actors and when hitting the breakpoint you can -drop into a PDB session that you can then use to: - -- Inspect variables in that context -- Step within that task or actor -- Move up or down the stack - -.. warning:: - - The Ray Debugger is an experimental feature and is currently unstable. Interfaces are subject to change. - -Getting Started ---------------- - -.. note:: - - On Python 3.6, the ``breakpoint()`` function is not supported and you need to use - ``ray.util.pdb.set_trace()`` instead. - -Take the following example: - -.. code-block:: python - - import ray - ray.init() - - @ray.remote - def f(x): - breakpoint() - return x * x - - futures = [f.remote(i) for i in range(2)] - print(ray.get(futures)) - -Put the program into a file named ``debugging.py`` and execute it using: - -.. code-block:: bash - - python debugging.py - - -Each of the 2 executed tasks will drop into a breakpoint when the line -``breakpoint()`` is executed. You can attach to the debugger by running -the following command on the head node of the cluster: - -.. code-block:: bash - - ray debug - -The ``ray debug`` command will print an output like this: - -.. code-block:: text - - 2021-07-13 16:30:40,112 INFO scripts.py:216 -- Connecting to Ray instance at 192.168.2.61:6379. - 2021-07-13 16:30:40,112 INFO worker.py:740 -- Connecting to existing Ray cluster at address: 192.168.2.61:6379 - Active breakpoints: - index | timestamp | Ray task | filename:lineno - 0 | 2021-07-13 23:30:37 | ray::f() | debugging.py:6 - 1 | 2021-07-13 23:30:37 | ray::f() | debugging.py:6 - Enter breakpoint index or press enter to refresh: - - -You can now enter ``0`` and hit Enter to jump to the first breakpoint. You will be dropped into PDB -at the break point and can use the ``help`` to see the available actions. Run ``bt`` to see a backtrace -of the execution: - -.. code-block:: text - - (Pdb) bt - /home/ubuntu/ray/python/ray/workers/default_worker.py(170)() - -> ray.worker.global_worker.main_loop() - /home/ubuntu/ray/python/ray/worker.py(385)main_loop() - -> self.core_worker.run_task_loop() - > /home/ubuntu/tmp/debugging.py(7)f() - -> return x * x - -You can inspect the value of ``x`` with ``print(x)``. You can see the current source code with ``ll`` -and change stack frames with ``up`` and ``down``. For now let us continue the execution with ``c``. - -After the execution is continued, hit ``Control + D`` to get back to the list of break points. Select -the other break point and hit ``c`` again to continue the execution. - -The Ray program ``debugging.py`` now finished and should have printed ``[0, 1]``. Congratulations, you -have finished your first Ray debugging session! - -Running on a Cluster --------------------- - -The Ray debugger supports setting breakpoints inside of tasks and actors that are running across your -Ray cluster. In order to attach to these from the head node of the cluster using ``ray debug``, you'll -need to make sure to pass in the ``--ray-debugger-external`` flag to ``ray start`` when starting the -cluster (likely in your ``cluster.yaml`` file or k8s Ray cluster spec). - -Note that this flag will cause the workers to listen for PDB commands on an external-facing IP address, -so this should *only* be used if your cluster is behind a firewall. - -Debugger Commands ------------------ - -The Ray debugger supports the -`same commands as PDB -`_. - -Stepping between Ray tasks --------------------------- - -You can use the debugger to step between Ray tasks. Let's take the -following recursive function as an example: - -.. code-block:: python - - import ray - - ray.init() - - @ray.remote - def fact(n): - if n == 1: - return n - else: - n_ref = fact.remote(n - 1) - return n * ray.get(n_ref) - - @ray.remote - def compute(): - breakpoint() - result_ref = fact.remote(5) - result = ray.get(result_ref) - - ray.get(compute.remote()) - - -After running the program by executing the Python file and calling -``ray debug``, you can select the breakpoint by pressing ``0`` and -enter. This will result in the following output: - -.. code-block:: python - - Enter breakpoint index or press enter to refresh: 0 - > /home/ubuntu/tmp/stepping.py(16)() - -> result_ref = fact.remote(5) - (Pdb) - -You can jump into the call with the ``remote`` command in Ray's debugger. -Inside the function, print the value of `n` with ``p(n)``, resulting in -the following output: - -.. code-block:: python - - -> result_ref = fact.remote(5) - (Pdb) remote - *** Connection closed by remote host *** - Continuing pdb session in different process... - --Call-- - > /home/ubuntu/tmp/stepping.py(5)fact() - -> @ray.remote - (Pdb) ll - 5 -> @ray.remote - 6 def fact(n): - 7 if n == 1: - 8 return n - 9 else: - 10 n_ref = fact.remote(n - 1) - 11 return n * ray.get(n_ref) - (Pdb) p(n) - 5 - (Pdb) - -Now step into the next remote call again with -``remote`` and print `n`. You an now either continue recursing into -the function by calling ``remote`` a few more times, or you can jump -to the location where ``ray.get`` is called on the result by using the -``get`` debugger comand. Use ``get`` again to jump back to the original -call site and use ``p(result)`` to print the result: - -.. code-block:: python - - Enter breakpoint index or press enter to refresh: 0 - > /home/ubuntu/tmp/stepping.py(14)() - -> result_ref = fact.remote(5) - (Pdb) remote - *** Connection closed by remote host *** - Continuing pdb session in different process... - --Call-- - > /home/ubuntu/tmp/stepping.py(5)fact() - -> @ray.remote - (Pdb) p(n) - 5 - (Pdb) remote - *** Connection closed by remote host *** - Continuing pdb session in different process... - --Call-- - > /home/ubuntu/tmp/stepping.py(5)fact() - -> @ray.remote - (Pdb) p(n) - 4 - (Pdb) get - *** Connection closed by remote host *** - Continuing pdb session in different process... - --Return-- - > /home/ubuntu/tmp/stepping.py(5)fact()->120 - -> @ray.remote - (Pdb) get - *** Connection closed by remote host *** - Continuing pdb session in different process... - --Return-- - > /home/ubuntu/tmp/stepping.py(14)()->None - -> result_ref = fact.remote(5) - (Pdb) p(result) - 120 - (Pdb) - - -Post Mortem Debugging ---------------------- - -Often we do not know in advance where an error happens, so we cannot set a breakpoint. In these cases, -we can automatically drop into the debugger when an error occurs or an exception is thrown. This is called *post-mortem debugging*. - -We will show how this works using a Ray serve application. Copy the following code into a file called -``serve_debugging.py``: - -.. code-block:: python - - import time - - from sklearn.datasets import load_iris - from sklearn.ensemble import GradientBoostingClassifier - - import ray - from ray import serve - - serve.start() - - # Train model - iris_dataset = load_iris() - model = GradientBoostingClassifier() - model.fit(iris_dataset["data"], iris_dataset["target"]) - - # Define Ray Serve model, - @serve.deployment(route_prefix="/iris") - class BoostingModel: - def __init__(self): - self.model = model - self.label_list = iris_dataset["target_names"].tolist() - - await def __call__(self, starlette_request): - payload = await starlette_request.json()["vector"] - print(f"Worker: received request with data: {payload}") - - prediction = self.model.predict([payload])[0] - human_name = self.label_list[prediction] - return {"result": human_name} - - # Deploy model - serve.start() - BoostingModel.deploy() - - time.sleep(3600.0) - -Let's start the program with the post-mortem debugging activated (``RAY_PDB=1``): - -.. code-block:: bash - - RAY_PDB=1 python serve_debugging.py - -The flag ``RAY_PDB=1`` will have the effect that if an exception happens, Ray will -drop into the debugger instead of propagating it further. Let's see how this works! -First query the model with an invalid request using - -.. code-block:: bash - - python -c 'import requests; response = requests.get("http://localhost:8000/iris", json={"vector": [1.2, 1.0, 1.1, "a"]})' - -When the ``serve_debugging.py`` driver hits the breakpoint, it will tell you to run -``ray debug``. After we do that, we see an output like the following: - -.. code-block:: text - - Active breakpoints: - index | timestamp | Ray task | filename:lineno - 0 | 2021-07-13 23:49:14 | ray::RayServeWrappedReplica.handle_request() | /home/ubuntu/ray/python/ray/serve/backend_worker.py:249 - Traceback (most recent call last): - - File "/home/ubuntu/ray/python/ray/serve/backend_worker.py", line 242, in invoke_single - result = await method_to_call(*args, **kwargs) - - File "serve_debugging.py", line 24, in __call__ - prediction = self.model.predict([payload])[0] - - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/ensemble/_gb.py", line 1188, in predict - raw_predictions = self.decision_function(X) - - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/ensemble/_gb.py", line 1143, in decision_function - X = check_array(X, dtype=DTYPE, order="C", accept_sparse='csr') - - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/utils/validation.py", line 63, in inner_f - return f(*args, **kwargs) - - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/utils/validation.py", line 673, in check_array - array = np.asarray(array, order=order, dtype=dtype) - - File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/numpy/core/_asarray.py", line 83, in asarray - return array(a, dtype, copy=False, order=order) - - ValueError: could not convert string to float: 'a' - - Enter breakpoint index or press enter to refresh: - -We now press ``0`` and then Enter to enter the debugger. With ``ll`` we can see the context and with -``print(a)`` we an print the array that causes the problem. As we see, it contains a string (``'a'``) -instead of a number as the last element. - -In a similar manner as above, you can also debug Ray actors. Happy debugging! - -Debugging APIs --------------- - -See :ref:`package-ref-debugging-apis`. diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-failures.rst b/doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-failures.rst deleted file mode 100644 index 046c3e1bb2d20..0000000000000 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-failures.rst +++ /dev/null @@ -1,272 +0,0 @@ -.. _observability-troubleshoot-failures: - -Troubleshooting Failures -======================== - -What Kind of Failures Exist in Ray? ------------------------------------ - -Ray consists of 2 major APIs. ``.remote()`` to create a task/actor and :func:`ray.get ` to get the result. -Debugging Ray means identifying and fixing failures from remote processes that run functions and classes (task and actor) created by the ``.remote`` API. - -Ray APIs are future APIs (indeed, it is :ref:`possible to convert Ray object references to standard Python future APIs `), -and the error handling model is the same. When any remote tasks or actors fail, the returned object ref will contain an exception. -When you call ``get`` API to the object ref, it raises an exception. - -.. code-block:: python - - import ray - @ray.remote - def f(): - raise ValueError - - # Raises a ValueError. - ray.get(f.remote()) - -In Ray, there are 3 types of failures. See exception APIs for more details. - -- **Application failures**: This means the remote task/actor fails by the user code. In this case, ``get`` API will raise the :func:`RayTaskError ` which includes the exception raised from the remote process. -- **Intentional system failures**: This means Ray is failed, but the failure is intended. For example, when you call cancellation APIs like ``ray.cancel`` (for task) or ``ray.kill`` (for actors), the system fails remote tasks and actors, but it is intentional. -- **Unintended system failures**: This means the remote tasks and actors failed due to unexpected system failures such as processes crashing (for example, by out-of-memory error) or nodes failing. - - 1. `Linux Out of Memory killer `_ or :ref:`Ray OOM killer ` kills processes with high memory usages to avoid out-of-memory. - 2. The machine shuts down (e.g., spot instance termination) or a :term:`raylet ` is crashed (e.g., by an unexpected failure). - 3. System is highly overloaded or stressed (either machine or system components like Raylet or :term:`GCS `), which makes the system unstable and fail. - -Debugging Application Failures ------------------------------- - -Ray distributes users' code to multiple processes across many machines. Application failures mean bugs in users' code. -Ray provides a debugging experience that's similar to debugging a single-process Python program. - -print -~~~~~ - -``print`` debugging is one of the most common ways to debug Python programs. -:ref:`Ray's task and actor logs are printed to the Ray driver ` by default, -which allows you to simply use the ``print`` function to debug the application failures. - -Debugger -~~~~~~~~ - -Many Python developers use a debugger to debug Python programs, and `Python pdb `_) is one of the popular choices. -Ray has native integration to ``pdb``. You can simply add ``breakpoint()`` to actors and tasks code to enable ``pdb``. View :ref:`Ray Debugger ` for more details. - -.. _troubleshooting-out-of-memory: - -Debugging Out of Memory ------------------------ - -Before reading this section, it is recommended to understand Ray's :ref:`Memory Management ` model. - -- To find if your cluster has out-of-memory problems, view :ref:`How to Detect Out-of-Memory Errors? `. -- To find what leaks the memory, view :ref:`Find per Task and Actor Memory Usage `. -- If your head node has high memory usage, view :ref:`Head Node Out-of-Memory Error `. -- If your memory usage is high due to high parallelism, view :ref:`Reduce Parallelism `. -- If you want to profile per task and actor memory usage, view :ref:`Profile Task and Actor Memory Usage `. - -What's the Out-of-Memory Error? -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -Memory is a limited resource. When a process requests memory and the OS fails to allocate memory, the OS executes a routine to free up memory -by killing a process that has high memory usage (via SIGKILL) to avoid the OS becoming unstable. It is called `Linux Out of Memory killer `_. - -One of the common problems of the Linux out-of-memory killer is that processes are killed by a SIGKILL without Ray noticing it. -Since SIGKILL cannot be handled by processes, it makes Ray difficult to raise a proper error message -and take proper actions for fault tolerance. -To solve this problem, Ray built and enabled (from Ray 2.2) an application-level :ref:`memory monitor `, -which keeps monitoring the memory usage of the host and kills the Ray workers before the Linux out-of-memory killer kicks in. - -.. _troubleshooting-out-of-memory-how-to-detect: - -How to Detect Out-of-Memory Errors? -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -If tasks or actors are killed by the Linux out-of-memory killer, Ray worker processes are unable to catch and display an exact root cause -because SIGKILL cannot be handled by processes. If you call ``ray.get`` into the tasks and actors that were executed from the dead worker, -it will raise an exception with one of the following error messages (which indicates the worker is killed unexpectedly). - -.. code-block:: bash - - Worker exit type: UNEXPECTED_SY STEM_EXIT Worker exit detail: Worker unexpectedly exits with a connection error code 2. End of file. There are some potential root causes. (1) The process is killed by SIGKILL by OOM killer due to high memory usage. (2) ray stop --force is called. (3) The worker is crashed unexpectedly due to SIGSEGV or other unexpected errors. - -.. code-block:: bash - - Worker exit type: SYSTEM_ERROR Worker exit detail: The leased worker has unrecoverable failure. Worker is requested to be destroyed when it is returned. - -Also, you can use the `dmesg `_ CLI command to verify the processes are killed by the Linux out-of-memory killer. - -.. image:: ../images/dmsg.png - :align: center - -If the worker is killed by Ray's memory monitor, they are automatically retried (see the :ref:`link ` for the detail). -If tasks or actors cannot be retried, they raise an exception with -a much cleaner error message when you call ``ray.get`` to it. - -.. code-block:: bash - - ray.exceptions.OutOfMemoryError: Task was killed due to the node running low on memory. - - Task was killed due to the node running low on memory. - Memory on the node (IP: 10.0.62.231, ID: e5d953ef03e55e26f13973ea1b5a0fd0ecc729cd820bc89e4aa50451) where the task (task ID: 43534ce9375fa8e4cd0d0ec285d9974a6a95897401000000, name=allocate_memory, pid=11362, memory used=1.25GB) was running was 27.71GB / 28.80GB (0.962273), which exceeds the memory usage threshold of 0.95. Ray killed this worker (ID: 6f2ec5c8b0d5f5a66572859faf192d36743536c2e9702ea58084b037) because it was the most recently scheduled task; to see more information about memory usage on this node, use `ray logs raylet.out -ip 10.0.62.231`. To see the logs of the worker, use `ray logs worker-6f2ec5c8b0d5f5a66572859faf192d36743536c2e9702ea58084b037*out -ip 10.0.62.231.` - Top 10 memory users: - PID MEM(GB) COMMAND - 410728 8.47 510953 7.19 ray::allocate_memory - 610952 6.15 ray::allocate_memory - 711164 3.63 ray::allocate_memory - 811156 3.63 ray::allocate_memory - 911362 1.25 ray::allocate_memory - 107230 0.09 python test.py --num-tasks 2011327 0.08 /home/ray/anaconda3/bin/python /home/ray/anaconda3/lib/python3.9/site-packages/ray/dashboard/dashboa... - - Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. - -Ray memory monitor also periodically prints the aggregated out-of-memory killer summary to Ray drivers. - -.. code-block:: bash - - (raylet) [2023-04-09 07:23:59,445 E 395 395] (raylet) node_manager.cc:3049: 10 Workers (tasks / actors) killed due to memory pressure (OOM), 0 Workers crashed due to other reasons at node (ID: e5d953ef03e55e26f13973ea1b5a0fd0ecc729cd820bc89e4aa50451, IP: 10.0.62.231) over the last time period. To see more information about the Workers killed on this node, use `ray logs raylet.out -ip 10.0.62.231` - (raylet) - (raylet) Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. Consider provisioning more memory on this node or reducing task parallelism by requesting more CPUs per task. To adjust the kill threshold, set the environment variable `RAY_memory_usage_threshold` when starting Ray. To disable worker killing, set the environment variable `RAY_memory_monitor_refresh_ms` to zero. - -Ray Dashboard's :ref:`metrics page ` and :ref:`event page ` also provides the out-of-memory killer-specific events and metrics. - -.. image:: ../images/oom-metrics.png - :align: center - -.. image:: ../images/oom-events.png - :align: center - -.. _troubleshooting-out-of-memory-task-actor-mem-usage: - -Find per Task and Actor Memory Usage -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -If tasks or actors are failed by Out-of-memory errors, they are retried based on :ref:`retry policies `. -However, it is often more desirable to find the root causes of memory issues and fix them instead of relying on fault tolerance mechanisms. -This section explains how to debug out-of-memory errors in Ray. - -First, find the tasks and actors that use high memory usage. View the :ref:`per task and actor memory usage graph ` for more details. -The memory usage from the per component graph uses RSS - SHR. See the below for reasoning. - -Alternatively, you can also use the CLI command `htop `_. - -.. image:: ../images/htop.png - :align: center - -See the ``allocate_memory`` row. See two columns, RSS and SHR. - -SHR usage is typically the memory usage from the Ray object store. The Ray object store allocates 30% of host memory to the shared memory (``/dev/shm``, unless you specify ``--object-store-memory``). -If Ray workers access the object inside the object store using ``ray.get``, SHR usage increases. Since the Ray object store supports the :ref:`zero-copy ` -deserialization, several workers can access the same object without copying them to in-process memory. For example, if -8 workers access the same object inside the Ray object store, each process' ``SHR`` usage increases. However, they are not using 8 * SHR memory (there's only 1 copy in the shared memory). -Also note that Ray object store triggers :ref:`object spilling ` when the object usage goes beyond the limit, which means the memory usage from the shared memory won't exceed 30% -of the host memory. - -It means when there are out-of-memory issues from a host, it is due to RSS usage from each worker. We recommend to calculate per -process memory usage by RSS - SHR because SHR is for Ray object store as explained above. That said, the total memory usage is typically -``SHR (object store memory usage, 30% of memory) + sum(RSS - SHR from each ray proc) + sum(RSS - SHR from system components. e.g., raylet, GCS. Usually small)``. - -.. _troubleshooting-out-of-memory-head: - -Head Node Out-of-Memory Error -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -First, check the head node memory usage from the metrics page. Find the head node address from the cluster page. - -.. image:: ../images/head-node-addr.png - :align: center - -And then check the memory usage from the head node from the node memory usage view inside the Dashboard :ref:`metrics view `. - -.. image:: ../images/metrics-node-view.png - :align: center - -Ray head node has more memory-demanding system components such as GCS or the dashboard. -Also, driver runs from a head node by default. If the head node has the same memory capacity as worker nodes -and if you execute the same number of tasks and actors from a head node, it can easily have out-of-memory problems. -In this case, we recommend you not running any tasks and actors to the head node. This is achievable by specifying ``--num-cpus=0`` when starting a head node by ``ray start --head`` (if you use Kuberay, view ``). - -.. _troubleshooting-out-of-memory-reduce-parallelism: - -Reduce Parallelism -~~~~~~~~~~~~~~~~~~ - -High parallelism can trigger out-of-memory errors. For example, imagine -you have 8 training workers that perform the data preprocessing -> training. -If you load too much data into each worker, the total memory usage (``training worker mem usage * 8``) can exceed the -memory capacity. - -You can verify it by looking at the :ref:`per task and actor memory usage graph `. And the task metrics. - -First, see the memory usage of a ``allocate_memory`` task. It is total 18GB. -At the same time, you can verify 15 concurrent tasks running. - -.. image:: ../images/component-memory.png - :align: center - -.. image:: ../images/tasks-graph.png - :align: center - -It means each task uses about 18GB / 15 == 1.2 GB. To reduce the parallelism, - -- `Limit the max number of running tasks `_. -- increase the ``num_cpus`` options for :func:`ray.remote`. Modern hardware typically has 4GB of memory per CPU, so you can choose the CPU requirements accordingly. For this example, we specified 1 CPU per each ``allocate_memory`` task. If we double the CPU requirements, you can only run half of tasks (7) at the same time, meaning memory usage won't exceed 9GB. - -.. _troubleshooting-out-of-memory-profile: - -Profile Task and Actor Memory Usage -~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - -It is also possible tasks and actors use more memory than you expect. For example, actors or tasks can have a memory leak or have unnecessary copies. - -View :ref:`Memory Profiling Ray Tasks and Actors ` to learn how to memory profile individual actors and tasks. - - -Running out of file descriptors (``Too may open files``) --------------------------------------------------------- - -In a Ray cluster, arbitrary two system components can communicate with each other and make 1 or more connections. -For example, some workers may need to communicate with GCS to schedule actors (worker <-> GCS connection). -Your driver can invoke actor methods (worker <-> worker connection). - -Ray can support 1000s of raylets and 10000s of worker processes. When a Ray cluster gets larger, -each component can have an increasing number of network connections which requires file descriptors. - -Linux typically limits the default file descriptors per process to 1024. When there are -more than 1024 connections to the component, it can raise error messages below. - -.. code-block:: bash - - Too may open files - -It is especially common for the head node GCS process because it is a centralized -component that many other components in Ray communicate with. When you see this error message, -we recommend you adjust the max file descriptors limit per process via the ``ulimit`` command. - -We recommend you apply ``ulimit -n 65536`` to your host configuration. However, you can also selectively apply it for -Ray components (view below example). Normally, each worker has 2~3 connections to GCS. Each raylet has 1~2 connections to GCS. -65536 file descriptors can handle 10000~15000 of workers and 1000~2000 of nodes. -If you have more workers, you should consider using a higher number than 65536. - -.. code-block:: bash - - # Start head node components with higher ulimit. - ulimit -n 65536 ray start --head - - # Start worker node components with higher ulimit. - ulimit -n 65536 ray start --address - - # Start a Ray driver with higher ulimit. - ulimit -n 65536 - -If that fails, double-check that the hard limit is sufficiently large by running ``ulimit -Hn``. -If it is too small, you can increase the hard limit as follows (these instructions work on EC2). - -* Increase the hard ulimit for open file descriptors system-wide by running - the following. - - .. code-block:: bash - - sudo bash -c "echo $USER hard nofile 65536 >> /etc/security/limits.conf" - -* Logout and log back in. diff --git a/doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-hangs.rst b/doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-hangs.rst deleted file mode 100644 index 0725e8863bb16..0000000000000 --- a/doc/source/ray-observability/user-guides/troubleshoot-apps/troubleshoot-hangs.rst +++ /dev/null @@ -1,16 +0,0 @@ -.. _observability-troubleshoot-hangs: - -Troubleshooting Hangs -===================== - -Observing Ray Work ------------------- - -You can run ``ray stack`` to dump the stack traces of all Ray workers on -the current node. This requires ``py-spy`` to be installed. See the `Troubleshooting page `_ for more details. - -This document discusses some common problems that people run into when using Ray -as well as some known problems. If you encounter other problems, please -`let us know`_. - -.. _`let us know`: https://github.com/ray-project/ray/issues diff --git a/doc/source/rllib/rllib-training.rst b/doc/source/rllib/rllib-training.rst index 7b0529e54320f..bcbd9a82ff188 100644 --- a/doc/source/rllib/rllib-training.rst +++ b/doc/source/rllib/rllib-training.rst @@ -608,4 +608,4 @@ hangs or performance issues. Next Steps ---------- -- To check how your application is doing, you can use the :ref:`Ray dashboard`. \ No newline at end of file +- To check how your application is doing, you can use the :ref:`Ray dashboard `. \ No newline at end of file diff --git a/doc/source/train/getting-started.rst b/doc/source/train/getting-started.rst index a7105abed9db8..5b357df754323 100644 --- a/doc/source/train/getting-started.rst +++ b/doc/source/train/getting-started.rst @@ -190,4 +190,4 @@ Here are examples for some of the commonly used trainers: Next Steps ---------- -* To check how your application is doing, you can use the :ref:`Ray dashboard`. +* To check how your application is doing, you can use the :ref:`Ray dashboard `. diff --git a/doc/source/tune/getting-started.rst b/doc/source/tune/getting-started.rst index ccf805d833269..b3a75a09b2db2 100644 --- a/doc/source/tune/getting-started.rst +++ b/doc/source/tune/getting-started.rst @@ -164,4 +164,4 @@ Next Steps * Check out the :ref:`Tune tutorials ` for guides on using Tune with your preferred machine learning library. * Browse our :ref:`gallery of examples ` to see how to use Tune with PyTorch, XGBoost, Tensorflow, etc. * `Let us know `__ if you ran into issues or have any questions by opening an issue on our Github. -* To check how your application is doing, you can use the :ref:`Ray dashboard`. +* To check how your application is doing, you can use the :ref:`Ray dashboard `. From b32a6b858417040409677910ac84978399e3d5e2 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Mon, 22 May 2023 10:28:38 -0700 Subject: [PATCH 28/29] fixing links Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/cluster/images/graphs.png | Bin 0 -> 150078 bytes doc/source/ray-core/api/index.rst | 4 +- .../user-guides/debug-apps/debug-failures.rst | 272 +++++++++++++++ .../user-guides/debug-apps/debug-hangs.rst | 16 + .../user-guides/debug-apps/debug-memory.rst | 56 +++ .../debug-apps/general-troubleshoot.rst | 217 ++++++++++++ .../user-guides/debug-apps/index.md | 11 + .../debug-apps/optimize-performance.rst | 325 ++++++++++++++++++ .../user-guides/debug-apps/ray-debugging.rst | 325 ++++++++++++++++++ 9 files changed, 1224 insertions(+), 2 deletions(-) create mode 100644 doc/source/cluster/images/graphs.png create mode 100644 doc/source/ray-observability/user-guides/debug-apps/debug-failures.rst create mode 100644 doc/source/ray-observability/user-guides/debug-apps/debug-hangs.rst create mode 100644 doc/source/ray-observability/user-guides/debug-apps/debug-memory.rst create mode 100644 doc/source/ray-observability/user-guides/debug-apps/general-troubleshoot.rst create mode 100644 doc/source/ray-observability/user-guides/debug-apps/index.md create mode 100644 doc/source/ray-observability/user-guides/debug-apps/optimize-performance.rst create mode 100644 doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst diff --git a/doc/source/cluster/images/graphs.png b/doc/source/cluster/images/graphs.png new file mode 100644 index 0000000000000000000000000000000000000000..2cd41f5b9b2784ad4ba0e44f9d93ad1e1aa4b2c9 GIT binary patch literal 150078 zcmcF~1y@^Lw>7jBFW%x%+$j=Vi(3i=x8m;ZR;+k&D^R>Z305q)yHngD1b5e)=Y8J$ z-Cyu!oH5QgezNyIYt1>=+>t7uW#6Kcpd%n4yp@-e0wEy0MnOP8%tS?o@6i<#dw{Qy zTqWeyQQ<#7RI_jJZDKcRZ8tSX3pY<=7jpzl2S9{Wx6l{BAzyzkYMlv&?o;>RtAQ5bT>c2kV<49DShvO+!9oRQ_WPuEbAL=}ks~wj0>`I{=80E9T_+iV7-OKq?6O3WGERoITJx z8((MbG3(gIz4#H$gu9?P=7r10?}xR_nh4?^*}@^3mF9xj6+gn5@~WZQ~&wl7-nRw95h z+loz5DnO1jQ%;*3u|Y$lYQ^BilxE+kw;G!b&q07_a(c-gY5i%n^i=&!W$ip-wq%q!dKl@r*}$7>rGfiAhiD0X;(&0f851I679^K4_Yo@^ebVB z1R32dnwT#XJ-GMeb}vsy(PHY-lE9drbKxclTyHI(%I!tq#W3jrbJY0;zU^CC2~^?n zW|mTctQ1e46G^#I9mg|F38ytbwlv-<-M zs!2L>5h&6m&uOkwFLy2efWHwOVXkxgDEd9oq*|)U&)0n*9-WD<(+D-NqaA81JV%o| zZtTdl*I^Mu3c2_-lIpZRVeE0VfTHf;kUQS5_0s+_0?U1%h8S=9uy-1L8$9D|JL5C# z+heG3e-Zr>i$Ga`yNd4f|M?K45o|7EY_ZG>VU6aJ2SU%E zq|bTYwB#6~BU_@XtLZo`ijDz}uU-^tn%vRq#>UgUM{RhQ44eCNZ2W!amOC!%s>mYRp1W5n&jpVL zT6eRnW0itvNFks04@f!qkl8c?dO)QFkty%=Js(_C^3KUgF5*;^G(LOBSSR zZOq{soXns4%(Yb<<5QcP2F;fBS=O0{tIdu__lv*Jb>%e_Xz?(yzqD1Xdr!0cO`;G& z_P|4r$jHbsGd9k74mOyXnRVWNMj0%ej={nE=67B`B$vz`lRfA>cPwESD4_T&$`-W9pCa!Zy|0Z??sPgK-erv zv=P6;T8^*RxmG|}a2&Gm$^Ch2Ea_iYwres`30u?j1PqwAT;b{ZqN(xIQPxXrvih9+ z$5*9*Q}A>+36S=DLW-`Rdh@ z+4^u@>v}_>tGmdUHr!KQworS5Amn+Rmse`N#Ns7^3h}+)rJN5NhZTGuS7X&gj`4K! zsCI?rD9m@0@SW%K{kxqnys z{auOH;kF@DXJG+>^l<&@&zvj~l~teyTR^a(nvJa(6M8K8B$y1hKMp=oqZv5)~Y= zQJq*=kr${Nn3=(hduKW~_QfSyZDj1@hMn^uMr~BY3F-QFgJGk2WoMr+Al4)xWiH@S zV{A0S>Tr25B%W+*o(7wX_xde*7p>;BxTO|0u9%W?43|J|ezPYf%FPi~G2N+)jsg_V zhvT!O<42(TUxLI7eF_TN6eg9WwzN1Oe$O4eqC}v@mvhBT7ze6hrOUf(QtZ=?w982VutOIur?-#zE3?9i@rey z*+<)nwJo04)V(z!6KIAsS1roNWbZ|;%T_K45V zCYf6FwaFsb_N2^X5HLP^ShanYgs7~VD+b6%TNI;ROkB$oJLgh6J$GqV(~kS=c_2<<0@4*?8Pa)qyQ}Z(;anFc-GP3DJlyrEJs%sM+ zd(#Zo_V%W-FP{Ykn~6wT)OA=k_HSNys{CjR4$x#^ugNr!YTxJETC5IA$+)i^lbyq) zv?sSW)3neALQf~Y=g7J$i&b2y4w*;D0R~PNJTv4|7yQ>Llo(VOq9|yK0yCX8W@g<^< z-a?&Anh)M3!MVv4Lie6jp?yV)gmS-Gx4bDNtN;LBm(x=bT73EN2!}^Zv0n-rpFjp0 zKs+K6$jIZe9o-fkPGMa|c217yR4GBJo6UjzD+isiI+#^!gH7`tD`l@qa9-zdmV@Cm zV1J*)%Fd>bBW6hMd49{WG3q62r^bO*=lLm{8!!6t7K-y5z8yshz<>j#9}uD@j7w}^>vtlD1?7c;+XHjlsjuu`MU z_1~!t{ekIST5EX{@R{<3*fEEsxX$xqJh?AUWqK#3ED_Os=Xl`bYlQ@?9KRi6n@4v1 zr0!%z-iPMvs)B;xOPkEHaB?Zr&ZXxT2YMV-6+JCws@wu~_UYNx!R~m`PqZY)*3QBA z_sFcdCp2sM`HVxdx!?nJ4GsL*UT1>{$i0uQ?vl24y6gvD=8Z5vijN|awjYKka}eUM z!sS!Rxu%T0ntE(jo2McD8a*;PNlExA;R5~U#sR@@g8w9u+H%tI3vK=Mwfa>xIE>M7 ze_Lx;nADT}8fT0o6y<~OrZR*# zwBMa+W4=x6nL^*#r}sEok+os(xx0YL!w!ddg|mw8h&89_;)JhhN8KfV&nE`2B7b>) zZdO_AmVEHw;^$W?-lhV?TK&$+EJFa7P%B4q(*v5YR#7PfX;t47HX z^h``S%*LeEA?2_j9H32O+E=w%TC%bc>{@-s`rL9USv*DGpW_qN{gCvyi^4#Z`-93v z86F1qZrs zD2?XgqV=_}aVyai$OHznDqF%A{DMTDT4P#oNKSG{jn^bqeHQaQPcsZyh%nDt#)TCn z)wtvAP16*SB#nolTxsk-U=tfO1krEc0qLf==~q-fV?#H9O%MXvTbdkyJY-GP$e3Q? zVzW8e|DF?Jt-F2h?GrLB?BCv(Lr-qlvojwM{d~+Hc&8ghh=MVHe62oPl+4OLk|bX; z33xkx!AO8{`xJmhCaeSpYlA)LP~c8jU^@UOhK&^8F#;Y0%E~f5*91P7Y8MuiXWfT~$HIy((=&3m2mEExA_~m7 z_VgUpHT!OO06c2%(2TSZ%LAFF0>nfIh*7eI+315My)v~ z15fSm%|Uy@&qgAtY34o_#K3UQugz$g$a^4SgXvQ1eG6sJACZGP`F~^6xn!wRXviw4 zH@UUP6gL0U(l=UIFJy%gakjOC!ywKWPAu zP|yLAc^u#aRpxhl$9r_^?RaF{7;h{}7#5me5Q?oGhTUtyTwZ)~dEYIqoY0o)D>GQd zQG>VmN5;qh!5jmpI9Q&%lkMZMlG^fUIY1PHh(sO^jijV!5lZL8+x+VJ9T5}>8iIhp zXkv#2*D44^cd#8D`Aj$sT0|Qr5q!6xvrkGo35fM$AD-Im0`h*W_6zXwr06o#Dg?K^ zsrb{e766mhoYIjPL?S=(ZG^QI%;7S^KK&3Rt%>1Uk2-y@Kx$Ny7z zxN-Uf1OOU9|ZZ$8$oyO#sv+Vd*+bCVflJht!0hbBuf}a?nm*?F2q? zY}8dkKn^L>{mNC0Y;o;(f`+$`C40LL#{JL!tZ)6ZT_sJ4g3ZT%v<(D*6p>OC?ons{ zltF>D?3)OM?^|2^9nf1W;VrCTv9Bmq67Z~RZ7WFUVqv= zV+NI%lfD(-kP9_vzQMza$jHi;o_1p$rXfqzrXpqA-e@rBn(DT6czLu-HCHt-2!Ym$ zi&v}ACTQn$Y8$R470uvWC ze}*Rpe7D2j`Hba z`%NvNT}}grOG3J@{XJ2IUk3OywzNp+YaT~HE?i`2%<t^{EHAURw)(qWUDh;K zb-<+qau{^UgVNKw-;=)S!r3X|B#v~?Ac z)7I7R05HRL<95Q56O{xzJvb{Z{!w-i$S|L}KaWlpd-oolX2{+Q`B#RO(pNWAz5-FK#qnpr9m| zfw&!4z@n?cJr-H7LM#`cF7yTwq8N6}fLBVzSVI8PQCcZXm#+l~DUXHpoMXBsg%n~O z_w`a2(W<@uiUR;i7Qj%|e}HnM#J|K4_i&L&a16^ZU`+`Rb~Rxsb>J@NW*xiu}W zm}sk@fcCebZkbZ(gm!w%KzXJ09{``~+Onj7MefWpLy1%HflQk;II%K4yue*D2y?l-yx!o)!nnAAs{?Swh?`q& zNpp}TC2HP0OIvuXtBWitX=`M3RE3pb;!0s1&JH=1lJs|W_DDfKUf^gNq)G8_Rsboc zG+N7p8C^V};BQtomD3C)YK71UP#O$=y!EteVrseLYaGm`VU`C6Z z=Zg#pqF;DK73(S4&~Y7$icmdJG#4(OtS9@MO?eOXa#0r=szqLsWr6a?wtM*R5*5|_$tszfdD;0fv!@x#Jb0nQ)yaa8cB2e-%}O~{~$f;*drEn90_v6jbTmX)qfesB{;aCC7IXj^8ByA5awzJZyZENn>#Uzv zRMcxv3R~yLN`i;{t4j_szTtfv7yHe)A?k`^>l!#C-GNy!zQ?f2(qNS~r|$@_cep0e z@PZ11H~I-tjd$_%FifI^!>x^xC2xV*;v6v_qKzVJwk9W*2RA{Pf9=pv;;n$(D;0ID zbgoi<6c>A}VM?hfExpga*q;hBW;QVw5?eVJ2Kw5sRu;8AZid!Oj@pMPwS36Q8kF+X zi|g~US2Wo)COL*BjMAciqFC=)nVXyEm;g!PRR&q@chq=T8;v+oQqE8B8}tmW zgyU0E%o7KDWP)5u;2Op0xwOxNhY<;V68=`Q!kbr}I3le=R`&L=qbSnby74AV%Wll-Y$&~ok+^{3Zn?;v9jvK9|;qx01 z3o1=Cp*%%_ORR86e11MQq`b8p$wzzxI*5S7$yToZE|a*{8?u77gAdP{?3Y0uGjHiXX4Bqne-V8l zZI`6ZvXqq-kzh6SBukVsN%%^*9%&gEySdidVaC;bCi2soKGs0U>zL(*s6Pgzhi`g6 zwL~W7obwkH+S?28MLynYlH;qs|4jYqwZRGnCF- z9RovBXL((kSmCl({q^VTBQbGPQ*Ot$XXfk`5wfNiM*J;P0*o@0PpZj63<-@n5nKEz z=x4g%nuA9EI5le>4`yrJfOg8E+dHWldz7zAX(nXHpubCd1m| z#}XY@t`bF(;NTOA-Lf;x3^x^G9T|s9i386e5pabiVS_3&GxHa?^fgmPR@Tb(!iG$E z_%|sNGC{wOu<=~FY~6~>&4}*H>nr?78G*C0B~rDaFF;gzs$2^@uk;{#mWruN8Ns@) zfuFL%o3cY1nnOCNN%EJ8io#^nN%9Hu>4iJ{G zmFPz=J?Sw>&pqG!YMPrpZ};j)JBZU5^9;1Ve@wfZ_4U80kj(cc&wW;iF$)KEkyH|K6qT*6$7&jYBkTzig1 zn5wo9k3iYm-!u2Nj+8R-)+4<{4--z>GIevm;iz&Yg!j_+xqw?M7lJTeEIc)$BE6a$ z{oroNJl*IxJ?2QAJMMbFEtFmy-+i;JNM&f`}f7d`QuSswoGiqEPsS&enUt<3{ltG{ifl%(Um@gOShR z4@d+xXrJ5eUp0kcX(lPQ+)c&HjlIPIh_D+#29_Lhng+cAZ$X3aQZ4$4NGTXHhU^8G z#eEXvo&(;0%rmdwyb9qKsy{u6~7vUF>4r{*v!nH=u(C*99Y5jn?4;k!L5U=pcm%npX z_N9Gn=ETI$%q96ufEBaTLkzz_rf7P%)Dre{zQDUQsTg58*idK%R3Zn1!SK^ZCDjp_ z(+@iK+#E|gx0s?7RuF;r6EwFSy7U_7_ocJC;K9e z;Vee@<#ff7yD70Vyh0>mm(nJA$c^-i)}eY-GX3sRLyZ=H!MOx8R_!zISkSCh9fZ!WAOp9`eh8MMUP z&FpZuvnx4|<`v@Usxkk1Sv3remC+n`m1q5&@CC8|O0I%6(cleXT1pXdqQs_8q8)Dt ztcf;>NqD^T0*RYDrC->^Oe^gd5~B&*-$7G9Go0X_dU6Z;eh#+R!j)XNnhJfg7wO@_ zFabDpzQHB6W9fHJ&m?20u94!$>#bo$4|}zs1ike*TJ7%uufUbbgC$W$?|^{!OYGCj zQ$u4bzS_CAA*DM(#fHHi`Fsw;K(HaB;o0a9jx*#bI}I?RAEoc$jvhM^Skta z8~Q%jNDd4Oue>q$&XKj$B7}>*DJk*NbPX>n4e_;}FNpB8{FNnPE$qUQk9?apw1W_J zGZ};bbCHnrv^yL*+Y(kARIyTqvsWEm-FbClN;Ta~S~fPcN2LXxihTY-x&$a0xS$<1 z5IS8dS7Q6U-ZtJjkXek6`eyQ%KC4HQRH^#TzMPa5Nh-8|;tdpJZNF#zb{Ot`NOM^L zEAjl^gL@bNR7?c+N^1Q~X)LAUkT9|J*W-EWski>$N8fQ)Afz5Auq5M17-U9Qf$-bf+A zefr>^9G&7)IDr=7w_WQU>acNRO`yP@7ev|54q{BLOb!bdBbuMjIr>7ZVyXK+)!JkU zs$i3hr1jN$*9#om>hV^N>h$h0J+q_U9$?VVp61-pGz?UbS99gg^}UXU$28CYl>d2_ zwB>A(#fAzXt?CeMtrJSE8kO~X7ScvEdp|dLPj;Tw6!7_>f(YLIP1`kEQ^d^eMTmZ) zV$i7_YsAdrjhlx`-N%E)1c$gI73Oz*K>P0?Jk$tz2K+gtrQIg>t5LRwkrhG-BO>HU z?H_Wq64lyf*3QPJD9P3-=kgNNB8pX(?UTR>NY8U zqMp-F8lq&y8ahtH)kj%h9u87wSxOz>1si?O+}E|RENSp-FOIZ5PH}G&M}QafO0}2I zN$@0aP?5g%ZEB~6C95!Q%hBVh&&?)$0Xh`I^xNC3%a}of+x6F0a!h9^uLv-_qwXGg zY_-+@&KNDFd>P7WZ3lyMCKjJ+>lIKGqoWDX@7UHFp0g7^S31OeUjgt34H3p#XU?&% z!bdD8@Iqm@9HH6H(Uu}*1N)+*K>6ZyvOA{bfji)Ss+_e%QE#lX9PS}IsZkYe!|eVE zZVN6hwy0UaoK{QPb`jv&5m!xKa;aTkcW9hmM*y>ix)-p#mDg)M2|+^9Ior7*Yuk& zIP!BA9wwpQ%(WAKeqt2Rf^7w@qI|OmI`}&*SXfXoHq3yBTKVyOzsi!r9NKX zfKuH4jbB3@nB^S6;h%yV-NdeZ>6ZmDLBi!lWzzFAwHl7G#3O<~x! zKW&RXS4kI1+Ib2=|3X)3dtvP44?q?z&ok{5BM*i(tfk@VW0VU9 zW#qDF(+cregWloCpEHIN)y%o9l#ve}&3@l*LicOc{?lKo1aDOSZD4pCGuo;<{rqPQ zU(kR4`}BWIinG7g#s69VvxvVz`S*mV{NJkocV!SdoPqv(j!wb zD-oFa{~a!>1m>glNGO@$0_)8(S)@!bO;4(0Q#cMu>Ue}XNl=TscV@k`cZ1IQ74$$b zAP_;ax%OsFH6t^w*uechOnlBAd>Gua&-9i`e458EKn%hz{_S9_^TFACDYN)&gS_rP ze zPFc$CQTEMUTkCtGK0ZFD=mJsouAbPs|F-Vkz(a=! z3dnXl_c{6{GVr=TP|kEl0zLaaNAhu!@&yh1vTjQOA=p|sy?F3%?1}^`=JW^6>5kSL zv93=4*V?+!D~R#$)@O)KrRFeU-~smCRHd*KU|UuJb@t}W@X0f<3m?TZ?|=l4LkVNJ z2As?QU>%m2b61-FZKp8J`9!t9c<=ofS+ZweCOZvu1C}3#IQ%?V?0Y7H1VH^$!>T8z zK5{7GZBg|e{UxA1!z3hzfW$l}tx?;5_}o>sfKrT;sK@y5m7?6Z#TgZ`c~ z`~{~0IQcGZ%^qR{j(&HpGjn~YGj8Yc&+F|_6+;$?$C0<5*wMe9T>xCz1jH=uE&q2M zi-yPOwTe#z;Nln$~NOH8h$QxEMLFUwSJqJs>2R8kVd#vLz++3nz6lJhBU(Z@d%~y6>$+QUT z{o|7Q`l$U`dL%p+oV=1iK5D+<9+RAFFvX=y;^dyWZZo7xf!KmDai-2)#ZWCQNtu(A zSEg)tN~j9I-F%%}IQ-+6{B>QoOE-M%2OWcS3p>BWuSc&^AnuH0bekAdW<_az9R5u!s9`665(mI9nA7gVVyh=w{UP+F*ry*&ku@OVb?P^2Ypl@3L_e}-ah>sveUs=M9 zvRA>W7Ob^XWgqK4j6dE74uUPjq(4fTj%aTZvg*UsDvh7XBpVOrnqfn5$bM7gTH{ax@UTGX|N=$8&#%f7R1VCG6# z_{YVLXG#Ct*Tz1`aU9u#@`_q0*y_NBt6*fSf8_rG2 z-I&&ka?7M}eBl1{pBX8%{xTh6!7(9Ad^b3cjG3)=e4$_;(J!nh$jX#U7cY!6e0dVf zubh{?lcCiOV$} zgq3AGwc`Q7-Ll79rEr>DBK9C4v2j`#5=f_%?U%#H4!x^edvdp-`GnwR_-(Zt+to?y zNBdLlOKkgd)=7S^=jnWP|H5QLw1NLs%Wdn+$XxrANb!4++#YAu&4-)gwcf*ZrY`|c zNY&RXhpUs)11&Zsg@vszsN(^5e0M`9kL-qzk0qsX_qah8VQ1a~aW9YF&l`8SYD=E! z5!g4u#Zi_GxqK*y5+KKjk^hCcC-2f`1|)~7Cp-r?Dl@{If_na7(afce`uE} z;4y^IX=_FJ`BoKkYo|od2dgQ=^>MTzV8q5HI4$?ZHM~TN6NvB6R)oe}DhhFcYmF=y zA&)B!&oS-yS&rQNyq#g(j;=e)(uEd@IzP6r2m?DxhifkneGxL6I%;AAIhb+i%sLpV}V`btWNq*@nJ86|Autu9c^OFNQA()dKMm$4l{U ztvBv@O#gmjl>U520yq*JUVE=fboIOsBTL|vY3;esu(q+PDnawlnDe>cMfnT%%SHPg zrT6{@KjizZ5+(VlJ%LsUzjwelFZ_|ul~aFbfIt_-3TNnwmr@^j~FB@FJYD z?+t<`L|il}&n%7Qovo#5zjg+s$M$2pG?yCf^31q+pcvjn)nJNGYxwXSDhCj)DGe9Kv3pM zZJL+j$u9`u?5_7kBr)q{<|mHhu~>1 zhse@u&i85WwKH{!nz=(sy#7T~=Yy`poG75Kal=O@C@9j5YY-X@b410QHEa;}`AYe? z&EI|lW$37L){ih_jwIgwSyJ$(jO3dJ$^bYnX|&Ke_Jc+pFnP#XqVogOPi@BUpV_(G zj*(8jkT86CXYON`tR(MpbtoTsr~0|o^+&H+oL7S)SS6!~akVY}qu32#?GF1TlbhQ( zPUPV-r6E$GED7das5w2OF{i5bj*cR_;2?-KK|g~^4Xe7KBx$SPkV+cg(Cb%x*kr94 zIy@WSqPDt!&22k<1r^5-s<4H*;YrYBeO6_CPtlc*$n7UGKJF8zkKh{bH9HIF{u>qAk$AQ}`;F6aIJ7^RV)#Pkm* zVtv6Pqb>Jc@#)HUyJc&UIP3C+d3rsjs-PkkwUXi_47u9uoH&bOt>HiQb5$)IK7XAf zg>slR{|kz|8qJ+lh+_`V_Qwrn5H=ygu931$`d!X&_h`#=(uvdUK)`i95+gy4zTgA9 zqAc=lSEYYg3?^=E4ZKKn5W|kX1G<4*l3?0OpZtO8+g1I$r#K?7eXWj`-XzF$efnhK zOtM!IyDYsq&zpJhbz%ZQUKnA(zST(Kz|`F}h0bOm#JO1ZJuajqBPI0~M68Vd!z6** zv9;G#`EVzrL@ToHs$h^SClu~;UWjxSguJm_4S3!9#F*h0UUVmrsu^9VrRFs{dU+%u z?#HEwN7z z3EM}crV84n&$}zzLzB-HGUW98i+>l3X#5BH-$_9#+_7f$V)<+Zt|zRmzH#|%Acf!$ z&-Vg`ONDEG>~sxo#V9FNB@U=+tbN-t6S&$@gck=gQ`9r<=w&s4B3&$;V%%eJ=Zocd zj;XUim>2;a0U(2va(k+Xit7#!HCU>ojCo}05{Cx=BAJ{pFRO=K1c2jC}ZX zfY(Jcn5MPSO0i$ethnS^Y583@`6GWTni#*gmibj#L7$dxYGdOB?}V*46)j%3rJI>;eZ z=EHhF)4OT|!qCqgL>)};Ixpdw7PLS|aH%!`J1rD{1##HCA?B|SnQ&irL0d{m;lXf= z1yz5Wc&pUi;c9Q!A6Dnj4R8{k2dobJFD=BwTzQQUt%DJQnTG1HsJVg1viCQVq^4CMXXdry|^_fwELx zxoB4bZ@FcD2`j5ik54WiLVxukynf2x7I!()5CJK$4)-{dy+eUVF3k^AfF9{?RIu=` z_53Nr>9<`Q9DO`7lD}w?c~f<|Tvy`#+Wyf~LQuhXHfyJEP~U`D&8f)$Nhi|QB;G4! z|3__MnS@4N*pfN3M$+pow77F$4GzfGQ>f#GoWFTWgFhoVD=m3qYH~FE*1M? zR>Cwl9__=ndcQ2fwJ(b~uuY$oXO}q$w$@SusVGZy@<6Y@wue60vmNZV41V;(9>2hm zke!|(qJm1m=WE$TgC_f~gty;)#$}?r2ATZS#dJWP)7|+pY%8|8kQPf!*b{8ycgyeG zk;FO*_Nwgt1@Q10y{lNug(R6%`RTDG1hC#P01Jw>rTbI@)2^oasC4G}d(e1SY+zjLMI z>63e6&Ao>N4ps!(oxdEAfh5uh^%Dm{=S1lk{l(+SL5dP>V3KO`vOM2CI{huAkuZONNZy&m>(GDwrMr=gP6WMM2{y41(Be_H-aY~G21 zbJ0vNG}h1FT!<9NAty{BEl0R?77$@9PK}F7D>02GjgRCnh$nU?g(MuWLK_8^XI*+4 zc+!nx3c3`g9@Kr68L{Ped7?_q(S_!5+>$mmLiDCz&3T=-N}H+KZ|a+^B37X+$Ewm? z8S+k_tURH@Fbi)Q7chlr*)P#R@3vO#4mPtiO-7v46IplR_-*?0!$tR<9jsQG zq4VQH--jaZb?g!8-QrwsZw9fEmuUQ{zW%s0b*c?nVKObL(89%b?V-05;S*^?bzOVJ zR|UvLq@jHNg=<8luLJkA9ODplY(EaUVlBVzip6GbZXRbRq&5` zK?RD$lr^3cYR@H>Qbf31Ht!f z{x_$$N%0@fYh&WagTNZLljc_!5HIxr@|=V9rY}W2z#y2WXf4d=r!{O%Qqx9vyy3f7 zBUrMCA-Bhsd?Mseg-(CpF)zrKnAdJ?aB?57 zOf=;@oUD!vWF!%=X}t+w>Zzke$$?GHFUlKm4fLEcN>4-9O;qTR`55}Sybf>LdbLDf zI9$75kBp2?w!?1jo){WVkDDpieb&FI9BZ_>&bi}2KE1S#pQgYMbRL|nTlv)KClLNl z*zm3@v!ogKgL)sYwxx_N9p_tW&|Ih$T-y16|LDJ0Znt}o=Qy2hw^oxA7AenrE=+enPZLIA`_-tEpH_RHjAGE8Rz6w44UoQaNtT#w=_}K-j{bBvwL@?GvK?)H{yLf&S(lldK($K?*6 z%@gLDZlzukPy>6uVu`)-u#v4yog#`9JCYJR>45x8GctGBXSEqkXA)M5d zVf|i#Vr8Y=dT(){tX>9`ZM(2<9RpO_EH{ug9WYtHL@noC2@w$LkdQC1Jo2&04?tEX zH?}g1dMk9yu;GaoAxY*1dn*x3y1O*Sk5{k?!<0?!Z(a!0mTllW5_c7q{>)al3eEa795zf!P4X?89@nYm z%xUF*#gwai#o{d$oz8 z!gG3hU#PKr_XQnWPC15Ph@Ovn!`0^UEYMoKRJ8D%JsIU#D=R9^FHEqKH9l$&q0?7E zanii(30pCV?z)R^f~S(iy4Hyv%kvnIn6`J^IT#%Gv_! z(UHhdcqekh7HL2NOz7xzshy~?gtGBK|1254DBk!wU3#=CiFqs$JJ@+Amai%y?Tb(k z6f3Dx_{m)9am4_yZKaUQB!<JQY6-kFe`iOLiOaW^OW)L%J|n^%K@aug*r{8$kTKj zA{?-pZQ3)$xLR#AiXtPMC9!AYk=6Y>7df*M*&U6N;RsON90%rf$^ z5heTV-USsTN`xS%In7E64kn%6SJn5unpPRU{--*k&S;7<(TdJU{BWq6#{J*_?&UyTos_wfdE@aeuGrL=lz z+b33HY+o ze|!ApCzj=+4-$-;5-xB1CVKDEU#AsxV>cV6nmc%ZjW8h8WE);M$6Z$o%3a5b3}Hoc znk-ObnjiMVSm;{C_y5X3C1a&u9r3curR?wl&YuIuSDv<&*%~!5D0m`pSjcvX@!d;N z@ap~yjnp?zjpBk*p!R2>W)Bzk#5>f%$r_hwAzMD;7h}zjB${C^_m{z9wab3UJ?Nyl zEj~;aqpXh~UV7MRcP`&w$nX2FedDy%86pYDdR-T6NxIy=zhNK+9%WZ@@jO<@-^_|H;ZZd6rWho3WwMdPoQq|-j^f+@IuFZ_~Pmg z%+q)`LSZpEcRFq-|>jnL~*}yO)oAVJFUT7+C&ca-PZs|1VT-_X^ zvK-GLOV2jmv1vNab$1UzL>~5@dRYQ)kZy7n?G@h$9L`JFGk#jd7#nSgY+Z7=uLv%D zeeEZ#-_z}EH$ur_x|K|j_M6?{ZB2&Y-G$@H@Z)~KM(c$lk$`);evaqlqMLr-Zl+RU zABkEmb)K3e0mW>8FYU;w#d$C>Jx$Q#(Y1J#!Equ?dgd~E2cp2)T z`^zrU<275K6hqpW41xm6^>h2}qDl&-@e;$@rIv)qFMWgJspFPmw5jOcWP`hpik#iz zpnb+!0{P}}**~hnC(GPT{-rn$Lf*x~!-|BY9ZpH`#@kVXM)DC?(wHAn`nN>LB7)t6 zy^tPZ!LR4YCgh)7$XxZ(3U)87zN8C9{tk@HWT zuX}{?dN~B$^i?CO?LA)+_CF6M zjjeh(cjr-K<^zoaap-^;#9En!0@2tlq6bAypE z+fgov6i%5=-RIgu&w|pgg{CrM4cfU4J{(53n{_vvJWrnlxXU{$C0v&AGVeMa&xW7& zvqlzI))!{ksu1^f?-=*29LJJVU|NmuAbtC@E%%Eat(_@45fMz~gNjjXVDb+0>&VDV z|D2o9cB2xTK<6IIV|=)dl`Z7BAJ9`OjeO!7g8IPFA<0O~D!L60N~1UbjOVmcP@}Yk zchG8%Q-7^IE4HQLG9FBF_+q3?5EceO_GgEH6X*3#r#gg8q#;&%iHmh zr*;nv2tknG1|uUsnplO#Wp*8>7>~HwP3Nj`TZbcnh*;YqT;M1mKAJ~@k)$0eH#iI$ zBKdk&uMAfwpc1aa_3(<@kn%Aj^{X4`0h)ZU{qx0)n`Kjr-X^7DlSf&j8{W8VEF=AjKNb@*cvAe@J5O?=(nyF)WHQ~Y3c^F$-E04&cl~ew1bbY@fk|u%7xW7OA2zg?2>F{9edcyHIBYM@ zwg`wxEL>gTJXX;fo&53mbsm;4N^He-s4ZqY*ckiGx4u5y6>Q~7znO8K);t~W@!Y7H zKK{LEH(!gG!q=07VQD=#8DYye6B%ruz)Jk4@-{QZ6FZ$qje~Bpn!AEUPLK$W^7Fai zj>6YiDwNxYNhm6FYGL=Ag^G&OVNohJ+LAvjiR90cJOXa~loa(sf{j5+XDfb#8A83=VRtEX-Cp}C!%(+73zOAnI}3taLui;0G@v9Sf+^bBE#67D#S5mg zTg{N81W(YEh0X+({SL`(++H?W8NX6%zT4cp-%VnvdK!6ORcLlUA*K+s62l_cRUw_Z z+CgkPdv%HY#pcN)i<;m~W-hcariS2=M-dx^v+O!B*m(^od_4YN0vRh~;xsR)1$? zy9arVJa?^0ZWnNVQR!c1@IqY$6FJZ_WN2lW({k>MGfn%;&Ah$^656_#V)7QCiQIH3 zkH96M$W}@8@bMX+{T)|61+9l-K)+r5A5bAf(>d3ggAcdqhOCd zt7Qo9%G2&5f{Pej;I%|TwqCw|HZ1%=u;(+aiW6m=LR}GWMjh&P6`|$1Gu^g%3{{Ko zM?@M%%jPNH$`SJM+?5Y#FXSeLFGBta>q^~!T{uSAXZE3(D6Wc~R!emUbbC)=f0O(o zX=_zkyPNq`Pn|M|YtAz`6h>Om769)Cg=99gU!TKCsnGDaeZ@e>z!{+|Zm>R0louoc z$i4B2IT>DDQT5-?RD+EF&Rng^2GvA(T97cQhb6&sUMEy(Td0{GJi8Klq-=6Oe2Z72 z;J*rqYMgSIa^+evBuBIAK=2Sb=JhytQ>9O|-rg_BD~~V}l%gfbZ>Wi|VvKAD)i#^? zX;3l$WHgP{zLInGk)qUV?L_+ee)=hTWspsn2&NU4q<@Ph9(3#)>-N5Cd_6woh=XrF zKH*@_f?@66YO|jpIJbnZ@!j?#D~!wkvaK#FR^x-GNut@{@T$k9^Zo=DyAqRLoqaB= zQxNIXvGn(%s_)8XGe7N7u{up$o#SzkmsWejtFPCYT-~_YMBEc`Xpm_Im{UJ}PLVBC z(;=Mq*|z%Xa$c_Gc@>P|Bq4iW)myw?Bx8x1AUr6Zmz=q|uuXZe_;6lfy|wKa&^>Il z7}k>fY#UBX(rfJy-T&cM%)|}0J%uJ6nbA$Em_3i$e9_(H8RMP0Yonb^Wa{po8XnG; zzsc@<$w*e~)WgXy3{BBz-O(MFmUj8x!t(ItDl)z67lV`NxWca)Z}(sG&Cn-CCIezK z=|~r|kEyBM++X8*yzYW2B=_{Jt-o*x`zB88BUnmXLT-&$X4ne1#{X7%HG1=4UUvLF zCX!pteMxL7BWuZ16^y z>bAq?HSk1$dKwu~qEEKlVJWQh)d-b}fgCj{i{L@kr6gO^kf5&QgX~ zbSKc3&}FSwV79|X{s$59*PfTmpD|Y-arIUSI(jaPgd@ppG4rRIekk6gCR)DMga0t8 zc9zIsVH1h@RxO_Yb;i@6`qZrf^G)`GXsx{g0YL~EZX{;ppOK5RV64fNb-mFWXTO~Q z@{u2|emT0ARzU@`#q{Q?1tBq2Rn(vMGO`YJRp{zis7g#r=;3Ga2TY)hZ}ppv@Z(eP z?NoE`=C8G$E|qUapB^fCg*S3p9Hb#3@t)tDMAHKfOm13uHQX+C%AFa`}OJW?;BuazbZ*oHKW4xAB86=DF0NY1pgho zdLJ&&m~iwBx_a+glt93gbL$;WGu*>VClsB~&u2vTr=rL|is|LM$KA?Sd09i#BbSZB z0&Bv|t@reJiHA*_iSWS7PL;e6Pc9Hw)&F>c_6?DmvF(}RzQi!%l~GIDZ9!$jZNgST z$BN)E9lNVT>f~Ysk&y6rV@7J4+{Wy|(+8dX99=(Ur;P`LOf(k6NOPaY`j#{N!;dFh zKZq&^0%}ns1GQUOOm>#n{0*D^=!T#4?m52&upmk@d&Ax})wv3?cpK4C1%CLFc!{pm zc10homV%nLlAt$N^NZ-mpA}6vC%#&3_w=@rux|rvHVy9|^bA!D4Qz?(^=~|O{0#_r zn0*(j$I=P=s#=-)p)q`DP+tKPYp{ab8*Pshf2Nkh*rIP|um4nS^=}Nf>U;UkKD7Q? zFnml&e8kt9J*?vW_~^6ZON6Qa>);G(cXQHA*8Qdb$|_{-+?DhAw$lH%{z-QwLAmc` zMbieDN9y0Gl0%DsWnL_0j>za0ApgU4!k}bS)q#|B(mA^kn~ou2s{yizJf;?GDT0_V zg&4k`(}=T3%{=XV9CPq4($q9@63()}pQ1tqK3~7cMlFOtv)@@&&Q1>|mtUrkTBy}1 zOP}>sE{QeJhPSoFfR8Ua;ZHmJ06iOMBX1zqC}Pm4#s+%5bJ5&!DX z_U|;C-tg7Rp@AZ-14WWDx+E3tkJbh7Q5{Wq5jKF`#?Ox=;EOelak7DL{`U(6T(+Oh z{xtGvrhQ6oaSNs$S)ia<44GQiPeN6Nm^x#W)G6GL+yk(U1W8#4|Hrojw)}y3$}@ zqXh%b1ysA)ro*p>PL0K1L9wy~l%QQW(BRva+i*GX23!@#A)kLE1V2|8&sU+vgdl}W zFwTpEmQ^8enaXni6>Wp$X7%Z^{rp98Z?C%W*BTvu zdcz2Xul|k`2{{uCpUwR8;)oJ{#Dy`HTQ!PNXRM9(WaU@ea&d9Fpu=C2vTy_w4~KhmZR zP=8R<(o&vICDB&~gZ)hQ#QB?BuZaXZSK6(o&dWgMA=2pazDV%=IFB+;c5H_ez*Qy} zg(jwXqEoXh?d(DtZ;^E80Ufg+?PX>G;F4*S^1psfB0lcAOGx-r;af+w<&R3o>>Mb{ zv>Tt>8t<$fp2~ADA+cdWRvjxajY}$|X#3ay}+QW3Z$`!$3PkxBrKK zJp>~?O6XwvuJUnHYu|q*N6tuvj!iLcZcg`>+l~@9@oxd0O93|4u1a37SlW+&x0u>GGa9 z65%X>nWY1%sHG|kGlOYuyN7Fo#<7`i2>=?Af3Eqr(Os9zv}LTqE3?idsx;S^8*lhZ zuktP*6Kgo!k_FKQwd3$b%Fv_576b|M$FP7~n&$IwS#Lv- z)NR7)Yu*u_c*0G)-CR@}QWAc1>bVpc4OzL+3h2Cj`NrDL23(s`7QYgA8aXby8tva+ zb9T>6X?m`(WzzOqJlceB_~|o%Dv2KE(CZoUN#qmgN`Y!EL@7cdKCpf<*Qu{?$@AEP z6SqQuf6h4Hy4IRf_}N^C;jo_mg)`fXI#L(&tG}w5ZMF^O20UUxm~6b%Z- zg1h=LWLu+Y5>k6=?eOqa>nob<=3`Z8zf|mH?O2}OJwV#4vYSe8Cu<_ zh|o=S+^LN9SA`IN=ddQ)Yh1Z)oW2%zl_=6@Ufwpik%3x$_a%Qq@mEfGfili161<#` zP^p_JQM2DR#j6tWI`?+LoLmy4hwV4Y{(@kp4u2bLIcG1k-0B3ru|j@2)d8-9?6YsC zNsefac6KE+W^)rS7s@-UfJ8@?Xq{^v1orf^PuE!Nubt_Uu^jMnEJ~RnAxV-$aa;a~C24x#0by9Og#@ zP@kg_+b}zM7AAkD^L+c0e~QBNyEW$&&|QdidWRJmywQh>t@I4FSQkD1mF%|v{Rw~m z)(S2=!762@>6*?vUnr7bkL1fKgz4R~PefFS_p!;yy4Ex=|2gG$3L zDXHmyJK>RK3O4&J##~)pRrR!bN;FCVH4}iFg2Z#$;|oV;Yf^7RVMHR{2=ldxwMN$F zt&i--c`|ftuJGZKx>F!vXNV}2MU&`(9=vMFQk8>yd{%o0bv2M|1<&sO~cI9ELoCW zRxtB}ZWbAd^2*uG>({Tz)f9R0_#!2Qvz{i<2YbMTex#`yW1^=1^FnJv|29AfiUM<* zH7}n_e$LjqCP-jjA8VmRgpgU`@yj~xjJJ27kMt{kr0EP$wzfYp%+rBCLc{X3+(6G>G8M!Xd>AtR= z7#>doZ>a>$3BY-C%d>ig4D2KsVN{$9W#7h`g0kW&?-oO32qx~guHkzZ>r^CFRa(4{ ziwUn#6VGmZWt3GJ6Mi!8xh6%ikhOON0c@ z#%~XIOEfxa!S;IKQws|VnW+hrRP(BxbE<0F1;|!_c2d^jjug(MCIi&5}skDGp`#`yUtNsJI{uVV~a zrI&Bf!u}Q_k4ms1!1?&6dk(uxvMSWqgq`=bT*qA?>jburc z#CXTbQ9a?XBw>8cTcOddpt#1*{jDur)E+e^(*Lwm`p2xN$ZR0|!eASqsq+f3eyVsw zAVr0SLY|zNxd!7gq=+f&U{_Q{r3(st*66@k1IMVTRDh57rE49GFw5O6>HON;Yv*goHM*iotF( zwsQz%AY2}RgtWphGfcHmg;$i^L1zJVLG}Dro|UT9Y|GG*ij7H;BK!2|6W@-LW+MX2Svq#C+E}X{{!YL7Q$bhbbA?+@r4;5uTZ1uC zGE5s3NAu5E+kt>zIL}_$p&%TdH*Zz`Jw*Mbp_W=NzIs2#&Tlmi~IJg2W@x-Q~RpWyPPlzu)gd$f}{w`G{~az&z#`j*M0rh z?21L4egK{=B7KHPr}UCcq_Yp9B0#jtc^>mqRR1&MSKbNp8G6j@cT39QS9U!=*ZxXJ zzW=CT!y_!3T@i6VQ`~?3JODDyXk2J{LWJ+)kUTLuE$h0hL$|#p6_fc*4_qcb*xjm` zZ>TstpBeqCP`&%b8~l0XrwiR!&$6ghV{%Q*C2Cx3=yg^d01erVU1=Y;8a~;9zPZ?P~E(TxKB*5ax ziBT6xC9jYf`kt$=bq{_pe@l5#eV~;S3%`+SC8{JRJRkn-m3aY<%np0q6MzVU$ayI0 ztt;uqjtfMw$Hh*+DP)ARY>@G=-9O~8v%g4|G~gRpLh50soU$6^otSt12An%va7Jc# zIykNVIDMT(4O@l0&&c`LfZgDP;6T^&*LImoWjxeCzdfC_6F}sYIVptjhig8b(!Ibb z`Z)OZEX+`>5zpNr^9H_+lINN{({%OkCh!THok_y;(^G!;4XF|cppSL3q2jmHTneP* zy?aNvLE+V&e=Z|i-2II9#mcPtDTQ>YsWMl#g8%i$jgAefx{;6apLeqe4I4zQPZch# z36+FDmsNOu*iuW_+5KugIH6r*Ayb110?IO8@y`As;{r>})Mq)y%nHUsQCg(<2`@6v zIjsmcQgY<1FyUyjx`zsWe6OlZe3<0IDBIWP;$~EDq+HDA%?9@oECrVYPTe`1_UAA6 z4nz-wV$G)@v(?<`aC1}p#D^U3mmrZ)W+x8OuKj7Uarw^agOet$qW;I_&rTn1xQCvc zC^|pMp~u_4)!DICqyL|zK~{XpzZc=Zv4Q_TNl+2KtC{~V9q`q8PpvP|SuqyECHlY9 z8F)qi9_&K-O?avGt5^Sh)4>`f7AkC<0;iN*ed~0b$qGOzy65Q3;9~$Mp;&2!5^Vt` z(F$@PKRxc(C+ zI)yp<#Z6|EK7)jb-#;D&5I^Ufm_FNJ2?!S@`)Dfirzwq#^FI6|c?7miTy35T@aJY2gQV_IuxEVZK z*j=wM)pF2!GW2cPsnLmbNpllPOmWCB*Z{nbW9z#`0=v5|)-~ z{`E23+<4VGEDmnM^gO|^!ZYpa#&4IG?(PS3UB%;;BO&c>LWgZn!XPB?-8hXO*igQ3 zp<{@v*h2^sTMMTI0pYMq%vg47PTfjTlQ{y3#*9)e@3-lys@&kEz+j_8L;fL?|;qa3dq?WmurpO z9{UUL$S_x~NAnbvm5Vh|!1LWORg00hk%0^RyJ^E0Eeh9xe@?bM(y4WSRfcEs>Q)1r zR1B(~TU;C{`F7~`;!W>H6?m!#ybCVFLjML63!8jiLbY+H&pdAmlDyW!X|yM2`y36u z%!@U(8fs>1VSpOB6^g{*AroF`&B;@T`65gf1~b;d;}};{P*jK=-UQ5LK*R!a0N?_w ztZf0LvfBO*oxiEk^W!iD@RCwf)3%p?rL?|$1ODr-KYs{7sg$wSbp7ncbF+b^+I`X6 zd#b{eoNM?};7HfT;9q7eK1!m7h*pd0M>-&3d*yj-nnsJ3k9f*qV7<~B3Pc^FIPgqA zW6mNW2pxLFKp$CdiveI5?xX@0MnE>jeF25GWt~8=z|=(=kK^W|$1+VEOhr#m9;*E| zLkY~2nAepr=;p^KCw{h0j8X4SFEeZj9jcfF;vU!$UB`D1GQ*|^q5l0A;Q)jFjHss& zwAQNOtb>>;estBCknl~kDm$lV;!Oje`{L6xnLdDM>XZn+SAIHOKE6z=#0i&_rHmUK zhG9oY?r2YTUC+>pXL$VECvGZC$o*WeFDeRxhx%2C(Z7;c-YY*_(hyF;B&@C;mo$A( z;%$XNqfAt!&ItF@{ERx1SU{;^|L0Z4mFIB>@=vBXX%erH7N4uw0rQ5Z$HkdQm#zbb zI=tud!Q(GGrK>6;H4I9345oV;TVMXF@c`OCRz{Pu4f+wAA!gCc0d z>&`6-+!uL!Na2#P&c)vm`U|i@DBQ@5e0j7E;Ov`qj?Gf6xE9NIN9;TT1B+LOzuz(2p>O#hle*~ ze@!%Re06wt^SJz=Rdpj+Xnh)Re-tn@Wv@5;%bFtK20uVnPfsB0>GwF;eP2wV$0@!_ z<{M7L$OG&r`G5aHP0=u3j?PYYfwY#CbSTJ!b+{M+pj&Yv&UR==81j|NQN^O`)j6Q) zd!AirZ66vA#3=^^in8{~x7{PoAI?PD&tJofJwCQQd=kPCQG#j%=4-Mq_duI*JkkKxL5EM0mMDg-B*7q;oy(N6YGPPlvD4E{RW7!VGICv|%$-L^7GS3I)a03y z^6XL>cJC%QBT`cdPj3V_s`@OUw*2}RZ)M>~l8hc^-3;b*)B5ng)>lX?D#rHp(dp`@ z>S$|c7eixQIy+@_yd0F3c~FaSp`JmbqwmoqEH$JB@=S4fODTY>emY6ATQve0n(cvc?W<0 z{vENDP5-nLW^*a?v4veuz0wBb{6u+B0Ir81aK$xMN#M+|7tmy+xI z`xjSLPD-jrTcPY_=La)0Tso>*eUkSXHCCQxcJGWRX|q~Yp#a1rInChr_!$$BmKDY& z@uk`zC3bx^mVS zmS}sFxejk+_4$4O?wIhsFnslG^0JYaLjhtpm<niyR3!ZL4oIS5N3i)ie3>@# zN_jd_enf0gNc{X6clqwn=&Xw+AfmgF8bm!_Czem5#p;sdocvmqtB=T6;1~v}=Czg< z$8nG&5%4(Z{-to*Ky}c7jfNW&qzihb^DKKtG#$1kX_NxZB%`iBGawv9k{}ZTGCMm*%OR^AGj~&I zX4NTg{C@m<&~pFW9JyzTR_8WI(hF%cd?TDWc%*6$KABs7yNmYu=ReBJH;!{KQql=% z@V+s@IH?Qr_32IYsELV7uZIIJj%FME1o~15N$wWZK*HPUy7K||G+{DF?aXC@ixcqs zRzD_NarOuL9RW4&yk@}hgm}ti^k=q_nn-Hu#)!p&3%4N6b7SFaO1PW;r3;mr@evy6QY*i^} z9?R9=hK442BVovQO9ibtC_g9>zpb~0g(2-6>J0q-LjYf-WWpkkmLGrp@HqUdbQBDK z`XKcs6j*u>H!%BYCSH*&aW+ePz}s7RTkvuG{V2=K7r(7V9=_HeBHlC??Ys{u`txZTDwS&$EVyyA{)%?9xt7-YX3JiNQn*g?y%vw6Ot7 zb??SG|DzBlCQi9830+dd%9$pnra(i{)=>H;jK^OIsv-*MC{)x#3H=Qij=FmD1g(6M zzD_YY9i&j7<>!|qf`bB%tztRRm9_kskZfyZ2k!TB5TtvJjFL$bBG8V7`|d-9b5<}g z5$iJ=)+>`=;}5wkX1P;q8=EvkjWHqDbZH&RO>|f9_^|`pm1zn7#6&&%(c{U_)E>|^ z2qBc-y|~nVg%FZWOU%N1e0f0P}Pr%~8!>})M%}DC^G1iK*e>AD>Pr!9)+lTc+ zn7*I~(mPrix^R}kGux-Xb$uHygF^~V8>jDTj}`z-WD5hN4TwbC(Z~_Pui>UE3^qUG zxBV$((xb)umM(np92_m;&@bkxv|oel=k*(1V}irbt9_3A43&nR%m9*6qM<0L<+3$oUn)nR zQK+uFxMK%$tg282L9C}=Y0gGO(%5K7I{KRrr$0PX5HP~FqVG{|ngu9PS4vBwta~=U z`!2)&Y#i1R7tp3HZ8SWelK1w`3o{kwG8`U zRPCQG$KKWKU0x&&?WhVIEgZ54Z>`m?H~Xc83AHNsKL}qHIVl37+ru-S$uKvWTKhC` zo}zVraB{*T$NjxZw&R*AO7>};H-5mPD;h?P9gYn_0D522Yy9WkCTwFVj9ub6sfQE= ze~zvIO1Iad`fn9qP{)T!83&XdnV=l99F;E$)QZq;_ih_o+n7iR`i#dA%frcyuh^UDvUw)^Hldp_jkZLtj9S;}>(Gb!x(cEkTJTA-k+vj6terF)3XN3l zGqZ3{ylvFAu{E7s*VX?XCsT+iPL7K`D3pc2l87h@Un)1 zD4SU|A0GZXC_cZ9Vck(kRG|KrV>>Z9xyBp&xr?{Mua0TKX-h z5CaEj*3Nxni;Bw4?E?2JV%6;aSN-dGk@+SeqdA}f5GvVEBl07!gnoAabWbZ5wyA(sB;yFJ}jQY$@ z!-*=x!oqs+Q8cC_99gpXo{hYOe}8iq=*Z4+$6-X9_|;s~^so>!NR(_P_2uj(g&Sc% z_GSz&$=OfX*IJ!R(+ob$gtczvpmpB!i;L3)EyJ9E1Hwz7AH|=MTT~Qh^qAm?{jF;T z>OuoNH{QrT$ACS(=F44MtHzFiyWz)>z(5ADVDXM;PcYIaw#wF9LbnUIeSwdkYNzlV zmWZULYitBQV_KDb&844=anY(Y$fkc$<@~6jA@+X4rfBp4tudn;sOK2ooF-jw^0f8M zJGLp7)kaE$VT_NC_SM=0c-S6A%$KwIEFi1`O9U?=!rD`3Ql`JjsS(>lm zz$*~mhr+42o>H!NQ@(twXySH(aPMz7zqz-3O=vq+mQz=U-xG80dNtBknB@mnH!~4Z zSiC=9mvk^+S8Y8`4Se>yt0T*^#Pa>w>M&42^ksV#rP#ab+&Y8W_b?mYj%iuhIBc%n zxRmL&#jpfyByAg2MZ3Y1lYnv zbHBB{Cs`m9QF2u|sRS6m>hwajVqr9&3$3>sTTQqfxYp2zCZqezgbxaRD|8x{8MB1^ z^HYKg-XvP5P7W0o?tmk6aRR4cEYK5M)tf{)3Cj*4t7S$R61b~{+3%;zOy%C!0sTvi ztj(On>-`r3O!|6Wlt^J+QB>lTg{y|KmDQIczo)Q}PUFXor+X1L*9ZE>LM;Ka)I#Qs z>G}h0&Fn(@7bQ94%hLm^onbOXfXFSjLWE4~EoKMu`O zkqT{<&y^BrKA>3=2`{9lRI|pz!~4njO|`h!wz9SV(4Yy>p+rQ6J=*R(v=jS|o|cYI zZ}%KWKrp8qaEs%xK(vbSbc_KB>-@(lnEyMsLS{LfVQC*g>gVT&`vNgEG;~DcRSkCZ zvp*(4$BE1Na-VN=a}&I*#?wQ@Wkzk0Cns!$MMYW*Wby^rjx8rhcbEI_0`)RWeD2?8 z8@vp?S0OMaShL5y-%wVYC?)o0RO(YSgH}bAOAoq~bPp?9M9k>>$d)tiZ@}tVpUl}- zf^z(9vo(C7&Sp*4Q@SWw`zXb5+>lwcYODfvgZlxY&*eTjHoD90xpZO@w81Jzdo=E_Gg$^V z(7@+%bYkrHpKaQ$;SBT6?uhft$_{oWP=9i(J(!VIHX@B%Ur!fsNyRtxN9;U4S?#+D zXBj#>Wq5+j*VeOt3=fpY*OKw~_WbuH2*D8g1?FyATLW8=(Q~Z{BiDgz_ZE)f6=fM1q?#N94p08Rzureg8hdV} zU3AoiH|1CUkQvpQr2FL;(qX;SqCLcQ5zK6xq~c8@RXW{Yqg34x#mFSr*5cjXdTVRz zm${~r`sTA@hyN^W%d+E^r%%Xj1MiD@2*5#4RW7$b`uP+`bb%&LgfS;8N5}Qg_>twE*9@x_Q)uv6aMxRhzTA5egl412LG;&DECYK8| zaI{|580FR#6{QZv%TOk#r9M23xY0ZXLj?KX!CWB6aada7{f8Z4it$?QXRlgtuQ26xo?QFwzuusI^z%$(lGEJ zNVF%TTF;}P6&Hgb4siYx(FM%WN1vV_j&}J0Fv81yk={O>p70+0nSf)m@`HnXY!vXi za6KNkU7I?f8vt#bNXN%pk(|o5tdEqx69VPI`&MUy+uQl?YhBE1dxebPh11{9N!OD` zcZwmos?4tdP4AisWXj49?1=>LiHI{Yh`dgyK*F&Xm1XtpZZU)I1s9ayfS4GZ#yPYv zKnPSD=?^))!HEE#k z`cf1Uskdm8&->5OGCM1!+j-Ly6M)+By~5^KP0h*u+3@)Ca*xr|MO%?NecliHk`nbX zQFWsE3M7rlbpYM@nw!0cRBV=d5-z*Dy9=~ZDKdsk2$1aU?Oi=QaK24qmKT$!$Z@c- z0qJwlAf$p5F*ZM~%&RH^Qhr9X)Lst;|F;)lAqL&qht^UD&*<)U+Up^L=i~++G@cn5 zkxEArty)&LhH4al8{v#l;r$av>fd{^-lH?V9A>ZQfe>C zp4HonJ-{mm?y)C|0odPCT%y`b%m&jSNwtP7a*qX0YL3hOrFFc#Ti46O#etKnC5dT` z)6Z`c&DjL```ez<+BPD1mf^^)?CdC06i)I;pORhq-x(RPq!x}&PAV3wTbOx21oCy{Z6eZn%G{x2!aCh%NRk0`<@T3d!0J2)~ z&bzyN|Fq(8U*wd$xQK`Vk&D6rP-_bWssz4lLnj_l&lTZp?J~^-)EoGG`yu<8A)zaw6*4jF~#7KAU%ziP{>KxH_cNK@3wJ3TetGDK7a9 zM8)mxnetWXxEaCD;Ar8^&m@wu?q-<856|!Q<-qFaof*{Dk{E2 zMLioF8m?w->pef`Y|uwne2;gwteWTn5=d_xh}7h5?f$XN4F)bUN*9WW zr#HSJt~E)e)~mCV($M%?fSp=0GC$ur$7S2#eDUI0fl2u0rp3K5DUah)z8w%@dq=ra z>C2579^!e@mA6pu758C8$TCY(;Lbfi-XWVBq1^spe<|RTnV`~JEE>)Cqd*HZe~beW*k?2wmOWDN@h;&pXB^;gy z^jgdYlxZ;RuaZ`F4#wNJWMs1*E0~V=_m)3wR-nXLzy$|BI0!@083%H;do?&{T|1-7 zaM$_&0l7sXTC(D!>Wl-wS$z{+^QFKOc^nV8e^!644v9oH$dvC((;tjRhvOw5 z8eZ39GT`HamRjfC36LcsR2b-}Dl3_Vt(kxXmv359+c~?cod0QDmMVS12b@-rzcZQ0ZTo^HIIWjx@E=T2q zg?B3Pd>yC*N$@!QtS10;;lztK+|jU&l*~-y_0WsK!9i9wHqhIQhHlv7M5J1b3p~56 z-k7Zz)zb&kM|f3nH8p@Ud_XTL$=%}z3VmoV4J=JeVkN@R-xmDcZ#es|{2jz59hdg= z1g~G4AK$w1^Ur~uZY!rv8E8UjPcmF-bBnNL?981M#W z=K#);WHiV6C^Y;4OsfUMIOP@OqkH#en}bm5oz8B`B&|U!!JwlABEf;Lwtc-P-Mqkr z6D(SEj~YJ5GWOffTjsI2slUN{Cq1j!avZGGubE8sr*97!8~uF)1^gKD!eu0e9&d6a zH&y`qp4^dwa36Fj06RJlgVf*Qegk86r2`e zlGqotc@E1S^FQ@0O@VE62EOSI<(m1v3*WE8j~fHADf&1#_4VpXpC3*4e4nlN>Z_K> zgKzYM4Yi9pzfQz~2fSJeHqDqjHah_TqvpCPx}L@Ag`E6x1n=Ivnr*XYF0F9e`%S>k$@$az(#-$%_5>x2oxSl{ z^2rq)7}K{DaA}=oQ?&7UwR5u(!5A_Q;Su0OnVKsrB>C{y^z& zXYbqtM^ELxDW`C3GX*uDef9~(L;-=eT~?1K zr~Q)3bU@DFXvJw^@0He}?xMSBnh#kQYs~k-xn(Nd7Cx01qUKT;FW2l7D-4;^hYH!) z;;^txe(`tg>=@P+VtChaG!&NFL#%ODvotwYZ0ibteS&nTO&er-JuJ_UI@5+cJlIlvCE zyVYv#LXgl`Hqqj9+lET3shbMggXxNue)7szWzO_+q=AO%-M{jxJ9T`16 z?GlDb9i2<3LU0l1h-m|i7a7pi3MI0zk~c|tkLP`GELl-fuKzYBX(%JU>MI~SF1&qn z#q?XSf<7YbYblW0@d`^I%YaDAUilC32EsP(R$rp4w7x4+6i?cLV&MJ4rn-18MZ-ZV zRlK4mK32@sfp_~JUYK%LGzlSSa{~ofQ&qrejaMFBWj8dvcL3t&t$r7CczSw+=44kl zCYgB|ctn9de5o}EB&uj3gi6&pYw76yqZX3|v!=nw+>!w$iJRjfvh1`!*!27Yf?yaB zEyaTTjMu}z%E#)x?Z0blCl+GvVQr;$FN%NbsVPjI?~$16>FKJq|AtI^UH#EY6AiQX z_af6YB35lYvin}DexiD6>cG${IP zoV}wcTfF&1LIxk*NIEdkkU!QI`pNI|esSnK0GK0bGCmj{oN0HsLK_gFp2+zd92 zY`Iu_mQPD1A@m&yISP628zF_R4WqbNTfT2+XrFy2zZ!C-X<)+Dp!u-{aC3>DWvomBq;gy zzM^)T4u8lK(vXkq{jW8({G4i8(i8a%xzfzCuRPXH?3DAMn^1OCs7DB6c$7D?=@fC+ux_VoK9gMvSeQ*ZO*X!3c;8p{iycBA$ zf#|+?q|K8#ZSqWl*a)B_2MrBIb6)E zvcrbX_wCJCv^0d6G!PxPXVWi6`!&9{s;GC%_h2d(2GF-^-@bkUA)Xw!(8g~ZcqpvQylWr^1cv8g9(`;xFF!QW*yP ziriRN7F0z{Q)aTb{2+vZSqN0WtdXk)=iB=F4|d~GCk-JACgNBys`yCL^9=BF$H!Ov zCvUOEa#0bqoP4_lGLfSGwEXQ4t_AE!ZjRU*+tHs6msCK7 zwvhNm>>B<*dzopFbT7F0E<>hTb{vAw})&dAWz2zzFwEJp>H2TW_+> zgJ|{_+LL`#&-q#hiEL|=7<`-A*-6X9%v|Okm6MZGTVKm9EIcr>h6gIdS)dF8_O9q? zC6KcSfcoN^!ko}hcdOR??{%zCn*ck58(o%(Lb?M3S$wsQoKNK7XJv|7d zF1h|6Hu^P%Ei_P-owyJTd$!>2pPgj^cbA^QX2s#l^MB=?K=}=_U?l|Ps|^JN20nw4j@#ETf|@LhH3 zM~&9-ukXl8Ef>!U-pz~0(z6d)L371x>+6$7-=Kb=`(ad*ww!AQ{v;&^?UW%V^T(T+ zch2_P9iib*MvNrDtQaV)U;!UtC*o{ z%(nR8jEty()27~WZa6gPR>;*tQcW!mpchrvecspHXvjs+>1 zLIRZ=BRofTWTf0%+qVD?0@{|Fo12gF^uX-^Kq(a#!U|+cu%&8DiUo+R0kDn=WJ;lG z*#Z-jlW`l3k^3^JVV!W`K?Cjz4?PlqW8i5nw+6g(a6zO6f(-D*xw<}rnQrDU^YK=Z z&Gu(%{D$5-_7#Ah@b?ECK%(FNJA|~>rP}6+tBT|>kz%^7b_4^Wk6h3>zpWzI7Ki$4 zL&FC}espax{HAw%BpVFe1ooIwar> z1Mm2$xD@AQp^msq0dAbyh~sSS^S=5qtnlqLeNfaHx&`?N?#&7;i3j~yX^aM!+=h^z z9*5eDQA03&z(30T$`Guq;z=;_`!MKsuF63$s(5l?2HJ-BVYb%h#lwFfeUhQB=A38NsF))j(W072!^JuCQOY# z-o;*h9S`6%7~RUax+08DB%jjFV02ut8p#S)&w2P*0~_TL6I1@dDevM#lIa))C=K^( zv6U-%@)$b(Zqf!6xdvv#NEvB>IreA0yl3y-cvmXJlnwdzEf$ytS8P8kLuP;Z@qwET ztTp0YA+S5h$jJc1Dl0=P-`C5siJ`cA$d{@N6gUG7Z5Ioui-F4yGst){$1hgynEF;k z)n8fD)6*ALoVha+DWXa@Il6oJW*k;<0sM|&_?uC~Z zO}-(!`}HXtRPvIYMl-^~+d2M)0cXwM%<>nb?{ULAJEgO-$?fc%5Igue_|-t20k5W0 zpVrzfG-_>;I-t@`Dw>-I8ASx`dTjNdf=6i^W;x15;?$ta$p1w-cMmXv8ii?es!$ z(cn(vPu68pn`57xTb6flc<}+75+GN0IIZyn$tSO3X`6@8mXc zCrcjChLE~AK$fES&@pS|7Bw|xHaEZa4+sE4YqnH?-oO@{&KQyg_Ucu(c(v0gk>m9>VB7;L?dmH}{VZ9Wv+#~P zF8D0km8Wjo7##F5N7NgTnT`BUh-bE8gEM71mpN@7bYf{q&BuVm+IoWlKY9<~_|R3)+%D zQ!|s()13#yn3=jsUi*U8joeU3EDUf1-zw_k-%ED%oWTI94ou)nELnr+n?mWFUjWFm z8`1-GI@nSkDj3u`Y@h~euCPuUS;*c6bswnwqTwkn5Wi9_aY6yLNPgKU3lnj5Gv zC9f^c)d20q*KsoSyoZk^yLP&2)^FcNxzFjbvl7MvSsVU|5X3!NQRovmXjB=DGV)NQ zCJ))UiDiz5r^`RZ$0eykj9R=oOigo4TD*xCs>i6jita14j(N9Im1&!yFlLH)fIt9H zuq!cs|MYcVxpB34T;2?`h{#&CZ$w2oAyM$0=idfLJ1;eGn_$@SsR4}fGbbl?m%X>*S>FuO*N`RckoY~15ck)pM=l|sTZV|rR& zK1!j{cNbqRPED3x>#4YLvBx;0O3M-?S#3XBC@K;@PUN)Pm*H$>FLL$&=jHB*EDlk10_r3^6w z?>-<9Hmi*=^AcV<%oOa305p|>04P6fSJ$5bi$r%@_=m*!&GpC{&;icY1#p270}ort zw@XoRKr-ODw8|`bL;JEygr9zBy^Bsp3zQ>5@oECuuUIrV#SHW!%b||+6!EUR>(`~8 zMJY*#xl}hdGjx#OBjm4!hgEBWZY3f#pQr$-tzYRIp-Jr@B#E1-(zBM!+^*YRMsS1*q^yJl_1)~!=qp_6Kh&bMzIlP zDv=DX1_Kzhtbov5!NV&ePnRi!hajf~%!jt_?qD*s3U6kSE@!eT6Bs-X1R}hAqC*$^ zuMXeUZG0YmlbL4@)@=TFwjP;8neinT-b}jW0o}<;-c0tIni|kwbYNiv35k)RNrxK+ zSmP(M2=9Tf{osHiQ#DGaKnl2y(uepwtv{BO?8X%2cJ=owGB)BVT7V-E<2C-%K?DS00=7bS{VE)V2@GU{jBsYGdGJa^P>)Dx{-Y*4FzX?I)Ie%qs_ULCwuA znK_xog=OUJk{vg3*({sx!=vgomK(%@sdK+>n*C+gP$3gJtfm@>c~F#}>$i<$T6zX2 zL)GjDdS;DO=#h&75%;y*ZPYoSC@aw|MJ_2LQkxv}3hJR5wFkm3(0;&-Pa=%SzEq`M zShi+{Lxm++-3UYPYm);Pi_X5aKwD@iVFV)?hV>(}vpc+gN)5+p$}n$WT>{1-0NqS4 z9q}5dvoeA&I5aeL&pX0Z;W9b;jo0({vUF?2Xevskrj+>dCDLCkR z_2`mDRjC~yai6)ipM$0DOBek*vDL)T(8AJkt)ed*O7gLf0M%`uRCUP8*}0;5IFQk) z?*HP(AhPHtQ^qFvHG7Do>EF3aqm+`^IvyW6x$y$?vH-t`Ex zr7D}W`q$n3(sT`HWwiu^Q%On59?YS5vS`vS?cg@8o=;e~=|?M&iJX_`k5)JOoo&Yi zNw7uAET>wQUoirW_;z|u!`0V-QAo0^m@2F%SGTzwZ_uFRl-UJ-q@z zOgY zl~^~mnhsav-TQaDqQ>_RXa*(Qd`ItlIZXX7+i$7I(0!l_Q!$69%i`&kU)oifhF0sO znV6XrArnOtyF`?2AFf*i#^jECFg3zkaiEffr!TYZ)q)`c0CE%I@ zRyBfBQZ%xt(3K!;&w#aA+_%U;k)^r6x~I=o)dng^e^ zHRGa}YH~ZB#jAfvSv>>uTQcHWcI{%|)QD!w?q>p1trwn1+2N5P8)O3j{7a5t3- z^}=I=YEguHNjE?rSvSuDx6*?%wM#x0+Y4zozhtkEdtGv(rAJhA;5<~Z*l`~$DHv5z z<9silm1V5RoUF*ony!Y-Y5E3Wsvy%SrlsK(4Q`PkyX(fyOK?);n40nnyyOuUuA*xN z>|H^L#$g(B;?;edJ>JFNp1U$MJ)M)=JH=gO^fDrar+}4u8_DsGLpPciPFNY+rPz&d zn0cWqFhGn2&Z{+XsdQR)g!tgaqS&xbIn+UuelY1DRfhFhcgiLabm_4*8&C}f;i85; zkY!mfndIG|Ui?#$m$Fu^uF&Cbf@?-EUANdFPW#o6Qc|Fto7=btw4*G?hEDHFdykLR z1e}*;OoJ$aoC-u12)Ot~T_&^-&3lmpe;;5A0GRp`2>t{9QGW&5nwgn)>(N35t{-1P zDGSP5lL3qHN{eu!gyGD?!*QuT99XbGc#fMJJj%fEFn>AC9>Q* z%(&kzq0Nhm0AEA5y#tzpy>qq=!RA>cbDI4#l_*~}#SbPD2{kd{hz3r><<~}Wb?z!c z@eHJ>pGRP%BQYs!*$cUh4+O@Pl~yWALX zhIx+%eSmj7U9M^Xk%gEhzdFDn67v=0N;P_(h|QU#fhOysv+Pq`Jbtt+K!r>MVt@2s z=Zc=;iHiqCbXkt(3Iq2j5`fHs^q4o3A1pL*4;s`i)c<9E&9ZkBk@0??q&jOdCM^bM zWYD^9UC24H#V*ajOSIthIz)cHlm$JjxbnuEQ@B6fz%TR>h6kJvjgk{;E+@Ov6n?Nc zJv@ZG3d?ucLfeBxN6Gzv=4lj*pY5KClU&QP#GJ}1poxn~U?}!=1`iAB(IS<+35$h6 zL4y@3L&a)&Jg>*EW^EOiA{z)IQ&@=6X^^NxJjvu(HqQd#8`p+vJ0XF5x2%#dy; z1$v~yWyyb#v+GHez9ek6Bbh={he)ZzmCB!xZk1qX@7}|7LUk4mb}7wg?LU)41o_pzvR%5Q8bz6lvuQ<=&^-O<(@PBN zzaI~^$oZ-je%d{OfrOx@rK#sPr2Fhpl$`|$l{Ul@IVojq&M|P0WLFz+QyVB&VGZrx zw<1G%EBmd?Q`;?d7tWO{PD{6Sk7l>cE6q5~dAD34mzTlgO?PsC156fOS}sN8EBi;E zm?7)BwJD0g_;FX!BlLvAg|3?jGRcof5eS^=s$216fLCnsj^XIRH#Ou9#s`AtjJ5hbsMpDXPMOL8qj& zJFg7w@LK_=e3c_UprRN4c;y(f_jawXb9@j->a=)o!7as0C* z$U=+(%sdglBf_M4=h&M9seT{hl|`>B0xN!?)N#L*xFm1HnNJJjOw z0TB}$dtzcjwM}#LppW|G0=NQ0F8&_jqDEfjtXxB{9v{{#4NWqb3DHkc+8+Y~Zx>by zKoIBUq4;dtz`_v)J?^mqBW@&a1#Kd5W@9PxtG&`AWB|CHD+Rgf44#bcj9WM_3M=DN zCq3Jq9!D;CTKO>5aB>@T{_lc5BEPQgx#^^^-zkIn{hq;d%I5`TE!mH-$9o^pRnR4( zD#VLv{9poDKTYz$X(tNv^`JqUOn@;tb*{)rwo7{)l>0M&?p$*ua67~Uia5UYmNVw- z>ADJ6KHbOZKgZ$J4lqHfBK=FQ+Tbp}leBV_sOmvbJB@-U*~wfFc4*BoYHA9I{5&vtHK-k&hUP;1sIebdfKmGKwujRaHL#nG$7#0n<5{{nmE0fMA-i}GOj!GTo4x%fZ9Dck86FvaFKj?NX3b5Qv9tf7OXmO zX8}0W3F-)RToFqx-rC0G^rXkYNz%XVetv#_I?M;<<`IU+uP|#-_VA#WVG)XtD`o?h zz3AOtyN46W2fe+0D`1W6l~(}Qx&^nP+weV7XXpkngw@tJoHc~prvr0EO%TkP(^M#7 z?F{G%dWuJ~!DEr_vIHzU1v&wCBp5|u5gZ&ZBVmrl+a>06F8BKNLdF+T5KOUb1$=<# z=NItD_mAhM-UqWiy&T5RsmZrCbf}L9xq=Y&w1nO6D2YjkulHu!;yoEaa{@FDF6I8; z^#8-qr@+k^7>`ExvnX)0n~K#mxyGV90dM-K`4et_=_MNv?he_3Ax6ok&3g<4ITSf~ zIYnK)e3r|@##;mgojn>#s(P{B`hrS5nE8V=)h23JMr#Sl683<0HS`0eDGhnF*>#gKH zT(Oh*e$(1Uq4d*ABhVa@2f+p9c^Pb-+`Q5!X_m&N>%gGc<_eS6)zjb3HZn4Wp0z>G z9*>p>hPc|h&d)L1j_4z3(qCxy!}bn;^pXay6c*6FveqFms4xTlDtfr(8(&m-b}lYp zqPq+NC_j$)Ij*Sp!RqEf@>MQ$_yE`&9>igvN4Aa3%zCTJ#|HQlJ3PvCONju|S#QI~ z$M?J76%I(*XmZ~|&xM8Jg10)$ITAx;T1`}-T5Ev@QbRa`H)h(qa6+yq9(OK7((NH# z(HnhLP9Hiu*i-gnG7xxJQMd*MP6LY#G8rSv3e_!8HEg1 zVw2|@0===1rk5$$&x#PfS|879+m+zuzztN7nwo=vRT_YalLVhrwER1;J{#lC7Qi{_ zO2`BA&fednPyrexdVj2zZi5em z%FP{_{*C?k^lBs+ubrr5K~}_{PQ1O;)ZauLiI=m$l*U~g*Hrv?(U3^U`-PTxEnoue z6(9%zW46tg3`irEO=Bga5uTe-a(g=EBMybRVv{%(iX0T4gmfGP0(Ml5)w|YuXfPm1t}7 zA1{5f@X=WjB2s*Q#1d=c>d={_HV*0FqE<(~r~h{5$=}+(FfTK`b@Rn%Ar*&}C^=bh zXaBFI|0yz1loziz>kct;nl<+O0k%>9bjwe$?u(zCtO;#MXdCGbg{KO>vXob-1k5_{n$S&-NkOzBb#<9*1tUe1 ztUxgeV0w_mkQ5!gkA(cDoNG-TtNg_GQ;ul;S)9wAxTUpAlH@gW$gm1mYkzk)R?0Mk z6;JK--M^XOY#)s;QiHB~uxY3QsuWEt7Uo0q{yK2?s{c@O7uj(x@2tR8^m`1f}~LnPkCf5gwp&^xn#kvom>H{$P{4`LFi$QJP`9|o9^h~VNeZZFd zwBHQK2C|rP>L@LAmm|)TAg;U4p6cG+94n zHynZ%?auD7zQ-%-w~w7pk*ng2a&YxdPrj7%mt zvxx*XQVZygsDd{wOHd)k}1>A)qymQ^)oW~RE zyIIvowb2zhq`I83nQ7Yw)zJ{|%oPu~TEhwHKmWsE{#Ymp6#QmyHag&PkIrmCmB-G( zbv6@npXaxc?qX#}&WdFV+yIw{OTYq~W?<$AW_dpF0*Kl+_nOMO(E5vTKw4)Uq9cg3 z;IkbKLreatU7}d`8q!{w4jkF@3k$PN9?VaMvbq^C3ZSi3vBYU=-lK38Co3;p}2m7;wRvMSvxUwQJ+xSb(Yz2>n)8W~yeY?{m5 z_N_melhNJj#f;GTG0D}sj`{XMrVgl!;otY=UeT$j4E;JTggqGphhk&9H_#p`LvAat z?XJFe}kb@=o-xvS1H zgrTqsrUA+OthK)wSp){;Mz068ISrQi0#?S!HrnsY2_F~Cp{rzG2JxlZgZ{!}A9~Kj z0#_82L#Or2d&N*wBd0M3FN4Im4vv4alQ%Inp@+fEU=h7Z(fvN}*bc@6(2l>h{?vK) z>&C~;B0uk;&8&9%90$|&d`I+)L{KyFIm$aHJmuji8!?CWvMKv#L)Cb6y29wG2|wnq z8!u9kX~B*CiAV8y>1ty|Yzlt)E4`wd#v)lIqaznh>kc-!mFr+_&xm_p9 ztZqweuZ=ASCbGJx+^jAfw){Y7{!~2Vv-Y0`gTYc}8!sa2@XpkbZSnFG z`@Ds{d+qY5kw0lRGnHPmD1`C!$3b_ZDM%IT&3a;&Z+((*!wkjZQ}*H+Bk=@zdVvlD z)g~tP^1`czU5zdmG<8~Dl3y0j`=WWG{?!Y(UmsuYTk{=UqjL9-EibDJw8%hCEp4PE zK;jB*qT9`PINI4X2jYYk_)1=c6tY-yj@}TMbs*E4i5Cxa zoFa%^SQFasV~8;HS**KF)~>*Cc4^gLGdwV+60O@K)Xxa`s;=pOSAgd7n z_qSI&V5`3Go?Efx=ilrNL(vLuP4kcuXCF?u`RcL7U(-G!?V8v2_u?19GyMM*1MA_Z ziFe9ukZaiEdWenVIoXRBLxKC`A$`<$Yt*1iWJy!LyMMfFAVe4BOPa!k{s{V29Ky%W zgJZ+va`}4c^9yC6!%X>d-y-3jCT*fifs4J@iYch%jzLhy<7e7RMHP%lKC0`82!&dM zy;WWKbmcC+lgex<=ZR(aNJrs}FS`OI0=6STg0o!wuLtDHUpMwUKDaM4QJx#>S3=VfNObR)!?j<-mK?P_$W}Rhoe}N%WO%!{>i= zYyWsAbVC?uw?ruOf7Z(4lJ1NN@YsEbGO54`l`eJdeH16Ss*{M&lMLf7NJcIA>G?k8 zYuZ7iA|u{Jeo=73#Q5uo?;K={uU?&2q8~}#QvUDM!aEs1O9XR_mC`_IQ{^G3aX)Wx zJ*N?(T4iWuV-uaZt0dsKLyL-#mG%0`MP0+k_u~iq+1T#{O?E=9*E}!%a@N-HYNr7w z*?lDhT7sTSu|Wv_D|&iNAUWvj5(5h9rw*dAQ4vy{LDwCSsweSWwYAo&4*7P{>hBx| zB09G=O_A)fcCH1hlDu)j%=<*tuBnJuSW0b@?O_mpq1Q5Ct)&kHW~T_NQsUBBAg{b)w6`N9V?!Vmdj)%(e3Y>@@F6``A`;kqPHip2JRa4{v@Cq10$i&D zZw*L6>;Q0?)HMF=80$w33PJ>Vkxz3P93APF#yTd}5p-rMJx=b7{BLfQn36x4wKP|R zsynEOq~3HBARz)Bx!6jd+ree(-JMXPlnNYHDD8K(DhEWhP|H91y1dV8wROt`vy0Z_ ziy05c#>PILA~KA*u;w~Lk1;fGRUpxJYQ-hd@t$9btj9A3#-`WDCkzV;nnEM6MzE*` zkg4sy?KtA}69@8QSzyR1W03K&TB&F1%8XACGNU4usJyRXQNln_P=fBd^I)z^xQKm(2mu&>Qs{Q7MPRC$sp(;yzLsa?**o6R8&c7qG}J!RbFtAYWy%QAWeZ}1x5Jli7&|9ZDliMN&ftaj zsE`&F0341oL_0)d%7a@CZSY!57^(m_)fji2yy+qAs4@wRO_gt=7m_}q5(%+Ec$rXH z-gq=(ce8(<=;Whn3XFCfh3NkI-XZNzhCt1T5P|kRB}kdbbqJk+WzjxWyFigM@0&@* z>g=k&1CyAk8so{J$;(&`MZr1S0(k`(FsS?9 z(#AQ~`PrXIbI@LA#*%Q+6}_>s$vm@#9L5VGIj^U zzwSCX#Q4(=#l;xo-$!?2;>AKCrC=YU7eMXh9{RbselymA!+Xzn)#)ikTu)DQ1L zdM!k|1O-x|F|qUh|Fi)6PwgQnyevNz7_!qsR(r2VwUq|9c$IhO>#2Zr&^FMM5Tfg( zzYIIboCGXss`7Y}9~R<|(x!dmu$_g-Kgd6eWGh%ZdEX}`F_m{ISY6yl6}c!5XzKH1 zY6``7tX3|x7Z_nV{cK+_gd8p@+?M z++mm~qW8b^TDH4ySRsi5mGwjnglZVztbXFzbg+>qtGO+E{ndK@cQ^jO->|q2=tZP6z0kx44Lp|KH_!tOm2P5MS#3%B{b>9%sLtAA*c-7t0f7 z4mZ4S+n0C_qeOx0>A~gF?|*;XB9|;#be=WRpY5Ypo*as3?=v8p5qt9bYFmn^;pH<( zt;h^fx~iJrnUwlSGR#X^6|T|$qRfhPSbQi$t825O9nRWmB(9}(Wv8ZfL3U58&_w5t zO6c9aGOKbT@2ffy&(m)Dq#dKn8;Xk{iP{gtc$v)iZ5u`lWN-WW4Im`wI*YA}eE+;U}xAkJ>SwY+4~B7YKkNj+R9zCrVZJGUaPbit!a=CwP%Mnb;?j`mcyCthwS+Iu#U@> zn$mA645-cmvX#yl$WV->-pJ1lPFZGpxqM7CsKX+Ktu&~6mviL~idjCNIFrFmPPc`1+|LFeocW>3UOv>H`;OFRf7T1nUP zBN~Ejq_Z?#yr$IDG;yf(wI=#7-i#ctCg+*H?ES3%ZLZn|1EeqLmT>~}2Y3c29f&9_ z>B?8Z9djNeEJXi>3LQzDiKo>2tylH=UZ)!LZ=H2heBhF}Mp}9J=krmJA9&Xlagn}= z`d}|qbCU*2V*0;QGSI@wcR|*(v>?oYGv#}A7W60{3g%vXEvrdhOSH@lc~Ah!E!dR) zxbVR6sGnBz>w4OlLhOnrEsdSC95ex1VKh>#BfU>;9KRyu_11iy!(t=PfK-SiWez3q zuRqB@Qzl5Q-`P7_ZS=y6K{xj5!26@;ndEPXOebRw}V1FIN?lmJ4nNAj4++4ibL}Awjef~UqpWo4x+<>{&nOx@H`5D zs;wX7e1KG!+S*#&SZ2f5CYyf-2-vwerX*b;gl$JNk~-yH_fbYxpoye4}FMn6E&|EU9)~SMVyAYgslJSw6?UI9yOE^sjNqK zBn{p6)?kIY-^(Zd2P2t7NCJdQt4;nzv}TM{e|9AXw>)j6^mG#Z5OIM^S2F<~A!205 z577q{1)3sx#aryh-OIl=$3Y1o<%gSBOqMgd5NL0mj2o)`TeU_0+A35k>@e96O6T&7(xtVLWiTJxvKQa56Hd(#K?~G=l-npQvDlu*QDZ5NNs^+It zxzndlm?sYA54^9@22*)w3SjTDwwYcXq

($t+Zfop-e}`*puRhEQjkP;ZhrGH(rt;sB2B?_~Zz4^mDlhmzTtU06{{H?T zrHhYmXlk72G*aeoQ+D+5KgxycE)ptB*eK;y-70sE`pF-#+}1u&Gf^d+jfJB)lp$d! zNIsB8%YHcoF0TP#Tsx%68C~_G>d{U-Jj8keMmmtdv6eJ|M1Ff~=WJj68qB95!36$| z>~C#Xgg_@*5${dqe@v4LJArV8JYVoWv_}?ErQs91xgsD%1oAY{hv2T@7_g9KXT3Yx zkz(zs|0}w)H{Wsohp@1)@ZAtyrt;K{TUSThw}$k!L_ARN%d^S>ko?teda(3m9AZZ+ zIOXe|pWI)*?4#QScu@-VW1sE1qlF!*$bhgfW4iudf|~HiWe{*v_5L>Z{S*Z9omSU= zdlgEkIOIO;x%>o-Sg!vKz|0PdK|FlOdBS5xI1c68p?bW%8`JbUqqTVM@MHlQa2>fNqpB$HZ=@TU6%^%^US8MFf zP+NoU0U7)R)Z5va!gleW$`>~u&{F5QA>`+b?fPb|BH9GNfBsBn%r>FSYaGB?;aG%= z%DVIOrGxMCx_e6g$I#E!;|XVs%W4 zB^*J5L(1t-C!%KVnWU)FPxNsb7Mn{pQoS?>NYNFPsFWd(BX_*1*Drjp=QMe}uySEl zUi+vntzIcU%YGj`U|eH!7`g(s>kP~h3C8LyUpXa&h2w~I0L<~PlhZp! z&^>HF-U&XYwJlS%JzP{y0+a;63JHuIP())KP*%9es zD|~$@dS~!B5&|vbdZe(%9MFwf97G3k(hXOwl2`O1EzSKEKc3`XfhP!je3PC|vr=MI zu-lafWE{!|gnPtrkWmi0zy97%=?b&9ZV*lOoRoqgQyJ0q(S>$=_zIAH;vkO2`@fVk-tWC?6U})BKp+Qjbs>2{F?fP+3+Ola%uPl zEF)z9sJ~J*v~6o|>aC~gj2!9!xWyx7LP5xCR(AGmRnQAyDhF}|(K`IRsfHzqjvo#A z)5_oPXe``MkyTVw>I|@esvdCY6SK2YDjs}8Lja8J(zph~7c!8tig}I$u4zLfqk<+# zCdf^+aSm>)H^=*j+i0ftQ0FaX0T5zPZvg#KqvtL<(5dxN1*D9R^SIAZ;O3SYm3LNN zX!r(8f$5$9kEgQ?tFnu>wj$l#-6dTjEgb@zZs|rq>2B#px*KWObVv$_lynNx4bt^3 z-gD094}Nf60(inKWI`4jR+pR*AQ*qweDi!JXq`oMk1-;^>W?z8r9g3%^E=V>3 ze7)tiE61Y)9dKrYR!$Tu#;1d8rxjrj%r0}@NJd6&;GD6p!9hra3|;S+5^ZZ=jz=k7AAonIaJH2ei?>g4|cyX07 z+2MoP2p?OvXzzLtjH#&>u+maolOEv*QN=6P)zyK;g8cqxncoU%QE&TR3j>#MxLF_& zQYR`FH;ZtJ1G*EIyIFD(S<=Y)6^F@dVh%ErdT!otSDxn`m1#3|p5vBUfA;ME;yf`WD&c1Mx2|p=J_+=CJSkb3b2i8azk@ ziLr$LMu&JZ65aAI%5N}&BK?o=MhexvFdieCk*C<&s}RTw$oGzW1CE#CMgHt7Phoj76%AOfzjN2ug2f}~A% zH)*p?Ost4=WfkY;1*FXrS#0-s?g3#6AP4|Y#IIlPVN5ybS#6z#gC<)I>9OF+(~F9T z@CgWf$DZis5Xgk}Bypk7cPXz}=V;sD^ z(YS%Y2W{$LorDKN%D~!D=Ig(;Oo^z)006VU?t5TBK@GfeY+v=BREc`m)}$@Aw?Iij z9j^!kXA+=XY%2--TpS=zrzoANjY>rq;w{o1s-#qyfF!iKc^geaN!!sO3?^3K02(r{ zKUiseE)4)KLROBIyrioO0~0L1lRaA*D_i@#+Oqg$yUSNkm`GIMw%Go-z+$%|Bnh^? zfeR$y$%yH<`oS~4G}*J8czg!>iQ~T~1JwfqqBFF$Q!RZR*zCIDU~uTiQ=otZT{tBN zh86oYg9V!BGPNp*BQ>Z3wE{qgumOm6XJ=fA{Pb214i(9J-nnYc4Blh!Y@=s1=i(Gg z9r0mb#vD5CjML}~c%|%?Sp(eo_0aj)x`)Q&LDzltgl^dPY)z}mQ+Dvqu z>EKN&4r~w(iULy5+M}!0tS*RfJ+w`dG~U$ z7OeO_6N-o7b(x_(-snLSSc-Lw^!?V0@Ukx$(n+M=1GQN0h~7UD40$==A@VdK#ZxnO zN^}CcjK)YInCXRaeMK<1F*CGD^dwe=FBh@heNpw^&BFyq=8gw^huD(Dyb4BDfnlXh zA76bzg9XX$yEUV?KuN;2GPjZ=x@|+9&zEw@L(JdmaO6h}*D^HO{fnV`c{i4S>kvP@ z&f7eSQ8&S@7WC7Wn0v@hBsO6&qn!yhlP=dFDxb|#_9URxB;Ty z{y9838Ltomx={=dW}^WKA4yUZf(`zi+mlSDk^1JR%gZ=-oyqQ=d<1lTwks1 z?ZIKeqTlQYa&^MDJ%Eq}01&<;1txDMdoFj1)ig9~=e*wCu_}R7afpb-KoZal41_M{ z3fgbB^!{C3;bdgxfh>@g79CK21D-9wUUKsB#i-{ItJG=(A6I^(l$I8OLt{^0pG&#M zS4i3CdV-hWnF4u3bj?g8X^z1*Q5N(UejD7Yr81oYPSkc2uRxoPyU2BDwQAzS0d(< z<*)I{8R>WLXg(0G)Fs=whB&wM0kydOAz8CSa(_t3Epx;qK49rvIGzLD(J@t;tn=Bj zDcF!eg-(-bz1DYXii!p(lV8Ud)-Bl|3H@yvCCb$E^YXClby#1s0+#Jr?R=nY86gEl zL0c066&$yknl|*E+v*Fz4E#G((y6E?!yxK)Itb=_n6dr4B!s}yoi0H&2G&`M`7`l3 zT~8O(J4i_oP-B4}IpLfDP|8|(c!WDMsi(GjZgN$(-`9Iev^F=3d07&?bvw?h@Oq`D zHVYJUAt7(T-2t*&2~dHdT~N+DQw&sxVNpNV>!_$pZZ5?&ZqAvzx1+5gu{&P2A}AC; zeI%dPLHJALcP*OuXX4>3;rRH`Df#nt5=^R0nAGAKlZ8pq|Xu2fWJ-y|lVk9`4$Bgy5nv#_c0<6~Z3xQh=xtw$y zTrrLAu}0QZh|F-#UyWMMDQvN-FqalSzW5WhP&QQpgwUT|T^(suYSV1J9B>=9PeZavw z3&ex3BTe$YP{1lGDH$s=1}rQs$zomu$IUOJ<{#iEf!*`B2V|{7sU><7w!OT$oO@Hb{$OoS^FYe($#kG#0S*)mmK%TPGIh6(PFMS+`4+ea zO)%mxEiT4{nCPX0;tBlc`3_r#>GF{f6aq0sPz?eNH|bm7%nK+?I*`GzXg` zK2RtB0PCW<a*f|=^ zQ4C1<9M7)bDn`XRKOPW)vW6RwxWNGckq0r{1Y=pIcU8-%4hxD@Iop%{C_wH5bYv$& z>`Cv@V18u^4y9Bs&IvRF$%Zvh)Wtz2mlp@YS^_s7L#;FV;9TwV*&Bg)NFZoehP1-W zKgcqc*nzR;eqq!xKTs0jAR+eyKk{a0&W#d%B0%#qCwuK1-4>p-6Vd}mr=wBq)bFDS z@4S;gn(MGv55rRr{~aCSG~;5^O9sv_a)XPdY>B_A6#a<*P*W2unHN8d_KG5pN(GVX z3j>b_h6&VUTIjVHgzi1Hg_R;v;Is{u`8c*o=lAka){Ga}G2Q5-xX4sJl-yge{(ZH$ zhoc%0qzq=_O`k6mY*d(gz02$3ADSIcskhnn#VF(CIe|;{-_L_qY=9zFQ=0-F=o#{g z+x-Ec+uv@ash$nRoALU^&dTwYpMY6-8k&`zW^9k<+gLp7_jnIiTx>bCjY9g?l{84F z>8a6lBBh^w)w|+I98@-=>h_8+fc_{dI=ZSwvCeQ&k2S^pqLJo;v(ghD#5UA0W4+mo8EXGkwTaO)V$gCP`5HlDSmVdfVL&AnzcyouCHN%M!oTLsMX@} z=A6K0o8b+>xdR|)#qo9(XK3=KIbfjULu8ZGfv*&J3d1lHY_+PLFNQa7PYJfRISGTC znbb&*27ua$j?Ji5W!2TfBE76^=yp!J+Mr_fW7Yn1TG8w=zrX3nE)Bnj4Q6}LQhLkX zYg>I!TKO70Zypf108@{|8^Aa)aK-u`8(*F z?#=82Ye}nMgNAtc80?jYzuVicn7j08uQd_!ZQc6eJCDq+i2p8`C@22w2BXy^>W70Y z^mJwE!DV7FdGG4zXyrY?u8faN<{yU-eYJ4yVrfp*EgMil_mE{w<6H~#5++z2DPS|8 zQ7}@IMX9bfd=a^*SLdT1392!glQM}|M6*HB#7i9S*2s${_$ZS~*8SGQo_)8s!I&5R z?uWh(_WJtE1B5+9;}u$93=1%TF)cSCi!%tT4y?AZF1A@0Spy#XURr}}!Ma!O54QUV zHJ6^tNA1Tv(8Z4Jjt$i&VGJtR^_2Z{-)-huyDJQXSd5-ZQ+!C3?XV&LFW%}7Oa0?F zB^Ose7WOYN3>IT?nCLd0;Yw?mL)&dY_?N}(#|qqJdBUCQm2Y_-ADS0F?>{u?QnY(! zPHU7mB2txVek8uGJSxy;^-U@vdTzP4(8{BWZ2aX^B+fZ7fFFy&MZ|TYv#wL0dqasQ zp{{m+~*3OIi_SI3U)Pq=CyHmW~4|qowCx1Y1%a7~8F7oJidnjg73j zFNcZPz|qHwLf1>-%Y>FaVYN9mpYIeTo$w1yka#0iELe&$ryx9Nd~Vp>dtK0UVaUO*ITktTFct*e5N3QPI#mcHfm8RjT#qK>CRK3H zKz<{{Sgnc5xOWoy8lp0t{<)_}8<&yFF=U|W9h&Luw)rivQ%(HlaVtrb;TEFM!@V*aeY7cf1^Has_({kPR=KVi3`En6(%vgAIT>g)&y(&W<@~>EpCZMF zUsQH;Bk8`5EUd&@K%f&xX6^x=Cr$;EKQu|`wz&b4sKI3LmCuf0QByJ1hm&|vnKq!w~0k&5k1+Q}gD|!aj+wkMCt{Q#az_@X= zJn8V^Tho6tWv3^s=dlp%Z=vBh7@@@C{<<%=C`PuyDTr_$=7~@{*0PBK%GYobf6{O{ zy|Bi(tzLg`8i!!I(L(8DewTeo<$<~-aXj^qN7H^td5_J`G`HA)T_F!%A}>earq{XO96jcrPZGbY9~k-<_S-$9XS-a_(^DzCs|}ho>g!8k*`623 zx-S*j8{mXc!-}N#QvQj(Fh{LMU<=WkeD#WlMu7J4_7grORwmyuFH$Q`N}N>@2lo2p zz^7r_&sLu|k>qrlgv{+3`&a%?3s5VYDZ=u8-kCAYf>>>84xV*TN;kJb$y6Q-2mTvO zfT~sHP~R)N!vYtgYMTwhutGFL`Na3FR+WrMk*N^%4pb+E;NkC(sBWye&JZT3i8sny zyb7i&Z^7@0oS{cY4xlIeW9tl51o2-635m$1t%2YcsLE01Qb45@Hf8xMH_74-TB+Z` zCtU=QmhYc=O_M3;lI~9Ztil|C&3k($^IY7>7A1h<-U*B$CwEcbHhru_xR|$#nX(*v z7R4$G_FoJ!fuFa(A#2HHN4hiUVlXhkz=#K=p-$-;>5N}levXdOn5Ul5CdzqwwFZpy zzA<8*Se$yN=fwgten2;VIsr2XlK$sUbfFp(;E3ty>+AD7tX@51w`Z%O0R&B2#*5@d z)xOi_FkXsQ-DejU(Ea=Y!X3eE=P^Kp1Agqh#n&_eQYQKFBY(1UGD%|M-}4(~+MN^# zXxj{frZH*_hv*=veQ{n2M<{_W71!MBuOZ#ug>Ut#pzK|4R2)iW&`YSO&J)kk?Ai$*9zkCQOEECugTh zBZWnLwyMLe`chBL!^9#jKB&_9j=oYgdtj1H0#4R?DwSsrkyWE#h9+5QsS2|mhP4gZ zJRmMK)%DbrJenxL)RuXYx>&fK^8FIJtR7rB3%04Ato6p7H4J-*5ff!0%v0jAFmMn2 z`D5&QG$&r9YccqX4>AO+%XE3XI@Var={ePbitezrj5Bz%o~R6lxTbyU5ezxrfx92Q zBjt;NH?)N=eyf2BGD^B6JnxUO_n0|I(K6!-K!S-Va^BG<`xN(_O@$Qo-N~+D<^yW7 zJl^b;+`z_{1@LQl>(4>~O%Ytn2Y^QYM(b(Mh^B}ZKa!>}x1c@@6HIi)sr zu+$Ox(f3>rT)V-D5rIMgxJ<#ch3~VA)4!cXq1&B;=iB1mKO*3Psy*b`S+2L_7hYbL zrIH0)lusUDwi18|%yycFxAW|&X=wvjufXdAVHFgB#3$@^d6M(A^|8i2(zzvqGjwLg z0t`w-|FS==v)8hsFyIeDU*F!{re$T(rOBWFpa}zDTT2@o;!0j{ZFCnB2LVxFVu~9H z2gbu+`qz1yc+M{ukJmihz6z|Oq?k`E&JNBTzHJfVd|yj=n|#*^Y$)t+uMYY@832}{ z$5kH~D=!kw*F_ig!y(_?wr)2}bL@ z18Vsod!nQhmAZ&=HYmE4JPJ0nm~q|g8**#@sc?oib&^3Jek!H3q81|@zF`Zl;46fy z+*WE>-9aq3=20|}PNM->@}{cy&C|$0C;$M*F={36r)Fmrgt95g>zR^Az@w9?&h=gN z&eHE!KZ>Qbtu4wlP%A3+x}Kb0utB}Q?s(n!6kfV20t_`tW15l{o#@``?0YEl*%6f(3uJl z2D6_5Kvav~h8$B?0ZX4^QpPnKW7Z=NwsDYMkAGl5_1N}$_ zzkR+wbE0Al7q9T>`XqJ`2|vKTq&E9zNr))(eYG4rS~lZKQ)U&I(Vd!HkT+H124Yd* z@0pPg`9Rm!19m5CE4y634{_Q*pQrQB@ZnADB9nOeBADO`zB_c%LeLyP7)F*^^$rrg zy97C8QRWZ86sQ$PrsbJ?la;vTskNf@vXpr)@Tx(HrY^&Zq49Tmiq7FT3kC~*6Fic; zIt-kOXtI9kpT9;rE5nH@3yEA5DjlG1W>bkT5Z~p#z%57K>}a`~x!_wCG1I7hfo%r^ zJ8>}cKpoLHD#U+2mgsSQ|MaG7!s9UcE5Hb0EV};V5`HtZbifl9ODZEjR5@Y1Ucn-C z+oz_^OgEDtx8iXgtt+@n%xm+z)M@N@R8ZIt&Fxf>0zMs*Yp6A(%O@f=etZRKuyep4 zS=v8g50PYMHrMEN<_U&(X${lZEjZY5FMJpN0?jkz%N_^SyEzE0s=#rO5d98{l>zhsZ<{zygHdLylML+Zy8pMzuR5Ua& zSptI$_jW`0%Oen#dN0Bbgf#%(q@U%nuOqU954~6wss0*s>59t~Uj2>w#Ud2snLSVF zCMM)QeL?}bBmnq!4$Lti&q5r0V=dR383E&Hdl`iQh#>)e6f^L%;HWoSqKat|k*a0th#?X{o-z4*?#t@&YSdeB+LaU(1U+_&a1m*({E1gp>|L9y zEdVh!nj1)@0I><2zks_7C|zj4xI8m8`ujI-nq5jFgYrq`#|Ih^*?r+LwCx-f8@qAg z32@o;xagd3`D2X7l(f>VJstV?}agL5baff%LbsAKNOvvO? z>#DRJz8G1CWOu%r=r5>vx!xn{$y48yQGMvL)!nyb|Uo7IzQ zHp1MK@r}4TnRjRaK^UYKPtE<9eW|-vRf9eLa@p-65_(L()QO^;i!NeD2eL_K;}T#T z0WHglHZY$N&=ml9kTGT0dp~2frdJI+CphR{{2e>54X#G*F37cmYkzfg&-1SK8>E7a7<|9PI1+; zoGBM z8o#q7MZkg7L>-_e>lpdkEaHymM=3^2+{=ZQuD9^RoJ&zD>XZEmbNW%9ZW)Itckgwj z9#GlmmNSn;o`c5IM3*i9?r-97_rO0H{6Las=Dq_GzOp3qIWu)n>;GrJqWGCBD9nlbNcgo z>$^zNpa`i-8bvH$@iJ8JmuO6uCt z@^ygDFSQ{CF<-|ADGfYeq6D7JeyH88+a*qHb-wJ7F4xe>4@58rbZ(Ipz#Z5=ce?L$ zY8Ro1$d8k!Uq%%{2ad*U?=v}EsUkqVuF$HUoR~-QAKU=4ZUzo zGL0Qn`1nyY#XQeCegE$G3gl%-(qiZN&600fluQVdlMWi+#TdeTUn@SF53H{!)b-JZJdh6^vi*erpwC2CK{cN-C95sV+B5x_g@% z7=(#TC)JD3Y-++=P3&-5pN`;5Bi($MN(lG!hFG2lu zcS+I5gT|IAx(zSS&jUufVm|L^K;8w2(dPC(hmV(yGHAP6JHG?88TvHG=-T}tE#SYz z6c%38G1LVFwNRAf?4tUL4Ee^MgBFlL%!1c;OX zF=g8;9wY602!#wR$q|nH|0w!F__P}2UC74w~7d$rh^z5NnNL=>CieX zOh(~!Ym&~G#c3@HGL_=L4na-F41VXKZoP&mF2{Hs98_hd8qKIy7mrT^8xbb^VuqES z<+EwB^Y0;ALydevYy`MhS(%6`4X+rXvALRb?CPY;VKSbWleF|)KUp|Y%Z%S3zbOgu z;0-LkW{fmH>isl)$Pbv}mNz@ZlO{gt4?z_;5{O*$q{oKM{>9eXDRO1%V7AL4%pGaI zY^b+Bxp~#h$pahnu2x;zVMu}-3`EX`kzq>J(aM3O-QL4v8Lk8XM{hzrGB18gR#w`} zG69g7Iy0CVnS+oMN$Qv+@bT5@=zXB6NAn-t-vBbFJUx78=$f!DbNAq z9^mmJQfV9Nu3u;^iIk}`Yp3*+R6k$`t{7Vl_2H zQ3qcdAn)-+`!}2-i%Y7VvLxgdW38-&nWxhZNQ)TealFip?^f1eo?~44ts7q_-{poC zhsd5P@eYz#&WKA&roj|A|1-TL+u4cYwR3W*Ox)}JqzewB#_YH(nZ^fMnL)`C=5}0M zvXRT&^tIDuxnX$4kFN|F1`;>PQ1a^*7n=CZ|g65 zX$k;>c3l8Hr01?m>X#?wnPy=UFj$Nph8lk$JkBwFC)be}y|H=LH1b#Am;J0pnfeR` zx07C1^IdsHXbgyoZE5$&tYlL?+Me9%0=CE*(Z~y^UiApn{_Lj5ezjV&qN(ww^zo?y z%=&xqZ}^Z`k*G?OM(k3VNe1+HV>;&mlXn%m%riRLy%*iN+A8Wd*s;JIgqe?@q+js- z2^^)RJDtQK(rEYcq8|w}l>|jl7P^lYMKjaB><>~VW|YQzjXOX?_v=fzieF>wX{%wF zO1V03I1?;9Gr6%Ef&7NHVT1@=ugZ8x-y|HI3_`zI2>KaUu+gb%k8%NWMEM#ws);S$ z87GBrpmr&tRCE*;RcwTH@Q@xFtXW`rS>n_}IN`=m!BMj$`%FcqhRdp4`H(<$QoJ zeYxTN61U=kn==dt~P4$0XXxM7HofP_qmCfUVB&U6v&QDD`{C9xwm$I>B zk0r_sOE)OKvdsz&r7gY1nv9h>&mDeYxcknL!eksvyH`ChiHx73 zs~GR}7LB#t>JeL^^H7%Q&Adneoqg~gL!Klfk+Em-yHav-DOjfK-koB3f!(4)n!cu_?K zg_uWqnvaxGT3+x4GrghgYUiL7pAlC{dQEFa$nk}SJZ|MM)G^|lRe^sDoa8^=_H69X z4LZoNqwxHg>!#q{o($8L4`WZ>PCa9)IJN)!h+2=#l99}K_?p#)yRh|{v@nxYUX`vz zTQ&{0`Z)GQU!`j~G)Hq;%*L?1NcvlOPSko4@oNYYY>k2;wgopIbBqIo>D%eheT46M z*(m%hI9Q%vI5~b7??6BVB{S@8*w3Ha$NBp5-LbhZ`Cr9F2eqvH{w}?)Nf}e>;1keb zY%It}{-+Ao>f3gg8!Y|@no~+Fy%}b{UqV6s-%O2>n+&Tr1+@3MABJ0VT-vl`@Hq6! zS1c$Isp2AHcCV3SBl%)OF<%CASYLU)(f`Gz$tQMqazKlIJl3HX^R8<(mc0}c$s$b! zs`^Zy-n2uKLc~NrB+s(!eR&!3z$kHYn2bC7Ny>sWgQ-NK zqxjT}M^>4quTekzH0nE~f1qJm1AZ_dcgxW0d&79qU-njFIW5%3DTxH~t}wQou>E;z zkHxLAg*}zvHNOn~P#2;QN4#7Ae3K0-%8CZtmpoV!1o&iz(XJKpGqp&b9qPm8vy@Jx z+GDlHUjwV+V)asn(yeRx3Hvv$a86Q6tpxFC%W1yA3H^mJko=9Swqh`sjus&yIWqOv zsOSClk9}fDPiI{!8K zr;KZRH@6V3^arCnYmmLIev{|8buNugA-B3ksa5^k1j`>-SILz>Uc+9FdG+T>M~^W> z&0FR>3nf*5F9w3}z#b@Tq54u!Ahbj0R8KZM>4?v5Qqn0BMGzgr+^7M4zTdgO03SlY z*#743wg?DR`m^4)O-M#0t)sl(oY!D)HZ?Z~{VrV`iw1>0)Eqnb9>4or!`~F?Vlbb# zy#41&&m`)vJZS{1`+$MC$%T<%BHq-PVELM((@$=tA9*ZC8g=ijl~4ec6PD}P8CGyl$@r}Yg5U$ za7At?7qBu}5s9+A20C8?MhEt~tyKlTL6XncgB)Cjh5TYIJOlcsy^O~7dnE9L7~_lIe%Qq3h+C6HPFfPgX4 z^oig(m|pab@8RL?n}<1^{arl*kE{DPf^XYoOCaE)SkuvjVK4=n9!1)$t;e27>o^un z70>&QUzL#KhL|euB|2$tRaMLe2_9;(5wumQ3Vm>L))Le zv9aTXOIzF7SiDZ_<3R;hAZfNX?}_MT!^vm}tdV_H!N~TX&pe7fCqXB9LtHPVOg4&CF(wbA zv8Q=1uQlKU4e^J~v#U@}8&f>xQ{IUs!>T7+C~fS&MiW}Knp~TggM6f({v0v&y(zqi z#N9NXsIoRjzbILv8BR}a2+o9+=bt0fJ?i4URt*euKo#Zr_ji9A;SW6UYoJf_1a!9xs zy;WPxHZ{7#eP>P5%UPNR53a$kARy)Yhd9e@uk0p6zvvXGE4jyklcC4%wtI&mJUXeF z%tuk6P#Xl3rLP+d%0Z@TjDJ1$FSK?KSEZhQEd#MEhoB(f)6rhIT)6ufYHDegR($N< zzN+UCRW8%Mb}@1Q&zsG?^jUZGD=JvvM?_t29ss5r78ZqI8e^_q9;H#G%$u|0z*qB% zp&=n2o=|%lLR8Q^BWm{yiHUjddbHBpQvY#cWhP*DUusrZzr#7L4&QCtqKgTWdR0}fmj;UW5zF1gTp$!i8G1QPH+I0d>cF{LB6}HMa zm0W>K{zjh#zz zngtDqf`O0t5P872hA``UUjcF+y5oHxlj2k#1tKPLD2Ns{^YB0#sRNZVsepH|x5(Ju z9&)@@5}4iIe_yj7#FKD5`2qG&(4(*^Opx=vFF00PfUs}mwGavB_%ulmI+f^?lAClb zA!O85qTHo@opxI4d_WyRBB^_X|Iu>Mm7_9~KJusIbZt3;R7D+cX~>JUS5iJapAlG1 zK{v}f~obrEKfHBAEKHu&x z1ga)f5xv+-#=oc}<$My|SJ-edA)XTcnJwcGVY2ztSh$mG`koR zVrk`!b>;ioO#?3i)|2KZRLpJ6f-?=o9Y$)LAK)bf7$Y#wZ}H9)xc?HYi0ie5b{Uh{ z(p0ZsNT&UR9kQ%@k;IsgEGU7fSRoN^J*ik=wT&rZl3|Yg-2$~)Hw%2u_{pKhqVqdo&K6&W3& zON?432F>WD%l7MsN97BCgtdz*W!6f9w!3fnWz6KU2vo4GkAK2hMNq))p=$s7m9M(e zT@*__dU`PyD(tw+1NZBS!42_QoR%|-!FmXNBJe6nGw=rKOluH2j-gxGl-UBouhka z$T&DS|LhijZRB``dRGDOM&&4iMZU?IA6v0Fauagx#&G_N%iFFd2fsPQT)VgW%-(+7 z=n~YmPRT?q*}=aDbXQAW@bU3M7!j$QNJlh|g4dy+mgZ3O1Y}(dzB^)UGfQI^a!dpV zQQ@*uEv2KTpv@eL^0ZV5$T5xMmJ;rZQfu}0@S$O%1F~Ryq&+fa3?}-oj?bm~zJ9|t z9Yz+v1!BzYRTspVH|FqhQTp$YWqnmj%m1GIDL6PdW{8(Pxm?_~awy*}E6H(X!P-8* z|7R&WSYVwg;1TF|J$RhEOKkM8Tz#@vQJpxldE)sQuqV&G_w`okv7_ru2hm9vw+6 zBN=!m0Pn%2mhtZL(A35={pJQPKOWKO@r($>Y6Fix#naysGcFy##KK0ex78Ab9-bV9 zM1me?WWW1Ctqkak-=8JV0u#|z%c@5Tq^0LA(Bd>SL?s1kvkn&kHXhVo7zBo_bU4OK zKN4X<_4g{i$P$c*u*r1op`x*^Oy<#uGFd@@x zTgmq(GG}sW61n?q5*q}!S-5zFyL%@SbYA1Nle0(Uubq0-vPOp2a;6XvfaMi8nB$%9 zcRScs66eYFb4()ehoA-Z|)4dG+%P+$_B8hH6l3(ZYz%GV`e%!r=^@24M* zStap+<>N|_9-N`t^j$U$?%Iq$myIFfRnIHGp!t;kzkdacZugES#^<7*`Rs9!I7QGA zTD{&7rO6n@1ml!4V!ty>zZhdeRE%XxOb{E}10o0Bdfv@L3%<%11r}TVJ6;Ge*HLG| zb8~e<*|bRpcuNGIH@qURyc2XZm_&rMeO|fgJoFMUC{Fz?E6FZut4NHWcy4hI?#htX z*n?O5vGanDjLiPN-{>C0`?zg&b8e!*`#fW{52_JE{${wPe*4E}ox6}AAb4*8ZX$kq zF|f@A9BMDMx7lBXCWQ3R!VP_~H_LEDPdu0Cn zc>UDHW45O6h~R;eWyxaoqgxOeuC@$u16B;cuWAR23Bk!J_8X@gFK3t z7zghVODn23uEPripwXg)X2wq8gE{V*ih@dhM(ooz$wNbO7Tfraq>-YJ1)_BEUlO$y zR@`p&G&FV>TM|ohe6lH?PWqKLKqeGulgq@A6@VP-q0|3{ho!{t_^WE$8zhTbxeKs?*YF#r-5#8=MDBuEx;0+>J4zP6!R*47K5yu7Za#(!fDa6I$&1zAQ$ z2&eZ#MgLmD4?6v@m?~7W_1;i^qMJhp{0cD*4S+xf1AU>#ma$WW)oYxdJdUZUM0FMo z&EMwq1JZz!tzA3>64S$C$xDhVqX&vD{#!dBs%Z>*_^hk0(Ay@)l!zO$@be)s1h*g~ zmZwZjOA1&F-_33(P)4oat==&7{6SPx!(ohQi^^1_zd6O|lJSyZ7?KPGdM1GF27(@j zM7jEA4rLzv-dZxyR<6E3BWP%JiyJiqlTu(_4VbxfWi#;py4l~qn3870-#MU3if~_^ zD_V>*nbh`65LYQ<*Dl#?UV7ye07jpw;{s1d9&ARPJ);@D4GlccDU|ey#%7;^mqsVs zV=L`-eOm_h64m%2MbP&bz=8jFXQJS*vj;u>z>ry8rhpSE6)nIpu(GnA%$dd-r_&!E z9s>7}!u5j)^y4ZRvTXpo36K@6p9-*4IN9}n`20DE<=9|r?R(G*(A##1Ug_bf@ZP}a z`qgz;(Uo6NkXQkxHc1MZLQW>1;02pHk%3C$N5HE)9X&l$YXBX6S_{h*aXEYC<95@* z0_Iog#QRv!QDp9?{RH4p=y$aP8#BmqMn@nboEt1s_{tyc$pp80a}+7`xMMBY+WI}9 z>X+;7t*2Ziorl?Sv-QX0HPQR9CowlS48Qw9KS?(?g5~8bP%+?CRP^p&n7pOLC#Y2n z;$QWCwu=hBT}Vq$H}SMUAF10p*y*l&{n?Gt+Wu6JWQ$7*pHc2`@%npWIH?BCw*thOM;4)*^1 zhYJ$mpD#$9qN0(ei{AU9PiU=!Udpp))&U#ir;T?OFZ4UxaElNuGmwk%q2Zzii zs}!t72A2*{13qBJ9(^xu<@ST9KL3T1)bBwq=5hAhk@a<R%X63J+TEMkHV5PY^1eCEV^Ww+kV^>f+q!Ht3pDGwpum#=Igu}BN6iZk2e!eH zF+H7(p=7RG@ccYNWn3Pst3Ru3ciknJ6`P&lIq##jzxX{UhnV?-)oas(`A zVv;@Us58vy{l0y*fdLChMF6^FtS(vuLHM9U z4}IuFiLqZyO@R~0aF7T08L$w08WbPf{LVk7W;!oNzi@MPi_FY(3@$1%?>t{VKI>=q zNqZL-lWU!>$1Tm6Z@NHN4alk5 z{G=@O9XQ`BDv*3{_A+Gu)R^A!k$e49#eSZURGoo5Q0J`>0`ORVaJISKxNEAiTqPnD z0Ae|Y#P@k+#E6cp8J98df%FdKcfOnb3^YH-2Cd!Ewp!pl1_7xeTxqBvAC^?aGc){@ zv&z6HmaO9eB-m31uMLV`lV6~r30+=qV?rD+5Bc41wnaCd9-kgjM7>YnN>XFRWC;!3 zFQN8JWM}YKnG!qv9I{MS6dYYkz7jcMYqHO{-b1cnai}C@C=7^}GuwyF8$(au*0K&1)Z#l#VF9qm+x47vJ- z23gZjD6L0ugVL1aJZQ1A3WANHGP5;`IN&`k-X+PKol6PHF9nQLXbNrCdjMAGy+2?L z0u0KJA4CJhzJTGls0+p2-KOE{9Yxb}uSaalmGSu$*>Pp%BLCloTaSHc*a&`0RoyH`ly~&yC3Ky&K6G^ytnds*ytG>iR;ogk<|p?En`o5HB%?`*8P*S&0 zjkVJqP`?wyr`$Y7RhL^^mG2GMt5*Qh-mV}bK&^`pa;&gr1Ym-Yo}i>ao9X7!F1;xp z5Vu;VSgrwRapbWitQxu)vTaR?y(SZXpE}}lhup3Pg*0l3qP}7dBN(xj7eCA6pHtR( zh@TT_z)n^^%?uPNAh{5)x0E{ggM)E^KHDI`NCO3$yU_QA1>%T&PzHkRBkx1ot|oFB zl|c~3*>-h6Pl}#6WTv9Qj}!KM<>UpKnIkMC|1K`i`nkcQaP+6LX+`3TN6#M)vcpQn z_ZmA#d(^d1X|9{juJ#x5{gW2yJ->dT+bvpnTEHu5%t7r*jJ*Gxx1TNSA1pR~1r~Ua zhoPgZ3(B?(ASVYxvv4zPsn$@jMw<_W@aa`kvx^EX5DOC%?5Vm^;(@JC?2b8fy*!Sj z8IA%yWr++!7Th4c@;{~N59{X?8`5qb)T<^+*hrK}kXWDXd&|_?=5>9;?ki_0Lt^xF zd#nb6nX=QR)#wl|DS&S^##AP@#%+uZCV&#SOr>Ud$^Gs*gX0w6D?alUKPm) zt^T-npjVyeAw>GtBVq>X%9iOD*7;~a82T)_Ib8s6{qd5%!Qp&uRyJ9OLq}@ojwK)C z4L=e8%7B6DXHsxk9si6rQpX83dk2)~)g=FtFPy$_e_}v`Rbfpg*X5(3>MIuiSZN6} zB`mmi3p+4#bu#dF(IDW{=DA0QtQ5%ernu&&3wEqen7=G0ZIzKxRWGMt&ImOpnuAtc z!^8N(!XSi!wp*4?4|(&q3>+Ec1?pG)y)lCYri;IB=l0glZ{i>-RqCQD z1-s$}j3C1JgF2esp!AgbXQe@BJ2w-Hb3SCSZ$iF}nCpSQE{{gPD=BmGo^S`oF&-6M zdbw%(kucoJop;rp@y+G7((H04-|Phe6C4o#fqZhU^81A;;>VOctMtdNK<%7=+GBgU z`x(=8w)kn08_m|k&(3@Azl0L<01pMyNl8VWA|4Ien6}Pwkv&KeKF5ar`o;EM^cEJZ z3Q!qdq91Jk-W>$mtq~x>8DNOYCXm1kmj;^{VV-Po@+KYN`@eBr#|7e?ZYR6Cw5YdV z%9z8@DK^LB3qblqZC$&fn!^9%>#U=q3fp$Cv`7g^cS%W0mw*C83P?zUfJk>pNH<6~ z2sj`jEirVD?iK~x_%d1jQAY`P`%NnDgxp! zwW^)XU!qk|nb_|go9a3wcrjUNCviquK`H7L7!VQz>?mDFLv$~TcY#Ve9@Mf!KAI#h zi4pPPBSC`O7aUWMkD>K<`uZ&KB@@ilJo$1g;5qrB!TJmnm6Wuxb78>+^HQ|XIqf5B zRVvd6ANE_j*@t`|HCSJa&rQ6N8^dq~khcSIuCWI8K&A)+s7C>cPm<~@EVCh38pc2k01hFCyV-~bWeAC)qB2-xCqjZC(27gCDoXTTzSLBg z02BIHE+Bnkh_{ks!3dGo&VM}N?wsk&1lEXvNWmp076lTTU#70iz}(8m&|4u;DaV)c z;@NQO0|4*=apZ#4N*vkr@!ckA`BoSonAm#RKw4H(@~qXt2AN%xcM=rJNS00ptUpLC zT)%#(*}iKBS|;$&VV|Dr!G!<*{mXnfDhPzG8lR&;)^Gr_-`DK0Fc!d!rHUDtm=M%L zFO5~UQ2p`I!66JR@4>Pyzpix{=(0i1=fPj*x_l5&4Uk6;w~_oX(s%zGPi|CU>yo# zuDyIf&R6zIOHMm;I7B$p5gWZX7X}Ri$iX~IR`8vW6Kor9J{GM%-=j}P#D7o_=R%#S zwxJtj2GVR*!+A-L^w#}zr;`hL94c}D2I=ag({e3V5V{;7fwzF$d{}UDb~-%W9DmLM z_N@|`?60eB2qH$y0W-{`oCIc#Wh={qc z38t$pEv=RdEwZ3BzAx0|Kw^CJ@eQ${*A5C$!m%WV|7MKO4yXF0L-ew^I79FhXsv() z1K{pf!wRyD$FmxNYJaG3B)@luK1Qsy={5v}X@Jf`7`VJyURyd-QzAgQzj6#qYx2yt zt~E>3bsn1PX%)l64BFh?USEhcB@BP_RyJgFt~%-?-D?G15pNMUQxY^_O;HtW%W6ah zpDs>d&OPzhyVYZhrZ!xI8-92+*v^&{G@SKt3aIPM;r_}WoSW-zZI>mbq+Hwjk~}#$ zjV{F{At399a0QPxR1G8_L7WuWYTnAaUrnw*#9o1q`;Y@}&D6fEBHr(Icc_c|yY!FC zBo3ue&)>h-6++vbU0F6FAXpNh9YB&bct|$8pHSV;3}fNyufsSST8d+h!FM-C;*f9t zw!nA6zydAs(fiwvoKMkP!LuPPU`J8MnsBlK-{saRWfD=_u@~0{uU6I?__GBSeQ&Lil zmSK}WUF`Q}F8il2)mZyD`*=kJ0`QKSd}7G=*n^D$-@l`aLBY$#<(482)Q>!EY#N!u z7ka)V&;04=xqgK+LlR~DHaLVoJ9aejIrx5#S-b>L(k{u`ag%9&yT-#Ub0rK~lyyX7 zV=DPg{HXQM^28{OD%t4FmU0_VtGJW=dRC&mW&s)xdC9wt-nMI8!wY}Q2>`S(E(`^H zvvQa*Z74G%=gAnU4E3{Ru1=yj;MHIowe+`qnSB7kD<;%hB6fXnbu&}`-yBGXqzhY$KPq_ zZ&**~TU_JdP>Of8-C(J!tM5<>e%(9d66Nb~GQ6N$Hq8r^daY(<8J#8A4WNJi(y}oQ z2mg$v7MZECli89v4p!%HkvR-SLlDI72mZolHT%S9I(^Hm||9EJ;F z2r+59c@Fb}7c7KF56Di2WrwB1=S?T0rQBXlp}QiUu%kIffPXx^*pe-{zD5BxvL-+9 zEV}wTe~#)70|NIooi4a-4kW1bK0;z`{(jVO2xcmB=$@giF9wPlqIWC9V8Gsya8Ciy zhd?~h%MqZt-M_oyP5!zl@SwNjekKLj!h*hM594FCO&&ArewLU*)rZxQJOkE5p1ZHq zO5-{@I*TCx`0f3Gz90w<+XT%(+oP{%VsNivB~j zwnCql5GQAoN~_vRt5EY7-zx#+@2r?FZEX4!Rh;%RYrs%d@Zr@@jb9hHkKfwjaPx|F z-yiMY-6ALM8pLjQ_VG(Z;3i9B`i+Q-+MRSc-pbk${M@=;2Ud!F4fbbPXrQ^U&Em&s z+Zi2WzU{EDfe+1{F`t-W8P4SWK;FB#Ppx7*`C{L>s_7-3>v6qq(|v5k?BF)$6TiLOL?xFhf21#l2hB>c5UPM#kgJ?#;8A@%Jk#KiDM=&8K=BrmxR^r=woO2ZFi>JUUc0%& zkluT|)ytV9XXfOpKFk*$gd{dy4VHdb0a{ck)^5Ny1sqC7sc4IGDDU5qP?6>8apYLB zc6Zz(<}%-OyJVne-NPFN&U^ZLrp~3_=al7@5untn>%D>Go>jAdu<50mep$b}xyd^` zY@v*3Z*XqzYwt@vv{dP}?ftC#Ic$n0{JJTCM$UxgQPrcL-u>S73LT;!HH}}q#;3uQ zyuo=Il9;ID!MklP zt!BTv8If|1XI$Le12;DmnVI^`P*Q+N`Pb#&=wpA<<6#~3L~sJQiW0EQ z7HVL!==YeVgY1qpzjE~#1G@G0u!|d?EYBqeQ0_St#qB*hyx~>h72)kntJ;3ZGsD9g z8XkuKJ-}$b?dP1Vwn1lRCOjG$9Y5}{^LgVfjoest&Gv=#Q*zXvym+t^u`vVz(B;F^ zHEB~a{%&)g0pMNO(rZbXe0+^yWD%ArwIy2eapOrcH5-^APfLm4c@}d;1LcF#IC4g5mR*&DQbDR?6-DIJM zrR*M*_3PlL1R&$*;&INHOeIy0j%MFC@G&p_Xzy9D&Q?B0UuGKdzgmFr6|T}!E@~#g zrwZgH0FXo~?0WLsROI;bIp8O^gK$1m$5gEQ#rumG(cAiYgRy0i+0!QIx=zGwRjN<{+#w z_n46}OvYa~Df*?3Q1z$ZN!8W-nx`k+ysvs4XG?8iH7znQ`FKwnk$gb>hM#nThwpvk3#K&Cdm-vEjVeJ3Y;5P2~Dg10aAeR)1} z{s>GHNQ-IA$4-@euWHn^=Ix&|d7_XD*qN^@)lb&!zr-45zz+k)8aak7_lwxlI0zFd zwMY=6;*;I}BUGpwDNw^|WcRmuKnYz=Peo5UIeA5$|5Wf+AY{RqTkf8LO)kLgbwiSB z9XvH944_J&RV!o4-LpD$q^(dddXg9rBhT`NBfWTMWYfc%jD&^)hoH3B%;o+A&n98D z4X>COK$3iPe8wWC*luJ3c15;xfcEf$Q`3S({1q*8B1G(Q4Sg_oevxyr_IYc)0m zg7H>XP{6--oItZXLEY2&i#8YYgM+ixFGSC4ONk zAl#U8adf1YEF~VJUoAf}I@oNE-75LDt`2Q#S|m({?qMd~T6rQYEWEL)-(Q{=0HQ2i zcju&KEr@|KQE{!u@Rc?lXnJs{GZTV=bMJklvrDOot!sW99V1Gn{;bzQ>-9I9r&9NY%LDb|vi$MJ?^US7>O9XHPnl|F9bU8=h2|{7x%=5F?LE8Ii2T;;w8bMDYi8aG zXXaE^M%W7~7HZR^cb6G2n*#!VrPyM_K;5t>3E z-bY^29wgOuA`D<1`k z%A&{Uopq0qXO?K5JYg?h;7o>)1Orb>3y7Cb1`RxUFt>tCNDX~(_Qu9kcp1VT{;Gm~ zfB!iIheR7DoolMX_>z?ifCDup|8=?QckRE+Fplm{RLCmNs#=(}Fo>1xaH8ay#DYgt zwjb*OR6M$bJ1(NWAv5uNg&c1tH<<(h`jrY<6SSIdLd4$|FtBqRWE7RVvu^nNzP zlq_?Js861_I-2+gT?+MWO?85(OQN;)Zx9|4-u|2b^wJFngMAK-V!6P!XD;UU-+v|d zvQ`DS*xS?gzM2h^%#1}mF@ceV8I$z#t|?HS4uvE*5xM((fE2F2k%n|vnop>3E)~yM zun4G-ph=PR!c9}Up2OG+67=S3S4`kctf%we)aZU8r>k5WkaoQ&63w$5C)o6GPv*jK zp}5|OevJW^)XBG>mtK_&jdAP-bFg}>^pbOvCbl}QrbFgci4!QO99qsnj+qUJZe0o=$*#zWs0zUjX+s_jMg8PaAXeCqAW8p?`cz!{XxL|HfZZ(meb^TE4fa@V?YA`1!+Y!KBr%3BMTp_~9jx z@e~c+(bDPz_@CW67uV6mVM5PF|9s0S?B$HJyM|ZjQW$#hw-~-qlQ+YUnvd$NtA@(j=K!h?fO5B6GI?RmyB5$)V)VX2X8Y@Q24Pb z`u0-miQahONR0rN^L43ZnyZ+aoQEeOl##etbH&5pbf8WhoPGT>k1VZJ_95^_wj{)j z1`U5${(G(ur;C=U`0Ia8oy`78`u8XLRdNq!?#%K$%$XWAIbFTg>CW8e#6wdOZ52q+AAGR$_tfvLK3*y<-?R=aFr%fNpVLqGA*r9fpjLKhQ`dorQ z@M0wBZ+3}?!Fum~^XT>CAtwh4auydjIDWMW23I7cz9d1oG@}r)|AH{wD2>{~?^2qD zH8DmH-t-S85nNc;M2y!}bd(dTM@PHsRaX!oU~S{^)=sj!Y|Z}YaHIx}IR5;}3Va}~ z1b*zGM`k(E=Hv@%fYGOz5b9=~m2_maHQhx=R1l_(YywZ-r+1O;8SpZmGR&HX0z zSGDAXfGrOyS6JY7UG`SX!_{Y#3ymm<#&pp9;_La9lY?b?#KF2}H7<(=Tu9xVU((l= zdL~tLZi_c zwdIWuZbS)7G6@9+d#@}PUJD-zb&`p*Fqi#5KR@tiE>aj?caK`X;x0>@hbE}imPi+B zi`tDVDb=%Tn4xQDPMvKsv$&M`2GsspXgL@hnv!bZ4@T6v-&rzpF>4?rE7iwgeO}^L z%)|G`q#=x6;+3FdC@v`k`mhvg*E6;-1T_s>{*I3yPDRZ_H4<{A{CtzO5nhjG=JT zueOp^+Yy{3(8BVpDR=CAV;YVF`63iF4yeg%bTqb5ByeW2s(8bA%up_NiC9$(X%gyk zRXy6@H05#!?N1{x{GMYxiyABAgQ#LE_NplT3I00o_MYem8>%LkA|hC+qm8!h)ibPe z9)FZ!zQV9T1Z0+=jRp_HD3$msP(LLuu_}|=<-TYAI386VLORlsrd?l>{jKn1WZz|P z$RBz2ohuDl#IuuYjucI2SJv2keVryr`uXegdq3H|$#C}i3xDILgUsfS+i6L-Hh)AB zc~-lcdmb0P&irsrPeX(QN$n}^-AU=4#obZq{Y&4o(mVfpyScaj;+*XoLrQ%QUw`&* zItHXko14+*Yr{|237iMIuXd*x^~td$Vn1Qcqtb0dTfXbDZoNWdi+rJk<~rZ*b{`^P zHeF}HF8V=hf91UC49_MiEQW;L^Thhgw|y~?4}WRms};maVtl=8xQJ*z7pwklE5#bL z7*po6K6|rjc!yfEI}^C=GqxVowvJo2nSWjDea)1nJAT*?`d(ub&&djygVu$UH8Kaw zyqrs>2i)#fsa2^@wSI{}ep>CU`N=l*RKMDFpOd|(+GI%+x!%~Zez9>~w)8mM=uei7 zt^fh1mp|r5$TJMt$v?-MwsYlAqOeF4h zZt#KJG?4B|UHudW>aBYggC0Qv=<$C={7j-v4y5BwOCz)Vyl2$^%lwCiyV|o-tMBDG zIMN=yHtCjWBmC8sW=3CU>uYFj(mypMDT&w_w@;1oF)hidEIthcox<--6f*Mi%=nB% z5wSlNzWt0czVzxKaA8=Ag-o-E1#(Jyn-aJjra>n4QLv*Pi?%9I!mwxWc*!Mi`g0lO!Qw#+QY~ zR^P>+743cq* z*L(LAHc<}0M#I3sOpFB(M~XH|c(m-AHdCE>F?ZtRb>#1|(BFQ|UnAODGXAlydDYlu zzEsW}HbwR=&PXY;E7;TjvE0i(`_Ic~R(Mx8x0l zUj9h1_|5(A4|kjdSMN-A?S64pC3wQ~>O>qNdQGv+Ti7%CQgO#5Y0(re1anVNny#tz zK7QMFMs{{0+;BZW+jOy!UDh-jvsh$tb@B9O7NHDU@8#N0=kFSQ1GwvBd~XJq-Ltwg7k8CSH7W5G~43z-c}8N^5-$ zK^c;aH74;DnPoK!-ET1%~kI&s^y~+CrF(xmUBNfdi z|2Pm7zw7Q;xb7P$_?xJ2vTJoYmE0^BHBV!(yuO;8UEm8hsJk9>Sl|8P?zV&4biMBT zCZPE%=G)5!81-QXTgXMF?>Y5hLA2A(#A|WEq_sJ8Tev;s$9k9hO@V}oiAiJ>GV;l} zi~UjYvUdL7bct-T>#AfO7B1lITTd^ACxvWi1uan3odW9JFcG6}3=yY6|!CWCPFC!EPPB(h4ARbvCkpiH9^~ z8Y09a+S`>`^KW`I`f*^|l*fUeh&IV29QW?dF15-Nw!gM3dQeq5Orvsy>wBKAF za&SVRWxUxLxAFy! z%ypET&mDtpur+B{ck{E=_mr%Ku(jK@T2y?tF!mDL8L^E z_>Vuv4>ugFsJQR8;58NrI37o5?#xvt_pRU!ck^u5?EPrYi|&qZrx0^YiF;-~Jr?Ec z%Jr*~d zmt)i>{Y~xX_r3S2NlZ~ zbaxJP?W9?MFYsUUCbO1^sT?p1Zo`ept7jqmaHIND{K02Q>DUrzL0Z>zV;;fz8E zY`fQxQ6p_}VuR#m5qhoJx^P7*d*^Q@bp6+Y=M6r0~KYI4@DrN|yE1 z8m*M-XA(|Q1e>y77?Q0Fl-@ku3P8&T51rG?pvia2QHnew{L+750VUOBf$Byk8zh@S zZ+fEXvO)ovM^aq@-A2h}g_kA#l5R~%+9OEn!Yi=twPPxe8Y{~8hBsMPrvxL;J1ch~ z)ZeQ(!c~p5A72UJ8oBcaeI$ZPBky{CCoEYuO73+ZZILwOuB~R~>@DIH?an2(SU-C? z(kV64u{;9z?^9O(mO*OfB`Pm#g-!|Z9Fdf!pSjydM(7saWpCt{b117@2sDaeXNd9# zV!loCZKg%J)Jr3HAv@uLpts)J{ph*E_0$8|_j_<%-P;doX&YM`Qj?5CYwC|aI_a3d zu=u+QS>a((a~fWWzyT8Wfjy#`K89hZhzvn~e~Gcz-o?1awTdw+-@3n0crIHubi8f4 z3;H4#(}?)%jvvtTBa)K@r695hg_$`yTu~{y@>>s0-y`hPC&F4)gHx08bf050(p$q_ zxXR3M#diBsl3sHWcI=<~Icm#U2)+_~JNTR-7u0NI5=vp>Ej7$m9p< z3L@X&8=IeuJXe@`3;7C^6$fVqRgl8eIe3I#b@r_=lwhhGrb|9~!$6V)D~ZqVVyc_c zbvR;yl*rPN(bRLnH~u}#y4x*vKkYx4Tc?Bs(V0!ZW|h215xrvC2U7moXpveQ1V^s; zYg#QyPY}(Rb(ZCW84ni{ieJ81aG=93W}u_}vCSjI|F)3O=)DE+p$wb;uFc9?fn4%y z4}2CzaklS868dlsZo#iT@ZXNP*_0$3elf166Pf;Om;3-p}XHwckaOxGuft{AKpk@Z!cG zpN||AP!cEPWa3XNZ`CDLXnbLe)TcVG7dFnv}k;24wm|IdnzSOynEt;LaB)V z-_VGng%&zyK+p)w&?Fs!?EFgD{7qV(v@B`%-^{yHsfDB?#pr00j{40{e3uIguYVv~ z%AJ1jM)WBu9qh3oj;Gj4xp?@Mt5qk(bPFM<8LPgDZI77XYlW6u1vPmQQ6g?8g!$TX zLd=D#YJOaD$`s4D1f;IJvt4(|*)KS~68T;*GDC+*T1ku@Jd^5qe!eGR#+(#YSLel| z7He@rh`~knsia70$(Cr9v8gT0X`}Gonw=~c<&+jh6>p>ej6svP;H@nQF~0OTzBD`# z@I60Q?C(F^T9#O|yIb3r6EZlp(fR1iBR`*{?_6EQUFO1jOwi5{^qLFbqCf4_xp-El zL(7~S7ApCTb2SAK$t&O;H*V;aHa_ne55!X3u#LgN(ay2rX^)NAQm5f0@tZbA{8D~b zbe_gVD!t^9c-0o){KbEKUZ^i@&;jW+budDAHszU)o>1M!OPIP;!g-soP|?xRNkxSa`i8gs zHT!ds@k+eBo`G71p3jSHvFC3Z?aFg%p3<@jNg^>5JEB-=9beDiQsK_jWoh}G($5Kb z`71Ih3CBZI(#)-^`)&>I>20TK!;`bVBy~pi0=={BoKJN2{goO2USoEZ4L3)Qdr-bU z0hm^IpGm!AUF%*sa?e`2@LTsdEv_L=H(-zu>9X^=MY)Tnyy-T)W4kE~5WT&s7rl)4 zH5sy6M%*uz>P%b5L5NdDJihr}b=z%BAGzO_Oapwh*ngc;0;39g`}b4}y>riURE7DN3829~Jy*A-~2kNln~FD`86|6A9*= z0l>gV<5d8YRjam;pzzUACm3B~;WXKxz-?R1>)6WB$mSqT%rHzVyJYgq!1O9Q!=Ga7dUx_t!wl_F^i<_w{AbZ;T_ZI z+wk!-TyBpek`TC~*J|(l0%#a7L-Y9OZ)@(!ardm33nXr2%`0$HSLmwQSP0kLmnV{#-TlQ|SR50t@N&uX5z(Q0f||JWWIT$$vD9k`TbPWvp(H8Voh0yQS4jBj_Q#(E zzGfzDvrA1-K!uo*Ls`=sQRv>$?So8H=!g zS1`$Dr%)N4jFr?%C8@R6)(V|Xry~XVd1RHT6o%9y#-beIu&6%OJ9M;hCMXRF&a%Ms zPN^sGu2v8zgVj5NraDSsga^>Yz&Ea(>fkZejkufZ9lN z+5<|H_Gyi6{?2r0|31#J!97shCPj}wz!LQdsu$t0#}BS2vUJ)~?|$d6qe#(8wMv?q z$igntHIEDCeNrrZ&k50Sv6qC*Agf1{Uj%;kNI;%+z|>12&whtJ!?`F^5OVY&AEpL7 z_O@e&{KwF^_TbDKkU{GcUE_P|?Q?hJko$^O>am0}OtD;}qn(BjivUY|RJB>JK1;eFSBP8&cJ`hON@lPre71Ct=q|a7bcE^{- z@Q-Dt&6Y3!27gZLI_n{kKBahf86rWat|?r6BWy&H2r=xG^0};5z`3=Cw#u=7|>{J!j^?iW6AMbFjWn+ZMNp_SRQU(f|F zj)t*R7`P*!JK9eb9EAl0KwDY{t*{;{jUv>~DVU+GmIP3`pNw_xtshY155yFC zpo)ieSm^pGNlo*t;u%`7gPG%QVw`(c2p*Dg8VTE`XJxLZ-h!|i_|C;XMMbBEA^Z{m z3jXUlsDC{akI5)WKCz{@{27=pzWe*bV-tI{yce zhJyo+_MaEBvpx7Fg7bWf|J4Hg=MO%N%G&z7a=7?q$_)Je^ylOlIz4dP9~WGp5lqrth-9Wb^t~d)2GabcBe%1&6Ia5_h%7!$Nt2hD?JZh2aQeFVQXZ zdx-)hSMMIWVzFxYm_Y|RL}t?<79Iu3utkNL(4=s-#s+=dB^F|W5unN(80UD&O^hiA z&hbwXJ$~3ir56qBt`BP>8f*RE;5K8&Ng$b1)g_l@InayI6R){(?_CG{jPE~ycsVbj z%|b^G3bZ>Z`F`7;Kp89A2!H4{SZtZ1f=fFkQmP?FE~dLp>iaCmLdEkfAJ^H25V+3`eeNJx z(@A1!ydsl4V|>F^kAizi#5}dpZE)0|KUGT;IC=E#K-~CmwlUnK{O=WZ`T_b4c+cP% zY71v)5`763aKkH)%f~1%Xyupy|8a*IoRqV(3-BF020gg`FLv~>+OKA5UVBZ~VO_a9 zbso5&d1|_@)Uo97BLSoy0GK!shXI9s#|4R>Q7E9)19w~-7tnGWw}aeLXXi~ZVsPm- zF`$I;1OyZ!!Vl>OtwtEbT!)vpJB_EVwsTHw!L*^{V}p}bnDu%xQxrN9}{C?D@`{8Lxh;dn!!~hmBQJ^h2 zIXQJW>R=vLkH7E|C$wMrhfXok95egH4$2?fax5x3-swW2hk)w++Np-smbDz_0eAQ7 zH46(Mu1PpM!)c4ffV?i$W&oX6RwA~`A`4374IPGEDd($HSAErAQys;9t|j;sk>DPl z``*zS_#6sRjGJ8#=zkny*>Nn4V_G?nRWA4$Hij3~rJB|8haEo*1*VrYZr~l(G-?$J zUj|-VGBUDMS5}t%Nky{ec@-MGi+`usQ>XGZ;0dlx^KO9~IMD=3LSn?e*e$OTXMrvO z#Fzwazp*|WfG{_MfOomxD*vu8u_xsUI6d=!0pkMbllKQ(tX177ShM1|zn?K-`kZ;5IX z)y1Vemf5uLZJOV#-Xm&LootZ?NDhj4ZB;EK(2&{irKcEwSQ(bn-}5D*t-l&%&eSh* zzpc_LEG#rW&(ggZMNlt~P4CkY;>0U6c^twlSdt^WPScwa53Ts>UX1pZ>fS&pW3*~( zM#5+=_X`^F-Znh6SA!(Lgbi4~ATZ8j%wmcQ<)PO08g~)b=)+^t z`p_eak}mgig7<6?hOJQeQe-q1EtKw^mI{A7Q|!;>r0KnnAKgO%k@D*5T1QWNp%Hms zEnV&(Al|bkO6hF`Y1#qh#bB)QocpQSp7#PR8Cg1rCXs@`C^SH;>`fmPP|?6mK|ega zb7)jNLqQF!KaLqz9XE5ak0At4Lbw$J^f=CATmACAnF9Tuhvz29Orc`=qOeff+w>U` zuEH>SaUq)fwlwn_?U(RWXfIna6ea zGy*MiN|qKhAl+w~0uD(~{C zK^KXGcvjmo@Ti?Xkg^(>Ia}b^Ph*)dzbbc0jFR4kVR8 zk)H3{{{a{Q9|0v1_`qxC>#R|UNQmPHX?d?1iYjXoSD;^M+Lt=p_O}#pCNQL_6<9&a*7diSQ-U?H$h!}!s&&)H7v&QbDF&ZXO=z+w5-jcCn8?(^36>-Q=nxm zx!Bj{>LAr&RxU+|(~ARm`lq?$_x?)zYH3uh4n*&=q`hXTKrO+c;wegcU3qTQN_6v+ zu2E73@?a2NXyFoD4qxdIgn8kNyQF4j(Iq3E6^B*WtaA{7uy}4BMUG^kFz7qs;~iE9 zb-bDnhaA$vnRM~}Ae9_s5{@CMXoVFaS)@v}6M}=8pmXUW;3l@DQMY zZ0C}P4wJBziT&B-0c9PjoUhA*#Esduy;X^d+T!@>dkhm3LFp=SgyxJ(a?e8veat<73htWBXPiN#ibYzG%)Ph?{&QL8GuH` zcU~8yI?_Xhi*mGCqTVtxZaJroRJGutA(flVwJVe-_?VM)y(0McX`C%+@##O$^*k^t zRYu6v@f)GitteB{>GloPE znkY3#GA!gsUh!--{O#G-wKig~!P^lVSjf;4;iPG)Vbuzxx59mY)(uRJ zj|sb1=)f!5^!TFGcVMOu3#fTe0%+-CF;v74fzgQ_@u~S>X87dnEGJ=Uq%a;RV>+() zffUl7GDJ=Zds1=j1feN+ZAX$=EtjKdPtGcsSy7K4Q|@U!t}XGyvVg@uYSww?TK1y~ z1>Iadw&5SYtA01NG&0(mDHW4t10rmR)AW5h5znuZxv0E&wCgbox-}t2(!@hm-7(bd z#9>PjOsrb?go#D}6td^HEIJN?NRiN>(6>ut(59qjmFQIvBjrr{g5?YelrXm1r_U%bjQB3reP8qDkSXw;KCTr> z;+YOcI4^x)!;$HSy&L|44(=n+f}IRjh58q&N9uGONmdxnf#WcKyWM(@z5~#u;l&dptdVn`wezk;Gs?RzU9vKo|f}Dpm24pBa76d zOq9<&u8(oDBPQM;NZNNBsPlVzN)2uqTQpKl3OY(_oo1gTN=*X#QaU#l(mNqOR`f28D)ubCV)BRfw{&Mf}e#x>q$R zom|KX#L970UWuR1AE|jJ5HUT|e8idERrz!xRcL1fE2w)Jg}k2ombSrUb?b54cP?%@ zhbwop*rWGY|JewAl)?aa=x=0$LVMB|uA9e1b6=8n!f-=f?OF~{9n>mogRqo2f)emw zB)7dZXFHbdOnv>D3^D_+KXqwVe1J8n50<4xfxl~IJ@wZ zpfdGJb}IL@r4#11BlQeYD3=h*P-lNZs`s%Lp@l5Vi-LnrvO@^$I7{Cn>p#1>5++*v z3MWO;Q}4FoA_pGcl`b@v{!F@&nTR7jueh)DF4>a|!c_%xS)T?zLV>Wf#mcQ#2(nom z?n4-$Kbo4N0iv3&cS>K5ovht~5(2d<`0(xrlfT&qur{z-OR3tl8DSVBuq+Ft0on-0LiQZaD=h`0=|W1_`V3>Q?Ow z>`zMV(3uvnl|YA!dc2c+;&YSWGt4@>d&X z-rGrpkv&|JofzVq!|%Hqwf?YMbP9DEgKIW5A0*CJj?ZHTdfeUqatRx_S^G6OUI6yf z|JSbfzdPN;|93O||Nk+E*)M$}%x`*LxJbwjtvz~kM|p@Bc*KzgoM(EuSE5=0YM7fa zFe`DfGWPyBQB^ZeVUh~%IfodhX|CR>nUEw3!_|6v4zqRE)apw#2GGN}6eD>d3-XAf zkrtx2r>;RKzrN5u3buSdkPW@ge60ZjucS53RUYB>u4jM`o}GXXWe9$EN%693yyeJ3 z3xX?668bsn0y1r@d5r{?1Osd?g_f#f%|B{H4Yaq&@p6TIerDN?+I&M;Ba#P7p|=EB zPL`Zx?_u?IqPJghaQ>($J${F>9T?LgRA0ktq~;OH95W<@?1^fPN3`44B5I+!k8}-{ z4n7tz;(0b>g)-nN^Q2Q`h&HB^;K{2seb|!T|)%iC*9kDKtLoM!ax%p_(bNTpkW9NgClM^o(umNHf z$Q1G|%YEA4g1i+VRHdL4_1}i=G_LXk`~Rg^mfM_9yfdqkp|6CPcai)E6OiY+XZ1cs zi1l_UA!mpRwb|6nzY)&{sq+>4#R4pZn8K3({={@2i;k7*D(KU-F46~ltkCgqrTJnz z*AGI6q}%BMdVX)VUeI+%5X1wHjE@gj{z}Ic7Yw!3nQ^!-Zn#+LP3T$Qw=J5TP#n&! zEXo8)SzXFmAFX#7imu214c7ylnh=;UfjOYkc$q1;To7z!DZK8*fLRyrbrm0Bfwky!&m#6Q}~T{ljiW**itn7~|Ef(^t8A zYY82Ve@FdL8A%&kmft2Skh-+WvJld*&)okwv6pybMDv*VG{=zo>KTDQV)j^Q54LVV z0e{2N*?2zJYg02Dj5vCRaRD>i2Tiw0DQRgjvuP$g4Ex@>#*>mQV^PoREA7!~1s+&Suu}TQmcbiSQ-Ua^{uM`HZBtg`NPgY2 zv_uDwrpWQ#^|YQ|+gPAUTFonjrPkGXj+M-Vt+}kdSU07ccZs7oPg~H`rP-3wYj$^-nfd1wgJ2HtFxh8h>wX=H*pJxZC)TEFpm! z!bCOpSG73A?ENEGSYIJQp7ms1Kz=@xNx2}X{GO5v80!$hW2nt`ucCmOUaBy{!r6<& zdsXs!1jPt>`0rD53>Jvpj)@-kIYmaPl-ceIQ~6xbMq}YP zE_KNFJ>5zBsXy%{PCY9oW<61xIyHr&#yTfamJL#CKs^Ng%tD@nqs(ELNU@iWC;~L2 z<@CM9KvDg$ocj^r$2#$qYeA!@0r zN_*@D_0sl&WWBX`_ri^dUwa}bhRGY*CzBeblh$5zH2NFXq8fbT4~n2DU#?CU%;p?16@PF#9Z z+7YVN;p@jj+s-BEam0{g1R`u)@vu+<_Ze(aMTzpNU_k-YUM;^$Y(X}Swj$F**3tak ztsOV8$bq!>=UzcmRXyvKTrVRQbCxNDyuOW9^Dk9B0j|Wq>rUYEAl8>B7M{Hgj!{ql zX*wIMq^PLKCGHw7#~?+U4R_Gvx1DN>&a4O94tqh3O^dIfkP+gCQW?%Dk=n6LAA@#A}lM%peRa*a&mqSifUqwjTpm5vsW&_Szwf|e%$s=#5Z12BF8cSQ|y<0AofB*`{jk0nx_gG~bh0J;U~u>^!= zsE`nhY|Nr(`(GmTH~^eod|OY%@d7!^cZ~(7$?a>^w!p?8y4}ixpEAjhJ3JzyN*SNZ zWgS8&)CWRJbOaY*#NL_yJ-I6|*nM{6IcD581edSbYB>EDd-7_d+fS)hd{#WZ*O_&=P?U)Uq+3nRn|{_3?O0{oVlF zs=rS5(!6VR!jLfjwS1F2_IdV1W9rM%8tmTp?zo|>!4 zVxEwEuL|xnr3U%`+1P>#Nhn-kV{>~wEFfg?4_$$BgMzdvuin9pMDRNw3C3;Rn-n-VnMKG`BqueHBx zujf}oO`bNjWhE|yxWdG!0^{D>=r+4`abZd2KpbJBlh$VuR09QV`^V}d;VX`?`f6qY zZa+_kV42ixvP<-};3Idjzj2Cmv9;^m^G`^{)wnk`!>N-y%+n&VaF6cL*DbAo!ugce zh?7VuHjF{;=dWL^`S~Su|A(gU4#c_-yBCTiduAkr5R$Cy71?F)E!le$vP1R?4|3Z( zd+(LZ?2)~9$o^g4_xJYKgNz^HvI8MTOe*&|X8ju5u@i2<1$pJu`lN*TRd(ZjrRX$6;X?EYgfPE*f@TVpxw7ekSu; zllqt8mIR~u=m;a;#}vAFsY}Gr{xyia+t3{t(0T6pkl^kyR#rLJi_)k#>v%#v65eBA zP4D+8g=}tq92?6(Me!=DA%_7sIJMXl@5#8;)=dP_+tg|OWM%$DH9S7s(;*%0^scO5 z7XJNh3;!Vy_u{A9+$_S5)xL!Ez78rMo^wKOkJH6wZ|_75}(A%+LK#pA|W} zP#smd?ANo@NOwJ}oPjp(Px8M!|9xi?j(f@>TSZWY6rfKnjaRE5b55>?p@4%*tN3G+ zF!>&$NA?QK!s`XD_yNKdX-sRvgItK~i(Qz5BEgX3RUD7?anD3SK9oT z#bH&9QoutWL>*A%Z4_=LLAjo_Ca~jT^$KgdggUuS0{L44N!GBVWsZoyZYl6uSlB+>#ePV5l($TiL4hW;il<9{hdO^3F32L(E; z3j5>Zlb79l*6SUA&8VZrtj@%_Ryl^x759IuW5@bokj_jgrYII=bxk766jwl*dzmG! zSUp6lGXCOcvhzUYwRic7k@LBS8ZHv)g3tE(#KZ(&JxPF-qXo-syzyOarn&FLVu~Uo z1$>lJDppZTan7RUwf|W=;fx*)B)-G|K*TxFrf@BB_7pfjBTl2J9su5J^m;bW7jXgd6)^sm10@9BT z<-KGW6UA_t+tM@319x6K!K5vZ?Y4H=B>7GX#rLTD0RdHtii*OtWvw+Yvw@dn3Ur+MhKe&7Eo&>k;rMB1MQv7Z9m*)I6 zA);8<(Ad@A+Ur7_# zd_uxEYJ?g6Sx=y>hsl>J#+s9}>BGzb$!KBPh6C^YajyMVR?n32i2B|s4T9|H4t%sU z+O&bAd6<>Yv9?&YzFB2eq`;6T_rS{@*&J6;R+eSQU2DUh%dABNoYk0pG&hI*Jl4<$ zM+pfq=o!+1jGlVI>y#`Xc)$9tBw2(C<*12oQ&?F4!%DhAZw-N?BspbJT6v#&tLy46 z30_(^wSlS5UXwG{9iP&J`*g)xwDEm3!N_1Kk+`vS*(Z2~BEKyt2BG4Rh$=LpVe+$H zaY+pal|WqIpo``F9Ie~L9>5>FYuWu+vMtRQ1$l@;9(u<0GiamBTC8GdNPk9pGH~Ztybiv|*IF%H28MA&8-w$@( zxTmy3QSr{2h22P6-Rn`|Kd+84`^vPU)`lrwlY3~fuhQ$2p>c9@878gzabaDyP_a}c zn~x(BQWKw$U}l(!v&l9d?C2VwPbq)=xYcd+F3y}gTHlag!s7XP?o1m*nvUzb$Byf` zCR$(D>1L*vMow;QjkOMN*w$*f@TUf5D8`s2p(5XbyR}As+u+>h&w<uD4H#9XB4)bX#atjA8&xkOT#z!NAy3VfWz2j%yPiMcbA^#kZJ?e~9y^qSY zzP*WPO!ryIdfKe<@4Q}s9IK1?a8_KgeZxWRX_Iyls}W|}=jk00W=q6Op&Zo31>qS% zO1;jzw3q5HN;m8CLqbMYY7HNr$j|{9B~c|BVrkKcBT#PpRVthN^6wE0{|}D~HhaH5 z#Ay0^Y<+MZsd_ddc+DzV_VbsvyQGDM&B3K>z?)8&gZM^aDzz5(wO=uUT^yxJyfN+J z!N!w8QDX9{Tv)#$JoVLhk=*PAE)go!05w{>*B%${?QAk3NfIpa?hfbV%a;V9r`ZE7 zYyRU~!si5@Sd_7QhK5HBH0a1!*`fRq{BqQ|^pp|NtOS$gd$z5|`T-JMXMs%@EFpQr zvm;%D`wYd8T?xe%ha6KnI_|=q-*m8dgCZ{5wqj>{aEV5eHCw1B{Pl zZR;B2NC|VfG?X~ii=@2|=ZnHA&;FT*WJdRSXQwt98n4aUuWB3*$9C<}4CDKP^LQbp z+nKq^yw8p-nUzRNaa3?(3iY9oqV-?mdgW@8fR^L@_y16Y=UlgBO-v96ZVgbF6^N@& zPN>n>KGJt`3aD6T92qZq@)pljj>6Q{b<*zazRI(DCB_bL$^@j#pj3#N2pnkw^hv;j z=2t3ze?@K&S=Np~7ygkMn=A^v2ZULKOqgxprVy6{hpf91YDorTkncaCdWtH}^s>@3 z@x_ta=!Z+I0X)7`R7@DYR9?$hzEmZ(kDf%!-LYAYO^cFjui<^=Kfbi0C1oX@Aqj?j z@9E{)99QgkX24nB*6}0Wr?6wk@P-S{tx}SMu!mqfB}Ip?mjo^ub|*idfjhFOy(b3p z%6?~{@+wn^GACzp-|YLzA>$FMD1QUnjg9hR;G8YX$rh_L&P4UTN~rQGz& zTjpiM*!B>jhbA7UV^~(2#A=!s1Qwl;)%E6r&;o1hQG$=U+t0zUL-A`i-F4j)te!Mt z>@v~AFpbSqIvLrK?-?1$ZEVs3){jOw_FedZA+?Kd>#t8zFup4;OaCZrZgkD?r2}Jp z-+}`#ZbzZSQ2$dyE-Tr?0@ zJh0ZYcC@l)tPW%GCiXS&h;^AprN^Hyxon-{NK-rx6)zlZQEXnZ19ETZ)-pC!YcqoS zEp?AeJ-JAUI*qQP5!0F^P9!a^xH9b7_1bV+sp_}-6g65!9v)2FZNV8iB)6slLd+ey zH0UI?V9P5!?x>_JuB^8GxpO3a@Y zmCTx4$I&BZmO-m;#SIC>B?z8|bSyqE6KP&hc=0egLxtHn-ibE_x~?*dlCk8NmBZDu z($3ZI2?NnRCb;5}LR3Ne8oZp*Z`B{ZkzV_`U_*tDoG63vAeV{)s~j+Y1asR3hyToY z&))qF9x?N0HZ6-wwdMX6l!4?6Eor9BV=mgKQmq?yKKE<1h}zg}xW6&vWm!Ma{QF#u zcYMJns?T_cV~6f))5#s@X)d3Hxbo1ehkwJpF<_66tMz!LEkLHL;_ zOX@|-*x8vB^`E82dqU znwXg2h~F9c5wlu3q`2AYs8#0{oATOuaC*A`Ev>H#Gj$M`2AX>CnTVNUYYtTd)Z{*nmCoqY7bu&iVuijD73(FYWnGw?}?dRit;b zqo7=EPu3-SUYrb14YzL3kmo8T(Wkn6qsOLLZo?MT^YHj48i-ZmdBI(yv}bSa*{b=i#xfK!I;fYXolC55 z?i|*TlWlFSh&xm_`k1EOVEEBw{>8>Dt>z9a$RT&zGy~W1n=R{`H=q{V*w~@Rjld5r zjE%UG@*o;$85_5V9Z7SzaoU^p@Y=E*De)k2{GWc_xLL3u)lb$nu}=Qp&Xq)*v5|UX zzro?$1rO!r-(E7&<_TI;i<#T{v`Ho89aps{#mVhkKAR#MFcg= z7`GL3lw)Cc@%3hLRR0QmQ0r4+tG>^lKPR8KL4}3D?Y_%@@7_I^g)>vqN00tA`&@kr z&gk}7+R`=r4-%~y+AGY*VQr@k3|!Nnl3A8K0;57`lB;Ca?` z**9;n*6vhWTU*V}6Wq74D0s7W6Pmiql9c?Y#_7r}x1~rDMq38>jtPD%D+5fZ2tk%i ztc~3#2expcZ#aacyr$!J*hkV%p=dXFvH8)Q$K#)r({_T8BB5^!66E6Odw6{DLrWDR zx=Tb9zuTuFr9$B5cHwfEYr3)XSV+;xL3Zf?1ub!+Cb_p42R?nHx1shCU46f1BTDTD zC{^O(Bxx5{!u~a03!gmZ{kh-GnUtgnMo#J3Rm7s>u9wZx1(ToXka zZB$eJ0?TK7`$Z$}yL^%pUiHmA_>dssEP;zkyCApON^MQzynF9CIzwJ>+SQrv;z9Ml zl=6!;vRpA&1{D3q0<8oIn( zdjlas>FJH(VH1z!vQdf$%)=(xgnu|O0O5xI$BFl=F@&WD_btG9SUT@Fbl>JaLHE0; zcdGhY_P}{xn7I<`lYY|Zs8XicBk0HSzi0IM46Jp)ZIia@b z+wxHogH?7t#3~)sHJ!ST!kHD;vFqkV8sIa%)46o;=fK6bb|#C0itAbkskm!u2M+qb z*jQllB+7yJjEPQ^%Vo8OnEPe{t=@YVlP1&9;q>o8%f;GPT2k>iu*A9C(Y@l~ze3?V zX}`mw%^i`K=geP#Wd`!7=_(~>Vu0bj?ZltWE352vI7710nrTb z@$t$NdBUQbY6O5}gZC97Ga%>O&qU0WY=25PChOpGt`&l|yWTy5l)Z`dLe5Y4h*N*?#)!?GyHVp*QzEx88|N1moM4+n(Seu;+n$dACHhA6l*>Z*? zbPF~Km18Q```9;M=P&!`jcjZ_ZH=1-b>Uxbuf(f|iw!!29URyqigcDr`Q1UNE2C56 zOBJM~s#;DI`oeXEy`jMq0N6|h_O6Ju|Ej9uJ3A%nby$g6v}lP!4<-!VU3ZST&gz;j z1nBNHU8n5Xob?j;i!98oWtuG>ouex&+g{FQ!XaB*3jquM7IgiBH%q6a6w`9jnq+-_ zDhvKY2eyxr9@W^LQExcp)t53z;kiHd_D#FlrY~S-J=%Vto0U<9mrJlof{w~0--u3@V&_gf!b|7h(@4=+0SV9D*G_jie7r+D zT?Jz^ZL;n10)!Qr%F3b)zq5`!;(d-zI$|3 zS%o_P-<40Ym#cc**?zrovy8=K6sp*_hlfnX!_)JsbI(Q{DGGBAwnqYa-2PH<6soZ_ z{9Dzu9vK<}zG)0tolxn%`okj{kK+#F<{$OCd3o%wU&HXpcLip(*O3~yC9Uz0l=|VL z)k4L%?%HrxAAH73wskc&*T@@6QEh?PFSxiu^YR!%dSiQ@oovsFikC!XzKMWdh4j#$ zx-5el19|H-0hLJ#@4=Bh$i9_+-%-;SqYrP8hK^K^6vuKWg*$~sV~Y2~-u7ym_ddTt z=_@g*?~UofO=o>Kki-eh1)(s3A|cxI=awEeyBsX>p>bm>8q>T0XL)>(ww&U7#Fyu} zCo9vaazqdtauZ(Mv*G&dDXT5x*^{#Up)Dw+0fw`w(f-_aJQ~PsRa_rr_FZNyra9k&ljep@(uoWi*kYgCP}CNcx}UO4 zy;jLZzo!-~1{T-p%a&Yk8yJnm7bXstM#p{+O^<{Lf-sanbt2uOw7OY9LpX8UBS`W_xc6OE%`O1Q15=@KF3+4{d|z zl|_MOva0g$-yBsx@Gz7ktI-;c_wW6k=KC9jo`6p9SicSFKascz+cmQN1NFpXDf?mGl*9;$OJhq2$HAmoj zo_W~-3L0?ER`KeUk`gLVF(DxsDm037o~PW9U6rB0!1kJJV0hT>>}8=sShBlfcbUxl z_cXyqA;)3d0qIMp7eFX?wy>=%Qt2(X^o&- z{A%@HWoX5&%+lhE^E`h5t<*}X)hb%dJZng|W`wd`WKye1LTvNQ4EcDmJbswO*#h=W znBeu2puVSPicK};2nXpS^4qr@+l?{HKg$pzLyOnKfvKdNo?99jO+T>z>-FB??5sF% zw0M+Qs{1w1oXb(+1>UfeIo9tm4`SRqTU@8-m*4n3qna*~n}i6&^USxtzO5lem{G+= zc~phT?*A=^yg>i{ml1s{h{AHR;(^oB1tCt(w9iP~3z0F~bdS&&oy%TxJ9SN`hS<9C z?|hi?GF6MBEz+(Fm|iit!3W7ZMnQ2GQuJ<-T%L$BH~FtaTvo06_GIgZ{^4Of(_i1f zQQ13WU@fZ>5h7_6%Qf$otg3s)gK~8R`&2GXp#Kbe;HbrYuiSmWYHDgK;oG+#%YJ5A zNEVx@__wcJSeS06@yv)#!cdfd!G@xY8EzbkogFw`gVU~;vmw^WAPKok?i+1n2~iO8 zOl5tcJ_(=WKPh+;&zpOnFb_SdS}%frX}M?Q=g*&UoROHcXGHvZbKHTN zE3<{^zCi9}Wd-BY%iC`Mvz^k=WRp&QBP}gmAd%^@M5ddWUl#}*A~1!5=b3_wd)L?Q zpBWi*(9d~1&pGlkZsQnqA@a}WWSa+~gP3Uy`x1cO1UGo=#xuIe-{@XV*fP=_5i=b3LFu)zM%rtDsjU+ zDC5(nO$OOLxaN`{Uk`YHuSF1fQ|Xb~1^6(iH;IB1FW~WnwQd*IQ(6Uid5H45uEYOk z{njQpu`G5ma21dq?o#PIrm#cw*rhi+6gB^(uo|gBj`Y1QU3i~CyKs7zi=n)De4Yr8 zKZ^ZT*hZq+ue2s1b8{w9Kfmw8X7G8316^2}i#x*cjNk#(h%+o)&=k1vSwXPOgV4}# z{HK&18J&OjaiW+(p56}!*1si&NWK1fIb8vo1;+C7Iw&5YG+axjHjFX>_`zwWCR3MFYtv4Y7 zwgdNJZ(se@>J})#)UQk}EOwExkuSu|^$)s9(GksBPQh81y%g4=HH2do+{jgX@wDhA zxDlCKyrOMw@VC`8G(>Pv;Yx$B3u^18BNo<%zwdfJ{1;7N`_Ozc}*Tj;(*>1tGvJE5s5%|IZcOgJ{qrKYC#=V#(e@Dqn( z7(&yuYCZh9<~))irPNfb(e8fyv{VkGk`952iqcG_;2e1$AYyIh*dr*cL zz~~Sa3BCfViOIUn8xD&nE;)I52J>_L%l16LJafPGCn&gYK>9+3y}~ZDyE%+zZgm1~ z6UaV+(-8){;3jgJvBL02p1-)pc3BGJ3hpq_O8k~lR2c^`d$dy2nv6M%*r<~6ZU#yL zBIp^C03nV%%*)Hq$th^&5FUTXF{@`{5}?HHpOH5@s$&1IN3=kEJgBgkCWKl!9fy#R zkoo!Z`S`CHWJf(3Kd~delr<7Pg>Wt^5+j2yIbR>D@|Po7)MQqeZB)4OfZSSD1saqp zSeeuih2a(CG(!;9L8~jIi7>=<*0=_+>ee@F7B_YJW6xz5HXIclxmh%o_iH~v{`F2g zpL)OJ2W_oj0RhLgn_0>YY4H3|m{n3q-p2BtLwX_|z`D@;c5FE8LsUUY^$!u+GGD}o z!;e=Bwsn%G!&(gjVSTx5kk!T;Yx2S7Ye$DMT)i-eE;DMi)Obk8! zEa?20r7^KE+4x_!4M)Yc$4b)nrOReeL0#L&{}!l~!iKQN*ZcDY%gnQSH~0}pS9{(~ zTF=d1)z%uvOVJaBDl!m2m%_}{Q>P+nd> zIyTBCC^*pDi&2)b55?*eDg{+dS?bhfh=i!tX6dYoSa7OwIbJIs9v$tTm>@A7NWIPR zz0JQhA1|6cl8Zu^Sy-s8{d_A)0N{B-fO7xlu>TRC?Y`8>W0$DklAl+y#2sWsqiDTO zM&c*D_t`)-2Svb@60ovcL##zTP@{(Kl<*I`H z%$MrurvK(S%YtE&iNV+5@*EEbQ)L{~e0ECY$Zp1piVC=T7HqgB5ySPzW3M57r_9jc zn~R5EcP~b;7#>bgL6HXGm{=qX(NPb*GBGuEJ=w@cM@6|@eoHe0%=3#2*V9?&o_MD+w7`1@7bvL6 zUbiYVz?5>a$8Aq$cxc5}H?py#Wf1@Vv>idl3`?4+A&p$ALi>h|0vsOfv49hHFiLCj zVyV4e&(A7(D(+}=`OlWFQXXgj!ewj6dH>u4lw(IHL(b^qzso!=zVzfigS$e)*!XE+ zDxVm2svc1J(8HG6Ze7UA$sPP0k%#$Q9>+}%e~;AbYXK17sjF!%k8X_oJw3Kv9~5?& z{c{CduBs{!)&_@c+uEt^<-hw=ucvAp&e);we4NYX$jbdq4su>@RgY-{n&}-GlHg0- zBNmmOjtky(w%p`m)O{ZNb3CwTCQHq^RI>{X)KO?3m7wvd}xdOA~LsuppL6VY3GlUo&@1&$rEc@i@i?-mkf;u?gwo1V% z8!3|w$gi?;PC-ryqvkriQn<*I2!wjt!jbC>hZG~m4ioMO<*VMQbmy9HE3~hLacpx` zdiwiURdk92wTeC!X&O&@2w;^~rXeB@0$3#(# z*>{^o7vhT)QY4^Nq>b(ef(NeKr`V!JO1{4tVlie9r%{02m7z1#?u#W%KnPp#Te-?G z`ci3R2`>g4mY2f1))>gJ_qNBT^nZtt3q%qA zo(v(a9HZ}%I;ryRL`m`pv#U( z(MCx|NwUXwE+`ya)I*T)yA7EQ4{4CDr7L-UEXNK)n5l5twtZI^j{Q;be_DXi5=O53 z)!K6GHRJ(SVYy}0pUe^p=<(2DdYlE;m{rw-z&t$7rlFq$J9Z0BbwP#JdXMf|MIT6);c-UVk0NJ?uEjQ?~{cLWEL? zMeeq1x&i}n1Dk9#)WASuM}4njH2z42cG?_2$*u4S%V3d7s5sDWpauc0C{qdA-Y}+Yh$-0u;StCK;JKHz^<6-PYwOIO7R6R|b&bn!D3CO(#2JAtiX_RhxY_E9x}?2(kZ-!rU|$ z#PG6N3z#I)MbV}+_F6UtW^{9DS2@mZ_6n65&`6&_@@H8C6oWx_zw3NORgyHl%_|JeJEU-#&!$9)yZ ze;c*x*G3p)h zNa^C2)^W#G3!OYO9ydFGXH3$+7VgDoku6Q}JWyt@N5}SXF!*l5yQKhSh@<^0DG{jxVQw=f(eL8b2hRh-0t4hYElVb1Ajc z9GOz=Ts#Y|gTjg*?*ZTKMtCoXdg#8sPJz5HGjedff+_M~iE%*upK7VTVNd+7#;oz6F$#wbGt*(-tysNwetJX} z@*Vp6q(3C-G41cvyiD5)EC6kvbwPE3vUrhln_y4YR?4 zSRd0Q^MT*^@0sQ)Z$UhBeNrc2trqn8(}C#b02^OHH;D0fKnH999T*J ztK-<;AbzTQPfSrL?s=9L^W6My<7ud@>zTUOO(9A1Cn|h*A%AcVWp54bSQJgL`{d<;RCv8sC$39DgO=&4 z^^xIWkcy*2a)<-3C>4(0c#$>-=h%`BH=JnyH8l7Eq2WtNXi;Dds&QZ^1KWh9rC&Di z^8+QV!7QPxi~ZVp`@i%?@X&!914stoyrAb$Qc+^#o{)EUCri;z>5lr`wUyOW98oz& zr1TqdY^SvQpFxX%<^i;iqM|ye^zkSo0#oM%?gdEX6h4TCwuKfCo-=sLtZ(n8*JQ;a zR#nEv4IvH?3DyG%3HTpY#0r)8PH4o`*0LeYhJTN={TW1GICtA@)u(3aOWS-j{VjWn zDDlL2j;f>vtst0&&!+9qQ8Pyin~cNNl6Anri@bL@xADgTck(v>Y-D3C+3}FB?{%#m zfnJ-0lr$r=0VI;bmX^=c^Kb}TsQP8YK!lRl{iC7+xbYAKp_!$*zTSTkpXT3anm_RK zLji0N7Zun8Ol)k$_wPfgLI7ER$lrATerVK=f2?(ju zob;`GAcO^8*38Vx=)(u3s%cp`CSJYzjWp(B{?VpxmV=XHQ1u^yir~p|yIJ4cKM2W-o3~!SH|@NgQJPEI#*u4nkx`h50*h<@ z500|(@9mk2c#(9Wl8mU}l)9nEkq;jv2R^tn@sv@LuQto(p|~C)WyDtBM6hTfA#*4xEu4bXf+OsKeUyN^Ml@1M~z-ijjatA6+Kw304nXaxpLCgSB z`;Cp=)~$mWE}4LPXgN7KNDL9?EAw^(aBz;6 z1Fc`n`&yCEAAm(?Rbn)(?XJBe#Ziu=h;pbRp74Gj%Y9WYsb z-Gmhz|5r)SE3MdrrL2bU>dLR0UuSAsF13Cy(L$~ zrvC^PadPsCS~wntr*YpGf;AMy04GwR0*Ughfy;dvIh_TN+Hkyj#op-oI`89s@8WSk z1`WzMpjZpXhb zA@Qd73140w(DgmW_G2A3kFzVO(4j9{T{FqW^#3YtwbOCNV^CUAhC4nrC1Lpf<-xHd zj1{Yg6@2YCK}wh?4fXzav4^LrnORZzP%(x%>%C$y!+nBRVYy84>l=^1BM@VA2URsp zsLeO%Z!n>gygEtY_V9V#D8aNW^{XEzEm;^RD}s*|9hovBKBN}|8ZU|z3K%qW(9X=v z^a}{M2f|-V%N=G`*5T>tzRu2jw_iMe4tlYTt*s@nCr8n7xpsxCK`DUoz>S}qoH3Io z6OhL5H8rG#|9E}$kRqnr{{EDi3}v=*^L%CVjDS$W!E`0Fx0;S_wZ$)7K z4~T7e$uY4bVnn=9!A$vm^!w4VF%YTWZ9YH02WPnPpmsRHV!i1c8xBeN z<3**G(+n9OT!~d^^0m@Z)3dXu4_nn<6el2BIzr>ux3;G7B#izXyD3QPzYL+qFUns@ z^V&xN^- z)kD+mW$04o7A+6|*l;Tl#8&)9k7w1< zo1Wkf@wh0NmG`qG_vo>u8DwSPwn{vr!+G(LC3))2j5IqkA$oyf#=*LL##i~pGyXd# zSnhikZ2ag1KQfX7_8XWapsT|5C`5^5R&Lo&Gl*^7*$ib84l6?C969hP?7Z;~V z6SLa=O>xG_(j<&gzaC$*#m7XI%8G)I6c&t`rKPc%Sz=KUi;AjhZXU(~;Igo-0_&y*LPp?l=Xhic0IdaQ*Wp{#PaXogcuC>!b~JoI zPTYDavT-pZ6ho_`nyHrrX$Xb}uJ7LE5_I@>N_e28jRzKy8L^YX40949_85Hf$obD% ztRez^e1zdpkmJ^phkXcCqrqtrKUiSKsZw*~%T~{@(9vhJ$$l`iY~hfFSu7J%6DiYz zuiwu6@2X@fDoyrgfwaD5@S)T0RL|ezt%4$nXG9?+0aB&k`50<;_lSKNKy$68m5^Ot zM~2VlkOEPtfXt(-J-Oq!;07{dwQp<}2H)WpoX#obHXFrpbQF}-5oV1x?A3S%rnjhXF5DTTUNx_*B&1 zvXMX}9#t^)6$;Adt&hQ7_@DFg%*@P;t(@Ze;#r4AM*PfLSNc@DjSBkjO5v%isR@l^ zBxu(E0qCIpWJjyPDOSzGPv%_on;bo47k}e-kF($mr4r&yHU^mv7$hb zJkUW`i+~)l_KzGPY8akjUHI#@a&*aU?Y;RLYFf0Ai#a*zI&$$!cI|LQ3Py#Hvj&+_ zL8B<~%=t;$z)QiV=81n!prJQXH!(K88!`R{hUM}jl{1DIEG-LZ-#;sokt@ZG7?~WB zv$n>zZo1-xutA{L)z+ptKwlrcvjj(4WY71rn@RTps+ywhUq`1}=RXhUS_l-7(D&}k ze?={!^W%Sp8y|K+$ongBl<0oi)+eLX@>TZh2Vb#kYQ}D{+S6Tk*IoX#)FZ(jBfKU-q!3@X z(DT|f;wR(~W(@ud94fng7MyLdeesa-^Y;|{Yu}0*cu(zo3Z%isjQF+K>hWBhjT|dl zDw}~&+QGqT==W?JtS=uS9MEzjJ2*L(JI(UKa4drovb1w6ML(qoI(b*4u~8mxGZ#@{ zkZ~v;4Cbe4YO{V!+u=UpPmK6y+=iUN%`9mYIe|9%jq-g0S0dJDS zIWPZT$C>Nmrx{1Kbs-oSpx*67&=C<4BiWEAPbR9vops>y5X!(i8%5>HSwv6DDA1lf z6(}Yy?s_g}M;-_VJaor!&xuh#geqQ1xahjPdGf3G}H!G@nIT;xiSh zcgbgzxV3VhPLvv=tqtfl+md6p>CFz}@bk}p6YvbMtAScgj2agj{GPQ3U&6oqc4D*Q zh{?wCe=+Y1&OaECi?jlqfsFaTdvR21HQO@}AL zt1j?-{BznfoVQRuy`8Z>U1Y!34b#b0&z?Ne)D54koxx&Yt zOip*EA4tx*uFYz+*t~kaKAgn__9*~~G=3+UysFRQ+?8-woqe9ua6(k9NRUM|s`h9` zK^d-50RfjA6uDGxDOx-jI`BFl8E1&?Zlz#Y8`Ri@=I}XlQVEiIUhySCp0$TW5ivojE2C!3; zk(VnmSjp4X(uj7Cb0`fieSJ?ea2R7^bPxt~_VDZw>WX9H8``diS)mAmz`(Qei z^fbQD{`@B?oTH%IT0Bl`5U$ap#+5HI?juU|IybAWJnd(S-};erG^CvQi?is~N?Kd9K8$4Rf8PJ;PYc?q&E{WHa0=R)$$S0dH}a~b zW)+@%dt!2O8yjY&<$UKtTj1G#8$M>CCiuZ%-nw&yg_Y~K-BT$b0l$W_k`gs01W_!~ zr-8N1!U}&afq~XmgO19U;p5zr6yu25PoG}X!EUNBzsF;H=wOUV@!2M{V(EZkYpgi4 zJU`TdcV|~(n)CT{+2seWzrOMJg~T!)oYjAidd6w#@d@qm3>pS7`D}xNxA=nPjwPlp z8A5>p1T3>#iaY`R$$<<6q3s4m?|)BD_H6#n2g4@NGKmlwDyucW)A^RAgW=Jc)~R&I0*!_McsVfE@%eA7=lJ-v94x3X+eB}gF(CPR!(lV^ z8+l{SNAYl+3Ip*K!)vRqF8&*JyQ<$IGMemU<<;Cei!&b%;=E1Yg#7HVWL_J?N1zfs z>s~8cF5>}n3n})*&N53nI{NNaj1P=5a&oq3hsq%RjE)|W8Tk1hQmG616I6c>AG?~* z8O@`k9IC}SG|Bzx5RYLI0Ns|YrAvZ`Km-l>VkrsCNfD3Qb4Y0nCq)7B8zI2W}Ty_jW@qGD zXD?Oz0zefvId!tVvYlAguP3oG%1$pZa+*h%Y1?QrfBx!EXT%WU{>JT^G+DzlrKy>a zl#0k~NSRC|J*?paYe?yMmY0Hp^3;^{kwGu{*aOndmk(>+VnkD=2H78PD$`{n64P|4 za3gMoRuKMw6h|*&!lJ<6HF3~*gKX-X_N8*k^pWWfBDF{k>Wy8@(Dv+ zNX3iyjFGJ_%#a&Q7cHlXF+vLuM&HB@^1d0L=_)$6&^^sH3Mn_016~vaqi`JPX%elx zgDCXAjrWjY9O^7y*j>i#v^&?hGJKtQC_!6wo_{-Xij!Dd#L zS=m()`SC0nW{(ymQQm|L$bjh;H0&Hzm3W~N>NOul57JHqMnW#iOpJMH~AzS_WL7LvwS@K!a3iS(D4$LrWRavM3UR zPcH}Iyi$mFz+u2l1(Rywa>MC-@m-DQ5W(9|{W%|lOF#fr6KxaW4<%PuS8LpkAHjl2 zFd?0;RR1CYnz{Deoc*5z{V(|VeD&Ac7Y~e2^S=7RJ-)q7PG$Dt!=KGqCfa=0ok+GC zi#Rpr5KDF1=w}?V&~#Atl?^Jzo}Ia}=+NR^;$kS@tbpto>J78u_aTG+KZ12LuLgPMR*6zJ(Ci+K*G7>mO9JdkRp>@F06lU^-H3dolAIHJ_5KdgzJl z`va2Hsc_Rtj#A4908)s(V~pMV$Y!vSF#E6CO|W`=tbXI%EGeLCsio9XrcX>!>!H%a9CQ6Q-PG~$x`?K zN7GryWwmu(+dx31L`o?M0cip0knV2jmhO^nfkT&oh=72A#0}CZ2#Bk0E3UqStr(2L{BdGyTb$4i5M!jlWya0RjD`DP!`59wjbX zrbTMqSm9L1J=?k`Uq*!enRrUo2Wl9$>*DmF7MT*$YxFan^~6+HYkXRK67smEf=F4? zx+DKc^p)HrI>)b3iw0}8X{2;W0wu|176C}U&7r2_JNMKEW2#U7QbzO1su(|Eu0Qpb z&iPxm$b!zPX`G55wfLo=K>HKDuC_*ncWtiGTaKtjRbplgs>HU^DAU?vb$GWX z;#1R4+*zD46m=eC{JAvHQtxr)Y5B=M{q_~7u4K`8iDj?X@D5Xj%Z}UT*zr+U%nO8IjL~#HN**cHDJZkOD^PooSqu`;W|x^>3XCS zy!d7PeUA{UTzRF)KSW1P?P!T+HPif6*kJhIb#`URV^S{7efBA_n3%ZCBO*8nQziE} z&hc<})Oq!9u2DjBG%|D%^&ohX zw`N%y%bzIv=y=uPiZ~~s;;@7?f>%pV!HHFFGfDjP{gY3dn11C;JZV)Q=4J>V=)eCI zMM=w(Lo}1ANT=T<;1*9r$t^>sQFU%tL0BxHp(|Q_^6~q$vCMtkj9;%#zrIWDg*m7d z0Q|N$Y0^3s>F@#N_56ox1CkeO*OL zK)!^-yMA3)7t?s1E|ZoqO=sEQ%yi$<|C=_y?_$0A+azt;nx(pesGmiRb>~+*rwHrI z0@c<04fh%M-m3R~=@eg)IFK*6VntV!hn7oU7wW{noO{z_EgBzU)Vapyl%wx`JI#Z;sbBWN9R<+9EWv2R#D z&D#b(coYSeO`9;)_DTnw(%<_ACk`FqHthJ8c-bKRm zKYb7q229G3j=BV&m>xLOD4>p2ygXII zwie=VlL9$(T|+}uPmHs_Yd68LCd5YjRtG8}@qwLVe*X;h^ggbmM?y5Jgyh3E;m_dF z8D0EIwpS;{j3;8Q#UVUJG+CouoqlTDz1^6sq>9&(U{Z9C+qOvDxi8Z%*K#R&?R+Mj zc~?|9ueTx4Mw^(+IY^aGiZ125R8#RYoxt4n|1hu?xiBTjxZ8&dH5Bn4=nm4qw8I4I zpronW8>Q(sADh^3evl96q~+WyVyr@`mr=3;0V?p6BnalUCFEKpF39{Q5SV0|$aq>1 z0I!&Y>!@&#t?E^J=B}~bC?%tq)F85w#GBeM8N<<>`VbfGeE!%;4>2sZ~nVjt4+sfC@%*I;y zVlv8QvfY!Ci8EFob5Z1;RE(@uWs`d6tE}#1Tk%kWHs5#dZms8g;4WoAcW`}YXJxu- z_E+&zuU@tN=g2e;@}Rk44ZVUKagYY@_3m?OHO=4snxNC@?!UHhue_Y?moaqj>NJ~t z?t`hLP=hI6KF?~FF-KeV#$y~%>Xxr5PQLbN;4%yf4-gX%icSS`4A}6j72ma8B_6Wz zNax95&ZvH#BJG3>SQ~*1fQf*gOeepXfSsKqGan7W49BgbF6nQczoAYqUT!HU2%mq2 z9o5W`ynpUhF_fjq74%*716ptkKatQW1wK{P1CcFe86QgmDPJL1%pEhKGS9v%ii+fo zp|dsz<5zRrr&JF*=a}|@5nNCZCskLVlWC7yu2a1Hu&_YfIGdg(rPCxSC&rzR6x^-P zJ6ES#R5SXbZ6=Wd?{Y5xD*H`<_F8Jo)1O&b$Wfd2m!DZ_j69nM*2HO@R!qGrO_O{& z4-*-I;*Bg?oKix#S@EQh>L9yvzDdy?wnSM~J=cbJ-452QDcxlh+~i0(S@h%Irk<;4 zeIuD`Q1iMpeZGWRR{d<`DAq*@x56i5`0Rk(RL*~?yxoBqwA0V6nldb_w%#Jr?IDLG z(O0|CfK9IaVCty0+-t_WN!xF$m-!_1PFq71l9%w`uWt?nD)wAfzsRRAPf7XTkf^Du z5smBkexKQ!lH08|b;>8=U1S7DR$bs7GrX{zy|x&-0+qbpbcbYuVa~q$oW8S>@;@c` zH8SoC5TTO~;Q3GbAmHP<2yma|P6?;iiN76{q{TUm7$`5g{o{?xPpCSpcf=IGOZaZU z+7vo;GP}bTlr?v(_5ILezB0mXN1CxGl4sOqW2mO7KWB2DQ@`rqgb~^jW3`FY5JpcqR}wp(*3zZ+U{VXkTytQy%}d z_)1ZAm~rcd`|yEhZQb6*m8`5BWfIpzHPa&Kn;=wM`dujI4dg<~thJzh;-yFC(-bO< z2vW0%q0SLi!@YOcH^o@{&>Qr^z_Wj!$(*r)Su5rMQTNLDQGRdgg^GSD1Ekf$?AuHLOT-M=5jIlaw4u zFM9R;U=iBgrjn8QYul^>wAvk47JT8kSLi2hx*YTiPst)pwXA0wBiq_&2i`fS#Lpe8 z#=yIS-zZQLsz{4k?IfD1nWf3_82xmBJ<_^SF+LVW@pYnKUxW=Wu8ir|1gO(lKn@tH~8UXf5FAcKGz?(^K(5I+ApoG;1dPG;QLJ_RXXDV+PkSkkDNv83SQe58Q8(m9T% zmpSw?U-C_yGUWZ5^VtOt-T7W4NMAmvwVfJbkypdL^aseI!6bgLFv69Dq=TFg^$QWt znHPd16qS769o@@^OcF*CA9FoAUUQ`yx?dkDS5Vw(UCe*2T0#&wOt|>uP2{yJa z*2luw79UDY55-~3V7#~}`BD)VFQc!c(q?jPZOg#KIjEt|t=wmt2hOLEpfgtapt@s| z>lh;%z8Is!l}K2<**G{VyIDY*BzET?^c%9yKUT)VdB@WxE54m)>5ic-F`I zSemn=`ML`d>7gz8%Yo|Rg#Y(p-wkWT04?-FMn=>>dqG^N?q#GLAPs`5bck-Uo_FcQ-mbryU4bv`3m+U9E#+@M@-W zDzywIsfB-=R@T-U6uV9pGm%bCPL4O(k%3T1N8tAQX;s-W-lFy>9=GJBo1;amW5yktO@{Y9PqxGc*}Vjq zXT3liQlc+mZcfwMN-6mgf}s`=9Z?A18y~2=V0`ARyP8jvVK80m0?BTUHJ>&a-aR84 zDOpW1ZjK1KLb%bb{nwl>6Cv< zL=n#?XLv@vV~~fb0yOx{KcCS7LU^%!6ZgGI(O9_+A zI{~jy9pN;pSFBePq{ZZGE#!FQnM^k_EF$BwAsJ>o$#sxTL#I2e_C6OVq?UjYFAn3z(2uK_-x|n7+|CAR%+Js3^)nDmgzO%D)Xmt3cS(KAc&7w zK`$bqO6z3d=Z(e=0Cn9|%e~OF)KbQHQHr0BnCm~x@3XM~2sy6gc<<>M*0K1B$0eLG zjS~h5t;1H7gdsu{DNDQh@52)5HTow;dM^X&V${?ynwS;Y_=Ibf^1->nMf?_L90o-^ zq^*&uDEEBe5eMV^tX!1V^EsMl5wH4Tel86%P~emk`0jZh?@sph_hYA%*j)aj5hN#m zSZ;WQU19<2Z4q0k)dI7gJZ) z%_$mfsaqc@xXJcV0_rbjJU{VzMhRZF3O>gTSETa}{~en^NQxzl3dX6;!_iu_&D4;- zO%b)%@6XTIHa3jBoIjq`q@)f~1pcu4E=`i+3%H`-)7- zXK78BPB7|N=#S0TskLvclu=r}Hf$-(%zsw!a`(N`c|-V1Dzaq0LOQhbqIL-|8<;M( zN8Y;g=3^g^U>2DUZw#YH|CeRK6N*XvrKu@8KCe%aVTDkqA*8oh9%M2ADFZ~x%*`HE zGLxQ>y?xBp#c_-zmB(_ThGw1F>og9V76;<+HXOTpk58cRI)@n$xXt=q(j}>&!%7=2 z%tH8_I0~%qJ)ikKV}$J&mR>rZ@l96Q-A;*q`MQ_LZB@NflQe>0tybg)#&sO%| zmc*{FTSSow_SR;SkDwRNH@MA;NQdPXxFhBi~^TB-$-=k6MnfkEM zC}MFTh;)gqE-)`?x_*Yx{Nzj>QM?HoUlg%`ENd)PT2Glv7xnx1o?s;XQOQ0qJ=_V< zpg)b?o*;5E?n+{W_v-*IxU)dtGnC7<;SANGA%UmGtp$oqiBq7f&)!6?v{^(IdQMm` z5Q-+`DM15_4<~c^SKP0JV29Y|*&46P;PgFy7IAn;%;go7yRwpc>GUersEc08gB#Uc?^!HeQ^4g&jf#2XeQTBudg&V+mBz)zV=P2;|^ z*>}&@QUS0&b0_=FAsClu88Df6s7xz6<^grB#zV^qI6tlq#?Vl=eW^QI!@+kun!Pw?^rM%%gvAKpUL2yPMDHQm4zVxQcztA^9GRKb z9QGtnv&P8J-yd(Rvd4%$WOQ|L+&m*kla`f@Uj&r4!f0QEK)2AvDwR`?j!|R^w_|&| z_{8KS_w@)b#Hb=8pKFzyVUR(H#ct!tUGv`0w=!&7GQV&$?@UFLu9cq4^^EI>>#~q` z`n%-jyo53c<{Qc92EEu}`jI}xDC6OMuMO;pevjGx_@@rs1Kq5WL`o<19AsUD{ zko7Jno5)inRVfs%F55D$9zyi8DxJTnmODC@?zArLn`W{vd3m8OM5J?2~Nzl%SQLuI-&^rJ9 zV~-X%V9_YnF@zaay~m}E&vIIy)Ac1tFSt7nn=XlOE}G6L8;^=+7ou-YXC2n&1U`Eo zTfy7}Y$3m_N}76&qxr=F#7F!vz6W2C9!*b%ja%_K&cj=sGb9bbSZ$_J57fwe=a;{X z9=dLi_10EcDXB#cARI=D^h2{~Vd|&ERKvRyQtWiRJaL!aDwG(t{~SGZgJ5{1#a&q&$cB4X&-gnmJ|Myq$V7lpw^vug&tq>Wy9vx*n-}f_3wIJVb z{^UZ7jXfx+sIjx=o(ZZa4fgCZ;a4A<=i@IL_4Vp| zdj4&ViQELD*fp7cqy1`@D<%IQg(F$IyyA4rpelx%+RMa0Abz!G&kKTQ=XpyhljkyqIw0~p*tUHg27#X|! zdf7O5q-p1}^Yc^QF;p^OY^wK^vde*@DMFM^4xuSLhyi+#v2t@PHZ~`a`ABoB1Urr| zErqEN)(V0BDO0huGT5i^kU*JYjvjOmX?Z%#1i);u1u)o4W2! zNPQ)5sVE#BtFjXWFqSM&>w*z1y{%B@b9tYm1AyW7B2a-yxc+=Y0lUSpCv|vhrXr%9 zeiTqTb?YBdP&8gaQPT*z1XZJbs@bz0x3goxGmdm!lCR3^X;DY<6hVPHtk}Z{;o3ST zj7SV(-$j?+&jOuQKD$j><>hJCIsTe1p)ZInAbukfy_jTy`Fb&92p_k$<4Z-u6S={6 z)|RfDIzK#LA-?ub|7xh)(HOm>`cU7_>Q~;hK*Ki%q+EM<#>7>1rXRT$|3ya+4h?k= zgihf$iLyRyK0n$7MJ_}f%qOPF|(@E4;WM((9Ib3%9Eb{{MA zU<1fE+_i5&7}=ws?->cy+fvUH`L7E*=nQhc@yb)|SRZrSG#U?NZoflCJlaHQ2r07P z8j!ff-eAs;B|0-k>NGZE`JUovSJ|3vPgHyoa9OEbr*jb=4Imy zY;1#kXTP3sRm!oZP!VI68yVbDQ}bBQNgn%QL7ASu4JNw8s3=!>i~93*v!3%RIX^os zF8|2!+magiK;|DQmd0k>(UvftrXq-dGSdH z|D}NNLrEnla@upvz@bp>zK(MzwK&s?4jvpVP|-S__8b0P$AyXsNHr;mh3XVR-^5;k zPWJ-x$Fd}toox=$9y!>xY&2Pdw*PjASZI2N7=j}(PcfqZ? z@zAIWAoKo5XxDVaky;FE((i$6Eb`!1&)vodL-&66bf92ilJGT$`{N~qzfcu(84R?o zjz({F>fqi`mWGp5FGjnDn%d>{9n(Qzi5c+PqD&8(_w5n*o+W>3vi~f~DwL@ys`NaW ze(vM~L_R`;uT|xiL3vYZW{+Lcp>il?i;kZpBi4AHsK3y&XlJO!QRlY3G$0YB%#iqU zr7TD!Q^89<16NHgjEM=fe4?NnsI?uUR_So~`1!NZ>P`bdCjm8DNQn|&_gwacO1<|b z#*FtS2ApDa>)fDsmVu4-i-Abx!{qg{P3RBri=6?W5KEh43qz;V_TY@+@UWZ~4e%xM z>3p4Tjfc8GITVZQ6g4GK8TsT*PK>KYp26HdIU4al7e%r)$0I-7ILDgqy$U6DbaWWj zg_+<~C`-(7Ve(H;2eldo+~u?>x7Mn^8kwX;<12NbjKZ!zxvsk$*1s{=yS&D#c3wuC zm;5LxazDh-&smIIpy0E-w3N9yFUZh7aXW#I1ZuctIA*p?0zK*15_@XF{5~PnM~4;Q z+KTRw3*Eo)w%zEDi+0<@7?Q@k=yPiT#ptie-TDX%t#1Wy5rbqmW*R1~V8cw5z$@U?ON^NKW~jN{thSjTU^{iU!D;@y$;W^dfz%HwaarG!G7q zNor_tHXZcX?UiRVfUdWt&rP^$(OCJipoORfF<#~Z9ELH`=Y&YOqf*KSu4YNsc!eO# zGcq{^^u=Yacyc!7sQb-yRkiaxAN*vsEc$Z1;w*QyLd+Y~#DIjcqwiU!xLFZidt}#S z@>jzdE!+4*%0{_yTvgikYT<@A>$^9kcYIGym|qE!)z|3+8m=Q0D?MJeEXml{ta z4=h|EpQ}-yNE5ow*{iu-Fc9$FTrz`LvSoeHe5}^=QND8=y+-kH7D;$PT-Wj&hxE`W zc{h3wTU*SM(boP6AyOX4ps=uqw?xU0pQzI1^H0dv(H+igVFo`*PojyB1dpU69{}_$X z#Ype}v;gEmBKb@Bh6;VMh~1ylm%Vm3_|S@i5(DG>;@tfbVVocr;d|LZ_KK@Iygfv` zH(gKx6pXXy$5n)683yjB-skT&#nA5Z)%8`o#H%n=%I@*p-x} zL37ex)p(@+UB8KRvXYR9=tlOh^H-!UcB{5f7%UEbn+pDOR&IW}QQp)IvYs$TCIANp z+f3db;xG3u_zt%Yx5g^K$>x4&UXp3a1C-2;$3NVD?Dt)_kp*P!#gpBjmhAQpet~9n z3&o3@e}CBhqiJPbW4_viPQD;b$pvD!tenE&>}&_NQC~{l+x6@ER zB_nh4Z4DRB)NHBLCDQo48rHr5(8{*xp;{5Czm8|a*eD>mz_DibZ03ulj;!w&^EWnXyk5vZGlFhA&ND#i| z4a)j<+js;GQJIl}pN7vrN<~%hHr&X7(jQK*#I1{MuEc+G!zihsu1KU;hwTY2k^S)J z1WO(^zpt8zx-b0$9u`ssC!CEI^Pl;VLe&<0F*Wy0wD#K^?M6g zG0J7e7aXc;rq*7cBuKgcl)$let@dXVyi5PQ44-iD3`dcAe+(t%xrO3p#7k1`o{r!O zR|^p>pI641rY0gz%H3w~!r=oi+kW{BcHzTcEw04Y24m(4x zDxC@gzTj@_nIr?1zLNJZ;k(6}FUoTM5{L&=5u#;T@ubMEunDddl(j&&IVXXgL%jc1 zk|)X$FE)q~a91!``C*Kxu#jvQh$I(YIlmE-)bRPP!%@-|_zc;nDOovRwuT~Y^~7`F zbs$th7Fysiyyv4Y=cqc&-tzNKa2KqSskvE}&-9{|E|@g_41tjU7ds<*D}+SX#(mC(r~n}~T~gyG9K$-}eyneb2Pg%nvG#vF(*eaJGuWVln@PpltTkU7#R zxS}mUlTnt2@d=uL=BB&1UU;Ih?)$6%{_Mn9yuC%Gklu%hVZIJ^ndl$(3}`T943G^)gxb-E-|4eL*l7D)g;SUR+A z^f4CsXp}G8sQ4?p?l6*)iho*{@95uP{a#CiX!by=<$qb#%Dtqmx=6`#PjpB3FT5mQ zzD$xXJ`Eg=FeJp4$qDBx$BLfc;lh`BQBN9JW-%Hb_;Wh*p9lZaqn*?jxacv=NtS0k zKi}NR=vy2SdnhEJFRJnyt?gb&aUD*7k{Cj#I9~)*O*WH~>-dz9$nA+yN=Dy+_UqGk zyx~-{;T_9&sc&VxmDTVRmc5$OPh@Z_ zere)8P&q|0)V;d*9*6EIyYp+l`&k2Rg>RkD< zo+~ERoJb}&Zq}E4_ybaL>9Enw`I-{n^x!rzy{mnNolYyY)vllETqq~kCEt3@Lr~m! zkJ?!?U@SS5f7m8|rw}(XF*TIzux;JgNid5RE!Se7bM~y52=AsI>zjN!(KiCBS(1&0QV+f%3zhlXB$aqSNY;1(^LS<)Vse#OQTk7au;lg)2wG3aq zB-ubD{O`9OEX?IbS)y)u{=9m!m<;&!rw9of7 z7UTzYs&-Z_jMUmz4*}wyqkNW2L3KOpGvLK6$ePk5FRhH6_9zk@6P|xbZ|T1d;W8WRn$RY z;U0YTKAF*q2`lTa?TgHH9$_x7UgpYo4Lr7LlgvSy9LU3~g*|_gdhREB*st>$5bjLr zUj1kJqG6G52Z3B=TZ7}aE}M$;XYqEeQXl^=?#0897h9eL>-oZDfX!C-r&J33fF ze*apc>NG(UbFh)pFwf&PD|h-m+b6#@HyR~rfG*adW)M7 zQ-g0#r_tG8mPcj%+lHw@+wB22*Wwii&wrwsb#-a(eC;p|`@QN2K@dy7xX!$;i@8z5 zv%rnL?_LkwnVD-ElYgjh{WOC~Hb{WeM{SI3Z^$qINg{_XqVQc*(~q zfqT}%f-XAZ3<=aG_eHPL(DM+4kp_xZm~?oUyJ*tFd;%{27pDtEmIHC^g2nP zF|i04S55brunprYldipDgWLUcqz{VyfC-&ZBhREv$9G6Qi`M%Gqg4w;6yGty7dV#iAlqV4vYlJ_FO@^n+Xvw+G!xO_mD7E zUfYuw%s%i=arOygLy&|reG?&*O|lE1+l95Bzp0b^>gY#Ee*6d(j0YkCc=vfVlI zx7w2ss!bU$AK$mS)7dE`E%kh^qmqO(^_)BVi=t?psB*2`n3TMeV}CaWGRXS~!6Ct5QX8wJOh z@BEO2!n5v*^@P8*aLXBPvoxY~-BEvB=k%&hqjoGTecac8lltq!$dG4r@7~*@wBt?}VC+nBYxwP}j{oau?_9s*ACjk_nct4C)Lg!60la1cKA z-W;}e7%-f#d4Tu_d?^mjD%#{Ou5?Us3Gp?uwY<7Kl;lAvgeJ6ppFoq`1CYc-Y1(p> z*?^IBGVg=?tP_Tasku%IzZ@E`w(ZI*YsG1Y%%s(*uXN8jhlW#;AxWkn#I!uNpY zh7ph@?-;_ZV`=fjf$4l1ty!ebfDg=fd^r`wqz(GNlf#X|62Ip$lGcN;_>QxgqXxG) zJ3A{DsorKz^EdFLV_;@(^2mp#Q!YyEq1^i<=oVE3no0KXJ~EhwWl165%C(K17ucYXV>#(JTv%Mq<6jDT zjC6P0$YW{F$Va_n^@NR!y=B0d1_=AY*bnqPujqgLsDKdSmV$~p1 z3eOn~MZT!T*gvAc6jgmUlDOT~Wm$19ydkHR6l?laOi53&Bcq@|$NkK-@p75Dydu%S zlUE2_@V?jPz9Gsb<+ZipeVgRCXhsHrMfJ&uHem5j9VvxGY?utgoJD6uC_@)JAtIW z!R|Gi=Y-D*Q4Q9R3$kuiW=NOy*XNsC*1HC4&xYR?J>4fHeY$da=0wcv`rnD~)m<1p z6@MlL5(&gxKxiP4+PQhS{W}(y*AUrLjcmjH+ycEBf)WhRva`$Y=d}tIsV~#(UqeI& zKU@p2ilGMd_j8uOeJBMVGsiKdw^zQsqvN4O3&2y$fl*weBRl2a5i%Sc~E17Fu`+YXzx_Xt7d!Mw!hc`dFV(L?;`=% zL@|%bSY!iPE`LWyBm970kzG_2i9o!PZo8+AVg9$=Z`w8wHorG4vTh23+v(}w8JLud z6T(EZi3J_PkBe>3QILBi{ivRV{en9vr@Sz&qCySr>|WOMy*~tvlG>gqwpGco0}krR z1nWE7B1#)x`$r}H=9`-eOCI{{AT96-fLM zN#)wkE-Y&_zM)H@NSQi=BpCCDcWK#W0|*pe)K|wm2rGU4LBTa}KPdf)LEY!9FZcr) z%};fThCwk<^|Sx1u<2ZXeRG?d7;SOCdVOmvE3YnIxdv?Zw`m6_7au;ft zzx(0fp@bVN8VjJ1iHWxkJn2()ZW+!zJ!50HNgqatU%-~&ayUS;O^%O>!UTR~Scfq% z^f)-45QbQwH{M)d)E{gV64uHyWh~|idSlg1w-=V^7;N;CK+sVumH?!7I9y+HPmyxw zPwV1}nHoBTi5jI*K9NX8iB3t;3dl4NX%;|@He=JWoo(!yGY|}T4lyx{rNNB8?u}G=yhNLrE%51ObHglpQH9MImJV)=HE`K&g<7BLl=C@A?zTeP$fd zFJduV^l!&sIft(?5F%HI$n!i2iQ@em@ebkBEy2FghF-D26usy@>_QUVAwx^pOwon@ z4{Of3<{P~Wl+gglf?*D#Jl@2OQ_R;_Fgy!8vTY2eh)GnI6+FQC$Jf7HAc!%2gc%wM z2|iEI9^8f_0goNV6f+m&eft-eP=WiL6`njsVte6q!$%WtmSAh*9u(I3upkl5ZAUh9 zQ(Oe=R7*?C7_!_E2qkL{dMF%~f(8a27wa|&aioc0BOMWMMpEkvUt@4F}1)BbL5e(ttAy}0v_9Y#1ePi_aH)PYo;@Dz}Z8)bscnNp%1 zF>XyP=<{iL+4pNxQ9JA*m&*ZC#G?pRdNtTH2iT|rKP`$Tdg(Yn4XZt2!)5Lm#w!$)TdmG8B zklvaG=hq@5?jsNBwt%O{1;bEYCPavZ2lIig!hVU6DS7e-kGB9IpJJ)@5u`f=ZA3;Z z_rgCcM5qw$r1#`D?&-hMJ(TVF}=$Xgom;; z99RL_LJN*HaJ(6_RtGht3>A*4i%W24HiQGR{X057$mX17-$XV^If|4kNR%G*W5xwc zX3om&g~tp?mSL>|q6NvDW;4*At|>WB8yL6Z(jC1|2bVG*J|zxj`Hvr>)MO>QvlPXY{4wAgt8dj^^FwT2^-UBP9Q$VKpdf1MUDp%$P=gV0^ge zeB+ND$?Mav*kZB6%ch{L2o#|-cvJBf!a>hbLLNVeRnGt2IcN@nYk+WpRnqIm^Tv-h z+`V3)ECpQ~(Cozg?K8IFlI-me44p=_Ikx4Rh2Rosn*GOQg|+Fo`qJuTxxtW6FvU)! z`NfBBqGz1{iThL<@ZoL)yLP`SCZ+HG#kuk+#`{AH-@p*tey3}N(%MUf%m4IB%J?tt zrg8cPy+iPuMNwE;S&dn9fN%nsVlZ~ykWPO@pofffa7HW(Z)q< zTS={~#95p8spD3jG{4mO#M;INQCB5#?uZN;>#!%3Q%$}Z&|RmMQY{1*K6_x~@2!T5 zlg{OXtt~2|^-}iYm(cNq8WqHyP(XE>v2AT_VWWk>vf93SXlOB1AC}b&@Fg1gPX_Pa zeSoy&8s+&wEP-E}hx->zHy1G43G@Fcw8cShFz3k2{eFw~*3XZPdJC#safAi{c9f|5 zVGr4phoR7Rrox%Deo+bkfsTb?STUi9yUZ3J4TXuh04^FpQecqb9fPc1Xoz?lR#i#) zMn%xwORrvlxsX0Wl@4!^Ywzs0$+6V=-vDVoUc%~jsW>>x z`+P{lfq9I#BXmuqJKF4*REfDGr(js2xwf;Jtyl6SO7tl)siBW;k{5@0NWn?dG%mXE zAp_M13ft&pKS4jLlm`)i{mK4g=;~?dTg7PYcNp9LfHJxt&^lk5ZFyik_&`sUYs=L{ zN?Z-^GrMVPf4*vgLIri(t9kkNe{~3b+B0 z?g@Rzf7GV@G@>nFavQm|*|uDzSQR|S_e&?O$G&qg;M1q(A`mq~GFJ3Z+T2P*Ry)@1 z7wl^_wY5Po3SvsoUMuGZbu5CTvd0>a=cmrs9kXQ=a=`+Jcu*0Y2 zCBX#eJW(1E5L#PeSxBokV~@Jx{(W&jZ&oBD(N@Ei&*1AUO5;2`WA^#kbk%gt$dvp< zEMKK$)MK-#>M04yZDu7bftjsW7i?7Vyr63h85zMZ28N2V2J}s)8h9h0fB5m^he7!y zLaPXp97L5`mK}{^O+kGeAczf7UkisG4?`)@v9KK0|C)kmsy5f*dg^Jhe$%n!EL_AmXRQ2a6FvAYsSq; zv<%K^d%&({YVeLwP9>Hx!)NJ>$0;tEJLj}5*05zT1Wiov)IwXQT9=f`XODhIes;d( z`oyHhaP8+rz=9V#8F1&lbt5V&mN4ub+RrT4rdS~ z%i@>9MNDoJ@--E2$lv#>m6W=Zce*C3bdEG@O3UhDCK0lyrzdXL3-~O{(T#@^@FZCGdFIMU#Z4Eej0+aP2|)FeKC78%pC0_d8dO&9dmFUt7> zar1_a?xpI5!!2FRP2IP7gA`q?5YRJ-5q+U;*AzHLyHhdQb6$Kq-0$q91kJ;Kg#P7U zzK++?Z&nNi0SDz`emkdxSJMt-w4LR+u?_1`!cA00d>EpA{+Wxl7s93<9hP6(-JI(m zW?T=>0>^;x*)xh~k*TkxnFzp{0z9?BkqLMj3>_R&8fQq4k0D`XUj!7t&`}YJGXY5^ zV*Gn|Pv4UWF)00Y>Ri($*@anf-l~{(&aJ<&HfE4hv9Q}UfdeooirR;}+`R>;qey7n?|IJupB8`+{q}HwM^j(BZF^ri zKY>i)`?$Nzs7oaWXSKs3UC9Fx_l6lWrb8dGS_TcuA?C;-#Y6B}-tQzoz!t3kw5ZdUNQAhK%@ufzbjg zenhaBK;LD{J^k;a5GqkK_4VW!`jg~1Q8ArnHLgd`a+FhyJE9!-|C&PaJFfFN^XFZV z9ggzF5Frn3qQMjNhUK?91t@^1YRUr(EMT*E#mCwWpvvv-Z4)PMqXE=5qu2)LW4{1A z(t=6*nos}zhkkf-?R#@qE9q7SQ^D*VmYz<oxMDC87F9AZr-3e;xtH~)=?4A{m*DItix74f z{Ogzlcmusrn7O&Rs7QhJ&ez2B^zyP3OHe71cswM7%;)Upk=^NPw;L} zA$TNSzVgPFfb6N6W`a znU9T+i;p_5tPqq=Zhckmq|T06Z^r#b_AUF{sAX>K^E3-)03w4_VOsJD`_p$LoE(pl zpsg+fJ&rgkptE_dqqnxgEvpCZ6zJ-?h6XAlf(F-H)}0YGHBwB;l!?+D|9)}|Xa+5I zL@zqm^t(i6X5NLD1E3n#h%I5RTkX$wb_bJBI!t{N=OJ3|{CwR6dftgx%zDrjqU^$l zEBvnZyA7HO_zw)ImF}|ny$(4XGA@qp@H2s4Wp!^ z2e@Ql^#YwBxGSCtItIaj;Gl+KbCYOn+ZD>OaT)V!_kA>uwEJp<@WC}1Sr-=Xq z_`RekYqM61aK|;9)&)uFYe@-NXlrk;>%F>ZaduMk<7<&+h!C7qRx2P?X30w$>}yo_kq!o`xKL< zOR!~yw7EG-9Fy8A-ENxtAGLK%!xW*y@6Ld(qZoGDm1Q|T8iB28N9_o{`R z)r*>VRy>pZ-vnaW*E8Q1Iry64E8&jGQ?anT zi|mlv&B-6#oT_z+kghpDy|{P@KT%fpLz1UKqDbA?$2J5Iez1Lab-@`MuPlYVOD19n z6E{IR?$x4K+u5rAe(Pwq|hg4e}J2UyB zp-O_a#+a_&F?i>*-kmsJUR<=``^KpZu6ZYPC|hnoVaSNCLh7@4jrJ1^j!rB#aAj_oHvH z$0n!6uz`%L{%zP2N{!>cPa!M{x(HBBPw@Th1(8WRALYx7mvnk2`qH zX62>T)qV^zaGY1pSKWVK&%ns!uzL*?*KL!sakKO{9LdHO9&jgqVCaItV_dN|vvQn= z5`eihcX1&>LX{9EY|iP@0Ou>xJ+q*bzsp_!MMmL7RP2pt+ZiJ?p-b$wH!qfyl=KRn zB)O||8(gHGd|Nq#Qw^R3T1mHAevsjonMqzYPVAPO8~Y+wltPFuQ8>wzgR^q4nxEQu zX5$$ofhc@kypioEq`xf!f0!tTOjo)Rd4Q-=!CW8*a%W?mQ9Z*>#)kAX-? zhs|dnA8ppcg`m;aPJ5RsuHmMzh+Q%;FpLrM)0M)%$1FRi0)s^nFA*BnWm@qEibHWTeeAQgr?KylbVtEzVZxS@muqzX9fm2HiZPZIBfs-}qMYt#ebDsB|7bePs4CR1?J6SO0s_(@jfj+_bV!#-Nl15tlynIQNFPADqy?mWbgVy+wd*ctbKhvVh zbDhcJAhJ{h7!2X&z$48Bk*wcH4Ldu(Udl#WVLySSFNDgV}tNV4+4Eq(uA*7SO1 zHRt){c9l((FrJo{HG^%eA5Y&hz7_Hr2# z(lJ*TZ6C< zJX59lUQOP13oV?5ss}UDYwne@eA3%D;0&@l^-xm(oAKeNS1L`}v!Z~t->2Bi$d}|F zWx`7tNCt$E{azL8+4gRH&^6bawJ1-#t$DT}T)`GDAvqP1>piE~72scS?2F|-d7j@i zrzke*_5*hF5fe;(b#3Fd9DY>_fPms-7!&3qiie?qbKQS9p2_hW9s?*`}kO=Y2WDm_|K zq3HBSn)jb5@$W=>hTJ_v_&Ic}u|y$G46H-y<%?lf(q+|P%=@VKS5Z@=u@4X3qE#Ma z*@q}T2xfFkSKN7B*j9h(M%sG>KYKq-1DlitIsH*$u?WT&vADy~gIR7({&p(wsj~Db zA2V7{Z~gm*esAsSBE5B!WE`Nu9Dr0X(>Zw3^pALhGXh;Wkm^F#?nZM2t z#MqNN_7wdNzC6^WV2u}OrxLJ48bogB0-UoX@q36wWC6iLERW zi*$W`_jwaPI~XGX4XJr_TT_U7OM)`8JLKH#aStC7b6qDB->C1ln)}ZeS1kHONG>&T z(Qht8TpkLg_=gy9=%M<3EEFtRjSAH&7@VMxi=AV0AVTDAr@}hC2=IM_!ysrxKTHL$ zkdGcOCsrCR{$hi^@t_g~pZFW$HEG=kM^WXcTSVjJ#}}r1beum8t8ge#aCZ)brqF5~ znu4r6IIQ==1N8ww_zNf9`KN=CiuJCr6_6W!tc#R~@Q4DA!O^+Uj*2qYpB`L3|S z&1+gY6Cy#wP$)}X?)55zo9_%wsd%z%<{OINfl}V$LO{G5M*ZAiXXn#jzSOjB?89mV zQB!co5Q#DN^G=^hX#ZH3`FZfY-832Re^`(%s;V2^Sp!8ql%2$;U_foaa`u}BMw^)n z{X9kAiivbq7g_xrG>ECwpK;a;`J4CirYh(tAaBYc>E3QEiT&LCo;AMRO^8lj z^MqZAD^;aHJVyuia{Y3xf{87t%#bkp%aJ?o`=q>JJ?SxFJEoT_(nkmlP_IfDHGe zR4UeB{Bs-_&s+xnh&Ha6(WxXZF@WO{SPtzf9;O-(q|wRTnOhUBj|U7&h)588RXIe5vrFuYF~4$oSqQSa`gzqTfdBhj0qGM4nC zxfHnLuy<$ZZ>d6UE|USyZE!?EQewf+*RWKa)7ghlx?~yUL55x)nkVO@bD0{2;3#Xe z(F<^6k|5;U zy$v&f?tPw~7bz0Pv4L33Se^pFrCOUW&u-Jo=zF`nQ7R0Blui{) zSTx+WAdeSvkz(;f1n7TLu_Q&7mR2@a>>M1T+S*TeZ07)RabFo9VkSL3sNfg6>x0)h zr$DU`a9aonNHHC)Q9f7PvRO}T1;|%g1JU1)dYoT*H=Q;5h~uvYMc-Sp3lVl(Nx3>o zXwEKSfXcjKm8M;t&fWBP-1ORcxyKyid~(`hF~89TrK+>-v;kOtYnXW-Xz&e5B+b0-n%xvFDr6vtIDLVOS92Poh{TX* zk^KhkI|X*ke~d~qMUT=?R0qfHez0`AM9V645L8gvko?GU@g!3yyx`mW9M`XfnuEn1 zF7lyRuJ~%lvKz)QuG;PE>gxf^8*XJy2d@H*V=U;yoaHu;4JUT>rE>H&+v4Ss+wI?# zn|=MtW|NG3O3LBSk6I{KR*V303-w$gV)D*im*eAiR|rKDlcZ;hRai}V1!!GSWGjQ? zx*WASjygd7*xogEvcj()7!YM%Bjgjucf+rivk0{pys&q z$pTOe4%;XiL+5P7zR*ItkeH0vz|qp*a()s%Ak0ZD?+grZvE62U1|b4}%VuMT5EdYQ{r6 z6Bpn`&`zVl$BLEP!$FmDzbxiqBtTc>TL0h9j#!aiDWEKZW2Y^_w?h!LGaE%DIE44I zXEFZ#PjbKlqR1BE- zA4)01(V`Rr=??G@_&>P*-fsBqTZsY4g`I+Ab3}dKQ2<>j)-TCMnX!jXdI|sTo1WI#CS{tQSpJF-8a4y*e?BC2S43F`2 zZ@yShN=a%NJy0WCo)KGIVo9Qx>57WwL(Yk~htKZzt|;qxhF=6mb{3i! zFqlMZu86)+R6GW$VH^ngN_1*FH#Qhxm1KEVYFMT9((xZ>rK96rBGgHjK2DZehPsYz z1eh_KV|kjII-+01q2p(hkfP5X!Vm@?2p3mZ7(0TYcGyH38ykN&xceU-5+ZmD2Z|mR zn7zV=VZm)G~2stSAzyOU}ANsTvPcb<$V1W#*LC3m)J4ueDv`hZ#{5dqkx=luP+&1Yra|R1k>1+7~<$tcXv11%v z2;>-+Z^{>{6+$H~tI*;_4j8l{oC3z$mCJ=&A!i9FvbJwK8$o+>RlZb4R~ZE97+9F2 zb7KRLxl6J_nqvnR20Q)lA7?P2QZsW`2I0ADEhY3VJMk7w;QwahVLc=&_>vS9-?Ij} zkA|aOW+H&+npiqQvK+cJPy~Lf{jTfhS2ZSOrZ^Nt1Sk&zoZwmW6$c0o=O?jH8g6e& zVHv`}z<@h|rmileJR&LhWj!pPz#1xFx= zJUJt8!Q-sydPHh)?~$KeivGMmv92*$M3kjY>|(2NcA}OkoWW%Bd;e`-3j|1=686!cJKL1-4M%VY zxO54T2VvbF*B}IVDSF@ZaruXj(aqOuEZLt{8oXjl@7l8i3zw+22Xl~tQo)y(_E$vY zrH|#6f7mV2K{LeM($c`fBCOPFcN_R)?7}=<`(A&~65nb5v24Co_df8a;NauyPiQ(J z>G^|ozxgi7sQ2N5j>CM+LxDQ?5L$7>9F5L4@JaU!8C^E_Z7%|0sZY zS3nXV!1%fQCCSP$tB8%XcG@u|{GR4(l0vO6TIcKcg=2=!%i-9Mc7#sLv(#L871R>L#O%t{_Vxb_pAKTp zxgZ0W3W~RN^?cB(Z8LDxui=N7uUZioAz^&k?5L0X+qe-8VZud|ej{Y!KyEOiLy)bi6m)C)$n>&(E-+x)Y9VZ*Yb6fpkn5{vD7ngW`-CE3Hi{? zD;hUTF-Gt0lS`klpY^Y_a$!m~<+d^k|X-P0EYm+Dw`i!c&AtG@H-+P%l zy>n`!HCE|IiLz05Gn@w1mki4_y^S?rMSO9X;eojH)pBFHO3?&)_EQvuI-A_Ln%^{# zf`Z;ArnD{D-9xG~_?piIijd=r5zgH*%EOl}zOPCtm4E7Y4fkXTPg)z?lhMti6xe#J zj8R#=qZ*iD5P?A{t6||0KHchtQW3Xxig6mROq&ulx@C;`V0j5A)mjMAmBJ3~bJ_o8 z0ZeC&@i`q%4I;x`^QHas{Up`qXl$d#GJbW})&0yLD>1{*X4`Ry)_aX45w@t__r+R| zDTb_OOC)Yby3mhFD_@Ju?L^mtkjKyPD+7YWHc8=+Yqi(-PUrvJpOLS}reDhreAOAb z$a`(f5-;^PkT&#Naq+T!eXZS+@pzDbJG&DIq#-aP+VqjOPdY~WY3kC=HL;%0wKF#l zPp#KFx6kRij{+@zwpJ0OK4Es(bDjSdvp_9opqLdJ>}Kz*SFSFO)tF$>XrmDPLeb=- z#Ek)GL(txU?g&MybtgLpuB13oT{2HqiHLyv>9G3?6TZ)FGmh)*h?;M8ElH3e6U`v| zV&P`dys$mVfE_ zv?22|b`sn<`nU6#oC8y#(BzLT~_04l*y+$kHf2+!elhFgZE(R#*4#ZVO;c zrd5s+GmMmpUYP#z@@fRo=KMvEA`v->p{FN4g#AH~4ed)G%8wm}8IYqfF1`Q00-A)s zU{5nJF+mdJQNj=0taoL6!vrTRm3oi#zJkA<^K zeeNa9$``0*!hfVM3^i^7BFrRpsOgY_@~b)DE}1SI@F4;f{eFyy+miITYLTZNH#ZhP zEyXgiC}GQHnNmf-4^@gKi;H$g^azw!1zOZbS2$z5=kJG|33BvA3*ydyNSMi9^q5P0 zqzu!1k3OHDpvlsC)_m&)5~3*iaMwsVIlr8?V#T(W|XD`;X9&p{s7}X!J-1ujMnGSKCco50;QAC}r^~Sm4-U zSNj{D6CZ#5#@5c=GCE)%`%(B63(0EU?TIRwY8m4Cp!B9D(PB{r@LzaG!2|s3AD>=S zN_4a`UZ_0iVJbTOPW(s{nd3Nwd_DqCn;lkxkUwqj(KJFTz6RUU_J6Oc-KF`j5j_<3?aybNFUpbli7^V#uO;HQ z9yom`3Sc{?iA>jA~i2yZ=m=EWHP`Rvq~#RuariXdes6F zz9AD9bR-c6<|OBrCX@>UW2YCr_>c|f_H*Ee5!&?>`> z>Cx&lpHSd2LPs>378S~KQ`@Wko3Vg9O0wd9EhW{JHAR5<{%z;5^H!&rtt!KxP{~&D zEE9S40=3rlvuL@e1sX;F3Ap2lu%j}Bb#MUW(bBDv-1};!2EM5A@w^b`V%H~6HaaA# z)rp7FVU?C8{$?*^1R#85Q&2}Ah6EXLN z@8ptyZp>EU7hkczx-Sg9)qC$RmevJoD+l@;!GKpETJzoB_QSt|H4nHTJlI)JA@3LQ zqS7a#SV^#O66EGQekbFde=bA#P{4(muBXehF=9@u&$0NWlEurdpq0%gZN@UkGu_j8 zHlO}$-2AoJ4aG1LA|yCb`5ZoLiYR2bI%zzUe(DM_2-wJbjiUnET)fKfuUh4QB zX608<`M{ruFJa(r41s>de53!!QNn-#IEk#~4iG2My2fM%Q$e&0EubLH|FzWBfx4DJ2d2C8tQ$BbYc} z@1u%+qs~>Ll;HL1-)DuFmB^gM7#fvt{kkpL^az%n0vnuBL~} zOd$<09HK2aAe+ioaWqSg?6NGgoVo)@`}!k8{-nk^{Wt1#q!`Gz=JS=A2=2sNZJqM@ zg`G?O$Y5as-JAXL`Sae@(bgAv3~08Eha@Q!zaX_MoNRN@QosLEsr&tCVti$I)*&`Bf`nr6MnQO&)z9eg*|EMp8e?Gt z#Roqr|5)(}WxW%@b~M|EZ8)0k?%90(+H3cCnHDOdjs?!jdj4y9CcBexz4M17c$dv3 z`Edbi%gf@_;z@$1UkFX&Xqk5rUM?+pd;Iq`-*IN=t5|&Z>(p$PF@;cjE|Jpz=;S|t z8rK|=j*k1=!MH;~fp*n{y?#KI#DDRx0pql`B&@&yFtokCK z!tHST;LP!GSpopDErL&Yj{d|d8p0RLB9IJwUw=RN5{3%i49!*iK0{!cU7MFrnFg|T zl@?~3k==q<3LJCWs;yyL;`LZ#+1<4!6>!6ujrLr5@-9z_!D0E23}pS3ZGTt^=LRmU z-{F!)GrIqbbFPMM+dTsA?Kx_-?QEy2qF@I^vp4191wH1CXHRz_=LSRkUJy+^`(5dq z)um8o)P)7!xbxmeitK#coy_&`+Sa)=8!hMr8yoS_A6DBg49+)R@;aW9DheDT<`_m} z3r+V;PxI0%q?wqReHN!4Jn6>}MUm~mgYJ|zAJD>vepOkTo68JOO?CPD2PdT_8BLX> zf-tAWUn-wx55p*Wg*QM9w{?jjPk{`XQ3jNx$#TGt5~Ge@EPp2GYn}{ZxiS}hGG>ip zYP^UF!wCx^w=*)}+f}p9)SP*7&wB_5EkX2&(u|47fr_`v%{k2z6*n7Z1Kk4DGO$BYof*jsl&{CBvECY!jb+b3DV{? z_)|E;UIqd;45H7w(n}7ye-Snv{c$;+npCEc`8}~_RQ18Pk-|IO%KPfk zT;*{;nrN-i`dKn-3ihViD|ex z2-RUz1ax9Tp8;3f4D@+E6f3k<9Jn`5pH3Vfxe35C&2|!0jga(*BZ2V@RFJ105um_? zt-V#SD3#(07UR`yi-5CvmH}iLp>PAil!V^W&v_a}xQevEaMPR$s3ee}B8^-;`~WUf zNKLNriW@zLF)Y>|ZmYi5YSts*u|IeEkM2t~kH0y0JgFjzZ_P~7mBQ-Wjq1ljCb)TK zf2rZ*s1J~pLz66H^c)n&ZvlMvST~_;NUmzvFs7Q%my(vMsf%&C=h5UbH?>B@+e5Wh4Sm9#f;1U+f^5rD3Dmdso ze13C!O`Ln2-tUAbKjd6l+{(I?YdgQ_xCt!2dm@c`^f)L=jG$|2n64Isf01GVL#{Ut z5ImVoXb?&hf9(es^Bk?aQg=ZCBV>9ql)B~Rd)E$#O7d!SibrSX26&ndG1EM5+)a|v zHSJV|Xre*hqFkT`L!oYhNG-WKfqt*;NwI>TWB5DNisz`%;iGYF@)J1Bus|-_Q>dBt zuavU7Ay^O8dUF;dy9#(nzVs0UeAI;73Ql_kt5sjFCe!jE(fi7mtp*9w86#$#&h-Fi z0OzL?(BR3Cqq_X9-;O`QglUkUt^XKE?Ek*0)d3qkEQlj!h*XHR|1q1w$4-rDWMX1< z)a|uQm4>tGJA#L$Q~q5mU;)Mr?|?+x`SpF^kL1o!raxmDpS-xfdDyF%*YkrjeJWj= z7Ctx9Qq(zOPg6SBF%WJ(TD~%V;VO3~!}uJ-$N=lR^^>TnZ4&!9G`;kwSCk<$W=bps zs7P}cqxPhvT~sLp`rIVwXRf@UX~sO``{Q`-DYW$$$@gPDRv8&`A7>BuP)qf$c3Pij zm}-@Z-uX^KFR$H4bx~t!v-6VZ*Y~=tVhwh-<885Gx(KjYc^FQ6WP6YmjATY0|L`2W z7B%+IbhNe`MO5H>`q(;lnyb?nyM?=hvMJFJBjoE5O})dM{T?UnqBMGsr(R{9exdXx z4py+)c;WYj%c6Df%;!_ zC9X!q;ltTtIs%c$a#9q5#w&{}ODiPEOnYNy$1{QwDt-g%x+^!&P=ftv|8e7NYf4a~ zgne#Dz(Hfc2ts#~cyiwO6qQa<=Fl$-C!VfCmgq?!VUh$1Q{nF7wBL8UoB@TEdylAP zgvZ7){QU^BC3<4wpHsW|egU%jL**nII>UBx)Y5N|DRyF!ZB-n29ukfasl<-|2tBT1 zg-$vy<^)fP|u&CsJp2+kd*vY-^O!<3H@N%*lF$^2%wK+4pU4zF|JN*Su`- zQ}|VV%|qW;Z!qvP*oZI-GYl_TQA$Mq@rjJ->*4-HunaViX^3w(<)_roHsA` ztsJ+T_6$C|uZ!#ChVem=?^vwGgqQcXJ(piR0Yn&z{#hB!SjiSfOnN2Rbx8$* z&qXo&3;oy#{FXT48WMuCaKh9Ri#n9lk)zB$GY3hFJ8l&AxG~AtQcKj~WzMrpi*vk3 zdyeOe;MjqJg@gU4hd^vYFZgSoY~!AYGMNPzraAhR5V{na6_v78sCA*6C*j|qpaL_7 z!aw6DdL(P(ro3V4&q^k2MNzC~*LDRxN(eW-eaqa~>Mc1u*&O{E3{Ez}eP?5L6(VMz zrP>d_T7LS_6X7))!(5E{zs?`2MlWAoa7Q}RFH&wEJiU-T!bS4C9#AZOn_E{R(I10+ z?C>}DzU{<)d09>=>W3%6&MFy2HQ{gUH$o<)c^ba;eC+ma0#g;`SDnfqR;XwcH2*bx zGG;L%ZvOQ7<2zhWM%=Y`F4Z_n1b*mWs%&pu#%>mqw)M*PlZ|Jeri$aYWqSr;0*brSy?&6}QZ~Fq+*GEujn} zB4}!;u9eQBg|d8$GtLaLtvpoSq(I_O1}!I#8y)4r<{pTkwGzt=_R zmH%wAR}+&SE}V=g`W|l+Iy5+{N@2`RNY!Y8X|Zye=(mS~n4?0t-F&$;X_d{>E{c9> z%(9YBbW<=lfsmlnY9Z;v9haF z*uB}7cfCfId@AZATk3M96id7t5hPsA+C=-s_|LuWdj_&l<_Be+qrD9GQ?}6UvecBe#IGM2jk; z(I!%uwLi{(?$-o`u|sXvP8{z_qTj;%yX-Q_<2u0mS;cU+Bo7Rgv75YDk=8Uh{+Ski zx03%cVDFyNjs}2AmvfB&Uia})j&yxoWZ~#|7*_O4d)vH$%|1F zgy4?kZx36p!xT1<+8?z=b{M#OZDrjH!cXYPU=xDxq@T!9nT);jKs&rSYKA|SXIV-Un9+hhb9<##sd#eWqW z+_p_7#v_j8`g`rk=vc|khJ$oJN2%SKgz_tX7@O4_te_;Ggx-vN?Gu8CXQa*ek+rjd z)IZhRObqb-l}os4dNU-_QAOqseowlwZXd&+HMm$pgsj3BK!b}kwczhWI+*%Wo-N_Q zS0G8XWc=Nz8{F7HdHhW0t5yL^!Ot5OVoD*@Y^T(N>Mx1u_3Q3U#gQI568RYCZ}4PY z#s-D*xp~KOv{tP5d&zH1}6PXm-~*W768HRJOJ)nB6+y!CyiOi#a51Ll69h z8NNtr_ADfexfTqwkL}&suV|;DwB1YcojFH8VwKr;RKG`b% z-nzTBGE^7xeKzG!^f}J`fj%Z5BHz%b7KmGfNB%5W?yp#K1|GeV`x?x`r@}2M-~2BS z(H-yAXup^iAG=}G;i@`GU!WJ;&Fqqelp?ayk6){?53^vOjUrXM5ytalj z>3})I5pQ2(_vVwoCQIbd`fA+c+h`@;{^6hqVC|w zu_9=<-oH_IMQ8mtpq=GYl34ofA5lK9b$@ac&4p8e3W6`S8h%EZc-udckS${^Y?E?y zrqgA+O2mwhznIokgKfM1MD3chlct$G7UQC+#h_pbT?;6Lh!fQox|P zOf@}tG!4$eg^jjPcTZ0L+Mo6(nY_qm>53$6-N$BU=M50X(P78R{(}B|Oncen--c>6 zLixjdmAX&1;fxHWx*PK$qq(=s9OudNg845gSJ%u*svAexG%%^ zy>QJXyLiF^QH+Y#A;Uft$1k|r%5==@PEZVQCR)S?vJtBTRY?^mTJnpI#XJJEfx%Xa z#$rZ!D}%X=Tiyoi1+X%)rsp!0}hVV$^)h5s)2xXSHhEaM>BjvadKVudQZlWEg!yiTHk(pHuf< zNqpH{5k>LMwLm8mX6=K7tm8_~7wCAE+sxshv>aSiW~(})H)DctOw7AZkgrC|6IqQQ zy@cg$JvV7^9V_bQ>o*;EX46>Yf43&l5K5`p>Bh;BwPo?Cln8ta%}XkOJ2@W9Gcsyp zam`ZY7r3dP)e(ud(O@Ku>D6(yOzAz(*>p(=H+eXAKPvNBUri}?wMdluc}}kyO~|99 z+{@?(ZYvK1$_R;(V@h;TSS!*g<3A?)k3zHB2~+U>l!TX;*3=Y9#0M`_!!yk(${ zQE}KBw?EzYN>7dQ4xN0@ER-IlRUCvxTA2802!EuRGO#y2yRztOKxdTj3ooNS5{1ea zZSv1CxvY8OGv~f{wRDM)_q^7Ko7(v&WqVaXg|j?~lxRNN|JF4sM^v_Fc=6sn7&YuE zFbApP%r9B)gpP`mHPK){s>xD{Y_$@*tMkynzy=&YC3X}P$PpZB;>)Y{tfEmA^|AiU zjX1->2;{X}#97OXtWojnG8&GzS~`k*4mmjHe;&$eHC+0Q^NU~-`c4-Mnp5bEZ(3pH z6zE(DJ!54>C1sS!86CH9283VBtB-b)8>dP)ei91hMc02M8_XDU}*AS3~mF5N zVqz<&d`vMbtQ>19=A)bac^rS%x;hTrzyS&~gwWW6K$1AMukmdEz#!0B0Gj~{%S;A9 z>Z=s1B{!lH;Gv#f8kTqe{!I!hSn2PdhfFv3-}&zHLPS0HY2N2_Zr6hst)3(Kh2{)} zeuV{`Byavs>R3d2U7k{t15Ys+eX$&lMP;Xd?Jdnqq5Ddaly zlTJd0))0!fqnjAH=2QV+i;7~QqQaKYEQZyzO_kHmjsbNC7)<}rD8h*+ImKnNK1%T# zIVpl~;bO4VZf(b%O}Ea|**b8W^XRex#>Qc=P^R4o-i_Ku!T^02ZV9&ZyvjyQ`6-z0 z%hF`3AACy%$vWC)<-!RP#{$mC6&#d1XQ;}Z@Dv?gMO_^=SS&GDrglg221A#gF#NNu zsX3?ImG`qoK|!-*Xg#eS#o?}zalnW~lmyVVg?G)n|1S#w!<_**=XX$|<^G&eH6||L zg6oo5s}*wPny%TA&Zdf82>}-L@Iv73T{1O+$DNEvT$K%U;tRhQ)1v_z(HE7BK%7lY zB@?H?g}#o3MY2xQ$D!?vV%DDVaf}JA-AB)UKV=z_(1rY9llrS+}PoAy4994@0oB??oWy(Oz&^oFV^>`rxcpwkmNN4T40nj=y zI9Kg^x375$N#&>*Z@2No$C#4@6Ov_%AB4QT0ZXR~MarX@e!m+W(a&sgG6b((=kgBv zKJjxIdybU&j;iU`L`D7E9T?JNK=V03_W4;kB&aArE`{sY{I7`Opt~g1^WX*Qv7bOA+bbPc5>sdmUzNL-?J&=ncLk0-ge^20R%c+~CQZQ_m2wfRZ zJ?uEGh!2gC25uqJ?jRIdm9 zSyOwH*X0N&=4^HC{=COb`LO{EfjnzCz^f}c?00SMYyX4zgxhLS_&QIxYnaP>|I9n; z0Wy}pHg{XwKF9nppD#E2G0#FuU^_i0EXaIga1&#uoBi73|LQbZDG5x2KT z-gJB2{AYIIh8uH?7B7@tNT>&b;%-};dA#n+h|Slt%>&KXHMe`Jzu)ZnWtEnu>qhw` zoenDKEnGuJn`||dzFCBUh!#K8&g}MX{2qK|QC)q5x>1K&&DZxIKf^G3`*fCke$ZdL z+F`Sm-4YqeeLG?bd=yL3T}|x;kEAXre9k-?{vCxoot-+YE=$bcT%@4m&OF1D8+&hK z6Nyazt2g(OXKZ9?1V1! zh37_{WR+Hyo-NM7rP2d$5LW=6fF(mI1VBA2yqdW^S@q-wmu<7pol=FbvV;M_J2Em8 zh;y*$k6m%whK(E6n=60DF8D%>W2Q6x_QEF@&a?CLojlEl&Lp{^g|M(g7l@;Gtoev@ zLN@U6JI~PQ5#^s=HZ?YLt#6wD5rFpRPRJ(m`n6vdxz}Bf>&VpSl%m$tIKX3XrhAZV zRwhRUwjMj<_ZWm$7RS9%);^4CWgLFe;lH}o*F?RH$8o@GvQ2do=W z^Br1WKS`?Gxn6t_K)td5u4StEly+=P6@J&^8M0=L4OwVBjF}(L*YA&v{*rz7jv_1* zQU%l3N3XAbxi+CKl9x22vi?c) zP}g39-dBy3+~USmR&9!>q@jz)mlun3g0xyOPJd6958a3w9g-ego&rX@TrM`x`)~dE zl~Ca-$2Iet<1?R29yru;2yyquE_kBBTtwjUx|`W-8<>2qOhJQTv;DS?`Gr)D&Qj}v zyWt^GwYB3;w*gC^xVoB2ULG(e4aaglh>2_adyKmBCrUry2o;aQ`LrYu2TiXns$?Nd zNKvIYuKZi4e4`|6HlP_vf*}0ujg(LKzYOQ^)!*?#1BgjT-otN!_$leM&pkU!+qCl| zBTl*4sdqkOtVCZNHg9s?4^dAv?wrB%Y2;6z(gRj0z0z`u26BoJ zrvgAI{@FJQ+VmmEy{m^-U4S?^-~muzkwysuBO^X~!}aMbv`(5%rrGqKkavD((ev1K zF8TDS%xhKys{I9O$qu!x^Vp}$ha2z^a0BRBtGK$_ttlh3EO6Dh3yLBDHUj$J_rMx% z+uRtenx_EPjzdjEiB~yVE%u~?X~I%joAw+S*a^?xt-;a@V7Rz_p<|5Frn;0j@4Vv3 z_+2OwA=ju;swK}rB&tr(xphc z<;p=c4RlOD?|bcgKwKXThyruZ$n)CJ_1MvrLZr@!z>`#6<6>vZ9XHo|Jl1MLd|TzPlYQAn%Wi z%?=%ANp-I~GK9$%Xws%AjLgoy7D`_^FQ<`H_%uY{M-$MelFc@BzwZZ}9yigq5yw+J zW~`++-qiGr?*wja7{bN~n`d!xF(ustOXLH$Mztb>x;p-uCTo81mEEYBritSHFzE3gwS&a9{9Rndt^ znLdV%M&*UPyYfESX0ERA!B4N9-xnx&JIC3d80hckgoBbRjwm|^UoS8X4v0u*tF2ezO|ts+J+v8K z!YoUs+vBsQ;Hm8nPWjPImfQ2n$0*4DRzFN1khd*DoBhggh#ca0ngXBVjqO3#lsO_D zYmEmnWO$@0b8~a}Fl7U4-2hTr5t6B3c4hmo(t5Vxd^sF;TQw@M0>Ns9gN9UY9sa%| zZYGKNLtXtj;4qC`cBXqKS+%=!RRPWorykK)Lg}WEYu0BMFHl=vO?bD~_wEL7Zcgxz z<6HRna{<=q-Q};?08s<0zD@@cm_&ia%>AgR6pnB`>m1*!=pb%!*)I4X07s44IFjsT z0FEEDCqBY#2{5*m)G>9Z3e;q?M;%Z8mQvzJ!l~JAr<$c%s zW=mR9L;X2Ey_=|8Tq?x6T#&qeXpLq_couc9Br{%xqs-=WMg}G+utW{ZL%s~>KO+O} z4MV&vWU)xQT+%ngAB)Jwz#vj=v7CZD?g28d_YN;`eHJdJn%iz&mKGrI0%@G9l_76Q zT2f>9`W=M}dX!lQt1*Q;{XW5HUXZeU?N?j-a;mM#2pI&99FN)bXkqvb#vCEBxjo4W z%C&fX_U^?xXD4S@81pQ%N^!^uTj7QK7H}g+MwH3j^k0A_(RJQE814hKcuU(0=TG6f zYEk8v`P-W*3kU6g49&97dpX-kf3-0Cs>@i=qq<&AVRFi^-V9*dAGME|@V*sMC-7@w zFBm6%!fmrWf4N7W-p9ekB?28t6cKGHi-JQ!t!SBz|%-gtdM2F2dAtdtb z^0X;RNlk5{@z$}H#kQ%b{^fmieD%f^=a+rwd6)yZSHcTB*);hJoOLRN{8uOMBX3>b zA*~=qfmaLCaHzUdGfw+Y0mm2TkR+MaQliUWpkYBO=isq!ag3@kQGCw_}$qN{uqeI`GsM|+^=M- ztrrwvI+bE)?sbk9%{zusuK{WPNHMmAJe`udO;1s!0yW&QFmvsJWw#c%r6x0h<#eUy zro;9y?{+HPK#Q>2`URI@eHqRhf8kKiK` z4BW54CG-G45tkh|q6`K&up4M;ym(B3Mb0M23oa|r6pWaCIyP0L3Gn+81B2s~L~CjCXR1Z& zBC`1EE#=lMV}QEPuwYUV1^;bqvV<}h`Q_-NZ@Tr7uHJ{-AVH7nvR*wOt`0We-Y~Sg zMsjIBTBeV8CU=$Jt_y!eCO?+I;3J_v;`BhCM~Ta8hF!DM6iZ@+-RW7-p2(EV4F~7D zSgNhg#3&2%!e9NfL-%0Jk^mEqr@o_?z^&y2gFeLgJAu~xlt)Jw^sXoC${ z?4#R)v1tNQu$yh>?wtuA+(QNr_&hDb!3P010X>iw_NEDXfE&e(_Jfzbv12B#q4{b~ zCIif69-NFVtQDv4L-I$tSl#`wKh`mCp3f+Upco!VE8vm@E;09OpLk|_1>91 zd|&bQba%F^e1$$4-4i?49Ofrxd=~Ew3Em386%34{8Ddf}M=UO&Ae5e^!?OI>ZP=Mi zcM1d$z|8SJ8Z!m^s89D~v397qD4efu|0OJ#{^=zE&z{hX{NLyLGd5vK%)ybyzd+mZ zCQ2KfgX^VC))EO`4FO*>;1RI~8GUkv%Qe6#7YykBvq z|Idp6oYux6XCs-c*HnmwxjgIl{`VHq2YntUutR=dBNAS2*QwS@gpQj$J@avP>Q zVbJ&SR&`+&6DW=hx2ubt&Q6|lKi`b8l@2um4~lZ!cC@gnkPY^b2IKus?s^vR&HT0;YP$ z^K0)0q1zaehCpjys3z3!H?=ik3uhZ7|G%EjJD%$Qi{lL;WR#gLDKmSID4Xn6wp^R+ zS-uSlA%yG@vM!fgE?G&}NJ3WENLE~%YsAg(_~+*@59_-3em?K_Ij{G5y`GR-<%pAR zMCO-f;t4YPdP6*^I-A={;UTkq^<4<@&jn&^xOP!0diD02iOiC|ESXDGBy#?1Cpo28 zo+oa6$@1CiS)1hj?fZ#Vu*CUC z0Q+OsGp~(Bx?HVB7N?%rnhi%87M=`HN6(Ip1~&c3n))8SB&}vl&tPVP60XNBufBCK z&*-yO)k2wO**p#K{iI)rIq!)pt`ydmrPsJlp_#zpym%$?EuNG}B(6-iCW5u4#<5ce z{K~B`vi<&b&qU<(!xfe4Je@iXlqo_zlJhBkgf6~QtHk*BCnKFAaTr&IP9JRKSt*_T zk~Y`UyWxOF=|lZhZ8Tr0%<}CkC;CH?pK~`+I`8t8to&uDs5b5X_1e8H6JYmRRs2_k z1^hc31cq7)Me_i0L_${fcA@PpGX|)c2w@k1th5&!Px}O{hEQkCFNiI!3szx&0ob$` zAmQVBq*bGuke`0mW3>9X?X+4uSB7W8zbTIB=QlQ>wXCp_X@h(LCs>aLOX;~#LYJ1b zAg%%880$Pnle-bG24O?xNuq_Cn2MTO91u7{$%5eHAiq$n$}<4j6ui*Vwg=)}B_|4& zVN855gDU3sOM3}n`}v+!&xwYZQQLM z2rY~s{koK`917!BAKv>{cO-R?jkWkQY0_|h#&3$s$UJ?WtWo&bNdffM%klE#dOw>N z>^>B{wNtaMGN@YS>qV!d&(nJEgrmM$&<{8JtiS!nq{c<>teQuuuIjPe-}%>Aa!Bl# z`;O0}d>I3ZEh~pj%LdXsQe(&Q-HWk<9`nA~y}$kkybtI@X7qA*Uj+?mzMXg=D`}&S z>89vHd%>$(=N{ju?*b|vJ$m&4%$|DU9700gz`PT-5b%ClqJ5x!)O?2j4}?GH0i4zC zX!fR>V!+am8C6SyRdH;L8Fo6*;2x;BI)ss4YqvdyzF-^*FWJU14$c3T^`sA&l|@8J{7J_ z=q!H^XmZXUECT%TJ>2x4+PTQvEuha@3WFYhM+}$$gekICt;$B{>pDzhhri5Dr(>x>!o{#$U zT5vw9(-4;C!rCmbtz+h*cg(BU-`3Z2CSr#wG+JwC1&Ti72}z##w^HL89i9My0}Rng zx54JuQLvqIyQVTHRnTH^WB>*@3ypkEhHaIai6KN~@ui*0k?vrs+=ZtT8 zlb7GJ@6Yj7IZF9ck}|6XTae|jgFW$d?!#TVvS9&;D}W%rq;AAXAqq0+dUGoi7R}7b z>Eh{`eVeC!=LbmQ&h!W^GwCZMV+PvUTRg88J6|GJKWb2)5MfIAxsC>dLsyxnxm5f@j zQ(ubFU}3iG1+WVzb1_c$2i?H@#H<{d=7;&vb@ti`>>W5_f%s@csdx*!N9TLdpc|qF z1bK1t5PAXO>ziAY#|`gZ?G^r#)n$rPBOeX@PWABEzvJWP#K_vRN$pb1>Yo+I&K@&p zvE7FS5dKN?dt~sdJj~=qRij2`QE;BWL)a^PPXQ@FShPyr)My}duGC;@2NyP=94riO z2%1;$GE*6q4~zCi--}Cjb(k+3{=k>gx1+Mu^J6p-xdew40In_cPojNdsi+F586aeq zzSicwYl#Vyib*pU>b16)rX+c?vs?AHyorgM1WcXq80Hn$>enyk8w&_Kv>r6PY72K_ zux+;dJxnR{hj#}`-TPIUpGRi zHNx1|cSb)98{E0+2?;^KRkqO3IKNK>u}xf2@#|DGCqO7YMk)|~e#x^~1HcYxN?rz} z#K6!{-=8sAa7tI$)RN+&A0F=cFRZX6ck_CTAZOxbrtgO#c6UeAfdw2*zJC@7S&Y3k zuKIl7$G;yiM{%b8XW-MX-}L;K=g#wCvCzz2xt!?#d(6mKU*3hAgy~zJ6XEA+5aqFB ze;DqtsYLV81E*my3CP5QfFlB9DlvF3LP%#Ql&tHwYwgCtSwEC7R}mayab=Q#p7U72 zcXx-d-G#&s1m*m|HCe>%-A|Gjsue2E;LU>W-U3?Dn$fVG}3 z|0H>=)0Jm6uiR(Hq>bf82=5IUra-x+TYxT+*<#|fYhzT8B@77W;WU7BpGQk3$&_-C` zkHdDOGj}4EKS>$VpIegkmYmH*8PV6b#=5TcvIh_Bm_C`c?Av@W#A2h)pB1_u94&99 zgS|i}=>4PDt%pKrZc+g~$u^4*Ezc?Tr9ozI=V6?@Z&=F^E{(WnAe@#ybCKBQ`KW%0 z^nSqh96YTi)1jnQ*xvFRt7@!OjzEiUFS{FZuuMULl^&LzkToc447Au*p}NE>o4nj5`OdI zaJt2PF-BZe_BL=7q8%O_464KNJzU{{f0)@m2zBYly-cuHHClVw62Q+(T$vy+13WET z`C+Wna~jk^zVL?+Xr5bG>0ax ze{{~gURLvV7FJ_<8gkUH>*l6}GHw2!PPVB`k3I^vkm*Ex?Bc--vRMxF? zN*GISJ~HkqPVs*fY4=f!5<+3*fq=EHJoJ(TEZB>e)m@Gk$2 z2sE4Az9lXAOJ%-Rm4Sc-ix?s-wzk9oVqJQF%#l14H0LUUwm2E`Sh$$)&uyuXRJA^b zSb(VrFBu$qeQvPRp{GcpX@l_$6lbY^bYQan`a=@7PmucWu&eaIt(TFJQ4AA;j-@0C z_ty{C?G>cE(bDw8!WSb6sp(_88+X2FR!J&t<_WtK1X3=|LVxY&u&-#WgFqC6MoBJc zjzk3Pz(JC^J&tj)KlkeB{VOl9m*)KC3O1Ql!vB@Pv<97;gS+1ra-w#P$J{0aBmN>P zYUk6Q{#A^uik;PmH+hjNb}uj?SyvAmXQY=4`X@%~EIh-$5rL{H9Kq?SthVq(+o-SFKh#215sS^-n8e3 zyIBI8;!Q5v6N&0NMUv~zvCsNM_3cz6RT7B%hR{WX|-y&Apo|iT*s`a>}O;^ z@ysbQ;vyYU|QtegCYkF7~ZZ31(a^s1&>FWc+B{Uny7RkfU{Z zCQQU<`#^%`9s6p%Yv6IwYN+SfG~Pq%!t(LFMkHf^^`3INZp)lrTW`#H$gjRk>elxU zHB-lDM|X6bMZER>vG?w|_B|5GL)psi^cCV{UrXTLVbH_Xsb8XQ;XO=bn1v} zuQKj~O?dFS?(@9DU)gKnK0`|PI9f6BtohR6}JIKvVq8mpElNSJyz0OGCSy8nahPaF^txmJznH=uh-+!1( z+*g`ZvhZai)0dh1uzyaAi-McXM$I)aYjo6n%+k&!WRAMU?rM5p4LZ3UEx7+EHGf9I zx^me{UWeJ3EO)WW7ylBLO1)$AzQ%sPz)fs#|4KW5c4&`-wwJab#$+qZv__O;GA#~6|sE#?k#ps>oR1g-uE%<1>OAf~9; z1FI9p6UoTFBZf0R+BAsdOR+*YJj!(h2A9J{>gR4T8oqb(mmV31t7?!UFHmj$*<3u@ zm~7=|qzK8?yVIl~{pIV9;J@9`8!X3@XFZ$r6mz-g#a(02h5`&KT@{bj@*C#%Yh-bC z?W*1vCLt^7X=*A}TP{ob&1;rOR+*Qy1Fm=$)EAkC{l@HZaqGtam{Qa zXW=#B4?Fw|DaR^k-sEv`Ow|o2R?z0d{b?EO>zWjUXN;;4NARcydBj;Ne(As(Tu+Ui z^T+tJuxY6yDHyQa^whqGyQ1~c-X=oy+9gd(dL>JL{y-kcf18`()~w8LtDF0>#z)EZ zogQpJ4oD7x1?I|pH{WMJb>hC-hTcA*(4a<`^B=r;mhl{~ ze-ViVS}l*l9BT2Dv@E<)#9jUf%Y9U+|Ob-kyJ~-&4lpZm7@3%$WG@)~n0{F=xlnRU1dc&8y zI1?&@5Rb8|1ao0Dotg~Q!s6nFBF&KAmb-iMaOzh~4i3-~P^_Vae*QEDET`Lq<_|3;td2R(x4wf_p?a7 z%xkY1zM@l;P&`^v&Po{QW@7q1%X(I7U1cFxdg#L~VyjxdEQji8>}&xkS*$^>ft+|*!^Hd3 zPkx)s-qOkwpijRbC2JTlm>pnR39OOWM9TSohS*xwzG;3CVb${CYZNyxQ2+mP_4u=7 zWGzBxKQ>jJ{CIE7`nurMb-UIOlan9uMeQqJSWZ24Ojfa=YxN>z%jpz+Xsh43ja0LL F@;|)yNq+zU literal 0 HcmV?d00001 diff --git a/doc/source/ray-core/api/index.rst b/doc/source/ray-core/api/index.rst index eb5cdd9d0ef5a..2845ebe892ef6 100644 --- a/doc/source/ray-core/api/index.rst +++ b/doc/source/ray-core/api/index.rst @@ -10,5 +10,5 @@ Ray Core API utility.rst exceptions.rst cli.rst - ../../ray-observability/api/state/cli.rst - ../../ray-observability/api/state/api.rst + ../../ray-observability/reference/cli.rst + ../../ray-observability/reference/api.rst diff --git a/doc/source/ray-observability/user-guides/debug-apps/debug-failures.rst b/doc/source/ray-observability/user-guides/debug-apps/debug-failures.rst new file mode 100644 index 0000000000000..206adb88853e6 --- /dev/null +++ b/doc/source/ray-observability/user-guides/debug-apps/debug-failures.rst @@ -0,0 +1,272 @@ +.. _observability-debug-failures: + +Debugging Failures +================== + +What Kind of Failures Exist in Ray? +----------------------------------- + +Ray consists of 2 major APIs. ``.remote()`` to create a task/actor and :func:`ray.get ` to get the result. +Debugging Ray means identifying and fixing failures from remote processes that run functions and classes (task and actor) created by the ``.remote`` API. + +Ray APIs are future APIs (indeed, it is :ref:`possible to convert Ray object references to standard Python future APIs `), +and the error handling model is the same. When any remote tasks or actors fail, the returned object ref will contain an exception. +When you call ``get`` API to the object ref, it raises an exception. + +.. code-block:: python + + import ray + @ray.remote + def f(): + raise ValueError + + # Raises a ValueError. + ray.get(f.remote()) + +In Ray, there are 3 types of failures. See exception APIs for more details. + +- **Application failures**: This means the remote task/actor fails by the user code. In this case, ``get`` API will raise the :func:`RayTaskError ` which includes the exception raised from the remote process. +- **Intentional system failures**: This means Ray is failed, but the failure is intended. For example, when you call cancellation APIs like ``ray.cancel`` (for task) or ``ray.kill`` (for actors), the system fails remote tasks and actors, but it is intentional. +- **Unintended system failures**: This means the remote tasks and actors failed due to unexpected system failures such as processes crashing (for example, by out-of-memory error) or nodes failing. + + 1. `Linux Out of Memory killer `_ or :ref:`Ray OOM killer ` kills processes with high memory usages to avoid out-of-memory. + 2. The machine shuts down (e.g., spot instance termination) or a :term:`raylet ` is crashed (e.g., by an unexpected failure). + 3. System is highly overloaded or stressed (either machine or system components like Raylet or :term:`GCS `), which makes the system unstable and fail. + +Debugging Application Failures +------------------------------ + +Ray distributes users' code to multiple processes across many machines. Application failures mean bugs in users' code. +Ray provides a debugging experience that's similar to debugging a single-process Python program. + +print +~~~~~ + +``print`` debugging is one of the most common ways to debug Python programs. +:ref:`Ray's task and actor logs are printed to the Ray driver ` by default, +which allows you to simply use the ``print`` function to debug the application failures. + +Debugger +~~~~~~~~ + +Many Python developers use a debugger to debug Python programs, and `Python pdb `_) is one of the popular choices. +Ray has native integration to ``pdb``. You can simply add ``breakpoint()`` to actors and tasks code to enable ``pdb``. View :ref:`Ray Debugger ` for more details. + +.. _troubleshooting-out-of-memory: + +Debugging Out of Memory +----------------------- + +Before reading this section, it is recommended to understand Ray's :ref:`Memory Management ` model. + +- To find if your cluster has out-of-memory problems, view :ref:`How to Detect Out-of-Memory Errors? `. +- To find what leaks the memory, view :ref:`Find per Task and Actor Memory Usage `. +- If your head node has high memory usage, view :ref:`Head Node Out-of-Memory Error `. +- If your memory usage is high due to high parallelism, view :ref:`Reduce Parallelism `. +- If you want to profile per task and actor memory usage, view :ref:`Profile Task and Actor Memory Usage `. + +What's the Out-of-Memory Error? +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Memory is a limited resource. When a process requests memory and the OS fails to allocate memory, the OS executes a routine to free up memory +by killing a process that has high memory usage (via SIGKILL) to avoid the OS becoming unstable. It is called `Linux Out of Memory killer `_. + +One of the common problems of the Linux out-of-memory killer is that processes are killed by a SIGKILL without Ray noticing it. +Since SIGKILL cannot be handled by processes, it makes Ray difficult to raise a proper error message +and take proper actions for fault tolerance. +To solve this problem, Ray built and enabled (from Ray 2.2) an application-level :ref:`memory monitor `, +which keeps monitoring the memory usage of the host and kills the Ray workers before the Linux out-of-memory killer kicks in. + +.. _troubleshooting-out-of-memory-how-to-detect: + +How to Detect Out-of-Memory Errors? +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +If tasks or actors are killed by the Linux out-of-memory killer, Ray worker processes are unable to catch and display an exact root cause +because SIGKILL cannot be handled by processes. If you call ``ray.get`` into the tasks and actors that were executed from the dead worker, +it will raise an exception with one of the following error messages (which indicates the worker is killed unexpectedly). + +.. code-block:: bash + + Worker exit type: UNEXPECTED_SY STEM_EXIT Worker exit detail: Worker unexpectedly exits with a connection error code 2. End of file. There are some potential root causes. (1) The process is killed by SIGKILL by OOM killer due to high memory usage. (2) ray stop --force is called. (3) The worker is crashed unexpectedly due to SIGSEGV or other unexpected errors. + +.. code-block:: bash + + Worker exit type: SYSTEM_ERROR Worker exit detail: The leased worker has unrecoverable failure. Worker is requested to be destroyed when it is returned. + +Also, you can use the `dmesg `_ CLI command to verify the processes are killed by the Linux out-of-memory killer. + +.. image:: ../images/dmsg.png + :align: center + +If the worker is killed by Ray's memory monitor, they are automatically retried (see the :ref:`link ` for the detail). +If tasks or actors cannot be retried, they raise an exception with +a much cleaner error message when you call ``ray.get`` to it. + +.. code-block:: bash + + ray.exceptions.OutOfMemoryError: Task was killed due to the node running low on memory. + + Task was killed due to the node running low on memory. + Memory on the node (IP: 10.0.62.231, ID: e5d953ef03e55e26f13973ea1b5a0fd0ecc729cd820bc89e4aa50451) where the task (task ID: 43534ce9375fa8e4cd0d0ec285d9974a6a95897401000000, name=allocate_memory, pid=11362, memory used=1.25GB) was running was 27.71GB / 28.80GB (0.962273), which exceeds the memory usage threshold of 0.95. Ray killed this worker (ID: 6f2ec5c8b0d5f5a66572859faf192d36743536c2e9702ea58084b037) because it was the most recently scheduled task; to see more information about memory usage on this node, use `ray logs raylet.out -ip 10.0.62.231`. To see the logs of the worker, use `ray logs worker-6f2ec5c8b0d5f5a66572859faf192d36743536c2e9702ea58084b037*out -ip 10.0.62.231.` + Top 10 memory users: + PID MEM(GB) COMMAND + 410728 8.47 510953 7.19 ray::allocate_memory + 610952 6.15 ray::allocate_memory + 711164 3.63 ray::allocate_memory + 811156 3.63 ray::allocate_memory + 911362 1.25 ray::allocate_memory + 107230 0.09 python test.py --num-tasks 2011327 0.08 /home/ray/anaconda3/bin/python /home/ray/anaconda3/lib/python3.9/site-packages/ray/dashboard/dashboa... + + Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. + +Ray memory monitor also periodically prints the aggregated out-of-memory killer summary to Ray drivers. + +.. code-block:: bash + + (raylet) [2023-04-09 07:23:59,445 E 395 395] (raylet) node_manager.cc:3049: 10 Workers (tasks / actors) killed due to memory pressure (OOM), 0 Workers crashed due to other reasons at node (ID: e5d953ef03e55e26f13973ea1b5a0fd0ecc729cd820bc89e4aa50451, IP: 10.0.62.231) over the last time period. To see more information about the Workers killed on this node, use `ray logs raylet.out -ip 10.0.62.231` + (raylet) + (raylet) Refer to the documentation on how to address the out of memory issue: https://docs.ray.io/en/latest/ray-core/scheduling/ray-oom-prevention.html. Consider provisioning more memory on this node or reducing task parallelism by requesting more CPUs per task. To adjust the kill threshold, set the environment variable `RAY_memory_usage_threshold` when starting Ray. To disable worker killing, set the environment variable `RAY_memory_monitor_refresh_ms` to zero. + +Ray Dashboard's :ref:`metrics page ` and :ref:`event page ` also provides the out-of-memory killer-specific events and metrics. + +.. image:: ../images/oom-metrics.png + :align: center + +.. image:: ../images/oom-events.png + :align: center + +.. _troubleshooting-out-of-memory-task-actor-mem-usage: + +Find per Task and Actor Memory Usage +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +If tasks or actors are failed by Out-of-memory errors, they are retried based on :ref:`retry policies `. +However, it is often more desirable to find the root causes of memory issues and fix them instead of relying on fault tolerance mechanisms. +This section explains how to debug out-of-memory errors in Ray. + +First, find the tasks and actors that use high memory usage. View the :ref:`per task and actor memory usage graph ` for more details. +The memory usage from the per component graph uses RSS - SHR. See the below for reasoning. + +Alternatively, you can also use the CLI command `htop `_. + +.. image:: ../images/htop.png + :align: center + +See the ``allocate_memory`` row. See two columns, RSS and SHR. + +SHR usage is typically the memory usage from the Ray object store. The Ray object store allocates 30% of host memory to the shared memory (``/dev/shm``, unless you specify ``--object-store-memory``). +If Ray workers access the object inside the object store using ``ray.get``, SHR usage increases. Since the Ray object store supports the :ref:`zero-copy ` +deserialization, several workers can access the same object without copying them to in-process memory. For example, if +8 workers access the same object inside the Ray object store, each process' ``SHR`` usage increases. However, they are not using 8 * SHR memory (there's only 1 copy in the shared memory). +Also note that Ray object store triggers :ref:`object spilling ` when the object usage goes beyond the limit, which means the memory usage from the shared memory won't exceed 30% +of the host memory. + +It means when there are out-of-memory issues from a host, it is due to RSS usage from each worker. We recommend to calculate per +process memory usage by RSS - SHR because SHR is for Ray object store as explained above. That said, the total memory usage is typically +``SHR (object store memory usage, 30% of memory) + sum(RSS - SHR from each ray proc) + sum(RSS - SHR from system components. e.g., raylet, GCS. Usually small)``. + +.. _troubleshooting-out-of-memory-head: + +Head Node Out-of-Memory Error +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +First, check the head node memory usage from the metrics page. Find the head node address from the cluster page. + +.. image:: ../images/head-node-addr.png + :align: center + +And then check the memory usage from the head node from the node memory usage view inside the Dashboard :ref:`metrics view `. + +.. image:: ../images/metrics-node-view.png + :align: center + +Ray head node has more memory-demanding system components such as GCS or the dashboard. +Also, driver runs from a head node by default. If the head node has the same memory capacity as worker nodes +and if you execute the same number of tasks and actors from a head node, it can easily have out-of-memory problems. +In this case, we recommend you not running any tasks and actors to the head node. This is achievable by specifying ``--num-cpus=0`` when starting a head node by ``ray start --head`` (if you use Kuberay, view ``). + +.. _troubleshooting-out-of-memory-reduce-parallelism: + +Reduce Parallelism +~~~~~~~~~~~~~~~~~~ + +High parallelism can trigger out-of-memory errors. For example, imagine +you have 8 training workers that perform the data preprocessing -> training. +If you load too much data into each worker, the total memory usage (``training worker mem usage * 8``) can exceed the +memory capacity. + +You can verify it by looking at the :ref:`per task and actor memory usage graph `. And the task metrics. + +First, see the memory usage of a ``allocate_memory`` task. It is total 18GB. +At the same time, you can verify 15 concurrent tasks running. + +.. image:: ../images/component-memory.png + :align: center + +.. image:: ../images/tasks-graph.png + :align: center + +It means each task uses about 18GB / 15 == 1.2 GB. To reduce the parallelism, + +- `Limit the max number of running tasks `_. +- increase the ``num_cpus`` options for :func:`ray.remote`. Modern hardware typically has 4GB of memory per CPU, so you can choose the CPU requirements accordingly. For this example, we specified 1 CPU per each ``allocate_memory`` task. If we double the CPU requirements, you can only run half of tasks (7) at the same time, meaning memory usage won't exceed 9GB. + +.. _troubleshooting-out-of-memory-profile: + +Profile Task and Actor Memory Usage +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +It is also possible tasks and actors use more memory than you expect. For example, actors or tasks can have a memory leak or have unnecessary copies. + +View :ref:`Memory Profiling Ray Tasks and Actors ` to learn how to memory profile individual actors and tasks. + + +Running out of file descriptors (``Too may open files``) +-------------------------------------------------------- + +In a Ray cluster, arbitrary two system components can communicate with each other and make 1 or more connections. +For example, some workers may need to communicate with GCS to schedule actors (worker <-> GCS connection). +Your driver can invoke actor methods (worker <-> worker connection). + +Ray can support 1000s of raylets and 10000s of worker processes. When a Ray cluster gets larger, +each component can have an increasing number of network connections which requires file descriptors. + +Linux typically limits the default file descriptors per process to 1024. When there are +more than 1024 connections to the component, it can raise error messages below. + +.. code-block:: bash + + Too may open files + +It is especially common for the head node GCS process because it is a centralized +component that many other components in Ray communicate with. When you see this error message, +we recommend you adjust the max file descriptors limit per process via the ``ulimit`` command. + +We recommend you apply ``ulimit -n 65536`` to your host configuration. However, you can also selectively apply it for +Ray components (view below example). Normally, each worker has 2~3 connections to GCS. Each raylet has 1~2 connections to GCS. +65536 file descriptors can handle 10000~15000 of workers and 1000~2000 of nodes. +If you have more workers, you should consider using a higher number than 65536. + +.. code-block:: bash + + # Start head node components with higher ulimit. + ulimit -n 65536 ray start --head + + # Start worker node components with higher ulimit. + ulimit -n 65536 ray start --address + + # Start a Ray driver with higher ulimit. + ulimit -n 65536 + +If that fails, double-check that the hard limit is sufficiently large by running ``ulimit -Hn``. +If it is too small, you can increase the hard limit as follows (these instructions work on EC2). + +* Increase the hard ulimit for open file descriptors system-wide by running + the following. + + .. code-block:: bash + + sudo bash -c "echo $USER hard nofile 65536 >> /etc/security/limits.conf" + +* Logout and log back in. diff --git a/doc/source/ray-observability/user-guides/debug-apps/debug-hangs.rst b/doc/source/ray-observability/user-guides/debug-apps/debug-hangs.rst new file mode 100644 index 0000000000000..74fc4d34f5733 --- /dev/null +++ b/doc/source/ray-observability/user-guides/debug-apps/debug-hangs.rst @@ -0,0 +1,16 @@ +.. _observability-debug-hangs: + +Debugging Hangs +=============== + +Observing Ray Work +------------------ + +You can run ``ray stack`` to dump the stack traces of all Ray workers on +the current node. This requires ``py-spy`` to be installed. See the `Troubleshooting page `_ for more details. + +This document discusses some common problems that people run into when using Ray +as well as some known problems. If you encounter other problems, please +`let us know`_. + +.. _`let us know`: https://github.com/ray-project/ray/issues diff --git a/doc/source/ray-observability/user-guides/debug-apps/debug-memory.rst b/doc/source/ray-observability/user-guides/debug-apps/debug-memory.rst new file mode 100644 index 0000000000000..10d10620cb6bf --- /dev/null +++ b/doc/source/ray-observability/user-guides/debug-apps/debug-memory.rst @@ -0,0 +1,56 @@ +.. _ray-core-profiling: + +.. _ray-core-mem-profiling: + +Debugging Memory Issues +----------------------- + +To memory profile Ray tasks or actors, use `memray `_. +Note that you can also use other memory profiling tools if it supports a similar API. + +First, install ``memray``. + +.. code-block:: bash + + pip install memray + +``memray`` supports a Python context manager to enable memory profiling. You can write the ``memray`` profiling file wherever you want. +But in this example, we will write them to `/tmp/ray/session_latest/logs` because Ray dashboard allows you to download files inside the log folder. +This will allow you to download profiling files from other nodes. + +.. tab-set:: + + .. tab-item:: Actors + + .. literalinclude:: ../doc_code/memray_profiling.py + :language: python + :start-after: __memray_profiling_start__ + :end-before: __memray_profiling_end__ + + .. tab-item:: Tasks + + Note that tasks have a shorter lifetime, so there could be lots of memory profiling files. + + .. literalinclude:: ../doc_code/memray_profiling.py + :language: python + :start-after: __memray_profiling_task_start__ + :end-before: __memray_profiling_task_end__ + +Once the task or actor runs, go to the :ref:`Logs View ` of the dashboard. Find and click the log file name. + +.. image:: ../images/memory-profiling-files.png + :align: center + +Click the download button. + +.. image:: ../images/download-memory-profiling-files.png + :align: center + +Now, you have the memory profiling file. Running + +.. code-block:: bash + + memray flamegraph + +And you can see the result of the memory profiling! + diff --git a/doc/source/ray-observability/user-guides/debug-apps/general-troubleshoot.rst b/doc/source/ray-observability/user-guides/debug-apps/general-troubleshoot.rst new file mode 100644 index 0000000000000..96ffbab7cc021 --- /dev/null +++ b/doc/source/ray-observability/user-guides/debug-apps/general-troubleshoot.rst @@ -0,0 +1,217 @@ +.. _observability-general-troubleshoot: + +General Troubleshooting +======================= + +Ray sometimes has some aspects of its behavior that might catch +users off guard. There may be sound arguments for these design choices. + +In particular, users think of Ray as running on their local machine, and +while this is mostly true, this doesn't work. + +Environment variables are not passed from the driver to workers +--------------------------------------------------------------- + +**Issue**: If you set an environment variable at the command line, it is not passed to all the workers running in the cluster +if the cluster was started previously. + +**Example**: If you have a file ``baz.py`` in the directory you are running Ray in, and you run the following command: + +.. literalinclude:: ../../ray-core/doc_code/gotchas.py + :language: python + :start-after: __env_var_start__ + :end-before: __env_var_end__ + +**Expected behavior**: Most people would expect (as if it was a single process on a single machine) that the environment variables would be the same in all workers. It won’t be. + +**Fix**: Use runtime environments to pass environment variables explicity. +If you call ``ray.init(runtime_env=...)``, +then the workers will have the environment variable set. + + +.. literalinclude:: ../../ray-core/doc_code/gotchas.py + :language: python + :start-after: __env_var_fix_start__ + :end-before: __env_var_fix_end__ + + +Filenames work sometimes and not at other times +----------------------------------------------- + +**Issue**: If you reference a file by name in a task or actor, +it will sometimes work and sometimes fail. This is +because if the task or actor runs on the head node +of the cluster, it will work, but if the task or actor +runs on another machine it won't. + +**Example**: Let's say we do the following command: + +.. code-block:: bash + + % touch /tmp/foo.txt + +And I have this code: + +.. code-block:: python + + import os + + ray.init() + @ray.remote + def check_file(): + foo_exists = os.path.exists("/tmp/foo.txt") + print(f"Foo exists? {foo_exists}") + + futures = [] + for _ in range(1000): + futures.append(check_file.remote()) + + ray.get(futures) + + +then you will get a mix of True and False. If +``check_file()`` runs on the head node, or we're running +locally it works. But if it runs on a worker node, it returns ``False``. + +**Expected behavior**: Most people would expect this to either fail or succeed consistently. +It's the same code after all. + +**Fix** + +- Use only shared paths for such applications -- e.g. if you are using a network file system you can use that, or the files can be on s3. +- Do not rely on file path consistency. + + + +Placement groups are not composable +----------------------------------- + +**Issue**: If you have a task that is called from something that runs in a placement +group, the resources are never allocated and it hangs. + +**Example**: You are using Ray Tune which creates placement groups, and you want to +apply it to an objective function, but that objective function makes use +of Ray Tasks itself, e.g. + +.. code-block:: python + + from ray import air, tune + + def create_task_that_uses_resources(): + @ray.remote(num_cpus=10) + def sample_task(): + print("Hello") + return + + return ray.get([sample_task.remote() for i in range(10)]) + + def objective(config): + create_task_that_uses_resources() + + tuner = tune.Tuner(objective, param_space={"a": 1}) + tuner.fit() + +This will error with message: +ValueError: Cannot schedule create_task_that_uses_resources..sample_task with the placement group +because the resource request {'CPU': 10} cannot fit into any bundles for the placement group, [{'CPU': 1.0}]. + +**Expected behavior**: The above executes. + +**Fix**: In the ``@ray.remote`` declaration of tasks +called by ``create_task_that_uses_resources()`` , include a +``scheduling_strategy=PlacementGroupSchedulingStrategy(placement_group=None)``. + +.. code-block:: diff + + def create_task_that_uses_resources(): + + @ray.remote(num_cpus=10, scheduling_strategy=PlacementGroupSchedulingStrategy(placement_group=None)) + - @ray.remote(num_cpus=10) + +Outdated Function Definitions +----------------------------- + +Due to subtleties of Python, if you redefine a remote function, you may not +always get the expected behavior. In this case, it may be that Ray is not +running the newest version of the function. + +Suppose you define a remote function ``f`` and then redefine it. Ray should use +the newest version. + +.. code-block:: python + + @ray.remote + def f(): + return 1 + + @ray.remote + def f(): + return 2 + + ray.get(f.remote()) # This should be 2. + +However, the following are cases where modifying the remote function will +not update Ray to the new version (at least without stopping and restarting +Ray). + +- **The function is imported from an external file:** In this case, + ``f`` is defined in some external file ``file.py``. If you ``import file``, + change the definition of ``f`` in ``file.py``, then re-``import file``, + the function ``f`` will not be updated. + + This is because the second import gets ignored as a no-op, so ``f`` is + still defined by the first import. + + A solution to this problem is to use ``reload(file)`` instead of a second + ``import file``. Reloading causes the new definition of ``f`` to be + re-executed, and exports it to the other machines. Note that in Python 3, you + need to do ``from importlib import reload``. + +- **The function relies on a helper function from an external file:** + In this case, ``f`` can be defined within your Ray application, but relies + on a helper function ``h`` defined in some external file ``file.py``. If the + definition of ``h`` gets changed in ``file.py``, redefining ``f`` will not + update Ray to use the new version of ``h``. + + This is because when ``f`` first gets defined, its definition is shipped to + all of the workers, and is unpickled. During unpickling, ``file.py`` gets + imported in the workers. Then when ``f`` gets redefined, its definition is + again shipped and unpickled in all of the workers. But since ``file.py`` + has been imported in the workers already, it is treated as a second import + and is ignored as a no-op. + + Unfortunately, reloading on the driver does not update ``h``, as the reload + needs to happen on the worker. + + A solution to this problem is to redefine ``f`` to reload ``file.py`` before + it calls ``h``. For example, if inside ``file.py`` you have + + .. code-block:: python + + def h(): + return 1 + + And you define remote function ``f`` as + + .. code-block:: python + + @ray.remote + def f(): + return file.h() + + You can redefine ``f`` as follows. + + .. code-block:: python + + @ray.remote + def f(): + reload(file) + return file.h() + + This forces the reload to happen on the workers as needed. Note that in + Python 3, you need to do ``from importlib import reload``. + +This document discusses some common problems that people run into when using Ray +as well as some known problems. If you encounter other problems, please +`let us know`_. + +.. _`let us know`: https://github.com/ray-project/ray/issues diff --git a/doc/source/ray-observability/user-guides/debug-apps/index.md b/doc/source/ray-observability/user-guides/debug-apps/index.md new file mode 100644 index 0000000000000..53e9ee3e962e0 --- /dev/null +++ b/doc/source/ray-observability/user-guides/debug-apps/index.md @@ -0,0 +1,11 @@ +(observability-user-guides)= + +# Troubleshooting Applications + +These guides help you perform common debugging or optimization tasks for your distributed application on Ray: +* {ref}`observability-general-troubleshoot` +* {ref}`ray-core-mem-profiling` +* {ref}`observability-debug-hangs` +* {ref}`observability-debug-failures` +* {ref}`observability-optimize-performance` +* {ref}`ray-debugger` \ No newline at end of file diff --git a/doc/source/ray-observability/user-guides/debug-apps/optimize-performance.rst b/doc/source/ray-observability/user-guides/debug-apps/optimize-performance.rst new file mode 100644 index 0000000000000..cad5e6f312ac1 --- /dev/null +++ b/doc/source/ray-observability/user-guides/debug-apps/optimize-performance.rst @@ -0,0 +1,325 @@ +.. _observability-optimize-performance: + +Optimizing Performance +====================== + +No Speedup +---------- + +You just ran an application using Ray, but it wasn't as fast as you expected it +to be. Or worse, perhaps it was slower than the serial version of the +application! The most common reasons are the following. + +- **Number of cores:** How many cores is Ray using? When you start Ray, it will + determine the number of CPUs on each machine with ``psutil.cpu_count()``. Ray + usually will not schedule more tasks in parallel than the number of CPUs. So + if the number of CPUs is 4, the most you should expect is a 4x speedup. + +- **Physical versus logical CPUs:** Do the machines you're running on have fewer + **physical** cores than **logical** cores? You can check the number of logical + cores with ``psutil.cpu_count()`` and the number of physical cores with + ``psutil.cpu_count(logical=False)``. This is common on a lot of machines and + especially on EC2. For many workloads (especially numerical workloads), you + often cannot expect a greater speedup than the number of physical CPUs. + +- **Small tasks:** Are your tasks very small? Ray introduces some overhead for + each task (the amount of overhead depends on the arguments that are passed + in). You will be unlikely to see speedups if your tasks take less than ten + milliseconds. For many workloads, you can easily increase the sizes of your + tasks by batching them together. + +- **Variable durations:** Do your tasks have variable duration? If you run 10 + tasks with variable duration in parallel, you shouldn't expect an N-fold + speedup (because you'll end up waiting for the slowest task). In this case, + consider using ``ray.wait`` to begin processing tasks that finish first. + +- **Multi-threaded libraries:** Are all of your tasks attempting to use all of + the cores on the machine? If so, they are likely to experience contention and + prevent your application from achieving a speedup. + This is common with some versions of ``numpy``. To avoid contention, set an + environment variable like ``MKL_NUM_THREADS`` (or the equivalent depending on + your installation) to ``1``. + + For many - but not all - libraries, you can diagnose this by opening ``top`` + while your application is running. If one process is using most of the CPUs, + and the others are using a small amount, this may be the problem. The most + common exception is PyTorch, which will appear to be using all the cores + despite needing ``torch.set_num_threads(1)`` to be called to avoid contention. + +If you are still experiencing a slowdown, but none of the above problems apply, +we'd really like to know! Please create a `GitHub issue`_ and consider +submitting a minimal code example that demonstrates the problem. + +.. _`Github issue`: https://github.com/ray-project/ray/issues + +This document discusses some common problems that people run into when using Ray +as well as some known problems. If you encounter other problems, please +`let us know`_. + +.. _`let us know`: https://github.com/ray-project/ray/issues +.. _ray-core-timeline: + +Visualizing Tasks in the Ray Timeline +------------------------------------- + +The most important tool is the timeline visualization tool. To visualize tasks +in the Ray timeline, you can dump the timeline as a JSON file by running ``ray +timeline`` from the command line or ``ray.timeline`` from the Python API. + +To use the timeline, Ray profiling must be enabled by setting the +``RAY_PROFILING=1`` environment variable prior to starting Ray on every machine, and ``RAY_task_events_report_interval_ms`` must be larger than 0 (default 1000). + +.. code-block:: python + + ray.timeline(filename="/tmp/timeline.json") + +Then open `chrome://tracing`_ in the Chrome web browser, and load +``timeline.json``. + +.. _`chrome://tracing`: chrome://tracing + +.. _dashboard-profiling: + +Python CPU Profiling in the Dashboard +------------------------------------- + +The :ref:`ray-dashboard` lets you profile Ray worker processes by clicking on the "Stack Trace" or "CPU Flame Graph" +actions for active workers, actors, and jobs. + +.. image:: /images/profile.png + :align: center + :width: 80% + +Clicking "Stack Trace" will return the current stack trace sample using ``py-spy``. By default, only the Python stack +trace is shown. To show native code frames, set the URL parameter ``native=1`` (only supported on Linux). + +.. image:: /images/stack.png + :align: center + :width: 60% + +Clicking "CPU Flame Graph" will take a number of stack trace samples and combine them into a flame graph visualization. +This flame graph can be useful for understanding the CPU activity of the particular process. To adjust the duration +of the flame graph, you can change the ``duration`` parameter in the URL. Similarly, you can change the ``native`` +parameter to enable native profiling. + +.. image:: /images/flamegraph.png + :align: center + :width: 80% + +The profiling feature requires ``py-spy`` to be installed. If it is not installed, or if the ``py-spy`` binary does +not have root permissions, the dashboard will prompt with instructions on how to setup ``py-spy`` correctly: + +.. code-block:: + + This command requires `py-spy` to be installed with root permissions. You + can install `py-spy` and give it root permissions as follows: + $ pip install py-spy + $ sudo chown root:root `which py-spy` + $ sudo chmod u+s `which py-spy` + + Alternatively, you can start Ray with passwordless sudo / root permissions. + +Profiling Using Python's CProfile +--------------------------------- + +You can use Python's native cProfile `profiling module`_ to profile the performance of your Ray application. Rather than tracking +line-by-line of your application code, cProfile can give the total runtime +of each loop function, as well as list the number of calls made and +execution time of all function calls made within the profiled code. + +.. _`profiling module`: https://docs.python.org/3/library/profile.html#module-cProfile + +Unlike ``line_profiler`` above, this detailed list of profiled function calls +**includes** internal function calls and function calls made within Ray. + +However, similar to ``line_profiler``, cProfile can be enabled with minimal +changes to your application code (given that each section of the code you want +to profile is defined as its own function). To use cProfile, add an import +statement, then replace calls to the loop functions as follows: + +.. code-block:: python + + import cProfile # Added import statement + + def ex1(): + list1 = [] + for i in range(5): + list1.append(ray.get(func.remote())) + + def main(): + ray.init() + cProfile.run('ex1()') # Modified call to ex1 + cProfile.run('ex2()') + cProfile.run('ex3()') + + if __name__ == "__main__": + main() + +Now, when you execute your Python script, a cProfile list of profiled function +calls are printed on the terminal for each call made to ``cProfile.run()``. +At the very top of cProfile's output gives the total execution time for +``'ex1()'``: + +.. code-block:: bash + + 601 function calls (595 primitive calls) in 2.509 seconds + +Following is a snippet of profiled function calls for ``'ex1()'``. Most of +these calls are quick and take around 0.000 seconds, so the functions of +interest are the ones with non-zero execution times: + +.. code-block:: bash + + ncalls tottime percall cumtime percall filename:lineno(function) + ... + 1 0.000 0.000 2.509 2.509 your_script_here.py:31(ex1) + 5 0.000 0.000 0.001 0.000 remote_function.py:103(remote) + 5 0.000 0.000 0.001 0.000 remote_function.py:107(_remote) + ... + 10 0.000 0.000 0.000 0.000 worker.py:2459(__init__) + 5 0.000 0.000 2.508 0.502 worker.py:2535(get) + 5 0.000 0.000 0.000 0.000 worker.py:2695(get_global_worker) + 10 0.000 0.000 2.507 0.251 worker.py:374(retrieve_and_deserialize) + 5 0.000 0.000 2.508 0.502 worker.py:424(get_object) + 5 0.000 0.000 0.000 0.000 worker.py:514(submit_task) + ... + +The 5 separate calls to Ray's ``get``, taking the full 0.502 seconds each call, +can be noticed at ``worker.py:2535(get)``. Meanwhile, the act of calling the +remote function itself at ``remote_function.py:103(remote)`` only takes 0.001 +seconds over 5 calls, and thus is not the source of the slow performance of +``ex1()``. + + +Profiling Ray Actors with cProfile +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Considering that the detailed output of cProfile can be quite different depending +on what Ray functionalities we use, let us see what cProfile's output might look +like if our example involved Actors (for an introduction to Ray actors, see our +`Actor documentation here`_). + +.. _`Actor documentation here`: http://docs.ray.io/en/master/actors.html + +Now, instead of looping over five calls to a remote function like in ``ex1``, +let's create a new example and loop over five calls to a remote function +**inside an actor**. Our actor's remote function again just sleeps for 0.5 +seconds: + +.. code-block:: python + + # Our actor + @ray.remote + class Sleeper(object): + def __init__(self): + self.sleepValue = 0.5 + + # Equivalent to func(), but defined within an actor + def actor_func(self): + time.sleep(self.sleepValue) + +Recalling the suboptimality of ``ex1``, let's first see what happens if we +attempt to perform all five ``actor_func()`` calls within a single actor: + +.. code-block:: python + + def ex4(): + # This is suboptimal in Ray, and should only be used for the sake of this example + actor_example = Sleeper.remote() + + five_results = [] + for i in range(5): + five_results.append(actor_example.actor_func.remote()) + + # Wait until the end to call ray.get() + ray.get(five_results) + +We enable cProfile on this example as follows: + +.. code-block:: python + + def main(): + ray.init() + cProfile.run('ex4()') + + if __name__ == "__main__": + main() + +Running our new Actor example, cProfile's abbreviated output is as follows: + +.. code-block:: bash + + 12519 function calls (11956 primitive calls) in 2.525 seconds + + ncalls tottime percall cumtime percall filename:lineno(function) + ... + 1 0.000 0.000 0.015 0.015 actor.py:546(remote) + 1 0.000 0.000 0.015 0.015 actor.py:560(_remote) + 1 0.000 0.000 0.000 0.000 actor.py:697(__init__) + ... + 1 0.000 0.000 2.525 2.525 your_script_here.py:63(ex4) + ... + 9 0.000 0.000 0.000 0.000 worker.py:2459(__init__) + 1 0.000 0.000 2.509 2.509 worker.py:2535(get) + 9 0.000 0.000 0.000 0.000 worker.py:2695(get_global_worker) + 4 0.000 0.000 2.508 0.627 worker.py:374(retrieve_and_deserialize) + 1 0.000 0.000 2.509 2.509 worker.py:424(get_object) + 8 0.000 0.000 0.001 0.000 worker.py:514(submit_task) + ... + +It turns out that the entire example still took 2.5 seconds to execute, or the +time for five calls to ``actor_func()`` to run in serial. If you recall ``ex1``, +this behavior was because we did not wait until after submitting all five +remote function tasks to call ``ray.get()``, but we can verify on cProfile's +output line ``worker.py:2535(get)`` that ``ray.get()`` was only called once at +the end, for 2.509 seconds. What happened? + +It turns out Ray cannot parallelize this example, because we have only +initialized a single ``Sleeper`` actor. Because each actor is a single, +stateful worker, our entire code is submitted and ran on a single worker the +whole time. + +To better parallelize the actors in ``ex4``, we can take advantage +that each call to ``actor_func()`` is independent, and instead +create five ``Sleeper`` actors. That way, we are creating five workers +that can run in parallel, instead of creating a single worker that +can only handle one call to ``actor_func()`` at a time. + +.. code-block:: python + + def ex4(): + # Modified to create five separate Sleepers + five_actors = [Sleeper.remote() for i in range(5)] + + # Each call to actor_func now goes to a different Sleeper + five_results = [] + for actor_example in five_actors: + five_results.append(actor_example.actor_func.remote()) + + ray.get(five_results) + +Our example in total now takes only 1.5 seconds to run: + +.. code-block:: bash + + 1378 function calls (1363 primitive calls) in 1.567 seconds + + ncalls tottime percall cumtime percall filename:lineno(function) + ... + 5 0.000 0.000 0.002 0.000 actor.py:546(remote) + 5 0.000 0.000 0.002 0.000 actor.py:560(_remote) + 5 0.000 0.000 0.000 0.000 actor.py:697(__init__) + ... + 1 0.000 0.000 1.566 1.566 your_script_here.py:71(ex4) + ... + 21 0.000 0.000 0.000 0.000 worker.py:2459(__init__) + 1 0.000 0.000 1.564 1.564 worker.py:2535(get) + 25 0.000 0.000 0.000 0.000 worker.py:2695(get_global_worker) + 3 0.000 0.000 1.564 0.521 worker.py:374(retrieve_and_deserialize) + 1 0.000 0.000 1.564 1.564 worker.py:424(get_object) + 20 0.001 0.000 0.001 0.000 worker.py:514(submit_task) + ... + +Profiling for Developers +------------------------ +If you are developing Ray Core or debugging some system level failures, profiling the Ray Core could help. In this case, see :ref:`Profiling (Internal) `. diff --git a/doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst b/doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst new file mode 100644 index 0000000000000..afc98df070a61 --- /dev/null +++ b/doc/source/ray-observability/user-guides/debug-apps/ray-debugging.rst @@ -0,0 +1,325 @@ +.. _ray-debugger: + +Using the Ray Debugger +====================== + +Ray has a built in debugger that allows you to debug your distributed applications. It allows +to set breakpoints in your Ray tasks and actors and when hitting the breakpoint you can +drop into a PDB session that you can then use to: + +- Inspect variables in that context +- Step within that task or actor +- Move up or down the stack + +.. warning:: + + The Ray Debugger is an experimental feature and is currently unstable. Interfaces are subject to change. + +Getting Started +--------------- + +.. note:: + + On Python 3.6, the ``breakpoint()`` function is not supported and you need to use + ``ray.util.pdb.set_trace()`` instead. + +Take the following example: + +.. code-block:: python + + import ray + ray.init() + + @ray.remote + def f(x): + breakpoint() + return x * x + + futures = [f.remote(i) for i in range(2)] + print(ray.get(futures)) + +Put the program into a file named ``debugging.py`` and execute it using: + +.. code-block:: bash + + python debugging.py + + +Each of the 2 executed tasks will drop into a breakpoint when the line +``breakpoint()`` is executed. You can attach to the debugger by running +the following command on the head node of the cluster: + +.. code-block:: bash + + ray debug + +The ``ray debug`` command will print an output like this: + +.. code-block:: text + + 2021-07-13 16:30:40,112 INFO scripts.py:216 -- Connecting to Ray instance at 192.168.2.61:6379. + 2021-07-13 16:30:40,112 INFO worker.py:740 -- Connecting to existing Ray cluster at address: 192.168.2.61:6379 + Active breakpoints: + index | timestamp | Ray task | filename:lineno + 0 | 2021-07-13 23:30:37 | ray::f() | debugging.py:6 + 1 | 2021-07-13 23:30:37 | ray::f() | debugging.py:6 + Enter breakpoint index or press enter to refresh: + + +You can now enter ``0`` and hit Enter to jump to the first breakpoint. You will be dropped into PDB +at the break point and can use the ``help`` to see the available actions. Run ``bt`` to see a backtrace +of the execution: + +.. code-block:: text + + (Pdb) bt + /home/ubuntu/ray/python/ray/workers/default_worker.py(170)() + -> ray.worker.global_worker.main_loop() + /home/ubuntu/ray/python/ray/worker.py(385)main_loop() + -> self.core_worker.run_task_loop() + > /home/ubuntu/tmp/debugging.py(7)f() + -> return x * x + +You can inspect the value of ``x`` with ``print(x)``. You can see the current source code with ``ll`` +and change stack frames with ``up`` and ``down``. For now let us continue the execution with ``c``. + +After the execution is continued, hit ``Control + D`` to get back to the list of break points. Select +the other break point and hit ``c`` again to continue the execution. + +The Ray program ``debugging.py`` now finished and should have printed ``[0, 1]``. Congratulations, you +have finished your first Ray debugging session! + +Running on a Cluster +-------------------- + +The Ray debugger supports setting breakpoints inside of tasks and actors that are running across your +Ray cluster. In order to attach to these from the head node of the cluster using ``ray debug``, you'll +need to make sure to pass in the ``--ray-debugger-external`` flag to ``ray start`` when starting the +cluster (likely in your ``cluster.yaml`` file or k8s Ray cluster spec). + +Note that this flag will cause the workers to listen for PDB commands on an external-facing IP address, +so this should *only* be used if your cluster is behind a firewall. + +Debugger Commands +----------------- + +The Ray debugger supports the +`same commands as PDB +`_. + +Stepping between Ray tasks +-------------------------- + +You can use the debugger to step between Ray tasks. Let's take the +following recursive function as an example: + +.. code-block:: python + + import ray + + ray.init() + + @ray.remote + def fact(n): + if n == 1: + return n + else: + n_ref = fact.remote(n - 1) + return n * ray.get(n_ref) + + @ray.remote + def compute(): + breakpoint() + result_ref = fact.remote(5) + result = ray.get(result_ref) + + ray.get(compute.remote()) + + +After running the program by executing the Python file and calling +``ray debug``, you can select the breakpoint by pressing ``0`` and +enter. This will result in the following output: + +.. code-block:: python + + Enter breakpoint index or press enter to refresh: 0 + > /home/ubuntu/tmp/stepping.py(16)() + -> result_ref = fact.remote(5) + (Pdb) + +You can jump into the call with the ``remote`` command in Ray's debugger. +Inside the function, print the value of `n` with ``p(n)``, resulting in +the following output: + +.. code-block:: python + + -> result_ref = fact.remote(5) + (Pdb) remote + *** Connection closed by remote host *** + Continuing pdb session in different process... + --Call-- + > /home/ubuntu/tmp/stepping.py(5)fact() + -> @ray.remote + (Pdb) ll + 5 -> @ray.remote + 6 def fact(n): + 7 if n == 1: + 8 return n + 9 else: + 10 n_ref = fact.remote(n - 1) + 11 return n * ray.get(n_ref) + (Pdb) p(n) + 5 + (Pdb) + +Now step into the next remote call again with +``remote`` and print `n`. You an now either continue recursing into +the function by calling ``remote`` a few more times, or you can jump +to the location where ``ray.get`` is called on the result by using the +``get`` debugger comand. Use ``get`` again to jump back to the original +call site and use ``p(result)`` to print the result: + +.. code-block:: python + + Enter breakpoint index or press enter to refresh: 0 + > /home/ubuntu/tmp/stepping.py(14)() + -> result_ref = fact.remote(5) + (Pdb) remote + *** Connection closed by remote host *** + Continuing pdb session in different process... + --Call-- + > /home/ubuntu/tmp/stepping.py(5)fact() + -> @ray.remote + (Pdb) p(n) + 5 + (Pdb) remote + *** Connection closed by remote host *** + Continuing pdb session in different process... + --Call-- + > /home/ubuntu/tmp/stepping.py(5)fact() + -> @ray.remote + (Pdb) p(n) + 4 + (Pdb) get + *** Connection closed by remote host *** + Continuing pdb session in different process... + --Return-- + > /home/ubuntu/tmp/stepping.py(5)fact()->120 + -> @ray.remote + (Pdb) get + *** Connection closed by remote host *** + Continuing pdb session in different process... + --Return-- + > /home/ubuntu/tmp/stepping.py(14)()->None + -> result_ref = fact.remote(5) + (Pdb) p(result) + 120 + (Pdb) + + +Post Mortem Debugging +--------------------- + +Often we do not know in advance where an error happens, so we cannot set a breakpoint. In these cases, +we can automatically drop into the debugger when an error occurs or an exception is thrown. This is called *post-mortem debugging*. + +We will show how this works using a Ray serve application. Copy the following code into a file called +``serve_debugging.py``: + +.. code-block:: python + + import time + + from sklearn.datasets import load_iris + from sklearn.ensemble import GradientBoostingClassifier + + import ray + from ray import serve + + serve.start() + + # Train model + iris_dataset = load_iris() + model = GradientBoostingClassifier() + model.fit(iris_dataset["data"], iris_dataset["target"]) + + # Define Ray Serve model, + @serve.deployment(route_prefix="/iris") + class BoostingModel: + def __init__(self): + self.model = model + self.label_list = iris_dataset["target_names"].tolist() + + await def __call__(self, starlette_request): + payload = await starlette_request.json()["vector"] + print(f"Worker: received request with data: {payload}") + + prediction = self.model.predict([payload])[0] + human_name = self.label_list[prediction] + return {"result": human_name} + + # Deploy model + serve.start() + BoostingModel.deploy() + + time.sleep(3600.0) + +Let's start the program with the post-mortem debugging activated (``RAY_PDB=1``): + +.. code-block:: bash + + RAY_PDB=1 python serve_debugging.py + +The flag ``RAY_PDB=1`` will have the effect that if an exception happens, Ray will +drop into the debugger instead of propagating it further. Let's see how this works! +First query the model with an invalid request using + +.. code-block:: bash + + python -c 'import requests; response = requests.get("http://localhost:8000/iris", json={"vector": [1.2, 1.0, 1.1, "a"]})' + +When the ``serve_debugging.py`` driver hits the breakpoint, it will tell you to run +``ray debug``. After we do that, we see an output like the following: + +.. code-block:: text + + Active breakpoints: + index | timestamp | Ray task | filename:lineno + 0 | 2021-07-13 23:49:14 | ray::RayServeWrappedReplica.handle_request() | /home/ubuntu/ray/python/ray/serve/backend_worker.py:249 + Traceback (most recent call last): + + File "/home/ubuntu/ray/python/ray/serve/backend_worker.py", line 242, in invoke_single + result = await method_to_call(*args, **kwargs) + + File "serve_debugging.py", line 24, in __call__ + prediction = self.model.predict([payload])[0] + + File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/ensemble/_gb.py", line 1188, in predict + raw_predictions = self.decision_function(X) + + File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/ensemble/_gb.py", line 1143, in decision_function + X = check_array(X, dtype=DTYPE, order="C", accept_sparse='csr') + + File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/utils/validation.py", line 63, in inner_f + return f(*args, **kwargs) + + File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/sklearn/utils/validation.py", line 673, in check_array + array = np.asarray(array, order=order, dtype=dtype) + + File "/home/ubuntu/anaconda3/lib/python3.7/site-packages/numpy/core/_asarray.py", line 83, in asarray + return array(a, dtype, copy=False, order=order) + + ValueError: could not convert string to float: 'a' + + Enter breakpoint index or press enter to refresh: + +We now press ``0`` and then Enter to enter the debugger. With ``ll`` we can see the context and with +``print(a)`` we an print the array that causes the problem. As we see, it contains a string (``'a'``) +instead of a number as the last element. + +In a similar manner as above, you can also debug Ray actors. Happy debugging! + +Debugging APIs +-------------- + +See :ref:`package-ref-debugging-apis`. From 68019c166dda6829699c57a3556367e2304e1966 Mon Sep 17 00:00:00 2001 From: angelinalg <122562471+angelinalg@users.noreply.github.com> Date: Mon, 22 May 2023 14:24:33 -0700 Subject: [PATCH 29/29] fixing broken test and broken note Signed-off-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/cluster/configure-manage-dashboard.rst | 2 ++ doc/source/ray-observability/key-concepts.rst | 3 +-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/doc/source/cluster/configure-manage-dashboard.rst b/doc/source/cluster/configure-manage-dashboard.rst index 2502e850d29bc..100134271634e 100644 --- a/doc/source/cluster/configure-manage-dashboard.rst +++ b/doc/source/cluster/configure-manage-dashboard.rst @@ -146,10 +146,12 @@ To disable the dashboard, use the following arguments `--include-dashboard`. .. testcode:: :hide: + import ray ray.shutdown() .. testcode:: + import ray ray.init(include_dashboard=False) .. tab-item:: VM Cluster Launcher diff --git a/doc/source/ray-observability/key-concepts.rst b/doc/source/ray-observability/key-concepts.rst index a734e0248edda..1b48c12c9501a 100644 --- a/doc/source/ray-observability/key-concepts.rst +++ b/doc/source/ray-observability/key-concepts.rst @@ -171,9 +171,8 @@ Logging directory structure By default, Ray logs are stored in a ``/tmp/ray/session_*/logs`` directory. -:::{note} +..{note}: The default temp directory is ``/tmp/ray`` (for Linux and MacOS). To change the temp directory, specify it when you call ``ray start`` or ``ray.init()``. -::: A new Ray instance creates a new session ID to the temp directory. The latest session ID is symlinked to ``/tmp/ray/session_latest``.