Skip to content

Commit

Permalink
[Data] Remove deprecated APIs (ray-project#45850)
Browse files Browse the repository at this point in the history
This PR removes some APIs that have been "hard deprecated" (i.e., raise an error) for several releases.

Signed-off-by: Balaji Veeramani <[email protected]>
  • Loading branch information
bveeramani authored Jun 11, 2024
1 parent 93bc785 commit 8b224dc
Show file tree
Hide file tree
Showing 11 changed files with 20 additions and 209 deletions.
1 change: 0 additions & 1 deletion doc/source/data/api/dataset.rst
Original file line number Diff line number Diff line change
Expand Up @@ -105,7 +105,6 @@ I/O and Conversion
Dataset.write_tfrecords
Dataset.write_webdataset
Dataset.write_mongo
Dataset.write_datasource
Dataset.to_torch
Dataset.to_tf
Dataset.to_dask
Expand Down
76 changes: 18 additions & 58 deletions python/ray/data/dataset.py
Original file line number Diff line number Diff line change
Expand Up @@ -83,10 +83,8 @@
)
from ray.data.context import DataContext
from ray.data.datasource import (
BlockWritePathProvider,
Connection,
Datasink,
Datasource,
FilenameProvider,
ReadTask,
_BigQueryDatasink,
Expand All @@ -103,7 +101,7 @@
from ray.data.iterator import DataIterator
from ray.data.random_access_dataset import RandomAccessDataset
from ray.types import ObjectRef
from ray.util.annotations import Deprecated, DeveloperAPI, PublicAPI
from ray.util.annotations import DeveloperAPI, PublicAPI
from ray.util.scheduling_strategies import NodeAffinitySchedulingStrategy
from ray.widgets import Template
from ray.widgets.util import repr_with_fallback
Expand Down Expand Up @@ -2734,7 +2732,6 @@ def write_parquet(
try_create_dir: bool = True,
arrow_open_stream_args: Optional[Dict[str, Any]] = None,
filename_provider: Optional[FilenameProvider] = None,
block_path_provider: Optional[BlockWritePathProvider] = None,
arrow_parquet_args_fn: Callable[[], Dict[str, Any]] = lambda: {},
num_rows_per_file: Optional[int] = None,
ray_remote_args: Dict[str, Any] = None,
Expand All @@ -2750,10 +2747,9 @@ def write_parquet(
If pyarrow can't represent your data, this method errors.
By default, the format of the output files is ``{uuid}_{block_idx}.parquet``,
where ``uuid`` is a unique
id for the dataset. To modify this behavior, implement a custom
:class:`~ray.data.datasource.BlockWritePathProvider`
and pass it in as the ``block_path_provider`` argument.
where ``uuid`` is a unique id for the dataset. To modify this behavior,
implement a custom :class:`~ray.data.datasource.FilenameProvider` and pass it in
as the ``filename_provider`` argument.
Examples:
>>> import ray
Expand Down Expand Up @@ -2817,7 +2813,6 @@ def write_parquet(
try_create_dir=try_create_dir,
open_stream_args=arrow_open_stream_args,
filename_provider=filename_provider,
block_path_provider=block_path_provider,
dataset_uuid=self._uuid,
)
self.write_datasink(
Expand All @@ -2835,7 +2830,6 @@ def write_json(
try_create_dir: bool = True,
arrow_open_stream_args: Optional[Dict[str, Any]] = None,
filename_provider: Optional[FilenameProvider] = None,
block_path_provider: Optional[BlockWritePathProvider] = None,
pandas_json_args_fn: Callable[[], Dict[str, Any]] = lambda: {},
num_rows_per_file: Optional[int] = None,
ray_remote_args: Dict[str, Any] = None,
Expand All @@ -2853,9 +2847,8 @@ def write_json(
By default, the format of the output files is ``{uuid}_{block_idx}.json``,
where ``uuid`` is a unique id for the dataset. To modify this behavior,
implement a custom
:class:`~ray.data.file_based_datasource.BlockWritePathProvider`
and pass it in as the ``block_path_provider`` argument.
implement a custom :class:`~ray.data.datasource.FilenameProvider` and pass it in
as the ``filename_provider`` argument.
Examples:
Write the dataset as JSON file to a local directory.
Expand Down Expand Up @@ -2928,7 +2921,6 @@ def write_json(
try_create_dir=try_create_dir,
open_stream_args=arrow_open_stream_args,
filename_provider=filename_provider,
block_path_provider=block_path_provider,
dataset_uuid=self._uuid,
)
self.write_datasink(
Expand Down Expand Up @@ -3018,7 +3010,6 @@ def write_csv(
try_create_dir: bool = True,
arrow_open_stream_args: Optional[Dict[str, Any]] = None,
filename_provider: Optional[FilenameProvider] = None,
block_path_provider: Optional[BlockWritePathProvider] = None,
arrow_csv_args_fn: Callable[[], Dict[str, Any]] = lambda: {},
num_rows_per_file: Optional[int] = None,
ray_remote_args: Dict[str, Any] = None,
Expand All @@ -3036,9 +3027,9 @@ def write_csv(
By default, the format of the output files is ``{uuid}_{block_idx}.csv``,
where ``uuid`` is a unique id for the dataset. To modify this behavior,
implement a custom
:class:`~ray.data.datasource.BlockWritePathProvider`
and pass it in as the ``block_path_provider`` argument.
implement a custom :class:`~ray.data.datasource.FilenameProvider`
and pass it in as the ``filename_provider`` argument.
Examples:
Write the dataset as CSV files to a local directory.
Expand Down Expand Up @@ -3108,7 +3099,6 @@ def write_csv(
try_create_dir=try_create_dir,
open_stream_args=arrow_open_stream_args,
filename_provider=filename_provider,
block_path_provider=block_path_provider,
dataset_uuid=self._uuid,
)
self.write_datasink(
Expand All @@ -3127,7 +3117,6 @@ def write_tfrecords(
try_create_dir: bool = True,
arrow_open_stream_args: Optional[Dict[str, Any]] = None,
filename_provider: Optional[FilenameProvider] = None,
block_path_provider: Optional[BlockWritePathProvider] = None,
num_rows_per_file: Optional[int] = None,
ray_remote_args: Dict[str, Any] = None,
concurrency: Optional[int] = None,
Expand All @@ -3154,9 +3143,8 @@ def write_tfrecords(
By default, the format of the output files is ``{uuid}_{block_idx}.tfrecords``,
where ``uuid`` is a unique id for the dataset. To modify this behavior,
implement a custom
:class:`~ray.data.file_based_datasource.BlockWritePathProvider`
and pass it in as the ``block_path_provider`` argument.
implement a custom :class:`~ray.data.datasource.FilenameProvider`
and pass it in as the ``filename_provider`` argument.
Examples:
>>> import ray
Expand Down Expand Up @@ -3206,7 +3194,6 @@ def write_tfrecords(
try_create_dir=try_create_dir,
open_stream_args=arrow_open_stream_args,
filename_provider=filename_provider,
block_path_provider=block_path_provider,
dataset_uuid=self._uuid,
)
self.write_datasink(
Expand All @@ -3225,7 +3212,6 @@ def write_webdataset(
try_create_dir: bool = True,
arrow_open_stream_args: Optional[Dict[str, Any]] = None,
filename_provider: Optional[FilenameProvider] = None,
block_path_provider: Optional[BlockWritePathProvider] = None,
num_rows_per_file: Optional[int] = None,
ray_remote_args: Dict[str, Any] = None,
encoder: Optional[Union[bool, str, callable, list]] = True,
Expand All @@ -3246,7 +3232,7 @@ def write_webdataset(
This is only supported for datasets convertible to Arrow records.
To control the number of files, use :meth:`Dataset.repartition`.
Unless a custom block path provider is given, the format of the output
Unless a custom filename provider is given, the format of the output
files is ``{uuid}_{block_idx}.tfrecords``, where ``uuid`` is a unique id
for the dataset.
Expand All @@ -3271,8 +3257,9 @@ def write_webdataset(
already exist. Defaults to ``True``.
arrow_open_stream_args: kwargs passed to
``pyarrow.fs.FileSystem.open_output_stream``
block_path_provider: :class:`~ray.data.datasource.BlockWritePathProvider`
implementation to write each dataset block to a custom output path.
filename_provider: A :class:`~ray.data.datasource.FilenameProvider`
implementation. Use this parameter to customize what your filenames
look like.
num_rows_per_file: The target number of rows to write to each file. If
``None``, Ray Data writes a system-chosen number of rows to each file.
The specified value is a hint, not a strict limit. Ray Data might write
Expand All @@ -3292,7 +3279,6 @@ def write_webdataset(
try_create_dir=try_create_dir,
open_stream_args=arrow_open_stream_args,
filename_provider=filename_provider,
block_path_provider=block_path_provider,
dataset_uuid=self._uuid,
)
self.write_datasink(
Expand All @@ -3311,7 +3297,6 @@ def write_numpy(
try_create_dir: bool = True,
arrow_open_stream_args: Optional[Dict[str, Any]] = None,
filename_provider: Optional[FilenameProvider] = None,
block_path_provider: Optional[BlockWritePathProvider] = None,
num_rows_per_file: Optional[int] = None,
ray_remote_args: Dict[str, Any] = None,
concurrency: Optional[int] = None,
Expand All @@ -3325,11 +3310,11 @@ def write_numpy(
To control the number of number of blocks, call
:meth:`~ray.data.Dataset.repartition`.
By default, the format of the output files is ``{uuid}_{block_idx}.npy``,
where ``uuid`` is a unique id for the dataset. To modify this behavior,
implement a custom
:class:`~ray.data.datasource.BlockWritePathProvider`
and pass it in as the ``block_path_provider`` argument.
implement a custom :class:`~ray.data.datasource.FilenameProvider`
and pass it in as the ``filename_provider`` argument.
Examples:
>>> import ray
Expand Down Expand Up @@ -3381,7 +3366,6 @@ def write_numpy(
try_create_dir=try_create_dir,
open_stream_args=arrow_open_stream_args,
filename_provider=filename_provider,
block_path_provider=block_path_provider,
dataset_uuid=self._uuid,
)
self.write_datasink(
Expand Down Expand Up @@ -3607,30 +3591,6 @@ def write_bigquery(
concurrency=concurrency,
)

@Deprecated
@ConsumptionAPI(pattern="Time complexity:")
def write_datasource(
self,
datasource: Datasource,
*,
ray_remote_args: Dict[str, Any] = None,
**write_args,
) -> None:
"""Writes the dataset to a custom :class:`~ray.data.Datasource`.
Time complexity: O(dataset size / parallelism)
Args:
datasource: The :class:`~ray.data.Datasource` to write to.
ray_remote_args: Kwargs passed to ``ray.remote`` in the write tasks.
write_args: Additional write args to pass to the :class:`~ray.data.Datasource`.
""" # noqa: E501
raise DeprecationWarning(
"`write_datasource` is deprecated in Ray 2.9. Create a `Datasink` and use "
"`write_datasink` instead. For more information, see "
"https://docs.ray.io/en/master/data/api/doc/ray.data.Datasink.html.", # noqa: E501
)

@ConsumptionAPI(pattern="Time complexity:")
def write_datasink(
self,
Expand Down
8 changes: 0 additions & 8 deletions python/ray/data/datasource/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,6 @@
from ray.data.datasource.bigquery_datasink import _BigQueryDatasink
from ray.data.datasource.bigquery_datasource import BigQueryDatasource
from ray.data.datasource.binary_datasource import BinaryDatasource
from ray.data.datasource.block_path_provider import (
BlockWritePathProvider,
DefaultBlockWritePathProvider,
)
from ray.data.datasource.csv_datasink import _CSVDatasink
from ray.data.datasource.csv_datasource import CSVDatasource
from ray.data.datasource.datasink import Datasink, DummyOutputDatasink
Expand All @@ -17,7 +13,6 @@
)
from ray.data.datasource.file_based_datasource import (
FileBasedDatasource,
FileExtensionFilter,
_S3FileSystemWrapper,
)
from ray.data.datasource.file_datasink import (
Expand Down Expand Up @@ -70,20 +65,17 @@
"_BigQueryDatasink",
"BigQueryDatasource",
"BlockBasedFileDatasink",
"BlockWritePathProvider",
"Connection",
"_CSVDatasink",
"CSVDatasource",
"Datasink",
"Datasource",
"_SQLDatasink",
"SQLDatasource",
"DefaultBlockWritePathProvider",
"DefaultFileMetadataProvider",
"DummyOutputDatasink",
"FastFileMetadataProvider",
"FileBasedDatasource",
"FileExtensionFilter",
"FileMetadataProvider",
"FilenameProvider",
"_ImageDatasink",
Expand Down
26 changes: 0 additions & 26 deletions python/ray/data/datasource/block_path_provider.py

This file was deleted.

16 changes: 1 addition & 15 deletions python/ray/data/datasource/file_based_datasource.py
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@
_has_file_extension,
_resolve_paths_and_filesystem,
)
from ray.util.annotations import Deprecated, DeveloperAPI, PublicAPI
from ray.util.annotations import DeveloperAPI

if TYPE_CHECKING:
import pandas as pd
Expand All @@ -64,20 +64,6 @@
OPEN_FILE_MAX_ATTEMPTS = 10


@Deprecated
@PublicAPI(stability="beta")
class FileExtensionFilter(PathPartitionFilter):
def __init__(
self,
file_extensions: Union[str, List[str]],
allow_if_no_extension: bool = False,
):
raise DeprecationWarning(
"`FileExtensionFilter` is deprecated. Instead, set the `file_extensions` "
"parameter of `read_xxx()` APIs."
)


@DeveloperAPI
class FileBasedDatasource(Datasource):
"""File-based datasource for reading files.
Expand Down
10 changes: 0 additions & 10 deletions python/ray/data/datasource/file_datasink.py
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@
from ray.data._internal.util import _is_local_scheme, call_with_retry
from ray.data.block import Block, BlockAccessor
from ray.data.context import DataContext
from ray.data.datasource.block_path_provider import BlockWritePathProvider
from ray.data.datasource.datasink import Datasink
from ray.data.datasource.filename_provider import (
FilenameProvider,
Expand Down Expand Up @@ -37,7 +36,6 @@ def __init__(
try_create_dir: bool = True,
open_stream_args: Optional[Dict[str, Any]] = None,
filename_provider: Optional[FilenameProvider] = None,
block_path_provider: Optional[BlockWritePathProvider] = None,
dataset_uuid: Optional[str] = None,
file_format: Optional[str] = None,
):
Expand All @@ -59,13 +57,6 @@ def __init__(
if open_stream_args is None:
open_stream_args = {}

if block_path_provider is not None:
raise DeprecationWarning(
"`block_path_provider` has been deprecated in favor of "
"`filename_provider`. For more information, see "
"https://docs.ray.io/en/master/data/api/doc/ray.data.datasource.FilenameProvider.html", # noqa: E501
)

if filename_provider is None:
filename_provider = _DefaultFilenameProvider(
dataset_uuid=dataset_uuid, file_format=file_format
Expand All @@ -79,7 +70,6 @@ def __init__(
self.try_create_dir = try_create_dir
self.open_stream_args = open_stream_args
self.filename_provider = filename_provider
self.block_path_provider = block_path_provider
self.dataset_uuid = dataset_uuid
self.file_format = file_format

Expand Down
Loading

0 comments on commit 8b224dc

Please sign in to comment.