From 00829fee169b25a09de8857229a96d1c85fba68b Mon Sep 17 00:00:00 2001 From: simonsays1980 Date: Thu, 10 Oct 2024 11:26:25 +0200 Subject: [PATCH 01/46] [RLlib] Fix small bug in 'InfiniteLookBackBuffer.get_state/from_state'. (#47914) --- rllib/env/utils/infinite_lookback_buffer.py | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/rllib/env/utils/infinite_lookback_buffer.py b/rllib/env/utils/infinite_lookback_buffer.py index 269e3827ca205..cd84a8518097f 100644 --- a/rllib/env/utils/infinite_lookback_buffer.py +++ b/rllib/env/utils/infinite_lookback_buffer.py @@ -8,8 +8,10 @@ from ray.rllib.utils.serialization import gym_space_from_dict, gym_space_to_dict from ray.rllib.utils.spaces.space_utils import ( batch, + from_jsonable_if_needed, get_dummy_batch_for_space, get_base_struct_from_space, + to_jsonable_if_needed, ) @@ -71,12 +73,11 @@ def get_state(self) -> Dict[str, Any]: A dict containing all the data and metadata from the buffer. """ return { - "data": self.data, + "data": to_jsonable_if_needed(self.data, self.space) + if self.space + else self.data, "lookback": self.lookback, "finalized": self.finalized, - "space_struct": gym_space_to_dict(self.space_struct) - if self.space_struct - else self.space_struct, "space": gym_space_to_dict(self.space) if self.space else self.space, } @@ -92,16 +93,16 @@ def from_state(state: Dict[str, Any]) -> None: from the state dict. """ buffer = InfiniteLookbackBuffer() - buffer.data = state["data"] buffer.lookback = state["lookback"] buffer.finalized = state["finalized"] + buffer.space = gym_space_from_dict(state["space"]) if state["space"] else None buffer.space_struct = ( - gym_space_from_dict(state["space_struct"]) - if state["space_struct"] - else state["space_struct"] + get_base_struct_from_space(buffer.space) if buffer.space else None ) - buffer.space = ( - gym_space_from_dict(state["space"]) if state["space"] else state["space"] + buffer.data = ( + from_jsonable_if_needed(state["data"], buffer.space) + if buffer.space + else state["data"] ) return buffer From 92e43a255315b2e247360a5f58ca2436e96007d3 Mon Sep 17 00:00:00 2001 From: dentiny Date: Thu, 10 Oct 2024 02:35:07 -0700 Subject: [PATCH 02/46] [core] Add more debug string types (#47928) Followup on https://github.com/ray-project/ray/pull/47893, add more "blessed container types" to debug string function. Signed-off-by: dentiny --- src/ray/util/container_util.h | 27 +++++++++++++++++++++++ src/ray/util/tests/BUILD | 1 + src/ray/util/tests/container_util_test.cc | 21 ++++++++++++++++-- 3 files changed, 47 insertions(+), 2 deletions(-) diff --git a/src/ray/util/container_util.h b/src/ray/util/container_util.h index 1686c1459e250..d46da4d580d63 100644 --- a/src/ray/util/container_util.h +++ b/src/ray/util/container_util.h @@ -14,17 +14,20 @@ #pragma once +#include #include #include #include #include #include #include +#include #include #include #include "absl/container/flat_hash_map.h" #include "absl/container/flat_hash_set.h" +#include "absl/container/inlined_vector.h" #include "ray/util/logging.h" namespace ray { @@ -67,6 +70,13 @@ std::ostream &operator<<(std::ostream &os, DebugStringWrapper wrapper) { return os << wrapper.obj_; } +// TODO(hjiang): Implement debug string for `std::variant`. +template <> +inline std::ostream &operator<<(std::ostream &os, + DebugStringWrapper wrapper) { + return os << "(nullopt)"; +} + template std::ostream &operator<<(std::ostream &os, DebugStringWrapper> pair) { return os << "(" << debug_string(pair.obj_.first) << ", " @@ -93,6 +103,10 @@ std::ostream &operator<<(std::ostream &os, DebugStringWrapper> return os; } +template +std::ostream &operator<<(std::ostream &os, DebugStringWrapper> c) { + return c.StringifyContainer(os); +} template std::ostream &operator<<(std::ostream &os, DebugStringWrapper> c) { return c.StringifyContainer(os); @@ -120,6 +134,19 @@ std::ostream &operator<<(std::ostream &os, DebugStringWrapper> c) { return c.StringifyContainer(os); } +template +std::ostream &operator<<(std::ostream &os, + DebugStringWrapper> c) { + return c.StringifyContainer(os); +} + +template +std::ostream &operator<<(std::ostream &os, DebugStringWrapper> c) { + if (!c.obj_.has_value()) { + return os << debug_string(std::nullopt); + } + return os << debug_string(c.obj_.value()); +} template const typename C::mapped_type &map_find_or_die(const C &c, diff --git a/src/ray/util/tests/BUILD b/src/ray/util/tests/BUILD index 29cf17f706f7d..200425ea1a0da 100644 --- a/src/ray/util/tests/BUILD +++ b/src/ray/util/tests/BUILD @@ -7,6 +7,7 @@ cc_test( srcs = ["container_util_test.cc"], copts = COPTS, tags = ["team:core"], + linkstatic = True, deps = [ "//src/ray/util", "@com_google_absl//absl/container:flat_hash_map", diff --git a/src/ray/util/tests/container_util_test.cc b/src/ray/util/tests/container_util_test.cc index 0a34e91647a7b..590246fd2afd0 100644 --- a/src/ray/util/tests/container_util_test.cc +++ b/src/ray/util/tests/container_util_test.cc @@ -14,11 +14,13 @@ #include "ray/util/container_util.h" +#include + +#include #include +#include #include -#include "gtest/gtest.h" - namespace ray { template @@ -29,8 +31,18 @@ std::string debug_string_to_string(const T &t) { } TEST(ContainerUtilTest, TestDebugString) { + // Numerical values. ASSERT_EQ(debug_string_to_string(static_cast(2)), "2"); + + // String values. + ASSERT_EQ(debug_string_to_string(std::string_view{"hello"}), "hello"); + ASSERT_EQ(debug_string_to_string(std::string{"hello"}), "hello"); + + // Non-associative containers. ASSERT_EQ(debug_string_to_string(std::vector{1, 2}), "[1, 2]"); + ASSERT_EQ(debug_string_to_string(std::array{1, 2, 3}), "[1, 2, 3]"); + + // Associative containers. ASSERT_EQ(debug_string_to_string(std::set{1, 2}), "[1, 2]"); ASSERT_EQ(debug_string_to_string(std::unordered_set{2}), "[2]"); ASSERT_EQ(debug_string_to_string(absl::flat_hash_set{1}), "[1]"); @@ -54,6 +66,11 @@ TEST(ContainerUtilTest, TestDebugString) { ASSERT_EQ(debug_string_to_string(std::pair{3, "value"}), "(3, value)"); + // Optional. + ASSERT_EQ(debug_string_to_string(std::nullopt), "(nullopt)"); + ASSERT_EQ(debug_string_to_string(std::optional{}), "(nullopt)"); + ASSERT_EQ(debug_string_to_string(std::optional{"hello"}), "hello"); + // Composable: tuples of pairs of maps and vectors. ASSERT_EQ(debug_string_to_string( std::tuple>, std::map>{ From be14ef3498edb52184174c9b405eb7847a46750b Mon Sep 17 00:00:00 2001 From: Lonnie Liu <95255098+aslonnie@users.noreply.github.com> Date: Thu, 10 Oct 2024 03:11:47 -0700 Subject: [PATCH 03/46] [deps] add grpcio-tools into anyscale dependencies (#47955) so that it participates in the dependency resolving process Signed-off-by: Lonnie Liu --- python/requirements/anyscale-requirements.txt | 1 + python/requirements_compiled.txt | 4 ++++ 2 files changed, 5 insertions(+) diff --git a/python/requirements/anyscale-requirements.txt b/python/requirements/anyscale-requirements.txt index 3fef80ca3868a..c4145a74e8331 100644 --- a/python/requirements/anyscale-requirements.txt +++ b/python/requirements/anyscale-requirements.txt @@ -5,6 +5,7 @@ opentelemetry-sdk opentelemetry-exporter-otlp google-cloud-storage grpcio==1.60.0 +grpcio-tools pyyaml pyopenssl certifi diff --git a/python/requirements_compiled.txt b/python/requirements_compiled.txt index 5303ae2d23196..61318c9b0d1e5 100644 --- a/python/requirements_compiled.txt +++ b/python/requirements_compiled.txt @@ -689,6 +689,7 @@ grpcio==1.60.0 ; sys_platform != "darwin" # google-api-core # google-cloud-bigquery # grpcio-status + # grpcio-tools # mlagents-envs # opencensus-proto # opentelemetry-exporter-opencensus @@ -697,6 +698,8 @@ grpcio==1.60.0 ; sys_platform != "darwin" # tensorflow grpcio-status==1.48.2 # via google-api-core +grpcio-tools==1.48.2 + # via -r /ray/ci/../python/requirements/anyscale-requirements.txt gsutil==5.27 # via -r /ray/ci/../python/requirements/docker/ray-docker-requirements.txt gunicorn==20.1.0 @@ -1550,6 +1553,7 @@ protobuf==3.20.3 # google-cloud-bigquery-storage # googleapis-common-protos # grpcio-status + # grpcio-tools # mlagents-envs # mlflow # onnx From 59af152c93aca025cca405f73ac5d9ad9c4f5a95 Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Thu, 10 Oct 2024 18:10:52 +0200 Subject: [PATCH 04/46] [RLlib] Quick-fix for default RLModules in combination with a user-provided config-sub-dict (instead of a full `DefaultModelConfig`). (#47965) --- rllib/algorithms/bc/torch/bc_torch_rl_module.py | 2 +- rllib/core/rl_module/multi_rl_module.py | 14 +++++++++----- rllib/core/rl_module/rl_module.py | 2 +- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/rllib/algorithms/bc/torch/bc_torch_rl_module.py b/rllib/algorithms/bc/torch/bc_torch_rl_module.py index a547047d7f417..d06c323b124ef 100644 --- a/rllib/algorithms/bc/torch/bc_torch_rl_module.py +++ b/rllib/algorithms/bc/torch/bc_torch_rl_module.py @@ -11,7 +11,7 @@ class BCTorchRLModule(TorchRLModule): @override(RLModule) def setup(self): # __sphinx_doc_begin__ - # Build models from catalog + # Build models from catalog. self.encoder = self.catalog.build_encoder(framework=self.framework) self.pi = self.catalog.build_pi_head(framework=self.framework) diff --git a/rllib/core/rl_module/multi_rl_module.py b/rllib/core/rl_module/multi_rl_module.py index fb3e34f4339dd..43eddb909dea0 100644 --- a/rllib/core/rl_module/multi_rl_module.py +++ b/rllib/core/rl_module/multi_rl_module.py @@ -1,5 +1,5 @@ import copy -from dataclasses import dataclass, field +import dataclasses import logging import pprint from typing import ( @@ -553,7 +553,7 @@ def _check_module_exists(self, module_id: ModuleID) -> None: @PublicAPI(stability="alpha") -@dataclass +@dataclasses.dataclass class MultiRLModuleSpec: """A utility spec class to make it constructing MultiRLModules easier. @@ -666,7 +666,11 @@ def build(self, module_id: Optional[ModuleID] = None) -> RLModule: observation_space=self.observation_space, action_space=self.action_space, inference_only=self.inference_only, - model_config=self.model_config, + model_config=( + dataclasses.asdict(self.model_config) + if dataclasses.is_dataclass(self.model_config) + else self.model_config + ), rl_module_specs=self.rl_module_specs, ) # Older custom model might still require the old `MultiRLModuleConfig` under @@ -859,10 +863,10 @@ def get_rl_module_config(self): "module2: [RLModuleSpec], ..}, inference_only=..)", error=False, ) -@dataclass +@dataclasses.dataclass class MultiRLModuleConfig: inference_only: bool = False - modules: Dict[ModuleID, RLModuleSpec] = field(default_factory=dict) + modules: Dict[ModuleID, RLModuleSpec] = dataclasses.field(default_factory=dict) def to_dict(self): return { diff --git a/rllib/core/rl_module/rl_module.py b/rllib/core/rl_module/rl_module.py index f1fb5b337cc54..42aa0a780ed45 100644 --- a/rllib/core/rl_module/rl_module.py +++ b/rllib/core/rl_module/rl_module.py @@ -98,7 +98,7 @@ def build(self) -> "RLModule": observation_space=self.observation_space, action_space=self.action_space, inference_only=self.inference_only, - model_config=self.model_config, + model_config=self._get_model_config(), catalog_class=self.catalog_class, ) # Older custom model might still require the old `RLModuleConfig` under From 793ea9d7b4bf78f56e87149012b8ff90164fa3a1 Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Thu, 10 Oct 2024 18:12:38 +0200 Subject: [PATCH 05/46] [RLlib] Cleanup examples folder vol. 25: Remove some old API stack examples. (#47970) --- doc/source/rllib/rllib-examples.rst | 4 - doc/source/rllib/rllib-models.rst | 56 +----- rllib/BUILD | 175 ------------------ rllib/examples/cartpole_lstm.py | 94 ---------- rllib/examples/custom_model_api.py | 109 ----------- .../examples/custom_model_loss_and_metrics.py | 117 ------------ rllib/examples/learners/ppo_tuner.py | 61 ------ 7 files changed, 2 insertions(+), 614 deletions(-) delete mode 100644 rllib/examples/cartpole_lstm.py delete mode 100644 rllib/examples/custom_model_api.py delete mode 100644 rllib/examples/custom_model_loss_and_metrics.py delete mode 100644 rllib/examples/learners/ppo_tuner.py diff --git a/doc/source/rllib/rllib-examples.rst b/doc/source/rllib/rllib-examples.rst index 6457ebd171871..5a2c4dca69f63 100644 --- a/doc/source/rllib/rllib-examples.rst +++ b/doc/source/rllib/rllib-examples.rst @@ -254,12 +254,8 @@ RLModules - |old_stack| `How to using the "Repeated" space of RLlib for variable lengths observations `__: How to use RLlib's `Repeated` space to handle variable length observations. - |old_stack| `How to write a custom Keras model `__: - Example of using a custom Keras model. -- |old_stack| `How to register a custom model with supervised loss `__: Example of defining and registering a custom model with a supervised loss. - |old_stack| `How to train with batch normalization `__: - Example of adding batch norm layers to a custom model. -- |old_stack| `How to write a custom model with its custom API `__: Shows how to define a custom Model API in RLlib, such that it can be used inside certain algorithms. - |old_stack| `How to write a "trajectory ciew API" utilizing model `__: An example on how a model can use the trajectory view API to specify its own input. diff --git a/doc/source/rllib/rllib-models.rst b/doc/source/rllib/rllib-models.rst index 5e3badd3b8e3f..717c6bb196c6e 100644 --- a/doc/source/rllib/rllib-models.rst +++ b/doc/source/rllib/rllib-models.rst @@ -364,59 +364,7 @@ calculating head on top of your policy model. In order to expand a Model's API, define and implement a new method (e.g. ``get_q_values()``) in your TF- or TorchModelV2 sub-class. You can now wrap this new API either around RLlib's default models or around -your custom (``forward()``-overriding) model classes. Here are two examples that illustrate how to do this: - -**The Q-head API: Adding a dueling layer on top of a default RLlib model**. - -The following code adds a ``get_q_values()`` method to the automatically chosen -default Model (e.g. a ``FullyConnectedNetwork`` if the observation space is a 1D Box -or Discrete): - -.. literalinclude:: ../../../rllib/examples/_old_api_stack/models/custom_model_api.py - :language: python - :start-after: __sphinx_doc_model_api_1_begin__ - :end-before: __sphinx_doc_model_api_1_end__ - -Now, for your algorithm that needs to have this model API to work properly (e.g. DQN), -you use this following code to construct the complete final Model using the -``ModelCatalog.get_model_v2`` factory function (`code here `__): - -.. literalinclude:: ../../../rllib/examples/custom_model_api.py - :language: python - :start-after: __sphinx_doc_model_construct_1_begin__ - :end-before: __sphinx_doc_model_construct_1_end__ - -With the model object constructed above, you can get the underlying intermediate output (before the dueling head) -by calling ``my_dueling_model`` directly (``out = my_dueling_model([input_dict])``), and then passing ``out`` into -your custom ``get_q_values`` method: ``q_values = my_dueling_model.get_q_values(out)``. - - -**The single Q-value API for SAC**. - -Our DQN model from above takes an observation and outputs one Q-value per (discrete) action. -Continuous SAC - on the other hand - uses Models that calculate one Q-value only -for a single (**continuous**) action, given an observation and that particular action. - -Let's take a look at how we would construct this API and wrap it around a custom model: - -.. literalinclude:: ../../../rllib/examples/_old_api_stack/models/custom_model_api.py - :language: python - :start-after: __sphinx_doc_model_api_2_begin__ - :end-before: __sphinx_doc_model_api_2_end__ - -Now, for your algorithm that needs to have this model API to work properly (e.g. SAC), -you use this following code to construct the complete final Model using the -``ModelCatalog.get_model_v2`` factory function (`code here `__): - -.. literalinclude:: ../../../rllib/examples/custom_model_api.py - :language: python - :start-after: __sphinx_doc_model_construct_2_begin__ - :end-before: __sphinx_doc_model_construct_2_end__ - -With the model object constructed above, you can get the underlying intermediate output (before the q-head) -by calling ``my_cont_action_q_model`` directly (``out = my_cont_action_q_model([input_dict])``), and then passing ``out`` -and some action into your custom ``get_single_q_value`` method: -``q_value = my_cont_action_q_model.get_signle_q_value(out, action)``. +your custom (``forward()``-overriding) model classes. More examples for Building Custom Models @@ -505,7 +453,7 @@ Supervised Model Losses You can mix supervised losses into any RLlib algorithm through custom models. For example, you can add an imitation learning loss on expert experiences, or a self-supervised autoencoder loss within the model. These losses can be defined over either policy evaluation inputs, or data read from `offline storage `__. -**TensorFlow**: To add a supervised loss to a custom TF model, you need to override the ``custom_loss()`` method. This method takes in the existing policy loss for the algorithm, which you can add your own supervised loss to before returning. For debugging, you can also return a dictionary of scalar tensors in the ``metrics()`` method. Here is a `runnable example `__ of adding an imitation loss to CartPole training that is defined over a `offline dataset `__. +**TensorFlow**: To add a supervised loss to a custom TF model, you need to override the ``custom_loss()`` method. This method takes in the existing policy loss for the algorithm, which you can add your own supervised loss to before returning. For debugging, you can also return a dictionary of scalar tensors in the ``metrics()`` method. **PyTorch**: There is no explicit API for adding losses to custom torch models. However, you can modify the loss in the policy definition directly. Like for TF models, offline datasets can be incorporated by creating an input reader and calling ``reader.next()`` in the loss forward pass. diff --git a/rllib/BUILD b/rllib/BUILD index 469a5c57a9509..7c2259b84768e 100644 --- a/rllib/BUILD +++ b/rllib/BUILD @@ -2836,107 +2836,6 @@ py_test( args = ["--enable-new-api-stack", "--as-test"] ) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_local_cpu_torch", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=torch", "--config=local-cpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_local_cpu_tf2", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=tf2", "--config=local-cpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_local_gpu_torch", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples", "gpu"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=torch", "--config=local-gpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_local_gpu_tf2", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples", "gpu", "exclusive"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=tf2", "--config=local-gpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_remote_cpu_torch", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=torch", "--config=remote-cpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_remote_cpu_tf2", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=tf2", "--config=remote-cpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_remote_gpu_torch", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples", "gpu", "exclusive"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=torch", "--config=remote-gpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_remote_gpu_tf2", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples", "gpu", "exclusive"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=tf2", "--config=remote-gpu"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_multi_gpu_torch", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples", "multi_gpu", "exclusive"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=torch", "--config=multi-gpu-ddp"] -) - -#@OldAPIStack @HybridAPIStack -py_test( - name = "examples/learners/ppo_tuner_multi_gpu_tf2", - main = "examples/learners/ppo_tuner.py", - tags = ["team:rllib", "examples", "multi_gpu", "exclusive"], - size = "medium", - srcs = ["examples/learners/ppo_tuner.py"], - args = ["--framework=tf2", "--config=multi-gpu-ddp"] -) - # subdirectory: multi_agent/ # .................................... py_test( @@ -3256,56 +3155,6 @@ py_test( args = ["--as-test", "--framework=torch", "--stop-reward=-0.012", "--num-cpus=4"] ) -#@OldAPIStack -py_test( - name = "examples/cartpole_lstm_impala_tf2", - main = "examples/cartpole_lstm.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/cartpole_lstm.py"], - args = ["--run=IMPALA", "--as-test", "--framework=tf2", "--stop-reward=28", "--num-cpus=4"] -) - -#@OldAPIStack -py_test( - name = "examples/cartpole_lstm_impala_torch", - main = "examples/cartpole_lstm.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/cartpole_lstm.py"], - args = ["--run=IMPALA", "--as-test", "--framework=torch", "--stop-reward=28", "--num-cpus=4"] -) - -#@OldAPIStack -py_test( - name = "examples/cartpole_lstm_ppo_tf2", - main = "examples/cartpole_lstm.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "large", - srcs = ["examples/cartpole_lstm.py"], - args = ["--run=PPO", "--as-test", "--framework=tf2", "--stop-reward=28", "--num-cpus=4"] -) - -#@OldAPIStack -py_test( - name = "examples/cartpole_lstm_ppo_torch", - main = "examples/cartpole_lstm.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/cartpole_lstm.py"], - args = ["--run=PPO", "--as-test", "--framework=torch", "--stop-reward=28", "--num-cpus=4"] -) - -#@OldAPIStack -py_test( - name = "examples/cartpole_lstm_ppo_torch_with_prev_a_and_r", - main = "examples/cartpole_lstm.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "medium", - srcs = ["examples/cartpole_lstm.py"], - args = ["--run=PPO", "--as-test", "--framework=torch", "--stop-reward=28", "--num-cpus=4", "--use-prev-action", "--use-prev-reward"] -) - #@OldAPIStack py_test( name = "examples/centralized_critic_tf", @@ -3356,30 +3205,6 @@ py_test( args = ["--stop-iters=2"] ) -#@OldAPIStack -py_test( - name = "examples/custom_model_loss_and_metrics_ppo_tf", - main = "examples/custom_model_loss_and_metrics.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "small", - # Include the json data file. - data = ["tests/data/cartpole/small.json"], - srcs = ["examples/custom_model_loss_and_metrics.py"], - args = ["--run=PPO", "--stop-iters=1", "--framework=tf","--input-files=tests/data/cartpole"] -) - -#@OldAPIStack -py_test( - name = "examples/custom_model_loss_and_metrics_ppo_torch", - main = "examples/custom_model_loss_and_metrics.py", - tags = ["team:rllib", "exclusive", "examples"], - size = "small", - # Include the json data file. - data = ["tests/data/cartpole/small.json"], - srcs = ["examples/custom_model_loss_and_metrics.py"], - args = ["--run=PPO", "--framework=torch", "--stop-iters=1", "--input-files=tests/data/cartpole"] -) - py_test( name = "examples/custom_recurrent_rnn_tokenizer_repeat_after_me_tf2", main = "examples/custom_recurrent_rnn_tokenizer.py", diff --git a/rllib/examples/cartpole_lstm.py b/rllib/examples/cartpole_lstm.py deleted file mode 100644 index c7454161ab06b..0000000000000 --- a/rllib/examples/cartpole_lstm.py +++ /dev/null @@ -1,94 +0,0 @@ -# @OldAPIStack - -# TODO (sven): Move this script to `examples/rl_modules/...` - -import argparse -import os - -from ray.air.constants import TRAINING_ITERATION -from ray.rllib.examples.envs.classes.stateless_cartpole import StatelessCartPole -from ray.rllib.utils.metrics import ( - ENV_RUNNER_RESULTS, - EPISODE_RETURN_MEAN, - NUM_ENV_STEPS_SAMPLED_LIFETIME, -) -from ray.rllib.utils.test_utils import check_learning_achieved -from ray.tune.registry import get_trainable_cls - -parser = argparse.ArgumentParser() -parser.add_argument( - "--run", type=str, default="PPO", help="The RLlib-registered algorithm to use." -) -parser.add_argument("--num-cpus", type=int, default=0) -parser.add_argument( - "--framework", - choices=["tf", "tf2", "torch"], - default="torch", - help="The DL framework specifier.", -) -parser.add_argument("--use-prev-action", action="store_true") -parser.add_argument("--use-prev-reward", action="store_true") -parser.add_argument( - "--as-test", - action="store_true", - help="Whether this script should be run as a test: --stop-reward must " - "be achieved within --stop-timesteps AND --stop-iters.", -) -parser.add_argument( - "--stop-iters", type=int, default=200, help="Number of iterations to train." -) -parser.add_argument( - "--stop-timesteps", type=int, default=100000, help="Number of timesteps to train." -) -parser.add_argument( - "--stop-reward", type=float, default=150.0, help="Reward at which we stop training." -) - -if __name__ == "__main__": - import ray - from ray import air, tune - - args = parser.parse_args() - - ray.init() - - algo_cls = get_trainable_cls(args.run) - config = algo_cls.get_default_config() - - config.environment(env=StatelessCartPole).resources( - num_gpus=int(os.environ.get("RLLIB_NUM_GPUS", "0")) - ).framework(args.framework).reporting(min_time_s_per_iteration=0.1).training( - model={ - "use_lstm": True, - "lstm_cell_size": 32, - "lstm_use_prev_action": args.use_prev_action, - "lstm_use_prev_reward": args.use_prev_reward, - } - ) - - if args.run == "PPO": - config.training(num_epochs=5, vf_loss_coeff=0.0001, train_batch_size=512) - config.model["vf_share_layers"] = True - elif args.run == "IMPALA": - config.env_runners(num_env_runners=2) - config.resources(num_gpus=0) - config.training(vf_loss_coeff=0.01) - - stop = { - TRAINING_ITERATION: args.stop_iters, - NUM_ENV_STEPS_SAMPLED_LIFETIME: args.stop_timesteps, - f"{ENV_RUNNER_RESULTS}/{EPISODE_RETURN_MEAN}": args.stop_reward, - } - - tuner = tune.Tuner( - args.run, - param_space=config.to_dict(), - run_config=air.RunConfig( - stop=stop, - ), - ) - results = tuner.fit() - - if args.as_test: - check_learning_achieved(results, args.stop_reward) - ray.shutdown() diff --git a/rllib/examples/custom_model_api.py b/rllib/examples/custom_model_api.py deleted file mode 100644 index e1e6705bbf771..0000000000000 --- a/rllib/examples/custom_model_api.py +++ /dev/null @@ -1,109 +0,0 @@ -# @OldAPIStack -import argparse -from gymnasium.spaces import Box, Discrete -import numpy as np - -from ray.rllib.examples._old_api_stack.models.custom_model_api import ( - DuelingQModel, - TorchDuelingQModel, - ContActionQModel, - TorchContActionQModel, -) -from ray.rllib.models.catalog import ModelCatalog, MODEL_DEFAULTS -from ray.rllib.policy.sample_batch import SampleBatch -from ray.rllib.utils.framework import try_import_tf, try_import_torch - -tf1, tf, tfv = try_import_tf() -torch, _ = try_import_torch() - -parser = argparse.ArgumentParser() -parser.add_argument( - "--framework", - choices=["tf", "tf2", "torch"], - default="torch", - help="The DL framework specifier.", -) - -if __name__ == "__main__": - args = parser.parse_args() - - # Test API wrapper for dueling Q-head. - - obs_space = Box(-1.0, 1.0, (3,)) - action_space = Discrete(3) - - # Run in eager mode for value checking and debugging. - tf1.enable_eager_execution() - - # __sphinx_doc_model_construct_1_begin__ - my_dueling_model = ModelCatalog.get_model_v2( - obs_space=obs_space, - action_space=action_space, - num_outputs=action_space.n, - model_config=MODEL_DEFAULTS, - framework=args.framework, - # Providing the `model_interface` arg will make the factory - # wrap the chosen default model with our new model API class - # (DuelingQModel). This way, both `forward` and `get_q_values` - # are available in the returned class. - model_interface=DuelingQModel - if args.framework != "torch" - else TorchDuelingQModel, - name="dueling_q_model", - ) - # __sphinx_doc_model_construct_1_end__ - - batch_size = 10 - input_ = np.array([obs_space.sample() for _ in range(batch_size)]) - # Note that for PyTorch, you will have to provide torch tensors here. - if args.framework == "torch": - input_ = torch.from_numpy(input_) - - input_dict = SampleBatch(obs=input_, _is_training=False) - out, state_outs = my_dueling_model(input_dict=input_dict) - assert out.shape == (10, 256) - # Pass `out` into `get_q_values` - q_values = my_dueling_model.get_q_values(out) - assert q_values.shape == (10, action_space.n) - - # Test API wrapper for single value Q-head from obs/action input. - - obs_space = Box(-1.0, 1.0, (3,)) - action_space = Box(-1.0, -1.0, (2,)) - - # __sphinx_doc_model_construct_2_begin__ - my_cont_action_q_model = ModelCatalog.get_model_v2( - obs_space=obs_space, - action_space=action_space, - num_outputs=2, - model_config=MODEL_DEFAULTS, - framework=args.framework, - # Providing the `model_interface` arg will make the factory - # wrap the chosen default model with our new model API class - # (DuelingQModel). This way, both `forward` and `get_q_values` - # are available in the returned class. - model_interface=ContActionQModel - if args.framework != "torch" - else TorchContActionQModel, - name="cont_action_q_model", - ) - # __sphinx_doc_model_construct_2_end__ - - batch_size = 10 - input_ = np.array([obs_space.sample() for _ in range(batch_size)]) - - # Note that for PyTorch, you will have to provide torch tensors here. - if args.framework == "torch": - input_ = torch.from_numpy(input_) - - input_dict = SampleBatch(obs=input_, _is_training=False) - # Note that for PyTorch, you will have to provide torch tensors here. - out, state_outs = my_cont_action_q_model(input_dict=input_dict) - assert out.shape == (10, 256) - # Pass `out` and an action into `my_cont_action_q_model` - action = np.array([action_space.sample() for _ in range(batch_size)]) - if args.framework == "torch": - action = torch.from_numpy(action) - - q_value = my_cont_action_q_model.get_single_q_value(out, action) - assert q_value.shape == (10, 1) diff --git a/rllib/examples/custom_model_loss_and_metrics.py b/rllib/examples/custom_model_loss_and_metrics.py deleted file mode 100644 index ccb3d8e1acd07..0000000000000 --- a/rllib/examples/custom_model_loss_and_metrics.py +++ /dev/null @@ -1,117 +0,0 @@ -# @OldAPIStack - -# Users should just inherit the Learner and extend the loss_fn. -# TODO (sven): Move this example script to `examples/learners/...` - -"""Example of using custom_loss() with an imitation learning loss under the Policy -and ModelV2 API. - -The default input file is too small to learn a good policy, but you can -generate new experiences for IL training as follows: - -To generate experiences: -$ ./train.py --run=PG --config='{"output": "/tmp/cartpole"}' --env=CartPole-v1 - -To train on experiences with joint PG + IL loss: -$ python custom_loss.py --input-files=/tmp/cartpole -""" - -import argparse -from pathlib import Path -import os - -import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION -from ray.rllib.core import DEFAULT_MODULE_ID -from ray.rllib.examples._old_api_stack.models.custom_loss_model import ( - CustomLossModel, - TorchCustomLossModel, -) -from ray.rllib.models import ModelCatalog -from ray.rllib.utils.framework import try_import_tf -from ray.rllib.utils.metrics.learner_info import LEARNER_INFO, LEARNER_STATS_KEY -from ray.tune.registry import get_trainable_cls - -tf1, tf, tfv = try_import_tf() - -parser = argparse.ArgumentParser() -parser.add_argument( - "--run", type=str, default="PPO", help="The RLlib-registered algorithm to use." -) -parser.add_argument( - "--framework", - choices=["tf", "tf2", "torch"], - default="torch", - help="The DL framework specifier.", -) -parser.add_argument("--stop-iters", type=int, default=200) -parser.add_argument( - "--input-files", - type=str, - default=os.path.join( - os.path.dirname(os.path.abspath(__file__)), "../tests/data/cartpole/small.json" - ), -) - -if __name__ == "__main__": - ray.init() - args = parser.parse_args() - - # Bazel makes it hard to find files specified in `args` (and `data`). - # Look for them here. - if not os.path.exists(args.input_files): - # This script runs in the ray/rllib/examples dir. - rllib_dir = Path(__file__).parent.parent - input_dir = rllib_dir.absolute().joinpath(args.input_files) - args.input_files = str(input_dir) - - ModelCatalog.register_custom_model( - "custom_loss", - TorchCustomLossModel if args.framework == "torch" else CustomLossModel, - ) - - config = ( - get_trainable_cls(args.run) - .get_default_config() - .environment("CartPole-v1") - .framework(args.framework) - .env_runners(num_env_runners=0) - .training( - model={ - "custom_model": "custom_loss", - "custom_model_config": { - "input_files": args.input_files, - }, - }, - ) - # Use GPUs iff `RLLIB_NUM_GPUS` env var set to > 0. - .resources(num_gpus=int(os.environ.get("RLLIB_NUM_GPUS", "0"))) - ) - - stop = {TRAINING_ITERATION: args.stop_iters} - - tuner = tune.Tuner( - args.run, - param_space=config, - run_config=air.RunConfig(stop=stop, verbose=1), - ) - results = tuner.fit() - info = results.get_best_result().metrics["info"] - - # Torch metrics structure. - if args.framework == "torch": - assert LEARNER_STATS_KEY in info[LEARNER_INFO][DEFAULT_MODULE_ID] - assert "model" in info[LEARNER_INFO][DEFAULT_MODULE_ID] - assert "custom_metrics" in info[LEARNER_INFO][DEFAULT_MODULE_ID] - - # TODO: (sven) Make sure the metrics structure gets unified between - # tf and torch. Tf should work like current torch: - # info: - # learner: - # [policy_id] - # learner_stats: [return values of policy's `stats_fn`] - # model: [return values of ModelV2's `metrics` method] - # custom_metrics: [return values of callback: `on_learn_on_batch`] - else: - assert "model" in info[LEARNER_INFO][DEFAULT_MODULE_ID][LEARNER_STATS_KEY] diff --git a/rllib/examples/learners/ppo_tuner.py b/rllib/examples/learners/ppo_tuner.py deleted file mode 100644 index a27e292b9efa7..0000000000000 --- a/rllib/examples/learners/ppo_tuner.py +++ /dev/null @@ -1,61 +0,0 @@ -import argparse - -import ray -from ray import air, tune -from ray.air.constants import TRAINING_ITERATION -from ray.rllib.algorithms.ppo import PPOConfig - -LEARNER_CONFIG = { - "remote-cpu": {"num_learners": 1}, - "remote-gpu": {"num_learners": 1, "num_gpus_per_learner": 1}, - "multi-gpu-ddp": { - "num_learners": 2, - "num_gpus_per_learner": 1, - }, - "local-cpu": {}, - "local-gpu": {"num_gpus_per_learner": 1}, -} - - -def _parse_args(): - - parser = argparse.ArgumentParser() - parser.add_argument( - "--config", - type=str, - default="local-cpu", - ) - - parser.add_argument( - "--framework", - choices=["tf2", "torch"], # tf will be deprecated with the new Learner stack - default="torch", - ) - - return parser.parse_args() - - -if __name__ == "__main__": - args = _parse_args() - - ray.init() - - config = ( - PPOConfig() - .framework(args.framework) - .environment("CartPole-v1") - .learners(**LEARNER_CONFIG[args.config]) - ) - - print("Testing with learner config: ", LEARNER_CONFIG[args.config]) - print("Testing with framework: ", args.framework) - print("-" * 80) - tuner = tune.Tuner( - "PPO", - param_space=config.to_dict(), - run_config=air.RunConfig( - stop={TRAINING_ITERATION: 1}, - failure_config=air.FailureConfig(fail_fast="raise"), - ), - ) - tuner.fit() From 15c25b6c0fa0a0d2057b8247ae5043066ddcfe22 Mon Sep 17 00:00:00 2001 From: Chi-Sheng Liu Date: Fri, 11 Oct 2024 02:12:10 +0800 Subject: [PATCH 06/46] [Docs][KubeRay] Add YuniKorn Gang scheduling doc (#47850) Signed-off-by: Chi-Sheng Liu --- .../yunikorn-dashboard-apps-pending.png | Bin 0 -> 55273 bytes .../yunikorn-dashboard-apps-running.png | Bin 0 -> 67450 bytes .../cluster/kubernetes/k8s-ecosystem.md | 2 + .../kubernetes/k8s-ecosystem/yunikorn.md | 190 ++++++++++++++++++ 4 files changed, 192 insertions(+) create mode 100644 doc/source/cluster/kubernetes/images/yunikorn-dashboard-apps-pending.png create mode 100644 doc/source/cluster/kubernetes/images/yunikorn-dashboard-apps-running.png create mode 100644 doc/source/cluster/kubernetes/k8s-ecosystem/yunikorn.md diff --git a/doc/source/cluster/kubernetes/images/yunikorn-dashboard-apps-pending.png b/doc/source/cluster/kubernetes/images/yunikorn-dashboard-apps-pending.png new file mode 100644 index 0000000000000000000000000000000000000000..e5b56b483af0f809966c360ca7a361787ba710c4 GIT binary patch literal 55273 zcmd43by$^c^Dc}9Dgpuu(z0kIq#IO_?(XiCZcq_vkd|E1u;^}(?(XjHZq~l={rNoa z_wD1@``^9(!F$PF*IYAm&N=7cCoL(2@)Y+e0s;bxh_HYx0s@jJ0>V9~$M?ZEQ&Fj1 z;Ku{o4^ZS`yn3@mMpt?c$5)bk=B zyhacacrWi1zdhrs9_W09v~Ok@8xxy;q%Zc=FPJooIS(C%st(EAu_7eS;X6GH-~N0XE(L^dn{)a zuGS|BTZZf>L{bRwXCm0Br4Dy5@}CdZpc%!#hX}N?^73|PyE8s$M5`++PrN@AD$_)Y`uTRPY_6|Aet2I_ZU~it({5w1 zrN4ha*TDO~uJzkBD<_ABm-lvaB>T~$N35)@;1@sH*}ZuDkdu=$Ha1p3UPXlpE5Q4M zj-};pu39;`kiNP3bhZ+`l9CcH6rMh$2i^~!zJ}l=M5@6)JW2J>gJr&#kr~L6%h1); zy*%C7T3pmMGn;D0l#`Rg#Kff6sA6{7pVw*nUOSN1pUKf&US z*Jy1y=+hpXdy%6|p|G+tjS^*qC>c6K%rjJt%==DBTKn8dA(Z8GVw<`rNFE1k_FpM^=MYs{ZPoqQ*=M%2M1U$UzWa=WE$hmri>H~ zMx3%8|11dWc!t>2E?sZ6x7##xH8E*7I5@m)V&nHd%wIIe>wEb#Gw8klx$L|+?=TBg~sD~ypCn`zu!qn7STM?#}+$+7c zNGg@FvFgWjpU;$L{YGq8yPt4|9SJ6Pg*o;?nbxNEjRuM#iuB3_htlcWLE?e}q=6G9 z+8+J%9bs&&|J<_X{m8Vm$gshpv5Em)3K2eRLafku zxSN!Bg*tg;M3h3d**}5W*bl5zRYgUKB`O^qol?H0g~-F8sQ4UdN+{NgM0RJL%~S?) z@)U=R4@v#~+AMa83XwDmojqISgUdF~wGZUYlC>`0@_$x5dP{sjbzo;t<}HFf)TTCB zG`DVUzSRMrnU~;BtV*Yn;KN3zvSU2^#ORDQIBjOV|9tUaZX55f)jp?FQd3h?QWlh! zTFh>QU1gwMTT^ogk;*;K4@w&||x;&`_b5vRDjB>!Hy3=ot-l^pv*ExS|~;V#&(`mr3ndd4>FJ>J{Fh zlPZ20BEzdgQ)heo(h^Zcb%=|sbo-h~OpHhK$YR)sn1s0KD|R9@+HZ=Gu0ScQu&lKg z?avM5c`L{J;zuY&^3&_$M zyDQxB#HfrlJ>N5ZofsdVu5}$787XzHEGfAJOKnTI0(V$%hELclM#gH zZ`THLr_Oj8S+@sfX5VQEB}r>aR@^p)Tmfsc)bOghwzI{JRTuZ=-Pp(oh|F{(JYe0! zH~K`j+{Gc3$aV|Nw`{J^^O(oy5nlbN37pw{ zq*DP~6gS|}9)U`0i#~CTa}_)10ZkKH25G7=L3~9YTE;g-%UVMQkGg=jmv*x0(xiIG z-2{k1aq~JSIT9hH4qIQ{Hzeii3p)I{9Z5*?1Dz~5D=R|IcC9o$gD!5r{IXwzUD7MF zm7e+plODM_omZv?x`m3%jZMSawDS)v`#inKFw^4$e1oea$zCX6NNMl(kGVE}!qXu4 z(`9u(ag8;VKFp(EecoqaSpdc4+H2m76go;Uw2GYyzu^|NYu-3loA8F#bZr(j7h7x}cKef> zntagIaxcXN7**_hQ`|y@b22i-1PoTfx{)SotEO;Xt+UuY4exl%JWMn^XYe_$_(Rg` zw6G4{k<(qLs2D^svGDASjG^9blJZ$e`JqrRUqPp1sVZv0*%u!t)s=QuowKtP-xt(v9 zT?+fC&)eP-LDj!ILUnx>@huL6Vy%8G=%tBIxj~$xBi<_6k0(|wE#!o=@p4sfjcZnp z?pq(;5;1aC9OLaW3&$ql<2hQ+wVIpE>pBjRr@j`MC^rVm?uPe2T{i9f+iUjD zXARBe0xm}_x(+3+ka^>&7fS6vZBqOS3smbs=aL<&saId0xV<>v#*AZ)3ad=;l5>^m z-6>g4;gjE2qI*U88WNUHFuvbZnPD!66#ZUMKGylo#r2m!*G6x#RyCWgnCE}3X}Yqq z!okVOx-~pllYy~VdAwx=qh8#fbUiK$Y!~nA%1cF5(!tuOjNBw-(H?OPK%u|d! zEJjL=bu1SJJ(T8XKh{2DXffRT0nbB$u;n4^3&x?)0Iyb9l)hLPztVPv&8|Sv8uu96 z%*Tg+^ZgC^i>Hsit}=JN#$c~>jf%2&pQTikp`;<#Esi@AX87{%^^2!*kx$SEw&=_9 zq^$iY87~)`w%abGzA1%St}U+`tSo1d+2qzxJTmHQ3lK1Xo%A9ix}cmy<`GvH!P_Tf zcRa#ts;qr|+s)T=7A#|PIgwZ^MG z0-Ku2%z_D~pf76A#vR~=2~ZSP+;Yf9<1EVq&D-XZgnH$Y#+i8;f%X~=$CkdcVlLF5 zH6B-GM*}{KQnyZ|<5uQIJ4Qmvh91?ZWIM?FSGp~QzBeL#Qd_MIM^pBcCTGhYUKCar z7j5T1`44pB%DZ=UhOYyVHc3Pzg(IbXX6Y<^eTB`CH_>tKGPBZ5|nGQGl!4x~ZiRPY5vlSM*^EEZv_xNo{ch<>akyU=nPKGkMN>ty zF3{Ixqt<*TeJ?RG(LmNuAl>K^Jx<@-Swh`6zuhmOC!)}Tn}TzsY+#-v@NBmur45^Y z)|I#4ZQe-%gqv{!{G1bCfA0^E&}HmDmIexYWh;b}^lj+DD~qqWIp-U`|Mk24*H{7e zc6PCjdA~Bw&cCo|{f=IaN1m~!gF2zs=Ef;R4h#&S2ejl9jv=t*xRWQts3(HdHj-S? zOiXR)wJM3j6|lK!<;9j&-7Iz<2tigv)2?1Cp~h)wkeOBN2nKyEPSPQ{lu)Aeq;pJU zZdKVb8SiT&WVW|1EGvdX+NL9z+GJB(Q4@FhqiEiU#5uptZ5Wzh#l+5qj?3WIyWX~b ztJ<40?@rc0cKuX@+2|33aX{na5`0eHJV>d%vg^u{CK%uWJtB(d!Fd@OEt}~-XL%l* z-#J9re2d#wM)cC2Q^@-xHtdrGdwXr@ zoY0f*9(yXepJx&6zi!c2qgKr^R@a{R;ab(n#3oP>ml;qTRdG4f6^~YM&qb_g_&w%} z`*|x(nRVlVP~6eA`KGuSiV>rxwlzA_1rYGBHzpGtABsFqB`3@!6Yk-&h6|r==a*PI zvSs*!@MxyER&laxsHbOPVX43IO)KfIIheo5Q;!S}o0~*5HWT1EP+((BHCs$R%vg2D z(*SiYbtEg-qZ4_kU#2{1a9~}g%S`L|kDe7~-dlSU8;*}3$%GoEPIf0o`rqXjRFqH| zr4d~7=$;Q2%GFe*z39W86~wQ1cf5Wx7B$KPKbsIsCX`1HJw+K^J@HDIra?{R$@I+8 z-Zw(NCa@#?*rS!knTA+@X5_&W2#zk?&LSBh$y$0g4&LIo!=A>&jhR9)hL?`z?h0~& zhB4^q)}oMT#M~fy3Wz`TEc{4+Z5A~?{0XfY&V5=^QGcmubSVeE#pvr;_ecy@8AILV z{+Z&?Lnfad;xJ*{I)w>y);MAET&Isu{h!7}`1UNZoYy{5>GcbmBkIvFcf?^Hm}*p3 z3Q$NOtRz{~r=I%kzp4}TuH_qSz!+;x;&4bg!PsF6mJ!>c!LryWP<|!<~o11|P1cT z6RQ#@Zb+CYserv$OHt8NoKVC)1)NdFf`gxyOAH$1og(#-@y{_a9i42`^>u!flf_#7 zT{PzJT`P);ie=^(#X0iw@3|@VsOZvZDO=?g6>Tbe=ryu)FS}Q4XeMKDk)Mh4VpJTj zPl5`!3X(lt4OLK3AOm6I1(xHcv5cbT?Pf2YfWR^3qVRZ|nH#6KIO zFe>4mt&RnjCzzOwFU@_j;0a@JD;_FRC5MOJV4Oqp;IqQH3JZ_jEzIm-2s!&gDs=U zaCzFwmj2&rb3Y_yvuwI(lc~?*Qg?_k1uO4?xd@V}?3f&qvuf;v85eg3*G~0%5f3>y zi@vs%gaixyYOg1CPF@t|z*5b9Z&~7OOVcAmSQQ@5+2>5sQ+3&+D?K)yD2w;RIt)m4 zGl`_E8|91H=mch4!?2>0xh_vwc9TVB6v|$5O@$dpbJ^}f=2hdpM0YDQ!W9j%_8g6n zoaf|i86u`--wJis2r;JyJCY=a*5a1~I9P(%^*J({Fh4fiuIpTNLXphPl(6qT!3=?R zWnUQhZy!|h3-u;+(Tr$gZftCzcQA$znvG5(=M(r=5R*V)A|XAaufsY>QEnJpr+a$B zXeIpE!)Ra=WFma{9ugnOYmwNp-t$y5pV&1I@Gp#+K9DXg`RZGuSy_SiKU;2|Nb2?T z$KH)th@qi~)h#1hNz#9=pLvr(fk4aS8FB1L^pJBxfF?~M*BXNqqE^i>3i&++C4CR3Rg@`6_3snX5yjgqdCHKfz&O= z_7wqDx5ZJbBYaL|;h&+23Jt61BAsGpoeM4mGCVcv?B3e#B?nXu zNQ;g+PRsN92BOITsmDOTQ`lJ@k>7OdaydIsrV>id7VDm{WJn5ftLRMT;&Cw6HEEZ( zLbD$K^>=+!)7ZE;Gr+H2Klkt6x1WR3mEpsCbVCs9gW;x?m6hq~J^5>y8ApiFzT%x} znWR+eEO|wZ+>VtZWg7D022ne}TD)1aR(Cs0C<$y_#0bq4S5j)5lM+&JSEP=7)Tqm& z!CT&&f_!zd)9KwN6@Gj(Z%+@j?5-BmI5t{Cq9A9I9Mr`wC;I#y)aW;0%a$#l)n0 z@uFb}_&_Tn!kl=$&+i{4&dLe8vd1uAWaE+(YSFxm=e4Fgkkp+Pzfw&sz#EZi)+YXY7oXeRUNGLT3=}L0dBeRj@0&=`$`46N^3z@I zlaBkZe7Xik2ZslTx5sDAr9F)_GzcJJT`Qmr{`z&XI!1-6e9w@#yT|RPn!Gy9bS@_& zTTYQK%TJJ^PramUbAEgzy$oTd(7n3PP~uW&?8~t$8YAIq2@{iv&Ozm-!-weDDk$Np73wr=oI%QdJ=NRIG;IXgR-&k9=Pb3FSIh;7Ev+LfldKDCJKoSk$ z3pZ5*Gc(o6gdKR6q$qo@x}KOhNE+``R%gtmCwK_RS4Skyw05}Wg*%d*>#&cWEBXgNaVY3Ittug2A($yQ>Y*}t{iYwRT^-eJm zl$x()VtcU%9^1ZBfj_IcJ#sjOU(uznoWpnVwbv#njx6E+O57~YfvKvK4-5~}sA}r3r9I$EPzo zmu5rsl2Mk&aHc-hbt@{!toZdSGg(%Z?L(TpDsk42Ewz+I6OSqBoYoqh&J}DMaA%B5=v@+M)h)O2=^qO!qOz8Dp zyX(*AH&SPRfB*P6I}1zc#~wCS8Wxt3@$vEgf(FZu{ANIl-;Si`iUY77?-EOqI5l|H?P}fSE0`rN2Zb0^AAIC?SMAvApgsZ&vq@ z`uO5vVk)bv*|HOpFr}c=sQvjjVikiz%-J*AMSVdYSGM+E!*i|Yn2Bq1Ro_?8yjH1c6`N(vGhVfWgR z8ioL^85Ad0oeup;fGE9Y-D#tXKBCtv@pRcNx5JXRYcd`r)}W-PiUgeqQAOIw&KLe` z)}${Z0oM=uHv&?~0foyr|J;bzyy-g%m-C_i`}Yk~(5hd*=0Oh)d}XGC&ms?9vH2+C zm#M1gPEWEmJkdpuKca1BY{LPub`~^r_)?G=${t0~4R(PQhyNu0>XTPfwA%W%F0VDtkcsBRAl6%!$Z@KBSeK^S@)=n8P~V;`}JEN zP9yxA?a{2q%^&b+SR^Xy$cUV#TyL5>^S@^|cc|y(>FyDK@w8>5IGS*+R(W7=Y)V8- z>`1e<3HY5E)byW}hp?Z7%VH|8qoZSPzC9Ix+~|$$?d=U3gQm`u;Jy4ZbJWA^NLbDe z!m&?1hg&nVy;$OUu&}V!cwG*RH8eOK3rfqD4@#O1R&Zh_Rxm_nQh*aHtEv_#=QJ&3 zfUiwE{$0a)0ygKvWzcLe(ATd?*Z|EikE@;Xj&O=>vRD^bpJ53s*38d zwF7DLeVv1)8pP~A(3sLp^s=Z@rjE2*ls9i}ny!+7;yi0zwk8QeIgx5puX5rLA;1=I2zwQ$J5yJem*2mCuk%Li2YuP3q@=pKx^;uLTQ} zr%_LfQ9s_P;@va4PsmaNHItH{TkDY1Q3r9Xpknv4XOOkyw&@#;X!zxojE>F|LXLkI z5{{pjm&fmeI#Xsbg-81i`^5|MsWJ-?M8Vga{#i+R@9Za3)ozVupKVDJFqbmFw7CFV zeN8uiSff`xphF)OZ_u&3{eA2G2Y#uC8Sb7lHIa(txwd_~4w$W|Ow9c=g;&}Rl7yW9 zwzq%t1yxsbp<7H>SdZqYByifxx?Tqc27{1!`qEBwKHF4=Ub`^A6-EN>-rDw1rwww)x|T5^j396K+JWAg?%z_m}?y z`?6_ucei)#$ji%%?_xRp-UGx;1{<(z_TN~&{i_84O(i#X_mPTC74L;%&l(q!&5c8! zXX`xrzp|sS%2NgKJy%*Mke_^PNjs71(=JdJCl419g++B~X-;;+CxENsRBS zXNX=0+oiH1nYc|{_eYHWUg8`f9<;;P2T}kh?&# z77u?Uo2)yc6Wf3328h0c?x9?UWJ`1N@y^s(OH0e>Xqg{HdU`rJIk{$)oyq=u0|*36 z%*<}KBFUG};IIFy>+6&HzY7D5$MO>1z3vC`dw7^MxwBIg2n}95wY9Y+;&uBO6=nGU z6$bF1;M~5KcK_S_`HUtD^Z+x}j(5Kt9Zdn8T(?l3nhZi0ot=_G^8B&5gant}dViW& zG+Wj9#UorR3-Z59fPmnH`3wa`c1pGZ*gV4^Ga+jl0%Jg6{@!~ge}W~nx_d5a&mA_^Qw>+4^|uq$B@i&$unchG*T^%hgE?$2H%2 zivSD{lcH-hHGDTUl`JpP?@n60f8&H?}h2vYGv0oX8Jev=rRZ0KUeLi0rUpwT_7SN<|h0J zfhEJPn(leo%bmgf6$fU5!L~CVH*PP{+iZS%{tgVyQxxPpZ0vrs&|p@sCC#(?V`Oyc z=r@9uk=O$aJ`uip&mqV9Sq#2_TXsZ0UY_b9YVn60OG+j#HF27q`T#^`dS6o>a@AqJv?&gcBG@n`)#VdhWEP8W<|0z zG=X0TvUrL-UFe{jvCa22+6`yl-N|w-i#VISL=PQ(;_?wv-}+BHLi_?!C#k1%dNS>s z1A;6UjtK`bw(kYnH(9P0vk5XDx^t*zkXJ>PTcIr!(l&HCCcbn`d|op<7bPapY>zsQ zx41{mgI5IkbkWPciCv(#B|eh03-$Tdy-Du>^ojcOitA`Qy}gw!L!P~rY0LJ<)!lrf zxf34O?<0GE{QmKJ&q2l0T;*jANg|hFJX2!(v^YwC8$5JU5fl^z3JoZ? zGbB=!mXIWY&v9{Gb|zVXVrgXc=IX5e=5(r$_bO*YZhNBG z=zMRA{VVwg>2 zm6gX=d*aZDc;h1@)1v+)sdr>=-gNc#(UX&}df$wUjJ#3IQLc8##a+AnJj#n1s!Rie zFu7~cp;N$@tH>6zj4W3={V3^Df~F2sENd%$Y{)nzd2%XEro76%jXM2$jlx>*XPic< zQl2UhNeqzFa?hU1X5r5w5>VLGmf^FY= zAyMLH)9&aY{Pml!Zm*GTuz4)!*pl>b>#Y65-QRD}=+!-5lO-Q1Rq}jyTiYD9RCCFr zo;V_AQv4R+=R0{Ri8HY~Ks@ISd%b|&d4_+5-vv!NN&5WA3c8t(AH`Sxa97>fzAK9M z4K?-U9Q^jIih_{K!F;7N5=DVRt+Bbexv#GeNYo_YP9h>A7l&Q+M0_6I`iwg}J6i=oxQB#{@i&euMxp`rlAEyezd-aM&rD%C8zdqR4 zSG!PCJ!o-n&+}mM<4CkNK0dy@1sjkS!P3&mqPs{S5E@$Ak+44*ijslE zH^B=LMRq5mYvta(d%(7;s;U9l^onCj{lho)a?aC-yuA*=ES{G+b+4^}KakgXlBsO7 z8%@>88TFE@2XirY#WJ(2qoVtTTcwFvkki3UVLozd37cwM*xm8fJDtUn1BXH(yCd8s z1p>;V>NAt2ePkiFWPO@X`&wqQSc{68ND^(WkR&bLx=&*H7th0=d?27$vs8z5*4mW* zcKgtNyN|rqN3IpDI_kr(PM5K&H7T34KOwGeHq{$H{ie_B#S8!0>BxJ=oG%J~Da_Mr zA}`#(Un33{s;@7k$B>QdtNLuJXFZT-0X0o?Tbtv?U|JFbP-YOankxh7=O;`a4w}1Q zImBa`vnDEQIcV!G>G${de~6}KNG7;k^m`*?vzm|pqMpb|O&!j|=X2VB4ehIUyC|x% z`4eWXtoA%^E<5xYJw4$S73>gM$`=7xwElsC`i06saiySlR*bgAm&wUl#f=f|n<}85 z@wt}xR&udeE$UJCehlZ?JC~izVS;USu1J*RrQ_ng(Zw#&CtfHbYtmM@AslNzeO4 zfxmq|OGrssy1ltdNlpgl2fRFbOKWQ|KF44~gU(>8h~wU@0!p>*TJN_fnAL}+qqN(#zbo0%!hl8}~W0EA{U^7LAD=6ZUVFJGojPM4ZR$Hm3v z=H`Zk^m!qn$#d1#)L_x7GTSV-i-eQ2yr`iAciR}s*xKA=webrk;^WnCe?DTy!&CeF z_d8qLBf(Z9ef{&x%gKp}%Bm{M$&#$lP?TEaO7%KmirHFME>6x~C|^D64$x=;Y30=T zctv$}c3u-Qv=MnMH%Y7nXfb1$jKJwZuNW3VjDp?Jdo~L+Yrh5RE>N#u+N|1C#y2f& zPgfp=Z~|@Q=KTEcg-@}{H+Q+>bNc)z_G*Wn&7q7CdFlinSBDQDK1fSTJ2|n3cRWKy zC1f%T1Oe?AtdBFV*CD5oiwez`y|yi|gvRfgX4%^Lidxng5jS<7+(jfd`L@OmmxJ$C zLDrXgp0eX7vxh2UV{sy4Lecz#hm0_c0Mb;wc2CW^(dUotglVEMx?H+{=DjGt$);Vv z4jW?H<9jZO>r0O9M=dNA5TbdfncT0dcDp!i_QUi+!HWtDYg{^zm6N-nW z-YDg9yBueNvMxVApIQo1^=Y9o`C|Y`9OK9HLG3*~?w7||;o)+aZCxveCPV2lF);~% zE*VOf-~(~*V5tocce=m&@slSPfIeX}AO8e0(c>Cl1cgLgY%HBtoqFlfN{Djl+`s?>2S;U6`Tl?q(Fv@np&`i+ z`;P)nGax4U<4tf0MX4Gx!YYnxJ7P*uxMOp0ZH`9MD8BY~{i5wn`8w35XvrnwqCY?zs z-syh~tGRgK1SmXEjn>W4+ zUQ^{(@Z$}+kt(OVIBfh8q%IKGEr-&BHeC%tOe5g)sF^g9nUh0WY5b}S`7-4go$q$G zSa|91)cX-F{1(5j+hko^ZxPpVef6Nw(JJ9M0r^xt8be0mFfwj2l;pt}R#`O->lT|% z=R1-xzR>|P>dKf;AzTKVhkMj#_)^ihTrMHSFwvWf^q<1{2bIqRY2eUd#%-8%{aA#F zWqb+GRlYp>OTvkpwg?MMFSfzn{cOmMrrfF%-rOAKD;42E*tgV1D!uD+8jN_4Ex9vDv8F^~5ukfP;352>6R=U%UT@deTfEuOU=}KF$^9u^zju?LY z*aCF2m&Y4SuBYZ81-%FP9EY)aXh=>0merbDYfWK8& zyDo)~vewNXIGreRDCMcEC@WWJH{2gFw6;D7r;q|>Nt;}@shp{@&$hrwhPAR{)L6pu za{>cebO{I&qM?;z&#?m1va&8#qP1pgoQrE}YKn?%7MuM}g3Z@8Uc7wy5*vGr_kJPZ zk-*-J-gMWfb+HHN2Y5F?eV{5)P(Ug1a~&arxk|vYvke$=LfXvMNmh?8U|ai#KB#vl zwNeolZXe2!Qpi8so7)3hwLhgePQrb}$Yi?rWFY}7Xq(RIq)H=efArwQm9C_w??A|_ zUX|^qZz@@c9Ol`h-D&sh+hR{< zw)Z`8g&L8Rf)j#^<0y>}M(%iGV|T!Iffdi2VP*s(hL!+q_|+U70PWkI^10D`EqEJ| z=P(_eB8T%~>ZW&LAwzh_@$oUY`(=qV<@Qv0eqEghvK%8T>*&ysTA^+W;I%T->(t93 zG|^pnR4sMCfYtE37d~}%bw!FAU=l3@@bB*CMlKOs;&I~wOnHa!L)+!&s-<^4ovR`u zA})^BtN>Zn{)5<@wc4mJF(^aBYF|*&WSt~0Cnq)9^zzO$`=^hPkl^P_Ap#}F19!a3 zOpUWG@O)4+sO@Y%EtS?fiJ@e(If-Z=F?}|k2_u1W5t*33TkxjU(i}f@yK4Uus}y!j zV#do=wAzAcbJkljb>$}K!r`PGGHnPV_pIV1h`kkZpzbw*IZ?nsf+tynX>=WH_4 z(mxUt6U)mTot!Gl%h{lPYzzz{l9FY`#Uzj{`Pi_qyK~6}U5Q6ppFI2uq}pL&7!MwJ zr3co7ya=|gQBQ1|sl5gugZkUQ&w2xv5zK3VM^k_YP*PHk18a?Gc_#e$(O;<726|(d zGWPcN7F3n9Mdw?Z$=`W(AwUSNTRaiO_Y?hY*=I{6;#haa3%z{rKX?H0@l^)}R6K@} zjD^L1b2tkyZ|l>Q#Z}~&pD+IqkiWU!X=(AXu$+wMssXPnHtYc*cN!cN7Zri3m_z&b z=j&@XUMLYRZk6-VYJ7ZrMMVYpModf$!ua+rB^8xlK)@R+Dre&K@bK`^P#IIx%>tb! zc~^iH;g?(aTVwfcZEg8l^|z&r0PTW_8i0IN3w1wwc<`e4089XqIEXowp77f+GO-Hv z-AtL3VQU7>;WRO9tboXbgqh|8i>b1ePwafx`%OJ@YxrxbMJQVk*`SyOem%e`M0Ca%okl5q-LzkF3 zf^`9-6c{q^-u(tq(+`}w2bR3Fs0gfVEQ8*6*(~`n^-cAwKV6fzmeUoO6q0U-9TaPQ z&^@pX(D<$kz(asbs46Sxs84VjIA8J4Z(-i5}X}V?0*h>gsr*eNX_vKmo8L$P9nD1w?82eRcDmS`H z_g>A?VsC=!&Wl+lb-MCWxzC=IStTn{i05mK3COf(mb|EzASD5U&XxbocaVu@9#Rp`oFL@VZ~t6x3H$UB+3}UEE&nwFTlF zl@0#k4N4m(|B%E^87r%+iCm6!Kx<1%np$cr;|1!C+v6cAFh7;}R_BA++F_Z+M3Dig zZs?;E{ft4*0aX|en6W~>iVTeYHWa#fc`Q`B#cV1azvR-o!{(8OU zplbid-I#X0W&oqYHfd$Hrx%atXNDAXb8E|%J$_VjdB+*#-o!*g55c=9xK7dD`7`SF zdulXYB5qW2!Qu+3&fxBn+W%M1*QuQHh5xAJKdr`L1V98kQGhS^>fv?3Rd`&Ee+ZN2 zCj3dIch@X=k;wVDU)~Ro7uBEIwJz5{TW893c{ygO8n1-g=@k0qUVZV}wN=mK2C+73 z47Tpudaehf+qc7cE10zgcWVd+S%CetI1bMbmg=t$#lSq4m)8T3w@(8>0=r#L2?6dn zz5N)0lwJ~b&h|Cq!fc8zRbOiH(vJhC^iwvYatGDB2LYcwoE$*l6R1kcMR^|e_*>Pf z9P|uU>g{yW@2WF;HmwvXLC?durrfxBXMa$7mSyQ#ZVe7~d|W6o;veKc1>bL|Q`tT?YSbzXA3LtYWOE?8ka z(W*JZJ2Ubm7JBA5@%;JmP&)rjnM9#Bi^r*}RF*sjx>tkoj==xijsNa$w7*uCb-Oa! zR3>a7$wkBaL|9i6$A`!J*W*UFZ~{r^GICWcf@O|I&!#$}`)=GkUd?YXS<+Qq}MM3Q+ME03aP|v zkZMj(PbVfOtoALMd`PagJM?2t#&LG-J+PD47;WP*uXOa6HNb-TzP@`&q7nQ^3i#Ur zy~Xt9*qtIS9lZNSZrd9h90xy*ffrMUCbPJ!64k%uV2%ppu6kqKNAxr1#Bwa z03`&bFhS=A)J`AF&CO>HK7RZNQV_`5R-kOUzPTadcKS8Bx4K%U+wv4*V{flf<8**r z%xtFwdQD(p57bMxpr7b^x}D|PNnd}>sN3udN;Ocau|gopRK@?X_V4NkBSxR2D0?xZ z(4khs0ytOomwmktXZtI=#=hkwvdO}GdspDtWH~sL^+fiNDa+|(D=a6+viO44wi&S* z6yG^GAK&gQ?@*THI|n%Z>GtfRSsNtVuc6vgoul#G8wbpq(L9~BPvH5~+i9iLi)U$P zuQmcc1|Sf{{67X>u3fs>971*S&F>0D!XISCLPhsVe2 zz=;wQX8}AhMFZ^w2`KN)<;fPP>F?M>aq;dVgDzj+rUlUAtE{LPpPKUV_U?&g3GlU? zt=ZVyvobI!1r@-rU%ygQQzPFf)hUvLuJu!7WG2udWHJ<2R{oilm8CjI>GrF0cdm~V z^J*8%bIziFWHjQCX%~CUbJVtoiw}`zGW54YT(RB7pdahU1^A6=SELf_n#p-ix&0c3 z41i_{2^yb1eY&Ibj6vuJRbV2g{U)Gwrb^AuCXArKL^3R(VzmMJ6C`dmAe;d+z<>z{ z$xlvN`kg@i?d8@Q)KgO6j`IfHb*-Ptv(<+Hkb%UEk7@)8sHL`$xffW;|B1$*9g#r)tR{-u#9O=-0tt&UBoo@6bWN~evCn)N zAl%n-Hnod2vm-t(AmBjsvgLBeuU3<_F+oZ|0C83;H)bd${i}X2{i3a32}%#G`0edS zkFqa*kA{AT3Am(Vts)8t9CG^D2Jdo4&3IhyvV_z6`o?Q)pMPO;+`iGXgL90+(64Wg z#%KwCQ$RL5dPP<3*92Klib-gzsj|P$^?%F)bTG&dJ3zef;)Q?!*zROxWNenx(gY~T z$X(st*MLvX$;t5*=I7S~Suuo&4~(N)0JM4GRwtVUD6wMGf_JaS>)c&IJ0V7#9G%Dc zkd%}Zs8upq<333J`2e~=ORJuhriKGl5Fl}g*ZO?F8>epnQ64YF6zPZjvUzN+z5;4x zEI>;45|d?;XjSICx4uY%!cvQA$lo_p%Bb^j!uEVod<{J)G6aveYd&|nk{V#~(kl(f{Mn^GyIu{NAGEs@_~G?qi!Em&A;wi#Po! zKw4k}^PMSYc+0S&;}IiH9o@OinBgCr-&Gv=3}vLisqp;r3Gu2k79aq=abN~?vSAKGu$|7ro=F=cLc z(MK1~W(?W}M9v7;WEA{-+Yeif78G>Qpw*|anhOspFdJ2&vnuQA3quf!hp|XNh^#-BNPs} z0s{DTNq#X;>dR8&F$tOBJ~erAZg8fl&Yyi$!PgE09;JQsU@Hyv_ZRfi)e%F*CcHli z(7FNX%nK~6!MQiR_JEWCnijBgQg(X*>EwdM2W|{1zB{Z?>rx8j3IYPnQM78qiElH< zx(ss8@{ha67MQ52*VE=EA|eC)?2a`HyUQ}=X~K>6pj!v!anPB%99X512@8+CJvANO zD9B@@qj@taRr#QV0;ODlzyD?wOhG6-EG(8GR5pu-ii!jLB>(`?w|414J7#PQ%4&fd z(#GYquWf9c`R?^OfXkrn>Ckmwm-2Ai+N`xzqbK;Y!_I&Gx_q|Jsr(&FV`qEwjr%m8 zJ9EsXXYp`(JxsNEaG~bQh5fOeu#BQS=rJlN%B#z0rFY0Rugp^btsH#S0tN?7|d{Ri?F*w{p%MF#SV z!0|=~ly!^ez258<7xtgmpH%?Lf=r)^59*G^+q<>g9QMY`56RJ4DhR_QNOP^~-=FO& zme8MmI6e34qt?5??oI#UcNWkmY(@amGU$AQ%?VpA5Fo|ktEb^{Jq4v;Mof&#&f+>? zI_v3|JtRpnz>}rPX2DoMoG(88v3U%TM<#1PK)}nVANi@WzLuA1)oP)SXaa~y z;!iD*r|_5#;~H32hKrWxP$@b&#c?LReSNdLjN}}27(!z?BZ1r`+tYipHy4|kx~014PP69`jS zSO(WhTh7An^nQ29hTG`#+_NHq+M!&PdtlEZcnGgq&sr&ktorSCGN2+|{o)I6EW$iHWtlPPd zqvxn@jRP2o6k`K)BY;q?kp!m)Por*gJu?2JjCNTAThe?Yd`X+t;F&2cgzuU2D7Qfq zktF89V`72+;gOX9zvHUcYrmMj3p#u7t*cB`JEBV}7q)P(mLpeBYvq^yB=^BRgg&|= zdvon(05@Mq$~{r(ovtI7N(9JK7u3bnk+7T`d@QV}s3-@}+%?aut-VWF9_I@_0G1vd z9NaD4!}~e_ri`mQJ3D{-0fj34{f|I~d82Doo#NR-HOQ%@_Xo$@8^r*;41!-w*GqW!HH_)T5i|>GTrTY(do$vY< z9kp^;M~_e^5;vNE@pB|wq+E}>t;g}LY_oMtfYR_-q-m#=)2mbVLf<6aoBEfX`7kZ9 zO}f!Y!>HDWqz=YvDiSwrt`9K?-;9M~W52`+_aD+^kUNmD9QUGmPvUcddGWmQP4G)P zsQ_n;k9kV*MtQ{QH`fm0O{pzqmxGTFsCwg>yrtj!*I?VRQKyA@!h*Uyj4(~`KRiB} zeV$Fc1UK^-4k&%&e@Bv}(3+6}L zh2Ok+(|b%XkIeQiq>;i0bLeeCF7YuZA2A@N=n4yJV0Gb{hINys-NOtxtAtbZ1Ws8LDy=DC0PQTytoagU# z&gmTI>+AmB_jP^7dweci*GoF%c2Nb$w7?}X#bx5;{O`KBCKFI}KJW>NV~7}FB% zd00^31cdUPdHZV-Cmz%0c=4_E;ElB_xKb3&sHSkelVt`Bb<2{WHnW4f;E~MV30> zy6$BX60|F;W})YnU?G8NGc~{Bl$#Qy^4W-`2k^fQ1t=L{z*#3`u z2Ob7>zAt9cA>JVthe|Fm<3Dn$pISSb{>l$$7ze7{+NgE@^FK<|dU{{~SJ3zOid~QT z{I9Tc`&&oulmCzA@1K{PSwHsgSw;YIRxbYY`~SWH{`qO<(a1lb&ws!3pr7zR7x3Tj zlux%vM^|eWh`GKFiVnqJGYeJ9c9LSFZU246KYzNO6f<48VO!5b*P2gZEOzn^UFr?j zXj{Ag^P-YNAEWc^H@B!70vM-AeUx>EKAMJxzmF9ZFGe*omKf$}xbx13{QrFBzd!nO zw_DeUo%*u7_|ks!QRcNGw|a{!Ca$!$=RQ0!%>AM!MWf3_&C18f$rVMFLMVOWxfI9=1CrK@<~FMpJD?-WLxg9 zUjE_#dzZ165{Im@n55RvyPJ5g9Ud_#`8{!UZ02xXrj-$YiQl2Y#q~akR0+3-eyg`n z=N<8r@p12+T?u)sx47C@AJSvZcYG@)_|THCV6RzwxVJJTPt92sbscq61L9<<>rM%S zeO+(YPG71Sr1w~jAGj_#5nRJdsNUE3^67%-U}LEcb%47%PjpHvL*JjXG@~bJW|>M# zI~CZ-`Yiu_pFXvlc2tYa`}5gYN2W4=RhoUO{<1b8J}L9AgVw^d>O$hfl}}pni7{3( z+HOynPx&j{8I0_>`XaE~^LIdWtzQa(=8fjDnaTmS^rn*PVqM=_?V_edUiU}k9wRS2 z+EjGxYF)ylTQ@ie>jh4|+8S|P(NK?T@jZ0Dw8Ss0h;+j@V7M=Hg^#mMxZriAfMyZ< zg}DR&dKRBd_6Vi`EcD9#WwyLMeJGGZtAqT!bui6cV$o&H0%f!Y`c?B<04<0;F!dmdV(Yf9%lVt6uej9RBK+?1%PS1)2Zr z;&!a0?6+G8ckj7pdVj1uips^HSIX9x$~gL-!u=5gH&!pvqc8hv2S!adhzC}#%*xz; z_na$(v!8dl#xo`6|?)yj1okAzYV|7a(!?9Et9;F{y8&QT=C)SSrIR}!2ubf zn7LYsu~Vdr#49xqBPUK7=Jsx|+l&7;RjqHa+j2R-@Z)cj@%O48SXG>njy_)JE;>H? z=JdnotzT-0Js7k$eGX^YF0Qpohx$A(4>$3@(7qMrCI=(gz#o$7M{;+hWbL1xkcb== zUXhfNRH^oB^QfYq_kBij@t)U@<%4(a6edp{q&Kl+W;VE2B4imzV|nx-os^`L3iW7h zmCkp1I?ns&M+dupTq9?^x3mzNG}CtJUpMK~2)}ESajb7tdLhw^DH?8`&bHfA1s0n| z{3*;G7Jv6Ub+p)Yk%q;Y$#Q}A1?Ez+PQ|8R38O87%-I14cKO(^IS;3HESddV+^fdj6t9FEoiU;@xdUp0vA04+B`0dE#Iihx!dbvVdQqs%xHpl(0)q&19 z`yxdFd8YY)2SNQqbZxbCAB(qsZ&}T*-<$Fyl~Qjjf19IbeqFHP$pWiNwy7$KKxcAG zu(I^^#bXuSZE}|kf`X>sryo21ko+){r(EtnIw!?Lt0ch#eHk^}9ADJiJ<0VrLZ5^r z#5<{&9+2=+)|{JsFp|+Tv8(F|vF$=_>hzqqkGxz{#9P^lF`ovqK*D-qs~63>Mq(dF z5{!PECy4dmA>QHo<708LOyNHL5KTx6WYzmQ2#im!+!J=3Iy!plKXO9^?`_tm523;i=t~rxPNS{fZ=8>#nD_W}eUal|*EwDtEJ8 z!cWvi?}uEHoMQfsJ6yh5`setn+WX2JgT1*of2R){>Ev|XHIKX&teqARZ+N?y^3REpX39!RO2Qc@LF8QyYR=r$G&v>3 zAXkI)I=V+x9@*Km-c<5&O<)IM5P_D^TRR_S12hp}qJYUPRX_kcHfDD2TrS!#(t^Q& zrq)(3@?XQljDhKYonN6YOSw+O z$x?}5=mDl#0tup~lge?buPWJT8U*q+Ok&fS*%jjtd+zOFE{g``-+zCWG3*X5P8RntSJ479w}@J9Tm3@zp|r+qcc&sjMXl_HRlC+Y2`a|3T$^i_A}FYALjr4>X(1RN3zIg$wqJ5 zK6-w?d!yXzJsF2ZZIXPGswKnuv9J~~5xrAKbWy9){9OuqJ3&7+Gc{#;*=z0Bk>pHkYL!NI|6YHGeHPJMiy z+86J*fQqiHtW0B|_Ul(n<9ef-Y-?+KPo!k}GZd?UZ87VuW777FoV+}vsB`?=w=12S zhc_H(syN7Sog8N zVC=DR-`^7NXEEsPq}W(M4hJbIgMi|*QnshvVBzMT9vL~o#B?^t;ntZW0#?2DHa0Y! zX2(2Hk3*IuEG$eRfg22MJo$so{m?4{)^GgrgNvKH)TD`d2@;Ei$Bzlj&ByUiN?%WM zp0j6ZswSnSq>7yvzhda1wDh*Mgp>2>g=72sg?1=$Qjw&NZI4O3cTa=!MP{aznOO=j zCa5pZs2z1B3k?lrr926m9ts&~p*ku+JU}u7xf^LoSZrb8D@o?dva;BkC64`Q+bDfm zPo6ZjvTCrDw}AE~AH({xvQ`@1cJ>`fv$0ijf&}grpE-86bc);#eF&eN`II~xu^gaP zYt4VeiWV+CT~S8H7z^$Zr#P08sHl#Fgh5lR6ks5@;vr3FZ);FGv1;aDQh&dzQ!$ybuUV|f)8o~NJ)M3aR1ZfvZq zPzhnPK`2Vk1n?NW2Qxl2hV|t;<5zt7@&&BX;CT-4tVKmymoJ-Qt|V?5Q4CZEFx!Dp zc_9(=)B1xi2zx01jvvQfyCy4Z01dFRQ4iC_a7=S`)NW!Ox%nbWYwAR|iM~A)-P5Omj<1d3ia6mr05d1aSjp zVfs+j0DhJrQc3=f$P*e8vS;sJ_`<|FykLWe8-VpH*faiL9#krXEFejsWn#bC4cd@_xZf`kMw9kL}tM8@4|4*0=#Jd11B z_BUR7Fgi7*q^QWn!7(s2)Y{f|U0vO{0qBPUWE&F`*Hl$`czG)r7LB+Ys;iTL?I$EG zqE&~0;rsXR$2=h?;N|6g>hCY)V(`H-HYSFNlT+dPb*g~s_V#vQvGtKcS1~E1)VZp% z613jz{Jd3PiH^4R(TI+=wihp7b~H8~V`92#Xvli{G{e!O3JI#zr?ua{dGml=?(*d? z-@dirx3sr!)o@eo_uIKv_4b;Xsw!pMk^TGkclgJ^<%f-slM{5|7>J_BcA+JtKZ_4P z4-PKxw+6@(92yGTaRO>*bco~K_>+H-7_M}?s^_#a2K_&yaCU!$+d^~TEUWM_pztmrQ+9!1dO-))R+|CN2Km|5D zo$-+ou!rC`^svNOSv~0oUP6D0@v9h!3dQMbad*qxx2wW!xDj%to?|$LC6I}9uW_!ryKc}^e)c8gux9^#IfK0ye2MxS0!X5@iD75}c1;Ads~%xOVOP!@VT%26ob) z!ki0EDwG@nKMuC^v1LX6j12XC_Dsxv8av>VHm|EnMum%l^|FF*gbpf67NkFCA^;P9}r zf`aSpue~n6AiQ3J>HsA2`uaM4GL2_~v4JQCCLrV{)G17!l2TICT$(7ee2$kummo=W zoSmJ;6Bz_lIyR)IrzDh)DDnIDp^7kwFG}LkR_^tl_igI!(DJiPz>VWzB z-kDR2i*G^pgRbMW_J;?gv)$OH*j0qN#1^Na<$?hj7LWJu`JuRUG@a^R_mdVrF1sT;q4?*Y4$qL$im z74l3|BO@-z?JE~D+2(cr!!wpUBa)XhotBSc2YzFEtym#d<-`)F% zkHWcv2MyjBpQtk+7Lid#Qqs`an8h=T@Jzwy91f_+Cs^LI27iff8STibfL>I$sL!$E zSfK1g#Le9#7cO=AD|;h+iS9upL{EiHhfY*TNNBg@Iy_PmCdf9os-EJG7xjg-{hvGnW#=l2KRuI=KDkM1zJ+K^KZ@cHAMQbTAH5{MQx8v(0Hz*K@=&?DFJCSa!9tNUY~b&Gu#N1F zg+wey=|UhW#K$Kj0sj_>=Wbp70LA5Zz3t7}aEWN3!0(zrQ+Xgd3u(+Y5RU7wnSW z+t&7&d~8uS4t;BVy%Ivpty{O)f(sBEE!M#HqO?XZ(bUpH!$eSj%l!837m<-qH$o~C zvD)wj)iH<_Bm@FNLj3T7`^vNTM8JjNK}1B*8Q;q;l}y!+t0i~dj7Q``q*+~AVP;{W zkkA;jru6-;w@`reARR|cM5I5f?MV*baB?!Su<8UW!L90N&=ZSGyoI+7_od^xlL$kA zD1q1~aUQ~sJxhV~)K!~5*Y@v}FhRM9oq>B2#;wJoTNuZTx~m-+|h@?*Jx!mNAUni{#2_S4T63hPPXJ#kl! zjE)Q6dW@vL6eKWFz&J7WMK`CQptHF-$BNJn@%U3%(eh{A;A1Q-kK6|P`*BkbNSqK9 zJi>$qy|Uifb8SKEk(CAl59;geQYlGEv}J@-_ddOfjE!YpQc+O}^!6Tp#4tw=6Gmg> zwy;=Pnft^E@Ip7+r3FpBo&jpaUW z?GM&v5BF}1J<(qa2V*E?`m0x(koU>S?UQ)huBDyDn~3zkXBU%zK$u-yZTLB)AT2E| zcC0Tx(bIH@L1v9OQ?#Y~K6?`?ZVaDQ*#W?On+{`T!#6#T-% zMF&n1x}MkQ+IK#xzJAH4mvrP-Y5tRg9v9%ivp-*)ltehLg2y03M7(dG!E~-cz6NGV zjJ0y?-01<2=h~VxIuJx=s87%kll43&=%^r^?D`tYVaod|DGB0}%K*##{Yj~AUQKe~6OOAV9t{D>!ez?Lo#oN5Es~ z`hd$-^;=U=P$1W!*Fx`@o|fi1{~Pz{De7UE`lO`xB6OhQef5eKMi`#(Gc&6so{Rqd z)`Ck1aLT}8F|iU92WjWx-o4}F;v!{=N841)fAEh_ufLQO5ozNNf~($Go_czH&~HOA zrt-4((86(qQg(I$x3%q7=v3&FA)Bh4gZN1-A+%p;q<}9md!nQxV-Nve_yOhyz5A

xp6zz9;r|>Lt;)r64(F-_=!DA4UT)Iq9#)!odNZ^#y!>7#JEF zrXN}M{Y;fUMvnA_f|>B3vow+7D_&xziu>g<78abe6?dqM?&U;ZfD0JgxY}|pZb|7p zEz8u7{Z-YHkxZJTSS0&ByaJc|Cr)yky)LPgjEsEuPSGBUE~A0UU7gTvgy0+^;S zMsoqrAYC}MizJO^c4}; z%hV8p{l1L1#P);4RS4&+czIRALW&|OJvO$vr)M2-18P=I7Jr}%OJD0${zR;jcrRj& zNZn*a4fuldhAoyiR_aqafefr6={Oo%TEM4(Wkv6=3qs5T;4mpE>Frx0Ei42wEUfM0 zT44ABGKr-H(jCG^*DjsCdJ*^+JZA{*0Jh<{hzF9)C;+#Xx~)(_m1TJ1&=>4$Y+EQ4 z{lNy@xntyb=76;nwAdM$nT~dLxY8=sk_Fsv!`rv9$cOs-W8&k1wP|p+p<+UF23L?C z3RxIk@q3YwaT&-UoVkQMifg0A#m}BSD=NAH&je8GL;KKCX=rI3_(=7_oP&*xjf-m% zp#*ROk`Q~KELL)dmX{0@DDuYc9oe^tIJOCeaw7T9HKEvdC`kXr1qM>lPQb zA3Id?-+Gk`bN%gz_B0o@9}EmL-lZ{O0admjcLP++AW) zQogRL=;*jTuzHYsT>gb?MPa5xjx5^ z>7(W_Hue_XC`JhYRl{vVL(GUpeaQ*kIi8=Cl$TrrkRhn@0_X%RAq>IbF?n&gTU7v# zsHwG%3q9D$e|8U0w9}_feQju1UR*?zUG(;?iGqRx`g_0%usL<$+LJ|K%Pq;Q*rP{O zUvk#-xtG`8Kv`EzFv9Nu%!DpJYri-y6{l=q>;iuCo?RDpbxW{ZbR^tK^>?5r#&@vk zp_A+BIU3aTNEf&ZcGIJW4MiWVZ%k{MiqhicqkO1C&8E9q&~w>Ta1sOLTpEa zca*HU^QlYryLX2vDV-sM1!V-2UwZl^BybcGkOPqr6JP!ZaRWl712>;bpCOSn7qG?u z$m9Tj$t5VkFo}UmR!i%;aH=F`W=2Ov)z#Z0)JZc3B67$~Mz}b!nUDM1)9+YOp!@~- z^#1l0o{Bh-tWQt5$uPQyqb1v}D!y@-x|;mivDjEY+c*_{-G+x;iNBGo@;HRG3r7y*a7t;PIVEB=UDSRdI(TPNN zH%0@v-)>sl+7K*IHwD-`;Ef(+{s+Vb9)(pVeP@JbTbA*D=dL>PN89vf$puJhk90l3 z#u7r~Hglcefz^TFFKDzeD@L!_nJ-ZG=;#<;xI>TTd{l2&*LloHgJAWky^b`cVegK( znjPT+Yku6z8*$U0@v2V(czCztQ@4KtUn^9fI*fPp9&~09r2L`xZ?VHuTPR54e&|ay z6n}11V4KgqzHRhdq1v`E#i!2r&nzu5{FlG{L@yb=b4A@f#+b8bz1NYF?8>q2H+S}U z`JaQR`b2!h9;x`7^ADF?94_&*QSv1Jx75UKc3T%1KfdD=b$n^#xpDgM?z>EgF8$9$ zNnAF;t?06mN(aevzo=|(XKSEyVNp}w{w4qUDv`$J{uCEvsh{X?n0a;azu(87Vux^b zKY3=t>1{CB_W$T&^ogulbaw8LCTKreH*w;eu?Q{61^4>Ge-ZFAqz|Bqv9c+#%9 ze0>r}x#Key${O>Th9iP+*VjJ{%$ELX+y3Jc*;G4yL=u~p*3-v{&^kj!W$>HU{P8~; zqEBq6PMvR7`_IR9rGL!5rtosme(*v0p3g4S@nJo78(dk>&st~^{5)9JEPCmGWl`7? zObRufcHm9sMaALPsa+W0Q-eMKr}C?*=H7?T(mz5t#ROf1gsx*ynhhTH4KhZ^x>D z{?C6e#Y^*dCvwNuhvXM{jnh{;=UsLauh(23-R)Z+nR-cb&tVpp%9~taP1>tGMQ;i% zO|tKKHtz8*-IZAsUMw%wtP!O%6V$vvW|*5~PZ)0g1MUjc1VCiYe%f{M;ziINGmDG# zuHgV{0Gb0?Wz;Fkd;k~a_3Lf2?FJ`GVqb03B=9^O8P$BMiafY&06{aTrKu^l=mGw7 z1-5oEomECtlcfFa<`YSRe;z)11p0_jKWl{IOgopH~f<+bU08$$nOV{Zmd-x~y7u0x&NG_=xHDdn^R zNfp|*;`Gsw( zZLY7v>wr-YqBUx8IDqAN84}{@26_6nGPF5C-hF+x8o9-z-3<^3z$xSlAa4X3X&^LV zWZmxib903q#tuDd`jfv8mh`zfTWjm>?>SCs<7})Q2{u2faqmHQbaiy3=@y^oblaBlm8qU`GDCaaL|&LbSL9i0gORK0z8ZH z#wrCbjyd@*WyAOKl28`%Gf0-(_ssbW5i|iyZ#$A;e}`F>Z|3rHogI#GdjB52uAhrt5?F zjNOk9w!`7_cPcb$_SJzH@xwmD|9WxeDv{r-m4?x8)$S-joYxz`9+`esGP3w(h14{&lIh+ixG;_c#@Z`3v3ZqhjH{1J8NZ%e^_|LuyZziV&Q;DkyE7G1-(RM zNYBTQ+ge*m=Lk;QoPj^lwXiZ@hihYlS=R|%gD zLC04Jy@7Dq4SfZh5Dnt1BybiqCo1C7tA ztt?1TF?7G}bZ!h?CGKOfYxBLAR&^mDK##Do@@oGi<@fQgG_b2Yf4*&1hcys}=g^_S(b3Cn!EmoXUhA=j z<((W|F_0NULwoT0+^cs9&RC~B(l5@I%dU}jdwJ^|G(T_7SquG&sA;raGwP-+xLEs3TGr}O zgkPwc+eD~cQZZww{L77>@4s9iVk#M>>(QoqmhM|38JE_I^-LUxn$Y3*kBsYW&$ z$}JMRf~M6Q4oeU*sdYz{cAWTLxw3h`YFYQJP1~BDW9=jzFJbwkd4i=ckHuT|v7mjp z#3N=YvNKMwYdEK2@!mcOpX;=a!`JUkRUf#b&wpEFrgZ-4+^q4b<4c7NM$sb%1QK~Y zXBeZuiHU`QzRUd67r++wUV@EwMl9|Z??s{*ez~dv@-6fBOhQk|Bo)p6=**uRS$<=S zd%nu54-TLMK(E@_*0zKOXhP2tF;(W-(Hn3C!ixm)4uu4URk15YBI@Sn=9cf$#kN2Q zg78}yW+4DE5!T>GLOY5&AfHjPMJl9(m)3Pv|uC~5@3bekq z-w+Nxofsa@cV0BhkjVnz{w3%*QeTHaE}9C9HCzck$kWO1@DRZuhqR)3_sCgh=Fgp- ztME{6V%S}~UY$%2-V2{25rvDnGsFOJrU85=V-%~sp$4oPpVHLSbb`f8!qm)6v&2=% z;{wM2=<|eYEKB$O_9rjIhX5VGf2WI;g#|d$BgKiSsZBT&u?^MJG|H3&TiV*7dte3n zbNaLxYIa01-Bu2muv%JRx1yw`#()H>KM$5fjn=xW5z9bN&CSjxr=}{uJmX=$00MOt z86SDWPn%KC>TI$&k-jXZF0_!I>xDQ0JiW^g1bM#mi$zmK_K#ihHUC-~_0=PrWe zE)RVJgU|28M{yUx3zxNhR*FOc3G@+^IVeFWRC2;5Mn({3va+)~3mwd%m+y-$*)wr9 zvD9tY*`Hoh1>H4%bbIX~=|cJClCG50g6Qk7!!J~Hd`*|Zh5jo%A9zSWR1q#-{J8i5 zrsJi7U3i3%0qObzyZ%AcpTSmR*A_GFMICVusy>*eVYTfobRf~1!{91wJ3B3CHTvAQ z+z`l@R$=OG z>K63|eIVFAlnicI4tO|Rs1-0<3Jovnib|!1=H*|qJf>zRa3Hrfw!-v`aUzlzkn5ZrohRaq(E zb*u}uYh3OVxm<(dt&dtYNv1-2LmRmto}bmb&8)RLvwYQ~$#-z%L5sWRi*K}b=O`sS zt2{Z3MS2>3_$k%1R_7wUPs_yNQsQ41Bw(@nrb+^G~ z-)xWJkGq3YiRYfUx{d8Cp-rc0qA;{oy{DmC_lKI{!Yw8tuSk`J#~w!tR2*XmE=?Us z{?zuA9D$CBCY^}sS`;}Y1vUMl#ir>RLtd)DFDC~-(;^nW(&iEJQcz+F4yOFYJhgJP zyLii$H({JbU}uTxc0kPg_52p0Xz|0(1C$3T4a%pr=R{VjqdvbE(>iKc`N_mX=i0e_ z;)NVWd%BfTGy?R+9mo8i7@%F9t4mT1+d?4;FbFl3l|#@1SXo*EyZlk?vWy`R;CLZb z0ZWtt8N}LxDG&!dva7T#(}us6n8_X>2AI|D zT>j%n)VZe*Z{!&IqD6c8QUjG_WTf_u8=cU0-JR}r!$dl8w*YyWJ;9wq3oPGK7!XY=;_n_FH)I=gs}LWz!7dtN}=2bpa}2^UaB6w=57q#MgajxiQKmW zoDTGtzu@%+qW%g$4q8cg9TNZ%+$`V{?KJ=_LwF>Y4H}0xZ{Vm3Xe~`i*+-}NA~-lM zDao502!J8H&KQE>?OhEYR;LQxKCc|$iQsi@(w*ER{zS5}e%PWs#l`glOC8AoY*+S9 z8wk_q=CEUthQf5%@y0F<@}G-!n| zUIFgI$ny?neMXy@u7U?~lY(Y{``ynDMX>UM$HJA9l9BPBIkN)phcK4}83mH7ZE)Y^ zo^9Xmvwv~!ZJN;yE?V4Lm(a-Q=;_Hxx1XOpQ27&)kZj>6iDIC7g&qlEbUV=kB%Gjl z2UHHYDiRgK1Ly^lKJJ?pHm`ZKmy1iyC&-@w;@dt{{UL@dTFgp&dw4 z@>&>bX>Yf(urM6%BqAk6CviK8cx>7SMq>1dnZT}~*6S#B-$KaB((M42$n3dR6y@OP z2-);3fX@@oDlg#Le+ICJAnN$u_ZfvF zOuEvzeU8j2!E4YsYihz7d`E+Z9k$Kseacx5ekdea*aUo>~W{$IHxD9~(!q zTyz}Oj7WHv`gF)U%U1aKqezMRH+d|c&znDnlJPImalU)ZE%irOQir|$5~0rW`EI%s z6qBXzqygGBTN#L`5H*SQ$q=Trv39P2RlA2O^%GRgj+NlU{oZTVT zNc29swpM{T&;;@#0Nr=)_*Wh^$EM ziGgN=^%pe-A)z;f-WFV>r2hD!todsTDXU?%V59+rXluK+rl@sz+~NsG7$1hU9dkN5 z{hCJYmrmUFBq#I6eATf`C&=#{@7{ea#8|xV=YO>T+kECZtdR~MCA7*YK}h!P1JeKn z#xm`F7fcs|dnkF0TNEWfV)51rblq^H1z`235I=#k z^SFQ+0=ua+;3kI}tsChF2i@QhfucupU;?1x!t(^|gQ|Vl)v#aw(_+ZzyJ_vr>`NR^ zE*yg(5pTi3@EUn1x&0~$Q4H+HLV|+&VTwyk1I|tpMhG2BN=kx9>pMDLRvNe|{HDsF z?uBL_FTxzuFEurs+8@rKP|@nZQp5CuurNpPUJw$4I7GgLyB5&(?w-ejnUNOu=c7dT z22?NCRf%<-30HvUlXFe)>ecyQl_XFDoZq-J5?ji${rr(%@ZQiBVvO?%cuIYIFn#J) zyZ=s0V=^)_umj*W5#j$)*apeQL-m8WGGHx;l~r#{OAwcku=krR1GGcO9d$8tt*gxKo)?zj+8JOoJh5^e3w#dAm1manl}{{i6u$nftQqBT&_B-Z;%W#hPo#{AK4Rp zV}N|<=9Kx)SaOLI;&EAauX-(XO0MyQ?OVvpcQ__?kvXj9Of~0JdVsR5&x0eAN11Y# z)|z=d$$v#m3>H{1$rU)xYoAUCXR|1*YZ-HEtpcO)X6g3}caAR>R+cgnu^UpQeb)C z!1cq&k9i)cOBk$J2$H6~Xz*~#VmB!71T6ckf zX@PCvnezB|FZl$yZ{}v63B4m<C7;W_MTgZM4)TSl z7)i10rWksssQ50Sm(GVp?!*_4WASg|&tHCG)4{>Q#o49WF5T~z>d|CkAIX;Ywb|K| zNRwo8|Fu;LiMb`RP#Skhff8AgdHKXcb{00F$&{Alhqa7SJTK*`(-6mhenrNbDEQAM zZXIOgnDL4!{(3B!gro1wW_K1voR|R9+2Kk1Tf0epr86dnb4@KRe;?iZ)M;Z4rwBu1 z&axEv#Snb`lBP5)1xPCpPL6{;wX%Xgh4ynB28~-lOS?F1ATRF+YtZl^wqVm+w;qv0 z&JDA{a3sXOTL9|d>78KlxNw08!s6{2e`uH?+k`6X)-+jr^KVQaY*7}6;yfuak!;TY zj^I%;m$x6ooqrLrmda+Qh;`J z6B`*CD||tD25l^;FDOQd_Uxic4GMJnLv;|tdN(EHi_JFG(4>6+oOpmi1P4KuQ~KmW zqiDy#LgiQ4(9owh57Gh69EM12i-^H#VpbtSJ>pA1fHIVj5G_K&GVD78>}PvGkeD0h zZY-NAp&|hW03IA>Z>?6f?lt(qeQhJ<&cT~YPiq~S9M5eHO9ynHF*z6oeR6&382D_LlEe|BHVbYmwcny4McfZ=HOysgyKqN47<`svP2 zA>g+F;}B*iGEL6Nc)B^2^;>rwG0w<8w{)l$BPZZNv$ULYDY7Mmv%@o?=?t+FH`VL#I0w6yeiSMrPbk^j5h)Dpe1+Ye=vK7mMNjG)|Y8eR4ocllT#ndi9s(16k zz8JQ(w=ah&#)g6NC%wNDXWn5yF*2fjhe2w)iGhrU^UX1|X#e2Rp8m-o*(a?`>@r^+ zNDN}h$j+v4`F#`~v(F8896{8eRIEi=ACvPcLBa z8Z?brHQ&HR9A{(Op4hR=<5RdRCV&3;_H7(7rsH5m@vDO$!r6}25&J^dPhxD^A&v(Xc3HdZE9}*T;giMj4H63 z1v(I^1XV((DOdhtC7O&s^^3GPsHkP!jQ?dUfO%-+?r&{4THHg!QVmd2x!x8vsPpkR zG8@_qLeRdWK(y-2sjRJiie|X0i#9@R#rzFs#s&u;ToPy}$;``JKr4o6xO^9wP}At@ z>H;+YKV(9lAEV12h1#}=n2(?T{^|j1S2GwJban4T6!Z7+yw5q~S$_@_*h#L@)=usZ z*cQXcNJ&8n0|3FLW)WH+WqJ7sXnpmm&;_F1MOXtA`e0<9nhepUa*>+g5v5yMtKa(U zwBed3k(u41;mC27q-RI%+?V^NcZN^KYT6m+9GoH|_AB{g7wPeuIfaAU+V0}zMsBwu$xc1~eZJb|l5^*tIxHHeP+l?D{7h26 zQ2kkR>5sCDo?y0HhDdDa<39FM`Krs$ql-3b_D;9_nSAG|!m_G#%~YIDKjd-GlSF%8 zGtM9PYsy=lVon8Kr)7w2)TBKuVQDBka(+tGc5vpd-SJ(IDCt8}OO^siGW>>6cKGRV zM|Q=3YvO8sH8>sJT=Tr~UEh?m$u!$lqh0B|gKL+Tc1P@SihC!lqm$3Aa{QR4rGfOc z`t$vs$Bwf+CO5P2G<^5KugUfmIWOm=y_8K}bG5Gb{mx0JP3L<{beA1S{V&_`FLf~+ zSsym{FQN!(lDqcK{;sIKby1z6N>%EJ@GN7PAf2n!k%ImMwpxK9v$xw>WOM{*l6Bn37*Bt0xcc!hw;z>Cq5i1GLX&z}4siiGExhWVq1uy4Sft)``1<-W3!c=qpTszCeVwb7pQ1fe+v|ATtA& z_uB*ApkzYLQbK(83?@%K2?$7PGQkBZzZMA>v|%0_4(Wdq76!y>72vo6TW)qXJYLGL zczx}5l92cT4Y4prB`A!h9FYI!`TZX()n#Ph3xS#v4ndfCf1&y;9xHdCCa^oU+0a{D zT^Ukpv%~$!xwx07pCpj56@4{}3$Ykdl`LqrG&o)`_(Ch^3m4VUi}!xQ5n5eoKIE|> z30GEq^FrivK7)(75JNt5Pxd$7Vcye3;0%Ke~aM zXb*gEDQbt;Qu!=8vY;N=whud#v=0l+B|2{k(Vfx%1PK+$gRVqHI_7OBhMb)9vTKAD zEz%XRSuCV&U}J6$uC0cC>)y5xSXWrA;4mhe6TT5H?(#oi)tb2YyJI*&QS@A?op!N> zeb}xMsjZ!yf!^Jav7VJs^f@{m~Xs`P9GGF@kdZW&nqx$u6W&`{84iQ{W zWWO>m?z~(6PFJ_jW~%S&i|KnQWCcwWv*jX>x6UgcrHWRZblEiQj!<2{>Q@nRH>so6 zB1o%DK;TK~*&`y~ulTYyXnr|(nBwrE!LV~h>np)S-$K4u6!GacJRD&$_I4)QY<=|k zSF^6Qy|+KV`u%$g%YM_5ENZUoi%jMnKBIP$GFzMKX+5nyIdOvLevz5E-hWXsI=eM+ z)vm6$Sopau$F218a2nHD)8mE3DUaFSZr&Q!JU!oE1<1?7I;Gh;R z8O3==u|6qkC@D$wy(#R{2Pj)?eR=ELX%f3V-j>Toc!zsi$?aR6B&c0c8}2VXKyo!U zPQKFhEQK%0Jc(~|mxS}l6equSD_@R+@JJO21syGRc^xO_y6gc)<20eg6=#FCnbVSz zb?>~-zZ^T`J4ni_Yz5+8#RNE#RtXb+A?J-1YbJf_Hh&kG2h@ z<$2<-5v3RPxiX`t&ghM2yA|)^kh`ovO?g!+n{{6E>p`d6!fL-KW7D42QjU`y*;rT8 zHzSRD*C_aBv%IHbKzD4I8z%ZP%A{SOvRJ2_?G&N-dErVbrgHMl~qO+?DMo1TwKyaqps`kTAESUN3E#cwsnQqt}TElgzg0A&TQMko6Bk> zZcftRkT^Jor%=KGj)$K^9ZVfqu?-+FQRf|AVWv0*EVA#h%Fp)}pbu;g3gl}~;d`f# zAOKfG>hz)a+1UjcGYY^SbM)7rNzgbLoH+tHC}#GZ<>A3d*)VT!8K|}4zxu~R>$v-H zBIprk9BkW3kks*1sJL! z{G501+zY1&kOYt-4&nD$;tz~$$2}HbdzX@{hrS#N8n75nK(>)w4zek{b$vbxTVF#(BA@h5AZg0eFm``5*$1*I2ixtO=I#^_R&EPht~||IIjlcBA{gmmtd~8 z{lZTOK~U%*2Oz$|YC9h318YPZtPx=6`Q)S9oqj>%;x_xsE3Z%ms;V*0AyD}97y=;f zzWH(O{8yK){jJ|K^~4zh1n~LccLhx#keclt1uzrRm6YmS_0$HdK}C$s7jv)2)ncHB z#zuAC+qX4%<}fcXUgMWhR<^2qau~+q?!QMexxkJwIk|B=7Y2MxTf;%LDapw=N7@k- z$fnMP;7?lbO!??$=eZ^0hX(2N<2IwEMFq^3!dK(Emnxl2ON-XFHoD$iPJEV|8DP{u zRq%~_>%&-pP3;Iv=c|oHhifk@JGUZxidY7vcT%|Bnr}Bn7Oue%Gl|(ap)3^3! zIj^dt^JEDVTabXgCa>{`WBOcuMnpb)>j&p=cs>7$X%+V)7l&&Q)z^mPGL^AkC3$;_ zziG;}mtwD^WEw;8drm(2Ew`#^Ujwtw&gL&oZ68wEyiZaZCT;o`HZWS~HC`WmG)Cth ze;_{W66R4LlmIDRI4$8Tse1i78&Dww8+cso$Be_tllF3VhFqzbdXt)YQu$g?)QcBt zpTFBJg|}*L_If1H)z6pG7u_t|v6uOjvUpa-mTOE~$D-ZKq8|^fI9}Fw|NQRn^LE5) zYj4cxKDQrju5_X<%C9~)9gRL&NaNDOuXD1=VIi7{apL_suZ5@$WS%X5@IJ(9p|2 zk#{X*(z5&NtMLX6W7j)PORU)?^Hns(;gYLO<6%eZuN}>@uG2_YkKR*pQuxRFE#-sgrChgXEv4%5}{(;IVp(O1@G=artArmMTcgNPqYE60$u#m%%fv6`oy z?aDpe+Tu*zHj#bb!)WuJy;~2R0F#NoW^p6sUds|r*U`)GmO0&xi(h@K3}@fqr*Ghl zNfGY(>bnpr&@iSbL|pWqUiQo3gA`{E=RJKa)N!COB&nTjV$Es(`c#9T1}9{8u2-(W zvXqhm{`c+M=l65BNn@}weG0d}T!)Aebpoou?l;Adp}Cg95mptzI6XbRHAO(S`##pO z2enTu5G(fYihi6qaz}NpR429kTzKKFu=684L&75q^a(9q1)s;i4-}0a_~z1nG@ex{ zSzW*Og#xqcq`2|Yo2AU0F$Jgi*QHca#MMg1FKWg+YHi=IU-wYG>tvyPiiw#rjK4i6 z{GI&~aSqXe+g*zx3-W~Po=;MfMg<%y6dwRNx$!El3vCS=bx8hSfF7r(7sXj^1pYF| z$R}{74r)!(E)s~wsTEi$qr}__q1%)d-R=4?r6kRmeL^FmF+OmIE4`d5<QQxhB10xEOlH4nU3mYpLv@RMV6#FLPVM`L?R+>(L#DW7(U!+ZexHJ94J`W0t3Ei4HZ_VK(C2=3KK{ z($8=0$W-@6O`mTpu5Le_xWA=M{g_Q#pYQTF-_8QR>vnSM+?ki0RJxjUl5%{y7XM#) zZyiAMdY(zo2L6J_0O@~3Zbc1wvhujIC z^E>CgH^zH++;_+O=dIyjZ1-lZz4)&0n(I5~`~+8@*;Y*DZ{l6fO3A6PYfK3zd+x8X zNe3?sviqgY-LEc*O-pvk>~m&m%~g7}jOLi0zc0wT?uRa@dLZNWY^|y$z^?6rK=wfJ zr|-}peug3$T(psWB`ZEtnbp*cm?-Sv9Fl9Yt#c}Uxv*2j@ulh zHySj%;5iD-rdIs|ueXQhx56o4AR0{qH^;b_gQFunMJvq&v@i@+_NvZxq`mtS z@lo+bI!0!lY0Xi2_Mf==R=fE3>SPXsZL~Guf^mDZ63I;*CS@&YybXTicuN z5V3Rm1}w2k8_{l`4^5)laC521nj6q@!zNUBeRle5d{N zaxC%6+Dj&JoLvp~gB~M!gOU$L8|n3qv&Z3jX+$Tlt0GZpNnb;aj8k$yTHQSM*zK&@ zxsojXV1GJkEe;-U3^|TREad(;p2(Wtmf%~|Z8!q)nnOhs##|H3pmJ02&tZ%%__qr5 z-O-Y1PsO0Peli$#sI9%Q)2o!07n*L8HN?xr#K**>6oV4eP>|VRV3%u=@8~YJv0TV} z#z0+OC;uYsZe!Dw#FPSB(u}I!IB%XcxjIT^XLDqVfLcndxrrrIspkRuE3Mw>n+E+I zlJ!5Rw}(gd-tVFhMy@41a(H@oPrEa8RgZ7sAVi0I&!s5RSTRYV!(QW_==KZbQFhDo z;MGrNyx;7)J)~NSZI3i=O6NcDi_BI)zjq>_;gPSV-k^HX+_%FeGsmwNWgHR0csEsV zMt5CY107OT<^4gDRsrFZT`#qb^L{|<8wpWL*KbVOfP{F(3@pl9A1XHZStB#eTLN*R>MSzm+gh=C$(a^A}Uw zNMuWNeY<;Z#$Uan7edonUKpBQ(z?Ch?bA8q2AOYSpUag@#a!F|F?8`$y6Z{-{u#r< z!?e=9_l#NManR7zWV~<{E(qKP#GXrM6oLky_;?LD-?KL{60qb)J#rfiUO4_k+!&Ym zb#yHG_v5pl0$20@i|KH~x#7P=Ts@ZP{#rl%{r!C3zu)oqe+@MM#~oRtdH;P_kK7TU zNp}pFwmFRFPL05A@0L6T9YQM6 z2UeWy+s~1>m|q+>?=t$*Gcu+^{Re3EV7%O&hw=F5fmPeT#da|gzChm!`poRt-xqW4 z{m5^Zg}(i2za9oG4vhY-?Tq2kjsvDNy;)LUS%LR+14A77ME<3m=&$~&3Kw3Ciu77| zh)47Omm=g|=lC7sf1eo!#{YT9{$Gu?NBd962cvmm0(ppZLLzf|N1kp~7vZ?w{>u)# zXYYIpN>Y2+-A#A)*|!LcFz&Ngr#~oOXlRT=!d7O#`X_2s7Vg@p_%Ffaj-Bk)LEs8$4^ zT7LeiJPokNl&yIZ+5sL}4zP11V!y;8S`Dgh(D!h2t5Rn9)AaMIf9<8SO)|#g5AsKv z&p%U*BMbDQLG)mL@&XqAnnOj(hgqDpsYcKgy9zmK^viBQnS&a_?tGU4RK+2O0}UdJ zAD0Csp_C3#2G_sz&3ZE}NOnQp83d1DI{h&Lm{l6&N`Q8Oj4@LK;7-PRdf>pg4=yHd zYWE~hS7yNOLje^Xa%&|T%#cQSc{35JMYpKDkNw`t;0+jHq}6m<^IBw77?TyuMg&O41b7pnM3lGPcYYy5b ziodavY2zoy?oWQ(Aau&kj5pM|XJY1IvtO00$lJr%->Y7KSDo5%;l;C-(S}yGqTxZ> zCZ1z8#9gYv*|br^a<~|p+0v|sM?5KpP}-lHpNIYqh(Iv|ZU+a<$^hJ9Zfs0JP7d&1 zSf(6DkvbZ_fU+%gqLgH0_UBjukizxoks36%&G0}h=((h%^Y+5@{5*gNB@hQ7Z6_7S zJH5Ip2sUBStf!vnAm;|3{`$@VwBJNf_Cd%hD46wRG**?kBDBQN#y!uX_gmZBEq=7D zG7|ZaK%DsSp2u>36iMqJ*raeAVLK+hb$fZv&tE){QhyPBZ}8>a4L|ICv%Ks)4OK_G zFI(6Nl={-v7i>i`M_oVgeCWApP%}|OoO2;a0u)(dzfUO09b9S8w%LyS%2w3y&FF*w zIe(N`FkH`M|tPDKc`N?5KXU)xPJY0$Xc>oQ><{Hy+08RXEft^NY0NEwn}J zzo~(U$>9ll`XI*b40Q)6AR`%q#Gv5>DJ4u#LNYQ&vo0XmLyA5Ej&Lxy0Q?SE8?T%k za|ltHSo<^&EyqEtA8IhCUG;3IO8`Q=&(0nL@?2+PdS*~Mf$0bn6Iw3-)9j300tg=v z@8K5V0*Aq9)yw$!b~ZMJ#l@>#sqMfkhnithGu&FH03d=EJAmZ@HvF{A65Ilz1P|zT zH9+@4$2u7Yn#KZ9nS`!hOANPknUlPHD8Nc_HATQ<5fD%gPEK}!h;-5t5%Ga)DfGZ0 zq`^svF*t2Z`Qh?~-2wH>Ihmgco)G#t=18GO>Eh~YzdUFQLlGNG1vqqsN~u*cU=sm& z2;C$TF7uv@Yca>AZghW;R#gdN*EwAgQDf2!xKAN0XgIo=owTKOlYQtS4^fXSs80PVi zMRld!ax@yf4UFX^oCFYI)6)aUlY$8j0j3 z87iq*%3{=tZsvt{NjpV(PMFT4mkULkN}};sB>e^`$>&T*f?m#+DTJE3J|#ZRx?@3Q zET}EoqA~U)TUp5D{e*6s^Z>b&uALGop$W=)0(%}ev-pTLfq=M$~V zikCH+QWeD+TkB{LuIv0zV3YLA=a$GtQ!B0)K>;x)>~Kh0zVN#OR(Ck8s@O7HQPlTU z-K?p~nRG3*)eKv>$QHRpyTtS9i!RYt1haWjtm=#irIqtITCjz%k{IU zBCo|Dpz+zZo;FM{`DCM1KCJWEXuUZn@vDXVR<}K>@b;VFsLcs_!p}_io#~HbF^h7C zQ=SQ?Qe*Ahs`C~xkMG=_TRKg{h%ng9 z_crT6D%DvT zPHjCwM9XF$&;&qW0Im**4GC!dxCm89a6A^TX8~O-Xhc9i9Bn?r3lk33P3WTZGw}~G zB$V_3i2|-kr-~-<00+$#|JrD{BTj|8&YvSpPfI&(tiu7&MI8{5z?T)|NKT)~C{str zeL#$Zs9L^q76ej&WfI9iqYf;`phyU0asV+VFKU!nK9Q5N2Gd*!ORP#_&##3Z0OdZIeMxxcUw zHueU<^&6X;$+g^7_Y%Y^qoo<%+nWsD$jp`D)ViMEB4DEI{5D>q#Fl=)@I$pc zt#8PM-R7((wwiRhOcX2|(Vd6i&)HYkJVF@j6lrghe5oK_VZ%u=FHe6pTjhu-&Sw%z zPZjz8U4eO6$Pw;^LQ3A}{6@3U>HY+2N_l&>73c1df5U=%)d4Jj7c&DG&`7WflpM+^D zn;#)@P{5=or_fyJSZ7|&lRPWZYDeXC;-IpgXko>L;-Bue>c8DThXqK5v>Kk%MEmw3 z+LH7ns-^Nvgbs~#HA{op=)%>QNw48#V)2Ak+)nJJ;z=Z{ux6Z)W5xAdN54z<;h8UGr3ei^A^EW2mA{(Fu+h7^i_1=AfTy+nuBP#jV%ogV~Svq zNP!s!$YsFL0IA9xDlgDFgbqkm?;WVIL!Ufkh`WV9H) z;e|i;oWq?w-5?6?r=OGcVb@q``JK8U^2672nTg>&fqwjxCvQ$I4i(jo+Kz<2y!}4A z8FTMFHyD6#?!&qX)PrSZoh%N0aD<$4Oa|}i+EZx9NZA*!rmN(V5UXxuVeK{^3-mQ{ zVbDr%B`Pz}-66)-KnWukghcR3YI8=YVr(MFXh{nN6$Y{rxcnm*s0U?MTCE!&jS8;2 zQF?OK>s6PUOFEJfg)j_WI4ndM_YA#Ib(nm1RBmBQlzjZ#b=;;YE=jIlS}(t3EWK~N zSFez=z<5R4#=NnKcpm6uD=!w`*!*%=m+u@|>Sixjq;g5(?9Q+%;uQa0`tg&O(n^vb z(=9|WNx~3$$;pqnZe|%R>{6=iwgrph$CqD_mC|BTpv&SdZwA(gljUODy!Taf%TS`= z=#I3db1-vIIT$HKwSLE9S6#Hx%HLwiLeo$m@$U@_4$NbVDlSQq6!pC@D5Nkn`25lG zmGB;VtK5h%h9FjF4xc0VmVa$UsUCX|GRr>){T`%>#>vgb7ZwYRa!HAKbb2xJ!FM(&n}7vtg?> zd_ox7h|>Cpma(b`2PU z1!4_5yCNH>GH64Cgc7U_q2*=_-pri_@Nfpi!r=_fF$BZT01&AIt;of$pywbGt*A5v z)FW^h7ydcTZ;}!dN6{ z)<>3tcHEw#)xn_Hxeo%L(>+@*)2?2oAq>hl9y$sApgmwFUKK_%k+G}N`MlY2H`}C~ z(RHrGn6tjaq|v!26JZ)%d)@AWX9=|~log)NT=dCnv@KF-OgM9kt0Ow=0seMyP& zRNP*&*mGPinez1*q2zP*ESz;O3>~<`WyY$q*(Zpvpu}ug*R|7kZfx3t92}lG)`Z^1 zoj^;OLDIhDMWUJ|DGD=9-BbgmoOpYLWtIU+V0XE#dGdpX?bE;Ox#0x1l1Z(i*-+K} z&J$t%I_vNLk7|JyFk}$KeSHwA9qBr2fF@ zf-v`xDgXkN%`KrkWfw?Q06I~9cxwxKY*0MGx#$T=2!Ml#WdLgiNPF1y_4RdilL`~P z!3NU7!2zIkQc`WuLQGl&eJBtcgD9+s2m-}BEuaH*euP!5z(UUf1_)L%1ML$u0|4I& zlys1!;U078vK&!hj05c6N2eT+!&&|P`#w-^Y2A;4e*R_w@$7VZ-eu=v z=ZMTV5ABNjmfkr?*7Jw3n_V>-IB<`})%UFmEuCB6`gIBMJxy!r! zfg0=!xfkOz*|ua9C%ve>P$>1WX0cL2lZac*#Dv&5G<2F+t2ImD3oo40b9BLXMuO~~ zSa-WJ{}OViyf4%HNzGM=G!d~*oHv9&sy>!ql)js}(oBbo5dUse&H3Fvlguxk|C2eA zA>H-l+~bf}wkM9QF2_si%*RA`qz@CxY7yG`(-|ra#fl#*uF8+$eMu`f^p-28RApSM zr04fh!2EDMWAq)Tna~6W0zv98-bP7u7-!f$aw^f&4sN*<^YctZevmoNOibj1jxe2& z4=f>p(R!dO0hz)nW{8kp1XVi$j9DP_ zix-ym_U`uLS3v$4G8O1JLk*;?q(qF5lbxLs6x_?o4!{K1;SmiX6yoXux%2%yJkaKW zq#cox1On}+nH7j|ffSY|D24LMO~CcXr+Rpmh-l+}u2LmHE&*{0@KD%J4LG4Llr?*G zHLy-Q4+I%03;i*zAT2&A@rEvX>j<|y10^m)esTAMod=8!9|gt<8Kh7u6Zp%bfTrj^gz7aSIyLgi7?2?e!1wb!+bkx!QNZ-W2?l;r>oA{S*6pMU(f_(zL3yhvL0u%fxPgK-Mr@<5cJ37v+)(?kwZ~Rw5@8Iwb zlE4r3(A6zIH-fs5nf~y8dM2}AxvJy3PfuCcoUp<*&jouVrFn5t;?mM!@$=4trKL{A z;iAMXxy~f+kgn*9i2JKDY~*S7Ix@j?+shan`9*cik3?TBw(B3KATgqsE<@kE&){&g85Qf)c;h;C=Z25-6%$tu1 zl(x_q7-t@b00XphS5p%cRen)g|5_lgLlvpngsbZHq|es5nAV)OCL=>b00D~Ao?UUi z>g>C>LIACyWTFLg7$m%m+2$b?-)8Vdr=_Q#IsnWBgIRGd;1D3#pjFzS-Uz&KB+1pQ zHzaqj27#v|KwV6@fZzm`nV+MiYod#zm4|>gF=4w8&p!})bQHx{1&?JAwEP8Vgp{Uz zH(o6uAAz7Hj(arPT&4E1>UWxG2Gn|*C%@ke60aE(sYwdId&FI z=WtRqSNSCj~E~Gvb33#I=!_-GaB)z(mujrhOiMGjZ#%7B6@}rN3 zLg^$${zHnz6T(-YezPcKJT0$wSN1v3Y#X-dAKZGh)WMs;XK|r6x&VzPLn%(DI&Lxb z*||IUVY_TmMcVpczMSCE$ulwe$T=Xy>Q!U=``#ubTy2^6e_yppro8<}O5NgP*E<=C zsMNNbT9?H7m}uy?uMkUXLO$_To+VUnRjWH}$R}+}r6x*^6{~N|)b{k7QdxP* z^s;FD^L5xXEJo~mkr!-B6^Gq)-0y$O^6DT_9$t@U)4#-cgfA#-I3`u;%D26#Kjvnqpa)EgtbMFIx_(`cS$Eirek^Y_UCI%Q07Uzl)mab5d*^ zw)Smb%YR$>r`%k;Bpg3LLsembc-DIZ)osePS4KfryR06uh8l4!`8j8f`z$o4_Wt(p zgjpYgr<;3PP1`xVCqCwC;kREb&)yrb^@?cFnxT1L=0iYV zezQ`0#FvWl zKCawjX7R0=GIWs|l{=H5AxE|7+Q3%J$me0Eu&^dl#F5o4{fpv ze4V0STO-c5tPmqq5yz|@R-@EB{#>{T6^y=xDc(q_&(xFD7ubRANEx%N6+f1cV6d<1 z**pBvvR+9#Z(Hr>F0a7QE6(}H@!^{*YcI2RrEGAYUn304P=CB7+f%7Sec^<2muI$% z-D(D9+xQeQF?e{S|A)n4)JNIH`Rx4*qj7F?~_7V;=d{&#E1 zMICM)kTCA}6mrqc69{A~05lx2AA^nuMu&$k&(6!AgFov>Qc{ga3b=b1PUGwLmOvc6 zmqs(P&+Gh8)*{}LQsk|}?#Qp|$r8PunCI8R}GU$(k$rn8#PATyq5 zWAoh|TC0QWMg@*q?wcsqb_7C8MdTK_?PA?$XY#(Ti$Al~A0?zC|HN4ykJ1JD?P+5s zTv1dtm-ebE5Gj1OD}h}2r;R0WDL?bDg)45wTr;7YoiI)Sfh#{Y0Q_3Y*rdxBnB@?^F z|8xxv9r7s;TV9wQ=0p$j{Gt?!$BQ)(t>=oBi+k9oCiD``Dx@EW^JlbZJu9kx&Kuvr z5BT#8OhhDpVsE;_pLk}lSgY#chh2WxbWvCsAv=D%*C8V~N$kbb(ANz;f;nk*6Ny+Z zh(ynr&zZB{Ir^^)KZrY^5z-vVeYZN`&jH^z}C>_~*y1>3*9kNAS*(85i0^$zjz+Lm2;)(2ORy^XanFxVkJ{?j+eJvrMbf}3 z&ZSh9Z4%#~IU3AiqQ=H%vE1)#ELiJTBM1%m!!3=Lxa1uGr4vilmU zm2`Ihj0VQIH6tkR5&bD}LG%>!XY0?;JV7FN;z#^Yw@=W~*B^YeZZSB#dMqpwo6$Nd zP{M3g(vGW1JLr5+^kT&7NLNB+!{-?LY2uJCfpFlCjry`mA^ifOYP)%gD>vp%#T+!B zRY{*`wt(4gycCnR{e4P)qqXK`e0HH#^~#&C3stA+U+{I( z^rz-Cn(>no41d@ffnS2}J9+>D=>nVNZE{yFf@ zwB8@$X?|Iwyp9Hab4TWMf@qy&K-$g2-g>Sg~t>ykCR06(0Bb|rav(=)UH+VD(l^z6^BGKS0sOgjMa3ssn_SG+B_PRrqt zKiJvXP1g5b?Kdrg3>Lay@7_I=t3Qu{hXZZfznjOc-p>HbqUPnW1t`%2pZ|aep!E1Z z3xtqicub0cLB?^41^D0C;s0NT2K;Z@@obC*2oPH0{|1x5n62?Vf&P%jAMr;6|H2cT z4y23Jf8k3oFmwt28DRMEBmaLU0Zg-#WBljz^8=MK^W$s6RYQ2Je;DLE#>PrN7C-1ZOIBaI~k{4Ibo!Q zVEGlI-d5#=#F?d&+4aii7Ks2yw!z~;Bc)KnDY7>>Z_ZBYhALdM$JYYlTN#+m_C@A1 z8TPk?^W*QgCuUL<&(1kc+Bp15W&gPCL807aJlv*cPG;66(;RVJQ~LVRvv9K86q|-v zhL@S@rDiF3?oya_3x6<~C|M)p|C8(xKecSo7yMRdw$Z8gX-vEtI){{9-AsP9mT|Uf zkY+V^+qYkzG&(C~=;qm(?41W7P6vtsTf+-iJz&O9`;p{(^+0h6Y+V@fd>C;U@PS$R zp17{y*!~YWJ6_LmC0mwfEhHOUo6Ev3%J{cPk|Gt@m>fOQE4Q?)g-ghPeM6cHOE?qUo^VkLM+@|ZObB* zLnY+%%3(N*8tw;T%z{iiSzXRdCOgTRtJ@=sTO z+6<(Zd<(k6V!c}8PiguP>8YDgs*yg(Hbn4;{GoSDMH~GbDWOc8*4_5oMU|_+Cl4!I zCGc65$-{SN%5p8AW0B|O*4?^{LS6L3PG^zx7g(jL4kx3u3tL<|oK48Y>}=dtog9^I z`NCkES#GdVjB{a@@-<5A<5#zpjJFn47YJznSO*zte9bg5>@k!Vm%1a}I$$q7{*aB3 za2O5yecarcNAJVQ(65;-Ix3jP3Md^J&t~Pg zKjY?7LDO2;kV{>Rae6+g_1B#F^a9yjhHv@9a~WSBvS1Q=^|O!?PS?IgHhj?JYJRF3 zlF4lqU+~&a{`nok^@0NNR?An?oM^Tb_6VnT${()QBhe4*6h%U*+!c@v&!og|5u1Fm zuU?*N)n~KkIF8TcZ}@fKtW{T8_KYHQ9q!r7boF?nT*kPF#0qx0)W%+&pbzmW6vbmL zdUCd%C%oZzsKuLJcPo0VbJa0-9|Fij?Hz~jqHCF zM02)dUc(+IzS(qnXE4lw-x_zfC*M9^{gP6#lknFigMnuu?Vb{<#VHi6G3)Ww{{q9& ze)fJ}yt_On?xg@ThLVi!^6X-}`l?Rq^1V@Qds{#OrqQv}tJ?@Vs*QbWO1O!js9{7@ zCV+N%E-mMsw!(z;+8c|%=*LQY9}_Jatq|U1K!n#WYzla_WIb!bCFLim_l`*5#-XKRZXa((>i*$7Ym0YXCJq|zpYPwUDi60|wVgg*G@mszZe-I&z ze#w3o>-TK^PGY%s#h*{#tG(S=KDexv^>NyWQ1*IAHLI?>AL-+|KX{wU*V=7|*mDhf zj1s=AtWVOqOOs9I`t zi}UmYRgt%SB1TV)Iy-w4frFbNW@WCg%lz6lbhPRNH+}d@3Xy*|uk}Dhw{zG~Q)viW zN5^mNE%xqjvWk;-Ggj5bQQU80j_P}xzN`sS_s}9M3vR{iN;~y@;KLgv`zFdHw)T2V zk;BSsSK2t2v}$oFnhfpqoz8I<&z~j3BJDS>vculo`Xm;heH`{I(~3V(NLY042r3P5 z%((q)1hdZRQ=3wm%K0Xsh262F)XG>n5O*G#4&^g7|JW?SCpMLj;pOnKmmB};7ZL3u zD0+hq{Um@ixJ#p1h-hVYYWV#|)Pp8=k;gT+G~~aPZP({)TII_Uv*}$PN>68pQeaF) z6x#3hcGV0jTIho6N z)NsG++;?)sL9Xbw@2e=hf>*VdvIJveUkh>?OMJ^?3BBbj?Z?1qS~_dgvyS5~LpH5I zBsTim?wa03R%Lf|eZ_unhG)jkP2|R%u)a&2r!zr8xkgb@$)w&~ryYCMR>@w*QS7xSfE0nTmJh4*JnZY;|YGkC&TB6^(xv@CsrrPE0 zDDxUR1ncY|>wpN2L2m5pvhuy-Gc|*)0Xd9GMdfP2H0#ko3QQBwAHVae2NBkY1s=Sd#qE6LyVVv zw(}g3&q$lOSyzM5wsny*lNIgh?{Qq*nMR8(`fXNxqJ}9KQ{PkZa3eeH_uczRT;`RX z+6q|S;bS6o8)wpTEQ)X8aF;Et7nKLZ6yA9L$snl?5$fh6U4M)c@;m>%Cp*L!KmEZE zW|6Jlg3PJ9JfU>Y3gx@L3p+w0`;6S%=Kq?AIWM{Sg~-hV%$BXS;&@VTH1$Mr8)4bE z+l(gNi-@lnvHMay628vge8-NGz;XMx=z;V8+mWfh{)ehzLy}#V!MUOd@(J_KIRZ`N z>pY5YuqqFxsD|9`{jL)0aVXqOT9xHDeUzx$ZO+uO`7rP2!U2LgumI2lj+x)Ap|kK1 ze=PDl*9LqdVq@MHKSxy*(1R3}dl$>Ct>F2hsQeNh)9<5~^IlL|jV6FJHTeW`Ju!x0 z&`m$n5rC>IG<}p=*Z`pY;DIsJr2>l$hNp06Cl^`n8R9+WVo*T03FpNNRT};xUwv}> zDqu%Ww-=fZ;$QgH>gS=-vWy39{6M?DR=fD)K61hFcV@Ve!`+w?!{TX@(e1sGhBTd^ z2iLpC#6=!t!$&2HOYzw1w(>LO61*z(L@tVqyRqfjF%JxayI$x1<(UW$6L0Q~31AA3l~2_g!x)@@1@V?q4?K#*-7Gc`tk1#T9C`@d=!NpMw9d z>pEU=vpKIFs#QY|P!8#S2)Xj_#-^C5sjVN2DxRMsR8>gMe$Hs3P08cLYEM3zw_qTy zG2xa=-H1@Q&=qfiYpfshlHkOLtgO?yePC*cykbcDV&b6Wg3#7&yO??hN*#7eUxieE zT($Fk4~?}j@BC{hpbcBr@ZgZ8rOojc-9lPd2oXYh^Wsg%KKgbN+b)@#P?wBvED`gL z&ddJdb!}H6BMRJjz9E7>rDfU4g}W?93mV(>xvAb!Q)ak&wLJS-Pis#`h5{?kPbJ^D zk-4SwgSso=@0_d9`bmmR6Yf= zIW_9lcH4GQ{$3G%&0o@!^vn%(YUl`6<58$44U7HNp3tR6e5qzNVVS@w^x?pVmBy6k zu(IE#;Kj8K}Fnku#Po5H6yk z=4x_?&7Sfey_ou?g@m8C((ZS!s_=tFk!O~SJ6k;wV{^OIkF@u`{&p=l2_NwvF4M1l zgG)%LFr`<$S7-Uj`vXq|tuw=Y`kT{18@tCWOPDdX5zV~$({mwh#!J3TK}?QlRe?6~ zQpR6gMN|Tv8437@PvmWi!sCA|(|`7J6h2{4WBrrTKAs>tG^9+C7S7i)&#nXi#n1SkD8&4Kwd0?$(UIvwJmk6SakQc=UU7M^?7p*I zepOCTOdWOgHnLbUz1{IERUlC-Gwl}>D7_^d;GUk^V}b3nGQEe(!e4ik{qDb&yN&d} zjHM~LzxBn2)W4jij}~P+nB&d>Un$uk#TdU^3UC>Cf#GrT<^VTgV%75WsCz^qe+e-E z>?QrTocX{0wg3M-!0-1ei|FFhA;lPr`Og&-n7GPu`ktDItA1aN-s#Idq5tcJ`>j4> zb^6)2+HL~};n%14g$K!huJc0+Do;BKiqqd<`y}vF)U~uX-LaJ z#e{L&QQ~Z(c_a!Hs2ULPSzYarx^TAeRE#W-R51Uo~_+FXxFTKw7M0v~I#*Lpd1n2r+vLv%o{wJlimxZ1 zz3)gLDqHt`_CA5r$C%TPZ@|?U+1ayZy7_Li#LuKc{~k@idI#fV{3LJPnW@&dwquNG X!`@GOvaRn=Pgq=73YqitmFNEec8SyQ literal 0 HcmV?d00001 diff --git a/doc/source/cluster/kubernetes/images/yunikorn-dashboard-apps-running.png b/doc/source/cluster/kubernetes/images/yunikorn-dashboard-apps-running.png new file mode 100644 index 0000000000000000000000000000000000000000..11d626f2347c798e395bb26136c1fe9f156e91d3 GIT binary patch literal 67450 zcmd4(WmHw$`vwe~sHA{^v>=_*4GJ4+5Tx0Fbb~Z(NpZs%n=T4&^N`}dQr|&o- zlK#E+&g))IP5%E*@0doVrl!_7?+Cl>zF=ilpnLYMRXj@iZnbPE%do zVSTXk_wU`U^e_D!|K7RLEjurdiG^i*eJJ!^T)--L6kK# z=VJWrh0mFm9`7xWfqxH=oUq?6JvGfrYJtYKD-)?zuTzA z;TESWlljlX@BD_?+S*Q(TM-MppSF{G<>lv3U49xL9~XihzN33~b1`!h7!)*7YBq2g zQ>L@P=7WyXZ1LTq42n%0txg}w@JyO(`Ts0VkbiKnu*bQ>>({USFi8iAtGsTmjg5^d zC@74KjL68y)=@8Dxk)AV`?Z4S@`Ni|%1k);_}zG@8A?(RSsI__xxyG(Uz{UT7iYbp zf7h)xBsDFKRi}1!X9qFu+gXN=fl=eM1zYS0U0*ksk&!8Daj)wyfu>Y?G7jY(NhBkQ zRz8fd?bv!tLRq4_xzvfcNhjQ@Pw46KRMfgPm}s=be7fgM3)Rt4d?fw9an_=}gQ5Iy zK0ckuL!nRz97{^18aK>#K`1_j*m#NrW1VhO!E3yCGkVBGWvOUvV;f z{gb9GB>9=DvT|G*_Zk0(ZFq`xu(nk`(*!@?c+JtyP+3V?Ft-9txJ-tMT~+GY+C9!k z|7Y<%JG;8Z%dI9Z&kkmO!&a(G%gQDyY#KZ+j#;#;S2s3zIEb2yW>#rPer9x%?(kTC zo_VZIx}~Hzc82!k{Ji?={O|*N?9p3>$gT?wtQ%eu&vSQZb{6_Za>_&M|8EP`X|8tM zI6qt)Xz;qJ;9l@WBhWW6h-T3q@8Lz%t!rddYbH6J)aO-Khe!H)*B8jq$^?`fqobr1 z?O9@KK~<9uUXf$be!>5u@}K3B@!8IQfwJ5zhAT`jr*Dmy$w*0cB9S|T+3)S=2X)gg z*QxL#{lpZmZK*CZ&|=X7qzkBM2%bIjFtkk1&u@i|-y!G0E-L- z#*mq@wzyijTOy_F!RPiV7gs^1gzx?Re_zl&tE;Q4FG*-)ZtjoL6Biek1R=Nbyu7?l z>^K#eDjibmu-V%1xk`5aENe@d*yl{^> zVXCKbzG!NsT@iJmopAUU6CPIR(^HGE#h~7QABdn}VJT2Z7Ot+YW{&(+P+*-b>@hz- zA4ihnnqlHAs;&-IhMuqceH^u{xb91qCpDPp4C}NU-n_&7HZ=VEcQHtl1px=1jXSr7 zk(pUWdg&&Ybtyr3dLkMchC-KDd4?#IC0ts%J7bjxPDxvn2SpSYD?G;hkAiS?}mLIn{$mRaB@08te|v^OdWh)3bMm%vYm@ z>z@8-6@)9ehze?JOX2vS&4~? ze*5+f1cdnSZ%d?+5Goc%5i-}lq5wIQ3;7&oyvRsu>eS(%`9vF;zm`63TJl#`J2b>! zzu1v_BQ6etP8>A2sIf4zh$1AjRnOw74+SDhO-)TE*RYj3wf&*5ds0;Fngo=Yg+5}e z%C#zFe=kDRl%lw&l8Pc07UC>7<#DPCjq4enlZ&<-6PzYa_P+W0ukC$Ovp?C^CMJ%j zA-kk=9~dqqp)YcLc&|KNIfk&t<~{wxz8zXq(O2LCkj<(qD>>&zm!p#KP>uSUFK4h4 zPx}hea)z03BB!LfEsZQ&jCn{BdWToHCVR`@U&L4JOxJp-sVk={(;Xc-Z00drU4oJjL9>00_#pM;gB2^5JPdWx_wr07gaYL~9Cej0 z%Db^Sao8|SG#(GG?5XhIL%FK$5w&#qwa$iw1mRx{m$xQ|LQM#rc=?1i)@M6B$_4nS zK1`IArxEa?r72Jwql&YaH%{u!X0auyDrg-@vRGy4xehV=N=ndLkm@=4rWM<4LXp+D~J8u2TKA4CGM zNj(|1Rh?2K+wfwv1h9_x3bd)2s|u~=>J9zOEc{fykn@K}yo~ssR3+B%XTfhwa6^Qu zOYq29-O}!@PHbQ=&(?63Q1XaO{&3^Kv|kuR6{E@?jm$J574 zhT)Z48My3s+Vd5p_3Ad<6#4qL`aHtY{COw~Vb^f%YjzKn%*)J_ffy}?^<0bz)r=87 zU*&Ro9NzhgbC7%#S=EpmE}HTpBdk+@sC?5UE*3>bCOkJYbD(YW<%mwO@&MciUMqM( zJza^5?^(XrZ+{k>95U^YxG6lGDRYS6v0!YX@iE3U6CZ0YC%1r2#ljhKsZxvgb8>RN zI@^ycXx6zXweJVS4{e*n*8TN#eBzr)oOcbduF{ zuVZ1zYnoa-H;TF4sDyrT?=dCw|myPge5F)ta!?GKlLnX83~hvG#vJGFTFHganD6V}!;e(ke>5@-Ma)FvfwVr}BaNq} z`MqAe)UGsLGnKKvk9!NVZ;#jcwY9Yc+BMv3zx!)xapw6>wk&cO=6A2ITi>Lug3t-`{?<0Jg}rIFGZ#lWMl*Ykh*KqIA%G&EehrY=Ir2x@;ELIXMtU&RJF z`4+T(c=hd@jN|EePUN60O`01|jX~(jGgg--EvLofj|lCwNfQ`}_O8Pm$AsDCV_MJ7 z*x585VtS&6)TXRiMa7^kOD>1CdEelKL5~{@bdS^VgF-mb_9u*;#~OCf;xA^~YlhO0 zY9SH<5-GW0nK0g}>w`Wct@8&=M0wTCk1$KG9k+r3Yy z@*&mPn-%e-hj}&|ESgvorT2?$t+#j!xt65#6{yu7lc2zkO)KBScP|lo^f=xF3Dg{< zD!DP#osT}3C!<~P)gGYEj5mb%8!m@OBW=@_nzTr$`U{*_48Of~b7uW&Gpr&R zC*paDNA}BjIIs2fih<|!tEOKzYvTs($Fy&6+%4NDLrIe{xrxfy*_Kzzu9CfKN)v^3 zYx|pL^;~u@r<_}n=2JZdaX1YOCAKo%)Qvfu$fYM&L>3Mr+nY{k zui2~6+AZ6*#r)mcjvR?Aj-tsI`pCYaO%*$;I;Z{A6?{i7rR4tY3log5f|8PQYP(O> zQbSWzbWg;O?ICyh&$1g47wy_dn%|i?*sA(o;coPO9=|{~x)ug}qU!XgFOD{PAyX6a zg37xj#ANG%C8N!b^}V{{T6aS1fqv(+^WGWZe1YzC9DiBl;Y;pL)B_#cC-s!H$D%(2 ztroUc)F*^|R$h%)JPIMxu8q02{Bic_Pfj%lb`dj4-VywpujoT;MMv?nKuMommGGCy zy@e`!g}FLFaw;mt#1;Lbp2F$boptU@Uu9Lbchlc7hg$5Fm0^+ie`ky&yRXe_>Esxh zw`Wbvh1&P!3+yN{Rr3_O&FS?t=eCRI!c<~_bM`eVaakB$;6KWGfswkj?mV)$z*QaAQGaf&zR z)DP5XwIxh8?$dYfjeNN;q_Z1M=Ur9BVuX5-^pucOuxYYCL9Xdikm65(n{|ZoiWJ;x z>{spf+oXe@vOc6v*M7klzeFiXw-*tEp6s8_y3p1cOG=WT#%gJ83{Q2Hms`uIDV5lN zF}ymI9kY~Ayi39zE_r;|P-@Mi*1kKKo@Oz!Sb4k-Dg!GkYr{2oL-OA-@Xnpud)dl4 z+1X>0biPjnc($b&sLK5}7TJR3E=a0>{`{%TU^RZZPA`P3m@P?y?V)(&$Y>zO9epsr zhmZ;Hre|Q_!*MEB3+elc0`qOnK-ej-{}O5PV!HYRu@yLOsdG>z{w4JUv~6;T84Vp zMA09y5Pw@U_W90j-AICj<_He4Oww?Ff@x3%*Qj9ubpHoF? zsPzwAf;{!#MA_>PoJeqtZ6!Q9KP%@N9@1CI6yHQwu(7=6#TWa8Mu(;&r|noU_P;k5)(Z`A7sEmx@+k{BMpwRlJUJzkH`eF;R+K9J{gQ zw(|G0$S-8Mj)AMSn+=Q30eTgp<}=|Y2E78Sa=qbE%LhMO~9nu zwc2o!qZe`-4NKRge=r7(OMG$vLZh{%pL^H@mrJ>JS)lJiNy|^%aIy*!bljSYjH0!N zo*!|g$3F_mUhjh_M-n-{>NCl=U{!1G5K*B%?C*)!xIx$YxZ$8L_x;lEq`Two%qN*I z{q`~pfguXf*qwG^`u6KvT~dJ_i(3=bf5fP9o`j+x_Tr&5p&js1Xnyo!0x&`5=i+s^u z0wwNJ`jqrH5jXDR-6+}4)YrN)hyOw?7i^YjEe(yaT3=+blH$WX5|4I)c_! zm$-|p^UM1=qPxu|+6L7>eAo|dta4kCSH)ldmKWy9nwyg&O_R$)$F%*s-b%yutee>S zDfXY{-Ipokh{VAp)HG2~IgdnYxBp6XUBNM$T zq@dm$B$fl+#39zT%Qwq<3*jEiNs!q?{^WX7zHX3#B?} zs?wtLK6q#Hoj zxR6gfbLb%EO;XhIoslF!s)(4hjS}LD*pmO2dH=rGqVgd5ac=GsWIz<`0N|_#q z%2W`%PC?mj4yun!tlu!sG;!ZY3`ursERi~zzSG58H$H-&t-`h8HGHkC<~N~EK|C$) zS~Ow@x0})~LbJppgA06FFFmqw`qk1U?!SjOj=uv_hrblfBkJ z(anb!^@1`<%y}Z(ljDO{4KJEx%*RWpPOq+#VMre-zuh*}I2$EiQb8d^GAcGkwWLeO zW$X1R1|hbh9L%yDB~+HiY?GC3xu77*(#-BOluEIeMrwSn&;o~jZRuE=bZsnyKC>x= z47Em;_)|^>9why!-)QQePPpxcJz^j5Dg+BhI=FWLIr|#5YRd}-J=v=5H%P$ zY&}ztr!wUglN+2px78sf36r#}*-qS0OY(QYth02L8kt`h@LB&qwAAihl`dOz`wkSQ)I1q8b=pbsWbrDSbz5L+PyDT@H zWJ}J)=&W<(y|1kO&W%ePPH4fw>lNH=+b0wCo%XAAFZAAOsy>XDprU8Z&-UHWC>_d3 zSWKI6GLCcln7C=C;)UvcvG}L+&7*mz0{7KfQzPlrKrN|Q%JQm{XSxS!=U-c$4)@BX z_!3H;m*OmfNdsMqn+COb>@r)3NK>pE8J@||x` zCk1&vy;-uex0jQPS4S!;G-C)Sn@kkdiAt#`W#?6EhIbnksmmIt8Lj3pXsGt3jaH@$ zWE1+fQd26%78LYN67uKtmnda3J;mQx+pls=QW>pK2nn#2;<3q`l9bQQ${PGL$Dxzi zGxe!SAG>Hmt9<319UgX3PeOG$C4<+RE$e%nBKk^;OvEM48EZ9Z<+i!&P-G=zM%|=e zkR7VmdCVL6b_#>U(p+oWBzp5>%iVIrHtI$CQOqflWP~7Y5h;v!w}6_eEhDJBoQ6_A z4l`l7yWVbI-#siVjJELOF&S6iK{B~bAqjsb;&e0HqR7>#1$VR(lT4_3hH<)0&~NW< zKu?o=f{ZMGvUJ=%+($@4;rhOOl|9#B+~}S^LQnCdqY@&J7DikAMuxg)ow|K1ozmiv zYDh~tDPQZ$ooKY!&Jl4z3Bo`2XtA+B@2@;WO>LHkO3U*m%es5JV$JL|1G+}%43~%| zDVY^Ff!~&8PF{2qgAi2PLbdG|E&@b#=KpxeCog7D{c2TMb1^=6KQai3miYO3nRDv} zYKXmJd;ueen1&LPXxyY@FvNOx&QF148 zNtuS^y%(85u~Hn2NR^5O19I3|P)tn_u2)ImRJ}<LaE|N# zb9MfB>Z+P$f4p$&#i48*++@r~a)!mh8M)ozx8D{o(Jatr*E6ndKs438&1P^=KnPK; zfhSFM{c#RX+d-qWo2bT&-#Jf4Y3A!2J?A9B4jrm2t7*Hke}J@WtUSBFZj~nlo#i#J zU&yMCH)h&VJIN?$j-lSzNC;jlT&npxgf;)l;cN5nuO_oc@eR2%B0Ox&5;x8xtJQZk z`Ctn6ht|CROz|DgocEdXfMqw=LHFLMJu2w!qvGOn9L!Q=B_DZu6p>`4GBnTb|MRD@ ztN2oEj0dX5f@IiaFCj+>#?DPRc}WR|1UXD6Jh9_>ruhb_ffJqm&J#BhE?^fJ<0y**jZ zS=_QXUOf#VS`=!^&Y|LBGuN(d9>G0|ccwPipAp{e-If=*IMJ@++`2CAxM=6@VIbzQ z(QgP1huP*zlhH^NOkYTt+9$a_{&$4rbYfy+1_lce5fR2mlcwbn5zh<^48)q>##1#s zzh%XvLtwU#QPR>>YHqBcdOr|uTx-XZzPWhcGDA2P3dfr3P?+LGRAj>AFJ9fW>jsmA z2#$PsN)x^})?=F{fjK;9!5G=~W44Z#66rPAW%~P0bZGOVGInmIvYo83&Se&8Vp8Z5 zTj9yV=hRb19_4_acXh<|WjSut4;c(QDeg`bthsuXb&b@{VTYkj;BU*6lazS+ONdPg z{4;`CT~>bn^SNff_djLLevx>dEfH{VaA0AD{3M;;G$FxZd)_iI2+&}z&T#$%wN69v z){Lbk9Lk)i?%dp5VzktEbT-rV7pGDb*z}Q9p~|Y=OThlI{1TzpTf;wQg%mRl7#T7w zF)%C`(KJ*X568L*V>Pk_{-vxN{n}ewm-;hgC_}R{aGv@@0NZb8YinlrfQxhTLyU=*ziUJvUnxR7sH~+thRa&U%|uOqJ_M4?&RW4@LHWrr|ZIyYNhJ zFEf~na0Owr&h2l-GRtU)LoB?Tno>`3t+jh(Js)EH3jWIw^fTHOQ0>W< z8(sweMR)!Bpxl|0p%g2+#}u5Ns-&dY`Qa6ORZ5Qy|NoBHYIpXRqM{<8j%~cwKY#w5 zm6ZjWZalPbB55ROzNABxnRDfl9sI7#c6D_`mKbs@5>zb6jvkh$_;oLWGkmMO>shdk zCXv|DkwX&`6IG1KnHkFcK98zyJ2z(kPVAl962LioxVwvriOpOOrlh0@3k%D6Hvagg zDUIA8JiIu#e_bQ&Vv{WibtdrZ<^DY|v}}flA89DkgD8mN!T-{1e6lq)HQ576d6&R% zzow{jrRh1N6RfEjw?>FXwk(wG5lHdR?1A>EIcNd;`uY;EKwaX)+ge(l;p1ZxbLYj{ z4!(oYq!!yr%?CwHN`!1!-DE{7nmn@Lj%Q|K>hn{P$V2q&(J>m7x1%WN28MxW>-T?s;={V!P1BB@*#> z$YzNivG~U{BjVgh`%tFZ6^^7(^*M)%1Ji+~{djSq^|O#M}mYBCxsYPrpfu<7p?puJo6VDOzx z1k5s@`Nrm2c>@z41?3B!Dd=If1;~O zf~SV<_U&6q$&Mdye3Wss zea~f(?p8?#Ch;uCiRw)F*pV{n3SY#JU_2x2oWD0d=oNhLW^JV=3;E{X025PE)NuY( z7AX!o1i4ZbX?@U~tUSwC{{_rOzh3Y(~`%cTN;}Btjf~4H9~ZJFm84!sUf?w zgMc^F0A0X>J$`VH zl$2CXPL47sFYj4Et7ytA9-b2rPO`JJ+uGWM+>RY=ZRZZoY&dx$lICB0N#m7(d z)Bk&$OGeXDhkJKyLdwgX{!#+$lD&HM%JhF?hS@s5uf+ded*_Gof75*D&ZoGP{}$`c z9rLgMM_q9zi}1g`_MQJ1e9$b5O*KB{?wbIQnkT%hr1W^x{Ka}_ac~Lq1k(QoIE}ro zb=2dSSm)A{JDBJXXIvHZ+&jF6mdRoMchL^!QDlNc--zmBv<|$j{&Wd_ojwVX>L`(%vdQa@S%u zcrq-a5;=3uEcfW&C;ZS^x2*hgkE|WVCdge9IyF3eB_WG`-AnIsouDnH!kmcz@1>0m z<8C^zsL7BJcr6Pp<@n9Y80(2;Y>B(|<>~B3f0X>+-V~P(S~|70lZ}l(GH`D>)lbH* zNExJKBy@}rM*z0x=75E&X=`|3bi{dYAfTh0nmv8G!e+U|*C zG3SfDh0=oy9fSoS zDo@AX*_)XH59%@>xi1SeR&!;pD|VfU{n|Kbjn1NBUl&;j;Cl(R>UEHGzk#((+we#j zOZ!(A%zU+3b3Axg+vrhQci6GXW>N5cE9(;;hu^p-WPXsNftEYh{_!UwwpdyGfL6mC z1RvJxg=eU~K>9=Xj5)Ex4a=FOvBswPD~6}1rxg&jtU;!eva;x6Wx3r?cL}fpf`fG` zKkB)eg|*zLsra9Y34h*&z`S~6jd*@5azCl7VB@w|3l>Pj?!Dd2Yu zY1tw!v`^Ia+Z6GM@|?TzP`7UP|CQTbs)vm|*7q1SZd{m$sl&_c_q1XC%KR)>bCCJ| zj^GhT=I7m6E{l^^6=zcN)7FoPajUBJ1J!6DRv4N^8-bMzF+VWut*RgpDD6$M&q}o< z@&N)}y@)aD{CsDUTe~cGqIPyRRvQxf>n_@T)yp{J%^Lly9+Rv1iAuLIqw6Zqt>A@| zT!+c~pYMB27+zI5I{xnC-G(d5yF7RaWG!`{E6{J22Jb~<&$<#45_+5;x?Y|ZDKnIn zme#_K*5l(D1qCO2dwa*nVE|x^6lhy~lm(fo%wj}VRu)dj# z`-+`+rYXd}X4P+l{0vkwnqi2{>W*sm!!j{4%BKE{-hT&FFL7)JZDS=S8&-&d0-Y+mC7@0jUk`goqKD{__Bsbcx(_TzD~D0*7{WGNyv(g_W_ z?C)?m8L0-JN$(N^B|nQ9dC72fJjTi?9wjBtThEKJxcB;$r~r~NYJXNA0q@gYa&o>IZY}FQE`_QT zulQsZpyT3VmnCs~n#TM2{bW{p_#}XS#?6upUs$YL>7Z(>m=QKJsvd<|qn7TtP5uj|X*-H(inxjb=;0Ye@H z=7@MrO5*3wnY619a}o8ZSXd@S>#X9j0?ym50R$@KUSLVzIZlk0Tg693ML}7xu&^ph zOO<8*N*w9W?Wel@`WdKAc@& zW-)ijH-Qk`!esV3eO0$i?@QgJG`~Jv0Aq2DV01mt*oAK1`kesBJl~Ps>|$GDolc1i z2>~AKpsmAtQ%24tqZ(RI`-2^<06s68HM_Vy0XsW{Aez!U zI+(p0%Ir`I>>=~Lo4vRv1W#vOfw+ysxaUL3m|XCS=!SkIqCJGXqO2@Y$jzxem^7MI z7l_7?$s)QoJxM}tC|G2Z)s7aF--3g!hjNBKKSX0;x08Y$XD8ipj_Z_p{<^yT4+%t}a;Ii!Q_BD*F|H zBXB}RQ`Bo+Y$1>b0j-)9)``_|NgnDAiR-JY`;t(~FdSRbD=!eD>4Zxht8NCRa92nq zb4eqM2fX@*0~XnZR#ysRIJ~x@tHxudZUJwj$kgECDlJ3HxH1ya67%=wN>+$e1HS0@ zv;C%^?9TpGgI}XGIlZLYL&{}#-q(`zHLb|Ou}l(5*V2p)TyxfA1i1gz#fik*x0;%o z(x8!1{c*18cZwJ(2?=m!Hy~*lDg*-qcH1^RJ>A>8wL8}g;dDE;^zyoypVya>>E{X?v1BBGoCRkgQo+km2R zb8{1bEN+|W%lVC&S&fdReGFpmL!b?{7%4!*XUdYh*c>YXVXLC3Xm5A-owgrrN3?EZ zc$9h^$SZ-4ZfkQB2m`FAYqQ6%FSTSLKpNTHys-Kw4QT>Qh>xTXXa=tHZvTLZiAjRF zf+urxYXG|pyd-^KJYXuBBc)Ax4vqy`<~S-bm!}_Ntwl-Y@==5kSWH8NGiYK z*K|L020F-3CsRw^(N~w3!1NOm5_%T)l$4Z~`w|7*jyC}~=jG+?iD46?3CI4End~{9 zArsxv(Sa1cupBEk0x5O$R%3Xyfq{+=q!5qLpT8si-l~s5%3tb~xC!hXs;%wl;6Rj= z1cy6qjUOG2=-yl&AY)@=fuPg;qbv+|K3Qc?z^u8rv?LAz4;%xZ0l_kuOvv2Slou3H zDoK!MxVR-Qd-I^6sgwR#qz=R~Y8OgCwZ()#Li#;E{$jBc7YKoeWWX9M?7<~d-a#Oc z^vukWi#Nct6_NzUdU`g2Krx=j=6WqNIhb5zdeooC3IErxUtU*7BW*#%SmeUME-t-p zPP`xvJJWRD2c&!vS3uWR`E_=p+2yiI#Q_&B+^fMgBiUKK?}HlX?i${)bPIc^MlT<*1- zCoJ7(5Zb6YM>-5uk=s%ge1likR-;7)RgYD8o>PxFo+ayDtT-;?NRekIj^nyhMKa}E z=^So!51i&t>?V2AKVLP^RCHPC6ARPvS12%Fn77t?&%MlFb%X=ofmYEE{T|AW|17UI zaU7JF5;a|$q+>KW)tUE`z%Lk|=(d(>bn|at03_k7SXmWuM=R zzrPEA~rrgK7^3N`0D%!1PY+EZJx`b9JF{K2%r|YEN6HcgwNrLiHlag zJZRM|asct*<=enpa^c`UDjHfQkTzP57J}Ku4pquVfAS<&#H&6t(*y)z-~~WyY0g-C zTMcrWz5`HBm)fOHryn#|vXzIY!XhJE4cWEI%vGFk83Yg(R(j(rEXUYE>Y@^$XKuK;zDN}hI6vN+C^Bg0vKR)`qOyt#Au+Mv*RR2( z0>guY4*ls;P?nSB-uR$8G}d|usM3y(j><|(fV@XjTKb!4is>%1Du(q)!BjY{LXz&} zUp+LN*r=#sFksCx^NjN^V+nlrZ$rrPr|AbzV%}60{)myB1=x~tW_V`g_uG`hA7|w8 z3$c8db{sSXMdAd=5D&Mt^exTGlj;a~PFTU@Gb~b~!h*tHqauwNwuMK}LoY*vSrj{a zPaU>T{Tb`?C*1_)%qNGOBUDtnSb16@FxI5*MvK;)^r#dy-*cgM1*fI-?J2K}Z&|w? zNh0kwCurl4nx=wphi<{iTNds%Hm@NRQn~ZEXzd{}HI7BIg9eG#FWb;L z8C<;lX&qDPX{Ptil%eb-j$^=I#tZ}fmPm0y01sUOu@;bCInkib1>UkbRdbT3QF0q3 zKF+=d5!c4XMkTSD1(ec51Yp|s_xEugKLG3P^zoz1{-VF1UwZ)U(2&RZeS(m7i^7&F z^P!xkrY4|B&d~vdxQORvC6LkQL_=9BWTIKh>fR`4j1oLR#506<=4pATGDIGAt?C!4 zOGso+w}H^pgG90l3t!jl)Rz<$tpOqZ77KnrMt1gDD}nB0t!oJg(#6I7NF?G{iS;HE z(2ITx2v8A<@XrF$f3KUXiuKw8ow{o94m~|R>+9>_ZF6$oRyWse;tbW%3=Ivj#|E}9 za2?|xoVgD@*r#b`Ar6^>rphStn&(rd9U~z@puWwLm3t&C@nQgP-ubNb5ror$0s{GmMOuzn?}Rp9JxFl2rUWb z@T|=!AoMG4%^W^Qd_PqnZqTp1E1~Hwa^1N9y_JyFFH{+3Gwb>KPdD3= zp*Yx5hzAPVR62XHGy7b)+4G8Bt+RabC)&|Cp{Ik(cpI+Q1DBU>o#fggRR(fRzq9Kc z16?7=D`i$;+{K0{)gH`Kvu96sborbZhWiwLZ|2@oE5G*vs0Vnaay%MvR*+J|&aj)y z6_G!G{@nTzK%dtjc#j*kG62dohZXD-h?V_WQE`&}KJ$yH9uO-Z9v#KUh7}bRB_&OF zMKV^{%*b=KE>$mmsDfU9=C>4bd>{1f8;*aH5Nma5X=yoAZqwYyuI_H2mz$!k02vV+ zAi1+qQ0U!UUpdMkm9lT0)1W>dMd`08{a1X2oe2yo!(T37r1#ZE9Fx2^tf z%kOp9V(EDFdr#q9=IE!QDiX_lq^^HHHO1uXwpe-`pLcl2A*jqGhCQ7Ot*-9dc_mq& z9F#=&cfawec_bv{Qt*875SgMv zFEJb_vcI{WuRHRa?=~~k9;~bK<>%!ASOCf}fuZjGeLE|wEdUV!?aQoLYT6b^IFPAu zvcJy{J6v5!_R=|N92Zmm3Y<@`=?gIMy@mGNa#IU{TuckB_%FZRyMO=a@X+1Eqq(UG z{0nw3`1?yKDTO5_l446`^e-+hf~?h`Us|0#J~}!mEC%)`0LC{91{(yP3T&R5ni>TK z1#-9$A~Id?;np3+oKq)DQ=%isEhKaW%pPDo<+6%`tSnQepte3yHMNnBj#UsoYQ_u|Z}4A8cuBi;9RC3|L!T zH8nJh2n%C(+5Iqt@8))KYuNz6fK4&P+?Fo+=o(B>K|#IqtpREU(AeZ;1qh&V$7)>< z)&??nrfMo2)|6PH1wb03Nw_G~YX*EdDDOgJW6!3XXFw)B+!#S@f09q+2RLPO({iXA zEO0Wftv;}4K0ZEBDT556EOcC`3aUE*7_meyKOQXi1`+cBlE3ytzO8`?8>rfW6KYpG ztbw2l)&bx~7Qg1A!a_+InTriX!>uqCRDk1Oi9xU z307H6a8vG|oFsFYU}CSWtz8{Y*r=+i0-ZfbaQP>VVKo2a5(mvd3lSZ~py|sK{h~_y zl|GTn-DZ$11%-r05V{TKLkzgL2iRcn9TN?x#iYot2oHiZ1pIX^#l7xn^? z=^u*J_cC|(Qti!vC}}#rr$CljeGlEgJOwwe>WZmSpZQDJi_1KA50K)VgM03 zDgPmW>-+$l#jqJ@@<&%#0gD4pp!eL6w= zo6_Btp8z!v0Q(>tWFZhU{Z(0aeSY17{nNdLi^DYo0Ji$qYK1^*0POf}F!gZ*?J;mn z+ezQP-0s_yIh4zQA9HORP3^}i89(OU+Ryg+F1zs%Tf2Wq5_C;PAdqt(!8BB?fJFmF za!Pi)D`j`b`+xh5Y_x7)li#;uOV_3A_LetxJyG(vzx?kI=KTLZmleh1`GHom4l>ZfY@Mv0Ay7 zRzt`s;&QBe_u8w_B2(>bMLXN8qtow5X?M4!B_#`si{rQ~R6%wgd_D|_ zCh*GU=#R;TU7Z$rO71(g%WS=s*WUF)yM*m=VCL%-x_HIqWpKEOkO;evPEYGF;q!36 zW_DhZG5LGOKH>X;^+EU?h1OY!m<$i7ZCjU5R^8QmL`i!{X4PNYh$24_#mB{MriJj; zjwX41Khcl$|p4CK} z_@t{_T+R#ib5<7=0&8|Mq0Zx&JV-KqEM>Ml=o zgD>?u(?jZn>LielFR8QD%Mgc6&I`3Bo071B4Q@PC@sA%1b8{__NkR>m``w^`s&fI<7NFl!e({pocYwN^p zwjjoRNN%JwZF)R$0zAF3v624$Plv5>E@tK+_@dnzNb?{n0pI5~?ukJoU>PLURaPzq z1OV9idAK=dzdcz6FtM7Iv!kPSl^r#9Rb?fUMsfNyfX0#Z^z;Kisn<1h*$ zRz&%xuK;gG9MuA)Yezh(qnxqK znWes(^mqkqrInqMGFY)D7*o|MaN40_T-Wo=2;`*c>1hy32L=WNoHo_B3P95Z%oeo4 z(_<({MJ9@g53m^fOWpqP=D9Clzw!$R09gU3HTx3y!r_2A;&a}H0_K)f!07=B3JncS zfo9oy89O1jrMjtUR%qxSIQC1RF#zI@y|n4!L>Y*lo}QDF6WAXNb~V$a0wdZ1 z)H^USt#V7VfuFqqNjvS$iw|{xvKxe7-5Mt=fSfFb^FYI3FI_4eU^@6}#f&E)O~|mX zt}k{1@R@UX#C@EX0E0mz7uPcP@wUYcuqm@vIiNDgbN0Nh&sIQ*064=M#|>40tUi7E z1gg0@hyQi5K;L%`rO_fgOtr(1iIURH4D8z3PDYgE(_j|sW3TZkT>R9Uuy&%cq@=6~ z`Nq;S>NW?9jee92hoieUKm}^tCBEUaHc zs1PLrE34ZEV_QErH3SNnu|va4NJT!J_a)zuOxu)~nLlWl`FzxUe&7#>6_((r%~XZT zd2C5`TVJmK;Bini?_AmkJ>JvPQ?N2Pp?-c6U~elxX3B0rlmt{f04LfoGBQ&3|NgCB zZaJ1F5ptVDiwq=$lM%`cz!q6&W<;7+=_ z4ctt%d?vskz`kjp8Waqo#qI6ur~|W9cH5NCr{|aF6K7xFLgtm&{-8}X38GINIhSY8 z1eXou!{IRJ^gJmyEZv0d#iP`5+XV^2+DHMl)`^K4r-!zHOarwrfURpQD-sZp)MGf! z`a$jiFe7b$V3L1 zc#rgtCe{E{R+0R7f71N}Xl1*Q8kwkixi62&l>4X66pQ16K|N3n#v?ekLY0*C{H{wA z-j%^#)l-zZ{C=P5rI;8OlNM9G=M~vl@Ai$Q_)q`%nuYD45u3&EGJo?_Co78_W0IF* z!jX|&OZ0HL13^rYytua2xvsUuQPo09I#&Mb+9r?s{Wy&pnw3tQYa23 z`b+BSdNqE!ySKL|7k>gm1n?w4ZRG5rgQ>p(z+PHf8Whk#f&ggMO3*+9jknR!QOn87 zA`O$r7-pM&f5QY*VIXDD@rydx9Kzk?gu{8=#c(-rJ*|ZJXqi6#!g5V)SGZ`&j4QUPeYne!g~%xT9`#R1|=+_;`3j`R{X& zLHOg=qs&m!P*So3Ixa9D!&5q%nmGuBi0i%~S%@?EtxvJOU&QH!!Zt0(^^*(bvbdXe3Ro2XW zp`xk`{qxT2Q`8Rzd4EVM%89v|+1S`v_PM|aI}eY|$6eq)fUT|vXcooi4Fni{S&gz* zgmgxAKMGflRx|7d?3QS$D7KM#LO)W{SzukWGA~e3($O-q?mNW|cR1y5DJHtwy20qr zFH!Na*>6I%8wiG_fkkCxU|?Wm>d>epO(Y75LvPMrbZ062_gZEg2JH3m8+ z&e-Cp8#N_|3=eNb!~B4wn8rvk2I>D=PVBH@EPlb#ViL)p|!n!)!{Ee70H_tYbY9N>Yt`SH+ z=nseIcALC+(qjvaHMg-Y4o@qq&V#$=~&Aw_fM&Zh?BoMwB9SHgv%+vOf1-l z>=5Lkg0=5&3v4AD9HQ56A5%OOe>iwJFV(@p+4CW9y1@-invPB~MOpLX;-@09w}eXZzNZ7a42(3NogA)p}w z{Ffo}mOc9N7H`2gD^ASehmHR%Lp`Ptdo7GR{B znka#8iHQlF0tlCYWbqjuo};6qkdP4lGgQCkD%g>^mX=mK&MV-{oJ3oIiUQ#xdcFId z&dt8IU%{XyPeG=QfW75au&iHpGv3n!t}D;olJU9R8z}b|_*2nz?6$VW4!N-LjIRv3f@u44B)c0+Lbw^Q@?o_qI{_l#2cR#5 zMJCvxUj&*fI&Fvcks5Uo%j6n@rg1*8C~rET zsb8~;l#T)_FR;kk83A;op3;aB$A{if1-(|fKNMG-)=d7E z1{a+mcpwW2`^c(2*`qi9lwA}OvoYxY!NqDiy~kf+xDpY;2>QJt;RBs!66* zQ9cY$mIuSMt(xqPW}T^XQ-eSZL6QeOSw_{o*N+wHB0wCfbwBHGZ>OcA`tth4E#L#u zx=j@w4Gn+-FqzH5=H~w&s=hlA%Xj}@lcF+8Mpjm`k{KC2AtI8QgvcHZdzTclLP%Dq z$d;Lzkfe}=vZ=_P+5BG5=bZCB=jV@4=NQj(KlgoI@9`S%%Nn&L(hR;}8wA>Q=-|O% zCMjC->tW2&k~}<(&=d#@JA6q|z$O6?u1EU^@=qR~SHNhl{lXXGFH0S&6|Q_77uM)Y@aR+^7yi9>|n*MeouVzGR!CcJsoM(Vhg)fIywE<8=XR$yCaM zj3VYg{%D^T)GpAMIKxD|A3W?lD~0D=?ECizHS@(V7MOIeT3P>0%vmhTF#u(Or|IC} z;0bYsW|wVoQc^!2bbx$HW+v~Z*p5K?*RNl@gY3e9jHJ_UCmlyyK~4<}-v=>y@ZiCo zDzCVP;mi8^wR3%Y@f8>-bXHwL9OAB&&x4qRxvhMlcm-x5zZ_}*xS+3-3&VP4l zvdUdIt^WN!|N7^x#hupwo8CkG)*Zv$$rk_LuT)YH{%?%TKR1ShBGmFoTh6Zo8?+Q9# zD8!Iwz40qP+OQWD_J6&U$bNtZ^-A>{$nzeG^Vd>VwARt zK*r`MeQa?wWJjpf5$2q#zr}~e|4++Ie64{<&+pfEt4x)D|FkUa8*+You^zwd%HwYy z3Yw~RFZbIg?{a^?scESkpxeqgS4v;7o0=*`^%O(o!K$XmbKWn9#(0$lbNDph$tY74 zfB$`JOWEMK^vU&y^AS{f)YgAE6O{$+v%YLTqPyPp`_AFQR59uNAYSHMm-J2#=}1z23BJTE&UI%1&wS)kNw{jhS#U5-lACK_cXruGclcMvoV#-@T)KPIYIzNW(=4&lK+l7FL>Eu^MblK2RHw{i)I(TRj1v4!p9ZyD4OkPyPKia!`YK* zR&7>k-mAaPZzGI)i@B9n4@Z1wrt(f0*%AI^sPN-Kh0QAO{@UYy7gb-q-ZJy>f0d|w zASz@ZX^LaX*_eA{lzYSY&M>RB*Zim}jqkf)`yWr{KiAhEX1HHkbN4=-mw8h+5{lU8 z&eG|cbTo34`uFymcpTd^O{N;_KJ%*lRWs{J_xafGNsWY;MSEkplxo|x7%NZHYMu{g zW%s+NZ9XP7N3rAV-t$wohdz>h)tHNK@8XEw8S6_)xNr3)i*waovd;fhEIsE3`>1ij zQTv}?{XV6od3*nu@BI%nO1z#sTfT-A(!BbUl7D_b;|_WBdDdTY8)@ejJ*WBb#G59y zE8E#f!^k#BMv~NJ%=TQ)8XLav^43I%DnjGY86|}A%ZmEDMJum6e;a&5A|to~X)Inpr}|Kv{OR7}#l z6aN2xcK=+Gch(Z2g_@T9_qehtPhwzO?eFdjGErw@EzUfkLEKP6&Xxs47qu?TIg z^}NY#=iA8gDGA49rAyAY6YfRC(an1+Jj^({a#fUYoh#_RNYrBm4R1z5Kvvdr6|bX; zj^cigdf#o{kt-Hf)A^q0(! zv>=Jd$rgd}uF84mLXl%Ivr=r|`{RE?y}Rf_&qGR$fi1G4b*ahQDseOQ?tYQZe}3)@ zN>x#F-ACcRT=;}j+VSIdDb4e3jj7viP!ZBO#f2)}8{0)mnRwL4pP#$TZ*XE~vX&pE zb@GaP+e^vvc$;KHpVgNNiZ{k04!z{`r`@D}e_rXMU96BzLvxAvCB5@p&MJ0?sA#Rs z72MZOxk@oFa2ZxpU1?3RE;U{`IqAn=>>ttA*8g$ps)McITquvF22(+uwVX=FVtkJK z#T_%fcV<{f`q@iw|7Xd1lA{RQ;hx4}uN{=~s{SVD-fhWZI?dVS7Ac00=d?aIj5TTE ziDm?MCyopUF-QsMblvl0Bx9?)c6y(BMiis9%xCKk3i7l3qGo%XoA1SmTi!T@pEkDe zMYWdX6(^VqUrQft>JH!cl3gQZsh2{mT0mGb*`U$&uctq!L)y~}2Wn4)1G|MJ$nqVG z9v`Q_@KoBR@8Iy=Y@Qm&a3RSovZMgrp5nRZA4L8R%nDB)q5W|=!%J2 z6%0o(YB#hzna*o%{wN=q;OMo*QWYCgIsd}9?^mg{n~^T@gD&q62nzB^*oPhsB^sJ* z5V@$(!<9JTJ+P(yhX#F>kpYscHI25(yui!LtG~ic7*r5*C%>rZBD!2?fOnAYuj5vf zmp|?CR~0a!j!tm{=e_ABz*?9E0X;7t7uN%IfF;f*CPSdx08aScy9a5n`^^;RuOkNJ zlOKr{4h&2XsRjg)5z>o_!04Sgbt<;$7fPgsOI|gHJFjGGqU6@oqf2UmU~6mi9h5cV zH~Q*+Xs;WbhjOsF85m4tOiaw9N27zAceW@ee%3XIK$Z~^L2X6+`csv2m?RxT9FH9% zlR5>^)#3JSJ-+CtPZ`3MCQ5G-2!z|WU+3fqIG0a^LZt*u86>P{QFTf+du7-78_A&- zGb*1=j~`y*aclN0Mp+GGCvP12aeJ5&x%T70E`f9Loo>tfBuc;kx`r(_of3LVfoaX+ zPt%sx{f(2Otu1H#B@cG#-0XMy?A7VR72o5(6%*w;xy0EispA{v{MoKiv?TOjm5rV{-tSJ1xzFbq^zS60SEONc9hv_1b~ zNyY~KlVOiN02<5?#I!qYEiIW5eagC#`TUF=y?g>B+bts=9v+?uc2A)j2jRA1BC zipOGQ#S@~4J1;IKwmJF^c(ZHy!z7hb5D-7UfB*bcbX;?3*Ls67ciO;XG>>SGdOm&1 z%FWnG@)8woZ(|@O8p|(WY$auw_E1pRLPK`)B*SUfo8nDj0Rc+N%Hc6FNv9l$-4-NN z;3bojuL){xtaWDY5nt6&In!#`5V6ig{^Ps~TgyMT!M83ZzdlAmScbWg;4ncOP*YQ* z9l^3Os&EzR+f$Z%^Gy%f^&8(`zqpaPZS_1y^cawnlOH-1 zCFsbH`dsSNsmp3=_9({>?tS4xPHI$B(keOGU6PBHDkwNjL_DW@aGfmT_695+q@L`+ zwMBqr3dF==`7tnzjf{v}w;vlle4uLAz1{n6J37jff!2o>7HY*^yLabYE_m5%0+JGT zq4u`6G4DpmQMHaA@=!9kbcvAqMN3PIbs>1=!2LujLO|5$qKAo*?!U?;$E1pi^*f-h zDd?W1rImp60pScj8Bh;oQ5hM>+1c6sDE(^#!@>lyKcH3+6MJtcj5~XbpZ{BXJD;H7 z$gC^OQh+0l_XAfq@0bSvZIuFD#9w;E;Lqo$k%w&WJ9MciySSjdFLQ2PhXCW(V1dI-^ zy(~3{W#%YZ2{i{HIha{mUsJO{w_3NAJdgR(?b{eUYX*e@i?kws3yT@r8!UEoghYNf z3l{|`#_!+ez$LubT_^S#dAgm9g`U&G*!ZHRrUL|suV0_^y9kgPz51IsZw8839ciMX zqOhYbV)7t$AmnFIT;PdnYHLqsC4zk(92`8j*9;UB-n^?0?*#n*hHm+jG3nUcOXhr36+FlZ`v%LqJOb>#+Jk_6dlDlAv{5R7$EGT87VY;(8QB z**zew)6D8dMl9-^>c@ozN56~Zk^ZpU#ePCFJrR!w0sDouo!z&N4lz+tPD--TTBBsNw{WR} z6S#c&4IWZOMZ8+t$jAu0A2t@0C*08gvr%^@J7?h7iTn&xz9Ircv$RzCq^$ffGc= z9GslHEUqH7%E&Np(J?ZX6ICwZN~vF5PczF%OG@giso{?9Ut5Mq9cn)-Yilua@$&NW z!?d&}U!m3}imgCmCi-|iD~XDU!M8IqGu7iIgjiYcD5g3wxtkY!pPHJ&hFv zu3a|5-C5t*Sn-q*rIM0!X%C!C5nHbQeqm9O-!7co>ph21e)&e?}`KnTJ;GK*;QzIN(}@MkRLxbD80!> zz$QKikN^_)q;+jy!Dm4ouS!doA!6pEC0~PZ{4cB}EnQttT1Eg(%;5$zsnED@cbr{F zNr23Ymxl+o9PAq>ZU>Q3*}R{_NA{DlIbz#9c|vFF4pag%d?=ix%IjS^dSI?%2oW=> zBjZ~z(9k%O`h{$#Z*1&k0%7~##JvP1ZEa$(g!&yhqU*8$0T!^m*nOff3qj(yoUjUB z5qjZw?|h#b)fjSH+uQ%1Uzt>6KAjCR;c-})x`DwFFc%g+}Q5qYe{A)sHFF8H^!g>tp*-(6-! zT2@vtWkya8ZM;(2SAgrm!NL2PPnTi?$Rx2GITGBGAYv5()hX^P2L}fV8aOa`OQ(QY z0qBW}`UaRUA4)F#G6MsOm1Asd8U_Zz8y)qM6=eI;>ERk+V1Qj3a;><2{%zmDjIE{Vbck$}gUI3Bsaen&z+3j>6#Cuc(j6p_`hWk#W z)wed6WVhcPn4S*Xn%LIAo$m>eKh6rGNl=?!CD}zv3QO|%__&mm)QzsZGw07A#|SrQ zkdO>coOnb?CSD6Z1em2`$B$z*LD_MPhvy3lA}6O_OmoJlPY7)zBHChwE%KVZ@Y~Q# zo^=ts`}q(R75qOpZ{5P({CQJ?DN^4A`a##zeSp0vC@5wE!5Ci4*Mr^xI|m*llY;L= zLA9*+X~O8r{D`Maby?Zd8e@p7FrC<^Z)valIg4x>!iVnexO$TdSo>Js;5kn-Ph&mU ziml;qat$jlX=%YnGY>jJ1Kb`N7Q8m*U;^fOPih#?WJ-#<0+lS=gh5ZieKmtDK=gZ9Hb z;47M%^N7?Ct1WNGE$srq62Yl5!$Wii<>#Lv``*o81p72e%6r!d;{S zQwLc8jfzzr(kn?V7GvAtf((GAPDfAA57iz>8&}dxnwpwO<{$*(Vq<4lJZqK(36|J6 zpwltMJ0vvpeHk6-SUg9Qji4l1pJw9|01ib&MDiYQXW}=kctS|7YiNLnh&n0#*|R*r z+t6TFxy}NXXUL@Jvm$2`xzTqTY1C;<%kNQe@TX6)!pQo0EW+6MWXZ_xevd%wT3a6v zs*i|?fyN8bKzeIRE_Y~trAvrG*c>}17!!tJ%t3*EIJ5PSqhBkYA4b06rl3F~V0H0` ziWX&O+d&wJu7LT3@mt(Qa9C&p6hjz_&>Da?#9K)ws2vT@BNGgif9JhB2p`GqluX$k z+Z+gXp>897sITP(tgJCj3S?g_3sJh;o=0-Lbt~V`4-TmF7cSJF%kW26chugo5b|KjjZeWEu1#NLAna`V6mL^@6+*r6t-J}RXIq+sCDH^5T&(vd+D1uZb;ZV zIvv<#3i9%h18+jG>-_0Y@Zy`=R4Y2JNH@6uQDH5xo%HRk!*bq27027K8 z_#`L^V32fk4eJy2FqdRva#~>X!-NS<*SU4AIuuA$`w5nidZW()IY69Ij*xa&?(KO? zTU*bCQK~AspnB9VOpw^2YDcAzU&p#JLpoogYB}!5^cl6?K*&$XNWZqV0g0cvJo)j@ z(9ThB+U?2N;-g8B|If!byDf7I4hRo+k_qhVH43q{sO-0;xfm_7u!dz3anl!!tbF@!bLuJV;IUW%_b;WsF4 z?dbT6Bo37>K0Oq|jt&ksCGKa>?jR_k2)t=xY;5e{FprF#fBi5*$orQVlLn<;enrf=qYl4X!A@F_=Q1 zSzlL&cLJl0J{<}&D95*;uSbz2kkMYxC1ce|%dcN&UVwowa8}^@(V6AVq(n#4(z3my z3akTNNJ>h|&B-Y$CU%^M$I8aWr>*#HOh>-cJ7%YWJyT!&i;;|vsO$-G@;KaHzo+%H zwU(|fCQSg~I4!qhwSXuejuV?iCpXfPl$(`RXEZEFmJJpOa)D(ekO;z!SkOP~X-Vs=OOe52*3!gBLnhx+=o0_sGwQRL_ zPocg>M*%Lee()wf=jt_+Tdr63-pQGf9UXs2#QnBz;RbEnw(ZxiU$_-|DOchVW#IEf z97lmU+kK;2SxE^OK76+V+J3)_Dk|iEneEG|YV$P;;P^XpG#m{Yzl04mqFvjuHVNlU zYeAFq8CAv0u@3l98`tlrJ0XXDA5O~gfNSUQ{E!73hJp9I!n03LfBxcywAboE?+tJE zr*UzJ9D8=YgCS?h%h=MgtE1yVrAIeaxKevZ$M0EJJ@voDJ75$AsDw7hYjgb+?VO5| z5+0i@vy5jzg2LGc6yZ?GdptLp*+19YQ^!4-5D*Z6b`nhJo}CyyuZIINaLeQs6oP_+ zP^SQC!0P(fICiWmbi~--m=+S&a8ID6|5<2uBp!IIYI9Rl?GH|_$d&aK2fs&ncQG}t zzf@kM_a9P@VTYDEA4d(4XS*k-7KpD<~@N4pF3;;N`Z2h#mb6)-5m@Y1vt7^QeqZEPHOEf+6~I)6$5FEjSqgFCy$hc~I%S z!Jsm_O->5fLlD{=W#tt}%n=*uD0YK82GHWp&sVQ={rXQL{yyCg!_5U%Riw+{VAb-z zLmN96+;=~#6ev42Up~UCONVu}oKxww7F}bkgv^1)7-jL~YdcmyD0NVwqchUKco7;Q z5on=cTOM%04up22yS4QMkX`5$i^pFsVL#(f!{u19?Vmn*GCDSvk)Ce+-3jFrIBT+< z@6cG5YpO%Nfv7#Pg>5bxls}93>bU8k(8CKoyRod$9bve@ShZLvhv7 z5#2BhDU7m;3Is)8>*}5Zi+cWCURT!zDoD6U5_*TqEj8ed$xThw)YWY;FDSQ+`Q4JV zReL}d*ypd25daoMn+(cSWM4+WYcgJvK!cARyLj=U9_+=P-@VWHaB*@L;l{#6f|!8S zh}NAbU@Ex#>Q<^^$VpLAI~$u?nj_bkUZD408@X6{w3eb_!HJ1v`TS5yt!!Cv>)Tt0 zzg#C1K~1qdO+|6dvkA_lC`iJQeY~vSZTo}|484}Uv2h->Erz*_BwO@Yim*w+db^39 zVXaI63-7JV1E9pNu0MbOMsHMZN*c6DLTyhn6Bc^wr?Vz&D^b6>PKt;Lqt@>6HHt4H zB$5CnZ)1fbwe8sx2qVc~Cnx;Yo#{_V%c##V&JNLT&7y}MFd2!#VGVOKIb!9rC4`6r zWEV#Md(;wkSH+L+sU*+f- z2r+Nf- ze;%H#vOe@5wg^96?vsN$%g2${<-N&g7AC*EpzZ(j(@|YCd#j?!^zl-P@lpa@5>!J? z#Mk`yS0TIP^$(A#a(0mPKWZLoUO#YgL^+9V_y6*_yWOSlZK_xrnRD>WZYkG7hT z_@BSWzn(65hqu*a58`G+TTbM#{?Ct+c0w^h_Gd2GKhEtm^ZzH7Sl05Z${``~I%{cY ze!7m2F8F;)i&267|NOD%H1Tyke0NYdPptKH77_3LzrO}Nwi9p29_O3(;#4ig+x|bH zhRmLyxQ#|LbRshEje!mS|8L9DI!;gGw^Q-(LDkZlwP%Q%xxD`!Mn>{>377m#cFeZx zZK#zvYMPng|J8ilvG;{agh2N9&U?u-OHLvGxgh+zocPaU7~EnckYVNh_iN@AK8Kv{ zig=zDBUZtGe{e?~pMu2CKe0PRa(U6_-y$sH&)+Vo6z1}hycOSjKzum%ACC48B}%lh z@#Fvg^C~v%VM5wjJo%-(<=3rp!vFf1@d(ZTqS)|@GfbnEt-@BLug_LXjXshoW96O> zo$tG2V4rJ8e@Aeq$}GjL|5doi9KwzJ_v`=jPivFRf_ujaqiU0fgc`4ww}n57nWO5^ zCpJ7BuhcEDk9@M;{ik15MexS0H&-^_n1%Bf8}?=$s=U#?a^!mHSYFWg%xW>CXQE34 z{>I^st3A&c9UPZ_@r_PzFunb1FBkK_Z|*+em1{9+CHsk9*yP?1PTTbqP8^h z<1aTLmB2#>CT=^M zmcBj%{Gasn5VqbEyt($*&c|m9cLU~;ETd|S3p&X5I3{Lfa1b=j^W@}wDCdD@?6EgN zGa3>Sf`q6*HtSjeN(@bKX?FJKwl-=(HW85pG>$-JbuL{(=Sy@5(oj>oo=ya*0Iiww z^RTej_b5SKL9zRO6X|Lgm1N7u4`}0~r|^m||!S9|pS%fg6$v>R)gRFpNBf z^;B#Fs5NlVWfLOV77;P_TcGf3!tY=6anA%NafRUefZ_nlfnQ2Rd}%}FFej0xL}3O5 z^T+TRn_Spk;h7;KKiPUMj?No@)tj4}(JG^c^~E?{l^9_2a0GuXFKJ5rqjqH#9|;K; z_p+&}i3rVuR@}Kg8SMqe{o&&PJ%KSsO;uGM!p4LA81BQ(jas~_i*aVpz@^EvOb*hK z={k%aZA-DocwVIoxg}(-J>b#G?|Wq+p8g~@L~hqY6V>IAd++tFH-yiAYhFyuoqwuu zMOt9FhkWMESk<>0rK!|{osTFw6Dme3Z@pX736K|c=<-^R@4rxerH{M!$gCHpk4I0+ zxHnI~?3eC$uyBzI);L2Jp7nRNS88=floCTh%}C#UCsU;W+x%~W13y=f za&OqL_RgJhbDTd?dtk{xTdLDT+(q)qsnXJ<@)trmaQoexK(zZ&-vbT=ekI-A<3k*9#q4==q?rk`*07>k{lH^l zCFvL%Zcg<|c@Cofl$EW3n>TpF0% zbz**p9uau9fI;b_sHj5>3=c1Wa}hdu5*D0i7mr~&Dz+n>u=DiCG3$|8PHL_Cxk>P~K zGQH*O3`?gL&2e4FosVKXC(y>*BJWO9Qvl;Wf)EPai*a#DjpC8Rn$2H*dcDS-{Mdqkceh zaCz9e0sO^?WC#Q(&Y#A{0(E<4F+#okl(?h`xm$^fN%>A`IZ|U~w!xp)CR!X0^Vu)8 zc%RF4a#OT^8M&ymqP(#FvoP2^WKZpSL2{D?XVm9r&>vDb_~juf4md zWqY$xMN-Yzg!$?hk(^WnC6CRH{)yccK?^bygI*&cb$;6AV(L?LLZp>vXQmeGC1na6 zYQbsT)0Ca&G@gnh%T+Oy5u)&;l#*TT2x-k~@yUFm8|B_I-68yb#OPJ&>8(+ZWv4P~)xZU+$@z6I5(vJi5}rKi!ssxhES1zRlFUyF z3q=5S19K({^fmpw`Oy@>ABo2BD)4aT?;|%QfJP7p4iMR~8WP^n+Cj9wM87`XR2pVF zAZn5V^OM9WGNGaSJ37C?0@B&(@~7>EFF7a@bxgUva)ojHFC!JzU82qb+nVU{50r=L z8DLax(j~OQhL!G4(4D*P5n-^yHjH*@!gMkjNHF;PDu@L<&kr88*{l~67kapGY-Osv9b*ie(*D^!%TMv6`r%Wy;2=`I_$({e2+MHCH zpu&7j`oj2df#1darxwZjo?A0U+o)z20^+889vsYgdO;{s{&_}e6Eo>m_1-`w)l`Mu zuIHj77K8UGr{^y?*()=VvZ*a*_V_;Z*{T$@5cSzzB)Ohj+sRVbW=g0y5Fe89Eq~>U z>QL?WA}WgmiB(MCd}m^etW%#yZyHk*sB0fsg_EaN-cbB9l)7Cl;eZk!=PR2>QtgL4 z#h<*Qy5{Ik7`^BBI!v(VeujZNwe+Hs^x<0t?`|m6?d+e_xcjNy81b^^m8u`*@euO; zL7`+6TN7u5-e~9kS>SNDr$5Z*e3>>OQ9h0P%bz{BzFaG*2(|5Zko5gBH+xBL>zkke zO`SK5=k=sG3u8-1PXhz3(QI=2watpC?$&ETy;*dlf_aLfqDuuysSATE7>^JA(KZ{K zaRdNlU{UdEmhUUD#*-z(mkzTr<`R?^7k9L^5q+uHy8HL;ZR_k*zi@##JrGKdq$H!O zSD88*3vD3OGN^FP2X62eiP8mR+C^gs~_kIa2XhNqMjg$Sgn}$ zm5Kn?ED58(;N-_37_1GcocnPiR+qp@{Pkf7F~4^0i$Xml1#kzs!o`8jk5uhsaHnMi zYa4O}`|q!Ju{q+<3t;pZwE{}`)$`{+LEdzkpkankh?SKjh~HVm7APF#K0t2&Y5?Xp zJ-CPMaYBOEP~9G!7XT0#b4;B67L}Hj?WaGGK107u{g3d%Ln%agc50DkEmfGM~Q&P~2XluWC_Uw)z z-?B7N+J2&c$Ata~4p&|efV#j$BMNXL;#T*MA0}Z(VcG|p6T}}G8+*#SJ>B^mrTZOh zGHgs2e0-=V#Kpx=AH1Tcw~WdgDRJ-Aeds~|`e33CJ$>xl++WjD2Z?w<$skh>`6ypT z3_x~$QfO#~TQxm?eVrf_B=Z5OZ1qE}#1SCnh?7#8& z2S0n^oRN`|k`g$uZv2$nvQPiGID9M*T23-lVpDBQ0v>;Gt(x`U( zQn`P_F~&!voNck{@Z^!fE4()(38P4vTF0I6wDIfdkz&uufpbDmyL0;>`ku+xb085 zy_Mc;!kU_2Ap$v_@2aE}fb)$1K0D~-EssVFcmLP$FiyRCgItVx0+@V8!{&&-(ytYz zA&~UHSmCG^VdZYGuixeI*tdTTW1v80<(&oFcUeyl6Z5$E`72@UT;Im{JJrq#>|saf zHOzrLjkMV899=uuQ|_X!sYw%-eeVXXL55d)PzkeAd!r_W>?x<+F;s5X{F@f9a}k~= zH-)bDkk8pHO?POJZrK0gT%$|r_<2h|{gE2)IkNm=8lziU@72Vp`JUF++^dSRqOVud zw#ob1VBy3n=GbLS&CYvBo#Qjvt^OfT>DYS=K?M$sPE3E9t44qQ8W|t1k|U>5`k6Ca zW5Q11|ETZ9R|nZ1ZN{&|C&xGz_Brty!=(r^Hs|I{LTE?uV&-pE zx|c^AJ(zB`*DsojjSL&nEGr5nHB{N2A!AeAp72FV`iSy-2i>mES1alaDoJ_8PhR?? zyr8AXP9W1rQF&lH){J{SgqHHVCf9tNZ*G>0fy z3ciCHLZo|f@fdnwEQt$1%S(uYYh<=mupC=?s|d9phGUcGBBHKPI3UWP@9nKu(E$GejF$JB`T?}bJEeiS3(ST)<(En%w!UHHD1_bCKfC4N-u^Snw2_3f~4Y*rnWo5pF4u$+9 z`~6!+Okc41!8B()x3dQln(Jm}(2j$B&ZkDThq!dL*p?m!ZgKIE`4v6A8R&DM6#@%} zFM{wEie?jq#-=8y2V7=17EVF*_YX}{re}^C0LpSi8SGBL3|&-%Hzi!@4zkI!Qszy^ zs-+3jQ6PcWP)i&M9iXZ)6(RJ#aqzD~JWH5jVSXH*kCK08s?JDY#ltT-=8lp(q8B79ennPI1TMK^VdGLX?Oouxf4Y zFBZ!{JdH9L=WfS6eJWE%PUwzzP6R^$*E^KIP_43fFDD1>_)-F-yrQxaT8rT7%`nP$ zk`e$$u6d1>7gYE1J(NF6Mdk;p3<3ZnoHTkz^>!aR zI?%&2SL+KUId1$Vg>x45v)*Vm)(?tCq_gYc%r2Y!@89lyU0YtChKakw!J2bwPa$|s{QInFI0 zP+3}apF_&>`J33g)UWua%q>5T1H1DY;}`t>f!-l%j=bsP^T%T$-=gxgUt=24!|^Me zj~_hI3;q_(##(18-%r+Zc5L-&dTIK@AddEZ5Mf2hFx-!Pl$LQo&XPpiBb|Ebj>XtK zDOJaN=N&S|45MMI4QB5Insem>uAeZi(X)eIw6`x=VzDJ4Bc_FAnd@X9neDAW&ZyRq zhy3}^sVUd&tdH=QWbJ9R4i*s?zL?@UXu7LPXrknmqr3er>7{;du_2x7_4O82N%k^2 zMpfq_K>UXOE#p;@x|@!sX6^7SmftTG&OhVyKZrxcJV z$Q1e)Ph!~vbYH=QWN^q_>WRT2jf%+^^QQOJj?D8MJP-X31hj}4-S=~pI3fn9k}pcy z6#?FKN5%)vbTqtc>dGPaJD_=I*m#Mvcz_CE%K3diKQQP+%EBiw(g&^gDIn|R{JRfJiA86f2aQR<*E1(>&7bceg;a<%scUhiLagsw#a5Ts7F z4nLs30!U@t+2lzMV^N^INqN^Qz}zm|zxXe19tIH~Eex4-&38#Vh8_*!!I-HHiiOVX zE1k0`Q>EFUZHb_`g2FYH`}>1=3ithysTm$07|6p_0MXRd(o#ijRUSrGDX^0&P##X1 zt}Z^6uZsUXBG#r0~n8q(Wj*K`He_V|8z^}b=oCcbvros`2 zr~5VHDo-}{ zY!=#CI{A9a$lut`{P`7b_f~gKEVNlp>9c2`TcX89ZxcdJLb1#qwz^L$7oO>| ztzUW=6slafwy0}!ykvD(LgKxP@BB}-936Tc@0=q>d*pD%>b``;I`eVE}Q`OH3BUs8zby_1+SuolGbVU}g(0qBUS6Eb z5U@|`!8xUJp3|q-p;07Q@bL29z!*LVhL#WFEL%8eFoK9V>@en>1I$Iy6d1JiwG4V< zY~BGD=m96Drv+ihTY3t~)yDI&!A*>!LX%jHK{XgfM#cuq6aMhp^maJuB9%C>T~ONO zkXz|Wqn3ahW`gM_{tFqaGdbC}SsR>Tgh)W#m*`QzZ`s<}<^LfX`~r)q2s^i*$@9dn zO1>;pB?>|7F{uCfr2L*~6Q?m2?PVj*g`2n>0ChG*2?%c;gE=WF1Z8C&Ny)O;ujhaX z{=-v&*+QZw0*Y|0sC>Ok36`IF!t?ksPR{zTU+2+Z>1mbpmi(9{VM7h`8qqgzeNR`5 zE#G}5J3Ibp#)PI{7y~=dz~W<2WF2+?{;p^iE=;0DpN1ELoCH<^dY$%g_H*!W z6^z0w50GUyc=H`GKIPh^&Hz$Ct~DZ5z8aXO^KEkA36eftKFlSOPYsJ4Zm{D+r)kaPz zogds`MZqAYxTBJn4>K^b_Lt^5JbaU}`~44=ME}N?9{1Uv_l+k5I0Wb#qQA1F@Y!xS zzc90?7|dV@scksK5q8ApSVMD@`#0?yx7yMd8`PPviaZXcOe=fjyyh;?mf{@FvD34g zPQ;!5z^gDZaWST}3xrQ2w*+??IYph!>mT;s{FHwo@*9eN?&loLvH!t;_L|k@Rac8Q6qR-UJ4bp@Jb4!)H`Lp+io~#VkEF90Nm>o*MffZfX z6*zCvQw?5r=iNL#eEJF#+jE)jl}|>Y@3*%+f3nBhTUpz{&2;TT^O}pKzI8;`N{!6Y ztZtn0m^p8;q|}Y4sTT+nS@~s?ur)d9WGT#Ck*GnoswX z^AB#3R6`AQBcIXdqLalRMh)wqeCgb-8B6`6L#=sj$?i2HSFIQ0E#BX8YA^P>(}?pf z7}C=4#O6B5Ech1;yK>ew?+g2+5RkAcQ5LMymRR7lb;csCaV-5l)oegPY46I{HMi(8 z3^LO)G78%_AMxnXoe;VEzglO&gJI-kb~ffnPb_abVH4bjDa0(a1^{Ha zA_F-C)@40hm=Ang-A%G?^(354ix7C8oL@Ka5x>VV6%BFzAGQ3-y3hS=&GxX_=|4}` zI<8UtTIbmLF>B3;38{aLQt|-cEX+~Twz23fyZPjlL$gO;L8WQpd{xUeAqc;)L2UV9 zBN-YV2DNOlS(uTrm(lB0YAOvA(^Vi-7-S0Y2?su+$vStg0R~|t>0dN&kH4)rSO8p| zI3W|vhHG&Mx*?p%@~oghn4SIJMzZ1t9oS8HKQXH~U!xH#9St~G2__~cw410|y!LTU z`2$+Vz#o9|DAv#mkLj$c7 zm_bNS?X9ifH4I=NCRB`cB&Zn>8Ayrx`?FHae{n!3+M^@o*00ghpI23_z|j~`7jBco zd*aw2bh}_ufYamuLR=_(;skL-9p2Q$Bn9U#A!cJ7voc2nE)t9Y6eR!>QFf?sUH$~% z2he(DMa7erE9MiVWF+`}fH)B&twe}Ncc48-j1&Z)`uMS*_F2%Yy}e99aXaKJEG*8P zsR1m8jx3hu0m&V_5$<393xK6iX1R~zFqv>yFpwZ}ci+SOEux?oH4aH@TN~u{Hu%Ig zHaHC6E$TlYUbA)$w>NRd(|yiJWd{dgAcXMs@$q>sjvaDXD#Trap#(*+vCH-k*XM}! zIypH#1=?>I%H`eouC$mY8hD|Z$fW$I0G&Wm`%Je_7a-HGiSDWH1+5* z&VAnv7xWTdj*ndd5DmHlz@894#HzkO-Z)!k%*M;x4LbVLCEud19V-kbVvoj`xC<@6 zqF)2S#{F^j<1qX%~#E@^xA zQ``UP4|jj_FKJ9E4vQfN{hpPChEcTcZSCoi=T=Lsme0*hmzWm)x@oIc8Ex`2W2>CT zph~w*#-tHEA}+2Kgw+%bE*k$(p!<1FHVJx z-(^ism!MUg<59lK<8e7clMu_frJE+7Z}gb!(Kh4g*uJc_k@JSK9x9QLeHrq+x%^K4 z&K`I3!C_?hANTi5F;|XK=0fm9hkrv*M1->x{S({H5w8bgxew1gRx!5NDYE0R%eyEXz zWT#bXRK`hpk~xFGZZk9f#cW4PX4kM&0!K+mm^k^*b*-z$1m?Z|>>b|282D;MFyr|7 z+q~y*YjTg;^_UB(+*uA2eYh{(BwNAlMX%fuekC4Wz6>rJ1;0R_LY)e}RJu@wjYOZl z@;s^8&F7|@gh|*eHTcdm^EH0J-J#zc@$-j5b#KZCh+60`-boG`!CVThw+26R)|K$rC z0CXsg(B+Mc;01!6B5(rEfoR2hZalx+T;1GIcFc<&Y>{}`2c$%npeE}1idb)2_(k&Ah3FZjzJ|Civf>v(I)-;@<+ea)UX{rno?eV z1!WBa0^{=voI(LS69~{AGBO+!aChU)HNbI@EdPK+6rX;!QUm9C($TqsHHJ3(!Gql( z$KSq{!zoMAdG#*y$N=x&5u(|<(eph94}?B?ZF$*vTLAP`Z_3Jc@7i@0hXlfF3uxbz zr6UB&4Obv_kgvFVcz~dNo9JY1h>=rm+ zob$#FH7BPva^X?GDAs^#R2k+x9plH`E|fkOhX2};S_ zT26pI4`#uHDG4iWSEVOK;IjQ1Itnrd!n zSo(fpH1qQ{;zWE1L?(LPM9Lq%^=a1u*-vji*4O{SL{!9PfC~^&LuiXZzb|09hxi(d z_KCbnCSwhJq`pEcysIX1DWJOiGjyLM3>T4n9B$qulGYb4e1;*W>#OI|w+EiZH6s5^ zx*&SA=Ic!`L-WUvGKlv`9>v5oxQ{wi2FwixpCHx+oz=|6E#c>4>-`$t+~%cH%v(FIm+W+J@$oWNS>C}wLdn_gef37pabAI)c2V`; zXDtMN>a>!`DP4x|n*qEyhy@;y_>{lo8z%=;dw zw-@N!y%f1%vNQaL&gJ;P+tNPy=^w}@cFFXSdfe?Ld-3%T#qQLSlEaK!7%txZ_3K%{ zt0Ma9>IcLw&SpM~itnb0?*1t3938)t{tj{Y>KV3=_6z+PO5QZ(;;gCO56#{RT-ZkP zEav^xo@UGN*9!&giOv#B{&&VgNizSCkYvZ!l$)I2)z0LZ_S{}dlaOdJQf<}IB)tFH z0iFUK((Xqsr{a%oe-lz8_He6|>yI}5IbRbM{cR*ria~J zp9J53pe5VMYsS79U#YF)C2g-2D*9vi+VQ*Jsz#sG<{$Nq|9s}fyWWO5tJ{oc?G6T# zTzKW}k@I+uwd9)q`HhVmB-x{WwfPN&mox88w)5a;xsSuOCw;RwKQ>;z<&^7kenao9 z?g8Pz+#-+gtpl=O&EEN4(q5Gi7v%&yt{B}vTA+}B-z{WErr!7ZO7Eqv2P{gLmnFuN zE5x>p#_|=lZ`sT!UjF^3@O9m@yg0Li%Qm)*Omfbq%Q_0nk3GgLZ8iA!k=!}PDthC~ zN3*BY*OwatrB5U=~m=c$SMCn%fE;3(F_KDb=Cw%_83#f#tAM70RaT#Bz=f1ZtG;1d;Rgg1rW=4@l z?W>CV_qh1&B$T^IM$=1anBQM}YnH^K;o!^fu<3c#^IhN1=`_pMp=Ce*wD818L&eR4}XT(`xTmy1$uaA4xqIJ@Y7O$BSdrr*;KgQgr4$ zbk6v&u|?yzJYLQhZeio1+}layJiNTb$!6rnhbtoNNraRe2WPOq3SLwlk|VJ^v=Vll zB!HTx{?0Is1P8YYwS!92+&rU}=(!zQY0Qh|8eGrbT*$n?U5+-EPxL8S@1YlmR1fSJ z9D2fYNzQys&s9_V$<;z7eKovNtlN!0a`6(EACkCjnaj3bQ5jO_)FufyX8n0}%CsW< z|8e%-;aK|U>6p2iBkHSTXVll-mk`2 z`sWFz?~|rqZ{EohrmSrUOI; z;0=8D?!BztLUvfGx=UrvNzv>2-hP*q$GirkgRh0+q~MX}y`CTKco;m)n?TOT(ye~e|Dxui{* zTqn!*mCxm+?kSA4NxJ6QA;)&{?)8$?1cFn@homiJzKP#u^*ffj==-gb-kENqzS`>UKyS%$Nd+&!UY3zGGk(*F1}E1S-ZgQ!TlyBspm(Rjh>B*s>D{p>*tNz z2@l?G^-;PS%rJHJWKH?d^v2V@zv7=z1rE|P+46<%?N3a7XBxMM%}<`@tCGOGgJ+IC znfs96)}*PXsz}!JlwZ?=%lmLcmDKu;P@fmYiLPCV51zHIDrdD zEcLze!i&el$W$$Jo+jE|o~T(fh%(x%Ul`d;yFg>}smA$=&;23gGaJ`9+0=^GZxK8F zs=0-NnZs0?gZ6x!j^qEKcIrEUz{wMNUiH-J`!SE=(z{&dDz=Cw*NlxG; z+`uenCp^Jk_n-5zb9HRK$4Yt&&AF5}ly)udI-vs4`o{+hF9qLwh z>;05-@=(3^Pbb8;3w@7C9WH*K zYQE=1Q1~%*ZJk|{3nzmP)r4z^Cl=3sERK3aUaMicbwFq;V_!f))Lcx;&MJdA)l+rc zCpoD9XwI5kO*&ppc)V7wJV&!|#c$xeQqX3-*-UNrO;5rXw<{{hyDBL!CdM#VmG2uE z_;}??us2x`i$w%~+LJwB%x~Er{-S%?c`c*Q&G+78zoFFQ+%K+ntPtsbzT5ETY(3@F zFLeBucWg{XQc^G8;HV5O?ZU?pMZ4cA&GkZ?(tWn_)l`r9tQnY$GHSg1N=^0{x41+x z-ix$;N&ovDD#arB_>hz*%fJcGbv-BdU)`K}tVBn5#s_PK-;5p{IyYIe@$&3H5gzfQ z_(OxZF*PZYkRm?Bmiq6K@BRB)Ra9GG5%tf12}*_j?C$^hPaXu>-1}?iH#Tn<{`2!^a1*rqBZm)@>#%!E zUsY0qQ`B-x)`3k(>Cq@Ur|lcA zz3YFUo23(ZEu?C|Qi{Q2E(<2PEWi>$;CoBU5)xfP5SXPNJ>hT5c$!PlO(Ng2#kEsF z11)A4S%Xc1J~}X6k0$%|OGvdJzNCFFG~;$093`?&!^GR zfr^;ll89L>;0Mjs(onK-a)L-{3aY^R3plY{2Gm38{h-L-$L9p*Ts1fEqM#TV9W7(W z?Joxw4+{&lvQgoeg;p*x%Y~u!21p1t0=Nw*xhmNkO-YSHQ4tX+u6u25#9fZjRO1>z zYlM*4)6+A3I5*8wQ}+)G0oVzKVTOYJ*l?Y=TpuN2fdRT8u#Su@UtsOD)}*Y!6(n5dYP8S54V)m4g1rk zSQ6lyzzSp1;$dtosH46B*w7h(wb3REWlowWaJ0H|WfuVt{2WR?d|kO>g)*o1J=5<` z(xGG2(*G^MflF#SI=oPW0IWWAh(C8PpmTNUhJVbnwA9q-2~Yx)yaf1oY;Vggz~IQS z^2u3Aawd-;8zMddXchPcOsWN>1-T@7PQJ&rN2wV-0I>$kXVyV7ffpFW_Rt(eRu*|@;l@#z~uFHVZ$WVP_fuqN?jMAVIA=zsqKQ)gFieDsym# z??m)sal+Xe96T*-FtF@^yrE1*g2;{=j=8bq;ggsr15J+H<;xlv+X!nWatB`4?tLV9 zi&Emq-?)12ZCcvxmEVH^Vj-Ictsb`oLs<}=UWSJc4-62_z)ld#Fl-ZarqQz^^gyYC z3q}L!9~UYUcmb(hy;>%V*@)Maly2^phiOVAT2bg_*IwTy6W=pzxnwX}Q@D~0@h?~Vu4 z?Kymh4wb@Z4GMJhidr9nb9zEl^i|#&6!y45qodmicB1QO@2vjwg^#5^TnA_p>~TOG zu-JhHCZJLPieb$xb6PxekgWji0@9dEfN>kfsg~=2<{-?^PwirL>y|r)VG|qyxF9N$ zqS1Rz?@3Am+!sL!L>SA%v_Ksdd;2WR5`vCEg!yN2$m(}6B|!bQ!u%g>W55lBo5sd> zd&{>5%$*b##*zkt4C_jSAKX#>1MNS5VmzZU8QJ}-Z!eCHjP&$wfMX7dA^1eMCpK1~ zQ-+lvdfH@XP_=`W2p}OUN(Vk#F7M(}QY?Y);H3a(Y|Qj-CV(`KRwnufws-EJi-i_K zVTlf$??7&w>qCUahrq&tH%BizQMtzHiGs zJoJx^jmnnf0ag<|Px5Vt0}#7$nURqPy95tjic`iIJnUL`%-cx`{@hXILK>@ssAx0h6%-gmP67KNvCh!aGdF(~67rCzG98S1-7R*aZeXZ^WLIl{lYc*gH@IZz zJ%Ik&x{E4cas`rzyHg!F${F|XADNm87{2o)AYcPL-rgPU$+!o&O#r6x=)%IoGe5qQ zVudpV7;3%N_<&f^m;4#Mizde(PeAX4E;N87be!E5#&BIE_j-6gbxli3NVtKd3kE#| z7-BzPobXtJbQzMCo4U$`G~5$dSKw)31_$JH0&0qKa_B>A)lTmB1?d@rL|7-y&CMx> z2^gVUiI%`N0;RXUl~w1DA6|86C?b$o8D_v_4!baCKg&H=?GfdJ(wy^te-oOWWR%_+ zxMJW{AhnXFwW@o_6H3YZ6JldQ_W9D>oP~)xy1Foya06KdT0ltuxd0pC3v6duS*!~E zsnS+@lyTs5;GR4g&h9Y}ga<20@D!wD5+yx~=zc z*2Clq`vk@+=qrS~9g6@b7`@%J7Qm2rnCmJNh$nFQL8^dJqlVK-+X>Kyb#_Lg7lFSZ z+{OY0+6b5m=ZoG!mGm}rVjw7a84&?%jaH9zWF%N@AQ-i3K@}Rp2?bdrn9q3BBX+B} zW9i+-Aj~1OLa74a6g^=Ca(q%G_;7Q(sSt=LX(av{(3`o5327>+B}XR9^#Hkb)BQU? z<9tS3*h)a2sV+{MZqZ7!=%gOw0*W(cH|`@WudXUYOCAhYS5fgYnvGXNQ=asNPBhvT zR}>VCHC*iM=0WF7+ntB4fklJE2Lnl!VSI-h8a5oOoQMeA(gYYnQB75~@|GhBcoP#d z5QLquB@`ndE)wp2x%U!kQMB7J_e=v#yYJuAOJ7QaF_{0 zEn&2Qu}5d)l(CtQAw}Tg`qk34`5xqdKbxCD&A$tuJJjD0P_^1EIxpHU!rwyv)yeGy zN!R6|=g&R6z02=(5Has}vK>hZwtC14&VM*p1!AH(_ET-G{G#?OyrKtX;b@Eywz9ko zIz{?>2s}W6j)yPVeCOxgG4BwUApV6uB*nz}_HBN-?a&!I0^G$-C5a%o^!Jv_7r{y2EyY2ymb`Gm z!x%6qT9%H^{>Wc6)zy16Zmo(QM#moGg7cx)1zL-{0B^~(4^8N95@tY2z;5rZti(eG zLBbg4nYw|m$C5weDOs?)#a>_sSa%o0S(b*Hy0yqfxW=Ci?ZAzKo3LGi3FiS1F)*Uw zQ&DUjubc>G6zH=|?WzNh9`H7^dPz6@J-e%@F&bSOX9Ea4!OqSOL9nI1 z{uWX$EM{~BZ~}AhfQIbSg$ocG1jw_4AlIuGz^AOMi*Vs0URBA>%36jj8C(l_`C9n> z?GVTP26mm5lf$QxF@PYjZs!F<3Z&`asa?UQgpLk}rc^WksZ+s_RKT!QNeQv&W;81| z_XPGcj$O1<3AYiau!7-7ssJ~Yva()m4#z8SmVxpPY>A&9>7AXOJIOa6-$fvicnb=` zOsMzn#pdMV<#op*z(gj5J|v&WT){1YuoNE&bG{%vxO}wn!PXtv^zixf|Df=pLLm%;MD_y>U9f)tv6X~z*zTG{8w3yZ2DrbN zBCU>8fa?i{FU9-%)*#0Uqq+_-MOL;N4de72h*dQT?(+O}x{Zd#^0Wz@yvgpLN&EQm z=w2VJ4ODIRkQQ#3!F&penCNvR!rH2;gHuzayT<7MIm-A=aWP3&*y=_f0rt+{D+PfD zm5Gnl1o|sjNc4O5yorgiK|+c%a3e4kw-r?g%0WB;BpoE3c1y&1L=jGAl41>U0(*Hi zZ{+2YRHVYv;$j!_K=d}%jnN0Dr>1@>Wc3P$vB=q^f?2p=nS}U<>}q+&gHS3e0&X32 zBKF@d9T3Zn0UUS#3`tgk3&GP%`%qSkVh!HcBvnOaWxD$+0gdcfm=GcA=;(Ce=XHh5+~1=?m@!7wX;x?RT7j$;0M)aFtQpXX4BAlcc$!595q^cOIH zYjwc_4IDnJRML}gxEvziC&elnc?%xO-W#Y{;M52&m3z}6#Q~3TZhLuEV7P8tI19GJ zrcIl$D_hlEF>@Pd1^f)LHgGhc*FCebyzCH-1POABNCglEp(#0V+UDr}GiaWVS?s^0 zOJYBT=fZGNUD{;ghW`DyKWf*mA!C_G531Pg;+}!zGbp20GkpI;5Q?85<3Ku+E}xQ` zn$R?gv$$1DG!oZTFF=JXA2$o<4RTc&YehtK9Qqmc;)UVOo29ck`mhuZ3=BkpK;Yl+ zhnTkc9*Pu4$6N$?m>K)~?~ zp3UGB)QLI`u<5ZJ;3N1Q5dM&v`P^VyB}6|g8IPLZvX$Re8~F^j{=|X@q=Slzfv;X^ zq92I#magR!*CAfsfK*d>pdkyKpoQEHNEH%IZk@>us^gIg5E7CRU|qVd^t2eu4knK7 z-^G!=}0UBBKU0=orS4+DeG`l=EC{-b-hyF7u3)Nk;x0A{2Gd?lU77x1pG zvq;p%JH#!1aubj$R792JAkoFCHa2Enn8O#TkeZylkB*M~9_l($op|<&l-60;(~8j% z(y*Cu8#0+_JUoa<;}E$}Es*IVjwQpm83{ScJKS`2EA(6rGBKg(Lcl^+P1->R4%9sB zc0%mIswLbWsM>?V8#A?_^MSq>NrLF<(~+X~lv0nvkYFxm(!1~+KKvWj$vBkY=!~Kf z_OAi!e)?@h1YcCY>e)-h2t`B5!?r{&UftqaysnKwKVkxSEZ9sCBO&+gZT*~aV`4M+ zljvxv!-xA2XV9dDF#=EvlhYC+ACi$*N=O`>u2%$mSU9$zpdC7)9ejWu_^w!&yV8O} z4Sg>nBgu07M(zyiH&4l5w#Hap$bqoSaI3I7Bw=jC$487=gd*XNIERNORJO=@0kYWL zzMTth3-;PqLjnJbpCk`f?J4uD+_H)6fF|-cDM$}oV=z))Rh72p%3EaRkVOHV080f` z$Iko>IQpwgIdJDRBY@*LYc7u@U%O=~c?$mhXpnvPv z^PLEC!p==Q%8v!k&dv_$R#`mSO%Up!fk6{^2*aqZCjP*}w~SpxAf4H*frJvySf}sw zZP7`A?|au%w839Vzp?!FaeG5G{-XpQObAmo;agKv|HOo!gVgl4Iqfawx-*{2u+~CZ zhJ4GU5?gi`=9?-j!|V)STkl~$*3AQdKn&u(!cTaYn25{|)@?82+IxUu%(NiR6zD>Wa#Q!7CLdC6K94s}>$+E^P2;))m7Isg3;Np|F!2EUUFpUH8in z8_LL(GP~dFR@smK_anC@y;Q6*d;3!^`^qXSy$A2`a&y}`II!5QgI5p+0zgWNcu*Kv z#Yp2HKd!<$yE?M6?GY*EBH#Ki^n31%j!to6;&wTZv-3c zC_E(GX-qIffrRV>9=9ODqgxM_Fw|fyzZ(i^1fggrC9Td2lyc803CNX@jsXybt>(%e zlW4YM$3_PSgQKDX{AFQ)FpGp9<=LiHVfhSg%N9^_WJGEQT_X zkfMHtRevr7VE~aK5c?!p3bTQlsw$GCAe!W{3zHkvZx9XO%;gr_Qt%2(;%9ri7T|j% zN}`c)bQTvE2iAw9K(Zhm*$%KAywY1GPGsMo6t(@lyv`$KL1Y7B6=+0-8X82ga!H_% z-~(y`Ac5rx0ToVSdipFrCCshySx`^a9w=AP)XabV8YkQ{g@o?hI|k60t)F)}9aV%U z10+5{Ig!+D3ky?Ey=~02LP2jqW@c^^904=JYa2@lXCNHuJSWNi0Xu-@BVKmmiZMMESeQ#R<`r~e-JmwJzk9cj zYFZN#WV{;~fuz6eE@|izSel=QUG7tvSU~Tn>G7L-lPe|%sxSJ%HpVFWilQPPE2|!|_tuwGB;hKJ=* z7GIlV z^Vr7+@*RRAb(sRn&2Qhn;SG_fM@yvC!oD?H!sQ{`X*jr6_D$mkuSI=5iEs``NF!i^ zFsxDRdK$SM)(ReJ`Y|9}=xL~LQ?7-29JHi<$A8>^nGi!cbrZmO;P}ff^t;g0Bbso-m37j*jS6!5N^^SWrj^E)adW zJOC3rL1DWq3I2PUlv64n*7NVBrD*V`u*cGScHH&fvTYk64OsEwjLC>LsKxPvvOLGK zt+A^Mwm?4+BLdol_?a4X0Gz_Ad_n<6>Qf-?YJ`kg5SOw-;GqOJ;`q03VPnK3r}Fzq z#B**GlMr$Ol!=Rp$u_umEgIJb*$bFU0M5;z6IiZ!b&oJx(03^=WD3(#*U&pbZ@w{NKnVpk^@`1kQ zD9M@wAb!7--0jV$4^PCy%fy4|4)6_6362WzjJjPo57imInXx>878$~Wl*=Nwi_lB= z+DI|sLx*~>E|7SVj2mDM?lR>?8q{)_3jax@dJ(R?HCh*phEg`b~YuA2LZI+yS)GKKmE=PTR5xw_cC z=jV>CDxCkfzw4!;TXKk$`;u2!`uG3u*J4s~-6d(Msh4k^`kNJ>r5-5R!Par+-#`1W zOI$2K<9zY|^Unqpk7^`aY?lm=-*nr@T7uTL?(VM~?e2GZT2x+!ah0@e(SE-y4t@G# z)$(vHG$?*D!2fvl>KxyAl1IqFOZ6CG z#~>jGeziXSYu9~XMiLT5Z$4Ds#Qdi}B)ge0&9&ohlWP!oBE!g(Uz2L>nx44SZh&QX z=T6G@DRj0h5VMaitZ-u<8Jxr1M*(fMmeDqs06@W(S`<8o3=@I@Bt?LOx9LnEB|sZ{ z|{J6&3$JeIO3iNz*j4_r4F+G>jn zC#v0Ov({DNh$(FDs&2`)F+0PW)$mib>Na%eO|s<@M|R|MgtckrIoiFj5>e~$r5CXn zr?a|q@8O${qzyO{NJ7Wn-K`6+liO@G^z=C&KjOR-RNT3JJN5(m(=s4WpwI{j*@zL? zmZL%hI{+OagvM3c8pH3uel?eTp8bytz_@oW0Qr5Wnc+)?#*SNo{rUiU(10I_XvKkd z3qO6d*07;^qtU$s}k+GE|{PjcEt`+nwA4*Cjc zD(#A8cjL(dxPhedW-7`-*NN5>gk1Opq2$H`0Y%ix+FA-{Lx2%zbYh~SsJfAw7=%+Z zG6E#8hC>@kfpKd$0gYuOVMmUfQ&PH(vvhNcmF{YyI*vb}en>=+N{~SqG6xPZk}x=C zHGcozmSMExKL4_U8V=E2+qXk5k0_5t0^GcbSa;p^$Yyc#?mW%_OejAMd~e&fM`xuz z3WTmbU@mBU8gD0?>!CF!3&Vc^+N-zQ`s~)nN(O*8Fk!0{uW0(pc|D zk32e4YtHt!b_w7_K&_Ve;REw&8#E(HX9~R60lMM7B3u4@G`+vaCi;u|mI@J> zI$$~zO86r7)}u|C1-MEL>)jU z^x`+5#Po!@^I*x$>?~O+e2O2R*S~Qi0x2MNF=;AdQ1yJ43iKDKX08{Plt>W(KHy}< zJRSoJaTLg3HUXGLe&++-gf$iwYNleqUU0WUA%Xux=mEL`h6Iv&TL%XzV0Ord${kB zr27zDB_K-yYoRPsu4>8aUyT3v!KW`=_vvr4Uy-TKc3SFa@!M2-QT`FjS6Sbhg0X1} z$;pQ9rlWTUA`PwY2CK*DkcYxFO04+7K&pLs}WW`+S>Il9o&jy6LWaru{1R6D!#b(TF z-*?KY_a0?P$v)xKa;|)aiQ=taU_zw$)Q}C&v7w*!8nIVDe`yNHP0ijZ<{>n#Bme8F z*15&+?fX9a+BSz2kCk?*7AH?IAK~>z*TUqZRJD2H+w&E9J-!WN4bhY5v$Q$pc5AKY zw{!5E=H(leX&o!!J=b&X#<&d+Ydq(Q;e?J{_q&&za!*;?+_&gma*IAlbNf`nY(q|G zCqYHi!n*st#dN`V@6%z*jfmYvDg;eo$;R`IrImg!>;zV{`CLr^+Mo{)n{qQOxQ)#B zND?6ws;DqThYF^FX#Hb43xW|eO`#znn@XXj4M7GO0ZYyCQT5{DU7T~M z+*~i9v4I{BmI6A1WgGy4>%#;e$+N>VgO;8iYY0q~PlVE&{D|p1+}veG>X0IGtZg6*){td3tVM{MFJT3}%rF4E9%%FA_*v47eFVgo;`yU7L)6ss$e! zpA>bJ<5iS|_<{P5!1c7ntRI1p2$&;=qbO^gEr(9547DXJ#n4xxq1+nY+SUdz!pPhl zqp7YJ?jwN_q?rOccZw|50TXeBrDw=boc9aZ)X+QYhb0sj>tOQXfG!FqaspzJHV|o_ z^UuH-!c2IV5!`ntRQ1rf;2oDI=>p`gCaO?+3&6=G#WiII7s@GM4}kQZ%BWnsHaWMyUsux1UG4}c?(@W(4g2Xr=~Cjrx(Fc>G326XM(HA~I^MdGvHI^%SL zK7dD(ui&jCdQg;Qn>DeckIKsW2K1J+={TV9knsTr$B8u*A&zFYZo0lVI6@>L-!Uv4 zxF^VkaI^09AtxsXB_+B0HseQN%dM@gN}Ml>@9zF0bocg!PB0c=qSMNpoSJH)5g{8# zzHJ+NZiXno)sOpQJgE-y0w@?k!x9w}BP#`0q_D7~ySuxoY2T`d_GzJ)`zL(wDJ0-t zNnmnX-fjRFXfWY<gCBn$W1WLdPV%8yQ3S{~~`2psDJgNIaf}CM5iMXIEI4|Dy?HIenQf1SF zh7H=d=)0$wmT1-d4-t3$5T~lCbD6SNVu@@#wOI6n;?TQ+UIVskcbmWe99@#j%E(^S zqJ2!@H(RmhW&AN7$Wl+>_TFt4A&!9 zs2<454PB3FpVo5|p*nfpvU#vyg#PmFsavU6o(!f3oORko;FqoYWYSdlaLc83?eIX! zef$nPb%jS8H1fQL{H%)yMtk?q1yB-h9KPsVF{ikE>#mQRfaag2Q??p0)kk$C_uAPx zY%D}Ic2TY!QOY;*5{y|^XNCAxn`paWf1RqwlH;rSfvkk-w!H2mdFSqQY6{-L73OCb zkH>PQGU*h|jo2qeFly=F*uo&oLYc}Ds-pi?zwzi1|Jh(FmXO88&iw2WImN1D8*{^Z zqhqEzPQ+2!ZWO*dUu#)+Ny<+`dAaC0GcY9>b9Pr%aU#b9cL`^D>pXBL0{lDL5u|`o z;rs+;0aj?I%o<~nu*3coP5nrCDT#=P2nl89<&k1hw00;IS}H!=kta*3>&mk2b$tD z!2hD3z{7=?}pa+G47tFivViXc4 z@Vt73yHp6ussRL8%>xpU&6Ct1uOz`)FK zmJ%)fyxd&SN-_RQU%QvkgE)-%1s37jV5lzGxul#p6Qh5iAyhAR&BM4vFkjRuNk+>6 zh<;kApE+~SAQY6h&q!%fZsh$&o#g>#uh3Nmg((zD2zXF#uj1U{D_JYgc#)7Wj7SD@ zhaRX?!^4lXHJC!7NbVmZaq^bnGKyR--6{fhjp&*ZFjasDgL^Mu_9@Z_f(Tf&9nDaKgop;fqCK>Jzj&@PWicMEI#9!3BK- zG6=zSZTWKWFJM^3@$C(92}{C58`W zfhOU;2L@7w$q(h)u9_QcyQ%_OJw87YNHimXd8Acv*M7jKg%>s=83;!pG3)Px+8gc9 zG~k_}%z+&T?0$18-T=Bl;PjY`^*XRAiv=k>P;;*PXDI#|#ACFx);292**X*eP39@&}&_ ztUcHS!uS+~4A_f6L~Yq!e6Zx)!_nlTN=#0L7tlTrF>E^A%f(ZI!LoqvhoBLhX%+oT z`dxK?W5TRz@x{p2YugWtrc^AEd|4Ru2_&Ry4WPyHAmz(R+VfmeqsGb%W{oO$c8BtYR;5O@hn`uDTHhY0Lb!MN;zQA( zgRV0Q8{^e|1J{^CC1n{H-s#CL3ME7b7S7M8n%Qq7oUy*N>htNlM(PumuDl2>F=_J{ z5xJx-xAyNQaCeRukJ%0c5$Vt0G4k!E4{!|)*8b(8o|2mBOfV5!Shy1S)?qF>v_sNm zY5QZvX!$KY*IJ_GocE9K307MxoN4T}9hJI$c+Q4D2=m5UW?d1$-c+8Lq4IK5h?D&;V=3 zVp`s$TAW{Cq@J9Ox#fq^aDs0!KfM#KI6pUPd=Y2@ORArXe6gfxIO;TVB{*8zlu zMo=Hf2Uwkk0(p2VIHaOShT*W-KI~6raFv*MhHOWNREJqHx2)DE2dF&YCw>>mt&Ent_q+OMr%vqN4xx)9goe^%ycK zd{Q7I7=OEm-sIqoA^1%>FPLw-4_|i-j7l@*`x1xyPmf-N-Y0U9dC~O%BvMG7l57bS z?+NX3^=j}nz>!Bqh`?M*UZ8=@-1CAvEe#DA#f|XW)6l?Hy0T_x)NN4fJ#Xuusm9I4 zC99+34gi4rRAOqXuz-MOiF*+c8%#BuyM%a)y9PtNez0tiTc&fKCRMRKn6(5KXWR!Q zeP!&*i7MjGy9rx1-5-3$uou`0$>S3z=u9uMG}~4rJKgj66O8_&@WK*6jo8-SPF!ra z=Jtj&KOhTWFgS1nDvy+-#e~`&X(I{G1;@+1h`--4fA?+xx|kz{k-U5NirS4()14n` zjoN$&S-v@d@O?CqLzn?*AOQ!UfLo*&>f!9X3eFWWHBhi{Afjv}h0eU(V3wLWc52==GXA|y+xPwIL*_!;FH#1ss1fy^)dmhY*SR>kHa+}Q z>2c{utWZ>O7VRcFYmraAx=Y3WS_f@wip5HAx=m31xqWbexc91Y62)bkUtf=Io{;eI zm6@d5^m>q^qjo9XPn0iOcSdAbr#`MGE9Hc+&Sctnw)s)Q>%lGWv*fDeJj}DMUA|Qz z@oJU3N=oww`L}NW*IfbEQxtFTy$`1>t(4@u>e?G`^&rS&#=JhdXyCE3a-ULEBvH*I zzq&8f-XfOW{{a&h;mnd$lFXmvhJ){YWjFIO6P}*xWfe2oOJ=&u>`~pbH#=lf(ubxc zza3b)zlY=fB45T90)h8e!9Z2FVo}WQsg#YiwhguTO$5xRBOKjJ2+e)LsbXC8sf0HF z`-;z6P=&q6+$oMqmy7WRZ}prdTk;hIW;)}3+-(yc*#oJp$5ghzjkUp|PU<*u(;s#u za5%u#*^!brj?QDS*-h(FJJH20g2w0tq7_kFUk`Vcv5}Ge9hq(j7w9FP-ngEUoV;WE z_Oj>J=!y{T!Ospda3S>FFftBlQO1uo0C3pmB!xZ`HX#;^k%P z8h#`71t$}T438-+nBSw559}?og<7uj9;R!Ei=XTE9s?kY`^3Fes!l3HLk9I(ZeGO# zK%EcvEz%=!(>sC8)|{$o6u4<_VSx;FmX~<`@FOxqCdHQf29kpC-6_VE;uwRU07|=* zvfkDxK_oV4N5TgK0M|#L2{1E55?=DI2*y{nJY#{kop-*+Lk0g54-#|cfa2nBQB}Lr z?;3dAGkf+Q@4s87ra{l1m2(6z2;%C=qaOe_*zH*c*v)CChpgs^cL@c_Y(nB3rf|Z| z?e%Lyjc0>CiqYfOQ%WCh1KpoR?CwtD-HYz>q>$1*>lD!4cS$U8K3_Ze+Rw<1%@?)= z5xEX8tYp=lrJ7B%a?s+TCCKiZxITQ$oHOxQZKvJ0a;}vbWA4GN9|Wgg*A?vPAe^yE zjv@Qjm??MnS9NmTTP~YmrZK7Z&!s`R=C)5RCcW26o}0NXy74LL(XrCnt%Op+Z!R)f zEmRXchc*-Jggq=-c&=w(2TmNw6S?6M7IE^%m6>pte#+T1+YY-an=QCIzx-5l*`-Q?o+)G21w zfwy$g%xqrNHhJ&&r!U*JYY$#4L1Z`cRqdC?N}n$N^Pz$)_!$zU2u#KWKmbY{DC?np z;Bpx=Qb{O-Sd0Tj&b1mM0!$tT7$(FEyb7d~4JgnSoJm%5;Ng&lI*N&`HFG9GP*HGq zeIXwKW~yMiI1^iKcLX4KrGNj!pFgJiA(I<{2WTI7pq(ptg!QnHKUbt zt;7If!y9m9)LwHatSATlDFfB}2_NP7+Yn7c6~ZL$n1%)s;CQt0pfvOi3>0RlMPW zOqTekniIc~XSHVmLoG+-n_DHDRUQ5ypTDe8}&B%yuy}&r>dcx&v>uwQai;;{S zEpimBHW>m$C!LgicO}WLtrsyDu`BH{`;cJx_|>TnYc`IJkcee1o4)scKR#|^H7`7E zSF!C%v&rJLrl`l%>%kJvd^^{-rYr*wei)i1|M(#$P4PzDeaz2P;AoGklk{5AooaJ` z*(uJFC6A5Y$|e;{=5k$`8*kgjJ!p6m0Zw=IU^qj=a2rc+R-*^fktvOh2x?(wt){ zN@jV)+rN6HcUF1*&$ZnlgX#A;`nzltW=bO5HTQh18+aS`=!Uitvs=Q}yK0lK3i*W= zHi(`^EIJZ)6+g}znCw0%W;?RH;39ZIU#0b1P|=_*IVhSSdOX+Mj+9^4oc&OzZi#W- zE5I@^g~d~(DItntAya09z72j)9P3pqKBO(E(ICGJWjs%(XKfuF7Iy3)7U^--B$X4B zDmSgIJ;s0Xj84@Ece~CdrxpXBQc6@2yE_3SqscrAbpW;%T*Cn&BK|2M?vp-g^-JaL_ceq)&G$mQ zyi-Vq8?`4dsHEzwuM9_n+o+jmV}w-M%snL@@{QP^pAjM~6cp-#ETDhOvUHM|hoLEn zDoKEnfhjPCZv}f!1YIr6R>QQe)+#X!a*&iPhPD&wHcnUEPH=-A`~8Kn%RpCyVH!!# zWZ)2rmn@lYl+)1(T;}itISIHs{3CXef5P~gd3@~RwcjEPjz7*IN?pwo+@kin<} z%lzQ$$uJKJsu_eVqOHg!M>Yk;mHXBak2A>twcV?`4Qn zJ(hd71zsS!>v;KSS8D19ltbyar=biXrcp3Tw| z)7?3C{ng!uq!itlPeROph9xqTsV*4_PZQ7HBzo?9D?65^r&jRk<>e!*f1HmEB)#x` z^z$KK;Oc@7$2pGfdqKy@taNRZ6F4eeqfI#~bO(P-QHz+JCtlZ4kak)(JwH#gO9h@<@yvWPrtQSZP^-BH357SJZ|cR0>_*Ra2fb-3IJ#xB;YwG@(y)hr zxHy}}!Ihd$QET^kzJ~FFpV3Ykh0FDMo(U&y$*D$N#-)4$c!-_tg7(B!-;UyY&yL>u zv-s%4St9e=SlQrq;=K0ye#+gSBbcQ!i6&%Qb)%YoOm0ktPi22Kvg7R8UR6ZCEzs(D zfyL*gWAzO!x9OxQ?g;Z8`Ey^bg^TCnRt1k!gL7fqGPTVeb17IWI?E#_3jNOa^Ph4G zc8$Ck86LIla=NY&K66sYMj)l*D_-skR?|7LTWTjI44})%TVLva~bN|KaHMc$eo7cs$E) zGBHTl3{(N-ME;bFLb7st#6twX2j~-J6KPsG8u+qt0{r|aJ<^cMRf>HGA&pnW*S1}Z zD#-qjIW}XhS5YGDq!ul{vg6A{u7*hHKbwyaVG`|@4N4%`EkEB~Jpvmj$5g$ykXIG2 z421)f0Jp|vAh7ud@=@FX5-1y zD_b)=_k8(&?BWBMyDhZ115ln$^QiV8dw&c zh8|)p?%6ZZgah55Kw~~2?ntWJ2W|3Z-)=&;v8Sg8R||1=3qBQ2^$;3gkA-G6JT|xS z_cmGY)WDgOL(IXyDSZcv`>p?UzT#cCpOUcDt5k}!4-4gh{S2~7xw z^&exaC(uF0Y%~4Qa1xNrfni9mP;$Jv=fvI2$v=*K{+HQ#j5?liAsytQI?FpvVkqKUf zMEWnoPUn{!R1~zOehR{#>DZr)C-cr9e(LEMnkr7t)`dIhG&&R0~ z5~nAR#e@*)_mu70SgKpKN(g?Ez47Uf3mfkuOO6j`-{_q+8Nc~b$CAMLGpmPGQd;u&A{%TB0d(l;_~-rDT=ffhLR9)-G$I zckEM!5yWFQy<@G0S=6fIyLK~)eru@tbk>kaS7Nj3!IgD9W0&q*?HOsjlaat)iJ1=j!{0RwG>@sr9LMRRf2=+AOF^z5HE9txLtBJyrSX z>5dR)=Zd2?@5Bcd;;q7(`etg~$EURQI`g-!Q;+1^uhlM;NDLLMKAm1vX&1V7ohNZF zdaB>SXa_5k=!cF|=d*4bep6%|6*m1@_QPP7~!B7@s;%wJttl zDq23+p-At4QP6Rw2RV{dY{D0*97JgBaE63-%&0X2j=qywTm*rsH=2&Xd4%d!Mc-&( z)V2?FZ-s9$4$(*)yZQ5KoZ0*h2bJ^l+w>=U?hRg+reKX4&otJWIev-#?&^SJ=k`5U z$vh;ChWG8yI(^9YPRr7f%<+KM{R`7c;g7b;WYTl3@`>?ZOLtN#{Y|u|kl6oA#Q8?5 z_Z==)ua((cB)%Pkr?*Leyn znjdR_4$vBPdE39az4%w*hl#%93q=*g!}?QFan;quQm1mS9q(Sf`?WoQeTUekJZkCS zAUo}4XBRP_UibT_*=M%Q)cyhZ03TEU^j z6Ltvb&RY%(i<+M4Z@jW-k>FS-WxAYxj@8(7iQ8*g@RrkZeRi*#v459Xue%qwVZB0t z%~N8|*PS>5V%|J2@PD3}d1JsNj>ek%_%)H!HI;H?-4*07d9SP5?#&PHmft}`WbU1o zN+xbWyE#`oJd`g`_VCa-xcUCvcKp&0nT{;?uX-^jO2uC!6cZaws;bQ967n8TjL+)U z>#X-%$+c|hG2_2kN=W>|2`e?ym1o**C&(HXk7&f-=5xB4H86Cac2AhK!UW%4Udea+ zh2D53B$sO&XL}wG{GhX^iG4mYaNWR*+MLzv+u-rJ=JK|$+Uzfuju<P}pnsk)^?mR-k>;?O%gyCp*XNZh!WmxFhnGUFhZNV0i#rmFr%Sr1BKU3dw69NV zTr$fgtC91MXcK5`yg+E!Z$H-6KtHkG(MX!YjsfL%TqT!dyg#&Vwyh4(Nnd@LU+j^( z`k8Im{qpsqAMwvy*0(+hhLKoe6R34o|4sTKXr!^1?&0xZcV+HWp$BnEb>o=4UOA*EnLc| ze%sX38)A-oD>U?No*Pm_(%bJtm83#G*I)$sKDdMqRY5{H9J@S0^bg)wwhb zwimSyMJ|bTYxhe{g(sWvjc!_1+@Z>KH>xIMZc6BNdM?9)T#`lu8G-ORhP2qYwMk>F z?@`81BrvEP)IE?rSNZx@T!b;dsO4KHMv3s;Vyl(UQwKI5=pssv->%9{TzUSvc3-*bGOX39r4pUqef^1ByM*7B0PO=XZfXgj<3`DYiP0cPt&KT%^Zn==PuQ} zIHuO%E@AauxMT-wADtrQ^(&0ur8o^gMk;N%vyL?l`YJYB!I>J&ZAr-xUR)rd^jay7 z#=JNSLiN|#BPSX+BYj*wliDJuyNlct2hsS#Kj)EImgYa_(MQe6cY)G8szL<9;6z9I z%Tzu(>q+w;F4*ZPex{QKjZ#^)RU{gxnj#QERnNWi4>xE+6g z{%!)AtgXQHrUgn$9GFZB1CcT`-R}x8p}17jbyIz#gsICK`E&%Mk+wX`zwapSpQM(Geh=64 zugo9I1?eQ@XecYsDKL}@_GlFR_dD7!XGcFEE}7H1LFeV}*A(tw|Nb?BV0>|Y=ihfH z@C*L`y>krT1^<1G(%S!S#s7Wj7jIB{BHKiAG=siH$cMqp=r{MX1`k7>H#{6_B0-z& zhn{v}p?J;DnJNM1ji!@)|6Pmd$OK9`4Lvr@Bl(G=UlY$RW>69Z-6;T&}ac&_*(PL^d!iL_zJla zeO!iN@4s)Tl%0L6Q&hi#V)V{G9$*~hJIr{q~sxw4&6eyntuLGk(llwNNq=Jsa>j!18zkest zb8w8%birjW7_w;~qjd+u0tYxH%xt^*Zd84h+Q6{{dnrjusy4xrZZm`LMwTaRzcc(%|WXqIT5MP$HXF1+hX6$Q5i3bp-T} zBvT(-TP?VvLDI(S1!saA>H3v`!jSnxZ*lnfSHC|cP)DGu#3b6iEG(Eg0dQ(+aFETb zs=hu-z^ED>DKHtJ54*-MX3LXQi3zcsh$hfcy@TSJa z0<#_hu|eyjX*x{F!N;Oxy$&h@YF%(FFujUPOw0w13+cyTF#$~!Lq{|*TPTz)Qj3IO zlh+gsde{*O3>7<#<%{*OaoK7x<-wL>+Sr}flo`MiQ`Q4FG&pZ(jLB;icZs6*MmEyZ zD!n!h*#}@gNSyU?i@d2ZL(vEm85*RB7*o*<>M8mxRh5+&D6Xoj=K?qYLa+Ghq!~&$ zcrbvUMKZ_AvQX31`~?aE`X~JZ14r3@Oc%}tjIshHeYpEzJ{}oBJ5<8OOD{Q2>JStMT-(2gdQj=K>SM{NY-J%V2SvzlveMnAM5I3+3d} z#-C~H)-+aretrOwc&Ktp~wk zKjn8SM~QU}4LRV9XJ+n`77-ImZXQD$N03=1rr8{JiO#5?LB*qSlI0D}AqjTJe=745 zPXdK#FLcTreTMQG?qhjX=A`=ti$eeysO7Z=5?MUf?qG!WF2~J;RsLfbUOX zL=wdbAU9AO9Sg3;;zKW50U})9P^yEX^LP%}e$o`6yrV#-rK$%+F)=ko8e#%+2?&I! zgAN{am}s3HOHM^w3=c`S#9fk30vq&GF-|Lq=X;tq7ueoUtY zus^xENwd!{Fx~=x$H~!ACJudc5rS@q{B#HOeQ37w*i^?Q_Q z`Sdf@J8o6-f6uKnKQs-|Vi$(x0iY-1`m_hyYQ*fb>kkC$TLGhB0KL<2ID~%pQNW8eWG;8ocPY-Dyl5Hlpo9=Q0 zGq!Q3J)(ZdMbU6Lp~%U|1l6P0px5}rWbhLRixjNQ#{Bz%Il6c(N0DjqQ$bm3fn)F{}Q8(YM=wb18M8HmB%# zZ*W$Y#bM6&m4`#QwsD;kM<`P26mh0wsT5g4c0-gHk~M3MEku@v#$F;JvLxS_ zv4l9GY%!Lh9Q&RvOUH*}9|n!=eD^S}>wMRB&R^ekea~Mr*UbCA&-1*`eLwg6yMOoZ z<%j!SKehan_5KL3d|Lzp@{;0YB|;FrG7&l`hJ0QxU)e(`SY5X~Wo zCy*g>&=dal<_Y*#j?+p?mW{?2x+$rTVsAG}dcJIY+iY`jdN(+X zMgqtRnJj2f?-fUba7fu##ZC~&D)%HeZb10=(sG`_F&fg`+D0{A~l)-K=* zv!N5;WZg*uotE#8+!w`#JuvOP?z;{LmXamj=(cH@61zj#k z^aA;iNq=7pkOL6PBd`U5!_!&B#({K;*96|P2k1fI z%pkgd4%$Ruy*5}iqS+q+1&S-Eoq&Yc@(~xB)rYwJ0qoDM&Gq9Z0G?a;XcNs8fdm9= zS;Ol7`fnvN@JAq!*$7GCq`xBWj}s( z>@P(mam6Ubxt^INrpKU`1dM{aQcO99Z-Gv6GQ#jJ)ae!J_tD&YfF5j#nn0g+E&U|) z9tH!E7x05pC52}^WV1r+@@<;Hf?w0HR-@9YEV~t{;D-5hJ)t zyQ`sXV7$hrre!51u`V~b@*w1vuds0G3#^SiKRT?W{VjEP_hG0Cy$+U6iwB`d56U}$ zodsq4*Nyha#=*O$H=?hX4cu~ zCauS-X@8})b0QKN5T-glYz)#RM99wjBeb~%%0Qt$2;65ia7S?_9R?`!XPpvK%T7!8+zeq*rsb1g%?zC9c2`vd@Me@eR zdbbhZ!4p#MZ}b2r(5GHGT-5AupQJ5RY-dseg=9M|v#VKQnmAZbCi6TwCp7bJigi0Y ztiM!YIy!&2J|e#L{LFTuF4Y_(;%9253gNZm7onA~dv&N<^JhwaWL$d(359%G`s=gY zyAyJk1h;hU?N%#%WijS{;v!h2FCvIzu~gxW^V&i?aizpv!_)a+eRTtQzb~kfbQYKH9 zo-XNb!8?w>BV`yC`q-&ieaY@pyva^ZoZ#DDzU0g6r7SlT3YCwq6-yA--P&bL<{^yx zMdNLj)~F@FZM5z^qZ1C_f>0M|%q4uwrhskCsimp&ZBZuG8TZIk zmBrOB)g9{?^`!DFPGnr1kA;b_rI7xrw5X+);Q<6s^y_2VUyu5)_`KvmCblk27k)qw zWGqgqUg`C@T4xt2s7M$=)^9}IAR>R~(LCH#TmEkYBg{sALK__lU?5;^@0B3lDh&?x zqL^()`n&_33_quPARLw-R>t5iv-3B|l60muP%(DL(9MYR7w>A2;Pl<5QW6AMPY>$1=O^As1&_ zvpd`$=i)7*S;rRIS6w6OmKHEe-u_RYu5BHUyYDbS)DBIUSIR(LCc3W*jbbr#n|@zh zhvhJ?QI-c@xD_4x=%^av{WY8=>5`Lifd71JY+Yoyfl&NG_GdDbezA*A=;jx`H^M@S zb8Lc-?B4CT#GN<9zsyZ72X_SY#$=}i3G4@N)Ue2r)2kzQi!8TWi^$ffh;5sA^1AiPUec^}%lhqPTGvQ9Pn)>E6srM-FbcoSi^-(zOgY*E@>RjTH9bCf12 zF+=~jdAelR*$_jWQ-g18aEe<=b1g2im{q%d4_dH#J3UF}ECsn<6zVx@xhl{Nl52ge z4h8x7yl#yQgni@T1cNMTi3i9o=XPVB$Ukt|8xre2)GZ$h6AMD$q%Xmi^s39DPr9n| zZLj6!0E3S-O^jIM^$s*JY4RduRXGe!_`ptC>e_OB=$a@~3YYG#HBOW8YPQy#ksyjakhg~374l&LRZeGt-EAgKdtO&7NN z8z15E=CHW}w{&o@(aQNE16Ocdv5vdMTx+Y{N`?WNdpqY{fvgRse7N&_`m2SDoC&5) zAJKJGhi{Fn&M0KUQe~Lf&^=?`rLb66Zw#x2Vh*lY{^9IT`z@`sy$wvOHAj!egGX6pt&=L>zo`?IP2HTJZ~=S$d%k5*BN*LrAA5vaG_4m1&z z*WC_K%eDg|(k5WOOU#{mKIpx7R`sin$KHjRlqvrFXi=U`*Um+kk$n5Fk68ltpu;nK z;}nDJQmof%Y;{u$$#Q1`X-~$V5A$BV*}8VP zE&MSZf8{+6|9yYI4}@a(rrXO*txfqrT1i469j~p5L13A90|)7yp#7d1t{EeUJLDti zyLX;28jO361>NYhrhMmKONb=>>gZpvOq#YlQvj&lv$1(F-h2WmI-rc^umq84(XAYS zo2NXG%n%TDXzpXUSO930ap}&+t)q}?r;mYhWeC>RZvA-{w9cSSnZV3J>b=6^UGK~+ znG6^!>P-xZ3;D{~)JJLeqRKX^RJK8MGnDA%q$`_Xvpc84i3I@bLJ8%E4c&8xeW4!Lf4OruVP5on&JI;^W==f};vb_Rm!o?RWdA z4{&&9`Ln{_C_P(5P+nz$SimvCO=&i!$JFeSBvEcp7Z{(e0J%WQW9kodv4Nzi#O|Y@Syzutp(wk7p2oo3o|I@B30+ffbRa-gafp z?bQb_`Z}|MWx9eJK_^A*D=bM;!Q~#^3D%klC`XBR8vTCQATtB)SFf+(A!lY|R*tSP=#?a5f z05`r0EgY>zpMx{0`tBKfc)l+79}SZ}s=tTm|Iv$(xA)Bo9aTyXm-nZg;Mx_GS}xM+ G(SHDxG_FAa literal 0 HcmV?d00001 diff --git a/doc/source/cluster/kubernetes/k8s-ecosystem.md b/doc/source/cluster/kubernetes/k8s-ecosystem.md index 418deaa167fb0..7e7eb221b9512 100644 --- a/doc/source/cluster/kubernetes/k8s-ecosystem.md +++ b/doc/source/cluster/kubernetes/k8s-ecosystem.md @@ -9,6 +9,7 @@ k8s-ecosystem/ingress k8s-ecosystem/prometheus-grafana k8s-ecosystem/pyspy k8s-ecosystem/volcano +k8s-ecosystem/yunikorn k8s-ecosystem/kubeflow k8s-ecosystem/kueue k8s-ecosystem/istio @@ -18,6 +19,7 @@ k8s-ecosystem/istio * {ref}`kuberay-prometheus-grafana` * {ref}`kuberay-pyspy-integration` * {ref}`kuberay-volcano` +* {ref}`kuberay-yunikorn` * {ref}`kuberay-kubeflow-integration` * {ref}`kuberay-kueue` * {ref}`kuberay-istio` diff --git a/doc/source/cluster/kubernetes/k8s-ecosystem/yunikorn.md b/doc/source/cluster/kubernetes/k8s-ecosystem/yunikorn.md new file mode 100644 index 0000000000000..ad48ecc5d4759 --- /dev/null +++ b/doc/source/cluster/kubernetes/k8s-ecosystem/yunikorn.md @@ -0,0 +1,190 @@ +(kuberay-yunikorn)= + +# KubeRay integration with Apache YuniKorn + +[Apache YuniKorn](https://yunikorn.apache.org/) is a light-weight, universal resource scheduler for container orchestrator systems. It performs fine-grained resource sharing for various workloads efficiently on a large scale, multi-tenant, and cloud-native environment. YuniKorn brings a unified, cross-platform, scheduling experience for mixed workloads that consist of stateless batch workloads and stateful services. + +KubeRay's Apache YuniKorn integration enables more efficient scheduling of Ray Pods in multi-tenant Kubernetes environments. + +:::{note} + +This feature requires KubeRay version 1.2.2 or newer, and it's in alpha testing. + +::: + +## Step 1: Create a Kubernetes cluster with KinD +Run the following command in a terminal: + +```shell +kind create cluster +``` + +## Step 2: Install Apache YuniKorn + +You need to successfully install Apache YuniKorn on your Kubernetes cluster before enabling Apache YuniKorn integration with KubeRay. +See [Get Started](https://yunikorn.apache.org/docs/) for Apache YuniKorn installation instructions. + +## Step 3: Install the KubeRay operator with Apache YuniKorn support + +When installing KubeRay operator using Helm, pass the `--set batchScheduler.name=yunikorn` flag at the command line: + +```shell +helm install kuberay-operator kuberay/kuberay-operator --version 1.2.2 --set batchScheduler.name=yunikorn +``` + +## Step 4: Use Apache YuniKorn for gang scheduling + +This example uses gang scheduling with Apache YuniKorn and KubeRay. + +First, create a queue with a capacity of 4 CPUs and 6Gi of RAM by editing the ConfigMap: + +Run `kubectl edit configmap -n yunikorn yunikorn-defaults` + +Helm creates this ConfigMap during the installation of the Apache YuniKorn Helm chart. + +Add a `queues.yaml` config under the `data` key. The `ConfigMap` should look like the following: + +```yaml +apiVersion: v1 +kind: ConfigMap +metadata: + # Metadata for the ConfigMap, skip for brevity. +data: + queues.yaml: | + partitions: + - name: default + queues: + - name: root + queues: + - name: test + submitacl: "*" + parent: false + resources: + guaranteed: + memory: 6G + vcore: 4 + max: + memory: 6G + vcore: 4 +``` + +Save the changes and exit the editor. This configuration creates a queue named `root.test` with a capacity of 4 CPUs and 6Gi of RAM. + +Next, create a RayCluster with a head node with 1 CPU and 2GiB of RAM, and two workers with 1 CPU and 1GiB of RAM each, for a total of 3 CPU and 4GiB of RAM: + +```shell +# Path: kuberay/ray-operator/config/samples +# Configure the necessary labels on the RayCluster custom resource for Apache YuniKorn scheduler's gang scheduling: +# - `ray.io/gang-scheduling-enabled`: Set to `true` to enable gang scheduling. +# - `yunikorn.apache.org/app-id`: Set to a unique identifier for the application in Kubernetes, even across different namespaces. +# - `yunikorn.apache.org/queue`: Set to the name of one of the queues in Apache YuniKorn. +wget https://raw.githubusercontent.com/ray-project/kuberay/master/ray-operator/config/samples/ray-cluster.yunikorn-scheduler.yaml +kubectl apply -f ray-cluster.yunikorn-scheduler.yaml +``` + +Check the RayCluster that the KubeRay operator created: + +```shell +$ kubectl describe raycluster test-yunikorn-0 + +Name: test-yunikorn-0 +Namespace: default +Labels: ray.io/gang-scheduling-enabled=true + yunikorn.apache.org/app-id=test-yunikorn-0 + yunikorn.apache.org/queue=root.test +Annotations: +API Version: ray.io/v1 +Kind: RayCluster +Metadata: + Creation Timestamp: 2024-09-29T09:52:30Z + Generation: 1 + Resource Version: 951 + UID: cae1dbc9-5a67-4b43-b0d9-be595f21ab85 +# Other fields are skipped for brevity +```` + +Note the labels on the RayCluster: `ray.io/gang-scheduling-enabled=true`, `yunikorn.apache.org/app-id=test-yunikorn-0`, and `yunikorn.apache.org/queue=root.test`. + +:::{note} + +You only need the `ray.io/gang-scheduling-enabled` label when you require gang scheduling. If you don't set this label, YuniKorn schedules the Ray cluster without enforcing gang scheduling. + +::: + +Because the queue has a capacity of 4 CPU and 6GiB of RAM, this resource should schedule successfully without any issues. + +```shell +$ kubectl get pods + +NAME READY STATUS RESTARTS AGE +test-yunikorn-0-head-98fmp 1/1 Running 0 67s +test-yunikorn-0-worker-worker-42tgg 1/1 Running 0 67s +test-yunikorn-0-worker-worker-467mn 1/1 Running 0 67s +``` + +Verify the scheduling by checking the [Apache YuniKorn dashboard](https://yunikorn.apache.org/docs/#access-the-web-ui). + +```shell +kubectl port-forward svc/yunikorn-service 9889:9889 -n yunikorn +``` + +Go to `http://localhost:9889/#/applications` to see the running apps. + +![Apache YuniKorn dashboard](../images/yunikorn-dashboard-apps-running.png) + +Next, add an additional RayCluster with the same configuration of head and worker nodes, but with a different name: + +```shell +# Replace the name with `test-yunikorn-1`. +sed 's/test-yunikorn-0/test-yunikorn-1/' ray-cluster.yunikorn-scheduler.yaml | kubectl apply -f- +``` + +Now all the Pods for `test-yunikorn-1` are in the `Pending` state: + +```shell +$ kubectl get pods + +NAME READY STATUS RESTARTS AGE +test-yunikorn-0-head-98fmp 1/1 Running 0 4m22s +test-yunikorn-0-worker-worker-42tgg 1/1 Running 0 4m22s +test-yunikorn-0-worker-worker-467mn 1/1 Running 0 4m22s +test-yunikorn-1-head-xl2r5 0/1 Pending 0 71s +test-yunikorn-1-worker-worker-l6ttz 0/1 Pending 0 71s +test-yunikorn-1-worker-worker-vjsts 0/1 Pending 0 71s +tg-test-yunikorn-1-headgroup-vgzvoot0dh 0/1 Pending 0 69s +tg-test-yunikorn-1-worker-eyti2bn2jv 1/1 Running 0 69s +tg-test-yunikorn-1-worker-k8it0x6s73 0/1 Pending 0 69s +``` + +Apache YuniKorn creates the Pods with the `tg-` prefix for gang scheduling purpose. + +Go to `http://localhost:9889/#/applications` and to see `test-yunikorn-1` in the `Accepted` state but not running yet: + +![Apache YuniKorn dashboard](../images/yunikorn-dashboard-apps-pending.png) + +Because the new cluster requires more CPU and RAM than the queue allows, even though one of the Pods would fit in the remaining 1 CPU and 2GiB of RAM, Apache YuniKorn doesn't place the cluster's Pods until there's enough room for all of the Pods. Without using Apache YuniKorn for gang scheduling in this way, KubeRay would place one of the Pods, and only partially allocating the cluster. + +Delete the first RayCluster to free up resources in the queue: + +```shell +kubectl delete raycluster test-yunikorn-0 +``` + +Now all the Pods for the second cluster change to the `Running` state, because enough resources are now available to schedule the entire set of Pods: + +Check the Pods again to see that the second cluster is now up and running: + +```shell +$ kubectl get pods + +NAME READY STATUS RESTARTS AGE +test-yunikorn-1-head-xl2r5 1/1 Running 0 3m34s +test-yunikorn-1-worker-worker-l6ttz 1/1 Running 0 3m34s +test-yunikorn-1-worker-worker-vjsts 1/1 Running 0 3m34s +``` + +Clean up the resources: + +```shell +kubectl delete raycluster test-yunikorn-1 +``` From 9fa335ceb2335078a89a27f5d7a32ef1af74e4ca Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 10 Oct 2024 11:13:14 -0700 Subject: [PATCH 07/46] [core][logging] Remove actor/task prefix from the logs shown on the driver process (#47703) Signed-off-by: kaihsun --- python/ray/_private/worker.py | 54 +++++++++++++++++++++++------- python/ray/tests/test_logging_2.py | 24 +++++++++++++ 2 files changed, 65 insertions(+), 13 deletions(-) diff --git a/python/ray/_private/worker.py b/python/ray/_private/worker.py index e51315e70624f..b55bc01c09820 100644 --- a/python/ray/_private/worker.py +++ b/python/ray/_private/worker.py @@ -567,6 +567,17 @@ def debugger_port(self): worker_id = self.core_worker.get_worker_id() return ray._private.state.get_worker_debugger_port(worker_id) + @property + def job_logging_config(self): + """Get the job's logging config for this worker""" + if not hasattr(self, "core_worker"): + return None + job_config = self.core_worker.get_job_config() + if not job_config.serialized_py_logging_config: + return None + logging_config = pickle.loads(job_config.serialized_py_logging_config) + return logging_config + def set_debugger_port(self, port): worker_id = self.core_worker.get_worker_id() ray._private.state.update_worker_debugger_port(worker_id, port) @@ -1921,7 +1932,7 @@ def custom_excepthook(type, value, tb): sys.excepthook = custom_excepthook -def print_to_stdstream(data): +def print_to_stdstream(data, ignore_prefix: bool): should_dedup = data.get("pid") not in ["autoscaler"] if data["is_err"]: @@ -1938,7 +1949,7 @@ def print_to_stdstream(data): sink = sys.stdout for batch in batches: - print_worker_logs(batch, sink) + print_worker_logs(batch, sink, ignore_prefix) # Start time of this process, used for relative time logs. @@ -2029,7 +2040,9 @@ def time_string() -> str: _worker_logs_enabled = True -def print_worker_logs(data: Dict[str, str], print_file: Any): +def print_worker_logs( + data: Dict[str, str], print_file: Any, ignore_prefix: bool = False +): if not _worker_logs_enabled: return @@ -2109,11 +2122,19 @@ def color_for(data: Dict[str, str], line: str) -> str: else: color_pre = color_for(data, line) color_post = colorama.Style.RESET_ALL - print( - f"{color_pre}({prefix_for(data)}{pid}{ip_prefix}){color_post} " - f"{message_for(data, line)}", - file=print_file, - ) + + if ignore_prefix: + print( + f"{message_for(data, line)}", + file=print_file, + ) + else: + print( + f"{color_pre}({prefix_for(data)}{pid}{ip_prefix}){color_post} " + f"{message_for(data, line)}", + file=print_file, + ) + # Restore once at end of batch to avoid excess hiding/unhiding of tqdm. restore_tqdm() @@ -2163,7 +2184,6 @@ def listen_error_messages(worker, threads_stopped): error_message = _internal_kv_get(ray_constants.DEBUG_AUTOSCALING_ERROR) if error_message is not None: logger.warning(error_message.decode()) - while True: # Exit if received a signal that the thread should stop. if threads_stopped.is_set(): @@ -2184,7 +2204,8 @@ def listen_error_messages(worker, threads_stopped): "lines": [error_message], "pid": "raylet", "is_err": False, - } + }, + ignore_prefix=False, ) except (OSError, ConnectionError) as e: logger.error(f"listen_error_messages: {e}") @@ -2462,9 +2483,14 @@ def connect( ) worker.listener_thread.daemon = True worker.listener_thread.start() + # If the job's logging config is set, don't add the prefix + # (task/actor's name and its PID) to the logs. + ignore_prefix = global_worker.job_logging_config is not None + if log_to_driver: global_worker_stdstream_dispatcher.add_handler( - "ray_print_logs", print_to_stdstream + "ray_print_logs", + functools.partial(print_to_stdstream, ignore_prefix=ignore_prefix), ) worker.logger_thread = threading.Thread( target=worker.print_logs, name="ray_print_logs" @@ -2506,10 +2532,12 @@ def disconnect(exiting_interpreter=False): worker.logger_thread.join() worker.threads_stopped.clear() + # Ignore the prefix if the logging config is set. + ignore_prefix = worker.job_logging_config is not None for leftover in stdout_deduplicator.flush(): - print_worker_logs(leftover, sys.stdout) + print_worker_logs(leftover, sys.stdout, ignore_prefix) for leftover in stderr_deduplicator.flush(): - print_worker_logs(leftover, sys.stderr) + print_worker_logs(leftover, sys.stderr, ignore_prefix) global_worker_stdstream_dispatcher.remove_handler("ray_print_logs") worker.node = None # Disconnect the worker from the node. diff --git a/python/ray/tests/test_logging_2.py b/python/ray/tests/test_logging_2.py index eb32201d589b8..81fc8ed56abb3 100644 --- a/python/ray/tests/test_logging_2.py +++ b/python/ray/tests/test_logging_2.py @@ -424,6 +424,30 @@ def existing_factory(*args, **kwargs): assert record.__dict__["existing_factory"] +def test_text_mode_no_prefix(shutdown_only): + """ + If logging_config is set, remove the prefix that contains + the actor or task's name and their PIDs. + """ + script = """ +import ray +import logging +ray.init( + logging_config=ray.LoggingConfig(encoding="TEXT") +) +@ray.remote +class MyActor: + def print_message(self): + logger = logging.getLogger(__name__) + logger.info("This is a Ray actor") +my_actor = MyActor.remote() +ray.get(my_actor.print_message.remote()) +""" + stderr = run_string_as_driver(script) + assert "This is a Ray actor" in stderr + assert "(MyActor pid=" not in stderr + + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) From 3d8ea28f53cc6a59687050b15dad450fed14dd29 Mon Sep 17 00:00:00 2001 From: Sven Mika Date: Thu, 10 Oct 2024 20:49:24 +0200 Subject: [PATCH 08/46] [RLlib] Add framework-check to `MultiRLModule.add_module()`. (#47973) --- rllib/core/rl_module/multi_rl_module.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/rllib/core/rl_module/multi_rl_module.py b/rllib/core/rl_module/multi_rl_module.py index 43eddb909dea0..b140d9d13a96c 100644 --- a/rllib/core/rl_module/multi_rl_module.py +++ b/rllib/core/rl_module/multi_rl_module.py @@ -288,6 +288,19 @@ def add_module( # has `inference_only=False`. if not module.inference_only: self.inference_only = False + + # Check framework of incoming RLModule against `self.framework`. + if module.framework is not None: + if self.framework is None: + self.framework = module.framework + elif module.framework != self.framework: + raise ValueError( + f"Framework ({module.framework}) of incoming RLModule does NOT " + f"match framework ({self.framework}) of MultiRLModule! If the " + f"added module should not be trained, try setting its framework " + f"to None." + ) + self._rl_modules[module_id] = module # Update our RLModuleSpecs dict, such that - if written to disk - # it'll allow for proper restoring this instance through `.from_checkpoint()`. From b8dc709e560c45653e3b0c078dfabb000e613921 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Thu, 10 Oct 2024 13:30:18 -0700 Subject: [PATCH 09/46] [core][logging] Redefine worker initialization status and add context to the driver's system logs (#47948) Signed-off-by: kaihsun --- .../_private/ray_logging/logging_config.py | 7 ++++++ python/ray/_private/worker.py | 14 ++++++++++- python/ray/tests/test_logging_2.py | 23 +++++++++++++++++++ 3 files changed, 43 insertions(+), 1 deletion(-) diff --git a/python/ray/_private/ray_logging/logging_config.py b/python/ray/_private/ray_logging/logging_config.py index b944625aa8810..6788571f9e6aa 100644 --- a/python/ray/_private/ray_logging/logging_config.py +++ b/python/ray/_private/ray_logging/logging_config.py @@ -56,6 +56,13 @@ def __init__(self): "level": log_level, "handlers": ["console"], }, + "loggers": { + "ray": { + "level": log_level, + "handlers": ["console"], + "propagate": False, + } + }, } } diff --git a/python/ray/_private/worker.py b/python/ray/_private/worker.py index b55bc01c09820..0abfb5757692c 100644 --- a/python/ray/_private/worker.py +++ b/python/ray/_private/worker.py @@ -485,11 +485,17 @@ def __init__(self): # Cache the job id from initialize_job_config() to optimize lookups. # This is on the critical path of ray.get()/put() calls. self._cached_job_id = None + # Indicates whether the worker is connected to the Ray cluster. + # It should be set to True in `connect` and False in `disconnect`. + self._is_connected: bool = False @property def connected(self): """bool: True if Ray has been started and False otherwise.""" - return self.node is not None + return self._is_connected + + def set_is_connected(self, is_connected: bool): + self._is_connected = is_connected @property def node_ip_address(self): @@ -2509,6 +2515,9 @@ def connect( _setup_tracing() ray.__traced__ = True + # Mark the worker as connected. + worker.set_is_connected(True) + def disconnect(exiting_interpreter=False): """Disconnect this worker from the raylet and object store.""" @@ -2549,6 +2558,9 @@ def disconnect(exiting_interpreter=False): if ray_actor is not None: ray_actor._ActorClassMethodMetadata.reset_cache() + # Mark the worker as disconnected. + worker.set_is_connected(False) + @contextmanager def _changeproctitle(title, next_title): diff --git a/python/ray/tests/test_logging_2.py b/python/ray/tests/test_logging_2.py index 81fc8ed56abb3..b48b04e44a593 100644 --- a/python/ray/tests/test_logging_2.py +++ b/python/ray/tests/test_logging_2.py @@ -387,6 +387,29 @@ def print_message(self): for s in should_not_exist: assert s not in stderr + def test_text_mode_driver_system_log(self, shutdown_only): + script = """ +import ray +ray.init( + logging_config=ray.LoggingConfig(encoding="TEXT") +) +""" + stderr = run_string_as_driver(script) + should_exist = "timestamp_ns=" + assert should_exist in stderr + + +def test_structured_logging_with_working_dir(tmp_path, shutdown_only): + working_dir = tmp_path / "test-working-dir" + working_dir.mkdir() + runtime_env = { + "working_dir": str(working_dir), + } + ray.init( + runtime_env=runtime_env, + logging_config=ray.LoggingConfig(encoding="TEXT"), + ) + class TestSetupLogRecordFactory: @pytest.fixture From a8f2e8da21eb6cf662f740ebd2de26d1eade5704 Mon Sep 17 00:00:00 2001 From: Scott Lee Date: Thu, 10 Oct 2024 13:57:33 -0700 Subject: [PATCH 10/46] [Data] Refactor `plan_write_op` and `Datasink`s (#47942) ## Why are these changes needed? This PR refactors the `plan_write_op` and `Datasink` code paths to separate out (1) the step for writing blocks and (2) the step for collecting stats. The existing end behavior and outputs of `Write` operations are unchanged. ## Related issue number ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [x] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Scott Lee --- .../_internal/datasource/bigquery_datasink.py | 6 +- .../_internal/datasource/mongo_datasink.py | 6 +- .../_internal/datasource/parquet_datasink.py | 6 +- .../data/_internal/datasource/sql_datasink.py | 6 +- .../data/_internal/planner/plan_write_op.py | 42 ++++++--- python/ray/data/dataset.py | 10 ++- python/ray/data/datasource/datasink.py | 85 +++++++++++++++---- python/ray/data/datasource/file_datasink.py | 18 ++-- python/ray/data/tests/test_bigquery.py | 22 ++++- python/ray/data/tests/test_datasink.py | 4 +- python/ray/data/tests/test_formats.py | 12 +-- 11 files changed, 148 insertions(+), 69 deletions(-) diff --git a/python/ray/data/_internal/datasource/bigquery_datasink.py b/python/ray/data/_internal/datasource/bigquery_datasink.py index 7491540a5d73a..651216362ca8d 100644 --- a/python/ray/data/_internal/datasource/bigquery_datasink.py +++ b/python/ray/data/_internal/datasource/bigquery_datasink.py @@ -3,7 +3,7 @@ import tempfile import time import uuid -from typing import Any, Iterable, Optional +from typing import Iterable, Optional import pyarrow.parquet as pq @@ -70,7 +70,7 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: def _write_single_block(block: Block, project_id: str, dataset: str) -> None: from google.api_core import exceptions from google.cloud import bigquery @@ -127,5 +127,3 @@ def _write_single_block(block: Block, project_id: str, dataset: str) -> None: for block in blocks ] ) - - return "ok" diff --git a/python/ray/data/_internal/datasource/mongo_datasink.py b/python/ray/data/_internal/datasource/mongo_datasink.py index 5f731134f808e..78d56c81f0753 100644 --- a/python/ray/data/_internal/datasource/mongo_datasink.py +++ b/python/ray/data/_internal/datasource/mongo_datasink.py @@ -1,5 +1,5 @@ import logging -from typing import Any, Iterable +from typing import Iterable from ray.data._internal.datasource.mongo_datasource import ( _validate_database_collection_exist, @@ -26,7 +26,7 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: import pymongo _validate_database_collection_exist( @@ -46,5 +46,3 @@ def write_block(uri: str, database: str, collection: str, block: Block): block = builder.build() write_block(self.uri, self.database, self.collection, block) - - return "ok" diff --git a/python/ray/data/_internal/datasource/parquet_datasink.py b/python/ray/data/_internal/datasource/parquet_datasink.py index 796b3f48c4ae4..4dffa939d7727 100644 --- a/python/ray/data/_internal/datasource/parquet_datasink.py +++ b/python/ray/data/_internal/datasource/parquet_datasink.py @@ -57,13 +57,13 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: import pyarrow.parquet as pq blocks = list(blocks) if all(BlockAccessor.for_block(block).num_rows() == 0 for block in blocks): - return "skip" + return filename = self.filename_provider.get_filename_for_block( blocks[0], ctx.task_idx, 0 @@ -90,8 +90,6 @@ def write_blocks_to_path(): max_backoff_s=WRITE_FILE_RETRY_MAX_BACKOFF_SECONDS, ) - return "ok" - @property def num_rows_per_write(self) -> Optional[int]: return self.num_rows_per_file diff --git a/python/ray/data/_internal/datasource/sql_datasink.py b/python/ray/data/_internal/datasource/sql_datasink.py index 5efd6edb79277..dbf49a145714c 100644 --- a/python/ray/data/_internal/datasource/sql_datasink.py +++ b/python/ray/data/_internal/datasource/sql_datasink.py @@ -1,4 +1,4 @@ -from typing import Any, Callable, Iterable +from typing import Callable, Iterable from ray.data._internal.datasource.sql_datasource import Connection, _connect from ray.data._internal.execution.interfaces import TaskContext @@ -18,7 +18,7 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: with _connect(self.connection_factory) as cursor: for block in blocks: block_accessor = BlockAccessor.for_block(block) @@ -33,5 +33,3 @@ def write( if values: cursor.executemany(self.sql, values) - - return "ok" diff --git a/python/ray/data/_internal/planner/plan_write_op.py b/python/ray/data/_internal/planner/plan_write_op.py index 94d1d744b9f9b..69c35df1b6c13 100644 --- a/python/ray/data/_internal/planner/plan_write_op.py +++ b/python/ray/data/_internal/planner/plan_write_op.py @@ -1,3 +1,4 @@ +import itertools from typing import Callable, Iterator, List, Union from ray.data._internal.compute import TaskPoolStrategy @@ -9,32 +10,49 @@ MapTransformer, ) from ray.data._internal.logical.operators.write_operator import Write -from ray.data.block import Block -from ray.data.datasource.datasink import Datasink +from ray.data.block import Block, BlockAccessor +from ray.data.datasource.datasink import Datasink, WriteResult from ray.data.datasource.datasource import Datasource def generate_write_fn( datasink_or_legacy_datasource: Union[Datasink, Datasource], **write_args ) -> Callable[[Iterator[Block], TaskContext], Iterator[Block]]: - # If the write op succeeds, the resulting Dataset is a list of - # arbitrary objects (one object per write task). Otherwise, an error will - # be raised. The Datasource can handle execution outcomes with the - # on_write_complete() and on_write_failed(). def fn(blocks: Iterator[Block], ctx) -> Iterator[Block]: + """Writes the blocks to the given datasink or legacy datasource. + + Outputs the original blocks to be written.""" + # Create a copy of the iterator, so we can return the original blocks. + it1, it2 = itertools.tee(blocks, 2) if isinstance(datasink_or_legacy_datasource, Datasink): - write_result = datasink_or_legacy_datasource.write(blocks, ctx) + datasink_or_legacy_datasource.write(it1, ctx) else: - write_result = datasink_or_legacy_datasource.write( - blocks, ctx, **write_args - ) + datasink_or_legacy_datasource.write(it1, ctx, **write_args) + return it2 + + return fn + + +def generate_collect_write_stats_fn() -> Callable[ + [Iterator[Block], TaskContext], Iterator[Block] +]: + # If the write op succeeds, the resulting Dataset is a list of + # one Block which contain stats/metrics about the write. + # Otherwise, an error will be raised. The Datasource can handle + # execution outcomes with `on_write_complete()`` and `on_write_failed()``. + def fn(blocks: Iterator[Block], ctx) -> Iterator[Block]: + """Handles stats collection for block writes.""" + block_accessors = [BlockAccessor.for_block(block) for block in blocks] + total_num_rows = sum(ba.num_rows() for ba in block_accessors) + total_size_bytes = sum(ba.size_bytes() for ba in block_accessors) # NOTE: Write tasks can return anything, so we need to wrap it in a valid block # type. import pandas as pd + write_result = WriteResult(num_rows=total_num_rows, size_bytes=total_size_bytes) block = pd.DataFrame({"write_result": [write_result]}) - return [block] + return iter([block]) return fn @@ -46,9 +64,11 @@ def plan_write_op( input_physical_dag = physical_children[0] write_fn = generate_write_fn(op._datasink_or_legacy_datasource, **op._write_args) + collect_stats_fn = generate_collect_write_stats_fn() # Create a MapTransformer for a write operator transform_fns = [ BlockMapTransformFn(write_fn), + BlockMapTransformFn(collect_stats_fn), ] map_transformer = MapTransformer(transform_fns) return MapOperator.create( diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index 94c96a6a97210..2316afe958b8a 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -3709,13 +3709,15 @@ def write_datasink( datasink.on_write_start() self._write_ds = Dataset(plan, logical_plan).materialize() - blocks = ray.get(self._write_ds._plan.execute().block_refs) + # TODO: Get and handle the blocks with an iterator instead of getting + # everything in a blocking way, so some blocks can be freed earlier. + raw_write_results = ray.get(self._write_ds._plan.execute().block_refs) assert all( - isinstance(block, pd.DataFrame) and len(block) == 1 for block in blocks + isinstance(block, pd.DataFrame) and len(block) == 1 + for block in raw_write_results ) - write_results = [block["write_result"][0] for block in blocks] + datasink.on_write_complete(raw_write_results) - datasink.on_write_complete(write_results) except Exception as e: datasink.on_write_failed(e) raise diff --git a/python/ray/data/datasource/datasink.py b/python/ray/data/datasource/datasink.py index c523b5cd06c0c..0832e0539fd16 100644 --- a/python/ray/data/datasource/datasink.py +++ b/python/ray/data/datasource/datasink.py @@ -1,10 +1,51 @@ -from typing import Any, Iterable, List, Optional +import logging +from dataclasses import dataclass, fields +from typing import Iterable, List, Optional import ray from ray.data._internal.execution.interfaces import TaskContext from ray.data.block import Block, BlockAccessor from ray.util.annotations import DeveloperAPI +logger = logging.getLogger(__name__) + + +@dataclass +@DeveloperAPI +class WriteResult: + """Result of a write operation, containing stats/metrics + on the written data. + + Attributes: + total_num_rows: The total number of rows written. + total_size_bytes: The total size of the written data in bytes. + """ + + num_rows: int = 0 + size_bytes: int = 0 + + @staticmethod + def aggregate_write_results(write_results: List["WriteResult"]) -> "WriteResult": + """Aggregate a list of write results. + + Args: + write_results: A list of write results. + + Returns: + A single write result that aggregates the input results. + """ + total_num_rows = 0 + total_size_bytes = 0 + + for write_result in write_results: + total_num_rows += write_result.num_rows + total_size_bytes += write_result.size_bytes + + return WriteResult( + num_rows=total_num_rows, + size_bytes=total_size_bytes, + ) + @DeveloperAPI class Datasink: @@ -26,20 +67,16 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: """Write blocks. This is used by a single write task. Args: blocks: Generator of data blocks. ctx: ``TaskContext`` for the write task. - - Returns: - A user-defined output. Can be anything, and the returned value is passed to - :meth:`~ray.data.Datasink.on_write_complete`. """ raise NotImplementedError - def on_write_complete(self, write_results: List[Any]) -> None: + def on_write_complete(self, write_result_blocks: List[Block]) -> WriteResult: """Callback for when a write job completes. This can be used to "commit" a write output. This method must @@ -47,10 +84,27 @@ def on_write_complete(self, write_results: List[Any]) -> None: method fails, then ``on_write_failed()`` is called. Args: - write_results: The objects returned by every - :meth:`~ray.data.Datasink.write` task. + write_result_blocks: The blocks resulting from executing + the Write operator, containing write results and stats. + Returns: + A ``WriteResult`` object containing the aggregated stats of all + the input write results. """ - pass + write_results = [ + result["write_result"].iloc[0] for result in write_result_blocks + ] + aggregated_write_results = WriteResult.aggregate_write_results(write_results) + + aggregated_results_str = "" + for k in fields(aggregated_write_results.__class__): + v = getattr(aggregated_write_results, k.name) + aggregated_results_str += f"\t{k}: {v}\n" + + logger.info( + f"Write operation succeeded. Aggregated write results:\n" + f"{aggregated_results_str}" + ) + return aggregated_write_results def on_write_failed(self, error: Exception) -> None: """Callback for when a write job fails. @@ -111,10 +165,9 @@ def __init__(self): self.rows_written = 0 self.enabled = True - def write(self, block: Block) -> str: + def write(self, block: Block) -> None: block = BlockAccessor.for_block(block) self.rows_written += block.num_rows() - return "ok" def get_rows_written(self): return self.rows_written @@ -128,18 +181,18 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: tasks = [] if not self.enabled: raise ValueError("disabled") for b in blocks: tasks.append(self.data_sink.write.remote(b)) ray.get(tasks) - return "ok" - def on_write_complete(self, write_results: List[Any]) -> None: - assert all(w == "ok" for w in write_results), write_results + def on_write_complete(self, write_result_blocks: List[Block]) -> WriteResult: self.num_ok += 1 + aggregated_results = super().on_write_complete(write_result_blocks) + return aggregated_results def on_write_failed(self, error: Exception) -> None: self.num_failed += 1 diff --git a/python/ray/data/datasource/file_datasink.py b/python/ray/data/datasource/file_datasink.py index b09090f513af2..79d106f39ba3d 100644 --- a/python/ray/data/datasource/file_datasink.py +++ b/python/ray/data/datasource/file_datasink.py @@ -9,7 +9,7 @@ from ray.data._internal.util import _is_local_scheme, call_with_retry from ray.data.block import Block, BlockAccessor from ray.data.context import DataContext -from ray.data.datasource.datasink import Datasink +from ray.data.datasource.datasink import Datasink, WriteResult from ray.data.datasource.filename_provider import ( FilenameProvider, _DefaultFilenameProvider, @@ -114,7 +114,7 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: builder = DelegatingBlockBuilder() for block in blocks: builder.add_block(block) @@ -123,22 +123,20 @@ def write( if block_accessor.num_rows() == 0: logger.warning(f"Skipped writing empty block to {self.path}") - return "skip" + return self.write_block(block_accessor, 0, ctx) - # TODO: decide if we want to return richer object when the task - # succeeds. - return "ok" def write_block(self, block: BlockAccessor, block_index: int, ctx: TaskContext): raise NotImplementedError - def on_write_complete(self, write_results: List[Any]) -> None: - if not self.has_created_dir: - return + def on_write_complete(self, write_result_blocks: List[Block]) -> WriteResult: + aggregated_results = super().on_write_complete(write_result_blocks) - if all(write_results == "skip" for write_results in write_results): + # If no rows were written, we can delete the directory. + if self.has_created_dir and aggregated_results.num_rows == 0: self.filesystem.delete_dir(self.path) + return aggregated_results @property def supports_distributed_writes(self) -> bool: diff --git a/python/ray/data/tests/test_bigquery.py b/python/ray/data/tests/test_bigquery.py index 325de3eaa586d..67266a822c425 100644 --- a/python/ray/data/tests/test_bigquery.py +++ b/python/ray/data/tests/test_bigquery.py @@ -1,3 +1,4 @@ +from typing import Iterator from unittest import mock import pyarrow as pa @@ -10,6 +11,9 @@ import ray from ray.data._internal.datasource.bigquery_datasink import BigQueryDatasink from ray.data._internal.datasource.bigquery_datasource import BigQueryDatasource +from ray.data._internal.planner.plan_write_op import generate_collect_write_stats_fn +from ray.data.block import Block +from ray.data.datasource.datasink import WriteResult from ray.data.tests.conftest import * # noqa from ray.data.tests.mock_http_server import * # noqa from ray.tests.conftest import * # noqa @@ -197,6 +201,9 @@ def test_create_reader_table_not_found(self): class TestWriteBigQuery: """Tests for BigQuery Write.""" + def _extract_write_result(self, stats: Iterator[Block]): + return dict(next(stats).iloc[0])["write_result"] + def test_write(self, ray_get_mock): bq_datasink = BigQueryDatasink( project_id=_TEST_GCP_PROJECT_ID, @@ -204,11 +211,15 @@ def test_write(self, ray_get_mock): ) arr = pa.array([2, 4, 5, 100]) block = pa.Table.from_arrays([arr], names=["data"]) - status = bq_datasink.write( + bq_datasink.write( blocks=[block], ctx=None, ) - assert status == "ok" + + collect_stats_fn = generate_collect_write_stats_fn() + stats = collect_stats_fn([block], None) + write_result = self._extract_write_result(stats) + assert write_result == WriteResult(num_rows=4, size_bytes=32) def test_write_dataset_exists(self, ray_get_mock): bq_datasink = BigQueryDatasink( @@ -217,11 +228,14 @@ def test_write_dataset_exists(self, ray_get_mock): ) arr = pa.array([2, 4, 5, 100]) block = pa.Table.from_arrays([arr], names=["data"]) - status = bq_datasink.write( + bq_datasink.write( blocks=[block], ctx=None, ) - assert status == "ok" + collect_stats_fn = generate_collect_write_stats_fn() + stats = collect_stats_fn([block], None) + write_result = self._extract_write_result(stats) + assert write_result == WriteResult(num_rows=4, size_bytes=32) if __name__ == "__main__": diff --git a/python/ray/data/tests/test_datasink.py b/python/ray/data/tests/test_datasink.py index 7720784906017..714f03c6dfe30 100644 --- a/python/ray/data/tests/test_datasink.py +++ b/python/ray/data/tests/test_datasink.py @@ -1,4 +1,4 @@ -from typing import Any, Iterable +from typing import Iterable import pytest @@ -14,7 +14,7 @@ class MockDatasink(Datasink): def __init__(self, num_rows_per_write): self._num_rows_per_write = num_rows_per_write - def write(self, blocks: Iterable[Block], ctx: TaskContext) -> Any: + def write(self, blocks: Iterable[Block], ctx: TaskContext) -> None: assert sum(len(block) for block in blocks) == self._num_rows_per_write @property diff --git a/python/ray/data/tests/test_formats.py b/python/ray/data/tests/test_formats.py index 943e7d19bcffd..52cc95fc0335a 100644 --- a/python/ray/data/tests/test_formats.py +++ b/python/ray/data/tests/test_formats.py @@ -1,6 +1,6 @@ import os import sys -from typing import Any, Iterable, List +from typing import Iterable, List import pandas as pd import pyarrow as pa @@ -14,6 +14,7 @@ from ray.data._internal.execution.interfaces import TaskContext from ray.data.block import Block, BlockAccessor from ray.data.datasource import Datasink, DummyOutputDatasink +from ray.data.datasource.datasink import WriteResult from ray.data.datasource.file_meta_provider import _handle_read_os_error from ray.data.tests.conftest import * # noqa from ray.data.tests.mock_http_server import * # noqa @@ -239,7 +240,6 @@ def write(self, node_id: str, block: Block) -> str: block = BlockAccessor.for_block(block) self.rows_written += block.num_rows() self.node_ids.add(node_id) - return "ok" def get_rows_written(self): return self.rows_written @@ -255,7 +255,7 @@ def write( self, blocks: Iterable[Block], ctx: TaskContext, - ) -> Any: + ) -> None: data_sink = self.data_sink def write(b): @@ -266,11 +266,11 @@ def write(b): for b in blocks: tasks.append(write(b)) ray.get(tasks) - return "ok" - def on_write_complete(self, write_results: List[Any]) -> None: - assert all(w == "ok" for w in write_results), write_results + def on_write_complete(self, write_result_blocks: List[Block]) -> WriteResult: self.num_ok += 1 + aggregated_results = super().on_write_complete(write_result_blocks) + return aggregated_results def on_write_failed(self, error: Exception) -> None: self.num_failed += 1 From b679a206f401788b2606a36b3f29d906e2a6d65c Mon Sep 17 00:00:00 2001 From: PatricYan Date: Fri, 11 Oct 2024 06:36:55 +0800 Subject: [PATCH 11/46] Update handling-dependencies.rst, remove the redundant "is to" (#47968) remove the redundant "is to" from sentence Signed-off-by: PatricYan --- doc/source/ray-core/handling-dependencies.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/ray-core/handling-dependencies.rst b/doc/source/ray-core/handling-dependencies.rst index 8a2ac81edfc2b..ecec425bdc535 100644 --- a/doc/source/ray-core/handling-dependencies.rst +++ b/doc/source/ray-core/handling-dependencies.rst @@ -37,7 +37,7 @@ Concepts Preparing an environment using the Ray Cluster launcher ------------------------------------------------------- -The first way to set up dependencies is to is to prepare a single environment across the cluster before starting the Ray runtime. +The first way to set up dependencies is to prepare a single environment across the cluster before starting the Ray runtime. - You can build all your files and dependencies into a container image and specify this in your your :ref:`Cluster YAML Configuration `. From 07c72af16197cf9f1a8802b0d5f1c0e34314f0ac Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Thu, 10 Oct 2024 15:48:45 -0700 Subject: [PATCH 12/46] [serve] Fix failing test pow 2 scheduler on windows (#47975) ## Why are these changes needed? Fix `test_pow_2_replica_scheduler.py` on windows. Best guess is asyncio is slower on windows, so the shortened timeouts for some tests cause the tests to fail because tasks didn't get a chance to start/finish executing. Failing tests on windows: - `test_multiple_queries_with_different_model_ids` - `test_queue_len_cache_replica_at_capacity_is_probed` - `test_queue_len_cache_background_probing` ## Related issue number Closes https://github.com/ray-project/ray/issues/47950 Signed-off-by: Cindy Zhang --- .../ray/serve/tests/unit/test_pow_2_replica_scheduler.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/ray/serve/tests/unit/test_pow_2_replica_scheduler.py b/python/ray/serve/tests/unit/test_pow_2_replica_scheduler.py index e078885486c51..237a6c046a007 100644 --- a/python/ray/serve/tests/unit/test_pow_2_replica_scheduler.py +++ b/python/ray/serve/tests/unit/test_pow_2_replica_scheduler.py @@ -1253,7 +1253,7 @@ async def test_multiple_queries_with_different_model_ids(self, pow_2_scheduler): ), ] - done, _ = await asyncio.wait(tasks, timeout=0.01) + done, _ = await asyncio.wait(tasks, timeout=0.1) assert len(done) == len(tasks) assert all( @@ -1600,7 +1600,7 @@ async def test_queue_len_cache_replica_at_capacity_is_probed(pow_2_scheduler): s.replica_queue_len_cache.update(r1.replica_id, DEFAULT_MAX_ONGOING_REQUESTS) task = loop.create_task(s.choose_replica_for_request(fake_pending_request())) - done, _ = await asyncio.wait([task], timeout=0.01) + done, _ = await asyncio.wait([task], timeout=0.1) assert len(done) == 0 # 1 probe from scheduling requests # + 1 probe from when the replica set was updated with replica r1 @@ -1608,7 +1608,7 @@ async def test_queue_len_cache_replica_at_capacity_is_probed(pow_2_scheduler): # Now let the replica respond and accept the request, it should be scheduled. r1.set_queue_len_response(DEFAULT_MAX_ONGOING_REQUESTS - 1) - done, _ = await asyncio.wait([task], timeout=0.01) + done, _ = await asyncio.wait([task], timeout=0.1) assert len(done) == 1 assert (await task) == r1 @@ -1636,7 +1636,7 @@ async def test_queue_len_cache_background_probing(pow_2_scheduler): s.replica_queue_len_cache.update(r1.replica_id, 0) task = loop.create_task(s.choose_replica_for_request(fake_pending_request())) - done, _ = await asyncio.wait([task], timeout=0.01) + done, _ = await asyncio.wait([task], timeout=0.1) assert len(done) == 1 assert (await task) == r1 # 0 probes from scheduling requests From d2b9a6bb4f6046da836ac37e00ee7f8491b5b67f Mon Sep 17 00:00:00 2001 From: Hao Chen Date: Thu, 10 Oct 2024 16:16:00 -0700 Subject: [PATCH 13/46] [data] fix reading multiple parquet files with ragged ndarrays (#47961) ## Why are these changes needed? PyArrow infers parquet schema only based on the first file. This will cause errors when reading multiple files with ragged ndarrays. This PR fixes this issue by not using the inferred schema for reading. ## Related issue number Fixes https://github.com/ray-project/ray/issues/47960 --------- Signed-off-by: Hao Chen --- .../datasource/parquet_datasource.py | 27 +++++++++++-------- python/ray/data/tests/test_parquet.py | 26 ++++++++++++++++++ 2 files changed, 42 insertions(+), 11 deletions(-) diff --git a/python/ray/data/_internal/datasource/parquet_datasource.py b/python/ray/data/_internal/datasource/parquet_datasource.py index 8b06ecfaad60f..b688c2630d686 100644 --- a/python/ray/data/_internal/datasource/parquet_datasource.py +++ b/python/ray/data/_internal/datasource/parquet_datasource.py @@ -230,6 +230,14 @@ def __init__( # duplicating the partition data, we disable PyArrow's partitioning. dataset_kwargs["partitioning"] = None + # `read_schema` is the schema object that will be used to perform + # read operations. + # It should be None, unless user has specified the schema or columns. + # We don't use the inferred schema for read, because the pyarrow only infers + # schema based on the first file. Thus, files with different schemas will end + # up producing blocks with wrong schema. + # See https://github.com/ray-project/ray/issues/47960 for more context. + read_schema = schema pq_ds = get_parquet_dataset(paths, filesystem, dataset_kwargs) if schema is None: @@ -240,6 +248,7 @@ def __init__( schema = pa.schema( [schema.field(column) for column in columns], schema.metadata ) + read_schema = schema check_for_legacy_tensor_type(schema) @@ -247,17 +256,13 @@ def __init__( # Try to infer dataset schema by passing dummy table through UDF. dummy_table = schema.empty_table() try: - inferred_schema = _block_udf(dummy_table).schema - inferred_schema = inferred_schema.with_metadata(schema.metadata) + schema = _block_udf(dummy_table).schema.with_metadata(schema.metadata) except Exception: logger.debug( "Failed to infer schema of dataset by passing dummy table " "through UDF due to the following exception:", exc_info=True, ) - inferred_schema = schema - else: - inferred_schema = schema try: prefetch_remote_args = {} @@ -291,10 +296,10 @@ def __init__( self._pq_fragments = [SerializedFragment(p) for p in pq_ds.fragments] self._pq_paths = [p.path for p in pq_ds.fragments] self._meta_provider = meta_provider - self._inferred_schema = inferred_schema self._block_udf = _block_udf self._to_batches_kwargs = to_batch_kwargs self._columns = columns + self._read_schema = read_schema self._schema = schema self._file_metadata_shuffler = None self._include_paths = include_paths @@ -306,7 +311,7 @@ def __init__( self._pq_fragments, to_batches_kwargs=to_batch_kwargs, columns=columns, - schema=schema, + schema=self._read_schema, local_scheduling=self._local_scheduling, ) self._encoding_ratio = estimate_files_encoding_ratio(sample_infos) @@ -358,7 +363,7 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: meta = self._meta_provider( paths, - self._inferred_schema, + self._schema, num_fragments=len(fragments), prefetched_metadata=metadata, ) @@ -375,7 +380,7 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: to_batches_kwargs, default_read_batch_size_rows, columns, - schema, + read_schema, include_paths, partitioning, ) = ( @@ -383,7 +388,7 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: self._to_batches_kwargs, self._default_read_batch_size_rows, self._columns, - self._schema, + self._read_schema, self._include_paths, self._partitioning, ) @@ -394,7 +399,7 @@ def get_read_tasks(self, parallelism: int) -> List[ReadTask]: to_batches_kwargs, default_read_batch_size_rows, columns, - schema, + read_schema, f, include_paths, partitioning, diff --git a/python/ray/data/tests/test_parquet.py b/python/ray/data/tests/test_parquet.py index 15629e5ef0f79..23969d736f046 100644 --- a/python/ray/data/tests/test_parquet.py +++ b/python/ray/data/tests/test_parquet.py @@ -1291,6 +1291,32 @@ def _assert_equal(rows, expected): _assert_equal(ds.take_all(), expected_tuples) +def test_multiple_files_with_ragged_arrays(ray_start_regular_shared, tmp_path): + # Test reading multiple parquet files, each of which has different-shaped + # ndarrays in the same column. + # See https://github.com/ray-project/ray/issues/47960 for more context. + num_rows = 3 + ds = ray.data.range(num_rows) + + def map(row): + id = row["id"] + 1 + row["data"] = np.zeros((id * 100, id * 100), dtype=np.int8) + return row + + # Write 3 parquet files with different-shaped ndarray values in the + # "data" column. + ds.map(map).repartition(num_rows).write_parquet(tmp_path) + + # Read these 3 files, check that the result is correct. + ds2 = ray.data.read_parquet(tmp_path, override_num_blocks=1) + res = ds2.take_all() + res = sorted(res, key=lambda row: row["id"]) + assert len(res) == num_rows + for index, item in enumerate(res): + assert item["id"] == index + assert item["data"].shape == (100 * (index + 1), 100 * (index + 1)) + + if __name__ == "__main__": import sys From 301a3f031372643b870be072c9e96cc5f61e11d6 Mon Sep 17 00:00:00 2001 From: Ruiyang Wang <56065503+rynewang@users.noreply.github.com> Date: Thu, 10 Oct 2024 16:56:44 -0700 Subject: [PATCH 14/46] [core] Decouple create worker vs pop worker request. (#47694) Now, when you call PopWorker(), it finds an idle one or creates a worker. If a new worker is created, the worker is associated to the request and can only be used by it. This PR decouples the worker creation and the worker-to-task assignment, by adding an abstraction namely PopWorkerRequest. Now, if a req triggers a worker creation, the req is put into a queue. If there are workers ready, that is a PushWorker is called, either from a newly started worker or a released worker, Ray matches the first fitting request in the queue. This reduces latency. Later it can also be used to pre-start workers more meaningfully. Signed-off-by: Ruiyang Wang --- python/ray/tests/test_basic_4.py | 17 +- python/ray/tests/test_node_manager.py | 129 +++++++ src/ray/gcs/gcs_server/gcs_job_manager.cc | 2 +- src/ray/protobuf/common.proto | 1 + src/ray/raylet/worker.cc | 10 +- src/ray/raylet/worker.h | 1 - src/ray/raylet/worker_pool.cc | 417 ++++++++++++---------- src/ray/raylet/worker_pool.h | 112 +++--- src/ray/raylet/worker_pool_test.cc | 85 +++-- 9 files changed, 503 insertions(+), 271 deletions(-) diff --git a/python/ray/tests/test_basic_4.py b/python/ray/tests/test_basic_4.py index 47dae1fdeafb5..269062e789516 100644 --- a/python/ray/tests/test_basic_4.py +++ b/python/ray/tests/test_basic_4.py @@ -81,7 +81,11 @@ def get_num_workers(): time_waited = time.time() - start print(f"Waited {time_waited} for debug_state.txt to be updated") - # Check that no more workers started for a while. + # Check that no more workers started for a while. Note at initializtion there can + # be more workers prestarted and then idle-killed, so we tolerate at most one spike + # in the number of workers. + high_watermark = 16 + prev = high_watermark for i in range(100): # Sometimes the debug state file can be empty. Retry if needed. for _ in range(3): @@ -91,8 +95,17 @@ def get_num_workers(): time.sleep(0.05) else: break - assert num == 16 + if num >= high_watermark: + # spike climbing + high_watermark = num + prev = num + else: + # spike falling + assert num <= prev + prev = num time.sleep(0.1) + print(f"High watermark: {high_watermark}, prev: {prev}, num: {num}") + assert num == 16 @pytest.mark.skipif( diff --git a/python/ray/tests/test_node_manager.py b/python/ray/tests/test_node_manager.py index be746002b6dac..98e7dc27f6087 100644 --- a/python/ray/tests/test_node_manager.py +++ b/python/ray/tests/test_node_manager.py @@ -14,6 +14,13 @@ from ray._private.utils import get_num_cpus import time import sys +from ray._private.runtime_env.context import RuntimeEnvContext +from ray._private.runtime_env.plugin import RuntimeEnvPlugin +from typing import List, Optional +import logging +import tempfile +import collections +import shutil # This tests the queue transitions for infeasible tasks. This has been an issue @@ -396,6 +403,128 @@ def f(): assert used_worker_pids == worker_pids +MyPlugin = "HangOnSecondWorkerPlugin" +MY_PLUGIN_CLASS_PATH = "ray.tests.test_node_manager.HangOnSecondWorkerPlugin" +PLUGIN_TIMEOUT = 10 + + +class HangOnSecondWorkerPlugin(RuntimeEnvPlugin): + """ + The first worker will start up normally, but all subsequent workers will hang at + start up indefinitely. How it works: Ray RuntimeEnvAgent caches the modified context + so we can't do it in modify_context. Instead, we use a bash command to read a file + and hang forever. We don't have a good file lock mechanism in bash (flock is not + installed by default in macos), so we also serialize the worker startup. + """ + + name = MyPlugin + + def __init__(self): + # Each URI has a temp dir, a counter file, and a hang.sh script. + self.uris = collections.defaultdict(dict) + + def get_uris(self, runtime_env: "RuntimeEnv") -> List[str]: # noqa: F821 + return [runtime_env[self.name]] + + async def create( + self, + uri: Optional[str], + runtime_env, + context: RuntimeEnvContext, + logger: logging.Logger, + ) -> float: + d = self.uris[uri] + d["temp_dir"] = tempfile.mkdtemp() + logger.info(f"caching temp dir {d['temp_dir']} for uri {uri}") + d["counter_file"] = os.path.join(d["temp_dir"], "script_run_count") + with open(d["counter_file"], "w+") as f: + f.write("0") + d["hang_sh"] = os.path.join(d["temp_dir"], "hang.sh") + with open(d["hang_sh"], "w+") as f: + f.write( + f"""#!/bin/bash + +counter_file="{d['counter_file']}" + +count=$(cat "$counter_file") + +if [ "$count" -eq "0" ]; then + echo "1" > "$counter_file" + echo "first time run" + exit 0 +elif [ "$count" -eq "1" ]; then + echo "2" > "$counter_file" + echo "second time run, sleeping..." + sleep 1000 +fi +""" + ) + os.chmod(d["hang_sh"], 0o755) + return 0.1 + + def modify_context( + self, + uris: List[str], + runtime_env: "RuntimeEnv", # noqa: F821 + ctx: RuntimeEnvContext, + logger: logging.Logger, + ) -> None: + logger.info(f"Starting worker: {uris}, {runtime_env}") + if self.name not in runtime_env: + return + assert len(uris) == 1 + uri = uris[0] + hang_sh = self.uris[uri]["hang_sh"] + ctx.command_prefix += ["bash", hang_sh, "&&"] + + def delete_uri(self, uri: str, logger: logging.Logger) -> float: + temp_dir = self.uris[uri]["temp_dir"] + shutil.rmtree(temp_dir) + del self.uris[uri] + logger.info(f"temp_dir removed: {temp_dir}") + + +@pytest.fixture +def serialize_worker_startup(monkeypatch): + """Only one worker starts up each time, since our bash script is not process-safe""" + monkeypatch.setenv("RAY_worker_maximum_startup_concurrency", "1") + yield + + +@pytest.mark.parametrize( + "set_runtime_env_plugins", + [ + '[{"class":"' + MY_PLUGIN_CLASS_PATH + '"}]', + ], + indirect=True, +) +def test_can_reuse_released_workers( + serialize_worker_startup, set_runtime_env_plugins, ray_start_cluster +): + """ + Uses a runtime env plugin to make sure only 1 worker can start and all subsequent + workers will hang in runtime start up forever. We issue 10 tasks and test that + all the following tasks can still be scheduled on the first worker released from the + first task, i.e. tasks are not binded to the workers that they requested to start. + """ + cluster = ray_start_cluster + cluster.add_node(num_cpus=2) + ray.init(address=cluster.address) + + @ray.remote(runtime_env={"env_vars": {"HELLO": "WORLD"}, MyPlugin: "key"}) + def f(): + # Sleep for a while to make sure other tasks also request workers. + time.sleep(1) + print(f"pid={os.getpid()}, env HELLO={os.environ.get('HELLO')}") + return os.getpid() + + objs = [f.remote() for i in range(10)] + + pids = ray.get(objs) + for pid in pids: + assert pid == pids[0] + + if __name__ == "__main__": import sys diff --git a/src/ray/gcs/gcs_server/gcs_job_manager.cc b/src/ray/gcs/gcs_server/gcs_job_manager.cc index ce0599c3cc4bd..02b08ba7d77bc 100644 --- a/src/ray/gcs/gcs_server/gcs_job_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_job_manager.cc @@ -424,7 +424,7 @@ void GcsJobManager::OnNodeDead(const NodeID &node_id) { for (auto &data : result) { if (!data.second.is_dead() && NodeID::FromBinary(data.second.driver_address().raylet_id()) == node_id) { - RAY_LOG(DEBUG) << "Marking job: " << data.first << " as finished"; + RAY_LOG(DEBUG).WithField(data.first) << "Marking job as finished"; MarkJobAsFinished(data.second, [data](Status status) { if (!status.ok()) { RAY_LOG(WARNING) << "Failed to mark job as finished. Status: " << status; diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index dbd2fee683ee9..fb20006d57b96 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -710,6 +710,7 @@ message ActorCreationTaskSpec { // The dynamic options used in the worker command when starting a worker process for // an actor creation task. If the list isn't empty, the options will be used to replace // the placeholder string `RAY_WORKER_DYNAMIC_OPTION_PLACEHOLDER` in the worker command. + // Used by Java workers for JVM options. repeated string dynamic_worker_options = 5; // The max number of concurrent calls for default concurrency group of this actor. int32 max_concurrency = 6; diff --git a/src/ray/raylet/worker.cc b/src/ray/raylet/worker.cc index 6770be6bd70ec..82c7476b17fcd 100644 --- a/src/ray/raylet/worker.cc +++ b/src/ray/raylet/worker.cc @@ -15,6 +15,7 @@ #include "ray/raylet/worker.h" #include +#include #include "ray/raylet/format/node_manager_generated.h" #include "ray/raylet/raylet.h" @@ -42,7 +43,7 @@ Worker::Worker(const JobID &job_id, ip_address_(ip_address), assigned_port_(-1), port_(-1), - connection_(connection), + connection_(std::move(connection)), assigned_job_id_(job_id), runtime_env_hash_(runtime_env_hash), bundle_id_(std::make_pair(PlacementGroupID::Nil(), -1)), @@ -129,7 +130,12 @@ void Worker::Connect(std::shared_ptr rpc_client) } } -void Worker::AssignTaskId(const TaskID &task_id) { assigned_task_id_ = task_id; } +void Worker::AssignTaskId(const TaskID &task_id) { + assigned_task_id_ = task_id; + if (!task_id.IsNil()) { + task_assign_time_ = absl::Now(); + } +} const TaskID &Worker::GetAssignedTaskId() const { return assigned_task_id_; } diff --git a/src/ray/raylet/worker.h b/src/ray/raylet/worker.h index f59acb827ab15..9166eea619fca 100644 --- a/src/ray/raylet/worker.h +++ b/src/ray/raylet/worker.h @@ -233,7 +233,6 @@ class Worker : public WorkerInterface { RAY_CHECK(!task_spec.IsActorTask()); SetIsActorWorker(task_spec.IsActorCreationTask()); assigned_task_ = assigned_task; - task_assign_time_ = absl::Now(); root_detached_actor_id_ = assigned_task.GetTaskSpecification().RootDetachedActorId(); } diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index de417b23693d5..8293be8e29c32 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -62,6 +62,20 @@ bool RemoveWorker( const std::shared_ptr &worker) { return worker_pool.erase(worker) > 0; } + +// If both `ask` and `have` are set, they must match. If either of them is not set, it +// is considered a match. +bool OptionalMatches(const std::optional &ask, const std::optional &have) { + return !ask.has_value() || !have.has_value() || ask.value() == have.value(); +} + +// Similar to OptionalMatches, but for JobID or ActorID. +// TODO(ryw): use std::optional. +template +bool IdMatches(const IDType &ask, const IDType &have) { + return ask.IsNil() || have.IsNil() || ask == have; +} + } // namespace namespace ray { @@ -180,11 +194,10 @@ void WorkerPool::SetRuntimeEnvAgentClient( if (!runtime_env_agent_client) { RAY_LOG(FATAL) << "SetRuntimeEnvAgentClient requires non empty pointer"; } - runtime_env_agent_client_ = runtime_env_agent_client; + runtime_env_agent_client_ = std::move(runtime_env_agent_client); } -void WorkerPool::PopWorkerCallbackAsync(const TaskSpecification &task_spec, - const PopWorkerCallback &callback, +void WorkerPool::PopWorkerCallbackAsync(PopWorkerCallback callback, std::shared_ptr worker, PopWorkerStatus status) { // This method shouldn't be invoked when runtime env creation has failed because @@ -193,34 +206,17 @@ void WorkerPool::PopWorkerCallbackAsync(const TaskSpecification &task_spec, RAY_CHECK(status != PopWorkerStatus::RuntimeEnvCreationFailed); // Call back this function asynchronously to make sure executed in different stack. io_service_->post( - [this, task_spec, callback, worker, status]() { - PopWorkerCallbackInternal(task_spec, callback, worker, status); + [this, callback = std::move(callback), worker = std::move(worker), status]() { + PopWorkerCallbackInternal(callback, worker, status); }, "WorkerPool.PopWorkerCallback"); } -void WorkerPool::PopWorkerCallbackInternal(const TaskSpecification &task_spec, - const PopWorkerCallback &callback, +void WorkerPool::PopWorkerCallbackInternal(const PopWorkerCallback &callback, std::shared_ptr worker, PopWorkerStatus status) { RAY_CHECK(callback); - auto used = false; - if (worker && finished_jobs_.contains(task_spec.JobId()) && - task_spec.RootDetachedActorId().IsNil()) { - // When a job finishes, node manager will kill leased workers one time - // and worker pool will kill idle workers periodically. - // The current worker is already removed from the idle workers - // but hasn't been added to the leased workers since the callback is not called yet. - // We shouldn't add this worker to the leased workers since killing leased workers - // for this finished job may already happen and won't happen again (this is one time) - // so it will cause a process leak. - // Instead we fail the PopWorker and add the worker back to the idle workers so it can - // be killed later. - RAY_CHECK(status == PopWorkerStatus::OK); - callback(nullptr, PopWorkerStatus::JobFinished, ""); - } else { - used = callback(worker, status, /*runtime_env_setup_error_message*/ ""); - } + auto used = callback(worker, status, /*runtime_env_setup_error_message=*/""); if (worker && !used) { // The invalid worker not used, restore it to worker pool. PushWorker(worker); @@ -511,8 +507,7 @@ std::tuple WorkerPool::StartWorkerProcess( if (!IsIOWorkerType(worker_type)) { AdjustWorkerOomScore(proc.GetId()); } - MonitorStartingWorkerProcess( - proc, worker_startup_token_counter_, language, worker_type); + MonitorStartingWorkerProcess(worker_startup_token_counter_, language, worker_type); AddWorkerProcess(state, worker_type, proc, start, runtime_env_info, dynamic_options); StartupToken worker_startup_token = worker_startup_token_counter_; update_worker_startup_token_counter(); @@ -544,8 +539,7 @@ void WorkerPool::AdjustWorkerOomScore(pid_t pid) const { #endif } -void WorkerPool::MonitorStartingWorkerProcess(const Process &proc, - StartupToken proc_startup_token, +void WorkerPool::MonitorStartingWorkerProcess(StartupToken proc_startup_token, const Language &language, const rpc::WorkerType worker_type) { auto timer = std::make_shared( @@ -553,7 +547,7 @@ void WorkerPool::MonitorStartingWorkerProcess(const Process &proc, boost::posix_time::seconds( RayConfig::instance().worker_register_timeout_seconds())); // Capture timer in lambda to copy it once, so that it can avoid destructing timer. - timer->async_wait([timer, language, proc = proc, proc_startup_token, worker_type, this]( + timer->async_wait([timer, language, proc_startup_token, worker_type, this]( const boost::system::error_code e) mutable { // check the error code. auto &state = this->GetStateForLanguage(language); @@ -562,26 +556,17 @@ void WorkerPool::MonitorStartingWorkerProcess(const Process &proc, auto it = state.worker_processes.find(proc_startup_token); if (it != state.worker_processes.end() && it->second.is_pending_registration) { RAY_LOG(ERROR) - << "Some workers of the worker process(" << proc.GetId() + << "Some workers of the worker process(" << it->second.proc.GetId() << ") have not registered within the timeout. " - << (proc.IsAlive() + << (it->second.proc.IsAlive() ? "The process is still alive, probably it's hanging during start." : "The process is dead, probably it crashed during start."); - if (proc.IsAlive()) { - proc.Kill(); + if (it->second.proc.IsAlive()) { + it->second.proc.Kill(); } - PopWorkerStatus status = PopWorkerStatus::WorkerPendingRegistration; process_failed_pending_registration_++; - bool found; - bool used; - InvokePopWorkerCallbackForProcess(state.starting_workers_to_tasks, - proc_startup_token, - nullptr, - status, - &found, - &used); DeleteRuntimeEnvIfPossible(it->second.runtime_env_info.serialized_runtime_env()); RemoveWorkerProcess(state, proc_startup_token); if (IsIOWorkerType(worker_type)) { @@ -592,13 +577,33 @@ void WorkerPool::MonitorStartingWorkerProcess(const Process &proc, // We may have places to start more workers now. TryStartIOWorkers(language); if (worker_type == rpc::WorkerType::WORKER) { - TryPendingPopWorkerRequests(language); + TryPendingStartRequests(language); } starting_worker_timeout_callback_(); } }); } +void WorkerPool::MonitorPopWorkerRequestForRegistration( + std::shared_ptr pop_worker_request) { + auto timer = std::make_shared( + *io_service_, + boost::posix_time::seconds( + RayConfig::instance().worker_register_timeout_seconds())); + // Capture timer in lambda to copy it once, so that it can avoid destructing timer. + timer->async_wait([timer, pop_worker_request = std::move(pop_worker_request), this]( + const boost::system::error_code e) mutable { + auto &state = GetStateForLanguage(pop_worker_request->language); + auto &requests = state.pending_registration_requests; + auto it = std::find(requests.begin(), requests.end(), pop_worker_request); + if (it != requests.end()) { + // Fail the task... + PopWorkerStatus status = PopWorkerStatus::WorkerPendingRegistration; + PopWorkerCallbackAsync(pop_worker_request->callback, nullptr, status); + } + }); +} + Process WorkerPool::StartProcess(const std::vector &worker_command_args, const ProcessEnvironment &env) { if (RAY_LOG_ENABLED(DEBUG)) { @@ -978,73 +983,65 @@ void WorkerPool::PopDeleteWorker( } } -void WorkerPool::InvokePopWorkerCallbackForProcess( - absl::flat_hash_map - &starting_workers_to_tasks, - StartupToken startup_token, - const std::shared_ptr &worker, - const PopWorkerStatus &status, - bool *found, - bool *worker_used) { - *found = false; - *worker_used = false; - auto it = starting_workers_to_tasks.find(startup_token); - if (it != starting_workers_to_tasks.end()) { - *found = true; - const auto &callback = it->second.callback; - RAY_CHECK(callback); - // This method shouldn't be invoked when runtime env creation has failed because - // when runtime env is failed to be created, they are all - // invoking the callback immediately. - RAY_CHECK(status != PopWorkerStatus::RuntimeEnvCreationFailed); - if (worker && finished_jobs_.contains(it->second.task_spec.JobId()) && - it->second.task_spec.RootDetachedActorId().IsNil()) { - // If the job has finished, we should fail the PopWorker callback - // and add the worker back to the idle workers so it can be killed later. - // This doesn't apply to detached actor and its descendants - // since they can outlive the job. - RAY_CHECK(status == PopWorkerStatus::OK); - callback(nullptr, PopWorkerStatus::JobFinished, ""); - } else { - *worker_used = callback(worker, status, /*runtime_env_setup_error_message*/ ""); - } - starting_workers_to_tasks.erase(it); - } -} - void WorkerPool::PushWorker(const std::shared_ptr &worker) { // Since the worker is now idle, unset its assigned task ID. RAY_CHECK(worker->GetAssignedTaskId().IsNil()) << "Idle workers cannot have an assigned task ID"; + + // Find a task that this worker can fit. If there's none, put it in the idle pool. + // First find in pending_registration_requests, then in pending_start_requests. + std::shared_ptr pop_worker_request = nullptr; auto &state = GetStateForLanguage(worker->GetLanguage()); - bool found; - bool used; - InvokePopWorkerCallbackForProcess(state.starting_workers_to_tasks, - worker->GetStartupToken(), - worker, - PopWorkerStatus::OK, - &found, - &used); - RAY_LOG(DEBUG) << "PushWorker " << worker->WorkerId() << " used: " << used; - if (!used) { - // Put the worker to the idle pool. + { + auto it = std::find_if( + state.pending_registration_requests.begin(), + state.pending_registration_requests.end(), + [this, &worker](const std::shared_ptr &pop_worker_request) { + return WorkerFitsForTask(*worker, *pop_worker_request) == + WorkerUnfitForTaskReason::NONE; + }); + if (it != state.pending_registration_requests.end()) { + pop_worker_request = *it; + state.pending_registration_requests.erase(it); + } + } + if (!pop_worker_request) { + auto it = std::find_if( + state.pending_start_requests.begin(), + state.pending_start_requests.end(), + [this, &worker](const std::shared_ptr &pop_worker_request) { + return WorkerFitsForTask(*worker, *pop_worker_request) == + WorkerUnfitForTaskReason::NONE; + }); + if (it != state.pending_start_requests.end()) { + pop_worker_request = *it; + state.pending_start_requests.erase(it); + } + } + + if (pop_worker_request) { + bool used = pop_worker_request->callback(worker, PopWorkerStatus::OK, ""); + if (!used) { + // Retry PushWorker. Maybe it can be used by other tasks. + // Can we have tail call optimization for this? :) + return PushWorker(worker); + } + } else { state.idle.insert(worker); auto now = get_time_(); - if (found) { - // If the worker was just started, then we should consider it first when - // choosing which idle workers to kill because it is cold. - idle_of_all_languages_.push_front(std::make_pair(worker, now)); + if (worker->GetAssignedTaskTime() == absl::Time()) { + // If the worker never held any tasks, then we should consider it first when + // choosing which idle workers to kill because it is not warmed up and is slower + // than those workers who served tasks before. + // See https://github.com/ray-project/ray/pull/36766 + idle_of_all_languages_.emplace_front(worker, now); } else { idle_of_all_languages_.emplace_back(worker, now); } - } else if (!found) { - RAY_LOG(INFO) << "Worker not returned to the idle pool after being used. This may " - "cause a worker leak, worker id:" - << worker->WorkerId(); } // We either have an idle worker or a slot to start a new worker. if (worker->GetWorkerType() == rpc::WorkerType::WORKER) { - TryPendingPopWorkerRequests(worker->GetLanguage()); + TryPendingStartRequests(worker->GetLanguage()); } } @@ -1162,102 +1159,164 @@ void WorkerPool::KillIdleWorker(std::shared_ptr idle_worker, } WorkerUnfitForTaskReason WorkerPool::WorkerFitsForTask( - const WorkerInterface &worker, const TaskSpecification &task_spec) const { + const WorkerInterface &worker, const PopWorkerRequest &pop_worker_request) const { if (worker.IsDead()) { return WorkerUnfitForTaskReason::OTHERS; } - if (worker.GetLanguage() != task_spec.GetLanguage()) { + // These workers are exiting. So skip them. + if (pending_exit_idle_workers_.contains(worker.WorkerId())) { + return WorkerUnfitForTaskReason::OTHERS; + } + if (worker.GetLanguage() != pop_worker_request.language) { return WorkerUnfitForTaskReason::OTHERS; } - // Don't allow worker reuse across jobs or root detached actors. Reuse worker with - // unassigned job_id and root detached actor id is OK. - JobID job_id = worker.GetAssignedJobId(); - if (!job_id.IsNil() && job_id != task_spec.JobId()) { - return WorkerUnfitForTaskReason::ROOT_MISMATCH; + if (worker.GetWorkerType() != pop_worker_request.worker_type) { + return WorkerUnfitForTaskReason::OTHERS; } - ActorID root_detached_actor_id = worker.GetRootDetachedActorId(); - if (!root_detached_actor_id.IsNil() && - root_detached_actor_id != task_spec.RootDetachedActorId()) { + + if (!IdMatches(pop_worker_request.root_detached_actor_id, + worker.GetRootDetachedActorId())) { return WorkerUnfitForTaskReason::ROOT_MISMATCH; } - auto is_gpu = worker.GetIsGpu(); - if (is_gpu.has_value()) { - bool task_is_gpu = - task_spec.GetRequiredResources().Get(scheduling::ResourceID::GPU()) > 0; - if (is_gpu.value() != task_is_gpu) { - return WorkerUnfitForTaskReason::OTHERS; + // Only compare job id for actors not rooted to a detached actor. + if (pop_worker_request.root_detached_actor_id.IsNil()) { + if (!IdMatches(pop_worker_request.job_id, worker.GetAssignedJobId())) { + return WorkerUnfitForTaskReason::ROOT_MISMATCH; } } - auto is_actor_worker = worker.GetIsActorWorker(); - if (is_actor_worker.has_value() && - is_actor_worker.value() != task_spec.IsActorCreationTask()) { + // If the request asks for a is_gpu, and the worker is assigned a different is_gpu, + // then skip it. + if (!OptionalMatches(pop_worker_request.is_gpu, worker.GetIsGpu())) { + return WorkerUnfitForTaskReason::OTHERS; + } + // If the request asks for a is_actor_worker, and the worker is assigned a different + // is_actor_worker, then skip it. + if (!OptionalMatches(pop_worker_request.is_actor_worker, worker.GetIsActorWorker())) { return WorkerUnfitForTaskReason::OTHERS; } // TODO(clarng): consider re-using worker that has runtime envionrment // if the task doesn't require one. - if (worker.GetRuntimeEnvHash() != task_spec.GetRuntimeEnvHash()) { + if (worker.GetRuntimeEnvHash() != pop_worker_request.runtime_env_hash) { return WorkerUnfitForTaskReason::RUNTIME_ENV_MISMATCH; } // Skip if the dynamic_options doesn't match. if (LookupWorkerDynamicOptions(worker.GetStartupToken()) != - task_spec.DynamicWorkerOptionsOrEmpty()) { + pop_worker_request.dynamic_options) { return WorkerUnfitForTaskReason::DYNAMIC_OPTIONS_MISMATCH; } - // These workers are exiting. So skip them. - if (pending_exit_idle_workers_.contains(worker.WorkerId())) { - return WorkerUnfitForTaskReason::OTHERS; - } return WorkerUnfitForTaskReason::NONE; } -void WorkerPool::PopWorker(const TaskSpecification &task_spec, - const PopWorkerCallback &callback) { - RAY_LOG(DEBUG) << "Pop worker for task " << task_spec.TaskId() << " task name " - << task_spec.FunctionDescriptor()->ToString(); - auto &state = GetStateForLanguage(task_spec.GetLanguage()); +void WorkerPool::StartNewWorker( + const std::shared_ptr &pop_worker_request) { + auto start_worker_process_fn = [this]( + std::shared_ptr pop_worker_request, + const std::string &serialized_runtime_env_context) { + auto &state = GetStateForLanguage(pop_worker_request->language); + const std::string &serialized_runtime_env = + pop_worker_request->runtime_env_info.serialized_runtime_env(); - std::shared_ptr worker = nullptr; - auto start_worker_process_fn = [this](const TaskSpecification &task_spec, - State &state, - const std::string &serialized_runtime_env_context, - const PopWorkerCallback &callback) { PopWorkerStatus status = PopWorkerStatus::OK; - auto [proc, startup_token] = - StartWorkerProcess(task_spec.GetLanguage(), - rpc::WorkerType::WORKER, - task_spec.JobId(), - &status, - task_spec.DynamicWorkerOptionsOrEmpty(), - task_spec.GetRuntimeEnvHash(), - serialized_runtime_env_context, - task_spec.RuntimeEnvInfo()); + auto [proc, startup_token] = StartWorkerProcess(pop_worker_request->language, + pop_worker_request->worker_type, + pop_worker_request->job_id, + &status, + pop_worker_request->dynamic_options, + pop_worker_request->runtime_env_hash, + serialized_runtime_env_context, + pop_worker_request->runtime_env_info); if (status == PopWorkerStatus::OK) { RAY_CHECK(proc.IsValid()); WarnAboutSize(); - auto task_info = TaskWaitingForWorkerInfo{task_spec, callback}; - state.starting_workers_to_tasks[startup_token] = std::move(task_info); + state.pending_registration_requests.emplace_back(pop_worker_request); + MonitorPopWorkerRequestForRegistration(pop_worker_request); } else if (status == PopWorkerStatus::TooManyStartingWorkerProcesses) { // TODO(jjyao) As an optimization, we don't need to delete the runtime env // but reuse it the next time we retry the request. - DeleteRuntimeEnvIfPossible(task_spec.SerializedRuntimeEnv()); - state.pending_pop_worker_requests.emplace_back( - PopWorkerRequest{task_spec, callback}); + DeleteRuntimeEnvIfPossible(serialized_runtime_env); + state.pending_start_requests.emplace_back(std::move(pop_worker_request)); } else { - DeleteRuntimeEnvIfPossible(task_spec.SerializedRuntimeEnv()); - PopWorkerCallbackAsync(task_spec, callback, nullptr, status); + DeleteRuntimeEnvIfPossible(serialized_runtime_env); + PopWorkerCallbackAsync(std::move(pop_worker_request->callback), nullptr, status); } }; + const std::string &serialized_runtime_env = + pop_worker_request->runtime_env_info.serialized_runtime_env(); + + if (!IsRuntimeEnvEmpty(serialized_runtime_env)) { + // create runtime env. + GetOrCreateRuntimeEnv( + serialized_runtime_env, + pop_worker_request->runtime_env_info.runtime_env_config(), + pop_worker_request->job_id, + [this, start_worker_process_fn, pop_worker_request]( + bool successful, + const std::string &serialized_runtime_env_context, + const std::string &setup_error_message) { + if (successful) { + start_worker_process_fn(pop_worker_request, serialized_runtime_env_context); + } else { + process_failed_runtime_env_setup_failed_++; + pop_worker_request->callback( + nullptr, + PopWorkerStatus::RuntimeEnvCreationFailed, + /*runtime_env_setup_error_message*/ setup_error_message); + } + }); + } else { + start_worker_process_fn(pop_worker_request, ""); + } +} + +void WorkerPool::PopWorker(const TaskSpecification &task_spec, + const PopWorkerCallback &callback) { + RAY_LOG(DEBUG) << "Pop worker for task " << task_spec.TaskId() << " task name " + << task_spec.FunctionDescriptor()->ToString(); // Code path of actor task. RAY_CHECK(!task_spec.IsActorTask()) << "Direct call shouldn't reach here."; + auto pop_worker_request = std::make_shared( + task_spec.GetLanguage(), + rpc::WorkerType::WORKER, + task_spec.JobId(), + task_spec.RootDetachedActorId(), + /*is_gpu=*/task_spec.GetRequiredResources().Get(scheduling::ResourceID::GPU()) > 0, + /*is_actor_worker=*/task_spec.IsActorCreationTask(), + task_spec.RuntimeEnvInfo(), + task_spec.GetRuntimeEnvHash(), + task_spec.DynamicWorkerOptionsOrEmpty(), + [this, task_spec, callback]( + const std::shared_ptr &worker, + PopWorkerStatus status, + const std::string &runtime_env_setup_error_message) -> bool { + // We got a worker suitable for the task. Now let's check if the task is still + // executable. + if (worker && finished_jobs_.contains(task_spec.JobId()) && + task_spec.RootDetachedActorId().IsNil()) { + // When a job finishes, node manager will kill leased workers one time + // and worker pool will kill idle workers periodically. + // The current worker is already removed from the idle workers + // but hasn't been added to the leased workers since the callback is not called + // yet. We shouldn't add this worker to the leased workers since killing leased + // workers for this finished job may already happen and won't happen again (this + // is one time) so it will cause a process leak. Instead we fail the PopWorker + // and add the worker back to the idle workers so it can be killed later. + RAY_CHECK(status == PopWorkerStatus::OK); + callback(nullptr, PopWorkerStatus::JobFinished, ""); + // Not used + return false; + } + return callback(worker, status, runtime_env_setup_error_message); + }); + absl::flat_hash_map skip_reason_count; auto worker_fits_for_task_fn = - [this, &task_spec, &skip_reason_count]( + [this, &pop_worker_request, &skip_reason_count]( const std::pair, int64_t> &pair) -> bool { const auto &worker = pair.first; - WorkerUnfitForTaskReason reason = WorkerFitsForTask(*worker, task_spec); + WorkerUnfitForTaskReason reason = WorkerFitsForTask(*worker, *pop_worker_request); if (reason == WorkerUnfitForTaskReason::NONE) { return true; } @@ -1271,7 +1330,8 @@ void WorkerPool::PopWorker(const TaskSpecification &task_spec, } return false; }; - + auto &state = GetStateForLanguage(task_spec.GetLanguage()); + std::shared_ptr worker = nullptr; auto good_worker_it = std::find_if(idle_of_all_languages_.rbegin(), idle_of_all_languages_.rend(), worker_fits_for_task_fn); @@ -1284,45 +1344,19 @@ void WorkerPool::PopWorker(const TaskSpecification &task_spec, idle_of_all_languages_.erase(lit); } + // If there's an idle worker that fits the task, use it. + // Else, start a new worker. if (worker == nullptr) { RAY_LOG(DEBUG) << "No cached worker, cached workers skipped due to " << debug_string(skip_reason_count); - if (task_spec.HasRuntimeEnv()) { - // create runtime env. - RAY_LOG(DEBUG) << "GetOrCreateRuntimeEnv for task " << task_spec.TaskId(); - GetOrCreateRuntimeEnv( - task_spec.SerializedRuntimeEnv(), - task_spec.RuntimeEnvConfig(), - task_spec.JobId(), - [this, start_worker_process_fn, callback, &state, task_spec]( - bool successful, - const std::string &serialized_runtime_env_context, - const std::string &setup_error_message) { - if (successful) { - start_worker_process_fn( - task_spec, state, serialized_runtime_env_context, callback); - } else { - process_failed_runtime_env_setup_failed_++; - callback(nullptr, - PopWorkerStatus::RuntimeEnvCreationFailed, - /*runtime_env_setup_error_message*/ setup_error_message); - RAY_LOG(WARNING) << "Create runtime env failed for task " - << task_spec.TaskId() - << " and couldn't create the worker."; - } - }); - } else { - start_worker_process_fn(task_spec, state, "", callback); - } - } - - if (worker) { + StartNewWorker(pop_worker_request); + } else { RAY_CHECK(worker->GetAssignedJobId().IsNil() || worker->GetAssignedJobId() == task_spec.JobId()); RAY_LOG(DEBUG) << "Re-using worker " << worker->WorkerId() << " for task " << task_spec.DebugString(); stats::NumWorkersStartedFromCache.Record(1); - PopWorkerCallbackAsync(task_spec, callback, worker); + PopWorkerCallbackAsync(pop_worker_request->callback, worker, PopWorkerStatus::OK); } } @@ -1391,7 +1425,7 @@ void WorkerPool::DisconnectWorker(const std::shared_ptr &worker // This may add new workers to state.worker_processes // and invalidate the iterator, do not use `it` // after this call. - TryPendingPopWorkerRequests(worker->GetLanguage()); + TryPendingStartRequests(worker->GetLanguage()); } } @@ -1525,16 +1559,16 @@ void WorkerPool::TryStartIOWorkers(const Language &language) { TryStartIOWorkers(language, rpc::WorkerType::SPILL_WORKER); } -void WorkerPool::TryPendingPopWorkerRequests(const Language &language) { +void WorkerPool::TryPendingStartRequests(const Language &language) { auto &state = GetStateForLanguage(language); - if (state.pending_pop_worker_requests.empty()) { + if (state.pending_start_requests.empty()) { return; } - std::deque pending_pop_worker_requests; - state.pending_pop_worker_requests.swap(pending_pop_worker_requests); - for (const auto &pop_worker_request : pending_pop_worker_requests) { - PopWorker(pop_worker_request.task_spec, pop_worker_request.callback); + std::deque> pending_start_requests; + state.pending_start_requests.swap(pending_start_requests); + for (const auto &request : pending_start_requests) { + StartNewWorker(request); } } @@ -1585,6 +1619,11 @@ std::string WorkerPool::DebugString() const { << " workers: " << entry.second.registered_workers.size(); result << "\n- num " << Language_Name(entry.first) << " drivers: " << entry.second.registered_drivers.size(); + result << "\n- num " << Language_Name(entry.first) + << " pending start requests: " << entry.second.pending_start_requests.size(); + result << "\n- num " << Language_Name(entry.first) + << " pending registration requests: " + << entry.second.pending_registration_requests.size(); result << "\n- num object spill callbacks queued: " << entry.second.spill_io_worker_state.pending_io_tasks.size(); result << "\n- num object restore queued: " diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index 58ddc18870656..6d71290ca832f 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -73,7 +73,7 @@ enum PopWorkerStatus { /// \return true if the worker was used. Otherwise, return false /// and the worker will be returned to the worker pool. using PopWorkerCallback = - std::function worker, + std::function &worker, PopWorkerStatus status, const std::string &runtime_env_setup_error_message)>; @@ -437,11 +437,11 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// we didn't start a process. std::tuple StartWorkerProcess( const Language &language, - const rpc::WorkerType worker_type, + rpc::WorkerType worker_type, const JobID &job_id, PopWorkerStatus *status /*output*/, const std::vector &dynamic_options = {}, - const int runtime_env_hash = 0, + int runtime_env_hash = 0, const std::string &serialized_runtime_env_context = "{}", const rpc::RuntimeEnvInfo &runtime_env_info = rpc::RuntimeEnvInfo()); @@ -460,8 +460,7 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { virtual void WarnAboutSize(); /// Make this synchronized function for unit test. - void PopWorkerCallbackInternal(const TaskSpecification &task_spec, - const PopWorkerCallback &callback, + void PopWorkerCallbackInternal(const PopWorkerCallback &callback, std::shared_ptr worker, PopWorkerStatus status); @@ -504,19 +503,44 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { std::vector dynamic_options; }; - struct TaskWaitingForWorkerInfo { - /// The spec of task. - TaskSpecification task_spec; - /// The callback function which should be called when worker registered. - PopWorkerCallback callback; - }; - - /// Represents a PopWorker call. struct PopWorkerRequest { - TaskSpecification task_spec; + rpc::Language language; + rpc::WorkerType worker_type; + JobID job_id; // can be Nil + ActorID root_detached_actor_id; // can be Nil + std::optional is_gpu; + std::optional is_actor_worker; + rpc::RuntimeEnvInfo runtime_env_info; + int runtime_env_hash; + std::vector dynamic_options; + PopWorkerCallback callback; + + PopWorkerRequest(rpc::Language lang, + rpc::WorkerType worker_type, + JobID job, + ActorID root_actor_id, + std::optional gpu, + std::optional actor_worker, + rpc::RuntimeEnvInfo runtime_env_info, + int runtime_hash, + std::vector options, + PopWorkerCallback callback) + : language(lang), + worker_type(worker_type), + job_id(job), + root_detached_actor_id(root_actor_id), + is_gpu(gpu), + is_actor_worker(actor_worker), + runtime_env_info(std::move(runtime_env_info)), + runtime_env_hash(runtime_hash), + dynamic_options(std::move(options)), + callback(std::move(callback)) {} }; + // Starts a new worker that fulfills `pop_worker_request`. + void StartNewWorker(const std::shared_ptr &pop_worker_request); + /// An internal data structure that maintains the pool state per language. struct State { /// The commands and arguments used to start the worker process @@ -538,10 +562,13 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// the extra information of the process. Note that the shim process PID is the /// same with worker process PID, except worker process in container. absl::flat_hash_map worker_processes; - /// A map for looking up the task by the startup token of starting worker process. - absl::flat_hash_map starting_workers_to_tasks; - /// Pop worker requests that are pending due to maximum_startup_concurrency_. - std::deque pending_pop_worker_requests; + /// FIFO queue of pending requests with workers STARTED but pending registration. + /// If a request stays in this status for >= worker_register_timeout_seconds, we'll + /// fail the request and kill the worker process. + std::deque> pending_registration_requests; + /// FIFO queue of pending requests with workers NOT STARTED due to + /// maximum_startup_concurrency_. + std::deque> pending_start_requests; /// We'll push a warning to the user every time a multiple of this many /// worker processes has been started. int multiple_for_warning; @@ -569,10 +596,17 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// (due to worker process crash or any other reasons), remove them /// from `worker_processes`. Otherwise if we'll mistakenly /// think there are unregistered workers, and won't start new workers. - void MonitorStartingWorkerProcess(const Process &proc, - StartupToken proc_startup_token, + void MonitorStartingWorkerProcess(StartupToken proc_startup_token, const Language &language, - const rpc::WorkerType worker_type); + rpc::WorkerType worker_type); + + /// Start a timer to monitor the starting worker process. + /// Called when a worker process is started and waiting for registration for the + /// request. If the registration is not finished within the timeout, we'll failed the + /// request. Note we don't do anything to the worker process itself, as it's timed out + /// by MonitorStartingWorkerProcess. + void MonitorPopWorkerRequestForRegistration( + std::shared_ptr pop_worker_request); /// Get the next unallocated port in the free ports list. If a port range isn't /// configured, returns 0. @@ -598,10 +632,10 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// worker. void TryStartIOWorkers(const Language &language, const rpc::WorkerType &worker_type); - /// Try to fulfill pending PopWorker requests. + /// Try to fulfill pending_start_requests by trying to start more workers. /// This happens when we have more room to start workers or an idle worker is pushed. /// \param language The language of the PopWorker requests. - void TryPendingPopWorkerRequests(const Language &language); + void TryPendingStartRequests(const Language &language); /// Get either restore or spill worker state from state based on worker_type. /// @@ -626,29 +660,9 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// Call the `PopWorkerCallback` function asynchronously to make sure executed in /// different stack. - virtual void PopWorkerCallbackAsync(const TaskSpecification &task_spec, - const PopWorkerCallback &callback, + virtual void PopWorkerCallbackAsync(PopWorkerCallback callback, std::shared_ptr worker, - PopWorkerStatus status = PopWorkerStatus::OK); - - /// Try to find a task that is associated with the given worker process from the given - /// queue. If found, invoke its PopWorkerCallback. - /// \param workers_to_tasks The queue of tasks which waiting for workers. - /// \param startup_token The startup token representing the worker. - /// \param worker A new idle worker. If the worker is empty, we could also callback - /// to the task. - /// \param status The pop worker status which will be forwarded to - /// `PopWorkerCallback`. - /// \param found Whether the related task found or not. - /// \param worker_used Whether the worker is used by the task, only valid when found is - /// true. - void InvokePopWorkerCallbackForProcess( - absl::flat_hash_map &workers_to_tasks, - StartupToken startup_token, - const std::shared_ptr &worker, - const PopWorkerStatus &status, - bool *found /* output */, - bool *worker_used /* output */); + PopWorkerStatus status); /// We manage all runtime env resources locally by the two methods: /// `GetOrCreateRuntimeEnv` and `DeleteRuntimeEnvIfPossible`. @@ -706,18 +720,18 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { std::pair, ProcessEnvironment> BuildProcessCommandArgs( const Language &language, rpc::JobConfig *job_config, - const rpc::WorkerType worker_type, + rpc::WorkerType worker_type, const JobID &job_id, const std::vector &dynamic_options, - const int runtime_env_hash, + int runtime_env_hash, const std::string &serialized_runtime_env_context, const WorkerPool::State &state) const; void ExecuteOnPrestartWorkersStarted(std::function callback); // If this worker can serve the task. - WorkerUnfitForTaskReason WorkerFitsForTask(const WorkerInterface &worker, - const TaskSpecification &task_spec) const; + WorkerUnfitForTaskReason WorkerFitsForTask( + const WorkerInterface &worker, const PopWorkerRequest &pop_worker_request) const; /// For Process class for managing subprocesses (e.g. reaping zombies). instrumented_io_context *io_service_; diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 8cfd03201e34c..d945384b72774 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -44,7 +44,7 @@ std::vector LANGUAGES = {Language::PYTHON, Language::JAVA}; class MockWorkerClient : public rpc::CoreWorkerClientInterface { public: - MockWorkerClient() {} + MockWorkerClient() = default; void Exit(const rpc::ExitRequest &request, const rpc::ClientCallback &callback) { @@ -84,7 +84,7 @@ class MockWorkerClient : public rpc::CoreWorkerClientInterface { static std::unordered_map runtime_env_reference; -static int GetReferenceCount(const std::string serialized_runtime_env) { +static int GetReferenceCount(const std::string &serialized_runtime_env) { auto it = runtime_env_reference.find(serialized_runtime_env); return it == runtime_env_reference.end() ? 0 : it->second; } @@ -105,7 +105,7 @@ class MockRuntimeEnvAgentClient : public RuntimeEnvAgentClient { } else { runtime_env_reference[serialized_runtime_env] += 1; } - callback(true, "{\"dummy\":\"dummy\"}", ""); + callback(true, R"({"dummy":"dummy"})", ""); } }; @@ -159,17 +159,16 @@ class WorkerPoolMock : public WorkerPool { using WorkerPool::PopWorkerCallbackInternal; // Mock `PopWorkerCallbackAsync` to synchronized function. - void PopWorkerCallbackAsync(const TaskSpecification &task_spec, - const PopWorkerCallback &callback, + void PopWorkerCallbackAsync(PopWorkerCallback callback, std::shared_ptr worker, PopWorkerStatus status = PopWorkerStatus::OK) override { - PopWorkerCallbackInternal(task_spec, callback, worker, status); + PopWorkerCallbackInternal(callback, worker, status); } Process StartProcess(const std::vector &worker_command_args, const ProcessEnvironment &env) override { // Use a bogus process ID that won't conflict with those in the system - pid_t pid = static_cast(PID_MAX_LIMIT + 1 + worker_commands_by_proc_.size()); + auto pid = static_cast(PID_MAX_LIMIT + 1 + worker_commands_by_proc_.size()); last_worker_process_ = Process::FromPid(pid); worker_commands_by_proc_[last_worker_process_] = worker_command_args; startup_tokens_by_proc_[last_worker_process_] = @@ -195,10 +194,18 @@ class WorkerPoolMock : public WorkerPool { return total; } - int NumPendingPopWorkerRequests() const { + int NumPendingStartRequests() const { int total = 0; for (auto &entry : states_by_lang_) { - total += entry.second.pending_pop_worker_requests.size(); + total += entry.second.pending_start_requests.size(); + } + return total; + } + + int NumPendingRegistrationRequests() const { + int total = 0; + for (auto &entry : states_by_lang_) { + total += entry.second.pending_registration_requests.size(); } return total; } @@ -297,7 +304,7 @@ class WorkerPoolMock : public WorkerPool { // Create workers for processes and push them to worker pool. // \param[in] timeout_worker_number Don't register some workers to simulate worker // registration timeout. - void PushWorkers(int timeout_worker_number = 0) { + void PushWorkers(int timeout_worker_number, JobID job_id) { auto processes = GetProcesses(); for (auto it = processes.begin(); it != processes.end(); ++it) { auto pushed_it = pushedProcesses_.find(it->first); @@ -326,7 +333,7 @@ class WorkerPoolMock : public WorkerPool { auto worker = CreateWorker( it->first, is_java ? Language::JAVA : Language::PYTHON, - JOB_ID, + job_id, rpc::WorkerType::WORKER, runtime_env_hash, startup_tokens_by_proc_[it->first], @@ -372,7 +379,7 @@ class WorkerPoolMock : public WorkerPool { return true; }); if (push_workers) { - PushWorkers(timeout_worker_number); + PushWorkers(timeout_worker_number, task_spec.JobId()); } promise.get_future().get(); return popped_worker; @@ -839,7 +846,8 @@ TEST_F(WorkerPoolDriverRegisteredTest, MaximumStartupConcurrency) { started_processes.push_back(last_process); } ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); - ASSERT_EQ(0, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(0, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumPendingRegistrationRequests()); // Can't start a new worker process at this point. worker_pool_->PopWorker( @@ -848,7 +856,8 @@ TEST_F(WorkerPoolDriverRegisteredTest, MaximumStartupConcurrency) { PopWorkerStatus status, const std::string &runtime_env_setup_error_message) -> bool { return true; }); ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); - ASSERT_EQ(1, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(1, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumPendingRegistrationRequests()); std::vector> workers; // Call `RegisterWorker` to emulate worker registration. @@ -860,7 +869,10 @@ TEST_F(WorkerPoolDriverRegisteredTest, MaximumStartupConcurrency) { })); // Calling `RegisterWorker` won't affect the counter of starting worker processes. ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); - ASSERT_EQ(1, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(1, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, + worker_pool_->NumPendingRegistrationRequests()); + workers.push_back(worker); } @@ -872,7 +884,8 @@ TEST_F(WorkerPoolDriverRegisteredTest, MaximumStartupConcurrency) { PopWorkerStatus status, const std::string &runtime_env_setup_error_message) -> bool { return true; }); ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); - ASSERT_EQ(2, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(2, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumPendingRegistrationRequests()); // Call `OnWorkerStarted` to emulate worker port announcement. worker_pool_->OnWorkerStarted(workers[0]); @@ -881,7 +894,8 @@ TEST_F(WorkerPoolDriverRegisteredTest, MaximumStartupConcurrency) { // One pending pop worker request now can be fulfilled. ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY + 1, worker_pool_->GetProcessSize()); - ASSERT_EQ(1, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(1, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumPendingRegistrationRequests()); // Can't start a new worker process at this point. worker_pool_->PopWorker( @@ -891,22 +905,28 @@ TEST_F(WorkerPoolDriverRegisteredTest, MaximumStartupConcurrency) { const std::string &runtime_env_setup_error_message) -> bool { return true; }); ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY + 1, worker_pool_->GetProcessSize()); - ASSERT_EQ(2, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(2, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumPendingRegistrationRequests()); // Return a worker. worker_pool_->PushWorker(workers[0]); - // One more pending pop worker request can be fulfilled. + // The pushed worker fulfills a pending registration request, not a pending start + // request. ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY + 1, worker_pool_->GetProcessSize()); - ASSERT_EQ(1, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(2, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY - 1, + worker_pool_->NumPendingRegistrationRequests()); + ASSERT_EQ(0, worker_pool_->GetIdleWorkerSize()); // Disconnect a worker. worker_pool_->DisconnectWorker(workers[1], rpc::WorkerExitType::SYSTEM_ERROR); - // One more pending pop worker request can be fulfilled. + // We have 1 more slot to start a new worker process. ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumWorkersStarting()); ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY + 2, worker_pool_->GetProcessSize()); - ASSERT_EQ(0, worker_pool_->NumPendingPopWorkerRequests()); + ASSERT_EQ(1, worker_pool_->NumPendingStartRequests()); + ASSERT_EQ(MAXIMUM_STARTUP_CONCURRENCY, worker_pool_->NumPendingRegistrationRequests()); ASSERT_EQ(0, worker_pool_->GetIdleWorkerSize()); worker_pool_->ClearProcesses(); @@ -1189,6 +1209,11 @@ TEST_F(WorkerPoolDriverRegisteredTest, TestWorkerCapping) { auto task_spec = ExampleTaskSpec(/*actor_id=*/ActorID::Nil(), Language::PYTHON, job_id); auto worker = worker_pool_->PopWorkerSync(task_spec, false); + // Simulate running the task and finish. This is to set task_assign_time_. + RayTask task(task_spec); + worker->SetAssignedTask(task); + worker->AssignTaskId(TaskID::Nil()); + popped_workers.push_back(worker); ASSERT_TRUE(worker); ASSERT_EQ(worker->GetAssignedJobId(), job_id); @@ -1217,7 +1242,8 @@ TEST_F(WorkerPoolDriverRegisteredTest, TestWorkerCapping) { // Since the idle workers are killed in FIFO, we can assume the first entry in the idle // workers will be killed. auto mock_rpc_client_it = mock_worker_rpc_clients_.find(popped_workers[0]->WorkerId()); - ASSERT_EQ(mock_rpc_client_it->second->exit_count, 1); + ASSERT_EQ(mock_rpc_client_it->second->exit_count, 1) + << " expected pid " << popped_workers[0]->GetProcess().GetId(); ASSERT_EQ(mock_rpc_client_it->second->last_exit_forced, false); mock_rpc_client_it->second->ExitReplySucceed(); worker_pool_->TryKillingIdleWorkers(); @@ -1420,19 +1446,22 @@ TEST_F(WorkerPoolDriverRegisteredTest, TestJobFinishedForPopWorker) { task_spec = ExampleTaskSpec(/*actor_id=*/ActorID::Nil(), Language::PYTHON, job_id); pop_worker_status = PopWorkerStatus::OK; // This will start a new worker. + std::promise promise; worker_pool_->PopWorker( task_spec, [&](const std::shared_ptr worker, PopWorkerStatus status, const std::string &runtime_env_setup_error_message) -> bool { pop_worker_status = status; + promise.set_value(true); return false; }); auto process = worker_pool_->LastStartedWorkerProcess(); RAY_CHECK(process.IsValid()); ASSERT_EQ(1, worker_pool_->NumWorkersStarting()); - worker = worker_pool_->CreateWorker(Process()); + // Starts a worker for JOB_ID2. + worker = worker_pool_->CreateWorker(Process(), Language::PYTHON, job_id); worker->SetStartupToken(worker_pool_->GetStartupToken(process)); RAY_CHECK_OK(worker_pool_->RegisterWorker( worker, process.GetId(), worker_pool_->GetStartupToken(process), [](Status, int) { @@ -1446,8 +1475,10 @@ TEST_F(WorkerPoolDriverRegisteredTest, TestJobFinishedForPopWorker) { // Finish the job. worker_pool_->HandleJobFinished(job_id); - // This will trigger the PopWorker callback. + // This will trigger the PopWorker callback in async. worker_pool_->PushWorker(worker); + promise.get_future().get(); + ASSERT_EQ(pop_worker_status, PopWorkerStatus::JobFinished); ASSERT_EQ(worker_pool_->GetIdleWorkerSize(), 1); @@ -1807,7 +1838,7 @@ TEST_F(WorkerPoolDriverRegisteredTest, WorkerNoLeaks) { // No idle workers because no workers pushed. ASSERT_EQ(worker_pool_->GetIdleWorkerSize(), 0); // push workers. - worker_pool_->PushWorkers(); + worker_pool_->PushWorkers(0, task_spec.JobId()); // The worker has been pushed but not dispatched. ASSERT_EQ(worker_pool_->GetIdleWorkerSize(), 1); // Pop a worker and don't dispatch. @@ -2012,7 +2043,7 @@ TEST_F(WorkerPoolDriverRegisteredTest, WorkerReuseForPrestartedWorker) { const auto task_spec = ExampleTaskSpec(); worker_pool_->PrestartDefaultCpuWorkers(ray::Language::PYTHON, 1); - worker_pool_->PushWorkers(); + worker_pool_->PushWorkers(0, task_spec.JobId()); // One worker process has been prestarted. ASSERT_EQ(worker_pool_->GetProcessSize(), 1); ASSERT_EQ(worker_pool_->GetIdleWorkerSize(), 1); From 62b0205324bc0cff5fec17d68e688433bb351c92 Mon Sep 17 00:00:00 2001 From: dentiny Date: Fri, 11 Oct 2024 00:02:32 -0700 Subject: [PATCH 15/46] [core] Add metrics for gcs jobs (#47793) This PR adds metrics for job states within job manager. In detail, a gauge stats is sent via opencensus exporter, so running ray jobs could be tracked and alerts could be created later on. Fault tolerance is not considered, according to [doc](https://docs.ray.io/en/latest/ray-core/fault_tolerance/gcs.html), state is re-constructed at restart. On testing, the best way is to observe via opencensus backend (i.e. google monitoring dashboard), but not easy for open-source contributors; or to have a mock / fake exporter implementation, which I don't find in the code base. Signed-off-by: dentiny Co-authored-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com> --- src/ray/gcs/gcs_server/gcs_actor_manager.cc | 4 +- src/ray/gcs/gcs_server/gcs_job_manager.cc | 70 +++++++++++++++------ src/ray/gcs/gcs_server/gcs_job_manager.h | 29 +++++++-- src/ray/gcs/gcs_server/gcs_server.cc | 1 + src/ray/gcs/gcs_server/gcs_table_storage.h | 4 +- src/ray/stats/metric_defs.cc | 15 +++++ src/ray/stats/metric_defs.h | 4 ++ src/ray/util/counter_map.h | 5 +- 8 files changed, 102 insertions(+), 30 deletions(-) diff --git a/src/ray/gcs/gcs_server/gcs_actor_manager.cc b/src/ray/gcs/gcs_server/gcs_actor_manager.cc index de581a5e9405a..0c30514c1e32b 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_actor_manager.cc @@ -342,8 +342,8 @@ GcsActorManager::GcsActorManager( actor_gc_delay_(RayConfig::instance().gcs_actor_table_min_duration_ms()) { RAY_CHECK(worker_client_factory_); RAY_CHECK(destroy_owned_placement_group_if_needed_); - actor_state_counter_.reset( - new CounterMap>()); + actor_state_counter_ = std::make_shared< + CounterMap>>(); actor_state_counter_->SetOnChangeCallback( [this](const std::pair key) mutable { int64_t num_actors = actor_state_counter_->Get(key); diff --git a/src/ray/gcs/gcs_server/gcs_job_manager.cc b/src/ray/gcs/gcs_server/gcs_job_manager.cc index 02b08ba7d77bc..23432d1e51718 100644 --- a/src/ray/gcs/gcs_server/gcs_job_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_job_manager.cc @@ -15,17 +15,21 @@ #include "ray/gcs/gcs_server/gcs_job_manager.h" #include "ray/gcs/pb_util.h" +#include "ray/stats/metric.h" namespace ray { namespace gcs { void GcsJobManager::Initialize(const GcsInitData &gcs_init_data) { - for (auto &pair : gcs_init_data.Jobs()) { - const auto &job_id = pair.first; - const auto &job_table_data = pair.second; + for (const auto &[job_id, job_table_data] : gcs_init_data.Jobs()) { cached_job_configs_[job_id] = std::make_shared(job_table_data.config()); function_manager_.AddJobReference(job_id); + + // Recover [running_job_ids_] from storage. + if (!job_table_data.is_dead()) { + running_job_ids_.insert(job_id); + } } } @@ -82,28 +86,36 @@ void GcsJobManager::HandleAddJob(rpc::AddJobRequest request, auto time = current_sys_time_ms(); mutable_job_table_data.set_start_time(time); mutable_job_table_data.set_timestamp(time); - JobID job_id = JobID::FromBinary(mutable_job_table_data.job_id()); + const JobID job_id = JobID::FromBinary(mutable_job_table_data.job_id()); RAY_LOG(INFO) << "Adding job, job id = " << job_id << ", driver pid = " << mutable_job_table_data.driver_pid(); - auto on_done = [this, job_id, mutable_job_table_data, reply, send_reply_callback]( - const Status &status) { + auto on_done = [this, + job_id, + job_table_data = mutable_job_table_data, + reply, + send_reply_callback = + std::move(send_reply_callback)](const Status &status) { if (!status.ok()) { RAY_LOG(ERROR) << "Failed to add job, job id = " << job_id - << ", driver pid = " << mutable_job_table_data.driver_pid(); + << ", driver pid = " << job_table_data.driver_pid(); } else { - RAY_CHECK_OK(gcs_publisher_->PublishJob(job_id, mutable_job_table_data, nullptr)); - if (mutable_job_table_data.config().has_runtime_env_info()) { - runtime_env_manager_.AddURIReference( - job_id.Hex(), mutable_job_table_data.config().runtime_env_info()); + RAY_CHECK_OK(gcs_publisher_->PublishJob(job_id, job_table_data, /*done=*/nullptr)); + if (job_table_data.config().has_runtime_env_info()) { + runtime_env_manager_.AddURIReference(job_id.Hex(), + job_table_data.config().runtime_env_info()); } function_manager_.AddJobReference(job_id); RAY_LOG(INFO) << "Finished adding job, job id = " << job_id - << ", driver pid = " << mutable_job_table_data.driver_pid(); + << ", driver pid = " << job_table_data.driver_pid(); cached_job_configs_[job_id] = - std::make_shared(mutable_job_table_data.config()); + std::make_shared(job_table_data.config()); + + // Intentionally not checking return value, since the function could be invoked for + // multiple times and requires idempotency (i.e. due to retry). + running_job_ids_.insert(job_id); } - WriteDriverJobExportEvent(mutable_job_table_data); + WriteDriverJobExportEvent(job_table_data); GCS_RPC_SEND_REPLY(send_reply_callback, reply, status); }; @@ -122,7 +134,8 @@ void GcsJobManager::MarkJobAsFinished(rpc::JobTableData job_table_data, job_table_data.set_timestamp(time); job_table_data.set_end_time(time); job_table_data.set_is_dead(true); - auto on_done = [this, job_id, job_table_data, done_callback](const Status &status) { + auto on_done = [this, job_id, job_table_data, done_callback = std::move(done_callback)]( + const Status &status) { if (!status.ok()) { RAY_LOG(ERROR) << "Failed to mark job state, job id = " << job_id; } else { @@ -133,6 +146,13 @@ void GcsJobManager::MarkJobAsFinished(rpc::JobTableData job_table_data, } function_manager_.RemoveJobReference(job_id); WriteDriverJobExportEvent(job_table_data); + + // Update running job status. + auto iter = running_job_ids_.find(job_id); + RAY_CHECK(iter != running_job_ids_.end()); + running_job_ids_.erase(iter); + ++finished_jobs_count_; + done_callback(status); }; @@ -147,21 +167,28 @@ void GcsJobManager::HandleMarkJobFinished(rpc::MarkJobFinishedRequest request, rpc::SendReplyCallback send_reply_callback) { const JobID job_id = JobID::FromBinary(request.job_id()); - auto send_reply = [send_reply_callback, reply](Status status) { + auto send_reply = [send_reply_callback = std::move(send_reply_callback), + reply](Status status) { GCS_RPC_SEND_REPLY(send_reply_callback, reply, status); }; Status status = gcs_table_storage_->JobTable().Get( job_id, - [this, job_id, send_reply](Status status, + [this, job_id, send_reply](const Status &status, const std::optional &result) { if (status.ok() && result) { MarkJobAsFinished(*result, send_reply); - } else { + return; + } + + if (!result.has_value()) { RAY_LOG(ERROR) << "Tried to mark job " << job_id << " as finished, but there was no record of it starting!"; - send_reply(status); + } else if (!status.ok()) { + RAY_LOG(ERROR) << "Fails to mark job " << job_id << " as finished due to " + << status; } + send_reply(status); }); if (!status.ok()) { send_reply(status); @@ -438,5 +465,10 @@ void GcsJobManager::OnNodeDead(const NodeID &node_id) { RAY_CHECK_OK(gcs_table_storage_->JobTable().GetAll(on_done)); } +void GcsJobManager::RecordMetrics() { + ray::stats::STATS_running_jobs.Record(running_job_ids_.size()); + ray::stats::STATS_finished_jobs.Record(finished_jobs_count_); +} + } // namespace gcs } // namespace ray diff --git a/src/ray/gcs/gcs_server/gcs_job_manager.h b/src/ray/gcs/gcs_server/gcs_job_manager.h index 891e7ea20d25d..b74b24f3e1d1d 100644 --- a/src/ray/gcs/gcs_server/gcs_job_manager.h +++ b/src/ray/gcs/gcs_server/gcs_job_manager.h @@ -14,6 +14,14 @@ #pragma once +#include +#include +#include +#include +#include + +#include "absl/container/flat_hash_map.h" +#include "absl/container/flat_hash_set.h" #include "ray/common/runtime_env_manager.h" #include "ray/gcs/gcs_server/gcs_function_manager.h" #include "ray/gcs/gcs_server/gcs_init_data.h" @@ -88,7 +96,23 @@ class GcsJobManager : public rpc::JobInfoHandler { void WriteDriverJobExportEvent(rpc::JobTableData job_data) const; + /// Record metrics. + /// For job manager, (1) running jobs count gauge and (2) new finished jobs (whether + /// succeed or fail) will be reported periodically. + void RecordMetrics(); + private: + void ClearJobInfos(const rpc::JobTableData &job_data); + + void MarkJobAsFinished(rpc::JobTableData job_table_data, + std::function done_callback); + + // Running Job IDs, used to report metrics. + absl::flat_hash_set running_job_ids_; + + // Number of finished jobs since start of this GCS Server, used to report metrics. + int64_t finished_jobs_count_ = 0; + std::shared_ptr gcs_table_storage_; std::shared_ptr gcs_publisher_; @@ -104,11 +128,6 @@ class GcsJobManager : public rpc::JobInfoHandler { /// The cached core worker clients which are used to communicate with workers. rpc::CoreWorkerClientPool core_worker_clients_; - - void ClearJobInfos(const rpc::JobTableData &job_data); - - void MarkJobAsFinished(rpc::JobTableData job_table_data, - std::function done_callback); }; } // namespace gcs diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 174785681859b..805f2f521ed6e 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -789,6 +789,7 @@ void GcsServer::RecordMetrics() const { gcs_actor_manager_->RecordMetrics(); gcs_placement_group_manager_->RecordMetrics(); gcs_task_manager_->RecordMetrics(); + gcs_job_manager_->RecordMetrics(); execute_after( main_service_, [this] { RecordMetrics(); }, diff --git a/src/ray/gcs/gcs_server/gcs_table_storage.h b/src/ray/gcs/gcs_server/gcs_table_storage.h index b05cffc6beb3b..16133f2901389 100644 --- a/src/ray/gcs/gcs_server/gcs_table_storage.h +++ b/src/ray/gcs/gcs_server/gcs_table_storage.h @@ -111,8 +111,8 @@ class GcsTableWithJobId : public GcsTable { /// \param key The key that will be written to the table. The job id can be obtained /// from the key. /// \param value The value of the key that will be written to the table. - /// \param callback Callback that will be called after write finishes. - /// \return Status + /// \param callback Callback that will be called after write finishes, whether it + /// succeeds or not. \return Status for issuing the asynchronous write operation. Status Put(const Key &key, const Data &value, const StatusCallback &callback) override; /// Get all the data of the specified job id from the table asynchronously. diff --git a/src/ray/stats/metric_defs.cc b/src/ray/stats/metric_defs.cc index 114e6c07434d5..e12f68cd1bf4a 100644 --- a/src/ray/stats/metric_defs.cc +++ b/src/ray/stats/metric_defs.cc @@ -67,6 +67,21 @@ DEFINE_stats(actors, (), ray::stats::GAUGE); +/// Job related stats. +DEFINE_stats(running_jobs, + "Number of jobs currently running.", + /*tags=*/(), + /*buckets=*/(), + ray::stats::GAUGE); + +DEFINE_stats(finished_jobs, + "Number of jobs finished.", + // TODO(hjiang): Consider adding task completion status, for example, failed, + // completed in tags. + /*tags=*/(), + /*buckets=*/(), + ray::stats::COUNT); + /// Logical resource usage reported by raylets. DEFINE_stats(resources, // TODO(sang): Support placement_group_reserved_available | used diff --git a/src/ray/stats/metric_defs.h b/src/ray/stats/metric_defs.h index 44d77b8171594..d76c64e7f42f0 100644 --- a/src/ray/stats/metric_defs.h +++ b/src/ray/stats/metric_defs.h @@ -48,6 +48,10 @@ DECLARE_stats(tasks); /// Actor stats, broken down by state. DECLARE_stats(actors); +/// Job stats. +DECLARE_stats(running_jobs); +DECLARE_stats(finished_jobs); + /// Placement group stats, broken down by state. DECLARE_stats(placement_groups); diff --git a/src/ray/util/counter_map.h b/src/ray/util/counter_map.h index 6c7acbe046b8f..acd442b0e2c71 100644 --- a/src/ray/util/counter_map.h +++ b/src/ray/util/counter_map.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include "absl/container/flat_hash_map.h" #include "absl/container/flat_hash_set.h" @@ -35,7 +36,7 @@ template class CounterMap { public: - CounterMap(){}; + CounterMap() = default; CounterMap(const CounterMap &other) = delete; @@ -45,7 +46,7 @@ class CounterMap { /// Changes are buffered until `FlushOnChangeCallbacks()` is called to enable /// batching for performance reasons. void SetOnChangeCallback(std::function on_change) { - on_change_ = on_change; + on_change_ = std::move(on_change); } /// Flush any pending on change callbacks. From 211e7391e860ee6825cc759975a516d5d1ac35af Mon Sep 17 00:00:00 2001 From: Lonnie Liu <95255098+aslonnie@users.noreply.github.com> Date: Fri, 11 Oct 2024 08:21:46 -0700 Subject: [PATCH 16/46] upgrade grpcio version (#47982) to at least 1.66.1 this is already being overwritten to 1.66.1+ when during release tests Signed-off-by: Lonnie Liu --- python/requirements/anyscale-requirements.txt | 2 +- python/requirements_compiled.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/requirements/anyscale-requirements.txt b/python/requirements/anyscale-requirements.txt index c4145a74e8331..709e860a9e02d 100644 --- a/python/requirements/anyscale-requirements.txt +++ b/python/requirements/anyscale-requirements.txt @@ -4,7 +4,7 @@ opentelemetry-api opentelemetry-sdk opentelemetry-exporter-otlp google-cloud-storage -grpcio==1.60.0 +grpcio>=1.66.1 grpcio-tools pyyaml pyopenssl diff --git a/python/requirements_compiled.txt b/python/requirements_compiled.txt index 61318c9b0d1e5..a1043afc5b51b 100644 --- a/python/requirements_compiled.txt +++ b/python/requirements_compiled.txt @@ -682,7 +682,7 @@ graphql-core==3.2.3 # via moto greenlet==3.0.1 # via sqlalchemy -grpcio==1.60.0 ; sys_platform != "darwin" +grpcio==1.66.2 ; sys_platform != "darwin" # via # -r /ray/ci/../python/requirements.txt # -r /ray/ci/../python/requirements/anyscale-requirements.txt From 61a4220f564f1de740a4b2b0960be6bb661ed45e Mon Sep 17 00:00:00 2001 From: Chi-Sheng Liu Date: Sat, 12 Oct 2024 04:15:04 +0800 Subject: [PATCH 17/46] [Feat][Core] Implement single file module for runtime_env (#47807) Supports single file modules in `py_module` runtime_env. Signed-off-by: Chi-Sheng Liu --- doc/source/ray-core/handling-dependencies.rst | 10 +- python/ray/_private/runtime_env/packaging.py | 140 +++++++++++++----- python/ray/_private/runtime_env/py_modules.py | 37 +++-- python/ray/tests/conftest.py | 5 + .../ray/tests/test_runtime_env_packaging.py | 41 +++++ .../ray/tests/test_runtime_env_working_dir.py | 30 ++-- 6 files changed, 197 insertions(+), 66 deletions(-) diff --git a/doc/source/ray-core/handling-dependencies.rst b/doc/source/ray-core/handling-dependencies.rst index ecec425bdc535..2bc24462b9a85 100644 --- a/doc/source/ray-core/handling-dependencies.rst +++ b/doc/source/ray-core/handling-dependencies.rst @@ -329,8 +329,6 @@ To ensure your local changes show up across all Ray workers and can be imported ray.get(f.remote()) -Note: This feature is currently limited to modules that are packages with a single directory containing an ``__init__.py`` file. For single-file modules, you may use ``working_dir``. - .. _runtime-environments-api-ref: API Reference @@ -358,13 +356,15 @@ The ``runtime_env`` is a Python dictionary or a Python class :class:`ray.runtime Note: If the local directory contains symbolic links, Ray follows the links and the files they point to are uploaded to the cluster. - ``py_modules`` (List[str|module]): Specifies Python modules to be available for import in the Ray workers. (For more ways to specify packages, see also the ``pip`` and ``conda`` fields below.) - Each entry must be either (1) a path to a local directory, (2) a URI to a remote zip or wheel file (see :ref:`remote-uris` for details), (3) a Python module object, or (4) a path to a local `.whl` file. + Each entry must be either (1) a path to a local file or directory, (2) a URI to a remote zip or wheel file (see :ref:`remote-uris` for details), (3) a Python module object, or (4) a path to a local `.whl` file. - Examples of entries in the list: - ``"."`` - - ``"/local_dependency/my_module"`` + - ``"/local_dependency/my_dir_module"`` + + - ``"/local_dependency/my_file_module.py"`` - ``"s3://bucket/my_module.zip"`` @@ -380,8 +380,6 @@ The ``runtime_env`` is a Python dictionary or a Python class :class:`ray.runtime Note: For option (1), if the local directory contains a ``.gitignore`` file, the files and paths specified there are not uploaded to the cluster. You can disable this by setting the environment variable `RAY_RUNTIME_ENV_IGNORE_GITIGNORE=1` on the machine doing the uploading. - Note: This feature is currently limited to modules that are packages with a single directory containing an ``__init__.py`` file. For single-file modules, you may use ``working_dir``. - - ``excludes`` (List[str]): When used with ``working_dir`` or ``py_modules``, specifies a list of files or paths to exclude from being uploaded to the cluster. This field uses the pattern-matching syntax used by ``.gitignore`` files: see ``_ for details. Note: In accordance with ``.gitignore`` syntax, if there is a separator (``/``) at the beginning or middle (or both) of the pattern, then the pattern is interpreted relative to the level of the ``working_dir``. diff --git a/python/ray/_private/runtime_env/packaging.py b/python/ray/_private/runtime_env/packaging.py index 2766e6a7acab9..e66b705b3e863 100644 --- a/python/ray/_private/runtime_env/packaging.py +++ b/python/ray/_private/runtime_env/packaging.py @@ -135,6 +135,56 @@ def _dir_travel( excludes.pop() +def _hash_file_content_or_directory_name( + filepath: Path, + relative_path: Path, + logger: Optional[logging.Logger] = default_logger, +) -> bytes: + """Helper function to create hash of a single file or directory. + + This function will hash the path of the file or directory, + and if it's a file, it'll hash its content too. + """ + + BUF_SIZE = 4096 * 1024 + + sha1 = hashlib.sha1() + sha1.update(str(filepath.relative_to(relative_path)).encode()) + if not filepath.is_dir(): + try: + f = filepath.open("rb") + except Exception as e: + logger.debug( + f"Skipping contents of file {filepath} when calculating package hash " + f"because the file could not be opened: {e}" + ) + else: + try: + data = f.read(BUF_SIZE) + while len(data) != 0: + sha1.update(data) + data = f.read(BUF_SIZE) + finally: + f.close() + + return sha1.digest() + + +def _hash_file( + filepath: Path, + relative_path: Path, + logger: Optional[logging.Logger] = default_logger, +) -> bytes: + """Helper function to create hash of a single file. + + It'll hash the path of the file and its content to create a hash value. + """ + file_hash = _hash_file_content_or_directory_name( + filepath, relative_path, logger=logger + ) + return _xor_bytes(file_hash, b"0" * 8) + + def _hash_directory( root: Path, relative_path: Path, @@ -147,30 +197,13 @@ def _hash_directory( hash(file_name, file_content) to create a hash value. """ hash_val = b"0" * 8 - BUF_SIZE = 4096 * 1024 def handler(path: Path): - sha1 = hashlib.sha1() - sha1.update(str(path.relative_to(relative_path)).encode()) - if not path.is_dir(): - try: - f = path.open("rb") - except Exception as e: - logger.debug( - f"Skipping contents of file {path} when calculating package hash " - f"because the file could not be opened: {e}" - ) - else: - try: - data = f.read(BUF_SIZE) - while len(data) != 0: - sha1.update(data) - data = f.read(BUF_SIZE) - finally: - f.close() - + file_hash = _hash_file_content_or_directory_name( + path, relative_path, logger=logger + ) nonlocal hash_val - hash_val = _xor_bytes(hash_val, sha1.digest()) + hash_val = _xor_bytes(hash_val, file_hash) excludes = [] if excludes is None else [excludes] _dir_travel(root, excludes, handler, logger=logger) @@ -378,16 +411,16 @@ def _get_local_path(base_directory: str, pkg_uri: str) -> str: return os.path.join(base_directory, pkg_name) -def _zip_directory( - directory: str, +def _zip_files( + path_str: str, excludes: List[str], output_path: str, include_parent_dir: bool = False, logger: Optional[logging.Logger] = default_logger, ) -> None: - """Zip the target directory and write it to the output_path. + """Zip the target file or directory and write it to the output_path. - directory: The directory to zip. + path_str: The file or directory to zip. excludes (List(str)): The directories or file to be excluded. output_path: The output path for the zip file. include_parent_dir: If true, includes the top-level directory as a @@ -396,7 +429,10 @@ def _zip_directory( pkg_file = Path(output_path).absolute() with ZipFile(pkg_file, "w", strict_timestamps=False) as zip_handler: # Put all files in the directory into the zip file. - dir_path = Path(directory).absolute() + file_path = Path(path_str).absolute() + dir_path = file_path + if file_path.is_file(): + dir_path = file_path.parent def handler(path: Path): # Pack this path if it's an empty directory or it's a file. @@ -415,8 +451,8 @@ def handler(path: Path): to_path = dir_path.name / to_path zip_handler.write(path, to_path) - excludes = [_get_excludes(dir_path, excludes)] - _dir_travel(dir_path, excludes, handler, logger=logger) + excludes = [_get_excludes(file_path, excludes)] + _dir_travel(file_path, excludes, handler, logger=logger) def package_exists(pkg_uri: str) -> bool: @@ -451,6 +487,38 @@ def get_uri_for_package(package: Path) -> str: ) +def get_uri_for_file(file: str) -> str: + """Get a content-addressable URI from a file's content. + + This function will generate the name of the package by the file. + The final package name is: _ray_pkg_.zip of this package. + e.g., _ray_pkg_029f88d5ecc55e1e4d64fc6e388fd103.zip + + Examples: + + >>> get_uri_for_file("/my_file.py") # doctest: +SKIP + _ray_pkg_af2734982a741.zip + + Args: + file: The file. + + Returns: + URI (str) + + Raises: + ValueError if the file doesn't exist. + """ + filepath = Path(file).absolute() + if not filepath.exists() or not filepath.is_file(): + raise ValueError(f"File {filepath} must be an existing file") + + hash_val = _hash_file(filepath, filepath.parent) + + return "{protocol}://{pkg_name}.zip".format( + protocol=Protocol.GCS.value, pkg_name=RAY_PKG_PREFIX + hash_val.hex() + ) + + def get_uri_for_directory(directory: str, excludes: Optional[List[str]] = None) -> str: """Get a content-addressable URI from a directory's contents. @@ -515,7 +583,7 @@ def upload_package_to_gcs(pkg_uri: str, pkg_bytes: bytes) -> None: def create_package( - directory: str, + module_path: str, target_path: Path, include_parent_dir: bool = False, excludes: Optional[List[str]] = None, @@ -528,11 +596,11 @@ def create_package( logger = default_logger if not target_path.exists(): - logger.info(f"Creating a file package for local directory '{directory}'.") - _zip_directory( - directory, + logger.info(f"Creating a file package for local module '{module_path}'.") + _zip_files( + module_path, excludes, - target_path, + str(target_path), include_parent_dir=include_parent_dir, logger=logger, ) @@ -541,7 +609,7 @@ def create_package( def upload_package_if_needed( pkg_uri: str, base_directory: str, - directory: str, + module_path: str, include_parent_dir: bool = False, excludes: Optional[List[str]] = None, logger: Optional[logging.Logger] = default_logger, @@ -556,7 +624,7 @@ def upload_package_if_needed( Args: pkg_uri: URI of the package to upload. base_directory: Directory where package files are stored. - directory: Directory to be uploaded. + module_path: The module to be uploaded, either a single .py file or a directory. include_parent_dir: If true, includes the top-level directory as a directory inside the zip file. excludes: List specifying files to exclude. @@ -586,7 +654,7 @@ def upload_package_if_needed( f"{time.time_ns()}_{os.getpid()}_{package_file.name}" ) create_package( - directory, + module_path, package_file, include_parent_dir=include_parent_dir, excludes=excludes, diff --git a/python/ray/_private/runtime_env/py_modules.py b/python/ray/_private/runtime_env/py_modules.py index 551e22b3650a8..1066cbe6126df 100644 --- a/python/ray/_private/runtime_env/py_modules.py +++ b/python/ray/_private/runtime_env/py_modules.py @@ -10,6 +10,7 @@ delete_package, download_and_unpack_package, get_local_dir_from_uri, + get_uri_for_file, get_uri_for_directory, get_uri_for_package, install_wheel_package, @@ -71,15 +72,20 @@ def upload_py_modules_if_needed( elif isinstance(module, Path): module_path = str(module) elif isinstance(module, ModuleType): - # NOTE(edoakes): Python allows some installed Python packages to - # be split into multiple directories. We could probably handle - # this, but it seems tricky & uncommon. If it's a problem for - # users, we can add this support on demand. - if len(module.__path__) > 1: - raise ValueError( - "py_modules only supports modules whose __path__ has length 1." - ) - [module_path] = module.__path__ + if not hasattr(module, "__path__"): + # This is a single-file module. + module_path = module.__file__ + else: + # NOTE(edoakes): Python allows some installed Python packages to + # be split into multiple directories. We could probably handle + # this, but it seems tricky & uncommon. If it's a problem for + # users, we can add this support on demand. + if len(module.__path__) > 1: + raise ValueError( + "py_modules only supports modules whose __path__" + " has length 1 or those who are single-file." + ) + [module_path] = module.__path__ else: raise TypeError( "py_modules must be a list of file paths, URIs, " @@ -90,9 +96,13 @@ def upload_py_modules_if_needed( module_uri = module_path else: # module_path is a local path. - if Path(module_path).is_dir(): + if Path(module_path).is_dir() or Path(module_path).suffix == ".py": + is_dir = Path(module_path).is_dir() excludes = runtime_env.get("excludes", None) - module_uri = get_uri_for_directory(module_path, excludes=excludes) + if is_dir: + module_uri = get_uri_for_directory(module_path, excludes=excludes) + else: + module_uri = get_uri_for_file(module_path) if upload_fn is None: try: upload_package_if_needed( @@ -100,7 +110,7 @@ def upload_py_modules_if_needed( scratch_dir, module_path, excludes=excludes, - include_parent_dir=True, + include_parent_dir=is_dir, logger=logger, ) except Exception as e: @@ -136,7 +146,8 @@ def upload_py_modules_if_needed( upload_fn(module_path, excludes=None, is_file=True) else: raise ValueError( - "py_modules entry must be a directory or a .whl file; " + "py_modules entry must be a .py file, " + "a directory, or a .whl file; " f"got {module_path}" ) diff --git a/python/ray/tests/conftest.py b/python/ray/tests/conftest.py index ead0ec9648adb..4ee34d2b73c97 100644 --- a/python/ray/tests/conftest.py +++ b/python/ray/tests/conftest.py @@ -688,6 +688,11 @@ def tmp_working_dir(): with hello_file.open(mode="w") as f: f.write("world") + test_file_module = path / "file_module.py" + with test_file_module.open(mode="w") as f: + f.write("def hello():\n") + f.write(" return 'hello'\n") + module_path = path / "test_module" module_path.mkdir(parents=True) diff --git a/python/ray/tests/test_runtime_env_packaging.py b/python/ray/tests/test_runtime_env_packaging.py index 3c10826ad0361..c2a82318f5a4a 100644 --- a/python/ray/tests/test_runtime_env_packaging.py +++ b/python/ray/tests/test_runtime_env_packaging.py @@ -29,6 +29,7 @@ download_and_unpack_package, get_local_dir_from_uri, get_top_level_dir_from_compressed_package, + get_uri_for_file, get_uri_for_directory, get_uri_for_package, is_whl_uri, @@ -64,6 +65,14 @@ def random_string(size: int = 10): return "".join(random.choice(string.ascii_uppercase) for _ in range(size)) +@pytest.fixture +def random_file(tmp_path) -> Path: + p = tmp_path / (random_string(10) + ".py") + with p.open("w") as f: + f.write(random_string(100)) + yield p + + @pytest.fixture def random_dir(tmp_path) -> Path: subdir = tmp_path / "subdir" @@ -135,6 +144,38 @@ def random_zip_file_with_top_level_dir(tmp_path): yield str(path / ARCHIVE_NAME) +class TestGetURIForFile: + def test_invalid_file(self): + with pytest.raises(ValueError): + get_uri_for_file("/does/not/exist.py") + + with pytest.raises(ValueError): + get_uri_for_file("does/not/exist.py") + + def test_determinism(self, random_file): + # Check that it's deterministic for same data. + uris = {get_uri_for_file(str(random_file)) for _ in range(10)} + assert len(uris) == 1 + + # Append one line, should be different now. + with open(random_file, "a") as f: + f.write(random_string()) + + assert {get_uri_for_file(str(random_file))} != uris + + def test_relative_paths(self, random_file): + # Check that relative or absolute paths result in the same URI. + p = Path(random_file) + relative_uri = get_uri_for_file(os.path.relpath(p)) + absolute_uri = get_uri_for_file(str(p.resolve())) + assert relative_uri == absolute_uri + + def test_uri_hash_length(self, random_file): + uri = get_uri_for_file(str(random_file)) + hex_hash = uri.split("_")[-1][: -len(".zip")] + assert len(hex_hash) == 16 + + class TestGetURIForDirectory: def test_invalid_directory(self): with pytest.raises(ValueError): diff --git a/python/ray/tests/test_runtime_env_working_dir.py b/python/ray/tests/test_runtime_env_working_dir.py index f145eea151f89..e667b0c712b10 100644 --- a/python/ray/tests/test_runtime_env_working_dir.py +++ b/python/ray/tests/test_runtime_env_working_dir.py @@ -128,6 +128,7 @@ def call_ray_init(): runtime_env={ "py_modules": [ str(Path(tmp_working_dir) / "test_module"), + str(Path(tmp_working_dir) / "file_module.py"), Path(os.path.dirname(__file__)) / "pip_install_test-0.5-py3-none-any.whl", ] @@ -140,6 +141,7 @@ def call_ray_init(): "working_dir": tmp_working_dir, "py_modules": [ str(Path(tmp_working_dir) / "test_module"), + str(Path(tmp_working_dir) / "file_module.py"), Path(os.path.dirname(__file__)) / "pip_install_test-0.5-py3-none-any.whl", ], @@ -163,15 +165,16 @@ def reinit(): @ray.remote def test_import(): import test_module + import file_module assert TEST_IMPORT_DIR in os.environ.get("PYTHONPATH", "") - return test_module.one() + return test_module.one(), file_module.hello() if option == "failure": with pytest.raises(ImportError): ray.get(test_import.remote()) else: - assert ray.get(test_import.remote()) == 1 + assert ray.get(test_import.remote()) == (1, "hello") if option in {"py_modules", "working_dir_and_py_modules"}: @@ -205,9 +208,10 @@ def test_read(): class Actor: def test_import(self): import test_module + import file_module assert TEST_IMPORT_DIR in os.environ.get("PYTHONPATH", "") - return test_module.one() + return test_module.one(), file_module.hello() def test_read(self): assert TEST_IMPORT_DIR in os.environ.get("PYTHONPATH", "") @@ -216,11 +220,11 @@ def test_read(self): a = Actor.remote() if option == "failure": with pytest.raises(ImportError): - assert ray.get(a.test_import.remote()) == 1 + assert ray.get(a.test_import.remote()) == (1, "hello") with pytest.raises(FileNotFoundError): assert ray.get(a.test_read.remote()) == "world" elif option in {"working_dir_and_py_modules", "working_dir"}: - assert ray.get(a.test_import.remote()) == 1 + assert ray.get(a.test_import.remote()) == (1, "hello") assert ray.get(a.test_read.remote()) == "world" @@ -243,7 +247,10 @@ def call_ray_init(): ray.init( address, runtime_env={ - "py_modules": [os.path.join(tmp_working_dir, "test_module")] + "py_modules": [ + os.path.join(tmp_working_dir, "test_module"), + os.path.join(tmp_working_dir, "file_module.py"), + ] }, ) @@ -262,31 +269,32 @@ def reinit(): # Import in the driver. sys.path.insert(0, tmp_working_dir) import test_module + import file_module @ray.remote def test_import(): - return test_module.one() + return test_module.one(), file_module.hello() if option == "failure": with pytest.raises(Exception): ray.get(test_import.remote()) else: - assert ray.get(test_import.remote()) == 1 + assert ray.get(test_import.remote()) == (1, "hello") reinit() @ray.remote class Actor: def test_import(self): - return test_module.one() + return test_module.one(), file_module.hello() if option == "failure": with pytest.raises(Exception): a = Actor.remote() - assert ray.get(a.test_import.remote()) == 1 + assert ray.get(a.test_import.remote()) == (1, "hello") else: a = Actor.remote() - assert ray.get(a.test_import.remote()) == 1 + assert ray.get(a.test_import.remote()) == (1, "hello") def test_empty_working_dir(start_cluster): From 04098a6f2cd2724acd2ff03bbba1d6e6cd3b2a33 Mon Sep 17 00:00:00 2001 From: Matthew Owen Date: Fri, 11 Oct 2024 15:53:26 -0700 Subject: [PATCH 18/46] [data] Basic structured logging (#47210) Adds structured logging to Ray Data. This will allow users to configure logging to use any of the following: * A user's custom logging file (existing functionality) * A default TEXT logger (existing functionality) * A default JSON logger (new functionality) --------- Signed-off-by: Matthew Owen Signed-off-by: Matthew Owen Co-authored-by: Scott Lee Co-authored-by: Hao Chen --- python/ray/data/_internal/logging.py | 44 +++++++++++++++++++++++--- python/ray/data/_internal/logging.yaml | 9 ++++++ python/ray/data/tests/test_logging.py | 43 +++++++++++++++++++++++++ 3 files changed, 91 insertions(+), 5 deletions(-) diff --git a/python/ray/data/_internal/logging.py b/python/ray/data/_internal/logging.py index 0a9a5a8f7093a..ae0991e4adbe5 100644 --- a/python/ray/data/_internal/logging.py +++ b/python/ray/data/_internal/logging.py @@ -11,6 +11,12 @@ os.path.join(os.path.dirname(__file__), "logging.yaml") ) +# Env. variable to specify the encoding of the file logs when using the default config. +RAY_DATA_LOG_ENCODING_ENV_VAR_NAME = "RAY_DATA_LOG_ENCODING" + +# Env. variable to specify the logging config path use defaults if not set +RAY_DATA_LOGGING_CONFIG_ENV_VAR_NAME = "RAY_DATA_LOGGING_CONFIG" + # To facilitate debugging, Ray Data writes debug logs to a file. However, if Ray Data # logs every scheduler loop, logging might impact performance. So, we add a "TRACE" # level where logs aren't written by default. @@ -89,15 +95,43 @@ def _try_create_handler(self): def configure_logging() -> None: """Configure the Python logger named 'ray.data'. - This function loads the configration YAML specified by the "RAY_DATA_LOGGING_CONFIG" - environment variable. If the variable isn't set, this function loads the + This function loads the configration YAML specified by "RAY_DATA_LOGGING_CONFIG" + environment variable. If the variable isn't set, this function loads the default "logging.yaml" file that is adjacent to this module. + + If "RAY_DATA_LOG_ENCODING" is specified as "JSON" we will enable JSON logging mode + if using the default logging config. """ - config_path = os.environ.get("RAY_DATA_LOGGING_CONFIG", DEFAULT_CONFIG_PATH) - with open(config_path) as file: - config = yaml.safe_load(file) + + def _load_logging_config(config_path: str): + with open(config_path) as file: + config = yaml.safe_load(file) + return config + + # Dynamically load env vars + config_path = os.environ.get(RAY_DATA_LOGGING_CONFIG_ENV_VAR_NAME) + log_encoding = os.environ.get(RAY_DATA_LOG_ENCODING_ENV_VAR_NAME) + + if config_path is not None: + config = _load_logging_config(config_path) + else: + config = _load_logging_config(DEFAULT_CONFIG_PATH) + if log_encoding is not None and log_encoding.upper() == "JSON": + for logger in config["loggers"].values(): + logger["handlers"].remove("file") + logger["handlers"].append("file_json") + logging.config.dictConfig(config) + # After configuring logger, warn if RAY_DATA_LOGGING_CONFIG is used with + # RAY_DATA_LOG_ENCODING, because they are not both supported together. + if config_path is not None and log_encoding is not None: + logger = logging.getLogger(__name__) + logger.warning( + "Using `RAY_DATA_LOG_ENCODING` is not supported with " + + "`RAY_DATA_LOGGING_CONFIG`" + ) + def reset_logging() -> None: """Reset the logger named 'ray.data' to its initial state. diff --git a/python/ray/data/_internal/logging.yaml b/python/ray/data/_internal/logging.yaml index f72abf356f6dc..170d7c6605d85 100644 --- a/python/ray/data/_internal/logging.yaml +++ b/python/ray/data/_internal/logging.yaml @@ -4,16 +4,25 @@ disable_existing_loggers: False formatters: ray: format: "%(asctime)s\t%(levelname)s %(filename)s:%(lineno)s -- %(message)s" + ray_json: + class: ray._private.ray_logging.formatters.JSONFormatter filters: console_filter: (): ray.data._internal.logging.HiddenRecordFilter + core_context_filter: + (): ray._private.ray_logging.filters.CoreContextFilter handlers: file: class: ray.data._internal.logging.SessionFileHandler formatter: ray filename: ray-data.log + file_json: + class: ray.data._internal.logging.SessionFileHandler + formatter: ray_json + filename: ray-data.log + filters: [core_context_filter] console: class: ray._private.log.PlainRayHandler formatter: ray diff --git a/python/ray/data/tests/test_logging.py b/python/ray/data/tests/test_logging.py index de18e44523845..ace59dbcda920 100644 --- a/python/ray/data/tests/test_logging.py +++ b/python/ray/data/tests/test_logging.py @@ -124,6 +124,49 @@ def test_custom_config(reset_logging, monkeypatch, tmp_path): assert isinstance(logger.handlers[0], logging.StreamHandler) +def test_json_logging_configuration( + capsys, reset_logging, monkeypatch, shutdown_only, propagate_logs +): + monkeypatch.setenv("RAY_DATA_LOG_ENCODING", "JSON") + ray.init() + + configure_logging() + + logger = logging.getLogger("ray.data") + + # Ensure handlers correctly setup + handlers = logger.handlers + assert len(handlers) == 2 + assert sum(handler.name == "file_json" for handler in handlers) == 1 + assert sum(handler.name == "console" for handler in handlers) == 1 + + logger.info("ham") + logger.debug("turkey") + + log_path = os.path.join(get_log_directory(), "ray-data.log") + with open(log_path) as file: + log_contents = file.read() + + # Validate the log is in JSON format (a basic check for JSON) + assert all( + log_line.startswith("{") and log_line.endswith("}") + for log_line in log_contents.splitlines() + ) + + assert '"message": "ham"' in log_contents + assert '"message": "turkey"' in log_contents + + # Validate console logs are in text mode + console_log_output = capsys.readouterr().err + assert not any( + log_line.startswith("{") and log_line.endswith("}") + for log_line in console_log_output.splitlines() + ) + + assert "ham" in console_log_output + assert "turkey" not in console_log_output + + if __name__ == "__main__": import sys From dc5ed68ab85025e1bcb8edc10a6cde553fe6290c Mon Sep 17 00:00:00 2001 From: Dhyey Shah Date: Fri, 11 Oct 2024 19:02:38 -0400 Subject: [PATCH 19/46] [Core] Fix state api filter types and expand on error messages (#47963) Signed-off-by: dayshah --- python/ray/dashboard/state_aggregator.py | 6 ++++-- python/ray/tests/test_state_api.py | 4 ++-- python/ray/util/state/state_cli.py | 1 - 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/python/ray/dashboard/state_aggregator.py b/python/ray/dashboard/state_aggregator.py index c4ba56c954c20..b934f4c5c9e68 100644 --- a/python/ray/dashboard/state_aggregator.py +++ b/python/ray/dashboard/state_aggregator.py @@ -65,7 +65,7 @@ def _convert_filters_type( filter: List[Tuple[str, PredicateType, SupportedFilterType]], schema: StateSchema, -) -> List[Tuple[str, SupportedFilterType]]: +) -> List[Tuple[str, PredicateType, SupportedFilterType]]: """Convert the given filter's type to SupportedFilterType. This method is necessary because click can only accept a single type @@ -155,7 +155,7 @@ def data_source_client(self): def _filter( self, data: List[dict], - filters: List[Tuple[str, SupportedFilterType]], + filters: List[Tuple[str, PredicateType, SupportedFilterType]], state_dataclass: StateSchema, detail: bool, ) -> List[dict]: @@ -181,6 +181,8 @@ def _filter( if filter_column not in filterable_columns: raise ValueError( f"The given filter column {filter_column} is not supported. " + "Enter filters with –-filter key=value " + "or –-filter key!=value " f"Supported filter columns: {filterable_columns}" ) diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index ca5be44aee0c3..908faefcd9fd1 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -13,7 +13,7 @@ from ray._private.state_api_test_utils import get_state_api_manager from ray.util.state import get_job from ray.dashboard.modules.job.pydantic_models import JobDetails -from ray.util.state.common import Humanify +from ray.util.state.common import Humanify, PredicateType from ray._private.gcs_utils import GcsAioClient import yaml from click.testing import CliRunner @@ -328,7 +328,7 @@ def generate_runtime_env_info(runtime_env, creation_time=None, success=True): def create_api_options( timeout: int = DEFAULT_RPC_TIMEOUT, limit: int = DEFAULT_LIMIT, - filters: List[Tuple[str, SupportedFilterType]] = None, + filters: List[Tuple[str, PredicateType, SupportedFilterType]] = None, detail: bool = False, exclude_driver: bool = True, ): diff --git a/python/ray/util/state/state_cli.py b/python/ray/util/state/state_cli.py index ce40b9310cc03..d191b34b3c2c0 100644 --- a/python/ray/util/state/state_cli.py +++ b/python/ray/util/state/state_cli.py @@ -81,7 +81,6 @@ def _parse_filter(filter: str) -> Tuple[str, PredicateType, SupportedFilterType] filter[predicate_index[0] : predicate_index[1]], filter[predicate_index[1] :], ) - assert predicate == "=" or predicate == "!=" if len(key) == 0 or len(value) == 0: raise ValueError( From 1ae9025b348e587b4db59a059bdfc0e8d135566e Mon Sep 17 00:00:00 2001 From: Chi-Sheng Liu Date: Sun, 13 Oct 2024 04:53:56 +0800 Subject: [PATCH 20/46] [Chore][Core] Address PR 47807 comments (#48002) PR 47807 was auto-merged without applying the doc reviews, so this commit addresses them. Signed-off-by: Chi-Sheng Liu --- python/ray/_private/runtime_env/packaging.py | 21 ++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/python/ray/_private/runtime_env/packaging.py b/python/ray/_private/runtime_env/packaging.py index e66b705b3e863..dd9212556c47d 100644 --- a/python/ray/_private/runtime_env/packaging.py +++ b/python/ray/_private/runtime_env/packaging.py @@ -142,8 +142,8 @@ def _hash_file_content_or_directory_name( ) -> bytes: """Helper function to create hash of a single file or directory. - This function will hash the path of the file or directory, - and if it's a file, it'll hash its content too. + This function hashes the path of the file or directory, + and if it's a file, then it hashes its content too. """ BUF_SIZE = 4096 * 1024 @@ -156,7 +156,7 @@ def _hash_file_content_or_directory_name( except Exception as e: logger.debug( f"Skipping contents of file {filepath} when calculating package hash " - f"because the file could not be opened: {e}" + f"because the file couldn't be opened: {e}" ) else: try: @@ -177,7 +177,7 @@ def _hash_file( ) -> bytes: """Helper function to create hash of a single file. - It'll hash the path of the file and its content to create a hash value. + It hashes the path of the file and its content to create a hash value. """ file_hash = _hash_file_content_or_directory_name( filepath, relative_path, logger=logger @@ -490,9 +490,10 @@ def get_uri_for_package(package: Path) -> str: def get_uri_for_file(file: str) -> str: """Get a content-addressable URI from a file's content. - This function will generate the name of the package by the file. - The final package name is: _ray_pkg_.zip of this package. - e.g., _ray_pkg_029f88d5ecc55e1e4d64fc6e388fd103.zip + This function generates the name of the package by the file. + The final package name is _ray_pkg_.zip of this package, + where HASH_VAL is the hash value of the file. + For example: _ray_pkg_029f88d5ecc55e1e4d64fc6e388fd103.zip Examples: @@ -522,11 +523,11 @@ def get_uri_for_file(file: str) -> str: def get_uri_for_directory(directory: str, excludes: Optional[List[str]] = None) -> str: """Get a content-addressable URI from a directory's contents. - This function will generate the name of the package by the directory. + This function generates the name of the package by the directory. It'll go through all the files in the directory and hash the contents of the files to get the hash value of the package. - The final package name is: _ray_pkg_.zip of this package. - e.g., _ray_pkg_029f88d5ecc55e1e4d64fc6e388fd103.zip + The final package name is _ray_pkg_.zip of this package. + For example: _ray_pkg_029f88d5ecc55e1e4d64fc6e388fd103.zip Examples: From b53944012fc47a64b271f2de73c9dca951e01b50 Mon Sep 17 00:00:00 2001 From: dentiny Date: Sat, 12 Oct 2024 14:51:54 -0700 Subject: [PATCH 21/46] [core] Add a thread checker (#47966) `ThreadChecker` is a cheap runtime checker, which only takes two atomic operations, which - guards against threading invariants; - serves for better code readability, so code readers and developers are easy to understand thread execution status Signed-off-by: dentiny --- src/ray/util/BUILD | 7 ++++ src/ray/util/tests/BUILD | 12 +++++++ src/ray/util/tests/thread_checker_test.cc | 34 ++++++++++++++++++ src/ray/util/thread_checker.cc | 26 ++++++++++++++ src/ray/util/thread_checker.h | 43 +++++++++++++++++++++++ 5 files changed, 122 insertions(+) create mode 100644 src/ray/util/tests/thread_checker_test.cc create mode 100644 src/ray/util/thread_checker.cc create mode 100644 src/ray/util/thread_checker.h diff --git a/src/ray/util/BUILD b/src/ray/util/BUILD index b17521a4cdd68..bd35c3874218f 100644 --- a/src/ray/util/BUILD +++ b/src/ray/util/BUILD @@ -41,3 +41,10 @@ cc_library( "@nlohmann_json", ], ) + +cc_library( + name = "thread_checker", + hdrs = ["thread_checker.h"], + srcs = ["thread_checker.cc"], + visibility = ["//visibility:public"], +) diff --git a/src/ray/util/tests/BUILD b/src/ray/util/tests/BUILD index 200425ea1a0da..096b090f18542 100644 --- a/src/ray/util/tests/BUILD +++ b/src/ray/util/tests/BUILD @@ -1,6 +1,18 @@ load("@rules_cc//cc:defs.bzl", "cc_test") load("//bazel:ray.bzl", "COPTS") +cc_test( + name = "thread_checker_test", + srcs = ["thread_checker_test.cc"], + copts = COPTS, + size = "small", + tags = ["team:core"], + deps = [ + "//src/ray/util:thread_checker", + "@com_google_googletest//:gtest_main", + ], +) + cc_test( name = "container_util_test", size = "small", diff --git a/src/ray/util/tests/thread_checker_test.cc b/src/ray/util/tests/thread_checker_test.cc new file mode 100644 index 0000000000000..08bd8588ee7fa --- /dev/null +++ b/src/ray/util/tests/thread_checker_test.cc @@ -0,0 +1,34 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "src/ray/util/thread_checker.h" + +#include + +#include + +namespace ray { + +TEST(ThreadCheckerTest, BasicTest) { + ThreadChecker thread_checker; + // Pass at initialization. + ASSERT_TRUE(thread_checker.IsOnSameThread()); + // Pass when invoked at the same thread. + ASSERT_TRUE(thread_checker.IsOnSameThread()); + + auto thd = std::thread([&]() { ASSERT_FALSE(thread_checker.IsOnSameThread()); }); + thd.join(); +} + +} // namespace ray diff --git a/src/ray/util/thread_checker.cc b/src/ray/util/thread_checker.cc new file mode 100644 index 0000000000000..73a0072c75752 --- /dev/null +++ b/src/ray/util/thread_checker.cc @@ -0,0 +1,26 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "src/ray/util/thread_checker.h" + +namespace ray { + +bool ThreadChecker::IsOnSameThread() { + const auto cur_id = std::this_thread::get_id(); + std::thread::id uninitialized_id; + return thread_id_.compare_exchange_strong(uninitialized_id, cur_id) || + (uninitialized_id == cur_id); +} + +} // namespace ray diff --git a/src/ray/util/thread_checker.h b/src/ray/util/thread_checker.h new file mode 100644 index 0000000000000..622624859b753 --- /dev/null +++ b/src/ray/util/thread_checker.h @@ -0,0 +1,43 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Used to sanity check threading issues by checking current thread id. +// +// Example usage: +// ThreadChecker thread_checker{}; +// +// // Initialize on the thread at first usage. +// RAY_CHECK(thread_checker.ok()); +// +// // Check it's on the same thread. +// RAY_CHECK(thread_checker.ok()); + +#pragma once + +#include +#include + +namespace ray { + +class ThreadChecker { + public: + // Return true at initialization, or current invocation happens on the same thread as + // initialization. + bool IsOnSameThread(); + + private: + std::atomic thread_id_{}; +}; + +} // namespace ray From aca67b30243978e02832be4575efb5c5caaf47ae Mon Sep 17 00:00:00 2001 From: dentiny Date: Mon, 14 Oct 2024 00:39:57 -0700 Subject: [PATCH 22/46] [core] Add thread check to job mgr callback (#48005) This PR followup for comment https://github.com/ray-project/ray/pull/47793#discussion_r1794767402, and adds a thread checking to GCS job manager callback to make sure no concurrent access for data members. Signed-off-by: dentiny --- BUILD.bazel | 1 + src/ray/gcs/gcs_server/gcs_job_manager.cc | 10 ++++++++++ src/ray/gcs/gcs_server/gcs_job_manager.h | 5 +++++ 3 files changed, 16 insertions(+) diff --git a/BUILD.bazel b/BUILD.bazel index c121daaf99c8f..83660e6aa8cad 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -513,6 +513,7 @@ ray_cc_library( "@boost//:bimap", "@com_github_grpc_grpc//src/proto/grpc/health/v1:health_proto", "@com_google_absl//absl/container:btree", + "//src/ray/util:thread_checker", ], ) diff --git a/src/ray/gcs/gcs_server/gcs_job_manager.cc b/src/ray/gcs/gcs_server/gcs_job_manager.cc index 23432d1e51718..f68a764f600cc 100644 --- a/src/ray/gcs/gcs_server/gcs_job_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_job_manager.cc @@ -96,6 +96,8 @@ void GcsJobManager::HandleAddJob(rpc::AddJobRequest request, reply, send_reply_callback = std::move(send_reply_callback)](const Status &status) { + RAY_CHECK(thread_checker_.IsOnSameThread()); + if (!status.ok()) { RAY_LOG(ERROR) << "Failed to add job, job id = " << job_id << ", driver pid = " << job_table_data.driver_pid(); @@ -136,6 +138,8 @@ void GcsJobManager::MarkJobAsFinished(rpc::JobTableData job_table_data, job_table_data.set_is_dead(true); auto on_done = [this, job_id, job_table_data, done_callback = std::move(done_callback)]( const Status &status) { + RAY_CHECK(thread_checker_.IsOnSameThread()); + if (!status.ok()) { RAY_LOG(ERROR) << "Failed to mark job state, job id = " << job_id; } else { @@ -176,6 +180,8 @@ void GcsJobManager::HandleMarkJobFinished(rpc::MarkJobFinishedRequest request, job_id, [this, job_id, send_reply](const Status &status, const std::optional &result) { + RAY_CHECK(thread_checker_.IsOnSameThread()); + if (status.ok() && result) { MarkJobAsFinished(*result, send_reply); return; @@ -266,6 +272,8 @@ void GcsJobManager::HandleGetAllJobInfo(rpc::GetAllJobInfoRequest request, }; auto on_done = [this, filter_ok, request, reply, send_reply_callback, limit]( const absl::flat_hash_map &&result) { + RAY_CHECK(thread_checker_.IsOnSameThread()); + // Internal KV keys for jobs that were submitted via the Ray Job API. std::vector job_api_data_keys; @@ -447,6 +455,8 @@ void GcsJobManager::OnNodeDead(const NodeID &node_id) { << "Node failed, mark all jobs from this node as finished"; auto on_done = [this, node_id](const absl::flat_hash_map &result) { + RAY_CHECK(thread_checker_.IsOnSameThread()); + // If job is not dead and from driver in current node, then mark it as finished for (auto &data : result) { if (!data.second.is_dead() && diff --git a/src/ray/gcs/gcs_server/gcs_job_manager.h b/src/ray/gcs/gcs_server/gcs_job_manager.h index b74b24f3e1d1d..95f43c7e27ad2 100644 --- a/src/ray/gcs/gcs_server/gcs_job_manager.h +++ b/src/ray/gcs/gcs_server/gcs_job_manager.h @@ -31,6 +31,7 @@ #include "ray/rpc/worker/core_worker_client.h" #include "ray/rpc/worker/core_worker_client_pool.h" #include "ray/util/event.h" +#include "ray/util/thread_checker.h" namespace ray { namespace gcs { @@ -107,6 +108,10 @@ class GcsJobManager : public rpc::JobInfoHandler { void MarkJobAsFinished(rpc::JobTableData job_table_data, std::function done_callback); + // Used to validate invariants for threading; for example, all callbacks are executed on + // the same thread. + ThreadChecker thread_checker_; + // Running Job IDs, used to report metrics. absl::flat_hash_set running_job_ids_; From b10ffdcd5aa8d5a56af055b00994545ae3819e2d Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Tue, 15 Oct 2024 09:16:17 -0700 Subject: [PATCH 23/46] [Core] Update expired redis.crt (#48020) Signed-off-by: Jiajun Yao --- python/ray/tests/tls/README | 27 +++++++++++++++++++++++++++ python/ray/tests/tls/ca.txt | 2 +- python/ray/tests/tls/redis.crt | 26 +++++++++++++------------- 3 files changed, 41 insertions(+), 14 deletions(-) create mode 100644 python/ray/tests/tls/README diff --git a/python/ray/tests/tls/README b/python/ray/tests/tls/README new file mode 100644 index 0000000000000..0115bebbc7a10 --- /dev/null +++ b/python/ray/tests/tls/README @@ -0,0 +1,27 @@ +These files are generated with the following command: + +mkdir -p {str(tmp_path)}/tls +openssl genrsa -out {str(tmp_path)}/tls/ca.key 4096 +openssl req \ + -x509 -new -nodes -sha256 \ + -key {str(tmp_path)}/tls/ca.key \ + -days 3650 \ + -subj '/O=Redis Test/CN=Certificate Authority' \ + -out {str(tmp_path)}/tls/ca.crt +openssl genrsa -out {str(tmp_path)}/tls/redis.key 2048 +openssl req \ + -new -sha256 \ + -key {str(tmp_path)}/tls/redis.key \ + -subj '/O=Redis Test/CN=Server' | \ + openssl x509 \ + -req -sha256 \ + -CA {str(tmp_path)}/tls/ca.crt \ + -CAkey {str(tmp_path)}/tls/ca.key \ + -CAserial {str(tmp_path)}/tls/ca.txt \ + -CAcreateserial \ + -days 3650 \ + -out {str(tmp_path)}/tls/redis.crt +openssl dhparam -out {str(tmp_path)}/tls/redis.dh 2048 + + +See https://github.com/ray-project/ray/pull/40378/ for more details \ No newline at end of file diff --git a/python/ray/tests/tls/ca.txt b/python/ray/tests/tls/ca.txt index 55fca9ab4c0f4..3c937ac1ca773 100644 --- a/python/ray/tests/tls/ca.txt +++ b/python/ray/tests/tls/ca.txt @@ -1 +1 @@ -75703BF2CC43AFFC5692C7B72687A196C4040599 +75703BF2CC43AFFC5692C7B72687A196C404059A diff --git a/python/ray/tests/tls/redis.crt b/python/ray/tests/tls/redis.crt index 19baa8870fc92..53777f13eddf7 100644 --- a/python/ray/tests/tls/redis.crt +++ b/python/ray/tests/tls/redis.crt @@ -1,7 +1,7 @@ -----BEGIN CERTIFICATE----- -MIID4jCCAcoCFHVwO/LMQ6/8VpLHtyaHoZbEBAWZMA0GCSqGSIb3DQEBCwUAMDUx +MIID4jCCAcoCFHVwO/LMQ6/8VpLHtyaHoZbEBAWaMA0GCSqGSIb3DQEBCwUAMDUx EzARBgNVBAoMClJlZGlzIFRlc3QxHjAcBgNVBAMMFUNlcnRpZmljYXRlIEF1dGhv -cml0eTAeFw0yMzEwMTUwMjExNDNaFw0yNDEwMTQwMjExNDNaMCYxEzARBgNVBAoM +cml0eTAeFw0yNDEwMTUwNDM0MzlaFw0zNDEwMTMwNDM0MzlaMCYxEzARBgNVBAoM ClJlZGlzIFRlc3QxDzANBgNVBAMMBlNlcnZlcjCCASIwDQYJKoZIhvcNAQEBBQAD ggEPADCCAQoCggEBAMLnixXdFw0vS4ewwRfkYEcOWDBxCmW8jj9MUmQ/QZmd01Ic Ybs9AFf+t4p6Z5woggVLBNdjgYNnKCwP94cL4cGNKDIyP2NHl5IJ7KJHsBpeibys @@ -9,15 +9,15 @@ Ybs9AFf+t4p6Z5woggVLBNdjgYNnKCwP94cL4cGNKDIyP2NHl5IJ7KJHsBpeibys Tgqa88FSxTi5PenSpmvykPiK4ZY35ZG/j1hUqHFwmFESCLTQJZzFP5NmXUpumJG3 R2d4dpaYzrAKzHt1qnd8ByI2X/I8USCFbwEqunmsKxmFDZZWyOS/+d96THHEYJe3 ZImFMIXAdvdLou1MYfvO3tamXIyRh+zedr/Cp1kCAwEAATANBgkqhkiG9w0BAQsF -AAOCAgEAnVXfIi4NWv/VxbV/ylLkTLC5zY2bn+oiuiapPilCo4858fbdmiwKk7IC -mCksbY4MN2+nelKzypre8HwKDv+MoLKyGO3mo+m6P9xS0BIvMeYNJOgLld81dgP+ -pLShP3LN+5u26gmGjhABFEpUOolX5m75bRq8BPoA4hAg2CNVPZt7GVyPXWqgl/PA -OKWrtrsQBbzL1yLkZFgeDeWKPwr7qfAPKG3Hr4yhURv2jKzJa7u6F+zeqkNR+pab -SZRj1b6YPJLZDUznQ8eO5XiwtCqCkaMixfLKStTQ8RtL3AtSglitiISZ7MKF9tha -+EEMgKMhYvhhImOuaMydePT8BRqTL495Bd3lzDa9MhooID2Ur0qE/TmwPH5fCo1r -+olZHoTMrMftlWsJQVQrOafsvAM5Df+yqeyjQXdEUY0cLYp/q9Dg99WUl0sct2G0 -PDd165xsnEBCoBqvNdbYJjdKfFk2GaLgjQB2CntbhRgDz+pt6glXDTexD0mfSa30 -dJoB1wI94XerOxQdPur+XDL3i/W84HT9I5tpcw1ywr7QmFMGTRk+OlNhDBZGkHGu -guBQOn/zwbivtAuGYgXsX42FgRmCwSZOp5sth2x8D4zW5mu1QIbFWRDswCJqQXfd -h0oFOJE3nEmTQmSSsCN+jpV3U1pB5/r0CDNFZNCvZwfoW644HdA= +AAOCAgEAQgZ/uixRdNblqPSGhsFxeAxaZonBVg7akKG49Tk/wg/WwgCxvy76l0WS ++/GgWTXRNqM6BZbXggof/6Nh5pPGTJgcJmSORcEDNnv7R1FVrp4H2TPRVwf0g/NP +n3rqtM6WHIzJ9olv/U3J/U9vcyk42cuYscaJGHFMjoPnqq2ISltYb6lIbOxD6stz +oJPOnZNZyAILWMr5DCIZg67z2+ZZo4Y0epzmfdcg/Xlw6bV9xiOeLIENlG1Rz3a3 +Jw+zMAnU41gbRPZC0hZxq4K3qsCYTe/RmGz4YeF7eBXLHEdFQfz4mq2IzlCz6+Gh +BDgBQeMjqZzXsxk0BzHIPbiPD+WOUV4YZyap9D7b5nv4bDlQIAV2do6MuMLB16n1 +xVl98wbNCGpJvFeqzUlf7djQJmmDrz0egj05oEW7UQt9b1s3zswLZK4wAY0ZtTsZ +QBhuN6Ez/pD4ydsluPXQCBQaBtMlXMsMWCsLZ5N0luAhgqEl268PP03cEwwLCKk4 +Wk2k9Spj8ARKFykLxeLvlJSbfVUioKwduXBmcKdwBkJEoSqW51FlHXwPApq2r1uD +scWCRzs0H0t1H/BX2RHhDQrNvXZffBMkwTkXUOz5wLTHiynnKzgcBea0WZyhlOfK +0rl5SgsIeNbxydAL/TGydBpqA+MMmUpelEl6JEvQ8YAuhbNZGO4= -----END CERTIFICATE----- From e3d2668dc6093ea914449d6fce9a1599b76c4beb Mon Sep 17 00:00:00 2001 From: Blocka Date: Wed, 16 Oct 2024 02:14:56 +0800 Subject: [PATCH 24/46] [Docs] Replace Python version 3.8 with 3.9 in development.rst (#47857) Replace the Python version in development.rst from 3.8 to 3.9, including the wheel link. --- doc/source/ray-contribute/development.rst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/doc/source/ray-contribute/development.rst b/doc/source/ray-contribute/development.rst index fafa770675f69..1e699d2133cb4 100644 --- a/doc/source/ray-contribute/development.rst +++ b/doc/source/ray-contribute/development.rst @@ -92,8 +92,8 @@ RLlib, Tune, Autoscaler, and most Python files do not require you to build and c .. code-block:: shell - # For example, for Python 3.8: - pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-3.0.0.dev0-cp38-cp38-manylinux2014_x86_64.whl + # For example, for Python 3.9: + pip install -U https://s3-us-west-2.amazonaws.com/ray-wheels/latest/ray-3.0.0.dev0-cp39-cp39-manylinux2014_x86_64.whl 4. Replace Python files in the installed package with your local editable copy. We provide a simple script to help you do this: ``python python/ray/setup-dev.py``. Running the script will remove the ``ray/tune``, ``ray/rllib``, ``ray/autoscaler`` dir (among other directories) bundled with the ``ray`` pip package, and replace them with links to your local code. This way, changing files in your git clone will directly affect the behavior of your installed Ray. From f04ddd611b70307ff4f97f11a1c5300a23d74ad8 Mon Sep 17 00:00:00 2001 From: Mengjin Yan Date: Tue, 15 Oct 2024 11:58:53 -0700 Subject: [PATCH 25/46] [core] Fix the Paths in the Generated Monitoring Configs to Consider temp-dir (#47871) Today, when a user specifies the `--temp-dir` when starting ray, the paths in the default config files generated are still with the default root directory `/tmp/ray`. The issue is because the default config files are directly copied from the package when ray starts and the paths are all hardcoded. This PR fix the issue by generating the corresponding config files with the corresponding path at runtime. Also: * keep using the default path with the hard coded file in the `ray metrics launch-prometheus` command because the command can be executed with/without ray cluster started. The doc is updated accordingly to suggest the user to use manual mode to start prometheus is --temp-dir will be specified * Updated the tests to: * add verification of the monitoring config generation for the case of specify temp-dir * add verification of the paths in the config content Signed-off-by: Mengjin Yan --- doc/source/cluster/metrics.md | 14 ++- .../metrics/export/grafana/grafana.ini | 10 -- .../metrics/grafana_dashboard_factory.py | 3 - ...grafana_dashboard_provisioning_template.py | 10 -- .../metrics/grafana_datasource_template.py | 9 -- .../metrics/install_and_start_prometheus.py | 13 +-- .../dashboard/modules/metrics/metrics_head.py | 91 +++++++++++++------ .../dashboard/modules/metrics/templates.py | 49 ++++++++++ .../modules/tests/test_metrics_integration.py | 15 +++ python/ray/tests/test_metrics_head.py | 30 +++++- 10 files changed, 173 insertions(+), 71 deletions(-) delete mode 100644 python/ray/dashboard/modules/metrics/export/grafana/grafana.ini delete mode 100644 python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py delete mode 100644 python/ray/dashboard/modules/metrics/grafana_datasource_template.py create mode 100644 python/ray/dashboard/modules/metrics/templates.py diff --git a/doc/source/cluster/metrics.md b/doc/source/cluster/metrics.md index 6ef27ca5af76d..5aa19c89e5c67 100644 --- a/doc/source/cluster/metrics.md +++ b/doc/source/cluster/metrics.md @@ -22,6 +22,12 @@ For a quick demo, you can run Prometheus locally on your machine. Follow the qui ### Quickstart: Running Prometheus locally +```{admonition} Note +:class: note +If you need to change the root temporary directory by using "--temp-dir" in your ray +cluster setup, please follow the [manual steps](#optional-manual-running-prometheus-locally) to setup Prometheus locally. +``` + Run the following command to download and start Prometheus locally with a configuration that scrapes metrics from a local Ray Cluster. ```bash @@ -76,7 +82,7 @@ tar xvfz prometheus-*.tar.gz cd prometheus-* ``` -Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. +Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. If you specify the `--temp-dir={your_temp_path}` when starting the ray cluster, the config file will be at `{yout_temp_path}/session_latest/metrics/prometheus/prometheus.yml` ```yaml global: @@ -88,13 +94,17 @@ scrape_configs: - job_name: 'ray' file_sd_configs: - files: - - '/tmp/ray/prom_metrics_service_discovery.json' + - '/tmp/ray/prom_metrics_service_discovery.json' # or '${your_temp_path}/prom_metrics_service_discovery.json' if --temp-dir is specified ``` Next, start Prometheus: ```shell +# With default settings ./prometheus --config.file=/tmp/ray/session_latest/metrics/prometheus/prometheus.yml + +# With specified --temp-dir +./prometheus --config.file={your_temp_path}/session_latest/metrics/prometheus/prometheus.yml ``` ```{admonition} Note :class: note diff --git a/python/ray/dashboard/modules/metrics/export/grafana/grafana.ini b/python/ray/dashboard/modules/metrics/export/grafana/grafana.ini deleted file mode 100644 index e349fc5b29076..0000000000000 --- a/python/ray/dashboard/modules/metrics/export/grafana/grafana.ini +++ /dev/null @@ -1,10 +0,0 @@ -[security] -allow_embedding = true - -[auth.anonymous] -enabled = true -org_name = Main Org. -org_role = Viewer - -[paths] -provisioning = /tmp/ray/session_latest/metrics/grafana/provisioning diff --git a/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py b/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py index 527928fb4e92c..8d328c7084012 100644 --- a/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py +++ b/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py @@ -19,9 +19,6 @@ serve_deployment_dashboard_config, ) -METRICS_INPUT_ROOT = os.path.join(os.path.dirname(__file__), "export") -GRAFANA_CONFIG_INPUT_PATH = os.path.join(METRICS_INPUT_ROOT, "grafana") - GRAFANA_DASHBOARD_UID_OVERRIDE_ENV_VAR_TEMPLATE = "RAY_GRAFANA_{name}_DASHBOARD_UID" GRAFANA_DASHBOARD_GLOBAL_FILTERS_OVERRIDE_ENV_VAR_TEMPLATE = ( "RAY_GRAFANA_{name}_DASHBOARD_GLOBAL_FILTERS" diff --git a/python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py b/python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py deleted file mode 100644 index 254da09efad7b..0000000000000 --- a/python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py +++ /dev/null @@ -1,10 +0,0 @@ -DASHBOARD_PROVISIONING_TEMPLATE = """ -apiVersion: 1 - -providers: - - name: Ray # Default dashboards provided by OSS ray - folder: Ray - type: file - options: - path: {dashboard_output_folder} -""" diff --git a/python/ray/dashboard/modules/metrics/grafana_datasource_template.py b/python/ray/dashboard/modules/metrics/grafana_datasource_template.py deleted file mode 100644 index 01fc06bc3be9e..0000000000000 --- a/python/ray/dashboard/modules/metrics/grafana_datasource_template.py +++ /dev/null @@ -1,9 +0,0 @@ -GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1 - -datasources: - - name: {prometheus_name} - url: {prometheus_host} - type: prometheus - isDefault: true - access: proxy -""" diff --git a/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py b/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py index f060a98d6d114..541c9dfd6b52c 100644 --- a/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py +++ b/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py @@ -74,12 +74,13 @@ def install_prometheus(file_path): def start_prometheus(prometheus_dir): - # Currently, Ray never modifies this config file, so we can just use the - # hardcoded path. (It just copies it to a more user-friendly location, in - # MetricsHead._create_default_prometheus_configs.) - # However, if in the future Ray ever modifies this file at runtime, we'll - # need to use the user-friendly location instead, and reload the config - # file after it's updated by Ray. + # The function assumes the Ray cluster to be monitored by Prometheus uses the + # default configuration with "/tmp/ray" as the default root temporary directory. + # + # This is to support the `ray metrics launch-prometheus` command, when a ray cluster + # is not started yet and we have no way to get a `--temp-dir` anywhere. So we choose + # to use a hardcoded default value. + config_file = Path(PROMETHEUS_CONFIG_INPUT_PATH) if not config_file.exists(): diff --git a/python/ray/dashboard/modules/metrics/metrics_head.py b/python/ray/dashboard/modules/metrics/metrics_head.py index 04c80255fecb2..ae0d38e73b0a9 100644 --- a/python/ray/dashboard/modules/metrics/metrics_head.py +++ b/python/ray/dashboard/modules/metrics/metrics_head.py @@ -11,24 +11,24 @@ import ray.dashboard.optional_utils as dashboard_optional_utils import ray.dashboard.utils as dashboard_utils from ray._private.async_utils import enable_monitor_loop_lag -from ray._private.ray_constants import env_integer -from ray._private.utils import get_or_create_event_loop -from ray.dashboard.consts import ( - AVAILABLE_COMPONENT_NAMES_FOR_METRICS, - METRICS_INPUT_ROOT, - PROMETHEUS_CONFIG_INPUT_PATH, +from ray._private.ray_constants import ( + PROMETHEUS_SERVICE_DISCOVERY_FILE, + SESSION_LATEST, + env_integer, ) +from ray._private.utils import get_or_create_event_loop +from ray.dashboard.consts import AVAILABLE_COMPONENT_NAMES_FOR_METRICS from ray.dashboard.modules.metrics.grafana_dashboard_factory import ( generate_data_grafana_dashboard, generate_default_grafana_dashboard, generate_serve_deployment_grafana_dashboard, generate_serve_grafana_dashboard, ) -from ray.dashboard.modules.metrics.grafana_dashboard_provisioning_template import ( +from ray.dashboard.modules.metrics.templates import ( DASHBOARD_PROVISIONING_TEMPLATE, -) -from ray.dashboard.modules.metrics.grafana_datasource_template import ( GRAFANA_DATASOURCE_TEMPLATE, + GRAFANA_INI_TEMPLATE, + PROMETHEUS_YML_TEMPLATE, ) import psutil @@ -54,7 +54,6 @@ GRAFANA_HOST_DISABLED_VALUE = "DISABLED" GRAFANA_IFRAME_HOST_ENV_VAR = "RAY_GRAFANA_IFRAME_HOST" GRAFANA_DASHBOARD_OUTPUT_DIR_ENV_VAR = "RAY_METRICS_GRAFANA_DASHBOARD_OUTPUT_DIR" -GRAFANA_CONFIG_INPUT_PATH = os.path.join(METRICS_INPUT_ROOT, "grafana") GRAFANA_HEALTHCHECK_PATH = "api/health" @@ -75,13 +74,22 @@ def __init__(self, dashboard_head): PROMETHEUS_HOST_ENV_VAR, DEFAULT_PROMETHEUS_HOST ) default_metrics_root = os.path.join(self._dashboard_head.session_dir, "metrics") + session_latest_metrics_root = os.path.join( + self._dashboard_head.temp_dir, SESSION_LATEST, "metrics" + ) self._metrics_root = os.environ.get( METRICS_OUTPUT_ROOT_ENV_VAR, default_metrics_root ) - grafana_config_output_path = os.path.join(self._metrics_root, "grafana") + self._metrics_root_session_latest = os.environ.get( + METRICS_OUTPUT_ROOT_ENV_VAR, session_latest_metrics_root + ) + self._grafana_config_output_path = os.path.join(self._metrics_root, "grafana") + self._grafana_session_latest_config_output_path = os.path.join( + self._metrics_root_session_latest, "grafana" + ) self._grafana_dashboard_output_dir = os.environ.get( GRAFANA_DASHBOARD_OUTPUT_DIR_ENV_VAR, - os.path.join(grafana_config_output_path, "dashboards"), + os.path.join(self._grafana_config_output_path, "dashboards"), ) self._prometheus_name = os.environ.get( @@ -186,17 +194,34 @@ def _create_default_grafana_configs(self): """ Creates the grafana configurations that are by default provided by Ray. """ - grafana_config_output_path = os.path.join(self._metrics_root, "grafana") - - # Copy default grafana configurations - if os.path.exists(grafana_config_output_path): - shutil.rmtree(grafana_config_output_path) - os.makedirs(os.path.dirname(grafana_config_output_path), exist_ok=True) - shutil.copytree(GRAFANA_CONFIG_INPUT_PATH, grafana_config_output_path) + # Create grafana configuration folder + if os.path.exists(self._grafana_config_output_path): + shutil.rmtree(self._grafana_config_output_path) + os.makedirs(self._grafana_config_output_path, exist_ok=True) + + # Overwrite grafana's configuration file + grafana_provisioning_folder = os.path.join( + self._grafana_config_output_path, "provisioning" + ) + grafana_prov_folder_with_latest_session = os.path.join( + self._grafana_session_latest_config_output_path, "provisioning" + ) + with open( + os.path.join( + self._grafana_config_output_path, + "grafana.ini", + ), + "w", + ) as f: + f.write( + GRAFANA_INI_TEMPLATE.format( + grafana_provisioning_folder=grafana_prov_folder_with_latest_session + ) + ) # Overwrite grafana's dashboard provisioning directory based on env var dashboard_provisioning_path = os.path.join( - grafana_config_output_path, "provisioning", "dashboards" + grafana_provisioning_folder, "dashboards" ) os.makedirs( dashboard_provisioning_path, @@ -219,9 +244,7 @@ def _create_default_grafana_configs(self): prometheus_host = os.environ.get( PROMETHEUS_HOST_ENV_VAR, DEFAULT_PROMETHEUS_HOST ) - data_sources_path = os.path.join( - grafana_config_output_path, "provisioning", "datasources" - ) + data_sources_path = os.path.join(grafana_provisioning_folder, "datasources") os.makedirs( data_sources_path, exist_ok=True, @@ -297,14 +320,26 @@ def _create_default_prometheus_configs(self): self._metrics_root, "prometheus", "prometheus.yml" ) - # Copy default prometheus configurations + # Generate the default prometheus configurations if os.path.exists(prometheus_config_output_path): os.remove(prometheus_config_output_path) os.makedirs(os.path.dirname(prometheus_config_output_path), exist_ok=True) - # Currently Ray directly copies this file without modifying it at runtime. - # If Ray ever modifies this file at runtime, please ensure start_prometheus - # in install_and_start_prometheus.py is updated to reload the config file. - shutil.copy(PROMETHEUS_CONFIG_INPUT_PATH, prometheus_config_output_path) + + # This code generates the Prometheus config based on the custom temporary root + # path set by the user at Ray cluster start up (via --temp-dir). In contrast, + # start_prometheus in install_and_start_prometheus.py uses a hardcoded + # Prometheus config at PROMETHEUS_CONFIG_INPUT_PATH that always uses "/tmp/ray". + # Other than the root path, the config file generated here is identical to that + # hardcoded config file. + prom_discovery_file_path = os.path.join( + self._dashboard_head.temp_dir, PROMETHEUS_SERVICE_DISCOVERY_FILE + ) + with open(prometheus_config_output_path, "w") as f: + f.write( + PROMETHEUS_YML_TEMPLATE.format( + prom_metrics_service_discovery_file_path=prom_discovery_file_path + ) + ) @dashboard_utils.async_loop_forever(METRICS_RECORD_INTERVAL_S) async def record_dashboard_metrics(self): diff --git a/python/ray/dashboard/modules/metrics/templates.py b/python/ray/dashboard/modules/metrics/templates.py new file mode 100644 index 0000000000000..d78b570ff7bd5 --- /dev/null +++ b/python/ray/dashboard/modules/metrics/templates.py @@ -0,0 +1,49 @@ +GRAFANA_INI_TEMPLATE = """ +[security] +allow_embedding = true + +[auth.anonymous] +enabled = true +org_name = Main Org. +org_role = Viewer + +[paths] +provisioning = {grafana_provisioning_folder} +""" + +DASHBOARD_PROVISIONING_TEMPLATE = """ +apiVersion: 1 + +providers: + - name: Ray # Default dashboards provided by OSS ray + folder: Ray + type: file + options: + path: {dashboard_output_folder} +""" + +GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1 + +datasources: + - name: {prometheus_name} + url: {prometheus_host} + type: prometheus + isDefault: true + access: proxy +""" + +PROMETHEUS_YML_TEMPLATE = """# my global config +global: + scrape_interval: 10s # Set the scrape interval to every 10 seconds. Default is every \ +1 minute. + evaluation_interval: 10s # Evaluate rules every 10 seconds. The default is every 1 \ +minute. + # scrape_timeout is set to the global default (10s). + +scrape_configs: +# Scrape from each ray node as defined in the service_discovery.json provided by ray. +- job_name: 'ray' + file_sd_configs: + - files: + - '{prom_metrics_service_discovery_file_path}' +""" diff --git a/python/ray/dashboard/modules/tests/test_metrics_integration.py b/python/ray/dashboard/modules/tests/test_metrics_integration.py index 71f8630d1abc8..0d0ef13cdf068 100644 --- a/python/ray/dashboard/modules/tests/test_metrics_integration.py +++ b/python/ray/dashboard/modules/tests/test_metrics_integration.py @@ -3,7 +3,9 @@ import pytest +from ray.dashboard.consts import PROMETHEUS_CONFIG_INPUT_PATH from ray.dashboard.modules.metrics import install_and_start_prometheus +from ray.dashboard.modules.metrics.templates import PROMETHEUS_YML_TEMPLATE @pytest.mark.parametrize( @@ -38,5 +40,18 @@ def test_e2e(capsys): subprocess.run(["kill", str(pid)]) +def test_prometheus_config_content(): + # Test to make sure the content in the hardcoded file + # (python/ray/dashboard/modules/metrics/export/prometheus/prometheus.yml) will + # always be the same as the template (templates.py) used to generate prometheus + # config file when ray startup + PROM_DISCOVERY_FILE_PATH = "/tmp/ray/prom_metrics_service_discovery.json" + template_content = PROMETHEUS_YML_TEMPLATE.format( + prom_metrics_service_discovery_file_path=PROM_DISCOVERY_FILE_PATH + ) + with open(PROMETHEUS_CONFIG_INPUT_PATH) as f: + assert f.read() == template_content + + if __name__ == "__main__": sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/tests/test_metrics_head.py b/python/ray/tests/test_metrics_head.py index 2e358ce208f3c..d6010d88b3925 100644 --- a/python/ray/tests/test_metrics_head.py +++ b/python/ray/tests/test_metrics_head.py @@ -13,20 +13,35 @@ SERVE_GRAFANA_PANELS, ) from ray.tests.conftest import _ray_start +from ray._private.ray_constants import SESSION_LATEST logger = logging.getLogger(__name__) -def test_metrics_folder(): +@pytest.mark.parametrize( + "is_temp_dir_set, temp_dir_val", [(False, ""), (True, "/tmp/test-temp-dir")] +) +def test_metrics_folder_and_content(is_temp_dir_set, temp_dir_val): """ - Tests that the default dashboard files get created. + Tests that the default dashboard files get created. It also verifies paths in the + dashboard config files are set correctly. + + It checks both the default case and the case where the _temp_dir is specified. """ - with _ray_start(include_dashboard=True) as context: + with _ray_start( + include_dashboard=True, _temp_dir=temp_dir_val if is_temp_dir_set else None + ) as context: session_dir = context["session_dir"] + temp_dir = temp_dir_val if is_temp_dir_set else "/tmp/ray" assert os.path.exists( f"{session_dir}/metrics/grafana/provisioning/dashboards/default.yml" ) + with open( + f"{session_dir}/metrics/grafana/provisioning/dashboards/default.yml", "r" + ) as f: + assert f"path: {session_dir}/metrics/grafana/dashboards" in f.read() + assert os.path.exists( f"{session_dir}/metrics/grafana/dashboards" "/default_grafana_dashboard.json" @@ -34,8 +49,17 @@ def test_metrics_folder(): assert os.path.exists( f"{session_dir}/metrics/grafana/provisioning/datasources/default.yml" ) + assert os.path.exists(f"{session_dir}/metrics/grafana/grafana.ini") + with open(f"{session_dir}/metrics/grafana/grafana.ini", "r") as f: + assert ( + "provisioning = " + f"{temp_dir}/{SESSION_LATEST}/metrics/grafana/provisioning" in f.read() + ) + assert os.path.exists(f"{session_dir}/metrics/prometheus/prometheus.yml") + with open(f"{session_dir}/metrics/prometheus/prometheus.yml", "r") as f: + assert f"- '{temp_dir}/prom_metrics_service_discovery.json'" in f.read() @pytest.fixture From 347af0e3eab3fb32a7776849d9aa2fbb58373e5a Mon Sep 17 00:00:00 2001 From: Scott Lee Date: Tue, 15 Oct 2024 13:25:50 -0700 Subject: [PATCH 26/46] [Data] Link `PhysicalOperator` to its `LogicalOperator` (#47986) ## Why are these changes needed? It can be useful to know the `LogicalOperator` which was translated to the particular `PhysicalOperator`. For example, if we are trying to determine the exact type of Read operation, the PhysicalOperator will be a `Map` operator, which is ambiguous (we can only tell by examining the operator name, which is not reliable). This PR links each `PhysicalOperator` to its originating `LogicalOperator`s upon creation in the execution plan optimizer, so that we can access this information during execution. ## Related issue number ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [x] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Scott Lee --- .../execution/interfaces/physical_operator.py | 11 ++++- .../logical/rules/operator_fusion.py | 1 + python/ray/data/_internal/planner/planner.py | 13 ++++++ .../data/tests/test_execution_optimizer.py | 41 ++++++++++++++++--- 4 files changed, 59 insertions(+), 7 deletions(-) diff --git a/python/ray/data/_internal/execution/interfaces/physical_operator.py b/python/ray/data/_internal/execution/interfaces/physical_operator.py index fdb426264096b..be9bcc88ee8d5 100644 --- a/python/ray/data/_internal/execution/interfaces/physical_operator.py +++ b/python/ray/data/_internal/execution/interfaces/physical_operator.py @@ -12,7 +12,7 @@ ExecutionResources, ) from ray.data._internal.execution.interfaces.op_runtime_metrics import OpRuntimeMetrics -from ray.data._internal.logical.interfaces import Operator +from ray.data._internal.logical.interfaces import LogicalOperator, Operator from ray.data._internal.stats import StatsDict from ray.data.context import DataContext @@ -188,6 +188,9 @@ def __init__( self._estimated_num_output_bundles = None self._estimated_output_num_rows = None self._execution_completed = False + # The LogicalOperator(s) which were translated to create this PhysicalOperator. + # Set via `PhysicalOperator.set_logical_operators()`. + self._logical_operators: List[LogicalOperator] = [] def __reduce__(self): raise ValueError("Operator is not serializable.") @@ -205,6 +208,12 @@ def output_dependencies(self) -> List["PhysicalOperator"]: def post_order_iter(self) -> Iterator["PhysicalOperator"]: return super().post_order_iter() # type: ignore + def set_logical_operators( + self, + *logical_ops: LogicalOperator, + ): + self._logical_operators = list(logical_ops) + @property def target_max_block_size(self) -> Optional[int]: """ diff --git a/python/ray/data/_internal/logical/rules/operator_fusion.py b/python/ray/data/_internal/logical/rules/operator_fusion.py index 30fdb2dc4fa11..ab6730bc63dcd 100644 --- a/python/ray/data/_internal/logical/rules/operator_fusion.py +++ b/python/ray/data/_internal/logical/rules/operator_fusion.py @@ -325,6 +325,7 @@ def _get_fused_map_operator( ray_remote_args=ray_remote_args, ray_remote_args_fn=ray_remote_args_fn, ) + op.set_logical_operators(*up_op._logical_operators, *down_op._logical_operators) # Build a map logical operator to be used as a reference for further fusion. # TODO(Scott): This is hacky, remove this once we push fusion to be purely based diff --git a/python/ray/data/_internal/planner/planner.py b/python/ray/data/_internal/planner/planner.py index d47afc43b7158..3cc97f4db6cb0 100644 --- a/python/ray/data/_internal/planner/planner.py +++ b/python/ray/data/_internal/planner/planner.py @@ -127,5 +127,18 @@ def _plan(self, logical_op: LogicalOperator) -> PhysicalOperator: f"Found unknown logical operator during planning: {logical_op}" ) + # Traverse up the DAG, and set the mapping from physical to logical operators. + # At this point, all physical operators without logical operators set + # must have been created by the current logical operator. + queue = [physical_op] + while queue: + curr_physical_op = queue.pop() + # Once we find an operator with a logical operator set, we can stop. + if curr_physical_op._logical_operators: + break + + curr_physical_op.set_logical_operators(logical_op) + queue.extend(physical_op.input_dependencies) + self._physical_op_to_logical_op[physical_op] = logical_op return physical_op diff --git a/python/ray/data/tests/test_execution_optimizer.py b/python/ray/data/tests/test_execution_optimizer.py index 913c7ee1822bf..25b8b051d27c7 100644 --- a/python/ray/data/tests/test_execution_optimizer.py +++ b/python/ray/data/tests/test_execution_optimizer.py @@ -111,6 +111,9 @@ def test_read_operator(ray_start_regular_shared): physical_op.actual_target_max_block_size == DataContext.get_current().target_max_block_size ) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + assert physical_op.input_dependencies[0]._logical_operators == [op] def test_read_operator_emits_warning_for_large_read_tasks(): @@ -182,6 +185,9 @@ def test_from_operators(ray_start_regular_shared): assert isinstance(physical_op, InputDataBuffer) assert len(physical_op.input_dependencies) == 0 + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + def test_from_items_e2e(ray_start_regular_shared): data = ["Hello", "World"] @@ -253,6 +259,9 @@ def test_map_batches_operator(ray_start_regular_shared): assert len(physical_op.input_dependencies) == 1 assert isinstance(physical_op.input_dependencies[0], MapOperator) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + def test_map_batches_e2e(ray_start_regular_shared): ds = ray.data.range(5) @@ -393,6 +402,9 @@ def test_random_shuffle_operator(ray_start_regular_shared): == DataContext.get_current().target_shuffle_max_block_size ) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + def test_random_shuffle_e2e(ray_start_regular_shared, use_push_based_shuffle): ds = ray.data.range(12, override_num_blocks=4) @@ -430,6 +442,9 @@ def test_repartition_operator(ray_start_regular_shared, shuffle): == DataContext.get_current().target_max_block_size ) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + @pytest.mark.parametrize( "shuffle", @@ -506,6 +521,9 @@ def test_union_operator(ray_start_regular_shared, preserve_order): == DataContext.get_current().target_max_block_size ) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [union_op] + @pytest.mark.parametrize("preserve_order", (True, False)) def test_union_e2e(ray_start_regular_shared, preserve_order): @@ -578,22 +596,23 @@ def test_read_map_batches_operator_fusion(ray_start_regular_shared): physical_op.actual_target_max_block_size == DataContext.get_current().target_max_block_size ) + assert physical_op._logical_operators == [read_op, op] def test_read_map_chain_operator_fusion(ray_start_regular_shared): # Test that a chain of different map operators are fused. planner = Planner() read_op = get_parquet_read_logical_op(parallelism=1) - op = MapRows(read_op, lambda x: x) - op = MapBatches(op, lambda x: x) - op = FlatMap(op, lambda x: x) - op = Filter(op, lambda x: x) - logical_plan = LogicalPlan(op) + map1 = MapRows(read_op, lambda x: x) + map2 = MapBatches(map1, lambda x: x) + map3 = FlatMap(map2, lambda x: x) + map4 = Filter(map3, lambda x: x) + logical_plan = LogicalPlan(map4) physical_plan = planner.plan(logical_plan) physical_plan = PhysicalOptimizer().optimize(physical_plan) physical_op = physical_plan.dag - assert op.name == "Filter()" + assert map4.name == "Filter()" assert ( physical_op.name == "ReadParquet->Map()->MapBatches()" "->FlatMap()->Filter()" @@ -605,6 +624,7 @@ def test_read_map_chain_operator_fusion(ray_start_regular_shared): physical_op.actual_target_max_block_size == DataContext.get_current().target_max_block_size ) + assert physical_op._logical_operators == [read_op, map1, map2, map3, map4] def test_read_map_batches_operator_fusion_compatible_remote_args( @@ -1009,6 +1029,9 @@ def test_write_operator(ray_start_regular_shared, tmp_path): assert len(physical_op.input_dependencies) == 1 assert isinstance(physical_op.input_dependencies[0], MapOperator) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + def test_sort_operator( ray_start_regular_shared, @@ -1105,6 +1128,9 @@ def test_aggregate_operator(ray_start_regular_shared): == DataContext.get_current().target_shuffle_max_block_size ) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + def test_aggregate_e2e(ray_start_regular_shared, use_push_based_shuffle): ds = ray.data.range(100, override_num_blocks=4) @@ -1171,6 +1197,9 @@ def test_zip_operator(ray_start_regular_shared): == DataContext.get_current().target_max_block_size ) + # Check that the linked logical operator is the same the input op. + assert physical_op._logical_operators == [op] + @pytest.mark.parametrize( "num_blocks1,num_blocks2", From 680468ebf0d04c3549e24f71faf8bfed9043053e Mon Sep 17 00:00:00 2001 From: Gene Der Su Date: Tue, 15 Oct 2024 13:53:40 -0700 Subject: [PATCH 27/46] [Serve] Refactor helper function to add grpc address (#47720) ## Why are these changes needed? Refactor a helper function to add grpc address. ## Related issue number Closes https://github.com/anyscale/product/issues/30204 ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Gene Su --- python/ray/serve/_private/grpc_util.py | 5 +++++ python/ray/serve/_private/proxy.py | 8 ++++++-- python/ray/serve/tests/unit/test_grpc_util.py | 18 ++++++++++++++++++ 3 files changed, 29 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/_private/grpc_util.py b/python/ray/serve/_private/grpc_util.py index 095cd87ee14b6..d522f1e7a9f3b 100644 --- a/python/ray/serve/_private/grpc_util.py +++ b/python/ray/serve/_private/grpc_util.py @@ -81,3 +81,8 @@ class DummyServicer: def __getattr__(self, attr): # No-op pass through. Just need this to act as the callable. pass + + +def add_grpc_address(grpc_server: gRPCServer, server_address: str): + """Helper function to add a address to gRPC server.""" + grpc_server.add_insecure_port(server_address) diff --git a/python/ray/serve/_private/proxy.py b/python/ray/serve/_private/proxy.py index 157855c549895..d28a10e169cc1 100644 --- a/python/ray/serve/_private/proxy.py +++ b/python/ray/serve/_private/proxy.py @@ -40,7 +40,11 @@ SERVE_MULTIPLEXED_MODEL_ID, SERVE_NAMESPACE, ) -from ray.serve._private.grpc_util import DummyServicer, create_serve_grpc_server +from ray.serve._private.grpc_util import ( + DummyServicer, + add_grpc_address, + create_serve_grpc_server, +) from ray.serve._private.http_util import ( MessageQueue, convert_object_to_asgi_messages, @@ -1408,7 +1412,7 @@ async def run_grpc_server(self): service_handler_factory=self.grpc_proxy.service_handler_factory, ) - grpc_server.add_insecure_port(f"[::]:{self.grpc_port}") + add_grpc_address(grpc_server, f"[::]:{self.grpc_port}") # Dummy servicer is used to be callable for the gRPC server. Serve have a # custom gRPC server implementation to redirect calls into gRPCProxy. diff --git a/python/ray/serve/tests/unit/test_grpc_util.py b/python/ray/serve/tests/unit/test_grpc_util.py index 20d63dedaa097..c28fcd3e8e5dc 100644 --- a/python/ray/serve/tests/unit/test_grpc_util.py +++ b/python/ray/serve/tests/unit/test_grpc_util.py @@ -8,6 +8,7 @@ from ray import cloudpickle from ray.serve._private.grpc_util import ( DummyServicer, + add_grpc_address, create_serve_grpc_server, gRPCServer, ) @@ -15,6 +16,14 @@ from ray.serve.grpc_util import RayServegRPCContext +class FakeGrpcServer: + def __init__(self): + self.address = None + + def add_insecure_port(self, address): + self.address = address + + def fake_service_handler_factory(service_method: str, stream: bool) -> Callable: def foo() -> bytes: return f"{'stream' if stream else 'unary'} call from {service_method}".encode() @@ -120,6 +129,15 @@ def test_ray_serve_grpc_context_serializable(): assert deserialized_context.__dict__ == context.__dict__ +def test_add_grpc_address(): + """Test `add_grpc_address` adds the address to the gRPC server.""" + fake_grpc_server = FakeGrpcServer() + grpc_address = "fake_address:50051" + assert fake_grpc_server.address is None + add_grpc_address(fake_grpc_server, grpc_address) + assert fake_grpc_server.address == grpc_address + + if __name__ == "__main__": import sys From 5a65213b91bf0ec27a29b048c05ed2676065f5fc Mon Sep 17 00:00:00 2001 From: Scott Lee Date: Tue, 15 Oct 2024 14:10:53 -0700 Subject: [PATCH 28/46] [Data] Allow specifing both `num_cpus` and `num_gpus` for map APIs (#47995) ## Why are these changes needed? Currently, we enforce that `num_cpus` and `num_gpus` cannot be both set for map operations. This PR enables the user to specify both of these parameters, as we believe that with the recent improvements to the scheduler, Ray Data should be able to smoothly support this scenario. However, we will warn users that this is still an experimental feature. Updated doc example (updated for all map APIs): ![Screenshot at Oct 11 19-00-23](https://github.com/user-attachments/assets/2bfd6902-e2b4-4ae8-9550-604ec59d3b6b) ## Related issue number ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [x] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Scott Lee --- .../execution/operators/map_operator.py | 23 +++++++++++-------- python/ray/data/dataset.py | 18 +++++++++++++++ python/ray/data/grouped_data.py | 6 +++++ .../test_executor_resource_management.py | 20 +++++++++------- 4 files changed, 49 insertions(+), 18 deletions(-) diff --git a/python/ray/data/_internal/execution/operators/map_operator.py b/python/ray/data/_internal/execution/operators/map_operator.py index a3d98bbeee3bd..6a42e0c760af1 100644 --- a/python/ray/data/_internal/execution/operators/map_operator.py +++ b/python/ray/data/_internal/execution/operators/map_operator.py @@ -1,6 +1,7 @@ import copy import functools import itertools +import logging from abc import ABC, abstractmethod from collections import defaultdict, deque from typing import Any, Callable, Deque, Dict, Iterator, List, Optional, Set, Union @@ -37,6 +38,8 @@ from ray.data.context import DataContext from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy +logger = logging.getLogger(__name__) + class MapOperator(OneToOneOperator, ABC): """A streaming operator that maps input bundles 1:1 to output bundles. @@ -645,16 +648,16 @@ def _canonicalize_ray_remote_args(ray_remote_args: Dict[str, Any]) -> Dict[str, and should not be a serious limitation for users. """ ray_remote_args = ray_remote_args.copy() + + if ray_remote_args.get("num_cpus") and ray_remote_args.get("num_gpus"): + logger.warning( + "Specifying both num_cpus and num_gpus for map tasks is experimental, " + "and may result in scheduling or stability issues. " + "Please report any issues to the Ray team: " + "https://github.com/ray-project/ray/issues/new/choose" + ) + if "num_cpus" not in ray_remote_args and "num_gpus" not in ray_remote_args: ray_remote_args["num_cpus"] = 1 - if ray_remote_args.get("num_gpus", 0) > 0: - if ray_remote_args.get("num_cpus", 0) != 0: - raise ValueError( - "It is not allowed to specify both num_cpus and num_gpus for map tasks." - ) - elif ray_remote_args.get("num_cpus", 0) > 0: - if ray_remote_args.get("num_gpus", 0) != 0: - raise ValueError( - "It is not allowed to specify both num_cpus and num_gpus for map tasks." - ) + return ray_remote_args diff --git a/python/ray/data/dataset.py b/python/ray/data/dataset.py index 2316afe958b8a..ea0efdca8a14f 100644 --- a/python/ray/data/dataset.py +++ b/python/ray/data/dataset.py @@ -272,6 +272,12 @@ def map( If your transformation is vectorized like most NumPy or pandas operations, :meth:`~Dataset.map_batches` might be faster. + .. warning:: + Specifying both ``num_cpus`` and ``num_gpus`` for map tasks is experimental, + and may result in scheduling or stability issues. Please + `report any issues `_ + to the Ray team. + Examples: .. testcode:: @@ -417,6 +423,12 @@ def map_batches( If ``fn`` doesn't mutate its input, set ``zero_copy_batch=True`` to improve performance and decrease memory utilization. + .. warning:: + Specifying both ``num_cpus`` and ``num_gpus`` for map tasks is experimental, + and may result in scheduling or stability issues. Please + `report any issues `_ + to the Ray team. + Examples: Call :meth:`~Dataset.map_batches` to transform your data. @@ -973,6 +985,12 @@ def flat_map( transformation is vectorized like most NumPy and pandas operations, it might be faster. + .. warning:: + Specifying both ``num_cpus`` and ``num_gpus`` for map tasks is experimental, + and may result in scheduling or stability issues. Please + `report any issues `_ + to the Ray team. + Examples: .. testcode:: diff --git a/python/ray/data/grouped_data.py b/python/ray/data/grouped_data.py index c76d6cee7615f..e479908136a29 100644 --- a/python/ray/data/grouped_data.py +++ b/python/ray/data/grouped_data.py @@ -129,6 +129,12 @@ def map_groups( In general, prefer to use aggregate() instead of map_groups(). + .. warning:: + Specifying both ``num_cpus`` and ``num_gpus`` for map tasks is experimental, + and may result in scheduling or stability issues. Please + `report any issues `_ + to the Ray team. + Examples: >>> # Return a single record per group (list of multiple records in, >>> # list of a single record out). diff --git a/python/ray/data/tests/test_executor_resource_management.py b/python/ray/data/tests/test_executor_resource_management.py index d4d73956d6840..8bf3984ba7ccb 100644 --- a/python/ray/data/tests/test_executor_resource_management.py +++ b/python/ray/data/tests/test_executor_resource_management.py @@ -123,14 +123,18 @@ def test_resource_canonicalization(ray_start_10_cpus_shared): ) assert op._ray_remote_args == {"num_gpus": 2} - with pytest.raises(ValueError): - MapOperator.create( - _mul2_map_data_prcessor, - input_op=input_op, - name="TestMapper", - compute_strategy=TaskPoolStrategy(), - ray_remote_args={"num_gpus": 2, "num_cpus": 1}, - ) + op = MapOperator.create( + _mul2_map_data_prcessor, + input_op=input_op, + name="TestMapper", + compute_strategy=TaskPoolStrategy(), + ray_remote_args={"num_gpus": 2, "num_cpus": 1}, + ) + assert op.base_resource_usage() == ExecutionResources() + assert op.incremental_resource_usage() == ExecutionResources( + cpu=1, gpu=2, object_store_memory=inc_obj_store_mem + ) + assert op._ray_remote_args == {"num_gpus": 2, "num_cpus": 1} def test_execution_options_resource_limit(): From 340b955588566c56b540b39aa9671ed8baac7287 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Tue, 15 Oct 2024 16:19:41 -0700 Subject: [PATCH 29/46] [serve] deflake test_failure::test_replica_actor_died (#48034) Make `test_failure.py::test_replica_actor_died` more stable by waiting until one of the two actors is confirmed to be dead, before sending the next batch of requests. Signed-off-by: Cindy Zhang --- python/ray/serve/tests/test_failure.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/python/ray/serve/tests/test_failure.py b/python/ray/serve/tests/test_failure.py index 9763b591c63e6..3255241abd195 100644 --- a/python/ray/serve/tests/test_failure.py +++ b/python/ray/serve/tests/test_failure.py @@ -13,7 +13,12 @@ from ray.exceptions import RayActorError from ray.serve._private.common import DeploymentID from ray.serve._private.constants import SERVE_DEFAULT_APP_NAME -from ray.serve._private.test_utils import Counter, get_deployment_details, tlog +from ray.serve._private.test_utils import ( + Counter, + check_num_replicas_eq, + get_deployment_details, + tlog, +) def request_with_retries(endpoint, timeout=30): @@ -305,6 +310,8 @@ def check_health(self): tlog(f"Killing replica {replica_to_kill}") ray.kill(ray.get_actor(replica_to_kill, namespace="serve")) + wait_for_condition(check_num_replicas_eq, name="Dummy", target=1) + # The controller just health checked both of them, so it should not # be able to health check and notify the handle router in time. Then # we test that router can properly recognize that the replica has From 35d95a748893789de602bd3bc480a81668706e19 Mon Sep 17 00:00:00 2001 From: Mengjin Yan Date: Tue, 15 Oct 2024 17:11:00 -0700 Subject: [PATCH 30/46] Fix Style Changes Suggested in #47871 (#48030) ## Why are these changes needed? This PR fixed the style suggestions in https://github.com/ray-project/ray/pull/47871. The original PR got accidentally merged before the suggestions are fixed. ## Related issue number ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Mengjin Yan Co-authored-by: angelinalg <122562471+angelinalg@users.noreply.github.com> --- doc/source/cluster/metrics.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doc/source/cluster/metrics.md b/doc/source/cluster/metrics.md index 5aa19c89e5c67..ad9e42cd77dd5 100644 --- a/doc/source/cluster/metrics.md +++ b/doc/source/cluster/metrics.md @@ -24,8 +24,8 @@ For a quick demo, you can run Prometheus locally on your machine. Follow the qui ```{admonition} Note :class: note -If you need to change the root temporary directory by using "--temp-dir" in your ray -cluster setup, please follow the [manual steps](#optional-manual-running-prometheus-locally) to setup Prometheus locally. +If you need to change the root temporary directory by using "--temp-dir" in your Ray +cluster setup, follow these [manual steps](#optional-manual-running-prometheus-locally) to set up Prometheus locally. ``` Run the following command to download and start Prometheus locally with a configuration that scrapes metrics from a local Ray Cluster. @@ -82,7 +82,7 @@ tar xvfz prometheus-*.tar.gz cd prometheus-* ``` -Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. If you specify the `--temp-dir={your_temp_path}` when starting the ray cluster, the config file will be at `{yout_temp_path}/session_latest/metrics/prometheus/prometheus.yml` +Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. If you specify the `--temp-dir={your_temp_path}` when starting the Ray cluster, the config file is at `{your_temp_path}/session_latest/metrics/prometheus/prometheus.yml` ```yaml global: From f80434e078542fb6237622519274b49a718d6276 Mon Sep 17 00:00:00 2001 From: Matthew Owen Date: Tue, 15 Oct 2024 17:12:58 -0700 Subject: [PATCH 31/46] [data] removing unneeded test case (#48031) ## Why are these changes needed? This is a redundant test case with the following two asserts that broke things upstream. This PR removes this assert. ## Related issue number ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [x] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [x] Unit tests - [ ] Release tests - [ ] This PR is not tested :( Signed-off-by: Matthew Owen --- python/ray/data/tests/test_logging.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/ray/data/tests/test_logging.py b/python/ray/data/tests/test_logging.py index ace59dbcda920..152e14315ffb2 100644 --- a/python/ray/data/tests/test_logging.py +++ b/python/ray/data/tests/test_logging.py @@ -136,7 +136,6 @@ def test_json_logging_configuration( # Ensure handlers correctly setup handlers = logger.handlers - assert len(handlers) == 2 assert sum(handler.name == "file_json" for handler in handlers) == 1 assert sum(handler.name == "console" for handler in handlers) == 1 From 00d8c8acf73f70b97dd9325ea2547f32078bd395 Mon Sep 17 00:00:00 2001 From: Chi-Sheng Liu Date: Wed, 16 Oct 2024 22:07:22 +0800 Subject: [PATCH 32/46] [Docs][Minor] RuntimeEnv py_modules example wrong function name (#47788) In https://docs.ray.io/en/latest/ray-core/handling-dependencies.html#library-development ![Screenshot from 2024-09-22 22-07-05](https://github.com/user-attachments/assets/b8c056de-9007-4590-a139-759661c13d9a) See the red rectangle above, the function name is wrong. It should be `test_my_module`. Signed-off-by: Chi-Sheng Liu --- doc/source/ray-core/handling-dependencies.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/source/ray-core/handling-dependencies.rst b/doc/source/ray-core/handling-dependencies.rst index 2bc24462b9a85..8dfc883232fdf 100644 --- a/doc/source/ray-core/handling-dependencies.rst +++ b/doc/source/ray-core/handling-dependencies.rst @@ -327,7 +327,7 @@ To ensure your local changes show up across all Ray workers and can be imported # No need to import my_module inside this function. my_module.test() - ray.get(f.remote()) + ray.get(test_my_module.remote()) .. _runtime-environments-api-ref: From 9df5024246168c38bc5fb086d3bb68cb199df7e5 Mon Sep 17 00:00:00 2001 From: Chi-Sheng Liu Date: Thu, 17 Oct 2024 00:12:00 +0800 Subject: [PATCH 33/46] [Fix][Core] Change grpc_server_req_process_time_ms metric type to histogram (#47932) Signed-off-by: Chi-Sheng Liu Co-authored-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com> --- python/ray/tests/test_metrics_agent.py | 8 ++++++-- src/ray/stats/metric_defs.cc | 4 ++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/python/ray/tests/test_metrics_agent.py b/python/ray/tests/test_metrics_agent.py index adc6a74869271..990e917921b75 100644 --- a/python/ray/tests/test_metrics_agent.py +++ b/python/ray/tests/test_metrics_agent.py @@ -59,7 +59,9 @@ "ray_internal_num_spilled_tasks", # "ray_unintentional_worker_failures_total", # "ray_node_failure_total", - "ray_grpc_server_req_process_time_ms", + "ray_grpc_server_req_process_time_ms_sum", + "ray_grpc_server_req_process_time_ms_bucket", + "ray_grpc_server_req_process_time_ms_count", "ray_grpc_server_req_new_total", "ray_grpc_server_req_handling_total", "ray_grpc_server_req_finished_total", @@ -332,7 +334,9 @@ def test_cases(): # Make sure the gRPC stats are not reported from workers. We disabled # it there because it has too high cardinality. grpc_metrics = [ - "ray_grpc_server_req_process_time_ms", + "ray_grpc_server_req_process_time_ms_sum", + "ray_grpc_server_req_process_time_ms_bucket", + "ray_grpc_server_req_process_time_ms_count", "ray_grpc_server_req_new_total", "ray_grpc_server_req_handling_total", "ray_grpc_server_req_finished_total", diff --git a/src/ray/stats/metric_defs.cc b/src/ray/stats/metric_defs.cc index e12f68cd1bf4a..5d393acdce8d5 100644 --- a/src/ray/stats/metric_defs.cc +++ b/src/ray/stats/metric_defs.cc @@ -161,8 +161,8 @@ DEFINE_stats(operation_active_count, DEFINE_stats(grpc_server_req_process_time_ms, "Request latency in grpc server", ("Method"), - (), - ray::stats::GAUGE); + ({0.1, 1, 10, 100, 1000, 10000}, ), + ray::stats::HISTOGRAM); DEFINE_stats(grpc_server_req_new, "New request number in grpc server", ("Method"), From b49c166e7fa78c0eb2d0badfb27626ad8fabf169 Mon Sep 17 00:00:00 2001 From: Matthew Owen Date: Wed, 16 Oct 2024 11:02:46 -0700 Subject: [PATCH 34/46] [data] Adding in better json checking in test logging (#48036) ## Why are these changes needed? Previously this test simply checked if the first and last characters of a line were `{` or `}` to assert whether or not they were JSON lines. This updates the test to instead attempt to parse them as JSONs which is much more robust. ## Related issue number ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Matthew Owen Signed-off-by: Matthew Owen Co-authored-by: Scott Lee --- python/ray/data/tests/test_logging.py | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/python/ray/data/tests/test_logging.py b/python/ray/data/tests/test_logging.py index 152e14315ffb2..bc6c0ecc0c037 100644 --- a/python/ray/data/tests/test_logging.py +++ b/python/ray/data/tests/test_logging.py @@ -127,6 +127,8 @@ def test_custom_config(reset_logging, monkeypatch, tmp_path): def test_json_logging_configuration( capsys, reset_logging, monkeypatch, shutdown_only, propagate_logs ): + import json + monkeypatch.setenv("RAY_DATA_LOG_ENCODING", "JSON") ray.init() @@ -147,20 +149,19 @@ def test_json_logging_configuration( log_contents = file.read() # Validate the log is in JSON format (a basic check for JSON) - assert all( - log_line.startswith("{") and log_line.endswith("}") - for log_line in log_contents.splitlines() - ) + messages = [] + for log_line in log_contents.splitlines(): + log_dict = json.loads(log_line) # will error if not a json line + messages.append(log_dict["message"]) - assert '"message": "ham"' in log_contents - assert '"message": "turkey"' in log_contents + assert "ham" in messages + assert "turkey" in messages # Validate console logs are in text mode console_log_output = capsys.readouterr().err - assert not any( - log_line.startswith("{") and log_line.endswith("}") - for log_line in console_log_output.splitlines() - ) + for log_line in console_log_output.splitlines(): + with pytest.raises(json.JSONDecodeError): + json.loads(log_line) assert "ham" in console_log_output assert "turkey" not in console_log_output From cd553cd9b98a787270edf07bf8ac9e9a937e4502 Mon Sep 17 00:00:00 2001 From: matthewdeng Date: Wed, 16 Oct 2024 11:04:01 -0700 Subject: [PATCH 35/46] [train][tune] fix WandbLoggerCallback to reuse actors upon restore (#47985) This fixes an issue in the _restoration_ flow of Trials. The `_WandbLoggingActor` actor is cleaned up as part of the trial completion flow, but not appropriately handled during restoration, leading to leaked actors. This fix reuses the same actor across restoration. Signed-off-by: Matthew Deng --- python/ray/air/integrations/wandb.py | 5 +++++ .../ray/air/tests/test_integration_wandb.py | 20 +++++++++++++++++++ 2 files changed, 25 insertions(+) diff --git a/python/ray/air/integrations/wandb.py b/python/ray/air/integrations/wandb.py index fcd683ec0e7f7..2d2d5ffe27296 100644 --- a/python/ray/air/integrations/wandb.py +++ b/python/ray/air/integrations/wandb.py @@ -642,6 +642,11 @@ def log_trial_start(self, trial: "Trial"): def _start_logging_actor( self, trial: "Trial", exclude_results: List[str], **wandb_init_kwargs ): + # Reuse actor if one already exists. + # This can happen if the trial is restarted. + if trial in self._trial_logging_futures: + return + if not self._remote_logger_class: env_vars = {} # API key env variable is not set if authenticating through `wandb login` diff --git a/python/ray/air/tests/test_integration_wandb.py b/python/ray/air/tests/test_integration_wandb.py index abf1576407d79..f2f88aa523e32 100644 --- a/python/ray/air/tests/test_integration_wandb.py +++ b/python/ray/air/tests/test_integration_wandb.py @@ -483,6 +483,26 @@ def _handle_result(self, result): state = ray.get(actor.get_state.remote()) assert [metrics["training_iteration"] for metrics in state.logs] == [4, 5] + def test_wandb_restart(self, trial): + """Test that the WandbLoggerCallback reuses actors for trial restarts.""" + + logger = WandbLoggerCallback(project="test_project", api_key="1234") + logger._logger_actor_cls = _MockWandbLoggingActor + logger.setup() + + assert len(logger._trial_logging_futures) == 0 + assert len(logger._logging_future_to_trial) == 0 + + logger.log_trial_start(trial) + + assert len(logger._trial_logging_futures) == 1 + assert len(logger._logging_future_to_trial) == 1 + + logger.log_trial_start(trial) + + assert len(logger._trial_logging_futures) == 1 + assert len(logger._logging_future_to_trial) == 1 + def test_wandb_logging_process_run_info_hook(monkeypatch): """ From bfed0aa849ec3424afa59eb4b3340d9379487044 Mon Sep 17 00:00:00 2001 From: Scott Lee Date: Wed, 16 Oct 2024 11:09:18 -0700 Subject: [PATCH 36/46] [Data] Allow specifying insertion index when registering custom plan optimization `Rule`s (#48039) ## Why are these changes needed? Currently, when registering custom `LogicalPlan` or `PhysicalPlan` optimization `Rule`s, the only option is to append to the end of the registered rules. The optimization rules will then be executed in order. This is limiting because there are some cases where we want to specify the order of newly registered rules. For example, we may want to add our own custom rule prior to operator fusion taking place. This PR adds the ability to specify an insertion index when registering a custom logical/physical rule. The existing behavior of methods are unchanged. ## Related issue number ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [x] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [x] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Scott Lee --- python/ray/data/_internal/logical/optimizers.py | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/python/ray/data/_internal/logical/optimizers.py b/python/ray/data/_internal/logical/optimizers.py index 8371d70b70fcf..07d029275e3cb 100644 --- a/python/ray/data/_internal/logical/optimizers.py +++ b/python/ray/data/_internal/logical/optimizers.py @@ -1,4 +1,4 @@ -from typing import List, Type +from typing import List, Optional, Type from ray.data._internal.logical.interfaces import ( LogicalPlan, @@ -31,13 +31,19 @@ @DeveloperAPI -def register_logical_rule(cls: Type[Rule]): - _LOGICAL_RULES.append(cls) +def register_logical_rule(cls: Type[Rule], insert_index: Optional[int] = None): + if not insert_index: + _LOGICAL_RULES.append(cls) + else: + _LOGICAL_RULES.insert(insert_index, cls) @DeveloperAPI -def register_physical_rule(cls: Type[Rule]): - _PHYSICAL_RULES.append(cls) +def register_physical_rule(cls: Type[Rule], insert_index: Optional[int] = None): + if not insert_index: + _PHYSICAL_RULES.append(cls) + else: + _PHYSICAL_RULES.insert(insert_index, cls) class LogicalOptimizer(Optimizer): From 79051ff801206dc3008af9e63f893868b11c0ce5 Mon Sep 17 00:00:00 2001 From: Cindy Zhang Date: Wed, 16 Oct 2024 11:25:27 -0700 Subject: [PATCH 37/46] [serve] deflake test gcs failure (#48035) ## Why are these changes needed? Make sure that proxy has been updated with the expected set of replicas before killing the GCS server in `test_new_router_on_gcs_failure` and `test_proxy_router_updated_replicas_then_gcs_failure`. --------- Signed-off-by: Cindy Zhang Co-authored-by: Gene Der Su --- python/ray/serve/_private/proxy.py | 7 ++- python/ray/serve/tests/test_gcs_failure.py | 50 ++++++++++++++++------ 2 files changed, 42 insertions(+), 15 deletions(-) diff --git a/python/ray/serve/_private/proxy.py b/python/ray/serve/_private/proxy.py index d28a10e169cc1..2848c94d73bc0 100644 --- a/python/ray/serve/_private/proxy.py +++ b/python/ray/serve/_private/proxy.py @@ -7,7 +7,7 @@ import time from abc import ABC, abstractmethod from functools import partial -from typing import Any, Callable, Dict, Generator, List, Optional, Tuple, Type +from typing import Any, Callable, Dict, Generator, List, Optional, Set, Tuple, Type import grpc import starlette @@ -27,6 +27,7 @@ DeploymentID, EndpointInfo, NodeId, + ReplicaID, RequestMetadata, RequestProtocol, ) @@ -1293,6 +1294,10 @@ def _get_logging_config(self) -> Tuple: log_file_path = handler.baseFilename return log_file_path + def _dump_ingress_replicas_for_testing(self, route: str) -> Set[ReplicaID]: + _, handle, _ = self.http_proxy.proxy_router.match_route(route) + return handle._router._replica_scheduler._replica_id_set + def should_start_grpc_service(self) -> bool: """Determine whether gRPC service should be started. diff --git a/python/ray/serve/tests/test_gcs_failure.py b/python/ray/serve/tests/test_gcs_failure.py index 6bcacd239abe5..437722f4b3bde 100644 --- a/python/ray/serve/tests/test_gcs_failure.py +++ b/python/ray/serve/tests/test_gcs_failure.py @@ -1,6 +1,7 @@ import importlib import os import sys +from typing import Callable, Optional import pytest import requests @@ -8,10 +9,9 @@ import ray from ray import serve from ray._private.test_utils import wait_for_condition -from ray.serve._private.common import DeploymentID, ReplicaState from ray.serve._private.constants import SERVE_DEFAULT_APP_NAME from ray.serve._private.storage.kv_store import KVStoreError, RayInternalKVStore -from ray.serve._private.test_utils import check_apps_running, check_replica_counts +from ray.serve._private.test_utils import check_apps_running from ray.serve.context import _get_global_client from ray.serve.handle import DeploymentHandle from ray.serve.schema import ServeDeploySchema @@ -112,11 +112,20 @@ def call(): def router_populated_with_replicas( - handle: DeploymentHandle, threshold: int, + handle: Optional[DeploymentHandle] = None, + get_replicas_func: Optional[Callable] = None, check_cache_populated: bool = False, ): - replicas = handle._router._replica_scheduler._replica_id_set + """Either get router's replica set from `handle` directly, or use + `get_replicas_func` to get replica set. Then check that the number + of replicas in set is at least `threshold`. + """ + if handle: + replicas = handle._router._replica_scheduler._replica_id_set + else: + replicas = get_replicas_func() + assert len(replicas) >= threshold # Return early if we don't need to check cache @@ -142,6 +151,8 @@ def test_new_router_on_gcs_failure(serve_ha, use_proxy: bool): sent to replicas during GCS downtime. """ + _, client = serve_ha + @serve.deployment class Dummy: def __call__(self): @@ -161,7 +172,18 @@ def __call__(self): # waiting for the first request h._get_or_create_router() - wait_for_condition(router_populated_with_replicas, handle=h, threshold=1) + if use_proxy: + proxy_handles = ray.get(client._controller.get_proxies.remote()) + proxy_handle = list(proxy_handles.values())[0] + wait_for_condition( + router_populated_with_replicas, + threshold=2, + get_replicas_func=lambda: ray.get( + proxy_handle._dump_ingress_replicas_for_testing.remote("/") + ), + ) + else: + wait_for_condition(router_populated_with_replicas, threshold=2, handle=h) # Kill GCS server before a single request is sent. ray.worker._global_node.kill_gcs_server() @@ -208,8 +230,8 @@ def test_handle_router_updated_replicas_then_gcs_failure(serve_ha): wait_for_condition( router_populated_with_replicas, - handle=h, threshold=2, + handle=h, check_cache_populated=True, ) @@ -250,15 +272,15 @@ def test_proxy_router_updated_replicas_then_gcs_failure(serve_ha): config["deployments"][0]["num_replicas"] = 2 client.deploy_apps(ServeDeploySchema(**{"applications": [config]})) - # There is no way to directly check if proxy has received updated replicas, - # so just check for the status. After controller updates status with new - # replicas, proxy should instantly receive updates from long poll + proxy_handles = ray.get(client._controller.get_proxies.remote()) + proxy_handle = list(proxy_handles.values())[0] + wait_for_condition( - check_replica_counts, - controller=client._controller, - deployment_id=DeploymentID("GetPID", "default"), - total=2, - by_state=[(ReplicaState.RUNNING, 2, None)], + router_populated_with_replicas, + threshold=2, + get_replicas_func=lambda: ray.get( + proxy_handle._dump_ingress_replicas_for_testing.remote("/") + ), ) # Kill GCS server before router gets to send request to second replica From 37e244977a52a22afd94bcc1187088fc73c48ffa Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Wed, 16 Oct 2024 11:59:59 -0700 Subject: [PATCH 38/46] Revert "Fix Style Changes Suggested in #47871" (#48049) Reverts ray-project/ray#48030 --- doc/source/cluster/metrics.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doc/source/cluster/metrics.md b/doc/source/cluster/metrics.md index ad9e42cd77dd5..5aa19c89e5c67 100644 --- a/doc/source/cluster/metrics.md +++ b/doc/source/cluster/metrics.md @@ -24,8 +24,8 @@ For a quick demo, you can run Prometheus locally on your machine. Follow the qui ```{admonition} Note :class: note -If you need to change the root temporary directory by using "--temp-dir" in your Ray -cluster setup, follow these [manual steps](#optional-manual-running-prometheus-locally) to set up Prometheus locally. +If you need to change the root temporary directory by using "--temp-dir" in your ray +cluster setup, please follow the [manual steps](#optional-manual-running-prometheus-locally) to setup Prometheus locally. ``` Run the following command to download and start Prometheus locally with a configuration that scrapes metrics from a local Ray Cluster. @@ -82,7 +82,7 @@ tar xvfz prometheus-*.tar.gz cd prometheus-* ``` -Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. If you specify the `--temp-dir={your_temp_path}` when starting the Ray cluster, the config file is at `{your_temp_path}/session_latest/metrics/prometheus/prometheus.yml` +Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. If you specify the `--temp-dir={your_temp_path}` when starting the ray cluster, the config file will be at `{yout_temp_path}/session_latest/metrics/prometheus/prometheus.yml` ```yaml global: From 8bd6a9a82f900587e7a8870ac0ccc88a8b07ff14 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen <128072568+can-anyscale@users.noreply.github.com> Date: Wed, 16 Oct 2024 12:00:28 -0700 Subject: [PATCH 39/46] Revert "[Ray Core] Fix the Paths in the Generated Monitoring Configs to Consider temp-dir" (#48050) Reverts ray-project/ray#47871 --- doc/source/cluster/metrics.md | 14 +-- .../metrics/export/grafana/grafana.ini | 10 ++ .../metrics/grafana_dashboard_factory.py | 3 + ...grafana_dashboard_provisioning_template.py | 10 ++ .../metrics/grafana_datasource_template.py | 9 ++ .../metrics/install_and_start_prometheus.py | 13 ++- .../dashboard/modules/metrics/metrics_head.py | 91 ++++++------------- .../dashboard/modules/metrics/templates.py | 49 ---------- .../modules/tests/test_metrics_integration.py | 15 --- python/ray/tests/test_metrics_head.py | 30 +----- 10 files changed, 71 insertions(+), 173 deletions(-) create mode 100644 python/ray/dashboard/modules/metrics/export/grafana/grafana.ini create mode 100644 python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py create mode 100644 python/ray/dashboard/modules/metrics/grafana_datasource_template.py delete mode 100644 python/ray/dashboard/modules/metrics/templates.py diff --git a/doc/source/cluster/metrics.md b/doc/source/cluster/metrics.md index 5aa19c89e5c67..6ef27ca5af76d 100644 --- a/doc/source/cluster/metrics.md +++ b/doc/source/cluster/metrics.md @@ -22,12 +22,6 @@ For a quick demo, you can run Prometheus locally on your machine. Follow the qui ### Quickstart: Running Prometheus locally -```{admonition} Note -:class: note -If you need to change the root temporary directory by using "--temp-dir" in your ray -cluster setup, please follow the [manual steps](#optional-manual-running-prometheus-locally) to setup Prometheus locally. -``` - Run the following command to download and start Prometheus locally with a configuration that scrapes metrics from a local Ray Cluster. ```bash @@ -82,7 +76,7 @@ tar xvfz prometheus-*.tar.gz cd prometheus-* ``` -Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. If you specify the `--temp-dir={your_temp_path}` when starting the ray cluster, the config file will be at `{yout_temp_path}/session_latest/metrics/prometheus/prometheus.yml` +Ray provides a Prometheus config that works out of the box. After running Ray, you can find the config at `/tmp/ray/session_latest/metrics/prometheus/prometheus.yml`. ```yaml global: @@ -94,17 +88,13 @@ scrape_configs: - job_name: 'ray' file_sd_configs: - files: - - '/tmp/ray/prom_metrics_service_discovery.json' # or '${your_temp_path}/prom_metrics_service_discovery.json' if --temp-dir is specified + - '/tmp/ray/prom_metrics_service_discovery.json' ``` Next, start Prometheus: ```shell -# With default settings ./prometheus --config.file=/tmp/ray/session_latest/metrics/prometheus/prometheus.yml - -# With specified --temp-dir -./prometheus --config.file={your_temp_path}/session_latest/metrics/prometheus/prometheus.yml ``` ```{admonition} Note :class: note diff --git a/python/ray/dashboard/modules/metrics/export/grafana/grafana.ini b/python/ray/dashboard/modules/metrics/export/grafana/grafana.ini new file mode 100644 index 0000000000000..e349fc5b29076 --- /dev/null +++ b/python/ray/dashboard/modules/metrics/export/grafana/grafana.ini @@ -0,0 +1,10 @@ +[security] +allow_embedding = true + +[auth.anonymous] +enabled = true +org_name = Main Org. +org_role = Viewer + +[paths] +provisioning = /tmp/ray/session_latest/metrics/grafana/provisioning diff --git a/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py b/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py index 8d328c7084012..527928fb4e92c 100644 --- a/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py +++ b/python/ray/dashboard/modules/metrics/grafana_dashboard_factory.py @@ -19,6 +19,9 @@ serve_deployment_dashboard_config, ) +METRICS_INPUT_ROOT = os.path.join(os.path.dirname(__file__), "export") +GRAFANA_CONFIG_INPUT_PATH = os.path.join(METRICS_INPUT_ROOT, "grafana") + GRAFANA_DASHBOARD_UID_OVERRIDE_ENV_VAR_TEMPLATE = "RAY_GRAFANA_{name}_DASHBOARD_UID" GRAFANA_DASHBOARD_GLOBAL_FILTERS_OVERRIDE_ENV_VAR_TEMPLATE = ( "RAY_GRAFANA_{name}_DASHBOARD_GLOBAL_FILTERS" diff --git a/python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py b/python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py new file mode 100644 index 0000000000000..254da09efad7b --- /dev/null +++ b/python/ray/dashboard/modules/metrics/grafana_dashboard_provisioning_template.py @@ -0,0 +1,10 @@ +DASHBOARD_PROVISIONING_TEMPLATE = """ +apiVersion: 1 + +providers: + - name: Ray # Default dashboards provided by OSS ray + folder: Ray + type: file + options: + path: {dashboard_output_folder} +""" diff --git a/python/ray/dashboard/modules/metrics/grafana_datasource_template.py b/python/ray/dashboard/modules/metrics/grafana_datasource_template.py new file mode 100644 index 0000000000000..01fc06bc3be9e --- /dev/null +++ b/python/ray/dashboard/modules/metrics/grafana_datasource_template.py @@ -0,0 +1,9 @@ +GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1 + +datasources: + - name: {prometheus_name} + url: {prometheus_host} + type: prometheus + isDefault: true + access: proxy +""" diff --git a/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py b/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py index 541c9dfd6b52c..f060a98d6d114 100644 --- a/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py +++ b/python/ray/dashboard/modules/metrics/install_and_start_prometheus.py @@ -74,13 +74,12 @@ def install_prometheus(file_path): def start_prometheus(prometheus_dir): - # The function assumes the Ray cluster to be monitored by Prometheus uses the - # default configuration with "/tmp/ray" as the default root temporary directory. - # - # This is to support the `ray metrics launch-prometheus` command, when a ray cluster - # is not started yet and we have no way to get a `--temp-dir` anywhere. So we choose - # to use a hardcoded default value. - + # Currently, Ray never modifies this config file, so we can just use the + # hardcoded path. (It just copies it to a more user-friendly location, in + # MetricsHead._create_default_prometheus_configs.) + # However, if in the future Ray ever modifies this file at runtime, we'll + # need to use the user-friendly location instead, and reload the config + # file after it's updated by Ray. config_file = Path(PROMETHEUS_CONFIG_INPUT_PATH) if not config_file.exists(): diff --git a/python/ray/dashboard/modules/metrics/metrics_head.py b/python/ray/dashboard/modules/metrics/metrics_head.py index ae0d38e73b0a9..04c80255fecb2 100644 --- a/python/ray/dashboard/modules/metrics/metrics_head.py +++ b/python/ray/dashboard/modules/metrics/metrics_head.py @@ -11,24 +11,24 @@ import ray.dashboard.optional_utils as dashboard_optional_utils import ray.dashboard.utils as dashboard_utils from ray._private.async_utils import enable_monitor_loop_lag -from ray._private.ray_constants import ( - PROMETHEUS_SERVICE_DISCOVERY_FILE, - SESSION_LATEST, - env_integer, -) +from ray._private.ray_constants import env_integer from ray._private.utils import get_or_create_event_loop -from ray.dashboard.consts import AVAILABLE_COMPONENT_NAMES_FOR_METRICS +from ray.dashboard.consts import ( + AVAILABLE_COMPONENT_NAMES_FOR_METRICS, + METRICS_INPUT_ROOT, + PROMETHEUS_CONFIG_INPUT_PATH, +) from ray.dashboard.modules.metrics.grafana_dashboard_factory import ( generate_data_grafana_dashboard, generate_default_grafana_dashboard, generate_serve_deployment_grafana_dashboard, generate_serve_grafana_dashboard, ) -from ray.dashboard.modules.metrics.templates import ( +from ray.dashboard.modules.metrics.grafana_dashboard_provisioning_template import ( DASHBOARD_PROVISIONING_TEMPLATE, +) +from ray.dashboard.modules.metrics.grafana_datasource_template import ( GRAFANA_DATASOURCE_TEMPLATE, - GRAFANA_INI_TEMPLATE, - PROMETHEUS_YML_TEMPLATE, ) import psutil @@ -54,6 +54,7 @@ GRAFANA_HOST_DISABLED_VALUE = "DISABLED" GRAFANA_IFRAME_HOST_ENV_VAR = "RAY_GRAFANA_IFRAME_HOST" GRAFANA_DASHBOARD_OUTPUT_DIR_ENV_VAR = "RAY_METRICS_GRAFANA_DASHBOARD_OUTPUT_DIR" +GRAFANA_CONFIG_INPUT_PATH = os.path.join(METRICS_INPUT_ROOT, "grafana") GRAFANA_HEALTHCHECK_PATH = "api/health" @@ -74,22 +75,13 @@ def __init__(self, dashboard_head): PROMETHEUS_HOST_ENV_VAR, DEFAULT_PROMETHEUS_HOST ) default_metrics_root = os.path.join(self._dashboard_head.session_dir, "metrics") - session_latest_metrics_root = os.path.join( - self._dashboard_head.temp_dir, SESSION_LATEST, "metrics" - ) self._metrics_root = os.environ.get( METRICS_OUTPUT_ROOT_ENV_VAR, default_metrics_root ) - self._metrics_root_session_latest = os.environ.get( - METRICS_OUTPUT_ROOT_ENV_VAR, session_latest_metrics_root - ) - self._grafana_config_output_path = os.path.join(self._metrics_root, "grafana") - self._grafana_session_latest_config_output_path = os.path.join( - self._metrics_root_session_latest, "grafana" - ) + grafana_config_output_path = os.path.join(self._metrics_root, "grafana") self._grafana_dashboard_output_dir = os.environ.get( GRAFANA_DASHBOARD_OUTPUT_DIR_ENV_VAR, - os.path.join(self._grafana_config_output_path, "dashboards"), + os.path.join(grafana_config_output_path, "dashboards"), ) self._prometheus_name = os.environ.get( @@ -194,34 +186,17 @@ def _create_default_grafana_configs(self): """ Creates the grafana configurations that are by default provided by Ray. """ - # Create grafana configuration folder - if os.path.exists(self._grafana_config_output_path): - shutil.rmtree(self._grafana_config_output_path) - os.makedirs(self._grafana_config_output_path, exist_ok=True) - - # Overwrite grafana's configuration file - grafana_provisioning_folder = os.path.join( - self._grafana_config_output_path, "provisioning" - ) - grafana_prov_folder_with_latest_session = os.path.join( - self._grafana_session_latest_config_output_path, "provisioning" - ) - with open( - os.path.join( - self._grafana_config_output_path, - "grafana.ini", - ), - "w", - ) as f: - f.write( - GRAFANA_INI_TEMPLATE.format( - grafana_provisioning_folder=grafana_prov_folder_with_latest_session - ) - ) + grafana_config_output_path = os.path.join(self._metrics_root, "grafana") + + # Copy default grafana configurations + if os.path.exists(grafana_config_output_path): + shutil.rmtree(grafana_config_output_path) + os.makedirs(os.path.dirname(grafana_config_output_path), exist_ok=True) + shutil.copytree(GRAFANA_CONFIG_INPUT_PATH, grafana_config_output_path) # Overwrite grafana's dashboard provisioning directory based on env var dashboard_provisioning_path = os.path.join( - grafana_provisioning_folder, "dashboards" + grafana_config_output_path, "provisioning", "dashboards" ) os.makedirs( dashboard_provisioning_path, @@ -244,7 +219,9 @@ def _create_default_grafana_configs(self): prometheus_host = os.environ.get( PROMETHEUS_HOST_ENV_VAR, DEFAULT_PROMETHEUS_HOST ) - data_sources_path = os.path.join(grafana_provisioning_folder, "datasources") + data_sources_path = os.path.join( + grafana_config_output_path, "provisioning", "datasources" + ) os.makedirs( data_sources_path, exist_ok=True, @@ -320,26 +297,14 @@ def _create_default_prometheus_configs(self): self._metrics_root, "prometheus", "prometheus.yml" ) - # Generate the default prometheus configurations + # Copy default prometheus configurations if os.path.exists(prometheus_config_output_path): os.remove(prometheus_config_output_path) os.makedirs(os.path.dirname(prometheus_config_output_path), exist_ok=True) - - # This code generates the Prometheus config based on the custom temporary root - # path set by the user at Ray cluster start up (via --temp-dir). In contrast, - # start_prometheus in install_and_start_prometheus.py uses a hardcoded - # Prometheus config at PROMETHEUS_CONFIG_INPUT_PATH that always uses "/tmp/ray". - # Other than the root path, the config file generated here is identical to that - # hardcoded config file. - prom_discovery_file_path = os.path.join( - self._dashboard_head.temp_dir, PROMETHEUS_SERVICE_DISCOVERY_FILE - ) - with open(prometheus_config_output_path, "w") as f: - f.write( - PROMETHEUS_YML_TEMPLATE.format( - prom_metrics_service_discovery_file_path=prom_discovery_file_path - ) - ) + # Currently Ray directly copies this file without modifying it at runtime. + # If Ray ever modifies this file at runtime, please ensure start_prometheus + # in install_and_start_prometheus.py is updated to reload the config file. + shutil.copy(PROMETHEUS_CONFIG_INPUT_PATH, prometheus_config_output_path) @dashboard_utils.async_loop_forever(METRICS_RECORD_INTERVAL_S) async def record_dashboard_metrics(self): diff --git a/python/ray/dashboard/modules/metrics/templates.py b/python/ray/dashboard/modules/metrics/templates.py deleted file mode 100644 index d78b570ff7bd5..0000000000000 --- a/python/ray/dashboard/modules/metrics/templates.py +++ /dev/null @@ -1,49 +0,0 @@ -GRAFANA_INI_TEMPLATE = """ -[security] -allow_embedding = true - -[auth.anonymous] -enabled = true -org_name = Main Org. -org_role = Viewer - -[paths] -provisioning = {grafana_provisioning_folder} -""" - -DASHBOARD_PROVISIONING_TEMPLATE = """ -apiVersion: 1 - -providers: - - name: Ray # Default dashboards provided by OSS ray - folder: Ray - type: file - options: - path: {dashboard_output_folder} -""" - -GRAFANA_DATASOURCE_TEMPLATE = """apiVersion: 1 - -datasources: - - name: {prometheus_name} - url: {prometheus_host} - type: prometheus - isDefault: true - access: proxy -""" - -PROMETHEUS_YML_TEMPLATE = """# my global config -global: - scrape_interval: 10s # Set the scrape interval to every 10 seconds. Default is every \ -1 minute. - evaluation_interval: 10s # Evaluate rules every 10 seconds. The default is every 1 \ -minute. - # scrape_timeout is set to the global default (10s). - -scrape_configs: -# Scrape from each ray node as defined in the service_discovery.json provided by ray. -- job_name: 'ray' - file_sd_configs: - - files: - - '{prom_metrics_service_discovery_file_path}' -""" diff --git a/python/ray/dashboard/modules/tests/test_metrics_integration.py b/python/ray/dashboard/modules/tests/test_metrics_integration.py index 0d0ef13cdf068..71f8630d1abc8 100644 --- a/python/ray/dashboard/modules/tests/test_metrics_integration.py +++ b/python/ray/dashboard/modules/tests/test_metrics_integration.py @@ -3,9 +3,7 @@ import pytest -from ray.dashboard.consts import PROMETHEUS_CONFIG_INPUT_PATH from ray.dashboard.modules.metrics import install_and_start_prometheus -from ray.dashboard.modules.metrics.templates import PROMETHEUS_YML_TEMPLATE @pytest.mark.parametrize( @@ -40,18 +38,5 @@ def test_e2e(capsys): subprocess.run(["kill", str(pid)]) -def test_prometheus_config_content(): - # Test to make sure the content in the hardcoded file - # (python/ray/dashboard/modules/metrics/export/prometheus/prometheus.yml) will - # always be the same as the template (templates.py) used to generate prometheus - # config file when ray startup - PROM_DISCOVERY_FILE_PATH = "/tmp/ray/prom_metrics_service_discovery.json" - template_content = PROMETHEUS_YML_TEMPLATE.format( - prom_metrics_service_discovery_file_path=PROM_DISCOVERY_FILE_PATH - ) - with open(PROMETHEUS_CONFIG_INPUT_PATH) as f: - assert f.read() == template_content - - if __name__ == "__main__": sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/tests/test_metrics_head.py b/python/ray/tests/test_metrics_head.py index d6010d88b3925..2e358ce208f3c 100644 --- a/python/ray/tests/test_metrics_head.py +++ b/python/ray/tests/test_metrics_head.py @@ -13,35 +13,20 @@ SERVE_GRAFANA_PANELS, ) from ray.tests.conftest import _ray_start -from ray._private.ray_constants import SESSION_LATEST logger = logging.getLogger(__name__) -@pytest.mark.parametrize( - "is_temp_dir_set, temp_dir_val", [(False, ""), (True, "/tmp/test-temp-dir")] -) -def test_metrics_folder_and_content(is_temp_dir_set, temp_dir_val): +def test_metrics_folder(): """ - Tests that the default dashboard files get created. It also verifies paths in the - dashboard config files are set correctly. - - It checks both the default case and the case where the _temp_dir is specified. + Tests that the default dashboard files get created. """ - with _ray_start( - include_dashboard=True, _temp_dir=temp_dir_val if is_temp_dir_set else None - ) as context: + with _ray_start(include_dashboard=True) as context: session_dir = context["session_dir"] - temp_dir = temp_dir_val if is_temp_dir_set else "/tmp/ray" assert os.path.exists( f"{session_dir}/metrics/grafana/provisioning/dashboards/default.yml" ) - with open( - f"{session_dir}/metrics/grafana/provisioning/dashboards/default.yml", "r" - ) as f: - assert f"path: {session_dir}/metrics/grafana/dashboards" in f.read() - assert os.path.exists( f"{session_dir}/metrics/grafana/dashboards" "/default_grafana_dashboard.json" @@ -49,17 +34,8 @@ def test_metrics_folder_and_content(is_temp_dir_set, temp_dir_val): assert os.path.exists( f"{session_dir}/metrics/grafana/provisioning/datasources/default.yml" ) - assert os.path.exists(f"{session_dir}/metrics/grafana/grafana.ini") - with open(f"{session_dir}/metrics/grafana/grafana.ini", "r") as f: - assert ( - "provisioning = " - f"{temp_dir}/{SESSION_LATEST}/metrics/grafana/provisioning" in f.read() - ) - assert os.path.exists(f"{session_dir}/metrics/prometheus/prometheus.yml") - with open(f"{session_dir}/metrics/prometheus/prometheus.yml", "r") as f: - assert f"- '{temp_dir}/prom_metrics_service_discovery.json'" in f.read() @pytest.fixture From 4dfa033fd94401fd523f31028656ab23cf80d172 Mon Sep 17 00:00:00 2001 From: Gene Der Su Date: Wed, 16 Oct 2024 14:09:34 -0700 Subject: [PATCH 40/46] [Serve] refactor `add_grpc_address` helper (#48033) ## Why are these changes needed? Follow up on https://github.com/ray-project/ray/pull/47720 to move the helper into default_impl.py ## Related issue number ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Gene Su --- python/ray/serve/_private/default_impl.py | 6 ++++++ python/ray/serve/_private/grpc_util.py | 5 ----- python/ray/serve/_private/proxy.py | 7 ++----- python/ray/serve/tests/unit/test_grpc_util.py | 2 +- 4 files changed, 9 insertions(+), 11 deletions(-) diff --git a/python/ray/serve/_private/default_impl.py b/python/ray/serve/_private/default_impl.py index 71eb9296eb36f..6ff1b83056f39 100644 --- a/python/ray/serve/_private/default_impl.py +++ b/python/ray/serve/_private/default_impl.py @@ -10,6 +10,7 @@ DefaultDeploymentScheduler, DeploymentScheduler, ) +from ray.serve._private.grpc_util import gRPCServer from ray.serve._private.utils import get_head_node_id # NOTE: Please read carefully before changing! @@ -35,3 +36,8 @@ def create_deployment_scheduler( create_placement_group_fn=create_placement_group_fn_override or ray.util.placement_group, ) + + +def add_grpc_address(grpc_server: gRPCServer, server_address: str): + """Helper function to add a address to gRPC server.""" + grpc_server.add_insecure_port(server_address) diff --git a/python/ray/serve/_private/grpc_util.py b/python/ray/serve/_private/grpc_util.py index d522f1e7a9f3b..095cd87ee14b6 100644 --- a/python/ray/serve/_private/grpc_util.py +++ b/python/ray/serve/_private/grpc_util.py @@ -81,8 +81,3 @@ class DummyServicer: def __getattr__(self, attr): # No-op pass through. Just need this to act as the callable. pass - - -def add_grpc_address(grpc_server: gRPCServer, server_address: str): - """Helper function to add a address to gRPC server.""" - grpc_server.add_insecure_port(server_address) diff --git a/python/ray/serve/_private/proxy.py b/python/ray/serve/_private/proxy.py index 2848c94d73bc0..ee0e048fef5be 100644 --- a/python/ray/serve/_private/proxy.py +++ b/python/ray/serve/_private/proxy.py @@ -41,11 +41,8 @@ SERVE_MULTIPLEXED_MODEL_ID, SERVE_NAMESPACE, ) -from ray.serve._private.grpc_util import ( - DummyServicer, - add_grpc_address, - create_serve_grpc_server, -) +from ray.serve._private.default_impl import add_grpc_address +from ray.serve._private.grpc_util import DummyServicer, create_serve_grpc_server from ray.serve._private.http_util import ( MessageQueue, convert_object_to_asgi_messages, diff --git a/python/ray/serve/tests/unit/test_grpc_util.py b/python/ray/serve/tests/unit/test_grpc_util.py index c28fcd3e8e5dc..6270ecac31a7c 100644 --- a/python/ray/serve/tests/unit/test_grpc_util.py +++ b/python/ray/serve/tests/unit/test_grpc_util.py @@ -6,9 +6,9 @@ from google.protobuf.any_pb2 import Any as AnyProto from ray import cloudpickle +from ray.serve._private.default_impl import add_grpc_address from ray.serve._private.grpc_util import ( DummyServicer, - add_grpc_address, create_serve_grpc_server, gRPCServer, ) From 0056097f14dec3d3f5ad4c9fbaff216bcfd7b369 Mon Sep 17 00:00:00 2001 From: Chi-Sheng Liu Date: Thu, 17 Oct 2024 05:27:26 +0800 Subject: [PATCH 41/46] [Feat][Core] Support ExceptionGroup (#47887) See the description in the corresponding issue for details. The issue states that we should support `BaseExceptionGroup`, but I found that we don't support `BaseException` either, so it doesn't make sense to support `BaseExceptionGroup`. For `ExceptionGroup`, we need to override its `__new__` method as well. See https://docs.python.org/3/library/exceptions.html#ExceptionGroup for details. Signed-off-by: Chi-Sheng Liu Co-authored-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com> --- python/ray/exceptions.py | 52 ++++++- python/ray/tests/BUILD | 1 + python/ray/tests/test_exceptiongroup.py | 196 ++++++++++++++++++++++++ python/ray/tests/test_failure.py | 20 +++ 4 files changed, 263 insertions(+), 6 deletions(-) create mode 100644 python/ray/tests/test_exceptiongroup.py diff --git a/python/ray/exceptions.py b/python/ray/exceptions.py index d69cde6a283dc..33bb7a225c79e 100644 --- a/python/ray/exceptions.py +++ b/python/ray/exceptions.py @@ -1,7 +1,8 @@ import logging import os +import sys from traceback import format_exception -from typing import Optional, Type, Union +from typing import Optional, Union import colorama @@ -147,9 +148,21 @@ def __init__( assert traceback_str is not None - def make_dual_exception_type(self) -> Type: - """Makes a Type that inherits from both RayTaskError and the type of + def make_dual_exception_instance(self) -> "RayTaskError": + """Makes a object instance that inherits from both RayTaskError and the type of `self.cause`. Raises TypeError if the cause class can't be subclassed""" + # For normal user Exceptions, we subclass from both + # RayTaskError and the user exception. For ExceptionGroup, + # we special handle it because it has a different __new__() + # signature from Exception. + # Ref: https://docs.python.org/3/library/exceptions.html#exception-groups + if sys.version_info >= (3, 11) and isinstance( + self.cause, ExceptionGroup # noqa: F821 + ): + return self._make_exceptiongroup_dual_exception_instance() + return self._make_normal_dual_exception_instance() + + def _make_normal_dual_exception_instance(self) -> "RayTaskError": cause_cls = self.cause.__class__ error_msg = str(self) @@ -171,7 +184,35 @@ def __str__(self): cls.__name__ = name cls.__qualname__ = name - return cls + return cls(self.cause) + + def _make_exceptiongroup_dual_exception_instance(self) -> "RayTaskError": + cause_cls = self.cause.__class__ + error_msg = str(self) + + class cls(RayTaskError, cause_cls): + def __new__(cls, cause): + self = super().__new__(cls, cause.message, cause.exceptions) + return self + + def __init__(self, cause): + self.cause = cause + # BaseException implements a __reduce__ method that returns + # a tuple with the type and the value of self.args. + # https://stackoverflow.com/a/49715949/2213289 + self.args = (cause,) + + def __getattr__(self, name): + return getattr(self.cause, name) + + def __str__(self): + return error_msg + + name = f"RayTaskError({cause_cls.__name__})" + cls.__name__ = name + cls.__qualname__ = name + + return cls(self.cause) def as_instanceof_cause(self): """Returns an exception that's an instance of the cause's class. @@ -187,8 +228,7 @@ def as_instanceof_cause(self): return self # already satisfied try: - dual_cls = self.make_dual_exception_type() - return dual_cls(self.cause) + return self.make_dual_exception_instance() except TypeError as e: logger.warning( f"User exception type {type(self.cause)} in RayTaskError can't" diff --git a/python/ray/tests/BUILD b/python/ray/tests/BUILD index 06641d0d57776..7b3be66ef2373 100644 --- a/python/ray/tests/BUILD +++ b/python/ray/tests/BUILD @@ -291,6 +291,7 @@ py_test_module_list( "test_debug_tools.py", "test_distributed_sort.py", "test_environ.py", + "test_exceptiongroup.py", "test_get_or_create_actor.py", "test_ids.py", "test_list_actors.py", diff --git a/python/ray/tests/test_exceptiongroup.py b/python/ray/tests/test_exceptiongroup.py new file mode 100644 index 0000000000000..e8516a0a20574 --- /dev/null +++ b/python/ray/tests/test_exceptiongroup.py @@ -0,0 +1,196 @@ +import os +import sys +from textwrap import dedent + +import pytest + +import ray +from ray.exceptions import RayTaskError + +pytestmark = pytest.mark.skipif( + sys.version_info < (3, 11), + reason="ExceptionGroup is only available in Python 3.11+", +) + + +def test_baseexceptiongroup_task(ray_start_regular): + baseexceptiongroup = BaseExceptionGroup( # noqa: F821 + "test baseexceptiongroup", [BaseException("abc")] + ) + + @ray.remote + def task(): + raise baseexceptiongroup + + with pytest.raises(ray.exceptions.WorkerCrashedError): + ray.get(task.remote()) + + +def test_baseexceptiongroup_actor(ray_start_regular): + baseexceptiongroup = BaseExceptionGroup( # noqa: F821 + "test baseexceptiongroup", [BaseException("abc")] + ) + + @ray.remote + class Actor: + def f(self): + raise baseexceptiongroup + + with pytest.raises(ray.exceptions.ActorDiedError): + a = Actor.remote() + ray.get(a.f.remote()) + + +def test_except_exceptiongroup(ray_start_regular): + exceptiongroup = ExceptionGroup( # noqa: F821 + "test exceptiongroup", [ValueError(), TypeError()] + ) + + @ray.remote + def task(): + raise exceptiongroup + + @ray.remote + class Actor: + def f(self): + raise exceptiongroup + + try: + ray.get(task.remote()) + except Exception as ex: + assert isinstance(ex, RayTaskError) + assert isinstance(ex, ExceptionGroup) # noqa: F821 + assert len(ex.exceptions) == 2 + assert isinstance(ex.exceptions[0], ValueError) + assert isinstance(ex.exceptions[1], TypeError) + + try: + a = Actor.remote() + ray.get(a.f.remote()) + except Exception as ex: + assert isinstance(ex, RayTaskError) + assert isinstance(ex, ExceptionGroup) # noqa: F821 + assert len(ex.exceptions) == 2 + assert isinstance(ex.exceptions[0], ValueError) + assert isinstance(ex.exceptions[1], TypeError) + + +def test_except_star_exception(ray_start_regular): + @ray.remote + def task(): + raise ValueError + + @ray.remote + class Actor: + def f(self): + raise ValueError + + # TODO: Don't use exec() when we only support Python 3.11+ + # Here the exec() is used to avoid SyntaxError for except* for Python < 3.11 + python_code = dedent( + """\ + try: + ray.get(task.remote()) + except* RayTaskError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], RayTaskError) + assert isinstance(ex.exceptions[0], ValueError) + + try: + ray.get(task.remote()) + except* ValueError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], RayTaskError) + assert isinstance(ex.exceptions[0], ValueError) + + try: + a = Actor.remote() + ray.get(a.f.remote()) + except* RayTaskError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], RayTaskError) + assert isinstance(ex.exceptions[0], ValueError) + + try: + a = Actor.remote() + ray.get(a.f.remote()) + except* ValueError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], RayTaskError) + assert isinstance(ex.exceptions[0], ValueError) + """ + ) + exec(python_code) + + +def test_except_star_exceptiongroup(ray_start_regular): + exceptiongroup = ExceptionGroup( # noqa: F821 + "test exceptiongroup", [ValueError(), TypeError()] + ) + + @ray.remote + def task(): + raise exceptiongroup + + @ray.remote + class Actor: + def f(self): + raise exceptiongroup + + # TODO: Don't use exec() when we only support Python 3.11+ + # Here the exec() is used to avoid SyntaxError for except* for Python < 3.11 + python_code = dedent( + """\ + try: + ray.get(task.remote()) + except* RayTaskError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 2 + assert isinstance(ex.exceptions[0], ValueError) + assert isinstance(ex.exceptions[1], TypeError) + + try: + ray.get(task.remote()) + except* ValueError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], ValueError) + except* TypeError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], TypeError) + + try: + a = Actor.remote() + ray.get(a.f.remote()) + except* RayTaskError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 2 + assert isinstance(ex.exceptions[0], ValueError) + assert isinstance(ex.exceptions[1], TypeError) + + try: + a = Actor.remote() + ray.get(a.f.remote()) + except* ValueError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], ValueError) + except* TypeError as ex: + assert isinstance(ex, ExceptionGroup) + assert len(ex.exceptions) == 1 + assert isinstance(ex.exceptions[0], TypeError) + """ + ) + exec(python_code) + + +if __name__ == "__main__": + if os.environ.get("PARALLEL_CI"): + sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__])) + else: + sys.exit(pytest.main(["-sv", __file__])) diff --git a/python/ray/tests/test_failure.py b/python/ray/tests/test_failure.py index 072005becfc2a..cf4fb5adde583 100644 --- a/python/ray/tests/test_failure.py +++ b/python/ray/tests/test_failure.py @@ -322,6 +322,26 @@ def foo(): assert isinstance(ex, RayTaskError) +def test_baseexception_task(ray_start_regular): + @ray.remote + def task(): + raise BaseException("abc") + + with pytest.raises(ray.exceptions.WorkerCrashedError): + ray.get(task.remote()) + + +def test_baseexception_actor(ray_start_regular): + @ray.remote + class Actor: + def f(self): + raise BaseException("abc") + + with pytest.raises(ray.exceptions.ActorDiedError): + a = Actor.remote() + ray.get(a.f.remote()) + + @pytest.mark.skip("This test does not work yet.") @pytest.mark.parametrize("ray_start_object_store_memory", [10**6], indirect=True) def test_put_error1(ray_start_object_store_memory, error_pubsub): From dc735312cb50e8cd88bcedba7ee32a52dfb1b746 Mon Sep 17 00:00:00 2001 From: Scott Lee Date: Wed, 16 Oct 2024 15:18:19 -0700 Subject: [PATCH 42/46] [Data] Fix bug with inserting custom optimization rule at index 0 (#48051) ## Why are these changes needed? Fix the bug reported https://github.com/ray-project/ray/pull/48039#discussion_r1803592826 ## Related issue number ## Checks - [x] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [x] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [x] Unit tests - [ ] Release tests - [ ] This PR is not tested :( Signed-off-by: Scott Lee --- .../ray/data/_internal/logical/optimizers.py | 4 +- .../data/tests/test_execution_optimizer.py | 83 +++++++++++++++++++ 2 files changed, 85 insertions(+), 2 deletions(-) diff --git a/python/ray/data/_internal/logical/optimizers.py b/python/ray/data/_internal/logical/optimizers.py index 07d029275e3cb..3c89d658f1511 100644 --- a/python/ray/data/_internal/logical/optimizers.py +++ b/python/ray/data/_internal/logical/optimizers.py @@ -32,7 +32,7 @@ @DeveloperAPI def register_logical_rule(cls: Type[Rule], insert_index: Optional[int] = None): - if not insert_index: + if insert_index is None: _LOGICAL_RULES.append(cls) else: _LOGICAL_RULES.insert(insert_index, cls) @@ -40,7 +40,7 @@ def register_logical_rule(cls: Type[Rule], insert_index: Optional[int] = None): @DeveloperAPI def register_physical_rule(cls: Type[Rule], insert_index: Optional[int] = None): - if not insert_index: + if insert_index is None: _PHYSICAL_RULES.append(cls) else: _PHYSICAL_RULES.insert(insert_index, cls) diff --git a/python/ray/data/tests/test_execution_optimizer.py b/python/ray/data/tests/test_execution_optimizer.py index 25b8b051d27c7..04da5508b1fbf 100644 --- a/python/ray/data/tests/test_execution_optimizer.py +++ b/python/ray/data/tests/test_execution_optimizer.py @@ -1649,5 +1649,88 @@ def test_zero_copy_fusion_eliminate_build_output_blocks(ray_start_regular_shared ) +def test_insert_logical_optimization_rules(): + class FakeRule1: + pass + + class FakeRule2: + pass + + from ray.data._internal.logical.optimizers import ( + _LOGICAL_RULES, + register_logical_rule, + ) + from ray.data._internal.logical.rules.randomize_blocks import ( + ReorderRandomizeBlocksRule, + ) + + register_logical_rule(FakeRule1) + assert _LOGICAL_RULES == [ReorderRandomizeBlocksRule, FakeRule1] + + register_logical_rule(FakeRule2, 1) + assert _LOGICAL_RULES == [ReorderRandomizeBlocksRule, FakeRule2, FakeRule1] + + register_logical_rule(FakeRule1, 0) + assert _LOGICAL_RULES == [ + FakeRule1, + ReorderRandomizeBlocksRule, + FakeRule2, + FakeRule1, + ] + + +def test_insert_physical_optimization_rules(): + class FakeRule1: + pass + + class FakeRule2: + pass + + from ray.data._internal.logical.optimizers import ( + _PHYSICAL_RULES, + register_physical_rule, + ) + from ray.data._internal.logical.rules.inherit_target_max_block_size import ( + InheritTargetMaxBlockSizeRule, + ) + from ray.data._internal.logical.rules.operator_fusion import OperatorFusionRule + from ray.data._internal.logical.rules.set_read_parallelism import ( + SetReadParallelismRule, + ) + from ray.data._internal.logical.rules.zero_copy_map_fusion import ( + EliminateBuildOutputBlocks, + ) + + register_physical_rule(FakeRule1) + assert _PHYSICAL_RULES == [ + InheritTargetMaxBlockSizeRule, + SetReadParallelismRule, + OperatorFusionRule, + EliminateBuildOutputBlocks, + FakeRule1, + ] + + register_physical_rule(FakeRule2, 2) + assert _PHYSICAL_RULES == [ + InheritTargetMaxBlockSizeRule, + SetReadParallelismRule, + FakeRule2, + OperatorFusionRule, + EliminateBuildOutputBlocks, + FakeRule1, + ] + + register_physical_rule(FakeRule1, 0) + assert _PHYSICAL_RULES == [ + FakeRule1, + InheritTargetMaxBlockSizeRule, + SetReadParallelismRule, + FakeRule2, + OperatorFusionRule, + EliminateBuildOutputBlocks, + FakeRule1, + ] + + if __name__ == "__main__": sys.exit(pytest.main(["-v", __file__])) From c8dae968bce9bdb5543168622665b4a905a9e557 Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 16 Oct 2024 15:21:58 -0700 Subject: [PATCH 43/46] [chore][core][compiled-graph] Rename idx to local_idx or dag_idx in RayCG codebase (#47992) Signed-off-by: Kai-Hsun Chen --- .../experimental/test_execution_schedule.py | 768 +++++++++--------- 1 file changed, 391 insertions(+), 377 deletions(-) diff --git a/python/ray/dag/tests/experimental/test_execution_schedule.py b/python/ray/dag/tests/experimental/test_execution_schedule.py index 9477177fdc912..8633107a2c25b 100644 --- a/python/ray/dag/tests/experimental/test_execution_schedule.py +++ b/python/ray/dag/tests/experimental/test_execution_schedule.py @@ -35,12 +35,12 @@ def mock_init(self): pass -def generate_dag_graph_nodes(local_idx, dag_idx, actor_handle, requires_nccl): +def generate_dag_graph_nodes(exec_task_idx, task_idx, actor_handle, requires_nccl): graph_nodes = {} for op_type in _DAGNodeOperationType: graph_nodes[op_type] = _DAGOperationGraphNode( - _DAGNodeOperation(local_idx, op_type), - dag_idx, + _DAGNodeOperation(exec_task_idx, op_type), + task_idx, actor_handle, requires_nccl, ) @@ -52,8 +52,8 @@ class TestSelectNextNodes: Test whether `_select_next_nodes` function selects the next nodes for topological sort to generate execution schedule correctly. - dag_idx: Each DAG node has a unique global index. - local_idx: The DAG node's index in the actor's `executable_tasks` list. + task_idx: Each DAG node has a unique global index. + exec_task_idx: The DAG node's index in the actor's `executable_tasks` list. """ def test_two_candidates_on_same_actor(self, monkeypatch): @@ -73,19 +73,19 @@ def test_two_candidates_on_same_actor(self, monkeypatch): fake_actor = ActorHandle("fake_actor") # The DAG node has a global index of 1, and its index in the # actor's `executable_tasks` list is 0. - dag_idx_1 = 1 + task_idx_1 = 1 dag_node_1 = _DAGOperationGraphNode( _DAGNodeOperation(0, _DAGNodeOperationType.READ), - dag_idx_1, + task_idx_1, fake_actor, False, ) # The DAG node has a global index of 2, and its index in the # actor's `executable_tasks` list is 1. - dag_idx_2 = 2 + task_idx_2 = 2 dag_node_2 = _DAGOperationGraphNode( _DAGNodeOperation(1, _DAGNodeOperationType.READ), - dag_idx_2, + task_idx_2, fake_actor, False, ) @@ -113,38 +113,38 @@ def test_only_one_nccl_write(self, monkeypatch): execution schedule. """ monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) - fake_actor_1, dag_idx_1, local_idx_1 = ActorHandle("fake_actor_1"), 1, 0 - fake_actor_2, dag_idx_2, local_idx_2 = ActorHandle("fake_actor_2"), 2, 0 + fake_actor_1, task_idx_1, exec_task_idx_1 = ActorHandle("fake_actor_1"), 1, 0 + fake_actor_2, task_idx_2, exec_task_idx_2 = ActorHandle("fake_actor_2"), 2, 0 mock_graph = { - dag_idx_1: generate_dag_graph_nodes( - local_idx_1, dag_idx_1, fake_actor_1, True + task_idx_1: generate_dag_graph_nodes( + exec_task_idx_1, task_idx_1, fake_actor_1, True ), - dag_idx_2: generate_dag_graph_nodes( - local_idx_2, dag_idx_2, fake_actor_2, False + task_idx_2: generate_dag_graph_nodes( + exec_task_idx_2, task_idx_2, fake_actor_2, False ), } - del mock_graph[dag_idx_1][_DAGNodeOperationType.READ] - del mock_graph[dag_idx_1][_DAGNodeOperationType.COMPUTE] + del mock_graph[task_idx_1][_DAGNodeOperationType.READ] + del mock_graph[task_idx_1][_DAGNodeOperationType.COMPUTE] _add_edge( - mock_graph[dag_idx_1][_DAGNodeOperationType.WRITE], - mock_graph[dag_idx_2][_DAGNodeOperationType.READ], + mock_graph[task_idx_1][_DAGNodeOperationType.WRITE], + mock_graph[task_idx_2][_DAGNodeOperationType.READ], ) _add_edge( - mock_graph[dag_idx_2][_DAGNodeOperationType.READ], - mock_graph[dag_idx_2][_DAGNodeOperationType.COMPUTE], + mock_graph[task_idx_2][_DAGNodeOperationType.READ], + mock_graph[task_idx_2][_DAGNodeOperationType.COMPUTE], ) _add_edge( - mock_graph[dag_idx_2][_DAGNodeOperationType.COMPUTE], - mock_graph[dag_idx_2][_DAGNodeOperationType.WRITE], + mock_graph[task_idx_2][_DAGNodeOperationType.COMPUTE], + mock_graph[task_idx_2][_DAGNodeOperationType.WRITE], ) mock_actor_to_candidates = { - fake_actor_1: [mock_graph[dag_idx_1][_DAGNodeOperationType.WRITE]], + fake_actor_1: [mock_graph[task_idx_1][_DAGNodeOperationType.WRITE]], } next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) assert len(next_nodes) == 2 - assert next_nodes[0] == mock_graph[dag_idx_1][_DAGNodeOperationType.WRITE] - assert next_nodes[1] == mock_graph[dag_idx_2][_DAGNodeOperationType.READ] + assert next_nodes[0] == mock_graph[task_idx_1][_DAGNodeOperationType.WRITE] + assert next_nodes[1] == mock_graph[task_idx_2][_DAGNodeOperationType.READ] def test_two_nccl_writes(self, monkeypatch): """ @@ -164,67 +164,69 @@ def test_two_nccl_writes(self, monkeypatch): monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) fake_actor_1 = ActorHandle("fake_actor_1") - dag_idx_1_0, local_idx_1_0 = 1, 0 - dag_idx_1_1, local_idx_1_1 = 3, 1 + task_idx_1_0, exec_task_idx_1_0 = 1, 0 + task_idx_1_1, exec_task_idx_1_1 = 3, 1 fake_actor_2 = ActorHandle("fake_actor_2") - dag_idx_2_0, local_idx_2_0 = 2, 0 - dag_idx_2_1, local_idx_2_1 = 4, 1 + task_idx_2_0, exec_task_idx_2_0 = 2, 0 + task_idx_2_1, exec_task_idx_2_1 = 4, 1 # Run the test 10 times to ensure that the result of `_select_next_nodes` # is deterministic. for _ in range(20): mock_graph = { - dag_idx_1_0: generate_dag_graph_nodes( - local_idx_1_0, dag_idx_1_0, fake_actor_1, True + task_idx_1_0: generate_dag_graph_nodes( + exec_task_idx_1_0, task_idx_1_0, fake_actor_1, True ), - dag_idx_1_1: generate_dag_graph_nodes( - local_idx_1_1, dag_idx_1_1, fake_actor_1, False + task_idx_1_1: generate_dag_graph_nodes( + exec_task_idx_1_1, task_idx_1_1, fake_actor_1, False ), - dag_idx_2_0: generate_dag_graph_nodes( - local_idx_2_0, dag_idx_2_0, fake_actor_2, True + task_idx_2_0: generate_dag_graph_nodes( + exec_task_idx_2_0, task_idx_2_0, fake_actor_2, True ), - dag_idx_2_1: generate_dag_graph_nodes( - local_idx_2_1, dag_idx_2_1, fake_actor_2, False + task_idx_2_1: generate_dag_graph_nodes( + exec_task_idx_2_1, task_idx_2_1, fake_actor_2, False ), } - del mock_graph[dag_idx_1_0][_DAGNodeOperationType.READ] - del mock_graph[dag_idx_1_0][_DAGNodeOperationType.COMPUTE] - del mock_graph[dag_idx_2_0][_DAGNodeOperationType.READ] - del mock_graph[dag_idx_2_0][_DAGNodeOperationType.COMPUTE] + del mock_graph[task_idx_1_0][_DAGNodeOperationType.READ] + del mock_graph[task_idx_1_0][_DAGNodeOperationType.COMPUTE] + del mock_graph[task_idx_2_0][_DAGNodeOperationType.READ] + del mock_graph[task_idx_2_0][_DAGNodeOperationType.COMPUTE] _add_edge( - mock_graph[dag_idx_1_0][_DAGNodeOperationType.WRITE], - mock_graph[dag_idx_2_1][_DAGNodeOperationType.READ], + mock_graph[task_idx_1_0][_DAGNodeOperationType.WRITE], + mock_graph[task_idx_2_1][_DAGNodeOperationType.READ], ) _add_edge( - mock_graph[dag_idx_2_0][_DAGNodeOperationType.WRITE], - mock_graph[dag_idx_1_1][_DAGNodeOperationType.READ], + mock_graph[task_idx_2_0][_DAGNodeOperationType.WRITE], + mock_graph[task_idx_1_1][_DAGNodeOperationType.READ], ) _add_edge( - mock_graph[dag_idx_2_1][_DAGNodeOperationType.READ], - mock_graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE], + mock_graph[task_idx_2_1][_DAGNodeOperationType.READ], + mock_graph[task_idx_2_1][_DAGNodeOperationType.COMPUTE], ) _add_edge( - mock_graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE], - mock_graph[dag_idx_2_1][_DAGNodeOperationType.WRITE], + mock_graph[task_idx_2_1][_DAGNodeOperationType.COMPUTE], + mock_graph[task_idx_2_1][_DAGNodeOperationType.WRITE], ) _add_edge( - mock_graph[dag_idx_1_1][_DAGNodeOperationType.READ], - mock_graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE], + mock_graph[task_idx_1_1][_DAGNodeOperationType.READ], + mock_graph[task_idx_1_1][_DAGNodeOperationType.COMPUTE], ) _add_edge( - mock_graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE], - mock_graph[dag_idx_1_1][_DAGNodeOperationType.WRITE], + mock_graph[task_idx_1_1][_DAGNodeOperationType.COMPUTE], + mock_graph[task_idx_1_1][_DAGNodeOperationType.WRITE], ) mock_actor_to_candidates = { - fake_actor_1: [mock_graph[dag_idx_1_0][_DAGNodeOperationType.WRITE]], - fake_actor_2: [mock_graph[dag_idx_2_0][_DAGNodeOperationType.WRITE]], + fake_actor_1: [mock_graph[task_idx_1_0][_DAGNodeOperationType.WRITE]], + fake_actor_2: [mock_graph[task_idx_2_0][_DAGNodeOperationType.WRITE]], } next_nodes = _select_next_nodes(mock_actor_to_candidates, mock_graph) assert len(next_nodes) == 2 - assert next_nodes[0] == mock_graph[dag_idx_1_0][_DAGNodeOperationType.WRITE] - assert next_nodes[1] == mock_graph[dag_idx_2_1][_DAGNodeOperationType.READ] + assert ( + next_nodes[0] == mock_graph[task_idx_1_0][_DAGNodeOperationType.WRITE] + ) + assert next_nodes[1] == mock_graph[task_idx_2_1][_DAGNodeOperationType.READ] class TestBuildDAGNodeOperationGraph: @@ -237,7 +239,7 @@ class TestBuildDAGNodeOperationGraph: def check_edges_between_read_compute_write( self, graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], - dag_idx: int, + task_idx: int, expected_num_edges: List[Tuple[int, int]], ): """ @@ -246,54 +248,54 @@ def check_edges_between_read_compute_write( Args: graph: The operation graph generated by `_build_dag_node_operation_graph`. - dag_idx: The global index of the task used to access the task in + task_idx: The global index of the task used to access the task in `idx_to_task`. expected_num_edges: A list of tuples where each tuple contains the expected number of in-edges and out-edges for READ, COMPUTE, and WRITE operations. """ assert len(expected_num_edges) == 3 - assert len(graph[dag_idx]) == 3 - read_node = graph[dag_idx][_DAGNodeOperationType.READ] - compute_node = graph[dag_idx][_DAGNodeOperationType.COMPUTE] - write_node = graph[dag_idx][_DAGNodeOperationType.WRITE] + assert len(graph[task_idx]) == 3 + read_node = graph[task_idx][_DAGNodeOperationType.READ] + compute_node = graph[task_idx][_DAGNodeOperationType.COMPUTE] + write_node = graph[task_idx][_DAGNodeOperationType.WRITE] for idx, node in enumerate([read_node, compute_node, write_node]): assert node.in_degree == expected_num_edges[idx][0] assert len(node.out_edges) == expected_num_edges[idx][1] - assert (dag_idx, _DAGNodeOperationType.COMPUTE) in read_node.out_edges - assert (dag_idx, _DAGNodeOperationType.READ) in compute_node.in_edges - assert (dag_idx, _DAGNodeOperationType.WRITE) in compute_node.out_edges - assert (dag_idx, _DAGNodeOperationType.COMPUTE) in write_node.in_edges + assert (task_idx, _DAGNodeOperationType.COMPUTE) in read_node.out_edges + assert (task_idx, _DAGNodeOperationType.READ) in compute_node.in_edges + assert (task_idx, _DAGNodeOperationType.WRITE) in compute_node.out_edges + assert (task_idx, _DAGNodeOperationType.COMPUTE) in write_node.in_edges def check_edge_between_writer_and_reader( self, graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], - writer_dag_idx: int, - reader_dag_idx: int, + writer_task_idx: int, + reader_task_idx: int, ): """ Check whether the edge from writer's WRITE to reader's READ operation is added. Args: graph: The operation graph generated by `_build_dag_node_operation_graph`. - writer_dag_idx: The index of the task used to access the task + writer_task_idx: The index of the task used to access the task that the writer belongs to in `idx_to_task`. - reader_dag_idx: The index of the task used to access the task + reader_task_idx: The index of the task used to access the task that the reader belongs to in `idx_to_task`. """ - write_node = graph[writer_dag_idx][_DAGNodeOperationType.WRITE] - read_node = graph[reader_dag_idx][_DAGNodeOperationType.READ] + write_node = graph[writer_task_idx][_DAGNodeOperationType.WRITE] + read_node = graph[reader_task_idx][_DAGNodeOperationType.READ] - assert (reader_dag_idx, _DAGNodeOperationType.READ) in write_node.out_edges - assert (writer_dag_idx, _DAGNodeOperationType.WRITE) in read_node.in_edges + assert (reader_task_idx, _DAGNodeOperationType.READ) in write_node.out_edges + assert (writer_task_idx, _DAGNodeOperationType.WRITE) in read_node.in_edges def check_edge_between_compute_nodes( self, graph: Dict[int, Dict[_DAGNodeOperationType, _DAGOperationGraphNode]], - dag_idx_1: int, - dag_idx_2: int, + task_idx_1: int, + task_idx_2: int, ): """ Check whether the edge from COMPUTE with `bind_index` i to COMPUTE with @@ -301,18 +303,18 @@ def check_edge_between_compute_nodes( Args: graph: The operation graph generated by `_build_dag_node_operation_graph`. - dag_idx_1: The index of the task used to access the task in + task_idx_1: The index of the task used to access the task in `idx_to_task`. - dag_idx_2: The index of the task used to access the task in + task_idx_2: The index of the task used to access the task in `idx_to_task`. Note that both tasks belong to the same actor, and the `bind_index` of the second task is equal to the `bind_index` of the first task plus one. """ - compute_node_1 = graph[dag_idx_1][_DAGNodeOperationType.COMPUTE] - compute_node_2 = graph[dag_idx_2][_DAGNodeOperationType.COMPUTE] + compute_node_1 = graph[task_idx_1][_DAGNodeOperationType.COMPUTE] + compute_node_2 = graph[task_idx_2][_DAGNodeOperationType.COMPUTE] - assert (dag_idx_2, _DAGNodeOperationType.COMPUTE) in compute_node_1.out_edges - assert (dag_idx_1, _DAGNodeOperationType.COMPUTE) in compute_node_2.in_edges + assert (task_idx_2, _DAGNodeOperationType.COMPUTE) in compute_node_1.out_edges + assert (task_idx_1, _DAGNodeOperationType.COMPUTE) in compute_node_2.in_edges def test_edges_between_read_compute_write(self, monkeypatch): """ @@ -331,17 +333,17 @@ def test_edges_between_read_compute_write(self, monkeypatch): } fake_actor = "fake_actor" - dag_idx = 1 + task_idx = 1 actor_to_operation_nodes = { fake_actor: [ - list(generate_dag_graph_nodes(0, dag_idx, fake_actor, False).values()) + list(generate_dag_graph_nodes(0, task_idx, fake_actor, False).values()) ] } graph = _build_dag_node_operation_graph(idx_to_task, actor_to_operation_nodes) assert len(graph) == 1 self.check_edges_between_read_compute_write( - graph, dag_idx, [(0, 1), (1, 1), (1, 0)] + graph, task_idx, [(0, 1), (1, 1), (1, 0)] ) def test_edge_between_writer_and_reader(self, monkeypatch): @@ -354,8 +356,8 @@ def test_edge_between_writer_and_reader(self, monkeypatch): monkeypatch.setattr(ClassMethodNode, "__init__", mock_class_method_call_init) monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) - fake_actor_1, dag_idx_1 = "fake_actor_1", 1 - fake_actor_2, dag_idx_2 = "fake_actor_2", 2 + fake_actor_1, task_idx_1 = "fake_actor_1", 1 + fake_actor_2, task_idx_2 = "fake_actor_2", 2 idx_to_task = { 0: CompiledTask(0, InputNode()), 1: CompiledTask(1, ClassMethodNode()), @@ -367,12 +369,16 @@ def test_edge_between_writer_and_reader(self, monkeypatch): actor_to_operation_nodes = { fake_actor_1: [ list( - generate_dag_graph_nodes(0, dag_idx_1, fake_actor_1, False).values() + generate_dag_graph_nodes( + 0, task_idx_1, fake_actor_1, False + ).values() ) ], fake_actor_2: [ list( - generate_dag_graph_nodes(0, dag_idx_2, fake_actor_2, False).values() + generate_dag_graph_nodes( + 0, task_idx_2, fake_actor_2, False + ).values() ) ], } @@ -380,12 +386,12 @@ def test_edge_between_writer_and_reader(self, monkeypatch): assert len(graph) == 2 self.check_edges_between_read_compute_write( - graph, dag_idx_1, [(0, 1), (1, 1), (1, 1)] + graph, task_idx_1, [(0, 1), (1, 1), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, dag_idx_2, [(1, 1), (1, 1), (1, 0)] + graph, task_idx_2, [(1, 1), (1, 1), (1, 0)] ) - self.check_edge_between_writer_and_reader(graph, dag_idx_1, dag_idx_2) + self.check_edge_between_writer_and_reader(graph, task_idx_1, task_idx_2) def test_edge_between_compute_nodes(self, monkeypatch): """ @@ -399,22 +405,22 @@ def test_edge_between_compute_nodes(self, monkeypatch): monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) fake_actor = "fake_actor" - dag_idx_1, dag_idx_2 = 1, 2 + task_idx_1, task_idx_2 = 1, 2 idx_to_task = { 0: CompiledTask(0, InputNode()), - dag_idx_1: CompiledTask(dag_idx_1, ClassMethodNode()), - dag_idx_2: CompiledTask(dag_idx_2, ClassMethodNode()), + task_idx_1: CompiledTask(task_idx_1, ClassMethodNode()), + task_idx_2: CompiledTask(task_idx_2, ClassMethodNode()), 3: CompiledTask(3, MultiOutputNode()), } - idx_to_task[dag_idx_1].downstream_task_idxs = {dag_idx_2: fake_actor} + idx_to_task[task_idx_1].downstream_task_idxs = {task_idx_2: fake_actor} actor_to_operation_nodes = { fake_actor: [ list( - generate_dag_graph_nodes(0, dag_idx_1, fake_actor, False).values() + generate_dag_graph_nodes(0, task_idx_1, fake_actor, False).values() ), list( - generate_dag_graph_nodes(1, dag_idx_2, fake_actor, False).values() + generate_dag_graph_nodes(1, task_idx_2, fake_actor, False).values() ), ], } @@ -422,13 +428,13 @@ def test_edge_between_compute_nodes(self, monkeypatch): assert len(graph) == 2 self.check_edges_between_read_compute_write( - graph, dag_idx_1, [(0, 1), (1, 2), (1, 1)] + graph, task_idx_1, [(0, 1), (1, 2), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, dag_idx_2, [(1, 1), (2, 1), (1, 0)] + graph, task_idx_2, [(1, 1), (2, 1), (1, 0)] ) - self.check_edge_between_writer_and_reader(graph, dag_idx_1, dag_idx_2) - self.check_edge_between_compute_nodes(graph, dag_idx_1, dag_idx_2) + self.check_edge_between_writer_and_reader(graph, task_idx_1, task_idx_2) + self.check_edge_between_compute_nodes(graph, task_idx_1, task_idx_2) def test_two_actors(self, monkeypatch): """ @@ -443,35 +449,43 @@ def test_two_actors(self, monkeypatch): monkeypatch.setattr(ClassMethodNode, "__init__", mock_class_method_call_init) monkeypatch.setattr(MultiOutputNode, "__init__", mock_init) - fake_actor_1, dag_idx_1, dag_idx_3 = "fake_actor_1", 1, 3 - fake_actor_2, dag_idx_2, dag_idx_4 = "fake_actor_2", 2, 4 + fake_actor_1, task_idx_1, task_idx_3 = "fake_actor_1", 1, 3 + fake_actor_2, task_idx_2, task_idx_4 = "fake_actor_2", 2, 4 idx_to_task = { 0: CompiledTask(0, InputNode()), - dag_idx_1: CompiledTask(dag_idx_1, ClassMethodNode()), - dag_idx_2: CompiledTask(dag_idx_2, ClassMethodNode()), - dag_idx_3: CompiledTask(dag_idx_3, ClassMethodNode()), - dag_idx_4: CompiledTask(dag_idx_4, ClassMethodNode()), + task_idx_1: CompiledTask(task_idx_1, ClassMethodNode()), + task_idx_2: CompiledTask(task_idx_2, ClassMethodNode()), + task_idx_3: CompiledTask(task_idx_3, ClassMethodNode()), + task_idx_4: CompiledTask(task_idx_4, ClassMethodNode()), 5: CompiledTask(5, MultiOutputNode()), } - idx_to_task[dag_idx_1].downstream_task_idxs = {dag_idx_4: fake_actor_2} - idx_to_task[dag_idx_2].downstream_task_idxs = {dag_idx_3: fake_actor_1} + idx_to_task[task_idx_1].downstream_task_idxs = {task_idx_4: fake_actor_2} + idx_to_task[task_idx_2].downstream_task_idxs = {task_idx_3: fake_actor_1} actor_to_operation_nodes = { fake_actor_1: [ list( - generate_dag_graph_nodes(0, dag_idx_1, fake_actor_1, False).values() + generate_dag_graph_nodes( + 0, task_idx_1, fake_actor_1, False + ).values() ), list( - generate_dag_graph_nodes(1, dag_idx_3, fake_actor_1, False).values() + generate_dag_graph_nodes( + 1, task_idx_3, fake_actor_1, False + ).values() ), ], fake_actor_2: [ list( - generate_dag_graph_nodes(0, dag_idx_2, fake_actor_2, False).values() + generate_dag_graph_nodes( + 0, task_idx_2, fake_actor_2, False + ).values() ), list( - generate_dag_graph_nodes(1, dag_idx_4, fake_actor_2, False).values() + generate_dag_graph_nodes( + 1, task_idx_4, fake_actor_2, False + ).values() ), ], } @@ -479,19 +493,19 @@ def test_two_actors(self, monkeypatch): assert len(graph) == 4 self.check_edges_between_read_compute_write( - graph, dag_idx_1, [(0, 1), (1, 2), (1, 1)] + graph, task_idx_1, [(0, 1), (1, 2), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, dag_idx_2, [(0, 1), (1, 2), (1, 1)] + graph, task_idx_2, [(0, 1), (1, 2), (1, 1)] ) self.check_edges_between_read_compute_write( - graph, dag_idx_3, [(1, 1), (2, 1), (1, 0)] + graph, task_idx_3, [(1, 1), (2, 1), (1, 0)] ) self.check_edges_between_read_compute_write( - graph, dag_idx_4, [(1, 1), (2, 1), (1, 0)] + graph, task_idx_4, [(1, 1), (2, 1), (1, 0)] ) - self.check_edge_between_writer_and_reader(graph, dag_idx_1, dag_idx_4) - self.check_edge_between_writer_and_reader(graph, dag_idx_2, dag_idx_3) + self.check_edge_between_writer_and_reader(graph, task_idx_1, task_idx_4) + self.check_edge_between_writer_and_reader(graph, task_idx_2, task_idx_3) class TestGenerateActorToExecutionSchedule: @@ -564,7 +578,7 @@ def add_control_dependency( def test_single_actor_1(self, monkeypatch): """ - driver -> fake_actor.op (dag_idx_1) -> fake_actor.op (dag_idx_2) -> driver + driver -> fake_actor.op (task_idx_1) -> fake_actor.op (task_idx_2) -> driver Test the case where there is only one actor and no NCCL operations. Because there is no NCCL operation, all operations with smaller @@ -574,90 +588,90 @@ def test_single_actor_1(self, monkeypatch): monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) fake_actor = ActorHandle("fake_actor") - dag_idx_1, local_idx_1 = 1, 0 - dag_idx_2, local_idx_2 = 2, 1 + task_idx_1, exec_task_idx_1 = 1, 0 + task_idx_2, exec_task_idx_2 = 2, 1 graph = { - dag_idx_1: generate_dag_graph_nodes( - local_idx_1, dag_idx_1, fake_actor, False + task_idx_1: generate_dag_graph_nodes( + exec_task_idx_1, task_idx_1, fake_actor, False ), - dag_idx_2: generate_dag_graph_nodes( - local_idx_2, dag_idx_2, fake_actor, False + task_idx_2: generate_dag_graph_nodes( + exec_task_idx_2, task_idx_2, fake_actor, False ), } - self.add_edge_between_read_compute_write(graph[dag_idx_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_2]) - self.add_data_dependeny(graph[dag_idx_1], graph[dag_idx_2]) - self.add_control_dependency(graph[dag_idx_1], graph[dag_idx_2]) + self.add_edge_between_read_compute_write(graph[task_idx_1]) + self.add_edge_between_read_compute_write(graph[task_idx_2]) + self.add_data_dependeny(graph[task_idx_1], graph[task_idx_2]) + self.add_control_dependency(graph[task_idx_1], graph[task_idx_2]) actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) assert len(actor_to_execution_schedule) == 1 assert len(actor_to_execution_schedule[fake_actor]) == 6 assert actor_to_execution_schedule[fake_actor] == [ - graph[dag_idx_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2][_DAGNodeOperationType.WRITE].operation, ] def test_single_actor_2(self, monkeypatch): """ - driver -> fake_actor.op (dag_idx_1) -> fake_actor.op (dag_idx_2) -> driver + driver -> fake_actor.op (task_idx_1) -> fake_actor.op (task_idx_2) -> driver | | - -> fake_actor.op (dag_idx_3) - + -> fake_actor.op (task_idx_3) - - When the `dad_idx_1.WRITE` operation is picked, both `dag_idx_2.READ` and - `dag_idx_3.READ` operations should be zero in-degree. In this case, the one + When the `dad_idx_1.WRITE` operation is picked, both `task_idx_2.READ` and + `task_idx_3.READ` operations should be zero in-degree. In this case, the one with the smaller `bind_index` should be selected first. That is, - `dag_idx_2.READ` should be selected first. + `task_idx_2.READ` should be selected first. """ monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) fake_actor = ActorHandle("fake_actor") - dag_idx_1, local_idx_1 = 1, 0 - dag_idx_2, local_idx_2 = 2, 1 - dag_idx_3, local_idx_3 = 3, 2 + task_idx_1, exec_task_idx_1 = 1, 0 + task_idx_2, exec_task_idx_2 = 2, 1 + task_idx_3, exec_task_idx_3 = 3, 2 graph = { - dag_idx_1: generate_dag_graph_nodes( - local_idx_1, dag_idx_1, fake_actor, False + task_idx_1: generate_dag_graph_nodes( + exec_task_idx_1, task_idx_1, fake_actor, False ), - dag_idx_2: generate_dag_graph_nodes( - local_idx_2, dag_idx_2, fake_actor, False + task_idx_2: generate_dag_graph_nodes( + exec_task_idx_2, task_idx_2, fake_actor, False ), - dag_idx_3: generate_dag_graph_nodes( - local_idx_3, dag_idx_3, fake_actor, False + task_idx_3: generate_dag_graph_nodes( + exec_task_idx_3, task_idx_3, fake_actor, False ), } - self.add_edge_between_read_compute_write(graph[dag_idx_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_2]) - self.add_edge_between_read_compute_write(graph[dag_idx_3]) - self.add_data_dependeny(graph[dag_idx_1], graph[dag_idx_2]) - self.add_data_dependeny(graph[dag_idx_1], graph[dag_idx_3]) - self.add_control_dependency(graph[dag_idx_1], graph[dag_idx_2]) - self.add_control_dependency(graph[dag_idx_2], graph[dag_idx_3]) + self.add_edge_between_read_compute_write(graph[task_idx_1]) + self.add_edge_between_read_compute_write(graph[task_idx_2]) + self.add_edge_between_read_compute_write(graph[task_idx_3]) + self.add_data_dependeny(graph[task_idx_1], graph[task_idx_2]) + self.add_data_dependeny(graph[task_idx_1], graph[task_idx_3]) + self.add_control_dependency(graph[task_idx_1], graph[task_idx_2]) + self.add_control_dependency(graph[task_idx_2], graph[task_idx_3]) actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) assert len(actor_to_execution_schedule) == 1 assert len(actor_to_execution_schedule[fake_actor]) == 9 assert actor_to_execution_schedule[fake_actor] == [ - graph[dag_idx_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_3][_DAGNodeOperationType.READ].operation, - graph[dag_idx_3][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_3][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_3][_DAGNodeOperationType.READ].operation, + graph[task_idx_3][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_3][_DAGNodeOperationType.WRITE].operation, ] def test_two_actors_no_nccl(self, monkeypatch): """ - driver -> actor_1.op (dag_idx_1_1) -> actor_2.op (dag_idx_2_2) -> driver + driver -> actor_1.op (task_idx_1_1) -> actor_2.op (task_idx_2_2) -> driver | | - -> actor_2.op (dag_idx_2_1) -> actor_1.op (dag_idx_1_2) - + -> actor_2.op (task_idx_2_1) -> actor_1.op (task_idx_1_2) - Test the case where there are two actors and no NCCL operations. Because there is no NCCL operation, all operations with smaller @@ -667,35 +681,35 @@ def test_two_actors_no_nccl(self, monkeypatch): monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) fake_actor_1 = ActorHandle("fake_actor_1") - dag_idx_1_1, local_idx_1_1 = 1, 0 - dag_idx_1_2, local_idx_1_2 = 4, 1 + task_idx_1_1, exec_task_idx_1_1 = 1, 0 + task_idx_1_2, exec_task_idx_1_2 = 4, 1 fake_actor_2 = ActorHandle("fake_actor_2") - dag_idx_2_1, local_idx_2_1 = 2, 0 - dag_idx_2_2, local_idx_2_2 = 3, 1 + task_idx_2_1, exec_task_idx_2_1 = 2, 0 + task_idx_2_2, exec_task_idx_2_2 = 3, 1 graph = { - dag_idx_1_1: generate_dag_graph_nodes( - local_idx_1_1, dag_idx_1_1, fake_actor_1, False + task_idx_1_1: generate_dag_graph_nodes( + exec_task_idx_1_1, task_idx_1_1, fake_actor_1, False ), - dag_idx_2_1: generate_dag_graph_nodes( - local_idx_2_1, dag_idx_2_1, fake_actor_2, False + task_idx_2_1: generate_dag_graph_nodes( + exec_task_idx_2_1, task_idx_2_1, fake_actor_2, False ), - dag_idx_2_2: generate_dag_graph_nodes( - local_idx_2_2, dag_idx_2_2, fake_actor_2, False + task_idx_2_2: generate_dag_graph_nodes( + exec_task_idx_2_2, task_idx_2_2, fake_actor_2, False ), - dag_idx_1_2: generate_dag_graph_nodes( - local_idx_1_2, dag_idx_1_2, fake_actor_1, False + task_idx_1_2: generate_dag_graph_nodes( + exec_task_idx_1_2, task_idx_1_2, fake_actor_1, False ), } - self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) - self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_2]) - self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_1_2]) - self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) - self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) + self.add_edge_between_read_compute_write(graph[task_idx_1_1]) + self.add_edge_between_read_compute_write(graph[task_idx_1_2]) + self.add_edge_between_read_compute_write(graph[task_idx_2_1]) + self.add_edge_between_read_compute_write(graph[task_idx_2_2]) + self.add_data_dependeny(graph[task_idx_1_1], graph[task_idx_2_2]) + self.add_data_dependeny(graph[task_idx_2_1], graph[task_idx_1_2]) + self.add_control_dependency(graph[task_idx_1_1], graph[task_idx_1_2]) + self.add_control_dependency(graph[task_idx_2_1], graph[task_idx_2_2]) actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) assert len(actor_to_execution_schedule) == 2 @@ -703,64 +717,64 @@ def test_two_actors_no_nccl(self, monkeypatch): assert len(actor_to_execution_schedule[fake_actor_2]) == 6 assert actor_to_execution_schedule[fake_actor_1] == [ - graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.WRITE].operation, ] assert actor_to_execution_schedule[fake_actor_2] == [ - graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_2][_DAGNodeOperationType.WRITE].operation, ] def test_two_actors_with_nccl(self, monkeypatch): """ - driver -> actor_1.op (dag_idx_1_1) -> actor_2.op (dag_idx_2_2) -> driver + driver -> actor_1.op (task_idx_1_1) -> actor_2.op (task_idx_2_2) -> driver | | - -> actor_2.op (dag_idx_2_1) -> actor_1.op (dag_idx_1_2) - + -> actor_2.op (task_idx_2_1) -> actor_1.op (task_idx_1_2) - In this test, the communication between fake_actor_1 and fake_actor_2 is done - using NCCL. When the dag_idx_1.WRITE operation is picked, the dag_idx_2.READ + using NCCL. When the task_idx_1.WRITE operation is picked, the task_idx_2.READ operation is also added to the execution schedule because of the NCCL operation. """ monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) fake_actor_1 = ActorHandle("fake_actor_1") - dag_idx_1_1, local_idx_1_1 = 1, 0 - dag_idx_1_2, local_idx_1_2 = 4, 1 + task_idx_1_1, exec_task_idx_1_1 = 1, 0 + task_idx_1_2, exec_task_idx_1_2 = 4, 1 fake_actor_2 = ActorHandle("fake_actor_2") - dag_idx_2_1, local_idx_2_1 = 2, 0 - dag_idx_2_2, local_idx_2_2 = 3, 1 + task_idx_2_1, exec_task_idx_2_1 = 2, 0 + task_idx_2_2, exec_task_idx_2_2 = 3, 1 graph = { - dag_idx_1_1: generate_dag_graph_nodes( - local_idx_1_1, dag_idx_1_1, fake_actor_1, True + task_idx_1_1: generate_dag_graph_nodes( + exec_task_idx_1_1, task_idx_1_1, fake_actor_1, True ), - dag_idx_2_1: generate_dag_graph_nodes( - local_idx_2_1, dag_idx_2_1, fake_actor_2, True + task_idx_2_1: generate_dag_graph_nodes( + exec_task_idx_2_1, task_idx_2_1, fake_actor_2, True ), - dag_idx_2_2: generate_dag_graph_nodes( - local_idx_2_2, dag_idx_2_2, fake_actor_2, False + task_idx_2_2: generate_dag_graph_nodes( + exec_task_idx_2_2, task_idx_2_2, fake_actor_2, False ), - dag_idx_1_2: generate_dag_graph_nodes( - local_idx_1_2, dag_idx_1_2, fake_actor_1, False + task_idx_1_2: generate_dag_graph_nodes( + exec_task_idx_1_2, task_idx_1_2, fake_actor_1, False ), } - self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) - self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_2]) - self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_1_2]) - self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) - self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) + self.add_edge_between_read_compute_write(graph[task_idx_1_1]) + self.add_edge_between_read_compute_write(graph[task_idx_1_2]) + self.add_edge_between_read_compute_write(graph[task_idx_2_1]) + self.add_edge_between_read_compute_write(graph[task_idx_2_2]) + self.add_data_dependeny(graph[task_idx_1_1], graph[task_idx_2_2]) + self.add_data_dependeny(graph[task_idx_2_1], graph[task_idx_1_2]) + self.add_control_dependency(graph[task_idx_1_1], graph[task_idx_1_2]) + self.add_control_dependency(graph[task_idx_2_1], graph[task_idx_2_2]) actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) assert len(actor_to_execution_schedule) == 2 @@ -768,21 +782,21 @@ def test_two_actors_with_nccl(self, monkeypatch): assert len(actor_to_execution_schedule[fake_actor_2]) == 6 assert actor_to_execution_schedule[fake_actor_1] == [ - graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.WRITE].operation, ] assert actor_to_execution_schedule[fake_actor_2] == [ - graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, - # The order of `dag_idx_2_2.READ` and `dag_idx_2_2.COMPUTE` is important. - graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + # The order of `task_idx_2_2.READ` and `task_idx_2_2.COMPUTE` is important. + graph[task_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_2][_DAGNodeOperationType.WRITE].operation, ] def test_simulate_pp_2workers_2batches_1f1b_with_nccl(self, monkeypatch): @@ -799,94 +813,94 @@ def test_simulate_pp_2workers_2batches_1f1b_with_nccl(self, monkeypatch): monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) worker_1 = ActorHandle("worker_1") - dag_idx_1_1, local_idx_1_1 = 1, 0 - dag_idx_1_2, local_idx_1_2 = 2, 1 - dag_idx_1_3, local_idx_1_3 = 3, 2 - dag_idx_1_4, local_idx_1_4 = 4, 3 + task_idx_1_1, exec_task_idx_1_1 = 1, 0 + task_idx_1_2, exec_task_idx_1_2 = 2, 1 + task_idx_1_3, exec_task_idx_1_3 = 3, 2 + task_idx_1_4, exec_task_idx_1_4 = 4, 3 worker_2 = ActorHandle("worker_2") - dag_idx_2_1, local_idx_2_1 = 5, 0 - dag_idx_2_2, local_idx_2_2 = 6, 1 - dag_idx_2_3, local_idx_2_3 = 7, 2 - dag_idx_2_4, local_idx_2_4 = 8, 3 + task_idx_2_1, exec_task_idx_2_1 = 5, 0 + task_idx_2_2, exec_task_idx_2_2 = 6, 1 + task_idx_2_3, exec_task_idx_2_3 = 7, 2 + task_idx_2_4, exec_task_idx_2_4 = 8, 3 graph = { - dag_idx_1_1: generate_dag_graph_nodes( - local_idx_1_1, dag_idx_1_1, worker_1, True + task_idx_1_1: generate_dag_graph_nodes( + exec_task_idx_1_1, task_idx_1_1, worker_1, True ), - dag_idx_1_2: generate_dag_graph_nodes( - local_idx_1_2, dag_idx_1_2, worker_1, True + task_idx_1_2: generate_dag_graph_nodes( + exec_task_idx_1_2, task_idx_1_2, worker_1, True ), - dag_idx_1_3: generate_dag_graph_nodes( - local_idx_1_3, dag_idx_1_3, worker_1, False + task_idx_1_3: generate_dag_graph_nodes( + exec_task_idx_1_3, task_idx_1_3, worker_1, False ), - dag_idx_1_4: generate_dag_graph_nodes( - local_idx_1_4, dag_idx_1_4, worker_1, False + task_idx_1_4: generate_dag_graph_nodes( + exec_task_idx_1_4, task_idx_1_4, worker_1, False ), - dag_idx_2_1: generate_dag_graph_nodes( - local_idx_2_1, dag_idx_2_1, worker_2, False + task_idx_2_1: generate_dag_graph_nodes( + exec_task_idx_2_1, task_idx_2_1, worker_2, False ), - dag_idx_2_2: generate_dag_graph_nodes( - local_idx_2_2, dag_idx_2_2, worker_2, True + task_idx_2_2: generate_dag_graph_nodes( + exec_task_idx_2_2, task_idx_2_2, worker_2, True ), - dag_idx_2_3: generate_dag_graph_nodes( - local_idx_2_3, dag_idx_2_3, worker_2, False + task_idx_2_3: generate_dag_graph_nodes( + exec_task_idx_2_3, task_idx_2_3, worker_2, False ), - dag_idx_2_4: generate_dag_graph_nodes( - local_idx_2_4, dag_idx_2_4, worker_2, True + task_idx_2_4: generate_dag_graph_nodes( + exec_task_idx_2_4, task_idx_2_4, worker_2, True ), } - self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_3]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_4]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_3]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_4]) - self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_1]) - self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_2_2]) - self.add_data_dependeny(graph[dag_idx_2_2], graph[dag_idx_1_3]) - self.add_data_dependeny(graph[dag_idx_1_2], graph[dag_idx_2_3]) - self.add_data_dependeny(graph[dag_idx_2_3], graph[dag_idx_2_4]) - self.add_data_dependeny(graph[dag_idx_2_4], graph[dag_idx_1_4]) - self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) - self.add_control_dependency(graph[dag_idx_1_2], graph[dag_idx_1_3]) - self.add_control_dependency(graph[dag_idx_1_3], graph[dag_idx_1_4]) - self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) - self.add_control_dependency(graph[dag_idx_2_2], graph[dag_idx_2_3]) - self.add_control_dependency(graph[dag_idx_2_3], graph[dag_idx_2_4]) + self.add_edge_between_read_compute_write(graph[task_idx_1_1]) + self.add_edge_between_read_compute_write(graph[task_idx_1_2]) + self.add_edge_between_read_compute_write(graph[task_idx_1_3]) + self.add_edge_between_read_compute_write(graph[task_idx_1_4]) + self.add_edge_between_read_compute_write(graph[task_idx_2_1]) + self.add_edge_between_read_compute_write(graph[task_idx_2_2]) + self.add_edge_between_read_compute_write(graph[task_idx_2_3]) + self.add_edge_between_read_compute_write(graph[task_idx_2_4]) + self.add_data_dependeny(graph[task_idx_1_1], graph[task_idx_2_1]) + self.add_data_dependeny(graph[task_idx_2_1], graph[task_idx_2_2]) + self.add_data_dependeny(graph[task_idx_2_2], graph[task_idx_1_3]) + self.add_data_dependeny(graph[task_idx_1_2], graph[task_idx_2_3]) + self.add_data_dependeny(graph[task_idx_2_3], graph[task_idx_2_4]) + self.add_data_dependeny(graph[task_idx_2_4], graph[task_idx_1_4]) + self.add_control_dependency(graph[task_idx_1_1], graph[task_idx_1_2]) + self.add_control_dependency(graph[task_idx_1_2], graph[task_idx_1_3]) + self.add_control_dependency(graph[task_idx_1_3], graph[task_idx_1_4]) + self.add_control_dependency(graph[task_idx_2_1], graph[task_idx_2_2]) + self.add_control_dependency(graph[task_idx_2_2], graph[task_idx_2_3]) + self.add_control_dependency(graph[task_idx_2_3], graph[task_idx_2_4]) actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) assert len(actor_to_execution_schedule) == 2 assert len(actor_to_execution_schedule[worker_1]) == 12 assert len(actor_to_execution_schedule[worker_2]) == 12 assert actor_to_execution_schedule[worker_1] == [ - graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_3][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_3][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_3][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_4][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_4][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_4][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_3][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_3][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_3][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_4][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_4][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_4][_DAGNodeOperationType.WRITE].operation, ] assert actor_to_execution_schedule[worker_2] == [ - graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, - # The order of `dag_idx_2_3.READ` and `dag_idx_2_2.WRITE` is important. - graph[dag_idx_2_3][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_3][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_3][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_4][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_4][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_4][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + # The order of `task_idx_2_3.READ` and `task_idx_2_2.WRITE` is important. + graph[task_idx_2_3][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_3][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_3][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_4][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_4][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_4][_DAGNodeOperationType.WRITE].operation, ] def test_simulate_pp_2workers_2batches_1f1b_no_nccl(self, monkeypatch): @@ -904,97 +918,97 @@ def test_simulate_pp_2workers_2batches_1f1b_no_nccl(self, monkeypatch): monkeypatch.setattr(ActorHandle, "__init__", mock_actor_handle_init) worker_1 = ActorHandle("worker_1") - dag_idx_1_1, local_idx_1_1 = 1, 0 - dag_idx_1_2, local_idx_1_2 = 2, 1 - dag_idx_1_3, local_idx_1_3 = 3, 2 - dag_idx_1_4, local_idx_1_4 = 4, 3 + task_idx_1_1, exec_task_idx_1_1 = 1, 0 + task_idx_1_2, exec_task_idx_1_2 = 2, 1 + task_idx_1_3, exec_task_idx_1_3 = 3, 2 + task_idx_1_4, exec_task_idx_1_4 = 4, 3 worker_2 = ActorHandle("worker_2") - dag_idx_2_1, local_idx_2_1 = 5, 0 - dag_idx_2_2, local_idx_2_2 = 6, 1 - dag_idx_2_3, local_idx_2_3 = 7, 2 - dag_idx_2_4, local_idx_2_4 = 8, 3 + task_idx_2_1, exec_task_idx_2_1 = 5, 0 + task_idx_2_2, exec_task_idx_2_2 = 6, 1 + task_idx_2_3, exec_task_idx_2_3 = 7, 2 + task_idx_2_4, exec_task_idx_2_4 = 8, 3 # No NCCL operation. graph = { - dag_idx_1_1: generate_dag_graph_nodes( - local_idx_1_1, dag_idx_1_1, worker_1, False + task_idx_1_1: generate_dag_graph_nodes( + exec_task_idx_1_1, task_idx_1_1, worker_1, False ), - dag_idx_1_2: generate_dag_graph_nodes( - local_idx_1_2, dag_idx_1_2, worker_1, False + task_idx_1_2: generate_dag_graph_nodes( + exec_task_idx_1_2, task_idx_1_2, worker_1, False ), - dag_idx_1_3: generate_dag_graph_nodes( - local_idx_1_3, dag_idx_1_3, worker_1, False + task_idx_1_3: generate_dag_graph_nodes( + exec_task_idx_1_3, task_idx_1_3, worker_1, False ), - dag_idx_1_4: generate_dag_graph_nodes( - local_idx_1_4, dag_idx_1_4, worker_1, False + task_idx_1_4: generate_dag_graph_nodes( + exec_task_idx_1_4, task_idx_1_4, worker_1, False ), - dag_idx_2_1: generate_dag_graph_nodes( - local_idx_2_1, dag_idx_2_1, worker_2, False + task_idx_2_1: generate_dag_graph_nodes( + exec_task_idx_2_1, task_idx_2_1, worker_2, False ), - dag_idx_2_2: generate_dag_graph_nodes( - local_idx_2_2, dag_idx_2_2, worker_2, False + task_idx_2_2: generate_dag_graph_nodes( + exec_task_idx_2_2, task_idx_2_2, worker_2, False ), - dag_idx_2_3: generate_dag_graph_nodes( - local_idx_2_3, dag_idx_2_3, worker_2, False + task_idx_2_3: generate_dag_graph_nodes( + exec_task_idx_2_3, task_idx_2_3, worker_2, False ), - dag_idx_2_4: generate_dag_graph_nodes( - local_idx_2_4, dag_idx_2_4, worker_2, False + task_idx_2_4: generate_dag_graph_nodes( + exec_task_idx_2_4, task_idx_2_4, worker_2, False ), } - self.add_edge_between_read_compute_write(graph[dag_idx_1_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_2]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_3]) - self.add_edge_between_read_compute_write(graph[dag_idx_1_4]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_1]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_2]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_3]) - self.add_edge_between_read_compute_write(graph[dag_idx_2_4]) - self.add_data_dependeny(graph[dag_idx_1_1], graph[dag_idx_2_1]) - self.add_data_dependeny(graph[dag_idx_2_1], graph[dag_idx_2_2]) - self.add_data_dependeny(graph[dag_idx_2_2], graph[dag_idx_1_3]) - self.add_data_dependeny(graph[dag_idx_1_2], graph[dag_idx_2_3]) - self.add_data_dependeny(graph[dag_idx_2_3], graph[dag_idx_2_4]) - self.add_data_dependeny(graph[dag_idx_2_4], graph[dag_idx_1_4]) - self.add_control_dependency(graph[dag_idx_1_1], graph[dag_idx_1_2]) - self.add_control_dependency(graph[dag_idx_1_2], graph[dag_idx_1_3]) - self.add_control_dependency(graph[dag_idx_1_3], graph[dag_idx_1_4]) - self.add_control_dependency(graph[dag_idx_2_1], graph[dag_idx_2_2]) - self.add_control_dependency(graph[dag_idx_2_2], graph[dag_idx_2_3]) - self.add_control_dependency(graph[dag_idx_2_3], graph[dag_idx_2_4]) + self.add_edge_between_read_compute_write(graph[task_idx_1_1]) + self.add_edge_between_read_compute_write(graph[task_idx_1_2]) + self.add_edge_between_read_compute_write(graph[task_idx_1_3]) + self.add_edge_between_read_compute_write(graph[task_idx_1_4]) + self.add_edge_between_read_compute_write(graph[task_idx_2_1]) + self.add_edge_between_read_compute_write(graph[task_idx_2_2]) + self.add_edge_between_read_compute_write(graph[task_idx_2_3]) + self.add_edge_between_read_compute_write(graph[task_idx_2_4]) + self.add_data_dependeny(graph[task_idx_1_1], graph[task_idx_2_1]) + self.add_data_dependeny(graph[task_idx_2_1], graph[task_idx_2_2]) + self.add_data_dependeny(graph[task_idx_2_2], graph[task_idx_1_3]) + self.add_data_dependeny(graph[task_idx_1_2], graph[task_idx_2_3]) + self.add_data_dependeny(graph[task_idx_2_3], graph[task_idx_2_4]) + self.add_data_dependeny(graph[task_idx_2_4], graph[task_idx_1_4]) + self.add_control_dependency(graph[task_idx_1_1], graph[task_idx_1_2]) + self.add_control_dependency(graph[task_idx_1_2], graph[task_idx_1_3]) + self.add_control_dependency(graph[task_idx_1_3], graph[task_idx_1_4]) + self.add_control_dependency(graph[task_idx_2_1], graph[task_idx_2_2]) + self.add_control_dependency(graph[task_idx_2_2], graph[task_idx_2_3]) + self.add_control_dependency(graph[task_idx_2_3], graph[task_idx_2_4]) actor_to_execution_schedule = _generate_actor_to_execution_schedule(graph) assert len(actor_to_execution_schedule) == 2 assert len(actor_to_execution_schedule[worker_1]) == 12 assert len(actor_to_execution_schedule[worker_2]) == 12 assert actor_to_execution_schedule[worker_1] == [ - graph[dag_idx_1_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_2][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_3][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_3][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_3][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_1_4][_DAGNodeOperationType.READ].operation, - graph[dag_idx_1_4][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_1_4][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_2][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_3][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_3][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_3][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_1_4][_DAGNodeOperationType.READ].operation, + graph[task_idx_1_4][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_1_4][_DAGNodeOperationType.WRITE].operation, ] assert actor_to_execution_schedule[worker_2] == [ - graph[dag_idx_2_1][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_1][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, - # The order of `dag_idx_2_3.READ` and `dag_idx_2_2.WRITE` is important. + graph[task_idx_2_1][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_1][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_1][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_2][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_2][_DAGNodeOperationType.COMPUTE].operation, + # The order of `task_idx_2_3.READ` and `task_idx_2_2.WRITE` is important. # It is different from the case where there is an NCCL operation. - graph[dag_idx_2_2][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_3][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_3][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_3][_DAGNodeOperationType.WRITE].operation, - graph[dag_idx_2_4][_DAGNodeOperationType.READ].operation, - graph[dag_idx_2_4][_DAGNodeOperationType.COMPUTE].operation, - graph[dag_idx_2_4][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_2][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_3][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_3][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_3][_DAGNodeOperationType.WRITE].operation, + graph[task_idx_2_4][_DAGNodeOperationType.READ].operation, + graph[task_idx_2_4][_DAGNodeOperationType.COMPUTE].operation, + graph[task_idx_2_4][_DAGNodeOperationType.WRITE].operation, ] From 84fde99c0305abcee1be2b86bd8d66ab0dffdacb Mon Sep 17 00:00:00 2001 From: Kai-Hsun Chen Date: Wed, 16 Oct 2024 15:23:41 -0700 Subject: [PATCH 44/46] [core] Add an example for `RAY_DEDUP_LOGS_SKIP_REGEX` (#47913) Signed-off-by: kaihsun --- .../config/vocabularies/General/accept.txt | 2 + .../user-guides/configure-logging.md | 61 ++++++++++++++++--- 2 files changed, 55 insertions(+), 8 deletions(-) diff --git a/.vale/styles/config/vocabularies/General/accept.txt b/.vale/styles/config/vocabularies/General/accept.txt index df704e70a2721..9ffc6965e08cd 100644 --- a/.vale/styles/config/vocabularies/General/accept.txt +++ b/.vale/styles/config/vocabularies/General/accept.txt @@ -17,3 +17,5 @@ GKE namespace ARM breakpoint +deduplicate[s] +deduplication diff --git a/doc/source/ray-observability/user-guides/configure-logging.md b/doc/source/ray-observability/user-guides/configure-logging.md index 73c6102ab46ad..3be1af34cbffb 100644 --- a/doc/source/ray-observability/user-guides/configure-logging.md +++ b/doc/source/ray-observability/user-guides/configure-logging.md @@ -62,7 +62,7 @@ System logs may include information about your applications. For example, ``runt This is the log file of the agent containing logs of create or delete requests and cache hits and misses. For the logs of the actual installations (for example, ``pip install`` logs), see the ``runtime_env_setup-[job_id].log`` file (see below). - ``runtime_env_setup-ray_client_server_[port].log``: Logs from installing {ref}`Runtime Environments ` for a job when connecting with {ref}`Ray Client `. -- ``runtime_env_setup-[job_id].log``: Logs from installing {ref}`Runtime Environments ` for a Task, Actor or Job. This file is only present if a Runtime Environment is installed. +- ``runtime_env_setup-[job_id].log``: Logs from installing {ref}`runtime environments ` for a Task, Actor, or Job. This file is only present if you install a runtime environment. (log-redirection-to-driver)= @@ -136,13 +136,58 @@ The output is as follows: (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. 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. - +This feature is useful when importing libraries such as `tensorflow` or `numpy`, which may emit many verbose warning messages when you import them. + +Configure the following environment variables on the driver process **before importing Ray** to customize log deduplication: + +* Set ``RAY_DEDUP_LOGS=0`` to turn off this feature entirely. +* Set ``RAY_DEDUP_LOGS_AGG_WINDOW_S=`` to change the aggregation window. +* Set ``RAY_DEDUP_LOGS_ALLOW_REGEX=`` to specify log messages to never deduplicate. + * Example: + ```python + import os + os.environ["RAY_DEDUP_LOGS_ALLOW_REGEX"] = "ABC" + + import ray + + @ray.remote + def f(): + print("ABC") + print("DEF") + + ray.init() + ray.get([f.remote() for _ in range(5)]) + + # 2024-10-10 17:54:19,095 INFO worker.py:1614 -- Connecting to existing Ray cluster at address: 172.31.13.10:6379... + # 2024-10-10 17:54:19,102 INFO worker.py:1790 -- Connected to Ray cluster. View the dashboard at 127.0.0.1:8265 + # (f pid=1574323) ABC + # (f pid=1574323) DEF + # (f pid=1574321) ABC + # (f pid=1574318) ABC + # (f pid=1574320) ABC + # (f pid=1574322) ABC + # (f pid=1574322) DEF [repeated 4x across cluster] (Ray deduplicates logs by default. Set RAY_DEDUP_LOGS=0 to disable log deduplication, or see https://docs.ray.io/en/master/ray-observability/user-guides/configure-logging.html#log-deduplication for more options.) + ``` +* Set ``RAY_DEDUP_LOGS_SKIP_REGEX=`` to specify log messages to skip printing. + * Example: + ```python + import os + os.environ["RAY_DEDUP_LOGS_SKIP_REGEX"] = "ABC" + + import ray + + @ray.remote + def f(): + print("ABC") + print("DEF") + + ray.init() + ray.get([f.remote() for _ in range(5)]) + # 2024-10-10 17:55:05,308 INFO worker.py:1614 -- Connecting to existing Ray cluster at address: 172.31.13.10:6379... + # 2024-10-10 17:55:05,314 INFO worker.py:1790 -- Connected to Ray cluster. View the dashboard at 127.0.0.1:8265 + # (f pid=1574317) DEF + # (f pid=1575229) DEF [repeated 4x across cluster] (Ray deduplicates logs by default. Set RAY_DEDUP_LOGS=0 to disable log deduplication, or see https://docs.ray.io/en/master/ray-observability/user-guides/configure-logging.html#log-deduplication for more options.) + ``` ## Distributed progress bars (tqdm) From 059f4bad42ca03ddb1318bd2b41b5553024ef357 Mon Sep 17 00:00:00 2001 From: Matthew Owen Date: Wed, 16 Oct 2024 16:58:11 -0700 Subject: [PATCH 45/46] [Data] Adding in better framework for substituting logging handlers (#48056) ## Why are these changes needed? This provides a more extensible framework around substituting logging handlers within the ray data logging configuration. This will allow more substitutions to be added with more ease and less cruft. ## Related issue number ## Checks - [ ] I've signed off every commit(by using the -s flag, i.e., `git commit -s`) in this PR. - [ ] I've run `scripts/format.sh` to lint the changes in this PR. - [ ] I've included any doc changes needed for https://docs.ray.io/en/master/. - [ ] I've added any new APIs to the API Reference. For example, if I added a method in Tune, I've added it in `doc/source/tune/api/` under the corresponding `.rst` file. - [ ] I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/ - Testing Strategy - [ ] Unit tests - [ ] Release tests - [ ] This PR is not tested :( --------- Signed-off-by: Matthew Owen --- python/ray/data/_internal/logging.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/python/ray/data/_internal/logging.py b/python/ray/data/_internal/logging.py index ae0991e4adbe5..f62357940ad38 100644 --- a/python/ray/data/_internal/logging.py +++ b/python/ray/data/_internal/logging.py @@ -11,6 +11,10 @@ os.path.join(os.path.dirname(__file__), "logging.yaml") ) +# Dictionary of substitutions to be performed when using JSON mode. Handlers with names +# corresponding to keys will be replaced by those corresponding to values. +RAY_DATA_LOG_HANDLER_JSON_SUBSTITUTIONS = {"file": "file_json"} + # Env. variable to specify the encoding of the file logs when using the default config. RAY_DATA_LOG_ENCODING_ENV_VAR_NAME = "RAY_DATA_LOG_ENCODING" @@ -118,8 +122,12 @@ def _load_logging_config(config_path: str): config = _load_logging_config(DEFAULT_CONFIG_PATH) if log_encoding is not None and log_encoding.upper() == "JSON": for logger in config["loggers"].values(): - logger["handlers"].remove("file") - logger["handlers"].append("file_json") + for ( + old_handler_name, + new_handler_name, + ) in RAY_DATA_LOG_HANDLER_JSON_SUBSTITUTIONS.items(): + logger["handlers"].remove(old_handler_name) + logger["handlers"].append(new_handler_name) logging.config.dictConfig(config) From d5fa9a04ed841ea845887f43e06a0d2a81216c2d Mon Sep 17 00:00:00 2001 From: Jiajun Yao Date: Wed, 16 Oct 2024 17:29:55 -0700 Subject: [PATCH 46/46] [Core][RFC] RPC network error chaos test framework (#48007) Signed-off-by: Jiajun Yao --- BUILD.bazel | 15 +++ python/ray/tests/test_gcs_utils.py | 14 +++ src/ray/common/ray_config_def.h | 5 + src/ray/core_worker/core_worker_process.cc | 1 + src/ray/gcs/gcs_server/gcs_server_main.cc | 1 + src/ray/raylet/main.cc | 1 + src/ray/rpc/grpc_client.h | 52 +++++++--- src/ray/rpc/rpc_chaos.cc | 109 +++++++++++++++++++++ src/ray/rpc/rpc_chaos.h | 37 +++++++ src/ray/rpc/test/rpc_chaos_test.cc | 34 +++++++ 10 files changed, 258 insertions(+), 11 deletions(-) create mode 100644 src/ray/rpc/rpc_chaos.cc create mode 100644 src/ray/rpc/rpc_chaos.h create mode 100644 src/ray/rpc/test/rpc_chaos_test.cc diff --git a/BUILD.bazel b/BUILD.bazel index 83660e6aa8cad..7f497e3f6de1f 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -93,8 +93,10 @@ ray_cc_library( "src/ray/rpc/common.cc", "src/ray/rpc/grpc_server.cc", "src/ray/rpc/server_call.cc", + "src/ray/rpc/rpc_chaos.cc", ], hdrs = glob([ + "src/ray/rpc/rpc_chaos.h", "src/ray/rpc/client_call.h", "src/ray/rpc/common.h", "src/ray/rpc/grpc_client.h", @@ -1551,6 +1553,19 @@ ray_cc_test( ], ) +ray_cc_test( + name = "rpc_chaos_test", + size = "small", + srcs = [ + "src/ray/rpc/test/rpc_chaos_test.cc", + ], + tags = ["team:core"], + deps = [ + ":grpc_common_lib", + "@com_google_googletest//:gtest_main", + ], +) + ray_cc_test( name = "core_worker_client_pool_test", size = "small", diff --git a/python/ray/tests/test_gcs_utils.py b/python/ray/tests/test_gcs_utils.py index 82f34214046ef..c25beac6e598a 100644 --- a/python/ray/tests/test_gcs_utils.py +++ b/python/ray/tests/test_gcs_utils.py @@ -100,6 +100,20 @@ def test_kv_timeout(ray_start_regular): gcs_client.internal_kv_del(b"A", True, b"NS", timeout=2) +def test_kv_transient_network_error(shutdown_only, monkeypatch): + monkeypatch.setenv( + "RAY_testing_rpc_failure", + "ray::rpc::InternalKVGcsService.grpc_client.InternalKVGet=5," + "ray::rpc::InternalKVGcsService.grpc_client.InternalKVPut=5", + ) + ray.init() + gcs_address = ray._private.worker.global_worker.gcs_client.address + gcs_client = ray._raylet.GcsClient(address=gcs_address, nums_reconnect_retry=0) + + gcs_client.internal_kv_put(b"A", b"Hello", True, b"") + assert gcs_client.internal_kv_get(b"A", b"") == b"Hello" + + @pytest.mark.asyncio async def test_kv_basic_aio(ray_start_regular): gcs_client = gcs_utils.GcsAioClient( diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index bcc960158a078..674895d4b3892 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -835,6 +835,11 @@ RAY_CONFIG(std::string, REDIS_SERVER_NAME, "") // it will apply to all methods. RAY_CONFIG(std::string, testing_asio_delay_us, "") +/// To use this, simply do +/// export +/// RAY_testing_rpc_failure="method1=max_num_failures,method2=max_num_failures" +RAY_CONFIG(std::string, testing_rpc_failure, "") + /// The following are configs for the health check. They are borrowed /// from k8s health probe (shorturl.at/jmTY3) /// The delay to send the first health check. diff --git a/src/ray/core_worker/core_worker_process.cc b/src/ray/core_worker/core_worker_process.cc index 191788e7e0458..30042635dee7c 100644 --- a/src/ray/core_worker/core_worker_process.cc +++ b/src/ray/core_worker/core_worker_process.cc @@ -238,6 +238,7 @@ void CoreWorkerProcessImpl::InitializeSystemConfig() { RayConfig::instance().initialize(promise.get_future().get()); ray::asio::testing::init(); + ray::rpc::testing::init(); } void CoreWorkerProcessImpl::RunWorkerTaskExecutionLoop() { diff --git a/src/ray/gcs/gcs_server/gcs_server_main.cc b/src/ray/gcs/gcs_server/gcs_server_main.cc index 137efbaf9dd5d..18d7b83d896e4 100644 --- a/src/ray/gcs/gcs_server/gcs_server_main.cc +++ b/src/ray/gcs/gcs_server/gcs_server_main.cc @@ -70,6 +70,7 @@ int main(int argc, char *argv[]) { RayConfig::instance().initialize(config_list); ray::asio::testing::init(); + ray::rpc::testing::init(); // IO Service for main loop. instrumented_io_context main_service; diff --git a/src/ray/raylet/main.cc b/src/ray/raylet/main.cc index 8cc0c1f08ef1a..2b30f9068b6b6 100644 --- a/src/ray/raylet/main.cc +++ b/src/ray/raylet/main.cc @@ -275,6 +275,7 @@ int main(int argc, char *argv[]) { RAY_CHECK(stored_raylet_config.has_value()); RayConfig::instance().initialize(*stored_raylet_config); ray::asio::testing::init(); + ray::rpc::testing::init(); // Core worker tries to kill child processes when it exits. But they can't do // it perfectly: if the core worker is killed by SIGKILL, the child processes diff --git a/src/ray/rpc/grpc_client.h b/src/ray/rpc/grpc_client.h index 9037613968ceb..d86475fbdf6b5 100644 --- a/src/ray/rpc/grpc_client.h +++ b/src/ray/rpc/grpc_client.h @@ -23,6 +23,7 @@ #include "ray/common/status.h" #include "ray/rpc/client_call.h" #include "ray/rpc/common.h" +#include "ray/rpc/rpc_chaos.h" namespace ray { namespace rpc { @@ -148,15 +149,43 @@ class GrpcClient { const ClientCallback &callback, std::string call_name = "UNKNOWN_RPC", int64_t method_timeout_ms = -1) { - auto call = client_call_manager_.CreateCall( - *stub_, - prepare_async_function, - request, - callback, - std::move(call_name), - method_timeout_ms); - RAY_CHECK(call != nullptr); - call_method_invoked_ = true; + testing::RpcFailure failure = testing::get_rpc_failure(call_name); + if (failure == testing::RpcFailure::Request) { + // Simulate the case where the PRC fails before server receives + // the request. + RAY_LOG(INFO) << "Inject RPC request failure for " << call_name; + client_call_manager_.GetMainService().post( + [callback]() { + callback(Status::RpcError("Unavailable", grpc::StatusCode::UNAVAILABLE), + Reply()); + }, + "RpcChaos"); + } else if (failure == testing::RpcFailure::Response) { + // Simulate the case where the RPC fails after server sends + // the response. + RAY_LOG(INFO) << "Inject RPC response failure for " << call_name; + client_call_manager_.CreateCall( + *stub_, + prepare_async_function, + request, + [callback](const Status &status, Reply &&reply) { + callback(Status::RpcError("Unavailable", grpc::StatusCode::UNAVAILABLE), + Reply()); + }, + std::move(call_name), + method_timeout_ms); + } else { + auto call = client_call_manager_.CreateCall( + *stub_, + prepare_async_function, + request, + callback, + std::move(call_name), + method_timeout_ms); + RAY_CHECK(call != nullptr); + } + + call_method_invoked_.store(true); } std::shared_ptr Channel() const { return channel_; } @@ -167,7 +196,8 @@ class GrpcClient { /// Also see https://grpc.github.io/grpc/core/md_doc_connectivity-semantics-and-api.html /// for channel connectivity state machine. bool IsChannelIdleAfterRPCs() const { - return (channel_->GetState(false) == GRPC_CHANNEL_IDLE) && call_method_invoked_; + return (channel_->GetState(false) == GRPC_CHANNEL_IDLE) && + call_method_invoked_.load(); } private: @@ -179,7 +209,7 @@ class GrpcClient { /// The channel of the stub. std::shared_ptr channel_; /// Whether CallMethod is invoked. - bool call_method_invoked_ = false; + std::atomic call_method_invoked_ = false; }; } // namespace rpc diff --git a/src/ray/rpc/rpc_chaos.cc b/src/ray/rpc/rpc_chaos.cc new file mode 100644 index 0000000000000..373e3a9be60f0 --- /dev/null +++ b/src/ray/rpc/rpc_chaos.cc @@ -0,0 +1,109 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "ray/rpc/rpc_chaos.h" + +#include +#include + +#include "absl/synchronization/mutex.h" +#include "ray/common/ray_config.h" + +namespace ray { +namespace rpc { +namespace testing { +namespace { + +/* + RpcFailureManager is a simple chaos testing framework. Before starting ray, users + should set up os environment to use this feature for testing purposes. + To use this, simply do + export RAY_testing_rpc_failure="method1=3,method2=5" + Key is the RPC call name and value is the max number of failures to inject. +*/ +class RpcFailureManager { + public: + RpcFailureManager() { Init(); } + + void Init() { + absl::MutexLock lock(&mu_); + + failable_methods_.clear(); + + if (!RayConfig::instance().testing_rpc_failure().empty()) { + for (const auto &item : + absl::StrSplit(RayConfig::instance().testing_rpc_failure(), ",")) { + std::vector parts = absl::StrSplit(item, "="); + RAY_CHECK_EQ(parts.size(), 2UL); + failable_methods_.emplace(parts[0], std::atoi(parts[1].c_str())); + } + + std::random_device rd; + auto seed = rd(); + RAY_LOG(INFO) << "Setting RpcFailureManager seed to " << seed; + gen_.seed(seed); + } + } + + RpcFailure GetRpcFailure(const std::string &name) { + absl::MutexLock lock(&mu_); + + if (failable_methods_.find(name) == failable_methods_.end()) { + return RpcFailure::None; + } + + uint64_t &num_remaining_failures = failable_methods_.at(name); + if (num_remaining_failures == 0) { + return RpcFailure::None; + } + + std::uniform_int_distribution dist(0, 3); + int rand = dist(gen_); + if (rand == 0) { + // 25% chance + num_remaining_failures--; + return RpcFailure::Request; + } else if (rand == 1) { + // 25% chance + num_remaining_failures--; + return RpcFailure::Response; + } else { + // 50% chance + return RpcFailure::None; + } + } + + private: + absl::Mutex mu_; + std::mt19937 gen_; + // call name -> # remaining failures + std::unordered_map failable_methods_ ABSL_GUARDED_BY(&mu_); +}; + +static RpcFailureManager _rpc_failure_manager; + +} // namespace + +RpcFailure get_rpc_failure(const std::string &name) { + if (RayConfig::instance().testing_rpc_failure().empty()) { + return RpcFailure::None; + } + return _rpc_failure_manager.GetRpcFailure(name); +} + +void init() { _rpc_failure_manager.Init(); } + +} // namespace testing +} // namespace rpc +} // namespace ray diff --git a/src/ray/rpc/rpc_chaos.h b/src/ray/rpc/rpc_chaos.h new file mode 100644 index 0000000000000..cb0e614eead9f --- /dev/null +++ b/src/ray/rpc/rpc_chaos.h @@ -0,0 +1,37 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace ray { +namespace rpc { +namespace testing { + +enum class RpcFailure { + None, + // Failure before server receives the request + Request, + // Failure after server sends the response + Response, +}; + +RpcFailure get_rpc_failure(const std::string &name); + +void init(); + +} // namespace testing +} // namespace rpc +} // namespace ray diff --git a/src/ray/rpc/test/rpc_chaos_test.cc b/src/ray/rpc/test/rpc_chaos_test.cc new file mode 100644 index 0000000000000..75bced2592537 --- /dev/null +++ b/src/ray/rpc/test/rpc_chaos_test.cc @@ -0,0 +1,34 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "ray/rpc/rpc_chaos.h" + +#include + +#include "gtest/gtest.h" +#include "ray/common/ray_config.h" + +TEST(RpcChaosTest, Basic) { + RayConfig::instance().testing_rpc_failure() = "method1=0,method2=1"; + ray::rpc::testing::init(); + ASSERT_EQ(ray::rpc::testing::get_rpc_failure("unknown"), + ray::rpc::testing::RpcFailure::None); + ASSERT_EQ(ray::rpc::testing::get_rpc_failure("method1"), + ray::rpc::testing::RpcFailure::None); + // At most one failure. + ASSERT_FALSE(ray::rpc::testing::get_rpc_failure("method2") != + ray::rpc::testing::RpcFailure::None && + ray::rpc::testing::get_rpc_failure("method2") != + ray::rpc::testing::RpcFailure::None); +}