def _apply( self, fn: Any, remote_args: dict, block_list: BlockList, clear_input_blocks: bool, name: Optional[str] = None, ) -> BlockList: """Note: this is not part of the Dataset public API.""" context = DatasetContext.get_current() blocks_in = block_list.get_blocks_with_metadata() # Early release block references. if clear_input_blocks: block_list.clear() orig_num_blocks = len(blocks_in) results = [] if name is None: name = "map" name = name.title() map_bar = ProgressBar(name, total=orig_num_blocks) class BlockWorker: def ready(self): return "ok" def map_block_split(self, block: Block, input_files: List[str]) -> BlockPartition: return _map_block_split(block, fn, input_files) @ray.method(num_returns=2) def map_block_nosplit( self, block: Block, input_files: List[str]) -> Tuple[Block, BlockMetadata]: return _map_block_nosplit(block, fn, input_files) if not remote_args: remote_args["num_cpus"] = 1 BlockWorker = ray.remote(**remote_args)(BlockWorker) workers = [BlockWorker.remote() for _ in range(self.min_size)] tasks = {w.ready.remote(): w for w in workers} tasks_in_flight = collections.defaultdict(int) metadata_mapping = {} block_indices = {} ready_workers = set() while len(results) < orig_num_blocks: ready, _ = ray.wait(list(tasks.keys()), timeout=0.01, num_returns=1, fetch_local=False) if not ready: if (len(workers) < self.max_size and len(ready_workers) / len(workers) > 0.8): w = BlockWorker.remote() workers.append(w) tasks[w.ready.remote()] = w map_bar.set_description( "Map Progress ({} actors {} pending)".format( len(ready_workers), len(workers) - len(ready_workers))) continue [obj_id] = ready worker = tasks.pop(obj_id) # Process task result. if worker in ready_workers: results.append(obj_id) tasks_in_flight[worker] -= 1 map_bar.update(1) else: ready_workers.add(worker) map_bar.set_description( "Map Progress ({} actors {} pending)".format( len(ready_workers), len(workers) - len(ready_workers))) # Schedule a new task. while (blocks_in and tasks_in_flight[worker] < self.max_tasks_in_flight_per_actor): block, meta = blocks_in.pop() if context.block_splitting_enabled: ref = worker.map_block_split.remote( block, meta.input_files) else: ref, meta_ref = worker.map_block_nosplit.remote( block, meta.input_files) metadata_mapping[ref] = meta_ref tasks[ref] = worker block_indices[ref] = len(blocks_in) tasks_in_flight[worker] += 1 map_bar.close() new_blocks, new_metadata = [], [] # Put blocks in input order. results.sort(key=block_indices.get) if context.block_splitting_enabled: for result in ray.get(results): for block, metadata in result: new_blocks.append(block) new_metadata.append(metadata) else: for block in results: new_blocks.append(block) new_metadata.append(metadata_mapping[block]) new_metadata = ray.get(new_metadata) return BlockList(new_blocks, new_metadata)
def apply(self, fn: Any, remote_args: dict, blocks: BlockList) -> BlockList: blocks_in = list(blocks.iter_blocks_with_metadata()) orig_num_blocks = len(blocks_in) results = [] map_bar = ProgressBar("Map Progress", total=orig_num_blocks) class BlockWorker: def ready(self): return "ok" 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 if not remote_args: remote_args["num_cpus"] = 1 BlockWorker = ray.remote(**remote_args)(BlockWorker) self.workers = [BlockWorker.remote()] tasks = {w.ready.remote(): w for w in self.workers} ready_workers = set() while len(results) < orig_num_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: results.append(obj_id) map_bar.update(1) else: ready_workers.add(worker) # Schedule a new task. if blocks_in: block, meta = blocks_in.pop() ref = worker.process_block.remote(block, meta.input_files) tasks[ref] = worker map_bar.close() new_blocks, new_metadata = [], [] for result in ray.get(results): for block, metadata in result: new_blocks.append(block) new_metadata.append(metadata) return BlockList(new_blocks, new_metadata)
def apply(self, fn: Any, remote_args: dict, blocks: BlockList) -> BlockList: context = DatasetContext.get_current() blocks_in = blocks.get_blocks_with_metadata() orig_num_blocks = len(blocks_in) results = [] map_bar = ProgressBar("Map Progress", total=orig_num_blocks) class BlockWorker: def ready(self): return "ok" def map_block_split(self, block: Block, input_files: List[str]) -> BlockPartition: return _map_block_split(block, fn, input_files) @ray.method(num_returns=2) def map_block_nosplit( self, block: Block, input_files: List[str]) -> Tuple[Block, BlockMetadata]: return _map_block_nosplit(block, fn, input_files) if not remote_args: remote_args["num_cpus"] = 1 BlockWorker = ray.remote(**remote_args)(BlockWorker) self.workers = [BlockWorker.remote()] tasks = {w.ready.remote(): w for w in self.workers} metadata_mapping = {} ready_workers = set() while len(results) < orig_num_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: results.append(obj_id) map_bar.update(1) else: ready_workers.add(worker) # Schedule a new task. if blocks_in: block, meta = blocks_in.pop() if context.block_splitting_enabled: ref = worker.map_block_split.remote( block, meta.input_files) else: ref, meta_ref = worker.map_block_nosplit.remote( block, meta.input_files) metadata_mapping[ref] = meta_ref tasks[ref] = worker map_bar.close() new_blocks, new_metadata = [], [] if context.block_splitting_enabled: for result in ray.get(results): for block, metadata in result: new_blocks.append(block) new_metadata.append(metadata) else: for block in results: new_blocks.append(block) new_metadata.append(metadata_mapping[block]) return BlockList(new_blocks, new_metadata)
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)