def sort_impl(blocks: BlockList[T], key: SortKeyT, descending: bool = False) -> BlockList[T]: if len(blocks) == 0: return BlockList([], []) if isinstance(key, str): key = [(key, "descending" if descending else "ascending")] if isinstance(key, list): descending = key[0][1] == "descending" num_mappers = len(blocks) num_reducers = num_mappers boundaries = sample_boundaries(blocks, key, num_reducers) if descending: boundaries.reverse() @ray.remote(num_returns=num_reducers) def sort_block(block, boundaries): return BlockAccessor.for_block(block).sort_and_partition( boundaries, key, descending) @ray.remote(num_returns=2) def merge_sorted_blocks(*blocks: List[Block[T]]) -> Block[T]: if len(blocks) == 1: blocks = blocks[0] # Python weirdness return BlockAccessor.for_block(blocks[0]).merge_sorted_blocks( list(blocks), key, descending) map_results = np.empty((num_mappers, num_reducers), dtype=object) for i, block in enumerate(blocks): map_results[i, :] = sort_block.remote(block, boundaries) map_bar = ProgressBar("Sort Map", len(map_results)) map_bar.block_until_complete([ret[0] for ret in map_results]) map_bar.close() reduce_results = [] for j in range(num_reducers): ret = merge_sorted_blocks.remote(*map_results[:, j].tolist()) reduce_results.append(ret) merge_bar = ProgressBar("Sort Merge", len(reduce_results)) merge_bar.block_until_complete([ret[0] for ret in reduce_results]) merge_bar.close() blocks = [b for b, _ in reduce_results] metadata = ray.get([m for _, m in reduce_results]) return BlockList(blocks, metadata)
def from_items(items: List[Any], parallelism: int = 200) -> Dataset[Any]: """Create a dataset from a list of local Python objects. Examples: >>> ray.data.from_items([1, 2, 3, 4, 5]) Args: items: List of local Python objects. parallelism: The amount of parallelism to use for the dataset. Returns: Dataset holding the items. """ block_size = max(1, len(items) // parallelism) blocks: List[ObjectRef[Block]] = [] metadata: List[BlockMetadata] = [] i = 0 while i < len(items): builder = DelegatingArrowBlockBuilder() for item in items[i:i + block_size]: builder.add(item) block = builder.build() blocks.append(ray.put(block)) metadata.append( BlockAccessor.for_block(block).get_metadata(input_files=None)) i += block_size return Dataset(BlockList(blocks, metadata))
def from_items(items: List[Any], parallelism: int = 200) -> Dataset[Any]: """Create a dataset from a list of local Python objects. Examples: >>> ds.from_items([1, 2, 3, 4, 5]) Args: items: List of local Python objects. parallelism: The amount of parallelism to use for the dataset. Returns: Dataset holding the items. """ block_size = max(1, len(items) // parallelism) blocks: List[ObjectRef[Block]] = [] metadata: List[BlockMetadata] = [] i = 0 while i < len(items): builder = SimpleBlock.builder() for item in items[i:i + block_size]: builder.add(item) block = builder.build() blocks.append(ray.put(block)) metadata.append( BlockMetadata(num_rows=block.num_rows(), size_bytes=block.size_bytes(), schema=type(items[0]), input_files=None)) i += block_size return Dataset(BlockList(blocks, metadata))
def apply(self, fn: Any, remote_args: dict, blocks: BlockList[Any]) -> BlockList[Any]: map_bar = ProgressBar("Map Progress", total=len(blocks)) kwargs = remote_args.copy() kwargs["num_returns"] = 2 @ray.remote(**kwargs) def wrapped_fn(block: Block, meta: 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=meta.input_files) return new_block, new_meta refs = [ wrapped_fn.remote(b, m) for b, m in zip(blocks, blocks.get_metadata()) ] new_blocks, new_metadata = zip(*refs) map_bar.block_until_complete(list(new_blocks)) new_metadata = ray.get(list(new_metadata)) return BlockList(list(new_blocks), list(new_metadata))
def simple_shuffle(input_blocks: BlockList[T], output_num_blocks: int) -> BlockList[T]: input_num_blocks = len(input_blocks) @ray.remote(num_returns=output_num_blocks) def shuffle_map(block: Block[T]) -> List[Block[T]]: block = BlockAccessor.for_block(block) slice_sz = max(1, math.ceil(block.num_rows() / output_num_blocks)) slices = [] for i in range(output_num_blocks): slices.append( block.slice(i * slice_sz, (i + 1) * slice_sz, copy=True)) num_rows = sum(BlockAccessor.for_block(s).num_rows() for s in slices) assert num_rows == block.num_rows(), (num_rows, block.num_rows()) # Needed to handle num_returns=1 edge case in Ray API. if len(slices) == 1: return slices[0] else: return slices @ray.remote(num_returns=2) def shuffle_reduce( *mapper_outputs: List[Block[T]]) -> (Block[T], BlockMetadata): builder = DelegatingArrowBlockBuilder() assert len(mapper_outputs) == input_num_blocks 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 map_bar = ProgressBar("Shuffle Map", position=0, total=input_num_blocks) shuffle_map_out = [shuffle_map.remote(block) for block in 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() reduce_bar = ProgressBar("Shuffle Reduce", position=0, total=output_num_blocks) shuffle_reduce_out = [ shuffle_reduce.remote( *[shuffle_map_out[i][j] for i in range(input_num_blocks)]) for j in range(output_num_blocks) ] 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 limit(self, limit: int) -> "Dataset[T]": """Limit the dataset to the first number of records specified. Examples: >>> ds.limit(100).map(lambda x: x * 2).take() Time complexity: O(limit specified) Args: limit: The size of the dataset to truncate to. Returns: The truncated dataset. """ @ray.remote def get_num_rows(block: Block[T]) -> int: block = BlockAccessor.for_block(block) return block.num_rows() @ray.remote(num_returns=2) def truncate(block: Block[T], meta: BlockMetadata, count: int) -> (Block[T], BlockMetadata): block = BlockAccessor.for_block(block) logger.debug("Truncating last block to size: {}".format(count)) new_block = block.slice(0, count, copy=True) accessor = BlockAccessor.for_block(new_block) new_meta = BlockMetadata( num_rows=accessor.num_rows(), size_bytes=accessor.size_bytes(), schema=meta.schema, input_files=meta.input_files) return new_block, new_meta count = 0 out_blocks = [] out_metadata = [] for b, m in zip(self._blocks, self._blocks.get_metadata()): if m.num_rows is None: num_rows = ray.get(get_num_rows.remote(b)) else: num_rows = m.num_rows if count + num_rows < limit: out_blocks.append(b) out_metadata.append(m) elif count + num_rows == limit: out_blocks.append(b) out_metadata.append(m) break else: new_block, new_metadata = truncate.remote(b, m, limit - count) out_blocks.append(new_block) out_metadata.append(ray.get(new_metadata)) break count += num_rows return Dataset(BlockList(out_blocks, out_metadata))
def read_csv(paths: Union[str, List[str]], filesystem: Optional["pyarrow.fs.FileSystem"] = None, parallelism: int = 200, **arrow_csv_args) -> Dataset[ArrowRow]: """Create an Arrow dataset from csv files. Examples: # Read a directory of files in remote storage. >>> ds.read_csv("s3://bucket/path") # Read multiple local files. >>> ds.read_csv(["/path/to/file1", "/path/to/file2"]) # Read multiple directories. >>> ds.read_csv(["s3://bucket/path1", "s3://bucket/path2"]) Args: paths: A single file/directory path or a list of file/directory paths. A list of paths can contain both files and directories. filesystem: The filesystem implementation to read from. parallelism: The amount of parallelism to use for the dataset. arrow_csv_args: Other csv read options to pass to pyarrow. Returns: Dataset holding Arrow records read from the specified paths. """ import pyarrow as pa from pyarrow import csv import numpy as np paths, filesystem = _resolve_paths_and_filesystem(paths, filesystem) @ray.remote(num_returns=2) def csv_read(read_paths: List[str]): logger.debug(f"Reading {len(read_paths)} files.") tables = [] for read_path in read_paths: with filesystem.open_input_file(read_path) as f: tables.append( csv.read_csv( f, read_options=csv.ReadOptions(use_threads=False), **arrow_csv_args)) block = ArrowBlock(pa.concat_tables(tables)) return block, block.get_metadata(input_files=read_paths) res = [ csv_read.remote(read_paths) for read_paths in np.array_split(paths, parallelism) if len(read_paths) > 0 ] blocks, metadata = zip(*res) return Dataset(BlockList(blocks, ray.get(list(metadata))))
def from_arrow(tables: List[ObjectRef["pyarrow.Table"]], parallelism: int = 200) -> Dataset[ArrowRow]: """Create a dataset from a set of Arrow tables. Args: dfs: A list of Ray object references to Arrow tables. parallelism: The amount of parallelism to use for the dataset. Returns: Dataset holding Arrow records from the tables. """ @ray.remote def get_metadata(table: "pyarrow.Table") -> BlockMetadata: return BlockAccessor.for_block(table).get_metadata(input_files=None) metadata = [get_metadata.remote(t) for t in tables] return Dataset(BlockList(tables, ray.get(metadata)))
def from_pandas(dfs: List[ObjectRef["pandas.DataFrame"]], parallelism: int = 200) -> Dataset[ArrowRow]: """Create a dataset from a set of Pandas dataframes. Args: dfs: A list of Ray object references to pandas dataframes. parallelism: The amount of parallelism to use for the dataset. Returns: Dataset holding Arrow records read from the dataframes. """ import pyarrow as pa @ray.remote(num_returns=2) def df_to_block(df: "pandas.DataFrame"): block = ArrowBlock(pa.table(df)) return block, block.get_metadata(input_files=None) res = [df_to_block.remote(df) for df in dfs] blocks, metadata = zip(*res) return Dataset(BlockList(blocks, ray.get(list(metadata))))
def apply(self, fn: Any, remote_args: dict, blocks: BlockList[Any]) -> BlockList[Any]: map_bar = ProgressBar("Map Progress", total=len(blocks)) kwargs = remote_args.copy() kwargs["num_returns"] = 2 # Lazy init to avoid circular import. TODO(ekl) move these into a # separate remote functions file. global _remote_fn if _remote_fn is None: _remote_fn = ray.remote(map_block) refs = [ _remote_fn.options(**kwargs).remote(b, m, fn) for b, m in zip(blocks, blocks.get_metadata()) ] new_blocks, new_metadata = zip(*refs) map_bar.block_until_complete(list(new_blocks)) new_metadata = ray.get(list(new_metadata)) return BlockList(list(new_blocks), list(new_metadata))
def split(self, n: int, locality_hints: List[Any] = None) -> List["Dataset[T]"]: """Split the dataset into ``n`` disjoint pieces. This returns a list of sub-datasets that can be passed to Ray tasks and actors and used to read the dataset records in parallel. Examples: >>> # Split up a dataset to process over `n` worker actors. >>> shards = ds.split(len(workers), locality_hints=workers) >>> for shard, worker in zip(shards, workers): ... worker.consume.remote(shard) Time complexity: O(1) Args: n: Number of child datasets to return. locality_hints: A list of Ray actor handles of size ``n``. The system will try to co-locate the blocks of the ith dataset with the ith actor to maximize data locality. Returns: A list of ``n`` disjoint dataset splits. """ if n <= 0: raise ValueError(f"The num of splits {n} is not positive.") if locality_hints and len(locality_hints) != n: raise ValueError( f"The length of locality_hints {len(locality_hints)} " "doesn't equal the number of splits {n}.") block_refs = list(self._blocks) metadata_mapping = { b: m for b, m in zip(self._blocks, self._blocks.get_metadata()) } if locality_hints is None: return [ Dataset( BlockList(list(blocks), [metadata_mapping[b] for b in blocks])) for blocks in np.array_split(block_refs, n) ] # If the locality_hints is set, we use a two-round greedy algorithm # to co-locate the blocks with the actors based on block # and actor's location (node_id). # # The split algorithm tries to allocate equally-sized blocks regardless # of locality. Thus we first calculate the expected number of blocks # for each split. # # In the first round, for each actor, we look for all blocks that # match the actor's node_id, then allocate those matched blocks to # this actor until we reach the limit(expected number). # # In the second round: fill each actor's allocation with # remaining unallocated blocks until we reach the limit. ray.wait(block_refs, num_returns=len(block_refs)) def build_allocation_size_map(num_blocks: int, actors: List[Any]) -> Dict[Any, int]: """Given the total number of blocks and a list of actors, calcuate the expected number of blocks to allocate for each actor. """ num_actors = len(actors) num_blocks_per_actor = num_blocks // num_actors num_blocks_left = num_blocks - num_blocks_per_actor * n num_blocks_by_actor = {} for i, actor in enumerate(actors): num_blocks_by_actor[actor] = num_blocks_per_actor if i < num_blocks_left: num_blocks_by_actor[actor] += 1 return num_blocks_by_actor def build_block_refs_by_node_id( blocks: List[ObjectRef[Block]] ) -> Dict[str, List[ObjectRef[Block]]]: """Build the reverse index from node_id to block_refs. For simplicity, if the block is stored on multiple nodes we only pick the first one. """ block_ref_locations = ray.experimental.get_object_locations(blocks) block_refs_by_node_id = collections.defaultdict(list) for block_ref in blocks: node_ids = block_ref_locations.get(block_ref, {}).get("node_ids", []) node_id = node_ids[0] if node_ids else None block_refs_by_node_id[node_id].append(block_ref) return block_refs_by_node_id def build_node_id_by_actor(actors: List[Any]) -> Dict[Any, str]: """Build a map from a actor to its node_id. """ actors_state = ray.state.actors() return { actor: actors_state.get(actor._actor_id.hex(), {}).get("Address", {}).get("NodeID") for actor in actors } # expected number of blocks to be allocated for each actor expected_block_count_by_actor = build_allocation_size_map( len(block_refs), locality_hints) # the reverse index from node_id to block_refs block_refs_by_node_id = build_block_refs_by_node_id(block_refs) # the map from actor to its node_id node_id_by_actor = build_node_id_by_actor(locality_hints) allocation_per_actor = collections.defaultdict(list) # In the first round, for each actor, we look for all blocks that # match the actor's node_id, then allocate those matched blocks to # this actor until we reach the limit(expected number) for actor in locality_hints: node_id = node_id_by_actor[actor] matching_blocks = block_refs_by_node_id[node_id] expected_block_count = expected_block_count_by_actor[actor] allocation = [] while matching_blocks and len(allocation) < expected_block_count: allocation.append(matching_blocks.pop()) allocation_per_actor[actor] = allocation # In the second round: fill each actor's allocation with # remaining unallocated blocks until we reach the limit remaining_block_refs = list( itertools.chain.from_iterable(block_refs_by_node_id.values())) for actor in locality_hints: while len(allocation_per_actor[actor] ) < expected_block_count_by_actor[actor]: allocation_per_actor[actor].append(remaining_block_refs.pop()) assert len(remaining_block_refs) == 0, len(remaining_block_refs) return [ Dataset( BlockList( allocation_per_actor[actor], [metadata_mapping[b] for b in allocation_per_actor[actor]])) for actor in locality_hints ]
def apply(self, fn: Any, remote_args: dict, blocks: Iterable[Block]) -> Iterable[ObjectRef[Block]]: map_bar = ProgressBar("Map Progress", total=len(blocks)) class BlockWorker: def ready(self): return "ok" @ray.method(num_returns=2) 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 if not remote_args: remote_args["num_cpus"] = 1 BlockWorker = ray.remote(**remote_args)(BlockWorker) self.workers = [BlockWorker.remote()] metadata_mapping = {} tasks = {w.ready.remote(): w for w in self.workers} ready_workers = set() blocks_in = [(b, m) for (b, m) in zip(blocks, blocks.get_metadata())] blocks_out = [] while len(blocks_out) < len(blocks): ready, _ = ray.wait(list(tasks), timeout=0.01, num_returns=1, fetch_local=False) if not ready: if len(ready_workers) / len(self.workers) > 0.8: w = BlockWorker.remote() self.workers.append(w) tasks[w.ready.remote()] = w map_bar.set_description( "Map Progress ({} actors {} pending)".format( len(ready_workers), len(self.workers) - len(ready_workers))) continue [obj_id] = ready worker = tasks[obj_id] del tasks[obj_id] # Process task result. if worker in ready_workers: blocks_out.append(obj_id) map_bar.update(1) else: ready_workers.add(worker) # Schedule a new task. if blocks_in: block_ref, meta_ref = worker.process_block.remote( *blocks_in.pop()) metadata_mapping[block_ref] = meta_ref tasks[block_ref] = worker new_metadata = ray.get([metadata_mapping[b] for b in blocks_out]) map_bar.close() return BlockList(blocks_out, new_metadata)