Skip to content

Commit 49ccb05

Browse files
Devin PetersohnDevin Petersohn
authored andcommitted
Fix bug that causes workers to crash
1 parent a368735 commit 49ccb05

File tree

1 file changed

+28
-10
lines changed

1 file changed

+28
-10
lines changed

modin/pandas/io.py

Lines changed: 28 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -28,35 +28,46 @@
2828
def read_parquet(path, engine='auto', columns=None, **kwargs):
2929
"""Load a parquet object from the file path, returning a DataFrame.
3030
Ray DataFrame only supports pyarrow engine for now.
31-
3231
Args:
3332
path: The filepath of the parquet file.
3433
We only support local files for now.
3534
engine: Ray only support pyarrow reader.
3635
This argument doesn't do anything for now.
3736
kwargs: Pass into parquet's read_pandas function.
38-
3937
Notes:
4038
ParquetFile API is used. Please refer to the documentation here
4139
https://arrow.apache.org/docs/python/parquet.html
4240
"""
41+
return _read_parquet_pandas_on_ray(path, engine, columns, **kwargs)
42+
43+
44+
def _read_parquet_pandas_on_ray(path, engine, columns, **kwargs):
45+
from pyarrow.parquet import ParquetFile
46+
4347
if not columns:
4448
pf = ParquetFile(path)
4549
columns = [
4650
name for name in pf.metadata.schema.names
4751
if not PQ_INDEX_REGEX.match(name)
4852
]
49-
53+
num_splits = min(
54+
len(columns), RayBlockPartitions._compute_num_partitions())
5055
# Each item in this list will be a column of original df
5156
# partitioned to smaller pieces along rows.
5257
# We need to transpose the oids array to fit our schema.
53-
blk_partitions = [
54-
ray.get(_read_parquet_column.remote(path, col, kwargs))
55-
for col in columns
56-
]
57-
blk_partitions = np.array(blk_partitions).T
58-
59-
return DataFrame(block_partitions=blk_partitions, columns=columns)
58+
blk_partitions = np.array([
59+
_read_parquet_column._submit(
60+
args=(path, col, num_splits, kwargs),
61+
num_return_vals=num_splits + 1) for col in columns
62+
]).T
63+
remote_partitions = np.array([[RayRemotePartition(obj) for obj in row]
64+
for row in blk_partitions[:-1]])
65+
index_len = ray.get(blk_partitions[-1][0])
66+
index = pandas.RangeIndex(index_len)
67+
new_manager = PandasDataManager(
68+
RayBlockPartitions(remote_partitions), index, columns)
69+
df = DataFrame(data_manager=new_manager)
70+
return df
6071

6172

6273
# CSV
@@ -555,6 +566,13 @@ def _read_csv_with_offset_pandas_on_ray(fname, num_splits, start, end, kwargs, h
555566

556567
@ray.remote
557568
def _read_parquet_column(path, column, kwargs={}):
569+
import pyarrow.parquet as pq
570+
df = pq.read_pandas(path, columns=[column], **kwargs).to_pandas()
571+
# Append the length of the index here to build it externally
572+
return split_result_of_axis_func_pandas(0, num_splits,
573+
df) + [len(df.index)]
574+
# IMPORTANT: DO NOT DELETE THE CODE BELOW
575+
# Deleting this code for some reason causes workers to crash on high-CPU machines.
558576
df = pq.read_pandas(path, columns=[column], **kwargs).to_pandas()
559577
oids = _partition_pandas_dataframe(df, num_partitions=get_npartitions())
560578
return oids

0 commit comments

Comments
 (0)