Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit b2ed0da

Browse files
committed
Add ReadFromParquetBatched and ReadAllFromParquetBatched
IOs for reading arrow data from Parquet files
1 parent fd67fd3 commit b2ed0da

3 files changed

Lines changed: 208 additions & 56 deletions

File tree

sdks/python/apache_beam/io/parquetio.py

Lines changed: 99 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -39,19 +39,95 @@
3939
from apache_beam.io.iobase import RangeTracker
4040
from apache_beam.io.iobase import Read
4141
from apache_beam.io.iobase import Write
42+
from apache_beam.transforms import DoFn
43+
from apache_beam.transforms import ParDo
4244
from apache_beam.transforms import PTransform
4345

4446
if not (platform.system() == 'Windows' and sys.version_info[0] == 2):
4547
import pyarrow as pa
4648
import pyarrow.parquet as pq
4749

48-
__all__ = ['ReadFromParquet', 'ReadAllFromParquet', 'WriteToParquet']
50+
__all__ = ['ReadFromParquet', 'ReadAllFromParquet', 'ReadFromParquetBatched',
51+
'ReadAllFromParquetBatched', 'WriteToParquet']
52+
53+
54+
class _ArrowTableToRowDictionaries(DoFn):
55+
""" A DoFn that consumes an Arrow table and yields a python dictionary for
56+
each row in the table."""
57+
def process(self, table):
58+
num_rows = table.num_rows
59+
data_items = table.to_pydict().items()
60+
for n in range(num_rows):
61+
row = {}
62+
for column, values in data_items:
63+
row[column] = values[n]
64+
yield row
65+
66+
67+
class ReadFromParquetBatched(PTransform):
68+
"""A :class:`~apache_beam.transforms.ptransform.PTransform` for reading
69+
Parquet files as a `PCollection` of `pyarrow.Table`. This `PTransform` is
70+
currently experimental. No backward-compatibility guarantees."""
71+
72+
def __init__(self, file_pattern=None, min_bundle_size=0,
73+
validate=True, columns=None):
74+
""" Initializes :class:`~ReadFromParquetBatched`
75+
76+
An alternative to :class:`~ReadFromParquet` that yields each row group from
77+
the Parquet file as a `pyarrow.Table`. These Table instances can be
78+
processed directly, or converted to a pandas DataFrame for processing. For
79+
more information on supported types and schema, please see the pyarrow
80+
documentation.
81+
82+
.. testcode::
83+
84+
with beam.Pipeline() as p:
85+
dataframes = p \\
86+
| 'Read' >> beam.io.ReadFromParquetBatched('/mypath/mypqfiles*') \\
87+
| 'Convert to pandas' >> beam.Map(lambda table: table.to_pandas())
88+
89+
.. NOTE: We're not actually interested in this error; but if we get here,
90+
it means that the way of calling this transform hasn't changed.
91+
92+
.. testoutput::
93+
:hide:
94+
95+
Traceback (most recent call last):
96+
...
97+
IOError: No files found based on the file pattern
98+
99+
See also: :class:`~ReadFromParquet`.
100+
101+
Args:
102+
file_pattern (str): the file glob to read
103+
min_bundle_size (int): the minimum size in bytes, to be considered when
104+
splitting the input into bundles.
105+
validate (bool): flag to verify that the files exist during the pipeline
106+
creation time.
107+
columns (List[str]): list of columns that will be read from files.
108+
A column name may be a prefix of a nested field, e.g. 'a' will select
109+
'a.b', 'a.c', and 'a.d.e'
110+
"""
111+
112+
super(ReadFromParquetBatched, self).__init__()
113+
self._source = _create_parquet_source(
114+
file_pattern,
115+
min_bundle_size,
116+
validate=validate,
117+
columns=columns,
118+
)
119+
120+
def expand(self, pvalue):
121+
return pvalue.pipeline | Read(self._source)
122+
123+
def display_data(self):
124+
return {'source_dd': self._source}
49125

50126

51127
class ReadFromParquet(PTransform):
52128
"""A :class:`~apache_beam.transforms.ptransform.PTransform` for reading
53-
Parquet files. This `PTransform` is currently experimental. No
54-
backward-compatibility guarantees."""
129+
Parquet files as a `PCollection` of dictionaries. This `PTransform` is
130+
currently experimental. No backward-compatibility guarantees."""
55131

56132
def __init__(self, file_pattern=None, min_bundle_size=0,
57133
validate=True, columns=None):
@@ -86,8 +162,9 @@ def __init__(self, file_pattern=None, min_bundle_size=0,
86162
that are of simple types will be mapped into corresponding Python types.
87163
Records that are of complex types like list and struct will be mapped to
88164
Python list and dictionary respectively. For more information on supported
89-
types and schema, please see the pyarrow document.
165+
types and schema, please see the pyarrow documentation.
90166
167+
See also: :class:`~ReadFromParquetBatched`.
91168
92169
Args:
93170
file_pattern (str): the file glob to read
@@ -98,29 +175,29 @@ def __init__(self, file_pattern=None, min_bundle_size=0,
98175
columns (List[str]): list of columns that will be read from files.
99176
A column name may be a prefix of a nested field, e.g. 'a' will select
100177
'a.b', 'a.c', and 'a.d.e'
101-
"""
178+
"""
102179
super(ReadFromParquet, self).__init__()
103180
self._source = _create_parquet_source(
104181
file_pattern,
105182
min_bundle_size,
106183
validate=validate,
107-
columns=columns
184+
columns=columns,
108185
)
109186

110187
def expand(self, pvalue):
111-
return pvalue.pipeline | Read(self._source)
188+
return pvalue | Read(self._source) | ParDo(_ArrowTableToRowDictionaries())
112189

113190
def display_data(self):
114191
return {'source_dd': self._source}
115192

116193

117-
class ReadAllFromParquet(PTransform):
194+
class ReadAllFromParquetBatched(PTransform):
118195
"""A ``PTransform`` for reading ``PCollection`` of Parquet files.
119196
120197
Uses source ``_ParquetSource`` to read a ``PCollection`` of Parquet files or
121-
file patterns and produce a ``PCollection`` of Parquet records. This
122-
``PTransform`` is currently experimental. No backward-compatibility
123-
guarantees.
198+
file patterns and produce a ``PCollection`` of ``pyarrow.Table``, one for
199+
each Parquet file row group. This ``PTransform`` is currently experimental.
200+
No backward-compatibility guarantees.
124201
"""
125202

126203
DEFAULT_DESIRED_BUNDLE_SIZE = 64 * 1024 * 1024 # 64MB
@@ -140,7 +217,7 @@ def __init__(self, min_bundle_size=0,
140217
may be a prefix of a nested field, e.g. 'a' will select
141218
'a.b', 'a.c', and 'a.d.e'
142219
"""
143-
super(ReadAllFromParquet, self).__init__()
220+
super(ReadAllFromParquetBatched, self).__init__()
144221
source_from_file = partial(
145222
_create_parquet_source,
146223
min_bundle_size=min_bundle_size,
@@ -156,6 +233,14 @@ def expand(self, pvalue):
156233
return pvalue | self.label >> self._read_all_files
157234

158235

236+
class ReadAllFromParquet(PTransform):
237+
def __init__(self, **kwargs):
238+
self._read_batches = ReadAllFromParquetBatched(**kwargs)
239+
240+
def expand(self, pvalue):
241+
return pvalue | self._read_batches | ParDo(_ArrowTableToRowDictionaries())
242+
243+
159244
def _create_parquet_source(file_pattern=None,
160245
min_bundle_size=0,
161246
validate=False,
@@ -165,7 +250,7 @@ def _create_parquet_source(file_pattern=None,
165250
file_pattern=file_pattern,
166251
min_bundle_size=min_bundle_size,
167252
validate=validate,
168-
columns=columns
253+
columns=columns,
169254
)
170255

171256

@@ -244,13 +329,7 @@ def split_points_unclaimed(stop_position):
244329
else:
245330
next_block_start = range_tracker.stop_position()
246331

247-
num_rows = table.num_rows
248-
data_items = table.to_pydict().items()
249-
for n in range(num_rows):
250-
row = {}
251-
for column, values in data_items:
252-
row[column] = values[n]
253-
yield row
332+
yield table
254333

255334

256335
class WriteToParquet(PTransform):

0 commit comments

Comments
 (0)