Skip to content

Latest commit

 

History

History
157 lines (99 loc) · 9.8 KB

key-concepts.rst

File metadata and controls

157 lines (99 loc) · 9.8 KB

Key Concepts

To work with Ray Datasets, you need to understand how Datasets and Dataset Pipelines work. You might also be interested to learn about the execution model of Ray Datasets operations.

Datasets

Ray Datasets implements Distributed Arrow. A Dataset consists of a list of Ray object references to blocks. Each block holds a set of items in either an Arrow table or a Python list (for Arrow incompatible objects). Having multiple blocks in a dataset allows for parallel transformation and ingest of the data (e.g., into Ray Train <train-docs> for ML training).

The following figure visualizes a Dataset that has three Arrow table blocks, each block holding 1000 rows each:

image

Since a Dataset is just a list of Ray object references, it can be freely passed between Ray tasks, actors, and libraries like any other object reference. This flexibility is a unique characteristic of Ray Datasets.

Compared to Spark RDDs and Dask Bags, Ray Datasets offers a more basic set of features, and executes operations eagerly for simplicity. It is intended that users cast Datasets into more feature-rich dataframe types (e.g., ds.to_dask()) for advanced operations.

Dataset Pipelines

Datasets execute their transformations synchronously in blocking calls. However, it can be useful to overlap dataset computations with output. This can be done with a DatasetPipeline.

A DatasetPipeline is an unified iterator over a (potentially infinite) sequence of Ray Datasets, each of which represents a window over the original data. Conceptually it is similar to a Spark DStream, but manages execution over a bounded amount of source data instead of an unbounded stream. Ray computes each dataset window on-demand and stitches their output together into a single logical data iterator. DatasetPipeline implements most of the same transformation and output methods as Datasets (e.g., map, filter, split, iter_rows, to_torch, etc.).

Dataset Execution Model

This page overviews the execution model of Datasets, which may be useful for understanding and tuning performance.

Reading Data

Datasets uses Ray tasks to read data from remote storage. When reading from a file-based datasource (e.g., S3, GCS), it creates a number of read tasks equal to the specified read parallelism (200 by default). One or more files will be assigned to each read task. Each read task reads its assigned files and produces one or more output blocks (Ray objects):

image

In the common case, each read task produces a single output block. Read tasks may split the output into multiple blocks if the data exceeds the target max block size (2GiB by default). This automatic block splitting avoids out-of-memory errors when reading very large single files (e.g., a 100-gigabyte CSV file). All of the built-in datasources except for JSON currently support automatic block splitting.

Note

Block splitting is off by default. See the performance section <data_performance_tips> on how to enable block splitting (beta).

Deferred Read Task Execution

When a Dataset is created using ray.data.read_*, only the first read task will be executed initially. This avoids blocking Dataset creation on the reading of all data files, enabling inspection functions like ds.schema() and ds.show() to be used right away. Executing further transformations on the Dataset will trigger execution of all read tasks.

Dataset Transforms

Datasets use either Ray tasks or Ray actors to transform datasets (i.e., for .map, .flat_map, or .map_batches). By default, tasks are used (compute="tasks"). Actors can be specified with compute="actors", in which case an autoscaling pool of Ray actors will be used to apply transformations. Using actors allows for expensive state initialization (e.g., for GPU-based tasks) to be re-used. Whichever compute strategy is used, each map task generally takes in one block and produces one or more output blocks. The output block splitting rule is the same as for file reads (blocks are split after hitting the target max block size of 2GiB):

image

Shuffling Data

Certain operations like .sort and .groupby require data blocks to be partitioned by value. Datasets executes this in three phases. First, a wave of sampling tasks determines suitable partition boundaries based on a random sample of data. Second, map tasks divide each input block into a number of output blocks equal to the number of reduce tasks. Third, reduce tasks take assigned output blocks from each map task and combines them into one block. Overall, this strategy generates O(n^2) intermediate objects where n is the number of input blocks.

You can also change the partitioning of a Dataset using .random_shuffle or .repartition. The former should be used if you want to randomize the order of elements in the dataset. The second should be used if you only want to equalize the size of the Dataset blocks (e.g., after a read or transformation that may skew the distribution of block sizes). Note that repartition has two modes, shuffle=False, which performs the minimal data movement needed to equalize block sizes, and shuffle=True, which performs a full (non-random) distributed shuffle:

image

Memory Management

This section deals with how Datasets manages execution and object store memory.

Execution Memory

During execution, certain types of intermediate data must fit in memory. This includes the input block of a task, as well as at least one of the output blocks of the task (when a task has multiple output blocks, only one needs to fit in memory at any given time). The input block consumes object stored shared memory (Python heap memory for non-Arrow data). The output blocks consume Python heap memory (prior to putting in the object store) as well as object store memory (after being put in the object store).

This means that large block sizes can lead to potential out-of-memory situations. To avoid OOM errors, Datasets can split blocks during map and read tasks into pieces smaller than the target max block size. In some cases, this splitting is not possible (e.g., if a single item in a block is extremely large, or the function given to .map_batches returns a very large batch). To avoid these issues, make sure no single item in your Datasets is too large, and always call .map_batches with batch size small enough such that the output batch can comfortably fit into memory.

Note

Block splitting is off by default. See the performance section <data_performance_tips> on how to enable block splitting (beta).

Object Store Memory

Datasets uses the Ray object store to store data blocks, which means it inherits the memory management features of the Ray object store. This section discusses the relevant features:

Object Spilling: Since Datasets uses the Ray object store to store data blocks, any blocks that can't fit into object store memory are automatically spilled to disk. The objects are automatically reloaded when needed by downstream compute tasks:

image

Locality Scheduling: Ray will preferentially schedule compute tasks on nodes that already have a local copy of the object, reducing the need to transfer objects between nodes in the cluster.

Reference Counting: Dataset blocks are kept alive by object store reference counting as long as there is any Dataset that references them. To free memory, delete any Python references to the Dataset object.

Load Balancing: Datasets uses Ray scheduling hints to spread read tasks out across the cluster to balance memory usage.

Stage Fusion Optimization

To avoid unnecessary data movement in the distributed setting, Dataset pipelines will fuse compatible stages (i.e., stages with the same compute strategy and resource specifications). Read and map-like stages are always fused if possible. All-to-all dataset transformations such as random_shuffle can be fused with earlier map-like stages, but not later stages. For Datasets, only read stages are fused. This is since non-pipelined Datasets are eagerly executed except for their initial read stage.

You can tell if stage fusion is enabled by checking the Dataset stats <data_performance_tips> and looking for fused stages (e.g., read->map_batches).

Stage N read->map_batches->shuffle_map: N/N blocks executed in T
* Remote wall time: T min, T max, T mean, T total
* Remote cpu time: T min, T max, T mean, T total
* Output num rows: N min, N max, N mean, N total