mirror of
https://github.com/vale981/ray
synced 2025-03-05 10:01:43 -05:00
[data] Serialize parquet piece metadata in batches to reduce overheads
This commit is contained in:
parent
91171a194f
commit
95877be8ee
1 changed files with 16 additions and 20 deletions
|
@ -76,15 +76,14 @@ class ParquetDatasource(FileBasedDatasource):
|
|||
schema = pa.schema([schema.field(column) for column in columns],
|
||||
schema.metadata)
|
||||
|
||||
def read_pieces(serialized_pieces: List[str]) -> Iterator[pa.Table]:
|
||||
def read_pieces(serialized_pieces: str) -> Iterator[pa.Table]:
|
||||
# Implicitly trigger S3 subsystem initialization by importing
|
||||
# pyarrow.fs.
|
||||
import pyarrow.fs # noqa: F401
|
||||
|
||||
# Deserialize after loading the filesystem class.
|
||||
pieces: List["pyarrow._dataset.ParquetFileFragment"] = [
|
||||
cloudpickle.loads(p) for p in serialized_pieces
|
||||
]
|
||||
pieces: List["pyarrow._dataset.ParquetFileFragment"] = \
|
||||
cloudpickle.loads(serialized_pieces)
|
||||
|
||||
# Ensure that we're reading at least one dataset fragment.
|
||||
assert len(pieces) > 0
|
||||
|
@ -138,22 +137,19 @@ class ParquetDatasource(FileBasedDatasource):
|
|||
else:
|
||||
inferred_schema = schema
|
||||
read_tasks = []
|
||||
serialized_pieces = [cloudpickle.dumps(p) for p in pq_ds.pieces]
|
||||
if len(pq_ds.pieces) > PARALLELIZE_META_FETCH_THRESHOLD:
|
||||
metadata = _fetch_metadata_remotely(serialized_pieces)
|
||||
metadata = _fetch_metadata_remotely(pq_ds.pieces)
|
||||
else:
|
||||
metadata = _fetch_metadata(pq_ds.pieces)
|
||||
for piece_data in np.array_split(
|
||||
list(zip(pq_ds.pieces, serialized_pieces, metadata)),
|
||||
parallelism):
|
||||
list(zip(pq_ds.pieces, metadata)), parallelism):
|
||||
if len(piece_data) == 0:
|
||||
continue
|
||||
pieces, serialized_pieces, metadata = zip(*piece_data)
|
||||
pieces, metadata = zip(*piece_data)
|
||||
serialized_pieces = cloudpickle.dumps(pieces)
|
||||
meta = _build_block_metadata(pieces, metadata, inferred_schema)
|
||||
read_tasks.append(
|
||||
ReadTask(
|
||||
lambda pieces_=serialized_pieces: read_pieces(pieces_),
|
||||
meta))
|
||||
ReadTask(lambda p=serialized_pieces: read_pieces(p), meta))
|
||||
|
||||
return read_tasks
|
||||
|
||||
|
@ -172,32 +168,32 @@ class ParquetDatasource(FileBasedDatasource):
|
|||
|
||||
|
||||
def _fetch_metadata_remotely(
|
||||
pieces: List[bytes]
|
||||
pieces: List["pyarrow._dataset.ParquetFileFragment"]
|
||||
) -> List[ObjectRef["pyarrow.parquet.FileMetaData"]]:
|
||||
from ray import cloudpickle
|
||||
remote_fetch_metadata = cached_remote_fn(
|
||||
_fetch_metadata_serialization_wrapper)
|
||||
metas = []
|
||||
parallelism = min(len(pieces) // PIECES_PER_META_FETCH, 100)
|
||||
meta_fetch_bar = ProgressBar("Metadata Fetch Progress", total=parallelism)
|
||||
for pieces_ in np.array_split(pieces, parallelism):
|
||||
if len(pieces_) == 0:
|
||||
for pcs in np.array_split(pieces, parallelism):
|
||||
if len(pcs) == 0:
|
||||
continue
|
||||
metas.append(remote_fetch_metadata.remote(pieces_))
|
||||
metas.append(remote_fetch_metadata.remote(cloudpickle.dumps(pcs)))
|
||||
metas = meta_fetch_bar.fetch_until_complete(metas)
|
||||
return list(itertools.chain.from_iterable(metas))
|
||||
|
||||
|
||||
def _fetch_metadata_serialization_wrapper(
|
||||
pieces: List[bytes]) -> List["pyarrow.parquet.FileMetaData"]:
|
||||
pieces: str) -> List["pyarrow.parquet.FileMetaData"]:
|
||||
# Implicitly trigger S3 subsystem initialization by importing
|
||||
# pyarrow.fs.
|
||||
import pyarrow.fs # noqa: F401
|
||||
from ray import cloudpickle
|
||||
|
||||
# Deserialize after loading the filesystem class.
|
||||
pieces: List["pyarrow._dataset.ParquetFileFragment"] = [
|
||||
cloudpickle.loads(p) for p in pieces
|
||||
]
|
||||
pieces: List["pyarrow._dataset.ParquetFileFragment"] = \
|
||||
cloudpickle.loads(pieces)
|
||||
|
||||
return _fetch_metadata(pieces)
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue