ray.data.Dataset.map#
- Dataset.map(fn: Callable[[Dict[str, Any]], Dict[str, Any]], *, compute: ComputeStrategy | None = None, fn_args: Iterable[Any] | None = None, fn_kwargs: Dict[str, Any] | None = None, fn_constructor_args: Iterable[Any] | None = None, fn_constructor_kwargs: Dict[str, Any] | None = None, num_cpus: float | None = None, num_gpus: float | None = None, memory: float | None = None, concurrency: int | Tuple[int, int] | Tuple[int, int, int] | None = None, ray_remote_args_fn: Callable[[], Dict[str, Any]] | None = None, **ray_remote_args) Dataset[source]#
- Apply the given function to each row of this dataset. - Use this method to transform your data. To learn more, see Transforming rows. - You can use either a function or a callable class to perform the transformation. For functions, Ray Data uses stateless Ray tasks. For classes, Ray Data uses stateful Ray actors. For more information, see Stateful Transforms. - Tip - If your transformation is vectorized like most NumPy or pandas operations, - map_batches()might be faster.- Warning - Specifying both - num_cpusand- num_gpusfor map tasks is experimental, and may result in scheduling or stability issues. Please report any issues to the Ray team.- Examples - import os from typing import Any, Dict import ray def parse_filename(row: Dict[str, Any]) -> Dict[str, Any]: row["filename"] = os.path.basename(row["path"]) return row ds = ( ray.data.read_images("s3://anonymous@ray-example-data/image-datasets/simple", include_paths=True) .map(parse_filename) ) print(ds.schema()) - Column Type ------ ---- image ArrowTensorTypeV2(shape=(32, 32, 3), dtype=uint8) path string filename string - Time complexity: O(dataset size / parallelism) - Parameters:
- fn – The function to apply to each row, or a class type that can be instantiated to create such a callable. 
- compute – - The compute strategy to use for the map operation. - If - computeis not specified for a function, will use- ray.data.TaskPoolStrategy()to launch concurrent tasks based on the available resources and number of input blocks.
- Use - ray.data.TaskPoolStrategy(size=n)to launch at most- nconcurrent Ray tasks.
- If - computeis not specified for a callable class, will use- ray.data.ActorPoolStrategy(min_size=1, max_size=None)to launch an autoscaling actor pool from 1 to unlimited workers.
- Use - ray.data.ActorPoolStrategy(size=n)to use a fixed size actor pool of- nworkers.
- Use - ray.data.ActorPoolStrategy(min_size=m, max_size=n)to use an autoscaling actor pool from- mto- nworkers.
- Use - ray.data.ActorPoolStrategy(min_size=m, max_size=n, initial_size=initial)to use an autoscaling actor pool from- mto- nworkers, with an initial size of- initial.
 
- fn_args – Positional arguments to pass to - fnafter the first argument. These arguments are top-level arguments to the underlying Ray task.
- fn_kwargs – Keyword arguments to pass to - fn. These arguments are top-level arguments to the underlying Ray task.
- fn_constructor_args – Positional arguments to pass to - fn’s constructor. You can only provide this if- fnis a callable class. These arguments are top-level arguments in the underlying Ray actor construction task.
- fn_constructor_kwargs – Keyword arguments to pass to - fn’s constructor. This can only be provided if- fnis a callable class. These arguments are top-level arguments in the underlying Ray actor construction task.
- num_cpus – The number of CPUs to reserve for each parallel map worker. 
- num_gpus – The number of GPUs to reserve for each parallel map worker. For example, specify - num_gpus=1to request 1 GPU for each parallel map worker.
- memory – The heap memory in bytes to reserve for each parallel map worker. 
- concurrency – This argument is deprecated. Use - computeargument.
- ray_remote_args_fn – A function that returns a dictionary of remote args passed to each map worker. The purpose of this argument is to generate dynamic arguments for each actor/task, and will be called each time prior to initializing the worker. Args returned from this dict will always override the args in - ray_remote_args. Note: this is an advanced, experimental feature.
- ray_remote_args – Additional resource requirements to request from Ray for each map worker. See - ray.remote()for details.
 
 - See also - flat_map()
- Call this method to create new rows from existing ones. Unlike - map(), a function passed to- flat_map()can return multiple rows.
- map_batches()
- Call this method to transform batches of data.