From e54d618f03a57e0192c3cb8d3780fe9a2c9075b4 Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Thu, 6 Nov 2025 16:49:15 -0800 Subject: [PATCH 1/8] first draft --- .../common/callbacks/cloud_downloader.py | 4 +- .../common/utils/cloud_filesystem/__init__.py | 29 ++ .../cloud_filesystem/azure_filesystem.py | 82 ++++ .../common/utils/cloud_filesystem/base.py | 81 ++++ .../utils/cloud_filesystem/gcs_filesystem.py | 81 ++++ .../cloud_filesystem/pyarrow_filesystem.py | 344 +++++++++++++++ .../utils/cloud_filesystem/s3_filesystem.py | 233 ++++++++++ .../llm/_internal/common/utils/cloud_utils.py | 403 ++---------------- .../tests/common/utils/test_cloud_utils.py | 30 -- 9 files changed, 896 insertions(+), 391 deletions(-) create mode 100644 python/ray/llm/_internal/common/utils/cloud_filesystem/__init__.py create mode 100644 python/ray/llm/_internal/common/utils/cloud_filesystem/azure_filesystem.py create mode 100644 python/ray/llm/_internal/common/utils/cloud_filesystem/base.py create mode 100644 python/ray/llm/_internal/common/utils/cloud_filesystem/gcs_filesystem.py create mode 100644 python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py create mode 100644 python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py diff --git a/python/ray/llm/_internal/common/callbacks/cloud_downloader.py b/python/ray/llm/_internal/common/callbacks/cloud_downloader.py index 489e25333bfc..aa11ba310c6b 100644 --- a/python/ray/llm/_internal/common/callbacks/cloud_downloader.py +++ b/python/ray/llm/_internal/common/callbacks/cloud_downloader.py @@ -79,9 +79,7 @@ def on_before_download_model_files_distributed(self) -> None: paths = self.kwargs["paths"] start_time = time.monotonic() for cloud_uri, local_path in paths: - CloudFileSystem.download_files_parallel( - path=local_path, bucket_uri=cloud_uri - ) + CloudFileSystem.download_files(path=local_path, bucket_uri=cloud_uri) end_time = time.monotonic() logger.info( f"CloudDownloader: Files downloaded in {end_time - start_time} seconds" diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/__init__.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/__init__.py new file mode 100644 index 000000000000..492e13e61b1c --- /dev/null +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/__init__.py @@ -0,0 +1,29 @@ +"""Cloud filesystem module for provider-specific implementations. + +This module provides a unified interface for cloud storage operations across +different providers (S3, GCS, Azure) while allowing provider-specific optimizations. +""" + +from ray.llm._internal.common.utils.cloud_filesystem.azure_filesystem import ( + AzureFileSystem, +) +from ray.llm._internal.common.utils.cloud_filesystem.base import ( + BaseCloudFileSystem, +) +from ray.llm._internal.common.utils.cloud_filesystem.gcs_filesystem import ( + GCSFileSystem, +) +from ray.llm._internal.common.utils.cloud_filesystem.pyarrow_filesystem import ( + PyArrowFileSystem, +) +from ray.llm._internal.common.utils.cloud_filesystem.s3_filesystem import ( + S3FileSystem, +) + +__all__ = [ + "BaseCloudFileSystem", + "PyArrowFileSystem", + "GCSFileSystem", + "AzureFileSystem", + "S3FileSystem", +] diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/azure_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/azure_filesystem.py new file mode 100644 index 000000000000..d097ac5dbc1f --- /dev/null +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/azure_filesystem.py @@ -0,0 +1,82 @@ +"""Azure-specific filesystem implementation. + +This module provides an Azure-specific implementation that delegates to PyArrowFileSystem. +This maintains backward compatibility while allowing for future optimizations using +native Azure tools (azcopy, azure-storage-blob SDK). +""" + +from typing import List, Optional, Union + +from ray.llm._internal.common.utils.cloud_filesystem.base import BaseCloudFileSystem +from ray.llm._internal.common.utils.cloud_filesystem.pyarrow_filesystem import ( + PyArrowFileSystem, +) + + +class AzureFileSystem(BaseCloudFileSystem): + """Azure-specific implementation of cloud filesystem operations. + + **Note**: This implementation currently delegates to PyArrowFileSystem to maintain + stability. Optimized implementation using azure-storage-blob SDK and azcopy + will be added in a future PR. + """ + + @staticmethod + def get_file( + object_uri: str, decode_as_utf_8: bool = True + ) -> Optional[Union[str, bytes]]: + """Download a file from cloud storage into memory. + + Args: + object_uri: URI of the file (abfss:// or azure://) + decode_as_utf_8: If True, decode the file as UTF-8 + + Returns: + File contents as string or bytes, or None if file doesn't exist + """ + return PyArrowFileSystem.get_file(object_uri, decode_as_utf_8) + + @staticmethod + def list_subfolders(folder_uri: str) -> List[str]: + """List the immediate subfolders in a cloud directory. + + Args: + folder_uri: URI of the directory (abfss:// or azure://) + + Returns: + List of subfolder names (without trailing slashes) + """ + return PyArrowFileSystem.list_subfolders(folder_uri) + + @staticmethod + def download_files( + path: str, + bucket_uri: str, + substrings_to_include: Optional[List[str]] = None, + suffixes_to_exclude: Optional[List[str]] = None, + ) -> None: + """Download files from cloud storage to a local directory. + + Args: + path: Local directory where files will be downloaded + bucket_uri: URI of cloud directory + substrings_to_include: Only include files containing these substrings + suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + """ + PyArrowFileSystem.download_files( + path, bucket_uri, substrings_to_include, suffixes_to_exclude + ) + + @staticmethod + def upload_files( + local_path: str, + bucket_uri: str, + ) -> None: + """Upload files to cloud storage. + + Args: + local_path: The local path of the files to upload. + bucket_uri: The bucket uri to upload the files to, must start with + `abfss://` or `azure://`. + """ + PyArrowFileSystem.upload_files(local_path, bucket_uri) diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/base.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/base.py new file mode 100644 index 000000000000..ea8f3999b69b --- /dev/null +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/base.py @@ -0,0 +1,81 @@ +"""Abstract base class for cloud filesystem implementations. + +This module defines the interface that all cloud storage provider implementations +must follow, ensuring consistency across different providers while allowing +provider-specific optimizations. +""" + +from abc import ABC, abstractmethod +from typing import List, Optional, Union + + +class BaseCloudFileSystem(ABC): + """Abstract base class for cloud filesystem implementations. + + This class defines the interface that all cloud storage provider implementations + must implement. Provider-specific classes (S3FileSystem, GCSFileSystem, etc.) + will inherit from this base class and provide optimized implementations for + their respective cloud storage platforms. + """ + + @staticmethod + @abstractmethod + def get_file( + object_uri: str, decode_as_utf_8: bool = True + ) -> Optional[Union[str, bytes]]: + """Download a file from cloud storage into memory. + + Args: + object_uri: URI of the file (s3://, gs://, abfss://, or azure://) + decode_as_utf_8: If True, decode the file as UTF-8 + + Returns: + File contents as string or bytes, or None if file doesn't exist + """ + pass + + @staticmethod + @abstractmethod + def list_subfolders(folder_uri: str) -> List[str]: + """List the immediate subfolders in a cloud directory. + + Args: + folder_uri: URI of the directory (s3://, gs://, abfss://, or azure://) + + Returns: + List of subfolder names (without trailing slashes) + """ + pass + + @staticmethod + @abstractmethod + def download_files( + path: str, + bucket_uri: str, + substrings_to_include: Optional[List[str]] = None, + suffixes_to_exclude: Optional[List[str]] = None, + ) -> None: + """Download files from cloud storage to a local directory. + + Args: + path: Local directory where files will be downloaded + bucket_uri: URI of cloud directory + substrings_to_include: Only include files containing these substrings + suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + """ + pass + + @staticmethod + @abstractmethod + def upload_files( + local_path: str, + bucket_uri: str, + ) -> None: + """Upload files to cloud storage. + + Args: + local_path: The local path of the files to upload. + bucket_uri: The bucket uri to upload the files to, must start with + `s3://`, `gs://`, `abfss://`, or `azure://`. + """ + pass diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/gcs_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/gcs_filesystem.py new file mode 100644 index 000000000000..663d5e10a3ad --- /dev/null +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/gcs_filesystem.py @@ -0,0 +1,81 @@ +"""GCS-specific filesystem implementation. + +This module provides a GCS-specific implementation. +This maintains backward compatibility while allowing for future optimizations using +native GCS tools (gsutil, google-cloud-storage SDK). +""" + +from typing import List, Optional, Union + +from ray.llm._internal.common.utils.cloud_filesystem.base import BaseCloudFileSystem +from ray.llm._internal.common.utils.cloud_filesystem.pyarrow_filesystem import ( + PyArrowFileSystem, +) + + +class GCSFileSystem(BaseCloudFileSystem): + """GCS-specific implementation of cloud filesystem operations. + + **Note**: This implementation currently delegates to PyArrowFileSystem to maintain + stability. Optimized implementation using google-cloud-storage SDK and gsutil + will be added in a future PR. + """ + + @staticmethod + def get_file( + object_uri: str, decode_as_utf_8: bool = True + ) -> Optional[Union[str, bytes]]: + """Download a file from cloud storage into memory. + + Args: + object_uri: URI of the file (gs://) + decode_as_utf_8: If True, decode the file as UTF-8 + + Returns: + File contents as string or bytes, or None if file doesn't exist + """ + return PyArrowFileSystem.get_file(object_uri, decode_as_utf_8) + + @staticmethod + def list_subfolders(folder_uri: str) -> List[str]: + """List the immediate subfolders in a cloud directory. + + Args: + folder_uri: URI of the directory (gs://) + + Returns: + List of subfolder names (without trailing slashes) + """ + return PyArrowFileSystem.list_subfolders(folder_uri) + + @staticmethod + def download_files( + path: str, + bucket_uri: str, + substrings_to_include: Optional[List[str]] = None, + suffixes_to_exclude: Optional[List[str]] = None, + ) -> None: + """Download files from cloud storage to a local directory. + + Args: + path: Local directory where files will be downloaded + bucket_uri: URI of cloud directory + substrings_to_include: Only include files containing these substrings + suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + """ + PyArrowFileSystem.download_files( + path, bucket_uri, substrings_to_include, suffixes_to_exclude + ) + + @staticmethod + def upload_files( + local_path: str, + bucket_uri: str, + ) -> None: + """Upload files to cloud storage. + + Args: + local_path: The local path of the files to upload. + bucket_uri: The bucket uri to upload the files to, must start with `gs://`. + """ + PyArrowFileSystem.upload_files(local_path, bucket_uri) diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py new file mode 100644 index 000000000000..28c3644766f2 --- /dev/null +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py @@ -0,0 +1,344 @@ +"""PyArrow-based filesystem implementation for cloud storage. + +This module provides a PyArrow-based implementation of the cloud filesystem +interface, supporting S3, GCS, and Azure storage providers. +""" + +import os +from typing import List, Optional, Tuple, Union +from urllib.parse import urlparse + +import pyarrow.fs as pa_fs + +from ray.llm._internal.common.observability.logging import get_logger +from ray.llm._internal.common.utils.cloud_filesystem.base import BaseCloudFileSystem + +logger = get_logger(__name__) + + +class PyArrowFileSystem(BaseCloudFileSystem): + """PyArrow-based implementation of cloud filesystem operations. + + This class provides a unified interface for cloud storage operations using + PyArrow's filesystem abstraction. It supports S3, GCS, and Azure storage + providers. + """ + + @staticmethod + def get_fs_and_path(object_uri: str) -> Tuple[pa_fs.FileSystem, str]: + """Get the appropriate filesystem and path from a URI. + + Args: + object_uri: URI of the file (s3://, gs://, abfss://, or azure://) + If URI contains 'anonymous@', anonymous access is used. + Example: s3://anonymous@bucket/path + + Returns: + Tuple of (filesystem, path) + """ + anonymous = False + # Check for anonymous access pattern (only for S3/GCS) + # e.g. s3://anonymous@bucket/path + if "@" in object_uri and not ( + object_uri.startswith("abfss://") or object_uri.startswith("azure://") + ): + parts = object_uri.split("@", 1) + # Check if the first part ends with "anonymous" + if parts[0].endswith("anonymous"): + anonymous = True + # Remove the anonymous@ part, keeping the scheme + scheme = parts[0].split("://")[0] + object_uri = f"{scheme}://{parts[1]}" + + if object_uri.startswith("s3://"): + endpoint = os.getenv("AWS_ENDPOINT_URL_S3", None) + virtual_hosted_style = os.getenv("AWS_S3_ADDRESSING_STYLE", None) + fs = pa_fs.S3FileSystem( + anonymous=anonymous, + endpoint_override=endpoint, + force_virtual_addressing=(virtual_hosted_style == "virtual"), + ) + path = object_uri[5:] # Remove "s3://" + elif object_uri.startswith("gs://"): + fs = pa_fs.GcsFileSystem(anonymous=anonymous) + path = object_uri[5:] # Remove "gs://" + elif object_uri.startswith("abfss://"): + fs, path = PyArrowFileSystem._create_abfss_filesystem(object_uri) + elif object_uri.startswith("azure://"): + fs, path = PyArrowFileSystem._create_azure_filesystem(object_uri) + else: + raise ValueError(f"Unsupported URI scheme: {object_uri}") + + return fs, path + + @staticmethod + def _create_azure_filesystem(object_uri: str) -> Tuple[pa_fs.FileSystem, str]: + """Create an Azure filesystem for Azure Blob Storage or ABFSS. + + Args: + object_uri: Azure URI (azure://container@account.blob.core.windows.net/path or + abfss://container@account.dfs.core.windows.net/path) + + Returns: + Tuple of (PyArrow FileSystem, path without scheme prefix) + + Raises: + ImportError: If required dependencies are not installed. + ValueError: If the Azure URI format is invalid. + """ + try: + import adlfs + from azure.identity import DefaultAzureCredential + except ImportError: + raise ImportError( + "You must `pip install adlfs azure-identity` " + "to use Azure/ABFSS URIs. " + "Note that these must be preinstalled on all nodes in the Ray cluster." + ) + + # Parse and validate the Azure URI + parsed = urlparse(object_uri) + scheme = parsed.scheme.lower() + + # Validate URI format: scheme://container@account.domain/path + if not parsed.netloc or "@" not in parsed.netloc: + raise ValueError( + f"Invalid {scheme.upper()} URI format - missing container@account: {object_uri}" + ) + + container_part, hostname_part = parsed.netloc.split("@", 1) + + # Validate container name (must be non-empty) + if not container_part: + raise ValueError( + f"Invalid {scheme.upper()} URI format - empty container name: {object_uri}" + ) + + # Validate hostname format based on scheme + valid_hostname = False + if scheme == "abfss": + valid_hostname = hostname_part.endswith(".dfs.core.windows.net") + expected_domains = ".dfs.core.windows.net" + elif scheme == "azure": + valid_hostname = hostname_part.endswith( + ".blob.core.windows.net" + ) or hostname_part.endswith(".dfs.core.windows.net") + expected_domains = ".blob.core.windows.net or .dfs.core.windows.net" + + if not hostname_part or not valid_hostname: + raise ValueError( + f"Invalid {scheme.upper()} URI format - invalid hostname (must end with {expected_domains}): {object_uri}" + ) + + # Extract and validate account name + azure_storage_account_name = hostname_part.split(".")[0] + if not azure_storage_account_name: + raise ValueError( + f"Invalid {scheme.upper()} URI format - empty account name: {object_uri}" + ) + + # Create the adlfs filesystem + adlfs_fs = adlfs.AzureBlobFileSystem( + account_name=azure_storage_account_name, + credential=DefaultAzureCredential(), + ) + + # Wrap with PyArrow's PyFileSystem for compatibility + fs = pa_fs.PyFileSystem(pa_fs.FSSpecHandler(adlfs_fs)) + + # Return the path without the scheme prefix + path = f"{container_part}{parsed.path}" + + return fs, path + + @staticmethod + def _create_abfss_filesystem(object_uri: str) -> Tuple[pa_fs.FileSystem, str]: + """Create an ABFSS filesystem for Azure Data Lake Storage Gen2. + + This is a wrapper around _create_azure_filesystem for backward compatibility. + + Args: + object_uri: ABFSS URI (abfss://container@account.dfs.core.windows.net/path) + + Returns: + Tuple of (PyArrow FileSystem, path without abfss:// prefix) + """ + return PyArrowFileSystem._create_azure_filesystem(object_uri) + + @staticmethod + def _filter_files( + fs: pa_fs.FileSystem, + source_path: str, + destination_path: str, + substrings_to_include: Optional[List[str]] = None, + suffixes_to_exclude: Optional[List[str]] = None, + ) -> List[Tuple[str, str]]: + """Filter files from cloud storage based on inclusion and exclusion criteria. + + Args: + fs: PyArrow filesystem instance + source_path: Source path in cloud storage + destination_path: Local destination path + substrings_to_include: Only include files containing these substrings + suffixes_to_exclude: Exclude files ending with these suffixes + + Returns: + List of tuples containing (source_file_path, destination_file_path) + """ + file_selector = pa_fs.FileSelector(source_path, recursive=True) + file_infos = fs.get_file_info(file_selector) + + path_pairs = [] + for file_info in file_infos: + if file_info.type != pa_fs.FileType.File: + continue + + rel_path = file_info.path[len(source_path) :].lstrip("/") + + # Apply filters + if substrings_to_include: + if not any( + substring in rel_path for substring in substrings_to_include + ): + continue + + if suffixes_to_exclude: + if any(rel_path.endswith(suffix) for suffix in suffixes_to_exclude): + continue + + path_pairs.append( + (file_info.path, os.path.join(destination_path, rel_path)) + ) + + return path_pairs + + @staticmethod + def get_file( + object_uri: str, decode_as_utf_8: bool = True + ) -> Optional[Union[str, bytes]]: + """Download a file from cloud storage into memory. + + Args: + object_uri: URI of the file (s3://, gs://, abfss://, or azure://) + decode_as_utf_8: If True, decode the file as UTF-8 + + Returns: + File contents as string or bytes, or None if file doesn't exist + """ + try: + fs, path = PyArrowFileSystem.get_fs_and_path(object_uri) + + # Check if file exists + if not fs.get_file_info(path).type == pa_fs.FileType.File: + logger.info(f"URI {object_uri} does not exist.") + return None + + # Read file + with fs.open_input_file(path) as f: + body = f.read() + + if decode_as_utf_8: + body = body.decode("utf-8") + return body + except Exception as e: + logger.info(f"Error reading {object_uri}: {e}") + return None + + @staticmethod + def list_subfolders(folder_uri: str) -> List[str]: + """List the immediate subfolders in a cloud directory. + + Args: + folder_uri: URI of the directory (s3://, gs://, abfss://, or azure://) + + Returns: + List of subfolder names (without trailing slashes) + """ + # Ensure that the folder_uri has a trailing slash. + folder_uri = f"{folder_uri.rstrip('/')}/" + + try: + fs, path = PyArrowFileSystem.get_fs_and_path(folder_uri) + + # List directory contents + file_infos = fs.get_file_info(pa_fs.FileSelector(path, recursive=False)) + + # Filter for directories and extract subfolder names + subfolders = [] + for file_info in file_infos: + if file_info.type == pa_fs.FileType.Directory: + # Extract just the subfolder name without the full path + subfolder = os.path.basename(file_info.path.rstrip("/")) + subfolders.append(subfolder) + + return subfolders + except Exception as e: + logger.info(f"Error listing subfolders in {folder_uri}: {e}") + return [] + + @staticmethod + def download_files( + path: str, + bucket_uri: str, + substrings_to_include: Optional[List[str]] = None, + suffixes_to_exclude: Optional[List[str]] = None, + ) -> None: + """Download files from cloud storage to a local directory. + + Args: + path: Local directory where files will be downloaded + bucket_uri: URI of cloud directory + substrings_to_include: Only include files containing these substrings + suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + """ + try: + fs, source_path = PyArrowFileSystem.get_fs_and_path(bucket_uri) + + # Ensure the destination directory exists + os.makedirs(path, exist_ok=True) + + # Get filtered files to download + files_to_download = PyArrowFileSystem._filter_files( + fs, source_path, path, substrings_to_include, suffixes_to_exclude + ) + + # Download each file + for source_file_path, dest_file_path in files_to_download: + # Create destination directory if needed + dest_dir = os.path.dirname(dest_file_path) + if dest_dir: + os.makedirs(dest_dir, exist_ok=True) + + # Download the file + with fs.open_input_file(source_file_path) as source_file: + with open(dest_file_path, "wb") as dest_file: + dest_file.write(source_file.read()) + + except Exception as e: + logger.exception(f"Error downloading files from {bucket_uri}: {e}") + raise + + @staticmethod + def upload_files( + local_path: str, + bucket_uri: str, + ) -> None: + """Upload files to cloud storage. + + Args: + local_path: The local path of the files to upload. + bucket_uri: The bucket uri to upload the files to, must start with + `s3://`, `gs://`, `abfss://`, or `azure://`. + """ + try: + fs, dest_path = PyArrowFileSystem.get_fs_and_path(bucket_uri) + + pa_fs.copy_files( + source=local_path, + destination=dest_path, + source_filesystem=pa_fs.LocalFileSystem(), + destination_filesystem=fs, + ) + except Exception as e: + logger.exception(f"Error uploading files to {bucket_uri}: {e}") + raise diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py new file mode 100644 index 000000000000..b7724d0846b8 --- /dev/null +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py @@ -0,0 +1,233 @@ +"""S3-specific filesystem implementation using AWS CLI. + +This module provides an S3-specific implementation that uses AWS CLI for optimal +performance. This leverages native AWS tools for significantly faster transfers +compared to PyArrow-based implementations. +""" + +import os +import subprocess +import tempfile +from typing import List, Optional, Union + +from ray.llm._internal.common.observability.logging import get_logger +from ray.llm._internal.common.utils.cloud_filesystem.base import BaseCloudFileSystem + +logger = get_logger(__name__) + + +class S3FileSystem(BaseCloudFileSystem): + """S3-specific implementation of cloud filesystem operations using AWS CLI. + + This implementation uses AWS CLI (aws s3 cp, aws s3 ls) for optimal performance + when working with S3 storage. It provides significantly faster transfers + compared to PyArrow-based implementations, especially for large files. + """ + + @staticmethod + def _run_aws_command(cmd: List[str]) -> subprocess.CompletedProcess: + """Run an AWS CLI command and handle errors. + + Args: + cmd: List of command arguments (e.g., ['aws', 's3', 'cp', ...]) + + Returns: + CompletedProcess object from subprocess.run + + Raises: + subprocess.CalledProcessError: If the command fails + FileNotFoundError: If AWS CLI is not installed + """ + try: + result = subprocess.run( + cmd, + capture_output=True, + text=True, + check=True, + ) + return result + except FileNotFoundError: + raise FileNotFoundError( + "AWS CLI is not installed. Please install it using: " + "pip install awscli. For more info visit https://aws.amazon.com/cli/" + ) + except subprocess.CalledProcessError as e: + logger.error(f"AWS CLI command failed: {' '.join(cmd)}") + logger.error(f"Error output: {e.stderr}") + raise + + @staticmethod + def get_file( + object_uri: str, decode_as_utf_8: bool = True + ) -> Optional[Union[str, bytes]]: + """Download a file from cloud storage into memory. + + Args: + object_uri: URI of the file (s3://) + decode_as_utf_8: If True, decode the file as UTF-8 + + Returns: + File contents as string or bytes, or None if file doesn't exist + """ + if not object_uri.startswith("s3://"): + raise ValueError(f"Invalid S3 URI: {object_uri}") + + try: + # Create a temporary file to download to + with tempfile.NamedTemporaryFile(delete=False) as tmp_file: + tmp_path = tmp_file.name + + try: + # Download file using AWS CLI + cmd = ["aws", "s3", "cp", object_uri, tmp_path] + S3FileSystem._run_aws_command(cmd) + + # Read the file + mode = "r" if decode_as_utf_8 else "rb" + with open(tmp_path, mode) as f: + body = f.read() + + return body + finally: + # Clean up temporary file + if os.path.exists(tmp_path): + os.unlink(tmp_path) + + except subprocess.CalledProcessError as e: + # Check if file doesn't exist (AWS CLI returns non-zero exit code) + if "NoSuchKey" in e.stderr or "does not exist" in e.stderr.lower(): + logger.info(f"URI {object_uri} does not exist.") + return None + logger.info(f"Error reading {object_uri}: {e.stderr}") + return None + except Exception as e: + logger.info(f"Error reading {object_uri}: {e}") + return None + + @staticmethod + def list_subfolders(folder_uri: str) -> List[str]: + """List the immediate subfolders in a cloud directory. + + Args: + folder_uri: URI of the directory (s3://) + + Returns: + List of subfolder names (without trailing slashes) + """ + if not folder_uri.startswith("s3://"): + raise ValueError(f"Invalid S3 URI: {folder_uri}") + + # Ensure that the folder_uri has a trailing slash. + folder_uri = f"{folder_uri.rstrip('/')}/" + + try: + # Use AWS CLI to list objects with common prefix + cmd = ["aws", "s3", "ls", folder_uri] + result = S3FileSystem._run_aws_command(cmd) + + subfolders = [] + for line in result.stdout.strip().split("\n"): + if not line.strip(): + continue + # AWS CLI ls output format: "PRE folder_name/" or "timestamp size key" + # We're looking for lines starting with "PRE" (prefixes/directories) + if line.startswith("PRE"): + # Extract folder name: "PRE folder_name/" -> "folder_name" + folder_name = line.split()[-1].rstrip("/") + subfolders.append(folder_name) + + return subfolders + except Exception as e: + logger.info(f"Error listing subfolders in {folder_uri}: {e}") + return [] + + @staticmethod + def download_files( + path: str, + bucket_uri: str, + substrings_to_include: Optional[List[str]] = None, + suffixes_to_exclude: Optional[List[str]] = None, + ) -> None: + """Download files from cloud storage to a local directory. + + Args: + path: Local directory where files will be downloaded + bucket_uri: URI of cloud directory + substrings_to_include: Only include files containing these substrings + suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + """ + if not bucket_uri.startswith("s3://"): + raise ValueError(f"Invalid S3 URI: {bucket_uri}") + + try: + # Ensure the destination directory exists + os.makedirs(path, exist_ok=True) + + # Ensure bucket_uri has trailing slash for directory listing + source_uri = f"{bucket_uri.rstrip('/')}/" + + # Build AWS CLI command + cmd = ["aws", "s3", "cp", source_uri, path, "--recursive"] + + # AWS CLI filter logic: + # - By default, all files are included + # - --exclude removes files matching the pattern + # - --include adds files matching the pattern (even if excluded) + # - Order matters: filters are processed sequentially + + # If we have include filters, we need to exclude everything first, + # then include only what we want + if substrings_to_include: + # Exclude everything first + cmd.extend(["--exclude", "*"]) + # Then include files matching any of the substring patterns + for substring in substrings_to_include: + # Create wildcard pattern: *substring* matches files containing substring + pattern = f"*{substring}*" + cmd.extend(["--include", pattern]) + + # Add exclude filters (suffixes_to_exclude) + # These will exclude files ending with the specified suffixes + if suffixes_to_exclude: + for suffix in suffixes_to_exclude: + # Ensure suffix starts with * if it doesn't already + if not suffix.startswith("*"): + pattern = f"*{suffix}" + else: + pattern = suffix + cmd.extend(["--exclude", pattern]) + + # Run the download command + S3FileSystem._run_aws_command(cmd) + + except Exception as e: + logger.exception(f"Error downloading files from {bucket_uri}: {e}") + raise + + @staticmethod + def upload_files( + local_path: str, + bucket_uri: str, + ) -> None: + """Upload files to cloud storage. + + Args: + local_path: The local path of the files to upload. + bucket_uri: The bucket uri to upload the files to, must start with `s3://`. + """ + if not bucket_uri.startswith("s3://"): + raise ValueError(f"Invalid S3 URI: {bucket_uri}") + + try: + # Ensure bucket_uri has trailing slash for directory upload + dest_uri = f"{bucket_uri.rstrip('/')}/" + + # Build AWS CLI command for recursive upload + cmd = ["aws", "s3", "cp", local_path, dest_uri, "--recursive"] + + # Run the upload command + S3FileSystem._run_aws_command(cmd) + + except Exception as e: + logger.exception(f"Error uploading files to {bucket_uri}: {e}") + raise diff --git a/python/ray/llm/_internal/common/utils/cloud_utils.py b/python/ray/llm/_internal/common/utils/cloud_utils.py index 074f69539e85..df3a3acb9716 100644 --- a/python/ray/llm/_internal/common/utils/cloud_utils.py +++ b/python/ray/llm/_internal/common/utils/cloud_utils.py @@ -2,7 +2,6 @@ import inspect import os import time -from concurrent.futures import ThreadPoolExecutor from pathlib import Path from typing import ( Any, @@ -12,17 +11,19 @@ List, NamedTuple, Optional, - Tuple, TypeVar, Union, ) -# Use pyarrow for cloud storage access -import pyarrow.fs as pa_fs from pydantic import Field, field_validator from ray.llm._internal.common.base_pydantic import BaseModelExtended from ray.llm._internal.common.observability.logging import get_logger +from ray.llm._internal.common.utils.cloud_filesystem import ( + AzureFileSystem, + GCSFileSystem, + S3FileSystem, +) T = TypeVar("T") @@ -133,154 +134,35 @@ def bucket_path(self) -> str: class CloudFileSystem: - """A unified interface for cloud file system operations using PyArrow. + """A unified interface for cloud file system operations. This class provides a simple interface for common operations on cloud storage - systems (S3, GCS, Azure) using PyArrow's filesystem interface. + systems (S3, GCS, Azure) by delegating to provider-specific implementations + for optimal performance. """ @staticmethod - def get_fs_and_path(object_uri: str) -> Tuple[pa_fs.FileSystem, str]: - """Get the appropriate filesystem and path from a URI. - - Args: - object_uri: URI of the file (s3://, gs://, abfss://, or azure://) - If URI contains 'anonymous@', anonymous access is used. - Example: s3://anonymous@bucket/path - - Returns: - Tuple of (filesystem, path) - """ - anonymous = False - # Check for anonymous access pattern (only for S3/GCS) - # e.g. s3://anonymous@bucket/path - if "@" in object_uri and not ( - object_uri.startswith("abfss://") or object_uri.startswith("azure://") - ): - parts = object_uri.split("@", 1) - # Check if the first part ends with "anonymous" - if parts[0].endswith("anonymous"): - anonymous = True - # Remove the anonymous@ part, keeping the scheme - scheme = parts[0].split("://")[0] - object_uri = f"{scheme}://{parts[1]}" - - if object_uri.startswith("s3://"): - endpoint = os.getenv("AWS_ENDPOINT_URL_S3", None) - virtual_hosted_style = os.getenv("AWS_S3_ADDRESSING_STYLE", None) - fs = pa_fs.S3FileSystem( - anonymous=anonymous, - endpoint_override=endpoint, - force_virtual_addressing=(virtual_hosted_style == "virtual"), - ) - path = object_uri[5:] # Remove "s3://" - elif object_uri.startswith("gs://"): - fs = pa_fs.GcsFileSystem(anonymous=anonymous) - path = object_uri[5:] # Remove "gs://" - elif object_uri.startswith("abfss://"): - fs, path = CloudFileSystem._create_abfss_filesystem(object_uri) - elif object_uri.startswith("azure://"): - fs, path = CloudFileSystem._create_azure_filesystem(object_uri) - else: - raise ValueError(f"Unsupported URI scheme: {object_uri}") - - return fs, path - - @staticmethod - def _create_azure_filesystem(object_uri: str) -> Tuple[pa_fs.FileSystem, str]: - """Create an Azure filesystem for Azure Blob Storage or ABFSS. + def _get_provider_fs(bucket_uri: str): + """Get the appropriate provider-specific filesystem class based on URI. Args: - object_uri: Azure URI (azure://container@account.blob.core.windows.net/path or - abfss://container@account.dfs.core.windows.net/path) + bucket_uri: URI of the cloud storage (s3://, gs://, abfss://, or azure://) Returns: - Tuple of (PyArrow FileSystem, path without scheme prefix) + The appropriate filesystem class (S3FileSystem, GCSFileSystem, or AzureFileSystem) Raises: - ImportError: If required dependencies are not installed. - ValueError: If the Azure URI format is invalid. + ValueError: If the URI scheme is not supported """ - try: - import adlfs - from azure.identity import DefaultAzureCredential - except ImportError: - raise ImportError( - "You must `pip install adlfs azure-identity` " - "to use Azure/ABFSS URIs. " - "Note that these must be preinstalled on all nodes in the Ray cluster." - ) - - from urllib.parse import urlparse - - # Parse and validate the Azure URI - parsed = urlparse(object_uri) - scheme = parsed.scheme.lower() - - # Validate URI format: scheme://container@account.domain/path - if not parsed.netloc or "@" not in parsed.netloc: - raise ValueError( - f"Invalid {scheme.upper()} URI format - missing container@account: {object_uri}" - ) - - container_part, hostname_part = parsed.netloc.split("@", 1) - - # Validate container name (must be non-empty) - if not container_part: - raise ValueError( - f"Invalid {scheme.upper()} URI format - empty container name: {object_uri}" - ) - - # Validate hostname format based on scheme - valid_hostname = False - if scheme == "abfss": - valid_hostname = hostname_part.endswith(".dfs.core.windows.net") - expected_domains = ".dfs.core.windows.net" - elif scheme == "azure": - valid_hostname = hostname_part.endswith( - ".blob.core.windows.net" - ) or hostname_part.endswith(".dfs.core.windows.net") - expected_domains = ".blob.core.windows.net or .dfs.core.windows.net" - - if not hostname_part or not valid_hostname: - raise ValueError( - f"Invalid {scheme.upper()} URI format - invalid hostname (must end with {expected_domains}): {object_uri}" - ) - - # Extract and validate account name - azure_storage_account_name = hostname_part.split(".")[0] - if not azure_storage_account_name: - raise ValueError( - f"Invalid {scheme.upper()} URI format - empty account name: {object_uri}" - ) - - # Create the adlfs filesystem - adlfs_fs = adlfs.AzureBlobFileSystem( - account_name=azure_storage_account_name, - credential=DefaultAzureCredential(), - ) - - # Wrap with PyArrow's PyFileSystem for compatibility - fs = pa_fs.PyFileSystem(pa_fs.FSSpecHandler(adlfs_fs)) - - # Return the path without the scheme prefix - path = f"{container_part}{parsed.path}" - - return fs, path - @staticmethod - def _create_abfss_filesystem(object_uri: str) -> Tuple[pa_fs.FileSystem, str]: - """Create an ABFSS filesystem for Azure Data Lake Storage Gen2. - - This is a wrapper around _create_azure_filesystem for backward compatibility. - - Args: - object_uri: ABFSS URI (abfss://container@account.dfs.core.windows.net/path) - - Returns: - Tuple of (PyArrow FileSystem, path without abfss:// prefix) - """ - return CloudFileSystem._create_azure_filesystem(object_uri) + if bucket_uri.startswith("s3://"): + return S3FileSystem + elif bucket_uri.startswith("gs://"): + return GCSFileSystem + elif bucket_uri.startswith(("abfss://", "azure://")): + return AzureFileSystem + else: + raise ValueError(f"Unsupported URI scheme: {bucket_uri}") @staticmethod def get_file( @@ -289,109 +171,27 @@ def get_file( """Download a file from cloud storage into memory. Args: - object_uri: URI of the file (s3:// or gs://) + object_uri: URI of the file (s3://, gs://, abfss://, or azure://) decode_as_utf_8: If True, decode the file as UTF-8 Returns: File contents as string or bytes, or None if file doesn't exist """ - try: - fs, path = CloudFileSystem.get_fs_and_path(object_uri) - - # Check if file exists - if not fs.get_file_info(path).type == pa_fs.FileType.File: - logger.info(f"URI {object_uri} does not exist.") - return None - - # Read file - with fs.open_input_file(path) as f: - body = f.read() - - if decode_as_utf_8: - body = body.decode("utf-8") - return body - except Exception as e: - logger.info(f"Error reading {object_uri}: {e}") - return None + fs_class = CloudFileSystem._get_provider_fs(object_uri) + return fs_class.get_file(object_uri, decode_as_utf_8) @staticmethod def list_subfolders(folder_uri: str) -> List[str]: """List the immediate subfolders in a cloud directory. Args: - folder_uri: URI of the directory (s3:// or gs://) + folder_uri: URI of the directory (s3://, gs://, abfss://, or azure://) Returns: List of subfolder names (without trailing slashes) """ - # Ensure that the folder_uri has a trailing slash. - folder_uri = f"{folder_uri.rstrip('/')}/" - - try: - fs, path = CloudFileSystem.get_fs_and_path(folder_uri) - - # List directory contents - file_infos = fs.get_file_info(pa_fs.FileSelector(path, recursive=False)) - - # Filter for directories and extract subfolder names - subfolders = [] - for file_info in file_infos: - if file_info.type == pa_fs.FileType.Directory: - # Extract just the subfolder name without the full path - subfolder = os.path.basename(file_info.path.rstrip("/")) - subfolders.append(subfolder) - - return subfolders - except Exception as e: - logger.info(f"Error listing subfolders in {folder_uri}: {e}") - return [] - - @staticmethod - def _filter_files( - fs: pa_fs.FileSystem, - source_path: str, - destination_path: str, - substrings_to_include: Optional[List[str]] = None, - suffixes_to_exclude: Optional[List[str]] = None, - ) -> List[Tuple[str, str]]: - """Filter files from cloud storage based on inclusion and exclusion criteria. - - Args: - fs: PyArrow filesystem instance - source_path: Source path in cloud storage - destination_path: Local destination path - substrings_to_include: Only include files containing these substrings - suffixes_to_exclude: Exclude files ending with these suffixes - - Returns: - List of tuples containing (source_file_path, destination_file_path) - """ - file_selector = pa_fs.FileSelector(source_path, recursive=True) - file_infos = fs.get_file_info(file_selector) - - path_pairs = [] - for file_info in file_infos: - if file_info.type != pa_fs.FileType.File: - continue - - rel_path = file_info.path[len(source_path) :].lstrip("/") - - # Apply filters - if substrings_to_include: - if not any( - substring in rel_path for substring in substrings_to_include - ): - continue - - if suffixes_to_exclude: - if any(rel_path.endswith(suffix) for suffix in suffixes_to_exclude): - continue - - path_pairs.append( - (file_info.path, os.path.join(destination_path, rel_path)) - ) - - return path_pairs + fs_class = CloudFileSystem._get_provider_fs(folder_uri) + return fs_class.list_subfolders(folder_uri) @staticmethod def download_files( @@ -408,114 +208,10 @@ def download_files( substrings_to_include: Only include files containing these substrings suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) """ - try: - fs, source_path = CloudFileSystem.get_fs_and_path(bucket_uri) - - # Ensure the destination directory exists - os.makedirs(path, exist_ok=True) - - # Get filtered files to download - files_to_download = CloudFileSystem._filter_files( - fs, source_path, path, substrings_to_include, suffixes_to_exclude - ) - - # Download each file - for source_file_path, dest_file_path in files_to_download: - # Create destination directory if needed - dest_dir = os.path.dirname(dest_file_path) - if dest_dir: - os.makedirs(dest_dir, exist_ok=True) - - # Download the file - with fs.open_input_file(source_file_path) as source_file: - with open(dest_file_path, "wb") as dest_file: - dest_file.write(source_file.read()) - - except Exception as e: - logger.exception(f"Error downloading files from {bucket_uri}: {e}") - raise - - @staticmethod - def download_files_parallel( - path: str, - bucket_uri: str, - substrings_to_include: Optional[List[str]] = None, - suffixes_to_exclude: Optional[List[str]] = None, - max_concurrency: int = 10, - chunk_size: int = 64 * 1024 * 1024, - ) -> None: - """Multi-threaded download of files from cloud storage. - - Args: - path: Local directory where files will be downloaded - bucket_uri: URI of cloud directory - substrings_to_include: Only include files containing these substrings - suffixes_to_exclude: Exclude certain files from download - max_concurrency: Maximum number of concurrent files to download (default: 10) - chunk_size: Size of transfer chunks (default: 64MB) - """ - try: - fs, source_path = CloudFileSystem.get_fs_and_path(bucket_uri) - - # Ensure destination exists - os.makedirs(path, exist_ok=True) - - # If no filters, use direct copy_files - if not substrings_to_include and not suffixes_to_exclude: - pa_fs.copy_files( - source=source_path, - destination=path, - source_filesystem=fs, - destination_filesystem=pa_fs.LocalFileSystem(), - use_threads=True, - chunk_size=chunk_size, - ) - return - - # List and filter files - files_to_download = CloudFileSystem._filter_files( - fs, source_path, path, substrings_to_include, suffixes_to_exclude - ) - - if not files_to_download: - logger.info("Filters do not match any of the files, skipping download") - return - - def download_single_file(file_paths): - source_file_path, dest_file_path = file_paths - # Create destination directory if needed - dest_dir = os.path.dirname(dest_file_path) - if dest_dir: - os.makedirs(dest_dir, exist_ok=True) - - # Use PyArrow's copy_files for individual files, - pa_fs.copy_files( - source=source_file_path, - destination=dest_file_path, - source_filesystem=fs, - destination_filesystem=pa_fs.LocalFileSystem(), - use_threads=True, - chunk_size=chunk_size, - ) - return dest_file_path - - max_workers = min(max_concurrency, len(files_to_download)) - with ThreadPoolExecutor(max_workers=max_workers) as executor: - futures = [ - executor.submit(download_single_file, file_paths) - for file_paths in files_to_download - ] - - for future in futures: - try: - future.result() - except Exception as e: - logger.error(f"Failed to download file: {e}") - raise - - except Exception as e: - logger.exception(f"Error downloading files from {bucket_uri}: {e}") - raise + fs_class = CloudFileSystem._get_provider_fs(bucket_uri) + fs_class.download_files( + path, bucket_uri, substrings_to_include, suffixes_to_exclude + ) @staticmethod def download_model( @@ -536,16 +232,17 @@ def download_model( exclude_safetensors: If True, skip download of safetensor files """ try: - fs, source_path = CloudFileSystem.get_fs_and_path(bucket_uri) + # Get the provider-specific filesystem + fs_class = CloudFileSystem._get_provider_fs(bucket_uri) + + # Construct hash file URI + hash_uri = bucket_uri.rstrip("/") + "/hash" - # Check for hash file - hash_path = os.path.join(source_path, "hash") - hash_info = fs.get_file_info(hash_path) + # Try to download and read hash file + hash_content = fs_class.get_file(hash_uri, decode_as_utf_8=True) - if hash_info.type == pa_fs.FileType.File: - # Download and read hash file - with fs.open_input_file(hash_path) as f: - f_hash = f.read().decode("utf-8").strip() + if hash_content is not None: + f_hash = hash_content.strip() logger.info( f"Detected hash file in bucket {bucket_uri}. " f"Using {f_hash} as the hash." @@ -576,12 +273,11 @@ def download_model( safetensors_to_exclude = [".safetensors"] if exclude_safetensors else None - CloudFileSystem.download_files_parallel( + CloudFileSystem.download_files( path=destination_dir, bucket_uri=bucket_uri, substrings_to_include=tokenizer_file_substrings, suffixes_to_exclude=safetensors_to_exclude, - chunk_size=64 * 1024 * 1024, # 64MB chunks for large model files ) except Exception as e: @@ -597,20 +293,11 @@ def upload_files( Args: local_path: The local path of the files to upload. - bucket_uri: The bucket uri to upload the files to, must start with `s3://` or `gs://`. + bucket_uri: The bucket uri to upload the files to, must start with + `s3://`, `gs://`, `abfss://`, or `azure://`. """ - try: - fs, dest_path = CloudFileSystem.get_fs_and_path(bucket_uri) - - pa_fs.copy_files( - source=local_path, - destination=dest_path, - source_filesystem=pa_fs.LocalFileSystem(), - destination_filesystem=fs, - ) - except Exception as e: - logger.exception(f"Error uploading files to {bucket_uri}: {e}") - raise + fs_class = CloudFileSystem._get_provider_fs(bucket_uri) + fs_class.upload_files(local_path, bucket_uri) @staticmethod def upload_model( diff --git a/python/ray/llm/tests/common/utils/test_cloud_utils.py b/python/ray/llm/tests/common/utils/test_cloud_utils.py index 850fc2360018..e0d73572da81 100644 --- a/python/ray/llm/tests/common/utils/test_cloud_utils.py +++ b/python/ray/llm/tests/common/utils/test_cloud_utils.py @@ -6,7 +6,6 @@ import pyarrow.fs as pa_fs import pytest -from pytest import raises from ray.llm._internal.common.utils.cloud_utils import ( CloudFileSystem, @@ -282,35 +281,6 @@ async def slow_fetch(key: str): class TestCloudFileSystem: """Tests for the CloudFileSystem class.""" - @patch("pyarrow.fs.S3FileSystem") - def test_get_fs_and_path_s3(self, mock_s3fs): - """Test getting S3 filesystem and path.""" - mock_fs = MagicMock() - mock_s3fs.return_value = mock_fs - - fs, path = CloudFileSystem.get_fs_and_path("s3://bucket/key") - - assert fs == mock_fs - assert path == "bucket/key" - mock_s3fs.assert_called_once() - - @patch("pyarrow.fs.GcsFileSystem") - def test_get_fs_and_path_gcs(self, mock_gcsfs): - """Test getting GCS filesystem and path.""" - mock_fs = MagicMock() - mock_gcsfs.return_value = mock_fs - - fs, path = CloudFileSystem.get_fs_and_path("gs://bucket/key") - - assert fs == mock_fs - assert path == "bucket/key" - mock_gcsfs.assert_called_once() - - def test_get_fs_and_path_unsupported(self): - """Test unsupported URI scheme.""" - with raises(ValueError, match="Unsupported URI scheme"): - CloudFileSystem.get_fs_and_path("file:///tmp/file") - @patch("pyarrow.fs.S3FileSystem") def test_get_file(self, mock_s3fs): """Test getting a file from cloud storage.""" From 142b36dfdf3998efc771539a6e0797af619265f5 Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Fri, 7 Nov 2025 15:41:15 -0800 Subject: [PATCH 2/8] tests refactor fixes Signed-off-by: ahao-anyscale --- .../cloud_filesystem/pyarrow_filesystem.py | 105 +- .../utils/cloud_filesystem/s3_filesystem.py | 23 +- python/ray/llm/tests/common/cloud/__init__.py | 1 + .../common/cloud/test_cloud_filesystem.py | 79 ++ .../tests/common/cloud/test_mirror_config.py | 158 +++ .../common/cloud/test_pyarrow_filesystem.py | 544 +++++++++ .../ray/llm/tests/common/cloud/test_utils.py | 315 +++++ .../tests/common/utils/test_cloud_utils.py | 1075 ----------------- 8 files changed, 1204 insertions(+), 1096 deletions(-) create mode 100644 python/ray/llm/tests/common/cloud/__init__.py create mode 100644 python/ray/llm/tests/common/cloud/test_cloud_filesystem.py create mode 100644 python/ray/llm/tests/common/cloud/test_mirror_config.py create mode 100644 python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py create mode 100644 python/ray/llm/tests/common/cloud/test_utils.py delete mode 100644 python/ray/llm/tests/common/utils/test_cloud_utils.py diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py index 28c3644766f2..d38a6ba96981 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py @@ -5,6 +5,7 @@ """ import os +from concurrent.futures import ThreadPoolExecutor from typing import List, Optional, Tuple, Union from urllib.parse import urlparse @@ -276,12 +277,60 @@ def list_subfolders(folder_uri: str) -> List[str]: logger.info(f"Error listing subfolders in {folder_uri}: {e}") return [] + # @staticmethod + # def download_files( + # path: str, + # bucket_uri: str, + # substrings_to_include: Optional[List[str]] = None, + # suffixes_to_exclude: Optional[List[str]] = None, + # ) -> None: + # """Download files from cloud storage to a local directory. + + # Args: + # path: Local directory where files will be downloaded + # bucket_uri: URI of cloud directory + # substrings_to_include: Only include files containing these substrings + # suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + # """ + # try: + # fs, source_path = PyArrowFileSystem.get_fs_and_path(bucket_uri) + + # # Ensure the destination directory exists + # os.makedirs(path, exist_ok=True) + + # # Get filtered files to download + # files_to_download = PyArrowFileSystem._filter_files( + # fs, source_path, path, substrings_to_include, suffixes_to_exclude + # ) + + # # Download each file using PyArrow's copy_files API + # local_fs = pa_fs.LocalFileSystem() + # for source_file_path, dest_file_path in files_to_download: + # # Create destination directory if needed + # dest_dir = os.path.dirname(dest_file_path) + # if dest_dir: + # os.makedirs(dest_dir, exist_ok=True) + + # # Download the file using PyArrow's copy_files API + # pa_fs.copy_files( + # source=source_file_path, + # destination=dest_file_path, + # source_filesystem=fs, + # destination_filesystem=local_fs, + # ) + + # except Exception as e: + # logger.exception(f"Error downloading files from {bucket_uri}: {e}") + # raise + @staticmethod def download_files( path: str, bucket_uri: str, substrings_to_include: Optional[List[str]] = None, suffixes_to_exclude: Optional[List[str]] = None, + max_concurrency: int = 10, + chunk_size: int = 64 * 1024 * 1024, ) -> None: """Download files from cloud storage to a local directory. @@ -289,30 +338,68 @@ def download_files( path: Local directory where files will be downloaded bucket_uri: URI of cloud directory substrings_to_include: Only include files containing these substrings - suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + suffixes_to_exclude: Exclude certain files from download + max_concurrency: Maximum number of concurrent files to download (default: 10) + chunk_size: Size of transfer chunks (default: 64MB) """ try: fs, source_path = PyArrowFileSystem.get_fs_and_path(bucket_uri) - # Ensure the destination directory exists + # Ensure destination exists os.makedirs(path, exist_ok=True) - # Get filtered files to download + # If no filters, use direct copy_files + if not substrings_to_include and not suffixes_to_exclude: + pa_fs.copy_files( + source=source_path, + destination=path, + source_filesystem=fs, + destination_filesystem=pa_fs.LocalFileSystem(), + use_threads=True, + chunk_size=chunk_size, + ) + return + + # List and filter files files_to_download = PyArrowFileSystem._filter_files( fs, source_path, path, substrings_to_include, suffixes_to_exclude ) - # Download each file - for source_file_path, dest_file_path in files_to_download: + if not files_to_download: + logger.info("Filters do not match any of the files, skipping download") + return + + def download_single_file(file_paths): + source_file_path, dest_file_path = file_paths # Create destination directory if needed dest_dir = os.path.dirname(dest_file_path) if dest_dir: os.makedirs(dest_dir, exist_ok=True) - # Download the file - with fs.open_input_file(source_file_path) as source_file: - with open(dest_file_path, "wb") as dest_file: - dest_file.write(source_file.read()) + # Use PyArrow's copy_files for individual files, + pa_fs.copy_files( + source=source_file_path, + destination=dest_file_path, + source_filesystem=fs, + destination_filesystem=pa_fs.LocalFileSystem(), + use_threads=True, + chunk_size=chunk_size, + ) + return dest_file_path + + max_workers = min(max_concurrency, len(files_to_download)) + with ThreadPoolExecutor(max_workers=max_workers) as executor: + futures = [ + executor.submit(download_single_file, file_paths) + for file_paths in files_to_download + ] + + for future in futures: + try: + future.result() + except Exception as e: + logger.error(f"Failed to download file: {e}") + raise except Exception as e: logger.exception(f"Error downloading files from {bucket_uri}: {e}") diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py index b7724d0846b8..b1fe4c7bdca6 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py @@ -25,8 +25,8 @@ class S3FileSystem(BaseCloudFileSystem): """ @staticmethod - def _run_aws_command(cmd: List[str]) -> subprocess.CompletedProcess: - """Run an AWS CLI command and handle errors. + def _run_command(cmd: List[str]) -> subprocess.CompletedProcess: + """Run a command and handle errors. Args: cmd: List of command arguments (e.g., ['aws', 's3', 'cp', ...]) @@ -36,7 +36,7 @@ def _run_aws_command(cmd: List[str]) -> subprocess.CompletedProcess: Raises: subprocess.CalledProcessError: If the command fails - FileNotFoundError: If AWS CLI is not installed + FileNotFoundError: If the command is not installed """ try: result = subprocess.run( @@ -47,12 +47,11 @@ def _run_aws_command(cmd: List[str]) -> subprocess.CompletedProcess: ) return result except FileNotFoundError: - raise FileNotFoundError( - "AWS CLI is not installed. Please install it using: " - "pip install awscli. For more info visit https://aws.amazon.com/cli/" - ) + raise FileNotFoundError(f"Command '{cmd[0]}' is not installed.") except subprocess.CalledProcessError as e: - logger.error(f"AWS CLI command failed: {' '.join(cmd)}") + print(f"Command failed: {' '.join(cmd)}") + print(f"Error: {e.stderr}") + logger.error(f"Command failed: {' '.join(cmd)}") logger.error(f"Error output: {e.stderr}") raise @@ -80,7 +79,7 @@ def get_file( try: # Download file using AWS CLI cmd = ["aws", "s3", "cp", object_uri, tmp_path] - S3FileSystem._run_aws_command(cmd) + S3FileSystem._run_command(cmd) # Read the file mode = "r" if decode_as_utf_8 else "rb" @@ -123,7 +122,7 @@ def list_subfolders(folder_uri: str) -> List[str]: try: # Use AWS CLI to list objects with common prefix cmd = ["aws", "s3", "ls", folder_uri] - result = S3FileSystem._run_aws_command(cmd) + result = S3FileSystem._run_command(cmd) subfolders = [] for line in result.stdout.strip().split("\n"): @@ -198,7 +197,7 @@ def download_files( cmd.extend(["--exclude", pattern]) # Run the download command - S3FileSystem._run_aws_command(cmd) + S3FileSystem._run_command(cmd) except Exception as e: logger.exception(f"Error downloading files from {bucket_uri}: {e}") @@ -226,7 +225,7 @@ def upload_files( cmd = ["aws", "s3", "cp", local_path, dest_uri, "--recursive"] # Run the upload command - S3FileSystem._run_aws_command(cmd) + S3FileSystem._run_command(cmd) except Exception as e: logger.exception(f"Error uploading files to {bucket_uri}: {e}") diff --git a/python/ray/llm/tests/common/cloud/__init__.py b/python/ray/llm/tests/common/cloud/__init__.py new file mode 100644 index 000000000000..b189f6d32c9a --- /dev/null +++ b/python/ray/llm/tests/common/cloud/__init__.py @@ -0,0 +1 @@ +"""Tests for cloud filesystem functionality.""" diff --git a/python/ray/llm/tests/common/cloud/test_cloud_filesystem.py b/python/ray/llm/tests/common/cloud/test_cloud_filesystem.py new file mode 100644 index 000000000000..850e569d56f9 --- /dev/null +++ b/python/ray/llm/tests/common/cloud/test_cloud_filesystem.py @@ -0,0 +1,79 @@ +"""Tests for CloudFileSystem class.""" + +import os +import tempfile +from unittest.mock import patch + +from ray.llm._internal.common.utils.cloud_utils import CloudFileSystem + + +class TestCloudFileSystem: + """Tests for the CloudFileSystem class.""" + + @patch("ray.llm._internal.common.utils.cloud_utils.GCSFileSystem") + def test_download_model(self, mock_gcs_filesystem): + """Test downloading a model from cloud storage.""" + # Mock GCSFileSystem.get_file to return hash content + mock_gcs_filesystem.get_file.return_value = "abcdef1234567890" + + # Create temp directory for testing + with tempfile.TemporaryDirectory() as tempdir: + # Test downloading model + with patch.object(CloudFileSystem, "download_files") as mock_download: + CloudFileSystem.download_model(tempdir, "gs://bucket/model", False) + + # Check that hash file was processed + assert os.path.exists(os.path.join(tempdir, "refs", "main")) + with open(os.path.join(tempdir, "refs", "main"), "r") as f: + assert f.read() == "abcdef1234567890" + + # Verify get_file was called for hash file + mock_gcs_filesystem.get_file.assert_called_once_with( + "gs://bucket/model/hash", decode_as_utf_8=True + ) + + # Check that download_files was called correctly + mock_download.assert_called_once() + call_args = mock_download.call_args[1] + assert call_args["path"] == os.path.join( + tempdir, "snapshots", "abcdef1234567890" + ) + assert call_args["bucket_uri"] == "gs://bucket/model" + assert call_args["substrings_to_include"] == [] + assert call_args["suffixes_to_exclude"] is None + + @patch("ray.llm._internal.common.utils.cloud_utils.GCSFileSystem") + def test_upload_model(self, mock_gcs_filesystem): + """Test uploading a model to cloud storage.""" + # Create temp directory for testing + with tempfile.TemporaryDirectory() as tempdir: + hash = "abcdef1234567890" + # Create refs/main file + os.makedirs(os.path.join(tempdir, "refs"), exist_ok=True) + model_rev_path = os.path.join(tempdir, "refs", "main") + with open(model_rev_path, "w") as f: + f.write(hash) + + # Create snapshots/ folder + model_asset_path = os.path.join(tempdir, "snapshots", hash) + os.makedirs(model_asset_path) + + # Test uploading model + CloudFileSystem.upload_model(tempdir, "gs://bucket/model") + + # Check that upload_files was called twice - once for model assets and once for hash file + assert mock_gcs_filesystem.upload_files.call_count == 2 + + # Verify the calls were made with correct arguments + calls = mock_gcs_filesystem.upload_files.call_args_list + call_paths = { + call[0][0] for call in calls + } # Extract local_path from each call + call_uris = { + call[0][1] for call in calls + } # Extract bucket_uri from each call + + assert model_asset_path in call_paths + assert model_rev_path in call_paths + assert "gs://bucket/model" in call_uris + assert "gs://bucket/model/hash" in call_uris diff --git a/python/ray/llm/tests/common/cloud/test_mirror_config.py b/python/ray/llm/tests/common/cloud/test_mirror_config.py new file mode 100644 index 000000000000..ac55aa52cbe1 --- /dev/null +++ b/python/ray/llm/tests/common/cloud/test_mirror_config.py @@ -0,0 +1,158 @@ +"""Tests for mirror config classes.""" + +import pytest + +from ray.llm._internal.common.utils.cloud_utils import ( + CloudMirrorConfig, + LoraMirrorConfig, +) + + +class TestCloudMirrorConfig: + """Tests for the CloudMirrorConfig class.""" + + def test_valid_s3_uri(self): + """Test valid S3 URI.""" + config = CloudMirrorConfig(bucket_uri="s3://my-bucket/path") + assert config.bucket_uri == "s3://my-bucket/path" + assert config.storage_type == "s3" + + def test_valid_gcs_uri(self): + """Test valid GCS URI.""" + config = CloudMirrorConfig(bucket_uri="gs://my-bucket/path") + assert config.bucket_uri == "gs://my-bucket/path" + assert config.storage_type == "gcs" + + def test_valid_abfss_uri(self): + """Test valid ABFSS URI.""" + config = CloudMirrorConfig( + bucket_uri="abfss://container@account.dfs.core.windows.net/path" + ) + assert ( + config.bucket_uri == "abfss://container@account.dfs.core.windows.net/path" + ) + assert config.storage_type == "abfss" + + def test_valid_azure_uri(self): + """Test valid Azure URI.""" + config = CloudMirrorConfig( + bucket_uri="azure://container@account.blob.core.windows.net/path" + ) + assert ( + config.bucket_uri == "azure://container@account.blob.core.windows.net/path" + ) + assert config.storage_type == "azure" + + def test_none_uri(self): + """Test None URI.""" + config = CloudMirrorConfig(bucket_uri=None) + assert config.bucket_uri is None + assert config.storage_type is None + + def test_invalid_uri(self): + """Test invalid URI.""" + with pytest.raises( + ValueError, match='Got invalid value "file:///tmp" for bucket_uri' + ): + CloudMirrorConfig(bucket_uri="file:///tmp") + + def test_extra_files(self): + """Test extra files configuration.""" + config = CloudMirrorConfig( + bucket_uri="s3://bucket/path", + extra_files=[ + {"bucket_uri": "s3://bucket/file1", "destination_path": "/dest1"}, + {"bucket_uri": "s3://bucket/file2", "destination_path": "/dest2"}, + ], + ) + assert len(config.extra_files) == 2 + assert config.extra_files[0].bucket_uri == "s3://bucket/file1" + assert config.extra_files[0].destination_path == "/dest1" + + +class TestLoraMirrorConfig: + """Tests for the LoraMirrorConfig class.""" + + def test_valid_s3_config(self): + """Test valid S3 LoRA config.""" + config = LoraMirrorConfig( + lora_model_id="test-model", + bucket_uri="s3://my-bucket/lora-models", + max_total_tokens=1000, + ) + assert config.lora_model_id == "test-model" + assert config.bucket_uri == "s3://my-bucket/lora-models" + assert config.bucket_name == "my-bucket" + assert config.bucket_path == "lora-models" + + def test_valid_abfss_config(self): + """Test valid ABFSS LoRA config.""" + config = LoraMirrorConfig( + lora_model_id="test-model", + bucket_uri="abfss://container@account.dfs.core.windows.net/lora/models", + max_total_tokens=1000, + ) + assert config.lora_model_id == "test-model" + assert ( + config.bucket_uri + == "abfss://container@account.dfs.core.windows.net/lora/models" + ) + assert config.bucket_name == "container" + assert config.bucket_path == "lora/models" + + def test_valid_azure_config(self): + """Test valid Azure LoRA config.""" + config = LoraMirrorConfig( + lora_model_id="test-model", + bucket_uri="azure://container@account.blob.core.windows.net/lora/models", + max_total_tokens=1000, + ) + assert config.lora_model_id == "test-model" + assert ( + config.bucket_uri + == "azure://container@account.blob.core.windows.net/lora/models" + ) + assert config.bucket_name == "container" + assert config.bucket_path == "lora/models" + + def test_bucket_path_parsing(self): + """Test bucket path parsing for different URI formats.""" + # S3 with multiple path segments + config = LoraMirrorConfig( + lora_model_id="test", + bucket_uri="s3://bucket/path/to/model", + max_total_tokens=1000, + ) + assert config.bucket_name == "bucket" + assert config.bucket_path == "path/to/model" + + # ABFSS with multiple path segments + config = LoraMirrorConfig( + lora_model_id="test", + bucket_uri="abfss://container@account.dfs.core.windows.net/deep/nested/path", + max_total_tokens=1000, + ) + assert config.bucket_name == "container" + assert config.bucket_path == "deep/nested/path" + + def test_invalid_uri(self): + """Test invalid URI in LoRA config.""" + with pytest.raises( + ValueError, match='Got invalid value "file:///tmp" for bucket_uri' + ): + LoraMirrorConfig( + lora_model_id="test-model", + bucket_uri="file:///tmp", + max_total_tokens=1000, + ) + + def test_optional_fields(self): + """Test optional fields in LoRA config.""" + config = LoraMirrorConfig( + lora_model_id="test-model", + bucket_uri="s3://bucket/path", + max_total_tokens=1000, + sync_args=["--exclude", "*.tmp"], + ) + assert config.max_total_tokens == 1000 + assert config.sync_args == ["--exclude", "*.tmp"] diff --git a/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py b/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py new file mode 100644 index 000000000000..57604033fcb2 --- /dev/null +++ b/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py @@ -0,0 +1,544 @@ +"""Tests for PyArrowFileSystem class.""" + +import os +from unittest.mock import ANY, MagicMock, patch + +import pyarrow.fs as pa_fs +import pytest + +from ray.llm._internal.common.utils.cloud_filesystem.pyarrow_filesystem import ( + PyArrowFileSystem, +) + + +class TestPyArrowFileSystem: + """Tests for the PyArrowFileSystem class.""" + + @patch("pyarrow.fs.S3FileSystem") + def test_get_file(self, mock_s3fs): + """Test getting a file from cloud storage.""" + # Setup mock filesystem and file + mock_fs = MagicMock() + mock_s3fs.return_value = mock_fs + + # Mock file content and info + mock_file = MagicMock() + mock_file.read.return_value = b"test file content" + mock_fs.open_input_file.return_value.__enter__.return_value = mock_file + mock_fs.get_file_info.return_value.type = pa_fs.FileType.File + + # Test getting file as string (default) + content = PyArrowFileSystem.get_file("s3://bucket/test.txt") + assert content == "test file content" + + # Test getting file as bytes + content_bytes = PyArrowFileSystem.get_file( + "s3://bucket/test.txt", decode_as_utf_8=False + ) + assert content_bytes == b"test file content" + + # Test non-existent file + mock_fs.get_file_info.return_value.type = pa_fs.FileType.NotFound + assert PyArrowFileSystem.get_file("s3://bucket/nonexistent.txt") is None + + @patch("pyarrow.fs.GcsFileSystem") + def test_list_subfolders(self, mock_gcsfs): + """Test listing subfolders in cloud storage.""" + # Setup mock filesystem + mock_fs = MagicMock() + mock_gcsfs.return_value = mock_fs + + # Create mock file infos for directory listing + dir1 = MagicMock() + dir1.type = pa_fs.FileType.Directory + dir1.path = "bucket/parent/dir1" + + dir2 = MagicMock() + dir2.type = pa_fs.FileType.Directory + dir2.path = "bucket/parent/dir2" + + file1 = MagicMock() + file1.type = pa_fs.FileType.File + file1.path = "bucket/parent/file.txt" + + mock_fs.get_file_info.return_value = [dir1, dir2, file1] + + # Test listing subfolders + folders = PyArrowFileSystem.list_subfolders("gs://bucket/parent") + assert sorted(folders) == ["dir1", "dir2"] + + @patch.object(PyArrowFileSystem, "get_fs_and_path") + def test_list_subfolders_exception_handling(self, mock_get_fs_and_path): + """Test that list_subfolders returns empty list when get_fs_and_path raises exception.""" + # Make get_fs_and_path raise an exception + mock_get_fs_and_path.side_effect = ValueError("Example exception") + + # Test that list_subfolders handles the exception gracefully + folders = PyArrowFileSystem.list_subfolders("gs://bucket/parent") + assert folders == [] + + # Verify get_fs_and_path was called + mock_get_fs_and_path.assert_called_once_with("gs://bucket/parent/") + + @patch("pyarrow.fs.S3FileSystem") + def test_download_files(self, mock_s3fs): + """Test downloading files from cloud storage.""" + import tempfile + + # Setup mock filesystem + mock_fs = MagicMock() + mock_s3fs.return_value = mock_fs + + # Create mock file infos for listing + file_info1 = MagicMock() + file_info1.type = pa_fs.FileType.File + file_info1.path = "bucket/dir/file1.txt" + + file_info2 = MagicMock() + file_info2.type = pa_fs.FileType.File + file_info2.path = "bucket/dir/subdir/file2.txt" + + dir_info = MagicMock() + dir_info.type = pa_fs.FileType.Directory + dir_info.path = "bucket/dir/subdir" + + mock_fs.get_file_info.return_value = [file_info1, file_info2, dir_info] + + # Mock file content + mock_file = MagicMock() + mock_file.read.return_value = b"test content" + mock_fs.open_input_file.return_value.__enter__.return_value = mock_file + + # Create temp directory for testing + with tempfile.TemporaryDirectory() as tempdir: + # Test downloading files + PyArrowFileSystem.download_files(tempdir, "s3://bucket/dir") + + # Check that files were downloaded correctly + assert os.path.exists(os.path.join(tempdir, "file1.txt")) + assert os.path.exists(os.path.join(tempdir, "subdir", "file2.txt")) + + # Check content of downloaded files + with open(os.path.join(tempdir, "file1.txt"), "rb") as f: + assert f.read() == b"test content" + + @patch("pyarrow.fs.copy_files") + @patch("pyarrow.fs.S3FileSystem") + def test_upload_files(self, mock_s3fs, mock_copy_files): + """Test uploading files to cloud storage.""" + import tempfile + + # Setup mock filesystem + mock_fs = MagicMock() + mock_s3fs.return_value = mock_fs + + # Create temp directory for testing + with tempfile.TemporaryDirectory() as tempdir: + # Test uploading files + PyArrowFileSystem.upload_files(tempdir, "s3://bucket/dir") + + # Check that the files are copied + mock_copy_files.assert_called_once_with( + source=tempdir, + destination="bucket/dir", + source_filesystem=ANY, + destination_filesystem=ANY, + ) + + +class TestFilterFiles: + """Tests for the _filter_files method in PyArrowFileSystem.""" + + def test_filter_files_no_filters(self): + """Test filtering files with no inclusion or exclusion filters.""" + # Setup mock filesystem + mock_fs = MagicMock() + + # Create mock file infos + file_info1 = MagicMock() + file_info1.type = pa_fs.FileType.File + file_info1.path = "bucket/model/file1.txt" + + file_info2 = MagicMock() + file_info2.type = pa_fs.FileType.File + file_info2.path = "bucket/model/subdir/file2.json" + + dir_info = MagicMock() + dir_info.type = pa_fs.FileType.Directory + dir_info.path = "bucket/model/subdir" + + mock_fs.get_file_info.return_value = [file_info1, file_info2, dir_info] + + # Test filtering with no filters + result = PyArrowFileSystem._filter_files( + fs=mock_fs, source_path="bucket/model", destination_path="/local/dest" + ) + + # Should include all files, exclude directories + expected = [ + ("bucket/model/file1.txt", "/local/dest/file1.txt"), + ("bucket/model/subdir/file2.json", "/local/dest/subdir/file2.json"), + ] + assert sorted(result) == sorted(expected) + + # Verify filesystem was called correctly + mock_fs.get_file_info.assert_called_once() + call_args = mock_fs.get_file_info.call_args[0][0] + assert call_args.base_dir == "bucket/model" + assert call_args.recursive is True + + def test_filter_files_with_inclusion_substrings(self): + """Test filtering files with inclusion substrings.""" + # Setup mock filesystem + mock_fs = MagicMock() + + # Create mock file infos + file_info1 = MagicMock() + file_info1.type = pa_fs.FileType.File + file_info1.path = "bucket/model/config.json" + + file_info2 = MagicMock() + file_info2.type = pa_fs.FileType.File + file_info2.path = "bucket/model/weights.bin" + + file_info3 = MagicMock() + file_info3.type = pa_fs.FileType.File + file_info3.path = "bucket/model/tokenizer.json" + + mock_fs.get_file_info.return_value = [file_info1, file_info2, file_info3] + + # Test filtering with inclusion substrings + result = PyArrowFileSystem._filter_files( + fs=mock_fs, + source_path="bucket/model", + destination_path="/local/dest", + substrings_to_include=["config", "tokenizer"], + ) + + # Should only include files with "config" or "tokenizer" in path + expected = [ + ("bucket/model/config.json", "/local/dest/config.json"), + ("bucket/model/tokenizer.json", "/local/dest/tokenizer.json"), + ] + assert sorted(result) == sorted(expected) + + def test_filter_files_with_exclusion_suffixes(self): + """Test filtering files with exclusion suffixes.""" + # Setup mock filesystem + mock_fs = MagicMock() + + # Create mock file infos + file_info1 = MagicMock() + file_info1.type = pa_fs.FileType.File + file_info1.path = "bucket/model/model.bin" + + file_info2 = MagicMock() + file_info2.type = pa_fs.FileType.File + file_info2.path = "bucket/model/config.json" + + file_info3 = MagicMock() + file_info3.type = pa_fs.FileType.File + file_info3.path = "bucket/model/temp.tmp" + + file_info4 = MagicMock() + file_info4.type = pa_fs.FileType.File + file_info4.path = "bucket/model/log.txt" + + mock_fs.get_file_info.return_value = [ + file_info1, + file_info2, + file_info3, + file_info4, + ] + + # Test filtering with exclusion suffixes + result = PyArrowFileSystem._filter_files( + fs=mock_fs, + source_path="bucket/model", + destination_path="/local/dest", + suffixes_to_exclude=[".tmp", ".txt"], + ) + + # Should exclude files ending with .tmp or .txt + expected = [ + ("bucket/model/model.bin", "/local/dest/model.bin"), + ("bucket/model/config.json", "/local/dest/config.json"), + ] + assert sorted(result) == sorted(expected) + + def test_filter_files_with_both_filters(self): + """Test filtering files with both inclusion and exclusion filters.""" + # Setup mock filesystem + mock_fs = MagicMock() + + # Create mock file infos + file_info1 = MagicMock() + file_info1.type = pa_fs.FileType.File + file_info1.path = "bucket/model/config.json" + + file_info2 = MagicMock() + file_info2.type = pa_fs.FileType.File + file_info2.path = "bucket/model/config.tmp" + + file_info3 = MagicMock() + file_info3.type = pa_fs.FileType.File + file_info3.path = "bucket/model/weights.bin" + + file_info4 = MagicMock() + file_info4.type = pa_fs.FileType.File + file_info4.path = "bucket/model/tokenizer.json" + + mock_fs.get_file_info.return_value = [ + file_info1, + file_info2, + file_info3, + file_info4, + ] + + # Test filtering with both inclusion and exclusion + result = PyArrowFileSystem._filter_files( + fs=mock_fs, + source_path="bucket/model", + destination_path="/local/dest", + substrings_to_include=["config", "tokenizer"], + suffixes_to_exclude=[".tmp"], + ) + + # Should include files with "config" or "tokenizer" but exclude .tmp files + expected = [ + ("bucket/model/config.json", "/local/dest/config.json"), + ("bucket/model/tokenizer.json", "/local/dest/tokenizer.json"), + ] + assert sorted(result) == sorted(expected) + + +class TestPyArrowFileSystemAzureSupport: + """Tests for Azure/ABFSS support in PyArrowFileSystem.""" + + @patch("adlfs.AzureBlobFileSystem") + @patch("azure.identity.DefaultAzureCredential") + @patch("pyarrow.fs.PyFileSystem") + @patch("pyarrow.fs.FSSpecHandler") + def test_get_fs_and_path_abfss( + self, mock_handler, mock_pyfs, mock_cred, mock_adlfs + ): + """Test getting ABFSS filesystem and path.""" + mock_adlfs_instance = MagicMock() + mock_adlfs.return_value = mock_adlfs_instance + mock_pyfs_instance = MagicMock() + mock_pyfs.return_value = mock_pyfs_instance + + fs, path = PyArrowFileSystem.get_fs_and_path( + "abfss://container@account.dfs.core.windows.net/path/to/file" + ) + + assert fs == mock_pyfs_instance + assert path == "container/path/to/file" + + # Verify the adlfs filesystem was created with correct parameters + mock_adlfs.assert_called_once_with( + account_name="account", credential=mock_cred.return_value + ) + mock_handler.assert_called_once_with(mock_adlfs_instance) + mock_pyfs.assert_called_once_with(mock_handler.return_value) + + @patch("adlfs.AzureBlobFileSystem") + @patch("azure.identity.DefaultAzureCredential") + @patch("pyarrow.fs.PyFileSystem") + @patch("pyarrow.fs.FSSpecHandler") + def test_get_fs_and_path_azure( + self, mock_handler, mock_pyfs, mock_cred, mock_adlfs + ): + """Test getting Azure filesystem and path.""" + mock_adlfs_instance = MagicMock() + mock_adlfs.return_value = mock_adlfs_instance + mock_pyfs_instance = MagicMock() + mock_pyfs.return_value = mock_pyfs_instance + + fs, path = PyArrowFileSystem.get_fs_and_path( + "azure://container@account.blob.core.windows.net/path/to/file" + ) + + assert fs == mock_pyfs_instance + assert path == "container/path/to/file" + + # Verify the adlfs filesystem was created with correct parameters + mock_adlfs.assert_called_once_with( + account_name="account", credential=mock_cred.return_value + ) + + def test_abfss_uri_validation(self): + """Test ABFSS URI validation.""" + # Test valid URIs + valid_uris = [ + "abfss://container@account.dfs.core.windows.net/path", + "abfss://my-container@myaccount.dfs.core.windows.net/deep/nested/path", + ] + + for uri in valid_uris: + with patch("adlfs.AzureBlobFileSystem"), patch( + "azure.identity.DefaultAzureCredential" + ), patch("pyarrow.fs.PyFileSystem"), patch("pyarrow.fs.FSSpecHandler"): + # Should not raise an exception + PyArrowFileSystem._create_abfss_filesystem(uri) + + # Test invalid URIs + invalid_uris = [ + "abfss://container", # Missing @account + "abfss://@account.dfs.core.windows.net/path", # Empty container + "abfss://container@account.wrong.domain/path", # Wrong domain + "abfss://container@.dfs.core.windows.net/path", # Empty account + "abfss://container@account.dfs.core.windows.net", # No path (but this is actually valid) + ] + + for uri in invalid_uris[:-1]: # Skip the last one as it's actually valid + with pytest.raises(ValueError): + PyArrowFileSystem._create_abfss_filesystem(uri) + + def test_azure_uri_validation(self): + """Test Azure URI validation.""" + # Test valid URIs + valid_uris = [ + "azure://container@account.blob.core.windows.net/path", + "azure://container@account.dfs.core.windows.net/path", + "azure://my-container@myaccount.blob.core.windows.net/deep/nested/path", + ] + + for uri in valid_uris: + with patch("adlfs.AzureBlobFileSystem"), patch( + "azure.identity.DefaultAzureCredential" + ), patch("pyarrow.fs.PyFileSystem"), patch("pyarrow.fs.FSSpecHandler"): + # Should not raise an exception + PyArrowFileSystem._create_azure_filesystem(uri) + + # Test invalid URIs + invalid_uris = [ + "azure://container", # Missing @account + "azure://@account.blob.core.windows.net/path", # Empty container + "azure://container@account.wrong.domain/path", # Wrong domain + "azure://container@.blob.core.windows.net/path", # Empty account + ] + + for uri in invalid_uris: + with pytest.raises(ValueError): + PyArrowFileSystem._create_azure_filesystem(uri) + + def test_abfss_import_error(self): + """Test ImportError when adlfs is not available.""" + with patch( + "builtins.__import__", side_effect=ImportError("No module named 'adlfs'") + ): + with pytest.raises( + ImportError, match="You must `pip install adlfs azure-identity`" + ): + PyArrowFileSystem._create_abfss_filesystem( + "abfss://container@account.dfs.core.windows.net/path" + ) + + def test_azure_import_error(self): + """Test ImportError when adlfs is not available for Azure.""" + with patch( + "builtins.__import__", side_effect=ImportError("No module named 'adlfs'") + ): + with pytest.raises( + ImportError, match="You must `pip install adlfs azure-identity`" + ): + PyArrowFileSystem._create_azure_filesystem( + "azure://container@account.blob.core.windows.net/path" + ) + + @patch("adlfs.AzureBlobFileSystem") + @patch("azure.identity.DefaultAzureCredential") + @patch("pyarrow.fs.PyFileSystem") + @patch("pyarrow.fs.FSSpecHandler") + def test_abfss_anonymous_access_ignored( + self, mock_handler, mock_pyfs, mock_cred, mock_adlfs + ): + """Test that anonymous access pattern is ignored for ABFSS URIs.""" + mock_adlfs_instance = MagicMock() + mock_adlfs.return_value = mock_adlfs_instance + mock_pyfs_instance = MagicMock() + mock_pyfs.return_value = mock_pyfs_instance + + # ABFSS URI with @ symbol should not trigger anonymous access logic + fs, path = PyArrowFileSystem.get_fs_and_path( + "abfss://container@account.dfs.core.windows.net/path" + ) + + assert fs == mock_pyfs_instance + assert path == "container/path" + + # Verify that DefaultAzureCredential was used, not anonymous access + mock_cred.assert_called_once() + mock_adlfs.assert_called_once_with( + account_name="account", credential=mock_cred.return_value + ) + + @patch("adlfs.AzureBlobFileSystem") + @patch("azure.identity.DefaultAzureCredential") + @patch("pyarrow.fs.PyFileSystem") + @patch("pyarrow.fs.FSSpecHandler") + def test_get_file_abfss(self, mock_handler, mock_pyfs, mock_cred, mock_adlfs): + """Test getting a file from ABFSS storage.""" + # Setup mock filesystem and file + mock_adlfs_instance = MagicMock() + mock_adlfs.return_value = mock_adlfs_instance + mock_fs = MagicMock() + mock_pyfs.return_value = mock_fs + + # Mock file content and info + mock_file = MagicMock() + mock_file.read.return_value = b"test abfss content" + mock_fs.open_input_file.return_value.__enter__.return_value = mock_file + mock_fs.get_file_info.return_value.type = pa_fs.FileType.File + + # Test getting file as string (default) + content = PyArrowFileSystem.get_file( + "abfss://container@account.dfs.core.windows.net/test.txt" + ) + assert content == "test abfss content" + + # Verify the correct path was used + mock_fs.get_file_info.assert_called_with("container/test.txt") + mock_fs.open_input_file.assert_called_with("container/test.txt") + + @patch("adlfs.AzureBlobFileSystem") + @patch("azure.identity.DefaultAzureCredential") + @patch("pyarrow.fs.PyFileSystem") + @patch("pyarrow.fs.FSSpecHandler") + def test_list_subfolders_abfss( + self, mock_handler, mock_pyfs, mock_cred, mock_adlfs + ): + """Test listing subfolders in ABFSS storage.""" + # Setup mock filesystem + mock_adlfs_instance = MagicMock() + mock_adlfs.return_value = mock_adlfs_instance + mock_fs = MagicMock() + mock_pyfs.return_value = mock_fs + + # Create mock file infos for directory listing + dir1 = MagicMock() + dir1.type = pa_fs.FileType.Directory + dir1.path = "container/parent/subdir1" + + dir2 = MagicMock() + dir2.type = pa_fs.FileType.Directory + dir2.path = "container/parent/subdir2" + + file1 = MagicMock() + file1.type = pa_fs.FileType.File + file1.path = "container/parent/file.txt" + + mock_fs.get_file_info.return_value = [dir1, dir2, file1] + + # Test listing subfolders + folders = PyArrowFileSystem.list_subfolders( + "abfss://container@account.dfs.core.windows.net/parent" + ) + assert sorted(folders) == ["subdir1", "subdir2"] + + # Verify the correct path was used + mock_fs.get_file_info.assert_called_once() + call_args = mock_fs.get_file_info.call_args[0][0] + assert call_args.base_dir == "container/parent/" + assert call_args.recursive is False diff --git a/python/ray/llm/tests/common/cloud/test_utils.py b/python/ray/llm/tests/common/cloud/test_utils.py new file mode 100644 index 000000000000..3b8280d3d5bc --- /dev/null +++ b/python/ray/llm/tests/common/cloud/test_utils.py @@ -0,0 +1,315 @@ +"""Utility tests for cloud functionality.""" + +import asyncio + +import pytest + +from ray.llm._internal.common.utils.cloud_utils import ( + CloudObjectCache, + is_remote_path, + remote_object_cache, +) + + +class MockSyncFetcher: + def __init__(self): + self.call_count = 0 + self.calls = [] + + def __call__(self, key: str): + self.call_count += 1 + self.calls.append(key) + if key == "missing": + return -1 + return f"value-{key}" + + +class MockAsyncFetcher: + def __init__(self): + self.call_count = 0 + self.calls = [] + + async def __call__(self, key: str): + self.call_count += 1 + self.calls.append(key) + if key == "missing": + return -1 + return f"value-{key}" + + +class TestCloudObjectCache: + """Tests for the CloudObjectCache class.""" + + def test_sync_cache_basic(self): + """Test basic synchronous cache functionality.""" + fetcher = MockSyncFetcher() + cache = CloudObjectCache(max_size=2, fetch_fn=fetcher) + + # Test fetching a value (should be a miss) + assert cache.get("key1") == "value-key1" + assert fetcher.call_count == 1 + assert fetcher.calls == ["key1"] + + # Test cache hit (should not call fetcher) + assert cache.get("key1") == "value-key1" + assert fetcher.call_count == 1 # Count should not increase + assert fetcher.calls == ["key1"] # Calls should not change + + # Test cache size limit + assert cache.get("key2") == "value-key2" # Miss, should call fetcher + assert fetcher.call_count == 2 + assert fetcher.calls == ["key1", "key2"] + + assert ( + cache.get("key3") == "value-key3" + ) # Miss, should call fetcher and evict key1 + assert fetcher.call_count == 3 + assert fetcher.calls == ["key1", "key2", "key3"] + + assert len(cache) == 2 + + # Verify key1 was evicted by checking if it's fetched again + assert cache.get("key1") == "value-key1" # Miss, should call fetcher + assert fetcher.call_count == 4 + assert fetcher.calls == ["key1", "key2", "key3", "key1"] + + # Verify final cache state + assert len(cache) == 2 + assert "key3" in cache._cache # key3 should still be in cache + assert "key1" in cache._cache # key1 should be back in cache + assert "key2" not in cache._cache # key2 should have been evicted + + @pytest.mark.asyncio + async def test_async_cache_missing_object_expiration(self): + """Test cache expiration for missing objects in async mode.""" + fetcher = MockAsyncFetcher() + cache = CloudObjectCache( + max_size=2, + fetch_fn=fetcher, + missing_expire_seconds=1, # 1 second to expire missing object + exists_expire_seconds=3, # 3 seconds to expire existing object + missing_object_value=-1, + ) + + # Test missing object expiration + assert await cache.aget("missing") is -1 # First fetch + assert fetcher.call_count == 1 + assert fetcher.calls == ["missing"] + + # Should still be cached + assert await cache.aget("missing") is -1 # Cache hit + assert fetcher.call_count == 1 # No new fetch + assert fetcher.calls == ["missing"] + + await asyncio.sleep(1.5) # Wait for missing object to expire + assert await cache.aget("missing") is -1 # Should fetch again after expiration + assert fetcher.call_count == 2 # New fetch + assert fetcher.calls == ["missing", "missing"] + + @pytest.mark.asyncio + async def test_async_cache_existing_object_expiration(self): + """Test expiration of existing objects in async mode.""" + fetcher = MockAsyncFetcher() + cache = CloudObjectCache( + max_size=2, + fetch_fn=fetcher, + missing_expire_seconds=1, # 1 second to expire missing object + exists_expire_seconds=3, # 3 seconds to expire existing object + missing_object_value=-1, + ) + + # Test existing object expiration + assert await cache.aget("key1") == "value-key1" # First fetch + assert fetcher.call_count == 1 + assert fetcher.calls == ["key1"] + + # Should still be cached (not expired) + assert await cache.aget("key1") == "value-key1" # Cache hit + assert fetcher.call_count == 1 # No new fetch + assert fetcher.calls == ["key1"] + + await asyncio.sleep(1.5) # Not expired yet (exists_expire_seconds=3) + assert await cache.aget("key1") == "value-key1" # Should still hit cache + assert fetcher.call_count == 1 # No new fetch + assert fetcher.calls == ["key1"] # No change in calls + + await asyncio.sleep(2) # Now expired (total > 2 seconds) + assert await cache.aget("key1") == "value-key1" # Should fetch again + assert fetcher.call_count == 2 # New fetch + assert fetcher.calls == ["key1", "key1"] + + # Verify final cache state + assert len(cache) == 1 + + +class TestRemoteObjectCacheDecorator: + """Tests for the remote_object_cache decorator.""" + + @pytest.mark.asyncio + async def test_basic_functionality(self): + """Test basic remote_object_cache decorator functionality.""" + call_count = 0 + MISSING = object() + + @remote_object_cache( + max_size=2, + missing_expire_seconds=1, + exists_expire_seconds=3, + missing_object_value=MISSING, + ) + async def fetch(key: str): + nonlocal call_count + call_count += 1 + if key == "missing": + return MISSING + return f"value-{key}" + + # Test cache hit + assert await fetch("key1") == "value-key1" + assert call_count == 1 + assert await fetch("key1") == "value-key1" # Should hit cache + assert call_count == 1 # Count should not increase + + # Test cache size limit + assert await fetch("key2") == "value-key2" + assert call_count == 2 + assert await fetch("key3") == "value-key3" # Should evict key1 + assert call_count == 3 + + # Verify key1 was evicted + assert await fetch("key1") == "value-key1" + assert call_count == 4 + + @pytest.mark.asyncio + async def test_expiration(self): + """Test cache expiration for both missing and existing objects.""" + call_count = 0 + MISSING = object() + + @remote_object_cache( + max_size=2, + missing_expire_seconds=1, # 1 second to expire missing object + exists_expire_seconds=3, # 3 seconds to expire existing object + missing_object_value=MISSING, + ) + async def fetch(key: str): + nonlocal call_count + call_count += 1 + if key == "missing": + return MISSING + return f"value-{key}" + + # Test missing object expiration + assert await fetch("missing") is MISSING + assert call_count == 1 + assert await fetch("missing") is MISSING # Should hit cache + assert call_count == 1 + + await asyncio.sleep(1.5) # Wait for missing object to expire + assert await fetch("missing") is MISSING # Should fetch again + assert call_count == 2 + + # Test existing object expiration + assert await fetch("key1") == "value-key1" + assert call_count == 3 + assert await fetch("key1") == "value-key1" # Should hit cache + assert call_count == 3 + + await asyncio.sleep(1.5) # Not expired yet + assert await fetch("key1") == "value-key1" # Should still hit cache + assert call_count == 3 + + await asyncio.sleep(2) # Now expired (total > 3 seconds) + assert await fetch("key1") == "value-key1" # Should fetch again + assert call_count == 4 + + @pytest.mark.asyncio + async def test_error_handling(self): + """Test error handling in remote_object_cache decorator.""" + call_count = 0 + + @remote_object_cache(max_size=2) + async def fetch(key: str): + nonlocal call_count + call_count += 1 + if key == "error": + raise ValueError("Test error") + return f"value-{key}" + + # Test successful case + assert await fetch("key1") == "value-key1" + assert call_count == 1 + + # Test error case + with pytest.raises(ValueError, match="Test error"): + await fetch("error") + assert call_count == 2 + + # Verify error wasn't cached + with pytest.raises(ValueError, match="Test error"): + await fetch("error") + assert call_count == 3 + + @pytest.mark.asyncio + async def test_concurrent_access(self): + """Test concurrent access to cached function.""" + call_count = 0 + DELAY = 0.1 + + @remote_object_cache(max_size=2) + async def slow_fetch(key: str): + nonlocal call_count + call_count += 1 + await asyncio.sleep(DELAY) # Simulate slow operation + return f"value-{key}" + + # Launch multiple concurrent calls + tasks = [slow_fetch("key1") for _ in range(5)] + results = await asyncio.gather(*tasks) + + # All results should be the same + assert all(r == "value-key1" for r in results) + # Should only call once despite multiple concurrent requests + assert call_count == 1 + + +class TestIsRemotePath: + """Tests for the is_remote_path utility function.""" + + def test_s3_paths(self): + """Test S3 path detection.""" + assert is_remote_path("s3://bucket/path") is True + assert is_remote_path("s3://bucket") is True + assert is_remote_path("s3://anonymous@bucket/path") is True + + def test_gcs_paths(self): + """Test GCS path detection.""" + assert is_remote_path("gs://bucket/path") is True + assert is_remote_path("gs://bucket") is True + assert is_remote_path("gs://anonymous@bucket/path") is True + + def test_abfss_paths(self): + """Test ABFSS path detection.""" + assert ( + is_remote_path("abfss://container@account.dfs.core.windows.net/path") + is True + ) + assert is_remote_path("abfss://container@account.dfs.core.windows.net") is True + + def test_azure_paths(self): + """Test Azure path detection.""" + assert ( + is_remote_path("azure://container@account.blob.core.windows.net/path") + is True + ) + assert ( + is_remote_path("azure://container@account.dfs.core.windows.net/path") + is True + ) + + def test_local_paths(self): + """Test local path detection.""" + assert is_remote_path("/local/path") is False + assert is_remote_path("./relative/path") is False + assert is_remote_path("file:///local/path") is False + assert is_remote_path("http://example.com") is False diff --git a/python/ray/llm/tests/common/utils/test_cloud_utils.py b/python/ray/llm/tests/common/utils/test_cloud_utils.py deleted file mode 100644 index e0d73572da81..000000000000 --- a/python/ray/llm/tests/common/utils/test_cloud_utils.py +++ /dev/null @@ -1,1075 +0,0 @@ -import asyncio -import os -import sys -import tempfile -from unittest.mock import ANY, MagicMock, call, patch - -import pyarrow.fs as pa_fs -import pytest - -from ray.llm._internal.common.utils.cloud_utils import ( - CloudFileSystem, - CloudMirrorConfig, - CloudObjectCache, - LoraMirrorConfig, - is_remote_path, - remote_object_cache, -) - - -class MockSyncFetcher: - def __init__(self): - self.call_count = 0 - self.calls = [] - - def __call__(self, key: str): - self.call_count += 1 - self.calls.append(key) - if key == "missing": - return -1 - return f"value-{key}" - - -class MockAsyncFetcher: - def __init__(self): - self.call_count = 0 - self.calls = [] - - async def __call__(self, key: str): - self.call_count += 1 - self.calls.append(key) - if key == "missing": - return -1 - return f"value-{key}" - - -class TestCloudObjectCache: - """Tests for the CloudObjectCache class.""" - - def test_sync_cache_basic(self): - """Test basic synchronous cache functionality.""" - fetcher = MockSyncFetcher() - cache = CloudObjectCache(max_size=2, fetch_fn=fetcher) - - # Test fetching a value (should be a miss) - assert cache.get("key1") == "value-key1" - assert fetcher.call_count == 1 - assert fetcher.calls == ["key1"] - - # Test cache hit (should not call fetcher) - assert cache.get("key1") == "value-key1" - assert fetcher.call_count == 1 # Count should not increase - assert fetcher.calls == ["key1"] # Calls should not change - - # Test cache size limit - assert cache.get("key2") == "value-key2" # Miss, should call fetcher - assert fetcher.call_count == 2 - assert fetcher.calls == ["key1", "key2"] - - assert ( - cache.get("key3") == "value-key3" - ) # Miss, should call fetcher and evict key1 - assert fetcher.call_count == 3 - assert fetcher.calls == ["key1", "key2", "key3"] - - assert len(cache) == 2 - - # Verify key1 was evicted by checking if it's fetched again - assert cache.get("key1") == "value-key1" # Miss, should call fetcher - assert fetcher.call_count == 4 - assert fetcher.calls == ["key1", "key2", "key3", "key1"] - - # Verify final cache state - assert len(cache) == 2 - assert "key3" in cache._cache # key3 should still be in cache - assert "key1" in cache._cache # key1 should be back in cache - assert "key2" not in cache._cache # key2 should have been evicted - - @pytest.mark.asyncio - async def test_async_cache_missing_object_expiration(self): - """Test cache expiration for missing objects in async mode.""" - fetcher = MockAsyncFetcher() - cache = CloudObjectCache( - max_size=2, - fetch_fn=fetcher, - missing_expire_seconds=1, # 1 second to expire missing object - exists_expire_seconds=3, # 3 seconds to expire existing object - missing_object_value=-1, - ) - - # Test missing object expiration - assert await cache.aget("missing") is -1 # First fetch - assert fetcher.call_count == 1 - assert fetcher.calls == ["missing"] - - # Should still be cached - assert await cache.aget("missing") is -1 # Cache hit - assert fetcher.call_count == 1 # No new fetch - assert fetcher.calls == ["missing"] - - await asyncio.sleep(1.5) # Wait for missing object to expire - assert await cache.aget("missing") is -1 # Should fetch again after expiration - assert fetcher.call_count == 2 # New fetch - assert fetcher.calls == ["missing", "missing"] - - @pytest.mark.asyncio - async def test_async_cache_existing_object_expiration(self): - """Test expiration of existing objects in async mode.""" - fetcher = MockAsyncFetcher() - cache = CloudObjectCache( - max_size=2, - fetch_fn=fetcher, - missing_expire_seconds=1, # 1 second to expire missing object - exists_expire_seconds=3, # 3 seconds to expire existing object - missing_object_value=-1, - ) - - # Test existing object expiration - assert await cache.aget("key1") == "value-key1" # First fetch - assert fetcher.call_count == 1 - assert fetcher.calls == ["key1"] - - # Should still be cached (not expired) - assert await cache.aget("key1") == "value-key1" # Cache hit - assert fetcher.call_count == 1 # No new fetch - - await asyncio.sleep(1.5) # Not expired yet (exists_expire_seconds=3) - assert await cache.aget("key1") == "value-key1" # Should still hit cache - assert fetcher.call_count == 1 # No new fetch - assert fetcher.calls == ["key1"] # No change in calls - - await asyncio.sleep(2) # Now expired (total > 2 seconds) - assert await cache.aget("key1") == "value-key1" # Should fetch again - assert fetcher.call_count == 2 # New fetch - assert fetcher.calls == ["key1", "key1"] - - # Verify final cache state - assert len(cache) == 1 - - -class TestRemoteObjectCacheDecorator: - """Tests for the remote_object_cache decorator.""" - - @pytest.mark.asyncio - async def test_basic_functionality(self): - """Test basic remote_object_cache decorator functionality.""" - call_count = 0 - MISSING = object() - - @remote_object_cache( - max_size=2, - missing_expire_seconds=1, - exists_expire_seconds=3, - missing_object_value=MISSING, - ) - async def fetch(key: str): - nonlocal call_count - call_count += 1 - if key == "missing": - return MISSING - return f"value-{key}" - - # Test cache hit - assert await fetch("key1") == "value-key1" - assert call_count == 1 - assert await fetch("key1") == "value-key1" # Should hit cache - assert call_count == 1 # Count should not increase - - # Test cache size limit - assert await fetch("key2") == "value-key2" - assert call_count == 2 - assert await fetch("key3") == "value-key3" # Should evict key1 - assert call_count == 3 - - # Verify key1 was evicted - assert await fetch("key1") == "value-key1" - assert call_count == 4 - - @pytest.mark.asyncio - async def test_expiration(self): - """Test cache expiration for both missing and existing objects.""" - call_count = 0 - MISSING = object() - - @remote_object_cache( - max_size=2, - missing_expire_seconds=1, # 1 second to expire missing object - exists_expire_seconds=3, # 3 seconds to expire existing object - missing_object_value=MISSING, - ) - async def fetch(key: str): - nonlocal call_count - call_count += 1 - if key == "missing": - return MISSING - return f"value-{key}" - - # Test missing object expiration - assert await fetch("missing") is MISSING - assert call_count == 1 - assert await fetch("missing") is MISSING # Should hit cache - assert call_count == 1 - - await asyncio.sleep(1.5) # Wait for missing object to expire - assert await fetch("missing") is MISSING # Should fetch again - assert call_count == 2 - - # Test existing object expiration - assert await fetch("key1") == "value-key1" - assert call_count == 3 - assert await fetch("key1") == "value-key1" # Should hit cache - assert call_count == 3 - - await asyncio.sleep(1.5) # Not expired yet - assert await fetch("key1") == "value-key1" # Should still hit cache - assert call_count == 3 - - await asyncio.sleep(2) # Now expired (total > 3 seconds) - assert await fetch("key1") == "value-key1" # Should fetch again - assert call_count == 4 - - @pytest.mark.asyncio - async def test_error_handling(self): - """Test error handling in remote_object_cache decorator.""" - call_count = 0 - - @remote_object_cache(max_size=2) - async def fetch(key: str): - nonlocal call_count - call_count += 1 - if key == "error": - raise ValueError("Test error") - return f"value-{key}" - - # Test successful case - assert await fetch("key1") == "value-key1" - assert call_count == 1 - - # Test error case - with pytest.raises(ValueError, match="Test error"): - await fetch("error") - assert call_count == 2 - - # Verify error wasn't cached - with pytest.raises(ValueError, match="Test error"): - await fetch("error") - assert call_count == 3 - - @pytest.mark.asyncio - async def test_concurrent_access(self): - """Test concurrent access to cached function.""" - call_count = 0 - DELAY = 0.1 - - @remote_object_cache(max_size=2) - async def slow_fetch(key: str): - nonlocal call_count - call_count += 1 - await asyncio.sleep(DELAY) # Simulate slow operation - return f"value-{key}" - - # Launch multiple concurrent calls - tasks = [slow_fetch("key1") for _ in range(5)] - results = await asyncio.gather(*tasks) - - # All results should be the same - assert all(r == "value-key1" for r in results) - # Should only call once despite multiple concurrent requests - assert call_count == 1 - - -class TestCloudFileSystem: - """Tests for the CloudFileSystem class.""" - - @patch("pyarrow.fs.S3FileSystem") - def test_get_file(self, mock_s3fs): - """Test getting a file from cloud storage.""" - # Setup mock filesystem and file - mock_fs = MagicMock() - mock_s3fs.return_value = mock_fs - - # Mock file content and info - mock_file = MagicMock() - mock_file.read.return_value = b"test file content" - mock_fs.open_input_file.return_value.__enter__.return_value = mock_file - mock_fs.get_file_info.return_value.type = pa_fs.FileType.File - - # Test getting file as string (default) - content = CloudFileSystem.get_file("s3://bucket/test.txt") - assert content == "test file content" - - # Test getting file as bytes - content_bytes = CloudFileSystem.get_file( - "s3://bucket/test.txt", decode_as_utf_8=False - ) - assert content_bytes == b"test file content" - - # Test non-existent file - mock_fs.get_file_info.return_value.type = pa_fs.FileType.NotFound - assert CloudFileSystem.get_file("s3://bucket/nonexistent.txt") is None - - @patch("pyarrow.fs.GcsFileSystem") - def test_list_subfolders(self, mock_gcsfs): - """Test listing subfolders in cloud storage.""" - # Setup mock filesystem - mock_fs = MagicMock() - mock_gcsfs.return_value = mock_fs - - # Create mock file infos for directory listing - dir1 = MagicMock() - dir1.type = pa_fs.FileType.Directory - dir1.path = "bucket/parent/dir1" - - dir2 = MagicMock() - dir2.type = pa_fs.FileType.Directory - dir2.path = "bucket/parent/dir2" - - file1 = MagicMock() - file1.type = pa_fs.FileType.File - file1.path = "bucket/parent/file.txt" - - mock_fs.get_file_info.return_value = [dir1, dir2, file1] - - # Test listing subfolders - folders = CloudFileSystem.list_subfolders("gs://bucket/parent") - assert sorted(folders) == ["dir1", "dir2"] - - @patch("ray.llm._internal.common.utils.cloud_utils.CloudFileSystem.get_fs_and_path") - def test_list_subfolders_exception_handling(self, mock_get_fs_and_path): - """Test that list_subfolders returns empty list when get_fs_and_path raises exception.""" - # Make get_fs_and_path raise an exception - mock_get_fs_and_path.side_effect = ValueError("Example exception") - - # Test that list_subfolders handles the exception gracefully - folders = CloudFileSystem.list_subfolders("gs://bucket/parent") - assert folders == [] - - # Verify get_fs_and_path was called - mock_get_fs_and_path.assert_called_once_with("gs://bucket/parent/") - - @patch("pyarrow.fs.S3FileSystem") - def test_download_files(self, mock_s3fs): - """Test downloading files from cloud storage.""" - # Setup mock filesystem - mock_fs = MagicMock() - mock_s3fs.return_value = mock_fs - - # Create mock file infos for listing - file_info1 = MagicMock() - file_info1.type = pa_fs.FileType.File - file_info1.path = "bucket/dir/file1.txt" - - file_info2 = MagicMock() - file_info2.type = pa_fs.FileType.File - file_info2.path = "bucket/dir/subdir/file2.txt" - - dir_info = MagicMock() - dir_info.type = pa_fs.FileType.Directory - dir_info.path = "bucket/dir/subdir" - - mock_fs.get_file_info.return_value = [file_info1, file_info2, dir_info] - - # Mock file content - mock_file = MagicMock() - mock_file.read.return_value = b"test content" - mock_fs.open_input_file.return_value.__enter__.return_value = mock_file - - # Create temp directory for testing - with tempfile.TemporaryDirectory() as tempdir: - # Test downloading files - CloudFileSystem.download_files(tempdir, "s3://bucket/dir") - - # Check that files were downloaded correctly - assert os.path.exists(os.path.join(tempdir, "file1.txt")) - assert os.path.exists(os.path.join(tempdir, "subdir", "file2.txt")) - - # Check content of downloaded files - with open(os.path.join(tempdir, "file1.txt"), "rb") as f: - assert f.read() == b"test content" - - @patch("pyarrow.fs.GcsFileSystem") - def test_download_model(self, mock_gcsfs): - """Test downloading a model from cloud storage.""" - # Setup mock filesystem - mock_fs = MagicMock() - mock_gcsfs.return_value = mock_fs - - # Mock hash file - mock_hash_file = MagicMock() - mock_hash_file.read.return_value = b"abcdef1234567890" - mock_fs.open_input_file.return_value.__enter__.return_value = mock_hash_file - - # Mock file info for hash file - mock_fs.get_file_info.return_value.type = pa_fs.FileType.File - - # Create temp directory for testing - with tempfile.TemporaryDirectory() as tempdir: - # Test downloading model - with patch.object( - CloudFileSystem, "download_files_parallel" - ) as mock_download: - CloudFileSystem.download_model(tempdir, "gs://bucket/model", False) - - # Check that hash file was processed - assert os.path.exists(os.path.join(tempdir, "refs", "main")) - with open(os.path.join(tempdir, "refs", "main"), "r") as f: - assert f.read() == "abcdef1234567890" - - # Check that download_files_parallel was called correctly - mock_download.assert_called_once() - call_args = mock_download.call_args[1] - assert call_args["path"] == os.path.join( - tempdir, "snapshots", "abcdef1234567890" - ) - assert call_args["bucket_uri"] == "gs://bucket/model" - assert call_args["substrings_to_include"] == [] - assert call_args["suffixes_to_exclude"] is None - assert call_args["chunk_size"] == 64 * 1024 * 1024 - - @patch("pyarrow.fs.copy_files") - def test_upload_files(self, mock_copy_files): - """Test uploading files to cloud storage.""" - # Create temp directory for testing - with tempfile.TemporaryDirectory() as tempdir: - # Test uploading files - CloudFileSystem.upload_files(tempdir, "s3://bucket/dir") - - # Check that the files are copied - mock_copy_files.assert_called_once_with( - source=tempdir, - destination="bucket/dir", - source_filesystem=ANY, - destination_filesystem=ANY, - ) - - @patch("pyarrow.fs.copy_files") - def test_upload_model(self, mock_copy_files): - """Test uploading a model to cloud storage.""" - # Create temp directory for testing - with tempfile.TemporaryDirectory() as tempdir: - hash = "abcdef1234567890" - # Create refs/main file - os.makedirs(os.path.join(tempdir, "refs"), exist_ok=True) - model_rev_path = os.path.join(tempdir, "refs", "main") - with open(model_rev_path, "w") as f: - f.write(hash) - - # Create snapshots/ folder - model_asset_path = os.path.join(tempdir, "snapshots", hash) - os.makedirs(model_asset_path) - - # Test uploading model - CloudFileSystem.upload_model(tempdir, "gs://bucket/model") - - # Check that the files are copied - mock_copy_files.assert_has_calls( - [ - call( - source=model_rev_path, - destination="bucket/model/hash", - source_filesystem=ANY, - destination_filesystem=ANY, - ), - call( - source=model_asset_path, - destination="bucket/model", - source_filesystem=ANY, - destination_filesystem=ANY, - ), - ], - any_order=True, - ) - - -class TestIsRemotePath: - """Tests for the is_remote_path utility function.""" - - def test_s3_paths(self): - """Test S3 path detection.""" - assert is_remote_path("s3://bucket/path") is True - assert is_remote_path("s3://bucket") is True - assert is_remote_path("s3://anonymous@bucket/path") is True - - def test_gcs_paths(self): - """Test GCS path detection.""" - assert is_remote_path("gs://bucket/path") is True - assert is_remote_path("gs://bucket") is True - assert is_remote_path("gs://anonymous@bucket/path") is True - - def test_abfss_paths(self): - """Test ABFSS path detection.""" - assert ( - is_remote_path("abfss://container@account.dfs.core.windows.net/path") - is True - ) - assert is_remote_path("abfss://container@account.dfs.core.windows.net") is True - - def test_azure_paths(self): - """Test Azure path detection.""" - assert ( - is_remote_path("azure://container@account.blob.core.windows.net/path") - is True - ) - assert ( - is_remote_path("azure://container@account.dfs.core.windows.net/path") - is True - ) - - def test_local_paths(self): - """Test local path detection.""" - assert is_remote_path("/local/path") is False - assert is_remote_path("./relative/path") is False - assert is_remote_path("file:///local/path") is False - assert is_remote_path("http://example.com") is False - - -class TestCloudMirrorConfig: - """Tests for the CloudMirrorConfig class.""" - - def test_valid_s3_uri(self): - """Test valid S3 URI.""" - config = CloudMirrorConfig(bucket_uri="s3://my-bucket/path") - assert config.bucket_uri == "s3://my-bucket/path" - assert config.storage_type == "s3" - - def test_valid_gcs_uri(self): - """Test valid GCS URI.""" - config = CloudMirrorConfig(bucket_uri="gs://my-bucket/path") - assert config.bucket_uri == "gs://my-bucket/path" - assert config.storage_type == "gcs" - - def test_valid_abfss_uri(self): - """Test valid ABFSS URI.""" - config = CloudMirrorConfig( - bucket_uri="abfss://container@account.dfs.core.windows.net/path" - ) - assert ( - config.bucket_uri == "abfss://container@account.dfs.core.windows.net/path" - ) - assert config.storage_type == "abfss" - - def test_valid_azure_uri(self): - """Test valid Azure URI.""" - config = CloudMirrorConfig( - bucket_uri="azure://container@account.blob.core.windows.net/path" - ) - assert ( - config.bucket_uri == "azure://container@account.blob.core.windows.net/path" - ) - assert config.storage_type == "azure" - - def test_none_uri(self): - """Test None URI.""" - config = CloudMirrorConfig(bucket_uri=None) - assert config.bucket_uri is None - assert config.storage_type is None - - def test_invalid_uri(self): - """Test invalid URI.""" - with pytest.raises( - ValueError, match='Got invalid value "file:///tmp" for bucket_uri' - ): - CloudMirrorConfig(bucket_uri="file:///tmp") - - def test_extra_files(self): - """Test extra files configuration.""" - config = CloudMirrorConfig( - bucket_uri="s3://bucket/path", - extra_files=[ - {"bucket_uri": "s3://bucket/file1", "destination_path": "/dest1"}, - {"bucket_uri": "s3://bucket/file2", "destination_path": "/dest2"}, - ], - ) - assert len(config.extra_files) == 2 - assert config.extra_files[0].bucket_uri == "s3://bucket/file1" - assert config.extra_files[0].destination_path == "/dest1" - - -class TestLoraMirrorConfig: - """Tests for the LoraMirrorConfig class.""" - - def test_valid_s3_config(self): - """Test valid S3 LoRA config.""" - config = LoraMirrorConfig( - lora_model_id="test-model", - bucket_uri="s3://my-bucket/lora-models", - max_total_tokens=1000, - ) - assert config.lora_model_id == "test-model" - assert config.bucket_uri == "s3://my-bucket/lora-models" - assert config.bucket_name == "my-bucket" - assert config.bucket_path == "lora-models" - - def test_valid_abfss_config(self): - """Test valid ABFSS LoRA config.""" - config = LoraMirrorConfig( - lora_model_id="test-model", - bucket_uri="abfss://container@account.dfs.core.windows.net/lora/models", - max_total_tokens=1000, - ) - assert config.lora_model_id == "test-model" - assert ( - config.bucket_uri - == "abfss://container@account.dfs.core.windows.net/lora/models" - ) - assert config.bucket_name == "container" - assert config.bucket_path == "lora/models" - - def test_valid_azure_config(self): - """Test valid Azure LoRA config.""" - config = LoraMirrorConfig( - lora_model_id="test-model", - bucket_uri="azure://container@account.blob.core.windows.net/lora/models", - max_total_tokens=1000, - ) - assert config.lora_model_id == "test-model" - assert ( - config.bucket_uri - == "azure://container@account.blob.core.windows.net/lora/models" - ) - assert config.bucket_name == "container" - assert config.bucket_path == "lora/models" - - def test_bucket_path_parsing(self): - """Test bucket path parsing for different URI formats.""" - # S3 with multiple path segments - config = LoraMirrorConfig( - lora_model_id="test", - bucket_uri="s3://bucket/path/to/model", - max_total_tokens=1000, - ) - assert config.bucket_name == "bucket" - assert config.bucket_path == "path/to/model" - - # ABFSS with multiple path segments - config = LoraMirrorConfig( - lora_model_id="test", - bucket_uri="abfss://container@account.dfs.core.windows.net/deep/nested/path", - max_total_tokens=1000, - ) - assert config.bucket_name == "container" - assert config.bucket_path == "deep/nested/path" - - def test_invalid_uri(self): - """Test invalid URI in LoRA config.""" - with pytest.raises( - ValueError, match='Got invalid value "file:///tmp" for bucket_uri' - ): - LoraMirrorConfig( - lora_model_id="test-model", - bucket_uri="file:///tmp", - max_total_tokens=1000, - ) - - def test_optional_fields(self): - """Test optional fields in LoRA config.""" - config = LoraMirrorConfig( - lora_model_id="test-model", - bucket_uri="s3://bucket/path", - max_total_tokens=1000, - sync_args=["--exclude", "*.tmp"], - ) - assert config.max_total_tokens == 1000 - assert config.sync_args == ["--exclude", "*.tmp"] - - -class TestCloudFileSystemFilterFiles: - """Tests for the _filter_files method.""" - - def test_filter_files_no_filters(self): - """Test filtering files with no inclusion or exclusion filters.""" - # Setup mock filesystem - mock_fs = MagicMock() - - # Create mock file infos - file_info1 = MagicMock() - file_info1.type = pa_fs.FileType.File - file_info1.path = "bucket/model/file1.txt" - - file_info2 = MagicMock() - file_info2.type = pa_fs.FileType.File - file_info2.path = "bucket/model/subdir/file2.json" - - dir_info = MagicMock() - dir_info.type = pa_fs.FileType.Directory - dir_info.path = "bucket/model/subdir" - - mock_fs.get_file_info.return_value = [file_info1, file_info2, dir_info] - - # Test filtering with no filters - result = CloudFileSystem._filter_files( - fs=mock_fs, source_path="bucket/model", destination_path="/local/dest" - ) - - # Should include all files, exclude directories - expected = [ - ("bucket/model/file1.txt", "/local/dest/file1.txt"), - ("bucket/model/subdir/file2.json", "/local/dest/subdir/file2.json"), - ] - assert sorted(result) == sorted(expected) - - # Verify filesystem was called correctly - mock_fs.get_file_info.assert_called_once() - call_args = mock_fs.get_file_info.call_args[0][0] - assert call_args.base_dir == "bucket/model" - assert call_args.recursive is True - - def test_filter_files_with_inclusion_substrings(self): - """Test filtering files with inclusion substrings.""" - # Setup mock filesystem - mock_fs = MagicMock() - - # Create mock file infos - file_info1 = MagicMock() - file_info1.type = pa_fs.FileType.File - file_info1.path = "bucket/model/config.json" - - file_info2 = MagicMock() - file_info2.type = pa_fs.FileType.File - file_info2.path = "bucket/model/weights.bin" - - file_info3 = MagicMock() - file_info3.type = pa_fs.FileType.File - file_info3.path = "bucket/model/tokenizer.json" - - mock_fs.get_file_info.return_value = [file_info1, file_info2, file_info3] - - # Test filtering with inclusion substrings - result = CloudFileSystem._filter_files( - fs=mock_fs, - source_path="bucket/model", - destination_path="/local/dest", - substrings_to_include=["config", "tokenizer"], - ) - - # Should only include files with "config" or "tokenizer" in path - expected = [ - ("bucket/model/config.json", "/local/dest/config.json"), - ("bucket/model/tokenizer.json", "/local/dest/tokenizer.json"), - ] - assert sorted(result) == sorted(expected) - - def test_filter_files_with_exclusion_suffixes(self): - """Test filtering files with exclusion suffixes.""" - # Setup mock filesystem - mock_fs = MagicMock() - - # Create mock file infos - file_info1 = MagicMock() - file_info1.type = pa_fs.FileType.File - file_info1.path = "bucket/model/model.bin" - - file_info2 = MagicMock() - file_info2.type = pa_fs.FileType.File - file_info2.path = "bucket/model/config.json" - - file_info3 = MagicMock() - file_info3.type = pa_fs.FileType.File - file_info3.path = "bucket/model/temp.tmp" - - file_info4 = MagicMock() - file_info4.type = pa_fs.FileType.File - file_info4.path = "bucket/model/log.txt" - - mock_fs.get_file_info.return_value = [ - file_info1, - file_info2, - file_info3, - file_info4, - ] - - # Test filtering with exclusion suffixes - result = CloudFileSystem._filter_files( - fs=mock_fs, - source_path="bucket/model", - destination_path="/local/dest", - suffixes_to_exclude=[".tmp", ".txt"], - ) - - # Should exclude files ending with .tmp or .txt - expected = [ - ("bucket/model/model.bin", "/local/dest/model.bin"), - ("bucket/model/config.json", "/local/dest/config.json"), - ] - assert sorted(result) == sorted(expected) - - def test_filter_files_with_both_filters(self): - """Test filtering files with both inclusion and exclusion filters.""" - # Setup mock filesystem - mock_fs = MagicMock() - - # Create mock file infos - file_info1 = MagicMock() - file_info1.type = pa_fs.FileType.File - file_info1.path = "bucket/model/config.json" - - file_info2 = MagicMock() - file_info2.type = pa_fs.FileType.File - file_info2.path = "bucket/model/config.tmp" - - file_info3 = MagicMock() - file_info3.type = pa_fs.FileType.File - file_info3.path = "bucket/model/weights.bin" - - file_info4 = MagicMock() - file_info4.type = pa_fs.FileType.File - file_info4.path = "bucket/model/tokenizer.json" - - mock_fs.get_file_info.return_value = [ - file_info1, - file_info2, - file_info3, - file_info4, - ] - - # Test filtering with both inclusion and exclusion - result = CloudFileSystem._filter_files( - fs=mock_fs, - source_path="bucket/model", - destination_path="/local/dest", - substrings_to_include=["config", "tokenizer"], - suffixes_to_exclude=[".tmp"], - ) - - # Should include files with "config" or "tokenizer" but exclude .tmp files - expected = [ - ("bucket/model/config.json", "/local/dest/config.json"), - ("bucket/model/tokenizer.json", "/local/dest/tokenizer.json"), - ] - assert sorted(result) == sorted(expected) - - -class TestCloudFileSystemAzureSupport: - """Tests for Azure/ABFSS support in CloudFileSystem.""" - - @patch("adlfs.AzureBlobFileSystem") - @patch("azure.identity.DefaultAzureCredential") - @patch("pyarrow.fs.PyFileSystem") - @patch("pyarrow.fs.FSSpecHandler") - def test_get_fs_and_path_abfss( - self, mock_handler, mock_pyfs, mock_cred, mock_adlfs - ): - """Test getting ABFSS filesystem and path.""" - mock_adlfs_instance = MagicMock() - mock_adlfs.return_value = mock_adlfs_instance - mock_pyfs_instance = MagicMock() - mock_pyfs.return_value = mock_pyfs_instance - - fs, path = CloudFileSystem.get_fs_and_path( - "abfss://container@account.dfs.core.windows.net/path/to/file" - ) - - assert fs == mock_pyfs_instance - assert path == "container/path/to/file" - - # Verify the adlfs filesystem was created with correct parameters - mock_adlfs.assert_called_once_with( - account_name="account", credential=mock_cred.return_value - ) - mock_handler.assert_called_once_with(mock_adlfs_instance) - mock_pyfs.assert_called_once_with(mock_handler.return_value) - - @patch("adlfs.AzureBlobFileSystem") - @patch("azure.identity.DefaultAzureCredential") - @patch("pyarrow.fs.PyFileSystem") - @patch("pyarrow.fs.FSSpecHandler") - def test_get_fs_and_path_azure( - self, mock_handler, mock_pyfs, mock_cred, mock_adlfs - ): - """Test getting Azure filesystem and path.""" - mock_adlfs_instance = MagicMock() - mock_adlfs.return_value = mock_adlfs_instance - mock_pyfs_instance = MagicMock() - mock_pyfs.return_value = mock_pyfs_instance - - fs, path = CloudFileSystem.get_fs_and_path( - "azure://container@account.blob.core.windows.net/path/to/file" - ) - - assert fs == mock_pyfs_instance - assert path == "container/path/to/file" - - # Verify the adlfs filesystem was created with correct parameters - mock_adlfs.assert_called_once_with( - account_name="account", credential=mock_cred.return_value - ) - - def test_abfss_uri_validation(self): - """Test ABFSS URI validation.""" - # Test valid URIs - valid_uris = [ - "abfss://container@account.dfs.core.windows.net/path", - "abfss://my-container@myaccount.dfs.core.windows.net/deep/nested/path", - ] - - for uri in valid_uris: - with patch("adlfs.AzureBlobFileSystem"), patch( - "azure.identity.DefaultAzureCredential" - ), patch("pyarrow.fs.PyFileSystem"), patch("pyarrow.fs.FSSpecHandler"): - # Should not raise an exception - CloudFileSystem._create_abfss_filesystem(uri) - - # Test invalid URIs - invalid_uris = [ - "abfss://container", # Missing @account - "abfss://@account.dfs.core.windows.net/path", # Empty container - "abfss://container@account.wrong.domain/path", # Wrong domain - "abfss://container@.dfs.core.windows.net/path", # Empty account - "abfss://container@account.dfs.core.windows.net", # No path (but this is actually valid) - ] - - for uri in invalid_uris[:-1]: # Skip the last one as it's actually valid - with pytest.raises(ValueError): - CloudFileSystem._create_abfss_filesystem(uri) - - def test_azure_uri_validation(self): - """Test Azure URI validation.""" - # Test valid URIs - valid_uris = [ - "azure://container@account.blob.core.windows.net/path", - "azure://container@account.dfs.core.windows.net/path", - "azure://my-container@myaccount.blob.core.windows.net/deep/nested/path", - ] - - for uri in valid_uris: - with patch("adlfs.AzureBlobFileSystem"), patch( - "azure.identity.DefaultAzureCredential" - ), patch("pyarrow.fs.PyFileSystem"), patch("pyarrow.fs.FSSpecHandler"): - # Should not raise an exception - CloudFileSystem._create_azure_filesystem(uri) - - # Test invalid URIs - invalid_uris = [ - "azure://container", # Missing @account - "azure://@account.blob.core.windows.net/path", # Empty container - "azure://container@account.wrong.domain/path", # Wrong domain - "azure://container@.blob.core.windows.net/path", # Empty account - ] - - for uri in invalid_uris: - with pytest.raises(ValueError): - CloudFileSystem._create_azure_filesystem(uri) - - def test_abfss_import_error(self): - """Test ImportError when adlfs is not available.""" - with patch( - "builtins.__import__", side_effect=ImportError("No module named 'adlfs'") - ): - with pytest.raises( - ImportError, match="You must `pip install adlfs azure-identity`" - ): - CloudFileSystem._create_abfss_filesystem( - "abfss://container@account.dfs.core.windows.net/path" - ) - - def test_azure_import_error(self): - """Test ImportError when adlfs is not available for Azure.""" - with patch( - "builtins.__import__", side_effect=ImportError("No module named 'adlfs'") - ): - with pytest.raises( - ImportError, match="You must `pip install adlfs azure-identity`" - ): - CloudFileSystem._create_azure_filesystem( - "azure://container@account.blob.core.windows.net/path" - ) - - @patch("adlfs.AzureBlobFileSystem") - @patch("azure.identity.DefaultAzureCredential") - @patch("pyarrow.fs.PyFileSystem") - @patch("pyarrow.fs.FSSpecHandler") - def test_abfss_anonymous_access_ignored( - self, mock_handler, mock_pyfs, mock_cred, mock_adlfs - ): - """Test that anonymous access pattern is ignored for ABFSS URIs.""" - mock_adlfs_instance = MagicMock() - mock_adlfs.return_value = mock_adlfs_instance - mock_pyfs_instance = MagicMock() - mock_pyfs.return_value = mock_pyfs_instance - - # ABFSS URI with @ symbol should not trigger anonymous access logic - fs, path = CloudFileSystem.get_fs_and_path( - "abfss://container@account.dfs.core.windows.net/path" - ) - - assert fs == mock_pyfs_instance - assert path == "container/path" - - # Verify that DefaultAzureCredential was used, not anonymous access - mock_cred.assert_called_once() - mock_adlfs.assert_called_once_with( - account_name="account", credential=mock_cred.return_value - ) - - @patch("adlfs.AzureBlobFileSystem") - @patch("azure.identity.DefaultAzureCredential") - @patch("pyarrow.fs.PyFileSystem") - @patch("pyarrow.fs.FSSpecHandler") - def test_get_file_abfss(self, mock_handler, mock_pyfs, mock_cred, mock_adlfs): - """Test getting a file from ABFSS storage.""" - # Setup mock filesystem and file - mock_adlfs_instance = MagicMock() - mock_adlfs.return_value = mock_adlfs_instance - mock_fs = MagicMock() - mock_pyfs.return_value = mock_fs - - # Mock file content and info - mock_file = MagicMock() - mock_file.read.return_value = b"test abfss content" - mock_fs.open_input_file.return_value.__enter__.return_value = mock_file - mock_fs.get_file_info.return_value.type = pa_fs.FileType.File - - # Test getting file as string (default) - content = CloudFileSystem.get_file( - "abfss://container@account.dfs.core.windows.net/test.txt" - ) - assert content == "test abfss content" - - # Verify the correct path was used - mock_fs.get_file_info.assert_called_with("container/test.txt") - mock_fs.open_input_file.assert_called_with("container/test.txt") - - @patch("adlfs.AzureBlobFileSystem") - @patch("azure.identity.DefaultAzureCredential") - @patch("pyarrow.fs.PyFileSystem") - @patch("pyarrow.fs.FSSpecHandler") - def test_list_subfolders_abfss( - self, mock_handler, mock_pyfs, mock_cred, mock_adlfs - ): - """Test listing subfolders in ABFSS storage.""" - # Setup mock filesystem - mock_adlfs_instance = MagicMock() - mock_adlfs.return_value = mock_adlfs_instance - mock_fs = MagicMock() - mock_pyfs.return_value = mock_fs - - # Create mock file infos for directory listing - dir1 = MagicMock() - dir1.type = pa_fs.FileType.Directory - dir1.path = "container/parent/subdir1" - - dir2 = MagicMock() - dir2.type = pa_fs.FileType.Directory - dir2.path = "container/parent/subdir2" - - file1 = MagicMock() - file1.type = pa_fs.FileType.File - file1.path = "container/parent/file.txt" - - mock_fs.get_file_info.return_value = [dir1, dir2, file1] - - # Test listing subfolders - folders = CloudFileSystem.list_subfolders( - "abfss://container@account.dfs.core.windows.net/parent" - ) - assert sorted(folders) == ["subdir1", "subdir2"] - - # Verify the correct path was used - mock_fs.get_file_info.assert_called_once() - call_args = mock_fs.get_file_info.call_args[0][0] - assert call_args.base_dir == "container/parent/" - assert call_args.recursive is False - - -if __name__ == "__main__": - sys.exit(pytest.main(["-v", __file__])) From e29427f09dc2836ce726b5c8c78f44233d454a86 Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Fri, 7 Nov 2025 17:04:58 -0800 Subject: [PATCH 3/8] s3 tests Signed-off-by: ahao-anyscale --- .../utils/cloud_filesystem/s3_filesystem.py | 9 +- .../tests/common/cloud/test_s3_filesystem.py | 327 ++++++++++++++++++ 2 files changed, 332 insertions(+), 4 deletions(-) create mode 100644 python/ray/llm/tests/common/cloud/test_s3_filesystem.py diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py index b1fe4c7bdca6..15fd05f5e2f3 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py @@ -126,13 +126,14 @@ def list_subfolders(folder_uri: str) -> List[str]: subfolders = [] for line in result.stdout.strip().split("\n"): - if not line.strip(): + stripped_line = line.strip() + if not stripped_line: continue # AWS CLI ls output format: "PRE folder_name/" or "timestamp size key" - # We're looking for lines starting with "PRE" (prefixes/directories) - if line.startswith("PRE"): + # We're looking for lines containing "PRE" (prefixes/directories) + if stripped_line.startswith("PRE"): # Extract folder name: "PRE folder_name/" -> "folder_name" - folder_name = line.split()[-1].rstrip("/") + folder_name = stripped_line.split()[-1].rstrip("/") subfolders.append(folder_name) return subfolders diff --git a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py new file mode 100644 index 000000000000..27c8c2e4619b --- /dev/null +++ b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py @@ -0,0 +1,327 @@ +"""Tests for S3FileSystem class.""" + +import json +import os +import subprocess +import tempfile +from unittest.mock import MagicMock, patch + +import pytest + +from ray.llm._internal.common.utils.cloud_filesystem.s3_filesystem import ( + S3FileSystem, +) + + +class TestS3FileSystem: + """Tests for the S3FileSystem class.""" + + @patch("os.unlink") + @patch("os.path.exists") + @patch("builtins.open", create=True) + @patch("subprocess.run") + @patch("tempfile.NamedTemporaryFile") + @pytest.mark.parametrize( + "decode_as_utf_8,expected_content,expected_mode", + [ + (True, "test file content", "r"), + (False, b"test file content", "rb"), + ], + ) + def test_get_file( + self, + mock_tempfile, + mock_run, + mock_open, + mock_exists, + mock_unlink, + decode_as_utf_8, + expected_content, + expected_mode, + ): + """Test getting a file from S3 as string or bytes.""" + # Setup mock tempfile + mock_tmp_file = MagicMock() + mock_tmp_file.name = "/tmp/test_file_123" + mock_tmp_file.__enter__.return_value = mock_tmp_file + mock_tempfile.return_value = mock_tmp_file + + # Setup mock subprocess result + mock_result = MagicMock() + mock_result.returncode = 0 + mock_run.return_value = mock_result + + # Setup mock file reading + mock_file = MagicMock() + mock_file.read.return_value = expected_content + mock_open.return_value.__enter__.return_value = mock_file + mock_exists.return_value = True + + # Test getting file + content = S3FileSystem.get_file( + "s3://bucket/test.txt", decode_as_utf_8=decode_as_utf_8 + ) + + assert content == expected_content + mock_open.assert_called_once_with("/tmp/test_file_123", expected_mode) + + @patch("subprocess.run") + @pytest.mark.parametrize( + "stderr", + [ + "An error occurred (NoSuchKey) when calling the GetObject operation", + "The file does not exist", + ], + ) + def test_get_file_not_found(self, mock_run, stderr): + """Test getting a non-existent file from S3.""" + mock_run.side_effect = subprocess.CalledProcessError( + returncode=1, cmd=["aws", "s3", "cp"], stderr=stderr + ) + assert S3FileSystem.get_file("s3://bucket/nonexistent.txt") is None + + @patch("subprocess.run") + def test_get_file_invalid_uri(self, mock_run): + """Test getting a file with invalid URI.""" + with pytest.raises(ValueError, match="Invalid S3 URI"): + S3FileSystem.get_file("gs://bucket/test.txt") + + @patch("subprocess.run") + @pytest.mark.parametrize( + "uri,expected_path", + [ + ("s3://bucket/parent", "s3://bucket/parent/"), + ("s3://bucket/parent/", "s3://bucket/parent/"), + ], + ) + def test_list_subfolders(self, mock_run, uri, expected_path): + """Test listing subfolders in S3.""" + mock_result = MagicMock() + mock_result.returncode = 0 + mock_result.stdout = ( + "PRE folder1/\nPRE folder2/\n2024-01-01 12:00:00 12345 file.txt\n" + ) + mock_run.return_value = mock_result + + folders = S3FileSystem.list_subfolders(uri) + assert sorted(folders) == ["folder1", "folder2"] + assert mock_run.call_args[0][0][3] == expected_path + + @patch("subprocess.run") + def test_list_subfolders_exception(self, mock_run): + """Test listing subfolders when command fails.""" + mock_run.side_effect = Exception("Network error") + assert S3FileSystem.list_subfolders("s3://bucket/parent") == [] + + @patch("subprocess.run") + def test_list_subfolders_invalid_uri(self, mock_run): + """Test listing subfolders with invalid URI.""" + with pytest.raises(ValueError, match="Invalid S3 URI"): + S3FileSystem.list_subfolders("gs://bucket/parent") + + @patch("subprocess.run") + @pytest.mark.parametrize( + "uri,expected_path", + [ + ("s3://bucket/dir", "s3://bucket/dir/"), + ("s3://bucket/dir/", "s3://bucket/dir/"), + ], + ) + def test_download_files(self, mock_run, uri, expected_path): + """Test downloading files from S3.""" + mock_result = MagicMock() + mock_result.returncode = 0 + mock_run.return_value = mock_result + + with tempfile.TemporaryDirectory() as tempdir: + S3FileSystem.download_files(tempdir, uri) + call_args = mock_run.call_args[0][0] + assert call_args[3] == expected_path + assert "--recursive" in call_args + + @patch("subprocess.run") + def test_download_files_with_filters(self, mock_run): + """Test downloading files with inclusion and exclusion filters.""" + mock_result = MagicMock() + mock_result.returncode = 0 + mock_run.return_value = mock_result + + with tempfile.TemporaryDirectory() as tempdir: + S3FileSystem.download_files( + tempdir, + "s3://bucket/dir", + substrings_to_include=["config", "tokenizer"], + suffixes_to_exclude=[".tmp", "*.txt"], + ) + + call_args = mock_run.call_args[0][0] + assert "--exclude" in call_args + assert "--include" in call_args + assert "*config*" in call_args + assert "*tokenizer*" in call_args + assert "*.tmp" in call_args + assert "*.txt" in call_args + + @patch("subprocess.run") + def test_download_files_exception(self, mock_run): + """Test downloading files when command fails.""" + mock_run.side_effect = Exception("Network error") + with tempfile.TemporaryDirectory() as tempdir: + with pytest.raises(Exception, match="Network error"): + S3FileSystem.download_files(tempdir, "s3://bucket/dir") + + @patch("subprocess.run") + def test_download_files_invalid_uri(self, mock_run): + """Test downloading files with invalid URI.""" + with tempfile.TemporaryDirectory() as tempdir: + with pytest.raises(ValueError, match="Invalid S3 URI"): + S3FileSystem.download_files(tempdir, "gs://bucket/dir") + + @patch("subprocess.run") + @pytest.mark.parametrize( + "uri,expected_path", + [ + ("s3://bucket/dir", "s3://bucket/dir/"), + ("s3://bucket/dir/", "s3://bucket/dir/"), + ], + ) + def test_upload_files(self, mock_run, uri, expected_path): + """Test uploading files to S3.""" + mock_result = MagicMock() + mock_result.returncode = 0 + mock_run.return_value = mock_result + + with tempfile.TemporaryDirectory() as tempdir: + S3FileSystem.upload_files(tempdir, uri) + call_args = mock_run.call_args[0][0] + assert call_args[4] == expected_path + assert "--recursive" in call_args + + @patch("subprocess.run") + def test_upload_files_exception(self, mock_run): + """Test uploading files when command fails.""" + mock_run.side_effect = Exception("Network error") + with tempfile.TemporaryDirectory() as tempdir: + with pytest.raises(Exception, match="Network error"): + S3FileSystem.upload_files(tempdir, "s3://bucket/dir") + + @patch("subprocess.run") + def test_upload_files_invalid_uri(self, mock_run): + """Test uploading files with invalid URI.""" + with tempfile.TemporaryDirectory() as tempdir: + with pytest.raises(ValueError, match="Invalid S3 URI"): + S3FileSystem.upload_files(tempdir, "gs://bucket/dir") + + @patch("subprocess.run") + def test_run_command_success(self, mock_run): + """Test _run_command with successful execution.""" + mock_result = MagicMock() + mock_result.returncode = 0 + mock_run.return_value = mock_result + result = S3FileSystem._run_command(["aws", "s3", "ls"]) + assert result == mock_result + mock_run.assert_called_once_with( + ["aws", "s3", "ls"], capture_output=True, text=True, check=True + ) + + @patch("subprocess.run") + def test_run_command_file_not_found(self, mock_run): + """Test _run_command when command is not found.""" + mock_run.side_effect = FileNotFoundError() + with pytest.raises(FileNotFoundError, match="is not installed"): + S3FileSystem._run_command(["nonexistent", "command"]) + + @patch("subprocess.run") + def test_run_command_called_process_error(self, mock_run): + """Test _run_command when command fails.""" + mock_run.side_effect = subprocess.CalledProcessError( + returncode=1, cmd=["aws", "s3", "cp"], stderr="Access Denied" + ) + with pytest.raises(subprocess.CalledProcessError): + S3FileSystem._run_command(["aws", "s3", "cp", "s3://bucket/file", "/tmp"]) + + +class TestS3FileSystemIntegration: + """Integration tests for S3FileSystem (requires actual S3 access).""" + + def test_list_subfolders_real_s3(self): + """Test listing subfolders from real S3 bucket.""" + # Test listing subfolders in the parent directory which has actual subfolders + folders = S3FileSystem.list_subfolders("s3://air-example-data/rayllm-ossci/") + # Verify we get expected subfolders + assert isinstance(folders, list) + assert "meta-Llama-3.2-1B-Instruct" in folders + assert len(folders) > 0 + + def test_get_file_real_s3(self): + """Test getting a file from real S3 bucket.""" + # Test getting a small config file + content = S3FileSystem.get_file( + "s3://air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/config.json" + ) + assert content is not None + assert isinstance(content, str) + # Verify it's valid JSON + config = json.loads(content) + assert "model_type" in config or "vocab_size" in config + + def test_download_files_with_exclusion(self): + """Test downloading files with exclusion filter (exclude safetensors files).""" + with tempfile.TemporaryDirectory() as tempdir: + # Download files excluding safetensors + S3FileSystem.download_files( + tempdir, + "s3://air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/", + suffixes_to_exclude=[".safetensors"], + ) + + # Get list of downloaded files + downloaded_files = set() + for root, dirs, files in os.walk(tempdir): + for file in files: + rel_path = os.path.relpath(os.path.join(root, file), tempdir) + downloaded_files.add(rel_path) + + # Verify safetensors file is excluded + assert ( + "model.safetensors" not in downloaded_files + ), "safetensors file should be excluded" + + # Verify other files are downloaded + assert "config.json" in downloaded_files + assert "tokenizer.json" in downloaded_files + assert len(downloaded_files) > 0 + + def test_download_files_with_inclusion(self): + """Test downloading files with inclusion filter (include only .json files).""" + with tempfile.TemporaryDirectory() as tempdir: + # Download only .json files + S3FileSystem.download_files( + tempdir, + "s3://air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/", + substrings_to_include=[".json"], + ) + + # Get list of downloaded files + downloaded_files = set() + for root, dirs, files in os.walk(tempdir): + for file in files: + rel_path = os.path.relpath(os.path.join(root, file), tempdir) + downloaded_files.add(rel_path) + + # Verify only .json files are downloaded + expected_json_files = { + "config.json", + "generation_config.json", + "special_tokens_map.json", + "tokenizer.json", + "tokenizer_config.json", + } + assert ( + downloaded_files == expected_json_files + ), f"Expected {expected_json_files}, got {downloaded_files}" + + # Verify non-json files are excluded + assert "model.safetensors" not in downloaded_files + assert "README.md" not in downloaded_files + assert "LICENSE.txt" not in downloaded_files From 33030172a3fac58acb7f01fc0cf39a8a6d5e9a95 Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Mon, 10 Nov 2025 10:30:27 -0800 Subject: [PATCH 4/8] fix tests Signed-off-by: ahao-anyscale --- python/ray/llm/tests/common/cloud/__init__.py | 1 - python/ray/llm/tests/common/cloud/test_cloud_filesystem.py | 7 +++++++ python/ray/llm/tests/common/cloud/test_mirror_config.py | 6 ++++++ .../ray/llm/tests/common/cloud/test_pyarrow_filesystem.py | 5 +++++ python/ray/llm/tests/common/cloud/test_s3_filesystem.py | 5 +++++ python/ray/llm/tests/common/cloud/test_utils.py | 5 +++++ 6 files changed, 28 insertions(+), 1 deletion(-) delete mode 100644 python/ray/llm/tests/common/cloud/__init__.py diff --git a/python/ray/llm/tests/common/cloud/__init__.py b/python/ray/llm/tests/common/cloud/__init__.py deleted file mode 100644 index b189f6d32c9a..000000000000 --- a/python/ray/llm/tests/common/cloud/__init__.py +++ /dev/null @@ -1 +0,0 @@ -"""Tests for cloud filesystem functionality.""" diff --git a/python/ray/llm/tests/common/cloud/test_cloud_filesystem.py b/python/ray/llm/tests/common/cloud/test_cloud_filesystem.py index 850e569d56f9..0ed3ff662c42 100644 --- a/python/ray/llm/tests/common/cloud/test_cloud_filesystem.py +++ b/python/ray/llm/tests/common/cloud/test_cloud_filesystem.py @@ -1,9 +1,12 @@ """Tests for CloudFileSystem class.""" import os +import sys import tempfile from unittest.mock import patch +import pytest + from ray.llm._internal.common.utils.cloud_utils import CloudFileSystem @@ -77,3 +80,7 @@ def test_upload_model(self, mock_gcs_filesystem): assert model_rev_path in call_paths assert "gs://bucket/model" in call_uris assert "gs://bucket/model/hash" in call_uris + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/llm/tests/common/cloud/test_mirror_config.py b/python/ray/llm/tests/common/cloud/test_mirror_config.py index ac55aa52cbe1..08941e5c7a43 100644 --- a/python/ray/llm/tests/common/cloud/test_mirror_config.py +++ b/python/ray/llm/tests/common/cloud/test_mirror_config.py @@ -1,5 +1,7 @@ """Tests for mirror config classes.""" +import sys + import pytest from ray.llm._internal.common.utils.cloud_utils import ( @@ -156,3 +158,7 @@ def test_optional_fields(self): ) assert config.max_total_tokens == 1000 assert config.sync_args == ["--exclude", "*.tmp"] + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py b/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py index 57604033fcb2..876b37ae7a2c 100644 --- a/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py +++ b/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py @@ -1,6 +1,7 @@ """Tests for PyArrowFileSystem class.""" import os +import sys from unittest.mock import ANY, MagicMock, patch import pyarrow.fs as pa_fs @@ -542,3 +543,7 @@ def test_list_subfolders_abfss( call_args = mock_fs.get_file_info.call_args[0][0] assert call_args.base_dir == "container/parent/" assert call_args.recursive is False + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py index 27c8c2e4619b..478a4b130e5f 100644 --- a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py +++ b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py @@ -3,6 +3,7 @@ import json import os import subprocess +import sys import tempfile from unittest.mock import MagicMock, patch @@ -325,3 +326,7 @@ def test_download_files_with_inclusion(self): assert "model.safetensors" not in downloaded_files assert "README.md" not in downloaded_files assert "LICENSE.txt" not in downloaded_files + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) diff --git a/python/ray/llm/tests/common/cloud/test_utils.py b/python/ray/llm/tests/common/cloud/test_utils.py index 3b8280d3d5bc..e07239e9f5b6 100644 --- a/python/ray/llm/tests/common/cloud/test_utils.py +++ b/python/ray/llm/tests/common/cloud/test_utils.py @@ -1,6 +1,7 @@ """Utility tests for cloud functionality.""" import asyncio +import sys import pytest @@ -313,3 +314,7 @@ def test_local_paths(self): assert is_remote_path("./relative/path") is False assert is_remote_path("file:///local/path") is False assert is_remote_path("http://example.com") is False + + +if __name__ == "__main__": + sys.exit(pytest.main(["-v", __file__])) From 401db956d954300a76ea8832f2e7a44b69f05bf1 Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Mon, 10 Nov 2025 16:18:40 -0800 Subject: [PATCH 5/8] uri parse Signed-off-by: ahao-anyscale --- .../utils/cloud_filesystem/s3_filesystem.py | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py index 15fd05f5e2f3..ef1ab88e1739 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py @@ -6,6 +6,7 @@ """ import os +import re import subprocess import tempfile from typing import List, Optional, Union @@ -24,6 +25,10 @@ class S3FileSystem(BaseCloudFileSystem): compared to PyArrow-based implementations, especially for large files. """ + @staticmethod + def _parse_uri(uri: str): + return re.sub(r"^(s3://)[^/@]+@", r"\1", uri) + @staticmethod def _run_command(cmd: List[str]) -> subprocess.CompletedProcess: """Run a command and handle errors. @@ -68,8 +73,7 @@ def get_file( Returns: File contents as string or bytes, or None if file doesn't exist """ - if not object_uri.startswith("s3://"): - raise ValueError(f"Invalid S3 URI: {object_uri}") + object_uri = S3FileSystem._parse_uri(object_uri) try: # Create a temporary file to download to @@ -113,8 +117,7 @@ def list_subfolders(folder_uri: str) -> List[str]: Returns: List of subfolder names (without trailing slashes) """ - if not folder_uri.startswith("s3://"): - raise ValueError(f"Invalid S3 URI: {folder_uri}") + folder_uri = S3FileSystem._parse_uri(folder_uri) # Ensure that the folder_uri has a trailing slash. folder_uri = f"{folder_uri.rstrip('/')}/" @@ -156,6 +159,8 @@ def download_files( substrings_to_include: Only include files containing these substrings suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) """ + bucket_uri = S3FileSystem._parse_uri(bucket_uri) + if not bucket_uri.startswith("s3://"): raise ValueError(f"Invalid S3 URI: {bucket_uri}") @@ -215,8 +220,8 @@ def upload_files( local_path: The local path of the files to upload. bucket_uri: The bucket uri to upload the files to, must start with `s3://`. """ - if not bucket_uri.startswith("s3://"): - raise ValueError(f"Invalid S3 URI: {bucket_uri}") + + bucket_uri = S3FileSystem._parse_uri(bucket_uri) try: # Ensure bucket_uri has trailing slash for directory upload From 63c9e3e3935939e308e338bab8a7e0f236ded422 Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Wed, 12 Nov 2025 14:59:53 -0800 Subject: [PATCH 6/8] use boto instead of cli, fix tests Signed-off-by: ahao-anyscale --- .../cloud_filesystem/pyarrow_filesystem.py | 50 +-- .../utils/cloud_filesystem/s3_filesystem.py | 417 ++++++++++++------ .../llm/_internal/common/utils/cloud_utils.py | 6 +- .../common/cloud/test_pyarrow_filesystem.py | 89 +++- .../tests/common/cloud/test_s3_filesystem.py | 6 - 5 files changed, 360 insertions(+), 208 deletions(-) diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py index d38a6ba96981..64280714429d 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py @@ -37,6 +37,10 @@ def get_fs_and_path(object_uri: str) -> Tuple[pa_fs.FileSystem, str]: Returns: Tuple of (filesystem, path) """ + + if object_uri.startswith("pyarrow-"): + object_uri = object_uri[8:] + anonymous = False # Check for anonymous access pattern (only for S3/GCS) # e.g. s3://anonymous@bucket/path @@ -277,52 +281,6 @@ def list_subfolders(folder_uri: str) -> List[str]: logger.info(f"Error listing subfolders in {folder_uri}: {e}") return [] - # @staticmethod - # def download_files( - # path: str, - # bucket_uri: str, - # substrings_to_include: Optional[List[str]] = None, - # suffixes_to_exclude: Optional[List[str]] = None, - # ) -> None: - # """Download files from cloud storage to a local directory. - - # Args: - # path: Local directory where files will be downloaded - # bucket_uri: URI of cloud directory - # substrings_to_include: Only include files containing these substrings - # suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) - # """ - # try: - # fs, source_path = PyArrowFileSystem.get_fs_and_path(bucket_uri) - - # # Ensure the destination directory exists - # os.makedirs(path, exist_ok=True) - - # # Get filtered files to download - # files_to_download = PyArrowFileSystem._filter_files( - # fs, source_path, path, substrings_to_include, suffixes_to_exclude - # ) - - # # Download each file using PyArrow's copy_files API - # local_fs = pa_fs.LocalFileSystem() - # for source_file_path, dest_file_path in files_to_download: - # # Create destination directory if needed - # dest_dir = os.path.dirname(dest_file_path) - # if dest_dir: - # os.makedirs(dest_dir, exist_ok=True) - - # # Download the file using PyArrow's copy_files API - # pa_fs.copy_files( - # source=source_file_path, - # destination=dest_file_path, - # source_filesystem=fs, - # destination_filesystem=local_fs, - # ) - - # except Exception as e: - # logger.exception(f"Error downloading files from {bucket_uri}: {e}") - # raise - @staticmethod def download_files( path: str, diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py index ef1ab88e1739..688eea9c8cab 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py @@ -1,16 +1,19 @@ -"""S3-specific filesystem implementation using AWS CLI. +"""S3-specific filesystem implementation using boto3. -This module provides an S3-specific implementation that uses AWS CLI for optimal -performance. This leverages native AWS tools for significantly faster transfers -compared to PyArrow-based implementations. +This module provides an S3-specific implementation that uses boto3 (AWS SDK for Python) +for reliable and efficient S3 operations. """ import os import re -import subprocess -import tempfile +from concurrent.futures import ThreadPoolExecutor, as_completed +from pathlib import Path from typing import List, Optional, Union +import boto3 +from botocore.client import BaseClient +from botocore.config import Config + from ray.llm._internal.common.observability.logging import get_logger from ray.llm._internal.common.utils.cloud_filesystem.base import BaseCloudFileSystem @@ -18,47 +21,63 @@ class S3FileSystem(BaseCloudFileSystem): - """S3-specific implementation of cloud filesystem operations using AWS CLI. + """S3-specific implementation of cloud filesystem operations using boto3. - This implementation uses AWS CLI (aws s3 cp, aws s3 ls) for optimal performance - when working with S3 storage. It provides significantly faster transfers - compared to PyArrow-based implementations, especially for large files. + This implementation uses boto3 (AWS SDK for Python) for reliable and efficient + operations with S3 storage. """ @staticmethod - def _parse_uri(uri: str): - return re.sub(r"^(s3://)[^/@]+@", r"\1", uri) - - @staticmethod - def _run_command(cmd: List[str]) -> subprocess.CompletedProcess: - """Run a command and handle errors. + def _parse_s3_uri(uri: str) -> tuple[str, str]: + """Parse S3 URI into bucket and key. Args: - cmd: List of command arguments (e.g., ['aws', 's3', 'cp', ...]) + uri: S3 URI (e.g., s3://bucket/path/to/object) Returns: - CompletedProcess object from subprocess.run + Tuple of (bucket_name, key) Raises: - subprocess.CalledProcessError: If the command fails - FileNotFoundError: If the command is not installed + ValueError: If URI is not a valid S3 URI """ - try: - result = subprocess.run( - cmd, - capture_output=True, - text=True, - check=True, - ) - return result - except FileNotFoundError: - raise FileNotFoundError(f"Command '{cmd[0]}' is not installed.") - except subprocess.CalledProcessError as e: - print(f"Command failed: {' '.join(cmd)}") - print(f"Error: {e.stderr}") - logger.error(f"Command failed: {' '.join(cmd)}") - logger.error(f"Error output: {e.stderr}") - raise + # Remove user info if present + uri = re.sub(r"^(s3://)[^/@]+@", r"\1", uri) + + if not uri.startswith("s3://"): + raise ValueError(f"Invalid S3 URI: {uri}") + + # Remove s3:// prefix and split into bucket and key + path = uri[5:] # Remove "s3://" + parts = path.split("/", 1) + bucket = parts[0] + key = parts[1] if len(parts) > 1 else "" + + return bucket, key + + @staticmethod + def _get_s3_client(max_pool_connections: int = 50): + """Create a new S3 client instance with connection pooling. + + Args: + max_pool_connections: Maximum number of connections in the pool. + Should be >= max_workers for optimal performance. + + Returns: + boto3 S3 client with connection pooling configured + """ + # Configure connection pooling for better concurrent performance + config = Config( + max_pool_connections=max_pool_connections, + # Retry configuration for transient failures + retries={ + "max_attempts": 3, + "mode": "adaptive", # Adapts retry behavior based on error type + }, + # TCP keepalive helps with long-running connections + tcp_keepalive=True, + ) + + return boto3.client("s3", config=config) @staticmethod def get_file( @@ -73,39 +92,19 @@ def get_file( Returns: File contents as string or bytes, or None if file doesn't exist """ - object_uri = S3FileSystem._parse_uri(object_uri) - try: - # Create a temporary file to download to - with tempfile.NamedTemporaryFile(delete=False) as tmp_file: - tmp_path = tmp_file.name - - try: - # Download file using AWS CLI - cmd = ["aws", "s3", "cp", object_uri, tmp_path] - S3FileSystem._run_command(cmd) - - # Read the file - mode = "r" if decode_as_utf_8 else "rb" - with open(tmp_path, mode) as f: - body = f.read() - - return body - finally: - # Clean up temporary file - if os.path.exists(tmp_path): - os.unlink(tmp_path) - - except subprocess.CalledProcessError as e: - # Check if file doesn't exist (AWS CLI returns non-zero exit code) - if "NoSuchKey" in e.stderr or "does not exist" in e.stderr.lower(): - logger.info(f"URI {object_uri} does not exist.") - return None - logger.info(f"Error reading {object_uri}: {e.stderr}") - return None + bucket, key = S3FileSystem._parse_s3_uri(object_uri) + s3_client = S3FileSystem._get_s3_client() + + # Download file directly into memory + response = s3_client.get_object(Bucket=bucket, Key=key) + body = response["Body"].read() + + if decode_as_utf_8: + return body.decode("utf-8") + return body except Exception as e: logger.info(f"Error reading {object_uri}: {e}") - return None @staticmethod def list_subfolders(folder_uri: str) -> List[str]: @@ -117,26 +116,28 @@ def list_subfolders(folder_uri: str) -> List[str]: Returns: List of subfolder names (without trailing slashes) """ - folder_uri = S3FileSystem._parse_uri(folder_uri) + try: + bucket, prefix = S3FileSystem._parse_s3_uri(folder_uri) - # Ensure that the folder_uri has a trailing slash. - folder_uri = f"{folder_uri.rstrip('/')}/" + # Ensure that the prefix has a trailing slash + if prefix and not prefix.endswith("/"): + prefix = f"{prefix}/" - try: - # Use AWS CLI to list objects with common prefix - cmd = ["aws", "s3", "ls", folder_uri] - result = S3FileSystem._run_command(cmd) + s3_client = S3FileSystem._get_s3_client() + + # List objects with delimiter to get only immediate subfolders + response = s3_client.list_objects_v2( + Bucket=bucket, Prefix=prefix, Delimiter="/" + ) subfolders = [] - for line in result.stdout.strip().split("\n"): - stripped_line = line.strip() - if not stripped_line: - continue - # AWS CLI ls output format: "PRE folder_name/" or "timestamp size key" - # We're looking for lines containing "PRE" (prefixes/directories) - if stripped_line.startswith("PRE"): - # Extract folder name: "PRE folder_name/" -> "folder_name" - folder_name = stripped_line.split()[-1].rstrip("/") + # CommonPrefixes contains the subdirectories + for common_prefix in response.get("CommonPrefixes", []): + folder_path = common_prefix["Prefix"] + # Extract the folder name from the full prefix + # Remove the parent prefix and trailing slash + folder_name = folder_path[len(prefix) :].rstrip("/") + if folder_name: subfolders.append(folder_name) return subfolders @@ -144,66 +145,191 @@ def list_subfolders(folder_uri: str) -> List[str]: logger.info(f"Error listing subfolders in {folder_uri}: {e}") return [] + @staticmethod + def _calculate_optimal_workers( + num_files: int, total_size: int, default_max: int = 100, default_min: int = 10 + ) -> int: + """Calculate optimal number of workers based on file characteristics. + + Args: + num_files: Number of files to download + total_size: Total size of all files in bytes + default_max: Maximum workers to cap at + default_min: Minimum workers to use + + Returns: + Optimal number of workers between default_min and default_max + """ + if num_files == 0: + return default_min + + avg_file_size = total_size / num_files if total_size > 0 else 0 + + # Strategy: More workers for smaller files, fewer for larger files + if avg_file_size < 1024 * 1024: # < 1MB (small files) + # Use more workers for many small files + workers = min(num_files, default_max) + elif avg_file_size < 10 * 1024 * 1024: # 1-10MB (medium files) + # Use moderate workers + workers = min(num_files // 2, default_max // 2) + else: # > 10MB (large files) + # Use fewer workers since each download is bandwidth-intensive + workers = min(20, num_files) + + # Ensure workers is between min and max + return max(default_min, min(workers, default_max)) + + @staticmethod + def _download_single_file( + s3_client: BaseClient, bucket: str, key: str, local_file_path: str + ) -> tuple[str, bool]: + """Download a single file from S3. + + Args: + s3_client: Shared boto3 S3 client + bucket: S3 bucket name + key: S3 object key + local_file_path: Local path where file will be saved + + Returns: + Tuple of (key, success) + """ + try: + # Create parent directories if needed + os.makedirs(os.path.dirname(local_file_path), exist_ok=True) + + s3_client.download_file(bucket, key, local_file_path) + logger.debug(f"Downloaded {key} to {local_file_path}") + return key, True + except Exception as e: + logger.error(f"Failed to download {key}: {e}") + return key, False + @staticmethod def download_files( path: str, bucket_uri: str, substrings_to_include: Optional[List[str]] = None, suffixes_to_exclude: Optional[List[str]] = None, + max_workers: Optional[int] = None, ) -> None: - """Download files from cloud storage to a local directory. + """Download files from cloud storage to a local directory concurrently. Args: path: Local directory where files will be downloaded bucket_uri: URI of cloud directory substrings_to_include: Only include files containing these substrings suffixes_to_exclude: Exclude certain files from download (e.g .safetensors) + max_workers: Maximum number of concurrent downloads. If None, automatically + calculated based on file count and sizes (min: 10, max: 100) """ - bucket_uri = S3FileSystem._parse_uri(bucket_uri) - - if not bucket_uri.startswith("s3://"): - raise ValueError(f"Invalid S3 URI: {bucket_uri}") - try: + bucket, prefix = S3FileSystem._parse_s3_uri(bucket_uri) + # Ensure the destination directory exists os.makedirs(path, exist_ok=True) - # Ensure bucket_uri has trailing slash for directory listing - source_uri = f"{bucket_uri.rstrip('/')}/" - - # Build AWS CLI command - cmd = ["aws", "s3", "cp", source_uri, path, "--recursive"] - - # AWS CLI filter logic: - # - By default, all files are included - # - --exclude removes files matching the pattern - # - --include adds files matching the pattern (even if excluded) - # - Order matters: filters are processed sequentially - - # If we have include filters, we need to exclude everything first, - # then include only what we want - if substrings_to_include: - # Exclude everything first - cmd.extend(["--exclude", "*"]) - # Then include files matching any of the substring patterns - for substring in substrings_to_include: - # Create wildcard pattern: *substring* matches files containing substring - pattern = f"*{substring}*" - cmd.extend(["--include", pattern]) - - # Add exclude filters (suffixes_to_exclude) - # These will exclude files ending with the specified suffixes - if suffixes_to_exclude: - for suffix in suffixes_to_exclude: - # Ensure suffix starts with * if it doesn't already - if not suffix.startswith("*"): - pattern = f"*{suffix}" - else: - pattern = suffix - cmd.extend(["--exclude", pattern]) - - # Run the download command - S3FileSystem._run_command(cmd) + # Ensure prefix has trailing slash for directory listing + if prefix and not prefix.endswith("/"): + prefix = f"{prefix}/" + + # Create initial client for listing (will recreate with proper pool size later) + s3_client = S3FileSystem._get_s3_client() + + # List all objects in the bucket with the given prefix + paginator = s3_client.get_paginator("list_objects_v2") + pages = paginator.paginate(Bucket=bucket, Prefix=prefix) + + # Collect all files to download and track total size + files_to_download = [] + total_size = 0 + + for page in pages: + for obj in page.get("Contents", []): + key = obj["Key"] + size = obj.get("Size", 0) + + # Skip if it's a directory marker + if key.endswith("/"): + continue + + # Get the relative path (remove the prefix) + relative_path = key[len(prefix) :] + + # Apply include filters + if substrings_to_include: + if not any( + substr in relative_path for substr in substrings_to_include + ): + continue + + # Apply exclude filters + if suffixes_to_exclude: + if any( + relative_path.endswith(suffix.lstrip("*")) + for suffix in suffixes_to_exclude + ): + continue + + # Construct local file path + local_file_path = os.path.join(path, relative_path) + files_to_download.append((bucket, key, local_file_path)) + total_size += size + + # Download files concurrently + if not files_to_download: + logger.info(f"No files matching filters to download from {bucket_uri}") + return + + # Dynamically calculate workers if not provided + if max_workers is None: + max_workers = S3FileSystem._calculate_optimal_workers( + num_files=len(files_to_download), + total_size=total_size, + default_max=100, + default_min=10, + ) + logger.info( + f"Auto-selected {max_workers} workers for {len(files_to_download)} files " + f"({total_size / (1024**2):.1f} MB total)" + ) + + # Create shared client with proper connection pool size for downloads + s3_client = S3FileSystem._get_s3_client( + max_pool_connections=max_workers + 10 + ) + + failed_downloads = [] + + with ThreadPoolExecutor(max_workers=max_workers) as executor: + # Submit all download tasks with shared client + future_to_key = { + executor.submit( + S3FileSystem._download_single_file, + s3_client, # Pass shared client to each worker + bucket, + key, + local_path, + ): key + for bucket, key, local_path in files_to_download + } + + # Process completed downloads + for future in as_completed(future_to_key): + key, success = future.result() + if not success: + failed_downloads.append(key) + + # Report any failures + if failed_downloads: + logger.warning( + f"Failed to download {len(failed_downloads)} files: {failed_downloads[:5]}..." + ) + raise Exception( + f"Failed to download {len(failed_downloads)} out of {len(files_to_download)} files" + ) + + logger.info(f"Successfully downloaded {len(files_to_download)} files") except Exception as e: logger.exception(f"Error downloading files from {bucket_uri}: {e}") @@ -220,18 +346,39 @@ def upload_files( local_path: The local path of the files to upload. bucket_uri: The bucket uri to upload the files to, must start with `s3://`. """ - - bucket_uri = S3FileSystem._parse_uri(bucket_uri) - try: - # Ensure bucket_uri has trailing slash for directory upload - dest_uri = f"{bucket_uri.rstrip('/')}/" - - # Build AWS CLI command for recursive upload - cmd = ["aws", "s3", "cp", local_path, dest_uri, "--recursive"] - - # Run the upload command - S3FileSystem._run_command(cmd) + bucket, prefix = S3FileSystem._parse_s3_uri(bucket_uri) + + # Ensure prefix has trailing slash for directory upload + if prefix and not prefix.endswith("/"): + prefix = f"{prefix}/" + + s3_client = S3FileSystem._get_s3_client() + + local_path_obj = Path(local_path) + + # Walk through the local directory and upload each file + if local_path_obj.is_file(): + # Upload a single file + file_name = local_path_obj.name + s3_key = f"{prefix}{file_name}" if prefix else file_name + s3_client.upload_file(str(local_path_obj), bucket, s3_key) + logger.debug(f"Uploaded {local_path_obj} to s3://{bucket}/{s3_key}") + elif local_path_obj.is_dir(): + # Upload directory recursively + for file_path in local_path_obj.rglob("*"): + if file_path.is_file(): + # Calculate relative path from local_path + relative_path = file_path.relative_to(local_path_obj) + # Construct S3 key + s3_key = f"{prefix}{relative_path.as_posix()}" + # Upload file + s3_client.upload_file(str(file_path), bucket, s3_key) + logger.debug(f"Uploaded {file_path} to s3://{bucket}/{s3_key}") + else: + raise ValueError( + f"Path {local_path} does not exist or is not a file/directory" + ) except Exception as e: logger.exception(f"Error uploading files to {bucket_uri}: {e}") diff --git a/python/ray/llm/_internal/common/utils/cloud_utils.py b/python/ray/llm/_internal/common/utils/cloud_utils.py index df3a3acb9716..1808ec82ec3e 100644 --- a/python/ray/llm/_internal/common/utils/cloud_utils.py +++ b/python/ray/llm/_internal/common/utils/cloud_utils.py @@ -22,6 +22,7 @@ from ray.llm._internal.common.utils.cloud_filesystem import ( AzureFileSystem, GCSFileSystem, + PyArrowFileSystem, S3FileSystem, ) @@ -154,8 +155,9 @@ def _get_provider_fs(bucket_uri: str): Raises: ValueError: If the URI scheme is not supported """ - - if bucket_uri.startswith("s3://"): + if bucket_uri.startswith("pyarrow-"): + return PyArrowFileSystem + elif bucket_uri.startswith("s3://"): return S3FileSystem elif bucket_uri.startswith("gs://"): return GCSFileSystem diff --git a/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py b/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py index 876b37ae7a2c..8e8ea9cfe859 100644 --- a/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py +++ b/python/ray/llm/tests/common/cloud/test_pyarrow_filesystem.py @@ -2,6 +2,7 @@ import os import sys +import tempfile from unittest.mock import ANY, MagicMock, patch import pyarrow.fs as pa_fs @@ -81,10 +82,34 @@ def test_list_subfolders_exception_handling(self, mock_get_fs_and_path): # Verify get_fs_and_path was called mock_get_fs_and_path.assert_called_once_with("gs://bucket/parent/") + @patch("pyarrow.fs.copy_files") @patch("pyarrow.fs.S3FileSystem") - def test_download_files(self, mock_s3fs): - """Test downloading files from cloud storage.""" - import tempfile + def test_download_files_no_filters(self, mock_s3fs, mock_copy_files): + """Test downloading files from cloud storage without filters.""" + + # Setup mock filesystem + mock_fs = MagicMock() + mock_s3fs.return_value = mock_fs + + # Create temp directory for testing + with tempfile.TemporaryDirectory() as tempdir: + # Test downloading files without filters + PyArrowFileSystem.download_files(tempdir, "s3://bucket/dir") + + # Verify copy_files was called with correct arguments + mock_copy_files.assert_called_once_with( + source="bucket/dir", + destination=tempdir, + source_filesystem=mock_fs, + destination_filesystem=ANY, + use_threads=True, + chunk_size=64 * 1024 * 1024, + ) + + @patch("pyarrow.fs.copy_files") + @patch("pyarrow.fs.S3FileSystem") + def test_download_files_with_filters(self, mock_s3fs, mock_copy_files): + """Test downloading files from cloud storage with filters.""" # Setup mock filesystem mock_fs = MagicMock() @@ -97,37 +122,63 @@ def test_download_files(self, mock_s3fs): file_info2 = MagicMock() file_info2.type = pa_fs.FileType.File - file_info2.path = "bucket/dir/subdir/file2.txt" + file_info2.path = "bucket/dir/subdir/file2.json" + + file_info3 = MagicMock() + file_info3.type = pa_fs.FileType.File + file_info3.path = "bucket/dir/file3.tmp" dir_info = MagicMock() dir_info.type = pa_fs.FileType.Directory dir_info.path = "bucket/dir/subdir" - mock_fs.get_file_info.return_value = [file_info1, file_info2, dir_info] - - # Mock file content - mock_file = MagicMock() - mock_file.read.return_value = b"test content" - mock_fs.open_input_file.return_value.__enter__.return_value = mock_file + mock_fs.get_file_info.return_value = [ + file_info1, + file_info2, + file_info3, + dir_info, + ] # Create temp directory for testing with tempfile.TemporaryDirectory() as tempdir: - # Test downloading files - PyArrowFileSystem.download_files(tempdir, "s3://bucket/dir") + # Test downloading files with filters + PyArrowFileSystem.download_files( + tempdir, + "s3://bucket/dir", + substrings_to_include=["file1", "file2"], + suffixes_to_exclude=[".tmp"], + ) + + # Verify copy_files was called for each filtered file + assert mock_copy_files.call_count == 2 + + # Get all calls to copy_files + calls = mock_copy_files.call_args_list + + # Verify the calls (order may vary due to threading) + expected_sources = {"bucket/dir/file1.txt", "bucket/dir/subdir/file2.json"} + expected_dests = { + os.path.join(tempdir, "file1.txt"), + os.path.join(tempdir, "subdir", "file2.json"), + } + + actual_sources = {call.kwargs["source"] for call in calls} + actual_dests = {call.kwargs["destination"] for call in calls} - # Check that files were downloaded correctly - assert os.path.exists(os.path.join(tempdir, "file1.txt")) - assert os.path.exists(os.path.join(tempdir, "subdir", "file2.txt")) + assert actual_sources == expected_sources + assert actual_dests == expected_dests - # Check content of downloaded files - with open(os.path.join(tempdir, "file1.txt"), "rb") as f: - assert f.read() == b"test content" + # Verify all calls have correct filesystem and options + for call in calls: + assert call.kwargs["source_filesystem"] == mock_fs + assert call.kwargs["destination_filesystem"] is not None + assert call.kwargs["use_threads"] is True + assert call.kwargs["chunk_size"] == 64 * 1024 * 1024 @patch("pyarrow.fs.copy_files") @patch("pyarrow.fs.S3FileSystem") def test_upload_files(self, mock_s3fs, mock_copy_files): """Test uploading files to cloud storage.""" - import tempfile # Setup mock filesystem mock_fs = MagicMock() diff --git a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py index 478a4b130e5f..0c03b18499de 100644 --- a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py +++ b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py @@ -81,12 +81,6 @@ def test_get_file_not_found(self, mock_run, stderr): ) assert S3FileSystem.get_file("s3://bucket/nonexistent.txt") is None - @patch("subprocess.run") - def test_get_file_invalid_uri(self, mock_run): - """Test getting a file with invalid URI.""" - with pytest.raises(ValueError, match="Invalid S3 URI"): - S3FileSystem.get_file("gs://bucket/test.txt") - @patch("subprocess.run") @pytest.mark.parametrize( "uri,expected_path", From a0afccbc9727f15643289873c3d58a424d62de9c Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Wed, 12 Nov 2025 16:09:49 -0800 Subject: [PATCH 7/8] fix tests, add anonymous functionality Signed-off-by: ahao-anyscale --- .../cloud_filesystem/pyarrow_filesystem.py | 2 +- .../utils/cloud_filesystem/s3_filesystem.py | 54 +- .../tests/common/cloud/test_s3_filesystem.py | 474 ++++++++++++------ 3 files changed, 360 insertions(+), 170 deletions(-) diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py index 64280714429d..48f6ddbc4fd9 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py @@ -246,7 +246,7 @@ def get_file( body = body.decode("utf-8") return body except Exception as e: - logger.info(f"Error reading {object_uri}: {e}") + logger.warning(f"Error reading {object_uri}: {e}") return None @staticmethod diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py index 688eea9c8cab..f66dda86bb78 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/s3_filesystem.py @@ -5,12 +5,12 @@ """ import os -import re from concurrent.futures import ThreadPoolExecutor, as_completed from pathlib import Path from typing import List, Optional, Union import boto3 +from botocore import UNSIGNED from botocore.client import BaseClient from botocore.config import Config @@ -28,20 +28,23 @@ class S3FileSystem(BaseCloudFileSystem): """ @staticmethod - def _parse_s3_uri(uri: str) -> tuple[str, str]: + def _parse_s3_uri(uri: str) -> tuple[str, str, bool]: """Parse S3 URI into bucket and key. Args: - uri: S3 URI (e.g., s3://bucket/path/to/object) + uri: S3 URI (e.g., s3://bucket/path/to/object or s3://anonymous@bucket/path/to/object) Returns: - Tuple of (bucket_name, key) + Tuple of (bucket_name, key, is_anonymous) Raises: ValueError: If URI is not a valid S3 URI """ - # Remove user info if present - uri = re.sub(r"^(s3://)[^/@]+@", r"\1", uri) + # Check if anonymous@ prefix exists + is_anonymous = False + if uri.startswith("s3://anonymous@"): + is_anonymous = True + uri = uri.replace("s3://anonymous@", "s3://", 1) if not uri.startswith("s3://"): raise ValueError(f"Invalid S3 URI: {uri}") @@ -52,15 +55,16 @@ def _parse_s3_uri(uri: str) -> tuple[str, str]: bucket = parts[0] key = parts[1] if len(parts) > 1 else "" - return bucket, key + return bucket, key, is_anonymous @staticmethod - def _get_s3_client(max_pool_connections: int = 50): + def _get_s3_client(max_pool_connections: int = 50, anonymous: bool = False): """Create a new S3 client instance with connection pooling. Args: max_pool_connections: Maximum number of connections in the pool. Should be >= max_workers for optimal performance. + anonymous: Whether to use anonymous access to S3. Returns: boto3 S3 client with connection pooling configured @@ -75,6 +79,7 @@ def _get_s3_client(max_pool_connections: int = 50): }, # TCP keepalive helps with long-running connections tcp_keepalive=True, + signature_version=UNSIGNED if anonymous else None, ) return boto3.client("s3", config=config) @@ -93,8 +98,8 @@ def get_file( File contents as string or bytes, or None if file doesn't exist """ try: - bucket, key = S3FileSystem._parse_s3_uri(object_uri) - s3_client = S3FileSystem._get_s3_client() + bucket, key, is_anonymous = S3FileSystem._parse_s3_uri(object_uri) + s3_client = S3FileSystem._get_s3_client(anonymous=is_anonymous) # Download file directly into memory response = s3_client.get_object(Bucket=bucket, Key=key) @@ -104,7 +109,7 @@ def get_file( return body.decode("utf-8") return body except Exception as e: - logger.info(f"Error reading {object_uri}: {e}") + logger.error(f"Error reading {object_uri}: {e}") @staticmethod def list_subfolders(folder_uri: str) -> List[str]: @@ -117,13 +122,13 @@ def list_subfolders(folder_uri: str) -> List[str]: List of subfolder names (without trailing slashes) """ try: - bucket, prefix = S3FileSystem._parse_s3_uri(folder_uri) + bucket, prefix, is_anonymous = S3FileSystem._parse_s3_uri(folder_uri) # Ensure that the prefix has a trailing slash if prefix and not prefix.endswith("/"): prefix = f"{prefix}/" - s3_client = S3FileSystem._get_s3_client() + s3_client = S3FileSystem._get_s3_client(anonymous=is_anonymous) # List objects with delimiter to get only immediate subfolders response = s3_client.list_objects_v2( @@ -142,7 +147,7 @@ def list_subfolders(folder_uri: str) -> List[str]: return subfolders except Exception as e: - logger.info(f"Error listing subfolders in {folder_uri}: {e}") + logger.error(f"Error listing subfolders in {folder_uri}: {e}") return [] @staticmethod @@ -224,7 +229,7 @@ def download_files( calculated based on file count and sizes (min: 10, max: 100) """ try: - bucket, prefix = S3FileSystem._parse_s3_uri(bucket_uri) + bucket, prefix, is_anonymous = S3FileSystem._parse_s3_uri(bucket_uri) # Ensure the destination directory exists os.makedirs(path, exist_ok=True) @@ -234,7 +239,7 @@ def download_files( prefix = f"{prefix}/" # Create initial client for listing (will recreate with proper pool size later) - s3_client = S3FileSystem._get_s3_client() + s3_client = S3FileSystem._get_s3_client(anonymous=is_anonymous) # List all objects in the bucket with the given prefix paginator = s3_client.get_paginator("list_objects_v2") @@ -289,14 +294,10 @@ def download_files( default_max=100, default_min=10, ) - logger.info( - f"Auto-selected {max_workers} workers for {len(files_to_download)} files " - f"({total_size / (1024**2):.1f} MB total)" - ) # Create shared client with proper connection pool size for downloads s3_client = S3FileSystem._get_s3_client( - max_pool_connections=max_workers + 10 + max_pool_connections=max_workers + 10, anonymous=is_anonymous ) failed_downloads = [] @@ -322,14 +323,9 @@ def download_files( # Report any failures if failed_downloads: - logger.warning( + logger.error( f"Failed to download {len(failed_downloads)} files: {failed_downloads[:5]}..." ) - raise Exception( - f"Failed to download {len(failed_downloads)} out of {len(files_to_download)} files" - ) - - logger.info(f"Successfully downloaded {len(files_to_download)} files") except Exception as e: logger.exception(f"Error downloading files from {bucket_uri}: {e}") @@ -347,13 +343,13 @@ def upload_files( bucket_uri: The bucket uri to upload the files to, must start with `s3://`. """ try: - bucket, prefix = S3FileSystem._parse_s3_uri(bucket_uri) + bucket, prefix, is_anonymous = S3FileSystem._parse_s3_uri(bucket_uri) # Ensure prefix has trailing slash for directory upload if prefix and not prefix.endswith("/"): prefix = f"{prefix}/" - s3_client = S3FileSystem._get_s3_client() + s3_client = S3FileSystem._get_s3_client(anonymous=is_anonymous) local_path_obj = Path(local_path) diff --git a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py index 0c03b18499de..795eb7a10c90 100644 --- a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py +++ b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py @@ -2,12 +2,12 @@ import json import os -import subprocess import sys import tempfile from unittest.mock import MagicMock, patch import pytest +from botocore.exceptions import ClientError from ray.llm._internal.common.utils.cloud_filesystem.s3_filesystem import ( S3FileSystem, @@ -17,46 +17,33 @@ class TestS3FileSystem: """Tests for the S3FileSystem class.""" - @patch("os.unlink") - @patch("os.path.exists") - @patch("builtins.open", create=True) - @patch("subprocess.run") - @patch("tempfile.NamedTemporaryFile") + @patch("boto3.client") @pytest.mark.parametrize( - "decode_as_utf_8,expected_content,expected_mode", + "decode_as_utf_8,file_content,expected_content", [ - (True, "test file content", "r"), - (False, b"test file content", "rb"), + (True, b"test file content", "test file content"), + (False, b"test file content", b"test file content"), ], ) def test_get_file( self, - mock_tempfile, - mock_run, - mock_open, - mock_exists, - mock_unlink, + mock_boto_client, decode_as_utf_8, + file_content, expected_content, - expected_mode, ): """Test getting a file from S3 as string or bytes.""" - # Setup mock tempfile - mock_tmp_file = MagicMock() - mock_tmp_file.name = "/tmp/test_file_123" - mock_tmp_file.__enter__.return_value = mock_tmp_file - mock_tempfile.return_value = mock_tmp_file - - # Setup mock subprocess result - mock_result = MagicMock() - mock_result.returncode = 0 - mock_run.return_value = mock_result - - # Setup mock file reading - mock_file = MagicMock() - mock_file.read.return_value = expected_content - mock_open.return_value.__enter__.return_value = mock_file - mock_exists.return_value = True + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Mock get_object response + mock_body = MagicMock() + mock_body.read.return_value = file_content + mock_s3_client.get_object.return_value = { + "Body": mock_body, + "ContentLength": len(file_content), + } # Test getting file content = S3FileSystem.get_file( @@ -64,176 +51,383 @@ def test_get_file( ) assert content == expected_content - mock_open.assert_called_once_with("/tmp/test_file_123", expected_mode) + mock_s3_client.get_object.assert_called_once_with( + Bucket="bucket", Key="test.txt" + ) - @patch("subprocess.run") - @pytest.mark.parametrize( - "stderr", - [ - "An error occurred (NoSuchKey) when calling the GetObject operation", - "The file does not exist", - ], - ) - def test_get_file_not_found(self, mock_run, stderr): + @patch("boto3.client") + def test_get_file_not_found(self, mock_boto_client): """Test getting a non-existent file from S3.""" - mock_run.side_effect = subprocess.CalledProcessError( - returncode=1, cmd=["aws", "s3", "cp"], stderr=stderr + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Simulate NoSuchKey error + mock_s3_client.get_object.side_effect = ClientError( + { + "Error": { + "Code": "NoSuchKey", + "Message": "The specified key does not exist.", + } + }, + "GetObject", ) + assert S3FileSystem.get_file("s3://bucket/nonexistent.txt") is None - @patch("subprocess.run") + @patch("boto3.client") + def test_get_file_anonymous(self, mock_boto_client): + """Test getting a file from S3 with anonymous access.""" + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Mock get_object response + mock_body = MagicMock() + mock_body.read.return_value = b"anonymous content" + mock_s3_client.get_object.return_value = { + "Body": mock_body, + } + + # Test getting file with anonymous URI + content = S3FileSystem.get_file("s3://anonymous@bucket/test.txt") + + assert content == "anonymous content" + # Verify anonymous config was used (UNSIGNED signature) + assert mock_boto_client.called + + @patch("boto3.client") @pytest.mark.parametrize( - "uri,expected_path", + "uri,expected_prefix", [ - ("s3://bucket/parent", "s3://bucket/parent/"), - ("s3://bucket/parent/", "s3://bucket/parent/"), + ("s3://bucket/parent", "parent/"), + ("s3://bucket/parent/", "parent/"), ], ) - def test_list_subfolders(self, mock_run, uri, expected_path): + def test_list_subfolders(self, mock_boto_client, uri, expected_prefix): """Test listing subfolders in S3.""" - mock_result = MagicMock() - mock_result.returncode = 0 - mock_result.stdout = ( - "PRE folder1/\nPRE folder2/\n2024-01-01 12:00:00 12345 file.txt\n" - ) - mock_run.return_value = mock_result + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Mock list_objects_v2 response + mock_s3_client.list_objects_v2.return_value = { + "CommonPrefixes": [ + {"Prefix": f"{expected_prefix}folder1/"}, + {"Prefix": f"{expected_prefix}folder2/"}, + ] + } folders = S3FileSystem.list_subfolders(uri) assert sorted(folders) == ["folder1", "folder2"] - assert mock_run.call_args[0][0][3] == expected_path + mock_s3_client.list_objects_v2.assert_called_once_with( + Bucket="bucket", Prefix=expected_prefix, Delimiter="/" + ) + + @patch("boto3.client") + def test_list_subfolders_exception(self, mock_boto_client): + """Test listing subfolders when operation fails.""" + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client - @patch("subprocess.run") - def test_list_subfolders_exception(self, mock_run): - """Test listing subfolders when command fails.""" - mock_run.side_effect = Exception("Network error") + mock_s3_client.list_objects_v2.side_effect = Exception("Network error") assert S3FileSystem.list_subfolders("s3://bucket/parent") == [] - @patch("subprocess.run") - def test_list_subfolders_invalid_uri(self, mock_run): + def test_list_subfolders_invalid_uri(self): """Test listing subfolders with invalid URI.""" - with pytest.raises(ValueError, match="Invalid S3 URI"): - S3FileSystem.list_subfolders("gs://bucket/parent") + # list_subfolders catches all exceptions and returns empty list + result = S3FileSystem.list_subfolders("gs://bucket/parent") + assert result == [] - @patch("subprocess.run") + @patch("boto3.client") @pytest.mark.parametrize( - "uri,expected_path", + "uri,expected_prefix", [ - ("s3://bucket/dir", "s3://bucket/dir/"), - ("s3://bucket/dir/", "s3://bucket/dir/"), + ("s3://bucket/dir", "dir/"), + ("s3://bucket/dir/", "dir/"), ], ) - def test_download_files(self, mock_run, uri, expected_path): + def test_download_files(self, mock_boto_client, uri, expected_prefix): """Test downloading files from S3.""" - mock_result = MagicMock() - mock_result.returncode = 0 - mock_run.return_value = mock_result + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Mock paginator + mock_paginator = MagicMock() + mock_s3_client.get_paginator.return_value = mock_paginator + mock_paginator.paginate.return_value = [ + { + "Contents": [ + {"Key": f"{expected_prefix}file1.txt", "Size": 100}, + {"Key": f"{expected_prefix}file2.txt", "Size": 200}, + ] + } + ] + + # Mock download_file to do nothing + mock_s3_client.download_file = MagicMock() with tempfile.TemporaryDirectory() as tempdir: - S3FileSystem.download_files(tempdir, uri) - call_args = mock_run.call_args[0][0] - assert call_args[3] == expected_path - assert "--recursive" in call_args + S3FileSystem.download_files(tempdir, uri, max_workers=2) + + # Verify paginator was called correctly + mock_s3_client.get_paginator.assert_called_with("list_objects_v2") + mock_paginator.paginate.assert_called_once_with( + Bucket="bucket", Prefix=expected_prefix + ) + + # Verify files were downloaded + assert mock_s3_client.download_file.call_count == 2 - @patch("subprocess.run") - def test_download_files_with_filters(self, mock_run): + @patch("boto3.client") + def test_download_files_with_filters(self, mock_boto_client): """Test downloading files with inclusion and exclusion filters.""" - mock_result = MagicMock() - mock_result.returncode = 0 - mock_run.return_value = mock_result + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Mock paginator with various files + mock_paginator = MagicMock() + mock_s3_client.get_paginator.return_value = mock_paginator + mock_paginator.paginate.return_value = [ + { + "Contents": [ + {"Key": "dir/config.json", "Size": 100}, + {"Key": "dir/tokenizer.json", "Size": 200}, + {"Key": "dir/data.tmp", "Size": 300}, + {"Key": "dir/readme.txt", "Size": 400}, + {"Key": "dir/other.bin", "Size": 500}, + ] + } + ] + + # Mock download_file to do nothing + mock_s3_client.download_file = MagicMock() with tempfile.TemporaryDirectory() as tempdir: S3FileSystem.download_files( tempdir, "s3://bucket/dir", substrings_to_include=["config", "tokenizer"], - suffixes_to_exclude=[".tmp", "*.txt"], + suffixes_to_exclude=[".tmp", ".txt"], + max_workers=2, ) - call_args = mock_run.call_args[0][0] - assert "--exclude" in call_args - assert "--include" in call_args - assert "*config*" in call_args - assert "*tokenizer*" in call_args - assert "*.tmp" in call_args - assert "*.txt" in call_args - - @patch("subprocess.run") - def test_download_files_exception(self, mock_run): - """Test downloading files when command fails.""" - mock_run.side_effect = Exception("Network error") + # Should only download config.json and tokenizer.json + # (included by substring, not excluded by suffix) + assert mock_s3_client.download_file.call_count == 2 + + # Get the keys that were downloaded + downloaded_keys = [ + call[0][1] for call in mock_s3_client.download_file.call_args_list + ] + assert "dir/config.json" in downloaded_keys + assert "dir/tokenizer.json" in downloaded_keys + + @patch("boto3.client") + def test_download_files_no_matching_files(self, mock_boto_client): + """Test downloading when no files match filters.""" + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Mock paginator with files that won't match + mock_paginator = MagicMock() + mock_s3_client.get_paginator.return_value = mock_paginator + mock_paginator.paginate.return_value = [ + { + "Contents": [ + {"Key": "dir/file1.txt", "Size": 100}, + ] + } + ] + with tempfile.TemporaryDirectory() as tempdir: - with pytest.raises(Exception, match="Network error"): - S3FileSystem.download_files(tempdir, "s3://bucket/dir") + # This should not raise, just return without downloading + S3FileSystem.download_files( + tempdir, + "s3://bucket/dir", + substrings_to_include=["nonexistent"], + max_workers=2, + ) + + # Verify no files were downloaded + mock_s3_client.download_file.assert_not_called() + + @patch("boto3.client") + @patch("ray.llm._internal.common.utils.cloud_filesystem.s3_filesystem.logger") + def test_download_files_concurrent_failure(self, mock_logger, mock_boto_client): + """Test downloading files when some downloads fail.""" + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + + # Mock paginator + mock_paginator = MagicMock() + mock_s3_client.get_paginator.return_value = mock_paginator + mock_paginator.paginate.return_value = [ + { + "Contents": [ + {"Key": "dir/file1.txt", "Size": 100}, + {"Key": "dir/file2.txt", "Size": 200}, + ] + } + ] + + # Make download_file fail + mock_s3_client.download_file.side_effect = Exception("Download failed") + + with tempfile.TemporaryDirectory() as tempdir: + # Should complete without raising, but log errors + S3FileSystem.download_files(tempdir, "s3://bucket/dir", max_workers=2) + + # Verify error was logged for failed downloads + mock_logger.error.assert_called() + error_call = mock_logger.error.call_args_list[0][0][0] + assert "Failed to download" in error_call - @patch("subprocess.run") - def test_download_files_invalid_uri(self, mock_run): + def test_download_files_invalid_uri(self): """Test downloading files with invalid URI.""" with tempfile.TemporaryDirectory() as tempdir: with pytest.raises(ValueError, match="Invalid S3 URI"): S3FileSystem.download_files(tempdir, "gs://bucket/dir") - @patch("subprocess.run") + @patch("boto3.client") @pytest.mark.parametrize( - "uri,expected_path", + "uri,expected_prefix", [ - ("s3://bucket/dir", "s3://bucket/dir/"), - ("s3://bucket/dir/", "s3://bucket/dir/"), + ("s3://bucket/dir", "dir/"), + ("s3://bucket/dir/", "dir/"), ], ) - def test_upload_files(self, mock_run, uri, expected_path): - """Test uploading files to S3.""" - mock_result = MagicMock() - mock_result.returncode = 0 - mock_run.return_value = mock_result + def test_upload_files_directory(self, mock_boto_client, uri, expected_prefix): + """Test uploading a directory to S3.""" + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + mock_s3_client.upload_file = MagicMock() with tempfile.TemporaryDirectory() as tempdir: + # Create some test files + test_file1 = os.path.join(tempdir, "file1.txt") + test_file2 = os.path.join(tempdir, "subdir", "file2.txt") + os.makedirs(os.path.dirname(test_file2), exist_ok=True) + + with open(test_file1, "w") as f: + f.write("test1") + with open(test_file2, "w") as f: + f.write("test2") + S3FileSystem.upload_files(tempdir, uri) - call_args = mock_run.call_args[0][0] - assert call_args[4] == expected_path - assert "--recursive" in call_args - - @patch("subprocess.run") - def test_upload_files_exception(self, mock_run): - """Test uploading files when command fails.""" - mock_run.side_effect = Exception("Network error") + + # Verify files were uploaded + assert mock_s3_client.upload_file.call_count == 2 + + # Check the S3 keys that were used + uploaded_keys = [ + call[0][2] for call in mock_s3_client.upload_file.call_args_list + ] + assert f"{expected_prefix}file1.txt" in uploaded_keys + assert f"{expected_prefix}subdir/file2.txt" in uploaded_keys + + @patch("boto3.client") + def test_upload_files_single_file(self, mock_boto_client): + """Test uploading a single file to S3.""" + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + mock_s3_client.upload_file = MagicMock() + + with tempfile.TemporaryDirectory() as tempdir: + # Create a test file + test_file = os.path.join(tempdir, "single.txt") + with open(test_file, "w") as f: + f.write("test content") + + S3FileSystem.upload_files(test_file, "s3://bucket/dir/") + + # Verify single file was uploaded + mock_s3_client.upload_file.assert_called_once() + call_args = mock_s3_client.upload_file.call_args[0] + assert call_args[2] == "dir/single.txt" + + @patch("boto3.client") + def test_upload_files_exception(self, mock_boto_client): + """Test uploading files when operation fails.""" + # Setup mock S3 client + mock_s3_client = MagicMock() + mock_boto_client.return_value = mock_s3_client + mock_s3_client.upload_file.side_effect = Exception("Network error") + with tempfile.TemporaryDirectory() as tempdir: + # Create a test file + test_file = os.path.join(tempdir, "test.txt") + with open(test_file, "w") as f: + f.write("test") + with pytest.raises(Exception, match="Network error"): S3FileSystem.upload_files(tempdir, "s3://bucket/dir") - @patch("subprocess.run") - def test_upload_files_invalid_uri(self, mock_run): + def test_upload_files_invalid_uri(self): """Test uploading files with invalid URI.""" with tempfile.TemporaryDirectory() as tempdir: with pytest.raises(ValueError, match="Invalid S3 URI"): S3FileSystem.upload_files(tempdir, "gs://bucket/dir") - @patch("subprocess.run") - def test_run_command_success(self, mock_run): - """Test _run_command with successful execution.""" - mock_result = MagicMock() - mock_result.returncode = 0 - mock_run.return_value = mock_result - result = S3FileSystem._run_command(["aws", "s3", "ls"]) - assert result == mock_result - mock_run.assert_called_once_with( - ["aws", "s3", "ls"], capture_output=True, text=True, check=True + def test_upload_files_nonexistent_path(self): + """Test uploading from a path that doesn't exist.""" + with pytest.raises(ValueError, match="does not exist"): + S3FileSystem.upload_files("/nonexistent/path", "s3://bucket/dir") + + def test_parse_s3_uri(self): + """Test parsing S3 URIs.""" + # Standard URI + bucket, key, is_anon = S3FileSystem._parse_s3_uri( + "s3://bucket/path/to/file.txt" ) + assert bucket == "bucket" + assert key == "path/to/file.txt" + assert is_anon is False - @patch("subprocess.run") - def test_run_command_file_not_found(self, mock_run): - """Test _run_command when command is not found.""" - mock_run.side_effect = FileNotFoundError() - with pytest.raises(FileNotFoundError, match="is not installed"): - S3FileSystem._run_command(["nonexistent", "command"]) - - @patch("subprocess.run") - def test_run_command_called_process_error(self, mock_run): - """Test _run_command when command fails.""" - mock_run.side_effect = subprocess.CalledProcessError( - returncode=1, cmd=["aws", "s3", "cp"], stderr="Access Denied" + # Anonymous URI + bucket, key, is_anon = S3FileSystem._parse_s3_uri( + "s3://anonymous@bucket/file.txt" ) - with pytest.raises(subprocess.CalledProcessError): - S3FileSystem._run_command(["aws", "s3", "cp", "s3://bucket/file", "/tmp"]) + assert bucket == "bucket" + assert key == "file.txt" + assert is_anon is True + + # Bucket only + bucket, key, is_anon = S3FileSystem._parse_s3_uri("s3://bucket") + assert bucket == "bucket" + assert key == "" + assert is_anon is False + + def test_calculate_optimal_workers(self): + """Test worker calculation based on file characteristics.""" + # Many small files (< 1MB) + workers = S3FileSystem._calculate_optimal_workers( + num_files=50, total_size=50 * 500 * 1024 # 50 files * 500KB each + ) + assert workers == 50 # Should use many workers for small files + + # Medium files (1-10MB) + workers = S3FileSystem._calculate_optimal_workers( + num_files=50, total_size=50 * 5 * 1024 * 1024 # 50 files * 5MB each + ) + assert workers == 25 # Should use moderate workers + + # Large files (> 10MB) + workers = S3FileSystem._calculate_optimal_workers( + num_files=50, total_size=50 * 50 * 1024 * 1024 # 50 files * 50MB each + ) + assert workers == 20 # Should cap at 20 for large files + + # Zero files + workers = S3FileSystem._calculate_optimal_workers(num_files=0, total_size=0) + assert workers == 10 # Should return default_min class TestS3FileSystemIntegration: From 04385acd56d644023b747fc3876b087a32422f19 Mon Sep 17 00:00:00 2001 From: ahao-anyscale Date: Tue, 18 Nov 2025 15:52:35 -0800 Subject: [PATCH 8/8] fixing tests Signed-off-by: ahao-anyscale --- .../utils/cloud_filesystem/pyarrow_filesystem.py | 2 +- python/ray/llm/_internal/common/utils/cloud_utils.py | 1 + .../ray/llm/tests/common/cloud/test_s3_filesystem.py | 10 ++++++---- python/ray/llm/tests/common/utils/test_upload_utils.py | 4 ++-- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py index 48f6ddbc4fd9..aafab195b2e8 100644 --- a/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py +++ b/python/ray/llm/_internal/common/utils/cloud_filesystem/pyarrow_filesystem.py @@ -278,7 +278,7 @@ def list_subfolders(folder_uri: str) -> List[str]: return subfolders except Exception as e: - logger.info(f"Error listing subfolders in {folder_uri}: {e}") + logger.error(f"Error listing subfolders in {folder_uri}: {e}") return [] @staticmethod diff --git a/python/ray/llm/_internal/common/utils/cloud_utils.py b/python/ray/llm/_internal/common/utils/cloud_utils.py index 1808ec82ec3e..3c98fa572cdf 100644 --- a/python/ray/llm/_internal/common/utils/cloud_utils.py +++ b/python/ray/llm/_internal/common/utils/cloud_utils.py @@ -45,6 +45,7 @@ def is_remote_path(path: str) -> bool: or path.startswith("gs://") or path.startswith("abfss://") or path.startswith("azure://") + or path.startswith("pyarrow-") ) diff --git a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py index 795eb7a10c90..2917efcac114 100644 --- a/python/ray/llm/tests/common/cloud/test_s3_filesystem.py +++ b/python/ray/llm/tests/common/cloud/test_s3_filesystem.py @@ -436,7 +436,9 @@ class TestS3FileSystemIntegration: def test_list_subfolders_real_s3(self): """Test listing subfolders from real S3 bucket.""" # Test listing subfolders in the parent directory which has actual subfolders - folders = S3FileSystem.list_subfolders("s3://air-example-data/rayllm-ossci/") + folders = S3FileSystem.list_subfolders( + "s3://anonymous@air-example-data/rayllm-ossci/" + ) # Verify we get expected subfolders assert isinstance(folders, list) assert "meta-Llama-3.2-1B-Instruct" in folders @@ -446,7 +448,7 @@ def test_get_file_real_s3(self): """Test getting a file from real S3 bucket.""" # Test getting a small config file content = S3FileSystem.get_file( - "s3://air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/config.json" + "s3://anonymous@air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/config.json" ) assert content is not None assert isinstance(content, str) @@ -460,7 +462,7 @@ def test_download_files_with_exclusion(self): # Download files excluding safetensors S3FileSystem.download_files( tempdir, - "s3://air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/", + "s3://anonymous@air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/", suffixes_to_exclude=[".safetensors"], ) @@ -487,7 +489,7 @@ def test_download_files_with_inclusion(self): # Download only .json files S3FileSystem.download_files( tempdir, - "s3://air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/", + "s3://anonymous@air-example-data/rayllm-ossci/meta-Llama-3.2-1B-Instruct/", substrings_to_include=[".json"], ) diff --git a/python/ray/llm/tests/common/utils/test_upload_utils.py b/python/ray/llm/tests/common/utils/test_upload_utils.py index 7d8b828ad0c5..7ee970143e34 100644 --- a/python/ray/llm/tests/common/utils/test_upload_utils.py +++ b/python/ray/llm/tests/common/utils/test_upload_utils.py @@ -54,7 +54,7 @@ def test_upload_downloaded_hf_model(mock_copy_files): "ray.llm._internal.common.utils.upload_utils.get_model_entrypoint", return_value=model_dir, ): - upload_model_files(model_id, "s3://bucket/model-id") + upload_model_files(model_id, "pyarrow-s3://bucket/model-id") assert mock_copy_files.call_count == 2 mock_copy_files.assert_has_calls( @@ -87,7 +87,7 @@ def test_upload_custom_model(mock_copy_files): "ray.llm._internal.common.utils.upload_utils.get_model_entrypoint", return_value=model_dir, ): - upload_model_files(model_id, "s3://bucket/model-id") + upload_model_files(model_id, "pyarrow-s3://bucket/model-id") mock_copy_files.assert_called_once_with( source=Path(model_dir),