Skip to content

Commit

Permalink
[Data] Deprecate meta_provider parameter of read APIs (ray-project#…
Browse files Browse the repository at this point in the history
…48690)

Metadata providers are specific to current implementation of read APIs.
To enable us to change the implementation, this PR deprecates the
`meta_provider` parameter and decouples metadata providers from the
user-facing interface.
---------

Signed-off-by: Balaji Veeramani <[email protected]>
Signed-off-by: Balaji Veeramani <[email protected]>
  • Loading branch information
bveeramani authored Nov 12, 2024
1 parent ba22e74 commit d222ce1
Show file tree
Hide file tree
Showing 2 changed files with 71 additions and 32 deletions.
96 changes: 64 additions & 32 deletions python/ray/data/read_api.py
Original file line number Diff line number Diff line change
Expand Up @@ -735,6 +735,7 @@ def read_parquet(
:class:`~ray.data.Dataset` producing records read from the specified parquet
files.
"""
_emit_meta_provider_deprecation_warning(meta_provider)
_validate_shuffle_arg(shuffle)

if meta_provider is None:
Expand Down Expand Up @@ -848,10 +849,10 @@ class string
the filesystem is automatically selected based on the scheme of the paths.
For example, if the path begins with ``s3://``, the `S3FileSystem` is used.
parallelism: This argument is deprecated. Use ``override_num_blocks`` argument.
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
ray_remote_args: kwargs passed to :meth:`~ray.remote` in the read tasks.
arrow_open_file_args: kwargs passed to
`pyarrow.fs.FileSystem.open_input_file <https://arrow.apache.org/docs/\
Expand Down Expand Up @@ -897,6 +898,8 @@ class string
ValueError: if ``size`` contains non-positive numbers.
ValueError: if ``mode`` is unsupported.
"""
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = ImageFileMetadataProvider()

Expand Down Expand Up @@ -993,10 +996,10 @@ def read_parquet_bulk(
assumes that the tensors are serialized in the raw
NumPy array format in C-contiguous order (e.g. via
`arr.tobytes()`).
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
partition_filter: A
:class:`~ray.data.datasource.partitioning.PathPartitionFilter`. Use
with a custom callback to read only selected partitions of a dataset.
Expand All @@ -1023,6 +1026,8 @@ def read_parquet_bulk(
Returns:
:class:`~ray.data.Dataset` producing records read from the specified paths.
"""
_emit_meta_provider_deprecation_warning(meta_provider)

warnings.warn(
"`read_parquet_bulk` is deprecated and will be removed after May 2025. Use "
"`read_parquet` instead.",
Expand Down Expand Up @@ -1138,10 +1143,10 @@ def read_json(
python/generated/pyarrow.fs.FileSystem.html\
#pyarrow.fs.FileSystem.open_input_stream>`_.
when opening input files to read.
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
partition_filter: A
:class:`~ray.data.datasource.partitioning.PathPartitionFilter`.
Use with a custom callback to read only selected partitions of a
Expand Down Expand Up @@ -1174,6 +1179,8 @@ def read_json(
Returns:
:class:`~ray.data.Dataset` producing records read from the specified paths.
""" # noqa: E501
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = DefaultFileMetadataProvider()

Expand Down Expand Up @@ -1305,10 +1312,10 @@ def read_csv(
python/generated/pyarrow.fs.FileSystem.html\
#pyarrow.fs.FileSystem.open_input_stream>`_.
when opening input files to read.
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
partition_filter: A
:class:`~ray.data.datasource.partitioning.PathPartitionFilter`.
Use with a custom callback to read only selected partitions of a
Expand Down Expand Up @@ -1340,6 +1347,8 @@ def read_csv(
Returns:
:class:`~ray.data.Dataset` producing records read from the specified paths.
"""
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = DefaultFileMetadataProvider()

Expand Down Expand Up @@ -1421,10 +1430,10 @@ def read_text(
python/generated/pyarrow.fs.FileSystem.html\
#pyarrow.fs.FileSystem.open_input_stream>`_.
when opening input files to read.
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
partition_filter: A
:class:`~ray.data.datasource.partitioning.PathPartitionFilter`.
Use with a custom callback to read only selected partitions of a
Expand All @@ -1451,6 +1460,8 @@ def read_text(
:class:`~ray.data.Dataset` producing lines of text read from the specified
paths.
"""
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = DefaultFileMetadataProvider()

Expand Down Expand Up @@ -1530,10 +1541,10 @@ def read_avro(
python/generated/pyarrow.fs.FileSystem.html\
#pyarrow.fs.FileSystem.open_input_stream>`_.
when opening input files to read.
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
partition_filter: A
:class:`~ray.data.datasource.partitioning.PathPartitionFilter`.
Use with a custom callback to read only selected partitions of a
Expand All @@ -1559,6 +1570,8 @@ def read_avro(
Returns:
:class:`~ray.data.Dataset` holding records from the Avro files.
"""
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = DefaultFileMetadataProvider()

Expand Down Expand Up @@ -1654,6 +1667,8 @@ def read_numpy(
Returns:
Dataset holding Tensor records read from the specified paths.
""" # noqa: E501
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = DefaultFileMetadataProvider()

Expand Down Expand Up @@ -1753,10 +1768,10 @@ def read_tfrecords(
when opening input files to read. To read a compressed TFRecord file,
pass the corresponding compression type (e.g., for ``GZIP`` or ``ZLIB``),
use ``arrow_open_stream_args={'compression': 'gzip'}``).
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
partition_filter: A
:class:`~ray.data.datasource.partitioning.PathPartitionFilter`.
Use with a custom callback to read only selected partitions of a
Expand Down Expand Up @@ -1789,6 +1804,8 @@ def read_tfrecords(
"""
import platform

_emit_meta_provider_deprecation_warning(meta_provider)

tfx_read = False

if tfx_read_options and platform.processor() != "arm":
Expand Down Expand Up @@ -1907,6 +1924,8 @@ def read_webdataset(
.. _tf.train.Example: https://www.tensorflow.org/api_docs/python/tf/train/Example
""" # noqa: E501
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = DefaultFileMetadataProvider()

Expand Down Expand Up @@ -1995,10 +2014,10 @@ def read_binary_files(
`pyarrow.fs.FileSystem.open_input_file <https://arrow.apache.org/docs/\
python/generated/pyarrow.fs.FileSystem.html\
#pyarrow.fs.FileSystem.open_input_stream>`_.
meta_provider: A :ref:`file metadata provider <metadata_provider>`. Custom
metadata providers may be able to resolve file metadata more quickly and/or
accurately. In most cases, you do not need to set this. If ``None``, this
function uses a system-chosen implementation.
meta_provider: [Deprecated] A :ref:`file metadata provider <metadata_provider>`.
Custom metadata providers may be able to resolve file metadata more quickly
and/or accurately. In most cases, you do not need to set this. If ``None``,
this function uses a system-chosen implementation.
partition_filter: A
:class:`~ray.data.datasource.partitioning.PathPartitionFilter`.
Use with a custom callback to read only selected partitions of a
Expand All @@ -2023,6 +2042,8 @@ def read_binary_files(
Returns:
:class:`~ray.data.Dataset` producing rows read from the specified paths.
"""
_emit_meta_provider_deprecation_warning(meta_provider)

if meta_provider is None:
meta_provider = DefaultFileMetadataProvider()

Expand Down Expand Up @@ -3258,3 +3279,14 @@ def _validate_shuffle_arg(shuffle: Optional[str]) -> None:
f"Invalid value for 'shuffle': {shuffle}. "
"Valid values are None, 'files'."
)


def _emit_meta_provider_deprecation_warning(
meta_provider: Optional[BaseFileMetadataProvider],
) -> None:
if meta_provider is not None:
warnings.warn(
"The `meta_provider` argument is deprecated and will be removed after May "
"2025.",
DeprecationWarning,
)
7 changes: 7 additions & 0 deletions python/ray/data/tests/test_image.py
Original file line number Diff line number Diff line change
Expand Up @@ -275,6 +275,13 @@ def test_unidentified_image_error(ray_start_regular_shared):
with pytest.raises(ValueError):
ray.data.read_images(paths=file.name).materialize()

def test_custom_meta_provider_emits_deprecation_warning(ray_start_regular_shared):
with pytest.warns(DeprecationWarning):
ray.data.read_images(
paths=["example://image-datasets/simple/image1.jpg"],
meta_provider=FastFileMetadataProvider(),
)


class TestWriteImages:
def test_write_images(ray_start_regular_shared, tmp_path):
Expand Down

0 comments on commit d222ce1

Please sign in to comment.