def _get_block_metadata( self, paths: List[str], schema: Optional[Union[type, "pyarrow.lib.Schema"]], *, pieces: List["pyarrow.dataset.ParquetFileFragment"], prefetched_metadata: Optional[List["pyarrow.parquet.FileMetaData"]], ) -> BlockMetadata: if prefetched_metadata is not None and len(prefetched_metadata) == len( pieces): # Piece metadata was available, construct a normal # BlockMetadata. block_metadata = BlockMetadata( num_rows=sum(m.num_rows for m in prefetched_metadata), size_bytes=sum( sum( m.row_group(i).total_byte_size for i in range(m.num_row_groups)) for m in prefetched_metadata), schema=schema, input_files=paths, exec_stats=None, ) # Exec stats filled in later. else: # Piece metadata was not available, construct an empty # BlockMetadata. block_metadata = BlockMetadata( num_rows=None, size_bytes=None, schema=schema, input_files=paths, exec_stats=None, ) return block_metadata
def _map_block_split( block: Block, block_fn: BlockTransform, input_files: List[str], fn: Optional[UDF], *fn_args, **fn_kwargs, ) -> BlockPartition: output = [] stats = BlockExecStats.builder() if fn is not None: fn_args = (fn,) + fn_args for new_block in block_fn(block, *fn_args, **fn_kwargs): accessor = BlockAccessor.for_block(new_block) new_meta = BlockMetadata( num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=input_files, exec_stats=stats.build(), ) owner = DatasetContext.get_current().block_owner output.append((ray.put(new_block, _owner=owner), new_meta)) stats = BlockExecStats.builder() return output
def prepare_read( self, parallelism: int, dataset_factory: Callable[[], "tf.data.Dataset"], ) -> List[ReadTask]: """Return a read task that loads a TensorFlow dataset. Arguments: parallelism: This argument isn't used. dataset_factory: A no-argument function that returns the TensorFlow dataset to be read. """ if parallelism > 1: logger.warn( "`SimpleTensorFlowDatasource` doesn't support parallel reads. The " "`parallelism` argument will be ignored." ) def read_fn() -> Iterator[Block]: # Load the entire dataset into memory. block = list(dataset_factory()) # Store the data in a single block. yield block metadata = BlockMetadata( num_rows=None, size_bytes=None, schema=None, input_files=None, exec_stats=None, ) return [ReadTask(read_fn, metadata)]
def prepare_read(self, parallelism: int, n: int, num_columns: int) -> List[ReadTask]: _check_pyarrow_version() import pyarrow read_tasks: List[ReadTask] = [] block_size = max(1, n // parallelism) def make_block(count: int, num_columns: int) -> Block: return pyarrow.Table.from_arrays( np.random.randint(np.iinfo(np.int64).max, size=(num_columns, count), dtype=np.int64), names=[f"c_{i}" for i in range(num_columns)]) schema = pyarrow.Table.from_pydict( {f"c_{i}": [0] for i in range(num_columns)}).schema i = 0 while i < n: count = min(block_size, n - i) read_tasks.append( ReadTask(lambda count=count, num_columns=num_columns: make_block(count, num_columns), BlockMetadata(num_rows=count, size_bytes=8 * count * num_columns, schema=schema, input_files=None))) i += block_size return read_tasks
def _split_single_block( block_id: int, block: Block, meta: BlockMetadata, block_row: int, split_indices: List[int], ) -> Tuple[int, List[Tuple[ObjectRef[Block], BlockMetadata]]]: """Split the provided block at the given indices.""" split_result = [] block_accessor = BlockAccessor.for_block(block) prev_index = 0 # append one more entry at the last so we don't # need handle empty edge case. split_indices.append(block_row) for index in split_indices: logger.debug(f"slicing block {prev_index}:{index}") stats = BlockExecStats.builder() split_block = block_accessor.slice(prev_index, index, copy=True) accessor = BlockAccessor.for_block(split_block) split_meta = BlockMetadata( num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=meta.schema, input_files=meta.input_files, exec_stats=stats.build(), ) split_result.append((ray.put(split_block), split_meta)) prev_index = index return (block_id, split_result)
def prepare_read(self, parallelism: int): value = DatasetContext.get_current().foo meta = BlockMetadata(num_rows=1, size_bytes=8, schema=None, input_files=None) return [ReadTask(lambda: [[value]], meta)]
def process_block(self, block: Block, meta: BlockMetadata) -> (Block, BlockMetadata): new_block = fn(block) accessor = BlockAccessor.for_block(new_block) new_metadata = BlockMetadata(num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=meta.input_files) return new_block, new_metadata
def _map_block(block: Block, fn: Any, input_files: List[str]) -> (Block, BlockMetadata): new_block = fn(block) accessor = BlockAccessor.for_block(new_block) new_meta = BlockMetadata( num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=input_files) return new_block, new_meta
def prepare_read(self, parallelism: int, n: int, block_format: str = "list", tensor_shape: Tuple = (1, )) -> List[ReadTask]: read_tasks: List[ReadTask] = [] block_size = max(1, n // parallelism) # Example of a read task. In a real datasource, this would pull data # from an external system instead of generating dummy data. def make_block(start: int, count: int) -> Block: if block_format == "arrow": return pyarrow.Table.from_arrays( [np.arange(start, start + count)], names=["value"]) elif block_format == "tensor": tensor = TensorArray( np.ones(tensor_shape, dtype=np.int64) * np.expand_dims( np.arange(start, start + count), tuple(range(1, 1 + len(tensor_shape))))) return pyarrow.Table.from_pydict({"value": tensor}) else: return list(builtins.range(start, start + count)) i = 0 while i < n: count = min(block_size, n - i) if block_format == "arrow": _check_pyarrow_version() import pyarrow schema = pyarrow.Table.from_pydict({"value": [0]}).schema elif block_format == "tensor": _check_pyarrow_version() from ray.data.extensions import TensorArray import pyarrow tensor = TensorArray( np.ones(tensor_shape, dtype=np.int64) * np.expand_dims( np.arange(0, 10), tuple( range(1, 1 + len(tensor_shape))))) schema = pyarrow.Table.from_pydict({"value": tensor}).schema elif block_format == "list": schema = int else: raise ValueError("Unsupported block type", block_format) meta = BlockMetadata( num_rows=count, size_bytes=8 * count, schema=schema, input_files=None, exec_stats=None) read_tasks.append( ReadTask( lambda i=i, count=count: [make_block(i, count)], meta)) i += block_size return read_tasks
def _shuffle_reduce(*mapper_outputs: List[Block]) -> (Block, BlockMetadata): builder = DelegatingArrowBlockBuilder() for block in mapper_outputs: builder.add_block(block) new_block = builder.build() accessor = BlockAccessor.for_block(new_block) new_metadata = BlockMetadata(num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=None) return new_block, new_metadata
def _map_block_split(block: Block, fn: Any, input_files: List[str]) -> BlockPartition: output = [] for new_block in fn(block): accessor = BlockAccessor.for_block(new_block) new_meta = BlockMetadata(num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=input_files) owner = DatasetContext.get_current().block_owner output.append((ray.put(new_block, _owner=owner), new_meta)) return output
def read_datasource(datasource: Datasource[T], *, parallelism: int = 200, ray_remote_args: Dict[str, Any] = 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: remote_read = ray.remote(**ray_remote_args)(remote_read) else: remote_read = ray.remote(remote_read) calls: List[Callable[[], ObjectRef[Block]]] = [] metadata: List[BlockMetadata] = [] for task in read_tasks: calls.append(lambda task=task: remote_read.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 process_block( self, block: Block, input_files: List[str] ) -> Iterable[Tuple[Block, BlockMetadata]]: output = [] for new_block in fn(block): accessor = BlockAccessor.for_block(new_block) new_metadata = BlockMetadata( num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=input_files) owner = DatasetContext.get_current().block_owner output.append((ray.put(new_block, _owner=owner), new_metadata)) return output
def prepare_read(self, parallelism: int, **read_args): import pyarrow as pa def load_data(): data_raw = load_breast_cancer(as_frame=True) dataset_df = data_raw["data"] dataset_df["target"] = data_raw["target"] return [pa.Table.from_pandas(dataset_df)] meta = BlockMetadata( num_rows=None, size_bytes=None, schema=None, input_files=None, exec_stats=None, ) return [ReadTask(load_data, meta)]
def prepare_read( self, parallelism: int, n_per_block: int, ) -> List[ReadTask]: read_tasks: List[ReadTask] = [] meta = BlockMetadata( num_rows=1, size_bytes=n_per_block, schema=None, input_files=None, exec_stats=None, ) for _ in range(parallelism): read_tasks.append( ReadTask(lambda: [[np.ones(n_per_block, dtype=np.uint8)]], meta)) return read_tasks
def _get_block_metadata( self, paths: List[str], schema: Optional[Union[type, "pyarrow.lib.Schema"]], *, rows_per_file: Optional[int], file_sizes: List[Optional[int]], ) -> BlockMetadata: if rows_per_file is None: num_rows = None else: num_rows = len(paths) * rows_per_file return BlockMetadata( num_rows=num_rows, size_bytes=None if None in file_sizes else sum(file_sizes), schema=schema, input_files=paths, exec_stats=None, ) # Exec stats filled in later.
def reduce(random_shuffle: bool, random_seed: Optional[int], *mapper_outputs: List[Block]) -> (Block, BlockMetadata): stats = BlockExecStats.builder() builder = DelegatingBlockBuilder() for block in mapper_outputs: builder.add_block(block) new_block = builder.build() accessor = BlockAccessor.for_block(new_block) if random_shuffle: new_block = accessor.random_shuffle( random_seed if random_seed is not None else None) accessor = BlockAccessor.for_block(new_block) new_metadata = BlockMetadata( num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=accessor.schema(), input_files=None, exec_stats=stats.build(), ) return new_block, new_metadata
def _merge( reduce_fn, *all_mapper_outputs: List[List[Block]], reduce_args: Optional[List[Any]] = None, ) -> List[Union[BlockMetadata, Block]]: """ Returns list of [BlockMetadata, O1, O2, O3, ...output_num_blocks]. """ assert (len({ len(mapper_outputs) for mapper_outputs in all_mapper_outputs }) == 1), "Received different number of map inputs" stats = BlockExecStats.builder() if not reduce_args: reduce_args = [] num_rows = 0 size_bytes = 0 schema = None for i, mapper_outputs in enumerate(zip(*all_mapper_outputs)): block, meta = reduce_fn(*reduce_args, *mapper_outputs, partial_reduce=True) yield block block = BlockAccessor.for_block(block) num_rows += block.num_rows() size_bytes += block.size_bytes() schema = block.schema() del block yield BlockMetadata( num_rows=num_rows, size_bytes=size_bytes, schema=schema, input_files=None, exec_stats=stats.build(), )
def get_read_tasks( self, parallelism: int, ) -> List[ReadTask]: read_tasks: List[ReadTask] = [] n = self._n block_format = self._block_format tensor_shape = self._tensor_shape block_size = max(1, n // parallelism) # Example of a read task. In a real datasource, this would pull data # from an external system instead of generating dummy data. def make_block(start: int, count: int) -> Block: if block_format == "arrow": import pyarrow as pa return pa.Table.from_arrays( [np.arange(start, start + count)], names=["value"] ) elif block_format == "tensor": import pyarrow as pa tensor = np.ones(tensor_shape, dtype=np.int64) * np.expand_dims( np.arange(start, start + count), tuple(range(1, 1 + len(tensor_shape))), ) return BlockAccessor.batch_to_block(tensor) else: return list(builtins.range(start, start + count)) i = 0 while i < n: count = min(block_size, n - i) if block_format == "arrow": _check_pyarrow_version() import pyarrow as pa schema = pa.Table.from_pydict({"value": [0]}).schema elif block_format == "tensor": _check_pyarrow_version() import pyarrow as pa tensor = np.ones(tensor_shape, dtype=np.int64) * np.expand_dims( np.arange(0, 10), tuple(range(1, 1 + len(tensor_shape))) ) schema = BlockAccessor.batch_to_block(tensor).schema elif block_format == "list": schema = int else: raise ValueError("Unsupported block type", block_format) if block_format == "tensor": element_size = np.product(tensor_shape) else: element_size = 1 meta = BlockMetadata( num_rows=count, size_bytes=8 * count * element_size, schema=schema, input_files=None, exec_stats=None, ) read_tasks.append( ReadTask(lambda i=i, count=count: [make_block(i, count)], meta) ) i += block_size return read_tasks
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 read_datasource(datasource: Datasource[T], *, parallelism: int = 200, ray_remote_args: Dict[str, Any] = 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) read_spread_custom_resource_labels = os.getenv( "RAY_DATASETS_READ_SPREAD_CUSTOM_RESOURCE_LABELS", None) if read_spread_custom_resource_labels is not None: read_spread_custom_resource_labels = ( read_spread_custom_resource_labels.split(",")) round_robin_resource_provider = itertools.cycle( map(lambda resource: {resource: 0.001}, read_spread_custom_resource_labels)) else: round_robin_resource_provider = itertools.repeat({}) resource_iter = iter(round_robin_resource_provider) 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)