def read_hdf(cls, path_or_buf, key=None, mode="r", columns=None): if not columns: empty_pd_df = pandas.read_hdf(path_or_buf, start=0, stop=0) columns = empty_pd_df.columns num_partitions = RayBlockPartitions._compute_num_partitions() num_splits = min(len(columns), num_partitions) # Each item in this list will be a list of column names of the original df column_splits = (len(columns) // num_partitions if len(columns) % num_partitions == 0 else len(columns) // num_partitions + 1) col_partitions = [ columns[i:i + column_splits] for i in range(0, len(columns), column_splits) ] blk_partitions = np.array([ _read_hdf_columns._remote( args=(path_or_buf, cols, num_splits, key, mode), num_return_vals=num_splits + 1, ) for cols in col_partitions ]).T remote_partitions = np.array( [[PandasOnRayRemotePartition(obj) for obj in row] for row in blk_partitions[:-1]]) index_len = ray.get(blk_partitions[-1][0]) index = pandas.RangeIndex(index_len) new_query_compiler = PandasQueryCompiler( RayBlockPartitions(remote_partitions), index, columns) return new_query_compiler
def _read_parquet_pandas_on_ray(path, engine, columns, **kwargs): from pyarrow.parquet import ParquetFile if not columns: pf = ParquetFile(path) columns = [ name for name in pf.metadata.schema.names if not PQ_INDEX_REGEX.match(name) ] num_splits = min(len(columns), RayBlockPartitions._compute_num_partitions()) # Each item in this list will be a column of original df # partitioned to smaller pieces along rows. # We need to transpose the oids array to fit our schema. blk_partitions = np.array([ _read_parquet_column._submit(args=(path, col, num_splits, kwargs), num_return_vals=num_splits + 1) for col in columns ]).T remote_partitions = np.array( [[PandasOnRayRemotePartition(obj) for obj in row] for row in blk_partitions[:-1]]) index_len = ray.get(blk_partitions[-1][0]) index = pandas.RangeIndex(index_len) new_manager = PandasQueryCompiler(RayBlockPartitions(remote_partitions), index, columns) df = DataFrame(query_compiler=new_manager) return df
def read_parquet(cls, path, engine, columns, **kwargs): """Load a parquet object from the file path, returning a DataFrame. Ray DataFrame only supports pyarrow engine for now. Args: path: The filepath of the parquet file. We only support local files for now. engine: Ray only support pyarrow reader. This argument doesn't do anything for now. kwargs: Pass into parquet's read_pandas function. Notes: ParquetFile API is used. Please refer to the documentation here https://arrow.apache.org/docs/python/parquet.html """ from pyarrow.parquet import ParquetFile if not columns: pf = ParquetFile(path) columns = [ name for name in pf.metadata.schema.names if not PQ_INDEX_REGEX.match(name) ] num_partitions = RayBlockPartitions._compute_num_partitions() num_splits = min(len(columns), num_partitions) # Each item in this list will be a list of column names of the original df column_splits = (len(columns) // num_partitions if len(columns) % num_partitions == 0 else len(columns) // num_partitions + 1) col_partitions = [ columns[i:i + column_splits] for i in range(0, len(columns), column_splits) ] # Each item in this list will be a list of columns of original df # partitioned to smaller pieces along rows. # We need to transpose the oids array to fit our schema. blk_partitions = np.array([ _read_parquet_columns._remote( args=(path, cols, num_splits, kwargs), num_return_vals=num_splits + 1, ) for cols in col_partitions ]).T remote_partitions = np.array( [[PandasOnRayRemotePartition(obj) for obj in row] for row in blk_partitions[:-1]]) index_len = ray.get(blk_partitions[-1][0]) index = pandas.RangeIndex(index_len) new_query_compiler = PandasQueryCompiler( RayBlockPartitions(remote_partitions), index, columns) return new_query_compiler
def read_hdf(cls, path_or_buf, **kwargs): """Load a h5 file from the file path or buffer, returning a DataFrame. Args: path_or_buf: string, buffer or path object Path to the file to open, or an open :class:`pandas.HDFStore` object. kwargs: Pass into pandas.read_hdf function. Returns: DataFrame constructed from the h5 file. """ format = cls._validate_hdf_format(path_or_buf=path_or_buf) if format is None: ErrorMessage.default_to_pandas( "File format seems to be `fixed`. For better distribution consider saving the file in `table` format. " "df.to_hdf(format=`table`).") return cls.from_pandas( pandas.read_hdf(path_or_buf=path_or_buf, **kwargs)) columns = kwargs.get("columns", None) if not columns: empty_pd_df = pandas.read_hdf(path_or_buf, start=0, stop=0) columns = empty_pd_df.columns num_partitions = RayBlockPartitions._compute_num_partitions() num_splits = min(len(columns), num_partitions) # Each item in this list will be a list of column names of the original df column_splits = (len(columns) // num_partitions if len(columns) % num_partitions == 0 else len(columns) // num_partitions + 1) col_partitions = [ columns[i:i + column_splits] for i in range(0, len(columns), column_splits) ] blk_partitions = np.array([ _read_hdf_columns._remote( args=(path_or_buf, cols, num_splits, kwargs), num_return_vals=num_splits + 1, ) for cols in col_partitions ]).T remote_partitions = np.array( [[PandasOnRayRemotePartition(obj) for obj in row] for row in blk_partitions[:-1]]) index_len = ray.get(blk_partitions[-1][0]) index = pandas.RangeIndex(index_len) new_query_compiler = PandasQueryCompiler( RayBlockPartitions(remote_partitions), index, columns) return new_query_compiler
def read_feather(cls, path, columns=None, use_threads=True): """Read a pandas.DataFrame from Feather format. Ray DataFrame only supports pyarrow engine for now. Args: path: The filepath of the feather file. We only support local files for now. multi threading is set to True by default columns: not supported by pandas api, but can be passed here to read only specific columns use_threads: Whether or not to use threads when reading Notes: pyarrow feather is used. Please refer to the documentation here https://arrow.apache.org/docs/python/api.html#feather-format """ if columns is None: from pyarrow.feather import FeatherReader fr = FeatherReader(path) columns = [fr.get_column_name(i) for i in range(fr.num_columns)] num_partitions = RayBlockPartitions._compute_num_partitions() num_splits = min(len(columns), num_partitions) # Each item in this list will be a list of column names of the original df column_splits = (len(columns) // num_partitions if len(columns) % num_partitions == 0 else len(columns) // num_partitions + 1) col_partitions = [ columns[i:i + column_splits] for i in range(0, len(columns), column_splits) ] blk_partitions = np.array([ _read_feather_columns._remote(args=(path, cols, num_splits), num_return_vals=num_splits + 1) for cols in col_partitions ]).T remote_partitions = np.array( [[PandasOnRayRemotePartition(obj) for obj in row] for row in blk_partitions[:-1]]) index_len = ray.get(blk_partitions[-1][0]) index = pandas.RangeIndex(index_len) new_query_compiler = PandasQueryCompiler( RayBlockPartitions(remote_partitions), index, columns) return new_query_compiler
def _read_csv_from_file_pandas_on_ray(cls, filepath, kwargs={}): """Constructs a DataFrame from a CSV file. Args: filepath (str): path to the CSV file. npartitions (int): number of partitions for the DataFrame. kwargs (dict): args excluding filepath provided to read_csv. Returns: DataFrame or Series constructed from CSV file. """ empty_pd_df = pandas.read_csv(filepath, **dict(kwargs, nrows=0, skipfooter=0)) column_names = empty_pd_df.columns skipfooter = kwargs.get("skipfooter", None) partition_kwargs = dict(kwargs, header=None, names=column_names, skipfooter=0) with open(filepath, "rb") as f: # Get the BOM if necessary prefix = b"" if kwargs.get("encoding", None) is not None: prefix = f.readline() partition_kwargs["skiprows"] = 1 f.seek(0, os.SEEK_SET) # Return to beginning of file prefix_id = ray.put(prefix) partition_kwargs_id = ray.put(partition_kwargs) # Skip the header since we already have the header information cls._skip_header(f, kwargs) # Launch tasks to read partitions partition_ids = [] index_ids = [] total_bytes = os.path.getsize(filepath) # Max number of partitions available num_parts = RayBlockPartitions._compute_num_partitions() # This is the number of splits for the columns num_splits = min(len(column_names), num_parts) # This is the chunksize each partition will read chunk_size = max(1, (total_bytes - f.tell()) // num_parts) while f.tell() < total_bytes: start = f.tell() f.seek(chunk_size, os.SEEK_CUR) f.readline() # Read a whole number of lines partition_id = _read_csv_with_offset_pandas_on_ray._remote( args=( filepath, num_splits, start, f.tell(), partition_kwargs_id, prefix_id, ), num_return_vals=num_splits + 1, ) partition_ids.append([ PandasOnRayRemotePartition(obj) for obj in partition_id[:-1] ]) index_ids.append(partition_id[-1]) index_col = kwargs.get("index_col", None) if index_col is None: new_index = pandas.RangeIndex(sum(ray.get(index_ids))) else: new_index_ids = get_index.remote([empty_pd_df.index.name], *index_ids) new_index = ray.get(new_index_ids) new_query_compiler = PandasQueryCompiler( RayBlockPartitions(np.array(partition_ids)), new_index, column_names) if skipfooter: new_query_compiler = new_query_compiler.drop( new_query_compiler.index[-skipfooter:]) if kwargs.get("squeeze", False) and len( new_query_compiler.columns) == 1: return new_query_compiler[new_query_compiler.columns[0]] return new_query_compiler