def _test_equal_split_balanced(block_sizes, num_splits): blocks = [] metadata = [] total_rows = 0 for block_size in block_sizes: block = list(range(total_rows, total_rows + block_size)) blocks.append(ray.put(block)) metadata.append( BlockAccessor.for_block(block).get_metadata(None, None)) total_rows += block_size block_list = BlockList(blocks, metadata) ds = Dataset( ExecutionPlan(block_list, DatasetStats.TODO()), 0, False, ) splits = ds.split(num_splits, equal=True) split_counts = [split.count() for split in splits] assert len(split_counts) == num_splits expected_block_size = total_rows // num_splits # Check that all splits are the expected size. assert all([count == expected_block_size for count in split_counts]) expected_total_rows = sum(split_counts) # Check that the expected number of rows were dropped. assert total_rows - expected_total_rows == total_rows % num_splits # Check that all rows are unique (content check). split_rows = [row for split in splits for row in split.take(total_rows)] assert len(set(split_rows)) == len(split_rows)
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())
def from_pandas_refs( dfs: Union[ObjectRef["pandas.DataFrame"], List[ObjectRef["pandas.DataFrame"]]] ) -> Dataset[ArrowRow]: """Create a dataset from a list of Ray object references to Pandas dataframes. Args: dfs: A Ray object references to pandas dataframe, or a list of Ray object references to pandas dataframes. Returns: Dataset holding Arrow records read from the dataframes. """ if isinstance(dfs, ray.ObjectRef): dfs = [dfs] elif isinstance(dfs, list): for df in dfs: if not isinstance(df, ray.ObjectRef): raise ValueError( "Expected list of Ray object refs, " f"got list containing {type(df)}" ) else: raise ValueError( "Expected Ray object ref or list of Ray object refs, " f"got {type(df)}" ) context = DatasetContext.get_current() if context.enable_pandas_block: get_metadata = cached_remote_fn(_get_metadata) metadata = [get_metadata.remote(df) for df in dfs] return Dataset( ExecutionPlan(BlockList(dfs, ray.get(metadata)), DatasetStats.TODO()), 0, False, ) df_to_block = cached_remote_fn(_df_to_block, num_returns=2) res = [df_to_block.remote(df) for df in dfs] blocks, metadata = zip(*res) return Dataset( ExecutionPlan( BlockList(blocks, ray.get(list(metadata))), DatasetStats(stages={"from_pandas_refs": metadata}, parent=None), ), 0, False, )
def from_numpy(ndarrays: List[ObjectRef[np.ndarray]]) -> Dataset[ArrowRow]: """Create a dataset from a set of NumPy ndarrays. Args: ndarrays: A list of Ray object references to NumPy ndarrays. Returns: Dataset holding the given ndarrays. """ ndarray_to_block = cached_remote_fn(_ndarray_to_block, num_returns=2) res = [ndarray_to_block.remote(ndarray) for ndarray in ndarrays] blocks, metadata = zip(*res) return Dataset(BlockList(blocks, ray.get(list(metadata))), 0, DatasetStats.TODO())
def from_arrow_refs( tables: Union[ObjectRef[Union["pyarrow.Table", bytes]], List[ObjectRef[Union["pyarrow.Table", bytes]]]] ) -> Dataset[ArrowRow]: """Create a dataset from a set of Arrow tables. Args: tables: A Ray object reference to Arrow table, or list of Ray object references to Arrow tables, or its streaming format in bytes. Returns: Dataset holding Arrow records from the tables. """ if isinstance(tables, ray.ObjectRef): tables = [tables] get_metadata = cached_remote_fn(_get_metadata) metadata = [get_metadata.remote(t) for t in tables] return Dataset(BlockList(tables, ray.get(metadata)), 0, DatasetStats.TODO())
def from_pandas_refs( dfs: Union[ObjectRef["pandas.DataFrame"], List[ObjectRef["pandas.DataFrame"]]] ) -> Dataset[ArrowRow]: """Create a dataset from a list of Ray object references to Pandas dataframes. Args: dfs: A Ray object references to pandas dataframe, or a list of Ray object references to pandas dataframes. Returns: Dataset holding Arrow records read from the dataframes. """ if isinstance(dfs, ray.ObjectRef): dfs = [dfs] df_to_block = cached_remote_fn(_df_to_block, num_returns=2) res = [df_to_block.remote(df) for df in dfs] blocks, metadata = zip(*res) return Dataset(BlockList(blocks, ray.get(list(metadata))), 0, DatasetStats.TODO())