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. Parallelism may be limited by the number of items. 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 = DelegatingBlockBuilder() 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, exec_stats=BlockExecStats.TODO)) i += block_size return Dataset(BlockList(blocks, metadata), 0, DatasetStats.TODO())
class BlockOutputBuffer(object): """Generates output blocks of a given size given a stream of inputs. This class is used to turn a stream of items / blocks of arbitrary size into a stream of blocks of ``target_max_block_size``. The caller should check ``has_next()`` after each ``add()`` call, and call ``next()`` to get the next block when ``has_next()`` returns True. When all items have been added, the caller must call ``finalize()`` and then check ``has_next()`` one last time. Examples: >>> # Yield a stream of output blocks. >>> output = BlockOutputBuffer(udf, 500 * 1024 * 1024) >>> for item in generator(): ... output.add(item) ... if output.has_next(): ... yield output.next() ... output.finalize() ... if output.has_next() ... yield output.next() """ def __init__(self, block_udf: Optional[Callable[[Block], Block]], target_max_block_size: int): self._target_max_block_size = target_max_block_size self._block_udf = block_udf self._buffer = DelegatingBlockBuilder() self._returned_at_least_one_block = False self._finalized = False def add(self, item: Any) -> None: """Add a single item to this output buffer.""" assert not self._finalized self._buffer.add(item) def add_block(self, block: Block) -> None: """Add a data block to this output buffer.""" assert not self._finalized self._buffer.add_block(block) def finalize(self) -> None: """Must be called once all items have been added.""" assert not self._finalized self._finalized = True def has_next(self) -> bool: """Returns true when a complete output block is produced.""" if self._finalized: return not self._returned_at_least_one_block \ or self._buffer.num_rows() > 0 else: return self._buffer.get_estimated_memory_usage() > \ self._target_max_block_size def next(self) -> Block: """Returns the next complete output block.""" assert self.has_next() block = self._buffer.build() accessor = BlockAccessor.for_block(block) if self._block_udf and accessor.num_rows() > 0: block = self._block_udf(block) self._buffer = DelegatingBlockBuilder() self._returned_at_least_one_block = True return block