def read_datasource( datasource: Datasource[T], *, parallelism: int = 200, ray_remote_args: Dict[str, Any] = None, _spread_resource_prefix: Optional[str] = None, **read_args, ) -> Dataset[T]: """Read a dataset from a custom data source. Args: datasource: The datasource to read data from. parallelism: The requested parallelism of the read. Parallelism may be limited by the available partitioning of the datasource. read_args: Additional kwargs to pass to the datasource impl. ray_remote_args: kwargs passed to ray.remote in the read tasks. Returns: Dataset holding the data read from the datasource. """ # TODO(ekl) remove this feature flag. if "RAY_DATASET_FORCE_LOCAL_METADATA" in os.environ: read_tasks = datasource.prepare_read(parallelism, **read_args) else: # Prepare read in a remote task so that in Ray client mode, we aren't # attempting metadata resolution from the client machine. ctx = DatasetContext.get_current() prepare_read = cached_remote_fn(_prepare_read, retry_exceptions=False, num_cpus=0) read_tasks = ray.get( prepare_read.remote(datasource, ctx, parallelism, _wrap_s3_filesystem_workaround(read_args))) context = DatasetContext.get_current() stats_actor = get_or_create_stats_actor() stats_uuid = uuid.uuid4() stats_actor.record_start.remote(stats_uuid) def remote_read(i: int, task: ReadTask) -> MaybeBlockPartition: DatasetContext._set_current(context) stats = BlockExecStats.builder() # Execute the read task. block = task() if context.block_splitting_enabled: metadata = task.get_metadata() metadata.exec_stats = stats.build() else: metadata = BlockAccessor.for_block(block).get_metadata( input_files=task.get_metadata().input_files, exec_stats=stats.build()) stats_actor.record_task.remote(stats_uuid, i, metadata) return block if ray_remote_args is None: ray_remote_args = {} # Increase the read parallelism by default to maximize IO throughput. This # is particularly important when reading from e.g., remote storage. if "num_cpus" not in ray_remote_args: # Note that the too many workers warning triggers at 4x subscription, # so we go at 0.5 to avoid the warning message. ray_remote_args["num_cpus"] = 0.5 remote_read = cached_remote_fn(remote_read) if _spread_resource_prefix is not None: # Use given spread resource prefix for round-robin resource-based # scheduling. nodes = ray.nodes() resource_iter = _get_spread_resources_iter(nodes, _spread_resource_prefix, ray_remote_args) else: # If no spread resource prefix given, yield an empty dictionary. resource_iter = itertools.repeat({}) calls: List[Callable[[], ObjectRef[MaybeBlockPartition]]] = [] metadata: List[BlockPartitionMetadata] = [] for i, task in enumerate(read_tasks): calls.append( lambda i=i, task=task, resources=next(resource_iter): remote_read. options(**ray_remote_args, resources=resources).remote(i, task)) metadata.append(task.get_metadata()) block_list = LazyBlockList(calls, metadata) # Get the schema from the first block synchronously. if metadata and metadata[0].schema is None: block_list.ensure_schema_for_first_block() return Dataset( block_list, 0, DatasetStats( stages={"read": metadata}, parent=None, stats_actor=stats_actor, stats_uuid=stats_uuid, ), )
def simple_shuffle( input_blocks: BlockList, block_udf: Optional[Callable[[Block], Iterable[Block]]], output_num_blocks: int, *, random_shuffle: bool = False, random_seed: Optional[int] = None, map_ray_remote_args: Optional[Dict[str, Any]] = None, reduce_ray_remote_args: Optional[Dict[str, Any]] = None, _spread_resource_prefix: Optional[str] = None ) -> Tuple[BlockList, Dict[str, List[BlockMetadata]]]: input_blocks = input_blocks.get_blocks() if map_ray_remote_args is None: map_ray_remote_args = {} if reduce_ray_remote_args is None: reduce_ray_remote_args = {} if "scheduling_strategy" not in reduce_ray_remote_args: reduce_ray_remote_args = reduce_ray_remote_args.copy() reduce_ray_remote_args["scheduling_strategy"] = "SPREAD" input_num_blocks = len(input_blocks) if _spread_resource_prefix is not None: # Use given spread resource prefix for round-robin resource-based # scheduling. nodes = ray.nodes() map_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, map_ray_remote_args ) reduce_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, reduce_ray_remote_args ) else: # If no spread resource prefix given, yield an empty dictionary. map_resource_iter, reduce_resource_iter = itertools.tee(itertools.repeat({}), 2) shuffle_map = cached_remote_fn(_shuffle_map) shuffle_reduce = cached_remote_fn(_shuffle_reduce) map_bar = ProgressBar("Shuffle Map", position=0, total=input_num_blocks) shuffle_map_out = [ shuffle_map.options( **map_ray_remote_args, num_returns=1 + output_num_blocks, resources=next(map_resource_iter) ).remote(block, block_udf, i, output_num_blocks, random_shuffle, random_seed) for i, block in enumerate(input_blocks) ] # The first item returned is the BlockMetadata. shuffle_map_metadata = [] for i, refs in enumerate(shuffle_map_out): shuffle_map_metadata.append(refs[0]) shuffle_map_out[i] = refs[1:] # Eagerly delete the input block references in order to eagerly release # the blocks' memory. del input_blocks shuffle_map_metadata = map_bar.fetch_until_complete(shuffle_map_metadata) map_bar.close() # Randomize the reduce order of the blocks. if random_shuffle: random = np.random.RandomState(random_seed) random.shuffle(shuffle_map_out) reduce_bar = ProgressBar("Shuffle Reduce", position=0, total=output_num_blocks) shuffle_reduce_out = [ shuffle_reduce.options( **reduce_ray_remote_args, num_returns=2, resources=next(reduce_resource_iter) ).remote(*[shuffle_map_out[i][j] for i in range(input_num_blocks)]) for j in range(output_num_blocks) ] # Eagerly delete the map block references in order to eagerly release # the blocks' memory. del shuffle_map_out new_blocks, new_metadata = zip(*shuffle_reduce_out) reduce_bar.block_until_complete(list(new_blocks)) new_metadata = ray.get(list(new_metadata)) reduce_bar.close() stats = { "map": shuffle_map_metadata, "reduce": new_metadata, } return BlockList(list(new_blocks), list(new_metadata)), stats
def read_datasource(datasource: Datasource[T], *, parallelism: int = 200, ray_remote_args: Dict[str, Any] = None, _spread_resource_prefix: Optional[str] = None, **read_args) -> Dataset[T]: """Read a dataset from a custom data source. Args: datasource: The datasource to read data from. parallelism: The requested parallelism of the read. read_args: Additional kwargs to pass to the datasource impl. ray_remote_args: kwargs passed to ray.remote in the read tasks. Returns: Dataset holding the data read from the datasource. """ read_tasks = datasource.prepare_read(parallelism, **read_args) def remote_read(task: ReadTask) -> Block: return task() if ray_remote_args is None: ray_remote_args = {} # Increase the read parallelism by default to maximize IO throughput. This # is particularly important when reading from e.g., remote storage. if "num_cpus" not in ray_remote_args: # Note that the too many workers warning triggers at 4x subscription, # so we go at 0.5 to avoid the warning message. ray_remote_args["num_cpus"] = 0.5 remote_read = cached_remote_fn(remote_read) if _spread_resource_prefix is not None: # Use given spread resource prefix for round-robin resource-based # scheduling. nodes = ray.nodes() resource_iter = _get_spread_resources_iter(nodes, _spread_resource_prefix, ray_remote_args) else: # If no spread resource prefix given, yield an empty dictionary. resource_iter = itertools.repeat({}) calls: List[Callable[[], ObjectRef[Block]]] = [] metadata: List[BlockMetadata] = [] for task in read_tasks: calls.append( lambda task=task, resources=next(resource_iter): remote_read. options(**ray_remote_args, resources=resources).remote(task)) metadata.append(task.get_metadata()) block_list = LazyBlockList(calls, metadata) # Get the schema from the first block synchronously. if metadata and metadata[0].schema is None: get_schema = cached_remote_fn(_get_schema) schema0 = ray.get(get_schema.remote(next(iter(block_list)))) block_list.set_metadata( 0, BlockMetadata( num_rows=metadata[0].num_rows, size_bytes=metadata[0].size_bytes, schema=schema0, input_files=metadata[0].input_files, )) return Dataset(block_list)
def simple_shuffle( input_blocks: BlockList[T], output_num_blocks: int, *, random_shuffle: bool = False, random_seed: Optional[int] = None, map_ray_remote_args: Optional[Dict[str, Any]] = None, reduce_ray_remote_args: Optional[Dict[str, Any]] = None, _spread_resource_prefix: Optional[str] = None) -> BlockList[T]: if map_ray_remote_args is None: map_ray_remote_args = {} if reduce_ray_remote_args is None: reduce_ray_remote_args = {} input_num_blocks = len(input_blocks) if _spread_resource_prefix is not None: # Use given spread resource prefix for round-robin resource-based # scheduling. nodes = ray.nodes() map_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, map_ray_remote_args) reduce_resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, reduce_ray_remote_args) else: # If no spread resource prefix given, yield an empty dictionary. map_resource_iter, reduce_resource_iter = itertools.tee( itertools.repeat({}), 2) shuffle_map = cached_remote_fn(_shuffle_map) shuffle_reduce = cached_remote_fn(_shuffle_reduce) map_bar = ProgressBar("Shuffle Map", position=0, total=input_num_blocks) shuffle_map_out = [ shuffle_map.options(**map_ray_remote_args, num_returns=output_num_blocks, resources=next(map_resource_iter)).remote( block, i, output_num_blocks, random_shuffle, random_seed) for i, block in enumerate(input_blocks) ] # Eagerly delete the input block references in order to eagerly release # the blocks' memory. del input_blocks if output_num_blocks == 1: # Handle the num_returns=1 edge case which doesn't return a list. shuffle_map_out = [[x] for x in shuffle_map_out] map_bar.block_until_complete([x[0] for x in shuffle_map_out]) map_bar.close() # Randomize the reduce order of the blocks. if random_shuffle: random = np.random.RandomState(random_seed) random.shuffle(shuffle_map_out) reduce_bar = ProgressBar("Shuffle Reduce", position=0, total=output_num_blocks) shuffle_reduce_out = [ shuffle_reduce.options(**reduce_ray_remote_args, num_returns=2, resources=next(reduce_resource_iter)). remote(*[shuffle_map_out[i][j] for i in range(input_num_blocks)]) for j in range(output_num_blocks) ] # Eagerly delete the map block references in order to eagerly release # the blocks' memory. del shuffle_map_out new_blocks, new_metadata = zip(*shuffle_reduce_out) reduce_bar.block_until_complete(list(new_blocks)) new_metadata = ray.get(list(new_metadata)) reduce_bar.close() return BlockList(list(new_blocks), list(new_metadata))
def read_datasource( datasource: Datasource[T], *, parallelism: int = 200, ray_remote_args: Dict[str, Any] = None, _spread_resource_prefix: Optional[str] = None, **read_args, ) -> Dataset[T]: """Read a dataset from a custom data source. Args: datasource: The datasource to read data from. parallelism: The requested parallelism of the read. Parallelism may be limited by the available partitioning of the datasource. read_args: Additional kwargs to pass to the datasource impl. ray_remote_args: kwargs passed to ray.remote in the read tasks. Returns: Dataset holding the data read from the datasource. """ # TODO(ekl) remove this feature flag. force_local = "RAY_DATASET_FORCE_LOCAL_METADATA" in os.environ pa_ds = _lazy_import_pyarrow_dataset() if pa_ds: partitioning = read_args.get("dataset_kwargs", {}).get("partitioning", None) if isinstance(partitioning, pa_ds.Partitioning): logger.info( "Forcing local metadata resolution since the provided partitioning " f"{partitioning} is not serializable." ) force_local = True if force_local: read_tasks = datasource.prepare_read(parallelism, **read_args) else: # Prepare read in a remote task so that in Ray client mode, we aren't # attempting metadata resolution from the client machine. ctx = DatasetContext.get_current() prepare_read = cached_remote_fn( _prepare_read, retry_exceptions=False, num_cpus=0 ) read_tasks = ray.get( prepare_read.remote( datasource, ctx, parallelism, _wrap_arrow_serialization_workaround(read_args), ) ) context = DatasetContext.get_current() stats_actor = get_or_create_stats_actor() stats_uuid = uuid.uuid4() stats_actor.record_start.remote(stats_uuid) def remote_read(i: int, task: ReadTask, stats_actor) -> MaybeBlockPartition: DatasetContext._set_current(context) stats = BlockExecStats.builder() # Execute the read task. block = task() if context.block_splitting_enabled: metadata = task.get_metadata() metadata.exec_stats = stats.build() else: metadata = BlockAccessor.for_block(block).get_metadata( input_files=task.get_metadata().input_files, exec_stats=stats.build() ) stats_actor.record_task.remote(stats_uuid, i, metadata) return block if ray_remote_args is None: ray_remote_args = {} if "scheduling_strategy" not in ray_remote_args: ray_remote_args["scheduling_strategy"] = "SPREAD" remote_read = cached_remote_fn(remote_read) if _spread_resource_prefix is not None: if context.optimize_fuse_stages: logger.warning( "_spread_resource_prefix has no effect when optimize_fuse_stages " "is enabled. Tasks are spread by default." ) # Use given spread resource prefix for round-robin resource-based # scheduling. nodes = ray.nodes() resource_iter = _get_spread_resources_iter( nodes, _spread_resource_prefix, ray_remote_args ) else: # If no spread resource prefix given, yield an empty dictionary. resource_iter = itertools.repeat({}) calls: List[Callable[[], ObjectRef[MaybeBlockPartition]]] = [] metadata: List[BlockPartitionMetadata] = [] for i, task in enumerate(read_tasks): calls.append( lambda i=i, task=task, resources=next(resource_iter): remote_read.options( **ray_remote_args, resources=resources ).remote(i, task, stats_actor) ) metadata.append(task.get_metadata()) block_list = LazyBlockList(calls, metadata) # TODO(ekl) consider refactoring LazyBlockList to take read_tasks explicitly. block_list._read_tasks = read_tasks block_list._read_remote_args = ray_remote_args # Get the schema from the first block synchronously. if metadata and metadata[0].schema is None: block_list.ensure_schema_for_first_block() stats = DatasetStats( stages={"read": metadata}, parent=None, stats_actor=stats_actor, stats_uuid=stats_uuid, ) return Dataset( ExecutionPlan(block_list, stats), 0, False, )