Dataset pipelines allow Dataset transformations to be executed incrementally on *windows* of the base data, instead of on all of the data at once. This can be used for streaming data loading into ML training, or to execute batch transformations on large datasets without needing to load the entire dataset into cluster memory.
Dataset pipelines can be read in a streaming fashion by one consumer, or split into multiple sub-pipelines and read in parallel by multiple consumers for distributed traininmg.
A `DatasetPipeline <package-ref.html#datasetpipeline-api>`__ can be constructed in two ways: either by pipelining the execution of an existing Dataset (via ``Dataset.window``), or generating repeats of an existing Dataset (via ``Dataset.repeat``). Similar to Datasets, you can freely pass DatasetPipelines between Ray tasks, actors, and libraries. Get started with this synthetic data example:
..code-block:: python
import ray
def func1(i: int) -> int:
return i + 1
def func2(i: int) -> int:
return i * 2
def func3(i: int) -> int:
return i % 3
# Create a dataset and then create a pipeline from it.
You can also create a DatasetPipeline from a custom iterator over dataset creators using ``DatasetPipeline.from_iterable``. For example, this is how you would implement ``Dataset.repeat`` and ``Dataset.window`` using ``from_iterable``:
..code-block:: python
import ray
from ray.data.dataset_pipeline import DatasetPipeline
# Equivalent to ray.data.range(1000).repeat(times=4)
While most Dataset operations are per-row (e.g., map, filter), some operations apply to the Dataset as a whole (e.g., sort, shuffle). Per-row operations apply to rows in the pipeline independently in the same way they do in a normal Dataset. However, when used in a pipeline, holistic transforms like shuffle are applied separately to each window in the pipeline:
In this example, we pipeline the execution of a three-stage Dataset application to minimize GPU idle time. Let's take a look at a simple batch inference example:
Ignoring the output, the above script has three separate stages: loading, preprocessing, and inference. Assuming we have a fixed-sized cluster, and that each stage takes 100 seconds each, the cluster GPUs will be idle for the first 200 seconds of execution:
We can optimize this by *pipelining* the execution of the dataset with the ``.window()`` call, which returns a DatasetPipeline instead of a Dataset object. The pipeline supports similar transformations to the original Dataset:
Here we specified ``blocks_per_window=2``, which means that the Dataset is split into smaller sub-Datasets of two blocks each. Each transformation or *stage* of the pipeline is operating over these two-block Datasets in parallel. This means batch inference processing can start as soon as two blocks are read and preprocessed, greatly reducing the GPU idle time:
..image:: images/dataset-pipeline-2.svg
Pipelined Writes
~~~~~~~~~~~~~~~~
When calling ``write_<datasource>()`` on a pipeline, data is written separately for each window. This means that in the above example, JSON files will start being written as soon as the first window is finished, in a incremental / pipelined way.
Tune the throughput vs latency of your pipeline with the ``blocks_per_window`` setting. As a rule of thumb, higher parallelism settings perform better, however ``blocks_per_window == num_blocks`` effectively disables pipelining, since the DatasetPipeline will only contain a single Dataset. The other extreme is setting ``blocks_per_window=1``, which minimizes the latency to initial output but only allows one concurrent transformation task per stage:
..image:: images/dataset-pipeline-3.svg
You can also specify the size of each window using ``bytes_per_window``. In this mode, Datasets will determine the size of each window based on the target byte size, giving each window at least 1 block but not otherwise exceeding the target bytes per window. This mode can be useful to limit the memory usage of a pipeline. As a rule of thumb, the cluster memory should be at least 2-5x the window size to avoid spilling.
..code-block:: python
# Create a DatasetPipeline with up to 10GB of data per window.
pipe: DatasetPipeline = ray.data \
.read_binary_files("s3://bucket/image-dir") \
.window(bytes_per_window=10e9)
# -> INFO -- Created DatasetPipeline with 73 windows: 9120MiB min, 9431MiB max, 9287MiB mean
Datasets will warn you if the windows are too large or each window has insufficient parallelism (too few blocks). Check out the reported statistics for window size and blocks per window to ensure efficient pipeline execution.
Pipelines for ML Ingest
=======================
Dataset pipelines can also be used for streaming data loading into distributed training in Ray.
..note::
Ray Train is the standard libary for distributed training in Ray. Train will automatically create
and split DatasetPipelines for you. See :ref:`Configuring Training Datasets <air-ingest>`
for the recommended way to get started with distributed training.
Splitting pipelines for distributed ingest
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
Similar to how you can ``.split()`` a Dataset, you can also split a DatasetPipeline with the same method call. This returns a number of DatasetPipeline shards that share a common parent pipeline. Each shard can be passed to a remote task or actor.
**Code**:
..code-block:: python
# Create a pipeline that loops over its source dataset indefinitely.
workers = [TrainingWorker.remote(rank, s) for rank, s in enumerate(shards)]
...
**Pipeline**:
..image:: images/dataset-repeat-2.svg
Handling Epochs
~~~~~~~~~~~~~~~
It's common in ML training to want to divide data ingest into epochs, or repetitions over the original source dataset.
DatasetPipeline provides a convenient ``.iter_epochs()`` method that can be used to split up the pipeline into epoch-delimited pipeline segments.
Epochs are defined by the last call to ``.repeat()`` in a pipeline, for example:
..code-block:: python
pipe = ray.data.from_items([0, 1, 2, 3, 4]) \
.repeat(3) \
.random_shuffle_each_window()
for i, epoch in enumerate(pipe.iter_epochs()):
print("Epoch {}", i)
for row in epoch.iter_rows():
print(row)
# ->
# Epoch 0
# 2
# 1
# 3
# 4
# 0
# Epoch 1
# 3
# 4
# 0
# 2
# 1
# Epoch 2
# 3
# 2
# 4
# 1
# 0
Note that while epochs commonly consist of a single window, they can also contain multiple windows if ``.window()`` is used or there are multiple ``.repeat()`` calls.