diff --git a/BUILD.bazel b/BUILD.bazel index e42b50cccf43..28675b7083e2 100644 --- a/BUILD.bazel +++ b/BUILD.bazel @@ -2404,11 +2404,43 @@ ray_cc_test( ) ray_cc_test( - name = "gcs_export_event_test", + name = "gcs_job_manager_export_event_test", size = "small", - srcs = glob([ - "src/ray/gcs/gcs_server/test/export_api/*.cc", - ]), + srcs = ["src/ray/gcs/gcs_server/test/export_api/gcs_job_manager_export_event_test.cc"], + tags = [ + "no_windows", + "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 = "gcs_actor_manager_export_event_test", + size = "small", + srcs = ["src/ray/gcs/gcs_server/test/export_api/gcs_actor_manager_export_event_test.cc"], + tags = [ + "no_windows", + "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 = "gcs_node_manager_export_event_test", + size = "small", + srcs = ["src/ray/gcs/gcs_server/test/export_api/gcs_node_manager_export_event_test.cc"], tags = [ "no_windows", "team:core" diff --git a/doc/source/cluster/kubernetes/user-guides/persist-kuberay-operator-logs.md b/doc/source/cluster/kubernetes/user-guides/persist-kuberay-operator-logs.md index 496afd34e4a5..3cadfbb81938 100644 --- a/doc/source/cluster/kubernetes/user-guides/persist-kuberay-operator-logs.md +++ b/doc/source/cluster/kubernetes/user-guides/persist-kuberay-operator-logs.md @@ -19,7 +19,7 @@ helm repo add grafana https://grafana.github.io/helm-charts helm repo update # Install Loki with single replica mode -helm install loki grafana/loki -f https://raw.githubusercontent.com/grafana/loki/refs/heads/main/production/helm/loki/single-binary-values.yaml +helm install loki grafana/loki --version 6.21.0 -f https://raw.githubusercontent.com/grafana/loki/refs/heads/main/production/helm/loki/single-binary-values.yaml ``` ### Configure log processing @@ -48,7 +48,7 @@ Deploy the Fluent Bit deployment with the [Helm chart repository](https://github helm repo add fluent https://fluent.github.io/helm-charts helm repo update -helm install fluent-bit fluent/fluent-bit -f fluent-bit-config.yaml +helm install fluent-bit fluent/fluent-bit --version 0.48.2 -f fluent-bit-config.yaml ``` ### Install the KubeRay Operator @@ -75,7 +75,7 @@ Deploy the Grafana deployment with the [Helm chart repository](https://github.co helm repo add grafana https://grafana.github.io/helm-charts helm repo update -helm install grafana grafana/grafana -f datasource-config.yaml +helm install grafana grafana/grafana --version 8.6.2 -f datasource-config.yaml ``` ### Check the Grafana Dashboard diff --git a/python/ray/_private/ray_logging/constants.py b/python/ray/_private/ray_logging/constants.py index de84d510c16c..54552bdfe1d7 100644 --- a/python/ray/_private/ray_logging/constants.py +++ b/python/ray/_private/ray_logging/constants.py @@ -41,6 +41,9 @@ class LogKey(str, Enum): NODE_ID = "node_id" ACTOR_ID = "actor_id" TASK_ID = "task_id" + ACTOR_NAME = "actor_name" + TASK_NAME = "task_name" + TASK_FUNCTION_NAME = "task_func_name" # Logger built-in context ASCTIME = "asctime" diff --git a/python/ray/_private/ray_logging/filters.py b/python/ray/_private/ray_logging/filters.py index e7003022040a..91233a2b11c6 100644 --- a/python/ray/_private/ray_logging/filters.py +++ b/python/ray/_private/ray_logging/filters.py @@ -20,4 +20,13 @@ def filter(self, record): task_id = runtime_context.get_task_id() if task_id is not None: setattr(record, LogKey.TASK_ID.value, task_id) + task_name = runtime_context.get_task_name() + if task_name is not None: + setattr(record, LogKey.TASK_NAME.value, task_name) + task_function_name = runtime_context.get_task_function_name() + if task_function_name is not None: + setattr(record, LogKey.TASK_FUNCTION_NAME.value, task_function_name) + actor_name = runtime_context.get_actor_name() + if actor_name is not None: + setattr(record, LogKey.ACTOR_NAME.value, actor_name) return True diff --git a/python/ray/_private/services.py b/python/ray/_private/services.py index 0de0ba78405f..23461e950286 100644 --- a/python/ray/_private/services.py +++ b/python/ray/_private/services.py @@ -834,8 +834,8 @@ def start_ray_process( use_valgrind_profiler: bool = False, use_perftools_profiler: bool = False, use_tmux: bool = False, - stdout_file: Optional[str] = None, - stderr_file: Optional[str] = None, + stdout_file: Optional[IO[AnyStr]] = None, + stderr_file: Optional[IO[AnyStr]] = None, pipe_stdin: bool = False, ): """Start one of the Ray processes. @@ -1443,8 +1443,8 @@ def start_gcs_server( redis_address: str, log_dir: str, session_name: str, - stdout_file: Optional[str] = None, - stderr_file: Optional[str] = None, + stdout_file: Optional[IO[AnyStr]] = None, + stderr_file: Optional[IO[AnyStr]] = None, redis_password: Optional[str] = None, config: Optional[dict] = None, fate_share: Optional[bool] = None, diff --git a/python/ray/_private/worker.py b/python/ray/_private/worker.py index e62375c5e523..d2b0cf3b013d 100644 --- a/python/ray/_private/worker.py +++ b/python/ray/_private/worker.py @@ -531,6 +531,14 @@ def actor_name(self): def current_task_id(self): return self.core_worker.get_current_task_id() + @property + def current_task_name(self): + return self.core_worker.get_current_task_name() + + @property + def current_task_function_name(self): + return self.core_worker.get_current_task_function_name() + @property def current_node_id(self): return self.core_worker.get_current_node_id() diff --git a/python/ray/_raylet.pyx b/python/ray/_raylet.pyx index 944adba8fc33..f3d93dce33ba 100644 --- a/python/ray/_raylet.pyx +++ b/python/ray/_raylet.pyx @@ -261,6 +261,9 @@ cdef optional[ObjectIDIndexType] NULL_PUT_INDEX = nullopt # https://docs.python.org/3/library/contextvars.html#contextvars.ContextVar # It is thread-safe. async_task_id = contextvars.ContextVar('async_task_id', default=None) +async_task_name = contextvars.ContextVar('async_task_name', default=None) +async_task_function_name = contextvars.ContextVar('async_task_function_name', + default=None) class DynamicObjectRefGenerator: @@ -1815,7 +1818,8 @@ cdef void execute_task( return core_worker.run_async_func_or_coro_in_event_loop( async_function, function_descriptor, name_of_concurrency_group_to_execute, task_id=task_id, - func_args=(actor, *arguments), func_kwargs=kwarguments) + task_name=task_name, func_args=(actor, *arguments), + func_kwargs=kwarguments) return function(actor, *arguments, **kwarguments) @@ -1927,7 +1931,8 @@ cdef void execute_task( execute_streaming_generator_async(context), function_descriptor, name_of_concurrency_group_to_execute, - task_id=task_id) + task_id=task_id, + task_name=task_name) else: execute_streaming_generator_sync(context) @@ -3415,6 +3420,48 @@ cdef class CoreWorker: with nogil: CCoreWorkerProcess.GetCoreWorker().Exit(c_exit_type, detail, null_ptr) + def get_current_task_name(self) -> str: + """Return the current task name. + + If it is a normal task, it returns the task name from the main thread. + If it is a threaded actor, it returns the task name for the current thread. + If it is async actor, it returns the task name stored in contextVar for + the current asyncio task. + """ + # We can only obtain the correct task name within asyncio task + # via async_task_name contextvar. We try this first. + # It is needed because the core worker's GetCurrentTask API + # doesn't have asyncio context, thus it cannot return the + # correct task name. + task_name = async_task_name.get() + if task_name is None: + # if it is not within asyncio context, fallback to TaskName + # obtainable from core worker. + task_name = CCoreWorkerProcess.GetCoreWorker().GetCurrentTaskName() \ + .decode("utf-8") + return task_name + + def get_current_task_function_name(self) -> str: + """Return the current task function. + + If it is a normal task, it returns the task function from the main thread. + If it is a threaded actor, it returns the task function for the current thread. + If it is async actor, it returns the task function stored in contextVar for + the current asyncio task. + """ + # We can only obtain the correct task function within asyncio task + # via async_task_function_name contextvar. We try this first. + # It is needed because the core Worker's GetCurrentTask API + # doesn't have asyncio context, thus it cannot return the + # correct task function. + task_function_name = async_task_function_name.get() + if task_function_name is None: + # if it is not within asyncio context, fallback to TaskName + # obtainable from core worker. + task_function_name = CCoreWorkerProcess.GetCoreWorker() \ + .GetCurrentTaskFunctionName().decode("utf-8") + return task_function_name + def get_current_task_id(self) -> TaskID: """Return the current task ID. @@ -4822,6 +4869,7 @@ cdef class CoreWorker: specified_cgname: str, *, task_id: Optional[TaskID] = None, + task_name: Optional[str] = None, func_args: Optional[Tuple] = None, func_kwargs: Optional[Dict] = None, ): @@ -4868,6 +4916,9 @@ cdef class CoreWorker: try: if task_id: async_task_id.set(task_id) + if task_name is not None: + async_task_name.set(task_name) + async_task_function_name.set(function_descriptor.repr) if inspect.isawaitable(func_or_coro): coroutine = func_or_coro diff --git a/python/ray/autoscaler/_private/monitor.py b/python/ray/autoscaler/_private/monitor.py index a641652615df..90b5610f59ae 100644 --- a/python/ray/autoscaler/_private/monitor.py +++ b/python/ray/autoscaler/_private/monitor.py @@ -604,16 +604,6 @@ def log_resource_batch_data_if_desired( parser.add_argument( "--gcs-address", required=False, type=str, help="The address (ip:port) of GCS." ) - parser.add_argument( - "--redis-address", required=False, type=str, help="This is deprecated" - ) - parser.add_argument( - "--redis-password", - required=False, - type=str, - default=None, - help="This is deprecated", - ) parser.add_argument( "--autoscaling-config", required=False, diff --git a/python/ray/autoscaler/kuberay/ray-cluster.complete.yaml b/python/ray/autoscaler/kuberay/ray-cluster.complete.yaml index f51ac9d4c242..d57f5d6f23b9 100644 --- a/python/ray/autoscaler/kuberay/ray-cluster.complete.yaml +++ b/python/ray/autoscaler/kuberay/ray-cluster.complete.yaml @@ -108,7 +108,7 @@ spec: workerGroupSpecs: # the pod replicas in this group typed worker - replicas: 1 - minReplicas: 1 + minReplicas: 0 maxReplicas: 300 # logical group name, for this called small-group, also can be functional groupName: small-group diff --git a/python/ray/autoscaler/v2/instance_manager/cloud_providers/kuberay/cloud_provider.py b/python/ray/autoscaler/v2/instance_manager/cloud_providers/kuberay/cloud_provider.py index 19e236cb4d19..c1b8ddc2a31b 100644 --- a/python/ray/autoscaler/v2/instance_manager/cloud_providers/kuberay/cloud_provider.py +++ b/python/ray/autoscaler/v2/instance_manager/cloud_providers/kuberay/cloud_provider.py @@ -209,21 +209,25 @@ def _initialize_scale_request( cur_instances = self.instances # Get the worker groups that have pending deletes and the worker groups that - # have finished deletes. + # have finished deletes, and the set of workers included in the workersToDelete + # field of any worker group. ( worker_groups_with_pending_deletes, worker_groups_without_pending_deletes, - ) = self._get_workers_groups_with_deletes( - ray_cluster, set(cur_instances.keys()) - ) + worker_to_delete_set, + ) = self._get_workers_delete_info(ray_cluster, set(cur_instances.keys())) # Calculate the desired number of workers by type. num_workers_dict = defaultdict(int) - for _, cur_instance in cur_instances.items(): - if cur_instance.node_kind == NodeKind.HEAD: - # Only track workers. - continue - num_workers_dict[cur_instance.node_type] += 1 + worker_groups = ray_cluster["spec"].get("workerGroupSpecs", []) + for worker_group in worker_groups: + node_type = worker_group["groupName"] + # Handle the case where users manually increase `minReplicas` + # to scale up the number of worker Pods. In this scenario, + # `replicas` will be smaller than `minReplicas`. + num_workers_dict[node_type] = max( + worker_group["replicas"], worker_group["minReplicas"] + ) # Add to launch nodes. for node_type, count in to_launch.items(): @@ -242,6 +246,11 @@ def _initialize_scale_request( # Not possible to delete head node. continue + if to_delete_instance.cloud_instance_id in worker_to_delete_set: + # If the instance is already in the workersToDelete field of + # any worker group, skip it. + continue + num_workers_dict[to_delete_instance.node_type] -= 1 assert num_workers_dict[to_delete_instance.node_type] >= 0 to_delete_instances_by_type[to_delete_instance.node_type].append( @@ -321,6 +330,7 @@ def _submit_scale_request( # No patch required. return + logger.info(f"Submitting a scale request: {scale_request}") self._patch(f"rayclusters/{self._cluster_name}", patch_payload) def _add_launch_errors( @@ -392,9 +402,9 @@ def instances(self) -> Dict[CloudInstanceId, CloudInstance]: return copy.deepcopy(self._cached_instances) @staticmethod - def _get_workers_groups_with_deletes( + def _get_workers_delete_info( ray_cluster_spec: Dict[str, Any], node_set: Set[CloudInstanceId] - ) -> Tuple[Set[NodeType], Set[NodeType]]: + ) -> Tuple[Set[NodeType], Set[NodeType], Set[CloudInstanceId]]: """ Gets the worker groups that have pending deletes and the worker groups that have finished deletes. @@ -404,10 +414,13 @@ def _get_workers_groups_with_deletes( deletes. worker_groups_with_finished_deletes: The worker groups that have finished deletes. + worker_to_delete_set: A set of Pods that are included in the workersToDelete + field of any worker group. """ worker_groups_with_pending_deletes = set() worker_groups_with_deletes = set() + worker_to_delete_set = set() worker_groups = ray_cluster_spec["spec"].get("workerGroupSpecs", []) for worker_group in worker_groups: @@ -422,6 +435,7 @@ def _get_workers_groups_with_deletes( worker_groups_with_deletes.add(node_type) for worker in workersToDelete: + worker_to_delete_set.add(worker) if worker in node_set: worker_groups_with_pending_deletes.add(node_type) break @@ -429,7 +443,11 @@ def _get_workers_groups_with_deletes( worker_groups_with_finished_deletes = ( worker_groups_with_deletes - worker_groups_with_pending_deletes ) - return worker_groups_with_pending_deletes, worker_groups_with_finished_deletes + return ( + worker_groups_with_pending_deletes, + worker_groups_with_finished_deletes, + worker_to_delete_set, + ) def _fetch_instances(self) -> Dict[CloudInstanceId, CloudInstance]: """ diff --git a/python/ray/autoscaler/v2/monitor.py b/python/ray/autoscaler/v2/monitor.py index c9334244aa8c..558725f3e78f 100644 --- a/python/ray/autoscaler/v2/monitor.py +++ b/python/ray/autoscaler/v2/monitor.py @@ -198,16 +198,6 @@ def record_autoscaler_v2_usage(gcs_client: GcsClient) -> None: parser.add_argument( "--gcs-address", required=False, type=str, help="The address (ip:port) of GCS." ) - parser.add_argument( - "--redis-address", required=False, type=str, help="This is deprecated" - ) - parser.add_argument( - "--redis-password", - required=False, - type=str, - default=None, - help="This is deprecated", - ) parser.add_argument( "--autoscaling-config", required=False, diff --git a/python/ray/autoscaler/v2/tests/test_node_provider.py b/python/ray/autoscaler/v2/tests/test_node_provider.py index 02d84e376b8d..47483d3f61fa 100644 --- a/python/ray/autoscaler/v2/tests/test_node_provider.py +++ b/python/ray/autoscaler/v2/tests/test_node_provider.py @@ -492,6 +492,124 @@ def test_pending_deletes(self): }, ] + def test_increase_min_replicas_to_scale_up(self): + # Simulate the case where users manually increase the `minReplicas` field + # from 0 to $num_pods. KubeRay will create $num_pods worker Pods to meet the new + # `minReplicas`, even though the `replicas` field is still 0. + small_group = "small-group" + num_pods = 0 + assert ( + self.mock_client._ray_cluster["spec"]["workerGroupSpecs"][0]["groupName"] + == small_group + ) + for pod in self.mock_client._pod_list["items"]: + if pod["metadata"]["labels"]["ray.io/group"] == small_group: + num_pods += 1 + assert num_pods > 0 + self.mock_client._ray_cluster["spec"]["workerGroupSpecs"][0]["replicas"] = 0 + self.mock_client._ray_cluster["spec"]["workerGroupSpecs"][0][ + "minReplicas" + ] = num_pods + + # Launching a new node and `replicas` should be + # `max(replicas, minReplicas) + 1`. + self.provider.launch(shape={small_group: 1}, request_id="launch-1") + patches = self.mock_client.get_patches( + f"rayclusters/{self.provider._cluster_name}" + ) + assert len(patches) == 1 + assert patches[0] == { + "op": "replace", + "path": "/spec/workerGroupSpecs/0/replicas", + "value": num_pods + 1, + } + + def test_inconsistent_pods_raycr_scale_up(self): + """ + Test the case where the cluster state has not yet reached the desired state. + Specifically, the replicas field in the RayCluster CR does not match the actual + number of Pods. + """ + # Check the assumptions of the test + small_group = "small-group" + num_pods = 0 + for pod in self.mock_client._pod_list["items"]: + if pod["metadata"]["labels"]["ray.io/group"] == small_group: + num_pods += 1 + + assert ( + self.mock_client._ray_cluster["spec"]["workerGroupSpecs"][0]["groupName"] + == small_group + ) + desired_replicas = num_pods + 1 + self.mock_client._ray_cluster["spec"]["workerGroupSpecs"][0][ + "replicas" + ] = desired_replicas + + # Launch a new node. The replicas field should be incremented by 1, even though + # the cluster state has not yet reached the goal state. + launch_request = {"small-group": 1} + self.provider.launch(shape=launch_request, request_id="launch-1") + + patches = self.mock_client.get_patches( + f"rayclusters/{self.provider._cluster_name}" + ) + assert len(patches) == 1 + assert patches[0] == { + "op": "replace", + "path": "/spec/workerGroupSpecs/0/replicas", + "value": desired_replicas + 1, + } + + def test_inconsistent_pods_raycr_scale_down(self): + """ + Test the case where the cluster state has not yet reached the desired state. + Specifically, the replicas field in the RayCluster CR does not match the actual + number of Pods. + """ + # Check the assumptions of the test + small_group = "small-group" + num_pods = 0 + pod_to_delete = None + for pod in self.mock_client._pod_list["items"]: + if pod["metadata"]["labels"]["ray.io/group"] == small_group: + num_pods += 1 + pod_to_delete = pod["metadata"]["name"] + assert pod_to_delete is not None + + assert ( + self.mock_client._ray_cluster["spec"]["workerGroupSpecs"][0]["groupName"] + == small_group + ) + desired_replicas = num_pods + 1 + self.mock_client._ray_cluster["spec"]["workerGroupSpecs"][0][ + "replicas" + ] = desired_replicas + + # Terminate a node. The replicas field should be decremented by 1, even though + # the cluster state has not yet reached the goal state. + self.provider.terminate(ids=[pod_to_delete], request_id="term-1") + patches = self.mock_client.get_patches( + f"rayclusters/{self.provider._cluster_name}" + ) + assert len(patches) == 2 + assert patches == [ + { + "op": "replace", + "path": "/spec/workerGroupSpecs/0/replicas", + "value": desired_replicas - 1, + }, + { + "op": "replace", + "path": "/spec/workerGroupSpecs/0/scaleStrategy", + "value": { + "workersToDelete": [ + pod_to_delete, + ] + }, + }, + ] + if __name__ == "__main__": if os.environ.get("PARALLEL_CI"): diff --git a/python/ray/data/_internal/planner/exchange/sort_task_spec.py b/python/ray/data/_internal/planner/exchange/sort_task_spec.py index 827c4a2c7a51..7c67b3dbdefe 100644 --- a/python/ray/data/_internal/planner/exchange/sort_task_spec.py +++ b/python/ray/data/_internal/planner/exchange/sort_task_spec.py @@ -81,8 +81,9 @@ def validate_schema(self, schema: Optional[Union[type, "pyarrow.lib.Schema"]]): for column in self._columns: if column not in schema_names_set: raise ValueError( - "The column '{}' does not exist in the " - "schema '{}'.".format(column, schema) + f"You specified the column '{column}', but there's no such " + "column in the dataset. The dataset has columns: " + f"{schema_names_set}" ) @property diff --git a/python/ray/data/grouped_data.py b/python/ray/data/grouped_data.py index 8f7b7dde118d..427ea18b7bbf 100644 --- a/python/ray/data/grouped_data.py +++ b/python/ray/data/grouped_data.py @@ -1,3 +1,4 @@ +from functools import partial from typing import Any, Dict, Iterable, List, Optional, Tuple, Union from ray.data._internal.aggregate import Count, Max, Mean, Min, Std, Sum @@ -261,7 +262,10 @@ def wrapped_fn(batch, *args, **kwargs): # Change the name of the wrapped function so that users see the name of their # function rather than `wrapped_fn` in the progress bar. - wrapped_fn.__name__ = fn.__name__ + if isinstance(fn, partial): + wrapped_fn.__name__ = fn.func.__name__ + else: + wrapped_fn.__name__ = fn.__name__ # Note we set batch_size=None here, so it will use the entire block as a batch, # which ensures that each group will be contained within a batch in entirety. diff --git a/python/ray/data/tests/test_all_to_all.py b/python/ray/data/tests/test_all_to_all.py index cf0cb8b2b2e7..a6b173383145 100644 --- a/python/ray/data/tests/test_all_to_all.py +++ b/python/ray/data/tests/test_all_to_all.py @@ -1167,7 +1167,6 @@ def test_groupby_map_groups_multicolumn( ray_start_regular_shared, ds_format, num_parts, use_push_based_shuffle ): # Test built-in count aggregation - print(f"Seeding RNG for test_groupby_arrow_count with: {RANDOM_SEED}") random.seed(RANDOM_SEED) xs = list(range(100)) random.shuffle(xs) @@ -1190,6 +1189,33 @@ def test_groupby_map_groups_multicolumn( ] +def test_groupby_map_groups_with_partial(): + """ + The partial function name should show up as + +- Sort + +- MapBatches(func) + """ + from functools import partial + + def func(x, y): + return {f"x_add_{y}": [len(x["id"]) + y]} + + df = pd.DataFrame({"id": list(range(100))}) + df["key"] = df["id"] % 5 + + ds = ray.data.from_pandas(df).groupby("key").map_groups(partial(func, y=5)) + result = ds.take_all() + + assert result == [ + {"x_add_5": 25}, + {"x_add_5": 25}, + {"x_add_5": 25}, + {"x_add_5": 25}, + {"x_add_5": 25}, + ] + assert "MapBatches(func)" in ds.__repr__() + + def test_random_block_order_schema(ray_start_regular_shared): df = pd.DataFrame({"a": np.random.rand(10), "b": np.random.rand(10)}) ds = ray.data.from_pandas(df).randomize_block_order() diff --git a/python/ray/data/tests/test_arrow_block.py b/python/ray/data/tests/test_arrow_block.py index 22fe9f7ef7a0..00ed13363f20 100644 --- a/python/ray/data/tests/test_arrow_block.py +++ b/python/ray/data/tests/test_arrow_block.py @@ -91,7 +91,7 @@ def binary_dataset_single_file_gt_2gb(): ], ) def test_single_row_gt_2gb( - ray_start_regular_shared, + ray_start_regular, restore_data_context, binary_dataset_single_file_gt_2gb, col_name, @@ -124,7 +124,7 @@ def _id(row): ], ) def test_arrow_batch_gt_2gb( - ray_start_regular_shared, + ray_start_regular, parquet_dataset_single_column_gt_2gb, restore_data_context, op, @@ -158,10 +158,10 @@ def _id(x): print( f">>> Batch:\n" f"------\n" - "Column: 'id'" + "Column: 'id'\n" f"Values: {batch['id']}\n" f"------\n" - "Column: 'bin'" + "Column: 'bin'\n" f"Total: {total_binary_column_size / GiB} GiB\n" f"Values: {[str(v)[:3] + ' x ' + str(len(v)) for v in batch['bin']]}\n" ) diff --git a/python/ray/data/tests/test_execution_optimizer.py b/python/ray/data/tests/test_execution_optimizer.py index d657ce1c9d98..af7af855b187 100644 --- a/python/ray/data/tests/test_execution_optimizer.py +++ b/python/ray/data/tests/test_execution_optimizer.py @@ -1145,9 +1145,7 @@ def test_sort_validate_keys(ray_start_regular_shared): assert extract_values("id", ds.sort("id").take_all()) == list(range(10)) invalid_col_name = "invalid_column" - with pytest.raises( - ValueError, match=f"The column '{invalid_col_name}' does not exist" - ): + with pytest.raises(ValueError, match="there's no such column in the dataset"): ds.sort(invalid_col_name).take_all() ds_named = ray.data.from_items( @@ -1165,10 +1163,7 @@ def test_sort_validate_keys(ray_start_regular_shared): assert [d["col1"] for d in r1] == [7, 5, 3, 1] assert [d["col2"] for d in r2] == [8, 6, 4, 2] - with pytest.raises( - ValueError, - match=f"The column '{invalid_col_name}' does not exist in the schema", - ): + with pytest.raises(ValueError, match="there's no such column in the dataset"): ds_named.sort(invalid_col_name).take_all() @@ -1279,9 +1274,7 @@ def test_aggregate_e2e(ray_start_regular_shared, use_push_based_shuffle): def test_aggregate_validate_keys(ray_start_regular_shared): ds = ray.data.range(10) invalid_col_name = "invalid_column" - with pytest.raises( - ValueError, match=f"The column '{invalid_col_name}' does not exist" - ): + with pytest.raises(ValueError): ds.groupby(invalid_col_name).count() ds_named = ray.data.from_items( @@ -1308,7 +1301,7 @@ def test_aggregate_validate_keys(ray_start_regular_shared): with pytest.raises( ValueError, - match=f"The column '{invalid_col_name}' does not exist in the schema", + match="there's no such column in the dataset", ): ds_named.groupby(invalid_col_name).count() diff --git a/python/ray/includes/libcoreworker.pxd b/python/ray/includes/libcoreworker.pxd index 242c5f10dd49..87f5d59a8583 100644 --- a/python/ray/includes/libcoreworker.pxd +++ b/python/ray/includes/libcoreworker.pxd @@ -191,6 +191,8 @@ cdef extern from "ray/core_worker/core_worker.h" nogil: CJobID GetCurrentJobId() CTaskID GetCurrentTaskId() + const c_string GetCurrentTaskName() + const c_string GetCurrentTaskFunctionName() void UpdateTaskIsDebuggerPaused( const CTaskID &task_id, const c_bool is_debugger_paused) diff --git a/python/ray/runtime_context.py b/python/ray/runtime_context.py index 78f7cee9502d..5cacae69371b 100644 --- a/python/ray/runtime_context.py +++ b/python/ray/runtime_context.py @@ -124,7 +124,7 @@ def get_worker_id(self) -> str: @property @Deprecated(message="Use get_task_id() instead", warning=True) def task_id(self): - """Get current task ID for this worker or driver. + """Get current task ID for this worker. Task ID is the id of a Ray task. This shouldn't be used in a driver process. @@ -155,7 +155,7 @@ def f(): Returns: The current worker's task id. None if there's no task id. """ - # only worker mode has actor_id + # only worker mode has task_id assert ( self.worker.mode == ray._private.worker.WORKER_MODE ), f"This method is only available when the process is a\ @@ -165,7 +165,7 @@ def f(): return task_id if not task_id.is_nil() else None def get_task_id(self) -> Optional[str]: - """Get current task ID for this worker or driver. + """Get current task ID for this worker. Task ID is the id of a Ray task. The ID will be in hex format. This shouldn't be used in a driver process. @@ -201,7 +201,7 @@ def get_task_id(): Returns: The current worker's task id in hex. None if there's no task id. """ - # only worker mode has actor_id + # only worker mode has task_id if self.worker.mode != ray._private.worker.WORKER_MODE: logger.warning( "This method is only available when the process is a " @@ -212,12 +212,116 @@ def get_task_id(): return task_id.hex() if not task_id.is_nil() else None def _get_current_task_id(self) -> TaskID: - async_task_id = ray._raylet.async_task_id.get() - if async_task_id is None: - task_id = self.worker.current_task_id - else: - task_id = async_task_id - return task_id + return self.worker.current_task_id + + def get_task_name(self) -> Optional[str]: + """Get current task name for this worker. + + Task name by default is the task's funciton call string. It can also be + specified in options when triggering a task. + + Example: + + .. testcode:: + + import ray + + @ray.remote + class Actor: + def get_task_name(self): + return ray.get_runtime_context().get_task_name() + + @ray.remote + class AsyncActor: + async def get_task_name(self): + return ray.get_runtime_context().get_task_name() + + @ray.remote + def get_task_name(): + return ray.get_runtime_context().get_task_name() + + a = Actor.remote() + b = AsyncActor.remote() + # Task names are available for actor tasks. + print(ray.get(a.get_task_name.remote())) + # Task names are avaiable for async actor tasks. + print(ray.get(b.get_task_name.remote())) + # Task names are available for normal tasks. + # Get default task name + print(ray.get(get_task_name.remote())) + # Get specified task name + print(ray.get(get_task_name.options(name="task_name").remote())) + + .. testoutput:: + :options: +MOCK + + Actor.get_task_name + AsyncActor.get_task_name + get_task_name + task_nams + + Returns: + The current worker's task name + """ + # only worker mode has task_name + if self.worker.mode != ray._private.worker.WORKER_MODE: + logger.warning( + "This method is only available when the process is a " + f"worker. Current mode: {self.worker.mode}" + ) + return None + return self.worker.current_task_name + + def get_task_function_name(self) -> Optional[str]: + """Get current task function name string for this worker. + + Example: + + .. testcode:: + + import ray + + @ray.remote + class Actor: + def get_task_function_name(self): + return ray.get_runtime_context().get_task_function_name() + + @ray.remote + class AsyncActor: + async def get_task_function_name(self): + return ray.get_runtime_context().get_task_function_name() + + @ray.remote + def get_task_function_name(): + return ray.get_runtime_context().get_task_function_name() + + a = Actor.remote() + b = AsyncActor.remote() + # Task functions are available for actor tasks. + print(ray.get(a.get_task_function_name.remote())) + # Task functions are available for async actor tasks. + print(ray.get(b.get_task_function_name.remote())) + # Task functions are available for normal tasks. + print(ray.get(get_task_function_name.remote())) + + .. testoutput:: + :options: +MOCK + + [python modual name].Actor.get_task_function_name + [python modual name].AsyncActor.get_task_function_name + [python modual name].get_task_function_name + + Returns: + The current worker's task function call string + """ + # only worker mode has task_function_name + if self.worker.mode != ray._private.worker.WORKER_MODE: + logger.warning( + "This method is only available when the process is a " + f"worker. Current mode: {self.worker.mode}" + ) + return None + return self.worker.current_task_function_name @property @Deprecated(message="Use get_actor_id() instead", warning=True) diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index 1f26a483a7aa..eed702bb7438 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -622,6 +622,15 @@ def debug(address: str, verbose: bool): type=str, help="a JSON serialized dictionary mapping label name to label value.", ) +@click.option( + "--include-log-monitor", + default=None, + type=bool, + help="If set to True or left unset, a log monitor will start monitoring " + "the log files of all processes on this node and push their contents to GCS. " + "Only one log monitor should be started per physical host to avoid log " + "duplication on the driver process.", +) @add_click_logging_options @PublicAPI def start( @@ -668,6 +677,7 @@ def start( ray_debugger_external, disable_usage_stats, labels, + include_log_monitor, ): """Start Ray processes manually on the local machine.""" @@ -757,6 +767,7 @@ def start( no_monitor=no_monitor, tracing_startup_hook=tracing_startup_hook, ray_debugger_external=ray_debugger_external, + include_log_monitor=include_log_monitor, ) if ray_constants.RAY_START_HOOK in os.environ: diff --git a/python/ray/serve/_private/constants.py b/python/ray/serve/_private/constants.py index f4c2adac37d1..6e45bae19ceb 100644 --- a/python/ray/serve/_private/constants.py +++ b/python/ray/serve/_private/constants.py @@ -355,3 +355,17 @@ RAY_SERVE_FORCE_LOCAL_TESTING_MODE = ( os.environ.get("RAY_SERVE_FORCE_LOCAL_TESTING_MODE", "0") == "1" ) + +# Run sync methods defined in the replica in a thread pool by default. +RAY_SERVE_RUN_SYNC_IN_THREADPOOL = ( + os.environ.get("RAY_SERVE_RUN_SYNC_IN_THREADPOOL", "0") == "1" +) + +RAY_SERVE_RUN_SYNC_IN_THREADPOOL_WARNING = ( + "Calling sync method '{method_name}' directly on the " + "asyncio loop. In a future version, sync methods will be run in a " + "threadpool by default. Ensure your sync methods are thread safe " + "or keep the existing behavior by making them `async def`. Opt " + "into the new behavior by setting " + "RAY_SERVE_RUN_SYNC_IN_THREADPOOL=1." +) diff --git a/python/ray/serve/_private/local_testing_mode.py b/python/ray/serve/_private/local_testing_mode.py index 6ccc16cd3628..84525625a938 100644 --- a/python/ray/serve/_private/local_testing_mode.py +++ b/python/ray/serve/_private/local_testing_mode.py @@ -10,7 +10,10 @@ import ray from ray import cloudpickle from ray.serve._private.common import DeploymentID, RequestMetadata -from ray.serve._private.constants import SERVE_LOGGER_NAME +from ray.serve._private.constants import ( + RAY_SERVE_RUN_SYNC_IN_THREADPOOL, + SERVE_LOGGER_NAME, +) from ray.serve._private.replica import UserCallableWrapper from ray.serve._private.replica_result import ReplicaResult from ray.serve._private.router import Router @@ -66,6 +69,7 @@ def make_local_deployment_handle( deployment.init_args, deployment.init_kwargs, deployment_id=deployment_id, + run_sync_methods_in_threadpool=RAY_SERVE_RUN_SYNC_IN_THREADPOOL, ) try: logger.info(f"Initializing local replica class for {deployment_id}.") diff --git a/python/ray/serve/_private/replica.py b/python/ray/serve/_private/replica.py index 079e3039d5d0..23fc7d237f94 100644 --- a/python/ray/serve/_private/replica.py +++ b/python/ray/serve/_private/replica.py @@ -1,5 +1,6 @@ import asyncio import concurrent.futures +import functools import inspect import logging import os @@ -7,9 +8,9 @@ import threading import time import traceback +import warnings from abc import ABC, abstractmethod from contextlib import contextmanager -from functools import wraps from importlib import import_module from typing import ( Any, @@ -23,6 +24,7 @@ ) import starlette.responses +from anyio import to_thread from starlette.types import ASGIApp, Message import ray @@ -47,6 +49,8 @@ HEALTH_CHECK_METHOD, RAY_SERVE_COLLECT_AUTOSCALING_METRICS_ON_HANDLE, RAY_SERVE_REPLICA_AUTOSCALING_METRIC_RECORD_PERIOD_S, + RAY_SERVE_RUN_SYNC_IN_THREADPOOL, + RAY_SERVE_RUN_SYNC_IN_THREADPOOL_WARNING, RECONFIGURE_METHOD, SERVE_CONTROLLER_NAME, SERVE_LOGGER_NAME, @@ -274,6 +278,7 @@ def __init__( init_args, init_kwargs, deployment_id=self._deployment_id, + run_sync_methods_in_threadpool=RAY_SERVE_RUN_SYNC_IN_THREADPOOL, ) # Guards against calling the user's callable constructor multiple times. @@ -602,6 +607,11 @@ async def initialize(self, deployment_config: DeploymentConfig): self._user_callable_initialized = True if deployment_config: + await asyncio.wrap_future( + self._user_callable_wrapper.set_sync_method_threadpool_limit( + deployment_config.max_ongoing_requests + ) + ) await asyncio.wrap_future( self._user_callable_wrapper.call_reconfigure( deployment_config.user_config @@ -635,6 +645,11 @@ async def reconfigure(self, deployment_config: DeploymentConfig): if logging_config_changed: self._configure_logger_and_profilers(deployment_config.logging_config) + await asyncio.wrap_future( + self._user_callable_wrapper.set_sync_method_threadpool_limit( + deployment_config.max_ongoing_requests + ) + ) if user_config_changed: await asyncio.wrap_future( self._user_callable_wrapper.call_reconfigure( @@ -990,6 +1005,7 @@ def __init__( init_kwargs: Dict, *, deployment_id: DeploymentID, + run_sync_methods_in_threadpool: bool, ): if not (inspect.isfunction(deployment_def) or inspect.isclass(deployment_def)): raise TypeError( @@ -1003,6 +1019,8 @@ def __init__( self._is_function = inspect.isfunction(deployment_def) self._deployment_id = deployment_id self._destructor_called = False + self._run_sync_methods_in_threadpool = run_sync_methods_in_threadpool + self._warned_about_sync_method_change = False # Will be populated in `initialize_callable`. self._callable = None @@ -1033,7 +1051,7 @@ def _run_on_user_code_event_loop(f: Callable) -> Callable: f ), "_run_on_user_code_event_loop can only be used on coroutine functions." - @wraps(f) + @functools.wraps(f) def wrapper(self, *args, **kwargs) -> concurrent.futures.Future: return asyncio.run_coroutine_threadsafe( f(self, *args, **kwargs), @@ -1042,6 +1060,12 @@ def wrapper(self, *args, **kwargs) -> concurrent.futures.Future: return wrapper + @_run_on_user_code_event_loop + async def set_sync_method_threadpool_limit(self, limit: int): + # NOTE(edoakes): the limit is thread local, so this must + # be run on the user code event loop. + to_thread.current_default_thread_limiter().total_tokens = limit + def _get_user_callable_method(self, method_name: str) -> Callable: if self._is_function: return self._callable @@ -1082,17 +1106,89 @@ async def _send_user_result_over_asgi( else: await Response(result).send(scope, receive, send) - async def _call_func_or_gen(self, callable: Callable, *args, **kwargs) -> Any: + async def _call_func_or_gen( + self, + callable: Callable, + *, + args: Optional[Tuple[Any]] = None, + kwargs: Optional[Dict[str, Any]] = None, + request_metadata: Optional[RequestMetadata] = None, + generator_result_callback: Optional[Callable] = None, + run_sync_methods_in_threadpool_override: Optional[bool] = None, + ) -> Tuple[Any, bool]: """Call the callable with the provided arguments. This is a convenience wrapper that will work for `def`, `async def`, generator, and async generator functions. + + Returns the result and a boolean indicating if the result was a sync generator + that has already been consumed. """ - result = callable(*args, **kwargs) - if inspect.iscoroutine(result): - result = await result + sync_gen_consumed = False + args = args if args is not None else tuple() + kwargs = kwargs if kwargs is not None else dict() + run_sync_in_threadpool = ( + self._run_sync_methods_in_threadpool + if run_sync_methods_in_threadpool_override is None + else run_sync_methods_in_threadpool_override + ) + is_sync_method = ( + inspect.isfunction(callable) or inspect.ismethod(callable) + ) and not ( + inspect.iscoroutinefunction(callable) + or inspect.isasyncgenfunction(callable) + ) - return result + if is_sync_method and run_sync_in_threadpool: + is_generator = inspect.isgeneratorfunction(callable) + if is_generator: + sync_gen_consumed = True + if request_metadata and not request_metadata.is_streaming: + # TODO(edoakes): make this check less redundant with the one in + # _handle_user_method_result. + raise TypeError( + f"Method '{callable.__name__}' returned a generator. " + "You must use `handle.options(stream=True)` to call " + "generators on a deployment." + ) + + def run_callable(): + result = callable(*args, **kwargs) + if is_generator: + for r in result: + # TODO(edoakes): make this less redundant with the handling in + # _handle_user_method_result. + if request_metadata and request_metadata.is_grpc_request: + r = (request_metadata.grpc_context, r.SerializeToString()) + generator_result_callback(r) + + result = None + + return result + + # NOTE(edoakes): we use anyio.to_thread here because it's what Starlette + # uses (and therefore FastAPI too). The max size of the threadpool is + # set to max_ongoing_requests in the replica wrapper. + # anyio.to_thread propagates ContextVars to the worker thread automatically. + result = await to_thread.run_sync(run_callable) + else: + if ( + is_sync_method + and not self._warned_about_sync_method_change + and run_sync_methods_in_threadpool_override is None + ): + self._warned_about_sync_method_change = True + warnings.warn( + RAY_SERVE_RUN_SYNC_IN_THREADPOOL_WARNING.format( + method_name=callable.__name__, + ) + ) + + result = callable(*args, **kwargs) + if inspect.iscoroutine(result): + result = await result + + return result, sync_gen_consumed @property def user_callable(self) -> Optional[Callable]: @@ -1129,8 +1225,10 @@ async def initialize_callable(self) -> Optional[ASGIApp]: self._callable = self._deployment_def.__new__(self._deployment_def) await self._call_func_or_gen( self._callable.__init__, - *self._init_args, - **self._init_kwargs, + args=self._init_args, + kwargs=self._init_kwargs, + # Always run the constructor on the main user code thread. + run_sync_methods_in_threadpool_override=False, ) if isinstance(self._callable, ASGIAppReplicaWrapper): @@ -1192,7 +1290,7 @@ async def call_reconfigure(self, user_config: Any): ) await self._call_func_or_gen( getattr(self._callable, RECONFIGURE_METHOD), - user_config, + args=(user_config,), ) def _prepare_args_for_http_request( @@ -1264,6 +1362,7 @@ async def _handle_user_method_result( user_method_name: str, request_metadata: RequestMetadata, *, + sync_gen_consumed: bool, generator_result_callback: Optional[Callable], is_asgi_app: bool, asgi_args: Optional[ASGIArgs], @@ -1297,7 +1396,7 @@ async def _handle_user_method_result( # For the FastAPI codepath, the response has already been sent over # ASGI, but for the vanilla deployment codepath we need to send it. await self._send_user_result_over_asgi(result, asgi_args) - elif not request_metadata.is_http_request: + elif not request_metadata.is_http_request and not sync_gen_consumed: # If a unary method is called with stream=True for anything EXCEPT # an HTTP request, raise an error. # HTTP requests are always streaming regardless of if the method @@ -1382,12 +1481,20 @@ async def call_user_method( request_args[0], request_metadata, user_method_params ) - result = await self._handle_user_method_result( - await self._call_func_or_gen( - user_method, *request_args, **request_kwargs - ), + result, sync_gen_consumed = await self._call_func_or_gen( + user_method, + args=request_args, + kwargs=request_kwargs, + request_metadata=request_metadata, + generator_result_callback=generator_result_callback + if request_metadata.is_streaming + else None, + ) + return await self._handle_user_method_result( + result, user_method_name, request_metadata, + sync_gen_consumed=sync_gen_consumed, generator_result_callback=generator_result_callback, is_asgi_app=is_asgi_app, asgi_args=asgi_args, @@ -1412,8 +1519,6 @@ async def call_user_method( if receive_task is not None and not receive_task.done(): receive_task.cancel() - return result - @_run_on_user_code_event_loop async def call_destructor(self): """Explicitly call the `__del__` method of the user callable. @@ -1437,7 +1542,11 @@ async def call_destructor(self): try: if hasattr(self._callable, "__del__"): # Make sure to accept `async def __del__(self)` as well. - await self._call_func_or_gen(self._callable.__del__) + await self._call_func_or_gen( + self._callable.__del__, + # Always run the destructor on the main user callable thread. + run_sync_methods_in_threadpool_override=False, + ) if hasattr(self._callable, "__serve_multiplex_wrapper"): await getattr(self._callable, "__serve_multiplex_wrapper").shutdown() diff --git a/python/ray/serve/tests/BUILD b/python/ray/serve/tests/BUILD index 369b9a339c6f..9f3208084538 100644 --- a/python/ray/serve/tests/BUILD +++ b/python/ray/serve/tests/BUILD @@ -467,3 +467,25 @@ py_test_module_list( "//python/ray/serve:serve_lib", ], ) + + +# Test currently off-by-default behavior to run replica sync methods in a threadpool. +# TODO(edoakes): remove this once the FF is flipped on by default. +py_test_module_list( + size = "small", + env = {"RAY_SERVE_RUN_SYNC_IN_THREADPOOL": "1"}, + files = [ + "test_replica_sync_methods.py", + ], + name_suffix = "_with_run_sync_in_threadpool", + tags = [ + "exclusive", + "no_windows", + "team:serve", + ], + deps = [ + ":common", + ":conftest", + "//python/ray/serve:serve_lib", + ], +) diff --git a/python/ray/serve/tests/test_logging.py b/python/ray/serve/tests/test_logging.py index 061dafd75fe0..24f9a47a01a1 100644 --- a/python/ray/serve/tests/test_logging.py +++ b/python/ray/serve/tests/test_logging.py @@ -351,6 +351,9 @@ def fn(*args): "actor_id": ray.get_runtime_context().get_actor_id(), "worker_id": ray.get_runtime_context().get_worker_id(), "node_id": ray.get_runtime_context().get_node_id(), + "task_name": ray.get_runtime_context().get_task_name(), + "task_func_name": ray.get_runtime_context().get_task_function_name(), + "actor_name": ray.get_runtime_context().get_actor_name(), } @serve.deployment( @@ -369,6 +372,9 @@ def __call__(self, req: starlette.requests.Request): "actor_id": ray.get_runtime_context().get_actor_id(), "worker_id": ray.get_runtime_context().get_worker_id(), "node_id": ray.get_runtime_context().get_node_id(), + "task_name": ray.get_runtime_context().get_task_name(), + "task_func_name": ray.get_runtime_context().get_task_function_name(), + "actor_name": ray.get_runtime_context().get_actor_name(), } serve.run(fn.bind(), name="app1", route_prefix="/fn") @@ -418,6 +424,9 @@ def check_log(): f'"worker_id": "{resp["worker_id"]}", ' f'"node_id": "{resp["node_id"]}", ' f'"actor_id": "{resp["actor_id"]}", ' + f'"task_name": "{resp["task_name"]}", ' + f'"task_func_name": "{resp["task_func_name"]}", ' + f'"actor_name": "{resp["actor_name"]}", ' f'"deployment": "{resp["app_name"]}_fn", ' f'"replica": "{method_replica_id}", ' f'"component_name": "replica".*' @@ -430,6 +439,9 @@ def check_log(): f'"worker_id": "{resp2["worker_id"]}", ' f'"node_id": "{resp2["node_id"]}", ' f'"actor_id": "{resp2["actor_id"]}", ' + f'"task_name": "{resp2["task_name"]}", ' + f'"task_func_name": "{resp2["task_func_name"]}", ' + f'"actor_name": "{resp2["actor_name"]}", ' f'"deployment": "{resp2["app_name"]}_Model", ' f'"replica": "{class_method_replica_id}", ' f'"component_name": "replica".*' diff --git a/python/ray/serve/tests/test_replica_sync_methods.py b/python/ray/serve/tests/test_replica_sync_methods.py new file mode 100644 index 000000000000..d6485704138f --- /dev/null +++ b/python/ray/serve/tests/test_replica_sync_methods.py @@ -0,0 +1,127 @@ +import asyncio +import sys + +import pytest +import requests +from anyio import to_thread +from fastapi import FastAPI +from starlette.responses import PlainTextResponse + +import ray +from ray import serve +from ray._private.test_utils import SignalActor, wait_for_condition +from ray.serve._private.constants import RAY_SERVE_RUN_SYNC_IN_THREADPOOL + + +@pytest.mark.skipif( + not RAY_SERVE_RUN_SYNC_IN_THREADPOOL, + reason="Run sync method in threadpool FF disabled.", +) +@pytest.mark.parametrize("use_fastapi", [False, True]) +def test_not_running_in_asyncio_loop(serve_instance, use_fastapi: bool): + if use_fastapi: + fastapi_app = FastAPI() + + @serve.deployment + @serve.ingress(fastapi_app) + class D: + @fastapi_app.get("/") + def root(self): + with pytest.raises(RuntimeError, match="no running event loop"): + asyncio.get_running_loop() + + else: + + @serve.deployment + class D: + def __call__(self) -> str: + with pytest.raises(RuntimeError, match="no running event loop"): + asyncio.get_running_loop() + + serve.run(D.bind()) + # Would error if the check fails. + requests.get("http://localhost:8000/").raise_for_status() + + +@pytest.mark.skipif( + not RAY_SERVE_RUN_SYNC_IN_THREADPOOL, + reason="Run sync method in threadpool FF disabled.", +) +def test_concurrent_execution(serve_instance): + signal_actor = SignalActor.remote() + + @serve.deployment + class D: + def do_sync(self): + ray.get(signal_actor.wait.remote()) + + async def do_async(self): + await signal_actor.wait.remote() + + h = serve.run(D.bind()) + + sync_results = [h.do_sync.remote(), h.do_sync.remote()] + async_results = [h.do_async.remote(), h.do_async.remote()] + + wait_for_condition(lambda: ray.get(signal_actor.cur_num_waiters.remote()) == 4) + ray.get(signal_actor.send.remote()) + [r.result() for r in sync_results + async_results] + + +@pytest.mark.skipif( + not RAY_SERVE_RUN_SYNC_IN_THREADPOOL, + reason="Run sync method in threadpool FF disabled.", +) +@pytest.mark.parametrize("use_fastapi", [False, True]) +def test_context_vars_propagated(serve_instance, use_fastapi: bool): + if use_fastapi: + fastapi_app = FastAPI() + + @serve.deployment + @serve.ingress(fastapi_app) + class D: + @fastapi_app.get("/") + def root(self): + return PlainTextResponse( + serve.context._serve_request_context.get().request_id + ) + + else: + + @serve.deployment + class D: + def __call__(self) -> str: + return PlainTextResponse( + serve.context._serve_request_context.get().request_id + ) + + serve.run(D.bind()) + + r = requests.get("http://localhost:8000/", headers={"X-Request-Id": "TEST-ID"}) + r.raise_for_status() + # If context vars weren't propagated, the request ID would be empty. + assert r.text == "TEST-ID" + + +@pytest.mark.skipif( + not RAY_SERVE_RUN_SYNC_IN_THREADPOOL, + reason="Run sync method in threadpool FF disabled.", +) +def test_thread_limit_set_to_max_ongoing_requests(serve_instance): + @serve.deployment + class D: + async def __call__(self): + return to_thread.current_default_thread_limiter().total_tokens + + h = serve.run(D.bind()) + + # Check that it's set if max_ongoing_requests is defaulted. + assert h.remote().result() == 5 + + # Update to a custom value, check again. + h = serve.run(D.options(max_ongoing_requests=10).bind()) + assert h.remote().result() == 10 + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", "-s", __file__])) diff --git a/python/ray/serve/tests/unit/test_user_callable_wrapper.py b/python/ray/serve/tests/unit/test_user_callable_wrapper.py index b03c9ca7e39e..39188f32c421 100644 --- a/python/ray/serve/tests/unit/test_user_callable_wrapper.py +++ b/python/ray/serve/tests/unit/test_user_callable_wrapper.py @@ -4,7 +4,7 @@ import sys import threading from dataclasses import dataclass -from typing import AsyncGenerator, Callable, Generator, Optional +from typing import Any, AsyncGenerator, Callable, Dict, Generator, Optional, Tuple import pytest from fastapi import FastAPI @@ -90,13 +90,18 @@ async def basic_async_generator(n: int, raise_exception: bool = False): def _make_user_callable_wrapper( - callable: Optional[Callable] = None, *init_args, **init_kwargs + callable: Optional[Callable] = None, + *, + init_args: Optional[Tuple[Any]] = None, + init_kwargs: Optional[Dict[str, Any]] = None, + run_sync_methods_in_threadpool: bool = False, ) -> UserCallableWrapper: return UserCallableWrapper( callable if callable is not None else BasicClass, - init_args, - init_kwargs, + init_args or tuple(), + init_kwargs or dict(), deployment_id=DeploymentID(name="test_name"), + run_sync_methods_in_threadpool=run_sync_methods_in_threadpool, ) @@ -144,8 +149,11 @@ def test_calling_methods_before_initialize(): user_callable_wrapper.call_reconfigure(None).result() -def test_basic_class_callable(): - user_callable_wrapper = _make_user_callable_wrapper() +@pytest.mark.parametrize("run_sync_methods_in_threadpool", [False, True]) +def test_basic_class_callable(run_sync_methods_in_threadpool: bool): + user_callable_wrapper = _make_user_callable_wrapper( + run_sync_methods_in_threadpool=run_sync_methods_in_threadpool + ) user_callable_wrapper.initialize_callable().result() @@ -215,8 +223,11 @@ def test_basic_class_callable(): ).result() -def test_basic_class_callable_generators(): - user_callable_wrapper = _make_user_callable_wrapper() +@pytest.mark.parametrize("run_sync_methods_in_threadpool", [False, True]) +def test_basic_class_callable_generators(run_sync_methods_in_threadpool: bool): + user_callable_wrapper = _make_user_callable_wrapper( + run_sync_methods_in_threadpool=run_sync_methods_in_threadpool + ) user_callable_wrapper.initialize_callable().result() result_list = [] @@ -291,9 +302,12 @@ def test_basic_class_callable_generators(): assert result_list == [0] +@pytest.mark.parametrize("run_sync_methods_in_threadpool", [False, True]) @pytest.mark.parametrize("fn", [basic_sync_function, basic_async_function]) -def test_basic_function_callable(fn: Callable): - user_callable_wrapper = _make_user_callable_wrapper(fn) +def test_basic_function_callable(fn: Callable, run_sync_methods_in_threadpool: bool): + user_callable_wrapper = _make_user_callable_wrapper( + fn, run_sync_methods_in_threadpool=run_sync_methods_in_threadpool + ) user_callable_wrapper.initialize_callable().result() # Call non-generator function with is_streaming. @@ -325,9 +339,14 @@ def test_basic_function_callable(fn: Callable): ).result() +@pytest.mark.parametrize("run_sync_methods_in_threadpool", [False, True]) @pytest.mark.parametrize("fn", [basic_sync_generator, basic_async_generator]) -def test_basic_function_callable_generators(fn: Callable): - user_callable_wrapper = _make_user_callable_wrapper(fn) +def test_basic_function_callable_generators( + fn: Callable, run_sync_methods_in_threadpool: bool +): + user_callable_wrapper = _make_user_callable_wrapper( + fn, run_sync_methods_in_threadpool=run_sync_methods_in_threadpool + ) user_callable_wrapper.initialize_callable().result() result_list = [] @@ -366,36 +385,68 @@ def test_basic_function_callable_generators(fn: Callable): @pytest.mark.asyncio -async def test_user_code_runs_on_separate_loop(): +@pytest.mark.parametrize("run_sync_methods_in_threadpool", [False, True]) +async def test_user_code_runs_on_separate_loop(run_sync_methods_in_threadpool: bool): main_loop = asyncio.get_running_loop() class GetLoop: def __init__(self): self._constructor_loop = asyncio.get_running_loop() - def check_health(self): + async def check_health(self): check_health_loop = asyncio.get_running_loop() assert ( check_health_loop == self._constructor_loop ), "User constructor and health check should run on the same loop." return check_health_loop - def __call__(self) -> asyncio.AbstractEventLoop: + async def call_async(self) -> Optional[asyncio.AbstractEventLoop]: user_method_loop = asyncio.get_running_loop() assert ( user_method_loop == self._constructor_loop ), "User constructor and other methods should run on the same loop." + return user_method_loop - user_callable_wrapper = _make_user_callable_wrapper(GetLoop) + def call_sync(self): + if run_sync_methods_in_threadpool: + with pytest.raises(RuntimeError, match="no running event loop"): + asyncio.get_running_loop() + + user_method_loop = None + else: + user_method_loop = asyncio.get_running_loop() + assert ( + user_method_loop == self._constructor_loop + ), "User constructor and other methods should run on the same loop." + + return user_method_loop + + user_callable_wrapper = _make_user_callable_wrapper( + GetLoop, run_sync_methods_in_threadpool=run_sync_methods_in_threadpool + ) user_callable_wrapper.initialize_callable().result() - request_metadata = _make_request_metadata() + + # Async methods should all run on the same loop. + request_metadata = _make_request_metadata(call_method="call_async") user_code_loop = user_callable_wrapper.call_user_method( request_metadata, tuple(), dict() ).result() assert isinstance(user_code_loop, asyncio.AbstractEventLoop) assert user_code_loop != main_loop + # Sync methods should run on the same loop if run_sync_methods_in_threadpool is off, + # else run in no asyncio loop. + request_metadata = _make_request_metadata(call_method="call_sync") + user_code_loop = user_callable_wrapper.call_user_method( + request_metadata, tuple(), dict() + ).result() + if run_sync_methods_in_threadpool: + assert user_code_loop is None + else: + assert isinstance(user_code_loop, asyncio.AbstractEventLoop) + assert user_code_loop != main_loop + # `check_health` method asserts that it runs on the correct loop. user_callable_wrapper.call_user_health_check().result() @@ -412,7 +463,7 @@ def __call__(self) -> str: msg = "hello world" user_callable_wrapper = _make_user_callable_wrapper( AsyncInitializer, - msg, + init_args=(msg,), ) user_callable_wrapper.initialize_callable().result() request_metadata = _make_request_metadata() @@ -498,8 +549,11 @@ def stream(self, msg: serve_pb2.UserDefinedMessage): yield serve_pb2.UserDefinedResponse(greeting=f"Hello {msg.greeting} {i}!") -def test_grpc_unary_request(): - user_callable_wrapper = _make_user_callable_wrapper(gRPCClass) +@pytest.mark.parametrize("run_sync_methods_in_threadpool", [False, True]) +def test_grpc_unary_request(run_sync_methods_in_threadpool: bool): + user_callable_wrapper = _make_user_callable_wrapper( + gRPCClass, run_sync_methods_in_threadpool=run_sync_methods_in_threadpool + ) user_callable_wrapper.initialize_callable().result() grpc_request = gRPCRequest( @@ -518,8 +572,11 @@ def test_grpc_unary_request(): @pytest.mark.asyncio -def test_grpc_streaming_request(): - user_callable_wrapper = _make_user_callable_wrapper(gRPCClass) +@pytest.mark.parametrize("run_sync_methods_in_threadpool", [False, True]) +def test_grpc_streaming_request(run_sync_methods_in_threadpool: bool): + user_callable_wrapper = _make_user_callable_wrapper( + gRPCClass, run_sync_methods_in_threadpool=run_sync_methods_in_threadpool + ) user_callable_wrapper.initialize_callable() grpc_request = gRPCRequest( diff --git a/python/ray/tests/kuberay/test_autoscaling_config.py b/python/ray/tests/kuberay/test_autoscaling_config.py index 6ae9ab7fc4fe..12b5c239f4a6 100644 --- a/python/ray/tests/kuberay/test_autoscaling_config.py +++ b/python/ray/tests/kuberay/test_autoscaling_config.py @@ -82,7 +82,7 @@ def _get_basic_autoscaling_config() -> dict: }, "small-group": { "max_workers": 300, - "min_workers": 1, + "min_workers": 0, "node_config": {}, "resources": { "CPU": 1, @@ -95,7 +95,7 @@ def _get_basic_autoscaling_config() -> dict: # and modified max_workers. "gpu-group": { "max_workers": 200, - "min_workers": 1, + "min_workers": 0, "node_config": {}, "resources": { "CPU": 1, @@ -109,7 +109,7 @@ def _get_basic_autoscaling_config() -> dict: # and modified max_workers and node_config. "tpu-group": { "max_workers": 4, - "min_workers": 1, + "min_workers": 0, "node_config": {}, "resources": { "CPU": 1, diff --git a/python/ray/tests/test_logging_2.py b/python/ray/tests/test_logging_2.py index b48b04e44a59..9d5be165f9ac 100644 --- a/python/ray/tests/test_logging_2.py +++ b/python/ray/tests/test_logging_2.py @@ -54,11 +54,14 @@ def f(): "worker_id": runtime_context.get_worker_id(), "node_id": runtime_context.get_node_id(), "task_id": runtime_context.get_task_id(), + "task_name": runtime_context.get_task_name(), + "task_func_name": runtime_context.get_task_function_name(), } for attr in should_exist: assert hasattr(record, attr) assert getattr(record, attr) == expected_values[attr] assert not hasattr(record, "actor_id") + assert not hasattr(record, "actor_name") obj_ref = f.remote() ray.get(obj_ref) @@ -77,7 +80,10 @@ def f(self): "worker_id": runtime_context.get_worker_id(), "node_id": runtime_context.get_node_id(), "actor_id": runtime_context.get_actor_id(), + "actor_name": runtime_context.get_actor_name(), "task_id": runtime_context.get_task_id(), + "task_name": runtime_context.get_task_name(), + "task_func_name": runtime_context.get_task_function_name(), } for attr in should_exist: assert hasattr(record, attr) diff --git a/python/ray/tests/test_runtime_context.py b/python/ray/tests/test_runtime_context.py index 3835a7e7eb3a..23bb39c6ab4d 100644 --- a/python/ray/tests/test_runtime_context.py +++ b/python/ray/tests/test_runtime_context.py @@ -264,6 +264,136 @@ def test_auto_init(shutdown_only): assert ray.is_initialized() +def test_get_task_name(shutdown_only): + ray.init() + + # for a normal task + @ray.remote + def get_task_name_for_normal_task(): + return ray.get_runtime_context().get_task_name() + + expected_task_name = "normal_task_name" + task_name = ray.get( + get_task_name_for_normal_task.options(name=expected_task_name).remote() + ) + assert ( + task_name == expected_task_name + ), f"Check normal task name failed. expected={expected_task_name}, \ +actual={task_name}" + + # for an actor task + @ray.remote + class Actor: + def get_task_name_for_actor_task(self): + return ray.get_runtime_context().get_task_name() + + expected_task_name = "Actor.get_task_name_for_actor_task" + actor = Actor.remote() + task_name = ray.get(actor.get_task_name_for_actor_task.remote()) + assert ( + task_name == expected_task_name + ), f"Check actor task name failed. expected={expected_task_name}, \ +actual={task_name}" + + # for a threaded actor task + @ray.remote + class ThreadedActor: + def get_task_name_for_threaded_actor_task(self): + return ray.get_runtime_context().get_task_name() + + expected_task_name = "ThreadedActor.get_task_name_for_threaded_actor_task" + threaded_actor = ThreadedActor.options(max_concurrency=2).remote() + task_name = ray.get(threaded_actor.get_task_name_for_threaded_actor_task.remote()) + assert ( + task_name == expected_task_name + ), f"Check actor task name failed. expected={expected_task_name}, \ +actual={task_name}" + + # for a async actor task + @ray.remote + class AsyncActor: + async def get_task_name_for_async_actor_task(self): + return ray.get_runtime_context().get_task_name() + + expected_task_name = "AsyncActor.get_task_name_for_async_actor_task" + async_actor = AsyncActor.remote() + task_name = ray.get(async_actor.get_task_name_for_async_actor_task.remote()) + assert ( + task_name == expected_task_name + ), f"Check actor task name failed. expected={expected_task_name}, \ +actual={task_name}" + + +def test_get_task_function_name(shutdown_only): + ray.init() + + # for a normal task + @ray.remote + def get_task_function_name_for_normal_task(): + return ray.get_runtime_context().get_task_function_name() + + expected_task_function_name = __name__ + ".get_task_function_name_for_normal_task" + task_function_name = ray.get(get_task_function_name_for_normal_task.remote()) + assert ( + task_function_name == expected_task_function_name + ), f"Check normal task function failed. expected={expected_task_function_name}, \ +actual={task_function_name}" + + # for an actor task + @ray.remote + class Actor: + def get_task_function_name_for_actor_task(self): + return ray.get_runtime_context().get_task_function_name() + + expected_task_function_name = ( + __name__ + ".Actor.get_task_function_name_for_actor_task" + ) + actor = Actor.remote() + task_function_name = ray.get(actor.get_task_function_name_for_actor_task.remote()) + assert ( + task_function_name == expected_task_function_name + ), f"Check actor task function failed. expected={expected_task_function_name}, \ +actual={task_function_name}" + + # for a threaded actor task + @ray.remote + class ThreadedActor: + def get_task_function_name_for_threaded_actor_task(self): + return ray.get_runtime_context().get_task_function_name() + + expected_task_function_name = ( + __name__ + ".ThreadedActor.get_task_function_name_for_threaded_actor_task" + ) + threaded_actor = ThreadedActor.options(max_concurrency=2).remote() + task_function_name = ray.get( + threaded_actor.get_task_function_name_for_threaded_actor_task.remote() + ) + assert ( + task_function_name == expected_task_function_name + ), f"Check actor task function failed. expected={expected_task_function_name}, \ +actual={task_function_name}" + + # for a async actor task + @ray.remote + class AsyncActor: + async def get_task_function_name_for_async_actor_task(self): + return ray.get_runtime_context().get_task_function_name() + + expected_task_function_name = ( + __name__ + + ".test_get_task_function_name..AsyncActor.\ +get_task_function_name_for_async_actor_task" + ) + async_actor = AsyncActor.remote() + task_function_name = ray.get( + async_actor.get_task_function_name_for_async_actor_task.remote() + ) + assert ( + task_function_name == expected_task_function_name + ), f"Check actor task function failed. expected={expected_task_function_name}, \ +actual={task_function_name}" + + def test_async_actor_task_id(shutdown_only): ray.init() diff --git a/python/requirements/ml/data-requirements.txt b/python/requirements/ml/data-requirements.txt index 6e2baa5592fe..de91b5010c7c 100644 --- a/python/requirements/ml/data-requirements.txt +++ b/python/requirements/ml/data-requirements.txt @@ -1,8 +1,8 @@ # Used by CI for datasets and docs. # https://github.com/ray-project/ray/pull/29448#discussion_r1006256498 -dask[complete]==2022.10.1; python_version < '3.12' -distributed==2022.10.1; python_version < '3.12' +dask[complete]==2022.10.2; python_version < '3.12' +distributed==2022.10.2; python_version < '3.12' dask[complete]==2024.6.0; python_version >= '3.12' distributed==2024.6.0; python_version >= '3.12' aioboto3==11.2.0 diff --git a/python/requirements_compiled.txt b/python/requirements_compiled.txt index 9dd3caa3ecab..f3b39647b0e8 100644 --- a/python/requirements_compiled.txt +++ b/python/requirements_compiled.txt @@ -360,7 +360,7 @@ cython==0.29.37 # via # -r /ray/ci/../python/requirements/test-requirements.txt # gpy -dask==2022.10.1 ; python_version < "3.12" +dask==2022.10.2 ; python_version < "3.12" # via # -r /ray/ci/../python/requirements/ml/data-requirements.txt # distributed @@ -398,7 +398,7 @@ dill==0.3.7 # multiprocess distlib==0.3.7 # via virtualenv -distributed==2022.10.1 ; python_version < "3.12" +distributed==2022.10.2 ; python_version < "3.12" # via # -r /ray/ci/../python/requirements/ml/data-requirements.txt # dask diff --git a/release/ray_release/byod/requirements_byod_3.9.txt b/release/ray_release/byod/requirements_byod_3.9.txt index adff2b611647..f1dcf9ee13a8 100644 --- a/release/ray_release/byod/requirements_byod_3.9.txt +++ b/release/ray_release/byod/requirements_byod_3.9.txt @@ -144,6 +144,7 @@ ale-py==0.10.1 \ # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_byod_3.9.in + # gymnasium annotated-types==0.6.0 \ --hash=sha256:0641064de18ba7a25dee8f96403ebc39113d0cb953a01429249d5c7564666a43 \ --hash=sha256:563339e807e53ffd9c267e99fc6d9ea23eb8443c08f112651963e24e22f84a5d @@ -574,33 +575,39 @@ crcmod==1.7 \ # via # -c release/ray_release/byod/requirements_compiled.txt # gsutil -cryptography==38.0.1 \ - --hash=sha256:0297ffc478bdd237f5ca3a7dc96fc0d315670bfa099c04dc3a4a2172008a405a \ - --hash=sha256:10d1f29d6292fc95acb597bacefd5b9e812099d75a6469004fd38ba5471a977f \ - --hash=sha256:16fa61e7481f4b77ef53991075de29fc5bacb582a1244046d2e8b4bb72ef66d0 \ - --hash=sha256:194044c6b89a2f9f169df475cc167f6157eb9151cc69af8a2a163481d45cc407 \ - --hash=sha256:1db3d807a14931fa317f96435695d9ec386be7b84b618cc61cfa5d08b0ae33d7 \ - --hash=sha256:3261725c0ef84e7592597606f6583385fed2a5ec3909f43bc475ade9729a41d6 \ - --hash=sha256:3b72c360427889b40f36dc214630e688c2fe03e16c162ef0aa41da7ab1455153 \ - --hash=sha256:3e3a2599e640927089f932295a9a247fc40a5bdf69b0484532f530471a382750 \ - --hash=sha256:3fc26e22840b77326a764ceb5f02ca2d342305fba08f002a8c1f139540cdfaad \ - --hash=sha256:5067ee7f2bce36b11d0e334abcd1ccf8c541fc0bbdaf57cdd511fdee53e879b6 \ - --hash=sha256:52e7bee800ec869b4031093875279f1ff2ed12c1e2f74923e8f49c916afd1d3b \ - --hash=sha256:64760ba5331e3f1794d0bcaabc0d0c39e8c60bf67d09c93dc0e54189dfd7cfe5 \ - --hash=sha256:765fa194a0f3372d83005ab83ab35d7c5526c4e22951e46059b8ac678b44fa5a \ - --hash=sha256:79473cf8a5cbc471979bd9378c9f425384980fcf2ab6534b18ed7d0d9843987d \ - --hash=sha256:896dd3a66959d3a5ddcfc140a53391f69ff1e8f25d93f0e2e7830c6de90ceb9d \ - --hash=sha256:89ed49784ba88c221756ff4d4755dbc03b3c8d2c5103f6d6b4f83a0fb1e85294 \ - --hash=sha256:ac7e48f7e7261207d750fa7e55eac2d45f720027d5703cd9007e9b37bbb59ac0 \ - --hash=sha256:ad7353f6ddf285aeadfaf79e5a6829110106ff8189391704c1d8801aa0bae45a \ - --hash=sha256:b0163a849b6f315bf52815e238bc2b2346604413fa7c1601eea84bcddb5fb9ac \ - --hash=sha256:b6c9b706316d7b5a137c35e14f4103e2115b088c412140fdbd5f87c73284df61 \ - --hash=sha256:c2e5856248a416767322c8668ef1845ad46ee62629266f84a8f007a317141013 \ - --hash=sha256:ca9f6784ea96b55ff41708b92c3f6aeaebde4c560308e5fbbd3173fbc466e94e \ - --hash=sha256:d1a5bd52d684e49a36582193e0b89ff267704cd4025abefb9e26803adeb3e5fb \ - --hash=sha256:d3971e2749a723e9084dd507584e2a2761f78ad2c638aa31e80bc7a15c9db4f9 \ - --hash=sha256:d4ef6cc305394ed669d4d9eebf10d3a101059bdcf2669c366ec1d14e4fb227bd \ - --hash=sha256:d9e69ae01f99abe6ad646947bba8941e896cb3aa805be2597a0400e0764b5818 +cryptography==42.0.5 \ + --hash=sha256:0270572b8bd2c833c3981724b8ee9747b3ec96f699a9665470018594301439ee \ + --hash=sha256:111a0d8553afcf8eb02a4fea6ca4f59d48ddb34497aa8706a6cf536f1a5ec576 \ + --hash=sha256:16a48c23a62a2f4a285699dba2e4ff2d1cff3115b9df052cdd976a18856d8e3d \ + --hash=sha256:1b95b98b0d2af784078fa69f637135e3c317091b615cd0905f8b8a087e86fa30 \ + --hash=sha256:1f71c10d1e88467126f0efd484bd44bca5e14c664ec2ede64c32f20875c0d413 \ + --hash=sha256:2424ff4c4ac7f6b8177b53c17ed5d8fa74ae5955656867f5a8affaca36a27abb \ + --hash=sha256:2bce03af1ce5a5567ab89bd90d11e7bbdff56b8af3acbbec1faded8f44cb06da \ + --hash=sha256:329906dcc7b20ff3cad13c069a78124ed8247adcac44b10bea1130e36caae0b4 \ + --hash=sha256:37dd623507659e08be98eec89323469e8c7b4c1407c85112634ae3dbdb926fdd \ + --hash=sha256:3eaafe47ec0d0ffcc9349e1708be2aaea4c6dd4978d76bf6eb0cb2c13636c6fc \ + --hash=sha256:5e6275c09d2badf57aea3afa80d975444f4be8d3bc58f7f80d2a484c6f9485c8 \ + --hash=sha256:6fe07eec95dfd477eb9530aef5bead34fec819b3aaf6c5bd6d20565da607bfe1 \ + --hash=sha256:7367d7b2eca6513681127ebad53b2582911d1736dc2ffc19f2c3ae49997496bc \ + --hash=sha256:7cde5f38e614f55e28d831754e8a3bacf9ace5d1566235e39d91b35502d6936e \ + --hash=sha256:9481ffe3cf013b71b2428b905c4f7a9a4f76ec03065b05ff499bb5682a8d9ad8 \ + --hash=sha256:98d8dc6d012b82287f2c3d26ce1d2dd130ec200c8679b6213b3c73c08b2b7940 \ + --hash=sha256:a011a644f6d7d03736214d38832e030d8268bcff4a41f728e6030325fea3e400 \ + --hash=sha256:a2913c5375154b6ef2e91c10b5720ea6e21007412f6437504ffea2109b5a33d7 \ + --hash=sha256:a30596bae9403a342c978fb47d9b0ee277699fa53bbafad14706af51fe543d16 \ + --hash=sha256:b03c2ae5d2f0fc05f9a2c0c997e1bc18c8229f392234e8a0194f202169ccd278 \ + --hash=sha256:b6cd2203306b63e41acdf39aa93b86fb566049aeb6dc489b70e34bcd07adca74 \ + --hash=sha256:b7ffe927ee6531c78f81aa17e684e2ff617daeba7f189f911065b2ea2d526dec \ + --hash=sha256:b8cac287fafc4ad485b8a9b67d0ee80c66bf3574f655d3b97ef2e1082360faf1 \ + --hash=sha256:ba334e6e4b1d92442b75ddacc615c5476d4ad55cc29b15d590cc6b86efa487e2 \ + --hash=sha256:ba3e4a42397c25b7ff88cdec6e2a16c2be18720f317506ee25210f6d31925f9c \ + --hash=sha256:c41fb5e6a5fe9ebcd58ca3abfeb51dffb5d83d6775405305bfa8715b76521922 \ + --hash=sha256:cd2030f6650c089aeb304cf093f3244d34745ce0cfcc39f20c6fbfe030102e2a \ + --hash=sha256:cd65d75953847815962c84a4654a84850b2bb4aed3f26fadcc1c13892e1e29f6 \ + --hash=sha256:e4985a790f921508f36f81831817cbc03b102d643b5fcb81cd33df3fa291a1a1 \ + --hash=sha256:e807b3188f9eb0eaa7bbb579b462c5ace579f1cedb28107ce8b48a9f7ad3679e \ + --hash=sha256:f12764b8fffc7a123f641d7d049d382b73f96a34117e0b637b80643169cec8ac \ + --hash=sha256:f8837fe1d6ac4a8052a9a8ddab256bc006242696f03368a4009be7ee3075cdb7 # via # -c release/ray_release/byod/requirements_compiled.txt # pyopenssl @@ -650,9 +657,9 @@ cython==0.29.37 \ # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_byod_3.9.in -dask[complete]==2022.10.1 ; python_version < "3.12" \ - --hash=sha256:2e6765bb6011c97c59fd4792540df679c703100443fcd99c82b98d8697295822 \ - --hash=sha256:79d283326045700af0de7e2be57fd663499958c63638bf5076839cbcde64aa3f +dask[complete]==2022.10.2 ; python_version < "3.12" \ + --hash=sha256:42cb43f601709575fa46ce09e74bea83fdd464187024f56954e09d9b428ceaab \ + --hash=sha256:928003a97b890a14c8a09a01f15320d261053bda530a8bf191d84f33db4a63b8 # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_byod_3.9.in @@ -673,9 +680,9 @@ diskcache==5.6.3 \ --hash=sha256:2c3a3fa2743d8535d832ec61c2054a1641f41775aa7c556758a109941e33e4fc \ --hash=sha256:5e31b2d5fbad117cc363ebaf6b689474db18a1f6438bc82358b024abd4c2ca19 # via petastorm -distributed==2022.10.1 ; python_version < "3.12" \ - --hash=sha256:31abab8ecc76951875828a3689d47dc4f20226b3ec99a0dc1af6183d02dbe5fe \ - --hash=sha256:42c6fe7d3bea491e23ce020879c411f2ecfecdb4914a6cb6b4a63530a7b0fa70 +distributed==2022.10.2 ; python_version < "3.12" \ + --hash=sha256:53f0a5bf6efab9a5ab3345cd913f6d3f3d4ea444ee2edbea331c7fef96fd67d0 \ + --hash=sha256:ae4fffdb55c6cb510ba1cbdf2856563af80ebf93e5ceacb91c1ce79e7da108d8 # via # -c release/ray_release/byod/requirements_compiled.txt # dask @@ -1264,7 +1271,7 @@ gsutil==5.27 \ # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_byod_3.9.in -gymnasium==1.0.0 \ +gymnasium[atari]==1.0.0 \ --hash=sha256:9d2b66f30c1b34fe3c2ce7fae65ecf365d0e9982d2b3d860235e773328a3b403 \ --hash=sha256:b6f40e1e24c5bd419361e1a5b86a9117d2499baecc3a660d44dfff4c465393ad # via @@ -2203,9 +2210,9 @@ pygments==2.18.0 \ # via # -c release/ray_release/byod/requirements_compiled.txt # rich -pyopenssl==23.0.0 \ - --hash=sha256:c1cc5f86bcacefc84dada7d31175cae1b1518d5f60d3d0bb595a67822a868a6f \ - --hash=sha256:df5fc28af899e74e19fccb5510df423581047e10ab6f1f4ba1763ff5fde844c0 +pyopenssl==24.2.1 \ + --hash=sha256:4247f0dbe3748d560dcbb2ff3ea01af0f9a1a001ef5f7c4c647956ed8cbf0e95 \ + --hash=sha256:967d5719b12b243588573f39b0c677637145c7a1ffedcd495a487e58177fbb8d # via # -c release/ray_release/byod/requirements_compiled.txt # gcs-oauth2-boto-plugin diff --git a/release/ray_release/byod/requirements_ml_byod_3.9.in b/release/ray_release/byod/requirements_ml_byod_3.9.in index 7ef915a8a698..69ca2cc2c734 100644 --- a/release/ray_release/byod/requirements_ml_byod_3.9.in +++ b/release/ray_release/byod/requirements_ml_byod_3.9.in @@ -6,7 +6,6 @@ bitsandbytes boto3 cmake crc32c -dataset datasets decord deepspeed>=0.12.3 diff --git a/release/ray_release/byod/requirements_ml_byod_3.9.txt b/release/ray_release/byod/requirements_ml_byod_3.9.txt index 7eeec76c2886..8d9e5a044f47 100644 --- a/release/ray_release/byod/requirements_ml_byod_3.9.txt +++ b/release/ray_release/byod/requirements_ml_byod_3.9.txt @@ -1,4 +1,3 @@ - # # This file is autogenerated by pip-compile with python 3.9 # To update, run: @@ -118,12 +117,6 @@ aiosignal==1.3.1 \ # via # -c release/ray_release/byod/requirements_compiled.txt # aiohttp -alembic==1.12.1 \ - --hash=sha256:47d52e3dfb03666ed945becb723d6482e52190917fdb47071440cfdba05d92cb \ - --hash=sha256:bca5877e9678b454706347bc10b97cb7d67f300320fa5c3a94423e8266e2823f - # via - # -c release/ray_release/byod/requirements_compiled.txt - # dataset annotated-types==0.6.0 \ --hash=sha256:0641064de18ba7a25dee8f96403ebc39113d0cb953a01429249d5c7564666a43 \ --hash=sha256:563339e807e53ffd9c267e99fc6d9ea23eb8443c08f112651963e24e22f84a5d @@ -175,17 +168,12 @@ attrs==21.4.0 \ # aiohttp # jsonlines # jsonschema - # markdown-it-py backcall==0.2.0 \ --hash=sha256:5cbdbf27be5e7cfadb448baf0aa95508f91f2bbc6c6437cd9cd06e2a4c215e1e \ --hash=sha256:fbbce6a29f263178a1f7915c1940bde0ec2b2a967566fe1c65c1dfb7422bd255 # via # -c release/ray_release/byod/requirements_compiled.txt # ipython -banal==1.0.6 \ - --hash=sha256:2fe02c9305f53168441948f4a03dfbfa2eacc73db30db4a93309083cb0e250a5 \ - --hash=sha256:877aacb16b17f8fa4fd29a7c44515c5a23dc1a7b26078bc41dd34829117d85e1 - # via dataset bitsandbytes==0.43.1 \ --hash=sha256:52c1c7189a6ca006555a9663e544e75f40520a97a26e075411f9f9aca0771fcd \ --hash=sha256:a81c826d576d6d691c7b4a7491c8fdc0f37f769795d6ca2e54afa605d2c260a3 @@ -510,12 +498,6 @@ comm==0.2.0 \ # via # -c release/ray_release/byod/requirements_compiled.txt # ipywidgets -commonmark==0.9.1 \ - --hash=sha256:452f9dc859be7f06631ddcb328b6919c67984aca654e5fefb3914d54691aed60 \ - --hash=sha256:da2f38c92590f83de410ba1a3cbceafbc74fee9def35f9251ba9a971d6d66fd9 - # via - # -c release/ray_release/byod/requirements_compiled.txt - # rich configargparse==1.7 \ --hash=sha256:d249da6591465c6c26df64a9f73d2536e743be2f244eb3ebe61114af2f94f86b \ --hash=sha256:e7067471884de5478c58a511e529f0f9bd1c66bfef1dea90935438d6c23306d1 @@ -652,33 +634,39 @@ crcmod==1.7 \ # via # -c release/ray_release/byod/requirements_compiled.txt # gsutil -cryptography==38.0.1 \ - --hash=sha256:0297ffc478bdd237f5ca3a7dc96fc0d315670bfa099c04dc3a4a2172008a405a \ - --hash=sha256:10d1f29d6292fc95acb597bacefd5b9e812099d75a6469004fd38ba5471a977f \ - --hash=sha256:16fa61e7481f4b77ef53991075de29fc5bacb582a1244046d2e8b4bb72ef66d0 \ - --hash=sha256:194044c6b89a2f9f169df475cc167f6157eb9151cc69af8a2a163481d45cc407 \ - --hash=sha256:1db3d807a14931fa317f96435695d9ec386be7b84b618cc61cfa5d08b0ae33d7 \ - --hash=sha256:3261725c0ef84e7592597606f6583385fed2a5ec3909f43bc475ade9729a41d6 \ - --hash=sha256:3b72c360427889b40f36dc214630e688c2fe03e16c162ef0aa41da7ab1455153 \ - --hash=sha256:3e3a2599e640927089f932295a9a247fc40a5bdf69b0484532f530471a382750 \ - --hash=sha256:3fc26e22840b77326a764ceb5f02ca2d342305fba08f002a8c1f139540cdfaad \ - --hash=sha256:5067ee7f2bce36b11d0e334abcd1ccf8c541fc0bbdaf57cdd511fdee53e879b6 \ - --hash=sha256:52e7bee800ec869b4031093875279f1ff2ed12c1e2f74923e8f49c916afd1d3b \ - --hash=sha256:64760ba5331e3f1794d0bcaabc0d0c39e8c60bf67d09c93dc0e54189dfd7cfe5 \ - --hash=sha256:765fa194a0f3372d83005ab83ab35d7c5526c4e22951e46059b8ac678b44fa5a \ - --hash=sha256:79473cf8a5cbc471979bd9378c9f425384980fcf2ab6534b18ed7d0d9843987d \ - --hash=sha256:896dd3a66959d3a5ddcfc140a53391f69ff1e8f25d93f0e2e7830c6de90ceb9d \ - --hash=sha256:89ed49784ba88c221756ff4d4755dbc03b3c8d2c5103f6d6b4f83a0fb1e85294 \ - --hash=sha256:ac7e48f7e7261207d750fa7e55eac2d45f720027d5703cd9007e9b37bbb59ac0 \ - --hash=sha256:ad7353f6ddf285aeadfaf79e5a6829110106ff8189391704c1d8801aa0bae45a \ - --hash=sha256:b0163a849b6f315bf52815e238bc2b2346604413fa7c1601eea84bcddb5fb9ac \ - --hash=sha256:b6c9b706316d7b5a137c35e14f4103e2115b088c412140fdbd5f87c73284df61 \ - --hash=sha256:c2e5856248a416767322c8668ef1845ad46ee62629266f84a8f007a317141013 \ - --hash=sha256:ca9f6784ea96b55ff41708b92c3f6aeaebde4c560308e5fbbd3173fbc466e94e \ - --hash=sha256:d1a5bd52d684e49a36582193e0b89ff267704cd4025abefb9e26803adeb3e5fb \ - --hash=sha256:d3971e2749a723e9084dd507584e2a2761f78ad2c638aa31e80bc7a15c9db4f9 \ - --hash=sha256:d4ef6cc305394ed669d4d9eebf10d3a101059bdcf2669c366ec1d14e4fb227bd \ - --hash=sha256:d9e69ae01f99abe6ad646947bba8941e896cb3aa805be2597a0400e0764b5818 +cryptography==42.0.5 \ + --hash=sha256:0270572b8bd2c833c3981724b8ee9747b3ec96f699a9665470018594301439ee \ + --hash=sha256:111a0d8553afcf8eb02a4fea6ca4f59d48ddb34497aa8706a6cf536f1a5ec576 \ + --hash=sha256:16a48c23a62a2f4a285699dba2e4ff2d1cff3115b9df052cdd976a18856d8e3d \ + --hash=sha256:1b95b98b0d2af784078fa69f637135e3c317091b615cd0905f8b8a087e86fa30 \ + --hash=sha256:1f71c10d1e88467126f0efd484bd44bca5e14c664ec2ede64c32f20875c0d413 \ + --hash=sha256:2424ff4c4ac7f6b8177b53c17ed5d8fa74ae5955656867f5a8affaca36a27abb \ + --hash=sha256:2bce03af1ce5a5567ab89bd90d11e7bbdff56b8af3acbbec1faded8f44cb06da \ + --hash=sha256:329906dcc7b20ff3cad13c069a78124ed8247adcac44b10bea1130e36caae0b4 \ + --hash=sha256:37dd623507659e08be98eec89323469e8c7b4c1407c85112634ae3dbdb926fdd \ + --hash=sha256:3eaafe47ec0d0ffcc9349e1708be2aaea4c6dd4978d76bf6eb0cb2c13636c6fc \ + --hash=sha256:5e6275c09d2badf57aea3afa80d975444f4be8d3bc58f7f80d2a484c6f9485c8 \ + --hash=sha256:6fe07eec95dfd477eb9530aef5bead34fec819b3aaf6c5bd6d20565da607bfe1 \ + --hash=sha256:7367d7b2eca6513681127ebad53b2582911d1736dc2ffc19f2c3ae49997496bc \ + --hash=sha256:7cde5f38e614f55e28d831754e8a3bacf9ace5d1566235e39d91b35502d6936e \ + --hash=sha256:9481ffe3cf013b71b2428b905c4f7a9a4f76ec03065b05ff499bb5682a8d9ad8 \ + --hash=sha256:98d8dc6d012b82287f2c3d26ce1d2dd130ec200c8679b6213b3c73c08b2b7940 \ + --hash=sha256:a011a644f6d7d03736214d38832e030d8268bcff4a41f728e6030325fea3e400 \ + --hash=sha256:a2913c5375154b6ef2e91c10b5720ea6e21007412f6437504ffea2109b5a33d7 \ + --hash=sha256:a30596bae9403a342c978fb47d9b0ee277699fa53bbafad14706af51fe543d16 \ + --hash=sha256:b03c2ae5d2f0fc05f9a2c0c997e1bc18c8229f392234e8a0194f202169ccd278 \ + --hash=sha256:b6cd2203306b63e41acdf39aa93b86fb566049aeb6dc489b70e34bcd07adca74 \ + --hash=sha256:b7ffe927ee6531c78f81aa17e684e2ff617daeba7f189f911065b2ea2d526dec \ + --hash=sha256:b8cac287fafc4ad485b8a9b67d0ee80c66bf3574f655d3b97ef2e1082360faf1 \ + --hash=sha256:ba334e6e4b1d92442b75ddacc615c5476d4ad55cc29b15d590cc6b86efa487e2 \ + --hash=sha256:ba3e4a42397c25b7ff88cdec6e2a16c2be18720f317506ee25210f6d31925f9c \ + --hash=sha256:c41fb5e6a5fe9ebcd58ca3abfeb51dffb5d83d6775405305bfa8715b76521922 \ + --hash=sha256:cd2030f6650c089aeb304cf093f3244d34745ce0cfcc39f20c6fbfe030102e2a \ + --hash=sha256:cd65d75953847815962c84a4654a84850b2bb4aed3f26fadcc1c13892e1e29f6 \ + --hash=sha256:e4985a790f921508f36f81831817cbc03b102d643b5fcb81cd33df3fa291a1a1 \ + --hash=sha256:e807b3188f9eb0eaa7bbb579b462c5ace579f1cedb28107ce8b48a9f7ad3679e \ + --hash=sha256:f12764b8fffc7a123f641d7d049d382b73f96a34117e0b637b80643169cec8ac \ + --hash=sha256:f8837fe1d6ac4a8052a9a8ddab256bc006242696f03368a4009be7ee3075cdb7 # via # -c release/ray_release/byod/requirements_compiled.txt # pyopenssl @@ -694,10 +682,6 @@ dataproperty==1.0.1 \ # via # pytablewriter # tabledata -dataset==1.6.2 \ - --hash=sha256:77d362118f67a8cbb4848dbd30ab362b9fa7cfebdbfaf426c9c500cb38969a99 \ - --hash=sha256:dcca9ba7658473d3082b1adf87a650252a1cd665705b73fa7d4ee32116a107b9 - # via -r release/ray_release/byod/requirements_ml_byod_3.9.in datasets==2.14.0 \ --hash=sha256:1bb3d1c992a593949a8d3e445b358ac1db4ead00e6619ea2e5e7b6dfc222dde1 \ --hash=sha256:93081cc3d9d0ce860c81f950a3ba23d24704da2eacbe2722092ef4f6ae0ada96 @@ -867,68 +851,84 @@ fonttools==4.45.1 \ # via # -c release/ray_release/byod/requirements_compiled.txt # matplotlib -frozenlist==1.4.0 \ - --hash=sha256:007df07a6e3eb3e33e9a1fe6a9db7af152bbd8a185f9aaa6ece10a3529e3e1c6 \ - --hash=sha256:008eb8b31b3ea6896da16c38c1b136cb9fec9e249e77f6211d479db79a4eaf01 \ - --hash=sha256:09163bdf0b2907454042edb19f887c6d33806adc71fbd54afc14908bfdc22251 \ - --hash=sha256:0c7c1b47859ee2cac3846fde1c1dc0f15da6cec5a0e5c72d101e0f83dcb67ff9 \ - --hash=sha256:0e5c8764c7829343d919cc2dfc587a8db01c4f70a4ebbc49abde5d4b158b007b \ - --hash=sha256:10ff5faaa22786315ef57097a279b833ecab1a0bfb07d604c9cbb1c4cdc2ed87 \ - --hash=sha256:17ae5cd0f333f94f2e03aaf140bb762c64783935cc764ff9c82dff626089bebf \ - --hash=sha256:19488c57c12d4e8095a922f328df3f179c820c212940a498623ed39160bc3c2f \ - --hash=sha256:1a0848b52815006ea6596c395f87449f693dc419061cc21e970f139d466dc0a0 \ - --hash=sha256:1e78fb68cf9c1a6aa4a9a12e960a5c9dfbdb89b3695197aa7064705662515de2 \ - --hash=sha256:261b9f5d17cac914531331ff1b1d452125bf5daa05faf73b71d935485b0c510b \ - --hash=sha256:2b8bcf994563466db019fab287ff390fffbfdb4f905fc77bc1c1d604b1c689cc \ - --hash=sha256:38461d02d66de17455072c9ba981d35f1d2a73024bee7790ac2f9e361ef1cd0c \ - --hash=sha256:490132667476f6781b4c9458298b0c1cddf237488abd228b0b3650e5ecba7467 \ - --hash=sha256:491e014f5c43656da08958808588cc6c016847b4360e327a62cb308c791bd2d9 \ - --hash=sha256:515e1abc578dd3b275d6a5114030b1330ba044ffba03f94091842852f806f1c1 \ - --hash=sha256:556de4430ce324c836789fa4560ca62d1591d2538b8ceb0b4f68fb7b2384a27a \ - --hash=sha256:5833593c25ac59ede40ed4de6d67eb42928cca97f26feea219f21d0ed0959b79 \ - --hash=sha256:6221d84d463fb110bdd7619b69cb43878a11d51cbb9394ae3105d082d5199167 \ - --hash=sha256:6918d49b1f90821e93069682c06ffde41829c346c66b721e65a5c62b4bab0300 \ - --hash=sha256:6c38721585f285203e4b4132a352eb3daa19121a035f3182e08e437cface44bf \ - --hash=sha256:71932b597f9895f011f47f17d6428252fc728ba2ae6024e13c3398a087c2cdea \ - --hash=sha256:7211ef110a9194b6042449431e08c4d80c0481e5891e58d429df5899690511c2 \ - --hash=sha256:764226ceef3125e53ea2cb275000e309c0aa5464d43bd72abd661e27fffc26ab \ - --hash=sha256:7645a8e814a3ee34a89c4a372011dcd817964ce8cb273c8ed6119d706e9613e3 \ - --hash=sha256:76d4711f6f6d08551a7e9ef28c722f4a50dd0fc204c56b4bcd95c6cc05ce6fbb \ - --hash=sha256:7f4f399d28478d1f604c2ff9119907af9726aed73680e5ed1ca634d377abb087 \ - --hash=sha256:88f7bc0fcca81f985f78dd0fa68d2c75abf8272b1f5c323ea4a01a4d7a614efc \ - --hash=sha256:8d0edd6b1c7fb94922bf569c9b092ee187a83f03fb1a63076e7774b60f9481a8 \ - --hash=sha256:901289d524fdd571be1c7be054f48b1f88ce8dddcbdf1ec698b27d4b8b9e5d62 \ - --hash=sha256:93ea75c050c5bb3d98016b4ba2497851eadf0ac154d88a67d7a6816206f6fa7f \ - --hash=sha256:981b9ab5a0a3178ff413bca62526bb784249421c24ad7381e39d67981be2c326 \ - --hash=sha256:9ac08e601308e41eb533f232dbf6b7e4cea762f9f84f6357136eed926c15d12c \ - --hash=sha256:a02eb8ab2b8f200179b5f62b59757685ae9987996ae549ccf30f983f40602431 \ - --hash=sha256:a0c6da9aee33ff0b1a451e867da0c1f47408112b3391dd43133838339e410963 \ - --hash=sha256:a6c8097e01886188e5be3e6b14e94ab365f384736aa1fca6a0b9e35bd4a30bc7 \ - --hash=sha256:aa384489fefeb62321b238e64c07ef48398fe80f9e1e6afeff22e140e0850eef \ - --hash=sha256:ad2a9eb6d9839ae241701d0918f54c51365a51407fd80f6b8289e2dfca977cc3 \ - --hash=sha256:b206646d176a007466358aa21d85cd8600a415c67c9bd15403336c331a10d956 \ - --hash=sha256:b826d97e4276750beca7c8f0f1a4938892697a6bcd8ec8217b3312dad6982781 \ - --hash=sha256:b89ac9768b82205936771f8d2eb3ce88503b1556324c9f903e7156669f521472 \ - --hash=sha256:bd7bd3b3830247580de99c99ea2a01416dfc3c34471ca1298bccabf86d0ff4dc \ - --hash=sha256:bdf1847068c362f16b353163391210269e4f0569a3c166bc6a9f74ccbfc7e839 \ - --hash=sha256:c11b0746f5d946fecf750428a95f3e9ebe792c1ee3b1e96eeba145dc631a9672 \ - --hash=sha256:c5374b80521d3d3f2ec5572e05adc94601985cc526fb276d0c8574a6d749f1b3 \ - --hash=sha256:ca265542ca427bf97aed183c1676e2a9c66942e822b14dc6e5f42e038f92a503 \ - --hash=sha256:ce31ae3e19f3c902de379cf1323d90c649425b86de7bbdf82871b8a2a0615f3d \ - --hash=sha256:ceb6ec0a10c65540421e20ebd29083c50e6d1143278746a4ef6bcf6153171eb8 \ - --hash=sha256:d081f13b095d74b67d550de04df1c756831f3b83dc9881c38985834387487f1b \ - --hash=sha256:d5655a942f5f5d2c9ed93d72148226d75369b4f6952680211972a33e59b1dfdc \ - --hash=sha256:d5a32087d720c608f42caed0ef36d2b3ea61a9d09ee59a5142d6070da9041b8f \ - --hash=sha256:d6484756b12f40003c6128bfcc3fa9f0d49a687e171186c2d85ec82e3758c559 \ - --hash=sha256:dd65632acaf0d47608190a71bfe46b209719bf2beb59507db08ccdbe712f969b \ - --hash=sha256:de343e75f40e972bae1ef6090267f8260c1446a1695e77096db6cfa25e759a95 \ - --hash=sha256:e29cda763f752553fa14c68fb2195150bfab22b352572cb36c43c47bedba70eb \ - --hash=sha256:e41f3de4df3e80de75845d3e743b3f1c4c8613c3997a912dbf0229fc61a8b963 \ - --hash=sha256:e66d2a64d44d50d2543405fb183a21f76b3b5fd16f130f5c99187c3fb4e64919 \ - --hash=sha256:e74b0506fa5aa5598ac6a975a12aa8928cbb58e1f5ac8360792ef15de1aa848f \ - --hash=sha256:f0ed05f5079c708fe74bf9027e95125334b6978bf07fd5ab923e9e55e5fbb9d3 \ - --hash=sha256:f61e2dc5ad442c52b4887f1fdc112f97caeff4d9e6ebe78879364ac59f1663e1 \ - --hash=sha256:fec520865f42e5c7f050c2a79038897b1c7d1595e907a9e08e3353293ffc948e +frozenlist==1.4.1 \ + --hash=sha256:04ced3e6a46b4cfffe20f9ae482818e34eba9b5fb0ce4056e4cc9b6e212d09b7 \ + --hash=sha256:0633c8d5337cb5c77acbccc6357ac49a1770b8c487e5b3505c57b949b4b82e98 \ + --hash=sha256:068b63f23b17df8569b7fdca5517edef76171cf3897eb68beb01341131fbd2ad \ + --hash=sha256:0c250a29735d4f15321007fb02865f0e6b6a41a6b88f1f523ca1596ab5f50bd5 \ + --hash=sha256:1979bc0aeb89b33b588c51c54ab0161791149f2461ea7c7c946d95d5f93b56ae \ + --hash=sha256:1a4471094e146b6790f61b98616ab8e44f72661879cc63fa1049d13ef711e71e \ + --hash=sha256:1b280e6507ea8a4fa0c0a7150b4e526a8d113989e28eaaef946cc77ffd7efc0a \ + --hash=sha256:1d0ce09d36d53bbbe566fe296965b23b961764c0bcf3ce2fa45f463745c04701 \ + --hash=sha256:20b51fa3f588ff2fe658663db52a41a4f7aa6c04f6201449c6c7c476bd255c0d \ + --hash=sha256:23b2d7679b73fe0e5a4560b672a39f98dfc6f60df63823b0a9970525325b95f6 \ + --hash=sha256:23b701e65c7b36e4bf15546a89279bd4d8675faabc287d06bbcfac7d3c33e1e6 \ + --hash=sha256:2471c201b70d58a0f0c1f91261542a03d9a5e088ed3dc6c160d614c01649c106 \ + --hash=sha256:27657df69e8801be6c3638054e202a135c7f299267f1a55ed3a598934f6c0d75 \ + --hash=sha256:29acab3f66f0f24674b7dc4736477bcd4bc3ad4b896f5f45379a67bce8b96868 \ + --hash=sha256:32453c1de775c889eb4e22f1197fe3bdfe457d16476ea407472b9442e6295f7a \ + --hash=sha256:3a670dc61eb0d0eb7080890c13de3066790f9049b47b0de04007090807c776b0 \ + --hash=sha256:3e0153a805a98f5ada7e09826255ba99fb4f7524bb81bf6b47fb702666484ae1 \ + --hash=sha256:410478a0c562d1a5bcc2f7ea448359fcb050ed48b3c6f6f4f18c313a9bdb1826 \ + --hash=sha256:442acde1e068288a4ba7acfe05f5f343e19fac87bfc96d89eb886b0363e977ec \ + --hash=sha256:48f6a4533887e189dae092f1cf981f2e3885175f7a0f33c91fb5b7b682b6bab6 \ + --hash=sha256:4f57dab5fe3407b6c0c1cc907ac98e8a189f9e418f3b6e54d65a718aaafe3950 \ + --hash=sha256:4f9c515e7914626b2a2e1e311794b4c35720a0be87af52b79ff8e1429fc25f19 \ + --hash=sha256:55fdc093b5a3cb41d420884cdaf37a1e74c3c37a31f46e66286d9145d2063bd0 \ + --hash=sha256:5667ed53d68d91920defdf4035d1cdaa3c3121dc0b113255124bcfada1cfa1b8 \ + --hash=sha256:590344787a90ae57d62511dd7c736ed56b428f04cd8c161fcc5e7232c130c69a \ + --hash=sha256:5a7d70357e7cee13f470c7883a063aae5fe209a493c57d86eb7f5a6f910fae09 \ + --hash=sha256:5c3894db91f5a489fc8fa6a9991820f368f0b3cbdb9cd8849547ccfab3392d86 \ + --hash=sha256:5c849d495bf5154cd8da18a9eb15db127d4dba2968d88831aff6f0331ea9bd4c \ + --hash=sha256:64536573d0a2cb6e625cf309984e2d873979709f2cf22839bf2d61790b448ad5 \ + --hash=sha256:693945278a31f2086d9bf3df0fe8254bbeaef1fe71e1351c3bd730aa7d31c41b \ + --hash=sha256:6db4667b187a6742b33afbbaf05a7bc551ffcf1ced0000a571aedbb4aa42fc7b \ + --hash=sha256:6eb73fa5426ea69ee0e012fb59cdc76a15b1283d6e32e4f8dc4482ec67d1194d \ + --hash=sha256:722e1124aec435320ae01ee3ac7bec11a5d47f25d0ed6328f2273d287bc3abb0 \ + --hash=sha256:7268252af60904bf52c26173cbadc3a071cece75f873705419c8681f24d3edea \ + --hash=sha256:74fb4bee6880b529a0c6560885fce4dc95936920f9f20f53d99a213f7bf66776 \ + --hash=sha256:780d3a35680ced9ce682fbcf4cb9c2bad3136eeff760ab33707b71db84664e3a \ + --hash=sha256:82e8211d69a4f4bc360ea22cd6555f8e61a1bd211d1d5d39d3d228b48c83a897 \ + --hash=sha256:89aa2c2eeb20957be2d950b85974b30a01a762f3308cd02bb15e1ad632e22dc7 \ + --hash=sha256:8aefbba5f69d42246543407ed2461db31006b0f76c4e32dfd6f42215a2c41d09 \ + --hash=sha256:96ec70beabbd3b10e8bfe52616a13561e58fe84c0101dd031dc78f250d5128b9 \ + --hash=sha256:9750cc7fe1ae3b1611bb8cfc3f9ec11d532244235d75901fb6b8e42ce9229dfe \ + --hash=sha256:9acbb16f06fe7f52f441bb6f413ebae6c37baa6ef9edd49cdd567216da8600cd \ + --hash=sha256:9d3e0c25a2350080e9319724dede4f31f43a6c9779be48021a7f4ebde8b2d742 \ + --hash=sha256:a06339f38e9ed3a64e4c4e43aec7f59084033647f908e4259d279a52d3757d09 \ + --hash=sha256:a0cb6f11204443f27a1628b0e460f37fb30f624be6051d490fa7d7e26d4af3d0 \ + --hash=sha256:a7496bfe1da7fb1a4e1cc23bb67c58fab69311cc7d32b5a99c2007b4b2a0e932 \ + --hash=sha256:a828c57f00f729620a442881cc60e57cfcec6842ba38e1b19fd3e47ac0ff8dc1 \ + --hash=sha256:a9b2de4cf0cdd5bd2dee4c4f63a653c61d2408055ab77b151c1957f221cabf2a \ + --hash=sha256:b46c8ae3a8f1f41a0d2ef350c0b6e65822d80772fe46b653ab6b6274f61d4a49 \ + --hash=sha256:b7e3ed87d4138356775346e6845cccbe66cd9e207f3cd11d2f0b9fd13681359d \ + --hash=sha256:b7f2f9f912dca3934c1baec2e4585a674ef16fe00218d833856408c48d5beee7 \ + --hash=sha256:ba60bb19387e13597fb059f32cd4d59445d7b18b69a745b8f8e5db0346f33480 \ + --hash=sha256:beee944ae828747fd7cb216a70f120767fc9f4f00bacae8543c14a6831673f89 \ + --hash=sha256:bfa4a17e17ce9abf47a74ae02f32d014c5e9404b6d9ac7f729e01562bbee601e \ + --hash=sha256:c037a86e8513059a2613aaba4d817bb90b9d9b6b69aace3ce9c877e8c8ed402b \ + --hash=sha256:c302220494f5c1ebeb0912ea782bcd5e2f8308037b3c7553fad0e48ebad6ad82 \ + --hash=sha256:c6321c9efe29975232da3bd0af0ad216800a47e93d763ce64f291917a381b8eb \ + --hash=sha256:c757a9dd70d72b076d6f68efdbb9bc943665ae954dad2801b874c8c69e185068 \ + --hash=sha256:c99169d4ff810155ca50b4da3b075cbde79752443117d89429595c2e8e37fed8 \ + --hash=sha256:c9c92be9fd329ac801cc420e08452b70e7aeab94ea4233a4804f0915c14eba9b \ + --hash=sha256:cc7b01b3754ea68a62bd77ce6020afaffb44a590c2289089289363472d13aedb \ + --hash=sha256:db9e724bebd621d9beca794f2a4ff1d26eed5965b004a97f1f1685a173b869c2 \ + --hash=sha256:dca69045298ce5c11fd539682cff879cc1e664c245d1c64da929813e54241d11 \ + --hash=sha256:dd9b1baec094d91bf36ec729445f7769d0d0cf6b64d04d86e45baf89e2b9059b \ + --hash=sha256:e02a0e11cf6597299b9f3bbd3f93d79217cb90cfd1411aec33848b13f5c656cc \ + --hash=sha256:e6a20a581f9ce92d389a8c7d7c3dd47c81fd5d6e655c8dddf341e14aa48659d0 \ + --hash=sha256:e7004be74cbb7d9f34553a5ce5fb08be14fb33bc86f332fb71cbe5216362a497 \ + --hash=sha256:e774d53b1a477a67838a904131c4b0eef6b3d8a651f8b138b04f748fccfefe17 \ + --hash=sha256:edb678da49d9f72c9f6c609fbe41a5dfb9a9282f9e6a2253d5a91e0fc382d7c0 \ + --hash=sha256:f146e0911cb2f1da549fc58fc7bcd2b836a44b79ef871980d605ec392ff6b0d2 \ + --hash=sha256:f56e2333dda1fe0f909e7cc59f021eba0d2307bc6f012a1ccf2beca6ba362439 \ + --hash=sha256:f9a3ea26252bd92f570600098783d1371354d89d5f6b7dfd87359d669f2109b5 \ + --hash=sha256:f9aa1878d1083b276b0196f2dfbe00c9b7e752475ed3b682025ff20c1c1f51ac \ + --hash=sha256:fb3c2db03683b5767dedb5769b8a40ebb47d6f7f45b1b3e3b4b51ec8ad9d9825 \ + --hash=sha256:fbeb989b5cc29e8daf7f976b421c220f1b8c731cbf22b9130d8815418ea45887 \ + --hash=sha256:fde5bd59ab5357e3853313127f4d3565fc7dad314a74d7b5d43c22c6a5ed2ced \ + --hash=sha256:fe1a06da377e3a1062ae5fe0926e12b84eceb8a50b350ddca72dc85015873f74 # via # -c release/ray_release/byod/requirements_compiled.txt # aiohttp @@ -964,8 +964,9 @@ fugue-sql-antlr==0.2.0 \ # via # -c release/ray_release/byod/requirements_compiled.txt # fugue -future==0.18.3 \ - --hash=sha256:34a17436ed1e96697a86f9de3d15a3b0be01d8bc8de9c1dffd59fb8234ed5307 +future==1.0.0 \ + --hash=sha256:929292d34f5872e70396626ef385ec22355a1fae8ad29e1a734c3e43f9fbc216 \ + --hash=sha256:bd2968309307861edae1458a4f8a4f3598c03be43b97521076aebf5d94c07b05 # via # -c release/ray_release/byod/requirements_compiled.txt # petastorm @@ -1295,7 +1296,6 @@ greenlet==3.0.1 \ # via # -c release/ray_release/byod/requirements_compiled.txt # gevent - # sqlalchemy gsutil==5.27 \ --hash=sha256:681a2d844acdf05fac989da6dd406944ae11cb27a4cf3c9edef74d2585ab5f05 # via @@ -1431,9 +1431,9 @@ jupyterlab-widgets==3.0.11 \ # via # -c release/ray_release/byod/requirements_compiled.txt # ipywidgets -jupytext==1.13.6 \ - --hash=sha256:2160774e30587fb427213231f0267ed070ba4ede41cf6121dbb2b14225eb83ba \ - --hash=sha256:c6c25918ddb6403d0d8504e08d35f6efc447baf0dbeb6a28b73adf39e866a0c4 +jupytext==1.16.3 \ + --hash=sha256:1ebac990461dd9f477ff7feec9e3003fa1acc89f3c16ba01b73f79fd76f01a98 \ + --hash=sha256:870e0d7a716dcb1303df6ad1cec65e3315a20daedd808a55cb3dae2d56e4ed20 # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in @@ -1680,19 +1680,14 @@ lxml==4.9.4 \ # via # -c release/ray_release/byod/requirements_compiled.txt # sacrebleu -mako==1.3.0 \ - --hash=sha256:57d4e997349f1a92035aa25c17ace371a4213f2ca42f99bee9a602500cfd54d9 \ - --hash=sha256:e3a9d388fd00e87043edbe8792f45880ac0114e9c4adc69f6e9bfb2c55e3b11b - # via - # -c release/ray_release/byod/requirements_compiled.txt - # alembic -markdown-it-py==1.1.0 \ - --hash=sha256:36be6bb3ad987bfdb839f5ba78ddf094552ca38ccbd784ae4f74a4e1419fc6e3 \ - --hash=sha256:98080fc0bc34c4f2bcf0846a096a9429acbd9d5d8e67ed34026c03c61c464389 +markdown-it-py==2.2.0 \ + --hash=sha256:5a35f8d1870171d9acc47b99612dc146129b631baf04970128b568f190d0cc30 \ + --hash=sha256:7c9a5e412688bc771c67432cbfebcdd686c93ce6484913dccf06cb5a0bea35a1 # via # -c release/ray_release/byod/requirements_compiled.txt # jupytext # mdit-py-plugins + # rich markupsafe==2.1.3 \ --hash=sha256:05fb21170423db021895e1ea1e1f3ab3adb85d1c2333cbc2310f2a26bc77272e \ --hash=sha256:0a4e4a1aff6c7ac4cd55792abf96c915634c2b97e3cc1c7129578aa68ebd754e \ @@ -1757,7 +1752,6 @@ markupsafe==2.1.3 \ # via # -c release/ray_release/byod/requirements_compiled.txt # jinja2 - # mako # werkzeug matplotlib==3.7.4 \ --hash=sha256:0037d066cca1f4bda626c507cddeb6f7da8283bc6a214da2db13ff2162933c52 \ @@ -1829,6 +1823,12 @@ mdit-py-plugins==0.3.5 \ # via # -c release/ray_release/byod/requirements_compiled.txt # jupytext +mdurl==0.1.2 \ + --hash=sha256:84008a41e51615a49fc9966191ff91509e3c40b939176e643fd50a5c2196b8f8 \ + --hash=sha256:bb413d29f5eea38f31dd4754dd7377d4465116fb207585f97bf925588687c1ba + # via + # -c release/ray_release/byod/requirements_compiled.txt + # markdown-it-py memray==1.10.0 ; platform_system != "Windows" and sys_platform != "darwin" and platform_machine != "aarch64" \ --hash=sha256:0a21745fb516b7a6efcd40aa7487c59e9313fcfc782d0193fcfcf00b48426874 \ --hash=sha256:22f2a47871c172a0539bd72737bb6b294fc10c510464066b825d90fcd3bb4916 \ @@ -1868,7 +1868,7 @@ memray==1.10.0 ; platform_system != "Windows" and sys_platform != "darwin" and p # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in -modin==0.22.2 \ +modin==0.22.2 ; python_version < "3.12" \ --hash=sha256:532fe0bfb2dcf06c0ad2d467721ef489fd58bb3ef7150bcf4a7ddd1069be1e4d \ --hash=sha256:fa897dc59d5b9a8496be044185689fdd337b9f26cc81c4144b217a2a94d029bc # via @@ -1952,81 +1952,97 @@ msgpack==1.0.7 \ # via # -c release/ray_release/byod/requirements_compiled.txt # locust -multidict==6.0.4 \ - --hash=sha256:01a3a55bd90018c9c080fbb0b9f4891db37d148a0a18722b42f94694f8b6d4c9 \ - --hash=sha256:0b1a97283e0c85772d613878028fec909f003993e1007eafa715b24b377cb9b8 \ - --hash=sha256:0dfad7a5a1e39c53ed00d2dd0c2e36aed4650936dc18fd9a1826a5ae1cad6f03 \ - --hash=sha256:11bdf3f5e1518b24530b8241529d2050014c884cf18b6fc69c0c2b30ca248710 \ - --hash=sha256:1502e24330eb681bdaa3eb70d6358e818e8e8f908a22a1851dfd4e15bc2f8161 \ - --hash=sha256:16ab77bbeb596e14212e7bab8429f24c1579234a3a462105cda4a66904998664 \ - --hash=sha256:16d232d4e5396c2efbbf4f6d4df89bfa905eb0d4dc5b3549d872ab898451f569 \ - --hash=sha256:21a12c4eb6ddc9952c415f24eef97e3e55ba3af61f67c7bc388dcdec1404a067 \ - --hash=sha256:27c523fbfbdfd19c6867af7346332b62b586eed663887392cff78d614f9ec313 \ - --hash=sha256:281af09f488903fde97923c7744bb001a9b23b039a909460d0f14edc7bf59706 \ - --hash=sha256:33029f5734336aa0d4c0384525da0387ef89148dc7191aae00ca5fb23d7aafc2 \ - --hash=sha256:3601a3cece3819534b11d4efc1eb76047488fddd0c85a3948099d5da4d504636 \ - --hash=sha256:3666906492efb76453c0e7b97f2cf459b0682e7402c0489a95484965dbc1da49 \ - --hash=sha256:36c63aaa167f6c6b04ef2c85704e93af16c11d20de1d133e39de6a0e84582a93 \ - --hash=sha256:39ff62e7d0f26c248b15e364517a72932a611a9b75f35b45be078d81bdb86603 \ - --hash=sha256:43644e38f42e3af682690876cff722d301ac585c5b9e1eacc013b7a3f7b696a0 \ - --hash=sha256:4372381634485bec7e46718edc71528024fcdc6f835baefe517b34a33c731d60 \ - --hash=sha256:458f37be2d9e4c95e2d8866a851663cbc76e865b78395090786f6cd9b3bbf4f4 \ - --hash=sha256:45e1ecb0379bfaab5eef059f50115b54571acfbe422a14f668fc8c27ba410e7e \ - --hash=sha256:4b9d9e4e2b37daddb5c23ea33a3417901fa7c7b3dee2d855f63ee67a0b21e5b1 \ - --hash=sha256:4ceef517eca3e03c1cceb22030a3e39cb399ac86bff4e426d4fc6ae49052cc60 \ - --hash=sha256:4d1a3d7ef5e96b1c9e92f973e43aa5e5b96c659c9bc3124acbbd81b0b9c8a951 \ - --hash=sha256:4dcbb0906e38440fa3e325df2359ac6cb043df8e58c965bb45f4e406ecb162cc \ - --hash=sha256:509eac6cf09c794aa27bcacfd4d62c885cce62bef7b2c3e8b2e49d365b5003fe \ - --hash=sha256:52509b5be062d9eafc8170e53026fbc54cf3b32759a23d07fd935fb04fc22d95 \ - --hash=sha256:52f2dffc8acaba9a2f27174c41c9e57f60b907bb9f096b36b1a1f3be71c6284d \ - --hash=sha256:574b7eae1ab267e5f8285f0fe881f17efe4b98c39a40858247720935b893bba8 \ - --hash=sha256:5979b5632c3e3534e42ca6ff856bb24b2e3071b37861c2c727ce220d80eee9ed \ - --hash=sha256:59d43b61c59d82f2effb39a93c48b845efe23a3852d201ed2d24ba830d0b4cf2 \ - --hash=sha256:5a4dcf02b908c3b8b17a45fb0f15b695bf117a67b76b7ad18b73cf8e92608775 \ - --hash=sha256:5cad9430ab3e2e4fa4a2ef4450f548768400a2ac635841bc2a56a2052cdbeb87 \ - --hash=sha256:5fc1b16f586f049820c5c5b17bb4ee7583092fa0d1c4e28b5239181ff9532e0c \ - --hash=sha256:62501642008a8b9871ddfccbf83e4222cf8ac0d5aeedf73da36153ef2ec222d2 \ - --hash=sha256:64bdf1086b6043bf519869678f5f2757f473dee970d7abf6da91ec00acb9cb98 \ - --hash=sha256:64da238a09d6039e3bd39bb3aee9c21a5e34f28bfa5aa22518581f910ff94af3 \ - --hash=sha256:666daae833559deb2d609afa4490b85830ab0dfca811a98b70a205621a6109fe \ - --hash=sha256:67040058f37a2a51ed8ea8f6b0e6ee5bd78ca67f169ce6122f3e2ec80dfe9b78 \ - --hash=sha256:6748717bb10339c4760c1e63da040f5f29f5ed6e59d76daee30305894069a660 \ - --hash=sha256:6b181d8c23da913d4ff585afd1155a0e1194c0b50c54fcfe286f70cdaf2b7176 \ - --hash=sha256:6ed5f161328b7df384d71b07317f4d8656434e34591f20552c7bcef27b0ab88e \ - --hash=sha256:7582a1d1030e15422262de9f58711774e02fa80df0d1578995c76214f6954988 \ - --hash=sha256:7d18748f2d30f94f498e852c67d61261c643b349b9d2a581131725595c45ec6c \ - --hash=sha256:7d6ae9d593ef8641544d6263c7fa6408cc90370c8cb2bbb65f8d43e5b0351d9c \ - --hash=sha256:81a4f0b34bd92df3da93315c6a59034df95866014ac08535fc819f043bfd51f0 \ - --hash=sha256:8316a77808c501004802f9beebde51c9f857054a0c871bd6da8280e718444449 \ - --hash=sha256:853888594621e6604c978ce2a0444a1e6e70c8d253ab65ba11657659dcc9100f \ - --hash=sha256:99b76c052e9f1bc0721f7541e5e8c05db3941eb9ebe7b8553c625ef88d6eefde \ - --hash=sha256:a2e4369eb3d47d2034032a26c7a80fcb21a2cb22e1173d761a162f11e562caa5 \ - --hash=sha256:ab55edc2e84460694295f401215f4a58597f8f7c9466faec545093045476327d \ - --hash=sha256:af048912e045a2dc732847d33821a9d84ba553f5c5f028adbd364dd4765092ac \ - --hash=sha256:b1a2eeedcead3a41694130495593a559a668f382eee0727352b9a41e1c45759a \ - --hash=sha256:b1e8b901e607795ec06c9e42530788c45ac21ef3aaa11dbd0c69de543bfb79a9 \ - --hash=sha256:b41156839806aecb3641f3208c0dafd3ac7775b9c4c422d82ee2a45c34ba81ca \ - --hash=sha256:b692f419760c0e65d060959df05f2a531945af31fda0c8a3b3195d4efd06de11 \ - --hash=sha256:bc779e9e6f7fda81b3f9aa58e3a6091d49ad528b11ed19f6621408806204ad35 \ - --hash=sha256:bf6774e60d67a9efe02b3616fee22441d86fab4c6d335f9d2051d19d90a40063 \ - --hash=sha256:c048099e4c9e9d615545e2001d3d8a4380bd403e1a0578734e0d31703d1b0c0b \ - --hash=sha256:c5cb09abb18c1ea940fb99360ea0396f34d46566f157122c92dfa069d3e0e982 \ - --hash=sha256:cc8e1d0c705233c5dd0c5e6460fbad7827d5d36f310a0fadfd45cc3029762258 \ - --hash=sha256:d5e3fc56f88cc98ef8139255cf8cd63eb2c586531e43310ff859d6bb3a6b51f1 \ - --hash=sha256:d6aa0418fcc838522256761b3415822626f866758ee0bc6632c9486b179d0b52 \ - --hash=sha256:d6c254ba6e45d8e72739281ebc46ea5eb5f101234f3ce171f0e9f5cc86991480 \ - --hash=sha256:d6d635d5209b82a3492508cf5b365f3446afb65ae7ebd755e70e18f287b0adf7 \ - --hash=sha256:dcfe792765fab89c365123c81046ad4103fcabbc4f56d1c1997e6715e8015461 \ - --hash=sha256:ddd3915998d93fbcd2566ddf9cf62cdb35c9e093075f862935573d265cf8f65d \ - --hash=sha256:ddff9c4e225a63a5afab9dd15590432c22e8057e1a9a13d28ed128ecf047bbdc \ - --hash=sha256:e41b7e2b59679edfa309e8db64fdf22399eec4b0b24694e1b2104fb789207779 \ - --hash=sha256:e69924bfcdda39b722ef4d9aa762b2dd38e4632b3641b1d9a57ca9cd18f2f83a \ - --hash=sha256:ea20853c6dbbb53ed34cb4d080382169b6f4554d394015f1bef35e881bf83547 \ - --hash=sha256:ee2a1ece51b9b9e7752e742cfb661d2a29e7bcdba2d27e66e28a99f1890e4fa0 \ - --hash=sha256:eeb6dcc05e911516ae3d1f207d4b0520d07f54484c49dfc294d6e7d63b734171 \ - --hash=sha256:f70b98cd94886b49d91170ef23ec5c0e8ebb6f242d734ed7ed677b24d50c82cf \ - --hash=sha256:fc35cb4676846ef752816d5be2193a1e8367b4c1397b74a565a9d0389c433a1d \ - --hash=sha256:ff959bee35038c4624250473988b24f846cbeb2c6639de3602c073f10410ceba +multidict==6.0.5 \ + --hash=sha256:01265f5e40f5a17f8241d52656ed27192be03bfa8764d88e8220141d1e4b3556 \ + --hash=sha256:0275e35209c27a3f7951e1ce7aaf93ce0d163b28948444bec61dd7badc6d3f8c \ + --hash=sha256:04bde7a7b3de05732a4eb39c94574db1ec99abb56162d6c520ad26f83267de29 \ + --hash=sha256:04da1bb8c8dbadf2a18a452639771951c662c5ad03aefe4884775454be322c9b \ + --hash=sha256:09a892e4a9fb47331da06948690ae38eaa2426de97b4ccbfafbdcbe5c8f37ff8 \ + --hash=sha256:0d63c74e3d7ab26de115c49bffc92cc77ed23395303d496eae515d4204a625e7 \ + --hash=sha256:107c0cdefe028703fb5dafe640a409cb146d44a6ae201e55b35a4af8e95457dd \ + --hash=sha256:141b43360bfd3bdd75f15ed811850763555a251e38b2405967f8e25fb43f7d40 \ + --hash=sha256:14c2976aa9038c2629efa2c148022ed5eb4cb939e15ec7aace7ca932f48f9ba6 \ + --hash=sha256:19fe01cea168585ba0f678cad6f58133db2aa14eccaf22f88e4a6dccadfad8b3 \ + --hash=sha256:1d147090048129ce3c453f0292e7697d333db95e52616b3793922945804a433c \ + --hash=sha256:1d9ea7a7e779d7a3561aade7d596649fbecfa5c08a7674b11b423783217933f9 \ + --hash=sha256:215ed703caf15f578dca76ee6f6b21b7603791ae090fbf1ef9d865571039ade5 \ + --hash=sha256:21fd81c4ebdb4f214161be351eb5bcf385426bf023041da2fd9e60681f3cebae \ + --hash=sha256:220dd781e3f7af2c2c1053da9fa96d9cf3072ca58f057f4c5adaaa1cab8fc442 \ + --hash=sha256:228b644ae063c10e7f324ab1ab6b548bdf6f8b47f3ec234fef1093bc2735e5f9 \ + --hash=sha256:29bfeb0dff5cb5fdab2023a7a9947b3b4af63e9c47cae2a10ad58394b517fddc \ + --hash=sha256:2f4848aa3baa109e6ab81fe2006c77ed4d3cd1e0ac2c1fbddb7b1277c168788c \ + --hash=sha256:2faa5ae9376faba05f630d7e5e6be05be22913782b927b19d12b8145968a85ea \ + --hash=sha256:2ffc42c922dbfddb4a4c3b438eb056828719f07608af27d163191cb3e3aa6cc5 \ + --hash=sha256:37b15024f864916b4951adb95d3a80c9431299080341ab9544ed148091b53f50 \ + --hash=sha256:3cc2ad10255f903656017363cd59436f2111443a76f996584d1077e43ee51182 \ + --hash=sha256:3d25f19500588cbc47dc19081d78131c32637c25804df8414463ec908631e453 \ + --hash=sha256:403c0911cd5d5791605808b942c88a8155c2592e05332d2bf78f18697a5fa15e \ + --hash=sha256:411bf8515f3be9813d06004cac41ccf7d1cd46dfe233705933dd163b60e37600 \ + --hash=sha256:425bf820055005bfc8aa9a0b99ccb52cc2f4070153e34b701acc98d201693733 \ + --hash=sha256:435a0984199d81ca178b9ae2c26ec3d49692d20ee29bc4c11a2a8d4514c67eda \ + --hash=sha256:4a6a4f196f08c58c59e0b8ef8ec441d12aee4125a7d4f4fef000ccb22f8d7241 \ + --hash=sha256:4cc0ef8b962ac7a5e62b9e826bd0cd5040e7d401bc45a6835910ed699037a461 \ + --hash=sha256:51d035609b86722963404f711db441cf7134f1889107fb171a970c9701f92e1e \ + --hash=sha256:53689bb4e102200a4fafa9de9c7c3c212ab40a7ab2c8e474491914d2305f187e \ + --hash=sha256:55205d03e8a598cfc688c71ca8ea5f66447164efff8869517f175ea632c7cb7b \ + --hash=sha256:5c0631926c4f58e9a5ccce555ad7747d9a9f8b10619621f22f9635f069f6233e \ + --hash=sha256:5cb241881eefd96b46f89b1a056187ea8e9ba14ab88ba632e68d7a2ecb7aadf7 \ + --hash=sha256:60d698e8179a42ec85172d12f50b1668254628425a6bd611aba022257cac1386 \ + --hash=sha256:612d1156111ae11d14afaf3a0669ebf6c170dbb735e510a7438ffe2369a847fd \ + --hash=sha256:6214c5a5571802c33f80e6c84713b2c79e024995b9c5897f794b43e714daeec9 \ + --hash=sha256:6939c95381e003f54cd4c5516740faba40cf5ad3eeff460c3ad1d3e0ea2549bf \ + --hash=sha256:69db76c09796b313331bb7048229e3bee7928eb62bab5e071e9f7fcc4879caee \ + --hash=sha256:6bf7a982604375a8d49b6cc1b781c1747f243d91b81035a9b43a2126c04766f5 \ + --hash=sha256:766c8f7511df26d9f11cd3a8be623e59cca73d44643abab3f8c8c07620524e4a \ + --hash=sha256:76c0de87358b192de7ea9649beb392f107dcad9ad27276324c24c91774ca5271 \ + --hash=sha256:76f067f5121dcecf0d63a67f29080b26c43c71a98b10c701b0677e4a065fbd54 \ + --hash=sha256:7901c05ead4b3fb75113fb1dd33eb1253c6d3ee37ce93305acd9d38e0b5f21a4 \ + --hash=sha256:79660376075cfd4b2c80f295528aa6beb2058fd289f4c9252f986751a4cd0496 \ + --hash=sha256:79a6d2ba910adb2cbafc95dad936f8b9386e77c84c35bc0add315b856d7c3abb \ + --hash=sha256:7afcdd1fc07befad18ec4523a782cde4e93e0a2bf71239894b8d61ee578c1319 \ + --hash=sha256:7be7047bd08accdb7487737631d25735c9a04327911de89ff1b26b81745bd4e3 \ + --hash=sha256:7c6390cf87ff6234643428991b7359b5f59cc15155695deb4eda5c777d2b880f \ + --hash=sha256:7df704ca8cf4a073334e0427ae2345323613e4df18cc224f647f251e5e75a527 \ + --hash=sha256:85f67aed7bb647f93e7520633d8f51d3cbc6ab96957c71272b286b2f30dc70ed \ + --hash=sha256:896ebdcf62683551312c30e20614305f53125750803b614e9e6ce74a96232604 \ + --hash=sha256:92d16a3e275e38293623ebf639c471d3e03bb20b8ebb845237e0d3664914caef \ + --hash=sha256:99f60d34c048c5c2fabc766108c103612344c46e35d4ed9ae0673d33c8fb26e8 \ + --hash=sha256:9fe7b0653ba3d9d65cbe7698cca585bf0f8c83dbbcc710db9c90f478e175f2d5 \ + --hash=sha256:a3145cb08d8625b2d3fee1b2d596a8766352979c9bffe5d7833e0503d0f0b5e5 \ + --hash=sha256:aeaf541ddbad8311a87dd695ed9642401131ea39ad7bc8cf3ef3967fd093b626 \ + --hash=sha256:b55358304d7a73d7bdf5de62494aaf70bd33015831ffd98bc498b433dfe5b10c \ + --hash=sha256:b82cc8ace10ab5bd93235dfaab2021c70637005e1ac787031f4d1da63d493c1d \ + --hash=sha256:c0868d64af83169e4d4152ec612637a543f7a336e4a307b119e98042e852ad9c \ + --hash=sha256:c1c1496e73051918fcd4f58ff2e0f2f3066d1c76a0c6aeffd9b45d53243702cc \ + --hash=sha256:c9bf56195c6bbd293340ea82eafd0071cb3d450c703d2c93afb89f93b8386ccc \ + --hash=sha256:cbebcd5bcaf1eaf302617c114aa67569dd3f090dd0ce8ba9e35e9985b41ac35b \ + --hash=sha256:cd6c8fca38178e12c00418de737aef1261576bd1b6e8c6134d3e729a4e858b38 \ + --hash=sha256:ceb3b7e6a0135e092de86110c5a74e46bda4bd4fbfeeb3a3bcec79c0f861e450 \ + --hash=sha256:cf590b134eb70629e350691ecca88eac3e3b8b3c86992042fb82e3cb1830d5e1 \ + --hash=sha256:d3eb1ceec286eba8220c26f3b0096cf189aea7057b6e7b7a2e60ed36b373b77f \ + --hash=sha256:d65f25da8e248202bd47445cec78e0025c0fe7582b23ec69c3b27a640dd7a8e3 \ + --hash=sha256:d6f6d4f185481c9669b9447bf9d9cf3b95a0e9df9d169bbc17e363b7d5487755 \ + --hash=sha256:d84a5c3a5f7ce6db1f999fb9438f686bc2e09d38143f2d93d8406ed2dd6b9226 \ + --hash=sha256:d946b0a9eb8aaa590df1fe082cee553ceab173e6cb5b03239716338629c50c7a \ + --hash=sha256:dce1c6912ab9ff5f179eaf6efe7365c1f425ed690b03341911bf4939ef2f3046 \ + --hash=sha256:de170c7b4fe6859beb8926e84f7d7d6c693dfe8e27372ce3b76f01c46e489fcf \ + --hash=sha256:e02021f87a5b6932fa6ce916ca004c4d441509d33bbdbeca70d05dff5e9d2479 \ + --hash=sha256:e030047e85cbcedbfc073f71836d62dd5dadfbe7531cae27789ff66bc551bd5e \ + --hash=sha256:e0e79d91e71b9867c73323a3444724d496c037e578a0e1755ae159ba14f4f3d1 \ + --hash=sha256:e4428b29611e989719874670fd152b6625500ad6c686d464e99f5aaeeaca175a \ + --hash=sha256:e4972624066095e52b569e02b5ca97dbd7a7ddd4294bf4e7247d52635630dd83 \ + --hash=sha256:e7be68734bd8c9a513f2b0cfd508802d6609da068f40dc57d4e3494cefc92929 \ + --hash=sha256:e8e94e6912639a02ce173341ff62cc1201232ab86b8a8fcc05572741a5dc7d93 \ + --hash=sha256:ea1456df2a27c73ce51120fa2f519f1bea2f4a03a917f4a43c8707cf4cbbae1a \ + --hash=sha256:ebd8d160f91a764652d3e51ce0d2956b38efe37c9231cd82cfc0bed2e40b581c \ + --hash=sha256:eca2e9d0cc5a889850e9bbd68e98314ada174ff6ccd1129500103df7a94a7a44 \ + --hash=sha256:edd08e6f2f1a390bf137080507e44ccc086353c8e98c657e666c017718561b89 \ + --hash=sha256:f285e862d2f153a70586579c15c44656f888806ed0e5b56b64489afe4a2dbfba \ + --hash=sha256:f2a1dee728b52b33eebff5072817176c172050d44d67befd681609b4746e1c2e \ + --hash=sha256:f7e301075edaf50500f0b341543c41194d8df3ae5caf4702f2095f3ca73dd8da \ + --hash=sha256:fb616be3538599e797a2017cccca78e354c767165e8858ab5116813146041a24 \ + --hash=sha256:fce28b3c8a81b6b36dfac9feb1de115bab619b3c13905b419ec71d03a3fc1423 \ + --hash=sha256:fe5d7785250541f7f5019ab9cba2c71169dc7d74d0f45253f8313f436458a4ef # via # -c release/ray_release/byod/requirements_compiled.txt # aiohttp @@ -2219,6 +2235,12 @@ numpy==1.26.4 \ # triad # utilsforecast # xgboost +nvidia-nccl-cu12==2.20.5 \ + --hash=sha256:057f6bf9685f75215d0c53bf3ac4a10b3e6578351de307abad9e18a99182af56 \ + --hash=sha256:1fc150d5c3250b170b29410ba682384b14581db722b2531b0d8d33c595f33d01 + # via + # -c release/ray_release/byod/requirements_compiled.txt + # xgboost oauth2client==4.1.3 \ --hash=sha256:b8a81cc5d60e2d364f0b1b98f958dbd472887acaf1a5b05e21c28c31a2d6d3ac \ --hash=sha256:d486741e451287f69568a4d26d70d9acd73a2bbfa275746c535b4209891cccc6 @@ -2250,6 +2272,7 @@ packaging==23.0 \ # evaluate # fugue-sql-antlr # huggingface-hub + # jupytext # lightning-utilities # matplotlib # modin @@ -2263,7 +2286,7 @@ packaging==23.0 \ # transformers # typepy # utilsforecast -pandas==1.5.3 \ +pandas==1.5.3 ; python_version < "3.12" \ --hash=sha256:14e45300521902689a81f3f41386dc86f19b8ba8dd5ac5a3c7010ef8d2932813 \ --hash=sha256:26d9c71772c7afb9d5046e6e9cf42d83dd147b5cf5bcb9d97252077118543792 \ --hash=sha256:3749077d86e3a2f0ed51367f30bf5b82e131cc0f14260c4d3e499186fccc4406 \ @@ -2710,9 +2733,9 @@ pynvml==11.5.0 \ # via # -c release/ray_release/byod/requirements_compiled.txt # deepspeed -pyopenssl==23.0.0 \ - --hash=sha256:c1cc5f86bcacefc84dada7d31175cae1b1518d5f60d3d0bb595a67822a868a6f \ - --hash=sha256:df5fc28af899e74e19fccb5510df423581047e10ab6f1f4ba1763ff5fde844c0 +pyopenssl==24.2.1 \ + --hash=sha256:4247f0dbe3748d560dcbb2ff3ea01af0f9a1a001ef5f7c4c647956ed8cbf0e95 \ + --hash=sha256:967d5719b12b243588573f39b0c677637145c7a1ffedcd495a487e58177fbb8d # via # -c release/ray_release/byod/requirements_compiled.txt # gcs-oauth2-boto-plugin @@ -2963,95 +2986,86 @@ qpd==0.4.4 \ # via # -c release/ray_release/byod/requirements_compiled.txt # fugue -regex==2023.10.3 \ - --hash=sha256:00ba3c9818e33f1fa974693fb55d24cdc8ebafcb2e4207680669d8f8d7cca79a \ - --hash=sha256:00e871d83a45eee2f8688d7e6849609c2ca2a04a6d48fba3dff4deef35d14f07 \ - --hash=sha256:06e9abc0e4c9ab4779c74ad99c3fc10d3967d03114449acc2c2762ad4472b8ca \ - --hash=sha256:0b9ac09853b2a3e0d0082104036579809679e7715671cfbf89d83c1cb2a30f58 \ - --hash=sha256:0d47840dc05e0ba04fe2e26f15126de7c755496d5a8aae4a08bda4dd8d646c54 \ - --hash=sha256:0f649fa32fe734c4abdfd4edbb8381c74abf5f34bc0b3271ce687b23729299ed \ - --hash=sha256:107ac60d1bfdc3edb53be75e2a52aff7481b92817cfdddd9b4519ccf0e54a6ff \ - --hash=sha256:11175910f62b2b8c055f2b089e0fedd694fe2be3941b3e2633653bc51064c528 \ - --hash=sha256:12bd4bc2c632742c7ce20db48e0d99afdc05e03f0b4c1af90542e05b809a03d9 \ - --hash=sha256:16f8740eb6dbacc7113e3097b0a36065a02e37b47c936b551805d40340fb9971 \ - --hash=sha256:1c0e8fae5b27caa34177bdfa5a960c46ff2f78ee2d45c6db15ae3f64ecadde14 \ - --hash=sha256:2c54e23836650bdf2c18222c87f6f840d4943944146ca479858404fedeb9f9af \ - --hash=sha256:3367007ad1951fde612bf65b0dffc8fd681a4ab98ac86957d16491400d661302 \ - --hash=sha256:36362386b813fa6c9146da6149a001b7bd063dabc4d49522a1f7aa65b725c7ec \ - --hash=sha256:39807cbcbe406efca2a233884e169d056c35aa7e9f343d4e78665246a332f597 \ - --hash=sha256:39cdf8d141d6d44e8d5a12a8569d5a227f645c87df4f92179bd06e2e2705e76b \ - --hash=sha256:3b2c3502603fab52d7619b882c25a6850b766ebd1b18de3df23b2f939360e1bd \ - --hash=sha256:3ccf2716add72f80714b9a63899b67fa711b654be3fcdd34fa391d2d274ce767 \ - --hash=sha256:3fef4f844d2290ee0ba57addcec17eec9e3df73f10a2748485dfd6a3a188cc0f \ - --hash=sha256:4023e2efc35a30e66e938de5aef42b520c20e7eda7bb5fb12c35e5d09a4c43f6 \ - --hash=sha256:4a3ee019a9befe84fa3e917a2dd378807e423d013377a884c1970a3c2792d293 \ - --hash=sha256:4a8bf76e3182797c6b1afa5b822d1d5802ff30284abe4599e1247be4fd6b03be \ - --hash=sha256:4a992f702c9be9c72fa46f01ca6e18d131906a7180950958f766c2aa294d4b41 \ - --hash=sha256:4c34d4f73ea738223a094d8e0ffd6d2c1a1b4c175da34d6b0de3d8d69bee6bcc \ - --hash=sha256:4cd1bccf99d3ef1ab6ba835308ad85be040e6a11b0977ef7ea8c8005f01a3c29 \ - --hash=sha256:4ef80829117a8061f974b2fda8ec799717242353bff55f8a29411794d635d964 \ - --hash=sha256:58837f9d221744d4c92d2cf7201c6acd19623b50c643b56992cbd2b745485d3d \ - --hash=sha256:5a8f91c64f390ecee09ff793319f30a0f32492e99f5dc1c72bc361f23ccd0a9a \ - --hash=sha256:5addc9d0209a9afca5fc070f93b726bf7003bd63a427f65ef797a931782e7edc \ - --hash=sha256:6239d4e2e0b52c8bd38c51b760cd870069f0bdf99700a62cd509d7a031749a55 \ - --hash=sha256:66e2fe786ef28da2b28e222c89502b2af984858091675044d93cb50e6f46d7af \ - --hash=sha256:69c0771ca5653c7d4b65203cbfc5e66db9375f1078689459fe196fe08b7b4930 \ - --hash=sha256:6ac965a998e1388e6ff2e9781f499ad1eaa41e962a40d11c7823c9952c77123e \ - --hash=sha256:6c56c3d47da04f921b73ff9415fbaa939f684d47293f071aa9cbb13c94afc17d \ - --hash=sha256:6f85739e80d13644b981a88f529d79c5bdf646b460ba190bffcaf6d57b2a9863 \ - --hash=sha256:706e7b739fdd17cb89e1fbf712d9dc21311fc2333f6d435eac2d4ee81985098c \ - --hash=sha256:741ba2f511cc9626b7561a440f87d658aabb3d6b744a86a3c025f866b4d19e7f \ - --hash=sha256:7434a61b158be563c1362d9071358f8ab91b8d928728cd2882af060481244c9e \ - --hash=sha256:76066d7ff61ba6bf3cb5efe2428fc82aac91802844c022d849a1f0f53820502d \ - --hash=sha256:7979b834ec7a33aafae34a90aad9f914c41fd6eaa8474e66953f3f6f7cbd4368 \ - --hash=sha256:7eece6fbd3eae4a92d7c748ae825cbc1ee41a89bb1c3db05b5578ed3cfcfd7cb \ - --hash=sha256:7ef1e014eed78ab650bef9a6a9cbe50b052c0aebe553fb2881e0453717573f52 \ - --hash=sha256:81dce2ddc9f6e8f543d94b05d56e70d03a0774d32f6cca53e978dc01e4fc75b8 \ - --hash=sha256:82fcc1f1cc3ff1ab8a57ba619b149b907072e750815c5ba63e7aa2e1163384a4 \ - --hash=sha256:8d1f21af4c1539051049796a0f50aa342f9a27cde57318f2fc41ed50b0dbc4ac \ - --hash=sha256:90a79bce019c442604662d17bf69df99090e24cdc6ad95b18b6725c2988a490e \ - --hash=sha256:9145f092b5d1977ec8c0ab46e7b3381b2fd069957b9862a43bd383e5c01d18c2 \ - --hash=sha256:91dc1d531f80c862441d7b66c4505cd6ea9d312f01fb2f4654f40c6fdf5cc37a \ - --hash=sha256:979c24cbefaf2420c4e377ecd1f165ea08cc3d1fbb44bdc51bccbbf7c66a2cb4 \ - --hash=sha256:994645a46c6a740ee8ce8df7911d4aee458d9b1bc5639bc968226763d07f00fa \ - --hash=sha256:9b98b7681a9437262947f41c7fac567c7e1f6eddd94b0483596d320092004533 \ - --hash=sha256:9c6b4d23c04831e3ab61717a707a5d763b300213db49ca680edf8bf13ab5d91b \ - --hash=sha256:9c6d0ced3c06d0f183b73d3c5920727268d2201aa0fe6d55c60d68c792ff3588 \ - --hash=sha256:9fd88f373cb71e6b59b7fa597e47e518282455c2734fd4306a05ca219a1991b0 \ - --hash=sha256:a8f4e49fc3ce020f65411432183e6775f24e02dff617281094ba6ab079ef0915 \ - --hash=sha256:a9e908ef5889cda4de038892b9accc36d33d72fb3e12c747e2799a0e806ec841 \ - --hash=sha256:ad08a69728ff3c79866d729b095872afe1e0557251da4abb2c5faff15a91d19a \ - --hash=sha256:adbccd17dcaff65704c856bd29951c58a1bd4b2b0f8ad6b826dbd543fe740988 \ - --hash=sha256:b0c7d2f698e83f15228ba41c135501cfe7d5740181d5903e250e47f617eb4292 \ - --hash=sha256:b3ab05a182c7937fb374f7e946f04fb23a0c0699c0450e9fb02ef567412d2fa3 \ - --hash=sha256:b6104f9a46bd8743e4f738afef69b153c4b8b592d35ae46db07fc28ae3d5fb7c \ - --hash=sha256:ba7cd6dc4d585ea544c1412019921570ebd8a597fabf475acc4528210d7c4a6f \ - --hash=sha256:bc72c231f5449d86d6c7d9cc7cd819b6eb30134bb770b8cfdc0765e48ef9c420 \ - --hash=sha256:bce8814b076f0ce5766dc87d5a056b0e9437b8e0cd351b9a6c4e1134a7dfbda9 \ - --hash=sha256:be5e22bbb67924dea15039c3282fa4cc6cdfbe0cbbd1c0515f9223186fc2ec5f \ - --hash=sha256:be6b7b8d42d3090b6c80793524fa66c57ad7ee3fe9722b258aec6d0672543fd0 \ - --hash=sha256:bfe50b61bab1b1ec260fa7cd91106fa9fece57e6beba05630afe27c71259c59b \ - --hash=sha256:bff507ae210371d4b1fe316d03433ac099f184d570a1a611e541923f78f05037 \ - --hash=sha256:c148bec483cc4b421562b4bcedb8e28a3b84fcc8f0aa4418e10898f3c2c0eb9b \ - --hash=sha256:c15ad0aee158a15e17e0495e1e18741573d04eb6da06d8b84af726cfc1ed02ee \ - --hash=sha256:c2169b2dcabf4e608416f7f9468737583ce5f0a6e8677c4efbf795ce81109d7c \ - --hash=sha256:c55853684fe08d4897c37dfc5faeff70607a5f1806c8be148f1695be4a63414b \ - --hash=sha256:c65a3b5330b54103e7d21cac3f6bf3900d46f6d50138d73343d9e5b2900b2353 \ - --hash=sha256:c7964c2183c3e6cce3f497e3a9f49d182e969f2dc3aeeadfa18945ff7bdd7051 \ - --hash=sha256:cc3f1c053b73f20c7ad88b0d1d23be7e7b3901229ce89f5000a8399746a6e039 \ - --hash=sha256:ce615c92d90df8373d9e13acddd154152645c0dc060871abf6bd43809673d20a \ - --hash=sha256:d29338556a59423d9ff7b6eb0cb89ead2b0875e08fe522f3e068b955c3e7b59b \ - --hash=sha256:d8a993c0a0ffd5f2d3bda23d0cd75e7086736f8f8268de8a82fbc4bd0ac6791e \ - --hash=sha256:d9c727bbcf0065cbb20f39d2b4f932f8fa1631c3e01fcedc979bd4f51fe051c5 \ - --hash=sha256:dac37cf08fcf2094159922edc7a2784cfcc5c70f8354469f79ed085f0328ebdf \ - --hash=sha256:dd829712de97753367153ed84f2de752b86cd1f7a88b55a3a775eb52eafe8a94 \ - --hash=sha256:e54ddd0bb8fb626aa1f9ba7b36629564544954fff9669b15da3610c22b9a0991 \ - --hash=sha256:e77c90ab5997e85901da85131fd36acd0ed2221368199b65f0d11bca44549711 \ - --hash=sha256:ebedc192abbc7fd13c5ee800e83a6df252bec691eb2c4bedc9f8b2e2903f5e2a \ - --hash=sha256:ef71561f82a89af6cfcbee47f0fabfdb6e63788a9258e913955d89fdd96902ab \ - --hash=sha256:f0a47efb1dbef13af9c9a54a94a0b814902e547b7f21acb29434504d18f36e3a \ - --hash=sha256:f4f2ca6df64cbdd27f27b34f35adb640b5d2d77264228554e68deda54456eb11 \ - --hash=sha256:fb02e4257376ae25c6dd95a5aec377f9b18c09be6ebdefa7ad209b9137b73d48 +regex==2024.5.15 \ + --hash=sha256:0721931ad5fe0dda45d07f9820b90b2148ccdd8e45bb9e9b42a146cb4f695649 \ + --hash=sha256:10002e86e6068d9e1c91eae8295ef690f02f913c57db120b58fdd35a6bb1af35 \ + --hash=sha256:10e4ce0dca9ae7a66e6089bb29355d4432caed736acae36fef0fdd7879f0b0cb \ + --hash=sha256:119af6e56dce35e8dfb5222573b50c89e5508d94d55713c75126b753f834de68 \ + --hash=sha256:1337b7dbef9b2f71121cdbf1e97e40de33ff114801263b275aafd75303bd62b5 \ + --hash=sha256:13cdaf31bed30a1e1c2453ef6015aa0983e1366fad2667657dbcac7b02f67133 \ + --hash=sha256:1595f2d10dff3d805e054ebdc41c124753631b6a471b976963c7b28543cf13b0 \ + --hash=sha256:16093f563098448ff6b1fa68170e4acbef94e6b6a4e25e10eae8598bb1694b5d \ + --hash=sha256:1878b8301ed011704aea4c806a3cadbd76f84dece1ec09cc9e4dc934cfa5d4da \ + --hash=sha256:19068a6a79cf99a19ccefa44610491e9ca02c2be3305c7760d3831d38a467a6f \ + --hash=sha256:19dfb1c504781a136a80ecd1fff9f16dddf5bb43cec6871778c8a907a085bb3d \ + --hash=sha256:1b5269484f6126eee5e687785e83c6b60aad7663dafe842b34691157e5083e53 \ + --hash=sha256:1c1c174d6ec38d6c8a7504087358ce9213d4332f6293a94fbf5249992ba54efa \ + --hash=sha256:2431b9e263af1953c55abbd3e2efca67ca80a3de8a0437cb58e2421f8184717a \ + --hash=sha256:287eb7f54fc81546346207c533ad3c2c51a8d61075127d7f6d79aaf96cdee890 \ + --hash=sha256:2b4c884767504c0e2401babe8b5b7aea9148680d2e157fa28f01529d1f7fcf67 \ + --hash=sha256:35cb514e137cb3488bce23352af3e12fb0dbedd1ee6e60da053c69fb1b29cc6c \ + --hash=sha256:391d7f7f1e409d192dba8bcd42d3e4cf9e598f3979cdaed6ab11288da88cb9f2 \ + --hash=sha256:3ad070b823ca5890cab606c940522d05d3d22395d432f4aaaf9d5b1653e47ced \ + --hash=sha256:3cd7874d57f13bf70078f1ff02b8b0aa48d5b9ed25fc48547516c6aba36f5741 \ + --hash=sha256:3e507ff1e74373c4d3038195fdd2af30d297b4f0950eeda6f515ae3d84a1770f \ + --hash=sha256:455705d34b4154a80ead722f4f185b04c4237e8e8e33f265cd0798d0e44825fa \ + --hash=sha256:4a605586358893b483976cffc1723fb0f83e526e8f14c6e6614e75919d9862cf \ + --hash=sha256:4babf07ad476aaf7830d77000874d7611704a7fcf68c9c2ad151f5d94ae4bfc4 \ + --hash=sha256:4eee78a04e6c67e8391edd4dad3279828dd66ac4b79570ec998e2155d2e59fd5 \ + --hash=sha256:5397de3219a8b08ae9540c48f602996aa6b0b65d5a61683e233af8605c42b0f2 \ + --hash=sha256:5b5467acbfc153847d5adb21e21e29847bcb5870e65c94c9206d20eb4e99a384 \ + --hash=sha256:5eaa7ddaf517aa095fa8da0b5015c44d03da83f5bd49c87961e3c997daed0de7 \ + --hash=sha256:632b01153e5248c134007209b5c6348a544ce96c46005d8456de1d552455b014 \ + --hash=sha256:64c65783e96e563103d641760664125e91bd85d8e49566ee560ded4da0d3e704 \ + --hash=sha256:64f18a9a3513a99c4bef0e3efd4c4a5b11228b48aa80743be822b71e132ae4f5 \ + --hash=sha256:673b5a6da4557b975c6c90198588181029c60793835ce02f497ea817ff647cb2 \ + --hash=sha256:68811ab14087b2f6e0fc0c2bae9ad689ea3584cad6917fc57be6a48bbd012c49 \ + --hash=sha256:6e8d717bca3a6e2064fc3a08df5cbe366369f4b052dcd21b7416e6d71620dca1 \ + --hash=sha256:71a455a3c584a88f654b64feccc1e25876066c4f5ef26cd6dd711308aa538694 \ + --hash=sha256:72d7a99cd6b8f958e85fc6ca5b37c4303294954eac1376535b03c2a43eb72629 \ + --hash=sha256:7b59138b219ffa8979013be7bc85bb60c6f7b7575df3d56dc1e403a438c7a3f6 \ + --hash=sha256:7dbe2467273b875ea2de38ded4eba86cbcbc9a1a6d0aa11dcf7bd2e67859c435 \ + --hash=sha256:833616ddc75ad595dee848ad984d067f2f31be645d603e4d158bba656bbf516c \ + --hash=sha256:87e2a9c29e672fc65523fb47a90d429b70ef72b901b4e4b1bd42387caf0d6835 \ + --hash=sha256:8fe45aa3f4aa57faabbc9cb46a93363edd6197cbc43523daea044e9ff2fea83e \ + --hash=sha256:9e717956dcfd656f5055cc70996ee2cc82ac5149517fc8e1b60261b907740201 \ + --hash=sha256:9efa1a32ad3a3ea112224897cdaeb6aa00381627f567179c0314f7b65d354c62 \ + --hash=sha256:9ff11639a8d98969c863d4617595eb5425fd12f7c5ef6621a4b74b71ed8726d5 \ + --hash=sha256:a094801d379ab20c2135529948cb84d417a2169b9bdceda2a36f5f10977ebc16 \ + --hash=sha256:a0981022dccabca811e8171f913de05720590c915b033b7e601f35ce4ea7019f \ + --hash=sha256:a0bd000c6e266927cb7a1bc39d55be95c4b4f65c5be53e659537537e019232b1 \ + --hash=sha256:a32b96f15c8ab2e7d27655969a23895eb799de3665fa94349f3b2fbfd547236f \ + --hash=sha256:a81e3cfbae20378d75185171587cbf756015ccb14840702944f014e0d93ea09f \ + --hash=sha256:ac394ff680fc46b97487941f5e6ae49a9f30ea41c6c6804832063f14b2a5a145 \ + --hash=sha256:ada150c5adfa8fbcbf321c30c751dc67d2f12f15bd183ffe4ec7cde351d945b3 \ + --hash=sha256:b2b6f1b3bb6f640c1a92be3bbfbcb18657b125b99ecf141fb3310b5282c7d4ed \ + --hash=sha256:b802512f3e1f480f41ab5f2cfc0e2f761f08a1f41092d6718868082fc0d27143 \ + --hash=sha256:ba68168daedb2c0bab7fd7e00ced5ba90aebf91024dea3c88ad5063c2a562cca \ + --hash=sha256:bfc4f82cabe54f1e7f206fd3d30fda143f84a63fe7d64a81558d6e5f2e5aaba9 \ + --hash=sha256:c0c18345010870e58238790a6779a1219b4d97bd2e77e1140e8ee5d14df071aa \ + --hash=sha256:c3bea0ba8b73b71b37ac833a7f3fd53825924165da6a924aec78c13032f20850 \ + --hash=sha256:c486b4106066d502495b3025a0a7251bf37ea9540433940a23419461ab9f2a80 \ + --hash=sha256:c49e15eac7c149f3670b3e27f1f28a2c1ddeccd3a2812cba953e01be2ab9b5fe \ + --hash=sha256:c6a2b494a76983df8e3d3feea9b9ffdd558b247e60b92f877f93a1ff43d26656 \ + --hash=sha256:cab12877a9bdafde5500206d1020a584355a97884dfd388af3699e9137bf7388 \ + --hash=sha256:cac27dcaa821ca271855a32188aa61d12decb6fe45ffe3e722401fe61e323cd1 \ + --hash=sha256:cdd09d47c0b2efee9378679f8510ee6955d329424c659ab3c5e3a6edea696294 \ + --hash=sha256:cf2430df4148b08fb4324b848672514b1385ae3807651f3567871f130a728cc3 \ + --hash=sha256:d0a3d8d6acf0c78a1fff0e210d224b821081330b8524e3e2bc5a68ef6ab5803d \ + --hash=sha256:d0c0c0003c10f54a591d220997dd27d953cd9ccc1a7294b40a4be5312be8797b \ + --hash=sha256:d1f059a4d795e646e1c37665b9d06062c62d0e8cc3c511fe01315973a6542e40 \ + --hash=sha256:d347a741ea871c2e278fde6c48f85136c96b8659b632fb57a7d1ce1872547600 \ + --hash=sha256:d3ee02d9e5f482cc8309134a91eeaacbdd2261ba111b0fef3748eeb4913e6a2c \ + --hash=sha256:d99ceffa25ac45d150e30bd9ed14ec6039f2aad0ffa6bb87a5936f5782fc1569 \ + --hash=sha256:e38a7d4e8f633a33b4c7350fbd8bad3b70bf81439ac67ac38916c4a86b465456 \ + --hash=sha256:e4682f5ba31f475d58884045c1a97a860a007d44938c4c0895f41d64481edbc9 \ + --hash=sha256:e5bb9425fe881d578aeca0b2b4b3d314ec88738706f66f219c194d67179337cb \ + --hash=sha256:e64198f6b856d48192bf921421fdd8ad8eb35e179086e99e99f711957ffedd6e \ + --hash=sha256:e6662686aeb633ad65be2a42b4cb00178b3fbf7b91878f9446075c404ada552f \ + --hash=sha256:ec54d5afa89c19c6dd8541a133be51ee1017a38b412b1321ccb8d6ddbeb4cf7d \ + --hash=sha256:f5b1dff3ad008dccf18e652283f5e5339d70bf8ba7c98bf848ac33db10f7bc7a \ + --hash=sha256:f8ec0c2fea1e886a19c3bee0cd19d862b3aa75dcdfb42ebe8ed30708df64687a \ + --hash=sha256:f9ebd0a36102fcad2f03696e8af4ae682793a5d30b46c647eaf280d6cfb32796 # via # -c release/ray_release/byod/requirements_compiled.txt # diffusers @@ -3080,9 +3094,9 @@ requests==2.31.0 \ # torchtext # transformers # wandb -requests-oauthlib==1.3.1 \ - --hash=sha256:2577c501a2fb8d05a304c09d090d6e47c306fef15809d102b327cf8364bddab5 \ - --hash=sha256:75beac4a47881eeb94d5ea5d6ad31ef88856affe2332b9aafb52c6452ccf0d7a +requests-oauthlib==2.0.0 \ + --hash=sha256:7dd8a5c40426b779b0868c404bdef9768deccf22749cde15852df527e6269b36 \ + --hash=sha256:b3dffaebd884d8cd778494369603a9e7b58d29111bf6b41bdc2dcd87203af4e9 # via # -c release/ray_release/byod/requirements_compiled.txt # google-auth-oauthlib @@ -3098,12 +3112,13 @@ retry-decorator==1.1.1 \ # -c release/ray_release/byod/requirements_compiled.txt # gcs-oauth2-boto-plugin # gsutil -rich==12.6.0 \ - --hash=sha256:a4eb26484f2c82589bd9a17c73d32a010b1e29d89f1604cd9bf3a2097b81bb5e \ - --hash=sha256:ba3a3775974105c221d31141f2c116f4fd65c5ceb0698657a11e9f295ec93fd0 +rich==13.3.2 \ + --hash=sha256:91954fe80cfb7985727a467ca98a7618e5dd15178cc2da10f553b36a93859001 \ + --hash=sha256:a104f37270bf677148d8acb07d33be1569eeee87e2d1beb286a4e9113caf6f2f # via # -c release/ray_release/byod/requirements_compiled.txt # memray + # typer rouge-score==0.1.2 \ --hash=sha256:c7d4da2683e68c9abf0135ef915d63a46643666f848e558a1b9f7ead17ff0f04 # via lm-eval @@ -3127,105 +3142,107 @@ s3transfer==0.6.2 \ sacrebleu==2.4.2 \ --hash=sha256:611a581d205828912f0b05f806b110180087184d3be2dc650fda7a729d6ecb89 # via lm-eval -safetensors==0.4.1 \ - --hash=sha256:04157d008385bea66d12fe90844a80d4a76dc25ec5230b5bd9a630496d1b7c03 \ - --hash=sha256:04dd14f53f5500eb4c4149674216ba1000670efbcf4b1b5c2643eb244e7882ea \ - --hash=sha256:097e9af2efa8778cd2f0cba451784253e62fa7cc9fc73c0744d27212f7294e25 \ - --hash=sha256:0bd0afd95c1e497f520e680ea01e0397c0868a3a3030e128438cf6e9e3fcd671 \ - --hash=sha256:0ddd050e01f3e843aa8c1c27bf68675b8a08e385d0045487af4d70418c3cb356 \ - --hash=sha256:16d8bbb7344e39cb9d4762e85c21df94ebeb03edac923dd94bb9ed8c10eac070 \ - --hash=sha256:1a45dbf03e8334d3a5dc93687d98b6dc422f5d04c7d519dac09b84a3c87dd7c6 \ - --hash=sha256:1d568628e9c43ca15eb96c217da73737c9ccb07520fafd8a1eba3f2750614105 \ - --hash=sha256:1faf5111c66a6ba91f85dff2e36edaaf36e6966172703159daeef330de4ddc7b \ - --hash=sha256:2297b359d91126c0f9d4fd17bae3cfa2fe3a048a6971b8db07db746ad92f850c \ - --hash=sha256:2304658e6ada81a5223225b4efe84748e760c46079bffedf7e321763cafb36c9 \ - --hash=sha256:2536b11ce665834201072e9397404170f93f3be10cca9995b909f023a04501ee \ - --hash=sha256:257d59e40a1b367cb544122e7451243d65b33c3f34d822a347f4eea6fdf97fdf \ - --hash=sha256:25a043cbb59d4f75e9dd87fdf5c009dd8830105a2c57ace49b72167dd9808111 \ - --hash=sha256:270b99885ec14abfd56c1d7f28ada81740a9220b4bae960c3de1c6fe84af9e4d \ - --hash=sha256:285b52a481e7ba93e29ad4ec5841ef2c4479ef0a6c633c4e2629e0508453577b \ - --hash=sha256:2b6a2814278b6660261aa9a9aae524616de9f1ec364e3716d219b6ed8f91801f \ - --hash=sha256:2d54c2f1826e790d1eb2d2512bfd0ee443f0206b423d6f27095057c7f18a0687 \ - --hash=sha256:2d87d993eaefe6611a9c241a8bd364a5f1ffed5771c74840363a6c4ed8d868f6 \ - --hash=sha256:2fe6926110e3d425c4b684a4379b7796fdc26ad7d16922ea1696c8e6ea7e920f \ - --hash=sha256:303d2c0415cf15a28f8d7f17379ea3c34c2b466119118a34edd9965983a1a8a6 \ - --hash=sha256:313e8472197bde54e3ec54a62df184c414582979da8f3916981b6a7954910a1b \ - --hash=sha256:35803201d980efcf964b75a0a2aee97fe5e9ecc5f3ad676b38fafdfe98e0620d \ - --hash=sha256:39d36f1d88468a87c437a1bc27c502e71b6ca44c385a9117a9f9ba03a75cc9c6 \ - --hash=sha256:3b0b7b2d5976fbed8a05e2bbdce5816a59e6902e9e7c7e07dc723637ed539787 \ - --hash=sha256:3b30abd0cddfe959d1daedf92edcd1b445521ebf7ddefc20860ed01486b33c90 \ - --hash=sha256:3c1b1d510c7aba71504ece87bf393ea82638df56303e371e5e2cf09d18977dd7 \ - --hash=sha256:3cfd1ca35eacc635f0eaa894e5c5ed83ffebd0f95cac298fd430014fa7323631 \ - --hash=sha256:3f6a520af7f2717c5ecba112041f2c8af1ca6480b97bf957aba81ed9642e654c \ - --hash=sha256:413e1f6ac248f7d1b755199a06635e70c3515493d3b41ba46063dec33aa2ebb7 \ - --hash=sha256:4177b456c6b0c722d82429127b5beebdaf07149d265748e97e0a34ff0b3694c8 \ - --hash=sha256:42c3710cec7e5c764c7999697516370bee39067de0aa089b7e2cfb97ac8c6b20 \ - --hash=sha256:44e230fbbe120de564b64f63ef3a8e6ff02840fa02849d9c443d56252a1646d4 \ - --hash=sha256:48901bd540f8a3c1791314bc5c8a170927bf7f6acddb75bf0a263d081a3637d4 \ - --hash=sha256:53134226053e56bd56e73f7db42596e7908ed79f3c9a1016e4c1dade593ac8e5 \ - --hash=sha256:573b6023a55a2f28085fc0a84e196c779b6cbef4d9e73acea14c8094fee7686f \ - --hash=sha256:5d95ea4d8b32233910734a904123bdd3979c137c461b905a5ed32511defc075f \ - --hash=sha256:5f25297148ec665f0deb8bd67e9564634d8d6841041ab5393ccfe203379ea88b \ - --hash=sha256:645b3f1138fce6e818e79d4128afa28f0657430764cc045419c1d069ff93f732 \ - --hash=sha256:660ca1d8bff6c7bc7c6b30b9b32df74ef3ab668f5df42cefd7588f0d40feadcb \ - --hash=sha256:6ace9e66a40f98a216ad661245782483cf79cf56eb2b112650bb904b0baa9db5 \ - --hash=sha256:6fd80f7794554091836d4d613d33a7d006e2b8d6ba014d06f97cebdfda744f64 \ - --hash=sha256:780dc21eb3fd32ddd0e8c904bdb0290f2454f4ac21ae71e94f9ce72db1900a5a \ - --hash=sha256:791edc10a3c359a2f5f52d5cddab0df8a45107d91027d86c3d44e57162e5d934 \ - --hash=sha256:7a8f6f679d97ea0135c7935c202feefbd042c149aa70ee759855e890c01c7814 \ - --hash=sha256:7ef010e9afcb4057fb6be3d0a0cfa07aac04fe97ef73fe4a23138d8522ba7c17 \ - --hash=sha256:7ff8a36e0396776d3ed9a106fc9a9d7c55d4439ca9a056a24bf66d343041d3e6 \ - --hash=sha256:82571d20288c975c1b30b08deb9b1c3550f36b31191e1e81fae87669a92217d0 \ - --hash=sha256:82cbb8f4d022f2e94498cbefca900698b8ded3d4f85212f47da614001ff06652 \ - --hash=sha256:83c2cfbe8c6304f0891e7bb378d56f66d2148972eeb5f747cd8a2246886f0d8c \ - --hash=sha256:845be0aafabf2a60c2d482d4e93023fecffe5e5443d801d7a7741bae9de41233 \ - --hash=sha256:88b4653059c903015284a9722f9a46838c654257173b279c8f6f46dbe80b612d \ - --hash=sha256:8b58ba13a9e82b4bc3fc221914f6ef237fe6c2adb13cede3ace64d1aacf49610 \ - --hash=sha256:8f69903ff49cb30b9227fb5d029bea276ea20d04b06803877a420c5b1b74c689 \ - --hash=sha256:8ff8e41c8037db17de0ea2a23bc684f43eaf623be7d34906fe1ac10985b8365e \ - --hash=sha256:911b48dc09e321a194def3a7431662ff4f03646832f3a8915bbf0f449b8a5fcb \ - --hash=sha256:998fbac99ca956c3a09fe07cc0b35fac26a521fa8865a690686d889f0ff4e4a6 \ - --hash=sha256:9a82bc2bd7a9a0e08239bdd6d7774d64121f136add93dfa344a2f1a6d7ef35fa \ - --hash=sha256:9d16b3b2fcc6fca012c74bd01b5619c655194d3e3c13e4d4d0e446eefa39a463 \ - --hash=sha256:a257de175c254d39ccd6a21341cd62eb7373b05c1e618a78096a56a857e0c316 \ - --hash=sha256:a79e16222106b2f5edbca1b8185661477d8971b659a3c814cc6f15181a9b34c8 \ - --hash=sha256:ae2d5a31cfb8a973a318f7c4d2cffe0bd1fe753cdf7bb41a1939d45a0a06f964 \ - --hash=sha256:ae2f67f04ed0bb2e56fd380a8bd3eef03f609df53f88b6f5c7e89c08e52aae00 \ - --hash=sha256:ae5497adc68669db2fed7cb2dad81e6a6106e79c9a132da3efdb6af1db1014fa \ - --hash=sha256:b287304f2b2220d51ccb51fd857761e78bcffbeabe7b0238f8dc36f2edfd9542 \ - --hash=sha256:b2f8877990a72ff595507b80f4b69036a9a1986a641f8681adf3425d97d3d2a5 \ - --hash=sha256:bb4cb3e37a9b961ddd68e873b29fe9ab4a081e3703412e34aedd2b7a8e9cafd9 \ - --hash=sha256:bbc2ce1f5ae5143a7fb72b71fa71db6a42b4f6cf912aa3acdc6b914084778e68 \ - --hash=sha256:bda3d98e2bcece388232cfc551ebf063b55bdb98f65ab54df397da30efc7dcc5 \ - --hash=sha256:bdc0d039e44a727824639824090bd8869535f729878fa248addd3dc01db30eae \ - --hash=sha256:bfa2e20342b81921b98edba52f8deb68843fa9c95250739a56b52ceda5ea5c61 \ - --hash=sha256:c3807ac3b16288dffebb3474b555b56fe466baa677dfc16290dcd02dca1ab228 \ - --hash=sha256:c3c9f0ca510e0de95abd6424789dcbc879942a3a4e29b0dfa99d9427bf1da75c \ - --hash=sha256:c8ed5d2c04cdc1afc6b3c28d59580448ac07732c50d94c15e14670f9c473a2ce \ - --hash=sha256:cba01c6b76e01ec453933b3b3c0157c59b52881c83eaa0f7666244e71aa75fd1 \ - --hash=sha256:ce7a28bc8af685a69d7e869d09d3e180a275e3281e29cf5f1c7319e231932cc7 \ - --hash=sha256:d10a9f7bae608ccfdc009351f01dc3d8535ff57f9488a58a4c38e45bf954fe93 \ - --hash=sha256:d3ac139377cfe71ba04573f1cda66e663b7c3e95be850e9e6c2dd4b5984bd513 \ - --hash=sha256:d5b3defa74f3723a388bfde2f5d488742bc4879682bd93267c09a3bcdf8f869b \ - --hash=sha256:d784938534e255473155e4d9f276ee69eb85455b6af1292172c731409bf9adee \ - --hash=sha256:d784a98c492c751f228a4a894c3b8a092ff08b24e73b5568938c28b8c0e8f8df \ - --hash=sha256:d8a85e3e47e0d4eebfaf9a58b40aa94f977a56050cb5598ad5396a9ee7c087c6 \ - --hash=sha256:d93321eea0dd7e81b283e47a1d20dee6069165cc158286316d0d06d340de8fe8 \ - --hash=sha256:da52ee0dc8ba03348ffceab767bd8230842fdf78f8a996e2a16445747143a778 \ - --hash=sha256:dab431699b5d45e0ca043bc580651ce9583dda594e62e245b7497adb32e99809 \ - --hash=sha256:dac4bb42f8679aadc59bd91a4c5a1784a758ad49d0912995945cd674089f628e \ - --hash=sha256:e056fb9e22d118cc546107f97dc28b449d88274207dd28872bd668c86216e4f6 \ - --hash=sha256:e09000b2599e1836314430f81a3884c66a5cbabdff5d9f175b5d560d4de38d78 \ - --hash=sha256:e0ccb5aa0f3be2727117e5631200fbb3a5b3a2b3757545a92647d6dd8be6658f \ - --hash=sha256:e57a5ab08b0ec7a7caf30d2ac79bb30c89168431aca4f8854464bb9461686925 \ - --hash=sha256:e9a7ffb1e551c6df51d267f5a751f042b183df22690f6feceac8d27364fd51d7 \ - --hash=sha256:e9c80ce0001efa16066358d2dd77993adc25f5a6c61850e4ad096a2232930bce \ - --hash=sha256:eb2c1da1cc39509d1a55620a5f4d14f8911c47a89c926a96e6f4876e864375a3 \ - --hash=sha256:edcf3121890b5f0616aa5a54683b1a5d2332037b970e507d6bb7841a3a596556 \ - --hash=sha256:f603bdd8deac6726d39f41688ed353c532dd53935234405d79e9eb53f152fbfb \ - --hash=sha256:f8934bdfd202ebd0697040a3dff40dd77bc4c5bbf3527ede0532f5e7fb4d970f \ - --hash=sha256:fdb4adb76e21bad318210310590de61c9f4adcef77ee49b4a234f9dc48867869 \ - --hash=sha256:fdb58dee173ef33634c3016c459d671ca12d11e6acf9db008261cbe58107e579 +safetensors==0.4.3 \ + --hash=sha256:018b691383026a2436a22b648873ed11444a364324e7088b99cd2503dd828400 \ + --hash=sha256:01e4b22e3284cd866edeabe4f4d896229495da457229408d2e1e4810c5187121 \ + --hash=sha256:01feb3089e5932d7e662eda77c3ecc389f97c0883c4a12b5cfdc32b589a811c3 \ + --hash=sha256:02318f01e332cc23ffb4f6716e05a492c5f18b1d13e343c49265149396284a44 \ + --hash=sha256:02ef3a24face643456020536591fbd3c717c5abaa2737ec428ccbbc86dffa7a4 \ + --hash=sha256:03a4447c784917c9bf01d8f2ac5080bc15c41692202cd5f406afba16629e84d6 \ + --hash=sha256:084fc436e317f83f7071fc6a62ca1c513b2103db325cd09952914b50f51cf78f \ + --hash=sha256:0bf4f9d6323d9f86eef5567eabd88f070691cf031d4c0df27a40d3b4aaee755b \ + --hash=sha256:0d52c958dc210265157573f81d34adf54e255bc2b59ded6218500c9b15a750eb \ + --hash=sha256:0d5ffc6a80f715c30af253e0e288ad1cd97a3d0086c9c87995e5093ebc075e50 \ + --hash=sha256:0d9cd8e1560dfc514b6d7859247dc6a86ad2f83151a62c577428d5102d872721 \ + --hash=sha256:0dd37306546b58d3043eb044c8103a02792cc024b51d1dd16bd3dd1f334cb3ed \ + --hash=sha256:1139eb436fd201c133d03c81209d39ac57e129f5e74e34bb9ab60f8d9b726270 \ + --hash=sha256:19bbdf95de2cf64f25cd614c5236c8b06eb2cfa47cbf64311f4b5d80224623a3 \ + --hash=sha256:1ab6527a20586d94291c96e00a668fa03f86189b8a9defa2cdd34a1a01acc7d5 \ + --hash=sha256:1b89381517891a7bb7d1405d828b2bf5d75528299f8231e9346b8eba092227f9 \ + --hash=sha256:1f598b713cc1a4eb31d3b3203557ac308acf21c8f41104cdd74bf640c6e538e3 \ + --hash=sha256:22d21760dc6ebae42e9c058d75aa9907d9f35e38f896e3c69ba0e7b213033856 \ + --hash=sha256:22f3b5d65e440cec0de8edaa672efa888030802e11c09b3d6203bff60ebff05a \ + --hash=sha256:2a0deb16a1d3ea90c244ceb42d2c6c276059616be21a19ac7101aa97da448faf \ + --hash=sha256:2a1f4430cc0c9d6afa01214a4b3919d0a029637df8e09675ceef1ca3f0dfa0df \ + --hash=sha256:2d603846a8585b9432a0fd415db1d4c57c0f860eb4aea21f92559ff9902bae4d \ + --hash=sha256:2f85fc50c4e07a21e95c24e07460fe6f7e2859d0ce88092838352b798ce711c2 \ + --hash=sha256:309b10dbcab63269ecbf0e2ca10ce59223bb756ca5d431ce9c9eeabd446569da \ + --hash=sha256:3615a96dd2dcc30eb66d82bc76cda2565f4f7bfa89fcb0e31ba3cea8a1a9ecbb \ + --hash=sha256:38e2a8666178224a51cca61d3cb4c88704f696eac8f72a49a598a93bbd8a4af9 \ + --hash=sha256:393e6e391467d1b2b829c77e47d726f3b9b93630e6a045b1d1fca67dc78bf632 \ + --hash=sha256:3f9cdca09052f585e62328c1c2923c70f46814715c795be65f0b93f57ec98a02 \ + --hash=sha256:41a727a7f5e6ad9f1db6951adee21bbdadc632363d79dc434876369a17de6ad6 \ + --hash=sha256:420a98f593ff9930f5822560d14c395ccbc57342ddff3b463bc0b3d6b1951550 \ + --hash=sha256:446e9fe52c051aeab12aac63d1017e0f68a02a92a027b901c4f8e931b24e5397 \ + --hash=sha256:455d538aa1aae4a8b279344a08136d3f16334247907b18a5c3c7fa88ef0d3c46 \ + --hash=sha256:4f9bac020faba7f5dc481e881b14b6425265feabb5bfc552551d21189c0eddc3 \ + --hash=sha256:53c4879b9c6bd7cd25d114ee0ef95420e2812e676314300624594940a8d6a91f \ + --hash=sha256:5757e4688f20df083e233b47de43845d1adb7e17b6cf7da5f8444416fc53828d \ + --hash=sha256:585c9ae13a205807b63bef8a37994f30c917ff800ab8a1ca9c9b5d73024f97ee \ + --hash=sha256:5d07cbca5b99babb692d76d8151bec46f461f8ad8daafbfd96b2fca40cadae65 \ + --hash=sha256:5fc6775529fb9f0ce2266edd3e5d3f10aab068e49f765e11f6f2a63b5367021d \ + --hash=sha256:622afd28968ef3e9786562d352659a37de4481a4070f4ebac883f98c5836563e \ + --hash=sha256:6f9568f380f513a60139971169c4a358b8731509cc19112369902eddb33faa4d \ + --hash=sha256:70a5319ef409e7f88686a46607cbc3c428271069d8b770076feaf913664a07ac \ + --hash=sha256:74707624b81f1b7f2b93f5619d4a9f00934d5948005a03f2c1845ffbfff42212 \ + --hash=sha256:7c4fa560ebd4522adddb71dcd25d09bf211b5634003f015a4b815b7647d62ebe \ + --hash=sha256:7de32d0d34b6623bb56ca278f90db081f85fb9c5d327e3c18fd23ac64f465768 \ + --hash=sha256:840b7ac0eff5633e1d053cc9db12fdf56b566e9403b4950b2dc85393d9b88d67 \ + --hash=sha256:840caf38d86aa7014fe37ade5d0d84e23dcfbc798b8078015831996ecbc206a3 \ + --hash=sha256:8651c7299cbd8b4161a36cd6a322fa07d39cd23535b144d02f1c1972d0c62f3c \ + --hash=sha256:868ad1b6fc41209ab6bd12f63923e8baeb1a086814cb2e81a65ed3d497e0cf8f \ + --hash=sha256:88887f69f7a00cf02b954cdc3034ffb383b2303bc0ab481d4716e2da51ddc10e \ + --hash=sha256:89f9f17b0dacb913ed87d57afbc8aad85ea42c1085bd5de2f20d83d13e9fc4b2 \ + --hash=sha256:8c496c5401c1b9c46d41a7688e8ff5b0310a3b9bae31ce0f0ae870e1ea2b8caf \ + --hash=sha256:8cf18888606dad030455d18f6c381720e57fc6a4170ee1966adb7ebc98d4d6a3 \ + --hash=sha256:8d22c1a10dff3f64d0d68abb8298a3fd88ccff79f408a3e15b3e7f637ef5c980 \ + --hash=sha256:90964917f5b0fa0fa07e9a051fbef100250c04d150b7026ccbf87a34a54012e0 \ + --hash=sha256:9bfb92f82574d9e58401d79c70c716985dc049b635fef6eecbb024c79b2c46ad \ + --hash=sha256:9c6ad011c1b4e3acff058d6b090f1da8e55a332fbf84695cf3100c649cc452d1 \ + --hash=sha256:a11c374eb63a9c16c5ed146457241182f310902bd2a9c18255781bb832b6748b \ + --hash=sha256:a7cef55929dcbef24af3eb40bedec35d82c3c2fa46338bb13ecf3c5720af8a61 \ + --hash=sha256:a844cdb5d7cbc22f5f16c7e2a0271170750763c4db08381b7f696dbd2c78a361 \ + --hash=sha256:ae7613a119a71a497d012ccc83775c308b9c1dab454806291427f84397d852fd \ + --hash=sha256:b1648568667f820b8c48317c7006221dc40aced1869908c187f493838a1362bc \ + --hash=sha256:b1e31be7945f66be23f4ec1682bb47faa3df34cb89fc68527de6554d3c4258a4 \ + --hash=sha256:b277482120df46e27a58082df06a15aebda4481e30a1c21eefd0921ae7e03f65 \ + --hash=sha256:b7ffba80aa49bd09195145a7fd233a7781173b422eeb995096f2b30591639517 \ + --hash=sha256:b852e47eb08475c2c1bd8131207b405793bfc20d6f45aff893d3baaad449ed14 \ + --hash=sha256:bb4f8c5d0358a31e9a08daeebb68f5e161cdd4018855426d3f0c23bb51087055 \ + --hash=sha256:bbae3b4b9d997971431c346edbfe6e41e98424a097860ee872721e176040a893 \ + --hash=sha256:befdf0167ad626f22f6aac6163477fcefa342224a22f11fdd05abb3995c1783c \ + --hash=sha256:c0acbe31340ab150423347e5b9cc595867d814244ac14218932a5cf1dd38eb39 \ + --hash=sha256:c41e1893d1206aa7054029681778d9a58b3529d4c807002c156d58426c225173 \ + --hash=sha256:c59d51f182c729f47e841510b70b967b0752039f79f1de23bcdd86462a9b09ee \ + --hash=sha256:cd6fff9e56df398abc5866b19a32124815b656613c1c5ec0f9350906fd798aac \ + --hash=sha256:cdd0a3b5da66e7f377474599814dbf5cbf135ff059cc73694de129b58a5e8a2c \ + --hash=sha256:cf476bca34e1340ee3294ef13e2c625833f83d096cfdf69a5342475602004f95 \ + --hash=sha256:d0dd4a1db09db2dba0f94d15addc7e7cd3a7b0d393aa4c7518c39ae7374623c3 \ + --hash=sha256:d1456f814655b224d4bf6e7915c51ce74e389b413be791203092b7ff78c936dd \ + --hash=sha256:d14d30c25897b2bf19b6fb5ff7e26cc40006ad53fd4a88244fdf26517d852dd7 \ + --hash=sha256:d244bcafeb1bc06d47cfee71727e775bca88a8efda77a13e7306aae3813fa7e4 \ + --hash=sha256:d8815b5e1dac85fc534a97fd339e12404db557878c090f90442247e87c8aeaea \ + --hash=sha256:d88b33980222085dd6001ae2cad87c6068e0991d4f5ccf44975d216db3b57376 \ + --hash=sha256:d8c5093206ef4b198600ae484230402af6713dab1bd5b8e231905d754022bec7 \ + --hash=sha256:d9c289f140a9ae4853fc2236a2ffc9a9f2d5eae0cb673167e0f1b8c18c0961ac \ + --hash=sha256:dcf5705cab159ce0130cd56057f5f3425023c407e170bca60b4868048bae64fd \ + --hash=sha256:e011cc162503c19f4b1fd63dfcddf73739c7a243a17dac09b78e57a00983ab35 \ + --hash=sha256:e066e8861eef6387b7c772344d1fe1f9a72800e04ee9a54239d460c400c72aab \ + --hash=sha256:e0b2104df1579d6ba9052c0ae0e3137c9698b2d85b0645507e6fd1813b70931a \ + --hash=sha256:e375d975159ac534c7161269de24ddcd490df2157b55c1a6eeace6cbb56903f0 \ + --hash=sha256:e4119532cd10dba04b423e0f86aecb96cfa5a602238c0aa012f70c3a40c44b50 \ + --hash=sha256:e7dbbde64b6c534548696808a0e01276d28ea5773bc9a2dfb97a88cd3dffe3df \ + --hash=sha256:e9afd5358719f1b2cf425fad638fc3c887997d6782da317096877e5b15b2ce93 \ + --hash=sha256:ec4b52ce9a396260eb9731eb6aea41a7320de22ed73a1042c2230af0212758ce \ + --hash=sha256:edb5698a7bc282089f64c96c477846950358a46ede85a1c040e0230344fdde10 \ + --hash=sha256:ee463219d9ec6c2be1d331ab13a8e0cd50d2f32240a81d498266d77d07b7e71e \ + --hash=sha256:efcc860be094b8d19ac61b452ec635c7acb9afa77beb218b1d7784c6d41fe8ad \ + --hash=sha256:f5e6883af9a68c0028f70a4c19d5a6ab6238a379be36ad300a22318316c00cb0 \ + --hash=sha256:f9650713b2cfa9537a2baf7dd9fee458b24a0aaaa6cafcea8bdd5fb2b8efdc34 \ + --hash=sha256:faefeb3b81bdfb4e5a55b9bbdf3d8d8753f65506e1d67d03f5c851a6c87150e9 \ + --hash=sha256:fb9c65bd82f9ef3ce4970dc19ee86be5f6f93d032159acf35e663c6bea02b237 \ + --hash=sha256:fe746d03ed8d193674a26105e4f0fe6c726f5bb602ffc695b409eaf02f04763d \ + --hash=sha256:fef5d70683643618244a4f5221053567ca3e77c2531e42ad48ae05fae909f542 # via # -c release/ray_release/byod/requirements_compiled.txt # accelerate @@ -3346,9 +3363,9 @@ sentencepiece==0.1.96 \ # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in -sentry-sdk==1.37.1 \ - --hash=sha256:7cd324dd2877fdc861f75cba4242bce23a58272a6fea581fcb218bb718bd9cc5 \ - --hash=sha256:a249c7364827ee89daaa078bb8b56ece0b3d52d9130961bef2302b79bdf7fe70 +sentry-sdk==2.10.0 \ + --hash=sha256:545fcc6e36c335faa6d6cda84669b6e17025f31efbf3b2211ec14efe008b75d1 \ + --hash=sha256:87b3d413c87d8e7f816cc9334bff255a83d8b577db2b22042651c30c19c09190 # via # -c release/ray_release/byod/requirements_compiled.txt # wandb @@ -3444,6 +3461,12 @@ setproctitle==1.3.3 \ # via # -c release/ray_release/byod/requirements_compiled.txt # wandb +shellingham==1.5.4 \ + --hash=sha256:7ecfff8f2fd72616f7481040475a65b2bf8af90a56c89140852d1120324e8686 \ + --hash=sha256:8dbca0739d487e5bd35ab3ca4b36e11c4078f3a234bfce294b0a0291363404de + # via + # -c release/ray_release/byod/requirements_compiled.txt + # typer six==1.16.0 \ --hash=sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926 \ --hash=sha256:8abb2f1d86890a2dfb989f9a77cfcfd3e47c2a354b01111771326f8aa26e0254 @@ -3470,50 +3493,15 @@ smmap==5.0.1 \ # via # -c release/ray_release/byod/requirements_compiled.txt # gitdb -sniffio==1.3.0 \ - --hash=sha256:e60305c5e5d314f5389259b7f22aaa33d8f7dee49763119234af3755c55b9101 \ - --hash=sha256:eecefdce1e5bbfb7ad2eeaabf7c1eeb404d7757c379bd1f7e5cce9d8bf425384 +sniffio==1.3.1 \ + --hash=sha256:2f6da418d1f1e0fddd844478f41680e794e6051915791a034ff65e5f100525a2 \ + --hash=sha256:f4324edc670a0f49750a81b895f35c3adb843cca46f0530f79fc1babb23789dc # via # -c release/ray_release/byod/requirements_compiled.txt # anyio -sqlalchemy==1.4.17 \ - --hash=sha256:196fb6bb2733834e506c925d7532f8eabad9d2304deef738a40846e54c31e236 \ - --hash=sha256:1dd77acbc19bee9c0ba858ff5e4e5d5c60895495c83b4df9bcdf4ad5e9b74f21 \ - --hash=sha256:216ff28fe803885ceb5b131dcee6507d28d255808dd5bcffcb3b5fa75be2e102 \ - --hash=sha256:461a4ea803ce0834822f372617a68ac97f9fa1281f2a984624554c651d7c3ae1 \ - --hash=sha256:4b09191ed22af149c07a880f309b7740f3f782ff13325bae5c6168a6aa57e715 \ - --hash=sha256:4c5e20666b33b03bf7f14953f0deb93007bf8c1342e985bd7c7cf25f46fac579 \ - --hash=sha256:4d93b62e98248e3e1ac1e91c2e6ee1e7316f704be1f734338b350b6951e6c175 \ - --hash=sha256:5732858e56d32fa7e02468f4fd2d8f01ddf709e5b93d035c637762890f8ed8b6 \ - --hash=sha256:58c02d1771bb0e61bc9ced8f3b36b5714d9ece8fd4bdbe2a44a892574c3bbc3c \ - --hash=sha256:651cdb3adcee13624ba22d5ff3e96f91e16a115d2ca489ddc16a8e4c217e8509 \ - --hash=sha256:6fe1c8dc26bc0005439cb78ebc78772a22cccc773f5a0e67cb3002d791f53f0f \ - --hash=sha256:7222f3236c280fab3a2d76f903b493171f0ffc29667538cc388a5d5dd0216a88 \ - --hash=sha256:7dc3d3285fb682316d580d84e6e0840fdd8ffdc05cb696db74b9dd746c729908 \ - --hash=sha256:7e45043fe11d503e1c3f9dcf5b42f92d122a814237cd9af68a11dae46ecfcae1 \ - --hash=sha256:7eb55d5583076c03aaf1510473fad2a61288490809049cb31028af56af7068ee \ - --hash=sha256:82922a320d38d7d6aa3a8130523ec7e8c70fa95f7ca7d0fd6ec114b626e4b10b \ - --hash=sha256:8e133e2551fa99c75849848a4ac08efb79930561eb629dd7d2dc9b7ee05256e6 \ - --hash=sha256:949ac299903d2ed8419086f81847381184e2264f3431a33af4679546dcc87f01 \ - --hash=sha256:a2d225c8863a76d15468896dc5af36f1e196b403eb9c7e0151e77ffab9e7df57 \ - --hash=sha256:a5f00a2be7d777119e15ccfb5ba0b2a92e8a193959281089d79821a001095f80 \ - --hash=sha256:b0ad951a6e590bbcfbfeadc5748ef5ec8ede505a8119a71b235f7481cc08371c \ - --hash=sha256:b59b2c0a3b1d93027f6b6b8379a50c354483fe1ebe796c6740e157bb2e06d39a \ - --hash=sha256:bc89e37c359dcd4d75b744e5e81af128ba678aa2ecea4be957e80e6e958a1612 \ - --hash=sha256:bde055c019e6e449ebc4ec61abd3e08690abeb028c7ada2a3b95d8e352b7b514 \ - --hash=sha256:c367ed95d41df584f412a9419b5ece85b0d6c2a08a51ae13ae47ef74ff9a9349 \ - --hash=sha256:dde05ae0987e43ec84e64d6722ce66305eda2a5e2b7d6fda004b37aabdfbb909 \ - --hash=sha256:ee6e7ca09ff274c55d19a1e15ee6f884fa0230c0d9b8d22a456e249d08dee5bf \ - --hash=sha256:f1c68f7bd4a57ffdb85eab489362828dddf6cd565a4c18eda4c446c1d5d3059d \ - --hash=sha256:f63e1f531a8bf52184e2afb53648511f3f8534decb7575b483a583d3cd8d13ed \ - --hash=sha256:fdad4a33140b77df61d456922b7974c1f1bb2c35238f6809f078003a620c4734 - # via - # -c release/ray_release/byod/requirements_compiled.txt - # alembic - # dataset -sqlglot==20.4.0 \ - --hash=sha256:401a2933298cf66901704cf2029272d8243ee72ac47b9fd8784254401b43ee43 \ - --hash=sha256:9a42135d0530de8150a2c5106e0c52abd3396d92501ebe97df7b371d20de5dc9 +sqlglot==25.6.1 \ + --hash=sha256:c1fcbaa00429979f16fb8cea20279a8b3f5312e76d97abb8f8c6a9b21be450d7 \ + --hash=sha256:ea40f3bf8452e2c1a696fe120163190bd67e49b346336e7db6d34400b57b7601 # via # -c release/ray_release/byod/requirements_compiled.txt # fugue @@ -3571,9 +3559,9 @@ statsmodels==0.14.0 \ # via # -c release/ray_release/byod/requirements_compiled.txt # statsforecast -sympy==1.12 \ - --hash=sha256:c3588cd4295d0c0f603d0f2ae780587e64e2efeedb3521e46b9bb1d08d184fa5 \ - --hash=sha256:ebf595c8dac3e0fdc4152c51878b498396ec7f30e7a914d6071e674d49420fb8 +sympy==1.13.1 \ + --hash=sha256:9cebf7e04ff162015ce31c9c6c9144daa34a93bd082f54fd8f12deca4f47515f \ + --hash=sha256:db36cdc64bf61b9b24578b6f7bab1ecdd2452cf008f34faa33776680c26d66f8 # via # -c release/ray_release/byod/requirements_compiled.txt # torch @@ -3764,17 +3752,12 @@ tokenizers==0.15.2 \ # via # -c release/ray_release/byod/requirements_compiled.txt # transformers -toml==0.10.2 \ - --hash=sha256:806143ae5bfb6a3c6e736a764057db0e6a0e05e338b5630894a5f779cabb4f9b \ - --hash=sha256:b3bda1d108d5dd99f4a20d24d9c348e91c4db7ab1b749200bded2f839ccbe68f - # via - # -c release/ray_release/byod/requirements_compiled.txt - # jupytext tomli==2.0.1 \ --hash=sha256:939de3e7a6161af0c887ef91b7d41a53e7c5a1ca976325f429cb46ea9bc30ecc \ --hash=sha256:de526c12914f0c550d15924c62d72abc48d6fe7364aa87328337a31007fe8a4f # via # -c release/ray_release/byod/requirements_compiled.txt + # jupytext # pytest torch==2.3.0 \ --hash=sha256:09c81c5859a5b819956c6925a405ef1cdda393c9d8a01ce3851453f699d3358c \ @@ -3881,9 +3864,9 @@ tqdm-multiprocess==0.0.11 \ --hash=sha256:3ebdf03e7a675150fa0bbceaa9c3c64b8cb556e9ffafa4fe6c078e51820524aa \ --hash=sha256:a74002a1222ea9cbe8cdc9bd460108c6009be359621fbee9b92d0515d4d180f7 # via lm-eval -traitlets==5.14.0 \ - --hash=sha256:f14949d23829023013c47df20b4a76ccd1a85effb786dc060f34de7948361b33 \ - --hash=sha256:fcdaa8ac49c04dfa0ed3ee3384ef6dfdb5d6f3741502be247279407679296772 +traitlets==5.14.3 \ + --hash=sha256:9ed0579d3502c94b4b3732ac120375cda96f923114522847de4b3bb98b96b6b7 \ + --hash=sha256:b74e89e397b1ed28cc831db7aea759ba6640cb3de13090ca145426688ff1ac4f # via # -c release/ray_release/byod/requirements_compiled.txt # comm @@ -3900,9 +3883,9 @@ transformers==4.36.2 \ # -r release/ray_release/byod/requirements_ml_byod_3.9.in # lm-eval # peft -triad==0.9.3 \ - --hash=sha256:1862b5a78deb9d475c7747b605f2b32457e96c6719f8cbc4e7e95147f34f6f64 \ - --hash=sha256:e4dff41ffbb98bad4d9741c9dd632890cdfe0b873f23d76d2b5f9ca41d4440a7 +triad==0.9.8 \ + --hash=sha256:2c0ba7d83977c6d4e7b59e3cc70727f858014ef7676c62d184aa8e63f7bef5de \ + --hash=sha256:5b67673124891981daf8afbab44b2e6358932ca35ef3ff38a25bc3e0f6f03f17 # via # -c release/ray_release/byod/requirements_compiled.txt # adagio @@ -3930,9 +3913,9 @@ typepy[datetime]==1.3.2 \ # dataproperty # pytablewriter # tabledata -typer==0.9.0 \ - --hash=sha256:50922fd79aea2f4751a8e0408ff10d2662bd0c8bbfa84755a699f3bada2978b2 \ - --hash=sha256:5d96d986a21493606a358cae4461bd8cdf83cbf33a5aa950ae629ca3b51467ee +typer==0.12.3 \ + --hash=sha256:070d7ca53f785acbccba8e7d28b08dcd88f79f1fbda035ade0aecec71ca5c914 \ + --hash=sha256:49e73131481d804288ef62598d97a1ceef3058905aa536a1134f90891ba35482 # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in @@ -3942,7 +3925,6 @@ typing-extensions==4.8.0 \ # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in - # alembic # fastapi # huggingface-hub # ipython @@ -3954,9 +3936,9 @@ typing-extensions==4.8.0 \ # torch # typer # wandb -urllib3==1.26.18 \ - --hash=sha256:34b97092d7e0a3a8cf7cd10e386f401b3737364026c45e622aa02903dffe0f07 \ - --hash=sha256:f8ecc1bba5667413457c529ab955bf8c67b45db799d159066261719e328580a0 +urllib3==1.26.19 \ + --hash=sha256:37a0344459b199fce0e80b0d3569837ec6b6937435c5244e7fd73fa6006830f3 \ + --hash=sha256:3e3d753a8618b86d7de333b4223005f68720bcd6a7d2bcb9fbd2229ec7c1e429 # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in @@ -3965,9 +3947,9 @@ urllib3==1.26.18 \ # requests # responses # sentry-sdk -utilsforecast==0.0.23 \ - --hash=sha256:188daa121c528965e26a3a38f409b66a15f9eef2b44684cc9426f3ddb1146841 \ - --hash=sha256:290882da47ebc7887663c05c46c67e19bc63898220be444ca6173d0a5fdeee4a +utilsforecast==0.2.0 \ + --hash=sha256:3db4245da4e361f26c8eaeef216c2d1206b20defbb033bf11d3e66ce2b1d6ef8 \ + --hash=sha256:a4825bf8da547e3dc552f9b9a7a8159341a118c3a5d122191f09bc3683cba433 # via # -c release/ray_release/byod/requirements_compiled.txt # statsforecast @@ -3992,9 +3974,9 @@ wandb==0.17.0 \ # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in -wcwidth==0.2.12 \ - --hash=sha256:f01c104efdf57971bcb756f054dd58ddec5204dd15fa31d6503ea57947d97c02 \ - --hash=sha256:f26ec43d96c8cbfed76a5075dac87680124fa84e0855195a6184da9c187f133c +wcwidth==0.2.13 \ + --hash=sha256:3da69048e4540d84af32131829ff948f1e022c1c6bdb8d6102117aac784f6859 \ + --hash=sha256:72ea0c06399eb286d978fdedb6923a9eb47e1c486ce63e9b4e64fc18303972b5 # via # -c release/ray_release/byod/requirements_compiled.txt # prompt-toolkit @@ -4011,13 +3993,15 @@ widgetsnbextension==4.0.11 \ # via # -c release/ray_release/byod/requirements_compiled.txt # ipywidgets -xgboost==1.7.6 \ - --hash=sha256:127cf1f5e2ec25cd41429394c6719b87af1456ce583e89f0bffd35d02ad18bcb \ - --hash=sha256:1c527554a400445e0c38186039ba1a00425dcdb4e40b37eed0e74cb39a159c47 \ - --hash=sha256:281c3c6f4fbed2d36bf95cd02a641afa95e72e9abde70064056da5e76233e8df \ - --hash=sha256:4c34675b4d2678c624ddde5d45361e7e16046923e362e4e609b88353e6b87124 \ - --hash=sha256:59b4b366d2cafc7f645e87d897983a5b59be02876194b1d213bd8d8b811d8ce8 \ - --hash=sha256:b1d5db49b199152d62bd9217c98760207d3de86d2b9d243260c573ffe638f80a +xgboost==2.1.0 \ + --hash=sha256:19d145eb847b070c32342b1bf2d7331c102783e07a484f8b13b7d759d707c6b0 \ + --hash=sha256:43b16205689249d7509daf7a6ab00ad0e6c570b3a9c263cb32b26e39d9477bb3 \ + --hash=sha256:7144980923e76ce741c7b03a14d3bd7514db6de5c7cabe96ba95b229d274f5ca \ + --hash=sha256:73673c9bb85927db7fe2e3aed6df6d35dba708cfd6767cc63d4ea11dda2dede5 \ + --hash=sha256:74904b91c42524a6c32147fe5718569e78fb65911ff4499b053f81d0964514d4 \ + --hash=sha256:840a0c6e2119d8c8f260a5dace996ea064a267f62b301a25d7d452488a7ac860 \ + --hash=sha256:b2a456eb0f3d3e8fd8ab37e44ac288292bf8ea8744c294be9fd88713d27af810 \ + --hash=sha256:cedc2e386e686795735448fd4597533acacc5ba6fb47dd910c204c468b80bb96 # via # -c release/ray_release/byod/requirements_compiled.txt # -r release/ray_release/byod/requirements_ml_byod_3.9.in @@ -4134,103 +4118,103 @@ xxhash==3.4.1 \ # -c release/ray_release/byod/requirements_compiled.txt # datasets # evaluate -yarl==1.9.3 \ - --hash=sha256:09c19e5f4404574fcfb736efecf75844ffe8610606f3fccc35a1515b8b6712c4 \ - --hash=sha256:0ab5baaea8450f4a3e241ef17e3d129b2143e38a685036b075976b9c415ea3eb \ - --hash=sha256:0d155a092bf0ebf4a9f6f3b7a650dc5d9a5bbb585ef83a52ed36ba46f55cc39d \ - --hash=sha256:126638ab961633f0940a06e1c9d59919003ef212a15869708dcb7305f91a6732 \ - --hash=sha256:1a0a4f3aaa18580038cfa52a7183c8ffbbe7d727fe581300817efc1e96d1b0e9 \ - --hash=sha256:1d93461e2cf76c4796355494f15ffcb50a3c198cc2d601ad8d6a96219a10c363 \ - --hash=sha256:26a1a8443091c7fbc17b84a0d9f38de34b8423b459fb853e6c8cdfab0eacf613 \ - --hash=sha256:271d63396460b6607b588555ea27a1a02b717ca2e3f2cf53bdde4013d7790929 \ - --hash=sha256:28a108cb92ce6cf867690a962372996ca332d8cda0210c5ad487fe996e76b8bb \ - --hash=sha256:29beac86f33d6c7ab1d79bd0213aa7aed2d2f555386856bb3056d5fdd9dab279 \ - --hash=sha256:2c757f64afe53a422e45e3e399e1e3cf82b7a2f244796ce80d8ca53e16a49b9f \ - --hash=sha256:2dad8166d41ebd1f76ce107cf6a31e39801aee3844a54a90af23278b072f1ccf \ - --hash=sha256:2dc72e891672343b99db6d497024bf8b985537ad6c393359dc5227ef653b2f17 \ - --hash=sha256:2f3c8822bc8fb4a347a192dd6a28a25d7f0ea3262e826d7d4ef9cc99cd06d07e \ - --hash=sha256:32435d134414e01d937cd9d6cc56e8413a8d4741dea36af5840c7750f04d16ab \ - --hash=sha256:3cfa4dbe17b2e6fca1414e9c3bcc216f6930cb18ea7646e7d0d52792ac196808 \ - --hash=sha256:3d5434b34100b504aabae75f0622ebb85defffe7b64ad8f52b8b30ec6ef6e4b9 \ - --hash=sha256:4003f380dac50328c85e85416aca6985536812c082387255c35292cb4b41707e \ - --hash=sha256:44e91a669c43f03964f672c5a234ae0d7a4d49c9b85d1baa93dec28afa28ffbd \ - --hash=sha256:4a14907b597ec55740f63e52d7fee0e9ee09d5b9d57a4f399a7423268e457b57 \ - --hash=sha256:4ce77d289f8d40905c054b63f29851ecbfd026ef4ba5c371a158cfe6f623663e \ - --hash=sha256:4d6d74a97e898c1c2df80339aa423234ad9ea2052f66366cef1e80448798c13d \ - --hash=sha256:51382c72dd5377861b573bd55dcf680df54cea84147c8648b15ac507fbef984d \ - --hash=sha256:525cd69eff44833b01f8ef39aa33a9cc53a99ff7f9d76a6ef6a9fb758f54d0ff \ - --hash=sha256:53ec65f7eee8655bebb1f6f1607760d123c3c115a324b443df4f916383482a67 \ - --hash=sha256:5f74b015c99a5eac5ae589de27a1201418a5d9d460e89ccb3366015c6153e60a \ - --hash=sha256:6280353940f7e5e2efaaabd686193e61351e966cc02f401761c4d87f48c89ea4 \ - --hash=sha256:632c7aeb99df718765adf58eacb9acb9cbc555e075da849c1378ef4d18bf536a \ - --hash=sha256:6465d36381af057d0fab4e0f24ef0e80ba61f03fe43e6eeccbe0056e74aadc70 \ - --hash=sha256:66a6dbf6ca7d2db03cc61cafe1ee6be838ce0fbc97781881a22a58a7c5efef42 \ - --hash=sha256:6d350388ba1129bc867c6af1cd17da2b197dff0d2801036d2d7d83c2d771a682 \ - --hash=sha256:7217234b10c64b52cc39a8d82550342ae2e45be34f5bff02b890b8c452eb48d7 \ - --hash=sha256:721ee3fc292f0d069a04016ef2c3a25595d48c5b8ddc6029be46f6158d129c92 \ - --hash=sha256:72a57b41a0920b9a220125081c1e191b88a4cdec13bf9d0649e382a822705c65 \ - --hash=sha256:73cc83f918b69110813a7d95024266072d987b903a623ecae673d1e71579d566 \ - --hash=sha256:778df71c8d0c8c9f1b378624b26431ca80041660d7be7c3f724b2c7a6e65d0d6 \ - --hash=sha256:79e1df60f7c2b148722fb6cafebffe1acd95fd8b5fd77795f56247edaf326752 \ - --hash=sha256:7c86d0d0919952d05df880a1889a4f0aeb6868e98961c090e335671dea5c0361 \ - --hash=sha256:7eaf13af79950142ab2bbb8362f8d8d935be9aaf8df1df89c86c3231e4ff238a \ - --hash=sha256:828235a2a169160ee73a2fcfb8a000709edf09d7511fccf203465c3d5acc59e4 \ - --hash=sha256:8535e111a064f3bdd94c0ed443105934d6f005adad68dd13ce50a488a0ad1bf3 \ - --hash=sha256:88d2c3cc4b2f46d1ba73d81c51ec0e486f59cc51165ea4f789677f91a303a9a7 \ - --hash=sha256:8a2538806be846ea25e90c28786136932ec385c7ff3bc1148e45125984783dc6 \ - --hash=sha256:8dab30b21bd6fb17c3f4684868c7e6a9e8468078db00f599fb1c14e324b10fca \ - --hash=sha256:8f18a7832ff85dfcd77871fe677b169b1bc60c021978c90c3bb14f727596e0ae \ - --hash=sha256:946db4511b2d815979d733ac6a961f47e20a29c297be0d55b6d4b77ee4b298f6 \ - --hash=sha256:96758e56dceb8a70f8a5cff1e452daaeff07d1cc9f11e9b0c951330f0a2396a7 \ - --hash=sha256:9a172c3d5447b7da1680a1a2d6ecdf6f87a319d21d52729f45ec938a7006d5d8 \ - --hash=sha256:9a5211de242754b5e612557bca701f39f8b1a9408dff73c6db623f22d20f470e \ - --hash=sha256:9df9a0d4c5624790a0dea2e02e3b1b3c69aed14bcb8650e19606d9df3719e87d \ - --hash=sha256:aa4643635f26052401750bd54db911b6342eb1a9ac3e74f0f8b58a25d61dfe41 \ - --hash=sha256:aed37db837ecb5962469fad448aaae0f0ee94ffce2062cf2eb9aed13328b5196 \ - --hash=sha256:af52725c7c39b0ee655befbbab5b9a1b209e01bb39128dce0db226a10014aacc \ - --hash=sha256:b0b8c06afcf2bac5a50b37f64efbde978b7f9dc88842ce9729c020dc71fae4ce \ - --hash=sha256:b61e64b06c3640feab73fa4ff9cb64bd8182de52e5dc13038e01cfe674ebc321 \ - --hash=sha256:b7831566595fe88ba17ea80e4b61c0eb599f84c85acaa14bf04dd90319a45b90 \ - --hash=sha256:b8bc5b87a65a4e64bc83385c05145ea901b613d0d3a434d434b55511b6ab0067 \ - --hash=sha256:b8d51817cf4b8d545963ec65ff06c1b92e5765aa98831678d0e2240b6e9fd281 \ - --hash=sha256:b9f9cafaf031c34d95c1528c16b2fa07b710e6056b3c4e2e34e9317072da5d1a \ - --hash=sha256:bb72d2a94481e7dc7a0c522673db288f31849800d6ce2435317376a345728225 \ - --hash=sha256:c25ec06e4241e162f5d1f57c370f4078797ade95c9208bd0c60f484834f09c96 \ - --hash=sha256:c405d482c320a88ab53dcbd98d6d6f32ada074f2d965d6e9bf2d823158fa97de \ - --hash=sha256:c4472fe53ebf541113e533971bd8c32728debc4c6d8cc177f2bff31d011ec17e \ - --hash=sha256:c4b1efb11a8acd13246ffb0bee888dd0e8eb057f8bf30112e3e21e421eb82d4a \ - --hash=sha256:c5f3faeb8100a43adf3e7925d556801d14b5816a0ac9e75e22948e787feec642 \ - --hash=sha256:c6f034386e5550b5dc8ded90b5e2ff7db21f0f5c7de37b6efc5dac046eb19c10 \ - --hash=sha256:c99ddaddb2fbe04953b84d1651149a0d85214780e4d0ee824e610ab549d98d92 \ - --hash=sha256:ca6b66f69e30f6e180d52f14d91ac854b8119553b524e0e28d5291a724f0f423 \ - --hash=sha256:cccdc02e46d2bd7cb5f38f8cc3d9db0d24951abd082b2f242c9e9f59c0ab2af3 \ - --hash=sha256:cd49a908cb6d387fc26acee8b7d9fcc9bbf8e1aca890c0b2fdfd706057546080 \ - --hash=sha256:cf7a4e8de7f1092829caef66fd90eaf3710bc5efd322a816d5677b7664893c93 \ - --hash=sha256:cfd77e8e5cafba3fb584e0f4b935a59216f352b73d4987be3af51f43a862c403 \ - --hash=sha256:d34c4f80956227f2686ddea5b3585e109c2733e2d4ef12eb1b8b4e84f09a2ab6 \ - --hash=sha256:d61a0ca95503867d4d627517bcfdc28a8468c3f1b0b06c626f30dd759d3999fd \ - --hash=sha256:d81657b23e0edb84b37167e98aefb04ae16cbc5352770057893bd222cdc6e45f \ - --hash=sha256:d92d897cb4b4bf915fbeb5e604c7911021a8456f0964f3b8ebbe7f9188b9eabb \ - --hash=sha256:dd318e6b75ca80bff0b22b302f83a8ee41c62b8ac662ddb49f67ec97e799885d \ - --hash=sha256:dd952b9c64f3b21aedd09b8fe958e4931864dba69926d8a90c90d36ac4e28c9a \ - --hash=sha256:e0e7e83f31e23c5d00ff618045ddc5e916f9e613d33c5a5823bc0b0a0feb522f \ - --hash=sha256:e0f17d1df951336a02afc8270c03c0c6e60d1f9996fcbd43a4ce6be81de0bd9d \ - --hash=sha256:e2a16ef5fa2382af83bef4a18c1b3bcb4284c4732906aa69422cf09df9c59f1f \ - --hash=sha256:e36021db54b8a0475805acc1d6c4bca5d9f52c3825ad29ae2d398a9d530ddb88 \ - --hash=sha256:e73db54c967eb75037c178a54445c5a4e7461b5203b27c45ef656a81787c0c1b \ - --hash=sha256:e741bd48e6a417bdfbae02e088f60018286d6c141639359fb8df017a3b69415a \ - --hash=sha256:f7271d6bd8838c49ba8ae647fc06469137e1c161a7ef97d778b72904d9b68696 \ - --hash=sha256:fc391e3941045fd0987c77484b2799adffd08e4b6735c4ee5f054366a2e1551d \ - --hash=sha256:fc94441bcf9cb8c59f51f23193316afefbf3ff858460cb47b5758bf66a14d130 \ - --hash=sha256:fe34befb8c765b8ce562f0200afda3578f8abb159c76de3ab354c80b72244c41 \ - --hash=sha256:fe8080b4f25dfc44a86bedd14bc4f9d469dfc6456e6f3c5d9077e81a5fedfba7 \ - --hash=sha256:ff34cb09a332832d1cf38acd0f604c068665192c6107a439a92abfd8acf90fe2 +yarl==1.9.4 \ + --hash=sha256:008d3e808d03ef28542372d01057fd09168419cdc8f848efe2804f894ae03e51 \ + --hash=sha256:03caa9507d3d3c83bca08650678e25364e1843b484f19986a527630ca376ecce \ + --hash=sha256:07574b007ee20e5c375a8fe4a0789fad26db905f9813be0f9fef5a68080de559 \ + --hash=sha256:09efe4615ada057ba2d30df871d2f668af661e971dfeedf0c159927d48bbeff0 \ + --hash=sha256:0d2454f0aef65ea81037759be5ca9947539667eecebca092733b2eb43c965a81 \ + --hash=sha256:0e9d124c191d5b881060a9e5060627694c3bdd1fe24c5eecc8d5d7d0eb6faabc \ + --hash=sha256:18580f672e44ce1238b82f7fb87d727c4a131f3a9d33a5e0e82b793362bf18b4 \ + --hash=sha256:1f23e4fe1e8794f74b6027d7cf19dc25f8b63af1483d91d595d4a07eca1fb26c \ + --hash=sha256:206a55215e6d05dbc6c98ce598a59e6fbd0c493e2de4ea6cc2f4934d5a18d130 \ + --hash=sha256:23d32a2594cb5d565d358a92e151315d1b2268bc10f4610d098f96b147370136 \ + --hash=sha256:26a1dc6285e03f3cc9e839a2da83bcbf31dcb0d004c72d0730e755b33466c30e \ + --hash=sha256:29e0f83f37610f173eb7e7b5562dd71467993495e568e708d99e9d1944f561ec \ + --hash=sha256:2b134fd795e2322b7684155b7855cc99409d10b2e408056db2b93b51a52accc7 \ + --hash=sha256:2d47552b6e52c3319fede1b60b3de120fe83bde9b7bddad11a69fb0af7db32f1 \ + --hash=sha256:357495293086c5b6d34ca9616a43d329317feab7917518bc97a08f9e55648455 \ + --hash=sha256:35a2b9396879ce32754bd457d31a51ff0a9d426fd9e0e3c33394bf4b9036b099 \ + --hash=sha256:3777ce5536d17989c91696db1d459574e9a9bd37660ea7ee4d3344579bb6f129 \ + --hash=sha256:3986b6f41ad22988e53d5778f91855dc0399b043fc8946d4f2e68af22ee9ff10 \ + --hash=sha256:44d8ffbb9c06e5a7f529f38f53eda23e50d1ed33c6c869e01481d3fafa6b8142 \ + --hash=sha256:49a180c2e0743d5d6e0b4d1a9e5f633c62eca3f8a86ba5dd3c471060e352ca98 \ + --hash=sha256:4aa9741085f635934f3a2583e16fcf62ba835719a8b2b28fb2917bb0537c1dfa \ + --hash=sha256:4b21516d181cd77ebd06ce160ef8cc2a5e9ad35fb1c5930882baff5ac865eee7 \ + --hash=sha256:4b3c1ffe10069f655ea2d731808e76e0f452fc6c749bea04781daf18e6039525 \ + --hash=sha256:4c7d56b293cc071e82532f70adcbd8b61909eec973ae9d2d1f9b233f3d943f2c \ + --hash=sha256:4e9035df8d0880b2f1c7f5031f33f69e071dfe72ee9310cfc76f7b605958ceb9 \ + --hash=sha256:54525ae423d7b7a8ee81ba189f131054defdb122cde31ff17477951464c1691c \ + --hash=sha256:549d19c84c55d11687ddbd47eeb348a89df9cb30e1993f1b128f4685cd0ebbf8 \ + --hash=sha256:54beabb809ffcacbd9d28ac57b0db46e42a6e341a030293fb3185c409e626b8b \ + --hash=sha256:566db86717cf8080b99b58b083b773a908ae40f06681e87e589a976faf8246bf \ + --hash=sha256:5a2e2433eb9344a163aced6a5f6c9222c0786e5a9e9cac2c89f0b28433f56e23 \ + --hash=sha256:5aef935237d60a51a62b86249839b51345f47564208c6ee615ed2a40878dccdd \ + --hash=sha256:604f31d97fa493083ea21bd9b92c419012531c4e17ea6da0f65cacdcf5d0bd27 \ + --hash=sha256:63b20738b5aac74e239622d2fe30df4fca4942a86e31bf47a81a0e94c14df94f \ + --hash=sha256:686a0c2f85f83463272ddffd4deb5e591c98aac1897d65e92319f729c320eece \ + --hash=sha256:6a962e04b8f91f8c4e5917e518d17958e3bdee71fd1d8b88cdce74dd0ebbf434 \ + --hash=sha256:6ad6d10ed9b67a382b45f29ea028f92d25bc0bc1daf6c5b801b90b5aa70fb9ec \ + --hash=sha256:6f5cb257bc2ec58f437da2b37a8cd48f666db96d47b8a3115c29f316313654ff \ + --hash=sha256:6fe79f998a4052d79e1c30eeb7d6c1c1056ad33300f682465e1b4e9b5a188b78 \ + --hash=sha256:7855426dfbddac81896b6e533ebefc0af2f132d4a47340cee6d22cac7190022d \ + --hash=sha256:7d5aaac37d19b2904bb9dfe12cdb08c8443e7ba7d2852894ad448d4b8f442863 \ + --hash=sha256:801e9264d19643548651b9db361ce3287176671fb0117f96b5ac0ee1c3530d53 \ + --hash=sha256:81eb57278deb6098a5b62e88ad8281b2ba09f2f1147c4767522353eaa6260b31 \ + --hash=sha256:824d6c50492add5da9374875ce72db7a0733b29c2394890aef23d533106e2b15 \ + --hash=sha256:8397a3817d7dcdd14bb266283cd1d6fc7264a48c186b986f32e86d86d35fbac5 \ + --hash=sha256:848cd2a1df56ddbffeb375535fb62c9d1645dde33ca4d51341378b3f5954429b \ + --hash=sha256:84fc30f71689d7fc9168b92788abc977dc8cefa806909565fc2951d02f6b7d57 \ + --hash=sha256:8619d6915b3b0b34420cf9b2bb6d81ef59d984cb0fde7544e9ece32b4b3043c3 \ + --hash=sha256:8a854227cf581330ffa2c4824d96e52ee621dd571078a252c25e3a3b3d94a1b1 \ + --hash=sha256:8be9e837ea9113676e5754b43b940b50cce76d9ed7d2461df1af39a8ee674d9f \ + --hash=sha256:928cecb0ef9d5a7946eb6ff58417ad2fe9375762382f1bf5c55e61645f2c43ad \ + --hash=sha256:957b4774373cf6f709359e5c8c4a0af9f6d7875db657adb0feaf8d6cb3c3964c \ + --hash=sha256:992f18e0ea248ee03b5a6e8b3b4738850ae7dbb172cc41c966462801cbf62cf7 \ + --hash=sha256:9fc5fc1eeb029757349ad26bbc5880557389a03fa6ada41703db5e068881e5f2 \ + --hash=sha256:a00862fb23195b6b8322f7d781b0dc1d82cb3bcac346d1e38689370cc1cc398b \ + --hash=sha256:a3a6ed1d525bfb91b3fc9b690c5a21bb52de28c018530ad85093cc488bee2dd2 \ + --hash=sha256:a6327976c7c2f4ee6816eff196e25385ccc02cb81427952414a64811037bbc8b \ + --hash=sha256:a7409f968456111140c1c95301cadf071bd30a81cbd7ab829169fb9e3d72eae9 \ + --hash=sha256:a825ec844298c791fd28ed14ed1bffc56a98d15b8c58a20e0e08c1f5f2bea1be \ + --hash=sha256:a8c1df72eb746f4136fe9a2e72b0c9dc1da1cbd23b5372f94b5820ff8ae30e0e \ + --hash=sha256:a9bd00dc3bc395a662900f33f74feb3e757429e545d831eef5bb280252631984 \ + --hash=sha256:aa102d6d280a5455ad6a0f9e6d769989638718e938a6a0a2ff3f4a7ff8c62cc4 \ + --hash=sha256:aaaea1e536f98754a6e5c56091baa1b6ce2f2700cc4a00b0d49eca8dea471074 \ + --hash=sha256:ad4d7a90a92e528aadf4965d685c17dacff3df282db1121136c382dc0b6014d2 \ + --hash=sha256:b8477c1ee4bd47c57d49621a062121c3023609f7a13b8a46953eb6c9716ca392 \ + --hash=sha256:ba6f52cbc7809cd8d74604cce9c14868306ae4aa0282016b641c661f981a6e91 \ + --hash=sha256:bac8d525a8dbc2a1507ec731d2867025d11ceadcb4dd421423a5d42c56818541 \ + --hash=sha256:bef596fdaa8f26e3d66af846bbe77057237cb6e8efff8cd7cc8dff9a62278bbf \ + --hash=sha256:c0ec0ed476f77db9fb29bca17f0a8fcc7bc97ad4c6c1d8959c507decb22e8572 \ + --hash=sha256:c38c9ddb6103ceae4e4498f9c08fac9b590c5c71b0370f98714768e22ac6fa66 \ + --hash=sha256:c7224cab95645c7ab53791022ae77a4509472613e839dab722a72abe5a684575 \ + --hash=sha256:c74018551e31269d56fab81a728f683667e7c28c04e807ba08f8c9e3bba32f14 \ + --hash=sha256:ca06675212f94e7a610e85ca36948bb8fc023e458dd6c63ef71abfd482481aa5 \ + --hash=sha256:d1d2532b340b692880261c15aee4dc94dd22ca5d61b9db9a8a361953d36410b1 \ + --hash=sha256:d25039a474c4c72a5ad4b52495056f843a7ff07b632c1b92ea9043a3d9950f6e \ + --hash=sha256:d5ff2c858f5f6a42c2a8e751100f237c5e869cbde669a724f2062d4c4ef93551 \ + --hash=sha256:d7d7f7de27b8944f1fee2c26a88b4dabc2409d2fea7a9ed3df79b67277644e17 \ + --hash=sha256:d7eeb6d22331e2fd42fce928a81c697c9ee2d51400bd1a28803965883e13cead \ + --hash=sha256:d8a1c6c0be645c745a081c192e747c5de06e944a0d21245f4cf7c05e457c36e0 \ + --hash=sha256:d8b889777de69897406c9fb0b76cdf2fd0f31267861ae7501d93003d55f54fbe \ + --hash=sha256:d9e09c9d74f4566e905a0b8fa668c58109f7624db96a2171f21747abc7524234 \ + --hash=sha256:db8e58b9d79200c76956cefd14d5c90af54416ff5353c5bfd7cbe58818e26ef0 \ + --hash=sha256:ddb2a5c08a4eaaba605340fdee8fc08e406c56617566d9643ad8bf6852778fc7 \ + --hash=sha256:e0381b4ce23ff92f8170080c97678040fc5b08da85e9e292292aba67fdac6c34 \ + --hash=sha256:e23a6d84d9d1738dbc6e38167776107e63307dfc8ad108e580548d1f2c587f42 \ + --hash=sha256:e516dc8baf7b380e6c1c26792610230f37147bb754d6426462ab115a02944385 \ + --hash=sha256:ea65804b5dc88dacd4a40279af0cdadcfe74b3e5b4c897aa0d81cf86927fee78 \ + --hash=sha256:ec61d826d80fc293ed46c9dd26995921e3a82146feacd952ef0757236fc137be \ + --hash=sha256:ee04010f26d5102399bd17f8df8bc38dc7ccd7701dc77f4a68c5b8d733406958 \ + --hash=sha256:f3bc6af6e2b8f92eced34ef6a96ffb248e863af20ef4fde9448cc8c9b858b749 \ + --hash=sha256:f7d6b36dd2e029b6bcb8a13cf19664c7b8e19ab3a58e0fefbb5b8461447ed5ec # via # -c release/ray_release/byod/requirements_compiled.txt # aiohttp -zipp==3.17.0 \ - --hash=sha256:0e923e726174922dce09c53c59ad483ff7bbb8e572e00c7f7c46b88556409f31 \ - --hash=sha256:84e64a1c28cf7e91ed2078bb8cc8c259cb19b76942096c8d7b84947690cabaf0 +zipp==3.19.2 \ + --hash=sha256:bf1dcf6450f873a13e952a29504887c89e6de7506209e5b1bcc3460135d4de19 \ + --hash=sha256:f091755f667055f2d02b32c53771a7a6c8b47e1fdbc4b72a8b9072b3eef8015c # via # -c release/ray_release/byod/requirements_compiled.txt # importlib-metadata diff --git a/release/ray_release/config.py b/release/ray_release/config.py index 884292545f1d..c38fa4b2f7e4 100644 --- a/release/ray_release/config.py +++ b/release/ray_release/config.py @@ -81,11 +81,22 @@ def _test_definition_invariant( def parse_test_definition(test_definitions: List[TestDefinition]) -> List[Test]: + default_definition = {} tests = [] for test_definition in test_definitions: + if test_definition["name"] == "DEFAULTS": + default_definition = copy.deepcopy(test_definition) + continue + + # Add default values to the test definition. + test_definition = deep_update( + copy.deepcopy(default_definition), test_definition + ) + if "variations" not in test_definition: tests.append(Test(test_definition)) continue + variations = test_definition.pop("variations") _test_definition_invariant( test_definition, diff --git a/release/ray_release/tests/test_config.py b/release/ray_release/tests/test_config.py index 9b1f5adbb929..c7884b116880 100644 --- a/release/ray_release/tests/test_config.py +++ b/release/ray_release/tests/test_config.py @@ -93,6 +93,42 @@ def test_parse_test_definition(): parse_test_definition([invalid_test_definition]) +def test_parse_test_definition_with_defaults(): + test_definitions = yaml.safe_load( + """ + - name: DEFAULTS + working_dir: default_working_dir + - name: sample_test_with_default_working_dir + frequency: nightly + team: sample + cluster: + byod: + type: gpu + cluster_compute: compute.yaml + run: + timeout: 100 + script: python script.py + - name: sample_test_with_overridden_working_dir + working_dir: overridden_working_dir + frequency: nightly + team: sample + cluster: + byod: + type: gpu + cluster_compute: compute.yaml + run: + timeout: 100 + script: python script.py + """ + ) + test_with_default, test_with_override = parse_test_definition(test_definitions) + schema = load_schema_file() + assert not validate_test(test_with_default, schema) + assert not validate_test(test_with_override, schema) + assert test_with_default["working_dir"] == "default_working_dir" + assert test_with_override["working_dir"] == "overridden_working_dir" + + def test_schema_validation(): test = VALID_TEST.copy() diff --git a/release/release_data_tests.yaml b/release/release_data_tests.yaml index 752aaacb4fb0..1a7adae918be 100644 --- a/release/release_data_tests.yaml +++ b/release/release_data_tests.yaml @@ -1,8 +1,4 @@ -############### -# Reading tests -############### - -- name: read_parquet +- name: DEFAULTS group: data-tests working_dir: nightly_tests/dataset @@ -13,23 +9,17 @@ byod: type: gpu cluster_compute: multi_node_autoscaling_compute.yaml + +############### +# Reading tests +############### +- name: read_parquet run: timeout: 3600 script: python read_and_consume_benchmark.py s3://ray-benchmark-data/parquet/10TiB --format parquet --iterate - name: read_images - group: data-tests - working_dir: nightly_tests/dataset - - frequency: nightly - team: data - - cluster: - byod: - type: gpu - cluster_compute: multi_node_autoscaling_compute.yaml - run: timeout: 3600 script: python read_and_consume_benchmark.py s3://air-example-data-2/300G-image-data-synthetic-raw --format image --iterate @@ -39,17 +29,6 @@ ############### - name: count_parquet - group: data-tests - working_dir: nightly_tests/dataset - - frequency: nightly - team: data - - cluster: - byod: - type: gpu - cluster_compute: multi_node_autoscaling_compute.yaml - run: timeout: 600 script: python read_and_consume_benchmark.py s3://ray-benchmark-data/parquet/10TiB --format parquet --count diff --git a/src/mock/ray/gcs/gcs_server/gcs_node_manager.h b/src/mock/ray/gcs/gcs_server/gcs_node_manager.h index 3a8f22949fae..7a3efe197529 100644 --- a/src/mock/ray/gcs/gcs_server/gcs_node_manager.h +++ b/src/mock/ray/gcs/gcs_server/gcs_node_manager.h @@ -18,7 +18,11 @@ namespace gcs { class MockGcsNodeManager : public GcsNodeManager { public: - MockGcsNodeManager() : GcsNodeManager(nullptr, nullptr, nullptr, ClusterID::Nil()) {} + MockGcsNodeManager() + : GcsNodeManager(/*gcs_publisher=*/nullptr, + /*gcs_table_storage=*/nullptr, + /*raylet_client_pool=*/nullptr, + /*cluster_id=*/ClusterID::Nil()) {} MOCK_METHOD(void, HandleRegisterNode, (rpc::RegisterNodeRequest request, diff --git a/src/ray/core_worker/core_worker.h b/src/ray/core_worker/core_worker.h index 06d966881e67..b904790b22fd 100644 --- a/src/ray/core_worker/core_worker.h +++ b/src/ray/core_worker/core_worker.h @@ -228,6 +228,19 @@ class CoreWorker : public rpc::CoreWorkerServiceHandler { const TaskID &GetCurrentTaskId() const { return worker_context_.GetCurrentTaskID(); } + const std::string GetCurrentTaskName() const { + return worker_context_.GetCurrentTask() != nullptr + ? worker_context_.GetCurrentTask()->GetName() + : ""; + } + + const std::string GetCurrentTaskFunctionName() const { + return (worker_context_.GetCurrentTask() != nullptr && + worker_context_.GetCurrentTask()->FunctionDescriptor() != nullptr) + ? worker_context_.GetCurrentTask()->FunctionDescriptor()->CallSiteString() + : ""; + } + /// Controls the is debugger paused flag. /// /// \param task_id The task id of the task to update. diff --git a/src/ray/gcs/gcs_server/gcs_actor_manager.cc b/src/ray/gcs/gcs_server/gcs_actor_manager.cc index aa609bacb445..8bf3f3d484f1 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_actor_manager.cc @@ -749,11 +749,8 @@ Status GcsActorManager::RegisterActor(const ray::rpc::RegisterActorRequest &requ "explicitly connect to this namespace with ray.init(namespace=\"" << actor->GetRayNamespace() << "\", ...)"; - auto error_data_ptr = - gcs::CreateErrorTableData("detached_actor_anonymous_namespace", - stream.str(), - absl::GetCurrentTimeNanos(), - job_id); + auto error_data_ptr = gcs::CreateErrorTableData( + "detached_actor_anonymous_namespace", stream.str(), absl::Now(), job_id); RAY_LOG(WARNING) << error_data_ptr->SerializeAsString(); RAY_CHECK_OK( diff --git a/src/ray/gcs/gcs_server/gcs_actor_manager.h b/src/ray/gcs/gcs_server/gcs_actor_manager.h index dadb90498379..afdf55be80d6 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_manager.h +++ b/src/ray/gcs/gcs_server/gcs_actor_manager.h @@ -321,7 +321,7 @@ class GcsActorManager : public rpc::ActorInfoHandler { std::function destroy_owned_placement_group_if_needed, const rpc::CoreWorkerClientFactoryFn &worker_client_factory = nullptr); - ~GcsActorManager() = default; + ~GcsActorManager() override = default; void HandleRegisterActor(rpc::RegisterActorRequest request, rpc::RegisterActorReply *reply, diff --git a/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc b/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc index f562c0f9034e..5daa1a992257 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc +++ b/src/ray/gcs/gcs_server/gcs_actor_scheduler.cc @@ -27,17 +27,17 @@ GcsActorScheduler::GcsActorScheduler( instrumented_io_context &io_context, GcsActorTable &gcs_actor_table, const GcsNodeManager &gcs_node_manager, - std::shared_ptr cluster_task_manager, + ClusterTaskManager &cluster_task_manager, GcsActorSchedulerFailureCallback schedule_failure_handler, GcsActorSchedulerSuccessCallback schedule_success_handler, - std::shared_ptr raylet_client_pool, + rpc::NodeManagerClientPool &raylet_client_pool, rpc::CoreWorkerClientFactoryFn client_factory, std::function normal_task_resources_changed_callback) : io_context_(io_context), gcs_actor_table_(gcs_actor_table), gcs_node_manager_(gcs_node_manager), - cluster_task_manager_(std::move(cluster_task_manager)), + cluster_task_manager_(cluster_task_manager), schedule_failure_handler_(std::move(schedule_failure_handler)), schedule_success_handler_(std::move(schedule_success_handler)), raylet_client_pool_(raylet_client_pool), @@ -97,11 +97,11 @@ void GcsActorScheduler::ScheduleByGcs(std::shared_ptr actor) { const auto &owner_node = gcs_node_manager_.GetAliveNode(actor->GetOwnerNodeID()); RayTask task(actor->GetCreationTaskSpecification(), owner_node.has_value() ? actor->GetOwnerNodeID().Binary() : std::string()); - cluster_task_manager_->QueueAndScheduleTask(task, - /*grant_or_reject*/ false, - /*is_selected_based_on_locality*/ false, - /*reply*/ reply.get(), - send_reply_callback); + cluster_task_manager_.QueueAndScheduleTask(task, + /*grant_or_reject*/ false, + /*is_selected_based_on_locality*/ false, + /*reply*/ reply.get(), + send_reply_callback); } void GcsActorScheduler::ScheduleByRaylet(std::shared_ptr actor) { @@ -218,7 +218,7 @@ std::vector GcsActorScheduler::CancelOnNode(const NodeID &node_id) { } } - raylet_client_pool_->Disconnect(node_id); + raylet_client_pool_.Disconnect(node_id); return actor_ids; } @@ -531,7 +531,7 @@ void GcsActorScheduler::DoRetryCreatingActorOnWorker( std::shared_ptr GcsActorScheduler::GetOrConnectLeaseClient( const rpc::Address &raylet_address) { - return raylet_client_pool_->GetOrConnectByAddress(raylet_address); + return raylet_client_pool_.GetOrConnectByAddress(raylet_address); } bool GcsActorScheduler::KillActorOnWorker(const rpc::Address &worker_address, @@ -664,13 +664,13 @@ void GcsActorScheduler::HandleWorkerLeaseRejectedReply( void GcsActorScheduler::OnActorDestruction(std::shared_ptr actor) { if (!actor->GetAcquiredResources().IsEmpty()) { ReturnActorAcquiredResources(actor); - cluster_task_manager_->ScheduleAndDispatchTasks(); + cluster_task_manager_.ScheduleAndDispatchTasks(); } } void GcsActorScheduler::ReturnActorAcquiredResources(std::shared_ptr actor) { auto &cluster_resource_manager = - cluster_task_manager_->GetClusterResourceScheduler()->GetClusterResourceManager(); + cluster_task_manager_.GetClusterResourceScheduler()->GetClusterResourceManager(); cluster_resource_manager.AddNodeAvailableResources( scheduling::NodeID(actor->GetNodeID().Binary()), actor->GetAcquiredResources().GetResourceSet()); @@ -678,14 +678,13 @@ void GcsActorScheduler::ReturnActorAcquiredResources(std::shared_ptr a } size_t GcsActorScheduler::GetPendingActorsCount() const { - return cluster_task_manager_->GetInfeasibleQueueSize() + - cluster_task_manager_->GetPendingQueueSize(); + return cluster_task_manager_.GetInfeasibleQueueSize() + + cluster_task_manager_.GetPendingQueueSize(); } bool GcsActorScheduler::CancelInFlightActorScheduling( const std::shared_ptr &actor) { - return cluster_task_manager_->CancelTask( - actor->GetCreationTaskSpecification().TaskId()); + return cluster_task_manager_.CancelTask(actor->GetCreationTaskSpecification().TaskId()); } } // namespace gcs diff --git a/src/ray/gcs/gcs_server/gcs_actor_scheduler.h b/src/ray/gcs/gcs_server/gcs_actor_scheduler.h index 1ea66d0ddbe0..048d1da8939c 100644 --- a/src/ray/gcs/gcs_server/gcs_actor_scheduler.h +++ b/src/ray/gcs/gcs_server/gcs_actor_scheduler.h @@ -129,14 +129,14 @@ class GcsActorScheduler : public GcsActorSchedulerInterface { instrumented_io_context &io_context, GcsActorTable &gcs_actor_table, const GcsNodeManager &gcs_node_manager, - std::shared_ptr cluster_task_manager_, + ClusterTaskManager &cluster_task_manager_, GcsActorSchedulerFailureCallback schedule_failure_handler, GcsActorSchedulerSuccessCallback schedule_success_handler, - std::shared_ptr raylet_client_pool, + rpc::NodeManagerClientPool &raylet_client_pool, rpc::CoreWorkerClientFactoryFn client_factory = nullptr, std::function normal_task_resources_changed_callback = nullptr); - virtual ~GcsActorScheduler() = default; + ~GcsActorScheduler() override = default; /// Schedule the specified actor. /// If there is no available nodes then the actor would be queued in the @@ -377,7 +377,7 @@ class GcsActorScheduler : public GcsActorSchedulerInterface { /// Reference of GcsNodeManager. const GcsNodeManager &gcs_node_manager_; /// The cluster task manager. - std::shared_ptr cluster_task_manager_; + ClusterTaskManager &cluster_task_manager_; /// The handler to handle the scheduling failures. GcsActorSchedulerFailureCallback schedule_failure_handler_; /// The handler to handle the successful scheduling. @@ -385,7 +385,7 @@ class GcsActorScheduler : public GcsActorSchedulerInterface { /// The nodes which are releasing unused workers. absl::flat_hash_set nodes_of_releasing_unused_workers_; /// The cached raylet clients used to communicate with raylet. - std::shared_ptr raylet_client_pool_; + rpc::NodeManagerClientPool &raylet_client_pool_; /// The cached core worker clients which are used to communicate with leased worker. rpc::CoreWorkerClientPool core_worker_clients_; diff --git a/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc b/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc index e4ef13371b53..c166dbbf6398 100644 --- a/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.cc @@ -28,12 +28,12 @@ GcsAutoscalerStateManager::GcsAutoscalerStateManager( GcsNodeManager &gcs_node_manager, GcsActorManager &gcs_actor_manager, const GcsPlacementGroupManager &gcs_placement_group_manager, - std::shared_ptr raylet_client_pool) + rpc::NodeManagerClientPool &raylet_client_pool) : session_name_(session_name), gcs_node_manager_(gcs_node_manager), gcs_actor_manager_(gcs_actor_manager), gcs_placement_group_manager_(gcs_placement_group_manager), - raylet_client_pool_(std::move(raylet_client_pool)), + raylet_client_pool_(raylet_client_pool), last_cluster_resource_state_version_(0), last_seen_autoscaler_state_version_(0) {} @@ -396,7 +396,7 @@ void GcsAutoscalerStateManager::HandleDrainNode( raylet_address.set_ip_address(node->node_manager_address()); raylet_address.set_port(node->node_manager_port()); - const auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(raylet_address); + const auto raylet_client = raylet_client_pool_.GetOrConnectByAddress(raylet_address); raylet_client->DrainRaylet( request.reason(), request.reason_message(), diff --git a/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.h b/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.h index c00d8d465202..c592a7a484d6 100644 --- a/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.h +++ b/src/ray/gcs/gcs_server/gcs_autoscaler_state_manager.h @@ -29,12 +29,11 @@ class GcsResourceManager; class GcsAutoscalerStateManager : public rpc::autoscaler::AutoscalerStateHandler { public: - GcsAutoscalerStateManager( - const std::string &session_name, - GcsNodeManager &gcs_node_manager, - GcsActorManager &gcs_actor_manager, - const GcsPlacementGroupManager &gcs_placement_group_manager, - std::shared_ptr raylet_client_pool); + GcsAutoscalerStateManager(const std::string &session_name, + GcsNodeManager &gcs_node_manager, + GcsActorManager &gcs_actor_manager, + const GcsPlacementGroupManager &gcs_placement_group_manager, + rpc::NodeManagerClientPool &raylet_client_pool); void HandleGetClusterResourceState( rpc::autoscaler::GetClusterResourceStateRequest request, @@ -152,7 +151,7 @@ class GcsAutoscalerStateManager : public rpc::autoscaler::AutoscalerStateHandler const GcsPlacementGroupManager &gcs_placement_group_manager_; /// Raylet client pool. - std::shared_ptr raylet_client_pool_; + rpc::NodeManagerClientPool &raylet_client_pool_; // The default value of the last seen version for the request is 0, which indicates // no version has been reported. So the first reported version should be 1. diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.cc b/src/ray/gcs/gcs_server/gcs_node_manager.cc index a232ecf10903..15aa488cb65f 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_node_manager.cc @@ -29,14 +29,13 @@ namespace ray { namespace gcs { ////////////////////////////////////////////////////////////////////////////////////////// -GcsNodeManager::GcsNodeManager( - std::shared_ptr gcs_publisher, - std::shared_ptr gcs_table_storage, - std::shared_ptr raylet_client_pool, - const ClusterID &cluster_id) +GcsNodeManager::GcsNodeManager(std::shared_ptr gcs_publisher, + std::shared_ptr gcs_table_storage, + rpc::NodeManagerClientPool *raylet_client_pool, + const ClusterID &cluster_id) : gcs_publisher_(std::move(gcs_publisher)), gcs_table_storage_(std::move(gcs_table_storage)), - raylet_client_pool_(std::move(raylet_client_pool)), + raylet_client_pool_(raylet_client_pool), cluster_id_(cluster_id) {} void GcsNodeManager::WriteNodeExportEvent(rpc::GcsNodeInfo node_info) const { @@ -393,8 +392,8 @@ std::shared_ptr GcsNodeManager::RemoveNode( .WithField("ip", removed_node->node_manager_address()) << error_message.str(); RAY_LOG(WARNING) << error_message.str(); - auto error_data_ptr = - gcs::CreateErrorTableData(type, error_message.str(), current_time_ms()); + auto error_data_ptr = gcs::CreateErrorTableData( + type, error_message.str(), absl::FromUnixMillis(current_time_ms())); RAY_CHECK_OK(gcs_publisher_->PublishError(node_id.Hex(), *error_data_ptr, nullptr)); } diff --git a/src/ray/gcs/gcs_server/gcs_node_manager.h b/src/ray/gcs/gcs_server/gcs_node_manager.h index db258d4cb00c..b924fec264c9 100644 --- a/src/ray/gcs/gcs_server/gcs_node_manager.h +++ b/src/ray/gcs/gcs_server/gcs_node_manager.h @@ -50,7 +50,7 @@ class GcsNodeManager : public rpc::NodeInfoHandler { /// \param gcs_table_storage GCS table external storage accessor. explicit GcsNodeManager(std::shared_ptr gcs_publisher, std::shared_ptr gcs_table_storage, - std::shared_ptr raylet_client_pool, + rpc::NodeManagerClientPool *raylet_client_pool, const ClusterID &cluster_id); /// Handle register rpc request come from raylet. @@ -248,7 +248,7 @@ class GcsNodeManager : public rpc::NodeInfoHandler { /// Storage for GCS tables. std::shared_ptr gcs_table_storage_; /// Raylet client pool. - std::shared_ptr raylet_client_pool_; + rpc::NodeManagerClientPool *raylet_client_pool_ = nullptr; /// Cluster ID to be shared with clients when connecting. const ClusterID cluster_id_; 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 c60bcd43cc45..1aec60e9603c 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_manager.cc +++ b/src/ray/gcs/gcs_server/gcs_placement_group_manager.cc @@ -14,6 +14,8 @@ #include "ray/gcs/gcs_server/gcs_placement_group_manager.h" +#include + #include "ray/common/asio/asio_util.h" #include "ray/common/asio/instrumented_io_context.h" #include "ray/common/ray_config.h" @@ -181,15 +183,15 @@ rpc::PlacementGroupStats *GcsPlacementGroup::GetMutableStats() { GcsPlacementGroupManager::GcsPlacementGroupManager( instrumented_io_context &io_context, - std::shared_ptr scheduler, + GcsPlacementGroupSchedulerInterface *scheduler, std::shared_ptr gcs_table_storage, GcsResourceManager &gcs_resource_manager, std::function get_ray_namespace) : io_context_(io_context), - gcs_placement_group_scheduler_(std::move(scheduler)), + gcs_placement_group_scheduler_(scheduler), gcs_table_storage_(std::move(gcs_table_storage)), gcs_resource_manager_(gcs_resource_manager), - get_ray_namespace_(get_ray_namespace) { + get_ray_namespace_(std::move(get_ray_namespace)) { placement_group_state_counter_.reset( new CounterMap()); placement_group_state_counter_->SetOnChangeCallback( 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 d90fdccf3a8a..a7d91388e264 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_manager.h +++ b/src/ray/gcs/gcs_server/gcs_placement_group_manager.h @@ -237,7 +237,7 @@ class GcsPlacementGroupManager : public rpc::PlacementGroupInfoHandler { /// \param gcs_resource_manager Reference of GcsResourceManager. /// \param get_ray_namespace A callback to get the ray namespace. GcsPlacementGroupManager(instrumented_io_context &io_context, - std::shared_ptr scheduler, + GcsPlacementGroupSchedulerInterface *scheduler, std::shared_ptr gcs_table_storage, GcsResourceManager &gcs_resource_manager, std::function get_ray_namespace); @@ -480,8 +480,8 @@ class GcsPlacementGroupManager : public rpc::PlacementGroupInfoHandler { std::deque> infeasible_placement_groups_; /// The scheduler to schedule all registered placement_groups. - std::shared_ptr - gcs_placement_group_scheduler_; + /// Scheduler's lifecycle lies in [GcsServer]. + gcs::GcsPlacementGroupSchedulerInterface *gcs_placement_group_scheduler_ = nullptr; /// Used to update placement group information upon creation, deletion, etc. std::shared_ptr gcs_table_storage_; 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 6bc2737c14a6..85a94f863598 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc +++ b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.cc @@ -26,13 +26,13 @@ GcsPlacementGroupScheduler::GcsPlacementGroupScheduler( std::shared_ptr gcs_table_storage, const gcs::GcsNodeManager &gcs_node_manager, ClusterResourceScheduler &cluster_resource_scheduler, - std::shared_ptr raylet_client_pool) + rpc::NodeManagerClientPool &raylet_client_pool) : io_context_(io_context), return_timer_(io_context), gcs_table_storage_(std::move(gcs_table_storage)), gcs_node_manager_(gcs_node_manager), cluster_resource_scheduler_(cluster_resource_scheduler), - raylet_client_pool_(std::move(raylet_client_pool)) {} + raylet_client_pool_(raylet_client_pool) {} void GcsPlacementGroupScheduler::ScheduleUnplacedBundles( const SchedulePgRequest &request) { @@ -279,7 +279,7 @@ void GcsPlacementGroupScheduler::CancelResourceReserve( std::shared_ptr GcsPlacementGroupScheduler::GetOrConnectLeaseClient(const rpc::Address &raylet_address) { - return raylet_client_pool_->GetOrConnectByAddress(raylet_address); + return raylet_client_pool_.GetOrConnectByAddress(raylet_address); } std::shared_ptr 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 ec7ac53941bd..df16f025d082 100644 --- a/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h +++ b/src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h @@ -290,12 +290,11 @@ class GcsPlacementGroupScheduler : public GcsPlacementGroupSchedulerInterface { /// \param cluster_resource_scheduler The resource scheduler which is used when /// scheduling. /// \param lease_client_factory Factory to create remote lease client. - GcsPlacementGroupScheduler( - instrumented_io_context &io_context, - std::shared_ptr gcs_table_storage, - const GcsNodeManager &gcs_node_manager, - ClusterResourceScheduler &cluster_resource_scheduler, - std::shared_ptr raylet_client_pool); + GcsPlacementGroupScheduler(instrumented_io_context &io_context, + std::shared_ptr gcs_table_storage, + const GcsNodeManager &gcs_node_manager, + ClusterResourceScheduler &cluster_resource_scheduler, + rpc::NodeManagerClientPool &raylet_client_pool); virtual ~GcsPlacementGroupScheduler() = default; @@ -502,7 +501,7 @@ class GcsPlacementGroupScheduler : public GcsPlacementGroupSchedulerInterface { placement_group_leasing_in_progress_; /// The cached raylet clients used to communicate with raylets. - std::shared_ptr raylet_client_pool_; + rpc::NodeManagerClientPool &raylet_client_pool_; /// The nodes which are releasing unused bundles. absl::flat_hash_set nodes_of_releasing_unused_bundles_; diff --git a/src/ray/gcs/gcs_server/gcs_resource_manager.h b/src/ray/gcs/gcs_server/gcs_resource_manager.h index dff95380cd21..47ecf9fff3a3 100644 --- a/src/ray/gcs/gcs_server/gcs_resource_manager.h +++ b/src/ray/gcs/gcs_server/gcs_resource_manager.h @@ -67,7 +67,7 @@ class GcsResourceManager : public rpc::NodeResourceInfoHandler, NodeID local_node_id, std::shared_ptr cluster_task_manager = nullptr); - virtual ~GcsResourceManager() {} + virtual ~GcsResourceManager() = default; /// Handle the resource update. void ConsumeSyncMessage(std::shared_ptr message) override; diff --git a/src/ray/gcs/gcs_server/gcs_server.cc b/src/ray/gcs/gcs_server/gcs_server.cc index c51c14bbb0dd..a8ad05ff3c9d 100644 --- a/src/ray/gcs/gcs_server/gcs_server.cc +++ b/src/ray/gcs/gcs_server/gcs_server.cc @@ -64,7 +64,7 @@ GcsServer::GcsServer(const ray::gcs::GcsServerConfig &config, ClusterID::Nil(), RayConfig::instance().gcs_server_rpc_client_thread_num()), raylet_client_pool_( - std::make_shared(client_call_manager_)), + std::make_unique(client_call_manager_)), pubsub_periodical_runner_(io_context_provider_.GetIOContext()), periodical_runner_(io_context_provider_.GetDefaultIOContext()), is_started_(false), @@ -289,7 +289,7 @@ void GcsServer::InitGcsNodeManager(const GcsInitData &gcs_init_data) { RAY_CHECK(gcs_table_storage_ && gcs_publisher_); gcs_node_manager_ = std::make_unique(gcs_publisher_, gcs_table_storage_, - raylet_client_pool_, + raylet_client_pool_.get(), rpc_server_.GetClusterId()); // Initialize by gcs tables data. gcs_node_manager_->Initialize(gcs_init_data); @@ -323,7 +323,7 @@ void GcsServer::InitGcsHealthCheckManager(const GcsInitData &gcs_init_data) { void GcsServer::InitGcsResourceManager(const GcsInitData &gcs_init_data) { RAY_CHECK(cluster_resource_scheduler_ && cluster_task_manager_); - gcs_resource_manager_ = std::make_shared( + gcs_resource_manager_ = std::make_unique( io_context_provider_.GetDefaultIOContext(), cluster_resource_scheduler_->GetClusterResourceManager(), *gcs_node_manager_, @@ -446,25 +446,25 @@ void GcsServer::InitGcsActorManager(const GcsInitData &gcs_init_data) { const rpc::PushTaskReply &reply) { gcs_actor_manager_->OnActorCreationSuccess(std::move(actor), reply); }; - auto client_factory = [this](const rpc::Address &address) { - return std::make_shared(address, client_call_manager_); - }; RAY_CHECK(gcs_resource_manager_ && cluster_task_manager_); scheduler = std::make_unique( io_context_provider_.GetDefaultIOContext(), gcs_table_storage_->ActorTable(), *gcs_node_manager_, - cluster_task_manager_, + *cluster_task_manager_, schedule_failure_handler, schedule_success_handler, - raylet_client_pool_, - client_factory, + *raylet_client_pool_, + /*factory=*/ + [this](const rpc::Address &address) { + return std::make_shared(address, client_call_manager_); + }, /*normal_task_resources_changed_callback=*/ [this](const NodeID &node_id, const rpc::ResourcesData &resources) { gcs_resource_manager_->UpdateNodeNormalTaskResources(node_id, resources); }); - gcs_actor_manager_ = std::make_shared( + gcs_actor_manager_ = std::make_unique( std::move(scheduler), gcs_table_storage_, gcs_publisher_, @@ -480,23 +480,23 @@ void GcsServer::InitGcsActorManager(const GcsInitData &gcs_init_data) { // Initialize by gcs tables data. gcs_actor_manager_->Initialize(gcs_init_data); // Register service. - actor_info_service_.reset(new rpc::ActorInfoGrpcService( - io_context_provider_.GetDefaultIOContext(), *gcs_actor_manager_)); + actor_info_service_ = std::make_unique( + io_context_provider_.GetDefaultIOContext(), *gcs_actor_manager_); rpc_server_.RegisterService(*actor_info_service_); } void GcsServer::InitGcsPlacementGroupManager(const GcsInitData &gcs_init_data) { RAY_CHECK(gcs_table_storage_ && gcs_node_manager_); - gcs_placement_group_scheduler_ = std::make_shared( + gcs_placement_group_scheduler_ = std::make_unique( io_context_provider_.GetDefaultIOContext(), gcs_table_storage_, *gcs_node_manager_, *cluster_resource_scheduler_, - raylet_client_pool_); + *raylet_client_pool_); - gcs_placement_group_manager_ = std::make_shared( + gcs_placement_group_manager_ = std::make_unique( io_context_provider_.GetDefaultIOContext(), - gcs_placement_group_scheduler_, + gcs_placement_group_scheduler_.get(), gcs_table_storage_, *gcs_resource_manager_, [this](const JobID &job_id) { @@ -671,7 +671,7 @@ void GcsServer::InitGcsAutoscalerStateManager(const GcsInitData &gcs_init_data) *gcs_node_manager_, *gcs_actor_manager_, *gcs_placement_group_manager_, - raylet_client_pool_); + *raylet_client_pool_); gcs_autoscaler_state_manager_->Initialize(gcs_init_data); autoscaler_state_service_.reset(new rpc::autoscaler::AutoscalerStateGrpcService( @@ -826,7 +826,7 @@ std::shared_ptr GcsServer::GetOrConnectRedis() { RAY_CHECK(status.ok()) << "Failed to init redis gcs client as " << status; // Init redis failure detector. - gcs_redis_failure_detector_ = std::make_shared( + gcs_redis_failure_detector_ = std::make_unique( io_context_provider_.GetDefaultIOContext(), redis_client_, []() { RAY_LOG(FATAL) << "Redis connection failed. Shutdown GCS."; }); diff --git a/src/ray/gcs/gcs_server/gcs_server.h b/src/ray/gcs/gcs_server/gcs_server.h index 22ece4dda229..6c37e9d8210c 100644 --- a/src/ray/gcs/gcs_server/gcs_server.h +++ b/src/ray/gcs/gcs_server/gcs_server.h @@ -14,6 +14,8 @@ #pragma once +#include + #include "ray/common/asio/asio_util.h" #include "ray/common/asio/instrumented_io_context.h" #include "ray/common/ray_syncer/ray_syncer.h" @@ -218,9 +220,9 @@ class GcsServer { /// The `ClientCallManager` object that is shared by all `NodeManagerWorkerClient`s. rpc::ClientCallManager client_call_manager_; /// Node manager client pool. - std::shared_ptr raylet_client_pool_; + std::unique_ptr raylet_client_pool_; /// The gcs resource manager. - std::shared_ptr gcs_resource_manager_; + std::unique_ptr gcs_resource_manager_; /// The cluster resource scheduler. std::shared_ptr cluster_resource_scheduler_; /// The cluster task manager. @@ -230,15 +232,17 @@ class GcsServer { /// The gcs node manager. std::unique_ptr gcs_node_manager_; /// The health check manager. - std::shared_ptr gcs_healthcheck_manager_; + std::unique_ptr gcs_healthcheck_manager_; /// The gcs redis failure detector. - std::shared_ptr gcs_redis_failure_detector_; + std::unique_ptr gcs_redis_failure_detector_; /// The gcs actor manager. - std::shared_ptr gcs_actor_manager_; + std::unique_ptr gcs_actor_manager_; /// The gcs placement group scheduler. - std::shared_ptr gcs_placement_group_scheduler_; + /// [gcs_placement_group_scheduler_] depends on [raylet_client_pool_]. + std::unique_ptr gcs_placement_group_scheduler_; /// The gcs placement group manager. - std::shared_ptr gcs_placement_group_manager_; + /// [gcs_placement_group_manager_] depends on [gcs_placement_group_scheduler_]. + std::unique_ptr gcs_placement_group_manager_; /// Job info handler and service. std::unique_ptr gcs_job_manager_; std::unique_ptr job_info_service_; diff --git a/src/ray/gcs/gcs_server/test/export_api/gcs_node_manager_export_event_test.cc b/src/ray/gcs/gcs_server/test/export_api/gcs_node_manager_export_event_test.cc index 742716da4b45..61d2d0e8b932 100644 --- a/src/ray/gcs/gcs_server/test/export_api/gcs_node_manager_export_event_test.cc +++ b/src/ray/gcs/gcs_server/test/export_api/gcs_node_manager_export_event_test.cc @@ -41,7 +41,7 @@ class GcsNodeManagerExportAPITest : public ::testing::Test { public: GcsNodeManagerExportAPITest() { raylet_client_ = std::make_shared(); - client_pool_ = std::make_shared( + client_pool_ = std::make_unique( [this](const rpc::Address &) { return raylet_client_; }); gcs_publisher_ = std::make_shared( std::make_unique()); @@ -72,7 +72,7 @@ class GcsNodeManagerExportAPITest : public ::testing::Test { protected: std::shared_ptr gcs_table_storage_; std::shared_ptr raylet_client_; - std::shared_ptr client_pool_; + std::unique_ptr client_pool_; std::shared_ptr gcs_publisher_; instrumented_io_context io_service_; std::string log_dir_; @@ -81,7 +81,7 @@ class GcsNodeManagerExportAPITest : public ::testing::Test { TEST_F(GcsNodeManagerExportAPITest, TestExportEventRegisterNode) { // Test export event is written when a node is added with HandleRegisterNode gcs::GcsNodeManager node_manager( - gcs_publisher_, gcs_table_storage_, client_pool_, ClusterID::Nil()); + gcs_publisher_, gcs_table_storage_, client_pool_.get(), ClusterID::Nil()); auto node = Mocker::GenNodeInfo(); rpc::RegisterNodeRequest register_request; @@ -103,7 +103,7 @@ TEST_F(GcsNodeManagerExportAPITest, TestExportEventRegisterNode) { TEST_F(GcsNodeManagerExportAPITest, TestExportEventUnregisterNode) { // Test export event is written when a node is removed with HandleUnregisterNode gcs::GcsNodeManager node_manager( - gcs_publisher_, gcs_table_storage_, client_pool_, ClusterID::Nil()); + gcs_publisher_, gcs_table_storage_, client_pool_.get(), ClusterID::Nil()); auto node = Mocker::GenNodeInfo(); auto node_id = NodeID::FromBinary(node->node_id()); node_manager.AddNode(node); diff --git a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_mock_test.cc b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_mock_test.cc index 6beeb8b7504c..aca66ca39c09 100644 --- a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_mock_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_mock_test.cc @@ -42,7 +42,7 @@ class GcsActorSchedulerMockTest : public Test { std::make_unique(nullptr, nullptr, nullptr, ClusterID::Nil()); raylet_client = std::make_shared(); core_worker_client = std::make_shared(); - client_pool = std::make_shared( + client_pool = std::make_unique( [this](const rpc::Address &) { return raylet_client; }); local_node_id = NodeID::FromRandom(); auto cluster_resource_scheduler = std::make_shared( @@ -52,7 +52,7 @@ class GcsActorSchedulerMockTest : public Test { /*is_node_available_fn=*/ [](auto) { return true; }, /*is_local_node_with_raylet=*/false); - auto cluster_task_manager = std::make_shared( + cluster_task_manager = std::make_unique( local_node_id, cluster_resource_scheduler, /*get_node_info=*/ @@ -70,10 +70,10 @@ class GcsActorSchedulerMockTest : public Test { io_context, *actor_table, *gcs_node_manager, - cluster_task_manager, + *cluster_task_manager, [this](auto a, auto b, auto c) { schedule_failure_handler(a); }, [this](auto a, const rpc::PushTaskReply) { schedule_success_handler(a); }, - client_pool, + *client_pool, [this](const rpc::Address &) { return core_worker_client; }); auto node_info = std::make_shared(); node_info->set_state(rpc::GcsNodeInfo::ALIVE); @@ -82,14 +82,16 @@ class GcsActorSchedulerMockTest : public Test { worker_id = WorkerID::FromRandom(); gcs_node_manager->AddNode(node_info); } + std::shared_ptr raylet_client; instrumented_io_context io_context; std::shared_ptr store_client; std::unique_ptr actor_table; - std::unique_ptr actor_scheduler; std::unique_ptr gcs_node_manager; + std::unique_ptr cluster_task_manager; + std::unique_ptr actor_scheduler; std::shared_ptr core_worker_client; - std::shared_ptr client_pool; + std::unique_ptr client_pool; std::shared_ptr>> counter; MockCallback schedule_failure_handler; diff --git a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc index 48a0c0becbe9..6302ee02ed63 100644 --- a/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc @@ -12,10 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include + #include // clang-format off -#include "gtest/gtest.h" #include "ray/common/asio/instrumented_io_context.h" #include "ray/gcs/gcs_server/gcs_actor_scheduler.h" #include "ray/gcs/gcs_server/test/gcs_server_test_util.h" @@ -39,7 +40,7 @@ class GcsActorSchedulerTest : public ::testing::Test { store_client_ = std::make_shared(io_service_); gcs_table_storage_ = std::make_shared(io_service_); gcs_node_manager_ = std::make_shared( - gcs_publisher_, gcs_table_storage_, raylet_client_pool_, ClusterID::Nil()); + gcs_publisher_, gcs_table_storage_, raylet_client_pool_.get(), ClusterID::Nil()); gcs_actor_table_ = std::make_shared(store_client_); local_node_id_ = NodeID::FromRandom(); @@ -73,7 +74,7 @@ class GcsActorSchedulerTest : public ::testing::Test { io_service_, *gcs_actor_table_, *gcs_node_manager_, - cluster_task_manager_, + *cluster_task_manager_, /*schedule_failure_handler=*/ [this](std::shared_ptr actor, const rpc::RequestWorkerLeaseReply::SchedulingFailureType failure_type, @@ -84,7 +85,7 @@ class GcsActorSchedulerTest : public ::testing::Test { [this](std::shared_ptr actor, const rpc::PushTaskReply &reply) { success_actors_.emplace_back(std::move(actor)); }, - raylet_client_pool_, + *raylet_client_pool_, /*client_factory=*/ [this](const rpc::Address &address) { return worker_client_; }, /*normal_task_resources_changed_callback=*/ diff --git a/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc index 856261b032e4..2f281fa31844 100644 --- a/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_autoscaler_state_manager_test.cc @@ -61,7 +61,7 @@ class GcsAutoscalerStateManagerTest : public ::testing::Test { void SetUp() override { raylet_client_ = std::make_shared(); - client_pool_ = std::make_shared( + client_pool_ = std::make_unique( [this](const rpc::Address &) { return raylet_client_; }); cluster_resource_manager_ = std::make_unique(io_service_); gcs_node_manager_ = std::make_shared(); @@ -86,7 +86,7 @@ class GcsAutoscalerStateManagerTest : public ::testing::Test { *gcs_node_manager_, *gcs_actor_manager_, *gcs_placement_group_manager_, - client_pool_)); + *client_pool_)); } public: diff --git a/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc index fcc273aa4f9f..eb12d59dbdb3 100644 --- a/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc @@ -28,7 +28,7 @@ class GcsNodeManagerTest : public ::testing::Test { public: GcsNodeManagerTest() { raylet_client_ = std::make_shared(); - client_pool_ = std::make_shared( + client_pool_ = std::make_unique( [this](const rpc::Address &) { return raylet_client_; }); gcs_publisher_ = std::make_shared( std::make_unique()); @@ -37,13 +37,13 @@ class GcsNodeManagerTest : public ::testing::Test { protected: std::shared_ptr gcs_table_storage_; std::shared_ptr raylet_client_; - std::shared_ptr client_pool_; + std::unique_ptr client_pool_; std::shared_ptr gcs_publisher_; }; TEST_F(GcsNodeManagerTest, TestManagement) { gcs::GcsNodeManager node_manager( - gcs_publisher_, gcs_table_storage_, client_pool_, ClusterID::Nil()); + gcs_publisher_, gcs_table_storage_, client_pool_.get(), ClusterID::Nil()); // Test Add/Get/Remove functionality. auto node = Mocker::GenNodeInfo(); auto node_id = NodeID::FromBinary(node->node_id()); @@ -58,7 +58,7 @@ TEST_F(GcsNodeManagerTest, TestManagement) { TEST_F(GcsNodeManagerTest, TestListener) { gcs::GcsNodeManager node_manager( - gcs_publisher_, gcs_table_storage_, client_pool_, ClusterID::Nil()); + gcs_publisher_, gcs_table_storage_, client_pool_.get(), ClusterID::Nil()); // Test AddNodeAddedListener. int node_count = 1000; std::vector> added_nodes; diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc index 6cfd689ac168..1e3ef61060c8 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_mock_test.cc @@ -47,7 +47,7 @@ class GcsPlacementGroupManagerMockTest : public Test { gcs_placement_group_manager_ = std::make_unique(io_context_, - gcs_placement_group_scheduler_, + gcs_placement_group_scheduler_.get(), gcs_table_storage_, *resource_manager_, [](auto &) { return ""; }); diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc index d407332d8021..ad808b644b67 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_manager_test.cc @@ -89,7 +89,7 @@ class GcsPlacementGroupManagerTest : public ::testing::Test { io_service_, cluster_resource_manager_, *gcs_node_manager_, NodeID::FromRandom()); gcs_placement_group_manager_.reset(new gcs::GcsPlacementGroupManager( io_service_, - mock_placement_group_scheduler_, + mock_placement_group_scheduler_.get(), gcs_table_storage_, *gcs_resource_manager_, [this](const JobID &job_id) { return job_namespace_table_[job_id]; })); diff --git a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc index 8bbcaedefc63..093bdaf13fcc 100644 --- a/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc +++ b/src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc @@ -55,21 +55,21 @@ class GcsPlacementGroupSchedulerTest : public ::testing::Test { [](auto) { return true; }, /*is_local_node_with_raylet=*/false); gcs_node_manager_ = std::make_shared( - gcs_publisher_, gcs_table_storage_, raylet_client_pool_, ClusterID::Nil()); + gcs_publisher_, gcs_table_storage_, raylet_client_pool_.get(), ClusterID::Nil()); gcs_resource_manager_ = std::make_shared( io_service_, cluster_resource_scheduler_->GetClusterResourceManager(), *gcs_node_manager_, local_node_id); store_client_ = std::make_shared(io_service_); - raylet_client_pool_ = std::make_shared( + raylet_client_pool_ = std::make_unique( [this](const rpc::Address &addr) { return raylet_clients_[addr.port()]; }); scheduler_ = std::make_shared( io_service_, gcs_table_storage_, *gcs_node_manager_, *cluster_resource_scheduler_, - raylet_client_pool_); + *raylet_client_pool_); counter_.reset(new CounterMap()); } @@ -296,7 +296,7 @@ class GcsPlacementGroupSchedulerTest : public ::testing::Test { ABSL_GUARDED_BY(placement_group_requests_mutex_); std::shared_ptr gcs_publisher_; std::shared_ptr gcs_table_storage_; - std::shared_ptr raylet_client_pool_; + std::unique_ptr raylet_client_pool_; std::shared_ptr> counter_; }; diff --git a/src/ray/gcs/pb_util.h b/src/ray/gcs/pb_util.h index cb3c518072b2..0bdd056b0b8d 100644 --- a/src/ray/gcs/pb_util.h +++ b/src/ray/gcs/pb_util.h @@ -16,6 +16,7 @@ #include +#include "absl/time/time.h" #include "ray/common/constants.h" #include "ray/common/id.h" #include "ray/common/ray_config.h" @@ -61,27 +62,11 @@ inline std::shared_ptr CreateJobTableData( } /// Helper function to produce error table data. -inline std::shared_ptr CreateErrorTableData( +std::shared_ptr CreateErrorTableData( const std::string &error_type, const std::string &error_msg, - double timestamp, - const JobID &job_id = JobID::Nil()) { - uint32_t max_error_msg_size_bytes = RayConfig::instance().max_error_msg_size_bytes(); - auto error_info_ptr = std::make_shared(); - error_info_ptr->set_type(error_type); - if (error_msg.length() > max_error_msg_size_bytes) { - std::ostringstream stream; - stream << "The message size exceeds " << std::to_string(max_error_msg_size_bytes) - << " bytes. Find the full log from the log files. Here is abstract: " - << error_msg.substr(0, max_error_msg_size_bytes); - error_info_ptr->set_error_message(stream.str()); - } else { - error_info_ptr->set_error_message(error_msg); - } - error_info_ptr->set_timestamp(timestamp); - error_info_ptr->set_job_id(job_id.Binary()); - return error_info_ptr; -} + absl::Time timestamp, + const JobID &job_id = JobID::Nil()); /// Helper function to produce worker failure data. inline std::shared_ptr CreateWorkerFailureData( diff --git a/src/ray/gcs/pb_utils.cc b/src/ray/gcs/pb_utils.cc new file mode 100644 index 000000000000..9330f80c27ce --- /dev/null +++ b/src/ray/gcs/pb_utils.cc @@ -0,0 +1,47 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// TODO(hjiang): Move all functions from `pb_utils.h` to this implementation file. + +#include + +#include "absl/strings/str_format.h" +#include "ray/gcs/pb_util.h" + +namespace ray::gcs { + +std::shared_ptr CreateErrorTableData( + const std::string &error_type, + const std::string &error_msg, + absl::Time timestamp, + const JobID &job_id) { + uint32_t max_error_msg_size_bytes = RayConfig::instance().max_error_msg_size_bytes(); + auto error_info_ptr = std::make_shared(); + error_info_ptr->set_type(error_type); + if (error_msg.length() > max_error_msg_size_bytes) { + std::string formatted_error_message = absl::StrFormat( + "The message size exceeds %d bytes. Find the full log from the log files. Here " + "is abstract: %s", + max_error_msg_size_bytes, + std::string_view{error_msg}.substr(0, max_error_msg_size_bytes)); + error_info_ptr->set_error_message(std::move(formatted_error_message)); + } else { + error_info_ptr->set_error_message(error_msg); + } + error_info_ptr->set_timestamp(absl::ToUnixMillis(timestamp)); + error_info_ptr->set_job_id(job_id.Binary()); + return error_info_ptr; +} + +} // namespace ray::gcs diff --git a/src/ray/protobuf/gcs.proto b/src/ray/protobuf/gcs.proto index b34f5bd9f39b..2d0ff6dfac4a 100644 --- a/src/ray/protobuf/gcs.proto +++ b/src/ray/protobuf/gcs.proto @@ -165,6 +165,8 @@ message ErrorTableData { // The error message. string error_message = 3; // The timestamp of the error message. + // Unit: millisecond. + // TODO(hjiang): Update field naming from `timestamp` to `timestamp_millisec`. double timestamp = 4; } diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 879edff0bb6c..18384711de58 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -143,7 +143,7 @@ NodeManager::NodeManager( /*starting_worker_timeout_callback=*/ [this] { cluster_task_manager_->ScheduleAndDispatchTasks(); }, config.ray_debugger_external, - /*get_time=*/[]() { return absl::GetCurrentTimeNanos() / 1e6; }), + /*get_time=*/[]() { return absl::Now(); }), client_call_manager_(io_service), worker_rpc_pool_(client_call_manager_), core_worker_subscriber_(std::make_unique( @@ -1562,8 +1562,8 @@ void NodeManager::DisconnectClient(const std::shared_ptr &clie .WithField("node_id", self_node_id_.Hex()) .WithField("job_id", worker->GetAssignedJobId().Hex()) << error_message_str; - auto error_data_ptr = - gcs::CreateErrorTableData(type, error_message_str, current_time_ms(), job_id); + auto error_data_ptr = gcs::CreateErrorTableData( + type, error_message_str, absl::FromUnixMillis(current_time_ms()), job_id); RAY_CHECK_OK(gcs_client_->Errors().AsyncReportJobError(error_data_ptr, nullptr)); } } @@ -1762,9 +1762,11 @@ void NodeManager::ProcessPushErrorRequestMessage(const uint8_t *message_data) { auto const &type = string_from_flatbuf(*message->type()); auto const &error_message = string_from_flatbuf(*message->error_message()); + // TODO(hjiang): Figure out what's the unit for `PushErrorRequest`. double timestamp = message->timestamp(); JobID job_id = from_flatbuf(*message->job_id()); - auto error_data_ptr = gcs::CreateErrorTableData(type, error_message, timestamp, job_id); + auto error_data_ptr = gcs::CreateErrorTableData( + type, error_message, absl::FromUnixMillis(timestamp), job_id); RAY_CHECK_OK(gcs_client_->Errors().AsyncReportJobError(error_data_ptr, nullptr)); } @@ -2126,8 +2128,8 @@ void NodeManager::MarkObjectsAsFailed( << " object may hang forever."; std::string error_message = stream.str(); RAY_LOG(ERROR) << error_message; - auto error_data_ptr = - gcs::CreateErrorTableData("task", error_message, current_time_ms(), job_id); + auto error_data_ptr = gcs::CreateErrorTableData( + "task", error_message, absl::FromUnixMillis(current_time_ms()), job_id); RAY_CHECK_OK(gcs_client_->Errors().AsyncReportJobError(error_data_ptr, nullptr)); } } diff --git a/src/ray/raylet/scheduling/cluster_task_manager.h b/src/ray/raylet/scheduling/cluster_task_manager.h index 058c40f97fcf..752cbd6b3e98 100644 --- a/src/ray/raylet/scheduling/cluster_task_manager.h +++ b/src/ray/raylet/scheduling/cluster_task_manager.h @@ -161,6 +161,7 @@ class ClusterTaskManager : public ClusterTaskManagerInterface { const NodeID &self_node_id_; /// Responsible for resource tracking/view of the cluster. + /// TODO(hjiang): Use reference instead of shared pointer. std::shared_ptr cluster_resource_scheduler_; /// Function to get the node information of a given node id. diff --git a/src/ray/raylet/worker_pool.cc b/src/ray/raylet/worker_pool.cc index eda893e8efbe..5964063a3205 100644 --- a/src/ray/raylet/worker_pool.cc +++ b/src/ray/raylet/worker_pool.cc @@ -89,10 +89,10 @@ WorkerPool::WorkerPool(instrumented_io_context &io_service, const std::vector &worker_ports, std::shared_ptr gcs_client, const WorkerCommandMap &worker_commands, - const std::string &native_library_path, + std::string native_library_path, std::function starting_worker_timeout_callback, int ray_debugger_external, - const std::function get_time) + std::function get_time) : worker_startup_token_counter_(0), io_service_(&io_service), node_id_(node_id), @@ -105,15 +105,15 @@ WorkerPool::WorkerPool(instrumented_io_context &io_service, RayConfig::instance().worker_maximum_startup_concurrency() : maximum_startup_concurrency), gcs_client_(std::move(gcs_client)), - native_library_path_(native_library_path), - starting_worker_timeout_callback_(starting_worker_timeout_callback), + native_library_path_(std::move(native_library_path)), + starting_worker_timeout_callback_(std::move(starting_worker_timeout_callback)), ray_debugger_external(ray_debugger_external), first_job_registered_python_worker_count_(0), first_job_driver_wait_num_python_workers_( std::min(num_prestarted_python_workers, maximum_startup_concurrency_)), num_prestart_python_workers(num_prestarted_python_workers), periodical_runner_(io_service), - get_time_(get_time) { + get_time_(std::move(get_time)) { RAY_CHECK_GT(maximum_startup_concurrency_, 0); // We need to record so that the metric exists. This way, we report that 0 // processes have started before a task runs on the node (as opposed to the @@ -1043,7 +1043,7 @@ void WorkerPool::PushWorker(const std::shared_ptr &worker) { } void WorkerPool::TryKillingIdleWorkers() { - int64_t now = get_time_(); + const auto now = get_time_(); // Filter out all idle workers that are already dead and/or associated with // jobs that have already finished. @@ -1056,14 +1056,14 @@ void WorkerPool::TryKillingIdleWorkers() { } const auto &job_id = idle_worker->GetAssignedJobId(); - if (finished_jobs_.count(job_id) > 0) { + if (finished_jobs_.contains(job_id)) { // The job has finished, so we should kill the worker immediately. KillIdleWorker(idle_worker, it->second); it = idle_of_all_languages_.erase(it); } else { - if (it->second == -1 || - now - it->second > - RayConfig::instance().idle_worker_killing_time_threshold_ms()) { + if (now - it->second > + absl::Milliseconds( + RayConfig::instance().idle_worker_killing_time_threshold_ms())) { // The job has not yet finished and the worker has been idle for longer // than the timeout. num_killable_idle_workers++; @@ -1085,9 +1085,9 @@ void WorkerPool::TryKillingIdleWorkers() { auto it = idle_of_all_languages_.begin(); while (num_killable_idle_workers > num_desired_idle_workers && it != idle_of_all_languages_.end()) { - if (it->second == -1 || - now - it->second > - RayConfig::instance().idle_worker_killing_time_threshold_ms()) { + if (now - it->second > + absl::Milliseconds( + RayConfig::instance().idle_worker_killing_time_threshold_ms())) { RAY_LOG(DEBUG) << "Number of idle workers " << num_killable_idle_workers << " is larger than the number of desired workers " << num_desired_idle_workers << " killing idle worker with PID " @@ -1102,7 +1102,7 @@ void WorkerPool::TryKillingIdleWorkers() { } void WorkerPool::KillIdleWorker(std::shared_ptr idle_worker, - int64_t last_time_used_ms) { + absl::Time last_time_used) { // To avoid object lost issue caused by forcibly killing, send an RPC request to the // worker to allow it to do cleanup before exiting. We kill it anyway if the driver // is already exited. @@ -1122,8 +1122,8 @@ void WorkerPool::KillIdleWorker(std::shared_ptr idle_worker, } rpc_client->Exit( request, - [this, idle_worker, last_time_used_ms](const ray::Status &status, - const rpc::ExitReply &r) { + [this, idle_worker, last_time_used](const ray::Status &status, + const rpc::ExitReply &r) { RAY_CHECK(pending_exit_idle_workers_.erase(idle_worker->WorkerId())); if (!status.ok()) { RAY_LOG(ERROR) << "Failed to send exit request: " << status.ToString(); @@ -1149,8 +1149,7 @@ void WorkerPool::KillIdleWorker(std::shared_ptr idle_worker, // kill the worker (e.g., when the worker owns the object). Without this, // if the first N workers own objects, it can't kill idle workers that are // >= N+1. - idle_of_all_languages_.push_back( - std::make_pair(idle_worker, last_time_used_ms)); + idle_of_all_languages_.emplace_back(idle_worker, last_time_used); } }); } @@ -1311,7 +1310,7 @@ void WorkerPool::PopWorker(const TaskSpecification &task_spec, auto worker_fits_for_task_fn = [this, &pop_worker_request, &skip_reason_count]( - const std::pair, int64_t> &pair) -> bool { + const std::pair, absl::Time> &pair) -> bool { const auto &worker = pair.first; WorkerUnfitForTaskReason reason = WorkerFitsForTask(*worker, *pop_worker_request); if (reason == WorkerUnfitForTaskReason::NONE) { @@ -1544,6 +1543,7 @@ void WorkerPool::WarnAboutSize() { << "some discussion of workarounds)."; std::string warning_message_str = warning_message.str(); RAY_LOG(WARNING) << warning_message_str; + auto error_data_ptr = gcs::CreateErrorTableData( "worker_pool_large", warning_message_str, get_time_()); RAY_CHECK_OK(gcs_client_->Errors().AsyncReportJobError(error_data_ptr, nullptr)); diff --git a/src/ray/raylet/worker_pool.h b/src/ray/raylet/worker_pool.h index ef2e1e048635..3d7f456f82cb 100644 --- a/src/ray/raylet/worker_pool.h +++ b/src/ray/raylet/worker_pool.h @@ -25,6 +25,7 @@ #include #include +#include "absl/time/time.h" #include "ray/common/asio/instrumented_io_context.h" #include "ray/common/asio/periodical_runner.h" #include "ray/common/client_connection.h" @@ -200,7 +201,7 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// it times out to start a worker. /// \param ray_debugger_external Ray debugger in workers will be started in a way /// that they are accessible from outside the node. - /// \param get_time A callback to get the current time. + /// \param get_time A callback to get the current time in milliseconds. WorkerPool(instrumented_io_context &io_service, const NodeID node_id, const std::string node_address, @@ -212,10 +213,10 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { const std::vector &worker_ports, std::shared_ptr gcs_client, const WorkerCommandMap &worker_commands, - const std::string &native_library_path, + std::string native_library_path, std::function starting_worker_timeout_callback, int ray_debugger_external, - const std::function get_time); + std::function get_time); /// Destructor responsible for freeing a set of workers owned by this class. virtual ~WorkerPool() override; @@ -472,7 +473,7 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// TODO(scv119): replace dynamic options by runtime_env. const std::vector &LookupWorkerDynamicOptions(StartupToken token) const; - void KillIdleWorker(std::shared_ptr worker, int64_t last_time_used_ms); + void KillIdleWorker(std::shared_ptr worker, absl::Time last_time_used); /// Gloabl startup token variable. Incremented once assigned /// to a worker process and is added to @@ -587,7 +588,8 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { /// The pool of idle non-actor workers of all languages. This is used to kill idle /// workers in FIFO order. The second element of std::pair is the time a worker becomes /// idle. - std::list, int64_t>> idle_of_all_languages_; + std::list, absl::Time>> + idle_of_all_languages_; private: /// A helper function that returns the reference of the pool state @@ -792,7 +794,7 @@ class WorkerPool : public WorkerPoolInterface, public IOWorkerPoolInterface { PeriodicalRunner periodical_runner_; /// A callback to get the current time. - const std::function get_time_; + const std::function get_time_; /// Runtime env manager client. std::shared_ptr runtime_env_agent_client_; /// Stats diff --git a/src/ray/raylet/worker_pool_test.cc b/src/ray/raylet/worker_pool_test.cc index 0da18903fdfe..022c5055522a 100644 --- a/src/ray/raylet/worker_pool_test.cc +++ b/src/ray/raylet/worker_pool_test.cc @@ -14,8 +14,10 @@ #include "ray/raylet/worker_pool.h" -#include "gmock/gmock.h" -#include "gtest/gtest.h" +#include +#include + +#include "absl/time/time.h" #include "nlohmann/json.hpp" #include "ray/common/asio/asio_util.h" #include "ray/common/asio/instrumented_io_context.h" @@ -26,9 +28,8 @@ #include "src/ray/protobuf/runtime_env_agent.pb.h" using json = nlohmann::json; -namespace ray { -namespace raylet { +namespace ray::raylet { int MAXIMUM_STARTUP_CONCURRENCY = 15; int PYTHON_PRESTART_WORKERS = 15; @@ -140,7 +141,7 @@ class WorkerPoolMock : public WorkerPool { "", []() {}, 0, - [this]() { return current_time_ms_; }), + [this]() { return absl::FromUnixMillis(current_time_ms_); }), last_worker_process_(), instrumented_io_service_(io_service), error_message_type_(1), @@ -241,7 +242,7 @@ class WorkerPoolMock : public WorkerPool { size_t GetIdleWorkerSize() { return idle_of_all_languages_.size(); } - std::list, int64_t>> &GetIdleWorkers() { + std::list, absl::Time>> &GetIdleWorkers() { return idle_of_all_languages_; } @@ -2151,9 +2152,7 @@ TEST_F(WorkerPoolTest, RegisterFirstJavaDriverCallbackImmediately) { ASSERT_TRUE(callback_called); } -} // namespace raylet - -} // namespace ray +} // namespace ray::raylet int main(int argc, char **argv) { InitShutdownRAII ray_log_shutdown_raii( diff --git a/src/ray/util/BUILD b/src/ray/util/BUILD index 23d9f1e90150..87f8a57e8dea 100644 --- a/src/ray/util/BUILD +++ b/src/ray/util/BUILD @@ -55,3 +55,13 @@ cc_library( srcs = ["thread_checker.cc"], visibility = ["//visibility:public"], ) + +cc_library( + name = "shared_lru", + hdrs = ["shared_lru.h"], + visibility = ["//visibility:public"], + deps = [ + ":util", + "@com_google_absl//absl/container:flat_hash_map", + ], +) diff --git a/src/ray/util/shared_lru.h b/src/ray/util/shared_lru.h new file mode 100644 index 000000000000..8132e38b6f12 --- /dev/null +++ b/src/ray/util/shared_lru.h @@ -0,0 +1,207 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// SharedLruCache is a LRU cache, with all entries shared, which means a single entry +// could be accessed by multiple getters. All values are wrapped with shared pointer to +// avoid copy at get operation, meanwhile also useful to maintain memory validity at any +// time. +// +// Example usage: +// SharedLruCache cache{cap}; +// // Put a key-value pair into cache. +// cache.Put("key", std::make_shared("val")); +// +// // Get a key-value pair from cache. +// auto val = cache.Get("key"); +// // Check and consume `val`. +// +// TODO(hjiang): +// 1. Add template arguments for key hash and key equal, to pass into absl::flat_hash_map. +// 2. Provide a key hash wrapper to save a copy. +// 3. flat hash map supports heterogeneous lookup, expose `KeyLike` templated interface. +// 4. Add a `GetOrCreate` interface, which takes factory function to creation value. +// 5. For thread-safe cache, add a sharded container wrapper to reduce lock contention. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +#include "absl/container/flat_hash_map.h" +#include "src/ray/util/logging.h" + +namespace ray::utils::container { + +template +class SharedLruCache final { + public: + using key_type = Key; + using mapped_type = Val; + + // A `max_entries` of 0 means that there is no limit on the number of entries + // in the cache. + explicit SharedLruCache(size_t max_entries) : max_entries_(max_entries) {} + + SharedLruCache(const SharedLruCache &) = delete; + SharedLruCache &operator=(const SharedLruCache &) = delete; + + ~SharedLruCache() = default; + + // Insert `value` with key `key`. This will replace any previous entry with + // the same key. + void Put(Key key, std::shared_ptr value) { + RAY_CHECK(value != nullptr); + auto iter = cache_.find(key); + if (iter != cache_.end()) { + lru_list_.splice(lru_list_.begin(), lru_list_, iter->second.lru_iterator); + iter->second.value = std::move(value); + return; + } + + lru_list_.emplace_front(key); + Entry new_entry{std::move(value), lru_list_.begin()}; + cache_[std::move(key)] = std::move(new_entry); + + if (max_entries_ > 0 && lru_list_.size() > max_entries_) { + const auto &stale_key = lru_list_.back(); + cache_.erase(stale_key); + lru_list_.pop_back(); + } + + RAY_CHECK_EQ(lru_list_.size(), cache_.size()); + } + + // Delete the entry with key `key`. Return true if the entry was found for + // `key`, false if the entry was not found. In both cases, there is no entry + // with key `key` existed after the call. + bool Delete(const Key &key) { + auto it = cache_.find(key); + if (it == cache_.end()) { + return false; + } + lru_list_.erase(it->second.lru_iterator); + cache_.erase(it); + return true; + } + + // Look up the entry with key `key`. Return nullptr if key doesn't exist. + std::shared_ptr Get(const Key &key) { + const auto cache_iter = cache_.find(key); + if (cache_iter == cache_.end()) { + return nullptr; + } + lru_list_.splice(lru_list_.begin(), lru_list_, cache_iter->second.lru_iterator); + return cache_iter->second.value; + } + + // Clear the cache. + void Clear() { + cache_.clear(); + lru_list_.clear(); + } + + // Accessors for cache parameters. + size_t max_entries() const { return max_entries_; } + + private: + struct Entry { + // The entry's value. + std::shared_ptr value; + + // A list iterator pointing to the entry's position in the LRU list. + typename std::list::iterator lru_iterator; + }; + + using EntryMap = absl::flat_hash_map; + + // The maximum number of entries in the cache. A value of 0 means there is no + // limit on entry count. + const size_t max_entries_; + + // Stores key-value pairs. + EntryMap cache_; + + // The LRU list of entries. The front of the list identifies the most + // recently accessed entry. + std::list lru_list_; +}; + +// Same interfaces as `SharedLruCache`, but all cached values are +// `const`-specified to avoid concurrent updates. +template +using SharedLruConstCache = SharedLruCache; + +// Same interface and functionality as `SharedLruCache`, but thread-safe version. +template +class ThreadSafeSharedLruCache final { + public: + using key_type = Key; + using mapped_type = Val; + + // A `max_entries` of 0 means that there is no limit on the number of entries + // in the cache. + explicit ThreadSafeSharedLruCache(size_t max_entries) : cache_(max_entries) {} + + ThreadSafeSharedLruCache(const ThreadSafeSharedLruCache &) = delete; + ThreadSafeSharedLruCache &operator=(const ThreadSafeSharedLruCache &) = delete; + + ~ThreadSafeSharedLruCache() = default; + + // Insert `value` with key `key`. This will replace any previous entry with + // the same key. + void Put(Key key, std::shared_ptr value) { + std::lock_guard lck(mu_); + cache_.Put(std::move(key), std::move(value)); + } + + // Delete the entry with key `key`. Return true if the entry was found for + // `key`, false if the entry was not found. In both cases, there is no entry + // with key `key` existed after the call. + bool Delete(const Key &key) { + std::lock_guard lck(mu_); + return cache_.Delete(key); + } + + // Look up the entry with key `key`. Return std::nullopt if key doesn't exist. + // If found, return a copy for the value. + std::shared_ptr Get(const Key &key) { + std::lock_guard lck(mu_); + return cache_.Get(key); + } + + // Clear the cache. + void Clear() { + std::lock_guard lck(mu_); + cache_.Clear(); + } + + // Accessors for cache parameters. + size_t max_entries() const { return cache_.max_entries(); } + + private: + std::mutex mu_; + SharedLruCache cache_; +}; + +// Same interfaces as `SharedLruCache`, but all cached values are +// `const`-specified to avoid concurrent updates. +template +using ThreadSafeSharedLruConstCache = ThreadSafeSharedLruCache; + +} // namespace ray::utils::container diff --git a/src/ray/util/tests/BUILD b/src/ray/util/tests/BUILD index 2941d105cf91..b85c01f28ebf 100644 --- a/src/ray/util/tests/BUILD +++ b/src/ray/util/tests/BUILD @@ -194,3 +194,15 @@ cc_test( "@com_google_googletest//:gtest_main", ], ) + +cc_test( + name = "shared_lru_test", + srcs = ["shared_lru_test.cc"], + deps = [ + "//src/ray/util:shared_lru", + "@com_google_googletest//:gtest_main", + ], + size = "small", + copts = COPTS, + tags = ["team:core"], +) diff --git a/src/ray/util/tests/shared_lru_test.cc b/src/ray/util/tests/shared_lru_test.cc new file mode 100644 index 000000000000..7c47f4d1daf0 --- /dev/null +++ b/src/ray/util/tests/shared_lru_test.cc @@ -0,0 +1,76 @@ +// Copyright 2024 The Ray Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include "src/ray/util/shared_lru.h" + +#include + +#include +#include + +namespace ray::utils::container { + +namespace { +constexpr size_t kTestCacheSz = 1; +} // namespace + +TEST(SharedLruCache, PutAndGet) { + ThreadSafeSharedLruCache cache{kTestCacheSz}; + + // No value initially. + auto val = cache.Get("1"); + EXPECT_EQ(val, nullptr); + + // Check put and get. + cache.Put("1", std::make_shared("1")); + val = cache.Get("1"); + EXPECT_NE(val, nullptr); + EXPECT_EQ(*val, "1"); + + // Check key eviction. + cache.Put("2", std::make_shared("2")); + val = cache.Get("1"); + EXPECT_EQ(val, nullptr); + val = cache.Get("2"); + EXPECT_NE(val, nullptr); + EXPECT_EQ(*val, "2"); + + // Check deletion. + EXPECT_FALSE(cache.Delete("1")); + EXPECT_TRUE(cache.Delete("2")); + val = cache.Get("2"); + EXPECT_EQ(val, nullptr); +} + +// Testing senario: push multiple same keys into the cache. +TEST(SharedLruCache, SameKeyTest) { + ThreadSafeSharedLruCache cache{2}; + + cache.Put(1, std::make_shared(1)); + auto val = cache.Get(1); + EXPECT_NE(val, nullptr); + EXPECT_EQ(1, *val); + + cache.Put(1, std::make_shared(2)); + val = cache.Get(1); + EXPECT_NE(val, nullptr); + EXPECT_EQ(2, *val); +} + +TEST(SharedLruConstCache, TypeAliasAssertion) { + static_assert( + std::is_same_v, SharedLruCache>); +} + +} // namespace ray::utils::container