Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Data] Add performant way to read large tfrecord datasets #42277

Merged
merged 40 commits into from Feb 29, 2024

Conversation

martinbomio
Copy link
Contributor

@martinbomio martinbomio commented Jan 9, 2024

Why are these changes needed?

The main motivation for this PR is that ray.data.read_tfrcords yields suboptimal performance when reading large datasets.
This PR adds a default "fast" route for reading tf.records that relies on tfx-bsl decoder. This approach also infers the schema when no tf_schema is provided by doing a pass of the data to determine the cardinality of the feature lists.

Related issue number

N/A

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@@ -1564,13 +1567,50 @@ def read_tfrecords(
shuffle: If setting to "files", randomly shuffle input files order before read.
Defaults to not shuffle with ``None``.
file_extensions: A list of file extensions to filter files by.
schema_inference: Toggles the schema inference applied; applicable only if
tf_schema argument is missing. Defaults to True.
force_fast_read: Forces the fast read, failing if the proper dependencies
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added this option mostly for test purposes, let me know if there's a better way to do this

@@ -338,9 +338,13 @@ def _str2bytes(d):
assert ds_expected.take() == ds_actual.take()


@pytest.mark.parametrize("with_tf_schema", (True, False))
@pytest.mark.parametrize(

This comment was marked as outdated.

@martinbomio martinbomio force-pushed the martinbomio/fast-tfrecord-read branch 2 times, most recently from aa023a5 to 1223af6 Compare January 9, 2024 22:48
@martinbomio martinbomio changed the title feat: add performant way to read large tfrecord datasets [Data] feat: add performant way to read large tfrecord datasets Jan 9, 2024
self.tf_schema = tf_schema
self._tf_schema = tf_schema
self._fast_read = fast_read
self._batch_size = batch_size or 2048
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could we assign the default 2048 to a constant at the top of the file?

@@ -1490,6 +1490,9 @@ def read_tfrecords(
tf_schema: Optional["schema_pb2.Schema"] = None,
shuffle: Union[Literal["files"], None] = None,
file_extensions: Optional[List[str]] = None,
batch_size: Optional[int] = None,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could we name this variable something like fast_read_batch_size to indicate that this value will be only used with the fast path?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, we are kind of leaking to the users that there's an underlying fast and slow im,plementation, which might be confusing to the user

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good point. i think since we will mention that this is used for only the fast path in the docstring anyway, i was thinking it would be good to surface that fact explicitly with the parameter name as well.

return dataset.map_batches(
_unwrap_single_value_lists,
fn_kwargs={"col_lengths": list_sizes["max_list_size"]},
batch_format="pandas",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we rewrite _unwrap_single_value_lists to use the default (numpy) batch format? pandas is more memory intensive, so we would like to avoid specifying pandas batch format wherever possible.

@@ -1490,6 +1490,9 @@ def read_tfrecords(
tf_schema: Optional["schema_pb2.Schema"] = None,
shuffle: Union[Literal["files"], None] = None,
file_extensions: Optional[List[str]] = None,
batch_size: Optional[int] = None,
schema_inference: bool = True,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

similarly for this, maybe something like fast_read_auto_infer_schema?

@martinbomio martinbomio force-pushed the martinbomio/fast-tfrecord-read branch 2 times, most recently from 735fb4e to 8b29ca1 Compare January 17, 2024 16:09
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
# Dependency used for read_tfrecords function.
# Given that we only use the ExamplesToRecordBatchDecoder
# which is purley c++, we can isntall it with --no-dependencies.
pip install tfx-bsl==1.14.0 --no-dependencies
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this feels like a rather weird way to do it.. is this the intended way to use tfx-bsl? are there other more direct ways to import and use the logic in ExamplesToRecordBatchDecoder?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

other more direct ways like adding the dependency like any other dependency or are you thinking about something else?

I tried adding it as a direct dependency in the data test dependencies, but it conflicted with other existing dependencies.

We do not really need to bring transitive dependencies since the only thing we need to use is the ExamplesToRecordBatchDecoder which is self contained and doesn't need any extra dependency to work. Another approach we could take is to add this class to ray repo itself, I am not really familiar with that part of ray codebase but we would have to add the c class and add a python binding

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

conflicted with other existing dependencies.

could you elaborate on the specifics?

cc @can-anyscale // example that we might need multiple constraint files.

I am worrying that this is a hack, and might be not sustainable. Like is there any guarantee that when tfx-bsl has a new version in the future, this will still work? Do we expect users to all install tfx-bsl with no --no-dependencies

This is not a test-only dependency, but a data dependency. at the end of the days, for it to be useful, it needs to work with other dependencies in key workflows / workloads. Tests and CI are proxies to "ray works for users".

It seems that this is only used as part of the internal implementation, not part of ray data interface? if that is the case, can we fork https://github.com/tensorflow/tfx-bsl , give it another package name, remove the other parts that we do not need, and build it from source, and publish a tfs-bsl-ray package just for this?

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

or maybe we should try to resolve the dependency conflicts..

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we expect users to all install tfx-bsl with no --no-dependencies

yes, we do

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we expect users to all install tfx-bsl with no --no-dependencies

yes, we do

:) this is more of a question of ray code owners. I am asking @scottjlee to help here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we can always make the assumption (or force users to) install with the --no-dependencies flag, since they might be using the library's other features. But that would mean the dependency resolution would be put on the user, which we should try to avoid.

Signed-off-by: Scott Lee <sjl@anyscale.com>
python/ray/data/read_api.py Outdated Show resolved Hide resolved
python/ray/data/read_api.py Outdated Show resolved Hide resolved
python/ray/data/read_api.py Show resolved Hide resolved
# Dependency used for read_tfrecords function.
# Given that we only use the ExamplesToRecordBatchDecoder
# which is purley c++, we can isntall it with --no-dependencies.
pip install tfx-bsl==1.14.0 --no-dependencies
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we can always make the assumption (or force users to) install with the --no-dependencies flag, since they might be using the library's other features. But that would mean the dependency resolution would be put on the user, which we should try to avoid.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
@bveeramani bveeramani removed their assignment Feb 21, 2024
Copy link
Contributor

@c21 c21 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you @martinbomio for the contibution!
Looks solid overall. Having some comments. Thanks.

@@ -23,13 +33,23 @@ def __init__(
self,
paths: Union[str, List[str]],
tf_schema: Optional["schema_pb2.Schema"] = None,
fast_read: bool = False,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we change the name fast_read to read_with_tfx?
The fast_read looks a bit vague to me.

Comment on lines 36 to 37
fast_read: bool = False,
batch_size: Optional[int] = None,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let's also add the comment for these two parameters in docstring.

else:
yield from self._slow_read_stream(f, path)

def _slow_read_stream(self, f: "pyarrow.NativeFile", path: str) -> Iterator[Block]:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

_slow_read_stream -> _default_read_stream

)

def _fast_read_stream(self, f: "pyarrow.NativeFile", path: str) -> Iterator[Block]:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

_fast_read_stream -> tfx_read_stream

@@ -1583,6 +1583,8 @@ def read_tfrecords(
file_extensions: Optional[List[str]] = None,
concurrency: Optional[int] = None,
override_num_blocks: Optional[int] = None,
fast_read_batch_size: Optional[int] = None,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fast_read_batch_size -> tfx_read_batch_size

@@ -1583,6 +1583,8 @@ def read_tfrecords(
file_extensions: Optional[List[str]] = None,
concurrency: Optional[int] = None,
override_num_blocks: Optional[int] = None,
fast_read_batch_size: Optional[int] = None,
fast_read_auto_infer_schema: bool = True,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fast_read_auto_infer_schema -> tfx_read_auto_infer_schema

Returns:
A :class:`~ray.data.Dataset` that contains the example features.

Raises:
ValueError: If a file contains a message that isn't a ``tf.train.Example``.
"""
import platform

fast_read = False
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fast_read -> read_with_tfx

Comment on lines 1686 to 1691
if platform.processor() == "arm":
logger.warning(
"The fast strategy of this function depends on tfx-bsl, which is "
"currently not supported on devices with Apple silicon "
"(e.g. M1) and requires an environment with x86 CPU architecture."
)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we check if platform.processor() == "arm" before import ExamplesToRecordBatchDecoder? We don't need to print a warning here if user runs on arm.

Comment on lines 1693 to 1700
logger.warning(
"To use TFRecordDatasource with large datasets, please install"
" tfx-bsl package with pip install tfx_bsl --no-dependencies`."
)
logger.info(
"Falling back to slower strategy for reading tf.records. This "
"reading strategy should be avoided when reading large datasets."
)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's print out one warning message instead:

Please install tfx-bsl package with `pip install tfx_bsl --no-dependencies`. This can help speed up the reading of large TFRecord files.

Signed-off-by: Martin Bomio <martinbomio@spotify.com>
@martinbomio
Copy link
Contributor Author

@c21 thanks for the comments, I addressed them in 9aa260b

@c21
Copy link
Contributor

c21 commented Feb 26, 2024

Thank you @martinbomio. Can you address the CI test failure?

Screen Shot 2024-02-26 at 1 30 09 PM Screen Shot 2024-02-26 at 1 30 13 PM

Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Returns:
A :class:`~ray.data.Dataset` that contains the example features.

Raises:
ValueError: If a file contains a message that isn't a ``tf.train.Example``.
"""
import platform

tfx_read = False
Copy link
Contributor

@c21 c21 Feb 26, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @martinbomio, final requesst - can we add a config in DataContext - https://github.com/ray-project/ray/blob/master/python/ray/data/context.py . So user can disable the tfx read code path, in case they run into any bugs? By default, the config can set to true.

DataContext.get_current().enable_tfrecord_tfx_read

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@c21 done in 413c1f0


if platform.processor() != "arm":
try:
from tfx_bsl.cc.tfx_bsl_extension.coders import ( # noqa: F401
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit, will just import tfx_bsl work?



def _infer_schema_and_transform(dataset: "Dataset"):
list_sizes = dataset.aggregate(_MaxListSize(dataset.schema().names))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you add a comment on why this function is needed?
it seems that we'll read the datasource twice, one for aggregate, one for map_batches. will that not be less efficient?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@raulchen it will indeed mean an extra pass on the data. The reason it is needed is because tfx-bsl ExampleDecoder returns always list of lists when no schema is provided, and what this function is doing is infering the schema for those fields that are single value fields.

Performance wise, some of our benchmarks on this implementation (we have had it for a while running internally), gives us more than 15X improvements compared to the current implementation. Some of our datasets take ~30m to load with the ray native implementation compared to less than 2m with this tfx-bsl implementation. Let me know if you need more benchmark numbers, happy to provide more

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

thanks for the explanation. it's okay to proceed without more benchmarks for now.

Signed-off-by: Martin Bomio <martinbomio@spotify.com>
@c21
Copy link
Contributor

c21 commented Feb 27, 2024

Discussed offline w/ @martinbomio. For 2.10 release, let's introduce 3 configs in DataContext, and disable the tfx read by default (to be safe), and we can enable the tfx read in future release.

enable_tfrecords_tfx_read
tfrecords_tfx_read_batch_size
tfrecords_tfx_read_auto_infer_schema

Signed-off-by: Martin Bomio <martinbomio@spotify.com>
@raulchen
Copy link
Contributor

Discussed offline w/ @martinbomio. For 2.10 release, let's introduce 3 configs in DataContext, and disable the tfx read by default (to be safe), and we can enable the tfx read in future release.

enable_tfrecords_tfx_read
tfrecords_tfx_read_batch_size
tfrecords_tfx_read_auto_infer_schema

actually instead of adding flags in DataContext. I think it's better to just add them in read_tfrecords. that would be more self-contained and easier to manage. and I'm also concerned of adding too many flags in DataContext.

Also, since we are adding 3 parameters, it'd be cleaner to combine them in a single one e.g. tfx_read_options: TfxReadOptions, and mark it as experimental.

martinbomio and others added 2 commits February 28, 2024 14:32
Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Copy link
Contributor

@c21 c21 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM thanks @martinbomio!

logger = DatasetLogger(__name__)


@dataclass
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lint: add API annotation:

@PublicAPI(stability="alpha")

Comment on lines 763 to 775
# # if tfx read is enaled, we just return the dataset because, by default
# # tfx_read will be used in unit tests given that tfx-bsl dependency is
# # installed
# if tfx_read:
# return tf_ds

# read_op = tf_ds._plan._logical_plan.dag
# datasource_override = read_op._datasource
# datasource_override._tfx_read = tfx_read
# parallelism = read_opts.pop("parallelism", -1)
# ds = ray.data.read_datasource(datasource_override, parallelism=parallelism)

# return ds
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let's just delete this.

@c21
Copy link
Contributor

c21 commented Feb 28, 2024

Some CI test failures to fix:

_____________________ test_read_tfrecords[False-True-GZIP] _____________________
--
  |  
  | with_tf_schema = False, tfx_read = True, compression = 'GZIP'
  | ray_start_regular_shared = RayContext(dashboard_url='127.0.0.1:8265', python_version='3.9.16', ray_version='3.0.0.dev0', ray_commit='{{RAY_COMMIT_SHA}}')
  | tmp_path = PosixPath('/tmp/pytest-of-root/pytest-1/test_read_tfrecords_False_True1')
  |  
  | @pytest.mark.parametrize(
  | "with_tf_schema,tfx_read,compression",
  | [
  | (True, True, None),
  | (True, True, "GZIP"),
  | (True, False, None),
  | (False, True, None),
  | (False, True, "GZIP"),
  | (False, False, None),
  | ],
  | )
  | def test_read_tfrecords(
  | with_tf_schema,
  | tfx_read,
  | compression,
  | ray_start_regular_shared,
  | tmp_path,
  | ):
  | import pandas as pd
  | import tensorflow as tf
  |  
  | example = tf_records_empty()[0]
  |  
  | tf_schema = None
  | if with_tf_schema:
  | tf_schema = _features_to_schema(example.features)
  |  
  | path = os.path.join(tmp_path, "data.tfrecords")
  | with tf.io.TFRecordWriter(
  | path=path, options=tf.io.TFRecordOptions(compression_type=compression)
  | ) as writer:
  | writer.write(example.SerializeToString())
  |  
  | arrow_open_stream_args = None
  | if compression:
  | arrow_open_stream_args = {"compression": compression}
  |  
  | ds = read_tfrecords_with_tfx_read_override(
  | path,
  | tf_schema=tf_schema,
  | tfx_read=tfx_read,
  | arrow_open_stream_args=arrow_open_stream_args,
  | )
  |  
  | >       df = ds.to_pandas()
  |  
  | python/ray/data/tests/test_tfrecords.py:386:
  | _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
  | /rayci/python/ray/data/dataset.py:4372: in to_pandas
  | count = self.count()
  | /rayci/python/ray/data/dataset.py:2486: in count
  | [get_num_rows.remote(block) for block in self.get_internal_block_refs()]
  | /rayci/python/ray/data/dataset.py:4628: in get_internal_block_refs
  | blocks = self._plan.execute().get_blocks()
  | /rayci/python/ray/data/_internal/lazy_block_list.py:293: in get_blocks
  | blocks, _ = self._get_blocks_with_metadata()
  | /rayci/python/ray/data/_internal/lazy_block_list.py:327: in _get_blocks_with_metadata
  | meta = ray.get(refs_list.pop(-1))
  | /rayci/python/ray/_private/auto_init_hook.py:21: in auto_init_wrapper
  | return fn(*args, **kwargs)
  | /rayci/python/ray/_private/auto_init_hook.py:21: in auto_init_wrapper
  | return fn(*args, **kwargs)
  | /rayci/python/ray/_private/client_mode_hook.py:103: in wrapper
  | return func(*args, **kwargs)
  | /rayci/python/ray/_private/worker.py:2647: in get
  | values, debugger_breakpoint = worker.get_objects(object_refs, timeout=timeout)
  | _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _
  |  
  | self = <ray._private.worker.Worker object at 0x7fae7ab34e20>
  | object_refs = [ObjectRef(c76a79b2875a7251ffffffffffffffffffffffff0100000002e1f505)]
  | timeout = None
  |  
  | def get_objects(
  | self,
  | object_refs: list,
  | timeout: Optional[float] = None,
  | ):
  | """Get the values in the object store associated with the IDs.
  |  
  | Return the values from the local object store for object_refs. This
  | will block until all the values for object_refs have been written to
  | the local object store.
  |  
  | Args:
  | object_refs: A list of the object refs
  | whose values should be retrieved.
  | timeout: The maximum amount of time in
  | seconds to wait before returning.
  | Returns:
  | list: List of deserialized objects
  | bytes: UUID of the debugger breakpoint we should drop
  | into or b"" if there is no breakpoint.
  | """
  | # Make sure that the values are object refs.
  | for object_ref in object_refs:
  | if not isinstance(object_ref, ObjectRef):
  | raise TypeError(
  | f"Attempting to call `get` on the value {object_ref}, "
  | "which is not an ray.ObjectRef."
  | )
  |  
  | timeout_ms = int(timeout * 1000) if timeout is not None else -1
  | data_metadata_pairs = self.core_worker.get_objects(
  | object_refs,
  | self.current_task_id,
  | timeout_ms,
  | )
  | debugger_breakpoint = b""
  | for data, metadata in data_metadata_pairs:
  | if metadata:
  | metadata_fields = metadata.split(b",")
  | if len(metadata_fields) >= 2 and metadata_fields[1].startswith(
  | ray_constants.OBJECT_METADATA_DEBUG_PREFIX
  | ):
  | debugger_breakpoint = metadata_fields[1][
  | len(ray_constants.OBJECT_METADATA_DEBUG_PREFIX) :
  | ]
  | values = self.deserialize_objects(data_metadata_pairs, object_refs)
  | for i, value in enumerate(values):
  | if isinstance(value, RayError):
  | if isinstance(value, ray.exceptions.ObjectLostError):
  | global_worker.core_worker.dump_object_store_memory_usage()
  | if isinstance(value, RayTaskError):
  | >                   raise value.as_instanceof_cause()
  | E                   ray.exceptions.RayTaskError(ModuleNotFoundError): ray::_execute_read_task_split() (pid=96061, ip=172.16.0.3)
  | E                     File "/rayci/python/ray/data/_internal/lazy_block_list.py", line 637, in _execute_read_task_split
  | E                       for block in blocks:
  | E                     File "/rayci/python/ray/data/datasource/datasource.py", line 164, in __call__
  | E                       yield from result
  | E                     File "/rayci/python/ray/data/_internal/execution/operators/map_transformer.py", line 430, in __call__
  | E                       for block in blocks:
  | E                     File "/rayci/python/ray/data/_internal/execution/operators/map_transformer.py", line 371, in __call__
  | E                       for data in iter:
  | E                     File "/rayci/python/ray/data/datasource/file_based_datasource.py", line 267, in read_task_fn
  | E                       yield from read_files(read_paths)
  | E                     File "/rayci/python/ray/data/datasource/file_based_datasource.py", line 236, in read_files
  | E                       for block in read_stream(f, read_path):
  | E                     File "/rayci/python/ray/data/datasource/tfrecords_datasource.py", line 74, in _read_stream
  | E                       yield from self._tfx_read_stream(f, path)
  | E                     File "/rayci/python/ray/data/datasource/tfrecords_datasource.py", line 102, in _tfx_read_stream
  | E                       from tfx_bsl.cc.tfx_bsl_extension.coders import ExamplesToRecordBatchDecoder
  | E                   ModuleNotFoundError: No module named 'tfx_bsl'


@@ -1654,13 +1663,31 @@ def read_tfrecords(
By default, the number of output blocks is dynamically decided based on
input data size and available resources. You shouldn't manually set this
value in most cases.

tfx_read_options: Specifies read options when reading TFRecord files with TFX.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
tfx_read_options: Specifies read options when reading TFRecord files with TFX.
tfx_read_options: [Experimental] Specifies read options when reading TFRecord files with TFX.

datasets in production use cases. To use this implementation you should
install tfx-bsl with:
1. `pip install tfx_bsl --no-dependencies`
2. DatasetContext.get_current().enable_tfrecords_tfx_read = True
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

update this comment?

"Please install tfx-bsl package with"
" `pip install tfx_bsl --no-dependencies`."
" This can help speed up the reading of large TFRecord files."
)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

when user has specified tfx_read_options but tfx_bsl isn't installed, it'd be better to just throw an exception.
And I guess it's okay to not check platform here, as tfx_bsl may support arm in the future.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it does not support it right now, I find it a lot safer to not assume it will

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

when user has specified tfx_read_options but tfx_bsl isn't installed, it'd be better to just throw an exception.

So you are suggestion I just remove the whole block of trying to import and logging a warning which will also remove the fallback?

# parallelism = read_opts.pop("parallelism", -1)
# ds = ray.data.read_datasource(datasource_override, parallelism=parallelism)

# return ds
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this seems not useful any more?

@c21 c21 changed the title [Data] feat: add performant way to read large tfrecord datasets [Data] Add performant way to read large tfrecord datasets Feb 29, 2024
@aslonnie aslonnie self-requested a review February 29, 2024 22:57
Copy link
Collaborator

@aslonnie aslonnie left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dockerfile looks fine.

@c21 c21 merged commit 2c37909 into ray-project:master Feb 29, 2024
9 checks passed
hebiao064 pushed a commit to hebiao064/ray that referenced this pull request Mar 12, 2024
…t#42277)

The main motivation for this PR is that ray.data.read_tfrcords yields suboptimal performance when reading large datasets.
This PR adds a default "fast" route for reading tf.records that relies on tfx-bsl decoder. This approach also infers the schema when no tf_schema is provided by doing a pass of the data to determine the cardinality of the feature lists.

Signed-off-by: Martin Bomio <martinbomio@spotify.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Martin <martinbomio@gmail.com>
Co-authored-by: Scott Lee <sjl@anyscale.com>
Co-authored-by: Cheng Su <scnju13@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

7 participants