From 03a7f066acfdb1f75829f2896f55842413aa9069 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=BB=91=E9=A9=B0?= Date: Tue, 21 Jan 2025 12:03:22 +0800 Subject: [PATCH] add virtual cluster MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: 黑驰 --- BUILD.bazel | 30 + doc/source/index.rst | 1 + doc/source/virtual-cluster/cli.rst | 401 ++++++ .../virtual-cluster/design-overview.rst | 170 +++ doc/source/virtual-cluster/examples.rst | 314 +++++ .../virtual-cluster/getting-started.rst | 40 + .../images/lifecycle-diagram.svg | 1 + .../images/ray-virtual-cluster.png | Bin 0 -> 222060 bytes doc/source/virtual-cluster/key-concepts.rst | 64 + doc/source/virtual-cluster/management.rst | 168 +++ python/ray/_private/ray_constants.py | 2 + python/ray/_private/state.py | 70 +- python/ray/_private/worker.py | 4 + python/ray/_raylet.pyx | 1 + python/ray/dashboard/modules/job/cli.py | 22 + python/ray/dashboard/modules/job/common.py | 94 +- python/ray/dashboard/modules/job/job_agent.py | 2 + .../ray/dashboard/modules/job/job_manager.py | 107 +- .../dashboard/modules/job/job_supervisor.py | 15 + python/ray/dashboard/modules/job/sdk.py | 5 + .../tests/test_job_with_virtual_cluster.py | 1030 +++++++++++++++ .../ray/dashboard/modules/state/state_head.py | 6 + .../modules/virtual_cluster/__init__.py | 0 .../tests/test_virtual_cluster.py | 621 +++++++++ .../virtual_cluster/virtual_cluster_head.py | 137 ++ python/ray/dashboard/state_aggregator.py | 173 +++ python/ray/data/BUILD | 8 + python/ray/data/tests/conftest.py | 120 +- .../tests/test_strict_in_virtual_cluster.py | 421 ++++++ python/ray/includes/global_state_accessor.pxd | 7 +- python/ray/includes/global_state_accessor.pxi | 28 +- python/ray/includes/unique_ids.pxd | 27 + python/ray/includes/unique_ids.pxi | 51 + python/ray/runtime_context.py | 5 + python/ray/scripts/scripts.py | 312 +++++ python/ray/tests/conftest.py | 49 + python/ray/tests/test_state_api.py | 1 + python/ray/util/state/api.py | 52 +- python/ray/util/state/common.py | 25 + python/ray/util/state/state_manager.py | 20 + src/mock/ray/gcs/gcs_client/accessor.h | 30 +- src/mock/ray/gcs/gcs_client/gcs_client.h | 3 + src/ray/common/BUILD | 3 + src/ray/common/constants.h | 6 + src/ray/common/placement_group.h | 6 + src/ray/common/ray_config_def.ant.h | 30 + src/ray/common/ray_config_def.h | 2 + src/ray/common/simple_id.h | 67 + src/ray/common/status.cc | 1 + src/ray/common/status.h | 8 + src/ray/common/task/task_util.h | 16 + src/ray/common/virtual_cluster_id.h | 72 ++ src/ray/core_worker/core_worker.cc | 7 +- src/ray/gcs/gcs_client/accessor.ant.cc | 153 +++ src/ray/gcs/gcs_client/accessor.cc | 36 + src/ray/gcs/gcs_client/accessor.h | 67 + src/ray/gcs/gcs_client/gcs_client.cc | 5 +- src/ray/gcs/gcs_client/gcs_client.h | 6 + .../gcs/gcs_client/global_state_accessor.cc | 43 +- .../gcs/gcs_client/global_state_accessor.h | 12 +- .../gcs/gcs_client/test/gcs_client_test.cc | 1 + src/ray/gcs/gcs_server/gcs_actor_manager.cc | 36 +- src/ray/gcs/gcs_server/gcs_actor_manager.h | 36 + src/ray/gcs/gcs_server/gcs_init_data.cc | 18 +- src/ray/gcs/gcs_server/gcs_init_data.h | 15 + src/ray/gcs/gcs_server/gcs_job_manager.cc | 13 + src/ray/gcs/gcs_server/gcs_job_manager.h | 4 + src/ray/gcs/gcs_server/gcs_node_manager.cc | 28 +- src/ray/gcs/gcs_server/gcs_node_manager.h | 6 +- .../gcs_server/gcs_placement_group_manager.cc | 12 + .../gcs_server/gcs_placement_group_manager.h | 30 + .../gcs_placement_group_scheduler.cc | 18 +- .../gcs_placement_group_scheduler.h | 3 +- .../gcs/gcs_server/gcs_resource_manager.cc | 40 +- src/ray/gcs/gcs_server/gcs_resource_manager.h | 4 + src/ray/gcs/gcs_server/gcs_server.ant.cc | 37 + src/ray/gcs/gcs_server/gcs_server.cc | 74 +- src/ray/gcs/gcs_server/gcs_server.h | 7 + src/ray/gcs/gcs_server/gcs_table_storage.cc | 1 + src/ray/gcs/gcs_server/gcs_table_storage.h | 18 + src/ray/gcs/gcs_server/gcs_virtual_cluster.cc | 1139 +++++++++++++++++ src/ray/gcs/gcs_server/gcs_virtual_cluster.h | 575 +++++++++ .../gcs_server/gcs_virtual_cluster_manager.cc | 556 ++++++++ .../gcs_server/gcs_virtual_cluster_manager.h | 138 ++ .../gcs_server/test/gcs_server_rpc_test.cc | 11 + .../test/gcs_virtual_cluster_manager_test.cc | 1121 ++++++++++++++++ src/ray/gcs/pubsub/gcs_pub_sub.ant.cc | 63 + src/ray/gcs/pubsub/gcs_pub_sub.h | 9 + src/ray/protobuf/common.proto | 4 + src/ray/protobuf/gcs.proto | 25 + src/ray/protobuf/gcs_service.proto | 129 +- src/ray/protobuf/pubsub.proto | 3 + src/ray/protobuf/usage.proto | 1 + src/ray/pubsub/publisher.cc | 1 + src/ray/raylet/format/node_manager.fbs | 2 + src/ray/raylet/node_manager.ant.cc | 74 ++ src/ray/raylet/node_manager.cc | 41 +- src/ray/raylet/node_manager.h | 11 + .../scheduling/cluster_resource_scheduler.cc | 43 +- .../scheduling/cluster_resource_scheduler.h | 21 +- .../affinity_with_bundle_scheduling_policy.cc | 22 +- .../affinity_with_bundle_scheduling_policy.h | 11 +- .../policy/bundle_scheduling_policy.cc | 7 + .../policy/bundle_scheduling_policy.h | 7 +- .../policy/composite_scheduling_policy.h | 55 +- .../policy/hybrid_scheduling_policy.cc | 15 +- .../policy/hybrid_scheduling_policy.h | 17 +- .../policy/node_affinity_scheduling_policy.cc | 1 + .../policy/node_affinity_scheduling_policy.h | 13 +- .../policy/node_label_scheduling_policy.cc | 9 +- .../policy/node_label_scheduling_policy.h | 16 +- .../policy/random_scheduling_policy.cc | 4 +- .../policy/random_scheduling_policy.h | 14 +- .../scheduling/policy/scheduling_context.h | 1 + .../scheduling/policy/scheduling_options.h | 83 +- .../policy/spread_scheduling_policy.cc | 4 +- .../policy/spread_scheduling_policy.h | 16 +- src/ray/raylet/virtual_cluster_manager.cc | 96 ++ src/ray/raylet/virtual_cluster_manager.h | 63 + .../raylet/virtual_cluster_manager_test.cc | 85 ++ src/ray/raylet_client/raylet_client.cc | 7 +- src/ray/raylet_client/raylet_client.h | 4 +- src/ray/rpc/gcs_server/gcs_rpc_client.h | 36 + src/ray/rpc/gcs_server/gcs_rpc_server.h | 60 + src/ray/util/array.h | 17 +- src/ray/util/logging.h | 1 + src/ray/util/util.h | 7 + 127 files changed, 10018 insertions(+), 198 deletions(-) create mode 100644 doc/source/virtual-cluster/cli.rst create mode 100644 doc/source/virtual-cluster/design-overview.rst create mode 100644 doc/source/virtual-cluster/examples.rst create mode 100644 doc/source/virtual-cluster/getting-started.rst create mode 100644 doc/source/virtual-cluster/images/lifecycle-diagram.svg create mode 100644 doc/source/virtual-cluster/images/ray-virtual-cluster.png create mode 100644 doc/source/virtual-cluster/key-concepts.rst create mode 100644 doc/source/virtual-cluster/management.rst create mode 100644 python/ray/dashboard/modules/job/tests/test_job_with_virtual_cluster.py create mode 100644 python/ray/dashboard/modules/virtual_cluster/__init__.py create mode 100644 python/ray/dashboard/modules/virtual_cluster/tests/test_virtual_cluster.py create mode 100644 python/ray/dashboard/modules/virtual_cluster/virtual_cluster_head.py create mode 100644 python/ray/data/tests/test_strict_in_virtual_cluster.py create mode 100644 src/ray/common/ray_config_def.ant.h create mode 100644 src/ray/common/simple_id.h create mode 100644 src/ray/common/virtual_cluster_id.h create mode 100644 src/ray/gcs/gcs_client/accessor.ant.cc create mode 100644 src/ray/gcs/gcs_server/gcs_server.ant.cc create mode 100644 src/ray/gcs/gcs_server/gcs_virtual_cluster.cc create mode 100644 src/ray/gcs/gcs_server/gcs_virtual_cluster.h create mode 100644 src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.cc create mode 100644 src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.h create mode 100644 src/ray/gcs/gcs_server/test/gcs_virtual_cluster_manager_test.cc create mode 100644 src/ray/gcs/pubsub/gcs_pub_sub.ant.cc create mode 100644 src/ray/raylet/node_manager.ant.cc create mode 100644 src/ray/raylet/virtual_cluster_manager.cc create mode 100644 src/ray/raylet/virtual_cluster_manager.h create mode 100644 src/ray/raylet/virtual_cluster_manager_test.cc diff --git a/BUILD.bazel b/BUILD.bazel index 5ed81f23b102..75689fbcd290 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -493,6 +493,7 @@ ray_cc_library( name = "gcs_pub_sub_lib", srcs = [ "src/ray/gcs/pubsub/gcs_pub_sub.cc", + "src/ray/gcs/pubsub/gcs_pub_sub.ant.cc", ], hdrs = [ "src/ray/gcs/pubsub/gcs_pub_sub.h", @@ -1833,6 +1834,34 @@ ray_cc_test( ], ) +ray_cc_test( + name = "gcs_virtual_cluster_manager_test", + size = "small", + srcs = [ + "src/ray/gcs/gcs_server/test/gcs_virtual_cluster_manager_test.cc", + ], + tags = ["team:core"], + deps = [ + ":gcs_server_lib", + ":gcs_server_test_util", + ":gcs_test_util_lib", + ":ray_mock", + "@com_google_googletest//:gtest_main", + ], +) + +ray_cc_test( + name = "virtual_cluster_manager_test", + size = "small", + srcs = ["src/ray/raylet/virtual_cluster_manager_test.cc"], + tags = ["team:core"], + deps = [ + ":ray_mock", + ":raylet_lib", + "@com_google_googletest//:gtest_main", + ], +) + ray_cc_library( name = "gcs_table_storage_lib", srcs = glob( @@ -1956,6 +1985,7 @@ ray_cc_library( name = "gcs_client_lib", srcs = [ "src/ray/gcs/gcs_client/accessor.cc", + "src/ray/gcs/gcs_client/accessor.ant.cc", "src/ray/gcs/gcs_client/gcs_client.cc", ], hdrs = [ diff --git a/doc/source/index.rst b/doc/source/index.rst index cf1db0e55e07..d4dcbb9cdb8e 100644 --- a/doc/source/index.rst +++ b/doc/source/index.rst @@ -19,6 +19,7 @@ Ray RLlib More Libraries Ray Clusters + Ray Virtual Clusters Monitoring and Debugging Developer Guides Glossary diff --git a/doc/source/virtual-cluster/cli.rst b/doc/source/virtual-cluster/cli.rst new file mode 100644 index 000000000000..aab062b1a491 --- /dev/null +++ b/doc/source/virtual-cluster/cli.rst @@ -0,0 +1,401 @@ +Virtual Cluster Management +=========================== + +.. _virtual-cluster-cli: + +Create a Virtual Cluster +------------------------ + +**Description:** + +Creates a new Virtual Cluster with specified parameters, allowing you to define clusters with specific replica distributions and divisibility settings. + +**Command:** + +`ray vcluster create [OPTIONS]` + +**Options:** + +.. list-table:: + :widths: 20 16 16 16 60 + :header-rows: 1 + + * - Option + - Type + - Default + - Required + - Description + * - --address TEXT + - str + - None + - NO + - Specifies the Ray cluster address. If not provided, the **RAY_ADDRESS** environment variable is used. + * - --id TEXT + - str + - N/A + - YES + - Assigns a unique identifier to the Virtual Cluster being created. + * - --divisible + - bool + - False + - NO + - Determines if the Virtual Cluster is divisible into smaller logical or job clusters. + * - --replica-sets TEXT + - dict + - N/A + - YES + - JSON-serialized dictionary defining the replica sets for the cluster (e.g., `{"group1":2,"group2":3}`, `group1` and `group2` correspond to the node type names passed via the `RAY_NODE_TYPE_NAME` environment variable). + +Usage Examples +~~~~~~~~~~~~~~ + +**Example:** Creating a Divisible Virtual Cluster + +`ray vcluster create --id logical1 --divisible --replica-sets '{"group2":1}'` + +**Output:** + +`Virtual cluster 'logical1' created successfully` + +Update a Virtual Cluster +------------------------ + +**Description:** + +Update an existing Virtual Cluster with specified parameters. + +**Command:** + +`ray vcluster update [OPTIONS]` + +**Options:** + +.. list-table:: + :widths: 20 16 16 16 60 + :header-rows: 1 + + * - Option + - Type + - Default + - Required + - Description + * - --address TEXT + - str + - None + - NO + - Specifies the Ray cluster address. If not provided, the **RAY_ADDRESS** environment variable is used. + * - --id TEXT + - str + - N/A + - YES + - Assigns a unique identifier to the Virtual Cluster being created. + * - --divisible + - bool + - False + - NO + - Determines if the Virtual Cluster is divisible into smaller logical or job clusters. + * - --replica-sets TEXT + - dict + - N/A + - YES + - JSON-serialized dictionary defining the replica sets for the cluster (e.g., `{"group1":2,"group2":3}`, `group1` and `group2` correspond to the node type names passed via the `RAY_NODE_TYPE_NAME` environment variable). + * - --revision INTEGER + - int + - 0 + - NO + - Indicates the revision number for updating the Virtual Cluster. + +Usage Examples +~~~~~~~~~~~~~~ + +**Example 1:** Updating a Divisible Virtual Cluster + +`ray vcluster update --id logical1 --divisible --replica-sets '{"group2":2}'` + +**Output:** + +`Virtual cluster 'logical1' updated successfully` + +**Example 2:** Handling Updating Failure Due to Incorrect Revision + +`ray vcluster update --id logical1 --divisible --replica-sets '{"group1":2}' --revision 2` + +**Output:** + +`Failed to update virtual cluster 'logical1': The revision (2) is expired, the latest revision of the virtual cluster logical1 is 1736911613521214948` + +Remove a Virtual Cluster +------------------------ + +**Description:** + +Removes an existing Virtual Cluster by its unique identifier from your Ray environment. + +**Command:** + +`ray vcluster remove [OPTIONS] ` + +**Options:** + +.. list-table:: + :widths: 20 16 16 16 60 + :header-rows: 1 + + * - Option + - Type + - Default + - Required + - Description + * - --address TEXT + - str + - None + - NO + - Specifies the Ray cluster address. If not provided, the **RAY_ADDRESS** environment variable is used. + * - + - str + - N/A + - YES + - The unique identifier of the Virtual Cluster to be removed. + +**Usage Example:** + +**Example 1:** Removing a Virtual Cluster by ID + +`ray vcluster remove logical1` + +**Output:** + +`Virtual cluster 'logical1' removed successfully` + +**Example 2:** Handling Removal Failure Due to Non-Existent ID + +`ray vcluster remove unknownCluster` + +**Output:** + +`Failed to remove virtual cluster 'unknownCluster': The logical cluster unknownCluster does not exist.` + +List Virtual Clusters +--------------------- + +**Description:** + +Displays a summary of all Virtual Clusters in your Ray environment. By default, it presents a table listing each cluster's ID, divisibility status, and any subdivided clusters. The `--detail` flag enriches the output with comprehensive information, including replica distributions and node instance statuses. The `--format` option allows output customization in `default`, `json`, `yaml`, or `table` formats. + +**Command:** + +`ray list vclusters [OPTIONS]` + +**Options:** + +.. list-table:: + :widths: 20 16 60 + :header-rows: 1 + + * - Option + - Type + - Description + * - --format + - str + - Specify the output format: `default`, `json`, `yaml`, or `table`. + * - -f, --filter TEXT + - str + - Apply filter expressions to narrow down the list based on specific criteria. Multiple filters are combined using logical AND. + * - --limit INTEGER + - int + - Maximum number of entries to return (default: `100`). + * - --detail + - bool + - Include detailed information in the output. + * - --timeout INTEGER + - int + - Timeout in seconds for the API requests (default: `30`). + * - --address TEXT + - str + - Address of the Ray API server. If not provided, it is configured automatically. + +**Sample Output:** + +- Brief outputs: + +.. code-block:: text + + $ ray list vclusters + + ======== List: 2025-01-20 16:50:30.665928 ======== + Stats: + ------------------------------ + Total: 4 + + Table: + ------------------------------ + VIRTUAL_CLUSTER_ID DIVISIBLE DIVIDED_CLUSTERS REPLICA_SETS UNDIVIDED_REPLICA_SETS RESOURCES_USAGE + 0 kPrimaryClusterID True kPrimaryClusterID##job1: indivisible group0: 2 group0: 1 CPU: 2.0 / 41.0 + logical1: divisble group1: 1 group1: 1 memory: 2.000 GiB / 68.931 GiB + group2: 2 object_store_memory: 0.000 B / 23.793 GiB + 1 kPrimaryClusterID##job1 False {} group0: 1 group0: 1 CPU: 1.0 / 9.0 + memory: 1.000 GiB / 9.327 GiB + object_store_memory: 0.000 B / 4.663 GiB + 2 logical1 True logical1##job2: indivisible group2: 2 group2: 1 CPU: 1.0 / 16.0 + memory: 1.000 GiB / 29.802 GiB + object_store_memory: 0.000 B / 9.565 GiB + 3 logical1##job2 False {} group2: 1 group2: 1 CPU: 1.0 / 8.0 + memory: 1.000 GiB / 14.901 GiB + object_store_memory: 0.000 B / 4.783 GiB + +- Detailed outputs: + +.. code-block:: yaml + + $ ray list vclusters --detail + + --- + - virtual_cluster_id: kPrimaryClusterID + divisible: true + divided_clusters: + logical1: divisble + kPrimaryClusterID##job1: indivisible + replica_sets: + group0: 2 + group1: 1 + group2: 2 + undivided_replica_sets: + group1: 1 + group0: 1 + resources_usage: + CPU: 2.0 / 41.0 + object_store_memory: 0.000 B / 23.793 GiB + memory: 2.000 GiB / 68.931 GiB + visible_node_instances: + fe8e2961e1d7f72c8f9da7bea38ebb650cbee685f541e8ceedb2a8e3: + hostname: arconkube-40-100083029097 + template_id: group1 + is_dead: false + 740273507b09c082c33909e9134ce136d1743e0da1d5b68ec2574988: + hostname: arconkube-40-100083029138 + template_id: group0 + is_dead: false + 3505335a78b9955a1c2ed1de0a0fa92449b8011afddb621b2bab23d5: + hostname: arconkube-40-100083029093 + template_id: group0 + is_dead: false + undivided_nodes: + fe8e2961e1d7f72c8f9da7bea38ebb650cbee685f541e8ceedb2a8e3: + hostname: arconkube-40-100083029097 + template_id: group1 + is_dead: false + 740273507b09c082c33909e9134ce136d1743e0da1d5b68ec2574988: + hostname: arconkube-40-100083029138 + template_id: group0 + is_dead: false + +**Explanation:** + +- **Primary Cluster** (`kPrimaryClusterID`): + + - **Divisible:** `true` - can create sub-clusters. + - **Divided Clusters:** Includes `kPrimaryClusterID##job1` and `logical1`. + - **Replica Sets:** Distribution across `group2`, `group1`, and `group0`. + - **Visible Node Instances:** Lists active nodes with their details. + - **Undivided Nodes:** Empty, as all nodes are part of sub-clusters. + +- **Logical Cluster** (`logical1`): + + - **Divisible:** `true` - can be subdivided. + - **Replica Sets & Undivided Replica Sets:** Reflects replica distribution. + - **Visible Node Instances & Undivided Nodes:** Lists nodes associated exclusively with this logical cluster. + +**Filtering Options:** + +The `--filter` flag enables you to narrow down the list of Virtual Clusters based on specific attributes. Multiple `--filter` options can be specified, and they are concatenated using logical AND. Filter expressions support predicates such as `key=value` or `key!=value`, and string filter values are case-insensitive. + +- **Supported Filter Expressions** + + - **Divisibility:** + + - `divisible=true`: Lists only divisible clusters. + - `divisible=false`: Lists only indivisible clusters. + + - **Virtual Cluster ID:** + + - `virtual_cluster_id=vid1`: Retrieves information for the cluster with ID `vid1`. + +- **Usage Guidelines** + + - **Single Filter:** `ray list vclusters --detail --filter "divisible=true"` + - **Multiple Filters:** `ray list vclusters --detail --filter "divisible=true" --filter "virtual_cluster_id=kPrimaryClusterID"` + + **Note:** Combining multiple filters results in a logical AND operation, meaning only clusters that satisfy all filter conditions will be listed. + +Get Specific Virtual Cluster +---------------------------- + +**Description:** + +Fetches detailed information about a single Virtual Cluster identified by its `virtual_cluster_id`. + +**Command:** + +`ray get vclusters [OPTIONS]` + +**Options:** + +.. list-table:: + :widths: 20 16 60 + :header-rows: 1 + + * - Option + - Type + - Description + * - --format + - str + - Specify the output format: `default`, `json`, `yaml`, or `table`. + * - --timeout INTEGER + - int + - Timeout in seconds for the API requests (default: `30`). + * - --address TEXT + - str + - Address of the Ray API server. If not provided, it is configured automatically. + +Understanding Command Outputs +----------------------------- + +Each Virtual Cluster's information comprises several key fields: + +**Common Fields** + +- virtual_cluster_id: + + - A unique identifier for the Virtual Cluster. IDs may include suffixes (e.g., ##job1, ##logical1) indicating Job Clusters with specific job IDs or Logical Clusters. + +- divisible: + + - Indicates whether the cluster is Divisible (true) or Indivisible (false). + + - Divisible Cluster (true): Can be subdivided into Logical Clusters or Job Clusters. + - Indivisible Cluster (false): Cannot be subdivided and is used exclusively for hosting user-submitted jobs. + +- divided_clusters: + + - Lists sub-clusters that have been subdivided from the parent cluster. This field is empty for Indivisible Clusters. + +- replica_sets: + + - Details the distribution of replicas across different template groups within the cluster, excluding any inactive nodes. + +- undivided_replica_sets: + + - Similar to replica_sets but specifically for replicas not associated with any sub-cluster. + +- visible_node_instances: + + - A dictionary of visible node instances within the cluster, including: + + - Node ID: Unique identifier for each node. + - hostname: Network name of the node. + - template_id: Indicates the template group the node belongs to (e.g., group2). + - is_dead: Boolean flag indicating node status (false for active, true for inactive or failed). + +- undivided_nodes: + + - Visible nodes that are part of the cluster but not associated with any divided sub-cluster. diff --git a/doc/source/virtual-cluster/design-overview.rst b/doc/source/virtual-cluster/design-overview.rst new file mode 100644 index 000000000000..45da555a7d6a --- /dev/null +++ b/doc/source/virtual-cluster/design-overview.rst @@ -0,0 +1,170 @@ +Design Overview +=============== + +.. _virtual-cluster-design-overview: + +This page introduces the design overview of Ray virtual clusters: + +.. contents:: + :local: + +.. _virtual-cluster-management: + +Virtual Cluster Management +-------------------------- + +The metadata (state, node types and count, etc.) of virtual clusters are managed by GCS. Meanwhile, metadata has to be published to all raylets upon any changes (create/update/remove). This ensures raylets making the right scheduling decisions for tasks within the dedicated virtual cluster. + +**Create:** + +When creating a new virtual cluster, GCS approximately goes through the following steps: + +1. Pre-check: + + - The request has to specify a valid virtual cluster ID. + - The expected replica sets specified in the request have to be valid. + +2. Select enough unassigned nodes from the primary cluster. +3. Remove the selected nodes from the primary cluster, and add them to the new virtual cluster. +4. Flush the metadata of the newly created virtual cluster to Redis and publish it to all raylets. + +**Update:** + +When updating an existing virtual cluster, GCS approximately goes through the following steps: + +1. Pre-check: + + - The request has to specify a valid and existing virtual cluster ID. + - The expected replica sets specified in the request have to be valid. + - The revision specified in the request has to be the latest one. + - The request can not change the virtual cluster from divisible to indivisible (or vice-versa) on the fly. + +2. Select enough unassigned nodes from the primary cluster (if the virtual cluster expects to scale up), and select enough idle nodes to be removed from the virtual cluster (if the virtual cluster expects to scale down). +3. Update the node type and count of the primary cluster and the virtual cluster. +4. Flush the metadata of the updated virtual cluster to Redis and publish it to all raylets. + +Note, currently we do not support updating a job cluster on the fly. It is our plan to make it available in the next release. + +**Remove:** + +When removing a virtual cluster, GCS approximately goes through the following steps: + +1. Pre-check: The request has to specify a valid and existing virtual cluster ID. +2. If there is any jobs/actors/tasks/placement groups still running inside the virtual cluster, then this request should fail. +3. Return the virtual cluster's nodes back to its father cluster. +4. Flush the metadata of the newly removed virtual cluster to Redis and publish it to all raylets. + +.. _virtual-cluster-fault-tolerance: + +Virtual Cluster Fault Tolerance +------------------------------- + +Let nodes inside logical indivisible cluster be :math:`V_{indivisble}` , nodes inside divisible cluster be :math:`V_{divisible}`, nodes inside job cluster be :math:`V_{job}`, undivided nodes inside divisible cluster be :math:`V_{undivided}`, then: + +- For divisible cluster: :math:`V_{divisible} = V_{undivided} + \sum{V_{job}}`, :math:`V_{job}` is divided from :math:`V_{divisible}` +- For indivisible cluster: :math:`V_{indivisible} = V_{undivided}`, it means that the indivisble cluster can not be diveded any more and all nodes inside indivisible cluster are undivieded. + +**Raylet Failover** + +When Rayle dies or restarts, the GCS Server will detect that the node is dead and then update the status of the corresponding node in the virtual cluster, and the virtual cluster manager will periodically check for dead nodes and replace them with new ones. + +There are four scenarios in which the virtual cluster manager will replenish new nodes for the virtual cluster, and the state of the virtual cluster needs to be flushed or published: + +- When undivided nodes belong to primary cluster :math:`V_{primary}` are dead. + + 1. Just do nothing. + +- When undivided nodes belongs to logical divisible cluster :math:`V_{divisible}` are dead. + + 1. then select new alive undivided node with the same node type as the dead one from :math:`V_{primary}` to replace the dead one. + 2. Only flush the :math:`V_{divisible}` to the storage, without publishing. + +- When nodes belong to logical indivisible cluster :math:`V_{indivisible}` are dead. + + 1. Select new nodes from :math:`V_{primary}` to replace the dead nodes inside :math:`V_{indivisible}` + 2. Flush the :math:`V_{indivisible}` to storage and publish it to all the worker nodes. + +- When nodes belong to :math:`V_{job}` are dead, these nodes' state will also be dead in :math:`V_{divisible}` which the :math:`V_{job}` belongs to. + + 1. Select new alive undivided node with the same node type as the dead one from :math:`V_{primary}` to replace the dead one. + 2. If there are still dead nodes inside :math:`V_{job}`, then continue to select new alive undivided node with the same node type as the dead one from it's parent virtual cluster nodes :math:`V_{divisible}` to replace the dead one. + 3. If there are any dead node inside :math:`V_{job}` is replenished, then flush the :math:`V_{job}` to the storage and publish it to all the worker nodes. + 4. If there are any dead node inside :math:`V_{divisible}` (parent virtual cluster of :math:`V_{job}`) is replenished, then just flush the :math:`V_{divisible}`, without publishing. + +**GCS Server Failover** + +The recovery process is as follows: + +1. Replay primary cluster nodes based on the gcs node table. +2. Replay logical virtual clusters (including divisible & indivisible clusters) based on the virtual cluster table. +3. Repair divisible clusters :math:`V_{divisible}` based on the :math:`V_{job}` belongs to them: + + - Find the different nodes between :math:`V_{job}` and :math:`V_{divisible}`, let :math:`D = V_{job} - V_{divisible}` + - Remove dead nodes from :math:`V_{divisible}` based on the replica sets of :math:`D` and flush :math:`V_{divisible}` + +4. Replay all job clusters :math:`V_{job}` + +.. _virtual-cluster-lifecycle: + +Virtual Cluster Lifecycle +------------------------- + +In the context of Virtual Clusters, an Indivisible Cluster is essential for managing user-submitted jobs. This section provides a comprehensive explanation of the lifecycle of an Indivisible Cluster, detailing its creation, usage, and destruction processes within the hierarchical structure of Virtual Clusters. + +Lifecycle Stages +~~~~~~~~~~~~~~~~ + +**Creation** + +- Indivisible Cluster (logical) Creation + + - Indivisible Clusters are instantiated within the Primary Cluster through API calls. + - Designed to host multiple jobs simultaneously, enabling efficient resource utilization. + +- Job Cluster Creation + + - When a job is submitted to a Divisible Cluster with the Divisible Cluster specification, the system automatically creates a corresponding Job Cluster as a sub-cluster. + - Job Clusters are specialized Indivisible Clusters designed to handle individual jobs, providing isolation and dedicated resources. + +**Usage** + +- Indivisible Clusters (logical) + + - Capable of executing between one to multiple job workloads. + - Jobs within an Indivisible Cluster share the cluster's resources, allowing tasks from different jobs to run on the same worker nodes. + +- Job Clusters + + - Dedicated to executing a single job's workloads. + - Ensures isolation and dedicated resource allocation for each job, enhancing performance and stability. + +**Destruction** + +- Indivisible Cluster (logical) Destruction + + - Can be explicitly destroyed via API. + - Destruction fails if there are any detached placement groups, detached actors, or other resources still in use, ensuring active processes are not inadvertently terminated. + +- Job Cluster Destruction + + - Automatically attempts to destroy the Job Cluster once the associated job completes. + - If destruction fails due to existing detached resources, the system performs a lazy destruction: + + - Periodically checks for empty Job Clusters. + - Destroys Job Clusters that are no longer in use, maintaining resource hygiene over time. + +Lifecycle Diagram +~~~~~~~~~~~~~~~~~ + +The following diagram illustrates the lifecycle of an Indivisible Cluster within the Virtual Cluster hierarchy: + +.. figure:: images/lifecycle-diagram.svg + :align: center + :width: 650px + +.. _virtual-cluster-autoscaler: + +AutoScaler +---------- + +**NOTE:** For the time being, the OSS autoscaler only supports Ray clusters, while virtual clusters can only be manually updated by users. It is our high-priority plan to make the autoscaler support virtual clusters. diff --git a/doc/source/virtual-cluster/examples.rst b/doc/source/virtual-cluster/examples.rst new file mode 100644 index 000000000000..b7cf7b6443c4 --- /dev/null +++ b/doc/source/virtual-cluster/examples.rst @@ -0,0 +1,314 @@ +Examples +======== + +.. _virtual-cluster-deployment: + +This page introduces examples about Ray virtual clusters: + +.. contents:: + :local: + +RayCluster Deployment +--------------------- + +This section demonstrates how to submit and deploy Ray Cluster to kubernetes using `kubectl `_. The provided example showcases a RayCluster (`ray-cluster.sample.yaml `_) that interacts with kubernetes, retrieves RayCluster name, pods, services for cluster management. + + +**Create a ray cluster yaml file:** `ray-cluster.sample.yaml` + +.. code-block:: yaml + + # This example config does not specify resource requests or limits. + # For examples with more realistic resource configuration, see + # ray-cluster.complete.large.yaml and + # ray-cluster.autoscaler.large.yaml. + apiVersion: ray.io/v1 + kind: RayCluster + metadata: + name: raycluster-sample + spec: + rayVersion: '2.9.0' # should match the Ray version in the image of the containers + # Ray head pod template + headGroupSpec: + serviceType: ClusterIP # optional + rayStartParams: + dashboard-host: '0.0.0.0' + block: 'true' + #pod template + template: + metadata: + labels: + test: a + spec: + containers: + - name: ray-head + image: ray-image:antray-open + resources: + limits: + cpu: 1 + memory: 2Gi + ephemeral-storage: 2Gi + requests: + cpu: 500m + memory: 2Gi + ephemeral-storage: 2Gi + env: + - name: RAY_NODE_TYPE_NAME + value: group1 + ports: + - containerPort: 6379 + name: gcs-server + - containerPort: 8265 # Ray dashboard + name: dashboard + - containerPort: 10001 + name: client + workerGroupSpecs: + # the pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 5 + # logical group name, for this called small-group, also can be functional + groupName: small-group-0 + + rayStartParams: + block: 'true' + #pod template + template: + metadata: + labels: + test: a + spec: + containers: + - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' + image: ray-image:antray-open + env: + - name: RAY_NODE_TYPE_NAME + value: group0 + resources: + limits: + cpu: 1 + memory: 1Gi + ephemeral-storage: 1Gi + requests: + cpu: 500m + memory: 1Gi + ephemeral-storage: 1Gi + # the pod replicas in this group typed worker + - replicas: 1 + minReplicas: 1 + maxReplicas: 5 + # logical group name, for this called small-group, also can be functional + groupName: small-group-1 + rayStartParams: + block: 'true' + #pod template + template: + metadata: + labels: + test: a + spec: + containers: + - name: ray-worker # must consist of lower case alphanumeric characters or '-', and must start and end with an alphanumeric character (e.g. 'my-name', or '123-abc' + image: ray-image:antray-open + env: + - name: RAY_NODE_TYPE_NAME + value: group1 + resources: + limits: + cpu: 1 + memory: 1Gi + ephemeral-storage: 1Gi + requests: + cpu: 500m + memory: 1Gi + ephemeral-storage: 1Gi + +**Create RayCluster** + +`kubectl create -f ray-cluster.sample.yaml` + +**Get RayCluster** + +.. code-block:: text + + $ kubectl get rayclusters.ray.io --sort-by='{.metadata.creationTimestamp}' + NAME DESIRED WORKERS AVAILABLE WORKERS STATUS AGE + raycluster-sample 1 ready 3d14h + +**Get RayCluster Pods** + +.. code-block:: text + + $ kubectl get po -owide --sort-by='{.metadata.creationTimestamp}' -l ray.io/cluster=raycluster-sample + NAME READY STATUS RESTARTS AGE IP NODE NOMINATED NODE READINESS GATES + raycluster-sample-head-z779h 1/1 Running 0 3d14h 100.88.92.34 yg61001t 1/1 + raycluster-sample-worker-small-group-0-lf9gg 1/1 Running 1 3d14h 100.88.92.192 yg61001t 1/1 + raycluster-sample-worker-small-group-1-lf9gg 1/1 Running 1 3d14h 100.88.92.191 yg61001t 1/1 + +.. _virtual-cluster-simple-job: + +Simple Job +---------- + +This section demonstrates how to execute and submit Ray jobs to both Divisible and Indivisible Virtual Clusters using the Ray CLI. The provided example showcases a Python script (test.py) that interacts with different components within a Ray cluster, retrieves node IDs, and utilizes placement groups for resource management. + +.. code-block:: python + + import ray + import sys + + # Initialize Ray and connect to the existing cluster + ray.init(address='auto') + + @ray.remote + class Actor: + def __init__(self): + pass + + def run(self): + # Retrieve and return the node ID where this actor is running + return ray.get_runtime_context().get_node_id() + + @ray.remote + def hello(): + # Retrieve and return the node ID where this task is running + return ray.get_runtime_context().get_node_id() + + # Create a placement group with 1 CPU + pg = ray.util.placement_group( + bundles=[{"CPU": 1}], name="pg_name" + ) + ray.get(pg.ready()) + + # Execute a remote task to get the node ID + node_id_task = ray.get(hello.remote()) + print("node_id:task: ", node_id_task) + + # Create a detached actor and get its node ID + actor = Actor.options(name="test_actor", namespace="test", lifetime="detached").remote() + node_id_actor = ray.get(actor.run.remote()) + print("node_id:actor: ", node_id_actor) + + # Get the node ID associated with the placement group + placement_group_table = ray.util.placement_group_table(pg) + node_id_pg = placement_group_table["bundles_to_node_id"][0] + print("node_id:placement_group: ", node_id_pg) + +Submitting to a Divisible Cluster +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Submitting a job to a Divisible Cluster involves specifying the --virtual-cluster-id and defining the replica sets. + +**Command:** + +`ray job submit --working-dir . --virtual-cluster-id kPrimaryClusterID --replica-sets '{"group0": 1}' -- python test.py` + +**Command Breakdown:** + +- --working-dir .: Sets the current directory as the working directory for the job. +- --virtual-cluster-id kPrimaryClusterID: Specifies the Divisible Cluster named PrimaryCluster to which the job is submitted. +- --replica-sets '{"group0": 1}': Defines the replica set configuration, requesting 1 replica in group0. +- -- python test.py: Indicates the Python script to execute. + +Submitting to an Indivisible Cluster +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Submitting a job to a Indivisible Cluster involves specifying the --virtual-cluster-id + +**Command:** + +`ray job submit --working-dir . --virtual-cluster-id indivisibleLogicalID -- python test.py` + +**Command Breakdown:** + +- **--working-dir .:** Sets the current directory as the working directory for the job. +- **--virtual-cluster-id indivisibleLogicalID:** Specifies the Indivisible Cluster named indivisibleLogicalID to which the job is submitted. +- **--python test.py:** Indicates the Python script to execute. + +.. _virtual-cluster-raydata-job: + +RayData Job +----------- + +Let's now submit a RayData job. The whole process is as same as submitting a simple job, where the execution parallelism will be deduced based on the certain virtual cluster's resources, and operator executor, i.e. Ray Actors, will also be restricted inside the virtual cluster. + +A batch inference job +~~~~~~~~~~~~~~~~~~~~~ + +Since we only want to demonstrate the process, we simplify the job by mocking a inference model that returns True when the number of passengers is more than 2, and vice versa. + +.. code-block:: python + + # Solution 1: Batch Inference with a Self-Maintained Pool of Actors + print("Batch inference with a self-maintained pool of actors.") + import pandas as pd + import pyarrow.parquet as pq + import ray + + def load_trained_model(): + # A fake model that predicts whether tips were given based on + # the number of passengers in the taxi cab. + def model(batch: pd.DataFrame) -> pd.DataFrame: + # Give a tip if 2 or more passengers. + predict = batch["passenger_count"] >= 2 + return pd.DataFrame({"score": predict}) + return model + + model = load_trained_model() + model_ref = ray.put(model) + input_splits = [f"s3://anonymous@air-example-data/ursa-labs-taxi-data/downsampled_2009_full_year_data.parquet" + f"/fe41422b01c04169af2a65a83b753e0f_{i:06d}.parquet" + for i in range(12) ] + + ds = ray.data.read_parquet(input_splits) + + class CallableCls: + def __init__(self, model): + self.model = ray.get(model) + + def __call__(self, batch): + result = self.model(batch) + return result + + results = ds.map_batches( + CallableCls, + num_gpus=0, + batch_size=1024, + batch_format="numpy", + compute=ray.data.ActorPoolStrategy(min_size=1, max_size=5), + fn_constructor_args=(model_ref,)) + + print(results.take(5)) + +Name it as NYC_taxi_predict.pyand put it to `/path/to/project/NYC_taxi_predict.py`. + +Submitting to a Divisible Cluster +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Submitting the job is exactly as the same as before: + +**Command:** + +`ray job submit --working-dir /path/to/project/ --virtual-cluster-id kPrimaryClusterID --replica-sets '{"group0": 1}' -- python NYC_taxi_predict.py` + +**Command Breakdown:** + +- **--working-dir .:** Sets /path/to/project/ as the working directory for the job. +- **--virtual-cluster-id kPrimaryClusterID:** Specifies the Divisible Cluster named PrimaryCluster to which the job is submitted. +- **--replica-sets '{"group0": 1}':** Defines the replica set configuration, requesting 1 replica in group0. +- **--python NYC_taxi_predict.py:** Indicates the Python script to execute. + +Submitting to an Indivisible Cluster +~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + +Submitting a job to a Indivisible Cluster involves specifying the --virtual-cluster-id + +**Command:** + +`ray job submit --working-dir /path/to/project/ --virtual-cluster-id indivisibleLogicalID -- python NYC_taxi_predict.py` + +**Command Breakdown:** + +- **--working-dir .:** Sets the current directory as the working directory for the job. +- **--virtual-cluster-id indivisibleLogicalID:** Specifies the Indivisible Cluster named indivisibleLogicalID to which the job is submitted. +- **--python test.py:** Indicates the Python script to execute. diff --git a/doc/source/virtual-cluster/getting-started.rst b/doc/source/virtual-cluster/getting-started.rst new file mode 100644 index 000000000000..9244cc3ffb37 --- /dev/null +++ b/doc/source/virtual-cluster/getting-started.rst @@ -0,0 +1,40 @@ +.. _virtual-cluster-index: + +Ray Virtual Clusters Overview +============================= + +.. toctree:: + :hidden: + + Key Concepts + Design Overview + Virtual Cluster Management API + Virtual Cluster CLI + Examples + +As early as 2021, Ant Group began promoting and implementing virtual clusters feature. +In addition to being able to submit and carry out different kinds of activities within the same Ray Cluster, with the condition that these jobs be segregated from one another, our business requirement was to use the long-running Ray Cluster mode to solve the delayed pod provisioning issue in Job mode. +Furthermore, several companies anticipated that some jobs might be co-located to maximise resource utilisation and others may be segregated to improve stability within the same Ray Cluster. + +The concept of virtual clusters is also mentioned in the planning of the Ray 3.0 community. +The following is a quotation from the Ray 3.0 community planning's pertinent FAQ section about `virtual clusters `_: + + **Q: Aren't Ray clusters already multi-tenant?** + + Ray clusters are technically multi-tenant today, but do not provide sufficient isolation between jobs. For example, you cannot run two Ray Tune jobs on the same cluster, since they will both try to use all the cluster resources and this leads to hangs. Similar issues apply when trying to use multiple instances of Ray Data within even one job. + + What's missing is the ability to reserve a resource "slice" or "pool" within the cluster for workloads, which is what virtual clusters provide. + + + **Q: Isn't this just placement groups?** + + Yes, you can think of virtual clusters like placement groups v2. Placement groups v1 is not usable for multi-tenancy due to implementation / API limitations (i.e., no nesting / autoscaling). + + + **Q: What new problems does virtual clusters solve?** + + First, virtual clusters solve the multi-tenancy problem in Ray. Instead of your ML infra team needing to create custom infrastructure to manage new Ray clusters for each workload, they can queue jobs to run efficiently and with isolation within an existing multi-tenant Ray cluster. + + Second, virtual clusters allow applications to easily compose multipart workloads. For example, you can create a Service that easily launches fine-tuning jobs using Ray Train and Ray Data. Each fine-tuning job can run in a sub-virtual cluster with well defined resource allocation and isolation semantics. Without virtual clusters, these fine-tuning jobs can compete for cluster resources in an ad-hoc mechanism, without notion of isolation, locality, or priority. + + Finally, virtual clusters fully encapsulate both the resource requirements and application logic needed to run a Ray application. This means that an application using virtual clusters can more easily be run "serverlessly" on any physical cluster without resource allocation issues. diff --git a/doc/source/virtual-cluster/images/lifecycle-diagram.svg b/doc/source/virtual-cluster/images/lifecycle-diagram.svg new file mode 100644 index 000000000000..1441f5160a12 --- /dev/null +++ b/doc/source/virtual-cluster/images/lifecycle-diagram.svg @@ -0,0 +1 @@ +
Primary Cluster
Submit Job with Replica Sets
Execute 1 to N Jobs
Execute Single Job
API Destroy Attempt
Success
Failure
Job Completes
Success
Failure
Indivisible Cluster
Divisible Cluster
Job Cluster
Job Execution
Job Execution
Destruction Attempt
Destroyed
Destruction Failed
Destruction Attempt
Lazy Destruction Scheduled
Periodic Check
\ No newline at end of file diff --git a/doc/source/virtual-cluster/images/ray-virtual-cluster.png b/doc/source/virtual-cluster/images/ray-virtual-cluster.png new file mode 100644 index 0000000000000000000000000000000000000000..0f1c51bc0087603855a4905a03eb2f2a4f2dedeb GIT binary patch literal 222060 zcmeFZRajfy+BQll?k>fHySqEZp}3Y}#T|l`;_i~-?pEC0p?Gn34HDcp@A|H9AMJIm zf1T}v|6pcH#+=D`#u$&>_nm~RD$AlGeL{kQfIWg=6rzq_`d(}&$S%r z|9uxeA?M@&9)Cpmr=i_sj0O~x7}RGeaSh;y(@c0Ejx^EG1{8(=ls9W`(7HG4OhLp< zyoEFjGJ;P;L>N*4AON5#CEoh;=k>bxIXBf`40$<9aQYju%Tn8E!)}5>!kur{*xC5R z#GhAp-X=i@%5Nsnia!zAKG1Z-{DQ*%`hQRUuMPfxIu8n@pzS9O6l1;;W=t3X~MCRDO&4T4*_M!jPWTx^*$W8QRaIHlT+P?e7HGSl2Z*N0{BjBQQ z=rFKwuK>MiFi{5s{B{W#Sz+nrT5Y!Szkb}#dF_gC2X}{ zdx&a1VwcB(#fBIxlPK-srU6=q6)rJ)VA5y|gU6jnjCTeW7XKkV8WxspZ&OpcO8Bq3 z#RUTZn9@-RrIEtR#`|@_7!%v9j+j#VHHyod*^mPg?|li@l&B;#IF5XN|9d@=bDE-J9~BkzW23F zW@szV#Aj0C+GRZB^@;z&^~q^LZvkL#N7rJ%_Y^@zaF{8)iVW@iQUNiakeXY4<-r{p zU)N8gmJC|i$om#rkvcQv^WY`k#IT;q+%UBFU)b>j&1L|MvJrBjlizv_B2qL~YLw|~ z^srcm$Mf<_>bhzrLqZ{~Gg_0TDgZu((Rp8c9OkO>w9uT%jtGPIt7|?Ys>@G zApLLu#EE9M54$R_5EZ{xC+%HjywP3a*h5_5$z7EBm@Ox?CWmCKhvJ?Ax;l7T$;QFR zCiedrV=mCc3hFv1kM`FPO8G1RpJkyU9)2INL$0`}(#}MAz(f&y+pw#p-)f>X_+mA2 z!;fN~qyL#JaS;eOf56I9FF*MD@JDxJVqK8ZGg{TSCDUAC{&_ng?`kNfi?KY)a)Na z@FjMybt8P@7}pMIi3!hr)!%p3&#^Psj$d`3QqL~Ab`|JIPk-ufioiIKng{H-{hxd| zri8!Gy*dF&Cjm>x*Hnd3%DB%`rj>^Cf!5)%Z3yAxz5k4{oB~W7r|0#lJ`OH6Hm1^a za^u9A=Vy{G`-nbzx;l zi0o|DY%auLJEQhrIl>He~KjJ4Fse0DGS-HioS|*X(X^o0@b}?0gx=cr zu-m*_px?BjDvZpfAv$nfiH@)!efP`~;X;H&k4>ZF@@7B3{^v~QBA~l3hfeP|e^~Nx z>3zgc|L*McY^(`xqK(s7nVdP3YAa5!pht1Bd&KGWOcf>}(Kj3?x?09o$6!6hLfsWy6e zjKb$x|HnN4-;;^lAI5$)GN({5fG*?9Ax)gw=n~1nrvu`HtbcjexMf5!-qoVUo@qXs z%=Ev6L~{eCLorIUK1pp*5`jp%37G@IV+*BUvQ7RLo`<>~9gmxlsaO+q=MSk^P5STD zn;(2=^%)0!`}WNhy<8`GK54cMbIx}Ci-#aSji`TrvnY*Hx{908v6ATPqh$cv)bgSQ zvkpZyvw?A>wtbkksE_X;GSPYezx7jbP|!_eGBXi|Jvcov!-zS*I6MaUhgycXyZ^_C zV36|TZ8!qq+S%vLCu)Z}yAyCwrr*nfpplqx&KTJxNeF19h)D;&1k= z;>8n2Mf$HEN({Sy;-js#tL0w2Tnj|jmeXzfK=)ozgI09WxV9TEzXa|c#2AkKh(!a| zkM9`Pyf5N>j$QLg=)SNTtddD1?G-zl%d0-ru0v)EeO*Sklb0B{EjP%1G@VbERR2gY z6>;7sb^}i($m1}YxuK-+EDJUse)pAxQvj!d1Wz7`*>+v@F3<5sLA>}R%k6Q_2cstI zxPF%ZHd~kLgCP}-EW-89QZs`-sB<<~QZbKg{?cPWNaxY$nzYmx&C&Y4cx61QxloqE zLu2R{4=|^3ut+7*rGdwS{_(COtO3y2apwdFqzm0A%ynupOg_CCz}9B^_vvO3C6>EN zSZekGz4Zrg`^Bq1t~DX<$TT}UNe-k^d%cktGMi(0&BRAFk(+xe-c2b0yykl8*cMF_ zdO?4@YH}zIi1o;5ZA8l1)kqM;Kd{kaIyE%vI#T%<`a@jsj`QWc6 zmHsMwIIoyV^V)c^CAT$c>0EBgWpCDd|g0gP_IzibjRp2b1XJETi-39)C=yIq9*5 zSwS*CJB{07Z1M4vg(?Y6#s}48B=cbBgv0$xK?#2E1a}=Zt~Th;z|nd8rS!9P^qdbJ z0&VR8&*5Q*!h6ZjUh2$b=vo2- zNUZV4lJT)BYYVde{+<%cSI11+g;~qB${=Ayjg9)=*~A!5#P%Z!9juB z3bV0e>GQ6ov8*x!S;@`g8~y8jc+=oJ`(`F)&00FKEq*mPuPnmpZAc=zfb(FapGBP!51@D$)fKY>|^^F@mlpF zQY5L5H^|;E7xk2HRLKyeB0t_v`i+0)b-M-DD#Ct@RHZ7!&`PV&YxMz&S~f0 zX6C-iI{!$h=C;kL!=w=5dh5{yRrA6&@)g23<_d$-So zbP{s@9X8p>#U6As`fLDwA*Ch13)1HkU>8t1fvS`e-@rK zRn9c-K{$7+6J=@=U)?atCd$!jRjTSwr4%J`!Xg;b-?YlcNybET<;n=tsg0Azp9X|3~c%1S}gvMlIQe=!_N!Wv! zj24^Gg@n*a1eNqXNU%toJ%-z)Fs+w!iZrPi-DP67c8Xbt9oNvwgmbZ^kw-EntKF=n z2{~JdRAd38wN!FYjGC-K zt3jslAN3(a+uYmtXrtD;b->Tn3rt3-c3_J;_gxJwur`x+#3<-ye$8X2(U(Kkkw!(u z6v5I?8d&+(WR6b4@1*Q3O>k(P?7Lg+{J^nG;e5UQysdZH<+9OyQdrJw^^i6W5tNYe z_`XIp!7mnbwgw+F9lv9_*SR;EQc#9Frhc-I>ME>gmsyRzeq?&Ou{_!Vk8U*`aZ@|m zEnUk3;G7*o*n_z3~A{?(Mc-&rP1x`KwNgdpQuuF4W=reU>T)zit`d zr1y@PkEJVxPAkk>vz%^0`>n~)s2ahg($e5(+zR7@eQx=}3my3d=0g&T_#8uW;sNSH z16D^(shyXCfi|CI^FlyYu0`csjw`&f~qR%#&AmWhXnxgQUoYRE=1>QqNdSI5f$?Zu(!N3w~Xk2<)7pkZ7= z=Tn^2t0kQhUBb6MTRkDmJPl>52=lBtv0TTHTI-ht;}Y?eAJ=RbH5JgYvseR2V{rp? z-Ix06T9wZp$Tz-sUs^Uc#n-^WJ7KUqP3nW?DX7VctR-Zslg2 zJw#v6xfJ~)ktpHcHYXKd;o-MQs_K*SB9i*`G>7(!B})OK=y+m!=ZnuTwfI$s3(IRR zx?iIy49fWEmF!Dy*UhEZvkrc0K;SAH>wES<{Z^+Rb#pUlvHHC+fbG6xgH&2gt!Ur7 znOF4bp4pS+WG|J+W^k8r;w>jP#Ak?xs2`tzUh#hWW3Q()nVz`ly(p*sjAE&eOo@}_ zhUD_A`jeMj!SEZ&^G|yi(Y#_z6CC2pj4WwFi8ITQxcJ&;Z}nT0EDZ()`^yo@vm-1# zRLX{{-5pctk-;F)72-KdM=a2Uj_Qyml!N2%_gS(i3f{z~VuMHne?L2SQLoBLU% z{B6ZV!@buOkF@!4=l6vxsuvy;ZsAL4RBrwXUSd`rvRxNruOcILt@ZQxLfa?Y5x?r^ z$vnWrWTDL*%k}(8^{0i(N^&x87^GScIM+HYZr#Q$;`)=?AOoNIM-NyzM<#2AKM9v5@DPt(j5#uuLEJ zqv7k&U`VpNZXNjw+z!V+OJ;>RJdPH~PD&zX>*tkns+_69HMwfg*ce$SEu=e}@1|Oh z1$iQ4%LzblZ?GA*CZ;N{4@-EEdWOA?J^G5Gxfc&XNLB(To;=k;T-kRG$&-$sgmN>C z;vWT_=-jU80n2l5=j@Y{6?$vbN zWJ*sj-z#^e$1Aqcm>m*W-c<7K!dUdhhRYs+MGQ%N(Qj9OD9Ec9(CYnsHJ zs*;|;M%GfXRp&({Hv2Q3sK|$B4|ZHn#jMd=wY0HB%3B^WQzM3BQGLe?`T0ze{34@7 zgrk30XYEKPJ(O9~u9u%WQF!4ryuBz~5VItB*I@)REP>w(N0oX+B-|Qw(vJ&5W^28=Zpa zG*PMUqLJKWuA+Pl25#VVcKg)s@l7#3#}|akOHs)m6uqiLGpF^+*zxSsuCGVJX&zv5aI0T5px@)4 zoG>94N+|YiJu&-LFeTj{UQY{f63xc39L=&eq38!@5_@7XD;}ns=R0!R=AeoD3lrdz ze>7u^>FQ!f1^cA?-a1A>V@PBX-!6ywUylQIyKFi62v0>;iToLr-`y3)y*ws~@XHX) zkjxnC$K#Xi{wm0{ib26p^Q2G;rc@%~qu;sh7eNzY4OimZ0+V%`WdksUamPXaX&UZA zL-xJ0Bg|BA{xr!P;Vz$e3kwT7iCeUo8hZW*Yu2XO)TqXc1qqkg>ehx3)h|cqyx8r2cLdO3it8k#iu^$yodz$1R zgCzur7kv2h!06Qa^4-xmjmN|_gSdSdR?vD;%1lvq(~~8rMk!(Yq{apEej@IYS<$e_47Af zM6vsu1FGADjyXQ?)1Lh1LnR3GiL?l~87fA&*6C);VnPRN`Fam)x8$VCa9BM4n*LsA zXmDHA*1L8pG93Ac8-3e-2TxwJg0^m3@HHVY;X6DlqVp}KquXCL0rTu%zwU#E^3 z#<2~bmX%lzN*oEP^p+N=mH}Anfr7Y_bA0hDbsQ8whG^KcHbrFW%d7U`xx3$7eKT3K zDk5lcI2Xl2S9g~Sakc+jQw93<){qa0WN`f9bxrll$3c|b_zm9mu7@%nv`~BJas`(> zGSv2a9If$OR0+TOai!wmdvMac-%q(Z&nF{t&2rU*hbKBo@0*_bGj1G}BZ4rkCP?&wsxM7%XiGpQbTx$GuV zqzcgIgpgTBz<3{=>gRWTUkY60RGiowPn`8}3Fr(B>firXgQKOf@}XB>G9E>@);d2N zKvWr$6L+1Q2Em1`_lzxfH8_DPHDbyy9$hu}{J}#vy zk_N%g_NL30TGo6@QrcBl21Dl1{ibSWLAxeZkG|~+GD&jagYNlD!!}QSh6m*RF}rf> zh&57(kZ`5&w+k`%eQ_Za=VJ3G$aMJ)X_(Ncd%M42-Idw*$nyiUpZ1oZ5B@5{Z}w2; zn3D^^$orV*a5aMw?mY3^WfbU;8*WTEbTism(ukwq-3SazBf@hBD*I2%Blxk{X?<-A zlI-HoVp0UX1>U*Po^;x13qJ7fy<2Hqm*k6y&)L6+(h-wzUSZ)rEI`+(Zk^^PEsXs* zAG#-QOlu08>3yfOOWD)~bZBSlxLo^79z8@QEaqvC~=a zmmOBs@+;uY$8EFm2p1kF0xhjxZYg`a`(C{}^`u`st*;WDPTZj~TOIq_`Fb|KSijLu z7}ENX^VfQ+6>GVb*f-S9_>bTv{ewqX5UL0OdpfUzF6N^R|1t^#013Uud?udexAiCY zc9%|rx0`Wn4{sa7l5ks!7G3booR9lF?+lA$lvn`nh-&Zg=gd#Zhn8E@`b`$*9pkkX zErBK#H=iK%h7#2}?TY){DcUT?gUwZjBdjqjKc=AF(=5s;U02ZT8 z3W?Koz?0_OC!*Yt=^`&xk$2FMe$vp6z_y{f zu+Sr1r@a2X?q~Q6nkwD}??o0xS&b6x`jpgh*jT*|x@eUimPShv3BmA)!qn zU*&$JTM@irOvIJ!ti}!B)m_ILoPB$4N^d;lr$Jq6$RDOwC%!qE4_W4^wp;Lyao72EC7Y#oestKn9nEPq0F6+Sd!kT_}zq}jM=`=Dlf#)ov@a^9G~g~8A#yaahXo(9|Od3x5Vcr)k?d&+R1G?L~RA5Ty+^O+BbojQWN|f#TVmDqQ@CQ+3uH+y2Sd{lx+-QF2ixScP13qJn$Nd~_WGun>@# z_2rAcNh8tRV>s^(SFMD|4(hN53bouMdm9<0Fnhk~wZ^QI-BfC@ow$F^XEV++pfO1u zFYmf}Eax)YZJJ5M%>t2i-L+QawZ~r$tpA=bDZk0-1n$8_7%1mlXY+y?KM8FU5tb41 zM-C~aea0@D$QP+@C?44nDKg-{nJg|06pyi`z6cB5-bgw0Oj{N(o?i~?@yD`RNGdZu z$C5#lw?^(m@pfjVeD-+D%KZL}URMBmd1o;`NlG#sSg2M2eCitN2*tZevRet2h()HU6{>*jr@h`e#C7`8+)uc9`^y~=D@fFAs=a9#EBG=C!Z zwtd-}vC>W!!5RplO~KN=^m!HvYFulR0wW2qnzdV}gPagIJ)Tz!(`V!xWXyd(Ry2DT zRwcOU-_BNjEIzyrw&A&&DNdpMe;?o^@gz98pQELzl)q2{Zg*w*(3u@>@m!OpWw&Vh z4XTWeQKWD1S6f-V-Cy~S17LJ<}O-rlvfr$lk z8%KV;;QWU4o9wHP!9Axcb)ay|R3ff#j}a66K$zyrn^ z@SXvjh~lOYi~@sR8?|$O26Nzy{WVcaZF zDk{P9*7-ddMXlq<2&+azLTpE<9abm7L%BY8xr58b40Yr{sp*N&z>@fvI8N`=NaY-a znti5J5zCsI(#HmH7vAIDF&*y6B4JklSG4o}1|pA<+vSeG!-pRnPP<3tU)x)(o{n4U zHG)R7?yt^W)pbfFSPVNpSKq$<3TdC7ts)-WHxxUv`;6f%UDNN`AX}eX;oY&!qRi)g z#Ut;1;!O@K;sq#&AXo6$ES?XCpW;LK8p$yil*3oY%xiZGXLP}Jol5<=1;<5OPEW7J zqH3Evee;<_wL1&gVuMwdr{)+5kEA8qJpr-%7bS6ops>ie`s+=j|Hc&ewp z_*aI!Ct~+69L&&FhiVG_!`2C=jDv}0i0T3&?lZaT$!Z!N(*r0bgO3r?@Hpz&{|>{A z6p>uYXqlE8<^w8M>h*uTZ2E1?_Kf@Sb604~+%1mn^+=9Am|~W#q$S~u-WzwmB@OyL zii|(FQ2Pp+$Ds81Je_?!D(tX)A)9yST`JNHrwuie^f86rIaDCTF_Zu`de}cWhF11 zv%27a0o{%t6&BrSxV5D)*ftsM+3>zZc@`@q|F$P7xgByU|9z1kf*N%fS>olXPEn=# z)Xs&`Oh_z88|=(WZ+;NSSRtouwkbFJ6QqxqSJ8`6p`}Sy_N9^-7B=)#pJ<^B&pQ%( z6jiOAK)RfDwWJ%LG2H*~BL;TWAjpyAURF{RsoZe!ElD2dx<hgPNDWi!#i8B>9_?mToAplJK1qt* zu!yUuI9rCNaql8Dwa$!r)T5kVk0;rg-;`Mb7(k5gvLZX_i~4+zV^K`t2QNyddsy?^CJr z_5x0wtO*Xg$-esBJ-~CYvOu~e_+PFnq=O6^C$M(cHJx>V`NxRPLVc9Wr2az(w36_u zSAz|u?qloDpg>s_>~^(=^};gd4%NP4)M$7%p@epwZ0s=aE{jwmvhE^s+j`=wXkFNFiq z>;TJ!m?!PT=c$jqfk!eIk_=nPf+J7+XtQ|!=OVt5@0_L`3TvaOaWCqSL2}OhQo)); zY5HHR`#+gJAB9SENCTa|sNYS#6p}#aZ=5R)AnC_g&)n?MsK=uK7JSbO(Ac&+JL_gU z!zcUsHdh2GOKpwWodzGP`n$*Mkf2ow^b@p{tch?(I$z;-Y!%$TsC%jc;k>XcR#up) zNw8z*cs^uYk((f7&Gr4LmRqS@u5x-Q63uJ_D)9-dQV~3wzUD-Eu$O7I5aTk$Em(Y9 zgdbe9oG>oRv82!_{HDteQp=JL_i+28CXAyN_qAZXU5%tg1J!=bO?ASsd#hK^Or)Sg z;GL*7Y;{hwe_V?i|krxF-XBeZZ*mLH#u3+>XB~3~sj2B~cA_{NNB{1bjH+vh8CKACHOIfq8WL?lw%E z?dNiQ1|9_bj#{519+)#&|8#VRL)&!#)Qh(CTyQ;!ZS_sUYP?o**;aiIi4B{NWf3%2 zixs}?v=x;T+vpL)SDpMYOrO2CQaCz1D%YqC;GL^57-c}B3yo)7N?G&lD0xy;JQs>g0R9MD-9CP28mFevC*8aaikAc(31Q zS8UZoE|yehBy^F@X>Tiy3NvcZNY!OypGNN2d|>4*h*^z@>(zUTlYlv+RAfK%^YWsz zN=ouIk;>t%z@4hCJJetM<~tx;Zhv@Z5r@iY7rVqdgiXgFl^1Lgd6K!zoqzs=V>SKa*V&~! z(acz141OiEAiC;-`}j_M@YDsu5^5=fk({m~q5;q=`z(Z?)u+wsWWI|8v3>71fUw2< z!jCxX`y69({0V;bdS2qT`NH|_#{hVRJ?C`}uDqM|&+}bAYbuI5tNGsBS;N`9w8>0* zV$Uj!$`4nIJOV&+XQj*zU?7WO8+C`lSTX|xtz;yAIQLWycL=h!58_UjbD#0v{&GKE2EsspvuQjcL0 zDQP?@v~ciqKnkY{D`}Rs5yiV~s>%NA%M)D^Evl;*+o#dpr=_HLZfXu|z~j$^cQ~LS zEf4e$cQP!TtE5aLz7ab>;EXV7<{flyvyyb$QhhR4^7T(~DBg(%BWn`UV;0+Rwa?+b z=aQ51WeoCwt^8wVqYRVj;9@CiP`fg-^PR4(RPbMCMIYi@%x2mNU#(d3CkEQzcO zk&62zGSM%eCpeSe>3SZf^jvpd7Iqb`yXs^vyW0&KFcD)|vM&)H#(o9)yYMUb^X$Pa z?xlSmE$0todC?d$3fye3l^y;4Lq6skm#cZZ$4x)%z5RDR(mTrwf21|=JT!OGH2kHV zC~L9hZvoo=hKw)w03gumjY#61awtl+X`0Dam`^*6Y!UHEvNh(^CRiElq%{%Qo2~Z4 zQ#0K58rQDWNTGSBKlEgA>g#N>DQZdPyWWl|<2k?y>_}vz?{d6T^@v?r@eogwsPQzM zaqOoR)6(_%ju4L%mdX=#p7E>V!bgKFE*Qw}sEl2jRs#I|ttv-)Mg698+KC ztowA7|4EJt6{2 z1HHR$YYijJ>hI*A@U9Qk`y?%%vLvPKS5*p|PQh(=(#<x0bcf8TYSWgm1pdRwv9JAVUSj-NIPpfTnv5b981uM4V0@P?YRFQ!D? z_9Y~AcH<@C)nUmT9=;+s4)>r;uFfGB(?kGTf(dwWMrfs8H)R2LSq;yvpsMv8mCk21 zQb5JGp51qR26A7|YNvWOnE~-^?@jD;+?l4EBPKigYf12r?&|N}9^T9hZq_*EkTQE`+%;}XKVwRrJ#NWk z@E(UzP)gM@)m`;`y`Zn4#jigOF+!877&~6sbVl~bVIXp0X-JH)Y=MX1a!|2QB)z|B z?6V?r!o>>rR{XP&Kmtcm-J7KiB#L?li!@@MerLTH(|8;sg!67E>Jwh}`ON}vtyonF zf8_l1GV@dPLP+BWsj9;}J)-0F($oQOx0m^k*JIC`G>N1{O=JnGq_D32;W2!F6X@R%m z=|Q`F#*LTuxaZ()f;W7;R}L9_5!dYicA@JMiVix-5R~h+`e2zr!a2C4d11k(oDagw z$D|71oK;Qb6vh&H6IjVkqo1;_g=Um?Un2$8td58;UTnh^ZTUAQYnV=T1>Hn$m)gWS z4Q@}jA#TvhQq)fbWMS-!<#O+E>g`eOLFeE(IKoZPT!gniY-2eEPom*m*me0neZ5BW zW2u}nbJuL~#K8KjNTK0S`S`7Fqrr}+h5!;C=?IfmVmGJL8D&5BX4bmM0sGazO;WAmyqWoC^8jX zPv55eg?ub(hLXwWJKZd}FBpLsXaE#8jhdb=irU<&NE%L+T72SIACSKHRyq3a|A0bg zEbGUeto3{uIN-r&YUy6pZ<*oF19s$Q^E~ORE+}SoAI#88jxgW#l%kTVgyxg<8k#_k zdQHAUT4n(oY7h_|q`1U6w)+P(4-rvX%|>h}5=*LTkL2cCBFMm1G4+_-b`+c4)Y=Q- zGpNf6W5}$WZId+rh}MxM))u*~=W4XLVNj5+ed`cA0)7R*;q?Amp}aK`^s+l5ygQl; zG?kJdCecW5g?sJ2CsRg7#N}f&z>2|G2!8E(Q-<2Zo*>wh$IY+pW&bb%+X!PS_+!7e zLaSZwM`V0a+yoUDlYxwm*dxHDN9f6fo!0Frjcl!hOfGBt&7>47c(=IVKC?F?ts^9T ze_<%Gcrqxk-|dJE!gfKA%bcXbCyP^%rs?AvU3EQi-U07l@ACH93&QnFkGoQuI;al` z8QW14`dKT_#X`pK9uACw7j5q(w|J|yjLKbvu)s|5j^Fz)Mg)ih zE;y9jBxV*@L`X|w!8#K8K^*pv83Pak`**l@KK*TRC1}no8@X^JL6c|?7{7Bn_ez%j zH`Ie3B3#b4Mng1!4>U$6z`7gwcmqLFX@ceKWM6`f2sScYnSSol9)r|iTVjY{TnFPP z$?qx5@@AtZueYnxNII1z71w^ldbXNwj?qxvhJ~9?9rm;PoYNoLP)+TRdp5ZdT@+)N zu2=R=`{Cz$X0*KDDU>O6`yYM=pj8IHU(53$_({~MtJ`H+%}S>!P4zU3>ix@^vZ$i; zsn22ht9_7|mHdDf{JM#`v&lFr4YVW z;G!J>qwOC51s*+Tk&JZm3a`4j4rD*IK)ItAG5LID%->ReJ0vAz;`Bm6=9JScF{qEhuoeL}vU6)M^69Dxw*1IbDoCTZ?8m8LQIqL5)Vlmd zYX~h5sm~T}UFUB9wu(Y#=o3H#JxqrmbTDb?W58nanYwp}K;U=iyk%L|W&tR#IT(i ziQb$qjg;gB51llB=A>7AS24@}NbmYnA|-Qpa+%%XFe55KW7-LQexWDsTKndE9fI-@ z!IHJ%cgN=g&ym~s zG!iv_muuDa`&}fp=S#?1Q}2>FZKYpKxl%IMJ=LNl67O=Mgcw|e{4;ro&`hUg8B^AL zawNreXX)-20|I%X-k`P_c8E?GhiAlFO-tSEOQ=eJdp~6^!ryF07)Dk>+>w8L&+~Zn z)1T~vjIG0*$6&}zg1=w2wx0T%y`3xS^W1An$}fE5cfgEUr&DbLbTp^WV^i@QcDxt}2u0C}C8UfWp2r<|yRDJvAr zz1r`Sli=GsZCdMiuX+_xMmz=qIBy;1c8mx~#5_C*`z?XyZ8`o(ns)WVUcKk?32B$OGCtV6^iJD19LD`{np1Y3y}R3D<9o}fmCk`U455$eaB9HuvE^q1av zUVnebZg!O`sy6>X;l{GFzmK!M_!2q<-sYVc=0|Q2eY|5al8UL8;CCoH{Gu1aGsIs{ zv;5BWE&HbNmNDZFsTxSeC9b5Exc|j+a}v4O{-9#wQiPy+J-yip$Y|a2DQ|3evRSAx z^fpHE%%`OQUA+rDXbGt4dir2`dBy1~jCB6;FlnY+m#&3?1}InXW`|je<#s+^`K$q3 zf-Jd>?8m9af=FemoaI2EbcIOsD*>yAr@2jdfMl&N>zD9>5!PL&_c2jb4+&W-=JjeS zJ|jsV2{obndO6kPd+x-)I#Avi-vaP#{1q1j!3khnG!+4^QC&l{*a-#QCXkS@wIHf7 zy{643VEI~+R|9N*UhS$4>aLx)Rej%OY|@`|Qt#NW?DxlZzk-_rC)zq(pd{n_O7hY? z%5Q25e{0IBojN1RMg^_Ndgt%{@_2xgALvXW}&tsm&&z8$RGcUzj z!)t-;^#A6Hrr0M*e=RDOn^#ueF)>iN_#Mq2L$Dfn9HE+l@$4YJq6#o2AziI*bg;&% z6|CVujOOMB8Df+6uuaG*U!q5415(c|$1J1YUu^$0kmXoF%9)?anLPh*KveROuD`KDG$irE~^~@>YQ(979{*c&RUQ4 zr=f>Z7ja*z{z?6aKCIRnq7LO_Wr&JovdID;MH0O&ao!H%(pWe+)akO$)9HTR z?1L)ZT7Wk^X?IOMKap*f1WR0dTB)<>qNo`WcmRHUH6CK!gk~vh$8baCNa#PH_pHVeVU9`gdUQ+wBq*0 zKxD>nPoR2b8q>M)stIH*i&eYoY1A61da^O*{_BhPvJ?q$Wc#B(g)hlTmzJQuisB+TgQK_XH>=6j(`HiDD; z8`CRKF85hIzHD65uG6JcsO9_BXetK`*90uYQ0h>ZiYymMW@y62F>(uCi^h~wb}N2rN(V})*iteJRj0ZW1Riw;rxddHK@hIT(p9BXD5 zFmp7SnZBJl@)NF|*UlVKaU*Hc1(5#1vzM;*$E@)uJTp9W76Q&Hc(6i}d)JZ6YYMs& z9nnh<(XvuI3&JV_51mFL_F(1|?d4f#pTbS4Z1Mi!%jKK(B2o3shg_(^0!C*jNs1^o zUW_33kUabVp)^rHVa0vD6j;{RZUG$hjE&c45K}63du&6*Zi@tB1!)5Otu1YG{_;!i z?_k-hB;7|$mN;bKSXI6=zT$0fp4v%=fi7_B=I9Zt=Y;TKiuYVdqT20y8Mt>};8jIT zEkn)pSN>?dV3UfXzYxVGJqRZM`R6D_N!-NbLBk;e!`{MBZ%;JK6(|!=kCp)-V`E65pzb7}St+8(*W_ z^#$M$tLfixmDr_x7wPR!o1d*&vwZ7oO*foO?-xM9m#NfFHyc(`G7r9&eoi5=mW68P zcL^r2JGP3sun2bJ35|$Ux7@!Ea(YVek~YrX^oXfz^ES>3Ljc=r8<3YznCBS11P@`C zEOiIn?7FBChav#(g7_p?+5wMN2M96(XV0qVR5&qxis+pt2fchM_$5LVDn z_mF;%jEO`2YE&ukO0!+|YQ61^1)(?^X*VKf2E1F%`+((YpESo`@tTMd2RbQhsEzIxyEKn2S-BarCNelS{NZOYd6K)EF14?lJd+W zTg8(6z{bx6Xcl8>sWR;=iN|wX1owT3z${?r`*d!rh`6?|uD4-I?q2q-tYm$(j8qGc z^5o)^;41sF)Qp@o=SZ&l_;DvhN@s~B%R3p1@ahY&Gnj)g;LXx8NQ}-p?7iG7iEQbE z9YayZ({1Mnx13z(0FX5XM~0z`jvFaE3W#qgXgSdSfjve{8kTE;lKHjL#zL$HiM-fg zQtVDfyhsmEt}`8`RM*-^pJ#}#Jl(!1$^-=QNL)(9)#1v|_G17=BuiIwm9R^%gC&$a zcLtk8>-{fgfKc^!vQS9KyY13+R633GCF)rf^zjX*lEaodLR$Y1V{aLi*OIh>26uSKSnvQZ?(Xiv-Q9y*aCdhL?r!%bGiT1ZYu%aqt<`^EZ+3TAKUH1T{ZvuD zgC-tQRvvV|h81V7ch;q%wXSh=2Jbxj0~b3xRVi+x+1~(fo{}g1!XRPsJ{m<$c;r&;iCPN@9(=5i;B(A4Gn?R#fo4E^YII)JP9g$UW%FZ@27)SkAY>j#$D#M31sOllWj)tOdYn!vcOkBZDsg$*`rEu`Ko2*dDgxqctXi%h; zF)N%Qhb#Vb;y$p}3_KpkwW8otI`lqGvYL{0fo%{3Bz!XG&xlA-6EB@oGm0@ond4g4wrju;g>L zfqe5FV#fzjXZs6osb?l&aBiHP!56h5cK$vqhAzcjScN(AFmCkRNtIVT0}kkeC>L1L znk|+3rr!>naCY||_)eNWEQkCil0sfv(vG)HQQ}Iq?!U`>O^-whqTPC+%ar)ydyTcU zA3?+Zd~hP~84c+ub&=;3PE|}x%=FrkKy|?3QkM9xLmzr9$Rv1;C2cD83B+LFpNU?b zPNhi{%H~US75zhQI?eIAT3pO7D$a_uG5U_BkU1w%x~%z#--u|PI=V6yYvrsJZVGtA zGiFWUGN#6&{lllRUs4-=OueOU@wo)c^*q$X#{3M}+X2&O4Hw${=uPrL5gA!eoc7ms z)o^-m+=*28$A5hFifUJh)6^rzi(9?QXI;H z$?;ww7Zv;F-0M{qrMJ?|qsJOVNxVYadUaA}nGd~83MQiIEwiu9I!8ooRj9k#>L(Tn zdQk{nuzD`|Bot6dj3@NJ(j%I!5YEA~+tS`s3Q}5)+zDoZ&+QqDIiDb{yN_aFo6Pp{ znzOI&CiOhQt*IElq^Jxj#rV279N7|~*=vZ+pm@;uI8B7~~TbuwRTR`}XQpyx*DTOxlGq z<}xTZ5G}M_ED?x`6{^5Hm`W2=&G9=~5GB-Ci?}QXm1jZ&mB)toEo43H(bEakNS(uf zQZcR*qqnE{(tgeYO5X;#U`acAUlfKXTp8=TN}zH4IhRrn0t&zWBk^X0y_8jGt@iUW z=9WIIRvN$p;zV;U7f22prp5|Wg{dSj>;luTq6?8f$Vp^Vyy%7mqiC%XpsM^e%TZ*T zzhi8pejKI%z!(30bsyn#w*S*490*o4ExauCN5Iq>F;isaO14YS;mqBvs;y7KM687H z3|kUbTW;u1owMF{<4HM6;OA;r3WT*^pIXg$=^`FHjoZRdh3fU%Y_<5*z=K0PZ=S)x z=qNRbp#9RR;@WE{ioFWAd$Q$jF0bdWfdS-s*^K}m1rNXKOJfl7t!-~}`ZW)+Zy=)7 z!z5&mL7L%L8vTn*bVSRZY(LV2Ux6T0F>#b60XHf)4Zq7^1s#`3-AY=$5&}@E9f;5FN^5^5> zU!OWy1p3+JDNOFwLB2M~fhe_6LD29dCp1=;zo1&UEh>pd*crT8aTDh3J&>yVL~MWA z&oQd@o%#7)I~NNH$*DxKsqETW>{>W4(dQz!=_^6CYd+In?{hlLPZkhWQi_X%E`EGx zKsVRr@~3@do>(H_K{*D|S6VG^@{7++EzYO{loUJ)4CGSG5TCgGEnM~ZN(8Y$5OcvxyDp=+-H<%A& z3FzpmadV`4rwixIjb{)NXKY^8%1WJL&N$=Y*eWVLHikRo8D4<$Zl`LLG*4 za`Er9{qo7;$9)WhK1iASX9Ol)*KR0a6XW3v9%Sf=bXJiczpgKcvV@P^C= z|HOfk?oW&z^6S|y{N=F67#Ik|Y3MuZeBM!KfwxipJiGH?zjchEfgM^Tzjky`M@eZr zE;W@T0?z&F#ak_Q@LT=9J<^Rlo>6m!ZcbYFyJA$qs1_AT%2(9$*<0CN#Gws1y|RFM z$F}tY(^4q-Gi)BGYtLOWbA?h=kNaNxW^h-{xhS_67Ykm}_J4s){wGQXXXmd*wKK8MR!eJu! zwKLAtuZ#DcjH!o=bDW&82hzT^Ydw_ZBh*(8a`0!#1STFP zcaN{xn(|)42G3X_=jI?N}L%zdrjipr#9-p>KuK@D*Vb?nYplz8@ex)m+FVd?UDTG_1TB3p+p{ElXXJn zc!`-{1g+$)CCcCT{)h_B-l@fT_Uq5^2qzH^YhF>nY-qmy-L4S%#t^uAOxiCKld0u=fhGH)>YI@13`f^@)B}495TUoxz6A+=d zKeEhT`M;_T=hNZLW@&i6(Ad>kty=(D>kl;!QA9yhb36$3+C2N*&C{1GT+C51X`tr$mbWps@Vr*y5+mD(+2rJZh?xVp2+-FPs8JdK^31A*s<*Anb9INrbdLY zsz<{%y*0aBl<%zT*>9rAFo*>a180@1IDE>Pb=7KBtT99l9g{hr4E2X@d5 zm()Sz&10GD>~tx-T}0BIqy$V(`-DEgd~q{i<~`zR?`&My2b!o8lFZT7I0FCWvkP;* z(#YAN*tH8A)5sfSEnKAbq)Kd|;>~PcE!E+;ilvpIUJ>VS5yb+O~uYG?}F34XEn4MHxku|jU8(g zdzOc)riytQO9Sq7vn(-}>vcCO2D+vg^Wrbdl5>9RD-gGCa((C=CSjv(0c+_krCuL@ zT$t*9v8)ggoAIwEfkeGj%V)D4-xE^=xs|mcyRB|9k#`qPi1*a%iThYgmqzyk+)AmE z+G@?gpB`p6>}~afDGvN8gO~Es4Zj%~!!_@cKqr!2$$S=Su8u#JTnwbz+ z{edWrdEOm>)g&RI?lM-xl|Z{oU{DRcwpfXPML#T`(L~uZVa^*diwNCsPA>|F5#5*k zW17eF(Kr0{>!{zV`&;}i{?KOCenVzyMnIoJMOnz(O5D_ae%{5}HTy_EI$Ef>g|D4e z6ltKo&)ft9A7uRA_FRKW`)KTDmJ9FInWhP>8;Qh^M*W8t(td@J`8FUg&_T&OHza5!FV2JD+A#u` zW5mtGF(L%h(y~XA7M*wN$D+*SQQ5$g4aKmAbiOPilA?*&o3b-=SEf4&%58py||Zdr3LktuJ`Fp&N_3mPgm;=etia3 zeCaz|U-4w9wFeV9nHHm{YfYx8Vjn!`_*aWxJOfc5h9$4UGX|hdcu(U5i+T*g1LCCO3}}J(9ef%~zX`c-@hLFoaAIV2&9$d5BMqdb-qPm+Rc3H*GrVLfwggeH<+xl@e>>x0VpWLGP}RDuSo zG~{dIVJYCGwkZ?B1V&{(Bi}7hUAVblo{qbo2fogo_BYhHpD>g|eus&>p!LlGu+|b1 zJYo!D9|{#|_aluU;x}iW%1m&2UM!Zh? zr_xvFEDZW+8uMb%k987@*obViHuJnPgzOC-J3&2=$NfuLF)~}X&0Vl@9B84PcxwHs z38asSTX2b1;mnSdmR`vn)N)PKMS9G0El!(5bOyvb)k{V-BHT~@O6){i#npj4P6vUr z1vB6dMFX|0#&b;Dc@fl&4Oi_wCgD=}R=kvX^C1fH$e)i10rQlSQivtvGwrmoG!;pL zN}uf7EfWzwrtyAk&!j_BxwA5F$xJ}iz}{p*o~(*R)25JH{sb zahqG`5OvEDQ8ptPBeI+@JaVKSSfB6%G+@Ctv!CzO6Fd;~sF!tIVPHuH5e3~fU@;mX zMKAA8+*2Ix;~UM<&S1J9pL1cJ;tL zITT}zW{;>W^NR3a*J_d8Gt_3=+rPiACFMWR%}nFmTVO2gS^e-*n|x6CZWX@wr16-c z_PjsBrM?YXX+{UHN6Y88KvT3yP}(q> zT@21M7n$rQDp!iR3?(q*c69@8`r7J=GQS0Ne+U*x>!+OUL|=%1br9>l+W@B z{aU;{W;#7Kpy3M;HobdTz^vWD(!FrM4cM~lTe$a>bGy6B)59F*IX!&rdPB$BQIB!P^FU;;C@?DM7YjS9W3+*0FrnYS|5wElxyOfvf{XL~fxkb(O7fLn#-Fid*F{l+Xx zF{41Ni)_rokwiZTo_PzEi|if>hJlB+2Vk<@#%sFDH2JVy()S^k<{4z7{+-Ty=@jzL z&>a?!$A^tS#eQy422#5Gq?~tiaJzhNPd0Wm4+4_Rt4VJ|PhWgVe!i+^MuH>Z z@l52djoYbVHJYiRrXXP>U5w%?V ziK?J5a6<<~Kz(-~$roBgz5bSYyVpw$t~h1W$(UYaL;sZ}3m)ooXKXKQfx0N*BZ_LX zdWOvTV}u-SG`=#$#DlFFne=^!z+a$yv?Twr&93DC?Vwpb%wr@An4b!1KaH&(i9) z>ho65-1%X3*$Ysz9A^%w4ybab<&x96~16ZVJM7e>j>OtL*e}S z^QDeIb(5JDOZRRLrlZ!-`|@;`WczMl^xOr2{PB;leF6T+^HRWahl1t3?S_FJvN}&5 zq+z@P1|O`w!~>>?bg=7t#ZlREF3w?$p; z4NCew+7soaehD?QBUm~!msx5rD+j7iUH;?5ZRsPWZXF?AGIxlW`-R^%wdyXk>}sB; ztV{SRWMp)23BF39V0wu?FldBzzZSu0D@u~X^U`sBE7t`l*kdw~@M?DApZI}B(ox$ViKz-w+|eDurxQAuElvP3(J z(tx<$96kOd#pjWJqgVVZV{<;M5&x#^*DN)EL1~nWju$GrM-w>w8r0%%bD0(gr+xyP z$Wy*U2~=V({ve|c`tG&*Pee*yVSR8@=fs+V~PC@P?El3$Tm3-Kg!WOBYFo9(&sS-ZW#C|zU-nM0auU{JbLm;UWYB&b|@)3xA@uY$8E^s6^bzFdP`Y~T}n+eKM)VgpQviBm`IEtliZ zU8q}o>GfJ9$AQ(*%r<>Aiv)C2GYm~zDJ|GBQ@LMmb#bOJo^wv+BL%lZa{= z^lmWD9OR9jAbKCF;@gqtA^E68VRvfITp%|i&-g=g+&RnV&(_+rgKu{kazc6YBx_%@ zeO5JWB(Cn_Znd*mM$xi~^b3n|pmKRE?G|DLTcEQ<;@YBYRjK3XV?l z{f_nkD`)T?ZYPg@XtmDlh!%JWK`OWvaSl~p$W z#vG61G_Be7ALA2m9rB(tes7=piN~w#MEgLLWNg9ACFsd(5T<(M>llV@x+{R_w|RNQ zz`4ddD_B#t))6&!3FApmc0UwFlh22*nC{9e*;X<6E7zJ07i-)1sjb(NRwJUK<$$HE z=UXP9z#$~6Z*M$tCepsMkCl3s%4HR=&*M)N!b5D!no0`_bIQx)^T@?IN@Rh_?Jzx; zC-UUO`ya7P=~6B@yMkrqm6p!z@2{Yn8N#1Vp?Jv442jzne0azwzg$N$=O+e;8D)hb z)fCKZUj0N9=%lblHU`MA?dOBTuN$fD+=#AeOGs~`JPqGvrdaMX$aKy^`v>+LOX?heg zIsaNQ2}8=5_2WgUkF{)MA&Ch%V1?()_|( zZ{vu;wxK=m6Z023aZi8X`~dfnJQ@|cj=)0_F%DypYSZ!p>yKMRFGNU7?YpPLEg={q ziJ91-Io3T>Ai#-9)YG1RR9ne*tqaFwJ`+p3kinE6Gy3MG z-JEt^Lj*o#XNm`}n?nonzGb%6de`Qd_uRLvO4Bx7PvM6We}sgDg*Bidcl2Ie4Oz|c zgrugX4)rbyR+>VX(t%sXUw0YfN?L4bv^Z;fYJH}6Ivq)Sls_JE8GlNH>W{3^f_;-) z7DZ#hZ7`;g{MLTO$>CSfT^2kKJ{239uhIruc?BZ^iM4ZJYC}idK#%9qNUNhb3-d6M zJ|C!TWKM`x6TU%u-o?OK!@vBn(fZ>zbkk|LYwYiCW`!7~nIHF8>r8H&IMTVeG(q2v zuEmt(MZM?c9ju}mAl-yx7nAknR{8fTY#LIRmxc46%hKE*9t%4l6bF)_>>RiqW{M5i zEzeOjSNk-!8H+Z`2N#tz_nsG<727DA4(o)+#7tbYtb+_OV@Io!2%0AMcm~99tRWR} zTsGYHPhYq;x>iA6_jr!!OlS&^$uYtebJFG_rVgdy@->k$_mhW)JTU|#I}v3_CW<|z zQ>7f|ZaTG3mlK+W_PX#+IIPQ*F+ZEw*1)FNe5(LgCyeUuHdyUDUg5#`t*9r9tDvVh zJa)P7hzd;x8c32nHJ5EaqppTZ2!dK<40@r+2>5=)ce%J1prpO|hH`0EEeQ~ci>zNU z|7cBKWwh%WhSsDv9*LMfuxGE0$On2+jRCktdD=PRl+F7&Q>!?AT34bENXDBVt&e#* z-5K`arRY3bv~%oAL5*Cw-4_||K$FVn5io4eiO(^8im@k#8s-Ek9HT%h)F@##7_dz= zruQfJ*&7Uq{enf8kMa7Zg`eWzT{_38h%v>pSU=GC)%0O}`YpDx&xr}LK?t8p=rkhq`c^QD%X~2|hABmq1t7Mcb+WQM=y5=uxaX|L;%LedRr9ndO z_sI$;MrE3568tApkLLO~y>IlYiiuCNE7gmS+jWG~h(FePFvI~(I0F|a40bY99<^B1 zyMF9@g?%s34&SG1Lg0CeJM2xhm{Dg5l3gr9wE@H=9)%w>I3*)$7^AB6Z6vsuIE3Bo z&Gt<(lNtnPAwGFw?c_e$pKZjI;Zm{kxvwz7J9`ghaNF_XKCoM@D4g4V`*B&JRme8x zxD8w!I8b}ipk93=5(#f`2|^oioPM(50o69+?uK4pOf)YIdy%}f*_y&%&7ABt9fx0% zuIOkO=1KS(DX9+k*Y3N0w_@$&s%haARv_8%<5$iV6a%elE<33xP3(SlVYd&BE^G&=Z2hiF{N18yhnwVx;-EU{{7W40S_G(4>0v1Gu->eZC3+`IXaw=!vi30yM+M}&%}24 zm|uv5k;dce;*|;=HW6Rn72_@3FWWya5oK!g8Ef&`FScpSfxdTNGTl&#IPav&RMPxZ zw=j*um@&c-RPXIRCgA{odlH31CZ zu{%-Y6sHYv^j#dP2gR~l#gdyzjWJ&dsF1Gmu?98Or#r8pTyEgcOWBcGNRjv`5PeAA zB1gzEIuqQVH%bOTzkO!cqsNQSNlU0`M2;N2ET1moUnVwDuN zNBFpmCOiTjS*!%*G-y-7BIWS1VY0tB>`2xWVX6vl8t&$mlR<+OfPl{v?e|?fXDl5k zRZvOq$Q^;HZyKZmprpuR;8SRagS#h@@t>S3;zu8n`7AV~?9&p*#detKIG_oq29HJf zHh;Gtry)iPZ`W$FSSt`?ergdfpp}a&f8zZlX{D3Ba}S`)j2JM{v_amu!Ncn(Laz7r zYrM*5tl2GO8eH94{@oDYfHL(>B3y(5GKY+|2ir%gIR4pJ7=;<(Fn^$}QJc1WDwRWe z=!8gwkh4LVi*+br6e=#iXyj(>5ZT-yz1-a0KA_Gj8WSLAgje1$ORxm~=oFfixwqRg6MO>92OZxHU>bl3 zu}EUtk*k;57{>^xv8p^etKv{wsYXb38se32&)uJu^tGQ_=B;C$}*a^hLly6XHk)6DX zk`@$cxnRll>71@>aa8GCt{peF$6&JST?Ia;Q+Vl$xnYO@PXM;(4xdH~v*2DuIfVwq z8ZY^+WgG<<_b&8hJ_Hho?s%6SEvOd5l!#r1Hs)h-fN$`Q0_%t{=Xo&SBU*{o*EY${k3JRUhQ@nSp!rKX zOE)18_Nj%UlKY9(4K>@%$V63+ZXqLD9;V(IwdNEQBG6+P^6)W(f%OV@82pa&3BDjc zEbluTG#hAQV`Nuu$D9{NA@4NR!r{PUG*GM`VEjM$z_Rksg&ynF>}((|s>E*tA{aa5 z#0*mKra6RDelf!?8r!VIfImxos7Wquq!_xS8?50TzDe3DDv~k-Mx2n8P=IY$GS4!i zT5=FnnY^IReh*oP5})f)bJH=(hCs|Ny(N!)i7AYLexii(47 zmk2>M9a8%3uY}0ir=>Q8?6TDH>KG9+B)~E2f`+}(C7Ok4fBKAMb~OD&AsN+B(IE_B zeRQNds%l9eL-n>(`cSH4{Y3SGnqQehvW!ndOPlBtrZ${%qcyY9A#xBuspRT78tw0V zcWGNFb1DrARJjafj@=9gE^Pe4j0r=Wcj*v<1c6@;hZ+m7fh;Q72mqwRNabI~7)k-u zWHbkdJHOQ>BP6452O~&HnzDbDaK@McztSa1*ukO1b|xEb^|VdaNGWw0Mm$@1m=a*` zpyZyMM;9)F~w zVT;m|h?KdQe{5AlD_bBhE?YFXP{apFuU!mr>5516c(Kw4Jd*r6VqZHds6ii*JX;x~ zd=YUK2?Lqryk7~DHIM!Pa6ojHF{=b2A^3&@9zUJ&f~k3c8}o0evObuO}4sgeOR0vogp}2pl*r8+wkXiw_@65Wm?XM?=pdw zC4lWmMS%Mm_%sgvA7kc>mPNwZ8pWU`{=fPn-5>%HL5t!b*LYku;`7kE|gd zcf$F$hl+A$`Ea%U?#aRM&yA2h>%e*H%L4bJD!x`rymw>pUz;~O_#uV(5a&?xbFZ}v zsD_dJiNN=O5gnZ!GeAM9C@7YKx7~+*(1`anIz$CoyE&Yi{HFEpPW<<_LLQy*k!~L^ ziIh>f&@ok~h@z9Rzq+4O;Lp-Givb7L?D%U@_Z>w4y*0wrS>cfEBS^*k#A=LV5)4M| ztnLGTr!eMy*U%va0!WAF(31JT&kgvV_cFL9TWF#*RAC8cBzDR%TPw1(w_&(pgrVHO z1|#)87}uS}@Bd>Yz(IU|SljB;?{dhZN835wS-Kf#223BF;|+ALW@tFA`-RwS>_h(DC=tbn#fobOf0!Kv+GFhdA}^G8eq z7$97+F693e4gUg)cM$l7Gf#3f5h0Ab(6h4cy>^naiHA2D{KxQHYQA3Ai4fv)^8}^xSfnCiRU>lA$P`chz!SL9 zg?zvtq}Sh)`ya;Q{m}*=2nT&j@o%+%i56%vaX=8;VCqTw9^XyD|D}0n!c4>*n{{+@d0IQVagTjsCdF&Lr17lo|XSTR-ykEF#8jZ};?;ix}J;>BM{QfbWiFWmiq zKuy2~8CpXe5Oe6gk{p*s672^@W+C_w!EMlkz=N+OeX8USR*;jYi@*Op5usEd6xm|( z0qDW~J;QVV4;WrN*?lbkLBT01G^aQ~L)8CR32|q{m&0j!=;fcgf$xtG1(aPBbk*u9 zG(iA+BhwLip@1|(zG8LOzdsB81yuPNW*Thx+q*4)$6PcDG?h3LF%KKQM1j9Ubh=0x z;@|SSkNp4%zejm+xyVhy>owA#LV#b_Xt7skfGUoKk0qeC-~1uGti*1#R01N zWtO8-@vy9o1phAyx58wBS-!lYo9XcDjc4M?l;;b{ea1(a3;2g7b`tm~s61EC6IA?1 zCeS=S1D%)$o*`~mTgF#iq+Btp;{E9V>5~+>2K=+P%~h9CSMAp-EOfx!sq`)jO&35w z9qFILEd9l!?vRMmj_z^0}*#&LR0*~mUw?6V=xc9=w!IR-}eUt z;lbU)W{egKNc$^@{!5>Ih!vJY`F+eOhKc@y=Xs=n0#|HUyrup(!89j2L-T(K*r$rb z+vHbHO_6t97%V49*ND*Ha`px{_Ytt7ew`7r(;)q;ci@)iI~CK)%w82kPEcdW^v`c; z5`Lg+7r?>*fC?rX|5Mp!KKd*vJQtTY)M)?b{c&jUbP~=EQcXap0gna3mwyBN56t~} zL3n7kk|#680`R=1jMx4aBrxg_B8(2@@0$8H=l2%^I~_n@Pkij!2oiZ~VGl$aI-z3% zT3fY?%gW+^{J;u{LXz>1#+J28vaWzj5)Kj&6l{-AN|H7*$|EHs3zF-_pQ+2s6A29q zyCyJ2TiUC*a0Mh`y=!+79jHb?bu7BpUgK7A^fr6l$dA9e1X^Ok0ce2>id=o&=-JTO z{|Flz67JJ~jH(SSXjLTAX|cKX&MWG!%RS39!pv{a=y{$O;pxf~=o=CPLkLa6=o4`W zTfZV=;`f6B66j1W1LzkZJ6qs(76Zv(?8D^!A>G6EI&kAx+LM^+e2GF@7gm6avl zNT7y(s_{b`T@WPIn~c-D?Cusike-($4DaN{$tPW3Clc)MhxsR9B26Zr*8x`KR4N|OD=~;j2uIO10f7FS@vA;k1BK_^Ylmo&q5c&_MaJ_k-`0RqUV2GSjP-kwyvd8 zGt#|dTA4&4Cj%R9w$@Nc#-;wI{a!EtPfr2flHi+4F*xJ=6so2_T;2Dc!V1aa*Xo+^dVI$mWe+8%|l?PVC6PUa>rSK0MUDQXywQd3h& zkXF4?clP%s3&(VJk5Mos932ze=WN=N`N?{F+i90dTc2&M@(bo_XNq-SAQO${Ke{I; zQS7$F6<96b0_GO?4-8-WEXIcqRfbn3$NqI1N(JI2O7?}QpUQO@v0e3|%IJ8nTMdn` zx(FbX%C$H;bL-@!Le#IfyIiSpZ`D=HKV$pxnOWTNY^@k-Bv6V5pQ_J!2~uzG;5QfV zRbB2)z1{1;G*1cjxJOhKBg*?4fRllE@oGUC8){Jf$2=s#fUAfA^d%Kk`i+C@t4VTj z*5A&hF(e;b@c4fl6|ZOMTPgQY7`V&{Bkh+^!cC+FZ$r1-01DKt;Xu; z+k;h~GN<$B=>@vq?I@$J7Z4^=d#@f$$7OcA10en2I{1p&Q7SWy9b{#ho?9TRFh8;& zIeA`phPiT{_q5IuM@Bp@9|}L2)6;3y$7J`6DfTzHP1s!a*RwvRXys{K*@uLNql)Oc z8McoMr@H=3xlLre8#wMTf#niDYXkOvm|BdAi@9$OI2FGX99jLV7Qnl%A;O?1KE5ti zv!Mef1~4;kLRuc6!gIR(#5ba%D4{l4*Z*O!@WemvYPA zNbA=2HiX}1mj6tH4%ESNWrzLQmD2oCv=- z-_-ZL0O(IX7vFF1m;A$N+U+p;Bvf`)?X}gLGQMtn(3*%f1KvdU{%tgM)ys zfTX9q5TN`iFaWWV+S;uvF7rKr+{4b^=L!DT`-n8=OJa)RpULlrTbo2ddC$!@eMw?UzA_=TvO@-mr%r{vXsPvOZ`jo}TAuGoHz$`DSWN6X?(RB^E#7v270H5v zDRFcj&l4ANHVq7l$#^p7p^uvL=31ZW_Vy{?Y%9O-AM!~jm0^5^<-rO$A^9WL0OZMf_^VOP&WP2&jJR>Y1)oj zn?oV9A=b6oG^Q`u`o1akb%W6_&#<5V@~p#RCoXC&W>ac!h2yTfe$)~Z=z@NQzTaMq zf#t0D#O+n&{8P8t`VO8V0596bVTJkf;@)W{%O7yC?)4SubU0bAf3qe219%Nm)l396RkNHv92 zZlCORfgjXuX_s}XQU7hZlBQ-r4kSGj89Iv6{9T3EaPB*eX|S`ehq^8}?mt z&{!Z|l&|1rm$bou?b@9T;J!mK+KSklNn(%OUk0qpfyvKomg*EOj;GPrAL;Tvg2tw# zMd8@xtvBnO^E8@`e9Dy4Nq_%3oUDzyK63I;{L&+F5B78?t!y8kSQV)}^)vA0;)#Wd zhAvJS1IG4O3|Pc0Yx~{lDjkg)ZNkk^h@V^xaATCpD37{UK(%ThcEhCqL1Up>8IRB1 zP|_4!YwyOiFOVi;4^@;jCWALusI5sQm1j%7YI%FVS4 zQ;bfl5=5`k4U8vokQq81+fvLeKmeS*-fmhl8nN+pVM0QrLYq?UPp5RaDEHONQc7J< zQWB(G-K6VX0#Q63j_NWUJW*m$04o<1%R7zkiv}7^fZ{Th5wy72Jj22Y!2Ko|tzzpQ ztl*m4Ns5XqhS9CfGa(qqNz$i;ojVY0*ExxUtTA4bxe^C0lNRSzUS_)vnFq62Ij_0% ztCd8Z##ynno*xX>7u^r2iD8G~K>PsC1yd58AUs)L-dWj+t2 z`Tb-Uey5~WPU{$UE}c(5t;4@DS^yK~gZ>pxwvuf`nTS*G{B%-wFW=hKJ!oI%PD2L> zK#T|35JR=FTK*tM;=R<4*avMzx;b2x7t?cT_f+ew&8LXmaazZraNhUM8U9gyx7cfY z!^$pw#uucrjOgiAVv?w)>#BL)uBNO^$swP@Vntqv#M!dc{;CjuVVsfen65NguyoM- z>D?(qRselfc!n_rQ^)#`mXE`LCy;1790?5(HbNe>gy?yg+XklKdjDQL`u^Ui3$o>Qh2lG_l_%~)WweTmE0P$ZCAmd-W7=1 z*$1V^E@;^mr6#kTfmctk5<-^Y-`ewZ{Djv#o8)q-^CNFzfpFZc2EtHSt% zgoCni4!fVrqUw!|6x>=FZhMoU*U8E%h6t1hONnb)gH zu9Is`Uyn_WmLAvp&om?VuNd0tC^8MWnO2+s#-m-F9 z4QRM7-k+&Bd=licx7WnogUe*g9b$~I8oOjF%w%(@$w*pgCc)lE=aRAY0IW0z$py>>m;GXDhb=b^myaL&W@7h+9R1tPDK* z$#D~q%+LXBD&YHA&fv;*Ln7cDMk0!UbaOs0(TpFOZma25o*Idym}I(2kh@>R#AwIH z#^$RWvDN5_vYf6rs#)LJ(|;aOVSg-vGg_Pyj+EiZMhQttN-E7?D%)Rw+HN%a7(}(F zC_M8mC$wFs$soDY4AJDs?{1wu)9WE_d5l7uL>3O0odB?esE8=;#1$yBcyEP*ugb2= zE+zW-?U2b}d&|BMCGw~s;_dLaZSJ`#+3-muk-5{1Xpemz^wj3U*VD?C6-)dk*dSuO z6*973K(7k>!iZ}Zzuiq1SSqfC?XK^Go&5nv#;VFp3;XR&(%bZymiRT2@#E=G;R z5jn3DX-))?X7nxjdizZC~9`cUBySI^{ha@kuGwl_={K7frQxg zV|-@mUSOc|lZHvAiE`H|d)o3gzx!}an`oG-F9?VNwSaO^fEcZT5 z&)5lkd8w6Mc4UJJXf9;|O*Y{{E1WAWjE{+&^Ml?WdHwR5eAmcgu_lBR$THM;r=!=# z&dE&4&~<(c3PI$whROGKER3HJVZsMLc0F@0<8*RkSpVyGgx_=aO!RKWAHNL;k(H9}d5olA|%x9 zEpO-)cad4r2E;$v`L?p0>V%~+a^ZVD zUf}Mo9B0um-bO<0>;YqkJ=vq(Ry(=bqcJ?k$izct^AKo<9$l9QsxUO{m=q-5$277f zJVi!B>n6?5SRi&i`g|H%_*TK%K|A<6ACS0@B{T}T+zIowmmy)y)%D26BO}?Z;Zatj zA3ZO2ovsO?cOdu-zqdY)7etC9gpY4AY1WjDbMmbFX zQ_S-VU_N8vtV1@z_5We-Eu-RUx^+gS)#0 zr}4)1cHZ~f-}&CX&p7A)zjq9NEa=sAbycldHRmIBjX#(h)mdbE5d)t#T-kn^rc6`h zr0Z8ci#1t&R5IJ#xKB@h=jfYdjFr9>+)(frSXG6t$R8KY0-5q|O;6>~0n0vjFZf6I z_Z)gtoqJ_9xVT~F8LekUc)aeQMX&grsp~nsy--gnU`kJq;x;oS`>ajX>G9V7NN*IM z{nByq^k6UQI>XYE7B|i9pAGQ{18zmZFRmhR9~8SiL8JKYW|4k_bIeP~8_R}*9p;<* zYd+8f&E9hqPSISH&B^vSH*aMWCy0DaUKanmTV8=&Du~X8y5Ddz*O8Y=(CxQd=Q15i zfVX|e7EAduBqq|#_!$^-D3HNusQnCowk;rnu}LswT7@xPMdqFz{1 zki!a}9XGeE;xdJemTru|^``kFC5yVSr{^6PHjTNZUpuNh&?Q{kUeiR> zWRAN`gQyqn!YEBZuflG!lj5tf|Z>KCw^DH>k z?6vS6Oj{@*872f=cB3_KmJ8WE&2jW=0Gz)~yv5ct8p3l5GV|KqHFy5SGQVcBz5FfU zHL|rNt#QwJJOQ1}4yoUt6*q=({sz2rMprtn6uUo;ByT?22;Xb?t`-v)G;-8$i@z~p`b;5BA!k`z+vsoS1zLiq{V;F?AW>xh7s`hVg}-r&^GGNxr*T4N z&8$rzl`y0}jdq`uqOI@>N!OVq#}bSLXIK^ZERx~g(tL8~2eP7hd~Icx998#}0A)o3 z`if$-v*EA(acSsqewkQ%zO@b{7>y4DN)};{Dh^hw&rjj=-7zkHaGE3+Y5$c34Kp?D zei~pr@idB((U7#<3!NYHUOdAr+3f8AoV*5s-;OcW4$j#JcCgYuLS6 z9nx^%wF(T@n5|0!MOqV%0|Ka$w7|y*8J~Unn(0!z)hxc+0*oKC-1Nx^hrr^KsI~k# z=qi)Vx~!LnJj0I=HzPYM#bgJblEr zmtpHr1D`A(tyUdx_Cx28g^#b7RyOw){JKP>R2O3-(-E~_W(i-HfLCMEstC{G-F2!>5QKaaTM>%w_gL47E50N+!~4HADr|R-;vesmCerNJ)O^F zUOw+6a|_QW2#+@{@W*xTmnXdXyacp?UtsU|9&r=6ERh2VG(cMx0^kiHjm%`i-F=Yl z{1L^O&t1Ok%dWi#I9*|vhpl~JVM9IOafuq$B!k{hDacLWiRG=j|XJRA|p;zLfo_qb0k8eCDS8D0nLQ5_j|x#GI4#2 zbrZmnHk;-A2JO!Cg~8B3rY~jZOSNS22R_;p7Vc$PkOc+Y$1aa3s!d#0?2&~HIc(j+ zrb?Wt9*5)3>wcNYn&Jjzn6wSgZLxNg{h3mG?>w^F4c{|>OJzzf;r=4RhoTFj4lVlf z3m6zF!yh>j)fsXKC=oaBM<$c~h(M6mfbdqK0Hq9iHG*?qxi%{; zS`$X5ocuAUIWpcx2hmKVZ<*YzTf=#G1 zPJ8Z2+b%m(VLYZ*3e=7kTdx=DBC|6g+1O~kl10|kx2P`=e&K=#aX4%!S?hU-3NO|` z7A-Axb916Py0gXV>&g1L#?CeC_lV1lXL+VI^dWkhf^jIMGHAQsx`_`~*+>O8VSf;C zA=Sh*y`nRXlDHq9BW$aUtzfc8YZmUT}PG&48q=C$C_^F5~s}#i1{dWPvksc)< zbl!1^YkP~3z)$!n@HuvR)XR)CU^$f$*8dQWGRRa=%?}DaWV3}Hhq#s3`~<@Y=uj;!pSkp z^m&mRsf}X9a@rJqd92)U(MtMiHi=j`OSLc+da{DbC*bDD#8I8gYh%lr*$aBrGqz5& z0{uvLsmPE=pe-yB3Wn6EB5hPWe2JDDY~o3Tp^ z1evD|Q<63Gi7Mw%nGBsTqc(==rMEuQm&a6xt({?K_>8i2;i8-xNAXP6Fp^d>R~enI zYI&kuAZGfJbF4WfnzmLOmRs}|DZ(9!sK zM242M{6=erv|h(J%=Og4dyx}_pl6ti)~YJ$0$kH2eBv#6zGJT3pFfr9mYxcCiv8?Y zJd0)~m%OJg5Kj$s98ggCtn(P&qYv&4R%H4;$X5mh+}N;H)|a~OuSKbb5%hC8&Ri!B zoo7=Hh8q+%rN~7*6Vwx>j}%{*s6wHboR-GS&%ZN(towi-?}|m5DA3>Q3jrq6tZfzc z2PcABr#d#sn);Nl|NDJDdH^V1W5dFc6d%-f-w)NQF~FBw>z^`|tDq`Y*YeYRzPkkY z1Lxe_HM;d@P`OW7XeeE&o`-}Lq_l8#=+z`2;3f&8)DY_3=4CSgw>Qw!yc(b@- zt5E*Edu)KvucqP?oT&Y}^MTKde)>h3&1xaH>oF-M@+W0ujW!Sp=aFXnCorq|Y304H zv{)+MV$-+(DnkAji33{NAAKleQ$42VM|?tDF!^U(0c-EXID*i1Hd1W#J_}9ftL$b# zgv*}2@rx>$+kB*7_v_-sL6SoP+;}PcDWUNaa!leKanGBW-d(G!d$IjF@B2pbb3>9| zEMT}ioR<>HClfugkle_O;m~2cM|@hekN^*B^bZ%$ZU%@Z9aGeo&v}(5H^<4B52-{8 zNkl7>)@MB2dy}Sa%h9#u4D93U6`p*1XmzHDA!^F;4GnYi+yHV_S6B6}QX~FbW{G+- zS{AN$!V~j$+k>i@%XOa0qq&2FUtF<}yq`j(`ZCEFUh3!Pv9g~_&k+`pw-B+$jBj!o z`SW_Li*@PViU|1vp>gHh^zCiYAf4No(j!S_CV1n*GG|ZF?N1cJGtJ=zH~*HtUk&=J z=d!rpfQfl&c2v!`$14pr1}_plLm1=4yKvu0yQ|;C3vKNJANvp7aAeD2s7*9T4a!G2 zRGmhxsOBHtFIUQ*cC7MPb7GMLsd;(6q5yh*Q*_zT&;kCLo)1jM^}Y6f#h;DNZ6Ki8 zNy1vQj2kmttMC+D%t0hlBtRsPRs~tNZMy+vbK2q5r##|G#jHgukQ_zw!IrM5>7VNa zWT|*V<}jH1p^8+bGmG7p)#SQlw}nKf(JnRvWluXWyx!=CGr%)Uy`P5JzOn2Jy`c88 zp2*HO=8YBg-S8gWmhoe<-7Xd+@U~uw$=k68=OJORLMELX1gwdjSmmlUfz-X)pk>;4!8-qMCJ9(}&?aG-)FY)5&TJqXR6IMA`?G zJT)4O3=dDN&UUZlt5%cQt!L(>=h_D?N51jqPU@KZul*4=N#%I|H-(rCz z=&(+=r`jrPK4H1VT6kY9EQT)qQxukxlXDYL2?l51$&OE(Bo_=Z->?C)AytaN|FMaH zR4gE>lCEU}smI>D$`fv@Et@ag1(UebHEPy++SV%HsrS~|CTguJ#zS{y$Y%l*odI}s z<|fW@xUc2Tssex(WYEGzk;(l(3D~xNO3%By)@Au=gD~XIcIXta4QhUj{#$(CJ zWZ_eYi^TvA@LsgO_?NfiF3wh5fXT<)Y_sC~d+m>iaU)SuBCl1%3NvFsr>_;nl?s9~*|i_OlZgZ81cKLG$r+WdYd@iV#kyPL zzzi2#L$tl$GVI3Qr|0fQE;V>nG|#J!{8x9`p=Ak2;#VD?S@~AnHVUIyDHXpg)kx>R zK(eFkzXJO#M>mk=XZy{fw!mYhfXQ*vl&9e*xB&j!-z!cK{KfXF!Axs!1c?DX%Y(i19B>TUsrNX_Q~fC@e*cRM^fxoh{}qvQ-uK$? zoD;xsj;5s&Z@=yWqR~Il%CjExv zLND|_84`x&ek6{KW5)^&?6{Wb+tl4|+mj#{2T2hORJg)6^tWXiMX!@Lkk!xjsq6xo zvX*4PY(1E(Cp_k2(N5I$D^V^l9ivVQf&rUNH1*L#yPHA$TC3~qB0TM+x(NUH9*R&) zjd3B36fY+Wj4kJ0^jg5yvL`4a1#koe0Va5l)#l(boqphe6JM|Qktj4Y=ODxaVx(?8x8Pos^~zHtLtB58_1MP1BwYT?ELx(xQUW@t-&hPqI8WE&{ha zUZ;Z2rd3vn1!gDQi5w_ijm$Hxmuw-)%F%5!wSdAjh}dPVUPocXRRbGoF3+;_natR( zRUzCm4rz{D1I&E0l4)r$2KQ*Q*0QvyrmIxPj4J#KP+ki_T464atyhA~Dun+rs{{=Z znMjv@;VU;XcHZS#%3X_zz7zJub^4lEih@1*^PR+RHVu7Z4Sk%EBT~Q2U0O$*iNGUs^V>DuU~nBBZ0DyQDFRh6 z!%Lhve|m+KIZeS2AR z0GUbH)D~>9BN+yd0HeYq~bGfuW!1VM4@Z?X0DU2MyB$SRAiN1}b*;-YeGah@EeH)pP! z*}zc-*A))KC`69HTobm2ExgXl8bIB1i-!^FulK*(#Ns%S(r-?kJG9jT1^%wWV7&p5 zmy^B4quJr)*ZWgXoi0~%8bx8xbsbTSD=FQp@C>1kCjbyysgUfSb^0c**r~+~N{@YJ zV9+1Z>NFUWI;Ft`K~CUx2@4Ce;*HWi108d~F}3BuWise!v(2HdpL}v(d@)qjxi|WB zXf~#E@Fq{unA`Fk?b$cLXDvd!VSGbxjvRE=Z_i8g(rQ=l`^qwpyAeI{0zrfxC{3Cn z+ahzDn#YLyfoev$vp=bEsbDhv+OnKQJusAAUGVPb|JXFk`bU~zI&jwWL#hcgDMw+! z^K5(P?VeX-%ea9>0%d+@=^9ssp6}2l*Z7}Tt)Zn@YYG3PuvcXLBO9KXXb*HIs1fZ9Ur0OeG)K2%13iuP?JLtkcZQ7QIeSE1z_=iHx|89=ViE=UmpUg z1K$Kk3SCtgP4KJn4Sx#Y(D!~}2J&WB5)3spC>j_;b*r-7K@ruG)F7@pj(b`4j&n~a z!(HMqopV}dtoNK8Gofz&suY+Z(g|Ufd`sAWinrJK%l|uvzL&mV*-S4p?_R@WxdzYE z$(Tp04zeNh-iKdMNPp9+{oP=*q635?-1~vTC~)LU4_;fcB>NkF3z}F?ryXb2R`HkW zB>Mrk`U?};ii2Fcb=eTMjMdvBBE>xhs@9N6|*P1b>212xXL;3)zpA05%CXd4l~)zuS)}M{@f%bALY5M12CT^ zlGQMmHA@qd2wfeNb!dTtmOH(-8So^-kS$?>~$X|053R;!Ee$VqngEeVlKj; zhCxKjyb@)go#8Y2^inLRD#mqN-Vld=o*{!;--Q6H2)DOsg;4_cHJf~0guVV9*}dXN zw_->oPrqwXU7YG3uSm>Wf5!X5+a^wR{CF3co$}q-tUP7M>P=GS(;6xzLS8E}OwfY9 zhx}$-(ALtr!z}*e;zMLa*vy%Ldtq;vJnzr19rikKls?+IpTALATj%CPu%W;F&)gq_ z^XdoAqEkRYUD4}KfnIU6R?SDoe#S^IjS1W9htdfx@i_c@;;$E?6wDEcre1`y2U)`K zllYbVl|1L+b;$Syhe}6o8;?c0?)veVC}dX<#z(l`Rio1FV9}rz=mI8PBFX&2_=em1 zCk^^{vx~kDe&{uK$dv2b3hi`^o#&$U2gahBQ7y-KTtZyyND@vas_~*6)7=DfxmKwA zuesac5&MGS75g@FfSUP-oTw>gD|Tmyj15l0mUZT7AaWRT?pC_@P@+DkY-Lo9)q?+lX-ixDg`j>C}4PlPOQ=-1~4 zZQI%l81s=bptVOFTq4qtE*?w5@zi29#q{g`kg-+WhaQJ_wjoil4hZPm4r_ml$y+=gkegb6sb% zEFF(kYvDfqWuP<+Qkf(2zjQ*!nEv13;b@m@@#p-7Xm%EJw_iN{Xm`30ABRUKEcVw& zHws4}yaIY`edp$k&oI(VjK%x*yfcd|3Ue}}D9x2|S8S^G7#J8=V781$pX@JROFJqc z#sYEHNR%wog5^7z_kgPb^-m!Xx)AfBr%tL(1*mkLeiNzBf5UI@^<*~2Mi;<|;xz|& zGRsGUjVDGbU_yfVblL`y);$lA9xw@fFveA?T<`OfKfL+5kq_i`9iBO-U`wDe&C z1VqLme-oc|M(L9er~E{>t^eFrK$aZJZkD6_KOY9nDt86As2A;9eOa%9y*5!`$$V@& zGPGLU#0b!a*dz)J2F!&yQd(9J1kR6Bdj;mO8a+19*Enu_TCOmNEY}TH9eZBI=+z@> z2BDWKyO0_M1$vpRX{!MWY=5WXnGROh0i}8Mu`R34aPuOet)f-3Kl)+c1{Xf*OmtP8|F!7b@C2 zZuB5H{_sjV_@mQRAu_+Rj8!fxVNLvB+J^HFK>kB`pZNIk!iEe9=GUV5BS6xxVW+CZdB-)sa=rqAAu72_OZcJ)0jeB6egmxAc|(4x zBhWCrS#U6c1QIs`1|wIhw2No`AclI7DjV&NVHMV%DjSj&ts)nY@A=r;QySekwRFQF z2N%K|%)9r!l<%?Qx3H?) z+YH~A4jAW`I+wL=s13?wL>0Jc)35hhKxw{sP?9g4Ih?L<^}bNM4n7nbxWW((X|$OW z_kHq6Xhuj(*is|{jHof3grzefINv<6F`PvaGPasDCFF|dopSsvA#EJPZSox3YCg1K z^BIB=R;Q-Gd%WSM<^vmxXC))zN5$gd8=2lW?HR%YceXDLiIkDJIo>C@Ir*K+FwV@x zi%b#=#g23r8SxqG(jz+fhoc*u3e0lR^;nJVo_Lnos#h(c7UFSrWC3JLimxm;0Xbub z0q_TzNsNRgI{#9fykeeTgX>?dM=}0|n)$;%32nxC%^%vNdrdzndE)FL4t(A4ede>m z<)++RWSda?5l(zzDq52Bvmak1s0ZB&0z|>>Q|wXn7qTPE>vJ0DrFYK7xAXubqj=!< zD+{kB7sk61Yr!=^?X1Zk z$oPo-FDyE@*!Ck%`)$upGpKZ676XrK)W zks@AP`j^rq19>*a>O{=a-kV$Ic}Ch~Y;&VLa|BinqEXE!{KZs^jhwM#wilx9u%@IN zL7zP2A;e!2`X$@h-ZF=sh-t`awquGc%$IK0D<8=uED4K$Ey}n(;F=%aQj5AwWOB6j zw??;Hqnx=o@HdVe6m@1d%+y<>di5P#?-}nE+4la(GB{W_DnHfv zcL$sGVo4-M;+30Lje)9V#qh03pXB~aE%9H)KOkjS@J<)&0tJMne*jyj_IB`9-}ewf zt77j=JF%{y|(~O(bD_`SL3&~f67?HXb3B$3Yc$!$Nq=acv@}*}~ zezjZTsy;~n`ciCyf#ASZ$`nDs9))_J&pWo9K`>D&J$ z(wSv{Lq?h$xueaEp;Gund{)d=!_ai@MBfhYA&kTAIA5ku@(0c-)9etf6cIg6#PqAC zjVN>b?MIIG9rjb3UzzO77XI4(cdn{2@bz0G7cB}ROe$)Ig!dCfIk^gGLzhe6hqyh* z2l|+JAh-v&eAfb@UiPu%%|J0!Ab})oFDD;R0D)D(55xU+ZDA*3p(L-3a7gm87G-pW z75c1$Pk!BVnR%V>_%4X&v<3abuCORWXlEA|4XC>Y?0}5AXiy#Nf-x)4!`3w zIQtp3QE1X}Is?$8`26|s05%+o0|^Wy!CP8?YU%fV`V2HA_Xjk#r?3*Z5(9_`hq~RP zpEM~)?|GdUCG~m`DbuAXYlyU1t^ZUy&%gWALXarl+a?N7F5D+`tvzVO@6bS{L_IQ` zu2$b?ZI6SO6_cJ{)EL;2VV->+GyDf`%EG6sTiluWxpk?P16~>J!kCsc0H$1445;P?#8E^ z=vZ~~Cp`S;k0J;bCq`E$C~Lz=V)$wUdk|7vb}_l^@bQ8LBndq^JSLnf8u2lHY z&qenAr!4q8&l{SZ?~mwakAhGE#QG4=H!X-VO4%cnG>({3P%M>LK(OpTwRC@|GwEay z%{%?4%fUe^T!-1}2bHin!>RvlkV~|Gb~Ip9u)vNF0yZMQ11GkN)p@=n))5EDY**G^70o-ysVR(pc5rZleGEu>ZVPhmau} zpnEd^^Kaye8VHIcg%t#r4gVwG{d+lBr9?aQ#VGjKJos;@;Q#Jd1oJ<#Vepjc%D>l> z|I0-e%%AEsRD{(3<-Y$yuOx}nAd+l?afw9R|5W<^ADf5@F|av)^X2ou#d810yZ@M9 zK;+U+oJ4vJ|CDk5ADYNZ3=vqvPbU54k@J7J@82)=BtvxKTi(t~?El1C|98FmcXQAE zA@BGkXpHgy_6`64VEp%|=ij~m-!A??JQ$*0(Lerg&mM6WEDf#LjamH2&9iB2vI@gv z%?L$fmT;&HZGK*vG7Sy1HsMEcXa@9P)h+MpDRc~MOcLslm}x3CpCd7uFma{nI^Q}` z(WW?Lw?F<9Ky4Dp<5g~Gp8bz#^q+D2uj_GyF0M8sjgYcsWm27+1Q{dd_P+5XM=uBT zE-qbF2Lf3j|MQ{UJ&DQ^KMlViJBR7O1XPicLZzLv&6&eIjG3`=F%{esZO!w5UkaD(-X-qH&`Y^==~RAHvb%h0e9R< zHhr{e0=tCVor48i#+dpQnRu9H+1VxwJjCw=Z)&Fn<$;40FjlhxLwIvxvGVj^^Q}QR z6sAPW0ZxaK?1Y+s8S?xMV*Ob>nb3+9QPS7Ez$V~vo1wAN(^14957j7;IpPupl~#Rc zv*@X)9J`%o>&a(1Pp(B8P-p-1cB;iKvSsGCB2VU;CH^f1P?56A2@vwsP$!56ak2uo zNJA+#f#-`Xp2qr|zGjLZu79qByIA73!`Z3#H4mZF)S=V4jbB<0gzS;a5RNAp5}%g> zS-jM;zgVK>$HTQWM1G)v{@^4o;WS9dqW%vRwl1P-Nh9c_)ycHCAwao|y!4MX6**cs zEK`^1eM@_d9iX7jeoj!ET_Fa+0udf~!H)+EPx7k-ZFCawlDs2rp7)kv3lBKKzXo^dU&`%cwoSwOEYF?gb$SRumEFm=D+vKXSd% zlx&qX+hvD1-Rh$&ITj=@QFFTw1^DO6R*6V3NE63~&qC4Ye>tb$YXS#p;~NtHH;kq1 z$BE^wfIJy-GO1D`qG+d+4?T~NW1)4=-T@7HGWCjCZ%Kd-BvrQd#Sk{R+;$JujssVo zO;8k~&9dI-ZNH)T+DPd*b-AzJX|1m{E4gXrGuR&nBcHil^3m!FnJUBv?d6%p4c5x@UwgO zWqOS^SG=Rb`#MPXS^VD8)3?o>*pf>uCj+LyCfL&?Fun}tVm5!qs~`bd!Cr>ZT-ATG z{=A6HG7p~BoydGj5utRdBBGnGR7ViJ@$5NkzJxf7CC>Fc{%nmvL3V3wE&u)UkJg|; z!@dh0HW?aTb0!Yzs{Z-`vdv1vTx?;LKr#Cum3cZw&dkrWhTIm#Ex@qa%L-lZ$nT}D zQa}ovq=1r`AsTrn()gx~3lTO6il9zlLIV&?OMOSyI$MDe-d=#6s6KpK3x-vi{gZtU zzZ#s(No{HJ#laUP<{{>Py<3Z4T|l*8P?VF>=KiRsTP zaBAB$aKGYCbj@M3M>Ggdu9M2DKu>D?ArG-`8eP)gK)*A#%-+O%$38+^7B}HLl8?!? zh=K&doQ1t9KG=C38;nU$?&i58WfJ)w@W52<^v`b4gif~N-w~t1Xl5rjgjrEefLL`Y zVk5@IC5$|ow&rDq%F35!Zeqi>MLoQwWvv$?M%%BA_}UDyT1$b-^VmWFH|BpHkba(Z zmy!=x%ftx@3yVoALI?>BjT;<9@WY;X0ztiLn}|=;khJv3$HGpO%D__$Qc#zV+1ZJ| zi-)Sn{?&wVxWq7Cx!aV&l{T13W3K7y8(1pUM1&aK{5^mjs9$jINj!0^F?933$^ML%iAOUxO5FV zEh|HrV(l}=-gdjGVLg^C1sN&jTT%m@v)qm>oEDGoEB6E~65rS;-%KDF>Ms!O?Y!Uf zxX$mR3UXYz@|@1m3jNhL?QtyAFggN&=44DQ7DB$mN{kzcTx+iJFmLGUKBn!h|89MB zIipNLUEylREJnygdD%g?Q9trA6X6;~hmk`6JTf&7jUMrW8KW}o*8MgjvdnX$kN*qC zd4ixVzy0qUCMD$ICTE9|ynNV+4OZ^hcD~Z2f4VK{5-S!EaLK5jB-F4F-KITjxW>kU zM6Ses){)b13?y1$qr7t?0dfmM4X4MH1R$r|9N3iFTd{J#5O+3luo_5=>|n>GG(4E0 z-?sG~t#f24ybn=V&RK_a!usU$t4$<4%S1rE|$o>I;i>$tr1!MMEJ$1RgLp@Oyo_8&_%YR+DU6O{r~$l19$#U&{4Dw{gspsuJ=+-v|Y>jNEx)JL3#VAv%ktz7m} zXTK&mvD@e&k#M{oWWl`Qa@8x;ZuRVYIM*DaaR-SI?+g|xQR=~5{k5kj|K$a6jEaCS z39SyRZpRQ4(dvCm7yc$LeWg%}V7Ihf5^TS=#Q=+v4#o)TmZMV&6E7OG+5amZd3m9#~9elM4&e76T~mM6kt?~uo!qA8BUmGNd} zWZC)V0<;`z586HO++^6?`AV2s4#)$3?K1<^bJ1j(|udBG&0KJJh%FC;iFk7G5(yJN4#5$ zG6Qxm;XiY6h#Syq9%O;N)!N3Nm5B!W=twuBL|nBE@Q)U;g6|^uq*dA75Ise zK#!Kx+9LUG?rCscJ7Crwcd_zq`r!IElZPylNlE0Y-d4hKTYKb%MO_M3LaKnD))EEw zN5y}E`QF|yp+_7uff4dR?JLB$> zgVu4~kyMJgWn4@uECjTEIozmiCa0y=);>T_x||{Ba{D#(zzV)Sd}A*d_zruq!zYYR z|BHcorS|8*zy{NU&#!k8`5Cn@Pi&jlCcthHsG5bKQZM<$@GpECNsVl@Gz^%lm}_k7 z5^EIw&$j3Y3E(C5wWdheo@7CwlO6rTg4e!!ODyzhC4rpkHsXG5zFh-71A}zt3gVO(_y$^q@ zw!*0d^p@ymX1&<&aw~hCU5lgD3L{P&x^U+4|6zTjS?(jGca^Mhf}JxMwZxUk`AS#7EFhp0b8jarM(QtoL=B zmZmO=%)FP|n8*8A^r3YO6VHE?PO3xHxU2C2dhm{{u;OUQsITxkh7Xc$VtSzG$vAI*s(^?^{md) zziE_4V4@OY1qR+9sSwB8p=^Xig#GlqP7~OSi^Gy}oqvO$@6{N(`%_>-K;Litl(g;j z`@%Cex_NW+QtWz8Jvli!7thnGJS4{vgx2EfgCxyIjv?17+j;hl1&|?m;$gi@S7oIu^%^>&N9-TgkGujDs&}|?6RS7y>O#l;xDouZXM)|jy1I>IfFZxitS+BFf&%a&y_Vn$(J`S2-) z-Iif&M(5or{JiT@UvFL(vB4jn@tOB|{`FqfE8!rR>)5CUNNyO=H64*5@Sa05VnMl+ z-A<2P`dzyk1qFq&uI{zl7iwxPQnCOQ&RJar+54{`d#Qsd)Kk3ew4hNJBco)3(Tig( z>%-5lT(Jwbx4NcKPV+&fz-h?I0k>R(%l8TXj2RrLduoG`qb!y&>O2f<%hYp?1Kp*o z(sM*7GKhpjKz`(!8Wz)=vcnp{KmuOw3UdvOQUq*RWN3mW2St%? z2%MOX@z?D9mFZzB`Ca=?Ke8a<{_NGh7ztHx2H?xWG~BgBhd_cp3*-HDg^e)tQ`6- zDgF?C%Y$b-d?*j++Z*df4mR1#rl&DHu6GOR0X&&(PtZZ~CXMeE5!EX+Y2f4O*wbcV zBDvLF2@0(?FmJfGv;14N5J;Pbgh>;?Lq-w&tNpKj#o8*8(Ry!946p1 zCI-IWJM3jOebAMXyS6*k_8P!FN&;4$tTYVXp>9l1SmMuD){2Ve+K7s7H>Bg0aE^!YzcU5Z7dE-z0EkB2pk6G4w zy2vPpQc8q0g3B|L28P6O5L#c#9&)(#>E-bBrTzFd0XB{VXi1(~7?KCY3Ty`o1tM zH#avb$V<^QwxPjP#`z7Qhge=V+K_BqnDm0WU)FIHLTSmvM2?~i(x+woa1o+}o< zJ4gTC=B87k{Y(igMz}by&;6}Co3`eO6o;z5@B*R~37UYn;jv&!Riro;eKj zHR&{E+{PQ!>3ls;CKdbA!MFt~Vz%3>=V`Rh*`P#OkaXm9Gr1fK%1}mwzH1Y}8b>EK z->UjO1VTcBfal%Ky&Q&QxqpDe7|)+u4PJG)$yVGk@KZ_ z!~5dleLDu%aIQqP^B#(YYWS=1`-5E1uBRAMvFw^%zdYC;o(zDrXBU2tj3F$}8*zVh z1T5GFTP-<*4)5%5V$w=wni;96tcqQm0ZhLGkX3jTT!3p3i$9>g`tKkgHqUJ+@os;iv0k0t3nlm-ghWEtX;h>P<GWxhP>QdmWUDDH(xO` z=`hS%7&!y6R>5m<%y)s^=^cZwJe*?PhM$?2h#h?lx)ASQP$E;61v!i*G5J#YU`@3$ z?FgOTfS`oP%{fjET7wXtBv8|%tXR_MplG%Pp)t6q=(G4IggoTEZN!O-(A>{Q@5#ut zX^rt-1m-+;%A%lf*Xvc@k3n>W+mw=$O8{>PTo#j@Um-uy2|Vl&lR+md>{7e9pxU=S zBYB8q7B8y;zxb}Ohf^CBgDGQyoE?}=6Aw>I)sJ7V8P@zxQ%L>s?CaXWi_L6BTr42~ zv>qKHyD}EThmTdm{?wm<_4}ERV>(g%DtlN!B6l>a5EE}IJ#KhT&jL$NsWJkD{vmtM}c7oT?SXPE(DP)R6Kh=u;I(MDtq^vZj&SaE=#@lxZ z7A4jXx?x&ma{LZ|->Y^At5KhYv9|+7#qGKxzuKsi1o}Kbe4y5uh*lTuGBw&xZI6WT#Rksu4YAEcj1VIHE12?zSXN5*;_%#shv!Bbm^?w_vL zi_u7yX;%j?*MTZp7FUnRVt;;q|Lv8*wj%Ois~29Ifxk>afz;R{6svewMhsziqg`v6)n9mI+#*n06=XMzxaa|Lvw18NT8+vR@P>h+?2@6-F3G8z(@FLKIj zTJj}MY$s;ckKZ<(Hx6zy4a(z?Hjz>J8?1gDyB^$y$B6mQr+Riu{sBq_f|#)wGKuU+ zy@#eRQT6xPRatujZme3DaX<8;2>GZ?mB~bdCGRh>8x8`%{GK~ddM{_td}zbS=wxEk zm)S5PgX3fhT){xfC?d;k$gZrIE872V&>$OQjO@U$KL?#e1%L}70Rg4Te6tAjsa-s-{7)z9Bdq*;`gqV+wmC9t+!eUbdKSQ z-xa$uX(}9PU$w|lzkAIN&FrZFzwx7li%Xyq@$<&hs7w`wfWF}Xcl4)*9&Z5XIGcC3 z!%JsbWQBK6@c{XNBv!r9#YovP9|BQ-H8=v;;ojC}&(8aUK!_!Ro8Z>%Mm>+&HrHRl z{;!zdm5VOO9Yj>}Bug22lA9ZJJDCl`b8rtXyz;qtC;Qp!if16~y&XcE$G$GC_+ph2 zf+$PsJ}FdwBAKs)+^kkvaovwG2{Pgxl=`9g*k|u0! z-NJq3G2zdUiC%2=)3#q$iw$Eb59mf947E^|>ZU)5O_>wB(bni&e)Szh6+9cV{ljYN zbWbo(CNrePm53NiUpbj0`IwBzbVHx;Es4)^-X|VKj)NJ$rkUm@4*T<;MwJ^?Z-q!V z(tn(V46Eq3_fO2WsZsVGlHf+2v68N+moM2)!+*bDR0@!+OvpN$6cYUvU#S_{>n7dr{+zBfpgXI{n3W)Uc;lL((7h#kMWl z(^j0(pd;GHpUA0*7I3?-H8!Fn{{L|HRbg>$+nRykS~vtR+=E-t;K4OG36P+{-Mw%M zO>lSD-~+lRSEuNo4B0&3G{ z=P&uLXP(}~s=4x}kcn>-bi0dy4x9|F8L z<=)p)4v_t_g4Qu_M!T~CUI!^8NCr$qRzV&#W7{p0o`f?*YV&IJ$Gzi)Faa%4ylVW5!_ugY?+XBq6;VuvtWYqZy>vxQL4V*pdV)+TJQjSww+R*!W zBT|bhJ+8m#Mp9fJk?j&oUv>`9`bv4YAJ0 zv$;nlok?uoEBzk+tGpbWC?*c1Fnz4F7hz({{_@U~R+3Y2S5CN%EjuIW*=cftNLVJq zC!-m{;k(*>u09DW^+yX_AdaIr28Tov9YZe|YWrv5G-~#+6Vce*{=JD_%ymI`Y2n4* zx@?D#{L9|b9`I*V@S}3dR!Uoj$Ks5}Lsrvpo7fnngg%o~o_Jbp>ff z98sGz8)M!Uaylox+MSk~b~#*pXEZjSJ4)@hvV!^v2V&#!mUU-%AxcK_+p)gxH{OZ9 zwU@O`7M(<99jjyW!XzOGzYl27g>5GH)nVFYZ9l;_Vt6JT;V@`UrnaMJP^e2N{)&~n z(5P`FL1hFJh07!ZYultD2^A!*HECRNQfh63t~aVIUnZR|k@0*Q9SMM3#G;p(40??# z52s@%&+{E`5Dd|GjNnfn*YBsWb}J0kT7vHapHm z%?c|N%INgVbq0a>mupspZvYbC^{V_ET#%}%Rk~kCgUwG&p_An^Dj;qUJFi!ojwY@i zhV>)fdFM3uav3rIv6EnT{GD=|!SpYR^4{yeKyv0reYN#h{6~kpjDM z%2Tq)TYo>qA(6*L$o}KFUfjE%*Y=lvFz5Hb_a>y1c}$rVk~x&#MAS!giyO9>9Dh0c zB~V;J{d8riW!9Hzh&%v)veKRdFW&!xe}a^Ks6ghID_D;Aqr_;RV4>lCIcW+vHowxc zINY4;o9`hoHAlL<5VOj7`*!VcjLGlj#8`8l0rU06bO#h=VV+w9@wq~&4!(G31E@gb zHcnk9;(a3;M~g;~`vbwvitiB{E~N>g{5vw{5Q^6`C(DA~veoVYAR- z)@s%&X&}Vkz*3b8wE}OnDV#j-F4?D^8>=HkHA4c`3=e1RcB7P?I-Kq%3Y0qSOmpLZ zW$?L`?|-UVsy7W#&Y7c?-E=*}fzU791m36cRv5nXci$$D+npLrn#{R23a^@GEZ4tR zvMhdI)Rbi$IZX*cDbZvFMqA{YOH0>ss0-UZm$tJ8h-xV&>U<^+qEjMM3uw|XnTK=i zI+4sDnhRv?LA^T}hnLlf#tI`M4>Bh3_K7&$g$C4G^nA?8rzq)iq4rMgyWHK=4VhjI zl!JKm4o7GlCOmfd%&8tOxjazkWO}nD)Q?6`Mq2pPIt>Juz zTlDqigp(QCh0%vM!W|N?90M zM0brVZ4!doQxq?C!)qG+k(;bdT5qZ&wzT>sYbU|b$3l+8Rv(FQn;1t zH!~N!;OBC*RVr4`%kSdPnyR|GjR{Ypf4An<&p7p^O(B)+7ik(RWNkDXVUAa55_UI8-s4Fh;20|UjLUN9Mdfr`c}WGY z$7D<{{aT#w6@~V?L%S{y_i*l9rc~d8iM(o%8|MtXLM%2#QniPM(=t9^G`eMA8Ntb8 zorYC3?{HY3&$gZ!hgeYC55$|BZGL5l7k>N6x)mVQK-WC2retn0-H-?t?c$c{a<;?S zw$P71!xRF7Bk-`t@=PaYDh%O}h2-d8-+;;nB_=t+W@THYGovBrQ%v3x9QNZL$&?>m zH!Gj)wQc8<**(~|`gN6cX{RxJDF`8}Z+px8b{<I=$JW(M!3EM58?;b)O?ezH}& zuHSrk6&kUU`4k`poJ_FlINKPYWDzh|L#yVL6%nxFMa@#H9OSkR)|ixY>1- z)44Q_&C|zt4h{DWmk>*$KUBnD=v4GAg>IB5Sl}=#=JT6(&D|$vSILoEFNMlp5&#gk z!yL}))C}K&Mv-tTJBrC4!|yZL<_&W>1a52)krt2o<+)d9j3Ad(#Mf%8OT@<2cp3tg zPO0DW5>@YO?yo7`4mOTXOcOUI<_RYc$PK_8x2t|aUq*up=$t`wmg4pItQhSK^=RMy zPkT_I{o&8)R5`KipN`rGk#8d$wnjc%qQupAgI9{cWfo79zN!U^pT|c8{la~<{0lK~ z`s>RR>gR_>`_g7q`0P`Ko#e^DiLQl)IiK_|y86BN0w!-8*&|=AU4TgJ2c{vL`NIpu zBYuS?5AfkgA#F>ljKt7iQkXTvh$7Qoy`}smU!#!^y*4ivn?EA?!fp$u=;Gu!iA`cM3>6A0rpsSrDy z&#GUR9x-z(Dd8h)X;LJjme~x2-y!d&w}$lH9zHfYPxWeJKB~G=KudA?D~0tYuBbWo z$8#0f4C%Y#H;mr<6>;#~mwQ;pInypDqz`h!y}o>r z6wGp|X0~OA^T`6^9emF-Lb{RIqkh%4j76YAw(HaV5^pjm#ld^ij_2L#0bff(gphlM zcS|%gBU4mTY5ZGpoG|v&{0{eSuid)WU9kZ6<1Z4#Hw66RWTkT@3C~2-(fwi9k3w$5 zZCB1sJ3_Qif})TdAy8gjU$BG;?60_WBoA6q-+oX2Smkt~Q-auWv(a`w?)p8U4+YEf z_>iH*!3sM!oFj*xf9g{kyq#L@S0!B3c~E1B8K%oC`zeNrL9Y@4E3ra;3+jn+QSbF9ve*RDm2$GQho zJJ%u5p~i$sez73jYz@KKGrxfi%|tsXXk1(7^@v*Bo(jG{ey}1U^JV(;etnd9NkrIU zQg(f^kk{%EBUQg`^gdsYM!c5e5J*+K{XPukQ8c92%#J_~Ew!0=$99ktape^;fJweQ zQ=B8+aQvdk31w4!K;679bwK7iGd235IRYRn2Prs*0w$5ee5XfsvrYJ|2?oGY2wP#E zx2VM2@aCj#Ozhc`Xye>`_O=q>x9qfh-x>!xVz-FRyAlIxZTH;g>rNtWGEDuI^TNPN zvPEd`1w{(cB*MwkXdi?_>a9fN!V^($-PuEgrTNqmzARnV^8`t@KHZa(s3UwD(H6Jl zJ^@{YhUJ0)jQO7+j-pkdn zjY7-U>Ik;cW+rl`Mw?%YYMpx78_VCgp%OZwUpB4UwiA7XY`WO2{3^mh+)B~QRAz|E zMY)U?B#-&m8b!fJ1)>y}7>IlJa6^&CJenrUMA&gyuj7RM0po`R9>p8f@fZ=T+s}}k z|9HO1l$)UPLaZd)4oknl{vp{BTM+drxd=Lj=TrCvd#d!L*=I$OrCQN2Iz*dzx-3=$ zl2}5Yt({thUnXFQ$p_jS%kLT0SvJXW6CLvZt_jzqLhhySN&2JBK!zuI0qDQXR+dS( zr|ikc{NBew{ou5MA`Uf%7$qIS2Xn-B{M)q)))H|cacz`m9` z-4H)%r0?#eb@(bTdtgl6dyzE?R(^&jiF*7tiu@JHwz+35+4eJnNH=Bt)sIZ4QK6;0Wv*=|eh&WPbsdBf3yW=3RgbskQxKW%0h9F_K)cn}#lF0UT0Yy)&9I)+Qu~ zW_V}WF-X8R39?4LeGhtwdT->DPO6OqfD`douri^b#lzrh4eNe*)Z3*HvxGMI!b+1t z=)01(HEVgD&Z?2*JcT5&MWwrka{mbD$;|v5wA)0XVUTQu-qpZ-gSmt?;HbKWPJ8EQ z80sz@%shgtp0$7TIvtD;taCe08d;-Q&TbCxG>TbNeRLIuqG4;Z@s`6JXTeh?z_iZe zB%&wy!HCo96R9Iu0srFFh3*&<{Oq@wfa6C^JElqQ>SHoFezllNAH}b3`W^T}2YlO> zWlUd4)NDOBB;PcZxv6t!9CH6m1NLoxi1>v)#QcWxOcd34sk$&O$xF5&1;#Wr}4tPev)m$fIFACMlGzy5YJ4Cf}Np)r2aHAB&cELM-WP4Wz|C~r<3KU*s>RiOn0Pw<;nn0d zsGH7d??<`ci505%&?V&Z+aweww`PshAkV@&jF2Jr2AyAy^Z9F`q~F&JHktEdeI1C; zGLxDel+$&nCx)gDxf-I(x~`UxLQtQ8`!&?(c(P{cGrYEI zq*ak~XXfRVi=VBL036s@XVWIx=6F@PFn_({<1bNZ8e^cjZKik$kOz#K#U$T!NMT8t9=C(`dj0x^tSCFS zF`tJAv?{~f5ty_Hq{+G{i;$YaZY`*PTEZm)^sR3?9@}97J z8QnyB73{LxdJ0E}?SY)567i1#mI)Rw`?0A$$rG7gU-naJSi6Kn3>Qm&P@n0S#xJq^ zEoFH``(4T`#xZhdyzE@VWQ!3z61>V(&12$gr9QFrrWV<*{(Qs2&Cz-nqT-=E?RM@W zwT4ZSoj;{h`%C|P#>LQ z(zMW7$$o&p61ctj_XaHf0`tVdOAAtObF0-Cn{EQuhUo$ruoqYm{3!?WRQOeMkDr1LMf*lysmuLyWHKMgntcb+smBq zjJi!AvpKxvpRd*SxUdNM&7@u`6|+vp>s}f0;^#9>Zxl%kTL^9C7=vA`iFz>*n8bp6 z%T;3#My#^1H2@U-!xMKZ{MgTfnX-7c91NiIjTy?Z5A!w*d%}k<6QbUp`8_uEWl`jc zQFkSWlGM7&gdrl{7+Rg@LaUb_qE*V(FN9l8eBiRBc!srhpNYywSNEON**uacvNGbm zVyV5c^ro{TrYu%tvgFx(D)uX2HKkK+$M!101)ol*_dSOYRfqIcZAQ?wzn4GnRd`YG4qedA0qzus9;K%ddiiXr-yY=ZX>>Me8bmLkXaNJhSWv5{=ZL4pgnjFj0e8p%+N9o?}L@jM&`x8d>tscE8tH>n02I5%gpp{Z?GiRm*Hs;)vbhUHbBBc4_ zV~#ht{65FE(%WXXGyLao{w`Rt5Yc_EZ)PXu3LK2LJr@PIY7$;m?647{9h4wRnT~2V z4A?VCNBe?uF9xkI8}toGSYU{r0+o61??^76V=khp5<{WvMHB)CF&14Ho`_>~oaqzF z4|$B-$Ybt4!}kH{C^Sf>DmhX(ZrR?M`NNS8O-0_n>s?*hP48AX==S%r%KPD4vF5$3hPm$0LY9L~GLKsXq|LKr>VW*TM=1tEtB|f# z^Z8M+??WWLLaW~@t4cA+y$(TEN)R_FXHc&-D*cU}gkS^Q`b7>J zf5VxZSoi&&TJPSE0FU@z0|EB{0xbG4?)G+J3=c#TU5J(w;uYf|Y-`kT7J>}nuSy-bQ!I%MG}F6@St&Tkv1{3NUoX1kb zw~3|vwfYjYC-YU3ix&xF`->_s>~hhf)Z81jH7cUB)t&BGj@w0V)`R*swSDAcRr+^p zOvXILg*P7H!*)Yvnd#mos`CBJi1T{IYp$?;-pzxE7$^0o?2)K|v1L|;0!p*h1EL*m zr~qFh(DtWF!H`Yjwnj|Ve7T)d!Ko~3sxze-$~p4>Mh)>VA#}EtId@qumCYY`Wk?Q} zYSKVIea^d8!d`I~%b$Lk{q`4-{|kirQ1~J~F{tL}3_hpfr6(jYcqnGbu=&XMzXh;< z2Lf#Rv6F@I9%Z(9=Vqhh*105Q#Y2>8fvi%DoKv@4JTn`aMM9E_)AI66ZYl?fjF^<) zX62^@#u@6C%i!ImW)~aO>}!Fw9|SVYx(>U&^GpFoDsF|M(~FG`amP!IEdHOeNMDIien!dE=J66lE{{cI< zM!X-pW$e9W5Qw)VxZRa59qi}0;j-b@yWVP*=RovCoG*Rf@V`Ori(wCU&b}eRCm6xg zZ77YyIG8Ai$Xk94*7o#14j-K@#tYx%D~ws=v>47;AXDg)R>I?7?X=Jr4IaT6(~9v3 z#>0YMvw!?KK7WJsL&Z)d~b}CE*>L7{PbsLhdr~DEM`} zt&o;*N?Dzlx+K=LIOB=j;mpXcyPc8b?4uL{QI3t|Pfx8s>#$wdn!+i|i=iL@63&H+lfd~P*Z}0 zORqc%GT)4%5{y{fF19+JPri>6p1)qv7&Hk3<@`ej+QgJ*1)P=H$ z`qB}WRQ`geDp5WX?);@@&(yjoazU#EnQ1}7;<-yp-&!Xb>ew!~T^FfV#967Z&2P1F zVaJ~nU>=)I^*$xQ80*0nPfwii3SAH%bvOI@7?c}6D@BQVn@FRJY%*RGZl7{BJ58iK zrHZ@-^$F&Zq!pA!&vgB5T(HNIp(LphrS=G4(V~rgdwf6id7B7H0_lX{c8`b&MZW|B zHbJTh-{sRn=U2HJBP$X}KM6WxX#S8=iv7VM&N9jo`WeCtx&ps#cJ3$+)_hN53u<>`*@qiP<_6n2h z&6Gkgj-ha{^=X1!d3WtxbTJLKCcey1i~MTHw>=-6e9WcpkPNho?gJin^aXelFbHF4 z0xyv4i{H|cBltId$kD7m9Ll)!cjR+-ZI9u-Ozh&Z(RoWJ!9_s5qL!yfN2->N8bl;x zOD^IpBXx#P0M(BI^?$9*jG_Fa&ZT;YuCPuv>GMJ@g>WdvZL0bRd?ZK~l82MoL&=ug zrR(;&VkCq}oJ1XM@cCghYVAh!+!T=R_&SN;G02ruJ(x+{!R1-0zSqbc zzv;RgeFF_z?L%D+?2|0Q9XVlbg;oi38YD&Qrb8t#-9u_ENm<3U|GpQFVQiu(s7a^Xaz4~oj=1!|J zeAl+nU>(n|gNnzRUy*OA#Wo=__?+luJ5;3I+$P)Hn)am~Bh$;nWdN9h?@*?ZgNFWO zojSDw1F9hovSOaJEA6*Eh~C>uc)y@u@93w#J;o@pHnfLRYaKp%0%;>NeFzrDQ4G4# z=i-f74U-8Lhb-GZm!nZT?Q`Y(nf6FdUXd6|7kpc9DAc0e zeDUiff~@`2&y}rEq`Z}*1Z`FdqJdzwffG~qXLUXd-ArQ37!y1jg=JRM>jfjho};_W zEXMrOVfFT%`$t6g>yV-_uU0>gfozM%Lj*65;D0xso)N+*Nq4A`r@9GzX%Pna%u_44 zPK1xS5mYlOZwiB{g$&#Sa3m6n>#j7aNjN`Q?N1wnB1ySp4B9*?$u^&*0}!1Xc0*7$ z_scbxnsT|wT1w8}h}lwUHA7)G%KcY=5vPiR(Ay;Y zCc0vXN!bX&P%-KPnQy_bc)qYqp!&)qj74jS>KJ*azdV2A`bDZk4GRh9Cnq=-5&cGk zFT+(u&Ms}P%DBc+6!w#Hc_B;c$ID>h$UrNOK9fy;>#}!F$_&MRD**81aP|t24FA?jIZR?`W8@X9!Y+H0`BIL3?B&i;#j`_UP@O> z*jVbqmK6Bjt~{_YNxy%$SyapD@isZ0(2b)>aTfJr;9|!_fyTN44u!4}K{_@?&6`}o^&4^Dn41HmePnQV$~>P!dAO1-m#CEE%>hmd@JF#V0sIzZc>Zuc=H)8l$`o zOKT(6#RCbmJ5^EVKIf#)ZQJ_79Z%Qb1M{vFVxhnt3++gJdPrt(kQcBv94#>;>xm39aM)I`an*JgxO;(`?3eF1tF`bI^w>(&OK2ctZ5;ORrv#hfiJCVv6u7(w_yp)TQ=t|wuv**$ z0&tLy&O6s+G7JrT9?_{(*}v+pz)%IsnZ3!vJqwAD*F2*5Cg_9({bkr@CQ-Du4I9zeXfyPN%qL7i&w-ynBF?vgTBXLIbwn&x&E6=o?28?x zxFkp>pgWYb5RL6PE5GxzLLvT_Tmsla)sb>#Q)D-%-brCeqzmx;Bgp;H<@`Be8T|=W zy2}Z(&Eg;=QLeYWV5x}Bik8DDGUDp7Fm-3gh*s}DyND!CgwBqs#Y1EqcPxY7Xy?ny z_QrqJZAUv2^ZQyU;TkVG^+pbpp~GL?NWE~Bp#=7n8@FeG&@QpOe7QbngT2RRVyz!D z$dfcAb$MKtvg~*?k0+PFh%WX4rv1b5>o-19j@)QtEw&iqD6pXAvmhJ{11XgWkr_*Q zS8dKhPsiQKIGy@q>X}D?pJYfu*S0pL8(CDBv_`FV`Q2g~IEm~ck$Z`=<``J#N~0(j zibFRRFH{TDB)(P5U8MdE#ZiAX@bfj~JC9alhs~Gh<$%2zg8-9P2oiK%$Y>69pmsLp z=GVMA*WE3cCy3;5fGTx+9j1Yh+RI!UVH*K|WyBRP4HdF6(o$SI&GO_{fP{eS$h$(5KR0RC9 zG5ozh{QZHX3@4fmF7qo1PVEHDS58jfc`!EF4PeYEe4sXSbE$*%({SrNYWbz-rX?9} z#7IUC*t0FsW5N)-LbX{=^3T>HQmWi=eyba-&Xk!pQBW=vQ6nNdB}*r_`&HC}V| z;NJRQ@&F`JGMD#vP58!G+&rz7jgy;OO z7_bj+m#NqOTG*#Z-w&SLH?ezk)Uxvs%#*TWwAoiXVFNF=-*yzTB)~Yy(?2gaI=tH! zZ7AT#{0Wz2t?X4{y{1-Q5H06W87))q6kA?T**?Mrje|v~$O9x`l@gvbZX!z=>&v4l zCvKf^duR9upZ;4-|65f5`-5VVs)O<6)BPV`)AK2+c}`bgEslNfZ3M}(p%j@hE2z(U z!DU9qWRh~gb>Om{OZFg~U}P-VspOZ58Q_x5S1~Ryr{)mI100vMis>cLQOlXqn82Df zIKBRV^o;-W>HPhW{)Z1y3UG%+HPLJg44?gtZ)qeB9qZ$GWso*z#?tE#oys-%M|JfR z1<{tK7vjJ!I$W|TY|jMXoLc6$j0^X7cwOHA&lfcM>w@!r`VU)!pS4qfolN}cqh+wO zDQ`>U#VTcp{T496;0Wznfj+$&fcX1cNdCY208JtV>VR(#W^96ISH9cnmneOu zuhQgUq?Z3Gv#l&ejATaDMFKIX|5Rn%JC>XtrZ@2KxBdU>_je${al@|pyTd#Qd9P)g zBC~xy+6v#Xmzk|xlQ>@Aoj-kkJn_-FU1+G#nJYX!+mOEdJsc7fMMf2@LV$81azM?2 z@W$<2hB!$SH1Yp&90{6jP7}IJ#CXR`tvMRN+3ku< z2L_WoN&n0OK#k6LGkKyDZ?hi-W!6RscP{_0YuI1^6Qw-8I?!^z{vEj(Nyxwvgi65i zl;?4=&5g{ak3t~)Mlym>Hm$sCK`BFc9FSg2o&n%vKA?i3)Is{cEhGQ6$U9(R7S^iq zm)=VC1fhQ3n<_~Lfk!GTA)3;OJhlyv1l$*Etnd(>r9!c z*gN+jxYW?OovWd%b-4f37%j8A+5aeGRfhA&d-B(1{3t|MmzPo;e-Ej+Tl7*gA|?I| zG5G_`DAcKJ_tv$GHF9b_htq_5;KFqqTFjN7zi_!4ximT4=gX#)2o5H(WGWsJPXLrm zlRyy9$h4lqEA^zENkG{uYv&BAvwRw}ovZOE;e6Z3_Rqfj-(R2iViYFdD|Gq!EvI~* z`QKSGo_oobMZjjq4QiFjS!%M;$!J9bc?zk>G({}1#Opnmoe>tDRY9fAK9xfIspLOj z8|&@;SOkAKGROP?Md;&wx5_18ve(wgKg7g+m&B%h^_OYnUqKRFB)w*L=tvsBR8UxA z7M~psTo3lm*T3EkI3n2ZS|=+7IB6m-Ve4U*M&Ilc%+5ObgAcw5TO#*FXZ)&U$z&Q< zI|Jrs#$b>xQehGcxufkb)@8IOp5C0jO%}AQmfv}~Gb=7Br`#E6z&gCQtH5lSwi2ci@(R_o8U{3^I2w(_E`q&NG%h+>e{-19Sh#BkDm?vQCoK*KR6Q*4? zTf!io!}QgBl`*IF{AmRVBm@Q!Mk~MOI?M;;6YVpU)jX0UFG02E0< z0k;Pvh2+=%2JPN7z4ca0vO^iOAIUWA9&SFcOLJSSRE4~MYtiI(fxx286%4o0~FsMnu0$m4Tr%C;k12_G>S`W_1?eBUFNmg+-;$_^RnBAH_vh_ zA#fzOB>zPN5F!r|@oVtw=VC=-Xgjx>r$$iVQ9SChK>Ab5;}}Yw`&IIUlea z6gq@_+Fsi9uG<$;X*G;k_(f?^YZlLE$AL9$o2=7N_?aD%Z0Pb(C@{9*bwXX6_Yp@V zp24Y9F#V?so{DbSuQEKa;RpZi4}hTqeW!e~`6}YlM50oqihjpxBNm1|74k4!<3XH7 zs#)%d7nVT*HLH2>5qC!N7jfF>hFYTF(4r+U>yNE4iK<^f#wK?A)*j>AMX|!LI1>jQ znjBb@2-dng;E-`t26^<&9==k>Dlb=?mqp0Cd_*}OK;nz+unF+a!|B8#t@oUA4gt#p;6!Y!S!lXAe zn`SQ~LD&4I(tACl?kE|KdmcK<7X)EGUaVvDGAxaM^|2aW1wEWD_w>_``buRzk2YzuDl?EMlvFzL?1xy*85|<{pX>Xo2pm4^R|QDsOTUR+Z9#r*z=(=m zA&q%rDPcc%o?x2x9JReX_TFKuxNcFv9I$F3v$=c~+_CRD0pBnC-BFz*{= zriA18P8Y}Wij_7U?MUOPC(`JgMWE>vsIz3IzHc8Ny-HRco^R)Kwmt6o^Md#yQUH4b z!EJL;o@gUj?|aOd8F_JrjE%oCml^96 z&s0@wg<&4@M(_YGY+BRrd|U?A656B&<-NITH^52OefD5nzg?`}Rw>Vp+%p1LJo&x8 z=7m2`P6|p#q=VxF7zKq8G5_9?E2BgUDPk$v(g6DCfPU5-<}94t@cHRQSI&UQL1m5n z7l2XtZ;;Z5;`F3AZ#)!tj8vm!#?C!I0P*)P!0>|v1yRr`KLIkafY<)_2>+LUP;Ont zvubtfaq`2#BsNDezD@%mWu*_4)OwE+MNS6$bmCI0LwI>8;36#5c2F&ywaa~bh<;*O z%?j1RE()V|L3>IC2qF*`>|j#GYe@(p6ytbBAwzORQ$}s4?_2Lj+-&J6GJcAgJ3uJ{ zOeffRvT^CE;aY%GO~OPL&N`yCu{J8(*R8^^UHr2mdBNXdG5^DU3fp12RCoQ+sfYC* zP63bYT)mSxy69Z>FU6C4^w%!8=sA%9g)J2tM@?nae4ZgGPkpeGYxF*y!;MSG{Zdp) z(CKK|P}r*1Bf6zEomrn8G=BQ0)_3&$&Z;aHb6H45++_^Nr7=#V)$*Seuq4P_|J7Cd zzBm?}ulphj1R?OQ$wSTd!iLrJyIsVqw&UPMMbyj1TJJIH3Na&491?;>c1APx1sew5 z_TuizWd7VE_c&c8`;)p@_lu9r3*>mW-Yu)2JU-S48rm|v-wx)#yFL8utc3~&g|Z=A_5?frBG7Ooir_r3u2jX(2|zT zKR8f9AX+JSwuM^IN4mu=J-)nm8RLUq5ULbyBg|QE+Q841^MWEeuvAL^IKm5*$iqXz z&Mx)8lgNWQb{uD-hVAP_ri`5)`x5n8Sq|5>MYz?G|`-)em_l+>-R$zJ9zo`(~Rq7yuvbs=`$Wfn$YQRO?o zcL8M}K~UIIt!$$JG?q&*F1MiJu%lOA{^Ofn*@9yp5DrF>{*?lNM9f!$AA&i%0N}U! z8WsrrqT!XG5pH0|*ZmN#I&Kq6G5-mkro7(wKRh*~h!@$ol-B5+VJ%MElLlcS zk*(g=OtPDVU@$eXOK`|(4_7%4+n?E51k6v>yY!qbG?Z6B`}$6T&EA7L);fc(BhwE` zu3U0sIJL;a$VIXaQNY8m_E2SQV)`m$^9BaZOd5^1M!m4S7Asuntc4{mOb63-W9;Xs zv8+wME?LC%J;FBzK&(q#;@Iin&b5_%{X#T z#2(HE`?F4M{*#3U8^;{au)EPNfF5L?Zv?wioQ{sea}=Xcf81g`9;P9C16tMC$GQO! zK~b`AlTRWL@pz`^dwc)_NrZl@p&fW@Ts&^$^o=*kbq8u$;2zUPuU<0s25+3B+*sOo zVr2`?S=cm#=1yYzX3Nd_J_r291I01oQ^f&5P1$PuzzM!eY+Z z)9MJ$DAIk6HTicQ`f@}sFa`l5wbaeZgocA7tCA|GPsS-aeJt0cd;o%xW4@=r#@_(4t!5lQmodN=?@$;zX~MVr%bQ)SvJ79<)e_cw zm5^8IxA`05R!K^)ph!52)4ociv!UD<;W%) z8+$F|iAi+Hv;^cmm)m`$A^7`?Wf^bPsMbM`A7V&P04Zy0(D3~+)xf9mz_r-f-dkqf zb2@ITMVbt{-Y8` zKiHnB7hg3=iA)4LBgEKIQ6?`0l{{TFPmT#G1X~qhd3w=S$4`Pj_rMSpKH!q5Mj0@0 z9OjR6#0H1FL?VmoJl|r=_Cg$veg=XV0xl!cTNEF}+2oRhZAf-sV^znG56f{m?%S{Q>TE|kbC#Ybqc6o69Je) zM2RSD$)TjE>QHr9<pY$mr^Pznpf0&l!5{`h=FzBSeK(}>R2(z8YK~P07V@MrGw}sJ(JpR zL>ySu9i4JsAx%Hh=<`z2%U>A)DTVd4th@Q&Qv*& zM@sw@Mq2-!<~|;Z6M}3)d~TA0)6bW&*{{~GH{ADf82x3POAe# z%Hbh>##@zNgb?JA??v*XYQ7?1@3sbAx(B0aGtVnmeRyO{7B(%#e~fM#PMJ1(JA4Gf zrg($JUXk?8u*x%w7G|Y+#gPUG4?SJ~6jBRj3ISgXiP-#?C6!sjwbe zON}GDjh{K~o?9m>pKVyVF&2cH>#@(VQ=|(!*7Ehu_(0$gm$xnt=1oKr!v+pz!3n>i=5cS{6EZM}<8P;feWmRIc zIKuw;Tt@k;(vE3=x-rkuWbyl&;hosxN)0?4oK+DTz}NJR@=Fw=sh6MUOnH3It9rB$8MY$K)fFZi;le*3F~K+*jktUlIxWjUwyI(s}*;i%me5(7|;9&|0DG z7&g9xG(h+w9{7;=Kw_7=iJ#@!+^{IykGf+dKXe`H96b{wEwESzj{f4Jae@X}*Y zv^Unwkn!1xB9PSsQyhS}7S^C({k?>-vBI_-(!9STe9Qu5L<_+bt`u*kRM6#1u|+|{ z{qMufF8|7&Yl0E>n{>*&7fAM{4r87Cmv_TVlmrXFp*vXC%Z+bwh!`MZ-$(MXW_an} z0_=%)Y_~1#?akTCZ-z;f{TU1nngT4i42o$V=iEcYcA_FgktB`QRvfyo_QG564@^r* zg_iWiueN}EqWBd!-%;l+5q7n7Gzo>xNlCnF?m^$9zVO+Ajl%#C$C+S&%59`pxWmU& z+auMHD_hF^pNms~qUa}1EUE|NqcVqs6PG;gOiV!mSG}^Q&%9sv#C|a}6Nx6QbB*>HT94m0* z7o`yTjwYtT7!r}81DFUi=)=AP*yTp4QtUgEpvTBpnW-E75noC!)H2^796m zoboFZX2}%e3n<6K*(PV5X23%2Z8GUt>ub<2+r&c6oG5&^X6r%sWC6E$TNBrUNV{C+ z+V|$k!q%zr`XZjL86=q1dNx2Le9}H_hfR5My{7CWoFm(#K!J>^1dHa4=cM}Di#Xkb z;5P>`&CWd-hbxlo-Q4)qm+w*0Rz`u@J1Dom+Bcu>>X;-hZAsB}waqt1Q?=u`v9;ya zQ5ke%`d_!O1@zp*GFa#0(6hy3519@_Uy@32q*_sIp|1!1X{E*KU_G4g>AN<82`Sh> z)RPP8<((6rxk&mux*Rv3vfNLRma^a(&+K`pSCEJ1m*}B`z=R6r=ws{d)+wxu4;&R9 z&A*pe_BDEr^%u+zcYoNs-1My!dG-`L8qyazp3KWl1Fd&@V#Q){Q;L;cn$=2vR{T{I znGk~YoSl08_0f{-18Ym-9)V`rk~znA=Pl=Ch0^CH_a3^dfIs-3*9dN^P+RM7{J)A6 z=vhxOF^q2VuzNP4aBMt$|9&Av4KEwZFtiqX8$sM^woewW-@bK?m$;pjGq7f+2X$xn zEGHzlJ!$58t9pkLkV4Y0)8VedfR^aW#W&PsrfLKFADj(H9OU}^)cOX8hWjXE0#@bY zuza?&WNi?~rjbT)AC6`4E=WrrTTwjh8mRwTk84ef#nQBRH+Y8J^IY(L=h%f(G8eql z@q9F`(klk4rx-)R>b7?T&$2Xf2|1rw1m=hFEFHPws5x3ZC;Xa%TvSD1T_g&4bK>fgdty-7I+M+lK7I)mOpARCdf^1`l01PS!! zj7zP?Dbeyw+hxjC$xD0WDA+G&kO{%%fAV~)K$?X&gY!um?SolK*`NcO6!t9vlYZ5E$smSZC9d`w!u~mrsa+7Hq1^0f@ z+B~1e+3FK1Z7d#Piz%Eq5iEtCn81(oZMMUTnve~h!B0-qp%J(w2I5mNQPymRcZ-Ir z1`O|}N!DF*68r6d@K?T6yE*|Qp71rV?Y3;<8=4?Yas#PuaMSmliX-`-vkU!nBk;x8 zd?KJWS3hN*hO*AAy^Y@J>Q^cazmIwCgspYCDV*4XSEM#`rEz~>Stg

s%PWI%>z6 zu>j3eK<$=qhHoiwX-=0Zy%a)909UY>VqkOHXt)%S#zUouhYSr|YGBP;lH871?Gej8Sg9FelQDS|KyrhoH zsh-{^G#${$wgRi#^;EZ;18sC{pMU&iqJrh#%~$}J*)E)1+Pu4CRkC#ZV@9Ndv`F6z zCI@OTb*W-L2__olmDy-ZEOp&(wlKk%L_7viSF2McTCYM&TqJC-nC0T>KZ}D)+(3C1 z5EIYd_%%Vhhgq@4aI@KgPu+g|Aua0^sd!_V`$yLAQ0rho{gr^{EE&|#$CI7&0U4V% zd(rKDu)0j-&$O57ED8fL3lgl`EUdI0jJFscg+fa+aIoAdVn6T(1%y)}5NU!?D+Qd0 zQ)2ooJpVQS#Y=)TMOPjgXYdV*6Ke675)o(x7e#v3sJOv=F|QnK2L~^4-PZa$QNZz7 zGEyQ+AJj=l1 z_ug~Q`Q7KuA3IO7v-9k|FxQxKj5(Gy_h44KDt*MhG`gfEIm;Y#OeR-SVnhQNvsjXCdNEhaz0u{XT1=A8VUCaZVpp-~d57qj zk;O4s!j#4Dk0+1-w3Mj0_xj$)2l88GEA{t0Phi5~{N{Q4NyND5AXHIsseH}tG(aU`yL<``5FpRCT@gNOb`4YzMvN)@Y83L(Uk71$rSvRl0Zw;cn)SM4pE-v zkt>i%b?_YsQXHs0%7T%kW+1z2MJ}`1uX9$sq<8^+Py+lOZF}Uq#WWh#!&z|0RKySW!Q?_Wxvl9)I+v@oJYk^&!Qt5yu~vVL zM3E6|+;oE}14Sq@IJwbs3)I#zb=US;{890J%>A_N+N{!<10pE2 zPa#khwym@r1@Gom*Hn5fSBZd$iZf$B)ZK6TTThXw*LFTg3KCL0y{9ArQLOTU06G|C zBR54oaDrN0&T5+tjtbo}{pki7WucJzI`7Ae(jQHW{w2YuFVzfT@Ib&uF$2K$?-HY+ zSF#VC%G^`cJ<7^PtN5f&#ctyU$q_}-Hk|!KK++CtC2nM)kegST>5GB28}WASa>%r7 z_A-lf#EmlE=ub~okfeN6N^-LrCuLwNOirfB>?ddhhyY2~nO5t8+K#pUPqp%*gptyL zR*jEY0h~&CBMl*p`_kHZIqTZu$fP5cN?h~(%Yk3)bmYpHQanMQ$)LxjtmIY<Iab+F z9eP#tE|jFXvYy*=k~P<1n>f$x%!y5xysDtQ_5o*Re5ssTd&2W^%FgaI zB(h?BsX%o=Dxsk(7K<3t5D@m)@sh^h&sxhkfk4=`-l*eeCj#Oe6Etq0w-u68rW>Os zOP$Pz)-)m<=w`P+#{FRwnAgbKT9W6f)d&nPCg0+#IQ`4APLG4viw=uF2V}#G_t*C<-DEEa+%ryFV<4D;67&04 ztx5gn>5$JM#q2$SalajMYa>>(0-7C64zAStg~7P$9b;6QOrAy3CAELQeqC#|G3=)i zI{Yi2vweNJkB&#;>@r+H4qIq?(U+sbr%aIPTfgE38ASLb<>xkCa*0i)z}FhhMq}Fx z4*InmL9g3gorlSG%Pk0tQJ?81f+)#z1QS;YhndqQC6FEOVeYiC-K^^7J`+iaB>&6a z`G*Y9dq)6I3odbJm=ehUIyG0(Dr@y8D5b&e$mt{`VNAM+rUHIyaO4>&CD0D6-JGs@ zru)o&9_XZQ%-VPS88{YD@TH1{*=me0^5iZZ9%!!yhw$`EQ^8@*#pa$(@ex#mTEQ0u zP|mR9uEGCBIYFp1AErRF`>2rMd=~eu^|0RaG^y8B4^>Gy(3I}|Xj@_`iF90^MSBKY z0@P6FyXn4m^r3tr>!Ixi}R`c22SSC@6+(CclbeLk3 zzBBC}c`Sk`I}`n5x|Kl!RP_mS!<#&w9*_9uVX)u1#4?Dzde@mRrm{2ZbUwI0Uk<Y5xIIc+8ulh=?FX>T7PxdFB<6Q{_9eeD$a)>=wn$;e>AEau{#u9 z-L|qpT5=TedH#6WKvZu31T~T!@EXaW?JB5BtGpc!UeIJR9J3-wsFalCUsdf8(+*ni zs?=0=+F<5`2F zU7+xwjASrSk&q&>l$IzE7QI3GkfJ+2XfhfWvWnP0jHSrpev0?)^ti@P!S<9-q0e7B z7ReXNzoZEUe(QRIPeR*lx)!!jVJxIH^cWS`()*O}EopNHy=g9?E0_?U6V?zkV@{<|OCFrC&r~?ny!a1Tm9bbMM!XVoR z+yjhKgZRFO1i9&luqll5h8M5PX3MwvMq`4+we2HF?hCo>Jd%2BQGTU+`~_CU=L3`0%BbYLqJkz%=I@i+YW3;fnp4 zOcMw^oXQaam`jE1_Irx=%j(YiGWWMHv?kJvC(X5PVjrr|QdGVRH$e5lat({<+3`K= z@ZhLR0rJJ`zEx5=hNDMAdsfKaT{U7QLIm*;M2F-yEO z)#&BVv4F(o@G+sCzCrPy7(|Kdvu@gF7K-LZN3oN0JK$TI-p=>HYb((A8vrxRfjS+s zas+B}{4JJ*b1sebr2iv)6rl$55c+G8lm3B#R_4V^4wK&fkx^9r!(WpjPM{ zQf}#|z9ug5zG;FXX0=lRbSA#44DON*DXB|I#sY6WIIP{i73Al_iXe~eqo|Fx1G`!Y zcEyRb+Oo6iDSnJzEtFsJ-JNe+kf-zYof!&ir2aOU?i;oOW{W(pSn=MC#(PVDdp)Ho z_udD_l~p~#oz4MaCofOTt0xTDlYM{w4xm~Ka|VnBy#v(eoI^j=Y>%IQBeexJCzmX_sqdnH zg^kC#Jn3*-9l3N;`H#s;LS4OP)ppX>V>;b9@WE~TV~*=XiGLb(_$A~#y@YohL%`2q zWiYQTLzM-4m)0HdFKfP4^nrv1tU&K<` zfDHXLPSmIDK;!##WpGGpz;Kc97_CO1R5JDbap`c+_IEhy39c>9kEj$1sWd6fN_(fv zVg%(~nLeFG9J?e6Su5pA0qoB{f>c-pEq!|^Fo;0VNi6vOwpwWKu zAGmjGT+cI#gko>X!?$aXuaArs_WD2d?S#2#WYcNC#ZYU!9_ltqG^+mwx)?r>n7H(} z!DBNB_e;u?9Gq0XlyWa2Q2<6eCg*C>N+;$v*GiZ+1pF7s32_rFZ7PB#v#rZ$ zvr%NjawPyGZ^NB*KdhL_R}?XI!a!)SzpOb9qal{t4+mE!ouW8ih<@q82L_M~DK-dj zt6n2dMG8p|Xwf`uY8M$Jfr# z5TxrxZ}f%ZKE3CD2-8#gUGKlBv}mDoErGU)(= zigjZ{er8Az!d#fG)}_mDNZ1q_7mmt}S+GJc@B8O(91ema*==`}6#B2nJK(qi4b=tn zmhXguo~0?@;Jje3%tlo{xqU9{f6r#&0w<07wMi$hle=XL_{d`xog{4a$9w*@cLLRYML3+wv$6@yVWj?}cQK|8lBJdhjZn59@qOWt90xEWg zzBUULl7RdHHD`|#W-f?Gz=wGMVB_cGF0Gj1(D#oJ;qiznNn)M;e_Q^ z%$xiEI>|e)-63gY<>P(%O(4ut7O|g5`E%;>Tf1BpxJ?Dah-K4uBuLkI{6&Y?JxizG zd(@q061Zp^;np*#FR$zs(3wI?B&AuUBP%BPB-ID-fa-@!x5#ZC#87~xt1hP{>Lana zlfKK4(5d_7oCwNHN|{I5O|qWYmrm@~Ou&Afj-v82_Tgkwbv*05wJ-GjrG(oVXG$zi z1Ya;uUsigzY`Ui_t#(_7>(dO>3bh%=fTS|XWVgk#26vZT`HKn~(IqG01KA~kb1oA= z^YuzY(m+heb0Y%6VD0fs`%M5p0p;6$vOE|Ou|*L~${I4oq*JL`J6%p0Xc4l7ueMue zYQL2TeuSe(E`Sni0d0}8MPhB}fW|)Sn*@g-@s;&TONvqtwb=csTH+*-vXOHUoDhId z4yRvsd%o-ZoI57?u118S%iH`a#<;u5?CJd4YMula3V#PV8g|C7lEIZZ1v%{hv>m7? zL2noi-blIW#Rzcf+i#Bqp_Nc~}5qBlefmho^&Lv)>s=m3oe)h+6_cZnK zi`|ZY8ajq{d2H;7!!kMQlN&fMOSUAD?(jA>&DB`5Ab^5nnZwc;apL6f*#c-CgSo7% zk9B;)O~8-@6YPZ5Y;Njt;F?UW!o$i=kQwe79-|mQWn-cYPd1b!UY|U-@5A1TULP+O zG`ZEoykrk(*{u$V&`*F0YJ(ItcaNi?TCdRW!KM8OFS!(+Z^hIFFQkI>hry!zfTe?( z4C>c@xNJd;Da{ zb$(Z7I@b%YyyWHma#wqJUi6W5{OY)Ri*f~#5_q;;C1sSyVziGRC;N9k&F2Q(}P`;I%heP4+>->(F{-bnE`d{A;2HzK4;7$uAM zzWfLA{5SuJgNR8#xq)^`-jayk5EbK$e^ zR@BiTB%n!192mWk<+cY{tastM7}W<2?ZQzgn!Dc z{hS87V$}Fip)n2lm+Vz_$YUR*OR4HBWnbBOgjz9hu@cAc)y`}n?Gi^k(=4I; z&~He40BkrjY?<-(3T~a`8hQEdb9Uft0J(1{=>p+qXhz9LWfPB5& zF)vRIDyHp4(j4TLCIz&h_4i`!SO9+r{o1>pH~m^m*#E9FKtLx%e04qlar|6fb9e5> z<#o-2(CJMS=EE|>1vP!dBK;~PNHru6Aahyxz-oS)G(!na}WXrG~ z4g5P>vH)lUe(|biz#>C>^-!CYN*Rw5)h-wdOgF*8X8OYJX5yFIxiKmx2ACqZST5Sg zfHnJRRB4p7ip=wFXnn}BW*XVt{Xb3*w%Wj*8I~Kx!A)&aC(pp>&-0Ved7T})$pDlE z1!`U9SQtWfUKEzD?%dT+R+w3U1JY>cXkOaiF6#KRaw6jQn(h?0Pc|PYM~+DZAT9c`D8p3FVR%EFS@stAQ92>W9j=k)4G8mrP>ky&-m7Uay*# zs^#J!sJds$Y=ZTZSsUg?oB4Qyh%+Vw;EJ8kIFUZ3le+FG39Un8@RZq==KqHLf^>E}lcR zfpNT+RKzdRgx`||4#Rv}7F}=RX0hHV3liikLlp+CV&mg_uO(qGRxK02hsehWgcE!! zk4QkU6s**wKSFmV{O+E(L@kHF5Kb zlS=)xVZnb}iYB28FyBmFN?3Wfo?KQI{0;7Z!F z?=<4QfnTpgYdE*$}rWukAiF{_)->#OLH-elwHdG0e^u97>M8^D7vH2n@1_6wEUith9lMM+XGV4k}n+yaH?p zQt_wx=+8|D28-U8a}N*MtTz?zMfz5nzgSqkWBvECJ6+B`3??t8T95FfR}osDA&;#J zFjQ1jd{eJAvCRpmW7khICMj3iElTp#O1{bOiHdq~ppC3(R_Gxzbw335qzXX4yuO48 zU~d;I&mz`YWV`<9>;9m*3!dt}|Gjn`DozQpGA?w5K4<_HWPv41p_Nckv)a^JVdNMS zFv?EaftyWc845aee$x#FcgO^JE z36Hn?eNDAfTrjALxpRg_CS(9LqThLJ-s9o|qdm==ElH^6eYZBBVc3w2U3tP^*Q)4y zfXx(T0Oj|PMATEIzqavPxw9eXqUDyb!C4WdzMbKAFr}DEqrRMiwCROfl{ zd~^F~{IMyc<-O~*o{y>P1ee7~g~0P-)H4h2`#`vpg)zE=qC#F`#X=&Q&46!pDyK9k z{b=mX^~o{2078m}&sNPiMu{C^Tli?F?iZQ30qqc+ysWbj5AaRe6>iKTCwhefW%OQh zDYxNaP6hu{`4X&jBvSQpQ!Ow27!J3ch~>o`+)|gj8s)h1v4D26X>g*R?u31pw!sR~ zTv_JSqp~5v%9Cey^**eXMGlv@2rd;fC@_P`LPo`pz3His!!u?qy=xAg^m+z$u8_I~ zFW^NpcU4nJPv`UUeaEr7-YPv8gZ+(!m{IBB6fR)-FMjVm(Sg2--{&5U%Ll=h_1p*- z@&>KB?KwPm4!(w_ow4MqT>N2!S-&sPHJ*QUfKQ$IkodkwQFJh0O5r08_O93VI<|e! zkaVx3;J=I&jWcE7pOuGSGzk(X_&i}I3V+5PYz$mjjRRxTFRXUqxDtHBf&yf2OGX_C z5UQNDhzsLANs4@6iti_=`7iz-NBX07D3MEB_huc_Ptqxl=?sZf1?0UpP)E$JzjX43 zmtcdZ1c=R@$wM#}_?wV+@*bojvx>(;Z0!j5tpKm26tqZtM?7Uxp{1dXcb#J*wSE+x zvji3uTSkD<=d8;d*_2wH8i@oji(&*myP4|Jbj1(W(fi zzb7CT-xJ1Zh?+dIu+QJ!V(2Wh5A!45P4(QIxaGu@3B30@s&TzelbpzZ7e^>Oaqyf? zq`eY{=jxL)f|N;VVb6u6tR>IS$yXBfY^3L3Xw*Gq!YR~|0?t%lBzkPsg*CfzF-LG4 zcnH<^rz(chzf(d!1b{N+Uza+}!+sWVVfh`Oxf1~4Dmn23A52Bhe?LcmeMM;lA341Y zX+$*C*`22yD|LL#rEP@00Sou)&yL!P00tbv%zEU~B3IXq2pvX2XdSJBad7gu$etg| z-_`ifV8{i)0!x!v7i|CM^JAO(RlC z(~yOt+nJxHz4zK3;0&Ptyjpy%`wn^{CqLlr{88->7YzO$^c{4Q7=zDml`b)us7_C& zW#*;@5w}u(#^Ex>r$o9}UNXg#$;`4YPmU9rAqBp6E1^~|g^w7=EgoZ-VMC|at#P59 z-&SP#SEY5S4WyK)?5s({5Sgqp)-ziC{L2}hh>a(oTB+KUwB&N1Cx?_ov?rDo+NVz% zqL@=DvlZtx2!+JeHjBcN)li42TEyV`_4WVky?^_(z(*I_h8RI)B%rUA4oDduOaJ=7 zl9$z*H7d0--XnMXE!AB7W&$Uy0V;ePsj>;mvOz6))(XaIBjINZ#@72oHL-FEIdyWh zt9f4RUJ(|}pLwfr`c)pU#-O5{QBp+;_(9uZ(XB%a`sm_25Fuyb>xw*6$)*iM8AA4b!&veOCA&`PNGaVW%6KKr^Q zkU=typm@@^JWr|xtq0LEd!*#yX-N4j4OmgtiBBWy)t{)6cYA4trBD^@!|{H7Hn8N- zj?@WVX_EPrIX9FyaK9XVoqQrC@7z-%f~ONawk**LyNFJBBQzvpiGv+IUm&XIVQW~qvQPuT%o z6ObphrCKg&<4HF`TX#dn%mnwJo8sTz?%(YKM%oj#AHW<+rP^Hpjg;;VTN~OxAJkw? zA^t3&N-^YH6&eaFX-t2BZ{@};;U#*JYop`JY^ydr5FxGXSNa)rPk`50p%!uubK6;K z3LV?#EY3vW5y>*Iq6dzmRYR$^{l13Du%2C^Mx?lLXjeY3Ocgq~Wz=|B+C=$O_buG>OH#g?_ zrO+r>l|p)-MjN3K)8o*hjyJV?Xdq+6#SB3sx3h&(H!e z9_f?NiN#;5eu`=Fts!Vzoj>?H>-ePnjdcIdr2fxa9$*A@ufxf2{o0nL6zUY;j3Z!5 zQ3vg!Qi_glvNzyQpHH#vIwZ_6B)q9#X#qT=^Iy-n)^bdP%S$PGzRM3t^C;+i_5N(Y zFnIk17BMi^fPuxF!kL*(dPiY`CxHO|--qKr#^#?Ndayy8q#nd62CTl6pt|F74$IN+ zpKO;mI8p00A3|l;RkzH>CSt9PA4U|+Yo5&30fJw?dJZI-YexbJ zu_fr2-Fr^%(yn~v$3EFF_1-(9((=A3e`2M%r&;*Hpz-#4qa;QtJey9`dx|i`dqVMu zHeIqksj)AFmUt6#tBV)yjlKmn-`lGuVS|{VIWCGh3%0Bmbc=yG|#t!701iWl& zWmw1C52FuTF8?%`r?$9X6{&f;bd1@E>FM06sW$dMA={e6%s1#w|HlhJhjhbFRu+8V zY}a^deK0;P8Nuo#J+J?f{$DHPzlTIDH{cv|)X?vbz<);rgve#52!nB)S>X5_r4Z4Qbh>;Bk)A81VFcm#H4S|T zG?C}0UNT0YX507JAmkG!;^R;Hi;+TO`x9yg-hs-uQ-d;Q4utc*RJ6O=>3sS@Hw>I? zY#`t67iU(K5W&GaOT{<>#xXnmpFH7Hp$ergF6_gryWWWRds=)Ci6Tg~%gz*wRO(5& z_rIp}(Bb0{2EDZPvv$SBI$WFing~Vh|H=qfa4_n7MS!lm*Hs1}|@sJNoE3{~&qW#O0@_t`{;`TF~$p_!+TE z?rgz8D`bi#0t2l;@Agy~zibSXFH=LSg7)~}E-b2#$+n;=OqAm-gAe7PC?^NzYr;ZK zQ3Ru5?5bao{zfHgZ(y{wIyYZGPQX(G$hfNWN*A``JLi78 zZwMFLir;0zB30e zR+%RMa|13e_s$mMLzFhNFR%=tI^nqA;Ojg@R+R*7 z2=@;#g`H(^tg(3=}m4LrHN-1lmK^l%kTe~n|m8UkGI+~5z4wc*~nys7FIFblWhSeXXFKO?BIWSVOImRcf=g&g3Etv zWcKf6(Q5pXz4q<>&iYVAj+h6+%Zmp>^LIy1L4U5U`Tw3*Am|YUU^}<^*t`fEWOlVn z^}IHm-)6G$*?tFUn2qmd3VFwcZD_wruof&>H^^NaA9Ql43h&=@V&-pG6In-ucL}>) zL1G#NsbxT)qW9Pzt;Fu+SewXIXn(`EpVJfkdlnEoL%6niaCzOsI^H)n_x+%ttKH$^ za&_eFc$wSEt1qB%CPy1je8@7d zrn33Fe7Qe9Q1A!8c4A3DFRDh3Hw)|$cWsr!fg|jW*@PSG-(&NKuuyIq1XTqudezg= zS7KIH=$KOajjN$sAhJ2IwULc-L=Ng!3&LM^Kf-b0BS}9b@UqJYRVxPVDe?s`vogW_ z=b}cD0DDX3fpCN-=&V_8V25CL!c0OX9s+;p8m${xU{_%wu#Jf;`z&c%GmwVqg+6-w zk~u=2OrwCH*j}xG%+b_^S@VEcmJhqvQ!kb(q*@YNxknc5BT?C^r;ryM`gdSZU;-N& z48h6(G9R%`0gH; zzrP#bViQTV7p(IF14J$^RHV}T3w^mik}CHboGRw|>V2alQ@*!MMWhE21=K)L8Va7t z04Xx?ZVx^yuFL+%pgyA->U5_iC&bXW~-mlt?HCa|>siOEu>)ry6dlt{%ipqc*n3?~GfCUlTaPSTI3 zp1F4nZdQjc`$kK$v00Z|_AU#K^gY-yFjBoxc2&7iBku3%OImlkSU|Z^i4`#^_aacx z`iwq76&OQwYC?2f(=v6#+}uQ$;5!4+$JMP#A)RRjVcpN@%}T+_6wQv-)cBvt3jF*R z`I@5MX{UG!-f_1DF%Cn1k;t3ggk&G_v+tWrS0|sJbNfW)YLxcjO2+T|-d}JnVu7tq zw$V`d3y6--C#9kk20Dr0DoG~O=D$JyY`8sGks)X6-jU9%RtLV>vYQn;W=SzIqj}Z? zv81$Y6IrJciLvELM}t(>%%BBNPRJfY%E^nSt ztEGnfX2a=@$Q%jE_?b3OeGk7e*bOB!+X^xw;^K^!A>z9)zYzzX1zVoF%@SmxRPkg! zFCFJ}DT85X>wC}B<>oWHn*A$28rofaP68ruOT38Zv!0av_*H1i{6FX7XVHL0<@qdY zl`l)WNb>b;*qJneJmbbAGy5Vcl^8<=0XzC-f5{WE%1f;r#6E;nYM=5C8Z+vafh@V8AR-v~o5;hhU|^tqulB>;^u{ zc=g5Rb;PI{L{b^-ueA)L@$W<;U%pJ0s(3E(jZBTk;Z!pSGq9wa zHR&)g&um3+LeBJqcwwO!O3us&^b@vx4}^q|3u?XnF2{WeT#zTaxZ|sO2+slDO!UeB z7mOHQ@Q95ru9wl=$rHC}jg9;S(Mm~nTBac+n1W-Ez&uhhsyP_6BT}p)H~7L66V^cF z^aV2eiHUJ(BB80eDqn>*hAtd`28x0ItbQtGG+aizt5PZvFV;1Kz7BJSsgw)h1_g)t z85=emRgq_HJyb~u+5y6y<5_{cgAu+Pb_MqZ#!jz9a{Is}5Nq>5z|1`BU(%9{=mf+{ zyeGuJ8zAGPufly=9U!^#btOeFDjJyc()hK0?U5$b08_eXEgUH~7M55CM=d#Mj6{o+ zq&f%jeR$_WQ`!o029+W5{vob+OWh`xA2uu;D!;uQ9QU+5ueDY~3;nJ$uwu}~89v|! z6DvjBCZlD87&Zn08P-MmRyq^>nDb7qsI-MIk2Wj}Due@k3}1}$?@a7>%HQ1x;RTJ@ zAZ{$5@S4DFE^C8P{e>vntu9Mx$N{6=yMEJDvm{2&xOqrNq#Am9oO!Y~th7m5KV<*b zx4qpu*oR3Oz+5j2*h}YW{SjrN z-v&0hvK5|;_H2ZB509pF!6ZX_t&L5}vw#YNPz4GXoW9Qdg|D+yr`2A$<7x`0`2LP3 z`SXo0TVg}O+S1d#&$BwG)A{(Z0&MdX1i5@bGQv@oe@dP$aS@B`Icnb1$o^PHY!o;A zXRIbsB?yim<+tzNbouS&1RNdLd>G%S!c1fN(>q0^eIoH!n!&rCYzx}I7s>%&eV*gk z6>3R|D#Wmr>i22IvzFk0rZO{d)7oz#qfIGRfFrp?X^ z>Dm-H+Ws*~x@wk{Ie3Oo-T9Kg08%>aj-^2k&HX_ZE9`2ek7-6TMaT%|#)S>}OGQe9 z&Ko+iw-C&^e1a4aj1Ai}-Cb&KSYkrQz<*Bbs~M(2@O2{gG>2rN=rbgR2(PwwE2`CW zeZrSFyn^dGsc4W>M#%A}DB$Gb!3ypqdDPmEr4tYUUCQ=xSJNxkuGFZ}lDL$->(|AU z?RqqIoV<=VjRxU2()~KNIdlY;quRQc$~j)Dd7$ zO2hEwOZmNZ{3&D?-vGzW(^^bG`Q_>T7HI*)TT3>Zvl6yJ2yMr&0{~;07$MzeEX!D} zACdtT@pR3;jIKr-G)FMCt3PHlH4XicmMZm0@$t62T_p4!c?f1^XQA4^s`pJ1tX&Of zeU)p&QJ)G04#PAo(5PoAJawt#9I_#VEDdusy263i(g76ND&D>2`VIReJ)&08|H2!@ zzF=mxGGoQ%RjrVz^tu$b$CX1*&DC9Zk2Nh!sRF1ckIdkEy8gFAyxR?|j6vKk*ph0I z^u9c`i=w}Tglqzqz$znMrrK3%u&5Npf&SCbqg-Dw_uo%&xt`aLVup0Vn4?{#Pts=% zjeU6q)6?n-u~?uP=oxrLa5S-D2yZ2{jn75gCpa-NB1{>kNB7wBfQ5UNgJx4=BhKCx zk4gW^)+2;0U&>+FVFZO2#KSH!e!(n==kuV5ZSJVo&HI{tz+POhJc$Q_aaOe>?E6(l zEZf3z-}~c6@Yr#Y%|pn&;~?iB_`3RAi0`dOl*XL*XqgzWu98=5 z0T(CJo!ize`!^1-In=htZgyv>irb+_IU#MoMA43m8rph(m4O@u(;L_@K!Z(8u9C_J zm{8Eo(xz7k z(`o|bV@aa{_6rgz-1{2e{4f+L`};+Mz#=fF8Dag5I-mNww+}w};Gz0QLX3d~tJnAO z55&d$Wjm$b9s~gHNJRrKJ`Y#$L$a{cLxwt==^#4L$+XA4{AhHzWPDmP%yHcMK!gge zw*CB_vdRLN%QtmS#|VH$#2hd)#luX3{zL|T*Gx03vuJq1hsqKmLe|zidOpvHHMY;6 zpS;ImbNz1^ZU6WrIT9uyIg$ZWF_9*OWe>%D^i~d0ur-DYdYy!#1`nbRVgEztM~#g%4;wiUWEfkuEKnho~T1nn__Qn5%}#O+Jc zVOxO#yBn?Q>aWcLP!-C0sZ5YbW!>re_?%L7(m{Voe{mh_75dP;1^PWyHvi zPjzEUfiCKa<4&fe+*W4Fdb<3l1QtL#lVHnZYnSbG=y~-!(?0k-OYbY*!2u?qo5Gcn zkqJ7=c`lFtbMQU4&+2U2>1YT2?@E$2qP>I$^d5J*s=L0feBXvYo_Oeea8Rt%x7dD#=`@M^B9@zC5kTDxN-4_N!Fo^Ro|K73+qp|@_pkZ+?Y~)iv(!Uwns#~ zl(PiksQQ_;`AEjRaOK_%uuySxm2;CBU!7X##_wpW^+C5xEfeoeT~)t)r>OqM;J_0! zrCr~JB%pac}b5Xkz2pb?~9<%F|89bbkn8~h2+6H}NT>04Bjq-HhA7KQRldTEXqn#@N0Vk_~S5!0mz7jM$*Q@SHQ-*IG*2mD)QV z4Od^(H)RIT*@oa$|2M#AIv)23lvY*-Jv3?f^cWyux*Yb`nbXah&^eZW%g+~f{b<0X zZFT?!A2K(nG5SnzSPOSJrTd9(^yvxWYx<8}l}GN}f9h|=vO=Se%F%-e-EYrvGewQ) zaPZMPvAmaoYu(=cuF6i_fAi9SbJ&F?{1Ae*Zj>(0By~%PFHPse1K{z{pgJ~3hOP`| zg%*K2{Xf4eO=Yaw20dpg)^$m4w!xSWcckFTbfD@_-3C+ zg_fNm-5G-I{Vmzetrk!}ujSw9yUvMouf>@T*rz0#<$+`FY7)_PW+%?q$%g+V%kz|Z z`>)5QaRMOUzPSbU?Q1^1BcY)u6;DRO|3lI4E98~5g9`|MUm_niI{&q+(xjFAnl)DW zQMhjgR4ISaKlTnS8(G#6jkJ$Ie%FJ>MHG~TV=%9>;{(016t?+|=<1-aA%NYWv++d# z`PSeeYTt2Mt-bN{y1e5%3Axa9#ag92Go7CwjLkP6^=y19wwoD?8PY*?N(wEj95<*& zsS@gH!IOzWHU5+HWonTGIXnwn_fSaOT*u*vyrmG(^?|S^|O~>$ncz-b6-5MnaaBz7U_8WKKdAX03GE@bbM3cSo;McKEfN=mV+; zWJn5@LEfpUsldZS3D+YIQtHxNzf)vpJ~oK2TeoMh=DrlR?_R8YH|-y&j0xg*FZV|; zV?$Q|?Arj?m>2}uM`f>1g$=LAu>`t1Thre*3z)iZc(gjd*!Adfk^QZ=30XnlBZZ?Z z&+XLocRb4)pVTeN;E~3K!w!%KG8p<%5AZIZq3=x8GXboPTvgC9LIwxDp6e*5>1IC& zU1Eure`SSKPEh9KMwY5MXOMiN8*xNop^%?~h*64*|0 zOK)BDy9f7t?*WyOi7~{p(p7L#t?RHx=L}vA@^H*rB~RV6 zpKjeqJp%(!Dk;7qr|lM@;enk?#{>me()si?bXmBL4=|xjEMD z%~!&(Bn1KemVH+qCD?c3w><}cw9W1ijp?nH2MKB#hDUlh`EU0%C-2R@CbdFezf$4z zv|z`*i{wm9(0lBTS~v}R#%kjD_5ZEd!lVj0JJb5?Wi|^7+W%0V^G&{({ULj1f&ebW zSgHL_t)W)V=%Y*K6Cq&;gc?V-$IWU`Nto1ajZWp$#u7bSW^!^&jxX5b82U+o3!jO_&FAM|!d&p)q&)&Yl ztE28Nn1gy+%H*BD04=S@6(*jgm0avXzQ~#Ay`QW*6zdK|*cf7@h+X^S?`hEkVUiNC zXdS4Zk{P;_>w3dYvc`o$KmG4%1nbIfRxVJpLTb`+Q&d!ST09#>OpvP0|@(Br}IuDdx$+KOlukCg9WH zYs4_wv)wKyNL?ej@v?no3#Z?qFk*{^K?Hc5kiz^)L%-9iBi*e(jD^K7o3ovNCT;^N zQlxcsGV;2&9wj=H6tt1)G^?DJPdr8Lqg8ZSagVfp6su`y{<0b5S_Ixh#+2pz<4oGf z-Tub{#N|j>|AhQx7^xkBHoLPg9@nA%a9ULzr(RrMi?u2u1HS(h>%_7^Xc8vBpkrVp z3t2s9Nw0nUvE%Hy9;U}Yf;&ND%x|fhM{OI9e$PC=&uQ-#%sy>JK#xZZ{H-ep{VQ27 z%zyqCn?72m`{eQX>MGt*?$LpVmf%PtT?hvDDrrsH=;2p8W$PG_mYp} zCf&=!x|D=`mgA6=>EJMnZ)U#)0q}#LcAdKOQ%U^#PC|;dinArg=r_d13k9NP_v}}n zH`vDZyHd=H{h=_g$JV3KM2d0#EwAk9bY8<6g<>%ki8$aEPd>~|-w(Q{?LC>5rh2H& z)hEn#_t&_yfF1GWh+0{i6~Lvz%UXEPiF|(Uz!7a&;oaKi2+U>k1;^ zZ4#06K5KXx4owpxhZ$Lmp@|8xWu!yke(7HU(E;kX_KMbYBk5Mzi_wS=o`fQOieL=6+GGM_& zCG#D`PA@MWgI*PVrR~6U=hkcIM*%Uv>CLqD4U1Ci`~Sz*J4fdob=$&?8e5HR8;u%U zjcpr^ZQE93+ivW}wrzjEzUQ8M&mH&sD}x z=Tu`lXTuiwTQB$#`${u0qqK}%`3eZcKuFF)kFmTNluvGaEHek$V1N4ZZf{W3nkWVN zGame4I%?3G{3{Cj1=1tMFV9?x2NI6z9^EGBW5kow8CFFJ0|Zrj_vfLMz9Gm{?* z1*6|EinuhcSFSN*WZPj`#%*&@EcBCDpv7aCSia4aYemOVNfe&i<9W z^9S7$EPqow$Fl*%-Gh63)Y3-Z_y!F&7mpW^_x$TCS`bvAn!_n#!&<|+W9-W;IWy+V z=?mnsE$0iTxm{JF(l_o!6RW>6CTOBI89}I!fbg87e&krAX~}P9lpuXkX~Xs zuKU5g5|Zpzr%ZbLa?4Q}FGnySLrLa#^O}F`Fz6SyOCqPHdzcdsUIbCFpdH-TdbV~}Wrs>}YCZ4wW+*NUdY$Ok>?Hjx z&47iW<_%ByJExcrZ$7|7*L*}e0x1#twBKXx=((mI5J7*q@j>8s(ISKHK^Ih4<2X2g zZL%`NRY9)9tB}6_Sg<|4b|hm@7}*b&ebN4Jkn?Ww0KlHhW+%c-Hscv0Hg&Q9N$ zdXR_eetwa9>USn1lS5jW@7yn+3UbPlg&ZU+LcNjitHN8|YP0noI&56j!K#mkUC1gs zq(9Fm-n4$TN<;-RYbaQdM)$ks@We_UK`uM0O)*{X^8l)lmvbEY5WFL1CZZpR8GI{z z#vf{90#z##2FAQYD{nWhh7kfly~Mb6qkb!Bv%fqo4;zAT7O=8HO(-P@bFAo>?U=m% zXbLleQUV7L-#0m0>7PIBYoCTmt413bnfBdQKUdj~FAX)<^t3_r@q8Hp{(c_F8IAa? z*=ZVB7h$=Q>G)0JGjH1NdOkmzHN!l6gHrk#eBN_C`XBQL$Chq=JAR`MkM%fyzK{Sd z)RvD`7fNri|CbLV8@bG$b$q46bgZiA3W$Ic_;Ki z_W!%`It)gA@pd0`xeUay=@`6pF{)*_v|anc-+B7$Zgv;%YZO`p$GT^`d`wHL2yK7$ zD5<;(J#~CJKwh8-@!SniENNt&haPQR>ky1YKSR!!_ye}q?1E{v`H;j2`U3w}{^~t+ z%dq=gSY>Hn&pUW=abkta&NnTJOcntUbhcA*6Xrt8jJ&^{#8i5s)ypz+ixEnY#USFc z@C8yV=F)yZ5BD<1{B<=lPWyK0gR9s9a7}L1nH0qNyq2CWGWz5df6i@%mvWHY8oB2> z-$tqSMscivJ(cB#AAUFgX~^4g^~S%o8^aE>UpF0eV!ioVtBSX~xn}ksq3m+-tHFCY+j zU(xeCj_dmE&?IO%uFJ$GR=~NK?6zKe02yDQkbupj!g1_w_J113%}*z>u7DRT&obwb*@?DT6SXzvH^E?bzGCV2B*7=Bi@lsu-^N|m zYF^WCnBnxL?V4So3bCO6dY-JqgM)rsd`~Myk&7Gj@<27&82$&OX(pXv=LbBO4=c~Q ztLzwGB)2FI%8h&kaYjUu^jl~_s~SH zpNN*jOpn0P94vs}I}y9b6%xp<$9L5}q`&bda*;k_ISCY!VxeQwk_In)UMJ?Jt<@gc z(|^VR1h3_>D`dMcK@O9CMs#ur-wWyEx|tvPQdhA2&{oT%AeZ!kmL~i2_^7kzImDJd3t5`i*;p`KU={-JMJ~`dZnzHMG5i>9Ze2S z-9y2#ac*-nBwz2u2B}msJ?7dJ z?;yj@8M3$HeDTN$O2X`*AS+4u&qIw)kH#wc|9D9O-ed}Vjfynz- zsJDGagILV;ufPK(WAta*Jcbf2^d`u%pf!-Tyml^o^UV53Lm?CcHK~`VIisPZg_n4A zxhxBWx9uYSo0Oc4@acKs%s|Gh&H30NwUzS7cVu%JxA`9Py0-Y=rRnJ%@5}!#O@FQY zeO{~gxmW&cs-db1$@4Z{y?rCK0J|{5$6}vv1Ra`u`xOkQ?m4~ip~<91ZM+49gvBWQ z*A26P4+BHzQ{<{Luh*1ji(Q!BvD`aSC*0fBE64Lm^7p6?xHZSe$90{r7_?ev8%}iY zSguFS-42Jv2^e>R0x%BoY(uyV9qb%zdf!S8x4S%ih+WB_?sjGL8r1?V-*P;mVpV}W z9xiu^SfgDMW20%$@K+SGBHzmqAB7tdW3HuTYv>HmgMbRX*Mav7xgOHN?7?fk@K3jQ z2`2;pqhwseitm|m%6yabGyWRrkS67*>8t0i2(c-ZemgwyMunw$KGPtT+P}=sGgk@( zM9Z)p>i@9?;dZ~MhIhM*nv{<@YuCh-ypEfl6xZGfF5{<`HdI<18L$+o$suu%#&3qpgFvdU0 zJwKMk1Q&u|G@dqJFZ?$7SWlb*my{!w;`{A(G1R_AogC%p{1^$PzW$c{otL||@H5fO z1M8?~vn@Xs^!1Rx6KuG?*ELO>3G?Hz*{5goHeyJ3=MjZWVCuVBrDAAM_5}hZEp>vK zoaqJ~kRh5bt1)ggY7JQ3zvn8`zaM%>BiD>hGsi4$F1?N1OGls53wPgx zW!JGRkpB?$%*-|Ox?b0BIU6?`7KXTSUG`c6&9PxKSbMov{o=I3$*Et2pFHu12bJ}w z!MGW^>5^qvb}?+9%J{g$mDA0cxP&)dhsTul(sZ@>0|6x_{P3PLnX7088lF=g@qG}1 ztp8cJfTieKu8;WDUpP?>houq;L!W=tJb%XQ0ZD~JUXEj45Cl!W% zJh*g0Iv O^#iXN)cX*FA){Q0liytDk?5?$*}{dkm}Rk-_O9MzW&1-)B$Jq7yD^M zdlNN&WXzY}M10_NT>0=f494&d1htfN=>Ev)^Aoyz@5)nD z4EYD6RMoMx&}yb-P}g3jTAy@bQvVQcDgF6DbZ%jx+)j!P_xm&e|>q)t#0<{ zbM}EQ&hlo@^yxLeI>2FoR!1PB>#2pcZ75eF854uxB(}A|b+YGEBCbG@tTTUKrCE8?mJO1p> z&mmou4VOr#zSVi0JjkTcuE4}D%t?MRdC-{d4a<{&DIAU%x$G-%8x@Gm?}yT%i_#6V z7C1H`&1D-wAh#tV6M=pDl6Eb`94Rb#f`tTqG=t1BEvmWgD=g7+4m5r|hxvHx<6VzI zMEO-Z`(|mFBlpFLjTCI<^6hYRf(7@@f{w*=Wit*ge;raUu#E7eEge5-v+Q8iJ6ZMp ziJu1wP}e2*yw@sjCX2rwiMq_@)23Di%tgz9SCS)`pj$#jY^VROoyxOPx~o#-EE6WT z+@GS6G2`MfY3-;Hge=Bq(a@=Id^?2}I|csFi71WdRg!`{`Ohw5rx~was*%3V*q83= z6=KTPp+)5A$GEjfl;{t~tbZxqzg*%!^$zgM0D@m$FNl6k;kb;11Qt$Na~ItK;? zd6YJei%Da&n;-EdUsqt{ueY%EX#Ke8I}(nb8dd{sAc*X$kS@-xnL z>4CJ>YbNwQ4{8Zw3T;adfXIJvz_I$xu)7z~3QHZHCTqiApUgnsWgGXq*#r!$r%XBF zD%OsZjIr0*p$5_3igPjR3^QM@z$$jGwZ|K0o4U;~$DrYK1=3!M9}~B^kd5(B+cHLJ zOwHZzbc9uD@}6pRl7dX~=X2&cg)^-DwE;_TbAp5T7I0Tq{}aNcA$z~i4PT6XHdhhgfw^jpRCs-R=r->v=h6}oNbv+3mJ@)|jDC|uMx`OnRD#iPV6 zIT(i_i7NzN)@ZbWR8a=L02$n+x6M_X?oX@3>c>b8q6F9r z5p-0JUs2U8N|*aBaYRanhwlGvI{$4)v6y}vBp5xdZf^b_=CzN;`}KoA%`D1Xa$dR3 zEvp zhwz=1gDo`WNqGl6-e5qR*Vgi?76}j8$@+ZR6S=MEg8in+kHt^ zIOTUZb$_uB^GC^C)2S8=+AvXC6BaZooyK7)iv%amgKvU?r#l&0BMNCEGydCTeJR{L z-9Y9$O(OE>>81nbq*NDr}#jnw$wPG(#y-fFD1AC?oZ6%{eKNF z{HH2J@(X}`>;?KHUkZBqdD+Ou{}-%Q(#3#;ETB|y-qq$hqHy8J7suzvxuz{^->DvkyV8nM`C4>j zOvvvuUc*4av3c2JE;O>J*@aZq+{hNt4NvV&vjwX_6q`7JF`@v%5F~o8;1D96JitKU zxn3zuj)9ibQsi844Sooj~cdLk8@zd7L?pV-Yq!{xIT}`HHA>P!yjFPLoA{l)On<6 zVCv}furmE>?Mfseqtad(x7v25S08guNT7@9rgBA=W^UM%JQ)#kD=V39SE0#f0wEKA zCLU(ulp1Fdz4~8V_Wv0Hq$B0q||T*06lh`TBnV zZygosgFJUHGa;9LrjaVw8rs%u9XB_QeVR#<5GmUBm97%5l~vqF_Zznp2Mm0E|B6sj z1>F=n&26`|sWbs;l-RlyKORI82VLEm&1a^xKXADY6bhWLmA(wZLB0pWroj^?%6DJm zhI#cPM_EV9XMo$OH_XhFH>d2ycERdkPY2W5N-sl$yb+J$YN=V|I&zw{X6Q4|{4iKd z+1lgPNo?5k>Z!SkP{YmmF~mt^Y-jvBXUrRh39d6SL32^$KI^OuW!S`AoKpSDu!|Iev#0_@Jj~2XllCFL~wO7lR?-^{; z;fwYji=p*7S?jqQ3Ibiv=htSvibw(j@Q zYhxRByOo0JbH~0JwjD6pPrLm!0H^Qh$PCc0@qu>v*!q6;R_}v)V`O~T^FNWis}z3z zuZ?WV^17-4FnfpFFP03T{llTL_nEi!!QouD9pURQx{G~{syT?Z-S^Z=czj>hwXW&>_()8t#pZACJ6e(6Gd8qXazGhl#zkP*=BKK)#UZO zW@8=s9bwJXiYKA9qNoZ#q?6scND&Rg!v$gRm_F)KF``ugyj#TZeZ=`3Kuon6Ee+tV zIptU%rbe5X}2|2C&>1_gs@Y-bcP{E>f=!B^M|6W@0s4uN4HfAOFCm2)~7jM@L0+H6o zZr8sr*XSv|M$%IalK9^L7W8eWg-nhZTyY@seU%s)*=1Xfa}6^( zGxqfC`^DSxlhO7&(2UzYd|cbU#U{&ko(!^Ezvp=EoW%S`d`}%6>zR{#2eD$uiB0jV z2i~q4O|m4eLK%3L3 zRDE?w119wgh4rNSiHmeZS{Y6_Wt7lH$X8QeLFK@+Rr<1HR(X?pgcib*$FrT`G=~yr z3Ww2y5@+(^(#3WILN9rB&t*3yEy8DLVFuMP){P>}yL9f;nT%ViPxGV;S`#B;4T!PB z_rx_18lByS&NHiwb%H~Tb%_5El8tqg4$R4%E4i6EXagnNjGB*wwg+%AiZy8B>R?1{ z)#^~Ws0c>d1#K2eL>R5Z%`YFSC1hprGvJv1+~kI!Gq<*8OY>Bw@ZQE5aB?^{oc)TN zTZ`KGR!IE+dp^S9Jx5>HXYjhw9NFdbT#LHfE?w-#Ybl;+{~IX0np~~!CHTA|Y-nzD z-zV9>EiKN0;O}}>jg5~#agN%Ejn_z|scyrk?>K^SeT(T1$hPq${3RTp@N>&QfnaG`gzKlCQtYW1`cB#Jz*0RGx?dJ#ufldN4h?jpFFgbap%9j^E?lb^lgtQUL{_AdLuX< z(qBm*IFb`WgWk;4i6Et5d(y~|ECu<$sv2k2WQw&ed7cbV|IJ;P|sD2S7 zgB=I-%*mmDyM>HO5h;UWDl zxhAYmQBfRIS)JX_JOS+pcoha2q*2^@T97jAiY=@d^$Kx;2ju_O0*E}JqMH`5uD7)% zHt3S?`eUgchbx(_h50{_k3?Bt!BH$+5Xnw#&LpZVx|peqFRj@+ohFqo21u8~C#hN! zlS<-l(qg&!ixhopZrkrN_-4AC1gN3lAUDj@@~S~Ht$HdVTcb)`u2_WLm|Ct}8j6_u z6}P%j*R#C+*>N{dnB6aQvF#oJheHGyIecKEai|q1AJYjL^V7|qmrvJ#E2PD5c={!G ze(cb;xK}BTfh>>j6Ta7)hHk)Kt*R&-p_r)gg}cG7NfxDCeH^D$wRN2AT5qf)OfqzY z_LfK60CUgG7mb4bb%2|=ae-$wuE023>t6!uUd4Lp*l01XG5;Fww0VrWH)pysARl&7 zxR?u?vQw|?ZUSSqY8mXVbfI|W>>$?Z`2$obG15k~pkboDV?EZZbq*XeiyfQa^107- z`gB+Ckq9I2z;a412zdTGx3wQZl;!>hd=Qs~ zS|cgQ>t1*XvAV3|aSTK8e1LJ*R*{wbALsLLL!;QGyv5=@LxlHIfZeBtF<`WzmWR}+IR7Ur3 zafW9t9MBa052GVST|D&G7JiS!<8Qp#;%M(he+{V%GBwfsKlEwqz8|M&`z|^yHJB+* z6*KC>q&6?!MRyUzB?kvGee1}7zwcE!h(_P`@B@0(Op$SFlF}vRny2>rx%}cLp;qy? z5sGAdM1lpMowPwOokYbiBZD#Ww)K_&=G)i1B-^a>*xpc|r&ad^HFuxwzqNh-xvpTRl4Z zP+G}R*3YuR5QQ&B=+Z;3M1xoDIXFEQxA+9>^$7W7w|&b6Dk!&rXACL+%es7ZONbCY z^WMnt+%XHjjAfClwr;fSc~TqJRkzJM_1f+`B~2tPojCK`7YQMwJe>#xrd)7A3=ree zAdbQ}Ns*}GbLc5yiwC2N=b?-HYvxC)MwACht{q7<_V@zl8@vdl@48>LD{PS?%FT-4hKX50u3eP_^yaEL7`#_B89Apw zO!G|(=cUJonfd;^&%N`e@1;QmM+OCaJC+}x9)4SX)1H|qA`o!XGMGQyliL` zh^a*Vn&kzkaEL=uWq4TO_5C;pfE1Y|=;alj;W-Q;6mQ*sqb&b$F+lpMJOxB<2aRG~ zH}lt&r{^SipNVnPL#CbiLs{l|HE&5SsPf=xB$t zA%`&JF1q$n&-yFwn2DN(%KFe_qr*?DHJgQEYE?^EzB9olAJBbO6z_Jpnh2$izh{2& zBc!mTr&2qZAtn!!#QFnGcG`n(ww{ul?|*K_B(73!R4JD16~UBzoXL#W@q8^lfHvZg zE_H1}n8GKee=adktwcbO2FbFM4n0W2>lsrRYV$j5CoJXrvCQl&RP$ATB8M>|=f}aafy5Dl^7wKD?$VF->MeAWt(kriK7DZdO5*5mg z+t|Cx_eNiy;rs1&X57_S#VA}2kxp|}6+L5D&qv=)ruJ%bC`RqxgYaND82xm&V z4q%!(m!L8(QWmT%9<1Btxl#D$X#+n5fdN%nA~tU%SXu_bsLYW_ht0l!C~(r=-vD~) z%$|`kKiE&74Q_{qYB96K3RdatNkJrJ&@#Zk`4Orts!OVftiUA{?Yj}_oQr}nb@ETe zSx}LoXJUaKi~jp4%XS~(fYS_iv68DU?3RI-7go7p4ZH}qtbdP&@#h&o&Qu$=+;4^N_q+kI!*5=*u4nG#q9-5PL7aiFvhmG#+;*(2_ts9h#Kwg3P95q zv|2<)PG4~r*D-87nH0NW>0(xMXEcIv2HnZ{O%(0s6pS|_aHjr`65QLnz)ivsUM3UB zO6o}Mh-sw#A9T;t-vkpa2vxq&NFdv`*u1`xoWWNvjCdseVhkB$z+m#xe#FN%R~Wf$ z1d-x$f6AADPuO&Tu3?4@=NBHSp8G?} zNG>mRn1I6*q0!4Zd*s5gPW)r2%5?k9%~VLiBJXt@yLIw{)d|{oB&d!qg17^eZQ zu{RC97(|^g2J@u9X+g4t!0ek3RmJ6g1C=k`?H}(32#Ykah!EOX0RFMVZgaUu*Gx?O z{vAOrw|1gmi>jg=LEyhyQ~5zf)gED-Y}q(0gKs-*O)%&ZuHB;wangwMBmOifh(9{b zgAVZ2@#fx5+n5>8glY17Dw(_*Swi=ZLw`@>mKh8bS+a7Cm0G)Vu9{f-{<36I(@rjN z(Uzwz5X&}^qC48z$rDfo4O(&NC8j#b+C>PA^o5{f`R>s;?H4k`{Lsm41P`trSE6Qf zzL5V2g}G(AFj|TOSf((vICephxmICy09fut`;~Y+sflkwE+Frq|!YU>Ni9a zm!`|F+~la?Zf`v)G1AEpa4KLxS1BMDd~&*2=rEXi%q|?zx zdPVxYy@iw8I)X|}-OqcV@u)0s_7jh*M0&w+d{4jOYhD}BvB7f-1LJLDPmHKB-ngT^ zHS?B;yB*IEvT<2(k4)q7Pj=e_T`V-Q`ZbCQSm9~Y|KhwsG4x-*!tosWP^y6C^1h>V6T&lR!lEa zd_eCsJ6*)VtV=4$Xk>uD`_O1}a{WSjoD~}bh|gMm|DzNA_NY|KyyGoHy4B;lZf&-` z;Wug+K=qb9gzAB$22*|6!%0Pjsm{)pd($$aL82x|rEi61v-^L^`=Qe|Co;0*3t1s= zE$!8^ZSrXBquS+rm#r>DXLlVPfx-|W%jzTQ&J}p9yucQ#%8M2OQ!RWS11!GL@-VIK z1xprp+cX?Rr~7lf1SppmJy&Pq=+CESGnZ3qZ^tS&@hqtR)rU$aOg5f5e&=;h>2(biY|B7wCur+9 z-=VnUNWC)VyUySLTTd`I$bkzZZgF#4$#Cyh6X>BBG&j9t&IfH8N-^Qe}8=MR8IvS=qTn+NME82wH<8 zrNDB`RoOo?q;RP#0kjndeK_j8L?BwgYRS%@6oQBO<^ zV&+17*TdOghR9Ri(Qdw^_Hcq4d99gPqp-68N&Jv@(jU!-E3zoJP*DaTScbThdrMgL zY7@}jki`~FBxxTmGt2$@$)48_Xv6H@bLev$C&c1^HMaV%ai<^p`cy_HL6_c=h!d#B zxD@K_5v0<1MXhtkYS+l*9q|>(-fN1q1K779JU+a9LFu^mEC5Wr9iE(a;b{3m z9sFt=zHi0AUUAw|S`wqKzJ#JLWMmV;XoFPnTZ_g}sHNG5 ztpYWVN&`HUC0`4NG$9(eDDZvDoAN=CtGa5dhU9k};GJztks{+%1C<`jRNBcEFYDtR zDMd)Cz;6xj@1o0PGy@TYF!LjB2>WY~l+ulb>C&(P#I65F7n49mgxPOgU~w4u78IqZ zwqtXS_A6_H0*=pjgtWpzXu8Bj}u^JcX$ggsKoXPhp0Ly1QYN zx4bm*@8dMB^-_vOlpjCRa<<3#(hlf#ic;1&m;N31!MuK}=E3#LZMy7oheD}VY&@my zX9%D@ON5YXmFpU_ku{x2QXbii`IUsYY^Ba#koc8R(omq{!%VdyqxRLI9dKptQ#t68{ zhBMsSvL&^EDURv8IdKYhk_K^mF{GmN6Y!?cnpBe^?@UhESvlx{NehI5QyL9;8y4j| z$pNPqM&oQHU3*_{Y_$fGWt?wm6cDsa3JES{khXWUpdmb*>_!az+(`}4bOAcy0v=VI zL>jFrk#V8QE+`J4l5rd&NY)?d3i0T1PqN}d(ceS_m3K?uXRys_aq!caH*% zq05OhGYHDR5p&zTD3y!R?2$(j;g$=+{OKZMqorjLGD&x0KmW(v>`V0AZCf>PwlLwb zNnZ?rhX$TC7b`pU_>)lX3o|ZDa>i;Cou)QCm&UAE^SJX^fF5Q^^dpHgi@F646)N=K zb#oUuLkm7J}H-&P(To%(Kx=(ADkN zMRY}jL-IGF(;j>|#9iew(H0)Bu5zcTO56Az$uT}$0;0W_%%4DqQK857z=Iio-KYk~ zY#j~k0&>VqZYEu$;Qpll{4YVqnn^uSz81{v*ji9|_QN{FIXsvnAJsJW5Y0`dK4%&O zAxncio^C)+fIKbTN(LWDb0|i7Qwqg>49XS*-$QmjEEv&jTR!wXg0CTWDKdbGQPB}%Wv)a1lT(Qkgx(No$yW3o+vDc z%|c9vD#hx0qGxKU0wqxLRyPy<8NNI_18wqF4hBD@6UjfH#z?LniwXYdU;nWxny9S4 z=IjCSkXhAhwwIFjo}7I@gY5_AnVp~d`T+%VKvqYBoEqWL6Z1%t{6G856;KUlq1X(e z%{R_QTXx8KLzNFV#;@l8bfg`mp>l^jLfyk7P~Vd@l$fP4q}@qPd`k~Jr6&VeWRjV? zAYu%jY&RfxfWQD}M{H#2<^~DRUt$&7Jd)|4s!W%Q?O%T1&IE5=~ zmeA!8L+U1D`w3d7{cS^Y@>ybLFf&qM$R!=)(eY8xl?8F}WoM+6xS8o>i{OFUGriRs zG5qa+YTVoj))eLWbx$_N*a37PR1KLWIStk9a;66)66@4q#D(Q(oJv|d08d-{vP?2? z3c?A0EB|B5DrD{PRDnjT$d%KL!Ot)!LzYa-TljQMea{8W^)-=+rK%G@Mz-u zTfI$`9#e9<5q-ROIDr1%1J?X*wOtuEUi^IN2fbt4090X#A`_oX+kiW|rwg!ISCbjq zJl21kozar%ZOlr@qb>IXazeAeqwllmYiE)i91{xq$#v{Z;wiuO+(o|^x_s+0vMeNw zP1wg^inw3j|3P@0)>tG38IpnGsk1u}2A=uE)Y{Ar!tK$B%HR-Q2<5beJEJv|Sqm=e zyXc~qDVP*dVOBozDyXbIz9>YSQ2|IW-og@ze?iOe92*W1uy0l;ry%@84#;f*n?3=z z7>CqLxu&UUcSZepY6ViG-LTjpM4}!iJ1eVF? z1NPYsMVF1mnvi_v3!YWC)?1?RLp`xyx6ZVNmRK?IAoL(ygbUnEm2uEY@#bRrY9d)m zZwP~Qj$~~qFCZOb{SIcxwKO2=dU|bS)TopmEImVrQ;c?wr03Mbea)Y0-k{P{YERD> zBK0xX9J7H~69dry1BsTwha$V=!Od99fb34d2f@@RoHhMXx$k?&O_7IR8^QoBBkHTI zFXHeeH{7}Y2kH?2ESLcLJT0pjtEwb&(|E#=H1+(y;BdF7D-q4CJ(awdA0OYHCH@O! zc5NEBmNMiEl>n72^4K^OQvX+}`Ely7+F7G8t<67@duh@h)-*H;VEd16EHPksaj~CM z8y@^a&N^-z#f~J0;-KJ;C{)zb$--m$RwMR24@PdH-%rsJMD{rhm^QeXSw z+#}~TaLR4k>oxl@lSI^L2ELFRG<9?Sn&#KNuxBE@$OgR%_+c606@ESXA0#A7(J!qd z5i?ZY2*8iqUiDKb@e5t`+^8e8qT?^fU6Q*-DT=O2n*QW(`26>fh>7wjA{C@H6t1!G zACn`PU#ECzOn)9*f1hVTysf_D(i0i6qm(&EfVIrpKYDarGTPgm*r{2!Wbhz~h$fsQ zBqU1H7haG{S+hkGR)Q6x&GJ!*Pshn|%u_kEWla7PU>Mds;0Z7b{luosV+fiJG7Qs6SM=Kw$ zLIfZ9Y#LgYu3Hu+O4QNMJWGoofegw@@Inq@>DW+sq&WeRiDU?xX$JWjyhMjsuiI&+ zmsCeiQ#|ARuao@69)S< zixXUwkE_VuwKT40_ce#ZDkwam!Fb00xP>-?3qwMgR(2yWcHbLfPYpDLtfBluHJ}C2 zf1^MA6KR_Nhkfs6Wia%usM<5@WB2pghgt}vFUK^hES-Ng7F{2`#*< z)5fVLWG>xVX;;SM5EgYCVO1ESCZuOSD3%Wv$^;HMdb8_+{>QTI$N5gp6!T{$eLEMT zt@SrV?X!H-nV=(Z19-nno=2H@8N|4d^g|V$ZLxGJ-ZCLP2q+^*-fev^zD%c;{@^C0 z!32w-oNdOo=})R}2ks7)sI)p&ie$@)%O$g^qnGp3QP5z|HckB)3Oyt9Z_{SwK@ zYrUL6oDkSNlu_zgiR;kTnwPI+%ok6%RQ`QeKl`3QJ-qs1&B5-cg=3RdYAIa;LO(nt z*k^U|VMrEyKg^md6}qs3T5CeSNqxC(cjz)0#|_u6>;TnLNB7gTGD?+@RZ)hdSIGeO zkWs)7oJ$Cc#xfz{b1WGti?AGM=v5Jg4DYEwQLA*i2dAn-R&(BXx%?U>Xg=&r*c?Q* z+rw?7oz!Ez@B?N>&&jw<(ipIw?BB5&Ydi~PVP-^Za+oBhetlnn1si`+!Yc~|796UB zVB=MBkjGptXkP$@Tj`d{w)18VH^oz6B5!=(Hu29B)w$(AD#JHYO-7yZ8E$y9XG1Qb z;uye#jEs2sI)72f@pbL%!5Q)7l`N@G8_>SRX;O$7Nerpl_+@iPh?A#tIXY|fg#h2%Hf8rPTHw;g^Q3bMi{S?r&$3Tx(Y?e(YlN+ zX5&yF)?VL9N_HWZ(^pSjVDg376HBD&#y!i{uRL?2Nww+adnv_q_qiL8NeZ~+#;m;I z9k7}NsBEQ~f|1!_uATg|nfJHV?$xY3GqR4C8D51`^sQNZT)a?4#!v#WvU2|9u$jSm z4rPMiy7z7*Ua8s!Ahy^<2F5lo`Y(GJb4uIqNwG(E?7ejPL!bm-ua_E(UuHp7X6ee( zT@lxv7NLt$^+T^2Xot!YUJ&`N?JXQ(w~kb9U;I{KsLn1>)*%FEO8&R%vF5cEeR2Af zrS}`LJ_@QWcx6{M`toFf9#qm9zg6&Tw*`x}^K7?NpM?p2^`2?1oVqG74B#_~`Q9me z!A;+sb#D7QDrY)x=@El>Y^y{z$)L+2Yl8)d@d_+wU>8_z?8{Mte>rT}+B2*zd-}s? z1-M*!%u}8tZa~K~2CE5ux%Du;K-szP9A4Ezf(3Q?ZLAW;2HY;fNs51~KKTZ$djE27 zVQ5E)HSU}94U#n*yYN$847Z+ml|1GT+f>6uXH0{bwE!>1FX!v%#pIYca}(KKeeSlleP<>!*&yHtKnoL@fGO8RNgcs2YyfHm zOr&{4hP~-W1qq?3=ZGBqOiB;02{%_r=H`ncT{rvq<<2-cs$pigeU&RwCR>*8k+2h( zJv-nSn?thLNR7`S|Nng^0lfBFP9NgBpjx7XHMu zcRjEcJ%?Oy?cxZ+k|DR?x`nXC)0bAD!pMxjn46=EN|!sk8L|o=!||Bj&j!?VOiB|y z5c)oAt0C@ObAJ=!+vszu;HPel-%hHC!Dq%IPm{3B;A1=a2@Axe{@N9d*o+Ph{}yi+ zzp?Tg4+Mo(c1Na{&I}Sytcj+~7r;(JRQ&27`-e&gH&wRa5U_p-s1ZQ%>j}?G&O=Y} zIL<>%b$D*1`aJVQdM<7z5st`N2RS|7&0M3#$ZKjAqfcO3v5dF6%U#68VovqL-puUg z)}a6C!_MdP+g^f?SDr)iKtcjcc8#z!F(mkpo>hH7m1==b8wGqmK= zFRD&$b`U0t)-d%)=`~y|0~y+{QoOe?<&##f82eD!E-6NqVVRPB88H^{Cm1G{n$C2- z^W2WlA@STc)%|%}9UTZI0o)DtQ524XvxM^nzg^-a&ok@&-9w7fMulLMvpIJcgN2G? zm8IAcjA7^txWO@H7zg-(4)|B~e^GC2*vNNhn7|T;Z8IB6k;hM2ZJt{A+Xnv7<%b2= zi5KsI3!lQZQzx|To+l-{rwhw32%Qo3zfz6b@!f!PuTIY2SOgNE4dJK?xqLV{c%9wz zvbc#vZ5QT*Ld!Upu`UG6BdH#1sQ!IFXrwH47M zq1|_HJpE>?l2Nn}LDY3lb;S-0W6!VnHNR%<)18^QfeD#LZmcXghtw+z%ieo;)SkCV z3ht~_J=?vYw*Xhu@1Xew$9sY2FF^>(Vn8lhzH9n-9*6>H>Le!<)3E%n^zS=|{EYcG z;inyg{~ud#0Tfr4yblKn0fG}GxCRm+K(OF$!Gc?G3pNnkB@o;R5Zv9J2@LM;1b24^ z7+{d^=H379Z@22Jx^2BQc zg*~<9%`c=H9*2RPg(T6v8pPrkXzd)VnZYZkYss2vj5`M3gc35YE_k4!56W097?tD3 z7|n}#b-qLq13knEj*40??5ZIgsB|B`kwmUK^cs?VM4yP<>o0b2<`0$ar`+u)~e(qD5t7P=3hOcFf;7>B&t($qZrpr1Qy;6GczEc*Ka zjwc?@GDaV0q1KV_68+ZA+3%|e!Nyym67hU@)6@Jh*yVf)S5 z=Nkz#qRkR#L}Q7>7dA1Is6d%0ry_sCcZ(IpK=jPMd#2(1v|LZm&AR3(aJJ%*kupeI zdzfmyoxYMcj}x);pxYlcqCbCKrDQt&4-P%Y8CzrXq=d998L`hjR)lRshKDf@d8YI1 z#&ZrorQsXdKcYtp&!DL{MzAVKLg0!m^)sX%Lb^{B08r(c4qDo>+8a=Ww2-$cT0Xw0 zCPLtsx>;~tLp>I@pVrTEI>7|?8PbRbL+=(oSK%#-kG*;F&&Pv_KOG@o_^NGW*>}i| zPCv+)A$M~q5sST}^}E%4%>zE5@}xQ455fgIe3!s8f0)ij4G#XEG^0rNW1?8Rbb2-f z=YYRj<@{c2qakeB#Vj+up!DmK+{+;$%>$KuQ8_GY8FyerR+)eyVWJE4zBMT&KjAX& z$K`|*J+g#F4u8nP--RH50Ca=zcg^p@Q2DfOD1osOBHlWaWv-JxrS}1BFIUMoN`PWF zMK{49%4=t!P?noNEYK9oP29<6n@hj>dvJaJUca0WpE`_qSK{Pt{>Q;rR8$%62;6vI zeiIaM9=&~zYAZ;W;ob+6HdLDrD>$P4GV zxj{PjLM)O7a=4V`Qt5D!qE{&I623w26YtA#xX?mQQoL=gTqq#37heDa&4*8&nksNq ztXw#?S4@vAWgI}4Q3>d6ztqil+8p=04py1~ZP%OTeW!;PxU~@EQM@^QEntxl%gDpp zIqY7D@D{EoCn1R!>J7I|ig*aT9?5z^>tNaPiG={Grt!Fgan~;}7-nwXDeT2fwRW8Q1a$OTmpr2S4`Q99`b{jgdfp!z7ab2T1HLe z?~I@lU8-6?z~ouzQ5J&GZa9X@FTW~YZ>I#3!?;oh1& ziV@M{oQ`>O!$+TIBcZyTcJB*zj{PXT zi~Rid6rE6&Vcv8xlHQXqH6J&)PMkgC*>QlAUdpjvBDgn5WOy~OD%jJ1FN|ytLlu0KZe;EUpN!jYwkMy0I)$9w~ zWEca4Uvtu-Z!(L0Q#=qK2+3hu7Y2iE}HHpAFNpM6d}!<t0@eRfQE=Ve_{MRu384j43l0P$5 z{fp~L&2& zBDikkPS$^p_?T^tcAn!-QZ+NQui+m<#4}y ztPlnPGhYj4R+bX3adb~6o23igVljz^*0SkaNL^i4WbRVct_z7*kYM4`g+Q^|3FJ!f za+l#rg2~6_-u7;0TrOYVN*g?LiEeg7nRBIG@V$mq3UpBuJAd%q2ILT$X!(N<&)R+B z;qlqP22qGN>M$M7{*cfDV>tufdT9VT4|Ok|eK5g*R*?G4%FbyP14+hYCqKNXMtxn~ zxjir3%m!~(w-=qZMs_@$6KSl4#@>F=9ubu63!vz0biJ7taU%(CFP(%LKVlu&Y1qUj zcG-Bw$;Vso0}}oG(+?=G!z0k7wrUE#o1k}vOM0u9kR>`9&4b&FrZcsBq}RABM>aSw z##TzlP59dgTWaG(-a_I0W8U$ms@gRB>fH4Fre=8vVV`^0m|h}kObwynh$Kkiwm&o~ zn~s#u4K8gx-N1b1p3cgjx&zc<+JWvV9~$%Lr|ZC&IrSxIMHK6$R?pt3KZ`bv7a_9l zmD2}shdjhz(23tCU23zDG2xrk$4{AkN)H_moHzn9qLL8iuzkgC$FpHoZO;SP@&L3l zN^tFk$L93P&#i>~gTuiA+L*L`Pi@TyJv5TctEuJrXXT7|8TA!!6qaq#4s7M(l#Ili z;&+phO;qPmE#ENkH_~I*s3~|lVh5S>E0yPXH#cq-R%lql@7xoy zE;zE!Q|v{52^)1jY<8KxHpAWX@tnqTk7Y3mR91y^zcP#4Q=J$yP8nve&ykCu<2 zMs{|t5eq4*Y*dTB2isSlF{-`O@d)`u{?*!Rsks?(oS z2T74uno`@n@W8l^7Jwgoc{kfRO|OFX$>Qj=m@7#|GdP^-`mSs^{iXTo5Mqs?;js{3 ztm)H-R3(V#_;Q1)snKgFA5oN*qwd{F;w>c3MY`#B>2`1W!A#M4Vtu-U*ZxbtLaTbQ zXl9`0bqx!>`1!QX-cEV`kt4GH_VvUMJ&zBWempgLPRbx*({D=AjcrRqPE1dwL_YnQ z-j>{Sl=bED#iQa9s1ijllB+w;aGB!V(KpkB^_PL8!s&J1_Zzju6S_vbbhyXcDeN=Mb+oTSCg)~nUWAvsI zrAAbw9?T->I&`mZwn^Ll8d(EpLFct34UfB#y7y!M>7)ph;&{u~=YjN9Etvf^c;MyX z#E5i{9C^+ggalfzVllErAyVs47}dt|iMH&{(Y^Rx0MU}|_Qt$a|Aip+0ot{7Hkj0V zq~U;+PgMxPZ6{$2Lvv4BB;i(*bnvsQ=XXJI{8t-Z%{aTbI03e#q;_Wbp?!j!W9z!~ z@;Poz#8-acAk4lI=eJhoo8RQT!@i3Sv+j<(_m-x+z1%V)%wt1PccA^`Q-{im`k5tf z+iEzuwEX2*!9`S7XP)lzO5P!^5iL$)0}p8hJ(hL8L4=_m-dwMc9_*wsuF|OBwo_cn&v3Z} zf>vfg)Ou*^AiSc$iI@u?jiM4bP}uiAQ4H9;mpxyL3yv>kuiPnT4%*f-9^qi^Q7%*! zkW{gBZ<`HUmEMAyyKQQ&Bz?ZGK^4X5+i?Cu`bqT{5o0@38CiK&8BUCs2aW}fmNFZU zaiA~~wH1EMDf=yj!*4G**zT+NeP^w~;EDLF!SQTM@ZnwNO{XQOdyy z;gfSE>~oQqTi-qdH(2wS{P@SjX z6U@^;KE<{_RY&FZ5~o;yE`I`;g2oTZSEvA}E8%^Bv+T#vkkO+9+cKXKsWc1-f3j?! z1c~F0D|1^Y!a5oPA{%Rze=+u!D($LF(Vf6+F2o+cUK$9Glumm)PjA@HMf8=oVxhZR zU)=rR83h7x0gJ=AliXV4$AJay5ssgNV!EVp)7>ar&n`fV-X*^lgxPPxUL;+gK?bG^ zCWukDaQ`Jq96gtJ+^>k^2voKFPppbbznlp+w5WNM6cM;B5dC^=qhs@ zve%jDC`BP-HawLdyUErJ8W=ddN;!N8TiP`n{X25XEb(NER0ia#gr& zY-Q7F4RfL@Tn>s!oh%cG4#fo-=4qRfJ|ts!UW|TIBWJCj=Bzyo2qqE7yqxdK%(F{} z7qVnPqK}0XC>Nmq&NK8=vMJFko4d+Fc+nPU=_?}?ollP)!nsf1_E-P#yL&;(j%1RB zg>M%nMQ-ij?Ru5mY(?`5<06DpHjW3lpqeN+IRLM>J+vInaaUg5L z8W2>)HGJADCHA5}s|;Hb#1Ox{a0 zlCLXW`-WaAFsN|)6ZbCj>KP7vy#2*&WWOqz6uf!qvD15O{mziw`(UlllGUIsy>7Oy z3?|xX7jAL2b|0g$G`8L#{B6y|sb#as{IicRDa3%p$E>-;Nf?yy-kCDw zB904(An@q3I6C0yB=lj63fJ&=Qa6*S{;^jbF*bC1)0l`<&;IXH$GaJIT%ESMxzK#_ zkW(=Cpy6fV$!R~b-J88&a+z^f5g^!IikRZ8Ve4IIKD9Nny+q(}E}qkhOMAJ3VF@}@ zkv?2dnmaG1E0>s5GdRp&UiMj-*`K?Z92+j&+mT6w&e;)@pKiAm9G)a9aksqqw_GwZk(UlDZD16WT`BWy`$4DR>8mi`jE(HO6C~;HYG!D zqTkK9?^@0_ciLg#Codm!y|4>o(g#cZAuaaC$eAY69arv!v}(G>+4mXUPcGZ=I^xBU z0_xk))r^lyN|z9W=B&~Rrm+}iqpBXL)>b2J^|EjerHw>yz4-_E1c=l4*>ykMUfN_Q z(I=kRu~!l&VqMuYVZA40X@jNr@Y8RjCiT2pzDi$M)uIhfG6;EZWwh1zF%NERv&)Ht z_f3Q_PG;*FnjI_k5q;&YOZMJN+@1$l>USUCAk+PmUTvXgqch-^utfycTqX1)#=twv zm)_42aGw3=@+U*=kNkxc;3_u~$;gWM(6A)RO6728{dzqPd0VmYRz&1}_UzV@D++|*JSn)a{sGy*7ruf@ek$-u)ZjA*V`Y^;2_hmG zdAQ{%(M8z%%yQ%_(@XLv_>U*qv9uc?jP?X&nm(n6y6zK+w`s#J#spcRw_QgQPoH4{ z;NUqJy+Dfus+-AC8DWjjgT&})@d^HAHH3142E`OR^N6qF_|a;w=P29O;M)^~g(R&C z0YT!f#viZb=zAD$YYACRagyD`a~mt@R}5qkph)FESC$H*xu=9Fyx0LGftG%I6fpJ~ z-_t0hTZKL4OU9(ww@*G2Xoa`R*IF0Q!$a3m^I)5s(l9XY)Ef_wIlofK-W@-9}me)ms-M5a=A9~Nld`o@lv6fbKaqxY_IrKUw8!%L{p0~kRumDf?W3dDY30s*n16|i`Ib- zcY@H7^9LyxZEcKa(XQw|W?t!= zHoaTXhCg?Qr$-M|=_lT%4-n?9nEM@Pf4;b?i;d&VQJXfrm$22ty($w`MDs^I+Ce|| z_;z`65)MSqIdt(F$W_1!a1?>xf3Efh?2QSm*KVL|%CmAxz@*gw&%(Vzl zcTex#lsPmk_~}sA%5dFN7M~_6lPtfvANyPCMJ7QatSyyJuciD5xw(a1+OVtLq@V&6 zziZ89&fx-=)qU3c{jFK%p9ynerKi8*ITnZ6Mzs@NeCr3p2^@8wlGlT7kD)g#>~!G> zB{5qbXm-ra*)5-^!p}F>#gjHjd%v=8nM+1YXvf;OEXsb+`y<1`n^H>?RB!X^=%`w( zlIt<@G)ipuO5n}!z#`H163>PhR*8-jDoB5g@Sf-$Nl5gGZ5Fb`Pu7(<-8;YnW)$lYU<)c`6uI)wz;%?*m@u@7N>9tm9LPWh^}}tT@GRHGsUQkYJJ=MY6HiQ;UsAW zi}*fM{dbwUmWCTVl;IrPr`|{+@lg)63Dukw z8q`_t1{Ct#opoB5{^X*OV}Cuxs&6oJ5DP1e3f$^>Rc*}f_ZF!%2bXE5-igSu89m?D zu^3Qf1Kdp$P{ecW?UY}Q@02i1v_n@6_UKQ0`V$h$5SZXXu$j|k#w1UHgiV&Wy;kfE z$93B)yIZqn-9CPXGTh%wah-MC$F4`0Umi-CzD~_Ey}eyyofC)1VQnq75oq5hLtv3_ zaxa&DJjGf_B=WsiFv!ka@)C97y6ta&QQN$57z`5_@U6G@ZDfsX5wqas^gdZ#^HfdR z*CbQ%iR73L0L(nypda()ALbh317V>{8~q}W&<`=5-rR%j zlw`L@J45v_JhQb;xNrkSi9X@?fm4%P1r{1*l$y71>yT0(;~9|*1g>@ui*rnkuKPQS zG=jYVrrK*@gRoYj2i3s5dLoy6DSuBC*ZKdn{_jo+M=knAsyrQ^YH&U?!E=EzuYI+= zb{}=or9(Fdf9pw6Ig5v*w3(k!o{j%PdV`KPPSU}V%5&bYrP*_&$YK#M@ZYllsHD~$ zlXx&j!L+fT=k4@EYAh~v+zvq3Uabz+Bc<-ECz!e2;Gv1{Muk17ZjV+@fCB_A8+TJQ zot87NO!EZuwB*-r#|>aa%G2#LdeXl@qQGOdVE-(>B>v2CQSkSyW1i5$jn`zYKIAaP zg0;VFE2wCcV#s@>AS1@4w$$;k@Z@}>x=gb=a5G&$Sz9=%nKr0SjONy?*1^>>?hzK5 zX|cMnF%vvK+b|xm%zw_ei?hmXjgrRiOVZgD=htb{I$;l>VY1PrkGp+TQVu$oCf%kNA;CZ5RR>=y~%%_e@E(S9aQl{YyE5*OkHR} zdmFTM#2ou!@pT+_!^s%okSBq-lZFe|^;aXeO-TnXi=mXOlCbw98%Db>fQO5sH~C3= zCzv#$qS$|9AH~k&YSc3hSznU7xb6wC_p%h@pc_I1D`E`PeC=|4db5MI%gUO#`Xyp6 z?u~iv+M_22Zc8IALMaWV4UyTJ$f5pY2D+7RIG!05yit>Juzlw_4W5~ZL!$5XSs~{% z@G;L9?`%dSFkdVqk`Q*`Bk!5o$@trK9Q^(MY zli-0-3Z_dP{C5<0_2XO#WKj2)(6YSugdwRd2v?=jHMZU=%W9HF^;%;~(>aR2%qySDa&mt7#mK5srG!K<8zfpx;% zq>lI(tRBOZb-**Bj;wLV&U%-gc`yFKVlQbiaC3QPTzn{0eNZ7gRx=sFu2Y}O$+CT% zsX}Fkpc+A~yGshYwz~#?YP!C(_2ERzfp$Umgj~YftxH^Zj;RfY2`w^}KwJK8(B{&f zz{~2$9^!ciaHY1m<_Y_J-vi8?#tj>h6dA|NFyQ$$kme3D^*nq50mA=(u)iECP$Xn| z+)(eVGbuy@LL*M*9_2J09 znTQy0G!qj!?X(hkQM zSQ_8HznXTnrcm_#b};sl^)Q^Xfpc{3#hwhDdf_e6rd<1j0I-@6&Fismmu*yAS!5JEmHf{FFe zt*niWcoF%((YpvaA{?8H9w;DvhIswPZz)9gD*o`Q>b8tHsQF-)u)l@fC9y!GfQP91 z2m(O%9>=K@2fJ@wRH=vQUy4>Sok>^Z9bb|fl@KLPO?#}CYF)6}5q;MZ(;|vn5|SS+ z*!!WWiB3?v^$VSklQiO8Z{L#p53rtFVv~`NbUqQVB8q=Y@}FoZf+j0WI*{CfHVn%_{4P{FE*)c_WbcFPdpH4V zxR>4Ao@~`Q0en~2=Q0=_;WotY>4H%Z4}+l?1H3C8FYW?tWZskMIQXviUL6vgD}p=) z$$ijo1SK80cJ-|z>-p%lTYOj1mll-XAWqPOUAKFLfQlH)8Ljcc1tlPO_CID}c6S z@~WLE*52TOgtx~!Xh!J?V%}~KquMN{r+qMAHK@pF;z;W!%29{+ec7IkwZZc4>XyoW zZ$)gDLO#K4YGIh2-=Pw_>@3(0)OFOpGSJy1$6}|j)8x5Mt+CxL!qurIX%)p5MxO93 z`SO>uwg9;8dvFzH{-_lr5~P1K?|Lfznr$woY;YAg5P&%cV@rvKfYxg}x_#yJ;`Gna zvgRe${|H%xDj0B;?{~(jSYB~G5F9EsuXnfA5EYk;FePT}TY>5_RiQ!ldE1Uj$KQIRdUU_Op(APoZXm3Ml%ZvDnZ9651gM2X^|mi^h^)>08ABzPQC48B}aJdr?@?t!WvAslT|b zHB$b$Q|}4d@!?IdXEc(ceLiZ&-Vh>O8-r zdMc)}Qk0w%a(5=L64f=~BdDLWnUMiO-4wpp7?Uppwo7`J+?1v}b*{>BKitLj3cbR;0r>c5q6T*__s5j1W)9f{5KNVAEt}QrcdWjLbwY1p ze2Qx|ajcSXm3iO_NQLZdn^(;^FmZuxVZlVENmYS6baeTdDfp;v06;^|@cL#2~?NS~;tPVQsd+|T- zKk7dAt~5at&E*sAPsRk!g6Z^cdECF{FC8`n>h8s6s5({IwxK8zdb9}KTTw`WH6Oc7 zGN+;hC!shq7oFjb_`8;G8|2BIk99zf1FWs-0D5A2?c1)DT=zg^3OWWmMi*_wMld2R zj)|%}H9epnc>`6cItFD#XZ_7WUDYdse(dbiJU zn_(h*ZS1pY{ByLb@f&rrS?(3nsYD&rSe~$}!A3*QHaW<&6mBgAuQzh0I5GwiW#aOW z{H^?}-ymRgXy{oT4bPtDpkw+z_Y>6Mwy0qFB61Iy0s}b2b8FX?=@W14dUC4DzS7qT zJ#y}~%sol_(sPeIJ#PMz6*4k+a;lnpZoA6IXA{?5bh;H^G9kswnlPT@QNfnhX!xn~ ztA$abr;Ex9?6vf5WVKLEKR`@M1b{KJ6()hFcR29;|Ly6b6w>pBBh0}OqhI7@TlDvh zOu|{2OZU9_9J+$^wP0L&Is=doBU+oQt9#tz@deB>5E5Ri!pM^o*nMB<+E(tavK7=Z ztRFv^x3@7(@9}u2^pAx-&dw8IABth7_Nt9*&J=u8nYt@}!aJWdJ~{Sa9}B|jlND32 zBRt1=?5IrK2i)vii1UFZ#lX}3Jx!znJ@3+H?@Wy_Z?nsBdUFrdA?(FP+oMfrchMJ_ zJnS|%+k~J#p3YjrcZIvvI+87eS7cB{=b50FhZcp0?|9qX-xHJ|IX2@YOh$ptd>Zfg zKOucrX`mk`+^e%)9j`9`hk=}}h1Jf_l?yI7QFgmZf@xq&tIc<`fAdoIH`n)dY$}QZ z(ACu6Wj{PAS#Yj(1_gXK3w2};wcZH}UsxF@tfaTd2sd4Cws^vqPTzwmV5?#X#ukVr z_k=@IuW=IR8IhE(g+&FgmZyoWvINsI_F>Upi4*{2>VoL~^e=l4kjoa>zF{sr*D9(- zu>gym>+5>)y1dmUd888fX9oe&?ZfPoc%2e-H2+oA)%^OW?cg>Ui#y6iFR;;RIbf$w z+i@3s4Q;C^?!pqq*)^M zb38B7W_~tsqXVMIg8MR1Icpp)a~5KbnrKBiEvSV(tp`*M~I3mU4tV~ zSY-m{40{j_<(u!3B>iDsATsz)K#V-(Vkd49{=+=XSs0M%QHql)NMWqvX=rY|eEaNZ{|3_LN zS^z(qsF&XBJNW})klIM(9?arnpM6l|nKk%Fxv7@o;>L5nwqKJC4i!U*kqjSFEF7-A z=_^(Gkax8jROo!m=1hXX(}T*1s|~@)sHrnW9tcZpjXamf1-IlEL9zmL>cjCZ+3c{B z&=awhPD|V+xtC;%78Pt=u((GXCNGe0~JbxP}|0C<{^DU37 z-4nTao1gwB|PQ(HJMH(d|?*(79>UdISjdYIJKxe|=K(caik6gWFFmlIG z?O~8KE)p@o<8LG9e=d>dK#rs6dtw+&d3z}&^2#zr*KqDjhy!xU;qY~f<_l%~F>3ircO?pO+=xkJ;{W?{k=Ka$o3xz#{E`sJ!}LOsz55^|7P&J$ z!a$@OgV5GsHNV;TJ7Ecsu?Zc9!};gRs@qzVEOZ{4G1SMt-D#cDn_mUrT7dNB|;iVd7np2ApQ?SN-RZ{N;*-6^0_)2 zS#2!x&*vd`7f64t^&TC2)`iX}0vU#w-C{N-$_Kg#pq|!#^FA@HT(Coki(qF%ABGIq zcBCM}|LnTg^rWHbI_Wrb+BQ>Qx4I&ihY*s2jAPX91_8310w#6;mEr_!3juzS-B-H# zu+S~3z#>_ZD@us51gOlvL!o|tPQ^&;&`ubJ88?Ed@|KpTsv1tbT++PK?zgNS@Mudj zPWq>IA9bZf;u&o9oh4ISWyz}VZZyj2CCxaHOV?*qsC@DFRipBRdpV?D{&#AX>NC6l zUuQ~QDGBN($DMER!*`%<;LP7!2^(vOm;fxh4k^=1+18f2ruq-T6u43*QwYIPqw`(w zoPAhXSiVvXqIrTW93orIOJH1+4ugWeg)Xywir&^|+;MvtIqwmcd`nCDOHbSILz_^> zPO)S90*a7bl4XIq+mfqUreucs_Za1SFK94+iIY>oAQaPZhx^xryecWk8-_jW|Bmi_y5)QImR7**R1()aD<7)IHl=S z#|AtD(T*`J*QM#{v9Ik(_b%A(fkEmA9HV(8iQA2SQGuTS8jz2RM4lW_7VO*+^6CGy z6j0Z8K0r~wlEbZZTt0!i7d{7(STRBE%Y!H@$|FQ9tm7aCXKM}&9-T#{1mScp8>Wp= zQ08iL0KEUyp8>MGv%T{e5+$LDQ;!Ddg~9{xHgLbYK6UoY>U&I=z=on~ksFEdaR8Ne zm@7^4Kq0rUNk>c?&+!?`zjfjtlQB&P*wTaj5m_)UzfRo2+xC3H6RRH+ZE7>F_5SL` zWk)Kb$I!*ob5-9nKfP&x9~*H)r86qh`&e6hQh`t8y$_8l2a|k z+xP2JMvo%gD)B&s{r*aQpgs8F2jv+B#z2bk(aq}L!nMako+nn1(+yphBZ@oZ|1s_b zIQ=_PocZjWDqIMK+fqk1{vG89vPNRho!2>A`i9n)KF8<$PH}pznN^ek^lHuRVFzNx zVM~2tDm0#GkmT_cW4k8$kvA^8K^%>WCnwv@xqQhIjN8@0$u-mUtLzZa&6*RJK&$se);yiMN#Vma7r$1m z_j=+>u#MtbQiJ)qn1A~>E-L<<=8J~gIbUwq0r&pWnW+`SHXU-8Z+5sw?E>3fF*X#| zb^rUSL4enn5I;ZA0INFEo}Bo_sx-Fp}gX~>YMbo!G}Lc z*i+)s0@butazv{I&|Uq->M!8u<3cwkk6`#|<C%R-fJHcbkf1;`lh4Gl5foMJhiYIq_LN!oK0 znS&tj)vT-<%J^C-(Cvq`(gi~@IbW=c(|`BnnY04^=Uv;irT!NRwp06&q5sffzgdDp z;dZ)3MvICcA09my{Wm3gknhCdmW)1_ZN*G)z8?JuVKZc26|3)x`4WO2YTY^0j%RDYXZ<;v2pyzS<|uha4kK9 zw7Qu^w2|TaPZ^y!K?-l&C^yhKOL1zy*?K;NkQx%NRz!YB7A;`OaKe zflmX9ldk7pY>z<3mC!zs60A*&iedTA37@pFeM=Y?Xkj`k3AmMoGvK`-poESU`}$g}x_xW5Z1W+5 zt}a7;U00vwtH`Eo$dJaz9NMZ!Zp#ILCMuuz0pSd6FMd7)98U!s9?jRYEsDM^IS3|I8Mh zPTj1$sYM59C+|V~S5LJPerW8*^KY}C5tsdite~#M3n=dzGoGEamCuQ0Q#7qVHjg){ z3%pTA$+)}OwVk@EZ`)73PbHzYw8?#Sf=Yv-m5WInSH2#lK`u`fKTrib=kkb;WP1tQ zsf!Kkxal64;;e$n@G+xGuu*uYD(|@JBXMZ|_1&>i$qiQ|eE-p_=*MxTShV}0C zv;1~{5F}~!jt89`mUl4GS>jRnY+?N9VB+1Rv1QR z#HuvamGAV=`1l;@OsYcTb*Ft6k78qQSu?EE1w*lg0_cY>r$EchOP;2qpUjtp%!8$V zG6ws()rVsJ?TLolc^>>>fvsNWRk~De*?DN%`vEoQ(-a|H9?_^svy88TOs$)8)r0Y3 z*k;=vx1pSzZtA2)CaNzXPu0<1jfAc^DfnMe20j1Bre`tw8?s3NyzdM*T9?=L7191q zq@2_Fa;?pmRtY8F3vAxYtM`)9g`mb*(h2?Fj;bJuOjdEYOV8ec{U7~L_C|JAU6yez zGdSL;s6)-QK01*!0_8#yrKQiI&73}>J~%!fyi!*5A8aj&vVK+|D^3G_|3u;MoPStt z4%$E2vTir+KQ01M&{QfUyUU3|Na8in+LrMQSfxUw`@8BWK^+%j|LuIqyAIb(T^<_Y z^DWiOzA>-l>l5L!(ib&dRYo8G1skwvsocGMcZ@MmZygz-z{P%)8++nlK^nm!~&h0FhY+H{F~no? z`GNuFQzB9^{v_8lPSQUon<*>?0SxmZ?qwKfOU>B*v8COn`mYsdWlqhp8_JI23Q zpm(Tl0{VPixO_rq*-BsQWr_|P$7sro31palt6f|?Z^;g;=#iYkw|4|}n{zjXR=+S7 zLTD2fXWJs}^A=`}LuE4w;R4VA)8+{^YkHBp@Bj?2TMMyMOUv_yvO68X>bB)8%U(~m zEbbN!C$O7+`;mZAvNW+=u|Ndtgy-Tl_4gCM#Zhm(gzmZvpF)IglZ6XDqk`ASax^fI z{=rKk42XSE!V;9;caCWX9df%XcPyAIEjg+qMFp=A0Zq*RDf#*Vtw|7ITM)3G8`@9# zniEu>?}_~UE^5!JSbR|{nlU8}lVGZRrl}$Mcix5<098GjPB2n`dM9XpX9~x|X#gHq zd`S+#+Or2M$H0Bk)%ic)Y-uaxKlOK(X+%H z5`DU3QK3dWbBeG%j;E6`{tRDd0qm}9&ea9_W8q6B+eRk+q1Si)67PlM=G12k#>D4H}z2lhQTTPD@A0dEFQ6pQ$3 zrY(@%@}n6u`p*`;LOYC=b#o;__fXybwVLXucGfUICslYrQRv82c{T$(Rc9)Ck@SjY zZQ^e;{W@#f(S^*pRqN z=I2LzC231;UIoYEx4`g{DqQy;z9{`u$CGriq0m}T!!uAK7k{mbS|{aoVwS^sVjZ>6u%p{a&OIqS(rV!y1wxW1Bb4AGwCvqsr5y%0#g^R%`}wg$m6||a^d9cG0nAokH{2PFcNlY^@KSrm7k# ztpnS18E)YYVU3n6gty+jV4qC^-wpozKWAqQ21vrs_P1V4>a`45M?O%5*kD`ka`aKS z{o2_n*dNSy-TRU0tu&N<1hu_CKK_FANlhF8Rx5HNXM&qHTr+D6cmPD+Kw%u1TAWMB zniy#B6+L<_CUT;8eZ6!4izAOfErq_yRWprH!|Ohk@u^Y%T0sAb-tV@YatzCtCs^O- z=XHCmIsf->36GT92%xMZjy}9JsY$&Ba}*wPa~CbvGOC`A-$@QX8C02Ihi${e zTBOY!%R|CWYdYKndQnVQx}kr*;&#=!akEN=fG*z9WDM%1EjA=Ig1DJUpU#_SD@30> z|7iIDjmVZ;*9toNI$7aFD>n<#jWAw>ABH@ zGa)Be%GP9?pF4&-K*~hz`z^W*drZYDwx_c5zPz;lr~Dv)i1ObFOAL$uX{tXk0Kc>r z_@}A0z4PhbpH(*rDx$~b;bh9Tt@3d%<@ zi`pJRai|BLR@lu~hr~wkGkTjdLpp0M!mf-!*&{23&zibnBv$flXM1Rb&IuCjAgEn3 zrPmO5=Zac=HimQKz`Y#Rsfp}@T3$?TF>l!6M_fgu5n2HyG-kSZ zCXGC=DLy^iXdfn2dF=tYT#xn%^11Ii9lvcf$j_Cg%=`SS51IW3;Nmvsx0Bq8sal02 z{i2paeiVD~CJX@cDEM6KgEqLEDw&t$oTEK%`Pa7|Q47OkSvf+Ikpsb1!oHA1aq^V; z8qbB!dv3)!Px0XTK_mH!C?KQrLM*tX6j_r$5x|9!nk>N0!bb7s>!>f+%W!3sBTE@V ztUhbX>>v5=VfLn4A>aOzGanlgrZL1F=%?j>OI?txhOXwYb5pSv5-M2L-r{C54Ub*Oif;JUZ2G{gu#sushe$fOfr0OEcJYhd7o zeaKk(qi&IZDVK2DNCF)%Kc~V}K;~k-U5P^8rIEiF0lm$Yc)bX;mh|$>f9CJ*HDRY) zRPV0xj)cEWC60sDDBp@4>$I9RU%RLJ950N((7iq9J()x&z9I*0*?xTOObo`2q+z6JISk-Y%PP`OmRI;~p?yam!QveLx(quZwoMMgp%9Rrk9&o>mF_G@44b_E3 zF{xw|$BSpM1%rLUJtJ19wu?t{f7Avs{C|ACby!^MlJ*@WK!UrwdvLel)&vL;JP_R7 zB|r%7?(Xhx!QI`0YvZop%HDHk&Yttm_fKDohKug?ta_xX?)sIT>&(g*yxVmNGeY2H zN_WAHN$Ig2gi9^Q{1O(Nffr!Uc-oipPY4#8MF-nc(qYnzslZ@L&uLjdaTOR6_^pf^b^b)v4Sf)H;cbzExWQ z%C1O{J#Aig31*<-k}d@?&Pidc+KbeId|=!e0rm>H-S@qvA)mbv!A+ zVrI^2O!`?=R2f&mUQKbb3w5q{Ru3czF&Xwf>awLhOR>SLSSJeSBh(rSz?VZrm6iFN z?;@OD>@FvJuewP|vug+0_{wPnS?KZ#M!uu<`#{wv2Nz>zc~Lh3RTnYUJ6l6lx1VkV z7*%@nPtrvRx1hY7zluW_;dy%32Iyp2V=^@@j zlZ!`*ut6p$NmJFv547?s%)pdZKruw-?>bPjcXj5* z)ThKl?u1*SF{nidziLlogC9c2<&_Ys7bw)2uc0VVjW{tC)d?3OoK(acWfF*fb{_&^ z!K$xOKs==#4@hlHxz^WofjdvQS6ltw7UU_@YuUY)C&ruQoT(FV07WN)(q{^;>f7&& z(!Wn$KEjWCl|6uX3t?F7(#l^+(llNDd%7U11t4q_U4Ro)Py#J9^bqOy$5T5Ku$JoV z<07a^s6uk< zzk^U`AcvU`IUY>0TQK&;>FMQWYKm2uslbh$eQvHZaBHkn+?JS;yaY^1uz$7nx5CNd z&pRCqC0q!W>TAa$%&y|KNwxhGoJQ7M@jmKCt@|t~{E|+dq=IL-44dl)6i47hDxUIn^0ZM} zAS=~t8$C;5IThD9E;FS_S%P&^;0{Vu}UU;*Sg|9K1rrm;I z)|F~@%6Py3U=pah_Prs~mphqisE?on+&wgnb~P zd)Qa**mIGmwU!dR+@7XLF+&^f*>~H945R+3a^$A+jOlm37?)5;J)!W2U{?60@rcj) zXYKjz>7p7YYzb4iJ<)a-A8B%`@{2+T!w>xr-nSBnj&)Shs`cb6j_P*~^v8Yo)aH{C z+$#?=tr_80{Y6b8;M%v%@-S@l!PPCxJU7Daqk5&Q6d@KLUU&E>-?+>0_Ye&Q|7dZF z2|ngj-Iid`CI4d|H6PUijjPuajQmm_Ol{tWDV%<*{s6vvfve--_0Wr5`^`|O@JkE! z3AY*e%tp9R5*s~FoWU0N*TGp=0Lp<6!ePRHmP?!XP~)R|Mxj8M$H0yPrIz9kU>6JN zT-?Pt>1crP>&8MqoABc}EpTSDp7@N`w=EOv0|yoNiR~88xP?4B zcXr6%)}bM-GQ1}f6UNrz$UO?)*)vuIy}SAN!V<7;pMuJR?Hl?`J@S=59w^r!`4sub zOXX+q$Dk$&rHC*H&~0=7wMkyHYvHUJ9gZkA==YgI9P_tv&NhIQ`9OvZG$DwwOhtYI zU^77{B)#vxdSUN;&Rz#Tsqv3F!`DI2SJ;Y(N8N$p|li7{o04K|T)l;_~O2M}t9mEuyE9Ub1| zw&Ec-WZNRAb?88m+xfV1vEt4w{1!2U;N~U#^+b}sY@@jzG=vWsbafM|VHGh&#`h0VtxMUeZ!wi?taGXz`tW98exHekY+!|L6ChggwjPHuK9&?#5+x|(5q}izKc45ozlH-w zAvYAQ1u|kAO+-ccSjQ}WtgLnOA(gckXN=mId*v4X4>>J5&c!J3A(( zkg@!3;gVBM`lb^=c##l6pdCI?>?Os(CZIgSeFf+@wD34<8{$Ngu=+ELzO943rg64T zR3cYo?rVK#x2=qIv_@AkJlV+S-B(Ux#S8K1ptEt^ZHbXIm5CEziz)yA8>gOT%lb`ZEWJ z$63qgN9=eV;zglsUgLNCUWdph1wOw;-dAIDLPL)LVW3lwO+XL2=i_XK$_-!$5Zfi( zm@haf*`5ess{bTK=M-t`phT;?4wFG^d6)DRwx7&Pi`U;dlA#yW3^Usb`th7-mV?d|aih0X&f zM`_i)J^dU*767xU5NDnc`&b3{me*9%j{HxAG!k29fR}W&3Qrv-iK9`Q_?;rV)hP`T zA5x(&Wc9>xXF1N>#h6*p1$eJL%W)4lw2XpQm5Q0m!f4%y-PZsB<>YoNAMxOVqZd6# z6vIjBj~2m^^{~f}k|x|vF#we;*7MWJEwl+Te#Gq$q)41{5|XbRz;1=U@v?lV9(Z$2 zGJ&}uf89>ymS8C=dmt`Q{J9-TJ=E#L)AxEC?fb4qi#I1vv-;7|{af7*EEiY}Mis=b zREQnfMm@io)|RQcGrnubOv6Inyy)2i{5iD%QOtAaBXND3j)Aph$=ja762y|8;w0Ea z@15xu-Vq|&B5v@BSYh|Y!D`2*YV1B@J8i+?9y&+ApQomqhl_>q#BQXFxA^o zRN+mGFt}_oZsvz*M^09{!>K%d)}7o1UZ7h|hpTe7>pZ(T26{Ad%LhF4n9rt%jKn9L zp~ywJqHSxo(y3>cshE>roaHjY4Fiqz-=;fDNV=nC*qkgZG4a!X-ul1 zhCn~p5b^>{KfoA{J;3Db?7vUa)&=qL?AgM@x-p=1an8oJg6O=)rp)5rGaXM{|VGt#lFE50QC03$V0w3?Q?$RejkbWN^ zZHTUj*I3^|G;`yI0vOizo!T z`95oyoJbH$?)iNGJ_=-smv`S6SGxaU&kPIoPhb7PMw<)E{CdwqXP=Y{ie#4tpIbHK z30SwBO5});{rW!nW3-6)rvVgqcVH>uf)PgA`6R-K{f)f)W{HuroNVktfca|(dADT~HO=PJg zZ(uMr-1FJ@k`kp}tODZBOu=wDGM!?SThvH@sAp~4!`)uA9Ay!_3?vhHgfwBu%jI_( z4BiO(nQ`(Tb;l{FT~jg_doap!6NW0}0_c12OBpAm(U`t)x{g6ZK*asgicnf+RgeH4 z0T7v}gKcKt>*hPp0>krehkCpc{l!ok3;#fJh}Y^WZa=r+{JZd54nk*S=_PUg55PsO z_WDFgoKcEXXP#*OETN$-j|Sp4e^XXYrNs#S!!S~a5IBO?-UF07yaFYeAY(i3*nFq$ zqcz-wA2nlm!p=-#Xnd$B@p>6MdgacCRAbA3gv3jd*j9%@0U6KE?9_FqgNyEhc+L3P zae>K1LqN*c$rNTA`u_O?NsB3wHJ7ToRAoln>20N=Ypg(Q|3YS}T#@P7uctN8@Ch#y z%rX+^%8*2=G-Z7FgqB(IZnW|fuV@KvI76;G9XImq!(n0Xayf2|d7jm$zS#`w3<@nEIQ0ayh2-{+7Ny<%> z6eD%~Gvz~=wqfvx zn#^YQ?0}g#)Vf*nakJ1gG~^bRxyBh`KPSWPt&lPQR1`j3sA&fBoNboU7ga2{kdP_w z2CO67+H7x>KofVR83vR2bnsLkI=IZyPwn3YiZt{uD+u^_fo+nfFr+IcWXyu%7d3f) zX~u?)4$4IBKyFF8xxOic+ikXwtIV(r(4g4~P(>A?R6I_BH)UO!0iPrs1{DNNt-*2N}(FZ)o(WIEFos z^vzC%LEDAJj=cHP0Vhti(WhN;Z?r-3__K*^t?wse&?X#46)L|0ZHqlWbx{e2zrQ|b zB3H442o~Oz?U==4ys9UEhbKG7UV+P$j?P`;dhrO&~JDXTPcxF zS6{?ZYh|}*7Pgnw(J+Ea#Z`ic6hc{&^-TVc&YKMYQr!6Q zeC`p?RPF2cu#LxB8q)gH54>M60g+G1FY1?m)8@ZG>;y*W`ZN%F9zJdD3iFG)MDH214W`)tsd+ZvV&IQ1i`mX8KdO&Z`G>DNX)(%#zLE@bips4n+m z!Qje`QMec@MEvlSBc+*3i1# zN$aiOKkhc5`rcbYtIWKEB*}z@hzOAXpz~xjSl#pN91pD{A}j!D4r9nNf;6Gz7S;Ey z!IYmx?mW~p9UkT{;*zrUSL8uRtU!0T&Pc7ia%(Z>X%D;nAq^YXB4{8~;Xv*dgygRf z>lQ>lj96uB>O+hpJVHuR8qn918Ss|;Bf;f(sw;I_s2ogT9ZQiwEN6vEO1*=gzlj;6 zP_n%HeyNo_+mUlZ5Qu}@Tni^G4;sLiC_R;1YF&)V2pTSmvEm|&-^DYb=~;u2eA>{i z6;sKdDU^w1Y@;wL@}%>2p7B_P*6ki!ao6tRZl4LPzO-iaR#NFC)n&2!i{gr*sCWCn z%2ScRV>Q=S9!Qt%u#cb8j33P=s$HsLbytg_0fAt3789UIq!R<6w!TcMwOh$(7IXSO zTB~pbc(tSn`!z%%&8O!2aUf5~Z8FJIQkyXGNJrJ)H^oxE71zv+S$bYwqJmc?IU#=| zYKO_2u_3gUwJ=L()|nTFf)lKrqpAxF>{awlTK7Ohg~b8wZTt_KQRFcqW=5ycbNP=7 zn4>4(<(Imj*mP0V0=lng0@M;<=gZ^Xd2Mz& z53FHlXSd`jxPVZ%4O)8Fim1)uL?xGslV8S7cGEehuF zRep#-*f`TC@^j{JkvQjv;rgCsR?6vjGD96BOXk7r#&;ROmH;63ATBQHU&kNyIML?+ zhd}ggnj`(S(LElMNC08bzv zUK$?05n&+HzXSyT6qEj~Dkvud%s1%}q@2toITq)N8H_Lf+ish14$fURiNxn?ByWpb zQBl+EkGYE!DPw+zG|}ac$0EYN5JEsJWQpE46IV(-lFqBOiDH6F>zkEB0F|A4xcS50 z(Hzm1IP*V5lK(Ah{_6tetaXLw13^nxZwm7*Z!Fi(?>#TrX^EuTt1Soi-e=fXJ+SGl zIZdscCvtC1ID#FCT|Y5+kT;{GHHrg$C~^+6&PgbhZw?}|1e%&9ds1%c7 z({C@wb?~q}LqW~IFG+rtS70ZA-x9Z}PY3P8{8PjJU-FFq5^1~#W=cZVF5qAj%%ZL- z8j`T(nFKqMHQN>kPSSwuTMOUQu#MKoPSWn-UjcW*h1+ioNpU$oX=mt9k7sbGSoq#c{qhEdbg;%2#o1{#;&YqAcI>MC#Es|?6(GoO1B znozKM-?L@hqQwo|x}WaJjZ!V{QtfFywV@a@-}e@>ZRDtlVRN_D?Zf@Y*l|!qjvCRX zjGELy2~GWr1yF(K1SRkHH(liW(Ywwlt*qdR3*OcPi{f8XG)7zylcQ2k*{h=(Yik@j z0bF5w*YWmircl}VlTn&)Z}zlCm-L)HpG6&}gZK(dO8RwB_7)VAKC)N0O*Bj<@9z>2 zJTH^{%>KuddjlDdGBBn4ypu7l`+ECoCJipc_?NzxMDk7e_In0WacMmH#&K!&u8BB0 z*67C0cGz3)M`cswF#8_*o<(ReNz=KwW$2qdi?F^;on-AZ^`${5*oa9eWC4+9D0c=+ zW9=PjTcv7p+)WF}J?`tq{rCSd9Djc&DwIeDC^e?Et$B5HGf_<7O(V4L!Vc`a*;iXJHC|}OM1Orey1kdNvmI=k-!u44@@RopitPs8D)wR!0&L!B ze68QEm}CMpg7m|Av?s&0#Nb8U`SshDRimgukVgk2?F(ouy8 zs&gJQQ;FpwN4s3_9@#RAR5-?L+>Wwz61bzWT}^!{Zy6N6#*GmwSL~bn=%&rC%WyW- zPK{EZIE+k8p#C+}e5Rl^G(H>@uoDs}hJ9c06aKHYtkVke<$FdoE(5k9mvb#FBuS+` zG1il6X;b&1>PRT^(TTNyAxpGf4k%Xi6 zp(58??r1&l!^F$0)(pZyrl6k*aB6`X(B;)*yqYtIY;N%XBci6!=2#0@cC= zcoF{RTl{0Pe6j~>z=`qzehIw+%<1Z zy%r@0V;_(P8DS5rA%kKQ#GJJ)Qx8@2e4dL6S`Xd|fx!xe^S0k2D(qjk&ZO|7Xn?ew zv}SgTPbS)`IQ2hDV*cG+JLMoVJfCEZRQCp&W1T64&L+?)nCT6uI84Rs3a4QpYfAaz z2nRVyKB8lx*o2TTLb4Z5eT2NE7j&Y~T=-6?qVRPpo-mQoRMY`hvYOIFk@o-GH)fLG zC}}24c6!(~k0=t@vHI}%J2rHJNP$|7l}tXaP!ai~Mb`?oxz^Z0n)KeU@58s#G=;;) zf1H^8M|s%4kK!EaCqc4BI`(t9P_iV;TBmXcT5*QLXe_tyYB5Tp zLJ78Fd!UJd%1DWUkV^Qz|8v~uk}YHg@sV)@oBJtsx2;)h6JNri_@UU%BRrWaWif2%-^z*<%57^U;utX!%u6SC z)x(sAE;cRNxZyovvu{(}`!^1^MH`~s$J8~$+spViy9 zv#(z6-b27thNBSr|Fy#XZ9bjq&@sAY1B9Lhbom~!3lMV?6;V)|(9;T5C+uA~`cy0S72<}FplQr#b z-r)SPhfn5^fc9Ba1H1j^n9F<>v-Y1Ag@69!D-;BGbcpbnQv4ii6q#hDES$?-#{25{ z@J~NqUGFpAM~}ME1YbuD7Y9Ra5i)0YBOFZ~+17WvlS?hEOOPE2@vYuO(;^Sj|5h75 zxuZ`uQGfA^rO52OB`T}V(VS`Q7HWu^HGH_5d1S9O79tp=M1Vj_LHk;X5BNt;kR|lP z^$kHVBAs-Q^z^bCvr_I>n0hmRFHoOCg;|qQKAqmeN>zXV_-B#qJb-q(8yG+6E-v>r zPf~HB-0E_2#7MJoiqC#84;OeL@I#-8gqmhC$}_Ncw&mTAXP&9p`q`pFUnj{Z4Da)e z6l=|3Cw;xijo|K_{Imf?&;LFL71H{g3f@{qWB=>f9#fn(ct2xbad}^_H%jv(SH7DKe@&;n_4hfO|5)qPN5hfaBH#Zo&h!Mri9iTX03#h#VEI*FDV zftC2tvRG?F8Up3t&sqPtE+mIcU`o%D#Rl1zhPqP+kkmP%`ss`UHVhpS-6EDC5PZd- ztq% zuZD0>hyKJ>=scpp61yWajT{)7NT<8<+j1fj}+| zJCKf@R#=+)Q`)jx(MCz3EaszDuy@-@1Rc&h1slQUek~T-PJ!kgssHmt{u>5jhj38U zgf%`)L(y`Qm+u#NX1Z3EkF%atBueauhVLl*Onu5v4OrSyHe4ezHV3N{vRAB_} zR(Vr6CYh@%r!TuYX1lf{;@Gt>y6Oar)z31^3!tVieS+Fn1-iXMeHI|u7H=A1!HNL= zAO!6cxpPiM>xK-c+lQi@QAzkzJ0}lb@hzY@0l}&Ep|D_BQX4D?sB5M!%zheSPdj*0 zi4R9zI8ifb+@<|0H#Oh&0u_mm=^<=X&H4BU`U1_p8ZT9SEF+%sZRuYE5P zGSpKTX`xC#9*Hs{)HM8EcM<^w&HwpSJGG&IOEG1Se|9qOSC5)@sx|lS{~%ER%C^)r zp7F}#|E%}&w4dL6Lfo9-<#OYitku?jka)bXTpy6md&OPnuMtAj(atL_)N}@)Y*=8& zf?9Bz5YAu5$glZ2KdWI;T@1Dj@Sn6vJ665%e zXAJAMq>PUp1qE%r(E;k_=;C#RkY?7pdvy};qED-$pdfZZ!n!*BVJIRD|J83WqQk|2 zQJSoP=}k`CwQcO$qkQNzpG7Ip?S0P%ih!FM`8|UCy2A(OG-})?}FowL-NS~)hiz9Wf1lK>3i#9kDD@6Jnt7g!;q1W=RZBz z&ze?eJh>Dg>FtV)r`4NZgs4p(;*P|}6Z)F(R#r3CMTxw#e8>1&>byc7ff8|2_L{=>IgbWVAHVb-zimA|&hk zyY%AU-1c~sPiI9nlmv4s$@xqCh;3!Bxw~TV;!V99eB+a?i}(sYnxc$Xz2pi}Vu;qU z?#}Dtx=r^`>)(zX~E7AH^mYb}D|C@;bh>}MLZL}+LtHqs*?;td8)zLn&z4Q7AWDulJlm-Yf z;_*d=Bze7eid4}G8nnqkVzaqeHwD#Tj-PE;uo>2)d{H~eO-=%NixnPVpb49#i|>zx z0_|v}i4$({Pi>dEx-ILg>8@9sFZN^wWVFGcQd7ktx2zL)_rk3?EhA*ljMm4LXNN2A zmZ0WSTJESAS%M%7Ze6nU#|)fYU}yiz&(R@pcZGuJ>1w?elp17D!IF`oHTc|y_XH+2 zTbqw&?y&7%C&(m!J#YbjAD(A=eG#l0u)-Q>uW`8V*=W60H0K4H+mA@B%lQcY`y|Lj zTG?DJcj5bYn4&S{dq75v8_zP)e@|+lwlq+R=~um~VyRzhyc-p(tPuC(H;q6>Gl7_^ zxp9KW=-o_9+m-Dim>)L1@cu4liYFQ7XrkF)18@K zLB@ysQ^5z&ng}DcAnOwehaFFpDJbXJ)5`lmV%x^PGALc}iUPcKDJyL(D%p-4?WIXp z!px{O8T`nzdb5WlD8U8h?XzT`5IDB+GTgpaQb}qTlD-j7P+Pv&5U&b`;>0<#y)t03 z`_e}W`VfRCP|dt2+ePTQ}XhemU_L~fg81v(bev`X8>)!;p4ag zBbgvIgyg*K_i_%1u;PoiuMXM*&)((`w5bPb2|AHTbvP=Vc(jd$oklA?u4&@eu5J#* z3~bSVYvO!59&tTxml%D=WLj}*5U_?%tctbKT9pR&D)(1*YVln8(*BpN2)IZ>)p71o z-Du$eEk&%$cc{N2*txkR@sn=FnD236Z&Jjtmtyhd49Vk%jG_*@bx>x{<0F_ywV?5o z@_oJKuZiHbI+zK*7F?3NBpx{|Fgb-%Kvb+0xNV~`t*r36Wi5!&8BRdmtKkl66Jf3+ zvYbzClg(h-j^MiDm?(#Js~blH^Pk&?6L__u@T|CMVuN3lPM)5m@i{5*co(9EA*)=> ze9E7H(*?1qixEsbwtGe9l=I9!jQjIhPo6@66=pwu=lZZZSn#Rt$T$8QwEuN5#bJmQ}y$W$LkpnI%`1; zd}^+vyQdlVsM0^|Xz$BBBeoyUE<>1i(*~4u`mW&DsLh2!RwlhmZmj0L>!x-%J$m#=dfF62Y6NX$EKZTn-3n@RJW+)hq1IYrhSFvyKtbc_5j)DZiQHv z?c}6b3uy2^*T-^besP*$;68L<3v6qp8{eHm=GXH3~xc$@>oe znz0~;M{L359P{-Kx|C!D!8*(}0(~7s|T38-<0{A^oXyhmg9S!VAn|k?yZX)HT8@Ebu1o??diAqc=6MrS01a zETtZ4K);=-*5;AR+!L-AfHfj!ooitwV_k5fN~q)cqz5)usv?3n&TkXJ%lo3+gJ~Y&9FsF^yM{znyF1<973+;AnMeNu zX}+wi^2U+AOJ$_hgSO7=0#GBu%zV1XqokZ2C%&Z7N)gpX7JcD%DqM~fH1l+4aqeh9 z8Ldn(+bed6)2`UhAo)>uyPi`fm{2q)rimp`w4x~M!_P3o(Wg@V;0=w%jNp1|)SKqk zOX=8P9vfzDdf>6qAA|Pl4$>+nH3nn3qqqX&sgl#GnxU8BW8P>WH5|5>9?s9t=lvQZ z{vGD8d`U4mMAkKia-U*1s7tZqINYU0h(^i(xLd1gwB>DoYKL*IvC%T`(t0iBzJ}`g zoF^T9oyavbb?~c)+4ko$RiIQo#t2>}MK7%?>yO-*V9Nz#t`6g|S>|vSjgX0X$h-nXkHnc&_@q;)ASX6zZ2>l-^p$ zI+e>tm{=LWp6?OU3W{VUVr#-@&PMK)2Ik^qhnBvvHMRnYg<)AWVtB;`@ThUb)=jF_S z$ylpplEgM1RfcdWIWZZVL3WJ=zj035-PHsU8_u=b5ErS#E&el5@UI{sutgrn3U_r^ zPvdF1qywC7gZeutBl-Y*c`Su3^H4OV#)}?OEgMK?>i4LhdmkF3X}EKGqe?=!a~#!! zib)(V}E}5Y!!kCF|p}Y-BI?G3U*Se?sywnjwOMgLjyWGpJwx-X^ZXLmG z_-Pxx;VE_Fy8FrMDjGTpjK>Np%SY@B=|t+fAQ41TcY;B>B|_;TkMt4!hS=b{I4=XM zQZ9z&2LRy8DRJAk$7yUE$=XbT1%Hb|>a-iY#XX~9b4Z01Y^*miJ{~J|Pq?OHL(xD$ zCnj~YXX!DYlf`w?uwAoG#Mh8}`akjNe-D%WeVs}3$0z=J?)t=GOM!hLv#8xCc}nCt z*^6d;*ZOu`6_6c#Cp^O>D2s+{iM?y{x9ej)u;_Y6Gj>=KLf+Rc-hi#|Uz;KhpV+AE zU>XnYD{^>1m|1`0d%05DoJT}lQiv74O&Heom2T?P;GCgUV^(4>J{@woam`NkSNw2b zRcX~1DrIg3vSLZ9A&1SrAWfdOuxqs%@FmuRs=Rnc;( z_Ed{znLPHjnvtbcqKmdz7EuwlCN!-(q&if2e!+8Rq|pW&R1TFhg9nTHM?$Pt841@e-FE z?|*b4)oo+p<>{ENoW!CNV;U_R-HTx8i@#9pwcJom4jS2wz~$maJg5wN#6|>d9MLOZ zO~ZGv6P4dPeIH>vUu$2z+A)5asAjqMR1dahA?>X0Yf!H$bzhl|vk#VPJ{(M%0I57ps}*VJO-{*=Evb@dA$b!#KpO4gymEIas_r|YJNWo0@E zXvg9bc805yv-~c56B>q?fy#a^f=HK zQ^s})aNISVVpg0-2!-x=QLgs8V2y;->)A*uA@6=!;t4i$`yTC-sexpz;^5?91?z zXzeS1&)kBvv5SY(W~fCR!WECqxLhq3Qv$PWVOiT2*8CYOo-c_cs-;BBw{d~l^=-r^ zDE~jMGdpCj0DHblg^_r%7=^6NZ#K3)_JQc;hpOic(r@q0)4cF2f;{GbgO~aZc%?g% z1j~rCqk1QwZqlF{upK2r{9L{JZQ4_9$>XFacl=A1nAFMdvjmWFkSmsnEDop-LejE> znf!xfCV%WzP)RGCx?g6YAl_g^Jhk7(|7?x9o4MycDAF^uTuHiLp@6d`TkqfD4Cekx z+~$7i)uu(4mRGXXRM%n{w*6>LudPZj+Baa*hE%df*{sH)*!z6P&#D#`B*a!gtvG^GMyiL#VCmurmh8uWJdd=M`%9 zRt1$CjQLIepSh*KZL71VwS&nQ_a@d!ipMgMS=-Ye$?2q^QmC};bvZ^iZ(d?OH>pnj zy~}fLi(@3w;vpRqSWL-$QAbn6F!TYk^)uANY+Qn=hoSE{ z^28)iIdXc^aeg-3c&sCe{-4{8=o(4olYJvrK%b`f5T4OZ<7uQeo0^x z%BB>>u&;U;S;4+3&Kb7(lq%dyJ~LvVFQFZ+bLR;<0Nl{YFW{?@7{;rUA{=c!SWSv# zQfW!gqe+{!Agaf;yF!#vY5;rMP2+iA!y`FT*8g#4;&)cWc%Sqf--*@jBvb>}C+v=d z7`63?DV=b*IKh=Z$J#mj&vq~J1T3C5){$OUQ*^jFiNmv+;Hpo;0+Q8q zH+#^_F`rKWi`5fKwJ6&2GLi{din6tpXZ`6rKV7f4!* z&d>hmj;pCpJKW&cZqcQ7?c3YPjzxtUYml;C()g)mrVll2-K`@vuAo@CxXJkpe_h5t zh3_7HCcG(ocJ@7c^q#(9f9Bs zT)WQdM(D&@=`p1DbgC?3l@>AbXp`Lcz@Is91rGP**aUTaC8*xZz>vBoJl;_0%AV|4 zW=UAHF=J$d>iyDex_M?%BU@^}jL1ecCS?bc^d#y)-H9{sYV^(-GuTAup^Y0j2y-21D&ITu)%{N6? zoZj*B*vLAA*mR*+`v-7$Mh=V<)h##oBh_S&F%$Lv!i$#JDyq}a;4lrg2B+1&~X>n&eyg}2m^k(ycr=IZF|Ajmn+;4n}6{p}ARmW4E$w9>EG zb+JS`4#a(_j)2@ zgicW*um8ar*>y2#MU3#p>hS6`8>EZu-t_xc>Yw5?NImP8#}ms^;lPQ_8)aa&d^5DVo4Mv?(tS7opyrp=4!loYiXNni@g>CY;y(Dn~GNa0D2;LfpiRDKD>{vpl91Yj|qM{N2OsviVi- zIL1`J)v@1|I9VihzVem2(>7+v*ZBQyWKh4U!LPGeb>bi5r(wbJ?@P73!U!*G>NGAha$yJq)1X7n*sSC7sngQWy*&Ts~Ym8ur+6K9`Q`j{IK zp+-i{#(_+O(vdi85rX#pqwI{WIrg;>OCxQa!%xEgkPN1&yUU!|@G@tzBgfE}Y0Tho z$=aE@KF4U~5?T%38sTq3czK@VM6Ht$>AD>oM{HGznV0LlqEH#j^3|#}xmBFOx$8Q2 zCgJ(IX{;ypUYb?X5OYu>@BryR&sbTRJtjH#e!CXS)q(@JDPxW-6`h_vzA>9)zUS62 z&4*}d=d{4e+xkUk{;)msWeN35xQnS1i|{4A&X`lp+JU)v#de8X7@^BWa_ZW1anFpEOxO${sxJ9G=v~M`3ekj4R~G@b4BQZ8a(tXY+&X!pu73V6vytRWn%ze-73r0uvD5?ZR}krraCupgSKOH)VjSA# zmwLP#U(C#MTz>q|U!LVEmN`1Ns9%B}7TWJQh)IH~E9yGi&C4K+8vtvl2ikf#|FH&8 z+uV9#_0=!tc5m|GHw1&dSIiyuSnmy& z?$$BH&>R_%R?~(yWqwXqt2mEJW7L-FYMQ?bn3>KV`g8Ws)L)KMW6To~Y005#M~rH{ z!tb^+e`kkp3Cz&#!?_skVAVlg^z)|18#=ev=rk=!eZlfJI$A4r+)2IRG3vLq&JNSh z$O0h7QOkHoR7_0;;W-W>S*?6qCz6W%gfeTJ0LH*??i5~%?i`OArjdk001vV5I^OvU zC8&fH;{KAQA6bcNL7iSP8H8#36ebUQFFrF9nI-5%hwM<`0yexp!R-L_LaD)vj?Xgo z?1Rvfjk?BrwiITb+vgyN=D~L7p*MTy%@>twi4brLpf~er;DJdJ;7T}tJ4tiXf?d{Y ztbTi|G$W2L+<8j<-Z5JNM=U=t4aMP<$A$uXvMgZB&Crq~NauT@Z06*APsuh9P#Pdn z9eo0tQ;lIZ4{U}O3(%R%GXo?v^52dgkM>|E(ZIQ>1M`HE_A#XJK6c-P1A>wlU6p$? z%6H>co9}zlNQ!76)NAg{o*=`j?K#CYjk|28KJ;#QGCjZ34$jCD=<;JH;1z>Vn2^>M z)0II%fxS4+&M4PG=3VMiEq&ZXX=NwMMm>4_T~2{`O)*$8v3?ehCHFv^uf}$p=d#&V zcko7n2#A&ysI$Gzf)K5h3|VzaLpq&!jqr3;iw#lQrcj3D}n}dndLa zM3NL=K7MbW9wy<;Kt8QhR+e24?gnHT-T@sc%A&v(WCU6ZU+ZfA4O>*;^yV*yw|;tN zrwgq}wYXt*9NkeCF#Q^HlWVzF%^w$Z85s+j>orr%@Da6DPTdzwYqSLYy;QEZ436F% zzyGohqhUkf<3c5&hQ4=zCrMhsgQspNZ5ZQ_lr~5n1kc_xIoRFRjdAf)g(%wmkqK!V z*NG%Tt{O`Hd@x~?1jD#mj1JiZX$P8lI{SC-hk1>ZXQ-H&AM^RX1OWP=_E&?`M-b-Y z%TmpHt?nd~DeBE3nz%lW3~HxiL2UDo)6E~vRDwXb-?g-Gc0|Fhrw236_6}Z2Vwp9m zph0#ON_lt5Px7W8zP!&Z+yd)yXp&W$Zy~jPh?HA+JUvsr9-f4zCeoj;X7;?-XPU2v zmNb=e3&JLDH?$cc!HkEzfSAi{F}bP%JbJhUbO|KCCnFe?@T6;41;j;myd~M)xH=Dp zKgc7Q`#_iRzt#oBaRNo3%%1oq)pWD{HnwUL$YWK8tdtF;%9kjk!uKS)YmX-a==l5@ zk-E*52|R~T3%jA-EsdfM(HS)?9^BayMe(x}e0$bSgS)723G8fN%~QZdJf8LB?F~!? zj;v{yjz5Teg*l+vVJ}h@QVj59xu<=!EIf)0vuQbDYHq+n1Lkh@%@(?O2tbBdq$i@?`Tsefd z(3~irW_?=qaE1)B7ZO@ORmyIWa5Ng>Ma-ak4l>?y5PWziPa*mFpqlnM)IfEAbpQaBEQm^WQ_}`I%;(#z zf+$1iy?FWXa=)~CwHq5CR!1T0nAWq+#gtJmR4?*z_oLrZn$<}Np>Z>_!Yy4yxgSTeviIN zK+nUz#w6-HZ=hN_9d=BpUPl8B`y>vu;{d!a&63}v|KtZ5uqpUBL-lJ6qN>iXJIyH^ zbV;i|le4Gj_P6%0M=?^xKv6MjYI&Ix4fXK*I7{R-1igmiOps*wK%4wp3m5`TAX9aK zPayPTCzc3o$8dP<}l-sGdVOrHFQr7U9xOgK5VJL zAMGpr>g+*SA~h}kr`XNiyUc3W=*ti9q?(gU!OSa^D+q|H26`#@XP5|BW^*q6Zq3<9 z!lg=v-aShAUNYFX9jX{#(b>Z7talhYh6{>@8 zc5@axKm8HR94f2{hd*24h^8KQ2kKu_ys)~#7R2k~Ecx8?9)!Hy;Oy63%a&AlEtnp?1yct!V_z_e`Cr|&- zzFNR$N-+ObJe<^#S~It(r`z#W;pZPH2#8;fC)LTPbOm|}7!Je?n(tNO`<@t7d==g_ zauovw=96EsQDzp9q&_J`TMI;!|F$$m(6kHvmOnZI=_<}UaV9^dFu9Kyu|E9z&h#EZ zvI=Q`gJ4c)O_(o_gX$fGTRjI>ar8BHWjD-zye@`9fXXn{*Z;cJA+h~H->`aCJ3#+R zN1xTHX&U)Aq+F4z%)rC{9_s(m|6KDq`rYqqO}CWa?e%8%-LF6x+9bnqQwVWJxtzqs zWPw1X{@5Y%=J*gubBx5AEez74A*8)%P(x_Z3+uUQiD2Dc7yU^D8C`P6DGwujw?q=B;)lF z0fPOh;a@j7{!pp+&NCAtnpXU~sIiUzugig})ws4<3>c!gN#gN3So6e!KhXVMrflb& zNxYYY-9MurI~x>~3kx^5M*_rhd+hm~dM+Y#vl5`IN~PQKMy_D*xM^5D3W?R1zO@CS zvKEho1VY8F2q!!3KYEAQ1+A+IDplP~)x?p~Td$4ml;z3t<`~8<@$}0_`x1)OTwi_& zeQ~%%5>sd$*pWh6U;kJG&2lPlu6JBXD|gcK*f|ro6QGM*{~DMxa>jA)Na>K8cL0o` zy~}%D>O#ceMsw9UCsq_%d=|3>Y+{;6k+r9;J)Yv#=|R)sd&fP@TfMD<1_Tu}M%aBbEaQ!u_|&Dy!_rPz5J7s~2#m>D{g60rATzhVpucevzJ zq(XH#OH@^(Y10BJ)X?*ov&4dCap@_;u8=c3v1rqn-YgKprQuL65K)ZNyueovhM;(U z;oQn=7{*1{O*8&0X<7PlvfD7sqUR4~_9g-r`M*Ri@?#O8yS8P4z`##FHqMwavKBhz z*9VwVQIRneve;U{B_d_xa7wUz&r{+4dSpr}o=1j2j2ka7nRa648s!UJFYOyX!81qfO>Ql?s{ z4J(@xlzde=8u^{N!#$6Igpru)v&0K8aYXSC%6yqYtOMnYl6S)C;PR|__jBwUXvvBw z(N!~C7`!n0*%nqU5FGfRn045PpR?)8#LpKovl%lz$}NC1D!Cst1WWpU!95{B!OV=KOK|qD71Ym^u!5avY#b30rTY1w-k0e3@Nsa2ashdS2 zxN(h#hymQxX3<>uNEw}Dz+cr?IRd;k5t(CGp9!y3)BcEK~M<7(Y+X*WS zbWl>{Z1ejzg7(>!5)$}cj$-t9Q)zCs$g~N)Q_8x6#^?b>%J+?nbWC^hFhymBkxEta zRh%(hN5QPiGwHzf~1T0v|Zovm*N9skea?n+}x^8wfKW|yQsq&Fp> za&m~~w8Z)MCm_*UV6MJjb@r$yJvJFUUpx}q7X`;uaZ|ICOi004Vq!HbrTIs+{H4kc zPAmIcY9sVoVDq-|kJmsCZtSjY=V9*Qc3thf@y}aI&rwf(&v#u8&!!pU-EJ1xd&FbE zA@!*qLd{LRawD0ff!GVLiA6);7)t=KD#`GY>CpM|o@bxklHi;EN=nCyb@#cj`>}c0 z@YMJ3E$RfyC)_9`Lc=A^Si|K+X$lIH$u|quW!~qD-NX>s1-J9*Iv`HNn`y z4`p1T+>N8_Fnm?CVgj=_#P6XCP(zH$=CD0D?izJGE3+23#?W@0dW z2XpTYs&cA-Dqk}FXE`akbZ-IJQ25s8tP58lMqT)H@#4O`<0IL;erTyB_;&r!QS4|P z3dXpqgD+wIe1c=m|MX8mJXDb;J9@gb(Zk)Ff4n@Qo37S_K091pI}%cCK!9envPx8P z@>BX7mKN&*`&r%GHKBZ5P1K17ZeM3JWy2Hip_>p!n@A7ni1s+@3K)|H1~8^k|Kduy zl)%B92U71S1I3i{hkIJ0P~9II7 z0|j(PouwDm@vlqK;6&<0hWJY~10QD<&)EaNckt=Aq0iK9FL~;(ve|1|7k$C3??RLF zo{8#`LCcBsl}B->6|)wjNDH5Cou_NGY9FW^iG418lHv;|H+#0Vx+)FOA*rScK(g0pCXBvT-74RnN8TmSWyHJ6daY%FY? z@D2zAq6)Q?;@bcuE%N>ZA$J%KA>tt|s9i{qkZHUyuLEG&FXHp7C~M=e$9DUfcaE+( zfP!)7aF7v8i#-As7X=sv`_hfG?5q~2bwlaxD|@t^Dtp_@8x^TOp(PpmBNA}ik!_=( zy5)o$22hp7@GOdxjBTMHyBJ|8r-+8I|MlhT9zApim83*WqfM8&D-t%-wifOfZ5)kL z5KN~7-#|Yl_1xJNP2#2OrqTZ(1gwtj%NJngiNWLi8qKzFXpxYfN8>uEOkK~TaLDjk z5m{OmCjU5qe}s@mze~{>$-BrfU-)4L8>fl=8j*X~)?%Jv8)uW3L6Q%6 zicTGGCPkXGb83XcljfREn@YaN-@J-2)nFGQ;Za+j(D{D{G$9#fDH=75B1SV+5=UhwyNX1+dITn5ztj zvyYrP$_3d`zezanUjD}aHNIeF;XNcrM|Av7yB^<|84kFxgdVC_C6J}l&Hm>%WHoa# zeNV1fbDhncMOL0K7Ld-8glxSq8s1jh#H|IRMIs}c7@^JJ!8H{x0zoo-lj0*Q)Jiq^ z-&lY3$?k+%uRp(p4f0ElY0?EHNpJ9|HNJiuqDl9stxG4yJO|^RMG@ypCsVHSx)%gQ zmTi^>-7J{bvgTr7_QhL{*@_fnGKDOZRM9c{<|jjJpE)jc`aH!gUp^x`P_3+9yby7a zIN-*1#H;)SP_&KEhTkHe>vYgzh!00vd`e0CT7x^#OH`Nr;?O||qh4g#dxqyI_WTlU zNq4X*{&`%u7hRFO^JdQJ!+Fp@@66*2EDt}=6elD^%PjiA`F8;3@kq?r^S2{9ZM zi8j-Y0UG6rHmV;BI4bb+UhVdEMv4V@;O(lwI%=RgHrtBDkR^J)%wlYU zb(u6L`H>l?nL|=SMC%Zc4MB`);!X2CWDme8v~UY9bEA}gCyE`=AP}3wwzM0msUHT| zoiHi8H97!W@~$7H`zgYu)V(_OnLURWa}x!%rX!E3?e^o1*wMo-p1(unS*dw38*WoC z0r2fjxj#a&Qj^JoG6Yy5Z%JF#t0rDa&((h6jqLs*AE5GUG$xtyknN-K;0=?=+p7+; z4}Z_lVqf51D2}(~2D*kxg3~l%9Ey6tBDAzsR zF_4PyI*u~A;_$H{s!@r2rCOjLn^kZ))OD(!PTRZW>1cHkOWu5Uxd(r+x+@2Nuz~P`FF{0 z1At^;X@Uxp2VKe1J8S}U5s?)8R!mL>S(rW`*-GlzPy z>V#g_bRC^CXn3!QesRBU;hD9n{h--#qJ^C~tS+|o8VJ@gy{n>N!|Uormwh00fv`(R znXt%<2qRi#V(^*&8~J_s8JWuU7dP7cNV3a-T-Vlyl-T5CE}WgsGQ!4Re@XS`A_&!usq z4HY!v$MubsvtR^__16f3IuVZIl=xw45SsD!7Y4HyQlFBuzc?elOc*DMA3~&9TK@WI zhyCsQ&AS+Jvai?+qNcw`i*PB>~9aXjgH{PLrJ* zd2l2)ff9(>?**7tO4N0`NG7q)Z+?owpInSYL3FN$>|o*6G_hLv9!)BCB4S*Dog<70 zp{&t?OH#WtwI;UZoBn^X0H9|+nXWij2h?_IP<%a)+0*)STRji=DUCK}yA>B;H&tNK z^mUV61>N{~qFbgW09s%ku13oW^CG$Nn5Ovf?0Qmv3}YH89!j4k zF*4;G2cJ#x_Y!Elhm7EsFMQAr1v?p~eCc6p^xy^b4w)<EU21)Okz zo>Zw0yR{L;1F4YS?gaU;njJ)@lmggHrN!`yTO(n1%GMOp7YpclsS9Z&gO_?K8e!yN zBE2yI_U4FAxPd6*RE)(FfJ)wmWVlY4T^#MD>bZaib@_)+=iYo=p(t28lMTg1NAuUv z?8hRZth!R%`ErF^1zU50D{JHWk`ZyhCzbzdrK+yja{ROx`&_S(FTo)|!^Ri8#w}oG z0rdiwKmNX+z=%>M8p4+uk(ShYZF8%w0}1fH`g}UC7({IZ)Znxg5OB=~zq z3@)6ZDLcUhV9aNkO}-5os!GDMG7WF&-^iY7Idx#}URoitvHkVX4MW+biZFZ(ChdC)j>}ojelpOaji%ov&3|b4Z$c zZ||>2=P2W+>31dFxIBsXtBRy#<`hFJ=qTjBD$GyUJiQQBSD+yIIaL{I+;09NZ=I(X zhqhOe#2aAT{JkrWEM+E1zjc2mi6I0N^EvO;AHEklu0W+RN3A^I{~BKNRuU8X_M3u& zG9|IcO3Sh^#bg=~*iX|As8^sNrBwl?8w{*{sdS}{N*FqBz@;4|t^Q|B<2Z~5pMfz+ zIO4qj3R8rm!QH2^-!dP(C&+iosOq((FBPO1eNpvwl5_z zxnxJgS7k+&vemFj(CqBoUG^prdJ`)IoDsFDln&o9M@b^41ryeDYn?7XQH^mD)G2~gT&!_W~i1iv+hcFwlGbuOkHPO9jG;{;~%zX1=28fm@mW2 zG^mbxq&I&-S9k8FZica&IITVxFl{4YsOYp}#!jXa1KF@JtCY{BJz}ZomVw}UvMol$ zxx_;Aj}M%FjyHB6U7|vk$XMx;Lyj{GqK__L0k5jP8JH7Mlsa-|1oFpio)Vl$K!UGP{k7{ia+LPMH4j{{91^tK1v%~MU??^Rk>4}qBsGfLSr?^v)w zK^-|7a4XInnR6q6uJDuG7*eps%F7`VS=&hqOhKo#V>B{?Amr>-foOt z7GTsHptb5St8k=*jx4@D8X>CmARCpCD*L_6aNz^iW#^WGF`EQqdw~xtnDkk%po>XH zOS;pxpOaySS%l%0y}0rlJ6@$OYn% z+-a;z=#+Vs@j})aYxm9XZ8nf^)&mjT&LKf$!4Jv16WMvSHkv?!+RkIS`YWkjTOzyf zF|QQu$!1Acy^T2jO|r8LbxKb35oYYs`!4F,HyYVMzOzQ4!tzeV(IwR`|7uhS~d z9UOO5r$<=8^Tq`X_I9xw4cJWr1_sc+=U*q<|~*F=_M(~ z39a(n5<2SVZ`4;pQQ@yF(Ys;}*Y%Vc6UL9FZ>WzjNDJd{B#316e(x>~M~u7ayz8P9 z1uuBD^SH*nBrBM8P4|m>jCXZMLIm>tStHOQc+PS3Z8e>DC#gnkeg-LsTu9Cd4}QCA z4MY%7bB29gS5XM+5iC+w6RbmzR7bFOc!<++cQ(N#(0;!vMUP|e*sKtdf7!H?sd}rI zw5Rt~I*=S9kxYpmNu}^$oE1FdO2PVbVrpMWB=_oy1+xExR3cRSL?A<#SHX)=TY4;o zPscwLG5vlYM(1xpeA9+~GF@HO4!8v8R&H)BB+BP)gRk@BwoUw=>BnF9ob~Jzbexer zM?2Lp9gBMWMOG|^{Y6%g3f3L8H8s3$;|3ZyHx-@;)H-Z_Q(p|}sDk4C^^1J!LV&d6 zm|oYPG}^`KiL`F5PJp8^D3A!-Klx#9XQcGuC41|OJX*ML*#QJ?8@JH%KAld1GJTS2 zq`pBbUe|8}y!>8*UdbBMqXK24$BF5_m7D|wMJqjzTf3H~Vf+X`P9d@R@2@Xtt&GQ& zpt~<|!plxyNRW8?>dO%A$;QWK4Ff%gORW8|vL+`J<|u!+>C8rm&Q(%F@V*;7Y~{Sq zQj9NX5O$xoyL)%t6}fpyEnpPugmXgUH}9Wj5E&tE-gv$?|9B`U++9sP=}^YNR0omI z$@b0(xjGC5N=C3o`E2Ww3L8DHI%06m3Ya7atJN zr{17U+}?XPuBKL6QB`u<)-ZL7!>6QshD&4 zP>xcbWcGp(v7!l)5(>T>7?doj_K(YwJ3vz>sojuIfjWD~(>F{?sxZu<^1+x*~M zI`PLIAsy>u4TF2`adBljEKI4x&#?u;ic=%$u|+$+oI)qX;v?Gr@eL_Et*`b;(w|LHu|I2*d0I0j%c3VNI7uA-~P^(|eJnS8p>6b(eo?Ts^Mrc2-{g z7V+Ec(ptubm4$Zb49*>=H`i&l-3MLCVt2b~rbk;#S-Y(~9vM8pDt#uJ11N*X) zK)-#;4IgiTsAwS*JmCBN%23xm@X~DlxN!&c6FI!~&#gKJd2f_jl$6R;B5@3Sj|kEL zCER&)bM;rtq{}I)Qr8&FR5%c@+u1GJ#}|3D zNRURA$+(|S>>5q1I(R73_Um#xt5l|2V_hDphCNVweW7zuIJGI2#MaQ!rt^IVy2(ln zkGo#6h)RKelfE=n+4!)%amA1Ik;`r*Yv59mBd-3K0;wegzMl^mJ{Cz<5{pxPtWn-ydj9ld}1(4<+) zqK|)t2QrD1cwuY)f4(^_=Wkz0KW~|Cj!ZS$vV1SSf{7VwhcDo; zO_Lz)X=yYomB9VU{6Q=Et}>9^x{;@_G%IV6bj<2C;yP~?aXtrX1Kt(glla)v{4u|- z%FF=zRzT$3N~iNJNyo)TEPb-9iqgr8XJ4NXB4WON4q&={s5aV@@oQDRqRH!&>cwuo z6D&gv$`7^1(;u>^J$A+({&?e8GtLfHlev@d^vbAye%%!DR;zNdpx4;9r}F?gj0Id) z-g{NQF5N`hx{qd!ExsgQt+D@_Dy%_M$A4fP`uA~06KR6m$PU%*&~UwDVO8CqE}8W_ z%4~kMKMme!H!s9`{)CZQmwgqjWJi@=^SkVgZD4}`i-1u&{>lE^TFTjCm*sB7|6B^F z5cLxU`mLO4 z#J+7p0`&>~(@%^Z?FXQ%AaAgHKz-)Dr|y(hXzMs)46pr;W@2@4b4HSGwE{88Q+-qb zud5jx*!L|M__WRnGuoWxui++q**`1OQq2wI(jQr^vKNveZ-4Dth=fmMY~y#GH|0!Q z$$1+(Ez5|Do4gk@QY3B9x?E%;W@2zBfnz*m9lCN$o#O{d*TCt2AAGZT>D^pV8-3M&mzQ_&T=-j zc4;UeuPi6brML97FwQcjdmd4P2XLNIIR9E^wT8TV6aVi~M=P-w1 z1e1zDg`Z=0e@xu`+!C1m)T_C7#N#i~b-N|R|CoWX_pqSqdNv2%1Dt-~?@%$zdM2@G#z@xfMU_U=Un2HzI0irtarstQK(O_!W%BVGo|_ zdx*%>*|#n=(rY#qshFWS-89v1YJxB^OWp-)r9jV{WcdFD<*~7|HTyytj@*clVO3(s zgVuXlw??{c~rPYZlAnR|sG*iJd`|6MaV*ey*x|T;mB7LIb{`n2}vJZt7@=t0C zSb|Q!(5)O!r&zST1Iz#kalB1PSIreR?3m)qGaK==^34=P@qaw3buJwbw(G{Qq)%tju>W0g#VE+k#hz@=o6Q#la9Uv4_JQ z$yw;$evs0y^{SwZm_iP#yUQV)i`5@U88OJ}l~$B)fVR_{ zW+Qm9r=LNG@KFjC!4+wMFZSr?{m@BkZO3x_J~LfznziN0%a1}=K7D@wehGtT z@`XSkPekVVfYdhK=OI3%8>2t}P<+Tc?UQ8u!WcK}&p%o-bhMHa-*1OZ{9|ge{WUe4 zNn%V@>f{WU<`SxqfCx!d^E>sfNHd&E!)jP#;W3>+wZQj_ACC%)9kI*g51SJjd)L{v z<4PeMDi1e$nIJ!l%DsXH!)`H*)m2ft*gt1765{MAo&UlNRhW9OVTXz>KBk)D&1@GD zH&0c!H(!30ZoT%(9CUV!EjZ9&O&I&rh%vX<$I+gd6#WG@E;*0xz(7=g@-(sjKIw?; z@9aayx&kWgs*4&uT-~P=l^)3y6hZmdYhKB;bdvjvdo)Lvi=!gDmzrV?<;AhnAQyfM z5FVLY=MADRB1=UeUMXhY5Chjq=?^Ctd~E1N-or&?#q8wKeMtM`U@}_mD>#5n@SE(V6>})$akI7Z znOX_~<4;f#Q~0?Ce>OK=Q zi$#N(#>F|9@6#@Z@S5!NS?@>@vPn3}qd~UyHmYE7sJydZy4bfRK8RFaM$S&t=&=5B zDALWlDi*8`s>#_l{)hG-k>axV`;5y2YY&FuRIW|P(Q}NYXB~L_6?LNm{Jw5xtsCcC zM&8T_O?JYU-Zc*Mj8<#gW&Ze)m(ZY&bpPbh1hvcz<*DHL_l=NaQ_2+%rLD{%Bq(4=TY9BpA_ghh;NMM%%!@Y${*7h$ zqg^5Z3~9wrF!Pz|nm+Py-roAppHVqyLDiiT3zK;{11|`TNE&PQ&=jYINqpzY1}f`D zrr%dOlsjKC@6@aXca$oDHfez_QUMM$iqQbBs{r8Ma-6lPWR$8C-g|m_clVImDLL7Q z9Ie-pD`68It6vBpN;gfYyleAL)AhDC#T2F z`|7EaNZC>KyGL?;-OW%;D?8^T-nGzc(xUGjNDULoO~K9p(dYoPL>;G`H{OD>{vxPe zWR9fgll+vfh|8f|x>!>dX{S^h#^ipQi`TdRq!f8@M+RM=hl)t`9f7$ls>DT}SESay zGd)jl$FC!uD{lRpP(-#=do#)O=UXxgb>VHZb4%JEY+_rK7l`^iLCY86(N)SS*%J#2ac@5f6CIxdwtPi#{{GjhNs@mCiv2hx6V z^pg{7kHcq>FcPO1FvTrG`{zN{O6z@!LSL_dyC3k;4IYLrU!zIzlSA0!OzG&jn=uMEFzwz_g1~&^~iF=Lgpr_NQ`uK(g4_1u? zh-4V>|Hb!7(?3eNoIQYG7u&G?-W*!d+p2L@JJzK(z8&h!ojYk${K~_ZKn}_Oa3Y=} z<0-Oz$n4CTAGz%BbG0h@`p+5py*pAv*m`7sFiK2a(xBa-v!ZF*(DlK+atsNJI19M| zX2BYS&Z85wJ8+G3VhNCdSfi--`gx$z$D!g;*G^NckIu4FTR&|wx-s+N$Fif_eSCu?IvhLUD zC>VS{H#wwYfBj4Nb?_UGM$;U^FTgoxU|uViJ}--K7y7t)H+XG8vxd7wueyKzoR!&h zcO^icn=nf$G@)=*IYVun65$^GRDZB7^fF;2#GWMFJ zP^oyrQo>YdlS)6d;fRan{<^rcxYDK~r)$K9+yD7z?B3eiT?JlN#R*TR94XH=3DjLd zqG*3`*FVL>bX1YBj~Kf8eQoiYE!gKHe~;N3q5qmzu-Q`Mlo=PiEN=Ek4b`!x@nmV^ z@T7QYB7IYZ?%44G85P~gp>*(W)t*%UhvnQ#y7%B!Q>3${oQhai?MSmdB-W~_dIQ^8 z%E;}9{0GsipWB%nI&)_N4LsKY?#wBL{&%+{kT9cghNn}xw|95=I(+=Co{NW}Ckz+s zulzjA1mX*o*o*Mjbf{!Hm=*VRV?%6Rsh5cRX*S&;GYtGFO?I7T^S7VpA67Lrs1CVy z!?1K1{PE?UVLc$TO#3(;eUIMb#B(R+$A)p`yR-6UC?`@)L3K*F%?74okux+zUIk04 zS~`jhK(F>^xG^pLVHhg9`tJ>}p8pfN=8nff)KV%^j&YWfd6dH%ylV^hb$=yPz za?Q{z(r`58w}cOFqJ^b#&L%+76_KEnd)`)_#1eDDzEgMrHy}pbf52rzUM0ZsK3#XP z6JCmMF!@m>Qsxz~_NuaNZdYuNr8Bj4^#u~58BXuvbs$RhDt=`A&K$$#8egu)c3Jn! z7~QCYx^5quU6=z!ZlLGa4mP@j?kxC8ui!902Xq5GpQgFd=bPx~y3*!SqCzoYAs6>7 zZ0(bI0HWk2Raek5C!u0+VnsY99D#k46}~a99WLzWa!7jDyI$N-7c}t`q?;uaUSSOA zW=;`W%#Ghr4KKfyRd(UIGy8_vX_mK+?f&DYLSg!|C&q*lLh~?hyVCno4QLi7IJLga z7AO3W)Wo8~uZvR7cgTC4vM(qYOdvy8DFx?Hu_?HgvD`*7M>3hPB9XEjcIKP7+_y)l zNsW6oDCkuEO&X_`H?#jklQQ*)cB+T)dK8<4y#tHBoMntDy{5HFHYhX=$ZG#A(Dv0& zF#CN<=U;pm;r>|+00_mK86rh6xX3bhWsJIxa)thuc8_NG@P9e2HKE-<7xgAXG&X{{ zW{F+~tMzWg;ab!U{exBtcJ(4=B;u(6Fl3*4JJi++3^h$GIv*;1 z>Gp4`@^2^ZLROV-ndl+!#wjUc=8fwh8lDV<=-ajmBg1!tcT?es9Hl;A6>IrvZ(3ML zLxwiE`>uvOt(KCxVk5&VzE?<{an=RhoX6rA$hD?Yxx2=Pwz%^qTweVFfnU}fEy7+1 zm1Mp+Y;@LmkMhzS0Jhbf`VDHGl)VEW+&yVF?Ky;5?|m_-un%Z0rqZdj50Zy=ESdWE zOWs_wVNY+LEZWlcK{jPe5ZECB?=cn2H_jI^TcS`WKue z&>QT!I_ma~M&9p8}$$KrP#UER7x>LQflx zW$nNt7{!D)Y+EsUgQ-FC3qsJGiiTl*xp@b!`yD#xVRb^n@JikGc3-zXYnm*K#FX}DGc9dpm$@TiT-BRh1fZ<<+FEr%(psHP^k zEvTUKth1*nr1mHG76|y}oUVB0P>(N%?_W;#jFxXtK&P#UVM#}deZFyIEA6#Wr!GN8 zpK_hLaF>88;M_9vti}j)i4?f=!~Wy7!ktUK==9&x8e&R&h49G#O*Z;O9SIK!X8G^- z5i=j)as~|mEwg((F*(?)N$YZCms#*nMB$tL^Ov1p^3_!gQ?a_5Kiiw6MKQhb5L2AU z%&(lI0fn&~^fUY8>zC1<{TbUAnU7A%lX{i9ad?iK-9Es3Ae9>MkqfI0}> zfy=V;9wW4Lu5HVk7lO!d+ z9&O?c0cUtOtT@DW$xX+nMN3#;@rfQha-Lf&4)wwhr1vmR|JiUhIt%!~-#SlUSCq?g zxA^4@P#gCm_Cr^U5$fN;2 zp1z`Nr5#hudiMBo>&Yqq7za%XJ-ueL`k>>AwuDB9cRyZJ`j+8UNt{`?Uf#ek+s<6n z;@1FgCB|Hy|6n;s2gJ%st?{EwezJ|v_n+Q6zp+$;*<)Fpsg@YCymzg_N#rtyK6PVbBFV5YuBvXTe|AZJ)&4!DNPChrffsK>8&9{*qgSb# z^*~==FigfjfFa9(z2J*r3*Au6mgVFJQUZod2PQ+?YX4M&$a(tYr%BIj(j}}c1Z^eJ zuR^y>Nl$+>1kDZz?oD*ZF=$YAwWM*ngCX~Ob^c%k7meaHiCwdzQ* zDDfg=fBKe^Dl-4}NC3$mnmKt(%ESGmr@hO>bMyFN_sx0g`O-Doidg0Mln0B=kb+5; z-8k+PE*o83eWF2_CIxeysZkPV1m#)4-eenbq;Lg3(z)GWuMVHnS$3YKv&UvGf^oer`WSN)@n3iOJN^kJ3`5>8 z%LKoEv8#sa)++sTKwYM5UbbmTDq{d*Lfr#Dm2C)iFF@=$FD-Ns#>wl+-uPg4y*ic4 z7rvdTy1Q9mQJWGroOY{MX7z8LQ$d5@#qrqvr859)x>ml-RQmC#M*~oLEi8zY-VkF(Co`aX!8@?Lb{}xYDl2AArJDV+K!jddk||Zn)U}dXXh)p${G!*Ws}k|1Ukeo4s1Mbp zQuz}NMbYTC%1^2%pH|>rhYBv(|kLeHK=@Uxs4EH7#b0Uud*E^gwo|ycX zsjDABcgD<_l7-`>YR8Cu^b{U5q3&QFoEF!k{PTruO<_U&kZK`)Dz-h$&b}KT?iNR` z*EZjA8h+ou=@6R)%L`L_EJ8y~YFxUR+s_|XBi;V2FR(=^&`{uf=jP2VpM{e;)n_08 zA$gxnMR+b-1vi|+K15=vs3BwL9hy<-R?XK-ag)hD_eYo(zTG|}fRhY(Yl!gQKcX>_ zd4J_;v^Azgc;iNduFt}{zTmqHBMGa~HowwYZ@%)Ou}I5NF!(Z!ZPC08-ke+dvvl9d zvQ^9unm_HY;5=1(DNJ;afIEd@p`ZHn@%0g&f4yUz%KgHTlAMFX1Fb||%2xeuQ_Nf) znltRoi+x0+)vHmSJf@l~HN*0@y3wL$0S@yFl7Q-i^I+beE28C`cSK2?kBvCmr~RvNVno)U8b3MG4_2QGo6 zYafRgzQv4*N326vF+MS6cPPiTm+7^V1_>=Ke@f9W!7LJ!<&oO_EfKj*=IWmnX(^^J zmk*zQK>B6-T$S#jHQ_rYpNU->M_@1h*nnJc+eE(%CR=1N+!?#$L#`6?k{AfFYGa-x zXMyOx{aY&B9T`>!YP$uZQ7@>vk=b7U*N=dnx0pD?M2#O_rAZXJ4ORN+q!!483!hRSwKaMo zJsmHL^fcZQ$B7o6nCE>P66ri+Z+Ab3#_6o-AZb%S_ucx3v@vdKB^C&2L1z{@A8yVj z3~?$WB}Qfu>ztnk->ik!C+@xrIj-ly?sx9wb&>h(?a*1gv&K~+J^rRzEvcZZkdS*c z=z{FANaEK?XWdI4DPw|XCMp4A{>7E&GhLg#6lR3d$~-zPYX6g2T%+%ji->nZLOJ7L zd}bT*5gM>*;6pxCGcakSHEi+E&fq(Ow0VZq)+t7(v5AN+X12Qy-E9*&%3v1TltR7! zI@{e(g$*|U>^m!MwtGGKjuJ#KX1gsSk>r%km|vJVxUH+q;teVZQmNlS?NTQ~)t&XU zwq?5Uo_cRj5;tzxLoRZ!3r_ccwncg7j8|`l2&!<~#l({Ij(O*-h4r$o>5h7LEiNOf zu=_sGs`n(MeG0QJqV5FiLkNQmzEkSTZ8#OvEcU}FS>ZkG_4_uhPk+2FmIf1jZuyKG zf!%@nc``3G5DDWJVM1n$yAvO`xxpw!kmBw3IXlJjkLRJ$?(^7Jxo=ih6B%t6_wL#Z z%5X=pr)`BJL5O?&UK+rsB6)ZR=-DMYLr6P#l^^~8T=IKTgbF;2ZPQ=!fC)ao9$wzw zTBS`AU^X!OpZ)vq-3QOYQx|5m}cFD_7XH`*}qokeL5z5=%#`k~!;ZWW4ZVuiXnzwi=AYY#M zOc2f)e#hFr7#3XUtA6_`iLtTZNy@!gzeWb8Zw-0sXd(9@r3=*3=Xf@K2s)_0K2`3G zFNS_dgx2UW2t+v{H_~>=RE#wK=s%gYf2g-DA6>AHb4by&`Pt8jRrV1yOM1GPT(#Yj z4|;4SXTd#MmfE}g8uLj!CDJu6FH}!?x((URSXXhC-;+I?&s zj@xlE?MP)*h-SeG-}=@R1HRQs`GA2jA^B6nsL}tKi@0b}F_SkynuX)Dt}3elt3|hj z6d0|{bfV#{b!+Gp9Z#+J!O3j zZqW2enXBt99g+>f6E2L*33f4EqZ!T8W7(=x#eIq#$z6HyPSm8u=(@}jnQa>`gPw>v zWbNImZk&r=GT@d6meCiH2N(Ey(4@uYn6me@lnSvu8=Ge?>^a^a6bBYhrG9j&&@KNiN?DrzU|h- z#q%xZbbM;y{w;!F7+zmPt2p(dDHzOz=-`X>rCRzET(}+HbT|l+tYZxc9J(xOUr{Iy zLbTi*DNbz)K}{%9J^QgMXY+&^8>qu^PLvjYf^paAVtSHntkuYHX*mZQHgsPLuTA zKKh>b{r-U6-+gB0%v>{b%^cLYP6)%$D}n5Dk-AdM(Tfmhj>hcuw3uP5=FxQ&B^{f0 z1!3-iU`7HljJs%=lyelQfqr9w=@_yzCG z|NY%|=$5X(^K2{uR5n4s`I|V`bQb6|gH+C~RZeg*d;~etQ3FE#QNL0t1gZ#)aWNz) z^i>K@GD>uHzh?`NC$8(JYUxi7*63C-N$9&A?%TYQ_v|YQUnG}nO*=h>`Q3NZFMeW{ zB8Ta?`a%9SX+EwK5~0zA|HvK}nks}+N^efH@mS4FqRUSihZm4u=JdSU{f_1RwwviO zfAi^o$NNUfywTP2AIZA8eF&WGYuL=n?R#fhpUa3>kd_ye#v=~AE3!e zFy%5`TKY}NOkCC#G#={E!a`#QP9^YXV@sp^iduq``Wc&DsQS$xS4)#I1^SoE^ZgJH z9_usqD_YOZWlSahgtpg1)8{#VHQ7r;t`L=yWuWTk(gvl)~rpx)WTS(X}^6~0h0 zZ|+GR>FF=AaeMq?n~b*JGvaz!#;j4;O8 z5{siA@Er|0czJ%>doikZ)S4p}Ma$92_For zPoq<6mtviyo#2HNThVV!h!%sseDvLdRgxjk_H*gb-; zj*Cm&aCYlU3E-R%T*~~;WV^`xPYcnkwKg^nhN-d8EF0NQ4hMUZqvtLJunXbT02=vz% zTlS{`CCJOiU>=Cp1{%FB(4fy6V)S$>V%e7eHz4>A=|+fDG++(oH>*_zA#1<~;2vbJ zmeqYlloJhGZC6&mbSST2PCeLuoyr|MVlm?UdY+IS%W)Db(lp9LJ_Rc8-bh{#v`$)f zu4C8!N&=MJmp;ARb<*GmEGErGZh3m=V7E7Of1Y3M_vc12SzcP(nP~d|0Lg#ClXCKz zJzMzZ$pY;c-CBMG^yzS*dCAgH%WAul`Xx|6HE^yhgI`Za2hDYaT-(%*Z}q*IXd)`^ zH^E`1qlE`o;QgqBc+*&gcVSd|&~P@}i%O-s;8g^<+UC=$_NMonMUGWJ$G>~VvHbtD ztWf_?wlk+!>Q&y%n#3^tn?56anfxI(n&sOKs7)Vd^^S|^`hykLHnRScLr8PRJ(!Z0 z3LP@}_==M4234R*kTs_aW0lY%eNE9XlQ%WNtElM$U8V!4RAihk8j6|VoRqYQ5C|{) znNWjD=_NRe>i^q!k-t|{^<^t!xNr#N@c--;%fTQr;HNKyev{j<-Ki);M)57 zX(=C+Ilq=hE6j1cfF1By48j}f`n%Z;!CxM-mN`CLuFcA>EzA(_Q1BpO@1?1KNKx$f zYMr9-KC4j-D|nUEt30j(4pKjVznmF=hd$c-4E~^)e;HKUBZ%tZ%DQhr(-sVAoYI@`+%h3v1BhcD4htr4aD2u2ZUIu9R z$5{W;Ho%1HQfqQuzJE_**siD;7YS1jKU~z!N@A!vI@5b-Oab2nce^Afyx3v?n1>&a z=q8+q^#xJ{TVytz)dvn}Uic%i(Q`n}_)5-vU6TI`y+TmtVL@%$CRbvCKoytHXd~06 zJ4E_Gq%#}MbX!AQM1^xpBaKOCkA>c=szb429nV(Kd(ia<^^6g? z>Cnl7B!|#!!XjjX&5A(R8~s$>4ZCWG3hJ z*-*ln-0HyVzrp)|L3#h5=#FbyGsh(C%8?ZfW$#zSr8hwB>bnED7(BhG@g;k;djU;y z`b^Q_&ql+hwlJDq^KV8TT))8{$;1OEzIG^mIGgWzWPggtn_ZScAjAExNZCEUhFJ2J zRpip^JX&Li$Dt#tyR!Wy31{?$D!zu}f6|Bl`~sS9hCL0?M7U}y8F4s#nerW{PnsI5 z+w((6%~?^)FZs~jZ(ivlql&@KOx3}+y%aYuGH&?DnkQuA&TARVw>+()(O2Qv1tYk?kXfn;wxto9)}e)Ced$nswZUn?&+YaLJpqE`Tbk)3 zdqFt%LwmKv9&_us9Do8&`^l&7KX@Wi)C=n*(Ct+Tn7lOv2({ke_e>7Cc$hs6%sdt| zq0g44_Z9WL$CModn=Zl?{V27plpRk_g!D^Js20kOvrIc$hnlzA{vB<*;i9bUVh z2l<$TPM?v;!9o!0sN;WV8a8sCf~|v@`XS#&$vX2Hy7_$TPWox-WZu7NF*$;P1_i#& z*wJgFjRck+P9_XWv&$uD3Y0;J*I85FK|c+dIf?2PbO8cehM-?EZTxUHCN03?GiR|t z1{Q0;HkbU6sQ*7PAp@w+gW8RohW5BMwSV;NWniT&Uj*)HPGO2dEq9!To0rKuvgwF5 zzKKU1zGh%^F3{xF(P%v&V_>?MgT9RGsn>6^W&@-?O1;vhC@y&mCpjN@x49N*eqo@k zIrgg!A_Obb2}TPvnct)hn!x_mPbJ4L*d@t5R-_a>eTv5> zf3#fkdv{wK<9*BT-}->**{{%#uc-Z}J6`o5C*vRJEBgY0L}fXm^6gL8Q^kf%YC=8$ zDQ9|yA5tvs2Nfv27aNVMj#3WT%o{gm+dJK^qW)lUT>OtXmcq|fNn-`TFJEae#I}2n z;6J_YJZ@?o*?Tr82{u3ON%qZJ>Bbt(5krC<2q`i+a*#gOTkC%kF!Useg~c2Av_qk& z#+Vp5c&HopSj^kV^X1o4Nv~Kt;{mS|HA3x%dLYSh6*gqS=(hGqK;$l}w})%Mw)mBu z*7N2U;S+C_S3UTGoDp1GJb$gOmg@%4&{-i= zB}_N;De%}gapm*S-O~;hGD0=}YU`v6*gopW%4e(x$(p<{e_}~lo^wnwDi9ho#G-=+^4bh-qjlOf;rN(wW<5dd% zyp8%Y1LcnNrJPKA@N=?soeyv*8a@^`H5_Tx`fS^M|+vWDhBMs;r75Sv5>GliH>xMkBfzjzls|dauMD% zU@M7PNho2Zv;_f5!iJcW4|PcVz({7d$Q2A^iZHmOJOr=q53#^@aNDeXiK;ekd;PQT zP=M&S)c39WFi4?U3yJ!!ljAa>fQZ$)<#TmfUI92j&R4$)qQremKrOEt-t(e7`| z*}py^qE z2Qj9k;|CBSv|Q@*o(RF^hJGCZW3{sA9>~9JL^+G+aH$&*EH!-Y@ZD>Hzsuf)pfMcV z6&`KZ7vRC~#Yx6;dc9KKtX7>7uX}>Rzz2I}tO4Dq@NshDH zZH-8PFV%lgeT%FD-VYp^NJ=f1M-y_R!>auTXN3CjU)OWfTtw#}wm2CegRplBKD%KA|X4!rKjuK^E5UAMSa8*)7$HcE*XJ3WV^@&6o%9<1<{3T5sBWA4%Mk|W15BxmX|KECO;X3sz!4?)IW@5mX4{6GpeIY%zCm43(Q|G z&+nEzJ>VYbSX{MAIy#v{MG9*0@;4p>q54+g&n%OOf2SjImbUB2JWCYObeRxril1W6 zJrp2}Rrzb|!GuF+Hec64A6c?0yYEzjHtfD}8sx>bAd6dPQO!?o=G+;*>OM#U7uCxK z#$x_xWj~xhT?QJtCT6g|cs`o*%8-8zz5O;^=LfCFb|Q*w);^h)ZF_wZP}6z;C$rsh zty}9#ew<&9b7KsI7qtQlYBs2A8-QLnw6?@_c=xJubxUOLx}l;?PV2W2PqEgM(uekK|ZzfVpXzk7)oOqrWY~=FC7|Z;L(GuoAeHBI-5T;x-LHK6@6R9r+e6n{6k1vTbTu+Xi)3^5o9C-)~Gs`D}4B@ciy(9+L3r}h-HB5<&Lwdml5IGVgBh4c(o@-^j`Vufb;HG$BsKR25 zYf+686CBrV9438YVg|IoN#wh2@E(K+;P?5+iL-ysk z_Y|T+F>a6Tje*J;b28;jJT{}kJ^gBGYWiP2WqB7WB+?T+Ht2#(y1@AMP&~AzBEWw> zfY20v6*-PzCBh8sBQX&7v$WzX$yqF~-|f^Wyxr#Eof@U9-6&ALwkNxtPr4zV$v}_m zasOrnuDX}LgA)prrGpYK*SS#=TPQ|f*fu!G`%K4z>TyBW$WY1IOM7}lT0R_rv z{nN*wDq}P_D-v7f7_X>z$7Qi2{y3`nAo_T$`73u@2q)dUuSiUD)1nHm!3?e%+$;27 z`-HI3KQVjZ)8g^{xT~Yk((~q#jUBLc{Al~`=+x#E?}va8rC4ALTlcfR!e)><@Q_)! zkP-M&r+kJwZeUL}I>2;RVEA%iNd69yFu%7l8)_y7v{L{L{@& zS`Bu}viC?5F#E7caHE_FFIK>e4{T&wf)C8m9Vi>EKR(4)c=8k#zOH9t)>dV9TggJn zNm4wrl0h?mN(9mWhBK^zA$B|}Crw#iiFWzgf=KB6V=^My8)3A2%2|VmBc**0KRGk- zu!hyS(dmfDJ}+IXatj?k1cUzWiio+e`!@vZIP#eXMpkRsERo;E!C>i_J&LDW9lI7} zP9mm9PlIYeLU~De&To7aw_9B`#9^!3#^&r?k7UDAN&P*<+t4z0uv!@UYZf*bYlThYESV(|}6cz{G=Ql&=!T~&f@AdJ7>5IR+9`Rk9L4VwFX>s0q zN#N)qz8_XrN&`OcyqJ|3+GQGQEJ?)EL`@kyK^qOJeFn$hqWxiKaEYj>*z|?rQ`y50 zLfx+YkTQ>)J6Z8c>2gFh$+u8FEj&>=pUe%Pzi@hbSmtnfp+mUtC$koY*IkrwmeFjP zJC21sltad6v^3J`-aSARf$3iw2Xg)jJu%56p!fo!cVf2R^2})bc@nr)$s>D22M@1f zwm%+BDYt6d`#~>NVcx0R+IPpM9#v6cj5W!-8da?x8|p(IB*5g|;nrjcLog&A*~e5h zLzp>2e{59y9BO@JE>RzGqBz;Uv&Y0z$V+52N{a;uUgb0%RwDtQ3gl$cni#9qMa1a$ z5;yAiein{*HPv$WWk+BTqlFmK9S^5K#&-u7{#+laFM_o_XysHMThNEXWI2h|OFdjm60t>Ebhy9coQ!$mpg=<^l( zAc_Dh_L5mVHMTa^jbrP?pDi(FtR@O)15ZOQm~adEBJ#W@?|A_PVtEKK=qM7g@krk8 zI~%R*iXk*pkQTf;;FzS}k1<`^H7$U8!8F_;@`9(Vl&|&|x-iOvy5Srfi!e?0C&_f{ zUBrl$Ez;>;VF)KSvZTriACMjCjD}*nCNaQ65`h3)}SZ-9C*hfa-YftCp?JsRWG%;aIEW#>OVzVcIBlpan4+^f_$}onw1d z6`HAp^Upv%Yj)@JG1&u^S^AuqT2r3oK_}x@hM4#)Y(}8-3{=$Z_+5_R0H}m!{D3b8 zE0>h)l-Su~Qr47?BKe($@zc&xTfL$t570b7UN6!VLo|02Z5Zl1#Gs=2n4~`|xorn` z3`S;j6uV!1!o9Cr3OBg7r4RvrJsSn}GW?|46c6t!bbz+io&2qUQlfps3Vape<&Od` zK2V(dXi~JlKB%+49)_pd57otGe)ZE{xBnqmH~I;6&5Or4ykX=7wAzZvkYddbZbt>-;JYGsitP_ep_ zE|k+K!A-mb=Sz7F2~Qiu*A~hNBpy>CrdlPz1G`N&@s@XeeSK^g72QUjMAryH=f<0h zI5a%xO~nqua6#%VfWRr!b|O~6&10r;*z}Z-%4Oxqs$a^ot>!(rbh30~paS#FOqaEE zGE~lLQ}*wXQ)+an`fzs3sc6#Xh5ntvo-v0Jyj3VL2}r(8uaElP z3RFF*1MMlWd@~k2J{)jF-ADBCALESqnT3@T!HijaP5Ije?0^uYaM+i5I1!5!6}HrW z-SxWw0Htm8{Y2ua(Gk{|B5?DNQ@4vo9xDlsz$I0T->C0I8VzT04jKpngr4(<1^LiGGft(3u%mu1g zk52YBN2_C9Mi}H-K})<0C&b921B0#!$9LEk{OI+a1|5Cr+hSwbnF&1@qd%)mx#)ON z`^80U#;xN_46^!Wt>dvY;|9LANiA4%D~>J+@Tk`j{=`)pX;it^gy_-VNJC3}DPDZ#C@#yZIzUkCzDH|qFeWHJzP83o4l5eyS ziYFXym60eQy_hZ@h(l>Cbm&I;qLB~~kjuQpSp`o6X01|@xbSPFg_$`oFQP@otLl=X zb{)}i@s5LA!;h#MW_`j){ezx};d2evUpPn=?W173ELfq4nG{>tjzX0R`MJ>vWV(*c zOJE*v#C*TfP3nTXF-zWAyC{DtU~4Qr&1KYtcGxNd95`f)%gyB!5*E_GRM?U|c*ms{Kf* z<|yg$rR2<(GD`ScVz+NtTtr1&e4%hkI6Q;H^x?uqT^dRsxYlMRHD`8-BcZqS(LM;jNr zjjeg4$;0nC{>yy4SeaBTi%*4wA+0 zf^fj$t30lnPp?kQ#wzgODpoCV)%7261&(_^&`FK>BZ{PS%}OMBprVt;|;x8<`|RET!lI)HXfnnf)y|#YRbU zb~LEQoQ1ix;GCcTClC{;=OtZdi8Wge+k-12D>U11K9xQ_zrd&Upg;~;IJ4sPe~R*6`2 zCnH{=$vC4FXzm+7rPHoEc6ZGPh5bNIHC3I2Oajm%OY3)c95ZusIV@#mHOmF@L?Jxs zIlvwS0(wc~v#M50*d%44+0xa&FRfO+gm6BTP2l5dDO4)xJXW>ahZnG?%aFn#{Luj_ z*RkWzmO%0^jyacL-8_|70a~$?Ex^uj=0mV^CwK|~&k0jLgVo=v9uryFIbvo_CGnwk z(A)ozZu2Im45L!^BP!DB11PI9kiecElYD?FKOkku&W%*}P@3=L!67S`54G@4tbq$fr zT#+SbI&4jP#Ao|7UV+hSoWrfoAFH|&^0Zyo`@D$}KA6AbR&lO8N;?8-;XwK#YXd)e zH1K>PgvKIXWHy#KHa1ONFLKpCNe+H`4}R_0OkIM#E@fj0_ zWFJ5`#YoH`Dloy+YG|~GYE$YzA4E%o?^DA0E1|uNJ7n7761zh7@oB8crM9gNC0c>0 z*5xT6AyUy2|6n-+EbTliXei?K_I;71Cs)S3wr|oUn!DyH*V8OH7`kSU_b5ph05v&} ztqCuV4=|HKvifxWNcPyNW*Lq6gv-R)8HL%;7Z;V-LuniJR7oIKfLJ6|i+c13Ty8>8 z|HiI^sP*3ZEmM3h;=so7{<$Do#4Msi3(0JQmzVDtx62}G%e}!pUm0uo6AO>}j+sBs zC_6P>lYP+niE+{>K0;i(hBLG%R(0~v!zs;;D%}6J*{2Qyw;pVJ<0K@eOr1%g^eXOTz*?1K%<>0!>G=t_~l1=xPVIvKS@#2G`bP}J0R5y8iC zkd*^(DHI=M^FXx{Idr0ID|cyMES1pVPB@$>vG|HkFXX!?qZXR_fiAtXQH-PIL9!V| zZYOOyG>PbnKFA6)Mltq)Im%<7NA>~;s-)PCmzGvy$B=}-63}N?(;jmfjb;}WOVPk$ z9VJsxGbZ`BOD9kI#{tc57Gf}f`sY|PMime!ffX)*B#ni*l%(#^U7g!;2^=;u_kh>S& zPInZy5hu!e#n-Sx?pZ|dsc+rzvqp{^YN{dw|HO-7ov+HNLP1%NGXtW#nO}cY=GVH_ zb&#S)Zciq5Az2oZ7;7t%jb#M^sw5G* zc=O87!cY4U&~fB;>=J$@k@*(Y{arEeIk!Bm_LS!qDwXl-#oLKcHSl^1I*umiFAb#P zoS3+b7O$_(C6IKte=XJ8qa6;X3eL~Zi-Dq2gt%+4&8Jp_>~;{*dAecwZo4|^*y9I} z*@ps5Ef&+>v@M;Nm+LXezVFns9QMI;N;z`&Nr+hW9T{;}1YE5f9=dBCkd1IQHrIja zr-&`?yVyj0ui^J@iyuc$7AQ-JnCaXc@nAgwWX;`~5p&?O4t-LUpMQ1*khlEUXP-HW zlOpk}RvD2`753l$yzZ6Hx+XaUk- zbt(~80C3oqClrxU`dh-MO&MWAziiITV#D(=pJ>KwUPXr{CCf;D9s7enfj!V&h)@|~Jfj)g0w9m?OWjbMp9O@x zA2Xi^L1>*l&-Ny52LZChuo_j}&*t|*HpFYKXt=YO2RH9%qkxv`kjDO|ERTCIbLHmr z^l7&h&lmpetP+bQo162w({-m^&F7_~*?1~BnYx!CeQ#OlW(TXjt+YsR+&XO(tDz$! zD*%bRJIlk{<$?3}cfot?4$!<`DDv^w0HHsqLDP6ES(Rtt#I&RnAy{0rQS#$kX_~3p zPu_B{NU5>t9FxnWX=+gmB45{^yWAeeV0qn7AWJJ87IHk`O5=xYkv;Q&1%-3>F<0}# z9)jpS3w}~38RXVQd?DakZSH7ub!{6z9qZnHO5-0322|U<3K-+7F7n!`bWm%|Y*vLbv*9>& zt0Pn8#HoXJ2`V0yQhjIpK4bj{lS%ShURfDV*K^S}bA^@ml-=P17r(QrW3u~eB5?QA zqDvb<$_J(u^7?>?rFW%>24C&1Q<{Csh(2@R@o#8LiWtBo5ttBXch1D77*Y1~gqS?# z{JFoxU!DRNtt}k)wJ9+0OcPDj5LK)vXlGNr&Au5U-T-zyB6Ai6+%Ch;HCXyiRj+UV ztO@14pCFEI6NK3dhSNr&wf3S)j8J~^)a`S@tf9X0(4_b-ckttdJ$;42`{Ez&pXe zXflxq-xeA#X6&|N51}h2zk2_&;O?xLbt2WKZf0`qkxHaro3joO7FvA zEleBScd4Thpj%7ov%@x+VdeTAx&uj*qGFg8m1S_;7%Yzg!oB##@WZJUqNehVpQR1| zY~|W^@Wmeb(5AFhUeiW(MsmCICdl0-8s}ZM-r0-^?)JIUu+j`$9(kYT?*ZvMv_ zn|j~`NEa!D0nJ5h`r;iv7IFGYI(V5pDUOu6fE~ti@=#pYi+s&*hjw9MK^>4s)jTWo zsg^>kuhLr>ONKD2V2}Y!bR#37j@JN?rwF(>&RIKu$)5tFB%=uzD(kRTi7+-o4Vj9) z!oqLhe5nqjCA50@r-pn4IcvAthm_N0C+sal>U95S*O$S8f5b2UEd(|8lH~PA=QBZn z7pK#NG$8TV3X4N*(9(oqI5O3!=v)x~Rt* zp(BWJQTk*SPJ4Mm(e9D^1w0Qp$4?_z?= zT~MDl*r_9#s=$i?%)CR~$jNenqEk|nQ^7r!g*YIcQ%JFic@ zEQpe&OZS}Dyl3{TgHsas`=7_K%lXbW^Wr&&zvJgQpaB1@eFN$J=&L4Pz?Ym4b`<01 zsFIu)q_2KSs=aB4t&NRDz1MSNFXzs!uy}pRsRPukP|4)1-pBk|{4X&#Utb1P4RDi2 zlatd%?clyck4+k<`|4cb4d$D&!$Ya!?%|AQ^O?qITT%ZPRAasJ@R)>*PYWU0zYlO~ z2Wi#~9i|OxyAk4yB6(wLqa;^e<@x5DW~8Hua{{_R#=}$ZHwW`9NOh#be=krv9?#Io z2@Y@7*r9pJDU}8hXfH8^2^8{|ljb@jSCA)?wI>R&*DoTcyI}%EnB>22h8T2!pvhpV z5?e9IkPi$#-V6-QJ#}|38>Zh=ZlaRb8K@7n>^HM=FlwQov?ulfJwN{#D!`nb5$t^S z{%F}G8RWMtK2{Nohbm}4Q;%ZPRZ=H&yc~o5l{-x0PV@>7q+PS%a3`ppL2AtJSf{n+ zy6{&DHd=!i*bxpi5~vKFxV}n0oZ11?J3IjK0(Cl!eiWEP!qeKLoe!AIYs`0jRkuY~ z`p7>GA}4CZ-kkXlI)tnHpBJou?L}-+K2;igoHh4eqxEdo3EVXJ{gKkt&SJ@k#R40< z$ODCl%fNTPH?)W;9AJrtsa7s;)=T$hV}_IuuY>Px$TnQPdwK>@P;U2|5CUvdb9uGC zj9S;RsnJ(qVal5%cR4}ku%x78=d)wMDF0QA8EXgDKhShI8oLSBc{}q^ zr@@4hHlg#t*HD^8BePajV&g#uCAlkX2TMfQ`nP0=1kr>EM$O@D;kXb6aNNQ-&%rTH z>j>fS-~DNZQj|ciW;*M2&~|6T^8S^xxv?KK2laLX|Ce}VlWXv=sqw!#7?E%OM=%+L zmN99W*@5#VTR0BtuEl(|rp6M0&r?t|quNsiOuPgh43L|t=0g^K4@ z9w!v|pQo~W(GJf;v{$lVnNFEY!(6xtT&(%jRw~hj@SEa(hX&=Ggm~KadYzwhr|BCu zFPWP#(E-ANvWYFce`YABY4uzI9*jUO$Fw7~rUOMA2+fa1$&UKh5l=x|*qYCU0)*x~ z9mz65sI4;by6Ic0VUji)Maz^91@MCdpA*acM;9{*)B?R9n!BLa!yXZuzx#oh-^QOl z2&(nJPS+*6Kn^YA2hfWd`Up`^z;b?>*%0~W0^2&I5h(dZBjAXgPtCHL(UnDmIC-i7 zobuf~IJ`l+x;-G`j@GpJhOAZs}K)UtMtWao;A&UmYWl%@0hBexB`qW17(wvCp)!ci&~$#ympG z;Gn0@Xa{@eG3MeKU9A)St_b>Usm`z6tiY>PFX`~)wy8KdI0kPT$vh$$<2p;e)v+cx_>+J*?BoA^YEY~Z#k7@{~ij@%F!)SSWQ$$H6kF7 zUBTT}w{B(C5o%qHnYUkko@4}4xc{e^SStDRy5|KiCs*eXFTeIUFHPf(OZmGmgEKLXY1QqQTB1$B=;O&%}x@kepTkf!5-JgD(kgh;N2MOer zIAo%WtHdLXC@xLcmGy4 z@ylBxr5HUjbjzLP7rsnek`u${r7dicy2BZt{`}hD@1R-(IpdiL3(v#W(A5k zTc}ibt4z967vbN4QY=L4N#WpxlBGf|JB^%)wzQ6RR?uyGzeuDFMSSG;rWb1O*1J@& z6N|pao?th2=xBOVeFQzH25y^1DLeI%a)!Xug+P?cu|03(7+i+1)l`SiIg+h7a0gsH zK%Jbli=f{}e)xF}=H#a5=4MK^#_ly#7~p6wB!IOs{-4_D-b5ueNZ6L9KDza8ntSv8 zTPI~WT>UPV1`k?s0+-^*@1ZCE{6Qmv^~<9%aO=_^M1TpZa6H{B=@STO5p}RVNV%~u zqG3!HOodxZTDDtwA+9B(oZuB|cZg-~-TDRUS2*ZT?J|2J7NoaL>T)}Tvd{1M^hD~1 zy}v@z1LQnRK6zQ|U7n1E2!S9Fe>puTovY+H^dzt>SM}9RowEXWmPP4sn|H4WavVWD zCePyo0(-yK&6o#~#4C;2|6tX`4;pHAy2NuE_Q=8FGg17i)D(p^H&w_g z6_cO90ddl94m-(~>Y-1}CB*%OBh54I?moCA{2M_4j4>aw8L7vBKQV~Ipt(T9U(zWN z^Q!x-5y)h&6N=xH3Z+T91)vw12=qTD<^6Bt?t)n=Z?`7F=4no;rRAe$Y3@G3v)k95-mq>6(<-;41uw_D zSLzn0OPAf1SmabRAyAaDrn4v*sAx2TTd#pX-fu3?MUY<(0^fB0_UHPKXB+rBrvjeK zGeEj6;Ah?tG_eA@a5H~xg%!~*FLEJVXFm~pLMZy)<7#un^nhy^$Wk^s7HeM9zq4n; zGnr1)$JBff;-FkHc#dZq!aCX~rsVDYXF|4_zL0pz-ssmmDTk0Ie@%VWPEOq{))qjq z$fGS=29^Nwnj289$8XluE1{62<`x!^NV}f+Cw9%j>lZ$#bvqo9VshOvlLOa1-f*F7 z$FApjNr}4;RRn(T*7ZgDEiN1&B3g22R?yfpX2)CP}fA%g06=P^IaClbb!Rwi)n#>K}RNBAn_XllvX>#20^`Jda z?B>DX1L5q9aCc@G>-5R^k&`Ue3irFhbvk7Wsu&3!AO$aEYwnM)?B1-50s=jVbEZ10 zHz4b)y?2w)-LNpGJec&A``}^$^vzdRTvu1{vRgn-bg8t!6D@|1T~EeMbvGv7ER6t0 zd0p8m_Edk&;ZjAQ=mTbk^H31umbnYaT*W?cw zURJ8V`-1Jm{O>=IgeUAX8ZYTZ&4KH^=FNzA`bbQjDBstWFdXwaVNI*M;{KEwbiLNh zMAmuc{sr;4P}8|zDrvkHbsJKA=1KM@&CW*aT^(0tx1Ls9ThxdC1B^RgxH$LxD}ftV zM8A2^xEPA&&7%Dmhq5~N*5XCZZ$A!G%PMlhc>jCH_v<6Pn(3s=xr{Nsj%Mh!4CMA( zo-;vNB*-_h-j~)EshHgNb>Th~Pj97+uE)9Vpx=*I`GeZOe4Qs1f8m{w6~)d)K537- zD4RP$N%oK-(7yQkcrYRg|D`R6N?ezy+7814Z&f7`s}+-=7N;VjUoi!}-U)+e1$25j zT@$&n-#7jKvPjI2D)c;5L<)QKoJeUj-@lz+5kq)B%>^d8Gj~2K5no)Y>+fR6RQLb= z(@<1zxYAkm`L+Fgd`!oh<`qz=K#rY3MKD^))zkz;^rreQ)Of0L3jC?EK74vF>c6#V zbtmY7M>cIc2Xq06xE_3jsUt!rRWu)1?G(6}lpe>`7`n7y+z9)1>d$lKTl{^xpTby5 zI@+Vr*U@MA5Ka41LCpeeGwx_E3R;r#>lNXuYL2d z>>VqqFEaV{+vz<@cs#c)Whi7Q*Zul6`z0c2#J(9?J2S$8i0w=;L*JXxE)`9SXMnZ1 zX6{>xa#5Vt8F23|rZU>c@>0qoLRmJ&FS=z#4p+iD0RXBrCvCBn>?uXKxN_?uXr?-4%d+gUVsoPZ}zHM^o3-cakHzt*nIJ!q|z6XD>KHSKwyO9xhq&n4?11#c`=_nYUG% zh_)&19Td$@{g${|gr5@MSTi?%D*S6Wj`(o5T?}GzNA=6zmQ1j1+vUk;UIPxUd)7w` zJ_*a}6fEKWF&wE3`> z!l2=egX=*w-~4ffEioM3y<78_o*I|;vH)-779k$@rxx7+3_!AZ_2Vn@Zb_3Mo;C8e z2~1@Kgh!e{?c}GDniPk){N+Eabm#*7Ucm(_AG}I@5AfjOcr|VL)t6RZpIeq>Ct0D? zD?+N~^P(ks;ms?OFIgQF2Ju;aPa_Sf#XAB0n|+}+FI=fPLEQEiL(ikGaetguXcDQ< zRz{e@^NQH%aCsQ;%3M`gQ}Ra?s8#>5tXMilsf!O@7yENNavzL0Y(cCdvZia_@~LN^ zX<`*484kT>IvtbsTp}RFc+pFjnQ%A+_x_QCk z;mcA^IG1qdzb2OyZdffMrwCWY0cvtsr+~$(RPb#2WSuf>>4cDbyY%#l=ZmYi;H^Zg z-kfb$1QH=~30)cERN8`(gIJ-iE_&vvyo6w|nuZRUaV6Z%27SQv3)9)~8II5XlLtu^ zjNMcw8f51IpMlpiKc%4=jN%2Tr#C1i?_-JRcheWJ`U1+zDyP4t5UG?D5c&baVlxwRA8#?L^3 ztCd!R+oRO_*@SZ`7^IzR*H&&pjz4`9-^O_<$N$M5o) zj(mxF3O@L6yR0n1NB)CJsbb0k^a;szZkg!>HMF57Y!F)Dd2iF4KeA)lfKKz5c_gcc zGsNKwB|IhbF4dakOj&*A>CV++mFMr2WVI2QP@3^siCFfocALxUXuvBi++%sQJ_(hL z;C{pw2cUp2#BIAZVw@8uIFC|7hm1IsuwS8H6=mTrLE~AO+0dlF71X9z$*NX^xlcM) zSsyvYhh!k%d0N{UiGx?vW7tNDyxDm&VvvD-l8Eq6m@c9e5NyG%t`7VS+x@OXn8n&2 zd=6nY1F}}DmH*n&O#FZNdIuoOnq_UcJ#D+EZQHhO+qOMzd)l^bOxwn^ZQDKno^#K= z=l%XK?iaBl_KLM*?_8BxnUz(Q^*mYdXtJfwO);p=dbRo-8Tq{8k@Iv3hc`+39`#Lw zR7HDlk3;(K(w0f#$avX{Pb>;m@@uh+M%9d?p9EU2jGCZ<@ew-7jT@T6M`ajhiQGu8 zsr*P#_c^jyiH5~paq#y}WBs)HyRyI`ZQQ@t1MHHd)H4pKqfj~UvpZp&qw;+u0S5nz z3J^)m(2F|UZZ3mF$}|-c%nR4BK6+nfGs+>12Nq2*h6OB5e6yoR|FV%nFMR)A zS3Fci#a(*f95MVm*t;uZN27cR0GX)o=j8~E43&f>Vi82AZiEIVA`hAp3SMwnRLI?o zwfD;?y^R`gKP&FQhS!|fVPN2eLMUwNA&7r7LG)VTa`^U?iz+|(svx6v z!lxn2ZgsrQ>JTidECs#aHWY$u+^F-5mmsrv_~ESuqnXDCAvIuu?^qn(8Y3bHK#*v} z6(XuyN&}5<8m-U2LyFGwmZ>c&Scff_umFmikP%@^^M>uia^T5bS~(eW@bZ=%g)#%)J{?w6JmO(4~NzBVv`t z!fKeyHdP=Vje%3Z8et?cv%@i}dl$J5ilT;_Jw3)Pm3x}B#&MeswQHrrMUf(k4Q!YK zdx6)qvt}2reuyRKin?%e$99pgQi{d6!dqCo2{4cA>Gr`>TT}iNZqLk+?USl|DNv?8 zI<;~Y(vA}L*iFxnhISHf(5AwYe3~S2C+g7}&s{R91s##)9=Fp@0>#i=aDLi6<)EoW zPxD3cC!JPlJ*7|qY#bNZKYC;Lh291w?vhk(X(9Q&dxI!6!WkDgc@T181n%5mS8Zz2 zYT*_?yBBg7vEjGjY$9q$3wm(D=FhQ))1v)>bXwXMSNDf|sq-sc0shq~2pL-?2sS~H z^3e^P_w>&F86Qnb$P?k{S9duVVo@$ikp|CbN);_&t_{bxu6Em93~(M40h^vS@7 zsW|)2F>f7Q{`)gb^~=&X!J7gD9!Hwb)+S4Y?*KBGEx7(govXC!EjeD(T&cbB^{bSEE9?rk#Nd}R!w)PwY?C_EmRPZ>g zuOE`nzXiUvTupE6Om1%QvBan`Qx7^3TW{H!a~_zwUk${G$9O*G`txlHY_f|zg_+;A z8teYaryaqj4OphAq>#3M=2kyCBJC~*x6I?p+Y;#Pnbl6RF-IdTV&%I3LVN*8tKWJ! z23C7k`A`3suIvBH zVp28`3 zN~z6ld*#zfN3=C1B3BjiYVbE(LvE@naEFg-xq#6|7rVf&-+-%>am>v}h0{omC9YjI z6rMBT#1EZb2_oop;h|Mk6?-myT)I>VyIZ?yPgL8RdLr5B^nyJ;#?%=fe>p{sWy(vp z7mCGi#G`TFrl1;{$bJ{Ad=@?6y;VFQ9Xnyt!ki)E2Ozql_Iw)QY6+SqTaaict4+$Q zUAg4aEY@zWTIE0gR>bFtwo|`&e$Z!3UiD1YB~ns)c5hd3`;OBf4sEG4D>)Y40LY;v z-z$#nX%A}X$<34c)2|X_7B=MWs75M-L?;LQ(Vwwjh3Ga3IM*rjEL~@X9=gkB+mq1fUNmt#Vsuv&#K` z#D5mNRM-qf=Ir@C(4>fXzwWSCxvBT z0IBQdw3kC+bAyNfk*Tlh?V23Q_7Hg<>UK$#)eK#-AUf>bX zGqOq)+!mP}|5cU%jz|!4dm#i_gFxSRn0e2U_9O5J)pQKv`Pw=xL*P)2i6@R~0;@8` zjLowJ`)dRLoFMPvy%`mYy++uzfW3A`yrl+7a$)Rq>U!Ys0INliH{=|#?Am*40i<=- ztlp!vIdxKecW-Dq>`Z}-(X~f%tJ{RWjq~cQ219wsl`Igz0~akrGXTDUSl>{QrHJH- z?Qs_0!?keT!$~ht^a)!CX0AdfOm8iuBoTl2xa7WUaGE2%Xy?NGkwR__ChEjX<>Q_g z1zj!_tE0hBfmEW5LD>T2_p$x{L>@Bi9QOupil^z5_;`=tSD$E#`^KP^9LcvVsrho> zx&pQ`2*ymAkn8)Kp5o!|mz-UU{Dw6srSj2UZZewM5@Jc=F(wFzs7HADAr=ZgRgy6tg4N{-ksr$ z`Gw&y{eqz`l~oxYe{A=GIkBK$cxic6J)9YeXG!l51K#tg0VdTSU!{gkTay}e*Q-~d zrFas@rCd%4il#F%PUL$PVIS$5Pk8Oyt-HW`5Pm?tuZ|K5L?}7)o|(hb$Lt0DWN=8< z&C^<1Db&l~VrUbG+>%6VH!sm>+{c`Lx>+Q+I&4b^&wA)5#aDesz-w@Dlbaz;>&`p} zbrSN@C*oY};%`hVv9aS>g>Kb88n3#Qt#1-&z|!k$ZBPuX>;^A~t%vlEt$FULTX23D z4m-7-Ixs)@ux9%cFO=Y$z+KwOcn4~9v_Ts4<3a$U&Jyf$|IRgB{t|14mVW`t(aa<; zNS1Qq~~D;aWjkBioBHgw;DC6+MyB>dOBl*(_P0=H853!9-?&T-m+8=eZb zk!anXrGwSq7S89Gz^;fVw$8uqPhxJkQX(pVBnMc(@%i{)@VGK}xWJZ8MNjDqJYR^#!RTjj&Q7fx_&5W*ff$I{(m8 zSl6yHbtKkw?FNEMS%j1+MbOvl-UO1|uQV^2uW7xoo@kXPs>o{$_x6;z36b{s;>!P(bogLU7EGxllBp%uE97|2lnaS6$3B+M zo`f*%caw9>iSUMw!;@2qUxynw+6uhRK}ihc>u6bnxmZY!-krJFYN4m|HScTgFU*_T zZhVo6?C9kAE?H+r&5ebZ)u~~I$*iO3N{RWe=GFSHFc7P zx9;~l$i?fLZ%+EMjPa6$S^XS$A|}8xk$MFt9Kb7o&LR!KX>{72xA#0Zio@`EKYKf6 z+vY!g5G2sXvdh5GHx1=`7~vc7^N=-#V2+Q#L)@-czLBUi^HY*F{0_YlffV!ez!^;JCvw@4p5!*DzB-Sk+U3zRqWF2q=& zqUP#J%V%4v|B+j}dO4*cc%1OugzM+O@Y!Zz0<(i<(S}5)SL|}c?+FK1mP7<-J=Geg zbl8XqFn}Gi#q9XnPZ9D~`Vni$yMuRU7fJl^)lhWlwC%xi+7Q?&tuZk;_5E`NKpW@M z!6wyC_O!J4G)WNvmE5tRT=Xt2;Io+}_>VAelvBl*``b%VK%Urnn4>5sJzj2sIfm)3 zoW%{NFMPRm7(>swMwEfY7Y;nkW~iZ!c`g5tDubfZiH9h0r0llVeD_jL7-tih)0v zkv9Ci;u;6lq((^)<<%@kN!W;Aw<2;X^#x{^J|Jp1AQ5$?V`{iam%I|Oc}(PI9>>%C z9p!Sy?DU7?D}{3#dt!3kAJJ|0zp(&b*>+8osP8ADoxUNbO^xR%cu_7O17t+ecuQhY zCw3B|{ONWh?`Z51U$~aBRV0dfMyIEPtZ6v4cTf7375I?D>%>XhHaL84Ldx@e8~u~? z91wS_!-dn^)x8?+)CWR|yv{DiXaN6c_j;H3qO1^V|B?R`2ThY0Y)$=lQ;Y#nmv^a_ zs?xKW`rfXrWQG+1;f)=DUIf5a08BCKs6bdo=5$KYZ+R-v8+7dGz!p=FbzJXQ#RL$* zn!GSddU)&BVTvG)|JFFXLeBoqXEXtGDj=M-=T&Oly;V77qpcR%fetlz-I>Jfa<&2! z_Mgt{F+m!zhj83?fh9@+i+75vN(TrdMI_KHnEFGQTrCRw$;{RsBxdh?bKCCdaeffK z)6aDYC5M~7u)ZKb2fIqdi9yAsODt+brv+o$gsLYmmcF78Rl|9$=^{71_e0Y_&=2G~ zYLTLzrY7wJ4{OF=PJ}VeSrJsLBC-?hvy9;DZr}2JVajrSE=Zn4i6XI>If)UGfV56! zxTsnyX>j4m^9pd(KYo1nq{L%&{-Y@(aBnzKlo=^|0N}PuCI_K=Bj|~`iM&o|UFjN6 zY{W)Ay;D@)vXAMWFMV=xJBNnMZ$>`a9dgciN%eq)gK>y$TD1OfvT=4-envNWItqQ9 zsCr8W$={B5zYAmV=a9*qjL5NwgRFx%?Kt=l9Xf4xD|$c4gZ_Sym+z<}vX;OZnZBe%R~7GzBksr62{5h?&|X99LI90V#X9Qj>p`oKS+< z1i|<>V5?<%;Hlym(&f6{DoQ}p9;DBifkg)Zb;DRo7D3+cdo5U(3Rif8&o}7nU=Zh_ zgEYa**X)i}Y)dRE7>A`PqS%*GSv$&W2_S1weWGCNmMAdoesQm9ly3m0Sn(~cNmslA z(mfg=YkiU0RV>T$LGMi_cctHA_-u*bF1~3k!z?+Ud#%>{e$pXPUmTsd7Cf$_^xZpy z8Q-&hXhRQk@S~iEt{1T7AnJ4(8y(X0)BawwDRj$6H>Edwd6i+;wTK{NG}>A9CJ|K% zX2Gu(rMq`hfF{*@KZk_#&%dnDHw)Re2@haCke&NEI>a@3Z^8KE#jhRmgmC7tCFPzG zM2c|76#MMvk1{zfZl`NxUGvI^yIHo1#UpbXKFjSSDMQlY|u*UUs}2KOJnZc zkYCpVV>>f30%{0lz@iTT!}Wq@LKha&$9kZ3m={H&GfVLfY5AIF!*Vur9H_2UBx(Z9 zS|alFW}wE=Y@i@8H7oa? z7>H!GiVb941LZE$gk=~gJhH?RwENkAtDTWG<^xOMsmQL6l0>lnc<8Qz&3*>;s?X)1 zUo2C^ku1C7qKt*sM9~INGD?*2$ee(SvCJ+(1RxM1;lKDH4WBqHtk+csbS}CfDSiw? zUFEEz_*EP|@_aYJAR(8_4LqS}%e|;^9UK<>&?l-wrd)+f1_(+9V3Xkj%E1U%>q`(t zJ^)x^yd`YM^Oe$49&LQxtNa9{oA$`s^YS@ArRdLH=jI#0dU06ky~bjhA|n9g@F!Rx zU`eCPqhEtj*;$Z1;PNAz+-i5^x>_hX%!w}i1)I1I9JtU6JDyFYKrC6W=4xARgiB=3 zH(}1wlC)7;9#F1ZxjFyR0+ZPHJYfMxrM05?ap-f^T#Q~ee^+e#Ge^E32Oqq+)%B08 zTB;dg<0$T~q+{HA#sZAUK~08E5jaQR%VF3HL>W|7T4~0}^6xXK^Gh3dU zmZLRv&gs@+DV>{J3Ms+9cg)-V^t6ZWPa-_E_4U^tC2F6>2`g8916!IXbvO0hQo8!hr!z;rnLFY#-10s?De*N3LxlW26xzxh2m$N0RXrAURc1qhy*h^^0;>cUxEGSkX}u+`q{jgdw1M3 zi@)UgA^Ul12$=!hRlpAnP<}38!%3YeN_&xVVM4aVZSl>0=k<4sSwZ zEngmZ^IE!=UTu8*(iZ=PQCF2LBqWS3B!UcXl=Tj42{Mz*PKRTvN$?sm@^h6uO zYlVB`T4+cbe1=fmDnifb#+A!+DQIp;*2jI>L+bmz-Y!9PE|$d0JeD0^t$8p5*+9NI zIJ`bQ=gM59g8)%70jtboN;K)P4;H5m+qv$LawU!$2Mm0o8RYF)j(-w z{o0(}p~ObgZgp$Jp*q*#_U-`regM*FZ~&-2!F$J9z#eKRY*T*t@f;8A{scG@Lj@0p z4a>F(7^+xb%yl2155{ulli}~K%eGXPJ1ekhy|}t4oo=lD$MYZ{PYa%g0Z?*=*fxw6 z*(tFpOs)HEUS*itT&Q2B8{hl?)*EXh{YBolYUiUi(Pm^`L~GP=O~{mWs+1CmtR#

a6m&g5F;6~IAKd^twL?L!3 z9CojL6W;)EijHBe}?gV>uczV8@pjI-CDFG>rWsowJA5$`@hi9hFcyYRs0 zSZRiavvno>v*M9o1!Y>(PeM*3_O)xHPNQkO|E}uGH;(2anOr|@PIkTa3DQgO@`ZcM zr<-tWlYO`3D)!^PNRJVeN-6i^Bx{h+`>;V0S(%p=*Ct?yUq7qZhtUPdsmtVMg*kst z4j6MV626s|I*6rru@79iY7w4^^3|@1!ceQBiQv!wESGa&KiEo;Ib6K9{Vh>B7PnCg zm7!8CMhdqz*_nuah(Vo%%EbLGW2b)Kx30Pscyv7WzhGnw2?U&xr)1-B&(V&kh$$09 zqdoA@5_X&!!6pf2hTi&~v4D#aXBV(No$)0cp9Tf-Kf>tQ`-0m_B$vl!q6Jt91RCU+ z{^0ACVGIXgrHm5gZnV)o?L`3j?9XAB-QQ!PJMJ%eSCRR`;6Cvl%Xk#hseD1Oc2Uk07JmlzzLArGqW@7Uf7O$6KGK z?%m*S#oh36nTlk2Pk^o-)R%Sk$>SSI-Php+)=*;+rmNrE2>0U@NOxlJ#))fiiM}#{4RP3-6Qju_kONp zqO!`2Ll33R+`J0eAfAZ{*dP*$_H3x|KNI1yyENm33vG6WIlBLOyxJUyLbW+1vS2SC z)H((8_>K*3>hw(ue4##lkD3wlb%i@V+U(x{aE*y3D>JNQU`i-;I9OI7EIE~)HfZ4Z zM}}?l`~*%fxn3tVMP|1xpUiPUPVJNZM$#97XhVrgns?4}6CAV}x&wKD5oxVR^evz8 zwR-iG-!72GhyoI$hE{|C1L^>rhGV!0O1!}Nvk4K@1(dd+;7$Rcx>-12>l7E0dLV+o z3(p7rRTuH5(=q8g-@DZ*FD}Z@WDh)6%6%$}mXyPG5DgA$07yNs*K@JEK_N2ba;O59 z=r_;KOf8xI7oG3>wIv4dF}P}umz1GXaYM>Dcn{HQ7@vgTxqFn8^MnMmp)E&j>Nky~J57Wlh5Q|$ySI6OchS*_O4g9>yGs#mdmpEO^)sjN#mXJn^FUe{N;Pu$q^}WJapo1M z&lrnoP9FVy+I#i<>FKZKrTMu{wvMU*O`zLsI6#ruxNa>R0`D1cYodSD`>%<(0hp9X z!0{Y%;GPNK^pAolq|1`2cLKB1isbHcL@0|3!{3=eJ0gKQ0PiJ=6nDRk%m-?p9KrZR zR*fn%DsE$K4^m@lp)d64D4qW+A+P%5(SJ+xJ9vsxR<4CgxrygcXaO6CE9E+sx*pV9 zaY_*Io}Tc;6R3T=@Cl(CEs z@5##+;?*4@>$jBVovRbum&3|G(A~c4JFoQqjSc-y2W*WSitZ9fGMSL!f0`T(M2~73 zAY3B0RIUR=H)N%*q9zCnM{^RLo&g+Lcm}cGSGupUiqd}_WVxH))1~Q{MG{exB^W8? znUZQJ7E@33G?=b7Y#+O9`(BUdUd~9i&`dlv9#X7Ip^}O*CQWq7;IwgGA7q&`KnCAT zTiOs$0$~n{vA^lR8*;0vs{c9Tj}gn|Ds9+}m&GDJmq%DjivKFmkJzc}d*-e{2p(A&E{ zzZ-UJY&c4LHZ0S~xM>4}QVm?klPoy{KyVP$wqqV0}ebE6$eJx_NZEPE+Awi)|jV6O*> z4+Xr$r*en3CZDH*-uG3yuc*wb*e8A%yDCF*giIh-&Fi)#!9D6AIY8}isSVa7{=A;0 z-Q(n)&~2zIyP3CCxmD+*;)18bc!O$@<-xqn=v{7FP=nrnX?x9k8ULj9oD}b~gG=PO zCkh0D3=m+1l4!e@W*e6G)R=tQ*#M~I7VJiiCMT>xRSlhK(I+2@4!HAYz~rmdZV!u8 z&{g!ja{0O?%VxPc=w~@ebbFJg@OQm+j^|EDQ%`6^dpnX~lrl47rwF&gDdYm#80k>2 z9EP@xS)YHh9>ze9ayL$WR8h}#N&Jru2;!p!B6^;r-P%vDYb)t$%r)O=MW;=`&MNP4 z)R(Uu6s)z$mzLPcxJ{?u^EC61 z+Og{a2W@Zm%|G|Kg*bK{jRxuflZL3jnzE+|FQtU2!N~43yeYB30VkE z%#SknAY-y+J>1t~kS$YI{am*;;eB@0R=-wX>eC}Tp|^1-!N$k4B^Bon_Iz8=vK0cS z2Yh=VCpow<4?PR=|H0&k86EJW5nbK{GXZYd(Sq*Z-=QV|@gyeox&IbEIN)?kh?W$1 zo8D#E`c6Na&G=Z-vR2liypHhKxXn#BK5DFTp#J)*cZCwPGxbp>G9dHra`!xP%j$0y z*7roKt9jA?&>=vzVnhTpb?Yyh=wc?skJxL_|5or2xj_!Qzh<4?sI$cZO-2BbCNhK5 zHdgzkrkna0Ds@N?b%-?umSC|bp^*r>aYMdxCnt{<+@|;#wtat35@(Wo6OBUBLS{7d z6Tm(HhiEk*pa!7HN}d5DITKx17;MmgOA93c5_XETv#Mgf$x5QEnSc$8-f$PQ>%?kG zIm4FDzn0p?R(e#V#IvRe5tPqnAi+_~B)jCsLD)Je*DuMghEdTDY(Ipp)Mxqr-$bTB zU{YBDJeA|jl{}r}%ryT{vxyXtGf61{83bAWN;ct(iJ?XjUZD4BW%Y)cG0|5 z{G4m49?3&#BExh;^Dq0XVKQsW&fv1Fkrq!RIu0x%-K*N2W8zb8lu~Hz05*2IgbDLP z&eq@~Ubj1N2Dyd7#J=MHv|3;eii6{aD=tpFsa7hLIMjz84VNMd(7#`FQWhxm0QFN} z4ZWac7!5mv;aPefHQdFsta(m;-d#)m zlf{Nz`XLpJS#3$M%<}oeyC69}U00DSv`Qldx7y#M`>pZ(7WoEc(L%$%EQ~2tj`jcX zUISDRQ=XcI{_9?wR07pHWzpizTFiK)o;3>IaJQiUvRMElz{k4e_?ub5e*-?rzAvK% zwutSAk%V2jkd`-LD?@iE)1$sAOP=^sf?`UM*_suSz>W={$uW`~6E3l{EWllNCRtUi zLTiIa>TT>E>O#4Y&;kF)ME{I(0s#gRdg5Svm#V5*BxyKw?B{v&L%UQ@la-Fx+wriQ zr$Ta;>;)<=rFski!_59^DE{>V2x5m`ek2JqI#5_6#2D&9)UF%qaIc=xRW;?s()67r zV1&Dg|A&zOR{P(FeZRpBaAgbstZ{a+CNe@cv18QK`tZ@qMk6e2Tr`m}}z5G`l`<4r7`OZYcF|1VcE4KQZn z!pYjS8UJe4&);-#Dxy=jIk|trUO2licxnge;uw(y-#{U={|I5gve|r{7$I3Xwcui`* zXGh5#n2BU?z(c}=YB|ZtG$ya6)xW!yFCF!G@S={CVYTX{QQpR{VG|Nxl;4Wr!M#1+ zkkz2T|A>VDybtU`3?Sr`jpnVkDFUlzv9!#UeG_{tTSRs#y~d+e-7H_s=TG_Zm0Il4 z8>N6Q+yJLHhJ@#(ozSxiY;V%-3bXd#-v6IY`QMEQAplyi$WXCmgU{6I0}y4}Ks1mN z&W>%3&WjhP=5hNzTa=&46~hB!-a~V;->C&vcOWy;_~C~|16t@fBtmz!jj?d=1JXad z{{L?nfiaZ6%s$wTw$dqabPh>V0h!DybTvnLYp7{7FC=`I(wJB#Ug`34xkKQ;QW`Jz z1Z4|)6XiNNTpG~y+d9sTPm{7Cd22fT1cU$A9+oEnCiSW6qF2v(L3nmaqZ@3hXK1OR zX&)9>WG}@k=>B%Gs2K85ihJ|0Fa7W}7Q{PVug0L(yQx^95RlyGf%f2t0(0VA@n1rLf{uVZVJ ziv2{6G(WCb)1`Di^h*VKNOze7l!v{cw=HJ&?&9u;lWldX`Cz*eQd>!EnSgvuBeU|u z8uiR+ff zRBq3m=u=_?msAwpy+wGgX;d3xftaIFy3DarBM|JS&Dk_DuPbd(=V&rSYBs8HE@UYH0GQ1G#S3Q3E$afX7@*Tkhk7<`O=gVbT$XA+Yp=)(p;Q= zHB1<5lK<5tK)`^bBI$2u_a3*+s)Bx}*ZIW9RCgNG6-rP5S<$HSKpj!X3D>kIbgYUe z^sKoyaatbXE2i2!JPr`gT|ZzmaYe2x$-OLNDZqrig~V2`P;2cCsiPAF1VBLo4X~!@ z*DcJ(fGMrZ}=4=62Tv)Fkcl8v~tEncfp`*W$tiHG8H6&kVNbxVJzn15($;T{F ztBkCxZN-6sypkcJc6U@Gbx$|g<*)cCi}}8i1-#qzL=^N0Aw%9YegX)oWvgiBidKfC zF{Wxg#&8~z!uYoo1T>#-J4nKYhJQuAD4PLx$j)6`psgG^UX`bEsjM1Q6J`Aj2m=KI zA;E+MGalP5hg)Ts^}cSL=B`QSEMHJ|ihT+?Ts5ky^k07`%g0n_VQaR=9DeMY26rb` z&DsWfO>QjR*(P^66W&e#7-b$fZzR;xgjJlyC;*xv!8*wE0M z`H^n^gWC0|gp2=jghRelrwZ7hvv6$^*1U~tkQ3)8s)aRYDRErA}} zfoDn@c75;5UT&u;&kxKbZk+DL1sZ$=RrQ@5tvou~Q>#tAIqUX2EVIq&_9|4^kZ-I- z&bzbLT*>j|+h7j4Q|yY1>JlJyy>ZhW<}b{$$7$x4k#HlwU?CzqR?i4?RU-6Jl#@0k zyHg&UmR1LI#gL7eks(c_1)>K*Yc$V}XX3q6k9_c;Z+l&m!dR(uRU}=sXNC{#%z16g zDoSHiV@QV0NDxPp127{nz6m!i;uEbn#CP3SEj=7c0e@kk-3+3K>-PS}_Txb}c2Z3X zsbEi4l$HPeJIhi#0Ut~SMC+s_dgSL=KzN1S@&&}QU0jHbTr;=^842Pc9vd|1ROEQGQ%Z9lZ z=QC3Z1TZ9x&j`KI-$iQ!ytb@<{@7|mf z$1e&gPAzX0XlxYkWvxrH-L!a5Hxf4>MSYh-kHZ8ElcD?0`voh|89D)aF>}^OcBEI0 zM_2xZy4^nSdKx+T#x+n}6B9_+Lciz*VSSA9QfIhf2F71AR_kcQ4zri0nif|38;|;o z+jnpccTmDdZapy&+%z)G_HU#2e05GHkT$0q+`le9m~hka73EfT%2Z&-Gb=MKoK;a= zv`^$1EI#+z+Z_l;@fqrEtl$mhEP<)0&2CyTrP~U-r{5|$17;Hu7ipPXzKfgVjz8k($?K?X zNP-wSeNCX8Bat*s(-}bBf&|RK#8%f|1|86{@f;IJ)IZW1jrvk&627QR*8(oHCz>iJ zL~-HTj?I=Ei_^V#_ws`Gh2;_-jmS@$fAbnd-|zETNBap;4}h7eZ8YV^v>A=2JQ|f! zvdzlY2VTXz3OYGeEl*f;ot$OJt@Y_mc$n*4Y|YKQhHc)v*!V@u(L~pt3s#9%lKMPV z+5d!}f>#DVcT=~;xva4+tr_Q{m5XyeNe9Al(k=XM^vU5b=M;X%fv~s7qQgof-#bBA zPd#2{NbXNpub}!Z+Ii>{2{%VItix3vY>Zcs4Fa37xZ9J!xL226j4Nv;pJ?(l*BO2XvSTs^8OThk&%E z4eho~;zD7Cc1_(9k*8=G14y%2c@m_Xf}6W(7%}m$YD@NmRU_H!TFmAD*hq1|SDZ5V zpJn@X6pe^_v(Xvqy*(hMicBMhmGSXvjRnJ0ScZ4UKW5~qcR$HEu7$kYIJWIqts+6* z!R!r47dFMkG#{_NYt&k%3&UP#;wbP45f#a?xzj;>Yd1g5jU#GxvrcdH_<8 z=1}XV9UhxqyW^<`?NNYSvDUv$*xA}vUXJd(YPw!Mt9x{i*EyL^N)7x;N}S~LCpHG} z%%Y#?W(=y>QKVl-{bpf<+Lbx4M|OTiRvmNVI}#gxO+FRJ9Lc^!rHZZ{@G*g=Us>re%4< zljeZ_Si0>iae;|da(0u*+Cb5LQ=G9>XGAi1(|?>iO=0E86@to_Bm75x2#PO zyq&R=V;;{}1N>EFc~%b5);C>VbiiLfhrQZMSnY6dHlOQ|+FPi;r|F&F*Shm{4hUA; zW~-=gx9Kex;D=q!UxRd?(|uCU$;+M4clS_G!_3v_(Koa8g=gW_yE?hv#T`DgBX$oC zo7D34iZ8c=T*9X{&V8b@5wY)s2fNKmU&|nayx_7C;O~2egF-M9)Ly|c23#Loa}Mf_ zcdiP*VzW`KG_*1)ogzZs`D~Jht_Dm7Lma`G3FvLY%${2j6?ewG+nhexQtD|2z>R)G z4?BHLzh}ABW_uug^Jo}OGozoizW;QwwPoL2tBb(CObP8FW8q^A^n}OwB*TeEn2A#& zu(;TcHv;mv_6m4vOuNH-s`8iLhp5>1Oi7Rm3$W)CL9zd4k--};4DqL&2*RN~bu1Xz z$_U|b*}57MUx!O6rZZ7#zS^Bl1XTXT^23#lLf<|=R98)T>JM(c&DJY5znYFm&g=c9 zV|Ml7a6XbBuuj3){(c&UFB(hl;4XW@GGbXJ{48KxSFsBtU&ZF&ya>2b8VJ5px*vO_ zcu332+c5aHCCjN@;L=dd!$R zi!Gsex?fIyW7l_7V-P;Jj;qWo;{j`a@pzgkb6%M9r;ZXh7vTl62JJj;Vi^7C8e{vq zxyNjvTaL6u87FJo?&1DnP8?0C`-E8Vvr!}1qnFg1+xoMW%i~lRuMWAiH2RA< z`XEfbHv>uJC%X|S2hcBf+U>DcN&2?FHZ#6 zu3i?|;*Thu8e0p`oY;m|*O>00j0q(EHi1Mco#u@Q=h9%8FG7CW9P_wWWQ;0n4iV;& z;Da*C!bq~7i9IoG4qLPaGnCWi8DI^Q*t?WsMAb&dRU>NIxvv%Hg-9c!-ZDc8x&wWM zHak36F*u%Qdt0=ye2u!4Dh%-s64IiZt0}xQtJ|bHK`L0_6S^E~7mS(0S?yVU9z(3Y zx}*Wk^#{|DD0>r1n=yAO?iustNnuU#*W%RGW7$-NWhVSPHa3#;n|Jh3lg~eMV>%7; z-!1kOmYWamGmjz2R=ms>B?lG`7q@l_rNZik@y_%uU#)i|cB#A`F3qshS!XsNte{Fm~m_Rq<3uOD3|PlnzK6yBK<=vZ*X9rl2hhVwRtq za~!-83yY^HEbk@Qq>`+7Svdy9j5{yxRvq@K%vK}4=?R@Y@uMsB^M8|e&zjDtiMK}E zf1P)T&$>i!;B0HOl-_gEoDkM zBz%&4O0pjkqKos?8y?zA;uNGeCU%NJp+7KcM|HyO*k_=e)g>fNZLn)RuMLVpKP zhL|oQ2?_QjRWOF}ChvUhG~3!qf;5FOGX}>=ScPd%H&eR1Kp7%~Cx=__}>j^m2Qnss1tf%4a@&UDA3KRz7@`R};zg(K8_?ft}{*Zui4)TyM0I zdN`+-8^h+F8jp*IT}qXWyIFnlNC|BmJrO8wF`W;&2nOL@GTO;ZmM^E;=13yk=0f7y z=Fr@$OJboFHBUQHVe8&J=-{w@(%PGrl+~RHc1-l^zOjMc0|ux0tRlFn%_bkMm-8xo zI+EaW>!29#X&ePYVhuUqDv#S14a1+U@l3uep7aewNOR;|u#uMAzt_&d*b2tGSoZ!# ztya=6dg(6hLe@rLYpA#DM_?=p?AY7IcX|X5pQBANuXh_-@yM0d&?W+kxx1M-_4&y} z01m9P36#uO0r*q&G-IPlVQ$ZyRIswo47+ELUJN17hWY+GR7cEl{zx@c4(RK;u>Hd7 zAr>#2`~n*|=kSH&3JFxwrRtOkHS_TfscgQvBhxLBztI6l_pf_G>D+W)sNhCvWsS-{f8)P`1!_o+LPb(Sc*oBC5q;x!&U7c_q8-?0ci`}|kj$!rrC?K=8KH?r^ z!YuxD5=%J03ws(^whH^=zi*F-u2;|?D2X@fGggdmj5#II44LejpUGzL#@9GaNeqz! zZK9?1uFlw@e=ezkpEgw6S?xojLT=ZBE~z&fx>AR^rMB%cxjRMpZ6~JStI*f(w#RcO zyT&kz+5C#H3hs?lmBq|;o|X5cGf5}Mq|E7iD9E0U<}6PV)}8nmlXxR=uo{$%YcSK* zGv0V^DuC#=66LzT-{H&5!g9&Xl4XqhuJygxAD>SE=5831Qpg|+ZY&IA#KPFLo5|nl z07xxT&S0@of*+fs?1u}Tm$A!o0A+2IoKxF5ojI@f^1i1&!e%26dVw>-+Zd}#1wB>yHjLk03%#aa;5)8}!{^7%YU+!Si+r&heC%P=Pe_E01$q6S^`ymBy;QMXb-uxAEr7}1Ml3#X z$9yYKZ?l8P7H!J-GwqaP+qn|fy2*aj`me1K_6Fegq~mU;*+weZv5q(D{3KzX)(h^b zgWF#OX;tQfQ8ygU6_v0G&a^_h7hAHHbD*6R0>Mh>p+w4uI)M7v*JYyA)h zM{ji_u8MhkMdtu_TNeER3r?aS^;TH{J0FXOV0(F+P-VO$100fMy@Y($9voAxy_!cu zoMZM19;^sA?nIk*gW%6d;C^r6lpR(tx9ExvR^I820@!XL9Zz)9_lEmZAu3JC5$51CQ`B zcAqbk&-#STS1n{5%V^-17=`?FXcZ=eOLT0Jn(V1idjEsME`Mtrg1DES%_p~uaY=qA zHn+bnjKxdI#oNmjjb6J=%&H|#34HW>T2fHh#rDiy0B(6 zInJ7|i)3vY_Bja2N{|UG2w-FEtz1RhQspzEUIPNi_Vj5^?Ma}f0`^yc4M{>IhCL2W za;x*`IrLY*QzPrDF=CVxuRNj)lsCb)!yKq)%~0N$qMjTeoz}!kI7~E@nC{*`H2u_L z8|@wlH7S~e9QEn4d)m6Pz1{oD#p=j%`W9^ZFu6;{#n$&MSkwe1P-|H@?k%=q-Y5>e zX*NTbmI#bjeP4J>{IL9?K)3X><|6&j(h1zBX<#+z7$nA9AL3^Ou;!7l*f@YPdd9nF zGUqYVv&SAbd?s4ArF}@-@dZ2w;gwb-795$ir(F3p=+cytwExeJ3lDTr7Ful@nO_9w!n*lMRk@ zXi933>nGI(WO-TVnI+|MmJ3bQAR$W>#Hp`nR&U@8*-T9vu@lzRP`2~7nWJf}j;U-z zGS(Mi;J2$2Laut2bVx}0o_i3N)StQsW2fG{}=;j@LR_VURdgvap|6e}m}}R34X%KVm|u_b6dBlI78Z}7--1v}5&Ov=t)kYpUa9c| zBKw$~ajOSjgg%YDDftZW+S*R(mK?tAwPrrq6Cd4UyMTDux08DD9(fp;#($-%xGDwO zv$_#(Rf2TV_HUI(flQY?24?>sS8pBGR=2bd7fPYDMOvV^Yl{^x5WKh*FBaUbxCAMs zNO37p+%-UoLm)tlYp~!_+#Q0GFV8u@^Pczpu8V)z*;#wm&YD>>bKf&dxiMAH`}#-o zfm4EuMu?;=pB7R4GHo*86F5ZX!=QiX@P6g~Unk$k`<#gy%_*`C2|00GYe*x;rQQWW zJH|kE*_&51{V$2hAy~6S*Wd4l-!a&zI6DRFr)CwCQq@&X#3Y`qq4yxsA?n&1cxc}f}jWA{paTQ)+vXE0TzEQiY<8du2&#e zM+k`iR-%spC+-Dx5g*xAJl`hyGOwzyaa}<%33NEkdi7aX0g>E#mm}}uUKVbGO=M}H zC?JW?D%CDB2nFRvbsK0c1_^W7Ie*$qT4V&8OZ@V#T%N^o%0fn?g| zPB6Y>=;iJAqV^k3F}m6KoTHkf0~OAj_fWLTIXdarIzL?fge1we{Ptxi?e0-H4$AV@ z>aLBfXZ`6X!HtPUAw( z3F539lC$FB0O<|%(XtghYCpq227NxUM4CiPaxQu3Do04Pu6t63PrT-qHQX$BT6cl@W%Z(08-9?CUdo+bg)Ykox;sKJuV(@I2n-y8H z2IY)11f_AEULIj`lltszGSC0C3nhpot*sc;OpF<O+QPe+GwtvG{QkCozM{)BK);dEab{7nGXuaDU(Y}nB5AyB zoY{^=9$Z(=wO*X;F{csvhIP&}Y@~j9SFiia>`*YU9E_ucE#)nlc1`d5x7T%E`7y)&Ep7sK3uNND4RNcVjH= z&3XtHCN|@7&yIn}WN$W7;R*fcZD~B6<3~k%jd0S!KtA2Kz&>tTh|pRQ>0Nr>^dGvO zfQENQ2!lMs-~2+5$OzAz^pyi5`>z!pi|6#r_v$%e}MX}O}HE6sQ?qx zuOP4Zw@zNrYG18CYcpx&!PNm6>42wVGmcn{h=l$iH5z*Ueld`y*Mys09n z^p$>=O)T_3ERuhNppy7ln?_!KVOz3t<6#|q;S`F|xeBvHPmj+5>NK|bBtQm|lt<_3 zv~B3^A>0C5FWI=0sFV$-eKCG1x6EwvR4#V?$Eo|eykiAvftaQip9=`Ddd3^@M>A0$ivl7r_OcjUs5LmP)O@GOMGhnDTaUh?Dog=AM0eD{^d z7s$1RV&zVIC^EytZyntPLq#wbH`S2ecP7yLULTBM@)s9|%c%M?weZt|duU;z%_rO^ zJ9&AX>v-mWW!#jm#a1mA#fOIUIsg{GqGP!Lt0iB!*YeronS*n=3*V7Grq=Yr%AMbD zvN9YHqsdRui$2K;6H=T-@(%oOj>)=;sh4#m5VS!q;$;~$;+gk#9Sit=TvF>fa-IBh zAstpR>A^XmR(>F}BlJW`r0-0X(N6x6-*Ak;VQurtU<1EL2B&a7QI(>fRRj0!vyZ$M zm=;USH~m*UylQlV(cIO!PqN9y6qAMs6%8?yfTUt#VZxJq`6EH=GVq&zzVGeG=%x+& zTAQ~YLWXFB<~U5H8}%gzVSVclk2o`@gFbnYCAuFuO#W|(H=Y?|di<;T2byh(3@bdg z2AbdrMl}&}?Ruaf6*pmU($K@?yG`X0EQojaSKkCajFjHiQ3tPgt~tZ8hHXkxjcG~e zzXr$RdSk7Dgv@R%pH0oa;d4fB8pX}Nec${}N`C8i<)`~Vv|}F&x5z&7xQ1upj}P(O zQ-+PhXbLtm#ApgT!Kk_ya{QOp2U)Iqc6=@`#%WoCkAIyiC zrn>tuGBp;F)Ek}y+(CAtlXfLvm;R#;)(U9hqV@9JWE#^F0?_e2=k9%sm369R z=|Y@ZcJ9U_b4c;L9gX$H!c3V*Eu%^BQ-DNZdLehbCec%$8Om#4i*Mzs0n9z(BCV;N zk)XdXvz@u6(QE&$ia5$Gq6fJHsZ-%(EUHcv55rhsZDm$>74`V4j;3_{Az$W zoMS;m)+zfQ+n9jus97zvaOeLc+)fhpW%D+o`QiLIoy`(1$4OzCy2|f&>0C3UtRo4} ziH3`#z^2zJ&1g(jlDO>RQCuq(9nUj*lGHCjX$m)(Z^Y2?vR~%2G>2Wuc)-LB%)2Ar zsBcMr@&5q#ADJ*a`8GT}9$_~#teChP>^$EgDxAbZ(o~k1mU4KqP;tJ;*cK4$YJwN{ zDuRth4{i!KF4h@|dauR7%cRMBCbD+xKJI}LTunC0Yv#{DHyXkskz=zN6^eeoY8v3RopF-9nFK^qu+L@Sca@uJ`04aOG6 z|6zH=JjMEIb$+)Z7c&wzQK&3=^Wg^j4Nq{bg&IX>xq?3-NPzVbtXMAg*3n+PTjbed zF-wq#UFHafddbMQg7lX@pYOhbCe*S&?cp%g7&?JiavozZ$tQlbaUWO-Z7CXlf38tKL&?5qT@R~>oCXcGi}lG z;gm1u-_P+h%`sgvt5p3Tfuf|*MmvW3a(2=`(vujfss^MovUIj&x+na3%sD3dobkpk ze#F4dBQUyHm{>Hi4K0nrGF>;(PvIUjI~$K(C9-90Ehoi)E;6HE63eFM5X=IyH^ELG zwr*aIx%IyoF?hIF{^ABkMjyq0(*>&7MF1zXzNao2+L*;p5c^@XCbb3Yof?>u_ORR8 z|87G*LJ}vVEjF3NVeR+kZ7Xh%j3!n!F4gf;Z=MsWeNKwtEt`?wEqoXOCq9cpp7g(8 z?vFpl*rD`cmIq07kW-?gtzZ9-7*jRx#e=oN;DY-$fg|sEc;t>+VR33%06FJNH{k9E=H{1# z+|8j@Xoizq_2;6Ze;$)^G{0p(b=Z}iPxJ*y@klMm51fI<+}}?^)7k!Iek``0(b#hYK(VMBx3dw_{`m&Yx)JGJdsW`2ihtmXt}#*CP(;VUr}FLyK5 z7?<3}^%Tr7k2)yzr!~I-*IIBcXMJe=7SVn%$ogpi`igpHoCM=UN&MHla6j!qGM(@wn{LQIS7`;9!UF!LNNoPN>l((k9(q%^_m;10ZXO=1k1pSL294KGFXq;1y%-2O?F~}k7CfM1AtyMW^?EFVG($S<>J%OG80hlfKZ}!3cM722WlfOTP2Iv}*Xb0t;`+`mV zG?2^Yn{($meszgM1d4tqqa)WQyQfw8#<>u?GQ$$PAX^r1gNPtZ0>v|0z45xaJ-8`qwKCenmez?H#K|BcZf4l2tsGV79xz zTg++bJ(?}1ry*~nH>cP9>z(?mDnFE{OMKLp)l##&kg@BA06M6z`sV34*DA0Q=xJkY z41+2u1D5#`1+g%H^NR3%JMNep9a+{_I|lR%9cHqEWhEpC&%8bNruoqs?(e=nV-R&E zTH&E8y#)fw`@1qz!T+X~|Qqm@lSP7qvYYdSXGhJ8( z%~>qiG}bxuWI4+?#$}3O6j&k$-l+A@u1W3s(`uyEzu)3U>gUayUrJhUna~g*)o!Er(7~euw95&tgQ{LSaejCV=TuFaZ`X_{*Ra9B<#5l*+G?BEozU#4 z-G=Ojo=Ii5YX==jM`;qpiyJl(7D6l2U|R~tVAojb#-1Ja`l8z2$CH5;C&NvfcTpN3 z4CVi7)jPuAw6r%jSMo=Nw5aWN2Dx=4Qq3xMb#6{%LrGUxciV9J?)lBZ@}2nrJLqAS zbtmZl!R4*o`eusmT(g6aj!w7?W4Ol6a1CE-VZl zrVN2BL*Ao$N*El@m39Q=3&J<6j47XgLUc*(;Q&Bzo$oOEk)v7k6Zi%%{!{H@s)@OL zwI`wT=hNWECDJ*MR_uyc?0TQ_{2nJeHx(!<9dv3Fq@0{=M$n#W;+GBdc_=F^M7ACC z>r79ME(h){@63`e<E>8ZAUpm8r`=4ih6911~Sj%<0k}M zljuS8uzJrvn3_HdTLrE3t7h=& zG?LCta&%qsaOyYy%3gGslMA`}8cMG};oRH*->{ZW6!WMi7mCfuIyE#l{Af{m)FcQw)zHs1iTUv|r2zDR?j5(_5L){Vh4V zzigL{Eip(#8&Qe7`_Dfe%eDv}rfEwCVl8}`-tE$)-0B3|mlKDaqL`e=AW4KkANxa>Bc<$=(dW(Kfa zfOi&VCwA@8wI4N6fosze@)iZ1O4%mYIqiyiM!u@!ckkm(Fs(1@2DrOdr>uJj;W87) z`P@~zH4-i4Xzg@F`i&_o^6ersDVMVR#!QM2-orAluEi$&qJJ!V&{WFpX7bPb{r&*# zkGg;AkX6j#k)g`H^0gidYtjCnN-6`2uk^z0A2cZc>(KsS3w~&GPhbZf;R(r4=|Mhm zyoTZD8I7sC|^Ij_7UGSDK3!5e=~d7cFA}S zHo2+Pmi^2D$HHbYjgOKw=uxZMh~?er48jjnxKG*$v$YYY6y|qob5QGn+olT?09|up z^dMvW?3F?V&hWHRGIGe|c1ozMp&h%P_!RHG1XZSi;Ih&00A-fvF;vMmf9iRs9EBZp z#a}5j2_C-6ol!Kyp+!>I48qb>?p) zG^$fH_9uS7U;lUV+mVv*6kq%aekhe{YO%2x5NvGpmW?IAL$aRZAMWqQ zuIt0h>*q=vs$w^-ZEUSYuA?||rj^_85hi(!=?!CF(P4CGC0jCexKdi}# zZMJM`Q;C8_xA{Qg=-OA1r3CEr%7*r=r>mpKEZ{P8L&Jv&@;a-wC9)yFuA~NZig2X} zL<3x}jh(pTPDzXUUwKe2uVjeVCfhemy%mkS8uNmYD(f>y3ZZrU zZw^CJ1D9h6{$`8R!dX>yhOBs>@G>XLE)@1fi#Z2QDQu)g6!#k;<#l750;K&Wzb4?oQ)m=@{}w>EG77Fa#}WcI~muUi*_Us%sU{<$~Fc4|04fA>v?mM`R|3&q(G zR0q|le|q*~DX`JEjWuuHlc~aeIBcc|R`byamcHfGEn@K7?0>YKaP)ETP83Z?rIS;_ z@%VUE3@n3jn3AlCP+z@{mzRqAzQyir>Vv+mn?1uy;H)tJBwQ~0m~Af`YYG06@wD4F zgLx9!_;4ao)(SWy92Dn9z1p%2r`KPJ-wEPVP9s0$tE9cIXM9&{Nkg!xPwD~As(I$l z!uQWy|Jat2)u$^qCjD|FmNEAg+iHRXQp)X!j2(co`JDsBup)>d{6%YmCmXCHeO*AE z-|MS$ZDK^#Ghb^ng(;ozT%Lc5KO^MN)YM}pMoo89FZ%~9B>tZkRLKAB`d(1iM23$$ z^^z#tV8AJs=c&Zaaa@uv^TzVbWu)7}%IA;X@UU;b1k^sF=&4q^(=DyK!X$d?)XFhVUwee`> z2{Fa8`M;)i)h2X`?x!HyPt&i^!}4zBxdbES?b~sO`yZr!s&PLa+|$fHez8$P-G6yq zEZ?Ehqq8k2rn)Vh>fO*(86xlIdcC%fPXCUEpcK^sNSS9?=ahWQrV3+5|8^zGFqdx+ z+vOUZq1VVhXvKo>G)})W0i{XcK-kh_oTW(7D(AZii3im#vimbylt-psZMu94HvpyG zMMl+vx@+b;Pr_+0*-vbal|X)nf4~RJtp`BbEB0G18VE-J*}W8AOwF>SI%vGv_0Y94 z3@sZR=CFbe>@k1fUHyADpR5FhgrI&HDVq^QfF!Qm>#X*W3e~8~AQXO? zoaO8rv)UrD{&CL6C8-u}7+}`S3blR}QRAVD#+#T+nQ&~3f6@x$Qgj}mU&#pfnoa2n z7PS1g^*A`K%2P)ub)h)z#*KoeKWrM3_9qE02??79`(apbD<4rw8-nkh&aIlbhfhEh zP;|#qVUfQ8vcn2t$A4A=8x;mF_u{lS*C7@ffe+mB zM$wilnSQ!oh4_b#WQkS!5aD133a$ofU3ckpP*zqRoZAFlbC#E@IcLIF2OPE+Wo~z9 z+b;A5Y&@w`51J#}ZbVVu3x^L6XRh<86BiU&Bh0TL9iB4~&oPZY^1FoC!S3LbG1a^s z;c=BMscC6jk2GhD+2g-`-*Su&>2yRnCl{Rys&pxzTxAVRI(`|!zv^34y=UyTW-o|+ z%`Px)SXRw-w-(^XC3#9tbE);t)3xTuOV4BB`Euf z=4GX(dzm$M{`S)Q=L|Vv0cp4e((4GN#-ifcfk@sB7gZ1LAQ0bkjPy+0(T=_WXb|7^ zq5QVM@`ug~soKz$@+o_kMZYU@<)}%cG0ChE-3@Kh*WtDaTh>+X+7l^rcy(pP8JKx= zFT>?@iTenkUL5^bApQg(R&c>#U_Kxif z_3Uh9WX0-r>)BLk45;WyL4nhsO)e)rwbY$b^Jf*OnX=Nwm9sqs917H?9F=v51at)7 zc&b$cSyM~wY|VU(ha;?8>_tjp(5(4Mm8go@l~4_e^_m@)5?CU%d6637q&+;%bCAt! zhYD?e@Py|a3iI}tSeaOlSNU(^`JCy`-*_eqC2LuQYxQsMn&c;UlhjO$l=`{j>0Ih+ zy(46R&+raYXBB)Ay?`y7H-1Ef*_Ydb3nQnV6{*3SG8XP!FE}-9vuMq%?iefVEd+GD zgv)9c71__k-S9ZZUZtuE&6YcABMfEKsycyNbVr5Z3k8F9KGtT7XC6x8@a+nUot4BPf(;cLy|ew8M_;gXTm(wn&a#-isd+a6_AdZ$skZw=V{$NthS zSE!He@$QWXZ7J($3$svwUk{&NJCW$R;WaBf#D2KkVS(A|4vMREVaxm`()cY-s34u| zede`$Bq~G7m04`u0#QkP-tzy_1ZY?O;2`U~UgALnwsnI>IRCP7m$X03yG#&CfFd3q zjvf-_!ZkuHVGV)ejX_w$5X2`SX^rAbLyKdz^cj^6ZW z_z~*;JrZ@%-ftZhq#eyk`zGGp3U((G60m*=`6cfuN;%@7Ir&Yd|LN7;9C;$BW#41Z zb0w8b9k!bvdCCjDm^=1zyj`j-`5K5!jOr`alU?U4@sKnKkSRWQS}HqqE=d=&AcZUb|N`XSh|Fi?Z+Nc?zEK97XL096X><< zZGgs{pA36z$@m)oS(B;axuKtyI8>twuS>zR&6!1-b^E}eUvat4CjYX7y&B(~)5~l2!d5zfc2XB`s6d5ekq$?(Q<&i`CbHPY%{fa`K-| z?NevC=rmZMTJpC1{I1s5_@Vr-TSRNxs+}J%)*h&pvzy#zDD%c}#!SFR0IFJ!+G;<# zwr>-tFW<`jMf8>|@|`Q8_WqGzX{s*taNdtJnRRr(ZqasCYzXp3ua8=sXj`!K zKpm@m5h!m=JD2NWhltJCpGQlET@n$~2GLHOo-HdDFw4rigw)&9A&IQQ%J!O&w+4S6lbS;Uc4KfU z0{!2uJi=6NP8K;jEZ*Gn`K$ib`Ps>%$m%V5+trQgZyOr`RilOsl+ZP_f)vt;F zZ1!(t#g6Feh!!B%8%N91IIk+|1w@ziP}?QLF57Y#uQ|B5MrZj-6gJ71TCGGqJv`nl zYxTcP_&f`iv0g@vsgM1Yq|K5DN)dFLlNO`&OPV@?(TZnR>{cKzPdYdhVHx==wYBAa zU7>ZneU)jh|8}*2*#Azb~DIsGW z^4Xm&+Ar?UrD{%FuA3KA!N#T5gh@L_VAz zs%r^=h?Ems{_>|1yn8G>HISX3W6kJZ?qRrfu@IKQsNb`wRo9VSp0z*A0UV$Ni|(+e z28KP{jW+fZeOi9}R#O9?J)16EiAV9&ZaaZ0>UB9sZfP|f*<0Hx}rui)1nl)}LMZTm}0Cexv+_}Z_HwOYQ~2R}-$#OG zM1Q+5wCC59pR%au5`wyndz&E9WdKYzLlv{3BsNm@cJ72Sb#yjp?eh74HF-uCyWc!& zLAGHxs+g&5%+Cfud>XC*kn2`xI!Vf~h^o+cSrhc=s*?gpcefa{R+7ZQv)|4=IJ9hc zO~-xg01tbAGR;L``xIqliVTCql#w632VZ;%vKu!^s-*(ZsH1*9+7i+PzGTz>METT}Xc>AE0AIe#cw z_4PMnb2N(ojqlw$ttcuaN*q%2V$$c{)SNqUrajodB10K6=gZWK#N($_W|Y{o9(Ur2 z7JjAc_d`Y_YU_lU(v8`v>L3s9f+}6znD6ST9nA0={ar?sfKS4BWGrSDlIc&>{;oAH zo7|7nya`;bH&)WHBtO6B*WZ)i*>!uJA>TIGv%1~+Tg>s1ZcHB|mE5gp2~9q^%gpoI zDWGc3j2prCEsvT>&9OeGCJo{J&5u2+`T^#LnKe#mvXDi_Q9d}`n;m<@aQ5%rXCY%bLUt*ACW1T+Exikqm$2GM4Hf&UC2g4u@fMwvQ%;+L&-Ye% z{I`cH>xJg)2TcX+t;3BYC#3Qu>S_Yf^_6pKloHk7|4G7xZ8#$X#V-hYI}6;nrWmkI z5fL0_rQCOy!uUK?NUGNbnLgyxj^LO~_lrofjCpRfpwyQVpR-@9g&s}3a&Fn$64dM@ zQH_++k6tQ(R~#+2-cZdt;s{WDNmHFaH7+KDlEB@g3cku^(`W+1H^7FiAwqX|EcQ*U z8qHk|hV#>8iK0h4-^miJIzB8DjIgi3by)I~vu}mm7GIYy>Z^OUP})9xx@wj=)o#$! z9E>qztl9QoFQZTQcLL<^^sr}?-ZHM7+Gl}N{U#lA`)JO)uVw3?Hox#Nu^E1O_7*8Q z#8Jks|3<~Im_QH z?v%75;mFh`_r^^R5Vv`pEE23y^VSxck1Z#dJ}ua>KVhLhf}|6Ia>jzC?7R@ZnA~Zp(03 z^cf5O{cA&;wy7uqHae37cwu((7i5@afjx55p&&>IewvGe{!v;t(z8Jp`eSX5Ce=iK zB>2~AO>nuOvtsug1lU@Z)EWwZ-TAZj4|ZXY#I-5<@k4O#7vm0Vw^lle#yT2DbhGqk zzq+He#%pCDZ|O@dwxm;pQ&AUPag6`WKxa6W?Rujf4?$g5;u>B9tE*8#`Rw1mWUSdY z`lO$Klo`lYrK+^~6D7U?0u|y(h@%r05ghvK=`;XTH1B1tBzP zZMm|$2(o?Nn}0|__T6>e$`h2S7BuLj!z|9wZ3J3He#OPqLFCbC8+2oD|G#7ohvo`< z4duGfLQk8*wvlwNxv@IcS>R^tvgoNK`az=I;Hwf=8Xdq7#!%hLg|?j(1En*nGM3c;dPA5#qOw~~a-l>j7st&X3W6}H;lSw82mcayVLwqNYx z<3@2VWh62YAG3w$qAf=ayuD6(>tW4U+Y}J$`U* zn8bfF{;4a19k?$OcT9Ci3LEJ*WydxX=r_gXIaf*)ZdXSIv93n<)dV3&`SBj=)OC|5 zJ=G`YfnFY;6&z2-Q=)XYUZO@qjO1wL0NPal@B*cL>oSU3RQduDF>M^(4{xG3PWVYsEB8WWDM zq>}S-IKbZcNfvBE;VYWR?Iqh9mt&)DCE#SrttoI|KT}>cK3L$w`eKE>=dWIGe$T!F z2Jj^#5()j)SmUEbnu<-u$hNc^OiKN}$9Qr2A>g##$HQ|`3(OkSc9mTbd1vF;{o4LUbv zGTD!`^>Pn@)Ht~LMyjdGquM6@^RL^QF{~zjr&pSD+f-^g);+2mu`51{D5?H-Wt_Dc zClBMHz8DJ*HE*q4qMG_e7R(B?3IXPKHD;W^3KAI20lHe$Apdf^)%xlPjE{?~$~@TW z-DI^Eaf+1MF6dHVEjV>dgU&GP1}U72ErVC6%14+n89UgOhXJC5p)N`ODDN;9;W&&e zwYj~DvT#&U1H89F_C>#1GTx0p-hV}7oR}|!tV(6d`&ELJ#nERI<$e&lM-pGBcDCo4 zUk-}!x1=lhh~8MUoVe~k!65u3+wsRm8Ulh>p6^dc6y3YDOB*&Gv9yToa@M{T75pD) zU-#vE4^vuSG3VX=xjy>(#tMiCdkDwzp9(!gbM_J#=T;?;{Gh(0; zMb`Rt!pv1NN6fLSfIt_S+n)K2WIq{RY zyE8l7j8tDm8mJCa`F&C?+2kq(JET}RepdcriEot_qMt|>5oC9J|Gh?o06td1f;+~U zJszsIu@G*pn6DSt<44kvL<$Sja|fQwZG~5^Uxy)>+$p1Y^v93?uvHe?o_!u_?Zcx& zkI#!!lRg4RmZ695pDP|(@z*b*>FEIa6gY@L1+E3v)(IReTG5why zeU4s-G`;s42Ko^csa8M(0mV{>i^doS)9+u~*5MIV!P^005t7ba9}!X!eILSNeN7$@cI>qek7>U(yNJ)V>)P-q z>dQtWCx1V|y^RPGiL%zI92h?`(A@l}d%@x!P%_?nC#Gn7g;0c~b2b1Ayi*We=>3*1 z{>PW0jhVSAMWQvv^V22PZKS9Krb~HY-9KjPUw`>n>n&O@`V*JVmgbM|$z_ue>8-?< zazP{iNlop%Ht_FWFd|W>YFoy9Nj)5Q5XiQ)qBr*d?(j--I(peb-iU%c-3pu zr!tiFPj!08IuUN61|k*xCi}#@ncd}$WKe;8YfAI;x9%Y>FtRx@opzlRhcoFHeyWJ) z*TUGtXy&jZC)yDe25~~)^ryOU`~Xz7=)R(S58$CmwX&~J6o-(xB8VLvbuYWQHkw{E zCW>Z~xgm%R!$r7j{5ZVc0K30yvctW>ENM2^H9ZHr*5g*moZ^m- zmEG@{T9Rx9rFSt}aIC5?)Vpq>C#3A&H z(8#=bOPcoaO*+angm!Uk<_2}S*{9f|XQ^Dh0q$cM#JslBFM4-&bp|L^YpG20OL4>2 z^H+QOW0WMFKMO6o35lIf9E~^EvX;UM!o5tq>NU7=4*tUs9De!&FI#WKv%g4BL)n3U zD8m|$t(r*K@ifF3#qDSwl;3-&rZPi073J>JFBQjaFivweC@XEss$6}oyM^1;x*Y$j z`eFNM8lj(jaNX_6%fykqsMgXCNUSX=G%ow}okY3z_fN&H?&tMEOece#pWeDh@i4@R z#cz533)-n)RrJb1)LL)6RVgxj?DM22tRtoxR6#;@R9$UmG`xnCFZuO3A_~z>Y?uR8 zH=eUNExWZO7p1Kw10x8hs(p1K>b_H9R>oPZwDX&%3R9@sigUY#TB#2S-$C;f79TIV zRJ!aNi%awkUkzhRj~)5)=w1$AusTZ)E$PFJhIGyJhTlEXH6Qg^zemv*`)=*lkE9qd-5Fsv2j#+)j8 zI0TiI;#NNq7@q{DSh{|=pzxkh8y%CIa7=`RV(IDrH0FU=JafmKe1=)1l-t}K;|x0x z_j{O?-@eexDYyx<-rZgLRGu2O_itv;BXlFarrq?6(z3D`sF34`a)33Yi_J`A>vcXi z%p8_PbYY^ZbI#l~=`jR%tj((zk6j-M0AK znDNwWQ&T-`$@PNbKZrb>;hqLFe-TM|8x5ZDrzUoFh&A61f^C`toNB&kV=GQJ)oUAUJXOkf}SMqR--^D!HqOlMTAN}N2v%<5i? zm_H%P7Uvz!tQG{Mo{L6dmKLqq*)d~|72a+Pd6V6?UVgqTiB=g9T_#shQ?jG=4;$uNWfv-P zIQtQ#Zn6$D;HK8!QqI|1?=>PXm}10e9EgE5z~bPX^KKe0v~umFGo#y_EWIHRZuyfS z(be)kS3q6kb&)&!ga(jT9Q&FQ zq_w+cm6~>rrdXB#+2)-M(1oWbPh}`J;v@eeKhg88!8mHN1ZNr5V7kxMy>vYZR2CkyxfK9cnMym&dmkPcCgp*&l@vT4P2qe%k!% z$^T(XRM)eRYmYk$fBQt`LrgcYN=DVQrNDPGV7KNl(RRkmGl!!HZ#SmFUPb^E&4QvP zGgOQ6jQWlq2>mPcmm(=3&T4NTJA|Uy1fe)-p?CC}czxB6Ny{526W5i0WQyrnLKN@B zCI8u_^qK=7+=wpNkilOto*KC?4%gWY;ujT3fX>4`R%fIh*HNfq{fCvy_5@QPc`qD_ z==4;Ic#F%-sn|of;FqPPU*()Z_7k!|LUYpgN8+h^X4D9X|2;+4@^QKH01-t&;|Wy_ z%=mi{;Xv~Uf$W|SJo{PXMPRH`6xn_vxsv(WhC2+(up5vC_q+I{r(-oKS>B6O%s9{0 zmk7t)r^I*A-Bf5>8CZ9xipJc{vm7fkvWX+^#KUmlDeI`Ka>Ti~wQbD101@YTOS z?7gS7U3^UEqvhq>z{HJRrtBUeZ=-^JfeXa`pV>r3v8FkkCDw_O<=HR{?fh;{{ez~Y%Uw0Xoe|g8q zJI9nS8tD?mCGn;VcO%dJ_6;fe0Sf*g~^mrGR`xaX=n9{`dkTx=Qe*H(^(}a0mTv(+rXgiwhI~qh} z8iN*onO8p?v$8Ud?mO_VKxDkI-hLRxW>(KfotMv=?;*&lWnTSjy{Mrpefz!r{|5TG zFGqP?jb@w=D65ZkT|Hg&I5W1aUr|?KUHoSK|G4Lpimz--@hQ`yKf6NqO^SyXy~^3R zwNi4RB`Hfj)2dqT7&RxYea4l7(35hdx9w!+`!U{Pb=#2?RB*hi(U~7Pa~V$Szf#Az zm-Ljm+iO)AlLEW7w4y1ef$2Bh*KAv2i(3T;F#MuexPNI1Q1ox{ZypIw&{rn2YEi=q z54`C$w{V4#G9CNU@k1j6W38IANzoYBO25&4_S?m}1lX1p`mhG?WG&;EP$rHRpS{6l zDJ@vCmy7GD?i8v3l*%^6$WM;)-xvC4HmO3rL} zW_6vsm{VjE1}xyg*Klf_v(-{X_uh1WuJft0r_APhh6-*~QEX9bWBb|ehhMqq`?!!c zzgg}vi+fO+i|eP8$%TogB%-JXzxH?W;P!d{jg)9r%aE~7O#p_DZ|!UC;u68bK6 z!^%oibWO&+S(v=JBN-xfDAsJ`ENj74RL?DXx=x2UC}(tFCJjbe45=cKkr zej7|+4KKBJ;7ryZDBTDXeCxtda+2EH$9JeP!k$;#eXIRqob+MOv8#pO%*uB`f6SCd zrbfeqJ+G4W6Qxt5C!62wO*b|F!zvkPW@Z5zSuVG&%H6zdNZiHXzU|4Tj@9bk=IgAx zc3)9o;uNY&d|~=?PDqOVxlvnXqR7#%W{PX5uG6+g?Z!nng1y2B{NbxAVPBY}ep+Z% z`-+#nqobM3X_Ck8x*5ghyJ>VYsMD!~mgMjj$6Z5=C!Q{e=S(z^ zYlZXh;5*~_z_zGEtk~owReUe?A1%2bf(!%kPzx}Zl zG$Rfj;=PzZedA^C+J!nGX%Pzr1Qi6o=pSTv_PaaR@>oW;lqJUWSL$${j~UadMQj~k z@%Vw(JJK!S+4IV@%lGa|zGZ7|PTt)c`eu-|X`E%tkn~$?YpBIrHLJrn(74OAT0#|}!=yxYnoV3;bi7@dLr!GH&%P!zAWdzh)< zZOZp^SKUACgVG2X0@o$Tzcs}6J!Yqi7Y#lTPz33zwt6&WF3qb!APlK?PD_YL+y<{ z5t%Y-V+l{8`XQAm|2KfhvvEC_dn_Ya#wD%V=@``{(A(+hg z^jJ^a9v;@B&JPv26|eRYikoiYUQ_bwuvLSA(%gCL(;whzg~sYd+7_Dm1T=@whYoeM zBm3&HaXu3|6?no75LZj(RV7{G=OWUPuOP`O%7+22-iIJ42&#qX zOv~CFQ89tE>|NZxp4s1{z7W6IYZSW&=`aVKpQf%Os)atVj!wP4ht#1rZsn48E)%g^ zCjp6rgWFNDMAR7=KG3`~=({X)-o^z7rgA^r?43@nSFU{Hb7%us?C;^|1+fIV?gnl8 z(Jwy?n>OQ{h~82Qi}^ZRC6_OIPnF_<1L*543hd1y#BQ{+*w(w?<-Zi+Q~ zVm1E^*jlgW?K-OTmh~_{IL~S2B7($3soiXn)Gz%Rejq z6SbBjmLQ4;KwW1Ie(}6QaYLK|W+3YFPOg`1hLSlimi^Z}oBb~G0aqU%5WlQkevrJB ze1w5T_)dim{2<-ry9PxcN+>sy1Cr_Km2(qly@d5z9X)I`@0S~Igov#nmO{to;g%fN z6E2tKv}L;OQv!dE&rdhx5i}iaI0TtmF2Oq_vE<7TqUAQVU zh=DD4a&Z?Qe}8`s*I3IO#(m;foWwURnarM(&LOl zC;pI^o^Og?v2^VZe4a8BusMy6)C(}{`u`gH?x-f#t$jh!qX-I!h#;Ue>4MU$(u-87 zp?85K^j<}pH0eEnbVxv2LdPS$*U&=mHJH%*mvipbZM|rO3gQO_?H~F{*I`B;n z=$bSq{08#Jm?#NvMBb^Ry&7XOMS^6JG}ufr*TA$%4)kaztc@k*ZAq&vQTV1mdDqo%L8nauDcE;luaHWw}#9mG?YhLY^jz$Yd{ zEtXpV!M2adGe^R{ybl_LvK}Xeh#$#-qVEC?%-mhqi19?LdZUC7S3zym7~TYX->u#v z(}x`IQ%ccdljTm}KV3?V>f;r?DfGX#5=@9+w$f%czbPQ<7w;Fk;$)&=%uwi>o{s44 z>+jsQyF_LNNT}Wnag#H;vd_+6^GUlQ!it%Atf{@+FR*Ld*tF^9ZNuWm7DAuWz3R!7 zREeH)VJdj}1$@BH5mRLw+c_oppi^!zGo~%R(wC0^sin*GR&CRk?)1a5bSt)Selk-n zKf~#V2$KYzJ6;z&Ml$NfkTr^#67=eK9kiCRsPm%Eky2^yI#6GA4>?xJ@Q$>8D9*vw z=iu|ACunBK94H&83kY$f-AaCvMmsRmJt?E!mWEEzmmfma@c6)D-%eD3N{kpiAB%)R zvz-^!1~ltlXrTyC?A|jpl4$NIklNJoTbEin5|DQTm%ePy=y#A>Yb6ELFoZ!(ML#gNa6ySl`DJeXw3M?x&WJgKM@_;PtuO z@4SS>ztw0#iYHUNUtC%Isy$WD+}b+iE^vkgP(m9dlw zaI!(ZnoqBr($rWSMrA)#GqyY59*WWnNdcaP zQw|0~Dj~qWuHgHB^X=c8!bXW#xY?p(;iNOxG`8&^ErBq2RaJ!@TMiecd>17gfxmiOyBy8ogUqXYI_O$G}rB%a~IG`Z{RAAiu^B4h!kB2qMVUe-jdHIKU!xI z|3T+}4gX)8cXWeu=%(x#ws{^KO@3lTF_i>!45HASr%e|h6sbLv-`JcwKvB54Y;iLZ zucTHGx)GM8< z*-pK_#>~fSZFhNP{_=JI8qU}t;B|r=ammwCmVVOE8;Meg3_wpmL>EC*M<0#a!MT>P z7!x*7H0t_WK>DOTC(Nan>T?Fk5mRgkQlyI{b1o0IJKYjhSc3$v!N0@_H-)02rcOFa zP`&Z3`l7GGP@aX_@PRP{%Ut`uLTQUcXyi|L>HPgm2t**z?adS?Trb6Ymr=8VL%_8$ z(&+rWxCPjgk#kV=U#5E-+3=}O6eK+QF|;vzRfG(Z%w}I$E8CDrDH^?*y{y*JRFD!* zyJ=$7C3aM0M>69IVoTI!Xi~h0UVI*K3@TfPJJ~)3WgxK&%8DOgmgz?F48m-#g&hCZ z@?ZMiy0eXDj%nbG_>&8b&gpY?#QXRsNvaX=qUOgvJ6|P%1xg1t#}_iD{kO}Dx7>4! zI#@JedZI^gXKf(Lu0{yoOY_+zjIlgv^wY?cSa}E^Jt%{$8Z)!nS|Z5npnRsL@e^$v z?h$0`cFGC>jqVyxwd?}|XTT7Zwz<{O`Kd&MP5d=X-86N&RpgzX4+&2}NnymVVv^Y& zvEO9>%J?Xb#xq(`yHI{A5Z6=wBkKeD!R#poL<&H$rX^Ojp0bf(X6j}hH3Gyo(=xIX zRO6z2qiWVf`N)H`t3Qm&=F>m0f6K@BI!$(R)u0QUogPu)d2#ma+O=mw@=~wg0X9~x zqjK|@%dJ&=R1*$vgpLDP)2+a|$(Nxu2B7R3adycCt&y~zg9G8IY35x1#1>;CLea_$ zY{10UAzYnysx>25JDs#FtdW(aBaF{VVmg(!dPqzW|7M@^_muwgtRCF;7!}uY=1==)@;H+zy)Vb&BR(qmKJ~Gp*h~5iK_Bn;SWxT}1 zR$M*cG_qJjl+B55mhH908*YFEA_l~UNP=HC#N3=IHNM1b@!6nMSncxiLQ&+mopyzZ z(?+Ot*)SX2{91*SzbT*b_6=`JK>;cCQiduab|^{?L2jXIe3nS?J}iIX#XEoZKJcUd z$`hJ=F?!G*z!K~nUUfo$!iDX96mSG5R_HVvyVriudYd(z9@FHh72reh)c(~hR~>BH zumLY7swG!PDf)Uw{0LB5_aPmqFY2Rl2_pxpPpB+eld2LGJ@xb#-_RL1e9K;-#CTP< zkB`8}M}ax0P)EgY6%~%|Y!njHq}*#Hm^xp8J;Y$HEnYheg#vuf3C4QUal4;UHK@A< z_^*&>$mK5+Qop?$n#o`x_UZM2Gm2V%Fx)p!eo8ZTc*)44z41MM9dJtiaBhsfbqC1M z-Z1FUN}p(w0%$2VQv#@w{5DakS?=F4H@~6_Q_!FSUkygA^L<$+T=%eY=;G(bNm=L1 z_Dhljz?*?TsW7n{Iz|P)WwRqfSyMqE$6Zjs*H1{vyJ_d3hh~s1_z+M^_u-Eqj`!jc zscoa~*5d2rGcxhsc#C{$KK>O3U#w(vTp#Uy&9pR^6c=&t1a`{3f|q5 z@wtK41mdm|%QYYU$$3VSNzf?o5+~C-JR8a-Fq9*6YwLuS~T0{S*DIpT$)=G%EPgZj3fuYCkQ z)k7^aZV&XM2yE!>^{WYR>aBhZGdztNVe>G{o@yF7dY!q>mu<%>F%3d%l(@ARB(@$U zrcXeH-KeYjBoBalL1Vw3~b0)&GRYWxEC!+Icz9Fd;xQG%s2eFi** zQy(g7C7?#kft`k+(svhbOZ6h8R4*#6tPY}^_C@a_^3o<7Wp>XROtA+R46!>P2YARo zlKH>JKiZ{9viJA!*g`Tq~8!N;r+6e6ekGBKwoKfC~BAy z5k%&g71pZ1;h0cHg1Na&|D>LE?sE_P$7TNx>EVfi>*y*qXkshr$c>xbx13{mFH6=t z#x%)Nu%5u%MA36rcHoD)K^RKCZ5trwwjkL&S)^BKYh?uZ%J>@r=(F4>vRsY{G%FQx zB_lqK3CtlSis|I4=9q1E(>JY2=+?quTWvi)rd#zmRoZea!bQ4NTxVH0nv6ZzDzzr@ zn`2(w^vv~4yVyNQV(9Q@CbekoQaGfSN zntBmGbaNgxX=qzP7W-KDsB$v*dWl$Tb)F2P_ZN?qIR z#O(&WeQH^BC(wOVXI^wivVvpGaH&zq&#kY8W}_}`R1pTK*aMZl0=G5lyDh@?Yk9^s zjK1IWYhj-Kd@|R5{4Qa#9UATmAssGWWJp z943&}YnEg%+^Jj>vCXu|<{&`?ef;Ei71K(*LBvbZjIWr;j?4BhVd%icl^)wJ*ImOu zbMA@{S8DI4H^y(?u{Hz3`dC(O>#*+e?3-k+3wp62$p_n~rk+LXbH8!`aGT)XW_K$h zq(wU^e|wxDH|FQ8_8Z$jm+{|?YJ0xIHS)kPHu_0Dix_G8$qPQQ5%`o2K!cUyhi4^r zAt#4)0Y9{xdwdcfQ%9^2#N_~i-TxYj3`0Hptf14nhUo;m(Izr+?J*{O)7t;ydk}su zz9q;GYf>HB`Z~Xg-*@@hyZFlg*cl$EPL2mGiLB<$zyi~um z`i81mc$$~1<6Z=~;o5FI2G4u<7p#+-L7Q2RKZp)nywigh2EHn8Uy!IPRH(fFi?Xlbn~2Xn zVGH{p>Qm`>g%116dTei0t$-{PiXh+k5_0{1D4UKEn=B%U+rFdbMF10p@rmuMIAobeE|!^ zW*=s>v3K)O{jtt~4a6)E&8}F~6QDs<3@LWki5tUSY1arnPHHU+MksSoCd`$JC`{o00?{9ak9Uy44Q&!u zXQddFkFtM3&ukQ>$k+4LI4^XwR_NeG$-We)Gz@F8J0+HP3F_^>XJpmirT!{xT)k@B zAaT#`1kwsTaZP}p1hi`1c5lG)^ZROcS?VK7A!NGutlp|DPw_V@Gs*hkoz&YFj2=x+ z?ap<1jXFKsT7YhZ>DOzG$_pB`#wyOeRZN^rRroOe^9*Sk2O$|^<>g`1TtRGSf4|3H zbntfQu|jv>r0NOmd+I;LPsAkwHup-?x!3&o&}td%H_Nnfw`5 zn{HwEa8NYvx(#GMj*HoD*3k?U65+gN-?qNwkrJ_<;mH)L?6ybk_X^*H znPa7ar;}~zok8WrRUef<1~=k)ps{SROv2jn6)J5z9cPC{f-$IYJxI_t*_*T zD}I))7iJo2UuhD$9-kIM$R;Om*M1=$8w?6Xky6QARPki3%BXC+`1U?_y8$x^MsRXd z>r($DVfzdTTY-%_v(iJndnG4U;w}S3w+U+4n?6_b4QnyP5Xx&FT?n69tqe)#EUL-- zbhqQTZng7O`-beqxMd5F2EY`YFSd`NB9>U}A_HFbk(X3~*mV&f>Wm84aZwT^A=CjM0638Wp2 zCCuEQrDKavgmhX|!R`imFVJ{vlaXm3Yi1_$A-h$lZyalCwK=(z zy4^jSAe!7ZbN7mBb2FzaGH1(Vs=FO6J{TH-`sX6dm$B2i0lZM7gxL|X=g}^6EKe<@ zxFx?)RgI=Mv%bw?I)*fUpR2V6V*WR66v-ca}<8{eDl4}f_$xW}He!taxgDjGy zQieI!$IykKKUQ{CbHM!0FM_y+&xQ|-k$xZk0($UCwif)Z=clR=O-gsotqn5Vu|}8yzKQrC6LrK{%={am7>|Qj5lwlQ-Q}A8Me_ zkYVmH@ZH2pxBkl5H1&WzJHa81GZ$y*8RF#qe#FzS9u7n zkK*O-@#M7gBM&g&d|bbTlGtpmaFQJOTZ>cQQ-frxq_}bA9&5)j@{Z|;q1_k`VGGrR z-h2TS<5g2FPCqUVLNm17^N>;1Z;iD33M2k!oHtzzakXVL{x%poX2zkH4S>dcDtPV3 z5)>Riww&yD-PJz@(MU;Z)n9>YW7)+kxJcrllb?Kv`%?>q>*`Pj%V@(!l%X3lFGM+oq9 zu=9|KzCn6yS>r{WzWN?BzbOV_C^!p$OA~r8ED57Y^(8E+g3`leKla>x8Jat~B*X4Y zUUCEnC*ZQ?R#*#_jzJ|ozhQ4DN?e|Q+kRXKZ`>tXMJU=+D#MY5h;BZ(Cep4v_sXQ~ zqcGLk_gftdu?K6b?MiZ`%0X1~^sP^S>Fkeq*1-`Gt_*I^pu|nBz7;E^>bOw$dG|?j z?ql;ybI}g;Wj1Ak1SHn_E4SY}sHIYJ!wz(_6|;I^UmAk74!k?0+>lD`a5x&d2NjL7 z%j3h>CEEnTm@;TXrSLnU{pt`U&ptZw+Ip04>(^Rpr) zvk1@Sai^3L!Ozi*r;MBQAlt{PnzOnJE?=5Y7GopM9p!n59DS-~`_gmQB+#n2&HbFK zo6Qm!C%ric9&>{vceF}xIc_N@NVvY95Mqsi270~eOY(Mj_Wi|p^?7b3*U!i~D+=FJ zU@Kru*YnwUInkXR&!-zPM(N>QTm$@qu3}SJ;`Fyd@>d3bM0~_&_)J(`8#gz8dj;YA z=0qtxq7c%p%-&i;27pxvIYD5((bcwWS!CU1ebb-ga=_iyZ5vbQ>HNl^MllcYmNEJv zhHusSA)gv=ujOizI(DyI5xaQIsH^XYdsFH)IYM^n!cmIxbBm#sZ^!#&|6*)F?ph@e zR$T_vRK=hej=T8;zOu5u4T0^1!xkD$kiqA$lGvO$KGYu$ROqko20Y5D(Dt(U9*j-Y zNpfT4<4#POR3nWvkL&A?73dwJe|pUt&2PJXonx(cot_CZT#J1l-Qda1TEHpY#v&?) zN>#5Y@Y6aZ>Q(qAi$D1^+ftF&<;}@X;T-o{-~ zf;rwu`}RtvvSBR0rR?a-J;kKd>nA#jnbR}FTu+fDaf@OB(nz)w3#9M~va5urVm(4I z9u=rIIX+B(Yq61(0c1d4r6?aWq|L)~I2FqtA+W&C_QlJQ>CiI|*eqM4+@ZNp=lL}% z9N>~6Ig*vIB{)~nXK;HtLB`4anXa}NY}KMZKe(1A!^}M@p?#U=r&U@r1i(ALIF~Fy zeR@^4IxQQyDL)$-M0IYkk)$Z59!T0Td}VOk@9#o432xY_H8-j1bG@jX*TmF$le8W~ zQ?vTA-ZVTOeH5%2vxsi>j75>&+I`0~A$BWFGjWk{;mvYbZr{>Rjr z7jIVbfo*As8?%PViQ`OVJuEJ1BPsU+KcQKg3oOEM?{bu?#=YjvVRa!<6mz#s{8^g55r{Mmm?u53ky8U`$edfJn?%9r$xCa(H2Th82406Nydp&Gl$>Mr}Lw<4`TmcI2=i^|P`^$kMZ z6rr@36h%tiM=%wY71i&t6aKQP2)$zd0@=~{y$3h=9=v!4LTU$u)(t+*U#zyMXB+~# z9f}#2#f&L3Z1t3!`<;adx9&RiClyZSa`F3Na}}O5Q(`?E;j6qq&HS5AfdAv~b<&KV z0WI=jH`4`~Q_5?CHSM$x&Kq(`orQ|LE!gSeoW!~;cP{E%)c0%>xuO z>L+lCN`;04kKFeX@9-x}JnidlB)336Nx&Ahj)bBPxQzA(Um1t-o9}|jH2*BZ$Y^Xp zMGnCO>y1p>E>Yqxrcy*4gRYBnz_`T1mt15i+D^~6K4QJQnF(zmaA#cq7qdCE-=OWZ z;WqJ4)3R^yzkJ=A6YDLeB6A?nUsxMGg5&WCmDmAHBw!Vfy@;kK&dHE6^PuOrip#Am zgmP@ppJ^m9cl52%^J3a9(jOMmM(*?DN3{SMQ&xHE=zNCR`2HD$RAQ@Q^b!x?Ba)A7 zP*j|GO|zlD&|R}}t7B9YQ_xP@?W4(1{&&#>X*Rx7X{_bY zIOp)7D3n&l;J)GzC;r@lyaZ^| zv?TQ+R`D4bLeu6&5?Jf#TtB_Dd$FdjXttYhQ;at*Awq$KgYm#eYX%T2&tddr0aLQ9 zN5=;7Z$gav@bUM8k8_wUg?+L@e z!l^>(sur8;rg#|wghe;LHg&mDNZPMQkA*Ha5qGgR)4@x18@aN+)8m0IP z#=mIuCrA80fB$O)enNd*Pt4k_TAD6x^oIDsqRq3Dje^X6b7{zkfVeO7$hPJ4s3$+I z;y(#;-#+|Xv`QZ*Z877(#x%y}nQpU<$1>IFr7)MWg3Q}grPu#!;Qu9&wH;mt)g_uR zt~RiWXHvgkfGfru+kBbRti5~LObBLPSyt>JWHybvL&8*=hEV2U$Y>nMMq3}D} zCJU$1EXY;*aVt{@L)p_UmUAj_hD!6nqV6GjFfDzHlg|Fzk*ILC z17hOO=r!I~r`fTlRYOi^Uq<`Y4vHx%FlMcQ3Hk`{V*uDsMAM}}#J2o|14t{BV!0_lG4c1P7?^^3M<7K-Jq~#K)u)LMuDh!ZPl7} zdxaR3XX=c-+_lIJg@ozO;3E>3{~NO-!uOosWO1SDQaBvf8uLWP_4#f=E;YD{&ta)g dYtOM)cod<5#rK9<2(ICN^3uvurEiQs{13viLXrRg literal 0 HcmV?d00001 diff --git a/doc/source/virtual-cluster/key-concepts.rst b/doc/source/virtual-cluster/key-concepts.rst new file mode 100644 index 000000000000..dd56203339a3 --- /dev/null +++ b/doc/source/virtual-cluster/key-concepts.rst @@ -0,0 +1,64 @@ +Key Concepts +============ + +.. _virtual-cluster-key-concepts: + +This page introduces key concepts for Ray virtual clusters: + +.. contents:: + :local: + +Virtual Cluster +--------------- +**Virtual cluster** provides logical resource isolation, enables multi-tenancy by allowing +Ray cluster admins to easily manage multiple workloads on the same Ray cluster. + +A virtual cluster consists of a group of heterogeneous worker nodes and a collection of +child clusters, including :ref:`divisible cluster ` +and :ref:`indivisible cluster `: + +.. image:: images/ray-virtual-cluster.png + :align: center + :width: 800px + +Users can submit jobs for execution on the Ray virtual cluster through the HTTP API or +by using command-line tools. +See :ref:`Virtual Cluster Management API ` +and :ref:`Virtual Cluster CLI ` for more information. + +.. _virtual-cluster-divisible-cluster: + +Divisible Cluster +----------------- +**Divisible clusters** are clusters that can be divided into smaller clusters. Each child +cluster can be managed independently and can be scaled up or down based on the workload. + +.. _virtual-cluster-indivisible-cluster: + +Indivisible Cluster +------------------- +**Indivisible clusters** are clusters that cannot be divided into smaller clusters. Jobs +can be submitted to the indivisible cluster directly. + +.. _virtual-cluster-primary-cluster: + +Primary Cluster +--------------- +**Primary cluster** is, to some extent, equivalent to a Ray Cluster, encompassing all worker +nodes within the Ray Cluster. The Primary Cluster is a Divisible Cluster, capable of +creating Logical Clusters or Job Clusters. + +.. _virtual-cluster-logical-cluster: + +Logical Cluster +--------------- +**Logical Cluster** consists of two types of Virtual Clusters: Indivisible and Divisible + +- `Indivisible Cluster` cannot be further divided into sub-virtual clusters. Ray Jobs can be directly submitted to this kind of virtual cluster. One Indivisible Cluster can execute 1 to N job's workloads. All jobs are co-located, meaning that tasks from multiple jobs can run on the same node. +- `Divisible Cluster` on the other hand, can be further divided into sub-virtual clusters. When a job is submitted to a Divisible Cluster, the dashboard will automatically create a Job Cluster(a special type of Indivisible Cluster) for this job. (Note: Jobs submitting to a Divisible Cluster must provide the expected ReplicaSet information, including the required number of each NodeType). + +.. _virtual-cluster-job-cluster: + +Job Cluster +----------- +**Job Cluster** is a special type of Indivisible Cluster, which cannot be further subdivided into smaller virtual clusters. It is automatically created when jobs are submitted to a Divisible Cluster and is destroyed once job is dead and all corresponding tasks finished. It can only execute one job's workerloads. diff --git a/doc/source/virtual-cluster/management.rst b/doc/source/virtual-cluster/management.rst new file mode 100644 index 000000000000..1163fa621091 --- /dev/null +++ b/doc/source/virtual-cluster/management.rst @@ -0,0 +1,168 @@ +Virtual Cluster Management API +============================== + +.. _virtual-cluster-management-api: + +This page introduces the management API of the Ray virtual clusters: + +.. contents:: + :local: + +.. _virtual-cluster-create-or-update-virtual-cluster: + +Create or Update A Virtual Cluster +---------------------------------- + +To create or update a virtual cluster, users can send a POST request to the http endpoint at `/virtual_clusters`. + +**Simple Request:** + +.. code-block:: json + + { + "virtualClusterId":"virtual_cluster_1", // Unique id of the virtual cluster + "divisible":false, // Whether it is a divisible virtual cluster + "replicaSets": { // The node type (same as pod template id) and count that you expect to assign to this virtual cluster + "4c8g":1, + "8c16g":1 + }, + "revision":1734141542694321600 // The timestamp of the virtual cluster's most recent creation/update + } + +**Success Response:** + +.. code-block:: json + + { + "result":true, + "msg":"Virtual cluster created or updated.", + "data":{ + "virtualClusterId":"virtual_cluster_1", + "revision":1734141542694433731, // The timestamp that this creation/update request was enforced in gcs + "nodeInstances": { // The nodes that were actually assigned to this virtual cluster + "033141204224b43e67f01ec314ba45c16892298a23e83c5182eec355":{ // The node id used in gcs + "hostname": "ec2-33-141-204-224.us-west-2.compute.amazonaws.com", + "templateId": "4c8g" + }, + "033159116236f3f382597f5e05cadbc000655f862f389c41072cef73": { + "hostname": "ec2-33-159-116-236.us-west-2.compute.amazonaws.com", + "templateId": "8c16g" + } + } + } + } + +**Error Response:** + +- If there are not enough eligible nodes to be added or removed, the sample reply will be: + +.. code-block:: json + + { + "result":false, + "msg":"Failed to create or update virtual cluster virtual_cluster_1: No enough nodes to add to the virtual cluster. The replica sets that gcs can add at most are shown below. Use it as a suggestion to adjust your request or cluster.", + "data":{ + "virtualClusterId":"virtual_cluster_1", + "replicaSetsToRecommend":{ + "4c8g":1 + } + } + } + +The reply will tell you the replica sets that are allowed to added or removed at most. You can use this information to adjust your request. + +- If you update the virtual cluster with an expired revision, then the sample reply will be: + +.. code-block:: json + + { + "result":false, + "msg":"Failed to create or update virtual cluster virtual_cluster_1: The revision (0) is expired, the latest revision of the virtual cluster virtual_cluster_1 is 1736848949051567437", + "data":{ + "virtualClusterId":"virtual_cluster_1", + "replicaSetsToRecommend":{} + } + } + +Every time you want to update a virtual cluster, do it based on the latest revision (there might be more than one party of interest). The failure message in the reply above tells you the latest revision. You can also get it by accessing the GET API shown below. + +Remove A Virtual Cluster +------------------------ + +Users can remove a virtual cluster by sending a DELETE request to the http endpoint at `/virtual_clusters/{virtual_cluster_id}`. + +**Success Response:** + +.. code-block:: json + + { + "result":true, + "msg":"Virtual cluster virtual_cluster_1 removed.", + "data":{ + "virtualClusterId":"virtual_cluster_1" + } + } + +**Error Response:** + +If there are still jobs running in the virtual cluster, then the sample reply will be: + +.. code-block:: json + + { + "result":false, + "msg":"Failed to remove virtual cluster virtual_cluster_1: The virtual cluster virtual_cluster_1 can not be removed as it is still in use. ", + "data":{ + "virtualClusterId":"virtual_cluster_1" + } + } + +Get Virtual Clusters +-------------------- + +To get the metadata of all virtual clusters, users can send a GET request to the http endpoint at `/virtual_clusters`. + +**Success Response:** + +.. code-block:: json + + { + "result":true, + "msg":"All virtual clusters fetched.", + "data":{ + "virtualClusters":[ + { + "virtualClusterId":"virtual_cluster_1", + "divisible":false, + "isRemoved":false, + "nodeInstances":{ // The nodes assigned to this virtual cluster. + "033141204224b43e67f01ec314ba45c16892298a23e83c5182eec355":{ + "hostname":"ec2-33-141-204-224.us-west-2.compute.amazonaws.com", + "templateId":"4c8g" + }, + "033159116236f3f382597f5e05cadbc000655f862f389c41072cef73":{ + "hostname":"ec2-33-159-116-236.us-west-2.compute.amazonaws.com", + "templateId":"8c16g" + } + }, + "revision":1734141542694433731 // The timestamp of the virtual cluster's most recent creation/update. + }, + { + "virtualClusterId":"virtual_cluster_2", + "divisible":true, + "isRemoved":false, + "nodeInstances":{ + "0331761541565ea3c14fcc158a98e9a6eed9e0c3c6c86fa613ce6738":{ + "hostname":"ec2-33-176-154-156.us-west-2.compute.amazonaws.com", + "templateId":"8c16g" + }, + "0331280722461e5130088465a89bd8262738fbd301ae9ae06e1edf42":{ + "hostname":"ec2-33-128-72-246.us-west-2.compute.amazonaws.com", + "templateId":"4c8g" + } + }, + "revision":1734132897622670263 + } + ] + } + } diff --git a/python/ray/_private/ray_constants.py b/python/ray/_private/ray_constants.py index 18c525412e6e..652eea645000 100644 --- a/python/ray/_private/ray_constants.py +++ b/python/ray/_private/ray_constants.py @@ -118,6 +118,8 @@ def env_set_by_user(key): # the local working_dir and py_modules to be uploaded, or these files might get # garbage collected before the job starts. RAY_RUNTIME_ENV_URI_PIN_EXPIRATION_S_DEFAULT = 10 * 60 +# Environment variable to specify the virtual cluster ID a Ray job belongs to. +RAY_VIRTUAL_CLUSTER_ID_ENV_VAR = "VIRTUAL_CLUSTER_ID" # If set to 1, then `.gitignore` files will not be parsed and loaded into "excludes" # when using a local working_dir or py_modules. RAY_RUNTIME_ENV_IGNORE_GITIGNORE = "RAY_RUNTIME_ENV_IGNORE_GITIGNORE" diff --git a/python/ray/_private/state.py b/python/ray/_private/state.py index ccdc709a4f1b..118fa65f40cf 100644 --- a/python/ray/_private/state.py +++ b/python/ray/_private/state.py @@ -158,7 +158,7 @@ def _gen_actor_info(self, actor_table_data): } return actor_info - def node_table(self): + def node_table(self, virtual_cluster_id=None): """Fetch and parse the Gcs node info table. Returns: @@ -166,7 +166,7 @@ def node_table(self): """ self._check_connected() - return self.global_state_accessor.get_node_table() + return self.global_state_accessor.get_node_table(virtual_cluster_id) def job_table(self): """Fetch and parse the gcs job table. @@ -724,7 +724,7 @@ def update_worker_num_paused_threads(self, worker_id, num_paused_threads_delta): worker_id, num_paused_threads_delta ) - def cluster_resources(self): + def cluster_resources(self, virtual_cluster_id=None): """Get the current total cluster resources. Note that this information can grow stale as nodes are added to or @@ -738,23 +738,25 @@ def cluster_resources(self): # Calculate total resources. total_resources = defaultdict(int) - for node_total_resources in self.total_resources_per_node().values(): + for node_total_resources in self.total_resources_per_node( + virtual_cluster_id + ).values(): for resource_id, value in node_total_resources.items(): total_resources[resource_id] += value return dict(total_resources) - def _live_node_ids(self): + def _live_node_ids(self, virtual_cluster_id=None): """Returns a set of node IDs corresponding to nodes still alive.""" - return set(self.total_resources_per_node().keys()) + return set(self.total_resources_per_node(virtual_cluster_id).keys()) - def available_resources_per_node(self): + def available_resources_per_node(self, virtual_cluster_id=None): """Returns a dictionary mapping node id to available resources.""" self._check_connected() available_resources_by_id = {} all_available_resources = ( - self.global_state_accessor.get_all_available_resources() + self.global_state_accessor.get_all_available_resources(virtual_cluster_id) ) for available_resource in all_available_resources: message = gcs_pb2.AvailableResources.FromString(available_resource) @@ -769,11 +771,15 @@ def available_resources_per_node(self): return available_resources_by_id # returns a dict that maps node_id(hex string) to a dict of {resource_id: capacity} - def total_resources_per_node(self) -> Dict[str, Dict[str, int]]: + def total_resources_per_node( + self, virtual_cluster_id=None + ) -> Dict[str, Dict[str, int]]: self._check_connected() total_resources_by_node = {} - all_total_resources = self.global_state_accessor.get_all_total_resources() + all_total_resources = self.global_state_accessor.get_all_total_resources( + virtual_cluster_id + ) for node_total_resources in all_total_resources: message = gcs_pb2.TotalResources.FromString(node_total_resources) # Calculate total resources for this node. @@ -786,7 +792,7 @@ def total_resources_per_node(self) -> Dict[str, Dict[str, int]]: return total_resources_by_node - def available_resources(self): + def available_resources(self, virtual_cluster_id=None): """Get the current available cluster resources. This is different from `cluster_resources` in that this will return @@ -802,7 +808,9 @@ def available_resources(self): """ self._check_connected() - available_resources_by_id = self.available_resources_per_node() + available_resources_by_id = self.available_resources_per_node( + virtual_cluster_id + ) # Calculate total available resources. total_available_resources = defaultdict(int) @@ -879,13 +887,19 @@ def next_job_id(): @DeveloperAPI @client_mode_hook -def nodes(): +def nodes(virtual_cluster_id=None): """Get a list of the nodes in the cluster (for debugging only). Returns: Information about the Ray clients in the cluster. """ - return state.node_table() + if not virtual_cluster_id: + virtual_cluster_id = ray.get_runtime_context().virtual_cluster_id + elif type(virtual_cluster_id) is not str: + raise TypeError( + f"virtual_cluster_id must be a string, got {type(virtual_cluster_id)}" + ) + return state.node_table(virtual_cluster_id) def workers(): @@ -999,7 +1013,7 @@ def object_transfer_timeline(filename=None): @DeveloperAPI @client_mode_hook -def cluster_resources(): +def cluster_resources(virtual_cluster_id=None): """Get the current total cluster resources. Note that this information can grow stale as nodes are added to or removed @@ -1009,12 +1023,18 @@ def cluster_resources(): A dictionary mapping resource name to the total quantity of that resource in the cluster. """ - return state.cluster_resources() + if not virtual_cluster_id: + virtual_cluster_id = ray.get_runtime_context().virtual_cluster_id + elif type(virtual_cluster_id) is not str: + raise TypeError( + f"virtual_cluster_id must be a string, got {type(virtual_cluster_id)}" + ) + return state.cluster_resources(virtual_cluster_id) @DeveloperAPI @client_mode_hook -def available_resources(): +def available_resources(virtual_cluster_id=None): """Get the current available cluster resources. This is different from `cluster_resources` in that this will return idle @@ -1028,7 +1048,13 @@ def available_resources(): is currently not available (i.e., quantity is 0), it will not be included in this dictionary. """ - return state.available_resources() + if not virtual_cluster_id: + virtual_cluster_id = ray.get_runtime_context().virtual_cluster_id + elif type(virtual_cluster_id) is not str: + raise TypeError( + f"virtual_cluster_id must be a string, got {type(virtual_cluster_id)}" + ) + return state.available_resources(virtual_cluster_id) @DeveloperAPI @@ -1045,7 +1071,7 @@ def available_resources_per_node(): @DeveloperAPI -def total_resources_per_node(): +def total_resources_per_node(virtual_cluster_id=None): """Get the current total resources of each live node. Note that this information can grow stale as tasks start and finish. @@ -1054,6 +1080,12 @@ def total_resources_per_node(): A dictionary mapping node hex id to total resources dictionary. """ + if not virtual_cluster_id: + virtual_cluster_id = ray.get_runtime_context().virtual_cluster_id + elif type(virtual_cluster_id) is not str: + raise TypeError( + f"virtual_cluster_id must be a string, got {type(virtual_cluster_id)}" + ) return state.total_resources_per_node() diff --git a/python/ray/_private/worker.py b/python/ray/_private/worker.py index 225e904ae475..9e1b35139013 100644 --- a/python/ray/_private/worker.py +++ b/python/ray/_private/worker.py @@ -570,6 +570,10 @@ def current_cluster_and_job(self): assert isinstance(self.current_job_id, ray.JobID) return self.node.cluster_id, self.current_job_id + @property + def current_virtual_cluster_id(self): + return os.environ.get(ray_constants.RAY_VIRTUAL_CLUSTER_ID_ENV_VAR, "") + @property def runtime_env(self): """Get the runtime env in json format""" diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 110172b6ce97..ec5566a8d22e 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -144,6 +144,7 @@ from ray.includes.unique_ids cimport ( CObjectID, CPlacementGroupID, ObjectIDIndexType, + CVirtualClusterID, ) from ray.includes.libcoreworker cimport ( ActorHandleSharedPtr, diff --git a/python/ray/dashboard/modules/job/cli.py b/python/ray/dashboard/modules/job/cli.py index e61dceee53f7..3ddd14ff1a96 100644 --- a/python/ray/dashboard/modules/job/cli.py +++ b/python/ray/dashboard/modules/job/cli.py @@ -133,6 +133,20 @@ def job_cli_group(): "If not provided, one will be generated." ), ) +@click.option( + "--virtual-cluster-id", + type=str, + default=None, + required=False, + help=("Virtual Cluster ID to specify for the job. "), +) +@click.option( + "--replica-sets", + type=str, + default=None, + required=False, + help=("Replica Sets required by the job. "), +) @click.option( "--runtime-env", type=str, @@ -210,6 +224,8 @@ def submit( address: Optional[str], job_id: Optional[str], submission_id: Optional[str], + virtual_cluster_id: Optional[str], + replica_sets: Optional[str], runtime_env: Optional[str], runtime_env_json: Optional[str], metadata_json: Optional[str], @@ -252,6 +268,8 @@ def submit( address=address, job_id=submission_id, submission_id=submission_id, + virtual_cluster_id=virtual_cluster_id, + replica_sets=replica_sets, runtime_env=runtime_env, runtime_env_json=runtime_env_json, metadata_json=metadata_json, @@ -273,9 +291,13 @@ def submit( runtime_env_json=runtime_env_json, working_dir=working_dir, ) + if replica_sets is not None: + replica_sets = json.loads(replica_sets) job_id = client.submit_job( entrypoint=list2cmdline(entrypoint), submission_id=submission_id, + virtual_cluster_id=virtual_cluster_id, + replica_sets=replica_sets, runtime_env=final_runtime_env, metadata=metadata_json, entrypoint_num_cpus=entrypoint_num_cpus, diff --git a/python/ray/dashboard/modules/job/common.py b/python/ray/dashboard/modules/job/common.py index b928baab3aa2..fe37ab9a4a54 100644 --- a/python/ray/dashboard/modules/job/common.py +++ b/python/ray/dashboard/modules/job/common.py @@ -9,12 +9,17 @@ from ray._private import ray_constants from ray._private.event.export_event_logger import get_export_event_logger -from ray._private.gcs_utils import GcsAioClient +from ray._private.gcs_utils import GcsAioClient, GcsChannel from ray._private.runtime_env.packaging import parse_uri +from ray.core.generated import gcs_service_pb2_grpc from ray.core.generated.export_event_pb2 import ExportEvent from ray.core.generated.export_submission_job_event_pb2 import ( ExportSubmissionJobEventData, ) +from ray.core.generated.gcs_service_pb2 import ( + CreateJobClusterRequest, + RemoveVirtualClusterRequest, +) from ray.util.annotations import PublicAPI # NOTE(edoakes): these constants should be considered a public API because @@ -104,6 +109,8 @@ class JobInfo: #: The driver process exit code after the driver executed. Return None if driver #: doesn't finish executing driver_exit_code: Optional[int] = None + #: The job cluster this job belongs to. + job_cluster_id: Optional[str] = None def __post_init__(self): if isinstance(self.status, str): @@ -378,6 +385,62 @@ async def get_job_info(job_id: str): } +class VirtualClusterClient: + def __init__(self, gcs_address): + self._gcs_channel = GcsChannel(gcs_address=gcs_address, aio=True) + self._gcs_channel.connect() + + self._gcs_virtual_cluster_info_stub = ( + gcs_service_pb2_grpc.VirtualClusterInfoGcsServiceStub( + self._gcs_channel.channel() + ) + ) + + def build_job_cluster_id(self, job_id, virtual_cluster_id): + """ + Constructs a unique job cluster ID by combining + the virtual cluster ID and job ID. + Note: + The format needs to remain consistent with `virtual_cluster_id.h`. + """ + return f"{virtual_cluster_id}##{job_id}" + + async def create_job_cluster(self, job_id, virtual_cluster_id, replica_sets): + request = CreateJobClusterRequest( + job_id=job_id, + virtual_cluster_id=virtual_cluster_id, + replica_sets=replica_sets, + ) + + reply = await self._gcs_virtual_cluster_info_stub.CreateJobCluster(request) + + if reply.status.code != 0: + logger.warning( + f"Failed to create job cluster for job ID '{job_id}' in " + f"virtual cluster '{virtual_cluster_id}', " + f"message: {reply.status.message}" + ) + return None, reply.status.message + + return reply.job_cluster_id, None + + async def remove_job_cluster(self, job_cluster_id): + if job_cluster_id is None: + return + + request = RemoveVirtualClusterRequest( + virtual_cluster_id=job_cluster_id, + ) + + reply = await self._gcs_virtual_cluster_info_stub.RemoveVirtualCluster(request) + + if reply.status.code != 0: + logger.warning( + f"Failed to remove job cluster '{job_cluster_id}'," + f" message: {reply.status.message}" + ) + + def uri_to_http_components(package_uri: str) -> Tuple[str, str]: suffix = Path(package_uri).suffix if suffix not in {".zip", ".whl"}: @@ -426,6 +489,10 @@ class JobSubmitRequest: # to reserve for the entrypoint command, separately from any Ray tasks # or actors that are created by it. entrypoint_resources: Optional[Dict[str, float]] = None + # Optional virtual cluster ID for job. + virtual_cluster_id: Optional[str] = None + # Optional replica sets for job + replica_sets: Optional[Dict[str, int]] = None def __post_init__(self): if not isinstance(self.entrypoint, str): @@ -511,6 +578,31 @@ def __post_init__(self): f"got {type(v)}" ) + if self.virtual_cluster_id is not None and not isinstance( + self.virtual_cluster_id, str + ): + raise TypeError( + "virtual_cluster_id must be a string if provided, " + f"got {type(self.virtual_cluster_id)}" + ) + + if self.replica_sets is not None: + if not isinstance(self.replica_sets, dict): + raise TypeError( + "replica_sets must be a dict, " f"got {type(self.replica_sets)}" + ) + else: + for k in self.replica_sets.keys(): + if not isinstance(k, str): + raise TypeError( + "replica_sets keys must be strings, " f"got {type(k)}" + ) + for v in self.replica_sets.values(): + if not isinstance(v, int): + raise TypeError( + "replica_sets values must be integers, " f"got {type(v)}" + ) + @dataclass class JobSubmitResponse: diff --git a/python/ray/dashboard/modules/job/job_agent.py b/python/ray/dashboard/modules/job/job_agent.py index 95a8811a9294..795e72f7f1df 100644 --- a/python/ray/dashboard/modules/job/job_agent.py +++ b/python/ray/dashboard/modules/job/job_agent.py @@ -48,6 +48,8 @@ async def submit_job(self, req: Request) -> Response: submission_id=request_submission_id, runtime_env=submit_request.runtime_env, metadata=submit_request.metadata, + virtual_cluster_id=submit_request.virtual_cluster_id, + replica_sets=submit_request.replica_sets, entrypoint_num_cpus=submit_request.entrypoint_num_cpus, entrypoint_num_gpus=submit_request.entrypoint_num_gpus, entrypoint_memory=submit_request.entrypoint_memory, diff --git a/python/ray/dashboard/modules/job/job_manager.py b/python/ray/dashboard/modules/job/job_manager.py index 17e988f81561..f2c7cf651e8a 100644 --- a/python/ray/dashboard/modules/job/job_manager.py +++ b/python/ray/dashboard/modules/job/job_manager.py @@ -26,6 +26,7 @@ SUPERVISOR_ACTOR_RAY_NAMESPACE, JobInfo, JobInfoStorageClient, + VirtualClusterClient, ) from ray.dashboard.modules.job.job_log_storage_client import JobLogStorageClient from ray.dashboard.modules.job.job_supervisor import JobSupervisor @@ -74,6 +75,7 @@ def __init__(self, gcs_aio_client: GcsAioClient, logs_dir: str): self._logs_dir = logs_dir self._job_info_client = JobInfoStorageClient(gcs_aio_client, logs_dir) self._gcs_address = gcs_aio_client.address + self._virtual_cluster_client = VirtualClusterClient(self._gcs_address) self._cluster_id_hex = gcs_aio_client.cluster_id.hex() self._log_client = JobLogStorageClient() self._supervisor_actor_cls = ray.remote(JobSupervisor) @@ -103,6 +105,37 @@ def _get_job_driver_logger(self, job_id: str) -> logging.Logger: return job_driver_logger + async def _create_job_cluster( + self, + submission_id, + virtual_cluster_id: Optional[str], + replica_sets: Optional[Dict[str, int]], + ): + if ( + virtual_cluster_id is not None + and replica_sets is not None + and len(replica_sets) > 0 + ): + ( + job_cluster_id, + message, + ) = await self._virtual_cluster_client.create_job_cluster( + submission_id, + virtual_cluster_id, + replica_sets, + ) + # If cluster creation fails + if job_cluster_id is None: + message = f"Create Job Cluster Failed, {message}" + await self._job_info_client.put_status( + submission_id, + JobStatus.FAILED, + message=message, + ) + raise Exception(message) + return job_cluster_id + return virtual_cluster_id + async def _recover_running_jobs(self): """Recovers all running jobs from the status client. @@ -130,7 +163,9 @@ def _get_actor_for_job(self, job_id: str) -> Optional[ActorHandle]: return None async def _monitor_job( - self, job_id: str, job_supervisor: Optional[ActorHandle] = None + self, + job_id: str, + job_supervisor: Optional[ActorHandle] = None, ): """Monitors the specified job until it enters a terminal state. @@ -148,7 +183,9 @@ async def _monitor_job( self.monitored_jobs.remove(job_id) async def _monitor_job_internal( - self, job_id: str, job_supervisor: Optional[ActorHandle] = None + self, + job_id: str, + job_supervisor: Optional[ActorHandle] = None, ): timeout = float( os.environ.get( @@ -203,6 +240,9 @@ async def _monitor_job_internal( "`ray status` and specifying fewer resources for the " "job entrypoint." ) + await self._virtual_cluster_client.remove_job_cluster( + job_info.job_cluster_id + ) await self._job_info_client.put_status( job_id, JobStatus.FAILED, @@ -226,6 +266,9 @@ async def _monitor_job_internal( # actor is not created due to some unexpected errors. # We will set the job status to FAILED. logger.error(f"Failed to get job supervisor for job {job_id}.") + await self._virtual_cluster_client.remove_job_cluster( + job_info.job_cluster_id + ) await self._job_info_client.put_status( job_id, JobStatus.FAILED, @@ -258,6 +301,9 @@ async def _monitor_job_internal( elif isinstance(e, RuntimeEnvSetupError): logger.info(f"Failed to set up runtime_env for job {job_id}.") job_error_message = f"runtime_env setup failed: {e}" + await self._virtual_cluster_client.remove_job_cluster( + job_info.job_cluster_id + ) job_status = JobStatus.FAILED await self._job_info_client.put_status( job_id, @@ -272,6 +318,9 @@ async def _monitor_job_internal( job_error_message = ( f"Job supervisor actor could not be scheduled: {e}" ) + await self._virtual_cluster_client.remove_job_cluster( + job_info.job_cluster_id + ) await self._job_info_client.put_status( job_id, JobStatus.FAILED, @@ -282,6 +331,9 @@ async def _monitor_job_internal( f"Job supervisor for job {job_id} failed unexpectedly: {e}." ) job_error_message = f"Unexpected error occurred: {e}" + await self._virtual_cluster_client.remove_job_cluster( + job_info.job_cluster_id + ) job_status = JobStatus.FAILED await self._job_info_client.put_status( job_id, @@ -328,6 +380,7 @@ def _get_supervisor_runtime_env( user_runtime_env: Dict[str, Any], submission_id: str, resources_specified: bool = False, + virtual_cluster_id: Optional[str] = None, ) -> Dict[str, Any]: """Configure and return the runtime_env for the supervisor actor. @@ -338,6 +391,7 @@ def _get_supervisor_runtime_env( in #24546 for GPU detection and just use the user's resource requests, so that the behavior matches that of the user specifying resources for any other actor. + virtual_cluster_id: ID of the virtual cluster the job belongs to. Returns: The runtime_env for the supervisor actor. @@ -360,6 +414,10 @@ def _get_supervisor_runtime_env( # driver can use GPUs if it wants to. This will be removed from # the driver's runtime_env so it isn't inherited by tasks & actors. env_vars[ray_constants.NOSET_CUDA_VISIBLE_DEVICES_ENV_VAR] = "1" + + if virtual_cluster_id is not None: + env_vars[ray_constants.RAY_VIRTUAL_CLUSTER_ID_ENV_VAR] = virtual_cluster_id + runtime_env["env_vars"] = env_vars if os.getenv(RAY_STREAM_RUNTIME_ENV_LOG_TO_JOB_DRIVER_LOG_ENV_VAR, "0") == "1": @@ -372,7 +430,7 @@ def _get_supervisor_runtime_env( return runtime_env async def _get_scheduling_strategy( - self, resources_specified: bool + self, resources_specified: bool, virtual_cluster_id: Optional[str] ) -> SchedulingStrategyT: """Get the scheduling strategy for the job. @@ -387,6 +445,9 @@ async def _get_scheduling_strategy( Returns: The scheduling strategy to use for the job. """ + if virtual_cluster_id is not None: + return "DEFAULT" + if resources_specified: return "DEFAULT" @@ -426,6 +487,8 @@ async def submit_job( submission_id: Optional[str] = None, runtime_env: Optional[Dict[str, Any]] = None, metadata: Optional[Dict[str, str]] = None, + virtual_cluster_id: Optional[str] = None, + replica_sets: Optional[Dict[str, int]] = None, entrypoint_num_cpus: Optional[Union[int, float]] = None, entrypoint_num_gpus: Optional[Union[int, float]] = None, entrypoint_memory: Optional[int] = None, @@ -452,6 +515,8 @@ async def submit_job( env at ray cluster, task and actor level. metadata: Support passing arbitrary data to driver command in case needed. + virtual_cluster_id: Virtual Cluster this job belongs to. + replica_sets: Replia Sets required by this job. entrypoint_num_cpus: The quantity of CPU cores to reserve for the execution of the entrypoint command, separately from any tasks or actors launched by it. Defaults to 0. @@ -485,6 +550,16 @@ async def submit_job( # avoid duplicate monitoring of the same job. await self._recover_running_jobs_event.wait() + job_cluster_id = None + if ( + virtual_cluster_id is not None + and replica_sets is not None + and len(replica_sets) > 0 + ): + job_cluster_id = self._virtual_cluster_client.build_job_cluster_id( + submission_id, virtual_cluster_id + ) + logger.info(f"Starting job with submission_id: {submission_id}") job_info = JobInfo( entrypoint=entrypoint, @@ -496,6 +571,7 @@ async def submit_job( entrypoint_num_gpus=entrypoint_num_gpus, entrypoint_memory=entrypoint_memory, entrypoint_resources=entrypoint_resources, + job_cluster_id=job_cluster_id, ) new_key_added = await self._job_info_client.put_info( submission_id, job_info, overwrite=False @@ -506,6 +582,13 @@ async def submit_job( "Please use a different submission_id." ) + try: + virtual_cluster_id = await self._create_job_cluster( + submission_id, virtual_cluster_id, replica_sets + ) + except Exception: + return submission_id + driver_logger = self._get_job_driver_logger(submission_id) # Wait for the actor to start up asynchronously so this call always # returns immediately and we can catch errors with the actor starting @@ -520,13 +603,18 @@ async def submit_job( ] ) scheduling_strategy = await self._get_scheduling_strategy( - resources_specified + resources_specified, virtual_cluster_id ) if self.event_logger: self.event_logger.info( f"Started a ray job {submission_id}.", submission_id=submission_id ) + labels = {} + if virtual_cluster_id is not None: + # label is used to pass virtual_cluster_id to core worker + labels["virtual_cluster_id"] = virtual_cluster_id + driver_logger.info("Runtime env is setting up.") supervisor = self._supervisor_actor_cls.options( lifetime="detached", @@ -537,9 +625,10 @@ async def submit_job( resources=entrypoint_resources, scheduling_strategy=scheduling_strategy, runtime_env=self._get_supervisor_runtime_env( - runtime_env, submission_id, resources_specified + runtime_env, submission_id, resources_specified, virtual_cluster_id ), namespace=SUPERVISOR_ACTOR_RAY_NAMESPACE, + _labels=labels, ).remote( submission_id, entrypoint, @@ -556,7 +645,10 @@ async def submit_job( # Monitor the job in the background so we can detect errors without # requiring a client to poll. run_background_task( - self._monitor_job(submission_id, job_supervisor=supervisor) + self._monitor_job( + submission_id, + job_supervisor=supervisor, + ) ) except Exception as e: tb_str = traceback.format_exc() @@ -564,6 +656,9 @@ async def submit_job( f"Failed to start supervisor actor for job {submission_id}: '{e}'" f". Full traceback:\n{tb_str}" ) + await self._virtual_cluster_client.remove_job_cluster( + job_info.job_cluster_id + ) await self._job_info_client.put_status( submission_id, JobStatus.FAILED, diff --git a/python/ray/dashboard/modules/job/job_supervisor.py b/python/ray/dashboard/modules/job/job_supervisor.py index 15676b5b5647..2d820c073a09 100644 --- a/python/ray/dashboard/modules/job/job_supervisor.py +++ b/python/ray/dashboard/modules/job/job_supervisor.py @@ -21,6 +21,7 @@ JOB_ID_METADATA_KEY, JOB_NAME_METADATA_KEY, JobInfoStorageClient, + VirtualClusterClient, ) from ray.dashboard.modules.job.job_log_storage_client import JobLogStorageClient from ray.job_submission import JobStatus @@ -74,9 +75,11 @@ def __init__( cluster_id_hex: str, logs_dir: Optional[str] = None, ): + self._job_id = job_id gcs_aio_client = GcsAioClient(address=gcs_address, cluster_id=cluster_id_hex) self._job_info_client = JobInfoStorageClient(gcs_aio_client, logs_dir) + self._virtual_cluster_client = VirtualClusterClient(gcs_address) self._log_client = JobLogStorageClient() self._entrypoint = entrypoint @@ -415,6 +418,9 @@ async def run( ) self._kill_processes(proc_to_kill, signal.SIGKILL) + await self._virtual_cluster_client.remove_job_cluster( + curr_info.job_cluster_id + ) await self._job_info_client.put_status(self._job_id, JobStatus.STOPPED) else: # Child process finished execution and no stop event is set @@ -427,6 +433,9 @@ async def run( f"exited with code {return_code}" ) if return_code == 0: + await self._virtual_cluster_client.remove_job_cluster( + curr_info.job_cluster_id + ) await self._job_info_client.put_status( self._job_id, JobStatus.SUCCEEDED, @@ -446,6 +455,9 @@ async def run( "Job entrypoint command " f"failed with exit code {return_code}. No logs available." ) + await self._virtual_cluster_client.remove_job_cluster( + curr_info.job_cluster_id + ) await self._job_info_client.put_status( self._job_id, JobStatus.FAILED, @@ -458,6 +470,9 @@ async def run( f"command. {traceback.format_exc()}" ) try: + await self._virtual_cluster_client.remove_job_cluster( + curr_info.job_cluster_id + ) await self._job_info_client.put_status( self._job_id, JobStatus.FAILED, diff --git a/python/ray/dashboard/modules/job/sdk.py b/python/ray/dashboard/modules/job/sdk.py index b3b25e936fa0..ae70f0f90280 100644 --- a/python/ray/dashboard/modules/job/sdk.py +++ b/python/ray/dashboard/modules/job/sdk.py @@ -130,6 +130,8 @@ def submit_job( runtime_env: Optional[Dict[str, Any]] = None, metadata: Optional[Dict[str, str]] = None, submission_id: Optional[str] = None, + virtual_cluster_id: Optional[str] = None, + replica_sets: Optional[Dict[str, int]] = None, entrypoint_num_cpus: Optional[Union[int, float]] = None, entrypoint_num_gpus: Optional[Union[int, float]] = None, entrypoint_memory: Optional[int] = None, @@ -157,6 +159,7 @@ def submit_job( Args: entrypoint: The shell command to run for this job. submission_id: A unique ID for this job. + virtual_cluster_id: The virtual cluster this job belongs to. runtime_env: The runtime environment to install and run this job in. metadata: Arbitrary data to store along with this job. job_id: DEPRECATED. This has been renamed to submission_id @@ -228,6 +231,8 @@ def submit_job( req = JobSubmitRequest( entrypoint=entrypoint, submission_id=submission_id, + virtual_cluster_id=virtual_cluster_id, + replica_sets=replica_sets, runtime_env=runtime_env, metadata=metadata, entrypoint_num_cpus=entrypoint_num_cpus, diff --git a/python/ray/dashboard/modules/job/tests/test_job_with_virtual_cluster.py b/python/ray/dashboard/modules/job/tests/test_job_with_virtual_cluster.py new file mode 100644 index 000000000000..d6c1af292ca2 --- /dev/null +++ b/python/ray/dashboard/modules/job/tests/test_job_with_virtual_cluster.py @@ -0,0 +1,1030 @@ +import logging +import sys +import tempfile +from functools import partial +from pathlib import Path + +import pytest +import pytest_asyncio + +import ray +from ray._private.gcs_utils import GcsChannel +from ray._private.runtime_env.working_dir import upload_working_dir_if_needed +from ray._private.test_utils import ( + format_web_url, + wait_for_condition, + wait_until_server_available, +) +from ray._private.utils import hex_to_binary +from ray._raylet import PlacementGroupID +from ray.core.generated import gcs_service_pb2_grpc +from ray.core.generated.gcs_service_pb2 import CreateOrUpdateVirtualClusterRequest +from ray.dashboard.modules.job.common import ( + JOB_ACTOR_NAME_TEMPLATE, + SUPERVISOR_ACTOR_RAY_NAMESPACE, + JobStatus, +) +from ray.dashboard.tests.conftest import * # noqa +from ray.job_submission import JobSubmissionClient +from ray.runtime_env.runtime_env import RuntimeEnv +from ray.tests.conftest import _ray_start_virtual_cluster +from ray.util.placement_group import PlacementGroup + +TEMPLATE_ID_PREFIX = "template_id_" +kPrimaryClusterID = "kPrimaryClusterID" + +logger = logging.getLogger(__name__) + + +@pytest_asyncio.fixture +def job_sdk_client(request): + param = getattr(request, "param", {}) + ntemplates = param["ntemplates"] + with _ray_start_virtual_cluster( + do_init=True, + num_cpus=20, + num_nodes=4 * ntemplates + 1, + template_id_prefix=TEMPLATE_ID_PREFIX, + **param, + ) as res: + ip, _ = res.webui_url.split(":") + for node in res.worker_nodes: + agent_address = f"{ip}:{node._dashboard_agent_listen_port}" + assert wait_until_server_available(agent_address) + head_address = res.webui_url + assert wait_until_server_available(head_address) + res.wait_for_nodes() + yield ( + JobSubmissionClient(format_web_url(head_address)), + res.gcs_address, + res, + ) + + +async def create_virtual_cluster( + gcs_address, virtual_cluster_id, replica_sets, divisible=False +): + channel = GcsChannel(gcs_address, aio=True) + channel.connect() + gcs_virtual_cluster_info_stub = ( + gcs_service_pb2_grpc.VirtualClusterInfoGcsServiceStub(channel.channel()) + ) + request = CreateOrUpdateVirtualClusterRequest( + virtual_cluster_id=virtual_cluster_id, + divisible=divisible, + replica_sets=replica_sets, + ) + reply = await (gcs_virtual_cluster_info_stub.CreateOrUpdateVirtualCluster(request)) + assert reply.status.code == 0 + return reply.node_instances + + +@pytest.mark.parametrize( + "job_sdk_client", + [ + { + "_system_config": {"gcs_actor_scheduling_enabled": False}, + "ntemplates": 2, + }, + { + "_system_config": {"gcs_actor_scheduling_enabled": True}, + "ntemplates": 2, + }, + ], + indirect=True, +) +@pytest.mark.asyncio +async def test_indivisible_virtual_cluster(job_sdk_client): + head_client, gcs_address, cluster = job_sdk_client + virtual_cluster_id_prefix = "VIRTUAL_CLUSTER_" + node_to_virtual_cluster = {} + ntemplates = 2 + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + nodes = await create_virtual_cluster( + gcs_address, virtual_cluster_id, {TEMPLATE_ID_PREFIX + str(i): 3} + ) + for node_id in nodes: + assert node_id not in node_to_virtual_cluster + node_to_virtual_cluster[node_id] = virtual_cluster_id + + @ray.remote + class ControlActor: + def __init__(self): + self._nodes = set() + self._ready = False + + def ready(self): + self._ready = True + + def is_ready(self): + return self._ready + + def add_node(self, node_id): + self._nodes.add(node_id) + + def nodes(self): + return self._nodes + + for i in range(ntemplates): + actor_name = f"test_actors_{i}" + pg_name = f"test_pgs_{i}" + control_actor_name = f"control_{i}" + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + control_actor = ControlActor.options( + name=control_actor_name, namespace="control" + ).remote() + with tempfile.TemporaryDirectory() as tmp_dir: + path = Path(tmp_dir) + driver_script = """ +import ray +import time +import asyncio + +ray.init(address="auto") + +control = ray.get_actor(name="{control_actor_name}", namespace="control") + + +@ray.remote(max_restarts=10) +class Actor: + def __init__(self, control, pg): + node_id = ray.get_runtime_context().get_node_id() + ray.get(control.add_node.remote(node_id)) + self._pg = pg + + async def run(self, control): + node_id = ray.get_runtime_context().get_node_id() + await control.add_node.remote(node_id) + node_id = ray.util.placement_group_table(self._pg)["bundles_to_node_id"][0] + await control.add_node.remote(node_id) + await control.ready.remote() + while True: + await asyncio.sleep(1) + + async def get_node_id(self): + node_id = ray.util.placement_group_table(pg)["bundles_to_node_id"][0] + return (ray.get_runtime_context().get_node_id(), node_id) + + +pg = ray.util.placement_group( + bundles=[{{"CPU": 1}}], name="{pg_name}", lifetime="detached" +) +ray.get(pg.ready()) + + +@ray.remote +def hello(control): + node_id = ray.get_runtime_context().get_node_id() + ray.get(control.add_node.remote(node_id)) + + +ray.get(hello.remote(control)) +a = Actor.options(name="{actor_name}", + namespace="control", + num_cpus=1, + lifetime="detached").remote( + control, pg +) +ray.get(a.run.remote(control)) + """ + driver_script = driver_script.format( + actor_name=actor_name, + pg_name=pg_name, + control_actor_name=control_actor_name, + ) + test_script_file = path / "test_script.py" + with open(test_script_file, "w+") as file: + file.write(driver_script) + + runtime_env = {"working_dir": tmp_dir} + runtime_env = upload_working_dir_if_needed( + runtime_env, tmp_dir, logger=logger + ) + runtime_env = RuntimeEnv(**runtime_env).to_dict() + + job_id = head_client.submit_job( + entrypoint="python test_script.py", + entrypoint_memory=1, + runtime_env=runtime_env, + virtual_cluster_id=virtual_cluster_id, + ) + + def _check_ready(control_actor): + return ray.get(control_actor.is_ready.remote()) + + wait_for_condition(partial(_check_ready, control_actor), timeout=20) + + def _check_virtual_cluster( + control_actor, node_to_virtual_cluster, virtual_cluster_id + ): + nodes = ray.get(control_actor.nodes.remote()) + assert len(nodes) > 0 + for node in nodes: + assert node_to_virtual_cluster[node] == virtual_cluster_id + return True + + wait_for_condition( + partial( + _check_virtual_cluster, + control_actor, + node_to_virtual_cluster, + virtual_cluster_id, + ), + timeout=20, + ) + + supervisor_actor = ray.get_actor( + name=JOB_ACTOR_NAME_TEMPLATE.format(job_id=job_id), + namespace=SUPERVISOR_ACTOR_RAY_NAMESPACE, + ) + actor_info = ray.state.actors(supervisor_actor._actor_id.hex()) + driver_node_id = actor_info["Address"]["NodeID"] + assert node_to_virtual_cluster[driver_node_id] == virtual_cluster_id + + job_info = head_client.get_job_info(job_id) + assert ( + node_to_virtual_cluster[job_info.driver_node_id] == virtual_cluster_id + ) + + nodes_to_remove = ray.get(control_actor.nodes.remote()) + if driver_node_id in nodes_to_remove: + nodes_to_remove.remove(driver_node_id) + + to_remove = [] + for node in cluster.worker_nodes: + if node.node_id in nodes_to_remove: + to_remove.append(node) + for node in to_remove: + cluster.remove_node(node) + + def _check_recover( + nodes_to_remove, actor_name, node_to_virtual_cluster, virtual_cluster_id + ): + actor = ray.get_actor(actor_name, namespace="control") + nodes = ray.get(actor.get_node_id.remote()) + for node_id in nodes: + assert node_id not in nodes_to_remove + assert node_to_virtual_cluster[node_id] == virtual_cluster_id + return True + + wait_for_condition( + partial( + _check_recover, + nodes_to_remove, + actor_name, + node_to_virtual_cluster, + virtual_cluster_id, + ), + timeout=120, + ) + head_client.stop_job(job_id) + + +@pytest.mark.parametrize( + "job_sdk_client", + [ + { + "ntemplates": 2, + }, + ], + indirect=True, +) +@pytest.mark.asyncio +async def test_divisible_virtual_cluster(job_sdk_client): + head_client, gcs_address, cluster = job_sdk_client + virtual_cluster_id_prefix = "VIRTUAL_CLUSTER_" + node_to_virtual_cluster = {} + ntemplates = 1 + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + nodes = await create_virtual_cluster( + gcs_address, + virtual_cluster_id, + {TEMPLATE_ID_PREFIX + str(i): 2}, + True, + ) + for node_id in nodes: + assert node_id not in node_to_virtual_cluster + node_to_virtual_cluster[node_id] = virtual_cluster_id + + for node in cluster.worker_nodes: + if node.node_id not in node_to_virtual_cluster: + node_to_virtual_cluster[node.node_id] = kPrimaryClusterID + + @ray.remote + class ControlActor: + def __init__(self): + self._nodes = set() + self._ready = False + + def ready(self): + self._ready = True + + def is_ready(self): + return self._ready + + def add_node(self, node_id): + self._nodes.add(node_id) + + def nodes(self): + return self._nodes + + for i in range(ntemplates + 1): + actor_name = f"test_actors_{i}" + pg_name = f"test_pgs_{i}" + control_actor_name = f"control_{i}" + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + if i == ntemplates: + virtual_cluster_id = kPrimaryClusterID + control_actor = ControlActor.options( + name=control_actor_name, namespace="control" + ).remote() + with tempfile.TemporaryDirectory() as tmp_dir: + path = Path(tmp_dir) + driver_script = """ +import ray +import time +import asyncio + +ray.init(address="auto") + +control = ray.get_actor(name="{control_actor_name}", namespace="control") + + +@ray.remote(max_restarts=10) +class Actor: + def __init__(self, control, pg): + node_id = ray.get_runtime_context().get_node_id() + ray.get(control.add_node.remote(node_id)) + self._pg = pg + + async def run(self, control): + node_id = ray.get_runtime_context().get_node_id() + await control.add_node.remote(node_id) + node_id = ray.util.placement_group_table(self._pg)["bundles_to_node_id"][0] + await control.add_node.remote(node_id) + await control.ready.remote() + while True: + await asyncio.sleep(1) + + async def get_node_id(self): + node_id = ray.util.placement_group_table(pg)["bundles_to_node_id"][0] + return (ray.get_runtime_context().get_node_id(), node_id) + + +pg = ray.util.placement_group( + bundles=[{{"CPU": 1}}], name="{pg_name}", lifetime="detached" +) +ray.get(pg.ready()) + + +@ray.remote +def hello(control): + node_id = ray.get_runtime_context().get_node_id() + ray.get(control.add_node.remote(node_id)) + + +ray.get(hello.remote(control)) +a = Actor.options(name="{actor_name}", + namespace="control", + num_cpus=1, + lifetime="detached").remote( + control, pg +) +ray.get(a.run.remote(control)) + """ + driver_script = driver_script.format( + actor_name=actor_name, + pg_name=pg_name, + control_actor_name=control_actor_name, + ) + test_script_file = path / "test_script.py" + with open(test_script_file, "w+") as file: + file.write(driver_script) + + runtime_env = {"working_dir": tmp_dir} + runtime_env = upload_working_dir_if_needed( + runtime_env, tmp_dir, logger=logger + ) + runtime_env = RuntimeEnv(**runtime_env).to_dict() + + job_id = head_client.submit_job( + entrypoint="python test_script.py", + entrypoint_memory=1, + runtime_env=runtime_env, + virtual_cluster_id=virtual_cluster_id, + replica_sets={TEMPLATE_ID_PREFIX + str(i): 2}, + ) + + def _check_ready(control_actor): + return ray.get(control_actor.is_ready.remote()) + + wait_for_condition(partial(_check_ready, control_actor), timeout=20) + + def _check_virtual_cluster( + control_actor, node_to_virtual_cluster, virtual_cluster_id + ): + nodes = ray.get(control_actor.nodes.remote()) + assert len(nodes) > 0 + for node in nodes: + assert node_to_virtual_cluster[node] == virtual_cluster_id + return True + + wait_for_condition( + partial( + _check_virtual_cluster, + control_actor, + node_to_virtual_cluster, + virtual_cluster_id, + ), + timeout=20, + ) + + supervisor_actor = ray.get_actor( + name=JOB_ACTOR_NAME_TEMPLATE.format(job_id=job_id), + namespace=SUPERVISOR_ACTOR_RAY_NAMESPACE, + ) + actor_info = ray.state.actors(supervisor_actor._actor_id.hex()) + driver_node_id = actor_info["Address"]["NodeID"] + assert node_to_virtual_cluster[driver_node_id] == virtual_cluster_id + + job_info = head_client.get_job_info(job_id) + assert ( + node_to_virtual_cluster[job_info.driver_node_id] == virtual_cluster_id + ) + + nodes_to_remove = ray.get(control_actor.nodes.remote()) + if driver_node_id in nodes_to_remove: + nodes_to_remove.remove(driver_node_id) + + to_remove = [] + for node in cluster.worker_nodes: + if node.node_id in nodes_to_remove: + to_remove.append(node) + for node in to_remove: + cluster.remove_node(node) + + def _check_recover( + nodes_to_remove, actor_name, node_to_virtual_cluster, virtual_cluster_id + ): + actor = ray.get_actor(actor_name, namespace="control") + nodes = ray.get(actor.get_node_id.remote()) + for node_id in nodes: + assert node_id not in nodes_to_remove + assert node_to_virtual_cluster[node_id] == virtual_cluster_id + return True + + wait_for_condition( + partial( + _check_recover, + nodes_to_remove, + actor_name, + node_to_virtual_cluster, + virtual_cluster_id, + ), + timeout=120, + ) + head_client.stop_job(job_id) + + +@pytest.mark.parametrize( + "job_sdk_client", + [ + { + "ntemplates": 2, + }, + ], + indirect=True, +) +@pytest.mark.asyncio +async def test_job_access_cluster_data(job_sdk_client): + head_client, gcs_address, cluster = job_sdk_client + virtual_cluster_id_prefix = "VIRTUAL_CLUSTER_" + node_to_virtual_cluster = {} + + @ray.remote + class StorageActor: + def __init__(self): + self._nodes = set() + self._ready = False + self._driver_info = {} + self._actor_info = {} + self._normal_task_info = {} + + def ready(self): + self._ready = True + + def is_ready(self): + return self._ready + + def get_info(self): + return { + "driver": self._driver_info, + "actor": self._actor_info, + "normal_task": self._normal_task_info, + } + + def set_driver_info(self, key, value): + self._driver_info[key] = value + + def set_actor_info(self, key, value): + self._actor_info[key] = value + + def set_normal_task_info(self, key, value): + self._normal_task_info[key] = value + + ntemplates = 2 + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + nodes = await create_virtual_cluster( + gcs_address, virtual_cluster_id, {TEMPLATE_ID_PREFIX + str(i): 3} + ) + for node_id in nodes: + assert node_id not in node_to_virtual_cluster + node_to_virtual_cluster[node_id] = virtual_cluster_id + + for i in range(ntemplates): + storage_actor_name = f"storage_{i}" + if i == ntemplates: + virtual_cluster_id = kPrimaryClusterID + storage_actor = StorageActor.options( + name=storage_actor_name, namespace="storage", num_cpus=0 + ).remote() + + assert not ray.get(storage_actor.is_ready.remote()) + resource_accessor_name = f"accessor_{i}" + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + + with tempfile.TemporaryDirectory() as tmp_dir: + path = Path(tmp_dir) + driver_script = """ +import ray +import os + + +ray.init(address="auto") +storage = ray.get_actor(name="{storage_actor_name}", namespace="storage") + + +@ray.remote +def access_nodes(): + return ray.nodes() + + +@ray.remote +def access_cluster_resources(): + return ray.cluster_resources() + + +@ray.remote +def access_available_resources(): + return ray.available_resources() + + +@ray.remote +class ResourceAccessor: + def is_ready(self): + return True + + def nodes(self): + self._nodes = ray.nodes() + return self._nodes + + def total_cluster_resources(self): + self._total_cluster_resources = ray.cluster_resources() + return self._total_cluster_resources + + def available_resources(self): + self._available_resources = ray.available_resources() + return self._available_resources + + +accessor = ResourceAccessor.options( + name="{resource_accessor_name}", namespace="storage", num_cpus=0 +).remote() +ray.get(accessor.is_ready.remote()) + +ray.get(storage.ready.remote()) + +driver_nodes = ray.nodes() +driver_cluster_resources = ray.cluster_resources() +driver_available_resources = ray.available_resources() +ray.get(storage.set_driver_info.remote("nodes", driver_nodes)) +ray.get(storage.set_driver_info.remote("cluster_resources", driver_cluster_resources)) +ray.get( + storage.set_driver_info.remote("available_resources", driver_available_resources) +) + +actor_nodes = ray.get(accessor.nodes.remote()) +actor_cluster_resources = ray.get(accessor.total_cluster_resources.remote()) +actor_available_resources = ray.get(accessor.available_resources.remote()) +ray.get(storage.set_actor_info.remote("nodes", actor_nodes)) +ray.get(storage.set_actor_info.remote("cluster_resources", actor_cluster_resources)) +ray.get(storage.set_actor_info.remote("available_resources", actor_available_resources)) + +normal_task_nodes = ray.get(access_nodes.options(num_cpus=0).remote()) +normal_task_cluster_resources = ray.get( + access_cluster_resources.options(num_cpus=0).remote() +) +normal_task_available_resources = ray.get( + access_available_resources.options(num_cpus=0).remote() +) +ray.get(storage.set_normal_task_info.remote("nodes", normal_task_nodes)) +ray.get( + storage.set_normal_task_info.remote( + "cluster_resources", normal_task_cluster_resources + ) +) +ray.get( + storage.set_normal_task_info.remote( + "available_resources", normal_task_available_resources + ) +) + """ + driver_script = driver_script.format( + resource_accessor_name=resource_accessor_name, + storage_actor_name=storage_actor_name, + ) + test_script_file = path / "test_script.py" + with open(test_script_file, "w+") as file: + file.write(driver_script) + + runtime_env = {"working_dir": tmp_dir} + runtime_env = upload_working_dir_if_needed( + runtime_env, tmp_dir, logger=logger + ) + runtime_env = RuntimeEnv(**runtime_env).to_dict() + + job_id = head_client.submit_job( + entrypoint="python test_script.py", + entrypoint_memory=1, + runtime_env=runtime_env, + virtual_cluster_id=virtual_cluster_id, + ) + + wait_for_condition( + partial( + lambda storage_actor: ray.get(storage_actor.is_ready.remote()), + storage_actor, + ), + timeout=20, + ) + + def _check_only_access_virtual_cluster_nodes( + storage_actor, node_to_virtual_cluster, virtual_cluster_id + ): + cluster_info = ray.get(storage_actor.get_info.remote()) + expect_nodes = ray.nodes(virtual_cluster_id) + expect_total_cluster_resources = ray.cluster_resources( + virtual_cluster_id + ) + expect_available_resources = ray.available_resources(virtual_cluster_id) + + assert len(cluster_info) > 0 + assert cluster_info["driver"]["nodes"] == expect_nodes + assert ( + cluster_info["driver"]["cluster_resources"]["CPU"] + == expect_total_cluster_resources["CPU"] + ) + assert ( + cluster_info["driver"]["available_resources"]["CPU"] + == expect_available_resources["CPU"] + ) + assert cluster_info["actor"]["nodes"] == expect_nodes + assert ( + cluster_info["actor"]["cluster_resources"]["CPU"] + == expect_total_cluster_resources["CPU"] + ) + assert ( + cluster_info["actor"]["available_resources"]["CPU"] + == expect_available_resources["CPU"] + ) + assert cluster_info["normal_task"]["nodes"] == expect_nodes + assert ( + cluster_info["normal_task"]["cluster_resources"]["CPU"] + == expect_total_cluster_resources["CPU"] + ) + assert ( + cluster_info["normal_task"]["available_resources"]["CPU"] + == expect_available_resources["CPU"] + ) + + for node in cluster_info["driver"]["nodes"]: + node_id = node["NodeID"] + assert node_to_virtual_cluster[node_id] == virtual_cluster_id + for node in cluster_info["actor"]["nodes"]: + node_id = node["NodeID"] + assert node_to_virtual_cluster[node_id] == virtual_cluster_id + for node in cluster_info["normal_task"]["nodes"]: + node_id = node["NodeID"] + assert node_to_virtual_cluster[node_id] == virtual_cluster_id + return True + + wait_for_condition( + partial( + _check_only_access_virtual_cluster_nodes, + storage_actor, + node_to_virtual_cluster, + virtual_cluster_id, + ), + timeout=20, + ) + head_client.stop_job(job_id) + + +@pytest.mark.parametrize( + "job_sdk_client", + [ + { + "ntemplates": 2, + }, + ], + indirect=True, +) +@pytest.mark.asyncio +async def test_list_nodes(job_sdk_client): + head_client, gcs_address, cluster = job_sdk_client + virtual_cluster_id_prefix = "VIRTUAL_CLUSTER_" + node_to_virtual_cluster = {} + ntemplates = 2 + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + nodes = await create_virtual_cluster( + gcs_address, virtual_cluster_id, {TEMPLATE_ID_PREFIX + str(i): 3} + ) + for node_id in nodes: + assert node_id not in node_to_virtual_cluster + node_to_virtual_cluster[node_id] = virtual_cluster_id + + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + cluster_nodes = ray.nodes(virtual_cluster_id=virtual_cluster_id_prefix + str(i)) + for node in cluster_nodes: + assert node["NodeID"] in node_to_virtual_cluster + assert node_to_virtual_cluster[node["NodeID"]] == virtual_cluster_id + + assert len(ray.nodes()) == 9 + assert len(ray.nodes("")) == 9 + assert len(ray.nodes(None)) == 9 + + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + assert len(ray.nodes(virtual_cluster_id)) == 3 + + assert len(ray.nodes("FAKE")) == 0 + with pytest.raises(TypeError): + ray.nodes(1) + + +@pytest.mark.parametrize( + "job_sdk_client", + [ + { + "ntemplates": 2, + }, + ], + indirect=True, +) +@pytest.mark.asyncio +async def test_list_cluster_resources(job_sdk_client): + head_client, gcs_address, cluster = job_sdk_client + virtual_cluster_id_prefix = "VIRTUAL_CLUSTER_" + node_to_virtual_cluster = {} + ntemplates = 2 + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + nodes = await create_virtual_cluster( + gcs_address, virtual_cluster_id, {TEMPLATE_ID_PREFIX + str(i): 3} + ) + for node_id in nodes: + assert node_id not in node_to_virtual_cluster + node_to_virtual_cluster[node_id] = virtual_cluster_id + + total_resources = ray.cluster_resources() + assert len(total_resources) > 0, f"total_resources {total_resources} is empty" + assert total_resources["CPU"] > 0 + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + virtual_cluster_resources = ray.cluster_resources( + virtual_cluster_id=virtual_cluster_id_prefix + str(i) + ) + assert int(virtual_cluster_resources["CPU"]) == 60 + assert len(ray.cluster_resources("NON_EXIST_VIRTUAL_CLUSTER")) == 0 + with pytest.raises(TypeError): + ray.cluster_resources(1) + + available_resources = ray.available_resources() + assert ( + len(available_resources) > 0 + ), f"available_resources {available_resources} is empty" + assert available_resources["CPU"] > 0 + assert available_resources["CPU"] <= total_resources["CPU"] + assert ray.available_resources(None) == available_resources + for i in range(ntemplates): + virtual_cluster_id = virtual_cluster_id_prefix + str(i) + virtual_cluster_resources = ray.available_resources( + virtual_cluster_id=virtual_cluster_id_prefix + str(i) + ) + assert int(virtual_cluster_resources["CPU"]) > 0 + assert int(virtual_cluster_resources["CPU"]) < total_resources["CPU"] + assert len(ray.available_resources("NON_EXIST_VIRTUAL_CLUSTER")) == 0 + with pytest.raises(TypeError): + ray.available_resources(1) + + +@pytest.mark.parametrize( + "job_sdk_client", + [ + { + "_system_config": {"expired_job_clusters_gc_interval_ms": 1000}, + "ntemplates": 2, + }, + ], + indirect=True, +) +@pytest.mark.asyncio +async def test_detached_job_cluster(job_sdk_client): + head_client, gcs_address, cluster = job_sdk_client + virtual_cluster_id = "VIRTUAL_CLUSTER_0" + await create_virtual_cluster( + gcs_address, + virtual_cluster_id, + {TEMPLATE_ID_PREFIX + str(0): 2}, + True, + ) + await create_virtual_cluster( + gcs_address, + virtual_cluster_id + "_indivisible", + {TEMPLATE_ID_PREFIX + str(1): 2}, + True, + ) + + actor_name = "test_actor" + pg_name = "test_pg" + with tempfile.TemporaryDirectory() as tmp_dir: + path = Path(tmp_dir) + driver_script = """ +import ray +import time +import asyncio + +ray.init(address="auto") + +@ray.remote(max_restarts=10) +class Actor: + def __init__(self): + pass + + async def run(self): + pass + +pg = ray.util.placement_group( + bundles=[{{"CPU": 1}}], name="{pg_name}", lifetime="detached" +) + +a = Actor.options(name="{actor_name}", + namespace="test", + num_cpus=1, + lifetime="detached").remote() +ray.get(a.run.remote()) + """ + driver_script = driver_script.format( + actor_name=actor_name, + pg_name=pg_name, + ) + test_script_file = path / "test_script.py" + with open(test_script_file, "w+") as file: + file.write(driver_script) + + runtime_env = {"working_dir": tmp_dir} + runtime_env = upload_working_dir_if_needed(runtime_env, tmp_dir, logger=logger) + runtime_env = RuntimeEnv(**runtime_env).to_dict() + + def _successful_submit(head_client, runtime_env, virtual_cluster_id): + job_id = head_client.submit_job( + entrypoint="python test_script.py", + entrypoint_memory=1, + runtime_env=runtime_env, + virtual_cluster_id=virtual_cluster_id, + replica_sets={TEMPLATE_ID_PREFIX + str(0): 2}, + ) + + def _check_job_succeeded(head_client, job_id): + assert head_client.get_job_status(job_id) == JobStatus.SUCCEEDED + return True + + assert head_client.get_job_status(job_id) != JobStatus.FAILED + + wait_for_condition( + partial( + _check_job_succeeded, + head_client, + job_id, + ), + timeout=20, + ) + head_client.stop_job(job_id) + return True + + _successful_submit(head_client, runtime_env, virtual_cluster_id) + + def _failed_submit(): + job_id = head_client.submit_job( + entrypoint="python test_script.py", + entrypoint_memory=1, + runtime_env=runtime_env, + virtual_cluster_id=virtual_cluster_id, + replica_sets={TEMPLATE_ID_PREFIX + str(0): 2}, + ) + + def _check_job_failed(head_client, job_id): + assert head_client.get_job_status(job_id) == JobStatus.FAILED + assert ( + "No enough node instances to create the job cluster" + in head_client.get_job_info(job_id).message + ) + return True + + wait_for_condition( + partial( + _check_job_failed, + head_client, + job_id, + ), + timeout=20, + ) + + _failed_submit() + + def _remove_pg(): + pgs = ray.state.state.placement_group_table() + pg_id = None + for id, pg in pgs.items(): + if pg["state"] == "CREATED": + pg_id = id + break + assert pg_id is not None + + pg = PlacementGroup(PlacementGroupID(hex_to_binary(pg_id))) + ray.util.remove_placement_group(pg) + + _remove_pg() + + _failed_submit() + + actor = ray.get_actor(name=actor_name, namespace="test") + ray.kill(actor, no_restart=False) + + _failed_submit() + + actor = ray.get_actor(name=actor_name, namespace="test") + ray.kill(actor) + + wait_for_condition( + partial(_successful_submit, head_client, runtime_env, virtual_cluster_id), + timeout=50, + retry_interval_ms=5000, + ) + + _remove_pg() + + actor = ray.get_actor(name=actor_name, namespace="test") + ray.kill(actor) + + with tempfile.TemporaryDirectory() as tmp_dir: + path = Path(tmp_dir) + driver_script += """ +ray.kill(a) +ray.util.remove_placement_group(pg) + """ + test_script_file = path / "test_script.py" + with open(test_script_file, "w") as file: + file.write(driver_script) + + runtime_env = {"working_dir": tmp_dir} + runtime_env = upload_working_dir_if_needed( + runtime_env, tmp_dir, logger=logger + ) + runtime_env = RuntimeEnv(**runtime_env).to_dict() + + wait_for_condition( + partial( + _successful_submit, head_client, runtime_env, virtual_cluster_id + ), + timeout=50, + retry_interval_ms=5000, + ) + # Test that the job submission is successful if detached is + # early killed in job. + wait_for_condition( + partial( + _successful_submit, head_client, runtime_env, virtual_cluster_id + ), + timeout=50, + retry_interval_ms=5000, + ) + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/dashboard/modules/state/state_head.py b/python/ray/dashboard/modules/state/state_head.py index 9b55d1a1cfe5..87213fee2dcf 100644 --- a/python/ray/dashboard/modules/state/state_head.py +++ b/python/ray/dashboard/modules/state/state_head.py @@ -184,6 +184,12 @@ async def list_objects(self, req: aiohttp.web.Request) -> aiohttp.web.Response: record_extra_usage_tag(TagKey.CORE_STATE_API_LIST_OBJECTS, "1") return await handle_list_api(self._state_api.list_objects, req) + @routes.get("/api/v0/vclusters") + @RateLimitedModule.enforce_max_concurrent_calls + async def list_vclusters(self, req: aiohttp.web.Request) -> aiohttp.web.Response: + record_extra_usage_tag(TagKey.CORE_STATE_API_LIST_VCLUSTERS, "1") + return await handle_list_api(self._state_api.list_vclusters, req) + @routes.get("/api/v0/runtime_envs") @RateLimitedModule.enforce_max_concurrent_calls async def list_runtime_envs(self, req: aiohttp.web.Request) -> aiohttp.web.Response: diff --git a/python/ray/dashboard/modules/virtual_cluster/__init__.py b/python/ray/dashboard/modules/virtual_cluster/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/python/ray/dashboard/modules/virtual_cluster/tests/test_virtual_cluster.py b/python/ray/dashboard/modules/virtual_cluster/tests/test_virtual_cluster.py new file mode 100644 index 000000000000..7d229d853110 --- /dev/null +++ b/python/ray/dashboard/modules/virtual_cluster/tests/test_virtual_cluster.py @@ -0,0 +1,621 @@ +import logging +import os +import socket +import tempfile +import time + +import pytest +import requests + +import ray +from ray._private.ray_constants import DEFAULT_DASHBOARD_AGENT_LISTEN_PORT +from ray._private.test_utils import ( + format_web_url, + get_resource_usage, + wait_for_condition, + wait_until_server_available, +) +from ray.cluster_utils import Cluster +from ray.dashboard.tests.conftest import * # noqa +from ray.job_submission import JobStatus, JobSubmissionClient + +logger = logging.getLogger(__name__) + + +def create_or_update_virtual_cluster( + webui_url, virtual_cluster_id, divisible, replica_sets, revision +): + try: + resp = requests.post( + webui_url + "/virtual_clusters", + json={ + "virtualClusterId": virtual_cluster_id, + "divisible": divisible, + "replicaSets": replica_sets, + "revision": revision, + }, + timeout=10, + ) + result = resp.json() + print(result) + return result + except Exception as ex: + logger.info(ex) + + +def remove_virtual_cluster(webui_url, virtual_cluster_id): + try: + resp = requests.delete( + webui_url + "/virtual_clusters/" + virtual_cluster_id, timeout=10 + ) + result = resp.json() + print(result) + return result + except Exception as ex: + logger.info(ex) + + +@ray.remote +class SmallActor: + def pid(self): + return os.getpid() + + +@pytest.mark.parametrize( + "ray_start_cluster_head", + [ + { + "include_dashboard": True, + } + ], + indirect=True, +) +def test_create_and_update_virtual_cluster( + disable_aiohttp_cache, ray_start_cluster_head +): + cluster: Cluster = ray_start_cluster_head + assert wait_until_server_available(cluster.webui_url) is True + webui_url = cluster.webui_url + webui_url = format_web_url(webui_url) + + # Add two nodes to the primary cluster. + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "4c8g"}) + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "8c16g"}) + hostname = socket.gethostname() + revision = 0 + + def _check_create_or_update_virtual_cluster( + virtual_cluster_id, divisible, replica_sets + ): + nonlocal revision + resp = requests.post( + webui_url + "/virtual_clusters", + json={ + "virtualClusterId": virtual_cluster_id, + "divisible": divisible, + "replicaSets": replica_sets, + "revision": revision, + }, + timeout=10, + ) + resp.raise_for_status() + result = resp.json() + print(result) + assert result["result"] is True, resp.text + assert result["data"]["virtualClusterId"] == virtual_cluster_id + current_revision = result["data"]["revision"] + assert current_revision > revision + revision = current_revision + virtual_cluster_replica_sets = {} + for _, node_instance in result["data"]["nodeInstances"].items(): + assert node_instance["hostname"] == hostname + virtual_cluster_replica_sets[node_instance["templateId"]] = ( + virtual_cluster_replica_sets.get(node_instance["templateId"], 0) + 1 + ) + # The virtual cluster has the same node types and count as expected. + assert replica_sets == virtual_cluster_replica_sets + + # Create a new divisible virtual cluster. + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_1", + divisible=True, + replica_sets={"4c8g": 1, "8c16g": 1}, + ) + + # Update the virtual cluster with less nodes (scale down). + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_1", + divisible=True, + replica_sets={"4c8g": 1}, + ) + + # Update the virtual cluster with more nodes (scale up). + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_1", + divisible=True, + replica_sets={"4c8g": 1, "8c16g": 1}, + ) + + # Update the virtual cluster with zero node (make it empty). + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_1", + divisible=True, + replica_sets={}, + ) + + # `virtual_cluster_1` has released all nodes, so we can now + # create a new indivisible virtual cluster with two nodes. + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_2", + divisible=False, + replica_sets={"4c8g": 1, "8c16g": 1}, + ) + + # Update the virtual cluster with less nodes. + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_2", + divisible=False, + replica_sets={"4c8g": 1}, + ) + + # Update the virtual cluster with more nodes. + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_2", + divisible=False, + replica_sets={"4c8g": 1, "8c16g": 1}, + ) + + # Update the virtual cluster with zero node (make it empty). + _check_create_or_update_virtual_cluster( + virtual_cluster_id="virtual_cluster_2", divisible=False, replica_sets={} + ) + + +@pytest.mark.parametrize( + "ray_start_cluster_head", + [ + { + "include_dashboard": True, + } + ], + indirect=True, +) +@pytest.mark.parametrize("divisible", [True, False]) +def test_create_and_update_virtual_cluster_with_exceptions( + disable_aiohttp_cache, ray_start_cluster_head, divisible +): + cluster: Cluster = ray_start_cluster_head + assert wait_until_server_available(cluster.webui_url) is True + webui_url = cluster.webui_url + webui_url = format_web_url(webui_url) + + # Add two nodes to the primary cluster. + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "4c8g"}, resources={"4c8g": 1}) + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "8c16g"}, resources={"8c16g": 1}) + + # Create a new virtual cluster with a non-exist node type. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=divisible, + replica_sets={"16c32g": 1}, + revision=0, + ) + assert result["result"] is False + assert "No enough nodes to add to the virtual cluster" in result["msg"] + replica_sets = result["data"].get("replicaSetsToRecommend", {}) + # The primary cluster can fulfill none `16c32g` node to meet the + # virtual cluster's requirement. + assert replica_sets == {"16c32g": 0} + + # Create a new virtual cluster with node count that the primary cluster + # can not provide. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=divisible, + replica_sets={"4c8g": 2, "8c16g": 1}, + revision=0, + ) + assert result["result"] is False + assert "No enough nodes to add to the virtual cluster" in result["msg"] + replica_sets = result["data"].get("replicaSetsToRecommend", {}) + # The primary cluster can only fulfill one `4c8g` node and one `8c16g` to meet the + # virtual cluster's requirement. + assert replica_sets == {"4c8g": 1, "8c16g": 1} + + # Create a new virtual cluster with one `4c8g` node, which shall succeed. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=divisible, + replica_sets={"4c8g": 1}, + revision=0, + ) + assert result["result"] is True + revision = result["data"]["revision"] + + # Update the virtual cluster with an expired revision. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=divisible, + replica_sets={"4c8g": 2, "8c16g": 2}, + revision=0, + ) + assert result["result"] is False + assert "The revision (0) is expired" in str(result["msg"]) + + # Update the virtual cluster with node count that the primary cluster + # can not provide. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=divisible, + replica_sets={"4c8g": 2, "8c16g": 2}, + revision=revision, + ) + assert result["result"] is False + assert "No enough nodes to add to the virtual cluster" in result["msg"] + replica_sets = result["data"].get("replicaSetsToRecommend", {}) + # The primary cluster can only fulfill one `8c16g` + # node to meet the virtual cluster's requirement. + assert replica_sets == {"4c8g": 0, "8c16g": 1} + + if not divisible: + actor = SmallActor.options(resources={"4c8g": 1}).remote() + ray.get(actor.pid.remote(), timeout=10) + + # Update (scale down) the virtual cluster with one node in use. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=divisible, + replica_sets={}, + revision=revision, + ) + assert result["result"] is False + assert "No enough nodes to remove from the virtual cluster" in result["msg"] + replica_sets = result["data"].get("replicaSetsToRecommend", {}) + # The virtual cluster has one `4c8g` node in use. So we can fulfill none node. + assert replica_sets == {"4c8g": 0} + + # Create a new virtual cluster that the remaining nodes in the primary cluster + # are not enough. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_2", + divisible=divisible, + replica_sets={"4c8g": 1, "8c16g": 1}, + revision=0, + ) + assert result["result"] is False + assert "No enough nodes to add to the virtual cluster" in result["msg"] + replica_sets = result["data"].get("replicaSetsToRecommend", {}) + # The primary cluster lacks one `4c8g` node to meet the + # virtual cluster's requirement. + assert replica_sets == {"4c8g": 0, "8c16g": 1} + + +@pytest.mark.parametrize( + "ray_start_cluster_head", + [ + { + "include_dashboard": True, + }, + ], + indirect=True, +) +def test_remove_virtual_cluster(disable_aiohttp_cache, ray_start_cluster_head): + cluster: Cluster = ray_start_cluster_head + assert wait_until_server_available(cluster.webui_url) is True + webui_url = cluster.webui_url + webui_url = format_web_url(webui_url) + + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "4c8g"}, resources={"4c8g": 1}) + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "8c16g"}, resources={"8c16g": 1}) + + # Create a new divisible virtual cluster. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=True, + replica_sets={"4c8g": 1, "8c16g": 1}, + revision=0, + ) + assert result["result"] is True + + # Try removing a non-exist virtual cluster. + result = remove_virtual_cluster( + webui_url=webui_url, virtual_cluster_id="virtual_cluster_2" + ) + assert result["result"] is False + # The error msg should tell us the virtual cluster does not exit. + assert str(result["msg"]).endswith("does not exist.") + + # Remove the virtual cluster. This will release all nodes. + result = remove_virtual_cluster( + webui_url=webui_url, virtual_cluster_id="virtual_cluster_1" + ) + assert result["result"] is True + + # Create a new indivisible virtual cluster. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_2", + divisible=False, + replica_sets={"4c8g": 1, "8c16g": 1}, + revision=0, + ) + assert result["result"] is True + + # Create an actor that requires some resources. + actor = SmallActor.options(num_cpus=1, resources={"4c8g": 1}).remote() + ray.get(actor.pid.remote(), timeout=10) + + # Remove the virtual cluster. + result = remove_virtual_cluster( + webui_url=webui_url, virtual_cluster_id="virtual_cluster_2" + ) + # The virtual cluster can not be removed because some nodes + # are still in use. + assert result["result"] is False + assert "still in use" in result["msg"] + + ray.kill(actor, no_restart=True) + + # Wait for RESOURCE_VIEW sync message consumed by gcs. + time.sleep(3) + + # Remove the virtual cluster. + result = remove_virtual_cluster( + webui_url=webui_url, virtual_cluster_id="virtual_cluster_2" + ) + assert result["result"] is True + + +@pytest.mark.parametrize( + "ray_start_cluster_head", + [ + { + "include_dashboard": True, + } + ], + indirect=True, +) +def test_get_virtual_clusters(disable_aiohttp_cache, ray_start_cluster_head): + cluster: Cluster = ray_start_cluster_head + assert wait_until_server_available(cluster.webui_url) is True + webui_url = cluster.webui_url + webui_url = format_web_url(webui_url) + hostname = socket.gethostname() + + # Add two `4c8g` nodes and two `8c16g` nodes to the primary cluster. + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "4c8g"}) + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "4c8g"}) + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "8c16g"}) + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "8c16g"}) + + # Create a new indivisible virtual cluster with two `4c8g` nodes. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=False, + replica_sets={"4c8g": 2}, + revision=0, + ) + assert result["result"] is True + + # Create a new divisible virtual cluster with two `8c16g` nodes. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_2", + divisible=True, + replica_sets={"8c16g": 2}, + revision=0, + ) + assert result["result"] is True + + def _get_virtual_clusters(): + try: + resp = requests.get(webui_url + "/virtual_clusters") + resp.raise_for_status() + result = resp.json() + print(result) + assert result["result"] is True, resp.text + for virtual_cluster in result["data"]["virtualClusters"]: + if virtual_cluster["virtualClusterId"] == "virtual_cluster_1": + assert virtual_cluster["divisible"] == "false" + assert len(virtual_cluster["nodeInstances"]) == 2 + for _, node_instance in virtual_cluster["nodeInstances"].items(): + assert node_instance["hostname"] == hostname + assert node_instance["templateId"] == "4c8g" + revision_1 = virtual_cluster["revision"] + assert revision_1 > 0 + elif virtual_cluster["virtualClusterId"] == "virtual_cluster_2": + assert virtual_cluster["divisible"] == "true" + assert len(virtual_cluster["nodeInstances"]) == 2 + for _, node_instance in virtual_cluster["nodeInstances"].items(): + assert node_instance["hostname"] == hostname + assert node_instance["templateId"] == "8c16g" + revision_2 = virtual_cluster["revision"] + assert revision_2 > 0 + else: + return False + # `virtual_cluster_2` should have a more recent revision because it was + # created later than `virtual_cluster_1`. + assert revision_2 > revision_1 + return True + except Exception as ex: + logger.info(ex) + return False + + wait_for_condition(_get_virtual_clusters, timeout=10) + + +# Because raylet is responsible for task scheduling, gcs depends on the resource_view +# sync to get to know the pending/running tasks at each node. If the resource_view sync +# lags, gcs may mistakenly consider one node idle when removing node instances from +# a virtual cluster. In this case, we have to clean up the pending/running tasks at +# the node (just removed). This test makes sure the cleanup is correctly enforced. +@pytest.mark.parametrize( + "ray_start_cluster_head", + [ + { + "include_dashboard": True, + "_system_config": { + "gcs_actor_scheduling_enabled": False, + # Make the resource_view sync message lag. + "raylet_report_resources_period_milliseconds": 30000, + "local_node_cleanup_delay_interval_ms": 10, + }, + } + ], + indirect=True, +) +def test_cleanup_tasks_after_removing_node_instance( + disable_aiohttp_cache, ray_start_cluster_head +): + cluster: Cluster = ray_start_cluster_head + ip, _ = cluster.webui_url.split(":") + agent_address = f"{ip}:{DEFAULT_DASHBOARD_AGENT_LISTEN_PORT}" + assert wait_until_server_available(agent_address) + assert wait_until_server_available(cluster.webui_url) + webui_url = cluster.webui_url + webui_url = format_web_url(webui_url) + + # Add one 4c8g node to the primary cluster. + cluster.add_node(env_vars={"RAY_NODE_TYPE_NAME": "4c8g"}, num_cpus=4) + cluster.wait_for_nodes() + + # Create a virtual cluster with one 4c8g node. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=False, + replica_sets={"4c8g": 1}, + revision=0, + ) + assert result["result"] is True + revision = result["data"]["revision"] + + client = JobSubmissionClient(webui_url) + temp_dir = None + file_path = None + + try: + # Create a temporary directory + temp_dir = tempfile.mkdtemp() + + # Define driver: create two actors, requiring 4 cpus each. + driver_content = """ +import ray +import time + +@ray.remote +class SmallActor(): + def __init__(self): + pass + +actors = [] +for _ in range(2): + actors.append(SmallActor.options(num_cpus=4).remote()) +time.sleep(600) + """ + + # Create a temporary Python file. + file_path = os.path.join(temp_dir, "test_driver.py") + + with open(file_path, "w") as file: + file.write(driver_content) + + absolute_path = os.path.abspath(file_path) + + # Submit the job to the virtual cluster. + job = client.submit_job( + entrypoint=f"python {absolute_path}", + virtual_cluster_id="virtual_cluster_1", + ) + + def check_job_running(): + status = client.get_job_status(job) + return status == JobStatus.RUNNING + + wait_for_condition(check_job_running) + + def check_actors(): + actors = ray._private.state.actors() + # There is only one 4c8g node in the virtual cluster, we shall see + # one alive actor and one actor pending creation. + expected_states = {"ALIVE": 1, "PENDING_CREATION": 1} + actor_states = {} + for _, actor in actors.items(): + if actor["ActorClassName"] == "SmallActor": + actor_states[actor["State"]] = ( + actor_states.get(actor["State"], 0) + 1 + ) + if actor_states == expected_states: + return True + return False + + wait_for_condition(check_actors) + + # Scale down the virtual cluster, removing one node instance. + result = create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id="virtual_cluster_1", + divisible=False, + replica_sets={"4c8g": 0}, + revision=revision, + ) + # Because resource_view sync is lagging, the node instance was + # successfully removed. + assert result["result"] is True + + def check_actors_after_update(): + actors = ray._private.state.actors() + # If the node (just removed from the virtual cluster) cleans up + # its pending and running tasks, we shall see two dead actors now. + expected_states = {"DEAD": 2} + actor_states = {} + for _, actor in actors.items(): + if actor["ActorClassName"] == "SmallActor": + actor_states[actor["State"]] = ( + actor_states.get(actor["State"], 0) + 1 + ) + if actor_states == expected_states: + return True + return False + + wait_for_condition(check_actors_after_update) + + def check_running_and_pending_tasks(): + resources_batch = get_resource_usage( + gcs_address=cluster.head_node.gcs_address + ) + # Check each node's resource usage, making sure no running + # or pending tasks left. + for node in resources_batch.batch: + if "CPU" not in node.resources_available: + return False + if ( + len(node.resource_load_by_shape.resource_demands) > 0 + and node.resource_load_by_shape.resource_demands[ + 0 + ].num_ready_requests_queued + > 0 + ): + return False + return True + + wait_for_condition(check_running_and_pending_tasks) + + finally: + if file_path: + os.remove(file_path) + if temp_dir: + os.rmdir(temp_dir) + + +if __name__ == "__main__": + pass diff --git a/python/ray/dashboard/modules/virtual_cluster/virtual_cluster_head.py b/python/ray/dashboard/modules/virtual_cluster/virtual_cluster_head.py new file mode 100644 index 000000000000..ede5e12726f4 --- /dev/null +++ b/python/ray/dashboard/modules/virtual_cluster/virtual_cluster_head.py @@ -0,0 +1,137 @@ +import logging + +import aiohttp.web + +import ray.dashboard.optional_utils as dashboard_optional_utils +import ray.dashboard.utils as dashboard_utils +from ray.core.generated import gcs_service_pb2_grpc +from ray.core.generated.gcs_service_pb2 import ( + CreateOrUpdateVirtualClusterRequest, + GetVirtualClustersRequest, + RemoveVirtualClusterRequest, +) + +logger = logging.getLogger(__name__) +routes = dashboard_optional_utils.DashboardHeadRouteTable + + +class VirtualClusterHead(dashboard_utils.DashboardHeadModule): + def __init__(self, dashboard_head): + super().__init__(dashboard_head) + + @routes.get("/virtual_clusters") + @dashboard_optional_utils.aiohttp_cache(10) + async def get_all_virtual_clusters(self, req) -> aiohttp.web.Response: + reply = await self._gcs_virtual_cluster_info_stub.GetVirtualClusters( + GetVirtualClustersRequest() + ) + + if reply.status.code == 0: + data = dashboard_utils.message_to_dict( + reply, always_print_fields_with_no_presence=True + ) + for virtual_cluster_data in data.get("virtualClusterDataList", []): + virtual_cluster_data["virtualClusterId"] = virtual_cluster_data.pop( + "id" + ) + virtual_cluster_data["revision"] = int( + virtual_cluster_data.get("revision", 0) + ) + virtual_cluster_data["divisible"] = str( + virtual_cluster_data.pop("divisible", False) + ).lower() + + return dashboard_optional_utils.rest_response( + success=True, + message="All virtual clusters fetched.", + virtual_clusters=data.get("virtualClusterDataList", []), + ) + else: + logger.info("Failed to get all virtual clusters") + return dashboard_optional_utils.rest_response( + success=False, + message="Failed to get all virtual clusters: {}".format( + reply.status.message + ), + ) + + @routes.post("/virtual_clusters") + async def create_or_update_virtual_cluster(self, req) -> aiohttp.web.Response: + virtual_cluster_info_json = await req.json() + logger.info("POST /virtual_clusters %s", virtual_cluster_info_json) + + virtual_cluster_info = dict(virtual_cluster_info_json) + virtual_cluster_id = virtual_cluster_info["virtualClusterId"] + divisible = False + if str(virtual_cluster_info.get("divisible", False)).lower() == "true": + divisible = True + + request = CreateOrUpdateVirtualClusterRequest( + virtual_cluster_id=virtual_cluster_id, + divisible=divisible, + replica_sets=virtual_cluster_info.get("replicaSets", {}), + revision=int(virtual_cluster_info.get("revision", 0)), + ) + reply = await ( + self._gcs_virtual_cluster_info_stub.CreateOrUpdateVirtualCluster(request) + ) + data = dashboard_utils.message_to_dict( + reply, always_print_fields_with_no_presence=True + ) + + if reply.status.code == 0: + logger.info("Virtual cluster %s created or updated", virtual_cluster_id) + + return dashboard_optional_utils.rest_response( + success=True, + message="Virtual cluster created or updated.", + virtual_cluster_id=virtual_cluster_id, + revision=int(data.get("revision", 0)), + node_instances=data.get("nodeInstances", {}), + ) + else: + logger.info( + "Failed to create or update virtual cluster %s", virtual_cluster_id + ) + return dashboard_optional_utils.rest_response( + success=False, + message="Failed to create or update virtual cluster {}: {}".format( + virtual_cluster_id, reply.status.message + ), + virtual_cluster_id=virtual_cluster_id, + replica_sets_to_recommend=data.get("replicaSetsToRecommend", {}), + ) + + @routes.delete("/virtual_clusters/{virtual_cluster_id}") + async def remove_virtual_cluster(self, req) -> aiohttp.web.Response: + virtual_cluster_id = req.match_info.get("virtual_cluster_id") + request = RemoveVirtualClusterRequest(virtual_cluster_id=virtual_cluster_id) + reply = await self._gcs_virtual_cluster_info_stub.RemoveVirtualCluster(request) + + if reply.status.code == 0: + logger.info("Virtual cluster %s removed", virtual_cluster_id) + return dashboard_optional_utils.rest_response( + success=True, + message=f"Virtual cluster {virtual_cluster_id} removed.", + virtual_cluster_id=virtual_cluster_id, + ) + else: + logger.info("Failed to remove virtual cluster %s", virtual_cluster_id) + return dashboard_optional_utils.rest_response( + success=False, + message="Failed to remove virtual cluster {}: {}".format( + virtual_cluster_id, reply.status.message + ), + virtual_cluster_id=virtual_cluster_id, + ) + + async def run(self, server): + self._gcs_virtual_cluster_info_stub = ( + gcs_service_pb2_grpc.VirtualClusterInfoGcsServiceStub( + self.aiogrpc_gcs_channel + ) + ) + + @staticmethod + def is_minimal_module(): + return False diff --git a/python/ray/dashboard/state_aggregator.py b/python/ray/dashboard/state_aggregator.py index 44bc5e74ca1b..584397c465ec 100644 --- a/python/ray/dashboard/state_aggregator.py +++ b/python/ray/dashboard/state_aggregator.py @@ -28,6 +28,7 @@ SummaryApiResponse, TaskState, TaskSummaries, + VirtualClusterState, WorkerState, protobuf_message_to_dict, protobuf_to_task_state_dict, @@ -450,6 +451,178 @@ def transform(replies) -> ListApiResponse: self._thread_pool_executor, transform, replies ) + async def list_vclusters(self, *, option: ListApiOptions) -> ListApiResponse: + """List all virtrual cluster information from the cluster. + + Returns: + {virtual_cluster_id -> virtual_cluster_data_in_dict} + virtual_cluster_data_in_dict's schema is in VirtualClusterState + """ + try: + reply = await self._client.get_all_virtual_cluster_info( + timeout=option.timeout, filters=option.filters + ) + except DataSourceUnavailable: + raise DataSourceUnavailable(GCS_QUERY_FAILURE_WARNING) + + def transform(reply) -> ListApiResponse: + result = [] + for message in reply.virtual_clusters_view: + data = protobuf_message_to_dict( + message=message, + fields_to_decode=[], + ) + result.append(data) + + entries = {} + for entry in result: + entry = { + "virtual_cluster_id": entry["id"], + "divided_clusters": {}, + "divisible": entry["divisible"], + "replica_sets": {}, + "undivided_replica_sets": {}, + "visible_node_instances": entry.get("node_instance_views", {}), + "undivided_nodes": {}, + "resources_total": {}, + "resources_available": {}, + "resources_usage": {}, + } + entries[entry["virtual_cluster_id"]] = entry + + primary_cluster = entries["kPrimaryClusterID"] + + all_nodes = {} + cluster_nodes = {} + for id, entry in entries.items(): + cluster_nodes[id] = set() + if id != "kPrimaryClusterID" and "##" not in id: + primary_cluster["divided_clusters"][id] = ( + "divisble" if entry["divisible"] else "indivisible" + ) + elif "##" in id: + parent_cluster_id = id.split("##")[0] + entries[parent_cluster_id]["divided_clusters"][id] = ( + "divisble" if entry["divisible"] else "indivisible" + ) + all_nodes.update(entry["visible_node_instances"]) + # update cluster nodes to calculate template ids + for id, entry in entries.items(): + for sub_cluster_id in entry["divided_clusters"]: + cluster_nodes[id].update( + entries[sub_cluster_id]["visible_node_instances"].keys() + ) + cluster_nodes[id].update(entry["visible_node_instances"].keys()) + + # calculate template ids + for id, entry in entries.items(): + for node_id in cluster_nodes[id]: + node = all_nodes[node_id] + if not node["is_dead"] and node["template_id"] != "": + entry["replica_sets"][node["template_id"]] = ( + entry["replica_sets"].get(node["template_id"], 0) + 1 + ) + + def collect_all_sub_nodes(virtual_cluster_id): + ret = set() + entry = entries[virtual_cluster_id] + for sub_cluster_id in entry["divided_clusters"]: + ret.update(collect_all_sub_nodes(sub_cluster_id)) + ret.update(entry["visible_node_instances"].keys()) + return ret + + for id, entry in entries.items(): + divided_nodes = set() + for sub_cluster_id in entry["divided_clusters"]: + divided_nodes.update(collect_all_sub_nodes(sub_cluster_id)) + undivided_nodes = {} + for node_id, node in entry["visible_node_instances"].items(): + if node_id not in divided_nodes: + undivided_nodes[node_id] = node + if not node["is_dead"] and node["template_id"] != "": + entry["undivided_replica_sets"][node["template_id"]] = ( + entry["undivided_replica_sets"].get( + node["template_id"], 0 + ) + + 1 + ) + entry["undivided_nodes"] = undivided_nodes + full_nodes = divided_nodes.union(undivided_nodes.keys()) + for node_id in full_nodes: + node = all_nodes[node_id] + if not node["is_dead"] and node["template_id"] != "": + for resource, value in node["resources_total"].items(): + entry["resources_total"][resource] = ( + entry["resources_total"].get(resource, 0) + value + ) + for resource, value in node["resources_available"].items(): + entry["resources_available"][resource] = ( + entry["resources_available"].get(resource, 0) + value + ) + + def readable_memory(x: int): + if x >= 2**30: + return str(format(x / (2**30), ".3f")) + " GiB" + elif x >= 2**20: + return str(format(x / (2**20), ".3f")) + " MiB" + elif x >= 2**10: + return str(format(x / (2**10), ".3f")) + " KiB" + return str(format(x, ".3f")) + " B" + + for entry in entries.values(): + for node_id, node in entry["visible_node_instances"].items(): + if "resources_total" in node: + del node["resources_total"] + if "resources_available" in node: + del node["resources_available"] + for node_id, node in entry["undivided_nodes"].items(): + if "resources_total" in node: + del node["resources_total"] + if "resources_available" in node: + del node["resources_available"] + for resource, value in entry["resources_available"].items(): + entry["resources_available"][resource] = ( + entry["resources_total"][resource] - value + ) + if "memory" in resource: + entry["resources_available"][resource] = readable_memory( + entry["resources_available"][resource] + ) + for resource, value in entry["resources_total"].items(): + if "memory" in resource: + entry["resources_total"][resource] = readable_memory(value) + for resource, value in entry["resources_total"].items(): + if resource in ["memory", "CPU", "object_store_memory"]: + entry["resources_usage"][ + resource + ] = f"""{entry["resources_available"][resource]} / {value}""" + del entry["resources_available"] + del entry["resources_total"] + + result = list(entries.values()) + + num_after_truncation = len(result) + result = do_filter( + result, option.filters, VirtualClusterState, option.detail + ) + num_filtered = len(result) + + # Sort to make the output deterministic. + result.sort(key=lambda entry: entry["virtual_cluster_id"]) + + result = list(islice(result, option.limit)) + + return ListApiResponse( + result=result, + total=reply.total, + num_after_truncation=num_after_truncation, + num_filtered=num_filtered, + ) + + return await get_or_create_event_loop().run_in_executor( + self._thread_pool_executor, transform, reply + ) + async def list_runtime_envs(self, *, option: ListApiOptions) -> ListApiResponse: """List all runtime env information from the cluster. diff --git a/python/ray/data/BUILD b/python/ray/data/BUILD index 77e1b6350c64..45320138be12 100644 --- a/python/ray/data/BUILD +++ b/python/ray/data/BUILD @@ -41,6 +41,14 @@ py_test( deps = ["//:ray_lib", ":conftest"], ) +py_test( + name = "test_strict_in_virtual_cluster", + size = "small", + srcs = ["tests/test_strict_in_virtual_cluster.py"], + tags = ["team:data", "exclusive"], + deps = ["//:ray_lib", ":conftest"], +) + py_test( name = "test_strict_mode", size = "small", diff --git a/python/ray/data/tests/conftest.py b/python/ray/data/tests/conftest.py index 9e6f06ec93a0..f449df30861e 100644 --- a/python/ray/data/tests/conftest.py +++ b/python/ray/data/tests/conftest.py @@ -3,26 +3,35 @@ import posixpath import time from collections import defaultdict +from contextlib import contextmanager import numpy as np import pandas as pd import pyarrow as pa import pytest +import requests import ray import ray.util.state from ray._private.internal_api import get_memory_info_reply, get_state_from_address +from ray._private.ray_constants import DEFAULT_DASHBOARD_AGENT_LISTEN_PORT +from ray._private.test_utils import format_web_url, wait_until_server_available from ray._private.utils import _get_pyarrow_version from ray.air.constants import TENSOR_COLUMN_NAME from ray.air.util.tensor_extensions.arrow import ArrowTensorArray +from ray.cluster_utils import Cluster from ray.data import Schema from ray.data.block import BlockExecStats, BlockMetadata from ray.data.tests.mock_server import * # noqa +from ray.job_submission import JobSubmissionClient # Trigger pytest hook to automatically zip test cluster logs to archive dir on failure from ray.tests.conftest import * # noqa -from ray.tests.conftest import pytest_runtest_makereport # noqa -from ray.tests.conftest import _ray_start, wait_for_condition +from ray.tests.conftest import ( + _ray_start, + get_default_fixture_ray_kwargs, + wait_for_condition, +) @pytest.fixture(scope="module") @@ -717,3 +726,110 @@ def _assert(last_snapshot): @pytest.fixture(autouse=True, scope="function") def log_internal_stack_trace_to_stdout(restore_data_context): ray.data.context.DataContext.get_current().log_internal_stack_trace_to_stdout = True + + +@contextmanager +def _ray_start_cluster(**kwargs): + init_kwargs = get_default_fixture_ray_kwargs() + num_nodes = 0 + do_init = False + # num_nodes & do_init are not arguments for ray.init, so delete them. + if "num_nodes" in kwargs: + num_nodes = kwargs["num_nodes"] + del kwargs["num_nodes"] + if "do_init" in kwargs: + do_init = kwargs["do_init"] + del kwargs["do_init"] + elif num_nodes > 0: + do_init = True + init_kwargs.update(kwargs) + namespace = init_kwargs.pop("namespace") + cluster = Cluster() + remote_nodes = [] + for i in range(num_nodes): + if i > 0 and "_system_config" in init_kwargs: + del init_kwargs["_system_config"] + remote_nodes.append(cluster.add_node(**init_kwargs)) + # We assume driver will connect to the head (first node), + # so ray init will be invoked if do_init is true + if len(remote_nodes) == 1 and do_init: + ray.init(address=cluster.address, namespace=namespace) + yield cluster + # The code after the yield will run as teardown code. + ray.shutdown() + cluster.shutdown() + + +def create_or_update_virtual_cluster( + webui_url, virtual_cluster_id, allocation_mode, replica_sets, revision +): + try: + resp = requests.post( + webui_url + "/virtual_clusters", + json={ + "virtualClusterId": virtual_cluster_id, + "allocationMode": allocation_mode, + "replicaSets": replica_sets, + "revision": revision, + }, + timeout=10, + ) + result = resp.json() + print(result) + return result + except Exception as ex: + print(ex) + + +@pytest.fixture +def create_virtual_cluster(request): + param = getattr(request, "param", {}) + node_instances = param.get("node_instances", None) + virtual_cluster = param.get("virtual_cluster", None) + default_node_instances = [ + ("1c2g", 3), + ("2c4g", 3), + ("4c8g", 3), + ] + default_virtual_cluster = {} + for i in range(3): + default_virtual_cluster[f"VIRTUAL_CLUSTER_{i}"] = { + "allocation_mode": "mixed", + "replica_sets": { + "1c2g": 1, + "2c4g": 1, + "4c8g": 1, + }, + "revision": 0, + } + if node_instances is None or len(node_instances) == 0: + node_instances = default_node_instances + if virtual_cluster is None or len(virtual_cluster) == 0: + virtual_cluster = default_virtual_cluster + with _ray_start_cluster(do_init=True, num_nodes=1) as cluster: + webui_url = cluster.webui_url + ip, _ = webui_url.split(":") + webui_url = format_web_url(webui_url) + agent_address = f"{ip}:{DEFAULT_DASHBOARD_AGENT_LISTEN_PORT}" + assert wait_until_server_available(agent_address) + assert wait_until_server_available(cluster.webui_url) is True + for node_type, amount in node_instances: + num_cpus = node_type.split("c")[0] + for _ in range(amount): + cluster.add_node( + env_vars={"RAY_NODE_TYPE_NAME": node_type}, num_cpus=int(num_cpus) + ) + + for virtual_cluster_id, config in virtual_cluster.items(): + create_or_update_virtual_cluster( + webui_url=webui_url, + virtual_cluster_id=virtual_cluster_id, + allocation_mode=config.get("allocation_mode", "mixed"), + replica_sets=config.get("replica_sets", {}), + revision=config.get("revision", 0), + ) + + yield ( + cluster, + JobSubmissionClient(format_web_url(webui_url)), + ) diff --git a/python/ray/data/tests/test_strict_in_virtual_cluster.py b/python/ray/data/tests/test_strict_in_virtual_cluster.py new file mode 100644 index 000000000000..9fc9908949a8 --- /dev/null +++ b/python/ray/data/tests/test_strict_in_virtual_cluster.py @@ -0,0 +1,421 @@ +import logging +import sys +import tempfile +from pathlib import Path + +import pytest +import requests + +import ray +from ray._private.runtime_env.working_dir import upload_working_dir_if_needed +from ray._private.test_utils import format_web_url, wait_for_condition +from ray.dashboard.tests.conftest import * # noqa +from ray.data.tests.conftest import * # noqa +from ray.runtime_env.runtime_env import RuntimeEnv + +logger = logging.getLogger(__name__) + + +@ray.remote +class JobSignalActor: + def __init__(self): + self._ready = False + self._data = None + self._barrier = False + + def is_ready(self): + return self._ready + + def ready(self): + self._ready = True + + def unready(self): + self._ready = False + self._barrier = False + + def set_barrier(self): + self._barrier = True + + def get_barrier(self): + return self._barrier + + def data(self, data=None): + if data is not None: + self._data = data + return self._data + + +def submit_job_to_virtual_cluster( + job_client, tmp_dir, driver_script, virtual_cluster_id +): + path = Path(tmp_dir) + test_script_file = path / "test_script.py" + with open(test_script_file, "w+") as file: + file.write(driver_script) + + runtime_env = {"working_dir": tmp_dir} + runtime_env = upload_working_dir_if_needed(runtime_env, tmp_dir, logger=logger) + runtime_env = RuntimeEnv(**runtime_env).to_dict() + + job_id = job_client.submit_job( + entrypoint="python test_script.py", + entrypoint_memory=1, + runtime_env=runtime_env, + virtual_cluster_id=virtual_cluster_id, + ) + return job_id + + +def get_virtual_cluster_nodes(webui_url, virtual_cluster_id): + resp = requests.get( + webui_url + "/virtual_clusters", + timeout=10, + ) + resp.raise_for_status() + result = resp.json() + assert result["result"] is True, resp.text + assert len(result["data"]["virtualClusters"]) > 0 + virtual_clusters = result["data"]["virtualClusters"] + for vc in virtual_clusters: + if vc["virtualClusterId"] == virtual_cluster_id: + return vc + return None + + +def get_job_actors(webui_url, job_id=None): + resp = requests.get( + webui_url + "/logical/actors", + timeout=10, + ) + resp.raise_for_status() + result = resp.json() + assert result["result"] is True, resp.text + actors = result["data"]["actors"] + print(f"Fetched {job_id} actors: {actors}") + if job_id is not None: + final_actors = {} + for actor_id, info in actors.items(): + if info["jobId"] == job_id: + final_actors[actor_id] = info + return final_actors + return actors + + +def check_job_actor_in_virtual_cluster(webui_url, job_id, virtual_cluster_id): + webui_url = format_web_url(webui_url) + job_actors = get_job_actors(webui_url, job_id) + target_virtual_cluster = get_virtual_cluster_nodes(webui_url, virtual_cluster_id) + target_virtual_cluster_node_ids = set( + target_virtual_cluster["nodeInstances"].keys() + ) + for actor_id, actor_info in job_actors.items(): + assert actor_info["address"]["rayletId"] in target_virtual_cluster_node_ids + + +@pytest.mark.parametrize( + "create_virtual_cluster", + [ + { + "node_instances": [("1c2g", 2), ("2c4g", 2), ("8c16g", 4)], + "virtual_cluster": { + "VIRTUAL_CLUSTER_0": { + "allocation_mode": "mixed", + "replica_sets": { + "1c2g": 2, + }, + }, + "VIRTUAL_CLUSTER_1": { + "allocation_mode": "mixed", + "replica_sets": { + "2c4g": 2, + }, + }, + "VIRTUAL_CLUSTER_2": { + "allocation_mode": "mixed", + "replica_sets": { + "8c16g": 4, + }, + }, + }, + } + ], + indirect=True, +) +def test_auto_parallelism(create_virtual_cluster): + """Tests that the parallelism can be auto-deduced by + data size and current virtual cluster's resources. + """ + _, job_client = create_virtual_cluster + MiB = 1024 * 1024 + # (data size, expected parallelism in each virtual cluster) + TEST_CASES = [ + # Should all be avail_cpus * 2 + (1024, (4, 8, 64)), + # Should be MAX_PARALLELISM, MAX_PARALLELISM, avail_cpus * 2 + # MAX_PARALLELISM = 10MB(data size) / 1MB(block size) + (10 * MiB, (10, 10, 64)), + ] + + with tempfile.TemporaryDirectory() as tmp_dir: + for i in range(3): # 3 virtual clusters in total + signal_actor_name = f"storage_actor_{i}" + signal_actor = JobSignalActor.options( + name=signal_actor_name, namespace="storage", num_cpus=0 + ).remote() + # Just to make sure the actor is running, signal actor + # should not be "ready" at this time until the submitted job + # is running + assert not ray.get(signal_actor.is_ready.remote()) + for data_size, expected_parallelism in TEST_CASES: + driver_script = """ +import ray +import time +from ray.data._internal.util import _autodetect_parallelism +from ray.data.context import DataContext + + +data_size = {data_size} +signal_actor_name = "{signal_actor_name}" +print("Job is running, data_size: ", data_size, + "signal_actor_name: ", signal_actor_name) +ray.init(address="auto") +signal_actor = ray.get_actor(signal_actor_name, namespace="storage") +ray.get(signal_actor.ready.remote()) +target_max_block_size = DataContext.get_current().target_max_block_size +class MockReader: + def estimate_inmemory_data_size(self): + return data_size + + +final_parallelism, _, _ = _autodetect_parallelism( + parallelism=-1, + target_max_block_size=target_max_block_size, + ctx=DataContext.get_current(), + datasource_or_legacy_reader=MockReader(), +) + +while(True): + barrier = ray.get(signal_actor.get_barrier.remote()) + if barrier: + break + time.sleep(0.1) + +ray.get(signal_actor.data.remote(final_parallelism)) +ray.get(signal_actor.unready.remote()) +""" + driver_script = driver_script.format( + signal_actor_name=signal_actor_name, data_size=data_size + ) + submit_job_to_virtual_cluster( + job_client, tmp_dir, driver_script, f"VIRTUAL_CLUSTER_{i}" + ) + # wait for job running + wait_for_condition( + lambda: ray.get(signal_actor.is_ready.remote()), timeout=40 + ) + # Give finish permission to the job in case it's finished + # before we catched the "ready" signal + ray.get(signal_actor.set_barrier.remote()) + # wait for job finish + wait_for_condition( + lambda: not ray.get(signal_actor.is_ready.remote()), timeout=30 + ) + # retrieve job data and check + res = ray.get(signal_actor.data.remote()) + print( + f"Driver detected parallelism: {res}, " + f"expect[{i}]: {expected_parallelism[i]}" + ) + assert res == expected_parallelism[i] + + +def test_job_in_virtual_cluster(create_virtual_cluster): + """Tests that an e2e ray data job's actor can be restricted + only in the target virtual cluster + """ + cluster, job_client = create_virtual_cluster + + with tempfile.TemporaryDirectory() as tmp_dir: + for i in range(3): # 3 virtual clusters in total + signal_actor_name = f"storage_actor_{i}" + signal_actor = JobSignalActor.options( + name=signal_actor_name, namespace="storage", num_cpus=0 + ).remote() + # Just to make sure the actor is running, signal actor + # should not be "ready" at this time until the submitted job + # is running + assert not ray.get(signal_actor.is_ready.remote()) + driver_script = """ +import ray +import time + +ray.init(address="auto") +signal_actor = ray.get_actor("{signal_actor_name}", namespace="storage") +ray.get(signal_actor.ready.remote()) + +def map_fn(row): + row['species'] = row['variety'] + '!!!' + return row + +def flat_map_fn(row): + new_row = row.copy() + new_row['species'] = row['variety'] + '???' + return [row, new_row] + +ds = ray.data.read_csv("s3://anonymous@air-example-data/iris.csv") +ds = ds.map(map_fn).flat_map(flat_map_fn) + +while(True): + barrier = ray.get(signal_actor.get_barrier.remote()) + if barrier: + break + time.sleep(0.1) + +ray.get(signal_actor.unready.remote()) +""" + driver_script = driver_script.format(signal_actor_name=signal_actor_name) + job_id = submit_job_to_virtual_cluster( + job_client, tmp_dir, driver_script, f"VIRTUAL_CLUSTER_{i}" + ) + # wait for job running + wait_for_condition( + lambda: ray.get(signal_actor.is_ready.remote()), timeout=40 + ) + # Give finish permission to the job in case it's finished + # before we catched the "ready" signal + ray.get(signal_actor.set_barrier.remote()) + # wait for job finish + wait_for_condition( + lambda: not ray.get(signal_actor.is_ready.remote()), timeout=40 + ) + check_job_actor_in_virtual_cluster( + cluster.webui_url, job_id, f"VIRTUAL_CLUSTER_{i}" + ) + + +@pytest.mark.parametrize( + "create_virtual_cluster", + [ + { + "node_instances": [("2c4g", 1), ("8c16g", 1)], + "virtual_cluster": { + "VIRTUAL_CLUSTER_0": { + "allocation_mode": "mixed", + "replica_sets": { + "2c4g": 1, + }, + }, + "VIRTUAL_CLUSTER_1": { + "allocation_mode": "mixed", + "replica_sets": { + "8c16g": 1, + }, + }, + }, + } + ], + indirect=True, +) +def test_start_actor_timeout(create_virtual_cluster): + """Tests that when requesting resources that exceeding only + the virtual cluster but the whole cluster, will raise an exception + on timeout while waiting for actors.""" + + cluster, job_client = create_virtual_cluster + + # achievable & unachievable cpu requirements + TEST_CASES = [ + (1, 4), # for virtual cluster 0 + (6, 36), # for virtual cluster 1 + ] + with tempfile.TemporaryDirectory() as tmp_dir: + for i in range(2): # 2 virtual clusters in total + signal_actor_name = f"storage_actor_{i}" + signal_actor = JobSignalActor.options( + name=signal_actor_name, namespace="storage", num_cpus=0 + ).remote() + # Just to make sure the actor is running, signal actor + # should not be "ready" at this time until the submitted job + # is running + ray.get(signal_actor.is_ready.remote()) + driver_script = """ +import ray +import time +from ray.exceptions import GetTimeoutError + + +class UDFClass: + def __call__(self, x): + return x + +ray.init(address="auto") +signal_actor = ray.get_actor("{signal_actor_name}", namespace="storage") +ray.get(signal_actor.ready.remote()) +ray.data.DataContext.get_current().wait_for_min_actors_s = 10 +result = False +# Specify an unachievable resource requirement to ensure +# we timeout while waiting for actors. + +achievable_cpus = {achievable_cpus} +unachievable_cpus = {unachievable_cpus} +print("Requesting achievable_cpus: ", achievable_cpus) +try: + ray.data.range(10).map_batches( + UDFClass, + batch_size=1, + compute=ray.data.ActorPoolStrategy(size=1), + num_cpus=achievable_cpus, + ).take_all() + result = True +except GetTimeoutError as e: + print("This shouldn't happen") + result = False + +print("Requesting unachievable_cpus: ", unachievable_cpus) +try: + ray.data.range(10).map_batches( + UDFClass, + batch_size=1, + compute=ray.data.ActorPoolStrategy(size=1), + num_cpus=unachievable_cpus, + ).take_all() + result = False +except GetTimeoutError as e: + result = result and True + +print("Final result: ", result) + +while(True): + barrier = ray.get(signal_actor.get_barrier.remote()) + if barrier: + break + time.sleep(0.1) + +ray.get(signal_actor.data.remote(result)) +ray.get(signal_actor.unready.remote()) +""" + driver_script = driver_script.format( + signal_actor_name=signal_actor_name, + achievable_cpus=TEST_CASES[i][0], + unachievable_cpus=TEST_CASES[i][1], + ) + + submit_job_to_virtual_cluster( + job_client, tmp_dir, driver_script, f"VIRTUAL_CLUSTER_{i}" + ) + # wait for job running + wait_for_condition( + lambda: ray.get(signal_actor.is_ready.remote()), timeout=30 + ) + # Give finish permission to the job in case it's finished + # before we catched the "ready" signal + ray.get(signal_actor.set_barrier.remote()) + # wait for job finish + wait_for_condition( + lambda: not ray.get(signal_actor.is_ready.remote()), timeout=600 + ) + assert ray.get(signal_actor.data.remote()) + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/includes/global_state_accessor.pxd b/python/ray/includes/global_state_accessor.pxd index 0671ff289c03..16c6c24b9f44 100644 --- a/python/ray/includes/global_state_accessor.pxd +++ b/python/ray/includes/global_state_accessor.pxd @@ -15,6 +15,7 @@ from ray.includes.unique_ids cimport ( CObjectID, CWorkerID, CPlacementGroupID, + CVirtualClusterID, ) from ray.includes.common cimport ( CRayStatus, @@ -32,9 +33,9 @@ cdef extern from "ray/gcs/gcs_client/global_state_accessor.h" nogil: c_vector[c_string] GetAllJobInfo( c_bool skip_submission_job_info_field, c_bool skip_is_running_tasks_field) CJobID GetNextJobID() - c_vector[c_string] GetAllNodeInfo() - c_vector[c_string] GetAllAvailableResources() - c_vector[c_string] GetAllTotalResources() + c_vector[c_string] GetAllNodeInfo(const c_string virtual_cluster_id) + c_vector[c_string] GetAllAvailableResources(const c_string virtual_cluster_id) + c_vector[c_string] GetAllTotalResources(const c_string virtual_cluster_id) unordered_map[CNodeID, c_int64_t] GetDrainingNodes() unique_ptr[c_string] GetInternalKV( const c_string &namespace, const c_string &key) diff --git a/python/ray/includes/global_state_accessor.pxi b/python/ray/includes/global_state_accessor.pxi index 29f3acb832b1..52c11343b09b 100644 --- a/python/ray/includes/global_state_accessor.pxi +++ b/python/ray/includes/global_state_accessor.pxi @@ -66,14 +66,20 @@ cdef class GlobalStateAccessor: cjob_id = self.inner.get().GetNextJobID() return cjob_id.ToInt() - def get_node_table(self): + def get_node_table(self, virtual_cluster_id): cdef: c_vector[c_string] items c_string item CGcsNodeInfo c_node_info unordered_map[c_string, double] c_resources + cdef c_string cvirtual_cluster_id + + if virtual_cluster_id is None: + cvirtual_cluster_id = b"" + else: + cvirtual_cluster_id = virtual_cluster_id with nogil: - items = self.inner.get().GetAllNodeInfo() + items = self.inner.get().GetAllNodeInfo(cvirtual_cluster_id) results = [] for item in items: c_node_info.ParseFromString(item) @@ -135,16 +141,26 @@ cdef class GlobalStateAccessor: return c_string(result.get().data(), result.get().size()) return None - def get_all_available_resources(self): + def get_all_available_resources(self, virtual_cluster_id): cdef c_vector[c_string] result + cdef c_string cvirtual_cluster_id + if virtual_cluster_id is None: + cvirtual_cluster_id = b"" + else: + cvirtual_cluster_id = virtual_cluster_id with nogil: - result = self.inner.get().GetAllAvailableResources() + result = self.inner.get().GetAllAvailableResources(cvirtual_cluster_id) return result - def get_all_total_resources(self): + def get_all_total_resources(self, virtual_cluster_id): cdef c_vector[c_string] result + cdef c_string cvirtual_cluster_id + if virtual_cluster_id is None: + cvirtual_cluster_id = b"" + else: + cvirtual_cluster_id = virtual_cluster_id with nogil: - result = self.inner.get().GetAllTotalResources() + result = self.inner.get().GetAllTotalResources(cvirtual_cluster_id) return result def get_task_events(self): diff --git a/python/ray/includes/unique_ids.pxd b/python/ray/includes/unique_ids.pxd index 84f511ec5107..ba3454e2429c 100644 --- a/python/ray/includes/unique_ids.pxd +++ b/python/ray/includes/unique_ids.pxd @@ -216,3 +216,30 @@ cdef extern from "ray/common/id.h" namespace "ray" nogil: CPlacementGroupID Of(CJobID job_id) ctypedef uint32_t ObjectIDIndexType + + +cdef extern from "ray/common/simple_id.h" namespace "ray" nogil: + cdef cppclass CSimpleID[T]: + @staticmethod + T FromBinary(const c_string &binary) + + @staticmethod + const T Nil() + + size_t Hash() const + c_bool IsNil() const + c_bool operator==(const CBaseID &rhs) const + c_bool operator!=(const CBaseID &rhs) const + + c_string Binary() const + + +cdef extern from "ray/common/virtual_cluster_id.h" namespace "ray" nogil: + cdef cppclass CVirtualClusterID "ray::VirtualClusterID" \ + (CSimpleID[CVirtualClusterID]): + + @staticmethod + CVirtualClusterID FromBinary(const c_string &binary) + + @staticmethod + const CVirtualClusterID Nil() diff --git a/python/ray/includes/unique_ids.pxi b/python/ray/includes/unique_ids.pxi index 5435d26a45e1..4da4dc488d2f 100644 --- a/python/ray/includes/unique_ids.pxi +++ b/python/ray/includes/unique_ids.pxi @@ -23,6 +23,8 @@ from ray.includes.unique_ids cimport ( CWorkerID, CPlacementGroupID, CClusterID, + CSimpleID, + CVirtualClusterID, ) @@ -135,6 +137,47 @@ cdef class UniqueID(BaseID): cdef size_t hash(self): return self.data.Hash() +cdef class SimpleID: + + @classmethod + def from_binary(cls, id_bytes): + return cls(id_bytes) + + def binary(self): + raise NotImplementedError + + def is_nil(self): + raise NotImplementedError + + def __hash__(self): + return self.hash() + + def __eq__(self, other): + return type(self) == type(other) and self.binary() == other.binary() + + def __ne__(self, other): + return type(self) != type(other) or self.binary() != other.binary() + + def __bytes__(self): + return self.binary() + + def __repr__(self): + return self.__class__.__name__ + "(" + self.binary() + ")" + + def __str__(self): + return self.__repr__() + + def __reduce__(self): + return type(self), (self.binary(),) + + def redis_shard_hash(self): + # NOTE: The hash function used here must match the one in + # GetRedisContext in src/ray/gcs/tables.h. Changes to the + # hash function should only be made through std::hash in + # src/common/common.h. + # Do not use __hash__ that returns signed uint64_t, which + # is different from std::hash in c++ code. + return self.hash() cdef class TaskID(BaseID): cdef CTaskID data @@ -383,6 +426,12 @@ cdef class ClusterID(UniqueID): cdef CClusterID native(self): return self.data +cdef class VirtualClusterID(SimpleID): + cdef CVirtualClusterID data + + def __init__(self, id): + self.data = CVirtualClusterID.FromBinary(id) + # This type alias is for backward compatibility. ObjectID = ObjectRef @@ -443,4 +492,6 @@ _ID_TYPES = [ UniqueID, PlacementGroupID, ClusterID, + SimpleID, + VirtualClusterID, ] diff --git a/python/ray/runtime_context.py b/python/ray/runtime_context.py index 5cacae69371b..bb01fca8d805 100644 --- a/python/ray/runtime_context.py +++ b/python/ray/runtime_context.py @@ -482,6 +482,11 @@ def runtime_env(self): return RuntimeEnv.deserialize(self.get_runtime_env_string()) + @property + def virtual_cluster_id(self): + + return self.worker.current_virtual_cluster_id + @property def current_actor(self): """Get the current actor handle of this actor itself. diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 1bb2e536bd13..ac7519c1f3e8 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -1,4 +1,5 @@ import copy +import asyncio import json import logging import os @@ -11,6 +12,7 @@ import urllib.parse import warnings import shutil +from concurrent.futures import ThreadPoolExecutor from datetime import datetime from typing import Optional, Set, List, Tuple from ray.dashboard.modules.metrics import install_and_start_prometheus @@ -23,6 +25,10 @@ import yaml import ray +from ray.util.state.common import ( + DEFAULT_LIMIT, + DEFAULT_RPC_TIMEOUT, +) import ray._private.ray_constants as ray_constants import ray._private.services as services from ray._private.utils import ( @@ -33,6 +39,18 @@ from ray._private.internal_api import memory_summary from ray._private.storage import _load_class from ray._private.usage import usage_lib +from ray._private.gcs_aio_client import GcsAioClient +from ray._private.gcs_utils import GcsChannel +from ray.core.generated import gcs_service_pb2_grpc +from ray.core.generated.gcs_service_pb2 import ( + CreateOrUpdateVirtualClusterRequest, + GetVirtualClustersRequest, + RemoveVirtualClusterRequest, +) +from ray.dashboard.state_aggregator import StateAPIManager +from ray.util.state.common import ListApiOptions, VirtualClusterState +from ray.util.state.state_cli import AvailableFormat, format_list_api_output +from ray.util.state.state_manager import StateDataSourceClient from ray.autoscaler._private.cli_logger import add_click_logging_options, cf, cli_logger from ray.autoscaler._private.commands import ( RUN_ENV_TYPES, @@ -2622,6 +2640,299 @@ def add_command_alias(command, name, hidden): cli.add_command(new_command, name=name) +def _get_virtual_cluster_stub(address: Optional[str] = None): + address = services.canonicalize_bootstrap_address_or_die(address) + channel = GcsChannel(address, aio=False) + channel.connect() + return gcs_service_pb2_grpc.VirtualClusterInfoGcsServiceStub(channel.channel()) + + +@click.group("vcluster") +def vclusters_cli_group(): + """Create, update or remove virtual clusters.""" + pass + + +@vclusters_cli_group.command() +@click.option( + "--address", + type=str, + default=None, + required=False, + help=( + "Address of the Ray cluster to connect to. Can also be specified " + "using the RAY_ADDRESS environment variable." + ), +) +@click.option( + "--id", + type=str, + required=True, + help="Virtual Cluster ID to create.", +) +@click.option( + "--divisible", + is_flag=True, + default=False, + help="Whether the virtual cluster is divisible.", +) +@click.option( + "--replica-sets", + type=str, + default=None, + required=True, + help="JSON-serialized dictionary of replica sets.", +) +@add_click_logging_options +def create( + address: Optional[str], + id: str, + divisible: bool, + replica_sets: Optional[str], +): + """Create a new virtual cluster.""" + stub = _get_virtual_cluster_stub(address) + reply = stub.GetVirtualClusters(GetVirtualClustersRequest(virtual_cluster_id=id)) + if reply.status.code != 0: + cli_logger.error( + f"Failed to create virtual cluster '{id}': {reply.status.message}" + ) + sys.exit(1) + + if len(reply.virtual_cluster_data_list) > 0: + cli_logger.error(f"Failed to create virtual cluster '{id}': already exists") + sys.exit(1) + + if replica_sets is not None: + replica_sets = json.loads(replica_sets) + + request = CreateOrUpdateVirtualClusterRequest( + virtual_cluster_id=id, + divisible=divisible, + replica_sets=replica_sets, + ) + + reply = stub.CreateOrUpdateVirtualCluster(request) + + if reply.status.code == 0: + cli_logger.success(f"Virtual cluster '{id}' created successfully") + else: + cli_logger.error( + f"Failed to create virtual cluster '{id}': {reply.status.message}" + ) + sys.exit(1) + + +@vclusters_cli_group.command() +@click.option( + "--address", + type=str, + default=None, + required=False, + help=( + "Address of the Ray cluster to connect to. Can also be specified " + "using the RAY_ADDRESS environment variable." + ), +) +@click.option( + "--id", + type=str, + required=True, + help="Virtual Cluster ID to create.", +) +@click.option( + "--divisible", + is_flag=True, + default=False, + help="Whether the virtual cluster is divisible.", +) +@click.option( + "--replica-sets", + type=str, + default=None, + required=True, + help="JSON-serialized dictionary of replica sets.", +) +@click.option( + "--revision", + type=int, + required=True, + help="Revision number for the virtual cluster.", +) +@add_click_logging_options +def update( + address: Optional[str], + id: str, + divisible: bool, + replica_sets: Optional[str], + revision: int, +): + """Update an existing virtual cluster.""" + stub = _get_virtual_cluster_stub(address) + if replica_sets is not None: + replica_sets = json.loads(replica_sets) + + request = CreateOrUpdateVirtualClusterRequest( + virtual_cluster_id=id, + divisible=divisible, + replica_sets=replica_sets, + revision=revision, + ) + + reply = stub.CreateOrUpdateVirtualCluster(request) + + if reply.status.code == 0: + cli_logger.success(f"Virtual cluster '{id}' updated successfully") + else: + cli_logger.error( + f"Failed to update virtual cluster '{id}': {reply.status.message}" + ) + sys.exit(1) + + +@vclusters_cli_group.command() +@click.option( + "--address", + type=str, + default=None, + required=False, + help=( + "Address of the Ray cluster to connect to. Can also be specified " + "using the RAY_ADDRESS environment variable." + ), +) +@click.argument("virtual-cluster-id", type=str) +@add_click_logging_options +def remove(address: Optional[str], virtual_cluster_id: str): + """Remove a virtual cluster.""" + stub = _get_virtual_cluster_stub(address) + request = RemoveVirtualClusterRequest(virtual_cluster_id=virtual_cluster_id) + + reply = stub.RemoveVirtualCluster(request) + + if reply.status.code == 0: + cli_logger.success( + f"Virtual cluster '{virtual_cluster_id}' removed successfully" + ) + else: + cli_logger.error( + f"Failed to remove virtual cluster '{virtual_cluster_id}'" + f": {reply.status.message}" + ) + sys.exit(1) + + +def _get_available_formats() -> List[str]: + """Return the available formats in a list of string""" + return [format_enum.value for format_enum in AvailableFormat] + + +@vclusters_cli_group.command() +@click.option( + "--address", + type=str, + default=None, + required=False, + help=( + "Address of the Ray cluster to connect to. Can also be specified " + "using the RAY_ADDRESS environment variable." + ), +) +@click.option( + "--format", default="default", type=click.Choice(_get_available_formats()) +) +@click.option( + "-f", + "--filter", + help=( + "A key, predicate, and value to filter the result. " + "E.g., --filter 'key=value' or --filter 'key!=value'. " + "You can specify multiple --filter options. In this case all predicates " + "are concatenated as AND. For example, --filter key=value --filter key2=value " + "means (key==val) AND (key2==val2), " + "String filter values are case-insensitive." + ), + multiple=True, +) +@click.option( + "--limit", + default=DEFAULT_LIMIT, + type=int, + help=("Maximum number of entries to return. 100 by default."), +) +@click.option( + "--detail", + help=( + "If the flag is set, the output will contain data in more details. " + "Note that the API could query more sources " + "to obtain information in a greater detail." + ), + is_flag=True, + default=False, +) +@click.option( + "--timeout", + type=int, + default=DEFAULT_RPC_TIMEOUT, + required=False, + help="Timeout for the request.", +) +@add_click_logging_options +def list( + address: Optional[str], + timeout: Optional[int], + limit: Optional[int], + filter: Optional[str], + detail: Optional[bool], + format: Optional[str], +): + """Lists all virtual clusters and their information. + + Example: + `ray vcluster list` + """ + + async def run(address): + address = services.canonicalize_bootstrap_address_or_die(address) + gcs_aio_client = GcsAioClient(address=address) + gcs_channel = GcsChannel(gcs_address=address, aio=True) + gcs_channel.connect() + state_api_data_source_client = StateDataSourceClient( + gcs_channel.channel(), gcs_aio_client + ) + state_api_manager = StateAPIManager( + state_api_data_source_client, + thread_pool_executor=ThreadPoolExecutor( + thread_name_prefix="state_api_test_utils" + ), + ) + data = await state_api_manager.list_vclusters( + option=ListApiOptions( + timeout=timeout, + limit=limit, + filters=filter, + detail=detail, + ) + ) + await state_api_data_source_client._client_session.close() + fmt = AvailableFormat(format) + if detail and fmt == AvailableFormat.DEFAULT: + fmt = AvailableFormat.TABLE + + result = [VirtualClusterState(**vcluster) for vcluster in data.result] + + print( + format_list_api_output( + state_data=result, + schema=VirtualClusterState, + format=fmt, + detail=detail, + ) + ) + + asyncio.run(run(address)) + + cli.add_command(dashboard) cli.add_command(debug) cli.add_command(start) @@ -2654,6 +2965,7 @@ def add_command_alias(command, name, hidden): cli.add_command(metrics_group) cli.add_command(drain_node) cli.add_command(check_open_ports) +cli.add_command(vclusters_cli_group) try: from ray.util.state.state_cli import ( diff --git a/python/ray/tests/conftest.py b/python/ray/tests/conftest.py index ad9d983b4f89..98f34188a781 100644 --- a/python/ray/tests/conftest.py +++ b/python/ray/tests/conftest.py @@ -608,6 +608,55 @@ def _ray_start_cluster(**kwargs): cluster.shutdown() +@contextmanager +def _ray_start_virtual_cluster(**kwargs): + cluster_not_supported_ = kwargs.pop("skip_cluster", cluster_not_supported) + if cluster_not_supported_: + pytest.skip("Cluster not supported") + init_kwargs = get_default_fixture_ray_kwargs() + num_nodes = 0 + do_init = False + ntemplates = kwargs["ntemplates"] + # num_nodes & do_init are not arguments for ray.init, so delete them. + if "num_nodes" in kwargs: + num_nodes = kwargs["num_nodes"] + del kwargs["num_nodes"] + if "do_init" in kwargs: + do_init = kwargs["do_init"] + del kwargs["do_init"] + elif num_nodes > 0: + do_init = True + + kwargs.pop("ntemplates") + init_kwargs.update(kwargs) + namespace = init_kwargs.pop("namespace") + template_id_prefix = init_kwargs.pop("template_id_prefix") + cluster = Cluster() + remote_nodes = [] + for i in range(num_nodes): + if i > 0 and "_system_config" in init_kwargs: + del init_kwargs["_system_config"] + env_vars = {} + if i > 0: + env_vars = { + "RAY_NODE_TYPE_NAME": template_id_prefix + str((i - 1) % ntemplates) + } + remote_nodes.append( + cluster.add_node( + **init_kwargs, + env_vars=env_vars, + ) + ) + # We assume driver will connect to the head (first node), + # so ray init will be invoked if do_init is true + if len(remote_nodes) == 1 and do_init: + ray.init(address=cluster.address, namespace=namespace) + yield cluster + # The code after the yield will run as teardown code. + ray.shutdown() + cluster.shutdown() + + # This fixture will start a cluster with empty nodes. @pytest.fixture def ray_start_cluster(request, maybe_external_redis): diff --git a/python/ray/tests/test_state_api.py b/python/ray/tests/test_state_api.py index bb1651e6f9ca..12d6f400262c 100644 --- a/python/ray/tests/test_state_api.py +++ b/python/ray/tests/test_state_api.py @@ -3683,6 +3683,7 @@ def test_core_state_api_usage_tags(shutdown_only): TagKey.CORE_STATE_API_LIST_CLUSTER_EVENTS, TagKey.CORE_STATE_API_LIST_NODES, TagKey.CORE_STATE_API_LIST_OBJECTS, + TagKey.CORE_STATE_API_LIST_VCLUSTERS, TagKey.CORE_STATE_API_LIST_RUNTIME_ENVS, TagKey.CORE_STATE_API_LIST_WORKERS, TagKey.CORE_STATE_API_SUMMARIZE_ACTORS, diff --git a/python/ray/util/state/api.py b/python/ray/util/state/api.py index 7de2c7c6ee8d..0ad34cb549b5 100644 --- a/python/ray/util/state/api.py +++ b/python/ray/util/state/api.py @@ -26,6 +26,7 @@ ListApiOptions, NodeState, ObjectState, + VirtualClusterState, PlacementGroupState, PredicateType, RuntimeEnvState, @@ -248,13 +249,14 @@ def get( TaskState, List[ObjectState], JobState, + VirtualClusterState, ] ]: """Get resources states by id Args: resource_name: Resource names, i.e. 'workers', 'actors', 'nodes', - 'placement_groups', 'tasks', 'objects'. + 'placement_groups', 'tasks', 'objects', "vclusters". 'jobs' and 'runtime-envs' are not supported yet. id: ID for the resource, i.e. 'node_id' for nodes. options: Get options. See `GetApiOptions` for details. @@ -269,6 +271,7 @@ def get( - WorkerState for workers - TaskState for tasks - JobState for jobs + - VirtualClusterState for virtual clusters Empty list for objects if not found, or list of ObjectState for objects @@ -292,6 +295,7 @@ def get( StateResource.TASKS: "task_id", StateResource.OBJECTS: "object_id", StateResource.JOBS: "submission_id", + StateResource.VCLUSTERS: "virtual_cluster_id", } if resource not in RESOURCE_ID_KEY_NAME: raise ValueError(f"Can't get {resource.name} by id.") @@ -462,6 +466,7 @@ def list( RuntimeEnvState, WorkerState, ClusterEventState, + VirtualClusterState, ] ]: """List resources states @@ -1097,6 +1102,51 @@ def list_objects( ) +@DeveloperAPI +def list_vclusters( + address: Optional[str] = None, + filters: Optional[List[Tuple[str, PredicateType, SupportedFilterType]]] = None, + limit: int = DEFAULT_LIMIT, + timeout: int = DEFAULT_RPC_TIMEOUT, + detail: bool = False, + raise_on_missing_output: bool = True, + _explain: bool = False, +) -> List[VirtualClusterState]: + """List virtual clusters in the cluster. + + Args: + address: Ray bootstrap address, could be `auto`, `localhost:6379`. + If None, it will be resolved automatically from an initialized ray. + filters: List of tuples of filter key, predicate (=, or !=), and + the filter value. E.g., `("ip", "=", "0.0.0.0")` + String filter values are case-insensitive. + limit: Max number of entries returned by the state backend. + timeout: Max timeout value for the state APIs requests made. + detail: When True, more details info (specified in `VirtualClusterState`) + will be queried and returned. See + :class:`VirtualClusterState `. + raise_on_missing_output: When True, exceptions will be raised if + there is missing data due to truncation/data source unavailable. + _explain: Print the API information such as API latency or + failed query information. + + Returns: + List of + :class:`VirtualClusterState `. + + Raises: + RayStateApiException: if the CLI failed to query the data. + """ # noqa: E501 + return StateApiClient(address=address).list( + StateResource.VCLUSTERS, + options=ListApiOptions( + limit=limit, timeout=timeout, filters=filters, detail=detail + ), + raise_on_missing_output=raise_on_missing_output, + _explain=_explain, + ) + + @DeveloperAPI def list_runtime_envs( address: Optional[str] = None, diff --git a/python/ray/util/state/common.py b/python/ray/util/state/common.py index 5b04ac57c341..0f55e649d5f4 100644 --- a/python/ray/util/state/common.py +++ b/python/ray/util/state/common.py @@ -71,6 +71,7 @@ class StateResource(Enum): OBJECTS = "objects" RUNTIME_ENVS = "runtime_envs" CLUSTER_EVENTS = "cluster_events" + VCLUSTERS = "vclusters" @unique @@ -852,6 +853,28 @@ class ObjectState(StateSchema): ip: str = state_column(filterable=True) +@dataclass(init=not IS_PYDANTIC_2) +class VirtualClusterState(StateSchema): + """Virtual Cluster State""" + + #: The virtual cluster id. + virtual_cluster_id: str = state_column(filterable=True) + #: Whether the virtual cluster can split into many child virtual clusters or not. + divisible: bool = state_column(filterable=True) + #: Divided virtual clusters. + divided_clusters: dict = state_column(filterable=False) + #: Replica Sets of virtual cluster + replica_sets: dict = state_column(filterable=False) + #: Only display undivided replica sets + undivided_replica_sets: dict = state_column(filterable=False) + #: Total resources of the virtual cluster. + resources_usage: dict = state_column(filterable=False, detail=False) + #: Mapping from node id to its instance. + visible_node_instances: Optional[dict] = state_column(filterable=False, detail=True) + #: Only display undivided nodes + undivided_nodes: Optional[dict] = state_column(filterable=False, detail=True) + + @dataclass(init=not IS_PYDANTIC_2) class RuntimeEnvState(StateSchema): """Runtime Environment State""" @@ -1539,6 +1562,8 @@ def resource_to_schema(resource: StateResource) -> StateSchema: return WorkerState elif resource == StateResource.CLUSTER_EVENTS: return ClusterEventState + elif resource == StateResource.VCLUSTERS: + return VirtualClusterState else: assert False, "Unreachable" diff --git a/python/ray/util/state/state_manager.py b/python/ray/util/state/state_manager.py index a4afa825f466..e65aff529d3a 100644 --- a/python/ray/util/state/state_manager.py +++ b/python/ray/util/state/state_manager.py @@ -29,6 +29,8 @@ GetAllWorkerInfoRequest, GetTaskEventsReply, GetTaskEventsRequest, + GetAllVirtualClusterInfoRequest, + GetAllVirtualClusterInfoReply, ) from ray.core.generated.node_manager_pb2 import ( GetObjectsInfoReply, @@ -176,6 +178,9 @@ def register_gcs_client(self, gcs_channel: grpc.aio.Channel): self._gcs_task_info_stub = gcs_service_pb2_grpc.TaskInfoGcsServiceStub( gcs_channel ) + self._gcs_virtual_cluster_info_stub = ( + gcs_service_pb2_grpc.VirtualClusterInfoGcsServiceStub(gcs_channel) + ) def register_raylet_client( self, node_id: str, address: str, port: int, runtime_env_agent_port: int @@ -378,6 +383,8 @@ async def get_all_node_info( req_filters.state = GcsNodeInfo.GcsNodeState.Value(value) elif key == "node_name": req_filters.node_name = value + elif key == "virtual_cluster_id": + req_filters.virtual_cluster_id = value else: continue @@ -462,6 +469,19 @@ async def get_object_info( ) return reply + @handle_grpc_network_errors + async def get_all_virtual_cluster_info( + self, + timeout: int = None, + limit: int = RAY_MAX_LIMIT_FROM_DATA_SOURCE, + filters: Optional[List[Tuple[str, PredicateType, SupportedFilterType]]] = None, + ) -> Optional[GetAllVirtualClusterInfoReply]: + request = GetAllVirtualClusterInfoRequest(limit=limit) + reply = await self._gcs_virtual_cluster_info_stub.GetAllVirtualClusterInfo( + request, timeout=timeout + ) + return reply + async def get_runtime_envs_info( self, node_id: str, diff --git a/src/mock/ray/gcs/gcs_client/accessor.h b/src/mock/ray/gcs/gcs_client/accessor.h index e167347e1da4..3aab3fb5d104 100644 --- a/src/mock/ray/gcs/gcs_client/accessor.h +++ b/src/mock/ray/gcs/gcs_client/accessor.h @@ -187,7 +187,8 @@ class MockNodeResourceInfoAccessor : public NodeResourceInfoAccessor { public: MOCK_METHOD(Status, AsyncGetAllAvailableResources, - (const MultiItemCallback &callback), + (std::optional & virtual_cluster_id, + const MultiItemCallback &callback), (override)); MOCK_METHOD(void, AsyncResubscribe, (), (override)); MOCK_METHOD(Status, @@ -352,3 +353,30 @@ class MockInternalKVAccessor : public InternalKVAccessor { } // namespace gcs } // namespace ray + +namespace ray { +namespace gcs { + +class MockVirtualClusterInfoAccessor : public VirtualClusterInfoAccessor { + public: + MOCK_METHOD(Status, + AsyncGet, + (const VirtualClusterID &virtual_cluster_id, + const OptionalItemCallback &callback), + (override)); + MOCK_METHOD(Status, + AsyncGetAll, + (bool include_job_clusters, + bool only_include_indivisible_clusters, + (const MultiItemCallback &callback)), + (override)); + MOCK_METHOD(Status, + AsyncSubscribeAll, + ((const SubscribeCallback + &subscribe), + const StatusCallback &done), + (override)); +}; + +} // namespace gcs +} // namespace ray diff --git a/src/mock/ray/gcs/gcs_client/gcs_client.h b/src/mock/ray/gcs/gcs_client/gcs_client.h index a798ef77760d..168073943684 100644 --- a/src/mock/ray/gcs/gcs_client/gcs_client.h +++ b/src/mock/ray/gcs/gcs_client/gcs_client.h @@ -50,6 +50,7 @@ class MockGcsClient : public GcsClient { mock_placement_group_accessor = new MockPlacementGroupInfoAccessor(); mock_internal_kv_accessor = new MockInternalKVAccessor(); mock_task_accessor = new MockTaskInfoAccessor(); + mock_virtual_cluster_accessor = new MockVirtualClusterInfoAccessor(); GcsClient::job_accessor_.reset(mock_job_accessor); GcsClient::actor_accessor_.reset(mock_actor_accessor); @@ -59,6 +60,7 @@ class MockGcsClient : public GcsClient { GcsClient::worker_accessor_.reset(mock_worker_accessor); GcsClient::placement_group_accessor_.reset(mock_placement_group_accessor); GcsClient::task_accessor_.reset(mock_task_accessor); + GcsClient::virtual_cluster_accessor_.reset(mock_virtual_cluster_accessor); } MockActorInfoAccessor *mock_actor_accessor; MockJobInfoAccessor *mock_job_accessor; @@ -69,6 +71,7 @@ class MockGcsClient : public GcsClient { MockPlacementGroupInfoAccessor *mock_placement_group_accessor; MockInternalKVAccessor *mock_internal_kv_accessor; MockTaskInfoAccessor *mock_task_accessor; + MockVirtualClusterInfoAccessor *mock_virtual_cluster_accessor; }; } // namespace gcs diff --git a/src/ray/common/BUILD b/src/ray/common/BUILD index f5e6b9d8b9cd..ff7e664378ea 100644 --- a/src/ray/common/BUILD +++ b/src/ray/common/BUILD @@ -132,6 +132,8 @@ ray_cc_library( "common_protocol.h", "id.h", "id_def.h", + "simple_id.h", + "virtual_cluster_id.h", ], deps = [ ":constants", @@ -240,6 +242,7 @@ ray_cc_library( hdrs = [ "ray_config.h", "ray_config_def.h", + "ray_config_def.ant.h", "ray_internal_flag_def.h", ], deps = [ diff --git a/src/ray/common/constants.h b/src/ray/common/constants.h index d4df214f01a8..95728b63c3d7 100644 --- a/src/ray/common/constants.h +++ b/src/ray/common/constants.h @@ -56,6 +56,12 @@ constexpr int kPublicDNSServerPort = 53; constexpr char kEnvVarKeyJobId[] = "RAY_JOB_ID"; constexpr char kEnvVarKeyRayletPid[] = "RAY_RAYLET_PID"; +// Environment variable key for virtual cluster ID +constexpr char kEnvVarKeyVirtualClusterID[] = "VIRTUAL_CLUSTER_ID"; + +// Label key for virtual cluster ID +constexpr char kLabelVirtualClusterID[] = "virtual_cluster_id"; + /// for cross-langueage serialization constexpr int kMessagePackOffset = 9; diff --git a/src/ray/common/placement_group.h b/src/ray/common/placement_group.h index 223ec9afc5ea..07683ed6f7cb 100644 --- a/src/ray/common/placement_group.h +++ b/src/ray/common/placement_group.h @@ -105,6 +105,12 @@ class PlacementGroupSpecBuilder { message_->set_is_detached(is_detached); message_->set_max_cpu_fraction_per_node(max_cpu_fraction_per_node); message_->set_soft_target_node_id(soft_target_node_id.Binary()); + // This ensures that placement groups incorporate virtual cluster information during + // scheduling. + std::string virtual_cluster_id = std::getenv(kEnvVarKeyVirtualClusterID) + ? std::getenv(kEnvVarKeyVirtualClusterID) + : ""; + message_->set_virtual_cluster_id(virtual_cluster_id); for (size_t i = 0; i < bundles.size(); i++) { auto resources = bundles[i]; diff --git a/src/ray/common/ray_config_def.ant.h b/src/ray/common/ray_config_def.ant.h new file mode 100644 index 000000000000..32d22672b184 --- /dev/null +++ b/src/ray/common/ray_config_def.ant.h @@ -0,0 +1,30 @@ +// Copyright 2017 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. + +// This header file is used to avoid code duplication. +// It can be included multiple times in ray_config.h, and each inclusion +// could use a different definition of the RAY_CONFIG macro. +// Macro definition format: RAY_CONFIG(type, name, default_value). +// NOTE: This file should NOT be included in any file other than ray_config.h. + +/// The interval to replenish node instances of all the virtual clusters. +RAY_CONFIG(uint64_t, node_instances_replenish_interval_ms, 30000) + +/// The interval to check and delete expired job clusters. +RAY_CONFIG(uint64_t, expired_job_clusters_gc_interval_ms, 30000) + +// The interval of the delayed local node cleanup. When a node is removed out +// of the virtual cluster, the cleanup function can be delayed. It allows other handlers +// executed first, and leaves some time for tasks to finish (before force killing them). +RAY_CONFIG(uint64_t, local_node_cleanup_delay_interval_ms, 30000) diff --git a/src/ray/common/ray_config_def.h b/src/ray/common/ray_config_def.h index 3bc2955f8cf7..4eb48605e7cc 100644 --- a/src/ray/common/ray_config_def.h +++ b/src/ray/common/ray_config_def.h @@ -18,6 +18,8 @@ // Macro definition format: RAY_CONFIG(type, name, default_value). // NOTE: This file should NOT be included in any file other than ray_config.h. +#include "ray/common/ray_config_def.ant.h" + /// The duration between dumping debug info to logs, or 0 to disable. RAY_CONFIG(uint64_t, debug_dump_period_milliseconds, 10000) diff --git a/src/ray/common/simple_id.h b/src/ray/common/simple_id.h new file mode 100644 index 000000000000..1886a2e31d1c --- /dev/null +++ b/src/ray/common/simple_id.h @@ -0,0 +1,67 @@ +// Copyright 2017 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 "ray/common/id.h" + +namespace ray { + +template +struct SimpleID { + static T FromBinary(const std::string &binary); + static const T &Nil(); + + SimpleID() = default; + + size_t Hash() const; + bool IsNil() const; + const std::string &Binary() const { return id_; } + bool operator==(const T &rhs) const { return id_ == rhs.id_; } + bool operator!=(const T &rhs) const { return !(*this == rhs); } + + protected: + std::string id_; + mutable size_t hash_ = 0; +}; + +template +T SimpleID::FromBinary(const std::string &binary) { + T id; + id.id_ = binary; + return id; +} + +template +const T &SimpleID::Nil() { + static const T nil_id; + return nil_id; +} + +template +size_t SimpleID::Hash() const { + // Note(ashione): hash code lazy calculation(it's invoked every time if hash code is + // default value 0) + if (!hash_) { + hash_ = MurmurHash64A(id_.data(), id_.size(), 0); + } + return hash_; +} + +template +bool SimpleID::IsNil() const { + return *this == T::Nil(); +} + +} // namespace ray diff --git a/src/ray/common/status.cc b/src/ray/common/status.cc index bdb944dfcd37..eb2ec4f350b5 100644 --- a/src/ray/common/status.cc +++ b/src/ray/common/status.cc @@ -78,6 +78,7 @@ const absl::flat_hash_map kCodeToStr = { {StatusCode::InvalidArgument, "InvalidArgument"}, {StatusCode::ChannelError, "ChannelError"}, {StatusCode::ChannelTimeoutError, "ChannelTimeoutError"}, + {StatusCode::UnsafeToRemove, "UnsafeToRemove"}, }; const absl::flat_hash_map kStrToCode = []() { diff --git a/src/ray/common/status.h b/src/ray/common/status.h index e8206ba90aaf..57676878a799 100644 --- a/src/ray/common/status.h +++ b/src/ray/common/status.h @@ -106,6 +106,8 @@ enum class StatusCode : char { ChannelError = 35, // Indicates that a read or write on a channel (a mutable plasma object) timed out. ChannelTimeoutError = 36, + // Indicates that it is now unsafe to remove nodes from a (virtual) cluster. + UnsafeToRemove = 37, // If you add to this list, please also update kCodeToStr in status.cc. }; @@ -259,6 +261,10 @@ class RAY_EXPORT Status { return Status(StatusCode::ChannelTimeoutError, msg); } + static Status UnsafeToRemove(const std::string &msg) { + return Status(StatusCode::UnsafeToRemove, msg); + } + static StatusCode StringToCode(const std::string &str); // Returns true iff the status indicates success. @@ -314,6 +320,8 @@ class RAY_EXPORT Status { bool IsChannelTimeoutError() const { return code() == StatusCode::ChannelTimeoutError; } + bool IsUnsafeToRemove() const { return code() == StatusCode::UnsafeToRemove; } + // Return a string representation of this status suitable for printing. // Returns the string "OK" for success. std::string ToString() const; diff --git a/src/ray/common/task/task_util.h b/src/ray/common/task/task_util.h index 8d2f90467248..70cd0266ce1d 100644 --- a/src/ray/common/task/task_util.h +++ b/src/ray/common/task/task_util.h @@ -189,6 +189,11 @@ class TaskSpecBuilder { if (!root_detached_actor_id.IsNil()) { message_->set_root_detached_actor_id(root_detached_actor_id.Binary()); } + // This ensures that tasks incorporate virtual cluster information during scheduling. + std::string virtual_cluster_id = std::getenv(kEnvVarKeyVirtualClusterID) + ? std::getenv(kEnvVarKeyVirtualClusterID) + : ""; + message_->mutable_scheduling_strategy()->set_virtual_cluster_id(virtual_cluster_id); return *this; } @@ -273,6 +278,17 @@ class TaskSpecBuilder { if (!root_detached_actor_id.IsNil()) { message_->set_root_detached_actor_id(root_detached_actor_id.Binary()); } + + // This ensures that actors incorporate virtual cluster information during scheduling. + // Specifically, for the supervisor actor, the virtual cluster ID is included within + // the labels. + std::string virtual_cluster_id = message_->labels().contains(kLabelVirtualClusterID) + ? message_->labels().at(kLabelVirtualClusterID) + : (std::getenv(kEnvVarKeyVirtualClusterID) + ? std::getenv(kEnvVarKeyVirtualClusterID) + : ""); + + message_->mutable_scheduling_strategy()->set_virtual_cluster_id(virtual_cluster_id); return *this; } diff --git a/src/ray/common/virtual_cluster_id.h b/src/ray/common/virtual_cluster_id.h new file mode 100644 index 000000000000..7f1a0ecb28f8 --- /dev/null +++ b/src/ray/common/virtual_cluster_id.h @@ -0,0 +1,72 @@ +// Copyright 2017 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 "ray/common/simple_id.h" + +namespace ray { + +const std::string kPrimaryClusterID = "kPrimaryClusterID"; +const std::string kJobClusterIDSeperator = "##"; + +class VirtualClusterID : public SimpleID { + public: + using SimpleID::SimpleID; + + VirtualClusterID BuildJobClusterID(const std::string &job_id) const { + return VirtualClusterID::FromBinary(id_ + kJobClusterIDSeperator + job_id); + } + + bool IsJobClusterID() const { + return id_.find(kJobClusterIDSeperator) != std::string::npos; + } + + VirtualClusterID ParentID() const { + auto pos = id_.find(kJobClusterIDSeperator); + return pos == std::string::npos ? Nil() + : VirtualClusterID::FromBinary(id_.substr(0, pos)); + } + + std::string JobName() const { + auto pos = id_.find(kJobClusterIDSeperator); + return pos == std::string::npos ? "" + : id_.substr(pos + kJobClusterIDSeperator.size()); + } +}; + +inline std::ostream &operator<<(std::ostream &os, const ray::VirtualClusterID &id) { + os << id.Binary(); + return os; +} + +template <> +struct DefaultLogKey { + constexpr static std::string_view key = kLogKeyVirtualClusterID; +}; + +} // namespace ray + +namespace std { + +template <> +struct hash { + size_t operator()(const ray::VirtualClusterID &id) const { return id.Hash(); } +}; +template <> +struct hash { + size_t operator()(const ray::VirtualClusterID &id) const { return id.Hash(); } +}; + +} // namespace std diff --git a/src/ray/core_worker/core_worker.cc b/src/ray/core_worker/core_worker.cc index 1a9142efdf70..a052b02f4b30 100644 --- a/src/ray/core_worker/core_worker.cc +++ b/src/ray/core_worker/core_worker.cc @@ -1059,8 +1059,13 @@ void CoreWorker::ConnectToRayletInternal() { // NOTE: This also marks the worker as available in Raylet. We do this at the // very end in case there is a problem during construction. if (options_.worker_type == WorkerType::DRIVER) { + // Get virtual cluster id from worker env to put it into job table data + std::string virtual_cluster_id = std::getenv(kEnvVarKeyVirtualClusterID) + ? std::getenv(kEnvVarKeyVirtualClusterID) + : ""; + Status status = local_raylet_client_->AnnounceWorkerPortForDriver( - core_worker_server_->GetPort(), options_.entrypoint); + core_worker_server_->GetPort(), options_.entrypoint, virtual_cluster_id); RAY_CHECK_OK(status) << "Failed to announce driver's port to raylet and GCS"; } else { // TODO(hjiang): In the future this function should only accessed by driver, should diff --git a/src/ray/gcs/gcs_client/accessor.ant.cc b/src/ray/gcs/gcs_client/accessor.ant.cc new file mode 100644 index 000000000000..47e251844878 --- /dev/null +++ b/src/ray/gcs/gcs_client/accessor.ant.cc @@ -0,0 +1,153 @@ +// Copyright 2017 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/gcs/gcs_client/accessor.h" + +#include + +#include "ray/gcs/gcs_client/gcs_client.h" + +namespace ray { +namespace gcs { + +VirtualClusterInfoAccessor::VirtualClusterInfoAccessor(GcsClient *client_impl) + : client_impl_(client_impl) {} + +Status VirtualClusterInfoAccessor::AsyncGet( + const VirtualClusterID &virtual_cluster_id, + const OptionalItemCallback &callback) { + RAY_LOG(DEBUG).WithField(virtual_cluster_id) << "Getting virtual cluster info"; + rpc::GetVirtualClustersRequest request; + request.set_virtual_cluster_id(virtual_cluster_id.Binary()); + client_impl_->GetGcsRpcClient().GetVirtualClusters( + request, + [virtual_cluster_id, callback](const Status &status, + rpc::GetVirtualClustersReply &&reply) { + if (reply.virtual_cluster_data_list_size() == 0) { + callback(status, std::nullopt); + } else { + RAY_CHECK(reply.virtual_cluster_data_list_size() == 1); + callback(status, reply.virtual_cluster_data_list().at(0)); + } + RAY_LOG(DEBUG).WithField(virtual_cluster_id) + << "Finished getting virtual cluster info"; + }); + return Status::OK(); +} + +Status VirtualClusterInfoAccessor::AsyncGetAll( + bool include_job_clusters, + bool only_include_indivisible_clusters, + const MultiItemCallback &callback) { + RAY_LOG(DEBUG) << "Getting all virtual cluster info."; + rpc::GetVirtualClustersRequest request; + request.set_include_job_clusters(true); + request.set_only_include_indivisible_clusters(true); + client_impl_->GetGcsRpcClient().GetVirtualClusters( + request, [callback](const Status &status, rpc::GetVirtualClustersReply &&reply) { + callback( + status, + VectorFromProtobuf(std::move(*reply.mutable_virtual_cluster_data_list()))); + RAY_LOG(DEBUG) << "Finished getting all virtual cluster info, status = " + << status; + }); + return Status::OK(); +} + +Status VirtualClusterInfoAccessor::AsyncSubscribeAll( + const SubscribeCallback &subscribe, + const StatusCallback &done) { + RAY_CHECK(subscribe != nullptr); + const auto updated_subscribe = + [this, subscribe](const VirtualClusterID &virtual_cluster_id, + rpc::VirtualClusterTableData &&virtual_cluster_data) { + auto iter = virtual_clusters_.find(virtual_cluster_id); + if (iter != virtual_clusters_.end()) { + if (virtual_cluster_data.revision() < iter->second.revision()) { + RAY_LOG(WARNING) << "The revision of the received virtual cluster (" + << virtual_cluster_id << ") is outdated. Ignore it."; + return; + } + if (virtual_cluster_data.is_removed()) { + virtual_clusters_.erase(iter); + } else { + iter->second = virtual_cluster_data; + } + } else { + virtual_clusters_[virtual_cluster_id] = virtual_cluster_data; + } + + subscribe(virtual_cluster_id, std::move(virtual_cluster_data)); + }; + fetch_all_data_operation_ = [this, subscribe, updated_subscribe]( + const StatusCallback &done) { + auto callback = + [this, subscribe, updated_subscribe, done]( + const Status &status, + std::vector &&virtual_cluster_info_list) { + absl::flat_hash_set virtual_cluster_id_set; + for (auto &virtual_cluster_info : virtual_cluster_info_list) { + auto virtual_cluster_id = + VirtualClusterID::FromBinary(virtual_cluster_info.id()); + updated_subscribe(virtual_cluster_id, std::move(virtual_cluster_info)); + virtual_cluster_id_set.emplace(virtual_cluster_id); + } + + for (auto iter = virtual_clusters_.begin(); iter != virtual_clusters_.end();) { + auto curr_iter = iter++; + // If there is any virtual cluster not in `virtual_cluster_id_set`, it + // means the local node may miss the pub messages (when gcs removed + // virtual clusters) in the past. So we have to explicitely notify the + // subscriber to clean its local cache. + if (!virtual_cluster_id_set.contains(curr_iter->first)) { + auto virtual_cluster_data = std::move(curr_iter->second); + virtual_cluster_data.set_is_removed(true); + subscribe(curr_iter->first, std::move(virtual_cluster_data)); + virtual_clusters_.erase(curr_iter); + } + } + if (done) { + done(status); + } + }; + RAY_CHECK_OK(AsyncGetAll( + /*include_job_clusters=*/true, + /*only_include_indivisible_clusters=*/true, + callback)); + }; + subscribe_operation_ = [this, updated_subscribe](const StatusCallback &done) { + return client_impl_->GetGcsSubscriber().SubscribeAllVirtualClusters(updated_subscribe, + done); + }; + return subscribe_operation_( + [this, done](const Status &status) { fetch_all_data_operation_(done); }); +} + +void VirtualClusterInfoAccessor::AsyncResubscribe() { + RAY_LOG(DEBUG) << "Reestablishing subscription for virtual cluster info."; + auto fetch_all_done = [](const Status &status) { + RAY_LOG(INFO) + << "Finished fetching all virtual cluster information from gcs server after gcs " + "server or pub-sub server is restarted."; + }; + + if (subscribe_operation_ != nullptr) { + RAY_CHECK_OK(subscribe_operation_([this, fetch_all_done](const Status &status) { + fetch_all_data_operation_(fetch_all_done); + })); + } +} + +} // namespace gcs +} // namespace ray diff --git a/src/ray/gcs/gcs_client/accessor.cc b/src/ray/gcs/gcs_client/accessor.cc index bb5f9bb3d164..5ebe521ac8f1 100644 --- a/src/ray/gcs/gcs_client/accessor.cc +++ b/src/ray/gcs/gcs_client/accessor.cc @@ -650,6 +650,34 @@ Status NodeInfoAccessor::AsyncGetAll(const MultiItemCallback &callb return Status::OK(); } +Status NodeInfoAccessor::AsyncGetByVirtualClusterID( + const std::optional &virtual_cluster_id, + const MultiItemCallback &callback, + int64_t timeout_ms) { + RAY_LOG(DEBUG) << "Getting information of all nodes by virtual cluster id: " + << virtual_cluster_id.value(); + if (!virtual_cluster_id) { + return AsyncGetAll(callback, timeout_ms); + } + rpc::GetAllNodeInfoRequest request; + request.mutable_filters()->set_virtual_cluster_id(virtual_cluster_id.value()); + client_impl_->GetGcsRpcClient().GetAllNodeInfo( + request, + [callback](const Status &status, rpc::GetAllNodeInfoReply &&reply) { + std::vector result; + result.reserve((reply.node_info_list_size())); + for (int index = 0; index < reply.node_info_list_size(); ++index) { + result.emplace_back(reply.node_info_list(index)); + } + callback(status, std::move(result)); + RAY_LOG(DEBUG) << "Finished getting information of all nodes by virtual cluster " + "id, status = " + << status; + }, + timeout_ms); + return Status::OK(); +} + Status NodeInfoAccessor::AsyncSubscribeToNodeChange( const SubscribeCallback &subscribe, const StatusCallback &done) { RAY_CHECK(subscribe != nullptr); @@ -804,8 +832,12 @@ NodeResourceInfoAccessor::NodeResourceInfoAccessor(GcsClient *client_impl) : client_impl_(client_impl) {} Status NodeResourceInfoAccessor::AsyncGetAllAvailableResources( + const std::optional &virtual_cluster_id, const MultiItemCallback &callback) { rpc::GetAllAvailableResourcesRequest request; + if (virtual_cluster_id.has_value()) { + request.set_virtual_cluster_id(virtual_cluster_id.value()); + } client_impl_->GetGcsRpcClient().GetAllAvailableResources( request, [callback](const Status &status, rpc::GetAllAvailableResourcesReply &&reply) { @@ -817,8 +849,12 @@ Status NodeResourceInfoAccessor::AsyncGetAllAvailableResources( } Status NodeResourceInfoAccessor::AsyncGetAllTotalResources( + const std::optional &virtual_cluster_id, const MultiItemCallback &callback) { rpc::GetAllTotalResourcesRequest request; + if (virtual_cluster_id.has_value()) { + request.set_virtual_cluster_id(virtual_cluster_id.value()); + } client_impl_->GetGcsRpcClient().GetAllTotalResources( request, [callback](const Status &status, rpc::GetAllTotalResourcesReply &&reply) { callback(status, VectorFromProtobuf(std::move(*reply.mutable_resources_list()))); diff --git a/src/ray/gcs/gcs_client/accessor.h b/src/ray/gcs/gcs_client/accessor.h index 8f5bbdcda96a..94e13f7fcd59 100644 --- a/src/ray/gcs/gcs_client/accessor.h +++ b/src/ray/gcs/gcs_client/accessor.h @@ -18,6 +18,7 @@ #include "ray/common/id.h" #include "ray/common/placement_group.h" #include "ray/common/task/task_spec.h" +#include "ray/common/virtual_cluster_id.h" #include "ray/gcs/callback.h" #include "ray/rpc/client_call.h" #include "ray/util/sequencer.h" @@ -381,6 +382,16 @@ class NodeInfoAccessor { virtual Status AsyncGetAll(const MultiItemCallback &callback, int64_t timeout_ms); + /// Get information of all nodes from GCS asynchronously. + /// + /// \param virtual_cluster_id The ID of the virtual cluster to look up in GCS. + /// \param callback Callback that will be called after lookup finishes. + /// \return Status + virtual Status AsyncGetByVirtualClusterID( + const std::optional &virtual_cluster_id, + const MultiItemCallback &callback, + int64_t timeout_ms); + /// Subscribe to node addition and removal events from GCS and cache those information. /// /// \param subscribe Callback that will be called if a node is @@ -500,6 +511,7 @@ class NodeResourceInfoAccessor { /// \param callback Callback that will be called after lookup finishes. /// \return Status virtual Status AsyncGetAllAvailableResources( + const std::optional &virtual_cluster_id, const MultiItemCallback &callback); /// Get total resources of all nodes from GCS asynchronously. @@ -507,6 +519,7 @@ class NodeResourceInfoAccessor { /// \param callback Callback that will be called after lookup finishes. /// \return Status virtual Status AsyncGetAllTotalResources( + const std::optional &virtual_cluster_id, const MultiItemCallback &callback); /// Get draining nodes from GCS asynchronously. @@ -999,6 +1012,60 @@ class AutoscalerStateAccessor { GcsClient *client_impl_; }; +class VirtualClusterInfoAccessor { + public: + VirtualClusterInfoAccessor() = default; + explicit VirtualClusterInfoAccessor(GcsClient *client_impl); + virtual ~VirtualClusterInfoAccessor() = default; + + /// Get a virtual cluster data from GCS asynchronously by id. + /// + /// \param virtual_cluster_id The id of a virtual cluster to obtain from GCS. + /// \return Status. + virtual Status AsyncGet( + const VirtualClusterID &virtual_cluster_id, + const OptionalItemCallback &callback); + + /// Get all virtual cluster info from GCS asynchronously. + /// + /// \param callback Callback that will be called after lookup finished. + /// \return Status + virtual Status AsyncGetAll( + bool include_job_clusters, + bool only_include_indivisible_clusters, + const MultiItemCallback &callback); + + /// Subscribe to virtual cluster updates. + /// + /// \param subscribe Callback that will be called each time when a job updates. + /// \param done Callback that will be called when subscription is complete. + /// \return Status + virtual Status AsyncSubscribeAll( + const SubscribeCallback &subscribe, + const StatusCallback &done); + + /// Reestablish subscription. + /// This should be called when GCS server restarts from a failure. + /// PubSub server restart will cause GCS server restart. In this case, we need to + /// resubscribe from PubSub server, otherwise we only need to fetch data from GCS + /// server. + virtual void AsyncResubscribe(); + + private: + /// Save the fetch data operation in this function, so we can call it again when GCS + /// server restarts from a failure. + FetchDataOperation fetch_all_data_operation_; + + /// Save the subscribe operation in this function, so we can call it again when PubSub + /// server restarts from a failure. + SubscribeOperation subscribe_operation_; + + GcsClient *client_impl_; + + // Local cache of the virtual cluster data. It can be used for revision control. + absl::flat_hash_map virtual_clusters_; +}; + } // namespace gcs } // namespace ray diff --git a/src/ray/gcs/gcs_client/gcs_client.cc b/src/ray/gcs/gcs_client/gcs_client.cc index d805bd6330ce..508cf4295450 100644 --- a/src/ray/gcs/gcs_client/gcs_client.cc +++ b/src/ray/gcs/gcs_client/gcs_client.cc @@ -119,6 +119,7 @@ Status GcsClient::Connect(instrumented_io_context &io_service, int64_t timeout_m node_accessor_->AsyncResubscribe(); node_resource_accessor_->AsyncResubscribe(); worker_accessor_->AsyncResubscribe(); + virtual_cluster_accessor_->AsyncResubscribe(); }; rpc::Address gcs_address; @@ -133,7 +134,8 @@ Status GcsClient::Connect(instrumented_io_context &io_service, int64_t timeout_m std::vector{rpc::ChannelType::GCS_ACTOR_CHANNEL, rpc::ChannelType::GCS_JOB_CHANNEL, rpc::ChannelType::GCS_NODE_INFO_CHANNEL, - rpc::ChannelType::GCS_WORKER_DELTA_CHANNEL}, + rpc::ChannelType::GCS_WORKER_DELTA_CHANNEL, + rpc::ChannelType::RAY_VIRTUAL_CLUSTER_CHANNEL}, /*max_command_batch_size*/ RayConfig::instance().max_command_batch_size(), /*get_client=*/ [this](const rpc::Address &) { @@ -155,6 +157,7 @@ Status GcsClient::Connect(instrumented_io_context &io_service, int64_t timeout_m task_accessor_ = std::make_unique(this); runtime_env_accessor_ = std::make_unique(this); autoscaler_state_accessor_ = std::make_unique(this); + virtual_cluster_accessor_ = std::make_unique(this); RAY_LOG(DEBUG) << "GcsClient connected " << options_.gcs_address_ << ":" << options_.gcs_port_; diff --git a/src/ray/gcs/gcs_client/gcs_client.h b/src/ray/gcs/gcs_client/gcs_client.h index f179a8f06512..b5f50bcdb9d8 100644 --- a/src/ray/gcs/gcs_client/gcs_client.h +++ b/src/ray/gcs/gcs_client/gcs_client.h @@ -209,6 +209,11 @@ class RAY_EXPORT GcsClient : public std::enable_shared_from_this { return *autoscaler_state_accessor_; } + VirtualClusterInfoAccessor &VirtualCluster() { + RAY_CHECK(virtual_cluster_accessor_ != nullptr); + return *virtual_cluster_accessor_; + } + // Gets ClusterID. If it's not set in Connect(), blocks on a sync RPC to GCS to get it. virtual ClusterID GetClusterId() const; @@ -234,6 +239,7 @@ class RAY_EXPORT GcsClient : public std::enable_shared_from_this { std::unique_ptr task_accessor_; std::unique_ptr runtime_env_accessor_; std::unique_ptr autoscaler_state_accessor_; + std::unique_ptr virtual_cluster_accessor_; private: /// If client_call_manager_ does not have a cluster ID, fetches it from GCS. The diff --git a/src/ray/gcs/gcs_client/global_state_accessor.cc b/src/ray/gcs/gcs_client/global_state_accessor.cc index cc482cfc672e..916013eae1ec 100644 --- a/src/ray/gcs/gcs_client/global_state_accessor.cc +++ b/src/ray/gcs/gcs_client/global_state_accessor.cc @@ -88,12 +88,21 @@ JobID GlobalStateAccessor::GetNextJobID() { return promise.get_future().get(); } -std::vector GlobalStateAccessor::GetAllNodeInfo() { +std::vector GlobalStateAccessor::GetAllNodeInfo( + const std::optional &virtual_cluster_id) { // This method assumes GCS is HA and does not return any error. On GCS down, it // retries indefinitely. std::vector node_table_data; std::promise promise; - { + RAY_LOG(DEBUG) << "Getting all node info, virtual_cluster_id: " + << (virtual_cluster_id.has_value() ? virtual_cluster_id.value() : ""); + if (virtual_cluster_id) { + absl::ReaderMutexLock lock(&mutex_); + RAY_CHECK_OK(gcs_client_->Nodes().AsyncGetByVirtualClusterID( + virtual_cluster_id.value(), + TransformForMultiItemCallback(node_table_data, promise), + /*timeout_ms=*/-1)); + } else { absl::ReaderMutexLock lock(&mutex_); RAY_CHECK_OK(gcs_client_->Nodes().AsyncGetAll( TransformForMultiItemCallback(node_table_data, promise), @@ -115,27 +124,31 @@ std::vector GlobalStateAccessor::GetAllTaskEvents() { return task_events; } -std::vector GlobalStateAccessor::GetAllAvailableResources() { +std::vector GlobalStateAccessor::GetAllAvailableResources( + const std::optional &virtual_cluster_id) { std::vector available_resources; std::promise promise; - { - absl::ReaderMutexLock lock(&mutex_); - RAY_CHECK_OK(gcs_client_->NodeResources().AsyncGetAllAvailableResources( - TransformForMultiItemCallback(available_resources, - promise))); - } + RAY_LOG(DEBUG) << "Requesting all available resources, virtual_cluster_id: " + << (virtual_cluster_id.has_value() ? virtual_cluster_id.value() : ""); + absl::ReaderMutexLock lock(&mutex_); + RAY_CHECK_OK(gcs_client_->NodeResources().AsyncGetAllAvailableResources( + virtual_cluster_id, + TransformForMultiItemCallback(available_resources, + promise))); promise.get_future().get(); return available_resources; } -std::vector GlobalStateAccessor::GetAllTotalResources() { +std::vector GlobalStateAccessor::GetAllTotalResources( + const std::optional &virtual_cluster_id) { std::vector total_resources; std::promise promise; - { - absl::ReaderMutexLock lock(&mutex_); - RAY_CHECK_OK(gcs_client_->NodeResources().AsyncGetAllTotalResources( - TransformForMultiItemCallback(total_resources, promise))); - } + RAY_LOG(DEBUG) << "Requesting all total resources, virtual_cluster_id: " + << (virtual_cluster_id.has_value() ? virtual_cluster_id.value() : ""); + absl::ReaderMutexLock lock(&mutex_); + RAY_CHECK_OK(gcs_client_->NodeResources().AsyncGetAllTotalResources( + virtual_cluster_id, + TransformForMultiItemCallback(total_resources, promise))); promise.get_future().get(); return total_resources; } diff --git a/src/ray/gcs/gcs_client/global_state_accessor.h b/src/ray/gcs/gcs_client/global_state_accessor.h index 7bdcad401edf..1df3ed3183a3 100644 --- a/src/ray/gcs/gcs_client/global_state_accessor.h +++ b/src/ray/gcs/gcs_client/global_state_accessor.h @@ -61,7 +61,9 @@ class GlobalStateAccessor { /// Get all node information from GCS. /// /// \return A list of `GcsNodeInfo` objects serialized in protobuf format. - std::vector GetAllNodeInfo() ABSL_LOCKS_EXCLUDED(mutex_); + std::vector GetAllNodeInfo( + const std::optional &virtual_cluster_id = std::nullopt) + ABSL_LOCKS_EXCLUDED(mutex_); /// Get information of all task events from GCS Service. /// @@ -73,14 +75,18 @@ class GlobalStateAccessor { /// \return available resources of all nodes. To support multi-language, we serialize /// each AvailableResources and return the serialized string. Where used, it needs to be /// deserialized with protobuf function. - std::vector GetAllAvailableResources() ABSL_LOCKS_EXCLUDED(mutex_); + std::vector GetAllAvailableResources( + const std::optional &virtual_cluster_id = std::nullopt) + ABSL_LOCKS_EXCLUDED(mutex_); /// Get total resources of all nodes. /// /// \return total resources of all nodes. To support multi-language, we serialize /// each TotalResources and return the serialized string. Where used, it needs to be /// deserialized with protobuf function. - std::vector GetAllTotalResources() ABSL_LOCKS_EXCLUDED(mutex_); + std::vector GetAllTotalResources( + const std::optional &virtual_cluster_id = std::nullopt) + ABSL_LOCKS_EXCLUDED(mutex_); /// Get draining nodes. /// diff --git a/src/ray/gcs/gcs_client/test/gcs_client_test.cc b/src/ray/gcs/gcs_client/test/gcs_client_test.cc index 98eabb02fe9e..fb2a02a93897 100644 --- a/src/ray/gcs/gcs_client/test/gcs_client_test.cc +++ b/src/ray/gcs/gcs_client/test/gcs_client_test.cc @@ -367,6 +367,7 @@ class GcsClientTest : public ::testing::TestWithParam { std::promise promise; std::vector resources; RAY_CHECK_OK(gcs_client_->NodeResources().AsyncGetAllAvailableResources( + std::nullopt, [&resources, &promise](Status status, const std::vector &result) { EXPECT_TRUE(!result.empty()); diff --git a/src/ray/gcs/gcs_server/gcs_actor_manager.cc b/src/ray/gcs/gcs_server/gcs_actor_manager.cc index 7a76b9642336..1902e042736f 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_actor_manager.cc @@ -124,6 +124,11 @@ const ray::rpc::ActorDeathCause GenActorRefDeletedCause(const ray::gcs::GcsActor return death_cause; } +} // namespace + +namespace ray { +namespace gcs { + // Returns true if an actor should be loaded to registered_actors_. // `false` Cases: // 0. state is DEAD, and is not restartable @@ -163,11 +168,6 @@ bool OnInitializeActorShouldLoad(const ray::gcs::GcsInitData &gcs_init_data, } }; -} // namespace - -namespace ray { -namespace gcs { - bool is_uuid(const std::string &str) { static const boost::regex e( "[a-f0-9]{8}-[a-f0-9]{4}-4[a-f0-9]{3}-[89aAbB][a-f0-9]{3}-[a-f0-9]{12}"); @@ -217,6 +217,10 @@ rpc::ActorTableData::ActorState GcsActor::GetState() const { return actor_table_data_.state(); } +const std::string &GcsActor::GetVirtualClusterID() const { + return task_spec_->scheduling_strategy().virtual_cluster_id(); +} + ActorID GcsActor::GetActorID() const { return ActorID::FromBinary(actor_table_data_.actor_id()); } @@ -329,6 +333,7 @@ GcsActorManager::GcsActorManager( GcsPublisher *gcs_publisher, RuntimeEnvManager &runtime_env_manager, GcsFunctionManager &function_manager, + GcsVirtualClusterManager &gcs_virtual_cluster_manager, std::function destroy_owned_placement_group_if_needed, const rpc::CoreWorkerClientFactoryFn &worker_client_factory) : gcs_actor_scheduler_(std::move(scheduler)), @@ -340,6 +345,7 @@ GcsActorManager::GcsActorManager( std::move(destroy_owned_placement_group_if_needed)), runtime_env_manager_(runtime_env_manager), function_manager_(function_manager), + gcs_virtual_cluster_manager_(gcs_virtual_cluster_manager), actor_gc_delay_(RayConfig::instance().gcs_actor_table_min_duration_ms()) { RAY_CHECK(worker_client_factory_); RAY_CHECK(destroy_owned_placement_group_if_needed_); @@ -714,6 +720,18 @@ Status GcsActorManager::RegisterActor(const ray::rpc::RegisterActorRequest &requ const auto &actor_creation_task_spec = request.task_spec().actor_creation_task_spec(); auto actor_id = ActorID::FromBinary(actor_creation_task_spec.actor_id()); + auto virtual_cluster_id = + request.task_spec().scheduling_strategy().virtual_cluster_id(); + if (!virtual_cluster_id.empty()) { + auto virtual_cluster = + gcs_virtual_cluster_manager_.GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr || virtual_cluster->Divisible()) { + std::stringstream stream; + stream << "Invalid virtual cluster, virtual cluster id: " << virtual_cluster_id; + return Status::InvalidArgument(stream.str()); + } + } + auto iter = registered_actors_.find(actor_id); if (iter != registered_actors_.end()) { auto pending_register_iter = actor_to_register_callbacks_.find(actor_id); @@ -788,6 +806,10 @@ Status GcsActorManager::RegisterActor(const ray::rpc::RegisterActorRequest &requ request.task_spec().runtime_env_info()); } + for (auto &listener : actor_registration_listeners_) { + listener(actor); + } + // The backend storage is supposed to be reliable, so the status must be ok. RAY_CHECK_OK(gcs_table_storage_->ActorTaskSpecTable().Put( actor_id, @@ -1111,6 +1133,10 @@ void GcsActorManager::DestroyActor(const ActorID &actor_id, } } + for (auto &listener : actor_destroy_listeners_) { + listener(actor); + } + auto actor_table_data = std::make_shared(*mutable_actor_table_data); // The backend storage is reliable in the future, so the status must be ok. diff --git a/src/ray/gcs/gcs_server/gcs_actor_manager.h b/src/ray/gcs/gcs_server/gcs_actor_manager.h index 7431b1976cfb..ad66a3a4be97 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_manager.h +++ b/src/ray/gcs/gcs_server/gcs_actor_manager.h @@ -25,6 +25,7 @@ #include "ray/gcs/gcs_server/gcs_function_manager.h" #include "ray/gcs/gcs_server/gcs_init_data.h" #include "ray/gcs/gcs_server/gcs_table_storage.h" +#include "ray/gcs/gcs_server/gcs_virtual_cluster_manager.h" #include "ray/gcs/gcs_server/usage_stats_client.h" #include "ray/gcs/pubsub/gcs_pub_sub.h" #include "ray/rpc/gcs_server/gcs_rpc_server.h" @@ -37,6 +38,14 @@ namespace ray { namespace gcs { +// Returns true if an actor should be loaded to registered_actors_. +// `false` Cases: +// 0. state is DEAD, and is not restartable +// 1. root owner is job, and job is dead +// 2. root owner is another detached actor, and that actor is dead +bool OnInitializeActorShouldLoad(const ray::gcs::GcsInitData &gcs_init_data, + ray::ActorID actor_id); + /// GcsActor just wraps `ActorTableData` and provides some convenient interfaces to access /// the fields inside `ActorTableData`. /// This class is not thread-safe. @@ -176,6 +185,8 @@ class GcsActor { void UpdateState(rpc::ActorTableData::ActorState state); /// Get the state of this gcs actor. rpc::ActorTableData::ActorState GetState() const; + /// Get virtual cluster id of this actor + const std::string &GetVirtualClusterID() const; /// Get the id of this actor. ActorID GetActorID() const; @@ -264,6 +275,11 @@ using RestartActorCallback = std::function)>; using CreateActorCallback = std::function, const rpc::PushTaskReply &reply, const Status &status)>; +using ActorRegistrationListenerCallback = + std::function &)>; +using ActorDestroyListenerCallback = + std::function &)>; + /// GcsActorManager is responsible for managing the lifecycle of all actors. /// This class is not thread-safe. /// Actor State Transition Diagram: @@ -323,6 +339,7 @@ class GcsActorManager : public rpc::ActorInfoHandler { GcsPublisher *gcs_publisher, RuntimeEnvManager &runtime_env_manager, GcsFunctionManager &function_manager, + GcsVirtualClusterManager &gcs_virtual_cluster_manager, std::function destroy_owned_placement_group_if_needed, const rpc::CoreWorkerClientFactoryFn &worker_client_factory = nullptr); @@ -500,6 +517,18 @@ class GcsActorManager : public rpc::ActorInfoHandler { usage_stats_client_ = usage_stats_client; } + /// Add actor registration event listener. + void AddActorRegistrationListener(ActorRegistrationListenerCallback listener) { + RAY_CHECK(listener); + actor_registration_listeners_.emplace_back(std::move(listener)); + } + + /// Add actor destroy event listener. + void AddActorDestroyListener(ActorDestroyListenerCallback listener) { + RAY_CHECK(listener); + actor_destroy_listeners_.emplace_back(std::move(listener)); + } + private: const ray::rpc::ActorDeathCause GenNodeDiedCause( const ray::gcs::GcsActor *actor, std::shared_ptr node); @@ -707,6 +736,8 @@ class GcsActorManager : public rpc::ActorInfoHandler { RuntimeEnvManager &runtime_env_manager_; /// Function manager for GC purpose GcsFunctionManager &function_manager_; + /// Virtual cluster manager for scheduling purpose + GcsVirtualClusterManager &gcs_virtual_cluster_manager_; UsageStatsClient *usage_stats_client_; /// Run a function on a delay. This is useful for guaranteeing data will be @@ -724,6 +755,11 @@ class GcsActorManager : public rpc::ActorInfoHandler { // Make sure our unprotected maps are accessed from the same thread. // Currently protects actor_to_register_callbacks_. ThreadChecker thread_checker_; + /// Listeners which monitors the registration of actor. + std::vector actor_registration_listeners_; + + /// Listeners which monitors the destruction of actor. + std::vector actor_destroy_listeners_; // Debug info. enum CountType { diff --git a/src/ray/gcs/gcs_server/gcs_init_data.cc b/src/ray/gcs/gcs_server/gcs_init_data.cc index 55f952cfee06..570d12a7aa52 100644 --- a/src/ray/gcs/gcs_server/gcs_init_data.cc +++ b/src/ray/gcs/gcs_server/gcs_init_data.cc @@ -18,7 +18,7 @@ namespace ray { namespace gcs { void GcsInitData::AsyncLoad(Postable on_done) { // There are 5 kinds of table data need to be loaded. - auto count_down = std::make_shared(5); + auto count_down = std::make_shared(6); auto on_load_finished = Postable( [count_down, on_done]() mutable { if (--(*count_down) == 0) { @@ -36,6 +36,8 @@ void GcsInitData::AsyncLoad(Postable on_done) { AsyncLoadActorTaskSpecTableData(on_load_finished); AsyncLoadPlacementGroupTableData(on_load_finished); + + AsyncLoadVirtualClusterTableData(on_load_finished); } void GcsInitData::AsyncLoadJobTableData(Postable on_done) { @@ -94,5 +96,19 @@ void GcsInitData::AsyncLoadActorTaskSpecTableData(Postable on_done) { )); } +void GcsInitData::AsyncLoadVirtualClusterTableData(Postable on_done) { + RAY_LOG(INFO) << "Loading virtual cluster table data."; + RAY_CHECK_OK( + gcs_table_storage_.VirtualClusterTable().GetAll(std::move(on_done).TransformArg( + [this](absl::flat_hash_map result) + -> void { + virtual_cluster_table_data_ = std::move(result); + RAY_LOG(INFO) << "Finished loading virtual cluster table data, size = " + << virtual_cluster_table_data_.size(); + }) + + )); +} + } // namespace gcs } // namespace ray diff --git a/src/ray/gcs/gcs_server/gcs_init_data.h b/src/ray/gcs/gcs_server/gcs_init_data.h index 3fcabb904b2a..224f65191d79 100644 --- a/src/ray/gcs/gcs_server/gcs_init_data.h +++ b/src/ray/gcs/gcs_server/gcs_init_data.h @@ -64,6 +64,12 @@ class GcsInitData { return placement_group_table_data_; } + /// Get virtual cluster metadata. + const absl::flat_hash_map + &VirtualClusters() const { + return virtual_cluster_table_data_; + } + private: /// Load job metadata from the store into memory asynchronously. /// @@ -87,6 +93,11 @@ class GcsInitData { void AsyncLoadActorTaskSpecTableData(Postable on_done); + /// Load virtual cluster metadata from the store into memory asynchronously. + /// + /// \param on_done The callback when virtual cluster metadata is loaded successfully. + void AsyncLoadVirtualClusterTableData(Postable on_done); + protected: /// The gcs table storage. gcs::GcsTableStorage &gcs_table_storage_; @@ -105,6 +116,10 @@ class GcsInitData { absl::flat_hash_map actor_table_data_; absl::flat_hash_map actor_task_spec_table_data_; + + /// Virtual cluster metadata. + absl::flat_hash_map + virtual_cluster_table_data_; }; } // namespace gcs diff --git a/src/ray/gcs/gcs_server/gcs_job_manager.cc b/src/ray/gcs/gcs_server/gcs_job_manager.cc index e89fc0f7be41..30fa59eab5ff 100644 --- a/src/ray/gcs/gcs_server/gcs_job_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_job_manager.cc @@ -121,6 +121,19 @@ void GcsJobManager::HandleAddJob(rpc::AddJobRequest request, GCS_RPC_SEND_REPLY(send_reply_callback, reply, status); }; + auto virtual_cluster_id = mutable_job_table_data.virtual_cluster_id(); + if (!virtual_cluster_id.empty()) { + auto virtual_cluster = + gcs_virtual_cluster_manager_.GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr || virtual_cluster->Divisible()) { + std::stringstream stream; + stream << "Invalid virtual cluster, virtual cluster id: " << virtual_cluster_id; + auto status = Status::InvalidArgument(stream.str()); + on_done(status); + return; + } + } + Status status = gcs_table_storage_.JobTable().Put( job_id, mutable_job_table_data, {on_done, io_context_}); if (!status.ok()) { diff --git a/src/ray/gcs/gcs_server/gcs_job_manager.h b/src/ray/gcs/gcs_server/gcs_job_manager.h index 9534a475e092..48f952986ca8 100644 --- a/src/ray/gcs/gcs_server/gcs_job_manager.h +++ b/src/ray/gcs/gcs_server/gcs_job_manager.h @@ -26,6 +26,7 @@ #include "ray/gcs/gcs_server/gcs_function_manager.h" #include "ray/gcs/gcs_server/gcs_init_data.h" #include "ray/gcs/gcs_server/gcs_table_storage.h" +#include "ray/gcs/gcs_server/gcs_virtual_cluster_manager.h" #include "ray/gcs/pubsub/gcs_pub_sub.h" #include "ray/rpc/gcs_server/gcs_rpc_server.h" #include "ray/rpc/worker/core_worker_client.h" @@ -54,6 +55,7 @@ class GcsJobManager : public rpc::JobInfoHandler { GcsPublisher &gcs_publisher, RuntimeEnvManager &runtime_env_manager, GcsFunctionManager &function_manager, + GcsVirtualClusterManager &gcs_virtual_cluster_manager, InternalKVInterface &internal_kv, instrumented_io_context &io_context, rpc::CoreWorkerClientFactoryFn client_factory = nullptr) @@ -61,6 +63,7 @@ class GcsJobManager : public rpc::JobInfoHandler { gcs_publisher_(gcs_publisher), runtime_env_manager_(runtime_env_manager), function_manager_(function_manager), + gcs_virtual_cluster_manager_(gcs_virtual_cluster_manager), internal_kv_(internal_kv), io_context_(io_context), core_worker_clients_(client_factory) {} @@ -131,6 +134,7 @@ class GcsJobManager : public rpc::JobInfoHandler { ray::RuntimeEnvManager &runtime_env_manager_; GcsFunctionManager &function_manager_; + GcsVirtualClusterManager &gcs_virtual_cluster_manager_; InternalKVInterface &internal_kv_; instrumented_io_context &io_context_; /// The cached core worker clients which are used to communicate with workers. diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index 2914559cffc9..b2708299703f 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -33,12 +33,14 @@ GcsNodeManager::GcsNodeManager(GcsPublisher *gcs_publisher, gcs::GcsTableStorage *gcs_table_storage, instrumented_io_context &io_context, rpc::NodeManagerClientPool *raylet_client_pool, - const ClusterID &cluster_id) + const ClusterID &cluster_id, + GcsVirtualClusterManager &gcs_virtual_cluster_manager) : gcs_publisher_(gcs_publisher), gcs_table_storage_(gcs_table_storage), io_context_(io_context), raylet_client_pool_(raylet_client_pool), - cluster_id_(cluster_id) {} + cluster_id_(cluster_id), + gcs_virtual_cluster_manager_(gcs_virtual_cluster_manager) {} void GcsNodeManager::WriteNodeExportEvent(rpc::GcsNodeInfo node_info) const { /// Write node_info as a export node event if @@ -229,14 +231,32 @@ void GcsNodeManager::HandleGetAllNodeInfo(rpc::GetAllNodeInfoRequest request, filter_state = request.filters().state(); } std::string filter_node_name = request.filters().node_name(); - auto filter_fn = [&filter_node_id, &filter_node_name](const rpc::GcsNodeInfo &node) { + std::string filter_virtual_cluster_id = request.filters().virtual_cluster_id(); + RAY_LOG(DEBUG) << "GetAllNodeInfo request, limit = " << limit + << ", filter_node_id = " << filter_node_id + << ", filter_node_name = " << filter_node_name + << ", filter_virtual_cluster_id = " << filter_virtual_cluster_id; + auto filter_fn = [this, &filter_node_id, &filter_node_name, &filter_virtual_cluster_id]( + const rpc::GcsNodeInfo &node) { if (!filter_node_id.IsNil() && filter_node_id != NodeID::FromBinary(node.node_id())) { return false; } if (!filter_node_name.empty() && filter_node_name != node.node_name()) { return false; } - return true; + if (filter_virtual_cluster_id.empty()) { + return true; + } + auto virtual_cluster = + gcs_virtual_cluster_manager_.GetVirtualCluster(filter_virtual_cluster_id); + if (virtual_cluster == nullptr) { + return false; + } + bool contains = + virtual_cluster->ContainsNodeInstance(NodeID::FromBinary(node.node_id()).Hex()); + RAY_LOG(DEBUG) << "Filtering node " << node.node_id() << " in virtual cluster " + << filter_virtual_cluster_id << " contains: " << contains; + return contains; }; int64_t num_added = 0; int64_t num_filtered = 0; diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.h b/src/ray/gcs/gcs_server/gcs_node_manager.h index f2ee24be3ae1..2fa40d5056c3 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.h +++ b/src/ray/gcs/gcs_server/gcs_node_manager.h @@ -27,6 +27,7 @@ #include "ray/gcs/gcs_server/gcs_init_data.h" #include "ray/gcs/gcs_server/gcs_resource_manager.h" #include "ray/gcs/gcs_server/gcs_table_storage.h" +#include "ray/gcs/gcs_server/gcs_virtual_cluster_manager.h" #include "ray/gcs/pubsub/gcs_pub_sub.h" #include "ray/rpc/client_call.h" #include "ray/rpc/gcs_server/gcs_rpc_server.h" @@ -52,7 +53,8 @@ class GcsNodeManager : public rpc::NodeInfoHandler { gcs::GcsTableStorage *gcs_table_storage, instrumented_io_context &io_context, rpc::NodeManagerClientPool *raylet_client_pool, - const ClusterID &cluster_id); + const ClusterID &cluster_id, + GcsVirtualClusterManager &gcs_virtual_cluster_manager); /// Handle register rpc request come from raylet. void HandleGetClusterId(rpc::GetClusterIdRequest request, @@ -253,6 +255,8 @@ class GcsNodeManager : public rpc::NodeInfoHandler { rpc::NodeManagerClientPool *raylet_client_pool_ = nullptr; /// Cluster ID to be shared with clients when connecting. const ClusterID cluster_id_; + /// The gcs virtual cluster handler and service. + GcsVirtualClusterManager &gcs_virtual_cluster_manager_; // Debug info. enum CountType { diff --git a/src/ray/gcs/gcs_server/gcs_placement_group_manager.cc b/src/ray/gcs/gcs_server/gcs_placement_group_manager.cc index 889e47a68cb2..6edbcf000627 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_placement_group_manager.cc @@ -120,6 +120,10 @@ rpc::PlacementStrategy GcsPlacementGroup::GetStrategy() const { return placement_group_table_data_.strategy(); } +const std::string &GcsPlacementGroup::GetVirtualClusterID() const { + return placement_group_table_data_.virtual_cluster_id(); +} + const rpc::PlacementGroupTableData &GcsPlacementGroup::GetPlacementGroupTableData() const { return placement_group_table_data_; @@ -260,6 +264,10 @@ void GcsPlacementGroupManager::RegisterPlacementGroup( placement_group); AddToPendingQueue(placement_group); + for (auto &listener : placement_group_registration_listeners_) { + listener(placement_group); + } + RAY_CHECK_OK(gcs_table_storage_->PlacementGroupTable().Put( placement_group_id, placement_group->GetPlacementGroupTableData(), @@ -556,6 +564,10 @@ void GcsPlacementGroupManager::RemovePlacementGroup( infeasible_placement_groups_.erase(pending_it); } + for (auto &listener : placement_group_destroy_listeners_) { + listener(placement_group); + } + // Flush the status and respond to workers. placement_group->UpdateState(rpc::PlacementGroupTableData::REMOVED); placement_group->GetMutableStats()->set_scheduling_state( diff --git a/src/ray/gcs/gcs_server/gcs_placement_group_manager.h b/src/ray/gcs/gcs_server/gcs_placement_group_manager.h index 728b1eacf741..5f61e232de16 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_manager.h +++ b/src/ray/gcs/gcs_server/gcs_placement_group_manager.h @@ -85,6 +85,8 @@ class GcsPlacementGroup { placement_group_table_data_.set_ray_namespace(ray_namespace); placement_group_table_data_.set_placement_group_creation_timestamp_ms( current_sys_time_ms()); + placement_group_table_data_.set_virtual_cluster_id( + placement_group_spec.virtual_cluster_id()); SetupStates(); } @@ -132,6 +134,9 @@ class GcsPlacementGroup { /// Get the Strategy rpc::PlacementStrategy GetStrategy() const; + /// Get the Virtual Cluster ID associated with this PlacementGroup + const std::string &GetVirtualClusterID() const; + /// Get debug string for the placement group. std::string DebugString() const; @@ -218,6 +223,11 @@ class GcsPlacementGroup { std::optional last_metric_state_; }; +using PlacementGroupRegistrationListenerCallback = + std::function &)>; +using PlacementGroupDestroyListenerCallback = + std::function &)>; + /// GcsPlacementGroupManager is responsible for managing the lifecycle of all placement /// group. This class is not thread-safe. /// The placementGroup will be added into queue and set the status as pending first and @@ -391,6 +401,19 @@ class GcsPlacementGroupManager : public rpc::PlacementGroupInfoHandler { /// the returned rpc has any placement_group_data. virtual std::shared_ptr GetPlacementGroupLoad() const; + /// Add placement group registration event listener. + void AddPlacementGroupRegistrationListener( + PlacementGroupRegistrationListenerCallback listener) { + RAY_CHECK(listener); + placement_group_registration_listeners_.emplace_back(std::move(listener)); + } + + /// Add placement group destroy event listener. + void AddPlacementGroupDestroyListener(PlacementGroupDestroyListenerCallback listener) { + RAY_CHECK(listener); + placement_group_destroy_listeners_.emplace_back(std::move(listener)); + } + protected: /// For testing/mocking only. explicit GcsPlacementGroupManager(instrumented_io_context &io_context, @@ -509,6 +532,13 @@ class GcsPlacementGroupManager : public rpc::PlacementGroupInfoHandler { /// Total number of successfully created placement groups in the cluster lifetime. int64_t lifetime_num_placement_groups_created_ = 0; + /// Listeners which monitors the registration of placement group. + std::vector + placement_group_registration_listeners_; + + /// Listeners which monitors the destruction of placement group. + std::vector placement_group_destroy_listeners_; + // Debug info. enum CountType { CREATE_PLACEMENT_GROUP_REQUEST = 0, diff --git a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc index f85971208372..c3b7cdb24277 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc +++ b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc @@ -67,7 +67,9 @@ void GcsPlacementGroupScheduler::ScheduleUnplacedBundles( CreateSchedulingOptions(placement_group->GetPlacementGroupID(), strategy, placement_group->GetMaxCpuFractionPerNode(), - placement_group->GetSoftTargetNodeID()); + placement_group->GetSoftTargetNodeID(), + placement_group->GetVirtualClusterID()); + auto scheduling_result = cluster_resource_scheduler_.Schedule(resource_request_list, scheduling_options); @@ -470,21 +472,25 @@ SchedulingOptions GcsPlacementGroupScheduler::CreateSchedulingOptions( const PlacementGroupID &placement_group_id, rpc::PlacementStrategy strategy, double max_cpu_fraction_per_node, - NodeID soft_target_node_id) { + NodeID soft_target_node_id, + const std::string &virtual_cluster_id) { switch (strategy) { case rpc::PlacementStrategy::PACK: - return SchedulingOptions::BundlePack(max_cpu_fraction_per_node); + return SchedulingOptions::BundlePack(max_cpu_fraction_per_node, virtual_cluster_id); case rpc::PlacementStrategy::SPREAD: - return SchedulingOptions::BundleSpread(max_cpu_fraction_per_node); + return SchedulingOptions::BundleSpread(max_cpu_fraction_per_node, virtual_cluster_id); case rpc::PlacementStrategy::STRICT_PACK: return SchedulingOptions::BundleStrictPack( max_cpu_fraction_per_node, soft_target_node_id.IsNil() ? scheduling::NodeID::Nil() - : scheduling::NodeID(soft_target_node_id.Binary())); + : scheduling::NodeID(soft_target_node_id.Binary()), + virtual_cluster_id); case rpc::PlacementStrategy::STRICT_SPREAD: return SchedulingOptions::BundleStrictSpread( - max_cpu_fraction_per_node, CreateSchedulingContext(placement_group_id)); + max_cpu_fraction_per_node, + CreateSchedulingContext(placement_group_id), + virtual_cluster_id); default: RAY_LOG(FATAL) << "Unsupported scheduling type: " << rpc::PlacementStrategy_Name(strategy); diff --git a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h index 2fc76f9a2e22..f64ff3e5b01e 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h +++ b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h @@ -464,7 +464,8 @@ class GcsPlacementGroupScheduler : public GcsPlacementGroupSchedulerInterface { SchedulingOptions CreateSchedulingOptions(const PlacementGroupID &placement_group_id, rpc::PlacementStrategy strategy, double max_cpu_fraction_per_node, - NodeID soft_target_node_id); + NodeID soft_target_node_id, + const std::string &virtual_cluster_id); /// Try to release bundle resource to cluster resource manager. /// diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.cc b/src/ray/gcs/gcs_server/gcs_resource_manager.cc index cac3a930289f..142d00c31ca3 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.cc @@ -20,15 +20,18 @@ namespace ray { namespace gcs { -GcsResourceManager::GcsResourceManager(instrumented_io_context &io_context, - ClusterResourceManager &cluster_resource_manager, - GcsNodeManager &gcs_node_manager, - NodeID local_node_id, - ClusterTaskManager *cluster_task_manager) +GcsResourceManager::GcsResourceManager( + instrumented_io_context &io_context, + ClusterResourceManager &cluster_resource_manager, + GcsNodeManager &gcs_node_manager, + NodeID local_node_id, + GcsVirtualClusterManager &gcs_virtual_cluster_manager, + ClusterTaskManager *cluster_task_manager) : io_context_(io_context), cluster_resource_manager_(cluster_resource_manager), gcs_node_manager_(gcs_node_manager), local_node_id_(std::move(local_node_id)), + gcs_virtual_cluster_manager_(gcs_virtual_cluster_manager), cluster_task_manager_(cluster_task_manager) {} void GcsResourceManager::ConsumeSyncMessage( @@ -83,11 +86,24 @@ void GcsResourceManager::HandleGetAllAvailableResources( rpc::GetAllAvailableResourcesRequest request, rpc::GetAllAvailableResourcesReply *reply, rpc::SendReplyCallback send_reply_callback) { + const std::string &virtual_cluster_id = request.virtual_cluster_id(); + RAY_LOG(DEBUG) << "Getting all available resources, virtual_cluster_id: " + << virtual_cluster_id; auto local_scheduling_node_id = scheduling::NodeID(local_node_id_.Binary()); + auto virtual_cluster = + gcs_virtual_cluster_manager_.GetVirtualCluster(virtual_cluster_id); for (const auto &node_resources_entry : cluster_resource_manager_.GetResourceView()) { if (node_resources_entry.first == local_scheduling_node_id) { continue; } + if (!virtual_cluster_id.empty()) { + /// Filter out nodes that are not in the virtual cluster. + NodeID node_id = NodeID::FromBinary(node_resources_entry.first.Binary()); + if (virtual_cluster == nullptr || + !virtual_cluster->ContainsNodeInstance(node_id.Hex())) { + continue; + } + } rpc::AvailableResources resource; resource.set_node_id(node_resources_entry.first.Binary()); const auto &node_resources = node_resources_entry.second.GetLocalView(); @@ -122,11 +138,25 @@ void GcsResourceManager::HandleGetAllTotalResources( rpc::GetAllTotalResourcesRequest request, rpc::GetAllTotalResourcesReply *reply, rpc::SendReplyCallback send_reply_callback) { + const std::string &virtual_cluster_id = request.virtual_cluster_id(); + RAY_LOG(DEBUG) << "Getting all total resources, virtual_cluster_id: " + << virtual_cluster_id; auto local_scheduling_node_id = scheduling::NodeID(local_node_id_.Binary()); + auto virtual_cluster = + gcs_virtual_cluster_manager_.GetVirtualCluster(virtual_cluster_id); for (const auto &node_resources_entry : cluster_resource_manager_.GetResourceView()) { if (node_resources_entry.first == local_scheduling_node_id) { continue; } + if (!virtual_cluster_id.empty()) { + /// Filter out nodes that are not in the virtual cluster. + NodeID node_id = NodeID::FromBinary(node_resources_entry.first.Binary()); + if (virtual_cluster == nullptr || + !virtual_cluster->ContainsNodeInstance(node_id.Hex())) { + continue; + } + } + rpc::TotalResources resource; resource.set_node_id(node_resources_entry.first.Binary()); const auto &node_resources = node_resources_entry.second.GetLocalView(); diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index 2cc42c37ee5b..26141e028e49 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -23,6 +23,7 @@ #include "ray/gcs/gcs_server/gcs_init_data.h" #include "ray/gcs/gcs_server/gcs_node_manager.h" #include "ray/gcs/gcs_server/gcs_table_storage.h" +#include "ray/gcs/gcs_server/gcs_virtual_cluster_manager.h" #include "ray/gcs/gcs_server/state_util.h" #include "ray/raylet/scheduling/cluster_resource_manager.h" #include "ray/raylet/scheduling/cluster_task_manager.h" @@ -64,6 +65,7 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, ClusterResourceManager &cluster_resource_manager, GcsNodeManager &gcs_node_manager, NodeID local_node_id, + GcsVirtualClusterManager &gcs_virtual_cluster_manager, ClusterTaskManager *cluster_task_manager = nullptr); virtual ~GcsResourceManager() = default; @@ -198,6 +200,8 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, ClusterResourceManager &cluster_resource_manager_; GcsNodeManager &gcs_node_manager_; NodeID local_node_id_; + /// The gcs virtual cluster handler and service. + GcsVirtualClusterManager &gcs_virtual_cluster_manager_; ClusterTaskManager *cluster_task_manager_; /// Num of alive nodes in the cluster. size_t num_alive_nodes_ = 0; diff --git a/src/ray/gcs/gcs_server/gcs_server.ant.cc b/src/ray/gcs/gcs_server/gcs_server.ant.cc new file mode 100644 index 000000000000..c397e3470961 --- /dev/null +++ b/src/ray/gcs/gcs_server/gcs_server.ant.cc @@ -0,0 +1,37 @@ +// Copyright 2017 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/gcs/gcs_server/gcs_server.h" + +#include "ray/gcs/gcs_server/gcs_virtual_cluster_manager.h" + +namespace ray { +namespace gcs { +void GcsServer::InitGcsVirtualClusterManager(const GcsInitData &gcs_init_data) { + RAY_CHECK(gcs_table_storage_ && gcs_publisher_); + gcs_virtual_cluster_manager_ = std::make_shared( + io_context_provider_.GetDefaultIOContext(), + *gcs_table_storage_, + *gcs_publisher_, + cluster_resource_scheduler_->GetClusterResourceManager(), + periodical_runner_); + // Initialize by gcs tables data. + gcs_virtual_cluster_manager_->Initialize(gcs_init_data); + // Register service. + gcs_virtual_cluster_service_.reset(new rpc::VirtualClusterInfoGrpcService( + io_context_provider_.GetDefaultIOContext(), *gcs_virtual_cluster_manager_)); + rpc_server_.RegisterService(*gcs_virtual_cluster_service_); +} +} // namespace gcs +} // namespace ray \ No newline at end of file diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index 6bec28f6fcf4..549b2ef67f48 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -25,6 +25,7 @@ #include "ray/gcs/gcs_server/gcs_job_manager.h" #include "ray/gcs/gcs_server/gcs_placement_group_manager.h" #include "ray/gcs/gcs_server/gcs_resource_manager.h" +#include "ray/gcs/gcs_server/gcs_virtual_cluster_manager.h" #include "ray/gcs/gcs_server/gcs_worker_manager.h" #include "ray/gcs/gcs_server/store_client_kv.h" #include "ray/pubsub/publisher.h" @@ -105,6 +106,7 @@ GcsServer::GcsServer(const ray::gcs::GcsServerConfig &config, rpc::ChannelType::RAY_ERROR_INFO_CHANNEL, rpc::ChannelType::RAY_LOG_CHANNEL, rpc::ChannelType::RAY_NODE_RESOURCE_USAGE_CHANNEL, + rpc::ChannelType::RAY_VIRTUAL_CLUSTER_CHANNEL, }, /*periodical_runner=*/*pubsub_periodical_runner_, /*get_time_ms=*/[]() { return absl::GetCurrentTimeNanos() / 1e6; }, @@ -183,6 +185,9 @@ void GcsServer::DoStart(const GcsInitData &gcs_init_data) { // Init cluster resource scheduler. InitClusterResourceScheduler(); + // Init gcs virtual cluster manager. + InitGcsVirtualClusterManager(gcs_init_data); + // Init gcs node manager. InitGcsNodeManager(gcs_init_data); @@ -292,7 +297,8 @@ void GcsServer::InitGcsNodeManager(const GcsInitData &gcs_init_data) { gcs_table_storage_.get(), io_context_provider_.GetDefaultIOContext(), raylet_client_pool_.get(), - rpc_server_.GetClusterId()); + rpc_server_.GetClusterId(), + *gcs_virtual_cluster_manager_); // Initialize by gcs tables data. gcs_node_manager_->Initialize(gcs_init_data); // Register service. @@ -330,6 +336,7 @@ void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { cluster_resource_scheduler_->GetClusterResourceManager(), *gcs_node_manager_, kGCSNodeID, + *gcs_virtual_cluster_manager_, cluster_task_manager_.get()); // Initialize by gcs tables data. @@ -392,7 +399,25 @@ void GcsServer::InitClusterResourceScheduler() { NodeResources(), /*is_node_available_fn=*/ [](auto) { return true; }, - /*is_local_node_with_raylet=*/false); + /*is_local_node_with_raylet=*/false, + /*is_node_schedulable_fn=*/ + [this](scheduling::NodeID node_id, const SchedulingContext *context) { + // Check if the virtual cluster manager exists. + if (gcs_virtual_cluster_manager_ == nullptr || + context->virtual_cluster_id.empty() || + context->virtual_cluster_id == kPrimaryClusterID) { + return true; + } + auto node_instance_id = NodeID::FromBinary(node_id.Binary()).Hex(); + auto virtual_cluster = + gcs_virtual_cluster_manager_->GetVirtualCluster(context->virtual_cluster_id); + if (virtual_cluster == nullptr) { + return true; + } + RAY_CHECK(!virtual_cluster->Divisible()); + // Check if the node is contained within the specified virtual cluster. + return virtual_cluster->ContainsNodeInstance(node_instance_id); + }); } void GcsServer::InitClusterTaskManager() { @@ -421,6 +446,7 @@ void GcsServer::InitGcsJobManager(const GcsInitData &gcs_init_data) { *gcs_publisher_, *runtime_env_manager_, *function_manager_, + *gcs_virtual_cluster_manager_, kv_manager_->GetInstance(), io_context_provider_.GetDefaultIOContext(), client_factory); @@ -481,6 +507,7 @@ void GcsServer::InitGcsActorManager(const GcsInitData &gcs_init_data) { gcs_publisher_.get(), *runtime_env_manager_, *function_manager_, + *gcs_virtual_cluster_manager_, [this](const ActorID &actor_id) { gcs_placement_group_manager_->CleanPlacementGroupIfNeededWhenActorDead( actor_id); @@ -495,6 +522,25 @@ void GcsServer::InitGcsActorManager(const GcsInitData &gcs_init_data) { // Initialize by gcs tables data. gcs_actor_manager_->Initialize(gcs_init_data); + // Add event listeners + gcs_actor_manager_->AddActorRegistrationListener( + [this](const std::shared_ptr &actor) { + if (gcs_virtual_cluster_manager_ != nullptr) { + if (actor->IsDetached()) { + gcs_virtual_cluster_manager_->OnDetachedActorRegistration( + actor->GetVirtualClusterID(), actor->GetActorID()); + } + } + }); + gcs_actor_manager_->AddActorDestroyListener( + [this](const std::shared_ptr &actor) { + if (gcs_virtual_cluster_manager_ != nullptr) { + if (actor->IsDetached()) { + gcs_virtual_cluster_manager_->OnDetachedActorDestroy( + actor->GetVirtualClusterID(), actor->GetActorID()); + } + } + }); // Register service. actor_info_service_ = std::make_unique( io_context_provider_.GetDefaultIOContext(), *gcs_actor_manager_); @@ -520,6 +566,27 @@ void GcsServer::InitGcsPlacementGroupManager(const GcsInitData &gcs_init_data) { }); // Initialize by gcs tables data. gcs_placement_group_manager_->Initialize(gcs_init_data); + // Add event listeners + gcs_placement_group_manager_->AddPlacementGroupRegistrationListener( + [this](const std::shared_ptr &placement_group) { + if (gcs_virtual_cluster_manager_ != nullptr) { + if (placement_group->IsDetached()) { + gcs_virtual_cluster_manager_->OnDetachedPlacementGroupRegistration( + placement_group->GetVirtualClusterID(), + placement_group->GetPlacementGroupID()); + } + } + }); + gcs_placement_group_manager_->AddPlacementGroupDestroyListener( + [this](const std::shared_ptr &placement_group) { + if (gcs_virtual_cluster_manager_ != nullptr) { + if (placement_group->IsDetached()) { + gcs_virtual_cluster_manager_->OnDetachedPlacementGroupDestroy( + placement_group->GetVirtualClusterID(), + placement_group->GetPlacementGroupID()); + } + } + }); // Register service. placement_group_info_service_.reset(new rpc::PlacementGroupInfoGrpcService( io_context_provider_.GetDefaultIOContext(), *gcs_placement_group_manager_)); @@ -730,6 +797,7 @@ void GcsServer::InstallEventListeners() { // placement groups and the pending actors. auto node_id = NodeID::FromBinary(node->node_id()); gcs_resource_manager_->OnNodeAdd(*node); + gcs_virtual_cluster_manager_->OnNodeAdd(*node); gcs_placement_group_manager_->OnNodeAdd(node_id); gcs_actor_manager_->SchedulePendingActors(); gcs_autoscaler_state_manager_->OnNodeAdd(*node); @@ -755,6 +823,7 @@ void GcsServer::InstallEventListeners() { // All of the related placement groups and actors should be reconstructed when a // node is removed from the GCS. gcs_resource_manager_->OnNodeDead(node_id); + gcs_virtual_cluster_manager_->OnNodeDead(*node); gcs_placement_group_manager_->OnNodeDead(node_id); gcs_actor_manager_->OnNodeDead(node, node_ip_address); gcs_job_manager_->OnNodeDead(node_id); @@ -791,6 +860,7 @@ void GcsServer::InstallEventListeners() { const auto job_id = JobID::FromBinary(job_data.job_id()); gcs_task_manager_->OnJobFinished(job_id, job_data.end_time()); gcs_placement_group_manager_->CleanPlacementGroupIfNeededWhenJobDead(job_id); + gcs_virtual_cluster_manager_->OnJobFinished(job_data); }); // Install scheduling event listeners. diff --git a/src/ray/gcs/gcs_server/gcs_server.h b/src/ray/gcs/gcs_server/gcs_server.h index 4de913136244..b1315fc7e11f 100644 --- a/src/ray/gcs/gcs_server/gcs_server.h +++ b/src/ray/gcs/gcs_server/gcs_server.h @@ -67,6 +67,7 @@ struct GcsServerConfig { }; class GcsNodeManager; +class GcsVirtualClusterManager; class GcsActorManager; class GcsJobManager; class GcsWorkerManager; @@ -156,6 +157,9 @@ class GcsServer { /// Initialize gcs actor manager. void InitGcsActorManager(const GcsInitData &gcs_init_data); + /// Initialize gcs virtual cluster manager. + void InitGcsVirtualClusterManager(const GcsInitData &gcs_init_data); + /// Initialize gcs placement group manager. void InitGcsPlacementGroupManager(const GcsInitData &gcs_init_data); @@ -249,6 +253,9 @@ class GcsServer { std::unique_ptr gcs_node_manager_; /// The health check manager. std::shared_ptr gcs_healthcheck_manager_; + /// The gcs virtual cluster handler and service. + std::shared_ptr gcs_virtual_cluster_manager_; + std::unique_ptr gcs_virtual_cluster_service_; /// The gcs redis failure detector. std::unique_ptr gcs_redis_failure_detector_; /// The gcs placement group manager. diff --git a/src/ray/gcs/gcs_server/gcs_table_storage.cc b/src/ray/gcs/gcs_server/gcs_table_storage.cc index 43fa8f7cbfc5..e9ade486cbbf 100644 --- a/src/ray/gcs/gcs_server/gcs_table_storage.cc +++ b/src/ray/gcs/gcs_server/gcs_table_storage.cc @@ -205,6 +205,7 @@ template class GcsTable; template class GcsTableWithJobId; template class GcsTableWithJobId; template class GcsTable; +template class GcsTable; } // namespace gcs } // namespace ray diff --git a/src/ray/gcs/gcs_server/gcs_table_storage.h b/src/ray/gcs/gcs_server/gcs_table_storage.h index 63ba4da6c6a8..c029aa48acbf 100644 --- a/src/ray/gcs/gcs_server/gcs_table_storage.h +++ b/src/ray/gcs/gcs_server/gcs_table_storage.h @@ -17,6 +17,7 @@ #include #include +#include "ray/common/virtual_cluster_id.h" #include "ray/gcs/store_client/in_memory_store_client.h" #include "ray/gcs/store_client/observable_store_client.h" #include "ray/gcs/store_client/redis_store_client.h" @@ -32,6 +33,7 @@ using rpc::JobTableData; using rpc::PlacementGroupTableData; using rpc::ResourceUsageBatchData; using rpc::TaskSpec; +using rpc::VirtualClusterTableData; using rpc::WorkerTableData; /// \class GcsTable @@ -213,6 +215,15 @@ class GcsWorkerTable : public GcsTable { } }; +class GcsVirtualClusterTable + : public GcsTable { + public: + explicit GcsVirtualClusterTable(std::shared_ptr store_client) + : GcsTable(std::move(store_client)) { + table_name_ = TablePrefix_Name(TablePrefix::VIRTUAL_CLUSTER); + } +}; + /// \class GcsTableStorage /// /// This class is not meant to be used directly. All gcs table storage classes should @@ -227,6 +238,7 @@ class GcsTableStorage { placement_group_table_ = std::make_unique(store_client_); node_table_ = std::make_unique(store_client_); worker_table_ = std::make_unique(store_client_); + virtual_cluster_table_ = std::make_unique(store_client_); } virtual ~GcsTableStorage() = default; @@ -261,6 +273,11 @@ class GcsTableStorage { return *worker_table_; } + GcsVirtualClusterTable &VirtualClusterTable() { + RAY_CHECK(virtual_cluster_table_ != nullptr); + return *virtual_cluster_table_; + } + Status AsyncGetNextJobID(Postable callback) { RAY_CHECK(store_client_); return store_client_->AsyncGetNextJobID(std::move(callback)); @@ -274,6 +291,7 @@ class GcsTableStorage { std::unique_ptr placement_group_table_; std::unique_ptr node_table_; std::unique_ptr worker_table_; + std::unique_ptr virtual_cluster_table_; }; /// \class RedisGcsTableStorage diff --git a/src/ray/gcs/gcs_server/gcs_virtual_cluster.cc b/src/ray/gcs/gcs_server/gcs_virtual_cluster.cc new file mode 100644 index 000000000000..4a96f17266e0 --- /dev/null +++ b/src/ray/gcs/gcs_server/gcs_virtual_cluster.cc @@ -0,0 +1,1139 @@ +// Copyright 2017 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/gcs/gcs_server/gcs_virtual_cluster.h" + +namespace ray { +namespace gcs { + +ReplicaSets buildReplicaSets(const ReplicaInstances &replica_instances) { + ReplicaSets result; + for (const auto &[template_id, job_node_instances] : replica_instances) { + int32_t count = 0; + for (const auto &[_, node_instances] : job_node_instances) { + count += node_instances.size(); + } + result[template_id] = count; + } + return result; +} + +/// Calculate the difference between two replica instances. +ReplicaInstances ReplicaInstancesDifference(const ReplicaInstances &left, + const ReplicaInstances &right) { + ReplicaInstances result; + for (const auto &[template_id, job_node_instances] : left) { + for (const auto &[job_cluster_id, node_instances] : job_node_instances) { + auto right_iter = right.find(template_id); + if (right_iter == right.end()) { + result[template_id][job_cluster_id] = node_instances; + } else { + auto right_job_cluster_iter = right_iter->second.find(job_cluster_id); + if (right_job_cluster_iter == right_iter->second.end()) { + result[template_id][job_cluster_id] = node_instances; + } else { + for (const auto &[node_instance_id, node_instance] : node_instances) { + if (right_job_cluster_iter->second.find(node_instance_id) == + right_job_cluster_iter->second.end()) { + result[template_id][job_cluster_id][node_instance_id] = node_instance; + } + } + } + } + } + } + return result; +} + +std::string DebugString(const ReplicaInstances &replica_instances, int indent /* = 0*/) { + std::ostringstream stream; + stream << "{\n"; + for (const auto &[template_id, job_cluster_instances] : replica_instances) { + stream << std::string(indent + 2, ' ') << template_id << " : {\n"; + for (const auto &[job_cluster_id, node_instances] : job_cluster_instances) { + stream << std::string(indent + 4, ' ') << job_cluster_id << " : {\n"; + for (const auto &[node_instance_id, node_instance] : node_instances) { + stream << std::string(indent + 6, ' ') << node_instance_id << " : " + << node_instance->DebugString() << ",\n"; + } + stream << std::string(indent + 4, ' ') << "},\n"; + } + stream << std::string(indent + 2, ' ') << "},\n"; + } + stream << std::string(indent, ' ') << "}\n"; + return stream.str(); +} + +std::string DebugString(const ReplicaSets &replica_sets, int indent /* = 0*/) { + std::ostringstream stream; + stream << "{\n"; + for (const auto &[template_id, replica] : replica_sets) { + stream << std::string(indent + 2, ' ') << template_id << " : " << replica << ",\n"; + } + stream << std::string(indent, ' ') << "}\n"; + return stream.str(); +} + +///////////////////////// VirtualCluster ///////////////////////// +void VirtualCluster::UpdateNodeInstances(ReplicaInstances replica_instances_to_add, + ReplicaInstances replica_instances_to_remove) { + // Remove node instances from the virtual cluster. + RemoveNodeInstances(std::move(replica_instances_to_remove)); + // Insert node instances to the virtual cluster. + InsertNodeInstances(std::move(replica_instances_to_add)); + // Update the revision of the cluster. + revision_ = current_sys_time_ns(); +} + +void VirtualCluster::InsertNodeInstances(ReplicaInstances replica_instances) { + for (auto &[template_id, job_node_instances] : replica_instances) { + auto &cached_job_node_instances = visible_node_instances_[template_id]; + auto &replicas = replica_sets_[template_id]; + for (auto &[job_cluster_id, node_instances] : job_node_instances) { + auto &cached_node_instances = cached_job_node_instances[job_cluster_id]; + for (auto &[id, node_instance] : node_instances) { + node_instances_map_[id] = node_instance; + cached_node_instances[id] = std::move(node_instance); + ++replicas; + } + } + } +} + +void VirtualCluster::RemoveNodeInstances(ReplicaInstances replica_instances) { + for (auto &[template_id, job_node_instances] : replica_instances) { + auto template_iter = visible_node_instances_.find(template_id); + if (template_iter == visible_node_instances_.end()) { + RAY_LOG(WARNING) << "The template id " << template_id << " is not found in cluster " + << GetID(); + continue; + } + + auto replica_set_iter = replica_sets_.find(template_id); + RAY_CHECK(replica_set_iter != replica_sets_.end()); + + for (auto &[job_cluster_id, node_instances] : job_node_instances) { + auto job_cluster_iter = template_iter->second.find(job_cluster_id); + RAY_CHECK(job_cluster_iter != template_iter->second.end()); + + for (auto &[id, _] : node_instances) { + if (job_cluster_iter->second.erase(id) == 0) { + RAY_LOG(WARNING) << "The node instance " << id << " is not found in cluster " + << GetID(); + } else { + node_instances_map_.erase(id); + RAY_CHECK(--(replica_set_iter->second) >= 0); + } + } + + if (job_cluster_iter->second.empty()) { + template_iter->second.erase(job_cluster_iter); + } + } + + if (template_iter->second.empty()) { + visible_node_instances_.erase(template_iter); + } + + if (replica_set_iter->second == 0) { + replica_sets_.erase(replica_set_iter); + } + } +} + +bool VirtualCluster::LookupUndividedNodeInstances( + const ReplicaSets &replica_sets, + ReplicaInstances &replica_instances, + NodeInstanceFilter node_instance_filter) const { + bool success = true; + for (const auto &[template_id, replicas] : replica_sets) { + auto &template_node_instances = replica_instances[template_id]; + if (replicas <= 0) { + continue; + } + + auto iter = visible_node_instances_.find(template_id); + if (iter == visible_node_instances_.end()) { + success = false; + continue; + } + auto empty_iter = iter->second.find(kUndividedClusterId); + if (empty_iter == iter->second.end()) { + success = false; + continue; + } + + auto &job_node_instances = template_node_instances[kUndividedClusterId]; + for (const auto &[id, node_instance] : empty_iter->second) { + if (node_instance_filter != nullptr && !node_instance_filter(*node_instance)) { + continue; + } + job_node_instances.emplace(id, node_instance); + if (job_node_instances.size() == static_cast(replicas)) { + break; + } + } + if (job_node_instances.size() < static_cast(replicas)) { + success = false; + } + } + + return success; +} + +bool VirtualCluster::MarkNodeInstanceAsDead(const std::string &template_id, + const std::string &node_instance_id) { + auto iter = node_instances_map_.find(node_instance_id); + if (iter != node_instances_map_.end() && iter->second->template_id() == template_id) { + iter->second->set_is_dead(true); + return true; + } + return false; +} + +bool VirtualCluster::ContainsNodeInstance(const std::string &node_instance_id) { + return node_instances_map_.contains(node_instance_id); +} + +void VirtualCluster::ForeachNodeInstance( + const std::function &)> &fn) const { + if (fn == nullptr) { + return; + } + for (const auto &[template_id, job_node_instances] : visible_node_instances_) { + for (const auto &[job_cluster_id, node_instances] : job_node_instances) { + for (const auto &[_, node_instance] : node_instances) { + fn(node_instance); + } + } + } +} + +bool VirtualCluster::ReplenishUndividedNodeInstances( + const NodeInstanceReplenishCallback &callback) { + RAY_CHECK(callback != nullptr); + bool any_node_instance_replenished = false; + for (auto &[template_id, job_node_instances] : visible_node_instances_) { + auto iter = job_node_instances.find(kUndividedClusterId); + if (iter == job_node_instances.end()) { + continue; + } + + absl::flat_hash_map> + node_instances_to_add; + absl::flat_hash_map> + node_instances_to_remove; + for (auto &[node_instance_id, node_instance] : iter->second) { + if (node_instance->is_dead()) { + if (auto replenished_node_instance = callback(node_instance)) { + RAY_LOG(INFO) << "Replenish node instance " << node_instance->node_instance_id() + << " in virtual cluster " << GetID() << " with node instance " + << replenished_node_instance->node_instance_id(); + node_instances_to_remove.emplace(node_instance_id, node_instance); + node_instances_to_add.emplace(replenished_node_instance->node_instance_id(), + replenished_node_instance); + any_node_instance_replenished = true; + break; + } + } + } + for (auto &[node_instance_id, node_instance] : node_instances_to_remove) { + iter->second.erase(node_instance_id); + node_instances_map_.erase(node_instance_id); + } + for (auto &[node_instance_id, node_instance] : node_instances_to_add) { + iter->second.emplace(node_instance_id, node_instance); + node_instances_map_[node_instance_id] = node_instance; + } + } + + return any_node_instance_replenished; +} + +std::shared_ptr VirtualCluster::ReplenishUndividedNodeInstance( + std::shared_ptr node_instance_to_replenish) { + if (node_instance_to_replenish == nullptr || !node_instance_to_replenish->is_dead()) { + return nullptr; + } + + auto template_iter = + visible_node_instances_.find(node_instance_to_replenish->template_id()); + if (template_iter == visible_node_instances_.end()) { + return nullptr; + } + + auto job_iter = template_iter->second.find(kUndividedClusterId); + if (job_iter == template_iter->second.end()) { + return nullptr; + } + + std::shared_ptr replenished_node_instance; + for (auto &[node_instance_id, node_instance] : job_iter->second) { + if (!node_instance->is_dead()) { + replenished_node_instance = node_instance; + break; + } + } + if (replenished_node_instance != nullptr) { + job_iter->second.erase(replenished_node_instance->node_instance_id()); + job_iter->second.emplace(node_instance_to_replenish->node_instance_id(), + node_instance_to_replenish); + + node_instances_map_.erase(replenished_node_instance->node_instance_id()); + node_instances_map_[node_instance_to_replenish->node_instance_id()] = + node_instance_to_replenish; + } + + return replenished_node_instance; +} + +std::shared_ptr VirtualCluster::ToProto() const { + auto data = std::make_shared(); + data->set_id(GetID()); + data->set_divisible(Divisible()); + data->set_revision(GetRevision()); + for (auto &[template_id, job_node_instances] : visible_node_instances_) { + for (auto &[job_cluster_id, node_instances] : job_node_instances) { + for (auto &[id, node_instance] : node_instances) { + (*data->mutable_node_instances())[id] = std::move(*node_instance->ToProto()); + } + } + } + return data; +} + +std::shared_ptr VirtualCluster::ToView() const { + auto data = std::make_shared(); + data->set_id(GetID()); + data->set_divisible(Divisible()); + data->set_revision(GetRevision()); + for (auto &[template_id, job_node_instances] : visible_node_instances_) { + for (auto &[job_cluster_id, node_instances] : job_node_instances) { + for (auto &[id, node_instance] : node_instances) { + auto node_id = scheduling::NodeID( + NodeID::FromHex(node_instance->node_instance_id()).Binary()); + const auto &node_resources = cluster_resource_manager_.GetNodeResources(node_id); + (*data->mutable_node_instance_views())[id] = + std::move(*node_instance->ToView(node_resources)); + } + } + } + return data; +} + +std::string VirtualCluster::DebugString() const { + int indent = 2; + std::ostringstream stream; + stream << "VirtualCluster[" << GetID() << "]:\n"; + stream << std::string(indent, ' ') << "revision: " << GetRevision() << ",\n"; + stream << std::string(indent, ' ') << "replica_sets: {\n"; + for (const auto &[template_id, replicas] : replica_sets_) { + stream << std::string(indent + 2, ' ') << template_id << ": " << replicas << ",\n"; + } + stream << std::string(indent, ' ') << "}, \n"; + stream << std::string(indent, ' ') << "visible_node_instances: " + << ray::gcs::DebugString(visible_node_instances_, indent); + return stream.str(); +} + +///////////////////////// DivisibleCluster ///////////////////////// +void DivisibleCluster::LoadJobCluster(const std::string &job_cluster_id, + ReplicaInstances replica_instances) { + RAY_CHECK(VirtualClusterID::FromBinary(job_cluster_id).IsJobClusterID()); + RAY_CHECK(job_clusters_.find(job_cluster_id) == job_clusters_.end()); + + DoCreateJobCluster(job_cluster_id, std::move(replica_instances)); +} + +Status DivisibleCluster::CreateJobCluster(const std::string &job_cluster_id, + ReplicaSets replica_sets, + CreateOrUpdateVirtualClusterCallback callback, + ReplicaSets *replica_sets_to_recommend) { + if (!Divisible()) { + std::ostringstream ostr; + ostr << "The job cluster can not be created in indivisible cluster " << GetID(); + return Status::InvalidArgument(ostr.str()); + } + + auto iter = job_clusters_.find(job_cluster_id); + if (iter != job_clusters_.end()) { + std::ostringstream ostr; + ostr << "The job cluster " << job_cluster_id << " already exists."; + return Status::InvalidArgument(ostr.str()); + } + + ReplicaInstances replica_instances_to_add; + // Lookup undivided alive node instances based on `replica_sets_to_add`. + auto success = LookupUndividedNodeInstances( + replica_sets, replica_instances_to_add, [](const auto &node_instance) { + return !node_instance.is_dead(); + }); + if (!success) { + if (replica_sets_to_recommend != nullptr) { + *replica_sets_to_recommend = buildReplicaSets(replica_instances_to_add); + } + std::ostringstream ostr; + ostr << "No enough node instances to create the job cluster " << job_cluster_id; + return Status::OutOfResource(ostr.str()); + } + + auto job_cluster = + DoCreateJobCluster(job_cluster_id, std::move(replica_instances_to_add)); + // Flush and publish the job cluster data. + return async_data_flusher_(job_cluster->ToProto(), std::move(callback)); +} + +std::shared_ptr DivisibleCluster::DoCreateJobCluster( + const std::string &job_cluster_id, ReplicaInstances replica_instances_to_add) { + auto replica_instances_to_remove_from_current_cluster = replica_instances_to_add; + auto replica_instances_to_add_to_current_cluster = replica_instances_to_add; + for (auto &[template_id, job_node_instances] : + replica_instances_to_add_to_current_cluster) { + auto node_instances = std::move(job_node_instances[kUndividedClusterId]); + job_node_instances.erase(kUndividedClusterId); + job_node_instances[job_cluster_id] = std::move(node_instances); + } + UpdateNodeInstances(std::move(replica_instances_to_add_to_current_cluster), + std::move(replica_instances_to_remove_from_current_cluster)); + + // Create a job cluster. + auto job_cluster = + std::make_shared(job_cluster_id, cluster_resource_manager_); + job_cluster->UpdateNodeInstances(std::move(replica_instances_to_add), + ReplicaInstances()); + RAY_CHECK(job_clusters_.emplace(job_cluster_id, job_cluster).second); + return job_cluster; +} + +Status DivisibleCluster::RemoveJobCluster(const std::string &job_cluster_id, + RemoveVirtualClusterCallback callback) { + if (!Divisible()) { + std::ostringstream ostr; + ostr << "The job cluster " << job_cluster_id + << " can not be removed from indivisible cluster " << GetID(); + return Status::InvalidArgument(ostr.str()); + } + + auto iter = job_clusters_.find(job_cluster_id); + if (iter == job_clusters_.end()) { + return Status::NotFound("The job cluster " + job_cluster_id + " does not exist."); + } + auto job_cluster = iter->second; + + const auto &replica_instances_to_remove = job_cluster->GetVisibleNodeInstances(); + + auto replica_instances_to_add_to_current_cluster = replica_instances_to_remove; + auto replica_instances_to_remove_from_current_cluster = replica_instances_to_remove; + for (auto &[template_id, job_node_instances] : + replica_instances_to_remove_from_current_cluster) { + auto node_instances = std::move(job_node_instances[kUndividedClusterId]); + job_node_instances.erase(kUndividedClusterId); + job_node_instances[job_cluster_id] = std::move(node_instances); + } + UpdateNodeInstances(std::move(replica_instances_to_add_to_current_cluster), + std::move(replica_instances_to_remove_from_current_cluster)); + + // Update the job cluster. + // job_cluster->UpdateNodeInstances(ReplicaInstances(), + // std::move(replica_instances_to_remove)); + job_clusters_.erase(iter); + + auto data = job_cluster->ToProto(); + // Mark the data as removed. + data->set_is_removed(true); + // Flush and publish the job cluster data. + return async_data_flusher_(std::move(data), std::move(callback)); +} + +std::shared_ptr DivisibleCluster::GetJobCluster( + const std::string &job_cluster_id) const { + auto iter = job_clusters_.find(job_cluster_id); + return iter != job_clusters_.end() ? iter->second : nullptr; +} + +bool DivisibleCluster::InUse() const { return !job_clusters_.empty(); } + +bool DivisibleCluster::IsUndividedNodeInstanceIdle( + const gcs::NodeInstance &node_instance) const { + auto template_iter = visible_node_instances_.find(node_instance.template_id()); + if (template_iter == visible_node_instances_.end()) { + return false; + } + auto job_iter = template_iter->second.find(kUndividedClusterId); + if (job_iter == template_iter->second.end()) { + return false; + } + return job_iter->second.contains(node_instance.node_instance_id()); +} + +void DivisibleCluster::ForeachJobCluster( + const std::function &)> &fn) const { + if (fn == nullptr) { + return; + } + for (const auto &[_, job_cluster] : job_clusters_) { + fn(job_cluster); + } +} + +bool DivisibleCluster::ReplenishNodeInstances( + const NodeInstanceReplenishCallback &callback) { + RAY_CHECK(callback != nullptr); + bool any_node_instance_replenished = false; + for (const auto &[_, job_cluster] : job_clusters_) { + // Explicitly capture the `job_cluster` by value to avoid the compile error. + // e.g. error: 'job_cluster_id' in capture list does not name a variable + const auto &job_cluster_id = job_cluster->GetID(); + bool replenished = job_cluster->ReplenishNodeInstances( + [this, &job_cluster_id, &any_node_instance_replenished, &callback]( + std::shared_ptr node_instance) { + const auto &template_id = node_instance->template_id(); + if (auto replenished_node_instance = callback(node_instance)) { + RAY_LOG(INFO) << "Replenish node instance " + << node_instance->node_instance_id() << " in virtual cluster " + << job_cluster_id << " with node instance " + << replenished_node_instance->node_instance_id() + << " in primary cluster."; + ReplicaInstances replica_instances_to_add; + replica_instances_to_add[template_id][job_cluster_id] + [replenished_node_instance->node_instance_id()] = + replenished_node_instance; + + ReplicaInstances replica_instances_to_remove; + replica_instances_to_remove[template_id][job_cluster_id] + [node_instance->node_instance_id()] = + node_instance; + + UpdateNodeInstances(std::move(replica_instances_to_add), + std::move(replica_instances_to_remove)); + + any_node_instance_replenished = true; + return replenished_node_instance; + } + + auto replenished_node_instance = ReplenishUndividedNodeInstance(node_instance); + if (replenished_node_instance != nullptr) { + RAY_LOG(INFO) << "Replenish node instance " + << node_instance->node_instance_id() << " in job cluster " + << job_cluster_id << " with node instance " + << replenished_node_instance->node_instance_id() + << " in virtual cluster " << GetID(); + return replenished_node_instance; + } + + return replenished_node_instance; + }); + + if (replenished) { + // Flush and publish the job cluster data. + auto status = async_data_flusher_(job_cluster->ToProto(), nullptr); + if (!status.ok()) { + RAY_LOG(ERROR) << "Failed to flush and publish the job cluster " << job_cluster_id + << " data: " << status.message(); + } + } + } + + if (ReplenishUndividedNodeInstances(callback)) { + any_node_instance_replenished = true; + } + + return any_node_instance_replenished; +} + +///////////////////////// IndivisibleCluster ///////////////////////// +bool IndivisibleCluster::IsUndividedNodeInstanceIdle( + const gcs::NodeInstance &node_instance) const { + if (node_instance.is_dead()) { + return true; + } + auto node_id = + scheduling::NodeID(NodeID::FromHex(node_instance.node_instance_id()).Binary()); + const auto &node_resources = cluster_resource_manager_.GetNodeResources(node_id); + if (node_resources.normal_task_resources.IsEmpty() && + node_resources.total == node_resources.available) { + return true; + } + return false; +} + +bool IndivisibleCluster::InUse() const { + if (!detached_actors_.empty() || !detached_placement_groups_.empty()) { + return true; + } + for (const auto &[template_id, job_cluster_instances] : visible_node_instances_) { + for (const auto &[job_cluster_id, node_instances] : job_cluster_instances) { + for (const auto &[node_instance_id, node_instance] : node_instances) { + if (!IsUndividedNodeInstanceIdle(*node_instance)) { + return true; + } + } + } + } + return false; +} + +bool IndivisibleCluster::ReplenishNodeInstances( + const NodeInstanceReplenishCallback &callback) { + return ReplenishUndividedNodeInstances(callback); +} + +void IndivisibleCluster::OnDetachedActorRegistration(const ActorID &actor_id) { + detached_actors_.insert(actor_id); +} + +void IndivisibleCluster::OnDetachedActorDestroy(const ActorID &actor_id) { + detached_actors_.erase(actor_id); +} + +void IndivisibleCluster::OnDetachedPlacementGroupRegistration( + const PlacementGroupID &placement_group_id) { + detached_placement_groups_.insert(placement_group_id); +} + +void IndivisibleCluster::OnDetachedPlacementGroupDestroy( + const PlacementGroupID &placement_group_id) { + detached_placement_groups_.erase(placement_group_id); +} + +///////////////////////// PrimaryCluster ///////////////////////// +void PrimaryCluster::Initialize(const GcsInitData &gcs_init_data) { + // Let indivisible cluster be Vi, divisible cluster be Vd, job cluster be J, empty job + // cluster be E, then + // + // Nodes(Vd) = Σ(Nodes(J)) + Nodes(E). + // + // When node belongs to Vi is dead (it's a simple case): + // 1. Find a new replica instance from primary cluster to replace the dead one in Vi, + // then flush and publish Vi. + // + // When node belongs to J is dead(It will also be dead in Vd as both Vd and J hold the + // shared node instance). + // 1. Find a new replica instance from E to replace the dead one in J, then flush and + // publish J. + // 2. If there is no enough node instances in E, then find a new replica instance from + // primary cluster to replace the dead one in J, then: + // a. flush and publish J. + // b. flush Vd. + // 3. If there is no enough node instances in primary cluster, then wait for the new + // node to register. + // + // When node belongs to E is dead (it's simple case). + // 1. Find a new replica instance from primary cluster to replace the dead one in E, + // then flush Vd. + // 2. If there is no enough node instances in primary cluster, then wait for the new + // node to register. + // + // When failover happens, we need to load the logical clusters and job clusters from the + // GCS tables, and repair the Vd based on J. + // 1. Find the different node instances between Σ(Nodes(J)) and Nodes(Vd), let the + // difference be D. + // D = Σ(Nodes(J)) - Vd + // 2. Remove the dead node instances from Vd based on the replica sets of D and then + // flush Vd. + const auto &nodes = gcs_init_data.Nodes(); + for (const auto &[_, node] : nodes) { + if (node.state() == rpc::GcsNodeInfo::ALIVE) { + OnNodeAdd(node); + } + } + + absl::flat_hash_map + job_cluster_replica_instances_map; + for (const auto &[virtual_cluster_id, virtual_cluster_data] : + gcs_init_data.VirtualClusters()) { + // Convert the node instances to replica instances and mark the dead node instances. + auto replica_instances = toReplicaInstances(virtual_cluster_data.node_instances()); + for (auto &[_, job_node_instances] : replica_instances) { + for (auto &[_, node_instances] : job_node_instances) { + for (auto &[node_instance_id, node_instance] : node_instances) { + auto node_id = NodeID::FromHex(node_instance_id); + auto it = nodes.find(node_id); + if (it == nodes.end() || it->second.state() == rpc::GcsNodeInfo::DEAD) { + node_instance->set_is_dead(true); + } + } + } + } + // Stash the job clusters and load the logical clusters. + if (virtual_cluster_id.IsJobClusterID()) { + job_cluster_replica_instances_map[virtual_cluster_id] = + std::move(replica_instances); + } else { + // Load the logical cluster. + LoadLogicalCluster(virtual_cluster_data.id(), + virtual_cluster_data.divisible(), + std::move(replica_instances)); + } + } + + // Repair the divisible cluster's node instances to ensure that the node instances in + // the virtual clusters contains all the node instances in the job clusters. Calculate + // the different node instances between Σ(Nodes(J)) and Nodes(Ve), + // D = Σ(Nodes(J)) - Ve + for (auto &[job_cluster_id, replica_instances] : job_cluster_replica_instances_map) { + auto parent_cluster_id = job_cluster_id.ParentID().Binary(); + auto virtual_cluster = GetVirtualCluster(parent_cluster_id); + RAY_CHECK(virtual_cluster != nullptr); + auto replica_instances_to_repair = ReplicaInstancesDifference( + replica_instances, virtual_cluster->GetVisibleNodeInstances()); + if (replica_instances_to_repair.empty()) { + continue; + } + + ReplicaInstances replica_instances_to_remove; + auto replica_sets_to_repair = buildReplicaSets(replica_instances_to_repair); + // Lookup undivided dead node instances best effort. + virtual_cluster->LookupUndividedNodeInstances( + replica_sets_to_repair, + replica_instances_to_remove, + [](const auto &node_instance) { return node_instance.is_dead(); }); + RAY_LOG(INFO) << "Repair the divisible cluster " << parent_cluster_id + << " based on the job cluster " << job_cluster_id + << "\nreplica_sets_to_repair: " + << ray::gcs::DebugString(replica_sets_to_repair) + << "replica_instances_to_repair: " + << ray::gcs::DebugString(replica_instances_to_repair) + << "replica_instances_to_remove: " + << ray::gcs::DebugString(replica_instances_to_remove); + virtual_cluster->UpdateNodeInstances(std::move(replica_instances_to_repair), + std::move(replica_instances_to_remove)); + } + + // Replay the job clusters. + for (auto &[job_cluster_id, replica_instances] : job_cluster_replica_instances_map) { + auto parent_cluster_id = job_cluster_id.ParentID().Binary(); + if (parent_cluster_id == kPrimaryClusterID) { + LoadJobCluster(job_cluster_id.Binary(), std::move(replica_instances)); + } else { + auto logical_cluster = std::dynamic_pointer_cast( + GetLogicalCluster(parent_cluster_id)); + RAY_CHECK(logical_cluster != nullptr && logical_cluster->Divisible()); + logical_cluster->LoadJobCluster(job_cluster_id.Binary(), + std::move(replica_instances)); + } + } +} + +std::shared_ptr PrimaryCluster::GetLogicalCluster( + const std::string &logical_cluster_id) const { + auto iter = logical_clusters_.find(logical_cluster_id); + return iter != logical_clusters_.end() ? iter->second : nullptr; +} + +void PrimaryCluster::ForeachVirtualCluster( + const std::function &)> &fn) const { + if (fn == nullptr) { + return; + } + for (const auto &[_, logical_cluster] : logical_clusters_) { + fn(logical_cluster); + if (logical_cluster->Divisible()) { + auto divisible_cluster = + std::dynamic_pointer_cast(logical_cluster); + divisible_cluster->ForeachJobCluster(fn); + } + } + ForeachJobCluster(fn); +} + +std::shared_ptr PrimaryCluster::LoadLogicalCluster( + const std::string &virtual_cluster_id, + bool divisible, + ReplicaInstances replica_instances) { + std::shared_ptr logical_cluster; + if (divisible) { + logical_cluster = std::make_shared( + virtual_cluster_id, async_data_flusher_, cluster_resource_manager_); + } else { + logical_cluster = std::make_shared(virtual_cluster_id, + cluster_resource_manager_); + } + RAY_CHECK(logical_clusters_.emplace(virtual_cluster_id, logical_cluster).second); + + auto replica_instances_to_remove_from_primary_cluster = replica_instances; + UpdateNodeInstances(ReplicaInstances(), + std::move(replica_instances_to_remove_from_primary_cluster)); + + // Update the virtual cluster replica sets and node instances. + logical_cluster->UpdateNodeInstances(std::move(replica_instances), ReplicaInstances()); + return logical_cluster; +} + +Status PrimaryCluster::CreateOrUpdateVirtualCluster( + rpc::CreateOrUpdateVirtualClusterRequest request, + CreateOrUpdateVirtualClusterCallback callback, + ReplicaSets *replica_sets_to_recommend) { + // Calculate the node instances that to be added and to be removed. + ReplicaInstances replica_instances_to_add_to_logical_cluster; + ReplicaInstances replica_instances_to_remove_from_logical_cluster; + auto status = DetermineNodeInstanceAdditionsAndRemovals( + request, + replica_instances_to_add_to_logical_cluster, + replica_instances_to_remove_from_logical_cluster); + if (!status.ok()) { + // Calculate the replica sets that we can fulfill the + // request at most. It can be used as a suggestion to adjust the request if it fails. + if (replica_sets_to_recommend) { + if (status.IsOutOfResource()) { + *replica_sets_to_recommend = + buildReplicaSets(replica_instances_to_add_to_logical_cluster); + } else if (status.IsUnsafeToRemove()) { + *replica_sets_to_recommend = + buildReplicaSets(replica_instances_to_remove_from_logical_cluster); + } + } + return status; + } + + auto logical_cluster = GetLogicalCluster(request.virtual_cluster_id()); + if (logical_cluster == nullptr) { + // replica_instances_to_remove must be empty as the virtual cluster is a new one. + RAY_CHECK(replica_instances_to_remove_from_logical_cluster.empty()); + if (request.divisible()) { + logical_cluster = std::make_shared( + request.virtual_cluster_id(), async_data_flusher_, cluster_resource_manager_); + } else { + logical_cluster = std::make_shared(request.virtual_cluster_id(), + cluster_resource_manager_); + } + logical_clusters_[request.virtual_cluster_id()] = logical_cluster; + } + + // Update the main cluster replica sets and node instances. + // NOTE: The main cluster unnecessary to flush and pub data to other nodes. + auto replica_instances_to_add_to_primary_cluster = + replica_instances_to_remove_from_logical_cluster; + auto replica_instances_to_remove_from_primary_cluster = + replica_instances_to_add_to_logical_cluster; + UpdateNodeInstances(std::move(replica_instances_to_add_to_primary_cluster), + std::move(replica_instances_to_remove_from_primary_cluster)); + + // Update the virtual cluster replica sets and node instances. + logical_cluster->UpdateNodeInstances( + std::move(replica_instances_to_add_to_logical_cluster), + std::move(replica_instances_to_remove_from_logical_cluster)); + return async_data_flusher_(logical_cluster->ToProto(), std::move(callback)); +} + +Status PrimaryCluster::DetermineNodeInstanceAdditionsAndRemovals( + const rpc::CreateOrUpdateVirtualClusterRequest &request, + ReplicaInstances &replica_instances_to_add, + ReplicaInstances &replica_instances_to_remove) { + replica_instances_to_add.clear(); + replica_instances_to_remove.clear(); + + const auto &logical_cluster_id = request.virtual_cluster_id(); + auto logical_cluster = GetLogicalCluster(logical_cluster_id); + if (logical_cluster != nullptr) { + auto replica_sets_to_remove = + ReplicasDifference(logical_cluster->GetReplicaSets(), request.replica_sets()); + // Lookup idle node instances from the logical cluster based on + // `replica_sets_to_remove`. + // TODO(Shanly): Iterate the dead node instances in advance. + auto success = logical_cluster->LookupUndividedNodeInstances( + replica_sets_to_remove, + replica_instances_to_remove, + [logical_cluster](const auto &node_instance) { + if (logical_cluster->Divisible()) { + return true; + } + return logical_cluster->IsUndividedNodeInstanceIdle(node_instance); + }); + if (!success) { + return Status::UnsafeToRemove( + "No enough nodes to remove from the virtual cluster. The replica sets that gcs " + "can remove " + "at most are shown below. Use it as a suggestion to " + "adjust your request or cluster."); + } + } + + auto replica_sets_to_add = ReplicasDifference( + request.replica_sets(), + logical_cluster ? logical_cluster->GetReplicaSets() : ReplicaSets()); + // Lookup undivided alive node instances from main cluster based on + // `replica_sets_to_add`. + auto success = LookupUndividedNodeInstances( + replica_sets_to_add, replica_instances_to_add, [](const auto &node_instance) { + return !node_instance.is_dead(); + }); + if (!success) { + return Status::OutOfResource( + "No enough nodes to add to the virtual cluster. The replica sets that gcs can " + "add " + "at most are shown below. Use it as a suggestion to " + "adjust your request or cluster."); + } + return Status::OK(); +} + +void PrimaryCluster::OnNodeAdd(const rpc::GcsNodeInfo &node) { + const auto &template_id = node.node_type_name(); + auto node_instance_id = NodeID::FromBinary(node.node_id()).Hex(); + auto node_instance = std::make_shared(node_instance_id); + node_instance->set_template_id(template_id); + node_instance->set_hostname(node.node_manager_hostname()); + node_instance->set_is_dead(false); + + InsertNodeInstances( + {{template_id, + {{kUndividedClusterId, {{node_instance_id, std::move(node_instance)}}}}}}); +} + +void PrimaryCluster::OnNodeDead(const rpc::GcsNodeInfo &node) { + const auto &node_type_name = node.node_type_name(); + auto node_instance_id = NodeID::FromBinary(node.node_id()).Hex(); + OnNodeInstanceDead(node_instance_id, node_type_name); +} + +void PrimaryCluster::OnNodeInstanceDead(const std::string &node_instance_id, + const std::string &node_type_name) { + if (MarkNodeInstanceAsDead(node_type_name, node_instance_id)) { + return; + } + + for (const auto &[_, logical_cluster] : logical_clusters_) { + if (logical_cluster->MarkNodeInstanceAsDead(node_type_name, node_instance_id)) { + return; + } + } +} + +Status PrimaryCluster::RemoveVirtualCluster(const std::string &virtual_cluster_id, + RemoveVirtualClusterCallback callback) { + auto cluster_id = VirtualClusterID::FromBinary(virtual_cluster_id); + if (cluster_id.IsJobClusterID()) { + auto parent_cluster_id = cluster_id.ParentID().Binary(); + auto parent_cluster = GetVirtualCluster(parent_cluster_id); + if (parent_cluster == nullptr) { + std::ostringstream ostr; + ostr << "Failed to remove virtual cluster, parent cluster not exists, virtual " + "cluster id: " + << virtual_cluster_id; + auto message = ostr.str(); + return Status::NotFound(message); + } + if (!parent_cluster->Divisible()) { + std::ostringstream ostr; + ostr << "Failed to remove virtual cluster, parent cluster is not divisible, " + "virtual cluster id: " + << virtual_cluster_id; + auto message = ostr.str(); + return Status::InvalidArgument(message); + } + auto virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster != nullptr) { + JobCluster *job_cluster = dynamic_cast(virtual_cluster.get()); + if (job_cluster->InUse()) { + std::ostringstream ostr; + ostr << "Failed to remove virtual cluster, job cluster is in use, " + << "virtual cluster id: " << virtual_cluster_id; + auto message = ostr.str(); + return Status::Invalid(message); + } + } + DivisibleCluster *divisible_cluster = + dynamic_cast(parent_cluster.get()); + return divisible_cluster->RemoveJobCluster(virtual_cluster_id, callback); + } else { + return RemoveLogicalCluster(virtual_cluster_id, callback); + } +} + +Status PrimaryCluster::RemoveLogicalCluster(const std::string &logical_cluster_id, + RemoveVirtualClusterCallback callback) { + auto logical_cluster = GetLogicalCluster(logical_cluster_id); + if (logical_cluster == nullptr) { + return Status::NotFound("The logical cluster " + logical_cluster_id + + " does not exist."); + } + + // Check if the virtual cluster is in use. + ReplicaInstances in_use_instances; + if (logical_cluster->InUse()) { + std::ostringstream ostr; + ostr << "The virtual cluster " << logical_cluster_id + << " can not be removed as it is still in use. "; + auto message = ostr.str(); + RAY_LOG(ERROR) << message; + + return Status::UnsafeToRemove(message); + } + + const auto &replica_instances_to_remove = logical_cluster->GetVisibleNodeInstances(); + + auto replica_instances_to_add_to_primary_cluster = replica_instances_to_remove; + UpdateNodeInstances(std::move(replica_instances_to_add_to_primary_cluster), + ReplicaInstances()); + + // Update the logical_cluster cluster. + // logical_cluster->UpdateNodeInstances(ReplicaInstances(), + // std::move(replica_instances_to_remove)); + logical_clusters_.erase(logical_cluster_id); + + auto data = logical_cluster->ToProto(); + data->set_is_removed(true); + return async_data_flusher_(std::move(data), std::move(callback)); +} + +std::shared_ptr PrimaryCluster::GetVirtualCluster( + const std::string &virtual_cluster_id) { + if (virtual_cluster_id == kPrimaryClusterID) { + return shared_from_this(); + } + + // Check if it is a logical cluster + auto logical_cluster = GetLogicalCluster(virtual_cluster_id); + if (logical_cluster != nullptr) { + return logical_cluster; + } + // Check if it is a job cluster + auto job_cluster = GetJobCluster(virtual_cluster_id); + if (job_cluster != nullptr) { + return job_cluster; + } + // Check if it is a job cluster of any logical cluster + for (auto &[cluster_id, logical_cluster] : logical_clusters_) { + if (logical_cluster->Divisible()) { + DivisibleCluster *divisible_cluster = + dynamic_cast(logical_cluster.get()); + auto job_cluster = divisible_cluster->GetJobCluster(virtual_cluster_id); + if (job_cluster != nullptr) { + return job_cluster; + } + } + } + return nullptr; +} + +void PrimaryCluster::ForeachVirtualClustersData( + rpc::GetVirtualClustersRequest request, VirtualClustersDataVisitCallback callback) { + std::vector> virtual_cluster_data_list; + auto virtual_cluster_id = request.virtual_cluster_id(); + bool include_job_clusters = request.include_job_clusters(); + bool only_include_indivisible_cluster = request.only_include_indivisible_clusters(); + + auto visit_proto_data = [&](const VirtualCluster *cluster) { + if (include_job_clusters && cluster->Divisible()) { + auto divisible_cluster = dynamic_cast(cluster); + divisible_cluster->ForeachJobCluster( + [&](const auto &job_cluster) { callback(job_cluster->ToProto()); }); + } + if (only_include_indivisible_cluster && cluster->Divisible()) { + return; + } + if (cluster->GetID() != kPrimaryClusterID) { + // Skip the primary cluster's proto data. + callback(cluster->ToProto()); + } + }; + + if (virtual_cluster_id.empty()) { + // Get all virtual clusters data. + for (const auto &[_, logical_cluster] : logical_clusters_) { + visit_proto_data(logical_cluster.get()); + } + visit_proto_data(this); + return; + } + + if (virtual_cluster_id == kPrimaryClusterID) { + visit_proto_data(this); + return; + } + + auto logical_cluster = GetLogicalCluster(virtual_cluster_id); + if (logical_cluster != nullptr) { + visit_proto_data(logical_cluster.get()); + } +} + +void PrimaryCluster::ForeachVirtualClustersView( + rpc::GetAllVirtualClusterInfoRequest request, + VirtualClustersViewVisitCallback callback) const { + std::vector> virtual_cluster_view_list; + + auto visit_view_data = [&](const VirtualCluster *cluster) { + if (cluster->Divisible()) { + auto divisible_cluster = dynamic_cast(cluster); + divisible_cluster->ForeachJobCluster( + [&](const auto &job_cluster) { callback(job_cluster->ToView()); }); + } + callback(cluster->ToView()); + }; + + // Get all virtual clusters data. + for (const auto &[_, logical_cluster] : logical_clusters_) { + visit_view_data(logical_cluster.get()); + } + visit_view_data(this); +} + +void PrimaryCluster::ReplenishAllClusterNodeInstances() { + auto node_instance_replenish_callback = [this](auto node_instance) { + return ReplenishUndividedNodeInstance(std::move(node_instance)); + }; + + for (auto &[_, logical_cluster] : logical_clusters_) { + if (logical_cluster->ReplenishNodeInstances(node_instance_replenish_callback)) { + // Flush the logical cluster data. + auto status = async_data_flusher_(logical_cluster->ToProto(), nullptr); + if (!status.ok()) { + RAY_LOG(ERROR) << "Failed to flush logical cluster " << logical_cluster->GetID() + << " data, status: " << status.ToString(); + } + } + } + + if (ReplenishNodeInstances(node_instance_replenish_callback)) { + // Primary cluster proto data need not to flush. + } +} + +void PrimaryCluster::GCExpiredJobClusters() { + RAY_LOG(INFO) << "Start GC expired job clusters..."; + size_t total_expired_job_clusters = 0; + auto gc_job_cluster = [&total_expired_job_clusters]( + const std::shared_ptr &divisible_cluster) { + std::vector expired_job_clusters; + divisible_cluster->ForeachJobCluster( + [&expired_job_clusters, &total_expired_job_clusters](auto &job_cluster) { + if (!job_cluster->InUse() && job_cluster->IsJobDead()) { + expired_job_clusters.emplace_back(job_cluster->GetID()); + total_expired_job_clusters++; + } + }); + for (auto &job_cluster_id : expired_job_clusters) { + auto status = divisible_cluster->RemoveJobCluster(job_cluster_id, nullptr); + if (!status.ok()) { + RAY_LOG(ERROR) << "Failed to remove job cluster when gc expired job cluster: " + << job_cluster_id << ", error: " << status.message(); + } + } + }; + + gc_job_cluster(std::dynamic_pointer_cast(shared_from_this())); + + for (auto &[_, logical_cluster] : logical_clusters_) { + if (logical_cluster->Divisible()) { + gc_job_cluster(std::dynamic_pointer_cast(logical_cluster)); + } + } + RAY_LOG(INFO) << "Finished GC expired job clusters, total expired job clusters: " + << total_expired_job_clusters; +} + +} // namespace gcs +} // namespace ray \ No newline at end of file diff --git a/src/ray/gcs/gcs_server/gcs_virtual_cluster.h b/src/ray/gcs/gcs_server/gcs_virtual_cluster.h new file mode 100644 index 000000000000..6fac202f2b09 --- /dev/null +++ b/src/ray/gcs/gcs_server/gcs_virtual_cluster.h @@ -0,0 +1,575 @@ +// Copyright 2017 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 "absl/container/flat_hash_map.h" +#include "absl/container/flat_hash_set.h" +#include "ray/common/status.h" +#include "ray/common/virtual_cluster_id.h" +#include "ray/gcs/gcs_server/gcs_init_data.h" +#include "src/ray/protobuf/gcs.pb.h" +#include "src/ray/protobuf/gcs_service.pb.h" +#include "src/ray/raylet/scheduling/cluster_resource_manager.h" + +namespace ray { +namespace gcs { + +struct NodeInstance { + NodeInstance(const std::string &node_instance_id) { + node_instance_id_ = node_instance_id; + } + + const std::string &node_instance_id() const { return node_instance_id_; } + + const std::string &hostname() const { return hostname_; } + void set_hostname(const std::string &hostname) { hostname_ = hostname; } + + const std::string &template_id() const { return template_id_; } + void set_template_id(const std::string &template_id) { template_id_ = template_id; } + + bool is_dead() const { return is_dead_; } + void set_is_dead(bool is_dead) { is_dead_ = is_dead; } + + std::shared_ptr ToProto() const { + auto node_instance = std::make_shared(); + node_instance->set_template_id(template_id_); + node_instance->set_hostname(hostname_); + return node_instance; + } + + std::shared_ptr ToView( + const NodeResources &node_resources) const { + auto node_view = std::make_shared(); + node_view->set_template_id(template_id_); + node_view->set_hostname(hostname_); + node_view->set_is_dead(is_dead_); + auto resources_total = node_view->mutable_resources_total(); + resources_total->insert(node_resources.total.GetResourceMap().begin(), + node_resources.total.GetResourceMap().end()); + auto resources_available = node_view->mutable_resources_available(); + resources_available->insert(node_resources.available.GetResourceMap().begin(), + node_resources.available.GetResourceMap().end()); + return node_view; + } + + std::string DebugString() const { + std::ostringstream stream; + stream << "NodeInstance(" << node_instance_id_ << "," << hostname_ << "," + << template_id_ << ", " << is_dead_ << ")"; + return stream.str(); + } + + private: + std::string node_instance_id_; + std::string hostname_; + std::string template_id_; + bool is_dead_ = false; +}; + +static const std::string kUndividedClusterId = "NIL"; + +/// +/// | +/// |--> +/// | +/// |--> > +using ReplicaInstances = absl::flat_hash_map< + std::string, + absl::flat_hash_map< + std::string, + absl::flat_hash_map>>>; + +using ReplicaSets = absl::flat_hash_map; + +using CreateOrUpdateVirtualClusterCallback = std::function, const ReplicaSets *)>; + +using RemoveVirtualClusterCallback = CreateOrUpdateVirtualClusterCallback; + +using VirtualClustersDataVisitCallback = + std::function)>; + +using VirtualClustersViewVisitCallback = + std::function)>; + +using AsyncClusterDataFlusher = std::function, CreateOrUpdateVirtualClusterCallback)>; + +using NodeInstanceFilter = std::function; + +/// Calculate the difference between two replica sets. +template +ReplicaSets ReplicasDifference(const T1 &left, const T2 &right) { + ReplicaSets result; + for (const auto &[template_id, replicas] : left) { + auto right_iter = right.find(template_id); + if (right_iter == right.end()) { + result[template_id] = replicas; + } else { + if (right_iter->second < replicas) { + result[template_id] = replicas - right_iter->second; + } + } + } + return result; +} + +template +ReplicaInstances toReplicaInstances(const T &node_instances) { + ReplicaInstances result; + for (const auto &[id, node_instance] : node_instances) { + auto inst = std::make_shared(id); + inst->set_hostname(node_instance.hostname()); + inst->set_template_id(node_instance.template_id()); + result[node_instance.template_id()][kUndividedClusterId].emplace(id, std::move(inst)); + } + return result; +} + +ReplicaSets buildReplicaSets(const ReplicaInstances &replica_instances); + +std::string DebugString(const ReplicaInstances &replica_instances, int indent = 0); +std::string DebugString(const ReplicaSets &replica_sets, int indent = 0); + +using NodeInstanceReplenishCallback = std::function( + std::shared_ptr node_instance_to_replenish)>; + +class VirtualCluster { + public: + VirtualCluster(const std::string &id, + const ClusterResourceManager &cluster_resource_manager) + : id_(id), cluster_resource_manager_(cluster_resource_manager) {} + virtual ~VirtualCluster() = default; + + /// Get the id of the cluster. + virtual const std::string &GetID() const = 0; + + /// Whether the virtual cluster can splite into one or more child virtual clusters or + /// not. + virtual bool Divisible() const = 0; + + /// Get the revision number of the cluster. + uint64_t GetRevision() const { return revision_; } + + /// Get the replica sets corresponding to the cluster. + const ReplicaSets &GetReplicaSets() const { return replica_sets_; } + + /// Get the visible node instances of the cluster. + const ReplicaInstances &GetVisibleNodeInstances() const { + return visible_node_instances_; + } + + /// Update the node instances of the cluster. + /// + /// \param replica_instances_to_add The node instances to be added. + /// \param replica_instances_to_remove The node instances to be removed. + void UpdateNodeInstances(ReplicaInstances replica_instances_to_add, + ReplicaInstances replica_instances_to_remove); + + /// Lookup undivided node instances from `visible_node_instances_` + /// based on the desired final replica sets and node_instance_filter. + /// + /// \param replica_sets The demand final replica sets. + /// \param replica_instances The node instances lookuped best effort from the visible + /// node instances. + /// \param node_instance_filter The filter to check if the node instance is desired. + /// \return True if the lookup is successful, otherwise return an error. + bool LookupUndividedNodeInstances(const ReplicaSets &replica_sets, + ReplicaInstances &replica_instances, + NodeInstanceFilter node_instance_filter) const; + + /// Mark the node instance as dead. + /// + /// \param node_instance_id The id of the node instance to be marked as dead. + /// \return True if the node instance is marked as dead, false otherwise. + bool MarkNodeInstanceAsDead(const std::string &template_id, + const std::string &node_instance_id); + + /// Check the virtual cluster contains node instance + /// + /// \param node_instance_id The id of the node instance to check + /// \return True if the node instance is in this virtual cluster, false otherwise. + bool ContainsNodeInstance(const std::string &node_instance_id); + + /// Iterate all node instances of the virtual cluster. + /// + /// \param fn The callback to iterate all node instances. + void ForeachNodeInstance( + const std::function &)> &fn) const; + + /// Check if the virtual cluster is in use. + /// + /// \param in_use_instances The node instances that are still in use. + /// \return True if the virtual cluster is in use, false otherwise. + virtual bool InUse() const = 0; + + /// Convert the virtual cluster to proto data which usually is used for flushing + /// to redis or publishing to raylet. + /// \return A shared pointer to the proto data. + std::shared_ptr ToProto() const; + + /// Convert the virtual cluster to view which usually is used for displaying + /// to client + /// \return A shared pointer to the view. + std::shared_ptr ToView() const; + + /// Get the debug string of the virtual cluster. + virtual std::string DebugString() const; + + /// Check if the node instance is idle. If a node instance is idle, it can be + /// removed from the virtual cluster safely. + /// + /// \param node_instance The node instance to be checked. + /// \return True if the node instance is idle, false otherwise. + virtual bool IsUndividedNodeInstanceIdle( + const gcs::NodeInstance &node_instance) const = 0; + + /// Replenish the node instances of the virtual cluster. + /// + /// \param callback The callback to replenish the dead node instances. + /// \return True if any dead node instances are replenished, false otherwise. + virtual bool ReplenishNodeInstances(const NodeInstanceReplenishCallback &callback) = 0; + + /// Replenish the undivided node instances of the virtual cluster. + /// + /// \param callback The callback to replenish the dead node instances. + /// \return True if any dead node instances are replenished, false otherwise. + bool ReplenishUndividedNodeInstances(const NodeInstanceReplenishCallback &callback); + + /// Replenish a node instance. + /// + /// \param node_instance_to_replenish The node instance to replenish. + /// \return True if the node instances is replenished, false otherwise. + std::shared_ptr ReplenishUndividedNodeInstance( + std::shared_ptr node_instance_to_replenish); + + protected: + /// Insert the node instances to the cluster. + /// + /// \param replica_instances The node instances to be inserted. + void InsertNodeInstances(ReplicaInstances replica_instances); + + /// Remove the node instances from the cluster. + /// + /// \param replica_instances The node instances to be removed. + void RemoveNodeInstances(ReplicaInstances replica_instances); + + /// The id of the virtual cluster. + std::string id_; + /// Node instances that are visible to the cluster. + ReplicaInstances visible_node_instances_; + /// Replica sets to express the visible node instances. + ReplicaSets replica_sets_; + /// Version number of the last modification to the cluster. + uint64_t revision_{0}; + /// The mapping from node instance id to `NodeInstance` instance. + /// `node_instances_map_` and `visible_node_instances_` are two views of the same node + /// instance. + absl::flat_hash_map> node_instances_map_; + const ClusterResourceManager &cluster_resource_manager_; +}; + +class JobCluster; +class DivisibleCluster : public VirtualCluster { + public: + DivisibleCluster(const std::string &id, + const AsyncClusterDataFlusher &async_data_flusher, + const ClusterResourceManager &cluster_resource_manager) + : VirtualCluster(id, cluster_resource_manager), + async_data_flusher_(async_data_flusher) {} + + const std::string &GetID() const override { return id_; } + bool Divisible() const override { return true; } + + /// Load a job cluster to the divisible cluster. + /// + /// \param job_cluster_id The id of the job cluster. + /// \param replica_instances The replica instances of the job cluster. + void LoadJobCluster(const std::string &job_cluster_id, + ReplicaInstances replica_instances); + + /// Build the job cluster id. + /// + /// \param job_name The name of the job. + /// \return The job cluster id. + std::string BuildJobClusterID(const std::string &job_name) { + return VirtualClusterID::FromBinary(GetID()).BuildJobClusterID(job_name).Binary(); + } + + /// Create a job cluster. + /// + /// \param job_cluster_id The id of the job cluster. + /// \param replica_sets The replica sets of the job cluster. + /// \param[out] replica_sets_to_recommend The replica sets that we can fulfill the + /// request at most. It can be used as a suggestion to adjust the request if it fails. + /// \return Status The status of the creation. + Status CreateJobCluster(const std::string &job_cluster_id, + ReplicaSets replica_sets, + CreateOrUpdateVirtualClusterCallback callback, + ReplicaSets *replica_sets_to_recommend = nullptr); + + /// Remove a job cluster. + /// + /// \param job_cluster_id The id of the job cluster to be removed. + /// \param callback The callback that will be called after the job cluster is removed. + /// \return Status The status of the removal. + Status RemoveJobCluster(const std::string &job_cluster_id, + RemoveVirtualClusterCallback callback); + + /// Get the job cluster by the job cluster id. + /// + /// \param job_cluster_id The id of the job cluster. + /// \return The job cluster if it exists, otherwise return nullptr. + std::shared_ptr GetJobCluster(const std::string &job_cluster_id) const; + + /// Iterate all job clusters. + void ForeachJobCluster( + const std::function &)> &fn) const; + + /// Check if the virtual cluster is in use. + /// + /// \return True if the virtual cluster is in use, false otherwise. + bool InUse() const override; + + /// Check if the node instance is idle. If a node instance is idle, it can be + /// removed from the virtual cluster safely. + /// + /// \param node_instance The node instance to be checked. + /// \return True if the node instance is idle, false otherwise. + bool IsUndividedNodeInstanceIdle(const gcs::NodeInstance &node_instance) const override; + + /// Replenish the node instances of the virtual cluster. + /// + /// \param callback The callback to replenish the dead node instances. + /// \return True if any dead node instances are replenished, false otherwise. + bool ReplenishNodeInstances(const NodeInstanceReplenishCallback &callback) override; + + protected: + /// Do create a job cluster from the divisible cluster. + /// + /// \param job_cluster_id The id of the job cluster. + /// \param replica_instances_to_add The node instances to be added. + /// \return The created job cluster. + std::shared_ptr DoCreateJobCluster( + const std::string &job_cluster_id, ReplicaInstances replica_instances_to_add); + + // The mapping from job cluster id to `JobCluster` instance. + absl::flat_hash_map> job_clusters_; + // The async data flusher. + AsyncClusterDataFlusher async_data_flusher_; +}; + +class IndivisibleCluster : public VirtualCluster { + public: + IndivisibleCluster(const std::string &id, + const ClusterResourceManager &cluster_resource_manager) + : VirtualCluster(id, cluster_resource_manager) {} + IndivisibleCluster &operator=(const IndivisibleCluster &) = delete; + + const std::string &GetID() const override { return id_; } + bool Divisible() const override { return false; } + + /// Check if the virtual cluster is in use. + /// + /// \return True if the virtual cluster is in use, false otherwise. + bool InUse() const override; + + /// Check if the node instance is idle. If a node instance is idle, it can be + /// removed from the virtual cluster safely. + /// + /// \param node_instance The node instance to be checked. + /// \return True if the node instance is idle, false otherwise. + bool IsUndividedNodeInstanceIdle(const gcs::NodeInstance &node_instance) const override; + + /// Replenish the node instances of the virtual cluster. + /// + /// \param callback The callback to replenish the dead node instances. + /// \return True if any dead node instances are replenished, false otherwise. + bool ReplenishNodeInstances(const NodeInstanceReplenishCallback &callback) override; + + /// Handle detached actor registration. + void OnDetachedActorRegistration(const ActorID &actor_id); + + /// Handle detached actor destroy. + void OnDetachedActorDestroy(const ActorID &actor_id); + + /// Handle detached placement group registration. + void OnDetachedPlacementGroupRegistration(const PlacementGroupID &placement_group_id); + + /// Handle detached placement group destroy. + void OnDetachedPlacementGroupDestroy(const PlacementGroupID &placement_group_id); + + private: + // The references of detached actors + absl::flat_hash_set detached_actors_; + // The references of detached placement groups + absl::flat_hash_set detached_placement_groups_; +}; + +class JobCluster : public IndivisibleCluster { + public: + using IndivisibleCluster::IndivisibleCluster; + + /// Set Job as dead + void SetJobDead() { job_dead = true; } + + /// Check if job is dead + /// + /// \return True if the job is dead, false otherwise. + bool IsJobDead() const { return job_dead; } + + private: + // If the job is dead + bool job_dead = false; +}; + +class PrimaryCluster : public DivisibleCluster, + public std::enable_shared_from_this { + public: + PrimaryCluster(const AsyncClusterDataFlusher &async_data_flusher, + const ClusterResourceManager &cluster_resource_manager) + : DivisibleCluster( + kPrimaryClusterID, async_data_flusher, cluster_resource_manager) {} + PrimaryCluster &operator=(const PrimaryCluster &) = delete; + + /// Initialize with the gcs tables data synchronously. + /// This should be called when GCS server restarts after a failure. + /// + /// \param gcs_init_data. + void Initialize(const GcsInitData &gcs_init_data); + + /// Load a logical cluster to the primary cluster. + /// + /// \param virtual_cluster_id The id of the logical cluster. + /// \param divisible Whether the logical cluster is divisible or not. + /// \param replica_instances The replica instances of the logical cluster. + /// \return The loaded logical cluster. + std::shared_ptr LoadLogicalCluster( + const std::string &virtual_cluster_id, + bool divisible, + ReplicaInstances replica_instances); + + const std::string &GetID() const override { return kPrimaryClusterID; } + bool Divisible() const override { return true; } + + /// Create or update a new virtual cluster. + /// + /// \param request The request to create or update a virtual cluster. + /// \param callback The callback that will be called after the virtual cluster + /// is flushed. + /// \param[out] replica_sets_to_recommend The replica sets that we can fulfill the + /// request at most. It can be used as a suggestion to adjust the request if it fails. + /// \return Status. + Status CreateOrUpdateVirtualCluster(rpc::CreateOrUpdateVirtualClusterRequest request, + CreateOrUpdateVirtualClusterCallback callback, + ReplicaSets *replica_sets_to_recommend = nullptr); + + /// Get the virtual cluster by the logical cluster id. + /// + /// \param logical_cluster_id The id of the virtual cluster. + /// \return The logical cluster if it exists, otherwise return nullptr. + std::shared_ptr GetLogicalCluster( + const std::string &logical_cluster_id) const; + + /// Remove logical cluster by the logical cluster id. + /// + /// \param logical_cluster_id The id of the logical cluster to be removed. + /// \param callback The callback that will be called after the logical cluster is + /// removed. + /// \return Status The status of the removal. + Status RemoveLogicalCluster(const std::string &logical_cluster_id, + RemoveVirtualClusterCallback callback); + + /// Remove virtual cluster by the virtual cluster id. + /// + /// \param virtual_cluster_id The id of the virtual cluster to be removed. + /// \param callback The callback that will be called after the virtual cluster is + /// removed. + /// \return Status The status of the removal. + Status RemoveVirtualCluster(const std::string &virtual_cluster_id, + RemoveVirtualClusterCallback callback); + + /// Get virtual cluster by virtual cluster id + /// + /// \param virtual_cluster_id The id of virtual cluster + /// \return the virtual cluster + std::shared_ptr GetVirtualCluster( + const std::string &virtual_cluster_id); + + /// Iterate all virtual clusters. + /// + /// \param fn The function to be called for each logical cluster. + void ForeachVirtualCluster( + const std::function &)> &fn) const; + + /// Iterate virtual clusters data matching the request. + /// + /// \param request The request to get the virtual clusters data. + /// \param callback The callback to visit each virtual cluster data. + void ForeachVirtualClustersData(rpc::GetVirtualClustersRequest request, + VirtualClustersDataVisitCallback callback); + + /// Iterate virtual clusters view matching the request. + /// + /// \param request The request to get the virtual clusters view. + /// \param callback The callback to visit each virtual cluster view. + void ForeachVirtualClustersView(rpc::GetAllVirtualClusterInfoRequest request, + VirtualClustersViewVisitCallback callback) const; + + /// Handle the node added event. + /// + /// \param node The node that is added. + void OnNodeAdd(const rpc::GcsNodeInfo &node); + + /// Handle the node dead event. + /// + /// \param node The node that is dead. + void OnNodeDead(const rpc::GcsNodeInfo &node); + + /// Replenish dead node instances of all the virtual clusters. + void ReplenishAllClusterNodeInstances(); + + /// Garbage collect expired job clusters. + void GCExpiredJobClusters(); + + protected: + /// Handle the node dead event. + /// + /// \param node_instance_id The id of the node instance that is dead. + /// \param node_type_name The type name of the node instance that is dead. + void OnNodeInstanceDead(const std::string &node_instance_id, + const std::string &node_type_name); + + private: + /// Calculate the node instances that to be added and to be removed + /// based on the demand final replica sets inside the request. + /// + /// \param request The request to create or update virtual cluster. + /// \param node_instances_to_add The node instances that to be added. + /// \param node_instances_to_remove The node instances that to be removed. + /// \return status The status of the calculation. + Status DetermineNodeInstanceAdditionsAndRemovals( + const rpc::CreateOrUpdateVirtualClusterRequest &request, + ReplicaInstances &node_instances_to_add, + ReplicaInstances &node_instances_to_remove); + + /// The map of virtual clusters. + /// Mapping from virtual cluster id to the virtual cluster. + absl::flat_hash_map> logical_clusters_; +}; + +} // namespace gcs +} // namespace ray \ No newline at end of file diff --git a/src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.cc b/src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.cc new file mode 100644 index 000000000000..bb5693e26878 --- /dev/null +++ b/src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.cc @@ -0,0 +1,556 @@ +// Copyright 2017 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/gcs/gcs_server/gcs_virtual_cluster_manager.h" + +#include "gcs_actor_manager.h" +#include "gcs_virtual_cluster_manager.h" + +namespace ray { +namespace gcs { + +void GcsVirtualClusterManager::Initialize(const GcsInitData &gcs_init_data) { + primary_cluster_->Initialize(gcs_init_data); + if (periodical_runner_ != nullptr) { + // Periodically check and replenish all the dead node instances of all the virtual + // clusters. + periodical_runner_->RunFnPeriodically( + [this]() { primary_cluster_->ReplenishAllClusterNodeInstances(); }, + RayConfig::instance().node_instances_replenish_interval_ms(), + "ReplenishNodeInstances"); + + // Periodically check and GC all the expired job clusters of all the virtual + // clusters. + periodical_runner_->RunFnPeriodically( + [this]() { primary_cluster_->GCExpiredJobClusters(); }, + RayConfig::instance().expired_job_clusters_gc_interval_ms(), + "GCExpiredJobClusters"); + } + const auto &actor_task_specs = gcs_init_data.ActorTaskSpecs(); + for (const auto &[actor_id, actor_table_data] : gcs_init_data.Actors()) { + if (OnInitializeActorShouldLoad(gcs_init_data, actor_id)) { + if (actor_table_data.is_detached()) { + // If it's a detached actor, we need to handle registration event on FO + // by gcs virtual cluster manager. + const auto &actor_task_spec = map_find_or_die(actor_task_specs, actor_id); + OnDetachedActorRegistration( + actor_task_spec.scheduling_strategy().virtual_cluster_id(), actor_id); + } + } + } + for (const auto &[job_id, job_table_data] : gcs_init_data.Jobs()) { + if (job_table_data.is_dead()) { + const auto &virtual_cluster_id = job_table_data.virtual_cluster_id(); + if (virtual_cluster_id.empty()) { + continue; + } + auto job_cluster_id = VirtualClusterID::FromBinary(virtual_cluster_id); + if (!job_cluster_id.IsJobClusterID()) { + continue; + } + + auto virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr) { + continue; + } + JobCluster *job_cluster = dynamic_cast(virtual_cluster.get()); + job_cluster->SetJobDead(); + } + } + for (auto &[placement_group_id, placement_group_table_data] : + gcs_init_data.PlacementGroups()) { + if (placement_group_table_data.state() == rpc::PlacementGroupTableData::REMOVED) { + // ignore this pg... + continue; + } + if (placement_group_table_data.is_detached()) { + // If it's a detached placement group, we need to handle registration event on FO + // by gcs virtual cluster manager. + OnDetachedPlacementGroupRegistration( + placement_group_table_data.virtual_cluster_id(), placement_group_id); + } + } +} + +void GcsVirtualClusterManager::OnNodeAdd(const rpc::GcsNodeInfo &node) { + primary_cluster_->OnNodeAdd(node); +} + +void GcsVirtualClusterManager::OnNodeDead(const rpc::GcsNodeInfo &node) { + primary_cluster_->OnNodeDead(node); +} + +void GcsVirtualClusterManager::OnJobFinished(const rpc::JobTableData &job_data) { + // exit early when job without a virtual cluster id. + const auto &virtual_cluster_id = job_data.virtual_cluster_id(); + if (virtual_cluster_id.empty()) { + return; + } + + auto job_cluster_id = VirtualClusterID::FromBinary(virtual_cluster_id); + + if (!job_cluster_id.IsJobClusterID()) { + // exit early when this job does not belong to an job cluster. + return; + } + + std::string divisible_cluster_id = job_cluster_id.ParentID().Binary(); + + auto virtual_cluster = GetVirtualCluster(divisible_cluster_id); + if (virtual_cluster == nullptr) { + RAY_LOG(WARNING) << "Failed to remove job cluster " << job_cluster_id.Binary() + << " when handling job finished event, parent cluster not exists."; + return; + } + + if (!virtual_cluster->Divisible()) { + // this should not happen, virtual cluster should be divisible. + return; + } + + auto job_virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (job_virtual_cluster == nullptr) { + // this should not happen, job cluster should exist + return; + } + JobCluster *job_cluster = dynamic_cast(job_virtual_cluster.get()); + job_cluster->SetJobDead(); + if (job_cluster->InUse()) { + // job cluster is detached, do not remove it + RAY_LOG(INFO) << "Failed to remove job cluster " << job_cluster_id.Binary() + << " when handling job finished event, job cluster is detached."; + return; + } + + DivisibleCluster *divisible_cluster = + dynamic_cast(virtual_cluster.get()); + + auto status = divisible_cluster->RemoveJobCluster( + virtual_cluster_id, + [job_cluster_id](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + if (!status.ok() || !data->is_removed()) { + RAY_LOG(WARNING) << "Failed to remove job cluster " << job_cluster_id.Binary() + << " when handling job finished event. status: " + << status.message(); + } else { + RAY_LOG(INFO) << "Successfully removed job cluster " << job_cluster_id.Binary() + << " after handling job finished event."; + } + }); + if (!status.ok()) { + RAY_LOG(WARNING) << "Failed to remove job cluster " << job_cluster_id.Binary() + << " when handling job finished event. status: " << status.message(); + } +} + +std::shared_ptr GcsVirtualClusterManager::GetVirtualCluster( + const std::string &virtual_cluster_id) { + if (virtual_cluster_id.empty()) { + return nullptr; + } + return primary_cluster_->GetVirtualCluster(virtual_cluster_id); +} + +void GcsVirtualClusterManager::HandleCreateOrUpdateVirtualCluster( + rpc::CreateOrUpdateVirtualClusterRequest request, + rpc::CreateOrUpdateVirtualClusterReply *reply, + rpc::SendReplyCallback send_reply_callback) { + const auto &virtual_cluster_id = request.virtual_cluster_id(); + RAY_LOG(INFO) << "Start creating or updating virtual cluster " << virtual_cluster_id; + auto on_done = [reply, virtual_cluster_id, callback = std::move(send_reply_callback)]( + const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + if (status.ok()) { + RAY_CHECK(data != nullptr); + // Fill the node instances of the virtual cluster to the reply. + reply->mutable_node_instances()->insert(data->node_instances().begin(), + data->node_instances().end()); + // Fill the revision of the virtual cluster to the reply. + reply->set_revision(data->revision()); + RAY_LOG(INFO) << "Succeed in creating or updating virtual cluster " << data->id(); + } else { + RAY_LOG(WARNING) << "Failed to create or update virtual cluster " + << virtual_cluster_id << ", status = " << status.ToString(); + if (replica_sets_to_recommend) { + reply->mutable_replica_sets_to_recommend()->insert( + replica_sets_to_recommend->begin(), replica_sets_to_recommend->end()); + } + } + GCS_RPC_SEND_REPLY(callback, reply, status); + }; + + // Verify if the arguments in the request is valid. + auto status = VerifyRequest(request); + if (!status.ok()) { + on_done(status, nullptr, nullptr); + return; + } + + ReplicaSets replica_sets_to_recommend; + status = primary_cluster_->CreateOrUpdateVirtualCluster( + std::move(request), on_done, &replica_sets_to_recommend); + if (!status.ok()) { + on_done(status, nullptr, &replica_sets_to_recommend); + } +} + +void GcsVirtualClusterManager::HandleRemoveVirtualCluster( + rpc::RemoveVirtualClusterRequest request, + rpc::RemoveVirtualClusterReply *reply, + rpc::SendReplyCallback send_reply_callback) { + const auto &virtual_cluster_id = request.virtual_cluster_id(); + RAY_LOG(INFO) << "Start removing virtual cluster " << virtual_cluster_id; + auto on_done = [reply, virtual_cluster_id, callback = std::move(send_reply_callback)]( + const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + if (status.ok()) { + RAY_LOG(INFO) << "Succeed in removing virtual cluster " << virtual_cluster_id; + } else { + RAY_LOG(WARNING) << "Failed to remove virtual cluster " << virtual_cluster_id + << ", status = " << status.ToString(); + } + GCS_RPC_SEND_REPLY(callback, reply, status); + }; + + auto status = VerifyRequest(request); + + if (status.ok()) { + status = primary_cluster_->RemoveVirtualCluster(virtual_cluster_id, on_done); + } + if (!status.ok()) { + on_done(status, nullptr, nullptr); + } +} + +void GcsVirtualClusterManager::HandleGetVirtualClusters( + rpc::GetVirtualClustersRequest request, + rpc::GetVirtualClustersReply *reply, + rpc::SendReplyCallback send_reply_callback) { + RAY_LOG(DEBUG) << "Getting virtual clusters."; + primary_cluster_->ForeachVirtualClustersData( + std::move(request), [reply, send_reply_callback](auto data) { + reply->add_virtual_cluster_data_list()->CopyFrom(*data); + }); + GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); +} + +void GcsVirtualClusterManager::HandleGetAllVirtualClusterInfo( + rpc::GetAllVirtualClusterInfoRequest request, + rpc::GetAllVirtualClusterInfoReply *reply, + rpc::SendReplyCallback send_reply_callback) { + size_t limit = + (request.limit() > 0) ? request.limit() : std::numeric_limits::max(); + RAY_LOG(DEBUG) << "Getting virtual clusters info."; + + size_t count = 0; + size_t total = 0; + + primary_cluster_->ForeachVirtualClustersView( + std::move(request), [reply, send_reply_callback, limit, &count, &total](auto data) { + total += 1; + if (count >= limit) { + return; + } + count += 1; + reply->add_virtual_clusters_view()->CopyFrom(*data); + }); + reply->set_total(total); + GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK()); +} + +void GcsVirtualClusterManager::HandleCreateJobCluster( + rpc::CreateJobClusterRequest request, + rpc::CreateJobClusterReply *reply, + rpc::SendReplyCallback send_reply_callback) { + const auto &virtual_cluster_id = request.virtual_cluster_id(); + RAY_LOG(INFO) << "Start creating job cluster in virtual cluster: " + << virtual_cluster_id; + auto on_done = [reply, + virtual_cluster_id, + job_id = request.job_id(), + callback = std::move(send_reply_callback)]( + const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + if (status.ok()) { + reply->set_job_cluster_id(data->id()); + if (replica_sets_to_recommend) { + reply->mutable_replica_sets_to_recommend()->insert( + replica_sets_to_recommend->begin(), replica_sets_to_recommend->end()); + } + RAY_LOG(INFO) << "Succeed in creating job cluster in virtual cluster: " + << virtual_cluster_id << " for job: " << job_id; + } else { + RAY_LOG(ERROR) << "Failed to create job cluster in virtual cluster: " + << virtual_cluster_id << " for job: " << job_id + << ", status = " << status.ToString(); + } + GCS_RPC_SEND_REPLY(callback, reply, status); + }; + + auto virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr) { + std::ostringstream ostr; + ostr << "virtual cluster not exists: " << virtual_cluster_id; + std::string message = ostr.str(); + on_done(Status::NotFound(message), nullptr, nullptr); + return; + } + if (!virtual_cluster->Divisible()) { + std::ostringstream ostr; + ostr << " virtual cluster is not exclusive: " << virtual_cluster_id; + std::string message = ostr.str(); + on_done(Status::InvalidArgument(message), nullptr, nullptr); + return; + } + ReplicaSets replica_sets(request.replica_sets().begin(), request.replica_sets().end()); + + auto divisible_cluster = dynamic_cast(virtual_cluster.get()); + std::string job_cluster_id = divisible_cluster->BuildJobClusterID(request.job_id()); + + ReplicaSets replica_sets_to_recommend; + auto status = divisible_cluster->CreateJobCluster( + job_cluster_id, std::move(replica_sets), on_done, &replica_sets_to_recommend); + if (!status.ok()) { + on_done(status, nullptr, &replica_sets_to_recommend); + } +} + +Status GcsVirtualClusterManager::VerifyRequest( + const rpc::CreateOrUpdateVirtualClusterRequest &request) { + const auto &virtual_cluster_id = request.virtual_cluster_id(); + if (virtual_cluster_id.empty()) { + std::ostringstream ostr; + ostr << "Invalid request, the virtual cluster id is empty."; + std::string message = ostr.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + + if (virtual_cluster_id == primary_cluster_->GetID()) { + std::ostringstream ostr; + ostr << "Invalid request, " << virtual_cluster_id + << " can not be created or updated."; + auto message = ostr.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + + for (const auto &[template_id, replicas] : request.replica_sets()) { + if (replicas < 0) { + std::ostringstream ostr; + ostr << "Invalid request, replicas(" << replicas + << ") must >= 0, virtual_cluster_id: " << virtual_cluster_id; + auto message = ostr.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + + if (template_id.empty()) { + std::ostringstream ostr; + ostr << "Invalid request, template_id is empty, virtual_cluster_id: " + << virtual_cluster_id; + auto message = ostr.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + } + + if (auto logical_cluster = + primary_cluster_->GetLogicalCluster(request.virtual_cluster_id())) { + // Check if the revision of the virtual cluster is expired. + if (request.revision() != logical_cluster->GetRevision()) { + std::ostringstream ss; + ss << "The revision (" << request.revision() + << ") is expired, the latest revision of the virtual cluster " + << request.virtual_cluster_id() << " is " << logical_cluster->GetRevision(); + std::string message = ss.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + + // check if the request attributes are compatible with the virtual cluster. + if (request.divisible() != logical_cluster->Divisible()) { + std::ostringstream ostr; + ostr << "The requested attributes are incompatible with virtual cluster " + << request.virtual_cluster_id() << ". expect: (" + << logical_cluster->Divisible() << "), actual: (" << request.divisible() + << ")."; + std::string message = ostr.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + } + + return Status::OK(); +} + +Status GcsVirtualClusterManager::VerifyRequest( + const rpc::RemoveVirtualClusterRequest &request) { + const auto &virtual_cluster_id = request.virtual_cluster_id(); + if (virtual_cluster_id.empty()) { + std::ostringstream ostr; + ostr << "Invalid request, the virtual cluster id is empty."; + std::string message = ostr.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + + if (virtual_cluster_id == primary_cluster_->GetID()) { + std::ostringstream ostr; + ostr << "Invalid request, " << virtual_cluster_id << " can not be removed."; + auto message = ostr.str(); + RAY_LOG(ERROR) << message; + return Status::InvalidArgument(message); + } + return Status::OK(); +} + +Status GcsVirtualClusterManager::FlushAndPublish( + std::shared_ptr data, + CreateOrUpdateVirtualClusterCallback callback) { + auto on_done = [this, data, callback = std::move(callback)](const Status &status) { + // The backend storage is supposed to be reliable, so the status must be ok. + RAY_CHECK_OK(status); + if (data->divisible()) { + // Tasks can only be scheduled on the nodes in the indivisible cluster, so we just + // need to publish the indivisible cluster data. + if (callback) { + callback(status, std::move(data), nullptr); + } + return; + } + + RAY_CHECK_OK(gcs_publisher_.PublishVirtualCluster( + VirtualClusterID::FromBinary(data->id()), *data, nullptr)); + if (callback) { + callback(status, std::move(data), nullptr); + } + }; + + if (data->is_removed()) { + return gcs_table_storage_.VirtualClusterTable().Delete( + VirtualClusterID::FromBinary(data->id()), {on_done, io_context_}); + } + + // Write the virtual cluster data to the storage. + return gcs_table_storage_.VirtualClusterTable().Put( + VirtualClusterID::FromBinary(data->id()), *data, {on_done, io_context_}); +} + +void GcsVirtualClusterManager::OnDetachedActorRegistration( + const std::string &virtual_cluster_id, const ActorID &actor_id) { + if (virtual_cluster_id.empty()) { + return; + } + auto virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr) { + RAY_LOG(ERROR) << "Failed to process the registration of detached actor " << actor_id + << " as the virtual cluster " << virtual_cluster_id + << " does not exist."; + return; + } + if (!virtual_cluster->Divisible()) { + IndivisibleCluster *indivisible_cluster = + dynamic_cast(virtual_cluster.get()); + indivisible_cluster->OnDetachedActorRegistration(actor_id); + } +} + +void GcsVirtualClusterManager::OnDetachedActorDestroy( + const std::string &virtual_cluster_id, const ActorID &actor_id) { + if (virtual_cluster_id.empty()) { + return; + } + auto virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr) { + RAY_LOG(ERROR) << "Failed to process the destroy of detached actor " << actor_id + << " as the virtual cluster " << virtual_cluster_id + << " does not exist."; + return; + } + if (!virtual_cluster->Divisible()) { + IndivisibleCluster *indivisible_cluster = + dynamic_cast(virtual_cluster.get()); + indivisible_cluster->OnDetachedActorDestroy(actor_id); + } + if (VirtualClusterID::FromBinary(virtual_cluster_id).IsJobClusterID()) { + JobCluster *job_cluster = dynamic_cast(virtual_cluster.get()); + if (!job_cluster->InUse() && job_cluster->IsJobDead()) { + auto status = primary_cluster_->RemoveVirtualCluster(virtual_cluster_id, nullptr); + if (!status.ok()) { + RAY_LOG(WARNING) << "Failed to remove virtual cluster " << virtual_cluster_id + << " after handling detached actor destroy event. status: " + << status.message(); + } + } + } +} + +void GcsVirtualClusterManager::OnDetachedPlacementGroupRegistration( + const std::string &virtual_cluster_id, const PlacementGroupID &placement_group_id) { + if (virtual_cluster_id.empty()) { + return; + } + auto virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr) { + RAY_LOG(ERROR) << "Failed to process the registration of detached placement group " + << placement_group_id << " as the virtual cluster " + << virtual_cluster_id << " does not exist."; + return; + } + if (!virtual_cluster->Divisible()) { + IndivisibleCluster *indivisible_cluster = + dynamic_cast(virtual_cluster.get()); + indivisible_cluster->OnDetachedPlacementGroupRegistration(placement_group_id); + } +} + +void GcsVirtualClusterManager::OnDetachedPlacementGroupDestroy( + const std::string &virtual_cluster_id, const PlacementGroupID &placement_group_id) { + if (virtual_cluster_id.empty()) { + return; + } + auto virtual_cluster = GetVirtualCluster(virtual_cluster_id); + if (virtual_cluster == nullptr) { + RAY_LOG(ERROR) << "Failed to process the destroy of detached placement group " + << placement_group_id << " as the virtual cluster " + << virtual_cluster_id << " does not exist."; + return; + } + if (!virtual_cluster->Divisible()) { + IndivisibleCluster *indivisible_cluster = + dynamic_cast(virtual_cluster.get()); + indivisible_cluster->OnDetachedPlacementGroupDestroy(placement_group_id); + } + if (VirtualClusterID::FromBinary(virtual_cluster_id).IsJobClusterID()) { + JobCluster *job_cluster = dynamic_cast(virtual_cluster.get()); + if (!job_cluster->InUse() && job_cluster->IsJobDead()) { + auto status = primary_cluster_->RemoveVirtualCluster(virtual_cluster_id, nullptr); + if (!status.ok()) { + RAY_LOG(WARNING) + << "Failed to remove virtual cluster " << virtual_cluster_id + << " after handling detached placement group destroy event. status: " + << status.message(); + } + } + } +} + +} // namespace gcs +} // namespace ray \ No newline at end of file diff --git a/src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.h b/src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.h new file mode 100644 index 000000000000..cd92ca6dcb8c --- /dev/null +++ b/src/ray/gcs/gcs_server/gcs_virtual_cluster_manager.h @@ -0,0 +1,138 @@ +// Copyright 2017 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 "ray/gcs/gcs_server/gcs_init_data.h" +#include "ray/gcs/gcs_server/gcs_table_storage.h" +#include "ray/gcs/gcs_server/gcs_virtual_cluster.h" +#include "ray/gcs/pubsub/gcs_pub_sub.h" +#include "ray/rpc/gcs_server/gcs_rpc_server.h" + +namespace ray { + +class PeriodicalRunner; + +namespace gcs { + +/// This implementation class of `VirtualClusterInfoHandler`. +class GcsVirtualClusterManager : public rpc::VirtualClusterInfoHandler { + public: + explicit GcsVirtualClusterManager( + instrumented_io_context &io_context, + GcsTableStorage &gcs_table_storage, + GcsPublisher &gcs_publisher, + const ClusterResourceManager &cluster_resource_manager, + std::shared_ptr periodical_runner = nullptr) + : io_context_(io_context), + gcs_table_storage_(gcs_table_storage), + gcs_publisher_(gcs_publisher), + periodical_runner_(periodical_runner), + primary_cluster_(std::make_shared( + [this](auto data, auto callback) { + return FlushAndPublish(std::move(data), std::move(callback)); + }, + cluster_resource_manager)) {} + + /// Initialize with the gcs tables data synchronously. + /// This should be called when GCS server restarts after a failure. + /// + /// \param gcs_init_data. + void Initialize(const GcsInitData &gcs_init_data); + + /// Handle the node added event. + /// + /// \param node The node that is added. + void OnNodeAdd(const rpc::GcsNodeInfo &node); + + /// Handle the node dead event. + /// + /// \param node The node that is dead. + void OnNodeDead(const rpc::GcsNodeInfo &node); + + /// Handle the job finished event. + /// + /// \param job_data The job that is finished. + void OnJobFinished(const rpc::JobTableData &job_data); + + /// Get virtual cluster by virtual cluster id + /// + /// \param virtual_cluster_id The id of virtual cluster + /// \return the virtual cluster + std::shared_ptr GetVirtualCluster( + const std::string &virtual_cluster_id); + + /// Handle detached actor registration. + void OnDetachedActorRegistration(const std::string &virtual_cluster_id, + const ActorID &actor_id); + + /// Handle detached actor destroy. + void OnDetachedActorDestroy(const std::string &virtual_cluster_id, + const ActorID &actor_id); + + /// Handle detached placement group registration. + void OnDetachedPlacementGroupRegistration(const std::string &virtual_cluster_id, + const PlacementGroupID &placement_group_id); + + /// Handle detached placement group destroy. + void OnDetachedPlacementGroupDestroy(const std::string &virtual_cluster_id, + const PlacementGroupID &placement_group_id); + + protected: + void HandleCreateOrUpdateVirtualCluster( + rpc::CreateOrUpdateVirtualClusterRequest request, + rpc::CreateOrUpdateVirtualClusterReply *reply, + rpc::SendReplyCallback send_reply_callback) override; + + void HandleRemoveVirtualCluster(rpc::RemoveVirtualClusterRequest request, + rpc::RemoveVirtualClusterReply *reply, + rpc::SendReplyCallback send_reply_callback) override; + + void HandleGetVirtualClusters(rpc::GetVirtualClustersRequest request, + rpc::GetVirtualClustersReply *reply, + rpc::SendReplyCallback send_reply_callback) override; + + void HandleCreateJobCluster(rpc::CreateJobClusterRequest request, + rpc::CreateJobClusterReply *reply, + rpc::SendReplyCallback send_reply_callback) override; + + void HandleGetAllVirtualClusterInfo( + rpc::GetAllVirtualClusterInfoRequest request, + rpc::GetAllVirtualClusterInfoReply *reply, + rpc::SendReplyCallback send_reply_callback) override; + + Status VerifyRequest(const rpc::CreateOrUpdateVirtualClusterRequest &request); + + Status VerifyRequest(const rpc::RemoveVirtualClusterRequest &request); + + Status FlushAndPublish(std::shared_ptr data, + CreateOrUpdateVirtualClusterCallback callback); + + private: + /// io context. This is to ensure thread safety. Ideally, all public + /// funciton needs to post job to this io_context. + instrumented_io_context &io_context_; + /// The storage of the GCS tables. + GcsTableStorage &gcs_table_storage_; + /// The publisher of the GCS tables. + GcsPublisher &gcs_publisher_; + + /// The periodical runner to run `ReplenishAllClusterNodeInstances` task. + std::shared_ptr periodical_runner_; + /// The global cluster. + std::shared_ptr primary_cluster_; +}; + +} // namespace gcs +} // namespace ray \ No newline at end of file diff --git a/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc b/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc index 829c86ad2589..467626318b70 100644 --- a/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_server_rpc_test.cc @@ -375,6 +375,17 @@ TEST_F(GcsServerTest, TestNodeInfoFilters) { rpc::GetAllNodeInfoReply reply; RAY_CHECK_OK(client_->SyncGetAllNodeInfo(request, &reply)); + ASSERT_EQ(reply.node_info_list_size(), 1); + ASSERT_EQ(reply.num_filtered(), 2); + ASSERT_EQ(reply.total(), 3); + } + { + // Get by virtual_cluster_id + rpc::GetAllNodeInfoRequest request; + request.mutable_filters()->set_virtual_cluster_id("virt_1"); + rpc::GetAllNodeInfoReply reply; + RAY_CHECK_OK(client_->SyncGetAllNodeInfo(request, &reply)); + ASSERT_EQ(reply.node_info_list_size(), 1); ASSERT_EQ(reply.num_filtered(), 2); ASSERT_EQ(reply.total(), 3); diff --git a/src/ray/gcs/gcs_server/test/gcs_virtual_cluster_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_virtual_cluster_manager_test.cc new file mode 100644 index 000000000000..3528799b323f --- /dev/null +++ b/src/ray/gcs/gcs_server/test/gcs_virtual_cluster_manager_test.cc @@ -0,0 +1,1121 @@ +// Copyright 2017 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/gcs/gcs_server/gcs_virtual_cluster_manager.h" + +// clang-format off +#include "gtest/gtest.h" +#include "ray/gcs/gcs_server/test/gcs_server_test_util.h" +#include "ray/gcs/test/gcs_test_util.h" +#include "mock/ray/pubsub/publisher.h" +#include "mock/ray/pubsub/subscriber.h" + +// clang-format on + +namespace ray { +namespace gcs { +class GcsVirtualClusterManagerTest : public ::testing::Test { + public: + GcsVirtualClusterManagerTest() : cluster_resource_manager_(io_service_) { + gcs_publisher_ = std::make_unique( + std::make_unique()); + gcs_table_storage_ = std::make_unique(io_service_); + gcs_virtual_cluster_manager_ = std::make_unique( + *gcs_table_storage_, *gcs_publisher_, cluster_resource_manager_); + } + + instrumented_io_context io_service_; + std::unique_ptr gcs_publisher_; + std::unique_ptr gcs_table_storage_; + std::unique_ptr gcs_virtual_cluster_manager_; + ClusterResourceManager cluster_resource_manager_; +}; + +class MockGcsInitData : public GcsInitData { + public: + using GcsInitData::GcsInitData; + + void SetNodes( + const absl::flat_hash_map> &nodes) { + for (const auto &[node_id, node] : nodes) { + node_table_data_[node_id] = *node; + } + } + + void SetVirtualClusters( + const absl::flat_hash_map> + &virtual_clusters) { + for (const auto &[virtual_cluster_id, virtual_cluster] : virtual_clusters) { + virtual_cluster_table_data_[VirtualClusterID::FromBinary(virtual_cluster_id)] = + *virtual_cluster; + } + } +}; + +bool operator==(const ReplicaInstances &lhs, const ReplicaInstances &rhs) { + if (lhs.size() != rhs.size()) { + return false; + } + for (const auto &[template_id, job_node_instances] : lhs) { + if (!rhs.contains(template_id)) { + return false; + } + const auto &rhs_job_node_instances = rhs.at(template_id); + if (job_node_instances.size() != rhs_job_node_instances.size()) { + return false; + } + for (const auto &[job_cluster_id, node_instances] : job_node_instances) { + if (!rhs_job_node_instances.contains(job_cluster_id)) { + return false; + } + const auto &rhs_node_instances = rhs_job_node_instances.at(job_cluster_id); + if (node_instances.size() != rhs_node_instances.size()) { + return false; + } + for (const auto &[node_instance_id, node_instance] : node_instances) { + if (!rhs_node_instances.contains(node_instance_id)) { + return false; + } + const auto &rhs_node_instance = rhs_node_instances.at(node_instance_id); + if (node_instance->hostname() != rhs_node_instance->hostname() || + node_instance->template_id() != rhs_node_instance->template_id() || + node_instance->is_dead() != rhs_node_instance->is_dead()) { + return false; + } + } + } + } + return true; +} + +// lhs <= rhs +bool operator<=(const ReplicaInstances &lhs, const ReplicaInstances &rhs) { + if (lhs.size() > rhs.size()) { + return false; + } + for (const auto &[template_id, job_node_instances] : lhs) { + if (!rhs.contains(template_id)) { + return false; + } + const auto &rhs_job_node_instances = rhs.at(template_id); + if (job_node_instances.size() > rhs_job_node_instances.size()) { + return false; + } + for (const auto &[job_cluster_id, node_instances] : job_node_instances) { + if (!rhs_job_node_instances.contains(job_cluster_id)) { + return false; + } + const auto &rhs_node_instances = rhs_job_node_instances.at(job_cluster_id); + if (node_instances.size() > rhs_node_instances.size()) { + return false; + } + for (const auto &[node_instance_id, node_instance] : node_instances) { + if (!rhs_node_instances.contains(node_instance_id)) { + return false; + } + const auto &rhs_node_instance = rhs_node_instances.at(node_instance_id); + if (node_instance->hostname() != rhs_node_instance->hostname() || + node_instance->template_id() != rhs_node_instance->template_id() || + node_instance->is_dead() != rhs_node_instance->is_dead()) { + return false; + } + } + } + } + return true; +} + +bool operator==(const ReplicaSets &lhs, const ReplicaSets &rhs) { + if (lhs.size() != rhs.size()) { + return false; + } + for (const auto &[template_id, count] : lhs) { + if (!rhs.contains(template_id) || rhs.at(template_id) != count) { + return false; + } + } + return true; +} + +bool operator<=(const ReplicaSets &lhs, const ReplicaSets &rhs) { + if (lhs.size() > rhs.size()) { + return false; + } + for (const auto &[template_id, count] : lhs) { + if (!rhs.contains(template_id) || rhs.at(template_id) < count) { + return false; + } + } + return true; +} + +class VirtualClusterTest : public ::testing::Test { + public: + VirtualClusterTest() : cluster_resource_manager_(io_service_) { + async_data_flusher_ = [this](auto data, auto callback) { + virtual_clusters_data_[data->id()] = data; + if (callback != nullptr) { + callback(Status::OK(), data, nullptr); + } + return Status::OK(); + }; + } + + std::shared_ptr InitPrimaryCluster( + size_t node_count, + size_t template_count, + absl::flat_hash_map>> + *template_id_to_nodes = nullptr) { + auto primary_cluster = std::make_shared( + async_data_flusher_, cluster_resource_manager_); + + for (size_t i = 0; i < node_count; ++i) { + auto node = Mocker::GenNodeInfo(); + auto template_id = std::to_string(i % template_count); + node->set_node_type_name(template_id); + primary_cluster->OnNodeAdd(*node); + if (template_id_to_nodes != nullptr) { + (*template_id_to_nodes)[template_id].emplace(NodeID::FromBinary(node->node_id()), + node); + } + nodes_.emplace(NodeID::FromBinary(node->node_id()), node); + } + return primary_cluster; + } + + Status CreateVirtualCluster( + std::shared_ptr primary_cluster, + const std::string &virtual_cluster_id, + const absl::flat_hash_map &replica_sets, + bool divisible = true) { + rpc::CreateOrUpdateVirtualClusterRequest request; + request.set_virtual_cluster_id(virtual_cluster_id); + request.set_divisible(true); + request.set_revision(0); + request.mutable_replica_sets()->insert(replica_sets.begin(), replica_sets.end()); + auto status = primary_cluster->CreateOrUpdateVirtualCluster( + request, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + }); + return status; + } + + instrumented_io_context io_service_; + ClusterResourceManager cluster_resource_manager_; + AsyncClusterDataFlusher async_data_flusher_; + absl::flat_hash_map> nodes_; + absl::flat_hash_map> + virtual_clusters_data_; +}; + +class PrimaryClusterTest : public VirtualClusterTest { + public: + using VirtualClusterTest::VirtualClusterTest; +}; + +TEST_F(PrimaryClusterTest, NodeAddAndRemove) { + absl::flat_hash_map>> + template_id_to_nodes; + size_t node_count = 200; + size_t template_count = 10; + auto primary_cluster = + InitPrimaryCluster(node_count, template_count, &template_id_to_nodes); + + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + EXPECT_EQ(visiable_node_instances.size(), template_count); + for (auto &[template_id, job_node_instances] : visiable_node_instances) { + EXPECT_EQ(job_node_instances.size(), 1); + EXPECT_EQ(job_node_instances.begin()->first, ray::gcs::kUndividedClusterId); + EXPECT_EQ(job_node_instances.begin()->second.size(), node_count / template_count); + } + + size_t each_template_keeps_node_count = 8; + ASSERT_TRUE(node_count / template_count > each_template_keeps_node_count); + for (auto &[template_id, nodes] : template_id_to_nodes) { + auto iter = nodes.begin(); + for (; iter != nodes.end();) { + if (nodes.size() == each_template_keeps_node_count) { + break; + } + auto current_iter = iter++; + primary_cluster->OnNodeDead(*current_iter->second); + nodes.erase(current_iter); + } + } + for (auto &[template_id, job_node_instances] : visiable_node_instances) { + EXPECT_EQ(job_node_instances.size(), 1); + EXPECT_EQ(job_node_instances.begin()->first, ray::gcs::kUndividedClusterId); + EXPECT_EQ(job_node_instances.begin()->second.size(), node_count / template_count); + + size_t alive_count = 0; + for (const auto &[id, node_instance] : job_node_instances.begin()->second) { + if (!node_instance->is_dead()) { + alive_count++; + } + } + EXPECT_EQ(alive_count, each_template_keeps_node_count); + } + + each_template_keeps_node_count = 0; + for (auto &[template_id, nodes] : template_id_to_nodes) { + auto iter = nodes.begin(); + for (; iter != nodes.end();) { + if (nodes.size() == each_template_keeps_node_count) { + break; + } + auto current_iter = iter++; + primary_cluster->OnNodeDead(*current_iter->second); + nodes.erase(current_iter); + } + } + EXPECT_EQ(visiable_node_instances.size(), template_count); + for (auto &[template_id, job_node_instances] : visiable_node_instances) { + EXPECT_EQ(job_node_instances.size(), 1); + EXPECT_EQ(job_node_instances.begin()->first, ray::gcs::kUndividedClusterId); + EXPECT_EQ(job_node_instances.begin()->second.size(), node_count / template_count); + + size_t alive_count = 0; + for (const auto &[id, node_instance] : job_node_instances.begin()->second) { + if (!node_instance->is_dead()) { + alive_count++; + } + } + EXPECT_EQ(alive_count, each_template_keeps_node_count); + } +} + +TEST_F(PrimaryClusterTest, CreateOrUpdateVirtualCluster) { + size_t node_count = 200; + size_t template_count = 10; + auto primary_cluster = InitPrimaryCluster(node_count, template_count); + + std::string template_id_0 = "0"; + std::string template_id_1 = "1"; + size_t node_count_per_template = node_count / template_count; + + std::string virtual_cluster_id_0 = "virtual_cluster_id_0"; + ASSERT_TRUE(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_0, + {{template_id_0, 5}, {template_id_1, 10}}) + .ok()); + + { + // Check the logical cluster virtual_cluster_id_0 visible node instances. + auto logical_cluster = primary_cluster->GetLogicalCluster("virtual_cluster_id_0"); + ASSERT_NE(logical_cluster, nullptr); + const auto &visiable_node_instances = logical_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 has 5 nodes, template_id_1 has 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + 10); + + // Check that the revision changed. + EXPECT_NE(logical_cluster->GetRevision(), 0); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 remains template_count - 5 nodes, template_id_1 has + // template_count - 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), template_count); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template - 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template - 10); + + // Check that the revision unchanged. + EXPECT_NE(primary_cluster->GetRevision(), 0); + } + + // Create virtual_cluster_id_1 and check that the status is ok. + std::string virtual_cluster_id_1 = "virtual_cluster_id_1"; + ASSERT_TRUE(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_1, + {{template_id_0, node_count_per_template - 5}, + {template_id_1, node_count_per_template - 10}}) + .ok()); + + { + // Check the logical cluster virtual_cluster_id_1 visible node instances. + auto logical_cluster = primary_cluster->GetLogicalCluster("virtual_cluster_id_1"); + ASSERT_NE(logical_cluster, nullptr); + const auto &visiable_node_instances = logical_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 has 5 nodes, template_id_1 has 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template - 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template - 10); + + // Check that the revision changed. + EXPECT_NE(logical_cluster->GetRevision(), 0); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 remains template_count - 5 nodes, template_id_1 has + // template_count - 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), template_count - 2); + EXPECT_FALSE(visiable_node_instances.contains(template_id_0)); + EXPECT_FALSE(visiable_node_instances.contains(template_id_1)); + + // Check that the revision unchanged. + EXPECT_NE(primary_cluster->GetRevision(), 0); + } + + // Create virtual_cluster_id_2 and check that the status is succeed. + std::string virtual_cluster_id_2 = "virtual_cluster_id_2"; + ASSERT_TRUE(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_2, + {{template_id_0, 0}, {template_id_1, 0}}) + .ok()); + + { + auto logical_cluster = primary_cluster->GetLogicalCluster("virtual_cluster_id_2"); + ASSERT_NE(logical_cluster, nullptr); + ASSERT_EQ(logical_cluster->GetVisibleNodeInstances().size(), 2); + ASSERT_EQ(logical_cluster->GetVisibleNodeInstances().at(template_id_0).size(), 0); + ASSERT_EQ(logical_cluster->GetVisibleNodeInstances().at(template_id_1).size(), 0); + + ASSERT_EQ(logical_cluster->GetReplicaSets().size(), 2); + ASSERT_EQ(logical_cluster->GetReplicaSets().at(template_id_0), 0); + ASSERT_EQ(logical_cluster->GetReplicaSets().at(template_id_1), 0); + } + + { + // Create virtual_cluster_id_3 and check that the status is failed. + std::string virtual_cluster_id_3 = "virtual_cluster_id_3"; + ASSERT_FALSE(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_3, + {{template_id_0, 5}, {template_id_1, 10}}) + .ok()); + ASSERT_EQ(primary_cluster->GetLogicalCluster(virtual_cluster_id_3), nullptr); + } +} + +TEST_F(PrimaryClusterTest, CreateJobCluster) { + size_t node_count = 200; + size_t template_count = 10; + auto primary_cluster = InitPrimaryCluster(node_count, template_count); + + std::string template_id_0 = "0"; + std::string template_id_1 = "1"; + size_t node_count_per_template = node_count / template_count; + + std::string job_cluster_id_0 = primary_cluster->BuildJobClusterID("job_0"); + { + // Create job_cluster_id_0 and check that the status is ok. + auto status = primary_cluster->CreateJobCluster( + job_cluster_id_0, + {{template_id_0, 5}, {template_id_1, 10}}, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + }); + ASSERT_TRUE(status.ok()); + } + + auto job_cluster_0 = primary_cluster->GetJobCluster(job_cluster_id_0); + ASSERT_NE(job_cluster_0, nullptr); + { + // Check the job cluster job_cluster_id_0 visible node instances. + const auto &visiable_node_instances = job_cluster_0->GetVisibleNodeInstances(); + // Check that template_id_0 has 5 nodes, template_id_1 has 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + 10); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that job_cluster_id_0 in template_id_0 has 5 nodes, kUndividedClusterId in + // template_id_0 has template_count - 5 nodes. + EXPECT_EQ(visiable_node_instances.size(), template_count); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(job_cluster_id_0).size(), 5); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template - 5); + + // Check that job_cluster_id_0 in template_id_1 has 10 nodes, kUndividedClusterId in + // template_id_1 has template_count - 10 nodes. + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(job_cluster_id_0).size(), 10); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template - 10); + } + + std::string job_cluster_id_1 = primary_cluster->BuildJobClusterID("job_1"); + { + // Create job_cluster_id_1 and check that the status is ok. + auto status = primary_cluster->CreateJobCluster( + job_cluster_id_1, + {{template_id_0, node_count_per_template - 5}, + {template_id_1, node_count_per_template - 10}}, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + }); + ASSERT_TRUE(status.ok()); + } + + auto job_cluster_1 = primary_cluster->GetJobCluster(job_cluster_id_1); + ASSERT_NE(job_cluster_1, nullptr); + { + // Check the job cluster job_cluster_id_1 visible node instances. + const auto &visiable_node_instances = job_cluster_1->GetVisibleNodeInstances(); + // Check that template_id_0 has node_count_per_template - 5 nodes, template_id_1 has + // node_count_per_template - 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template - 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template - 10); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that job_cluster_id_0 in template_id_0 has 5 nodes, + // job_cluster_id_1 in template_id_0 has template_count - 5 nodes, kUndividedClusterId + // does not exist in template_id_0. + EXPECT_EQ(visiable_node_instances.size(), template_count); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(job_cluster_id_0).size(), 5); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(job_cluster_id_1).size(), + node_count_per_template - 5); + ASSERT_FALSE(visiable_node_instances.at(template_id_0).contains(kUndividedClusterId)); + + // Check that job_cluster_id_0 in template_id_1 has 10 nodes, + // job_cluster_id_1 in template_id_0 has template_count - 10 nodes, + // kUndividedClusterId does not exist in template_id_1. + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(job_cluster_id_0).size(), 10); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(job_cluster_id_1).size(), + node_count_per_template - 10); + ASSERT_FALSE(visiable_node_instances.at(template_id_0).contains(kUndividedClusterId)); + } +} + +TEST_F(PrimaryClusterTest, RemoveJobCluster) { + size_t node_count = 200; + size_t template_count = 10; + auto primary_cluster = InitPrimaryCluster(node_count, template_count); + + std::string template_id_0 = "0"; + std::string template_id_1 = "1"; + size_t node_count_per_template = node_count / template_count; + + std::string job_cluster_id_0 = primary_cluster->BuildJobClusterID("job_0"); + { + // Create job_cluster_id_0 and check that the status is ok. + auto status = primary_cluster->CreateJobCluster( + job_cluster_id_0, + {{template_id_0, 5}, {template_id_1, 10}}, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + }); + ASSERT_TRUE(status.ok()); + } + + auto job_cluster_0 = primary_cluster->GetJobCluster(job_cluster_id_0); + ASSERT_NE(job_cluster_0, nullptr); + { + // Check the job cluster job_cluster_id_0 visible node instances. + const auto &visiable_node_instances = job_cluster_0->GetVisibleNodeInstances(); + // Check that template_id_0 has 5 nodes, template_id_1 has 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + 10); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that job_cluster_id_0 in template_id_0 has 5 nodes, kUndividedClusterId in + // template_id_0 has template_count - 5 nodes. + EXPECT_EQ(visiable_node_instances.size(), template_count); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(job_cluster_id_0).size(), 5); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template - 5); + + // Check that job_cluster_id_0 in template_id_1 has 10 nodes, kUndividedClusterId in + // template_id_1 has template_count - 10 nodes. + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(job_cluster_id_0).size(), 10); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template - 10); + } + + { + auto status = primary_cluster->RemoveJobCluster( + job_cluster_id_0, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + ASSERT_TRUE(data->is_removed()); + }); + ASSERT_TRUE(status.ok()); + ASSERT_EQ(primary_cluster->GetJobCluster(job_cluster_id_0), nullptr); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 has node_count_per_template nodes. + EXPECT_EQ(visiable_node_instances.size(), template_count); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template); + + // Check that template_id_1 has node_count_per_template nodes. + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template); + } + + { + std::string job_cluster_id_1 = primary_cluster->BuildJobClusterID("job_1"); + // Remove the job cluster that does not exist. + auto status = primary_cluster->RemoveJobCluster( + job_cluster_id_1, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { ASSERT_FALSE(true); }); + ASSERT_TRUE(status.IsNotFound()); + } +} + +TEST_F(PrimaryClusterTest, RemoveVirtualCluster) { + size_t node_count = 200; + size_t template_count = 10; + auto primary_cluster = InitPrimaryCluster(node_count, template_count); + + std::string template_id_0 = "0"; + std::string template_id_1 = "1"; + size_t node_count_per_template = node_count / template_count; + + // Create virtual_cluster_id_0 and check that the status is succeed. + std::string virtual_cluster_id_0 = "virtual_cluster_id_0"; + ASSERT_TRUE(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_0, + {{template_id_0, 5}, {template_id_1, 10}}) + .ok()); + + { + // Check the logical cluster virtual_cluster_id_0 visible node instances. + auto logical_cluster = primary_cluster->GetLogicalCluster(virtual_cluster_id_0); + ASSERT_NE(logical_cluster, nullptr); + const auto &visiable_node_instances = logical_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 has 5 nodes, template_id_1 has 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), 2); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + 10); + + // Check that the revision changed. + EXPECT_NE(logical_cluster->GetRevision(), 0); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 remains template_count - 5 nodes, template_id_1 has + // template_count - 10 nodes. + EXPECT_EQ(visiable_node_instances.size(), template_count); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template - 5); + + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template - 10); + + // Check that the revision unchanged. + EXPECT_NE(primary_cluster->GetRevision(), 0); + } + + { + auto status = primary_cluster->RemoveVirtualCluster( + virtual_cluster_id_0, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + ASSERT_TRUE(data->is_removed()); + }); + ASSERT_TRUE(status.ok()); + ASSERT_EQ(primary_cluster->GetLogicalCluster(virtual_cluster_id_0), nullptr); + } + + { + // Check the primary cluster visible node instances. + const auto &visiable_node_instances = primary_cluster->GetVisibleNodeInstances(); + // Check that template_id_0 has node_count_per_template nodes. + EXPECT_EQ(visiable_node_instances.size(), template_count); + EXPECT_EQ(visiable_node_instances.at(template_id_0).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_0).at(kUndividedClusterId).size(), + node_count_per_template); + + // Check that template_id_1 has node_count_per_template nodes. + EXPECT_EQ(visiable_node_instances.at(template_id_1).size(), 1); + EXPECT_EQ(visiable_node_instances.at(template_id_1).at(kUndividedClusterId).size(), + node_count_per_template); + } + + { + // Remove the logical cluster that does not exist. + auto status = primary_cluster->RemoveVirtualCluster( + virtual_cluster_id_0, + [this](const Status &status, + std::shared_ptr data, + const ReplicaSets *replica_sets_to_recommend) { ASSERT_FALSE(true); }); + ASSERT_TRUE(status.IsNotFound()); + } +} + +TEST_F(PrimaryClusterTest, GetVirtualClusters) { + size_t node_count = 200; + size_t template_count = 10; + auto primary_cluster = InitPrimaryCluster(node_count, template_count); + + std::string template_id_0 = "0"; + std::string template_id_1 = "1"; + + std::string virtual_cluster_id_0 = "virtual_cluster_id_0"; + std::string virtual_cluster_id_1 = "virtual_cluster_id_1"; + + std::string job_cluster_id_0 = primary_cluster->BuildJobClusterID("job_0"); + ASSERT_TRUE(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_0, + {{template_id_0, 5}, {template_id_1, 5}}) + .ok()); + + ASSERT_TRUE(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_1, + {{template_id_0, 5}, {template_id_1, 5}}) + .ok()); + + ASSERT_TRUE(primary_cluster + ->CreateJobCluster(job_cluster_id_0, + {{template_id_0, 10}, {template_id_1, 10}}, + [this](const Status &status, + auto data, + const auto *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + }) + .ok()); + + auto virtual_cluster_0 = std::dynamic_pointer_cast( + primary_cluster->GetLogicalCluster(virtual_cluster_id_0)); + ASSERT_TRUE(virtual_cluster_0 != nullptr); + + std::string job_cluster_id_1 = virtual_cluster_0->BuildJobClusterID("job_1"); + ASSERT_TRUE(virtual_cluster_0 + ->CreateJobCluster(job_cluster_id_1, + {{template_id_0, 2}, {template_id_1, 2}}, + [this](const Status &status, + auto data, + const auto *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + }) + .ok()); + + { + rpc::GetVirtualClustersRequest request; + request.set_virtual_cluster_id(virtual_cluster_id_0); + request.set_include_job_clusters(false); + absl::flat_hash_map> + virtual_clusters_data_map; + primary_cluster->ForeachVirtualClustersData( + request, [this, &virtual_clusters_data_map](auto data) { + virtual_clusters_data_map.emplace(data->id(), data); + }); + ASSERT_EQ(virtual_clusters_data_map.size(), 1); + ASSERT_TRUE(virtual_clusters_data_map.contains(virtual_cluster_id_0)); + + virtual_clusters_data_map.clear(); + request.set_include_job_clusters(true); + primary_cluster->ForeachVirtualClustersData( + request, [this, &virtual_clusters_data_map](auto data) { + virtual_clusters_data_map.emplace(data->id(), data); + }); + ASSERT_EQ(virtual_clusters_data_map.size(), 2); + ASSERT_TRUE(virtual_clusters_data_map.contains(virtual_cluster_id_0)); + + auto job_cluster = virtual_cluster_0->GetJobCluster(job_cluster_id_1); + ASSERT_TRUE(job_cluster != nullptr); + ASSERT_TRUE(virtual_clusters_data_map.contains(job_cluster->GetID())); + + virtual_clusters_data_map.clear(); + request.set_include_job_clusters(true); + request.set_only_include_indivisible_clusters(true); + primary_cluster->ForeachVirtualClustersData( + request, [this, &virtual_clusters_data_map](auto data) { + virtual_clusters_data_map.emplace(data->id(), data); + }); + ASSERT_EQ(virtual_clusters_data_map.size(), 1); + ASSERT_FALSE(virtual_clusters_data_map.contains(virtual_cluster_id_0)); + ASSERT_TRUE(virtual_clusters_data_map.contains(job_cluster->GetID())); + } +} + +// ┌───────────────────────────────────────────────────┐ +// │ ┌───────────────────┐ ┌─────────┐ │ +// │ │ virtual_cluster_1 │ │ │ Exclusive │ +// │ │ Exclusive │ │ │ │ +// │ │ ┌─────────┐ │ │ job_0 │ │ +// │ │ │ job_1 │ │ │ │ │ +// │ │ │ │ │ │ │ │ +// │ │ └─────────┘ │ │ │ │ +// │ └───────────────────┘ │ │ │ +// │ ┌───────────────────┐ │ │ │ +// │ │ virtual_cluster_2 │ │ │ │ +// │ │ Mixed │ │ │ │ +// │ │ │ │ │ │ +// │ │ │ │ │ │ +// │ └───────────────────┘ └─────────┘ │ +// └───────────────────────────────────────────────────┘ +class FailoverTest : public PrimaryClusterTest { + public: + using PrimaryClusterTest::PrimaryClusterTest; + + std::shared_ptr InitVirtualClusters(size_t node_count, + size_t template_count) { + auto primary_cluster = InitPrimaryCluster(node_count, template_count); + + RAY_CHECK_OK(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_1_, + {{template_id_0_, 5}, {template_id_1_, 5}})); + + RAY_CHECK_OK(CreateVirtualCluster(primary_cluster, + virtual_cluster_id_2_, + {{template_id_0_, 5}, {template_id_1_, 5}}, + /*divisible=*/false)); + + job_cluster_id_0_ = primary_cluster->BuildJobClusterID("job_0"); + RAY_CHECK_OK(primary_cluster->CreateJobCluster( + job_cluster_id_0_, + {{template_id_0_, 4}, {template_id_1_, 4}}, + [this](const Status &status, auto data, const auto *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + })); + + auto virtual_cluster_1 = std::dynamic_pointer_cast( + primary_cluster->GetLogicalCluster(virtual_cluster_id_1_)); + RAY_CHECK(virtual_cluster_1 != nullptr); + + job_cluster_id_1_ = virtual_cluster_1->BuildJobClusterID("job_1"); + RAY_CHECK_OK(virtual_cluster_1->CreateJobCluster( + job_cluster_id_1_, + {{template_id_0_, 4}, {template_id_1_, 4}}, + [this](const Status &status, auto data, const auto *replica_sets_to_recommend) { + ASSERT_TRUE(status.ok()); + })); + primary_cluster_ = primary_cluster; + return primary_cluster; + } + + NodeID SelectFirstNode(const std::string &virtual_cluster_id, + const std::string &template_id) { + if (auto virtual_cluster = primary_cluster_->GetVirtualCluster(virtual_cluster_id)) { + ReplicaSets replica_sets{{template_id, 1}}; + ReplicaInstances replica_instances; + if (virtual_cluster->LookupUndividedNodeInstances( + replica_sets, replica_instances, nullptr)) { + return NodeID::FromHex( + replica_instances.at(template_id).at(kUndividedClusterId).begin()->first); + } + } + return NodeID::Nil(); + } + + std::string template_id_0_ = "0"; + std::string template_id_1_ = "1"; + std::string job_cluster_id_0_; + std::string job_cluster_id_1_; + std::string virtual_cluster_id_1_ = "virtual_cluster_id_1"; + std::string virtual_cluster_id_2_ = "virtual_cluster_id_2"; + std::shared_ptr primary_cluster_; +}; + +TEST_F(FailoverTest, FailoverNormal) { + size_t node_count = 40; + size_t template_count = 2; + auto primary_cluster = InitVirtualClusters(node_count, template_count); + ASSERT_EQ(virtual_clusters_data_.size(), 4); + + // Mock a gcs_init_data. + instrumented_io_context io_service; + gcs::InMemoryGcsTableStorage gcs_table_storage(io_service); + MockGcsInitData gcs_init_data(gcs_table_storage); + gcs_init_data.SetNodes(nodes_); + gcs_init_data.SetVirtualClusters(virtual_clusters_data_); + + // Failover to a new primary cluster. + auto new_primary_cluster = std::make_shared( + [this](auto data, auto callback) { + callback(Status::OK(), data, nullptr); + return Status::OK(); + }, + cluster_resource_manager_); + new_primary_cluster->Initialize(gcs_init_data); + + // Check the visible node instances and replica sets of the primary cluster are the + // same. + ASSERT_TRUE(new_primary_cluster->GetVisibleNodeInstances() == + primary_cluster->GetVisibleNodeInstances()); + ASSERT_TRUE(new_primary_cluster->GetReplicaSets() == primary_cluster->GetReplicaSets()); + + // Check the visible node instances and replica sets of the virtual clusters are the + // same. + primary_cluster->ForeachVirtualCluster( + [this, new_primary_cluster](const auto &virtual_cluster) { + auto new_virtual_cluster = + new_primary_cluster->GetVirtualCluster(virtual_cluster->GetID()); + ASSERT_TRUE(new_virtual_cluster != nullptr); + ASSERT_TRUE(new_virtual_cluster->GetVisibleNodeInstances() == + virtual_cluster->GetVisibleNodeInstances()); + ASSERT_TRUE(new_virtual_cluster->GetReplicaSets() == + virtual_cluster->GetReplicaSets()); + }); +} + +TEST_F(FailoverTest, FailoverWithDeadNodes) { + size_t node_count = 40; + size_t template_count = 2; + auto primary_cluster = InitVirtualClusters(node_count, template_count); + ASSERT_EQ(virtual_clusters_data_.size(), 4); + + std::vector dead_node_ids; + // Mock template_id_1_'s nodes dead in job_cluster_1 and job_cluster_2. + dead_node_ids.emplace_back(SelectFirstNode(job_cluster_id_0_, template_id_0_)); + dead_node_ids.emplace_back(SelectFirstNode(job_cluster_id_1_, template_id_0_)); + // Mock template_id_1_'s nodes dead in virtual_cluster_1 and virtual_cluster_2. + dead_node_ids.emplace_back(SelectFirstNode(virtual_cluster_id_1_, template_id_1_)); + dead_node_ids.emplace_back(SelectFirstNode(virtual_cluster_id_2_, template_id_1_)); + // Erase the dead nodes. + for (const auto &dead_node_id : dead_node_ids) { + const auto &dead_node = nodes_.at(dead_node_id); + primary_cluster->OnNodeDead(*dead_node); + nodes_.erase(dead_node_id); + } + + auto default_data_flusher = [this](auto data, auto callback) { + callback(Status::OK(), data, nullptr); + return Status::OK(); + }; + + { + // Mock a gcs_init_data. + instrumented_io_context io_service; + gcs::InMemoryGcsTableStorage gcs_table_storage(io_service); + MockGcsInitData gcs_init_data(gcs_table_storage); + gcs_init_data.SetNodes(nodes_); + gcs_init_data.SetVirtualClusters(virtual_clusters_data_); + + // Failover to a new primary cluster. + auto new_primary_cluster = + std::make_shared(default_data_flusher, cluster_resource_manager_); + new_primary_cluster->Initialize(gcs_init_data); + + // Check the visible node instances and replica sets of the primary cluster are the + // same. + ASSERT_TRUE(new_primary_cluster->GetVisibleNodeInstances() == + primary_cluster->GetVisibleNodeInstances()); + ASSERT_TRUE(new_primary_cluster->GetReplicaSets() == + primary_cluster->GetReplicaSets()); + + // Check the visible node instances and replica sets of the virtual clusters are the + // same. + primary_cluster->ForeachVirtualCluster( + [this, new_primary_cluster](const auto &logical_cluster) { + auto new_virtual_cluster = + new_primary_cluster->GetVirtualCluster(logical_cluster->GetID()); + ASSERT_TRUE(new_virtual_cluster != nullptr); + ASSERT_TRUE(new_virtual_cluster->GetVisibleNodeInstances() == + logical_cluster->GetVisibleNodeInstances()); + ASSERT_TRUE(new_virtual_cluster->GetReplicaSets() == + logical_cluster->GetReplicaSets()); + }); + } + + { + auto virtual_cluster_1 = std::dynamic_pointer_cast( + primary_cluster->GetVirtualCluster(virtual_cluster_id_1_)); + ASSERT_TRUE(virtual_cluster_1 != nullptr); + + // Assume that the dead nodes in job cluster 1 is replaced by a new alive one from + // primary cluster. + auto node_instance_replenish_callback = [primary_cluster](auto node_instance) { + return primary_cluster->ReplenishUndividedNodeInstance(std::move(node_instance)); + }; + ASSERT_TRUE( + virtual_cluster_1->ReplenishNodeInstances(node_instance_replenish_callback)); + ASSERT_TRUE(async_data_flusher_(virtual_cluster_1->ToProto(), nullptr).ok()); + + // Mock a gcs_init_data. + instrumented_io_context io_service; + gcs::InMemoryGcsTableStorage gcs_table_storage(io_service); + MockGcsInitData gcs_init_data(gcs_table_storage); + gcs_init_data.SetNodes(nodes_); + gcs_init_data.SetVirtualClusters(virtual_clusters_data_); + + // Failover to a new primary cluster. + auto new_primary_cluster = + std::make_shared(default_data_flusher, cluster_resource_manager_); + new_primary_cluster->Initialize(gcs_init_data); + + // Check the visible node instances and replica sets of the new primary cluster are + // less than the old primary cluster. + ASSERT_TRUE(new_primary_cluster->GetVisibleNodeInstances() <= + primary_cluster->GetVisibleNodeInstances()); + ASSERT_TRUE(new_primary_cluster->GetReplicaSets() <= + primary_cluster->GetReplicaSets()); + + // Check the visible node instances and replica sets of the virtual clusters are the + // same. + primary_cluster->ForeachVirtualCluster( + [this, new_primary_cluster](const auto &virtual_cluster) { + auto new_virtual_cluster = + new_primary_cluster->GetVirtualCluster(virtual_cluster->GetID()); + ASSERT_TRUE(new_virtual_cluster != nullptr); + ASSERT_TRUE(new_virtual_cluster->GetVisibleNodeInstances() == + virtual_cluster->GetVisibleNodeInstances()); + ASSERT_TRUE(new_virtual_cluster->GetReplicaSets() == + virtual_cluster->GetReplicaSets()); + }); + } +} + +TEST_F(FailoverTest, OnlyFlushJobClusters) { + size_t node_count = 40; + size_t template_count = 2; + auto primary_cluster = InitVirtualClusters(node_count, template_count); + ASSERT_EQ(virtual_clusters_data_.size(), 4); + + std::vector dead_node_ids; + // Mock template_id_1_'s nodes dead in job_cluster_1 and job_cluster_2. + dead_node_ids.emplace_back(SelectFirstNode(job_cluster_id_0_, template_id_0_)); + dead_node_ids.emplace_back(SelectFirstNode(job_cluster_id_1_, template_id_0_)); + // Mock template_id_1_'s nodes dead in virtual_cluster_1 and virtual_cluster_2. + dead_node_ids.emplace_back(SelectFirstNode(virtual_cluster_id_1_, template_id_1_)); + dead_node_ids.emplace_back(SelectFirstNode(virtual_cluster_id_2_, template_id_1_)); + // Erase the dead nodes. + for (const auto &dead_node_id : dead_node_ids) { + const auto &dead_node = nodes_.at(dead_node_id); + primary_cluster->OnNodeDead(*dead_node); + nodes_.erase(dead_node_id); + } + + auto default_data_flusher = [this](auto data, auto callback) { + callback(Status::OK(), data, nullptr); + return Status::OK(); + }; + + { + auto virtual_cluster_1 = std::dynamic_pointer_cast( + primary_cluster->GetVirtualCluster(virtual_cluster_id_1_)); + ASSERT_TRUE(virtual_cluster_1 != nullptr); + + // Assume that the dead nodes in job cluster 1 is replaced by a new alive one from + // primary cluster. + auto node_instance_replenish_callback = [primary_cluster](auto node_instance) { + return primary_cluster->ReplenishUndividedNodeInstance(std::move(node_instance)); + }; + ASSERT_TRUE( + virtual_cluster_1->ReplenishNodeInstances(node_instance_replenish_callback)); + // async_data_flusher_(virtual_cluster_1->ToProto(), nullptr); + + // Mock a gcs_init_data. + instrumented_io_context io_service; + gcs::InMemoryGcsTableStorage gcs_table_storage(io_service); + MockGcsInitData gcs_init_data(gcs_table_storage); + gcs_init_data.SetNodes(nodes_); + gcs_init_data.SetVirtualClusters(virtual_clusters_data_); + + RAY_LOG(INFO) << "Initialize"; + // Failover to a new primary cluster. + auto new_primary_cluster = + std::make_shared(default_data_flusher, cluster_resource_manager_); + new_primary_cluster->Initialize(gcs_init_data); + + auto new_job_cluster_1 = new_primary_cluster->GetVirtualCluster(job_cluster_id_1_); + ASSERT_TRUE(new_job_cluster_1 != nullptr); + // The dead node dead_node_ids[1] in job cluster 1 is replenished and the job cluster + // 1 is flushed. + ASSERT_FALSE(new_job_cluster_1->ContainsNodeInstance(dead_node_ids[1].Hex())); + + auto new_virtual_cluster_1 = + new_primary_cluster->GetVirtualCluster(virtual_cluster_id_1_); + ASSERT_TRUE(new_virtual_cluster_1 != nullptr); + // The dead node dead_node_ids[1] is repaired in virtual cluster 1 when fo. + ASSERT_FALSE(new_virtual_cluster_1->ContainsNodeInstance(dead_node_ids[1].Hex())); + // The dead node dead_node_ids[2] is still in the virtual cluster 1 as it is not + // flushed. + ASSERT_TRUE(new_virtual_cluster_1->ContainsNodeInstance(dead_node_ids[2].Hex())); + + // Check all the node instances in job cluster 1 are in the new virtual cluster 1. + new_job_cluster_1->ForeachNodeInstance( + [new_virtual_cluster_1](const auto &node_instance) { + ASSERT_TRUE(new_virtual_cluster_1->ContainsNodeInstance( + node_instance->node_instance_id())); + }); + } +} + +} // namespace gcs +} // namespace ray diff --git a/src/ray/gcs/pubsub/gcs_pub_sub.ant.cc b/src/ray/gcs/pubsub/gcs_pub_sub.ant.cc new file mode 100644 index 000000000000..c0b974e4da94 --- /dev/null +++ b/src/ray/gcs/pubsub/gcs_pub_sub.ant.cc @@ -0,0 +1,63 @@ +// Copyright 2017 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/gcs/pubsub/gcs_pub_sub.h" + +namespace ray { +namespace gcs { + +Status GcsPublisher::PublishVirtualCluster(const VirtualClusterID &id, + const rpc::VirtualClusterTableData &message, + const StatusCallback &done) { + rpc::PubMessage msg; + msg.set_channel_type(rpc::ChannelType::RAY_VIRTUAL_CLUSTER_CHANNEL); + msg.set_key_id(id.Binary()); + *msg.mutable_virtual_cluster_message() = message; + publisher_->Publish(msg); + if (done != nullptr) { + done(Status::OK()); + } + return Status::OK(); +} + +Status GcsSubscriber::SubscribeAllVirtualClusters( + const SubscribeCallback &subscribe, + const StatusCallback &done) { + // GCS subscriber. + auto subscribe_item_callback = [subscribe](rpc::PubMessage &&msg) { + RAY_CHECK(msg.channel_type() == rpc::ChannelType::RAY_VIRTUAL_CLUSTER_CHANNEL); + const VirtualClusterID id = VirtualClusterID::FromBinary(msg.key_id()); + subscribe(id, std::move(*msg.mutable_virtual_cluster_message())); + }; + auto subscription_failure_callback = [](const std::string &, const Status &status) { + RAY_LOG(WARNING) << "Subscription to virtual cluster channel failed: " + << status.ToString(); + }; + RAY_UNUSED(subscriber_->SubscribeChannel( + std::make_unique(), + rpc::ChannelType::RAY_VIRTUAL_CLUSTER_CHANNEL, + gcs_address_, + [done](const Status &status) { + if (done != nullptr) { + done(status); + } + }, + std::move(subscribe_item_callback), + std::move(subscription_failure_callback))); + + return Status::OK(); +} + +} // namespace gcs +} // namespace ray diff --git a/src/ray/gcs/pubsub/gcs_pub_sub.h b/src/ray/gcs/pubsub/gcs_pub_sub.h index 0b80b9278338..557687bee44d 100644 --- a/src/ray/gcs/pubsub/gcs_pub_sub.h +++ b/src/ray/gcs/pubsub/gcs_pub_sub.h @@ -21,6 +21,7 @@ #include "absl/container/flat_hash_map.h" #include "absl/synchronization/mutex.h" #include "ray/common/ray_config.h" +#include "ray/common/virtual_cluster_id.h" #include "ray/gcs/callback.h" #include "ray/pubsub/publisher.h" #include "ray/pubsub/subscriber.h" @@ -86,6 +87,10 @@ class GcsPublisher { Status PublishResourceBatch(const rpc::ResourceUsageBatchData &message, const StatusCallback &done); + Status PublishVirtualCluster(const VirtualClusterID &id, + const rpc::VirtualClusterTableData &message, + const StatusCallback &done); + /// Prints debugging info for the publisher. std::string DebugString() const; @@ -126,6 +131,10 @@ class GcsSubscriber { Status SubscribeAllWorkerFailures(const ItemCallback &subscribe, const StatusCallback &done); + Status SubscribeAllVirtualClusters( + const SubscribeCallback &subscribe, + const StatusCallback &done); + /// Prints debugging info for the subscriber. std::string DebugString() const; diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index 037ac47716d2..240685f1d859 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -120,6 +120,8 @@ message SchedulingStrategy { NodeAffinitySchedulingStrategy node_affinity_scheduling_strategy = 4; NodeLabelSchedulingStrategy node_label_scheduling_strategy = 5; } + // Hold virtual cluster-related information, enabling its use in scheduling logic. + string virtual_cluster_id = 11; } // Address of a worker or node manager. @@ -636,6 +638,8 @@ message PlacementGroupSpec { // Otherwise, the bundles can be placed elsewhere. // This only applies to STRICT_PACK pg. bytes soft_target_node_id = 11; + // virtual cluster this placement group belongs to. + string virtual_cluster_id = 12; } message ObjectReference { diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index 668626213689..6b98e18e5692 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -42,6 +42,7 @@ enum TablePrefix { PLACEMENT_GROUP = 17; KV = 18; ACTOR_TASK_SPEC = 19; + VIRTUAL_CLUSTER = 20; } // The channel that Add operations to the Table should be published on, if any. @@ -677,6 +678,8 @@ message PlacementGroupTableData { // The time that the last bundle of a placement group is committed at, // effectively when the placement group has been "scheduled". int64 placement_group_final_bundle_placement_timestamp_ms = 16; + // The virtual cluster this placement group belongs to. + string virtual_cluster_id = 17; } message JobTableData { @@ -706,5 +709,27 @@ message JobTableData { optional bool is_running_tasks = 11; // Address of the driver that started this job. Address driver_address = 12; + // The virtual cluster this job belongs to. + string virtual_cluster_id = 13; } /////////////////////////////////////////////////////////////////////////////// + +message NodeInstance { + // The Hostname address of the node instance. + string hostname = 1; + // The template id of the node instance. + string template_id = 2; +} + +message VirtualClusterTableData { + // The virtual cluster id. + string id = 1; + // Whether the virtual cluster can splite into many child virtual clusters or not. + bool divisible = 2; + // Mapping from node id to it's instance. + map node_instances = 3; + // Whether this virtual cluster is removed. + bool is_removed = 4; + // Version number of the last modification to the virtual cluster. + uint64 revision = 5; +} diff --git a/src/ray/protobuf/gcs_service.proto b/src/ray/protobuf/gcs_service.proto index 790253cc0326..4af85644cc58 100644 --- a/src/ray/protobuf/gcs_service.proto +++ b/src/ray/protobuf/gcs_service.proto @@ -229,6 +229,7 @@ message GetAllNodeInfoRequest { optional bytes node_id = 1; optional GcsNodeInfo.GcsNodeState state = 2; optional string node_name = 3; + optional string virtual_cluster_id = 4; } // Maximum number to return. @@ -270,14 +271,18 @@ message DeleteResourcesReply { GcsStatus status = 1; } -message GetAllAvailableResourcesRequest {} +message GetAllAvailableResourcesRequest { + string virtual_cluster_id = 1; +} message GetAllAvailableResourcesReply { GcsStatus status = 1; repeated AvailableResources resources_list = 2; } -message GetAllTotalResourcesRequest {} +message GetAllTotalResourcesRequest { + string virtual_cluster_id = 1; +} message GetAllTotalResourcesReply { GcsStatus status = 1; @@ -863,3 +868,123 @@ service TaskInfoGcsService { } /////////////////////////////////////////////////////////////////////////////// + +message CreateOrUpdateVirtualClusterRequest { + // The virtual cluster id. + string virtual_cluster_id = 1; + // Whether the virtual cluster can splite into many child virtual clusters or not. + bool divisible = 2; + // The replica set of the virtual cluster. + map replica_sets = 3; + // Version number of the last modification to the virtual cluster. + uint64 revision = 4; +} + +message CreateOrUpdateVirtualClusterReply { + GcsStatus status = 1; + // Version number of the last modification to the virtual cluster. + uint64 revision = 2; + // Mapping from node id to it's instance. + map node_instances = 3; + // The replica sets that gcs can fulfill at most to meet the reqest. + // This field is only used when the request failed and serving as a suggestion + // to adjust the requst or cluster. + map replica_sets_to_recommend = 4; +} + +message RemoveVirtualClusterRequest { + // ID of the virtual cluster to be removed. + string virtual_cluster_id = 1; +} + +message RemoveVirtualClusterReply { + GcsStatus status = 1; +} + +message NodeInstanceView { + // The Hostname address of the node instance. + string hostname = 1; + // The template id of the node instance. + string template_id = 2; + // Whether the node is dead + bool is_dead = 3; + // Total resources of the node. + map resources_total = 4; + // Available resources of the node. + map resources_available = 5; +} + +message VirtualClusterView { + // The virtual cluster id. + string id = 1; + // Whether the virtual cluster can splite into many child virtual clusters or not. + bool divisible = 2; + // Mapping from node id to it's instance. + map node_instance_views = 3; + // Version number of the last modification to the virtual cluster. + uint64 revision = 4; +} + +message GetAllVirtualClusterInfoRequest { + // Maximum number to return. + // If set, the exact `limit` returned do not have any ordering or selection + // guarantee. + optional int64 limit = 1; +} + +message GetAllVirtualClusterInfoReply { + GcsStatus status = 1; + repeated VirtualClusterView virtual_clusters_view = 2; + // Length of the corresponding resource without truncation. + int64 total = 3; +} + +message GetVirtualClustersRequest { + // ID of the virtual cluster to get. + // It will reply all clusters data if virtual_cluster_id is empty. + string virtual_cluster_id = 1; + // Wether include job clusters. + bool include_job_clusters = 2; + // It will reply indivisible clusters if only_include_indivisible_clusters is true. + bool only_include_indivisible_clusters = 3; +} + +message GetVirtualClustersReply { + GcsStatus status = 1; + repeated VirtualClusterTableData virtual_cluster_data_list = 2; +} + +message CreateJobClusterRequest { + // The job id. + string job_id = 1; + // ID of the virtual cluster that the job belongs to. + string virtual_cluster_id = 2; + // The replica set of the job cluster. + map replica_sets = 3; +} + +message CreateJobClusterReply { + GcsStatus status = 1; + // The job cluster id. + string job_cluster_id = 2; + // The replica sets that gcs can fulfill at most to meet the reqest. + // This field is only used when the request failed and serving as a suggestion + // to adjust the requst or cluster. + map replica_sets_to_recommend = 3; +} + +service VirtualClusterInfoGcsService { + // Create or update a virtual cluster. + rpc CreateOrUpdateVirtualCluster(CreateOrUpdateVirtualClusterRequest) + returns (CreateOrUpdateVirtualClusterReply); + // Remove a virtual cluster. + rpc RemoveVirtualCluster(RemoveVirtualClusterRequest) + returns (RemoveVirtualClusterReply); + // Get virtual clusters. + rpc GetVirtualClusters(GetVirtualClustersRequest) returns (GetVirtualClustersReply); + // Create job cluster. + rpc CreateJobCluster(CreateJobClusterRequest) returns (CreateJobClusterReply); + // Get all virtual clusters info for display. + rpc GetAllVirtualClusterInfo(GetAllVirtualClusterInfoRequest) + returns (GetAllVirtualClusterInfoReply); +} diff --git a/src/ray/protobuf/pubsub.proto b/src/ray/protobuf/pubsub.proto index 89d3ab55a76e..9bb1ed5c188c 100644 --- a/src/ray/protobuf/pubsub.proto +++ b/src/ray/protobuf/pubsub.proto @@ -45,6 +45,8 @@ enum ChannelType { RAY_LOG_CHANNEL = 8; /// A channel for reporting node resource usage stats. RAY_NODE_RESOURCE_USAGE_CHANNEL = 10; + /// A channel for virtual cluster changes. + RAY_VIRTUAL_CLUSTER_CHANNEL = 11; } /// @@ -69,6 +71,7 @@ message PubMessage { ErrorTableData error_info_message = 12; LogBatch log_batch_message = 13; NodeResourceUsage node_resource_usage_message = 15; + VirtualClusterTableData virtual_cluster_message = 20; // The message that indicates the given key id is not available anymore. FailureMessage failure_message = 6; diff --git a/src/ray/protobuf/usage.proto b/src/ray/protobuf/usage.proto index 7e615d3844b9..bb4b7c63e2a2 100644 --- a/src/ray/protobuf/usage.proto +++ b/src/ray/protobuf/usage.proto @@ -116,6 +116,7 @@ enum TagKey { CORE_STATE_API_SUMMARIZE_TASKS = 111; CORE_STATE_API_SUMMARIZE_ACTORS = 112; CORE_STATE_API_SUMMARIZE_OBJECTS = 113; + CORE_STATE_API_LIST_VCLUSTERS = 114; // Dashboard // {True, False} diff --git a/src/ray/pubsub/publisher.cc b/src/ray/pubsub/publisher.cc index 444de5da9fbd..f05cb136c5b1 100644 --- a/src/ray/pubsub/publisher.cc +++ b/src/ray/pubsub/publisher.cc @@ -263,6 +263,7 @@ std::unique_ptr SubscriptionIndex::CreateEntityState( case rpc::ChannelType::GCS_JOB_CHANNEL: case rpc::ChannelType::GCS_NODE_INFO_CHANNEL: case rpc::ChannelType::GCS_WORKER_DELTA_CHANNEL: + case rpc::ChannelType::RAY_VIRTUAL_CLUSTER_CHANNEL: return std::make_unique(RayConfig::instance().max_grpc_message_size(), /*max_buffered_bytes=*/-1); diff --git a/src/ray/raylet/format/node_manager.fbs b/src/ray/raylet/format/node_manager.fbs index 3ba28a3ff021..5ff7b8cfa5c2 100644 --- a/src/ray/raylet/format/node_manager.fbs +++ b/src/ray/raylet/format/node_manager.fbs @@ -171,6 +171,8 @@ table AnnounceWorkerPort { port: int; // The entrypoint of the job. Only populated if the worker is a driver. entrypoint: string; + // The virtual cluster this job belongs to. + virtual_cluster_id: string; } table AnnounceWorkerPortReply { diff --git a/src/ray/raylet/node_manager.ant.cc b/src/ray/raylet/node_manager.ant.cc new file mode 100644 index 000000000000..bc52ea6e53b5 --- /dev/null +++ b/src/ray/raylet/node_manager.ant.cc @@ -0,0 +1,74 @@ +// Copyright 2017 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/raylet/node_manager.h" + +namespace ray { + +namespace raylet { + +void NodeManager::CancelMismatchedLocalTasks( + const std::string &local_virtual_cluster_id) { + bool skip_actor = RayConfig::instance().gcs_actor_scheduling_enabled(); + auto predicate = [&local_virtual_cluster_id, + skip_actor](const std::shared_ptr &work) { + if (skip_actor && work->task.GetTaskSpecification().IsActorCreationTask()) { + return false; + } + if (work->task.GetTaskSpecification().GetSchedulingStrategy().virtual_cluster_id() != + local_virtual_cluster_id) { + return true; + } + return false; + }; + auto tasks_canceled = + cluster_task_manager_->CancelTasks(predicate, + rpc::RequestWorkerLeaseReply::SCHEDULING_FAILED, + "The node is removed from a virtual cluster."); + if (tasks_canceled) { + RAY_LOG(INFO) << "Tasks are cleaned up from cluster_task_manager because the " + "node is removed from virtual cluster."; + } + tasks_canceled = + local_task_manager_->CancelTasks(predicate, + rpc::RequestWorkerLeaseReply::SCHEDULING_FAILED, + "The node is removed from a virtual cluster."); + if (tasks_canceled) { + RAY_LOG(INFO) << "Tasks are cleaned up from local_task_manager because the " + "node is removed from virtual cluster."; + } + if (!cluster_resource_scheduler_->GetLocalResourceManager().IsLocalNodeIdle()) { + for (auto iter = leased_workers_.begin(); iter != leased_workers_.end();) { + auto curr_iter = iter++; + auto worker = curr_iter->second; + const auto &task_spec = worker->GetAssignedTask().GetTaskSpecification(); + if (skip_actor && task_spec.IsActorCreationTask()) { + continue; + } + if (task_spec.GetSchedulingStrategy().virtual_cluster_id() != + local_virtual_cluster_id) { + RAY_LOG(INFO).WithField(worker->WorkerId()) + << "Worker is cleaned because the node is removed from virtual cluster."; + DestroyWorker( + worker, + rpc::WorkerExitType::INTENDED_SYSTEM_EXIT, + "Worker is cleaned because the node is removed from virtual cluster."); + } + } + } +} + +} // namespace raylet + +} // namespace ray diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 2937f419d740..6ecfd1f83f26 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -320,7 +320,19 @@ NodeManager::NodeManager( [this]() { return object_manager_.PullManagerHasPullsQueued(); }, shutdown_raylet_gracefully, /*labels*/ - config.labels); + config.labels, + /*is_node_schedulable_fn=*/ + [this](scheduling::NodeID node_id, const SchedulingContext *context) { + if (virtual_cluster_manager_ == nullptr) { + return true; + } + if (context->virtual_cluster_id.empty()) { + return true; + } + auto node_instance_id = NodeID::FromBinary(node_id.Binary()); + return virtual_cluster_manager_->ContainsNodeInstance(context->virtual_cluster_id, + node_instance_id); + }); auto get_node_info_func = [this](const NodeID &node_id) { return gcs_client_->Nodes().Get(node_id); @@ -403,6 +415,18 @@ NodeManager::NodeManager( mutable_object_provider_ = std::make_unique( *store_client_, absl::bind_front(&NodeManager::CreateRayletClient, this), nullptr); + + virtual_cluster_manager_ = std::make_shared( + self_node_id_, /*local_node_cleanup_fn=*/[this]() { + auto timer = std::make_shared( + io_service_, + boost::posix_time::milliseconds( + RayConfig::instance().local_node_cleanup_delay_interval_ms())); + timer->async_wait([this, timer](const boost::system::error_code e) mutable { + CancelMismatchedLocalTasks( + virtual_cluster_manager_->GetLocalVirtualClusterID()); + }); + }); } std::shared_ptr NodeManager::CreateRayletClient( @@ -499,6 +523,18 @@ ray::Status NodeManager::RegisterGcs() { RAY_RETURN_NOT_OK( gcs_client_->Jobs().AsyncSubscribeAll(job_subscribe_handler, nullptr)); + // Subscribe to all virtual clusrter update notification. + const auto virtual_cluster_update_notification_handler = + [this](const VirtualClusterID &virtual_cluster_id, + rpc::VirtualClusterTableData &&virtual_cluster_data) { + virtual_cluster_manager_->UpdateVirtualCluster(std::move(virtual_cluster_data)); + }; + RAY_RETURN_NOT_OK(gcs_client_->VirtualCluster().AsyncSubscribeAll( + virtual_cluster_update_notification_handler, [](const ray::Status &status) { + RAY_CHECK_OK(status); + RAY_LOG(INFO) << "Finished subscribing all virtual cluster infos."; + })); + periodical_runner_->RunFnPeriodically( [this] { DumpDebugState(); @@ -1440,6 +1476,9 @@ void NodeManager::ProcessAnnounceWorkerPortMessageImpl( string_from_flatbuf(*message->entrypoint()), *job_config); + job_data_ptr->set_virtual_cluster_id( + string_from_flatbuf(*message->virtual_cluster_id())); + RAY_CHECK_OK( gcs_client_->Jobs().AsyncAdd(job_data_ptr, [this, client](Status status) { SendPortAnnouncementResponse(client, std::move(status)); diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 0458ee2d3198..2d955119ddee 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -35,6 +35,7 @@ #include "ray/raylet/runtime_env_agent_client.h" #include "ray/raylet/scheduling/cluster_resource_scheduler.h" #include "ray/raylet/scheduling/cluster_task_manager_interface.h" +#include "ray/raylet/virtual_cluster_manager.h" #include "ray/raylet/wait_manager.h" #include "ray/raylet/worker_killing_policy.h" #include "ray/raylet/worker_pool.h" @@ -745,6 +746,13 @@ class NodeManager : public rpc::NodeManagerServiceHandler, std::shared_ptr CreateRayletClient( const NodeID &node_id, rpc::ClientCallManager &client_call_manager); + /// Clean up the local (pending and running) tasks that have mismatched + /// virtual cluster id against the one to which the local node currently belongs. + /// + /// \param local_virtual_cluster_id The ID of the virtual cluster to which the local + /// node currently belongs. \return void + void CancelMismatchedLocalTasks(const std::string &local_virtual_cluster_id); + /// ID of this node. NodeID self_node_id_; /// The user-given identifier or name of this node. @@ -918,6 +926,9 @@ class NodeManager : public rpc::NodeManagerServiceHandler, std::unique_ptr memory_monitor_; std::unique_ptr mutable_object_provider_; + + /// The virtual cluster manager. + std::shared_ptr virtual_cluster_manager_; }; } // namespace raylet diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc index 48470991d45f..9b2245e8a4ad 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.cc +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.cc @@ -28,10 +28,13 @@ ClusterResourceScheduler::ClusterResourceScheduler( scheduling::NodeID local_node_id, const NodeResources &local_node_resources, std::function is_node_available_fn, - bool is_local_node_with_raylet) + bool is_local_node_with_raylet, + std::function + is_node_schedulable_fn) : local_node_id_(local_node_id), is_node_available_fn_(is_node_available_fn), - is_local_node_with_raylet_(is_local_node_with_raylet) { + is_local_node_with_raylet_(is_local_node_with_raylet), + is_node_schedulable_fn_(is_node_schedulable_fn) { Init(io_service, local_node_resources, /*get_used_object_store_memory=*/nullptr, @@ -47,8 +50,12 @@ ClusterResourceScheduler::ClusterResourceScheduler( std::function get_used_object_store_memory, std::function get_pull_manager_at_capacity, std::function shutdown_raylet_gracefully, - const absl::flat_hash_map &local_node_labels) - : local_node_id_(local_node_id), is_node_available_fn_(is_node_available_fn) { + const absl::flat_hash_map &local_node_labels, + std::function + is_node_schedulable_fn) + : local_node_id_(local_node_id), + is_node_available_fn_(is_node_available_fn), + is_node_schedulable_fn_(is_node_schedulable_fn) { NodeResources node_resources = ResourceMapToNodeResources( local_node_resources, local_node_resources, local_node_labels); Init(io_service, @@ -81,12 +88,16 @@ void ClusterResourceScheduler::Init( local_node_id_, *cluster_resource_manager_, /*is_node_available_fn*/ - [this](auto node_id) { return this->NodeAvailable(node_id); }); + [this](auto node_id) { return this->NodeAvailable(node_id); }, + /*is_node_schedulable*/ + is_node_schedulable_fn_); bundle_scheduling_policy_ = std::make_unique( *cluster_resource_manager_, /*is_node_available_fn*/ - [this](auto node_id) { return this->NodeAvailable(node_id); }); + [this](auto node_id) { return this->NodeAvailable(node_id); }, + /*is_node_schedulable_fn*/ + is_node_schedulable_fn_); } bool ClusterResourceScheduler::NodeAvailable(scheduling::NodeID node_id) const { @@ -151,11 +162,13 @@ scheduling::NodeID ClusterResourceScheduler::GetBestSchedulableNode( const std::string &preferred_node_id, int64_t *total_violations, bool *is_infeasible) { + const std::string &virtual_cluster_id = scheduling_strategy.virtual_cluster_id(); // The zero cpu actor is a special case that must be handled the same way by all // scheduling policies, except for HARD node affnity scheduling policy. if (actor_creation && resource_request.IsEmpty() && !IsHardNodeAffinitySchedulingStrategy(scheduling_strategy)) { - return scheduling_policy_->Schedule(resource_request, SchedulingOptions::Random()); + return scheduling_policy_->Schedule(resource_request, + SchedulingOptions::Random(virtual_cluster_id)); } auto best_node_id = scheduling::NodeID::Nil(); @@ -165,7 +178,8 @@ scheduling::NodeID ClusterResourceScheduler::GetBestSchedulableNode( scheduling_policy_->Schedule(resource_request, SchedulingOptions::Spread( /*avoid_local_node*/ force_spillback, - /*require_node_available*/ force_spillback)); + /*require_node_available*/ force_spillback, + /*virtual_cluster_id*/ virtual_cluster_id)); } else if (scheduling_strategy.scheduling_strategy_case() == rpc::SchedulingStrategy::SchedulingStrategyCase:: kNodeAffinitySchedulingStrategy) { @@ -178,8 +192,8 @@ scheduling::NodeID ClusterResourceScheduler::GetBestSchedulableNode( scheduling_strategy.node_affinity_scheduling_strategy().soft(), scheduling_strategy.node_affinity_scheduling_strategy() .spill_on_unavailable(), - scheduling_strategy.node_affinity_scheduling_strategy() - .fail_on_unavailable())); + scheduling_strategy.node_affinity_scheduling_strategy().fail_on_unavailable(), + virtual_cluster_id)); } else if (IsAffinityWithBundleSchedule(scheduling_strategy) && !is_local_node_with_raylet_) { // This scheduling strategy is only used for gcs scheduling for the time being. @@ -190,10 +204,12 @@ scheduling::NodeID ClusterResourceScheduler::GetBestSchedulableNode( scheduling_strategy.placement_group_scheduling_strategy() .placement_group_bundle_index()); best_node_id = scheduling_policy_->Schedule( - resource_request, SchedulingOptions::AffinityWithBundle(bundle_id)); + resource_request, + SchedulingOptions::AffinityWithBundle(bundle_id, virtual_cluster_id)); } else if (scheduling_strategy.has_node_label_scheduling_strategy()) { best_node_id = scheduling_policy_->Schedule( - resource_request, SchedulingOptions::NodeLabelScheduling(scheduling_strategy)); + resource_request, + SchedulingOptions::NodeLabelScheduling(scheduling_strategy, virtual_cluster_id)); } else { // TODO (Alex): Setting require_available == force_spillback is a hack in order to // remain bug compatible with the legacy scheduling algorithms. @@ -202,7 +218,8 @@ scheduling::NodeID ClusterResourceScheduler::GetBestSchedulableNode( SchedulingOptions::Hybrid( /*avoid_local_node*/ force_spillback, /*require_node_available*/ force_spillback, - preferred_node_id)); + preferred_node_id, + virtual_cluster_id)); } *is_infeasible = best_node_id.IsNil(); diff --git a/src/ray/raylet/scheduling/cluster_resource_scheduler.h b/src/ray/raylet/scheduling/cluster_resource_scheduler.h index 55b40477a64d..52b881cb47d1 100644 --- a/src/ray/raylet/scheduling/cluster_resource_scheduler.h +++ b/src/ray/raylet/scheduling/cluster_resource_scheduler.h @@ -35,6 +35,7 @@ namespace ray { +using raylet_scheduling_policy::SchedulingContext; using raylet_scheduling_policy::SchedulingOptions; using raylet_scheduling_policy::SchedulingResult; @@ -50,11 +51,14 @@ class ClusterResourceScheduler { /// with the local node. /// \param is_node_available_fn: Function to determine whether a node is available. /// \param is_local_node_with_raylet: Whether there is a raylet on the local node. - ClusterResourceScheduler(instrumented_io_context &io_service, - scheduling::NodeID local_node_id, - const NodeResources &local_node_resources, - std::function is_node_available_fn, - bool is_local_node_with_raylet = true); + ClusterResourceScheduler( + instrumented_io_context &io_service, + scheduling::NodeID local_node_id, + const NodeResources &local_node_resources, + std::function is_node_available_fn, + bool is_local_node_with_raylet = true, + std::function + is_node_schedulable_fn = nullptr); ClusterResourceScheduler( instrumented_io_context &io_service, @@ -65,7 +69,9 @@ class ClusterResourceScheduler { std::function get_pull_manager_at_capacity = nullptr, std::function shutdown_raylet_gracefully = nullptr, - const absl::flat_hash_map &local_node_labels = {}); + const absl::flat_hash_map &local_node_labels = {}, + std::function + is_node_schedulable_fn = nullptr); /// Schedule the specified resources to the cluster nodes. /// @@ -219,6 +225,9 @@ class ClusterResourceScheduler { bundle_scheduling_policy_; /// Whether there is a raylet on the local node. bool is_local_node_with_raylet_ = true; + /// Callback to check if node is schedulable. + std::function + is_node_schedulable_fn_; friend class ClusterResourceSchedulerTest; FRIEND_TEST(ClusterResourceSchedulerTest, PopulatePredefinedResources); diff --git a/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.cc b/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.cc index bfe46d314abc..f8eb2b92529e 100644 --- a/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.cc +++ b/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.cc @@ -20,8 +20,10 @@ namespace raylet_scheduling_policy { bool AffinityWithBundleSchedulingPolicy::IsNodeFeasibleAndAvailable( const scheduling::NodeID &node_id, const ResourceRequest &resource_request, - bool avoid_gpu_nodes) { + bool avoid_gpu_nodes, + const SchedulingContext *scheduling_context) { if (!(nodes_.contains(node_id) && is_node_alive_(node_id) && + is_node_schedulable_(node_id, scheduling_context) && nodes_.at(node_id).GetLocalView().IsFeasible(resource_request) && nodes_.at(node_id).GetLocalView().IsAvailable(resource_request))) { return false; @@ -53,8 +55,10 @@ scheduling::NodeID AffinityWithBundleSchedulingPolicy::Schedule( const auto &node_id_opt = bundle_location_index_.GetBundleLocation(bundle_id); if (node_id_opt) { auto target_node_id = scheduling::NodeID(node_id_opt.value().Binary()); - if (IsNodeFeasibleAndAvailable( - target_node_id, resource_request, /*avoid_gpu_nodes=*/false)) { + if (IsNodeFeasibleAndAvailable(target_node_id, + resource_request, + /*avoid_gpu_nodes=*/false, + options.scheduling_context.get())) { return target_node_id; } } @@ -66,8 +70,10 @@ scheduling::NodeID AffinityWithBundleSchedulingPolicy::Schedule( if (options.avoid_gpu_nodes) { for (const auto &iter : *(bundle_locations_opt.value())) { auto target_node_id = scheduling::NodeID(iter.second.first.Binary()); - if (IsNodeFeasibleAndAvailable( - target_node_id, resource_request, /*avoid_gpu_nodes=*/true)) { + if (IsNodeFeasibleAndAvailable(target_node_id, + resource_request, + /*avoid_gpu_nodes=*/true, + options.scheduling_context.get())) { return target_node_id; } } @@ -75,8 +81,10 @@ scheduling::NodeID AffinityWithBundleSchedulingPolicy::Schedule( // Find a target from all nodes. for (const auto &iter : *(bundle_locations_opt.value())) { auto target_node_id = scheduling::NodeID(iter.second.first.Binary()); - if (IsNodeFeasibleAndAvailable( - target_node_id, resource_request, /*avoid_gpu_nodes=*/false)) { + if (IsNodeFeasibleAndAvailable(target_node_id, + resource_request, + /*avoid_gpu_nodes=*/false, + options.scheduling_context.get())) { return target_node_id; } } diff --git a/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.h b/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.h index 0d8e46774826..8f61a2cd336d 100644 --- a/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/affinity_with_bundle_scheduling_policy.h @@ -27,11 +27,14 @@ class AffinityWithBundleSchedulingPolicy : public ISchedulingPolicy { scheduling::NodeID local_node_id, const absl::flat_hash_map &nodes, std::function is_node_alive, - const BundleLocationIndex &pg_location_index) + const BundleLocationIndex &pg_location_index, + std::function + is_node_schedulable) : local_node_id_(local_node_id), nodes_(nodes), is_node_alive_(is_node_alive), - bundle_location_index_(pg_location_index) {} + bundle_location_index_(pg_location_index), + is_node_schedulable_(is_node_schedulable) {} scheduling::NodeID Schedule(const ResourceRequest &resource_request, SchedulingOptions options) override; @@ -40,11 +43,13 @@ class AffinityWithBundleSchedulingPolicy : public ISchedulingPolicy { const absl::flat_hash_map &nodes_; std::function is_node_alive_; const BundleLocationIndex &bundle_location_index_; + std::function is_node_schedulable_; private: bool IsNodeFeasibleAndAvailable(const scheduling::NodeID &node_id, const ResourceRequest &resource_request, - bool avoid_gpu_nodes); + bool avoid_gpu_nodes, + const SchedulingContext *scheduling_context); }; } // namespace raylet_scheduling_policy } // namespace ray diff --git a/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.cc b/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.cc index 4469ddb87f53..616e56b573dd 100644 --- a/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.cc +++ b/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.cc @@ -110,6 +110,9 @@ BundleSchedulingPolicy::SelectCandidateNodes(const SchedulingContext *context) c RAY_UNUSED(context); absl::flat_hash_map result; for (const auto &entry : cluster_resource_manager_.GetResourceView()) { + if (is_node_schedulable_ != nullptr && !is_node_schedulable_(entry.first, context)) { + continue; + } if (is_node_available_ == nullptr || is_node_available_(entry.first)) { result.emplace(entry.first, &entry.second); } @@ -550,6 +553,10 @@ BundleStrictSpreadSchedulingPolicy::SelectCandidateNodes( absl::flat_hash_map candidate_nodes; for (const auto &entry : cluster_resource_manager_.GetResourceView()) { + if (is_node_schedulable_ && !is_node_schedulable_(entry.first, context)) { + continue; + } + if (is_node_available_ && !is_node_available_(entry.first)) { continue; } diff --git a/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.h b/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.h index 255a11957d70..c8bbc839915b 100644 --- a/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/bundle_scheduling_policy.h @@ -32,9 +32,12 @@ class BundleSchedulingPolicy : public IBundleSchedulingPolicy { public: explicit BundleSchedulingPolicy( ClusterResourceManager &cluster_resource_manager, - std::function is_node_available) + std::function is_node_available, + std::function + is_node_schedulable) : cluster_resource_manager_(cluster_resource_manager), is_node_available_(is_node_available), + is_node_schedulable_(is_node_schedulable), node_scorer_(new LeastResourceScorer()) {} protected: @@ -75,6 +78,8 @@ class BundleSchedulingPolicy : public IBundleSchedulingPolicy { ClusterResourceManager &cluster_resource_manager_; /// Function Checks if node is alive. std::function is_node_available_; + /// Function Checks if node is schedulable. + std::function is_node_schedulable_; /// Scorer to make a grade to the node. std::unique_ptr node_scorer_; }; diff --git a/src/ray/raylet/scheduling/policy/composite_scheduling_policy.h b/src/ray/raylet/scheduling/policy/composite_scheduling_policy.h index d5cf66ae8be3..7d623304d375 100644 --- a/src/ray/raylet/scheduling/policy/composite_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/composite_scheduling_policy.h @@ -32,24 +32,37 @@ namespace raylet_scheduling_policy { /// scheduling_policy according to the scheduling_type. class CompositeSchedulingPolicy : public ISchedulingPolicy { public: - CompositeSchedulingPolicy(scheduling::NodeID local_node_id, - ClusterResourceManager &cluster_resource_manager, - std::function is_node_available) - : hybrid_policy_( - local_node_id, cluster_resource_manager.GetResourceView(), is_node_available), - random_policy_( - local_node_id, cluster_resource_manager.GetResourceView(), is_node_available), - spread_policy_( - local_node_id, cluster_resource_manager.GetResourceView(), is_node_available), - node_affinity_policy_( - local_node_id, cluster_resource_manager.GetResourceView(), is_node_available), + CompositeSchedulingPolicy( + scheduling::NodeID local_node_id, + ClusterResourceManager &cluster_resource_manager, + std::function is_node_available, + std::function + is_node_schedulable) + : hybrid_policy_(local_node_id, + cluster_resource_manager.GetResourceView(), + is_node_available, + is_node_schedulable), + random_policy_(local_node_id, + cluster_resource_manager.GetResourceView(), + is_node_available, + is_node_schedulable), + spread_policy_(local_node_id, + cluster_resource_manager.GetResourceView(), + is_node_available, + is_node_schedulable), + node_affinity_policy_(local_node_id, + cluster_resource_manager.GetResourceView(), + is_node_available, + is_node_schedulable), affinity_with_bundle_policy_(local_node_id, cluster_resource_manager.GetResourceView(), is_node_available, - cluster_resource_manager.GetBundleLocationIndex()), + cluster_resource_manager.GetBundleLocationIndex(), + is_node_schedulable), node_label_scheduling_policy_(local_node_id, cluster_resource_manager.GetResourceView(), - is_node_available) {} + is_node_available, + is_node_schedulable) {} scheduling::NodeID Schedule(const ResourceRequest &resource_request, SchedulingOptions options) override; @@ -69,11 +82,17 @@ class CompositeBundleSchedulingPolicy : public IBundleSchedulingPolicy { public: explicit CompositeBundleSchedulingPolicy( ClusterResourceManager &cluster_resource_manager, - std::function is_node_available) - : bundle_pack_policy_(cluster_resource_manager, is_node_available), - bundle_spread_policy_(cluster_resource_manager, is_node_available), - bundle_strict_spread_policy_(cluster_resource_manager, is_node_available), - bundle_strict_pack_policy_(cluster_resource_manager, is_node_available) {} + std::function is_node_available, + std::function + is_node_schedulable) + : bundle_pack_policy_( + cluster_resource_manager, is_node_available, is_node_schedulable), + bundle_spread_policy_( + cluster_resource_manager, is_node_available, is_node_schedulable), + bundle_strict_spread_policy_( + cluster_resource_manager, is_node_available, is_node_schedulable), + bundle_strict_pack_policy_( + cluster_resource_manager, is_node_available, is_node_schedulable) {} SchedulingResult Schedule( const std::vector &resource_request_list, diff --git a/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.cc b/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.cc index 482bd6533f17..46e6fff971e5 100644 --- a/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.cc +++ b/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.cc @@ -104,7 +104,8 @@ scheduling::NodeID HybridSchedulingPolicy::ScheduleImpl( NodeFilter node_filter, const std::string &preferred_node, int32_t schedule_top_k_absolute, - float scheduler_top_k_fraction) { + float scheduler_top_k_fraction, + const SchedulingContext *scheduling_context) { // Nodes that are feasible and currently have available resources. std::vector> available_nodes; // Nodes that are feasible but currently do not have available resources. @@ -126,6 +127,9 @@ scheduling::NodeID HybridSchedulingPolicy::ScheduleImpl( if (force_spillback && node_id == preferred_node_id) { continue; } + if (!is_node_schedulable_(node_id, scheduling_context)) { + continue; + } if (IsNodeFeasible(node_id, node_filter, node_resources, resource_request)) { bool ignore_pull_manager_at_capacity = false; if (node_id == preferred_node_id) { @@ -195,7 +199,8 @@ scheduling::NodeID HybridSchedulingPolicy::Schedule( NodeFilter::kAny, options.preferred_node_id, options.schedule_top_k_absolute, - options.scheduler_top_k_fraction); + options.scheduler_top_k_fraction, + options.scheduling_context.get()); } // Try schedule on non-GPU nodes. @@ -206,7 +211,8 @@ scheduling::NodeID HybridSchedulingPolicy::Schedule( NodeFilter::kNonGpu, options.preferred_node_id, options.schedule_top_k_absolute, - options.scheduler_top_k_fraction); + options.scheduler_top_k_fraction, + options.scheduling_context.get()); if (!best_node_id.IsNil()) { return best_node_id; } @@ -220,7 +226,8 @@ scheduling::NodeID HybridSchedulingPolicy::Schedule( NodeFilter::kAny, options.preferred_node_id, options.schedule_top_k_absolute, - options.scheduler_top_k_fraction); + options.scheduler_top_k_fraction, + options.scheduling_context.get()); } } // namespace raylet_scheduling_policy diff --git a/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.h b/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.h index bf992e702882..e1cb33f1ccf1 100644 --- a/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/hybrid_scheduling_policy.h @@ -49,14 +49,18 @@ namespace raylet_scheduling_policy { /// class HybridSchedulingPolicy : public ISchedulingPolicy { public: - HybridSchedulingPolicy(scheduling::NodeID local_node_id, - const absl::flat_hash_map &nodes, - std::function is_node_alive) + HybridSchedulingPolicy( + scheduling::NodeID local_node_id, + const absl::flat_hash_map &nodes, + std::function is_node_alive, + std::function + is_node_schedulable) : local_node_id_(local_node_id), nodes_(nodes), is_node_alive_(is_node_alive), bitgen_(), - bitgenref_(bitgen_) {} + bitgenref_(bitgen_), + is_node_schedulable_(is_node_schedulable) {} scheduling::NodeID Schedule(const ResourceRequest &resource_request, SchedulingOptions options) override; @@ -121,7 +125,8 @@ class HybridSchedulingPolicy : public ISchedulingPolicy { NodeFilter node_filter, const std::string &preferred_node, int32_t schedule_top_k_absolute, - float scheduler_top_k_fraction); + float scheduler_top_k_fraction, + const SchedulingContext *scheduling_context); /// Identifier of local node. const scheduling::NodeID local_node_id_; @@ -134,6 +139,8 @@ class HybridSchedulingPolicy : public ISchedulingPolicy { mutable absl::BitGen bitgen_; /// Using BitGenRef to simplify testing. mutable absl::BitGenRef bitgenref_; + /// Function Checks if node is schedulable. + std::function is_node_schedulable_; FRIEND_TEST(HybridSchedulingPolicyTest, GetBestNode); FRIEND_TEST(HybridSchedulingPolicyTest, GetBestNodePrioritizePreferredNode); diff --git a/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.cc b/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.cc index 737aa33a80f8..cc0e5b9b227c 100644 --- a/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.cc +++ b/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.cc @@ -23,6 +23,7 @@ scheduling::NodeID NodeAffinitySchedulingPolicy::Schedule( scheduling::NodeID target_node_id = scheduling::NodeID(options.node_affinity_node_id); if (nodes_.contains(target_node_id) && is_node_alive_(target_node_id) && + is_node_schedulable_(target_node_id, options.scheduling_context.get()) && nodes_.at(target_node_id).GetLocalView().IsFeasible(resource_request)) { if (!options.node_affinity_spill_on_unavailable && !options.node_affinity_fail_on_unavailable) { diff --git a/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.h b/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.h index 7d186219e373..5b966f0964f7 100644 --- a/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/node_affinity_scheduling_policy.h @@ -28,13 +28,17 @@ namespace raylet_scheduling_policy { // hybrid policy will be used to select another node. class NodeAffinitySchedulingPolicy : public ISchedulingPolicy { public: - NodeAffinitySchedulingPolicy(scheduling::NodeID local_node_id, - const absl::flat_hash_map &nodes, - std::function is_node_alive) + NodeAffinitySchedulingPolicy( + scheduling::NodeID local_node_id, + const absl::flat_hash_map &nodes, + std::function is_node_alive, + std::function + is_node_schedulable) : local_node_id_(local_node_id), nodes_(nodes), is_node_alive_(is_node_alive), - hybrid_policy_(local_node_id_, nodes_, is_node_alive_) {} + hybrid_policy_(local_node_id_, nodes_, is_node_alive_, is_node_schedulable), + is_node_schedulable_(is_node_schedulable) {} scheduling::NodeID Schedule(const ResourceRequest &resource_request, SchedulingOptions options) override; @@ -43,6 +47,7 @@ class NodeAffinitySchedulingPolicy : public ISchedulingPolicy { const absl::flat_hash_map &nodes_; std::function is_node_alive_; HybridSchedulingPolicy hybrid_policy_; + std::function is_node_schedulable_; }; } // namespace raylet_scheduling_policy } // namespace ray diff --git a/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.cc b/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.cc index 8486693d111f..3090fb6683b5 100644 --- a/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.cc +++ b/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.cc @@ -29,7 +29,8 @@ scheduling::NodeID NodeLabelSchedulingPolicy::Schedule( const auto &node_label_scheduling_strategy = scheduling_strategy.node_label_scheduling_strategy(); // 1. Selet feasible nodes - auto hard_match_nodes = SelectFeasibleNodes(resource_request); + auto hard_match_nodes = + SelectFeasibleNodes(resource_request, options.scheduling_context.get()); if (hard_match_nodes.empty()) { return scheduling::NodeID::Nil(); } @@ -163,12 +164,14 @@ bool NodeLabelSchedulingPolicy::IsNodeLabelInValues( absl::flat_hash_map NodeLabelSchedulingPolicy::SelectFeasibleNodes( - const ResourceRequest &resource_request) const { + const ResourceRequest &resource_request, + const SchedulingContext *scheduling_context) const { absl::flat_hash_map candidate_nodes; for (const auto &pair : nodes_) { const auto &node_id = pair.first; const auto &node_resources = pair.second.GetLocalView(); - if (is_node_alive_(node_id) && node_resources.IsFeasible(resource_request)) { + if (is_node_alive_(node_id) && is_node_schedulable_(node_id, scheduling_context) && + node_resources.IsFeasible(resource_request)) { candidate_nodes.emplace(node_id, &pair.second); } } diff --git a/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.h b/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.h index 3b300b1c29e8..d222a2ba1fa4 100644 --- a/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/node_label_scheduling_policy.h @@ -24,13 +24,17 @@ namespace raylet_scheduling_policy { // Label based node affinity scheduling strategy class NodeLabelSchedulingPolicy : public ISchedulingPolicy { public: - NodeLabelSchedulingPolicy(scheduling::NodeID local_node_id, - const absl::flat_hash_map &nodes, - std::function is_node_alive) + NodeLabelSchedulingPolicy( + scheduling::NodeID local_node_id, + const absl::flat_hash_map &nodes, + std::function is_node_alive, + std::function + is_node_schedulable) : local_node_id_(local_node_id), nodes_(nodes), is_node_alive_(is_node_alive), - gen_(std::chrono::high_resolution_clock::now().time_since_epoch().count()) {} + gen_(std::chrono::high_resolution_clock::now().time_since_epoch().count()), + is_node_schedulable_(is_node_schedulable) {} scheduling::NodeID Schedule(const ResourceRequest &resource_request, SchedulingOptions options) override; @@ -50,7 +54,8 @@ class NodeLabelSchedulingPolicy : public ISchedulingPolicy { const ResourceRequest &resource_request) const; absl::flat_hash_map SelectFeasibleNodes( - const ResourceRequest &resource_request) const; + const ResourceRequest &resource_request, + const SchedulingContext *scheduling_context) const; absl::flat_hash_map FilterNodesByLabelMatchExpressions( @@ -71,6 +76,7 @@ class NodeLabelSchedulingPolicy : public ISchedulingPolicy { std::function is_node_alive_; /// Internally maintained random number generator. std::mt19937_64 gen_; + std::function is_node_schedulable_; }; } // namespace raylet_scheduling_policy } // namespace ray diff --git a/src/ray/raylet/scheduling/policy/random_scheduling_policy.cc b/src/ray/raylet/scheduling/policy/random_scheduling_policy.cc index 8de02436f4d5..33aed4666be5 100644 --- a/src/ray/raylet/scheduling/policy/random_scheduling_policy.cc +++ b/src/ray/raylet/scheduling/policy/random_scheduling_policy.cc @@ -47,7 +47,9 @@ scheduling::NodeID RandomSchedulingPolicy::Schedule( // requirement, the distribution might not be even. const auto &node_id = iter->first; const auto &node = iter->second; - if (is_node_available_(node_id) && node.GetLocalView().IsFeasible(resource_request) && + if (is_node_available_(node_id) && + is_node_schedulable_(node_id, options.scheduling_context.get()) && + node.GetLocalView().IsFeasible(resource_request) && node.GetLocalView().IsAvailable(resource_request, /*ignore_pull_manager_at_capacity*/ true)) { best_node = iter->first; diff --git a/src/ray/raylet/scheduling/policy/random_scheduling_policy.h b/src/ray/raylet/scheduling/policy/random_scheduling_policy.h index 485c34cb4219..42ba9f3690dc 100644 --- a/src/ray/raylet/scheduling/policy/random_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/random_scheduling_policy.h @@ -26,13 +26,17 @@ namespace raylet_scheduling_policy { /// requirement, the distribution might not be even. class RandomSchedulingPolicy : public ISchedulingPolicy { public: - RandomSchedulingPolicy(scheduling::NodeID local_node_id, - const absl::flat_hash_map &nodes, - std::function is_node_available) + RandomSchedulingPolicy( + scheduling::NodeID local_node_id, + const absl::flat_hash_map &nodes, + std::function is_node_available, + std::function + is_node_schedulable) : local_node_id_(local_node_id), nodes_(nodes), gen_(std::chrono::high_resolution_clock::now().time_since_epoch().count()), - is_node_available_(is_node_available) {} + is_node_available_(is_node_available), + is_node_schedulable_(is_node_schedulable) {} scheduling::NodeID Schedule(const ResourceRequest &resource_request, SchedulingOptions options) override; @@ -46,6 +50,8 @@ class RandomSchedulingPolicy : public ISchedulingPolicy { std::mt19937_64 gen_; /// Function Checks if node is alive. std::function is_node_available_; + /// Function Checks if node is schedulable. + std::function is_node_schedulable_; }; } // namespace raylet_scheduling_policy } // namespace ray diff --git a/src/ray/raylet/scheduling/policy/scheduling_context.h b/src/ray/raylet/scheduling/policy/scheduling_context.h index 11701459ee06..7cc6e1e27f32 100644 --- a/src/ray/raylet/scheduling/policy/scheduling_context.h +++ b/src/ray/raylet/scheduling/policy/scheduling_context.h @@ -26,6 +26,7 @@ namespace raylet_scheduling_policy { // Options that controls the scheduling behavior. struct SchedulingContext { virtual ~SchedulingContext() = default; + std::string virtual_cluster_id; }; struct BundleSchedulingContext : public SchedulingContext { diff --git a/src/ray/raylet/scheduling/policy/scheduling_options.h b/src/ray/raylet/scheduling/policy/scheduling_options.h index 6a44cec601e4..85858c93193b 100644 --- a/src/ray/raylet/scheduling/policy/scheduling_options.h +++ b/src/ray/raylet/scheduling/policy/scheduling_options.h @@ -42,43 +42,60 @@ enum class SchedulingType { // Options that controls the scheduling behavior. struct SchedulingOptions { - static SchedulingOptions Random() { + static SchedulingOptions Random(const std::string &virtual_cluster_id = std::string()) { + auto scheduling_context = std::make_unique(); + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions(SchedulingType::RANDOM, /*spread_threshold*/ 0, /*avoid_local_node*/ false, /*require_node_available*/ true, - /*avoid_gpu_nodes*/ false); + /*avoid_gpu_nodes*/ false, + /*max_cpu_fraction_per_node*/ 1.0, + std::move(scheduling_context)); } // construct option for spread scheduling policy. - static SchedulingOptions Spread(bool avoid_local_node, bool require_node_available) { + static SchedulingOptions Spread(bool avoid_local_node, + bool require_node_available, + const std::string &virtual_cluster_id = std::string()) { + auto scheduling_context = std::make_unique(); + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions(SchedulingType::SPREAD, /*spread_threshold*/ 0, avoid_local_node, require_node_available, - RayConfig::instance().scheduler_avoid_gpu_nodes()); + RayConfig::instance().scheduler_avoid_gpu_nodes(), + /*max_cpu_fraction_per_node*/ 1.0, + std::move(scheduling_context)); } // construct option for hybrid scheduling policy. static SchedulingOptions Hybrid(bool avoid_local_node, bool require_node_available, - const std::string &preferred_node_id = std::string()) { + const std::string &preferred_node_id = std::string(), + const std::string &virtual_cluster_id = std::string()) { + auto scheduling_context = std::make_unique(); + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions(SchedulingType::HYBRID, RayConfig::instance().scheduler_spread_threshold(), avoid_local_node, require_node_available, RayConfig::instance().scheduler_avoid_gpu_nodes(), /*max_cpu_fraction_per_node*/ 1.0, - /*scheduling_context*/ nullptr, + /*scheduling_context*/ std::move(scheduling_context), preferred_node_id); } - static SchedulingOptions NodeAffinity(bool avoid_local_node, - bool require_node_available, - std::string node_id, - bool soft, - bool spill_on_unavailable = false, - bool fail_on_unavailable = false) { + static SchedulingOptions NodeAffinity( + bool avoid_local_node, + bool require_node_available, + std::string node_id, + bool soft, + bool spill_on_unavailable = false, + bool fail_on_unavailable = false, + const std::string &virtual_cluster_id = std::string()) { + auto scheduling_context = std::make_unique(); + scheduling_context->virtual_cluster_id = virtual_cluster_id; if (spill_on_unavailable) { RAY_CHECK(soft) << "spill_on_unavailable only works with soft == true"; } @@ -92,13 +109,16 @@ struct SchedulingOptions { scheduling_options.node_affinity_soft = soft; scheduling_options.node_affinity_spill_on_unavailable = spill_on_unavailable; scheduling_options.node_affinity_fail_on_unavailable = fail_on_unavailable; + scheduling_options.scheduling_context = std::move(scheduling_context); return scheduling_options; } // construct option for affinity with bundle scheduling policy. - static SchedulingOptions AffinityWithBundle(const BundleID &bundle_id) { + static SchedulingOptions AffinityWithBundle( + const BundleID &bundle_id, const std::string &virtual_cluster_id = std::string()) { auto scheduling_context = std::make_unique(bundle_id); + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions( SchedulingType::AFFINITY_WITH_BUNDLE, /*spread_threshold*/ 0, @@ -110,9 +130,11 @@ struct SchedulingOptions { } static SchedulingOptions NodeLabelScheduling( - const rpc::SchedulingStrategy &scheduling_strategy) { + const rpc::SchedulingStrategy &scheduling_strategy, + const std::string &virtual_cluster_id = std::string()) { auto scheduling_context = std::make_unique(scheduling_strategy); + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions( SchedulingType::NODE_LABEL, /*spread_threshold*/ 0, @@ -127,36 +149,50 @@ struct SchedulingOptions { */ // construct option for soft pack scheduling policy. - static SchedulingOptions BundlePack(double max_cpu_fraction_per_node = 1.0) { + static SchedulingOptions BundlePack( + double max_cpu_fraction_per_node = 1.0, + const std::string &virtual_cluster_id = std::string()) { + auto scheduling_context = std::make_unique(); + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions(SchedulingType::BUNDLE_PACK, /*spread_threshold*/ 0, /*avoid_local_node*/ false, /*require_node_available*/ true, /*avoid_gpu_nodes*/ false, - /*max_cpu_fraction_per_node*/ max_cpu_fraction_per_node); + /*max_cpu_fraction_per_node*/ max_cpu_fraction_per_node, + std::move(scheduling_context)); } // construct option for strict spread scheduling policy. - static SchedulingOptions BundleSpread(double max_cpu_fraction_per_node = 1.0) { + static SchedulingOptions BundleSpread( + double max_cpu_fraction_per_node = 1.0, + const std::string &virtual_cluster_id = std::string()) { + auto scheduling_context = std::make_unique(); + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions(SchedulingType::BUNDLE_SPREAD, /*spread_threshold*/ 0, /*avoid_local_node*/ false, /*require_node_available*/ true, /*avoid_gpu_nodes*/ false, - /*max_cpu_fraction_per_node*/ max_cpu_fraction_per_node); + /*max_cpu_fraction_per_node*/ max_cpu_fraction_per_node, + std::move(scheduling_context)); } // construct option for strict pack scheduling policy. static SchedulingOptions BundleStrictPack( double max_cpu_fraction_per_node = 1.0, - scheduling::NodeID soft_target_node_id = scheduling::NodeID::Nil()) { + scheduling::NodeID soft_target_node_id = scheduling::NodeID::Nil(), + const std::string &virtual_cluster_id = std::string()) { + auto scheduling_context = std::make_unique(); + scheduling_context->virtual_cluster_id = virtual_cluster_id; SchedulingOptions scheduling_options = SchedulingOptions(SchedulingType::BUNDLE_STRICT_PACK, /*spread_threshold*/ 0, /*avoid_local_node*/ false, /*require_node_available*/ true, /*avoid_gpu_nodes*/ false, - /*max_cpu_fraction_per_node*/ max_cpu_fraction_per_node); + /*max_cpu_fraction_per_node*/ max_cpu_fraction_per_node, + std::move(scheduling_context)); scheduling_options.bundle_strict_pack_soft_target_node_id = soft_target_node_id; return scheduling_options; } @@ -164,7 +200,12 @@ struct SchedulingOptions { // construct option for strict spread scheduling policy. static SchedulingOptions BundleStrictSpread( double max_cpu_fraction_per_node = 1.0, - std::unique_ptr scheduling_context = nullptr) { + std::unique_ptr scheduling_context = nullptr, + const std::string &virtual_cluster_id = std::string()) { + if (scheduling_context == nullptr) { + scheduling_context = std::make_unique(); + } + scheduling_context->virtual_cluster_id = virtual_cluster_id; return SchedulingOptions(SchedulingType::BUNDLE_STRICT_SPREAD, /*spread_threshold*/ 0, /*avoid_local_node*/ false, diff --git a/src/ray/raylet/scheduling/policy/spread_scheduling_policy.cc b/src/ray/raylet/scheduling/policy/spread_scheduling_policy.cc index b06b77ecaa8a..8cf70956043c 100644 --- a/src/ray/raylet/scheduling/policy/spread_scheduling_policy.cc +++ b/src/ray/raylet/scheduling/policy/spread_scheduling_policy.cc @@ -47,7 +47,9 @@ scheduling::NodeID SpreadSchedulingPolicy::Schedule( if (node_id == local_node_id_ && options.avoid_local_node) { continue; } - if (!is_node_alive_(node_id) || !node.GetLocalView().IsFeasible(resource_request)) { + if (!is_node_alive_(node_id) || + !is_node_schedulable_(node_id, options.scheduling_context.get()) || + !node.GetLocalView().IsFeasible(resource_request)) { continue; } diff --git a/src/ray/raylet/scheduling/policy/spread_scheduling_policy.h b/src/ray/raylet/scheduling/policy/spread_scheduling_policy.h index 9fb4efcceb7e..edca388c15a2 100644 --- a/src/ray/raylet/scheduling/policy/spread_scheduling_policy.h +++ b/src/ray/raylet/scheduling/policy/spread_scheduling_policy.h @@ -26,10 +26,16 @@ namespace raylet_scheduling_policy { /// If there are no available nodes, fallback to hybrid policy. class SpreadSchedulingPolicy : public ISchedulingPolicy { public: - SpreadSchedulingPolicy(scheduling::NodeID local_node_id, - const absl::flat_hash_map &nodes, - std::function is_node_alive) - : local_node_id_(local_node_id), nodes_(nodes), is_node_alive_(is_node_alive) {} + SpreadSchedulingPolicy( + scheduling::NodeID local_node_id, + const absl::flat_hash_map &nodes, + std::function is_node_alive, + std::function + is_node_schedulable) + : local_node_id_(local_node_id), + nodes_(nodes), + is_node_alive_(is_node_alive), + is_node_schedulable_(is_node_schedulable) {} scheduling::NodeID Schedule(const ResourceRequest &resource_request, SchedulingOptions options) override; @@ -45,6 +51,8 @@ class SpreadSchedulingPolicy : public ISchedulingPolicy { size_t spread_scheduling_next_index_ = 0; /// Function Checks if node is alive. std::function is_node_alive_; + /// Function Checks if node is schedulable. + std::function is_node_schedulable_; }; } // namespace raylet_scheduling_policy } // namespace ray diff --git a/src/ray/raylet/virtual_cluster_manager.cc b/src/ray/raylet/virtual_cluster_manager.cc new file mode 100644 index 000000000000..1b12de94cf68 --- /dev/null +++ b/src/ray/raylet/virtual_cluster_manager.cc @@ -0,0 +1,96 @@ +// Copyright 2017 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/raylet/virtual_cluster_manager.h" + +#include "ray/common/virtual_cluster_id.h" + +namespace ray { + +namespace raylet { + +//////////////////////// VirtualClusterManager //////////////////////// +bool VirtualClusterManager::UpdateVirtualCluster( + rpc::VirtualClusterTableData virtual_cluster_data) { + RAY_LOG(INFO) << "Virtual cluster updated: " << virtual_cluster_data.id(); + if (virtual_cluster_data.divisible()) { + RAY_LOG(WARNING) << "Virtual cluster " << virtual_cluster_data.id() + << " is divisible, " + << "ignore it."; + return false; + } + + // The virtual cluster id of the input data. + const auto &input_virtual_cluster_id = virtual_cluster_data.id(); + + if (virtual_cluster_data.is_removed()) { + if (local_virtual_cluster_id_ == input_virtual_cluster_id) { + local_virtual_cluster_id_.clear(); + if (!VirtualClusterID::FromBinary(input_virtual_cluster_id).IsJobClusterID()) { + // The local node's virtual cluster is removed, we have to clean up local tasks + // (it is a no-op in most cases). + local_node_cleanup_fn_(); + } + } + virtual_clusters_.erase(input_virtual_cluster_id); + } else { + // Whether the local node in the input data. + bool local_node_in_input_virtual_cluster = + virtual_cluster_data.node_instances().contains(local_node_instance_id_); + + // The local node is removed from its current virtual cluster. + if (local_virtual_cluster_id_ == input_virtual_cluster_id && + !local_node_in_input_virtual_cluster) { + local_virtual_cluster_id_.clear(); + // Clean up the local tasks (it is a no-op in most cases). + local_node_cleanup_fn_(); + } else if (local_virtual_cluster_id_ != input_virtual_cluster_id && + local_node_in_input_virtual_cluster) { // The local node is added to a new + // virtual cluster. + local_virtual_cluster_id_ = input_virtual_cluster_id; + // There are chances that the pub message (removing the local node from a virtual + // cluster) was lost in the past, so we also have to clean up when adding the local + // node to a new virtual cluster (it is a no-op in most cases). + local_node_cleanup_fn_(); + } + + virtual_clusters_[input_virtual_cluster_id] = std::move(virtual_cluster_data); + } + return true; +} + +bool VirtualClusterManager::ContainsVirtualCluster( + const std::string &virtual_cluster_id) const { + return virtual_clusters_.find(virtual_cluster_id) != virtual_clusters_.end(); +} + +bool VirtualClusterManager::ContainsNodeInstance(const std::string &virtual_cluster_id, + const NodeID &node_id) const { + auto it = virtual_clusters_.find(virtual_cluster_id); + if (it == virtual_clusters_.end()) { + return false; + } + const auto &virtual_cluster_data = it->second; + RAY_CHECK(!virtual_cluster_data.divisible()); + + const auto &node_instances = virtual_cluster_data.node_instances(); + return node_instances.find(node_id.Hex()) != node_instances.end(); +} + +const std::string &VirtualClusterManager::GetLocalVirtualClusterID() const { + return local_virtual_cluster_id_; +} + +} // namespace raylet +} // namespace ray diff --git a/src/ray/raylet/virtual_cluster_manager.h b/src/ray/raylet/virtual_cluster_manager.h new file mode 100644 index 000000000000..4aaf616f1acf --- /dev/null +++ b/src/ray/raylet/virtual_cluster_manager.h @@ -0,0 +1,63 @@ +// Copyright 2017 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 "ray/common/id.h" +#include "src/ray/protobuf/gcs_service.pb.h" + +namespace ray { + +namespace raylet { + +class VirtualClusterManager { + public: + VirtualClusterManager(const NodeID &node_id, + std::function local_node_cleanup_fn) + : local_node_instance_id_(node_id.Hex()), + local_node_cleanup_fn_(local_node_cleanup_fn) {} + + /// Update the virtual cluster. + /// + /// \param virtual_cluster_data The virtual cluster data. + bool UpdateVirtualCluster(rpc::VirtualClusterTableData virtual_cluster_data); + + /// Check if the virtual cluster exists. + /// + /// \param virtual_cluster_id The virtual cluster id. + /// \return Whether the virtual cluster exists. + bool ContainsVirtualCluster(const std::string &virtual_cluster_id) const; + + /// Check if the virtual cluster contains the node instance. + /// + /// \param virtual_cluster_id The virtual cluster id. + /// \param node_id The node instance id. + /// \return Whether the virtual cluster contains the node instance. + bool ContainsNodeInstance(const std::string &virtual_cluster_id, + const NodeID &node_id) const; + + const std::string &GetLocalVirtualClusterID() const; + + private: + /// The virtual clusters. + absl::flat_hash_map virtual_clusters_; + /// The local node instance id. + std::string local_node_instance_id_; + std::function local_node_cleanup_fn_; + /// The (indivisible) virtual cluster to which the local node belongs. + std::string local_virtual_cluster_id_; +}; + +} // namespace raylet +} // end namespace ray diff --git a/src/ray/raylet/virtual_cluster_manager_test.cc b/src/ray/raylet/virtual_cluster_manager_test.cc new file mode 100644 index 000000000000..575ab79c6a89 --- /dev/null +++ b/src/ray/raylet/virtual_cluster_manager_test.cc @@ -0,0 +1,85 @@ +// Copyright 2017 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/raylet/virtual_cluster_manager.h" + +#include "absl/container/flat_hash_set.h" +#include "gtest/gtest.h" + +namespace ray { + +namespace raylet { + +class VirtualClusterManagerTest : public ::testing::Test {}; + +TEST_F(VirtualClusterManagerTest, UpdateVirtualCluster) { + VirtualClusterManager virtual_cluster_manager; + + std::string virtual_cluster_id_0 = "virtual_cluster_id_0"; + ASSERT_FALSE(virtual_cluster_manager.ContainsVirtualCluster("virtual_cluster_id")); + + rpc::VirtualClusterTableData virtual_cluster_data; + virtual_cluster_data.set_id(virtual_cluster_id_0); + virtual_cluster_data.set_divisible(true); + virtual_cluster_data.set_revision(100); + for (size_t i = 0; i < 100; ++i) { + auto node_id = NodeID::FromRandom(); + virtual_cluster_data.mutable_node_instances()->insert( + {node_id.Hex(), ray::rpc::NodeInstance()}); + } + ASSERT_FALSE(virtual_cluster_manager.UpdateVirtualCluster(virtual_cluster_data)); + ASSERT_FALSE(virtual_cluster_manager.ContainsVirtualCluster(virtual_cluster_id_0)); + + virtual_cluster_data.set_divisible(false); + ASSERT_TRUE(virtual_cluster_manager.UpdateVirtualCluster(virtual_cluster_data)); + ASSERT_TRUE(virtual_cluster_manager.ContainsVirtualCluster(virtual_cluster_id_0)); + + virtual_cluster_data.set_revision(50); + ASSERT_FALSE(virtual_cluster_manager.UpdateVirtualCluster(virtual_cluster_data)); + + virtual_cluster_data.set_revision(150); + ASSERT_TRUE(virtual_cluster_manager.UpdateVirtualCluster(virtual_cluster_data)); + + virtual_cluster_data.set_is_removed(true); + ASSERT_TRUE(virtual_cluster_manager.UpdateVirtualCluster(virtual_cluster_data)); + ASSERT_FALSE(virtual_cluster_manager.ContainsVirtualCluster(virtual_cluster_id_0)); +} + +TEST_F(VirtualClusterManagerTest, TestContainsNodeInstance) { + VirtualClusterManager virtual_cluster_manager; + std::string virtual_cluster_id_0 = "virtual_cluster_id_0"; + + rpc::VirtualClusterTableData virtual_cluster_data; + virtual_cluster_data.set_id(virtual_cluster_id_0); + virtual_cluster_data.set_divisible(false); + virtual_cluster_data.set_revision(100); + absl::flat_hash_set node_ids; + for (size_t i = 0; i < 100; ++i) { + auto node_id = NodeID::FromRandom(); + node_ids.emplace(node_id); + + virtual_cluster_data.mutable_node_instances()->insert( + {node_id.Hex(), ray::rpc::NodeInstance()}); + } + ASSERT_TRUE(virtual_cluster_manager.UpdateVirtualCluster(virtual_cluster_data)); + ASSERT_TRUE(virtual_cluster_manager.ContainsVirtualCluster(virtual_cluster_id_0)); + + for (const auto &node_id : node_ids) { + ASSERT_TRUE( + virtual_cluster_manager.ContainsNodeInstance(virtual_cluster_id_0, node_id)); + } +} + +} // namespace raylet +} // namespace ray diff --git a/src/ray/raylet_client/raylet_client.cc b/src/ray/raylet_client/raylet_client.cc index c08c13ebe661..43c46a23c391 100644 --- a/src/ray/raylet_client/raylet_client.cc +++ b/src/ray/raylet_client/raylet_client.cc @@ -114,10 +114,11 @@ Status RayletClient::AnnounceWorkerPortForWorker(int port) { } Status RayletClient::AnnounceWorkerPortForDriver(int port, - const std::string &entrypoint) { + const std::string &entrypoint, + const std::string &virtual_cluster_id) { flatbuffers::FlatBufferBuilder fbb; - auto message = - protocol::CreateAnnounceWorkerPort(fbb, port, fbb.CreateString(entrypoint)); + auto message = protocol::CreateAnnounceWorkerPort( + fbb, port, fbb.CreateString(entrypoint), fbb.CreateString(virtual_cluster_id)); fbb.Finish(message); std::vector reply; RAY_RETURN_NOT_OK(conn_->AtomicRequestReply(MessageType::AnnounceWorkerPort, diff --git a/src/ray/raylet_client/raylet_client.h b/src/ray/raylet_client/raylet_client.h index 51df62037fbd..82e370001300 100644 --- a/src/ray/raylet_client/raylet_client.h +++ b/src/ray/raylet_client/raylet_client.h @@ -316,7 +316,9 @@ class RayletClient : public RayletClientInterface { /// \param port The port. /// \param entrypoint The entrypoint of the driver's job. /// \return ray::Status. - Status AnnounceWorkerPortForDriver(int port, const std::string &entrypoint); + Status AnnounceWorkerPortForDriver(int port, + const std::string &entrypoint, + const std::string &virtual_cluster_id); /// Tell the raylet that the client has finished executing a task. /// diff --git a/src/ray/rpc/gcs_server/gcs_rpc_client.h b/src/ray/rpc/gcs_server/gcs_rpc_client.h index 24d928dc2986..78eae723de70 100644 --- a/src/ray/rpc/gcs_server/gcs_rpc_client.h +++ b/src/ray/rpc/gcs_server/gcs_rpc_client.h @@ -178,6 +178,10 @@ class GcsRpcClient { runtime_env_grpc_client_ = std::make_shared>(channel_, client_call_manager); + virtual_cluster_info_grpc_client_ = + std::make_shared>(channel_, + client_call_manager); + retryable_grpc_client_ = RetryableGrpcClient::Create( channel_, client_call_manager.GetMainService(), @@ -556,6 +560,36 @@ class GcsRpcClient { runtime_env_grpc_client_, /*method_timeout_ms*/ -1, ) + // Create or update a virtual cluster. + VOID_GCS_RPC_CLIENT_METHOD(VirtualClusterInfoGcsService, + CreateOrUpdateVirtualCluster, + virtual_cluster_info_grpc_client_, + /*method_timeout_ms*/ -1, ) + + // Remove a virtual cluster. + VOID_GCS_RPC_CLIENT_METHOD(VirtualClusterInfoGcsService, + RemoveVirtualCluster, + virtual_cluster_info_grpc_client_, + /*method_timeout_ms*/ -1, ) + + // Get virtual clusters. + VOID_GCS_RPC_CLIENT_METHOD(VirtualClusterInfoGcsService, + GetVirtualClusters, + virtual_cluster_info_grpc_client_, + /*method_timeout_ms*/ -1, ) + + // Create job cluster. + VOID_GCS_RPC_CLIENT_METHOD(VirtualClusterInfoGcsService, + CreateJobCluster, + virtual_cluster_info_grpc_client_, + /*method_timeout_ms*/ -1, ) + + // Get virtual clusters view. + VOID_GCS_RPC_CLIENT_METHOD(VirtualClusterInfoGcsService, + GetAllVirtualClusterInfo, + virtual_cluster_info_grpc_client_, + /*method_timeout_ms*/ -1, ) + std::pair GetAddress() const { return std::make_pair(gcs_address_, gcs_port_); } @@ -582,6 +616,8 @@ class GcsRpcClient { std::shared_ptr> runtime_env_grpc_client_; std::shared_ptr> autoscaler_state_grpc_client_; + std::shared_ptr> + virtual_cluster_info_grpc_client_; friend class GcsClientReconnectionTest; FRIEND_TEST(GcsClientReconnectionTest, ReconnectionBackoff); diff --git a/src/ray/rpc/gcs_server/gcs_rpc_server.h b/src/ray/rpc/gcs_server/gcs_rpc_server.h index 0bd521bc38a4..626668923a30 100644 --- a/src/ray/rpc/gcs_server/gcs_rpc_server.h +++ b/src/ray/rpc/gcs_server/gcs_rpc_server.h @@ -153,6 +153,9 @@ namespace rpc { #define INTERNAL_PUBSUB_SERVICE_RPC_HANDLER(HANDLER) \ RPC_SERVICE_HANDLER(InternalPubSubGcsService, HANDLER, -1) +#define VIRTUAL_CLUSTER_SERVICE_RPC_HANDLER(HANDLER) \ + RPC_SERVICE_HANDLER(VirtualClusterInfoGcsService, HANDLER, -1) + #define GCS_RPC_SEND_REPLY(send_reply_callback, reply, status) \ reply->mutable_status()->set_code((int)status.code()); \ reply->mutable_status()->set_message(status.message()); \ @@ -719,6 +722,62 @@ class InternalPubSubGrpcService : public GrpcService { InternalPubSubGcsServiceHandler &service_handler_; }; +class VirtualClusterInfoGcsServiceHandler { + public: + virtual ~VirtualClusterInfoGcsServiceHandler() = default; + + virtual void HandleCreateOrUpdateVirtualCluster( + CreateOrUpdateVirtualClusterRequest request, + CreateOrUpdateVirtualClusterReply *reply, + SendReplyCallback send_reply_callback) = 0; + + virtual void HandleRemoveVirtualCluster(RemoveVirtualClusterRequest request, + RemoveVirtualClusterReply *reply, + SendReplyCallback send_reply_callback) = 0; + + virtual void HandleGetVirtualClusters(GetVirtualClustersRequest request, + GetVirtualClustersReply *reply, + SendReplyCallback send_reply_callback) = 0; + + virtual void HandleCreateJobCluster(CreateJobClusterRequest request, + CreateJobClusterReply *reply, + SendReplyCallback send_reply_callback) = 0; + + virtual void HandleGetAllVirtualClusterInfo(GetAllVirtualClusterInfoRequest request, + GetAllVirtualClusterInfoReply *reply, + SendReplyCallback send_reply_callback) = 0; +}; + +class VirtualClusterInfoGrpcService : public GrpcService { + public: + /// Constructor. + /// + /// \param[in] handler The service handler that actually handle the requests. + explicit VirtualClusterInfoGrpcService(instrumented_io_context &io_service, + VirtualClusterInfoGcsServiceHandler &handler) + : GrpcService(io_service), service_handler_(handler){}; + + protected: + grpc::Service &GetGrpcService() override { return service_; } + + void InitServerCallFactories( + const std::unique_ptr &cq, + std::vector> *server_call_factories, + const ClusterID &cluster_id) override { + VIRTUAL_CLUSTER_SERVICE_RPC_HANDLER(CreateOrUpdateVirtualCluster); + VIRTUAL_CLUSTER_SERVICE_RPC_HANDLER(RemoveVirtualCluster); + VIRTUAL_CLUSTER_SERVICE_RPC_HANDLER(GetVirtualClusters); + VIRTUAL_CLUSTER_SERVICE_RPC_HANDLER(CreateJobCluster); + VIRTUAL_CLUSTER_SERVICE_RPC_HANDLER(GetAllVirtualClusterInfo); + } + + private: + /// The grpc async service object. + VirtualClusterInfoGcsService::AsyncService service_; + /// The service handler that actually handle the requests. + VirtualClusterInfoGcsServiceHandler &service_handler_; +}; + using JobInfoHandler = JobInfoGcsServiceHandler; using ActorInfoHandler = ActorInfoGcsServiceHandler; using NodeInfoHandler = NodeInfoGcsServiceHandler; @@ -729,6 +788,7 @@ using InternalKVHandler = InternalKVGcsServiceHandler; using InternalPubSubHandler = InternalPubSubGcsServiceHandler; using RuntimeEnvHandler = RuntimeEnvGcsServiceHandler; using TaskInfoHandler = TaskInfoGcsServiceHandler; +using VirtualClusterInfoHandler = VirtualClusterInfoGcsServiceHandler; } // namespace rpc } // namespace ray diff --git a/src/ray/util/array.h b/src/ray/util/array.h index 6c5d2df74235..7452fc965398 100644 --- a/src/ray/util/array.h +++ b/src/ray/util/array.h @@ -36,14 +36,17 @@ constexpr bool ArrayIsUnique(const std::array &arr) { // Returns the index of the first occurrence of value in the array. If value is not found, // it's a compile error. template -constexpr size_t IndexOf(const std::array &arr, const T &value) { - for (size_t i = 0; i < N; ++i) { - if (arr[i] == value) { - return i; - } +constexpr size_t IndexOf(const std::array &arr, const T &value, size_t index = 0) { + if (index < 0 || index == N) { + throw std::out_of_range("Value not found in array"); + // return index; } - // Throwing in constexpr context leads to a compile error. - throw "Value not found in array"; + + if (arr[index] == value) { + return index; + } + + return IndexOf(arr, value, index + 1); } } // namespace ray diff --git a/src/ray/util/logging.h b/src/ray/util/logging.h index 8b6401f99be2..f7bf7c899e50 100644 --- a/src/ray/util/logging.h +++ b/src/ray/util/logging.h @@ -102,6 +102,7 @@ inline constexpr std::string_view kLogKeyActorID = "actor_id"; inline constexpr std::string_view kLogKeyTaskID = "task_id"; inline constexpr std::string_view kLogKeyObjectID = "object_id"; inline constexpr std::string_view kLogKeyPlacementGroupID = "placement_group_id"; +inline constexpr std::string_view kLogKeyVirtualClusterID = "virtual_cluster_id"; // Define your specialization DefaultLogKey::key to get .WithField(t) // See src/ray/common/id.h diff --git a/src/ray/util/util.h b/src/ray/util/util.h index c93a0a041429..b9084b6af40d 100644 --- a/src/ray/util/util.h +++ b/src/ray/util/util.h @@ -148,6 +148,13 @@ inline int64_t current_sys_time_us() { return mu_since_epoch.count(); } +inline int64_t current_sys_time_ns() { + std::chrono::nanoseconds mn_since_epoch = + std::chrono::duration_cast( + std::chrono::system_clock::now().time_since_epoch()); + return mn_since_epoch.count(); +} + std::string GenerateUUIDV4(); /// A helper function to parse command-line arguments in a platform-compatible manner.