import ray from ray.util import iter import s3fs from time import perf_counter as clock ray.init() fs = s3fs.S3FileSystem(anon=False) files = sorted(fs.ls("anyscale-data/bandwidth-benchmark"))[:1000] @ray.remote def download(path): data = fs.open(path, 'rb').read() return data start = clock() iter = iter.from_iterators(files).for_each(download) results = iter.gather_sync() for data in results: x = 0 for byte in data: x ^= byte print(".", end="", flush=True) end = clock() print("Total download time: ", (end - start))
def test_from_iterators(ray_start_regular_shared): it = from_iterators([range(2), range(2)]) assert repr(it) == "ParallelIterator[from_iterators[shards=2]]" assert list(it.gather_sync()) == [0, 0, 1, 1]
def read_parquet( paths: Union[str, List[str]], num_shards: int, rowgroup_split: bool = True, shuffle: bool = False, shuffle_seed: int = None, columns: Optional[List[str]] = None, **kwargs, ) -> MLDataset: """Read parquet format data from hdfs like filesystem into a MLDataset. .. code-block:: python # create dummy data spark.range(...).write.parquet(...) # create MLDataset data = ray.util.data.read_parquet(...) # convert to TorchMLDataset ds = data.to_torch(feature_columns=..., label_column=...) # get the given shard data shard = ds.get_shard(0) # create the DataLoader from the shard data and this can be used for # the TorchTrainer data creator as well data = DataLoader(shard, batch_size=32) Args: paths (Union[str, List[str]): a single file path or a list of file path num_shards (int): the number of shards rowgroup_split (bool): whether split the files into shards based on rowgroup. If set False, each shard will have a list of files. shuffle (bool): whether shuffle the ParquetDatasetPiece order when divide into shards shuffle_seed (int): the shuffle seed columns (Optional[List[str]]): a list of column names to read kwargs: the other parquet read options Returns: A MLDataset """ pq_ds = pq.ParquetDataset(paths, **kwargs) pieces = pq_ds.pieces data_pieces = [] if rowgroup_split: # split base on rowgroup for piece in pieces: num_row_groups = piece.get_metadata().to_dict()["num_row_groups"] for i in range(num_row_groups): data_pieces.append( pq.ParquetDatasetPiece( piece.path, piece.open_file_func, piece.file_options, i, piece.partition_keys, )) else: # split base on file pieces data_pieces = pieces.copy() if len(data_pieces) < num_shards: raise ValueError(f"number of data pieces: {len(data_pieces)} should " f"larger than num_shards: {num_shards}") if shuffle: random_shuffle = random.Random(shuffle_seed) random_shuffle.shuffle(data_pieces) shards = [[] for _ in range(num_shards)] for i, item in enumerate(data_pieces): shard = shards[i % num_shards] if item.row_group is None: for number in item.get_metadata().to_dict()["num_row_groups"]: shard.append( pq.ParquetDatasetPiece( item.path, item.open_file_func, item.file_options, number, item.partition_keys, )) else: shard.append(item) for i, shard in enumerate(shards): shards[i] = ParquetSourceShard(shard, columns, pq_ds.partitions, i) it = para_iter.from_iterators(shards, False, "parquet") return MLDataset.from_parallel_it(it, batch_size=0, repeated=False)