/
core.py
599 lines (512 loc) · 18.6 KB
/
core.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
from __future__ import absolute_import, division, print_function
from distutils.version import LooseVersion
import warnings
import toolz
from ...core import DataFrame, new_dd_object
from ....bytes.compression import compress
from ....base import tokenize
from ....compatibility import PY3, string_types
from ....bytes.core import get_fs_token_paths
from ....bytes.utils import infer_storage_options
from ....utils import import_required, natural_sort_key
__all__ = ("read_parquet", "to_parquet")
# ----------------------------------------------------------------------
# User API
def read_parquet(
path,
columns=None,
filters=None,
categories=None,
index=None,
storage_options=None,
engine="auto",
gather_statistics=None,
):
"""
Read ParquetFile into a Dask DataFrame
This reads a directory of Parquet data into a Dask.dataframe, one file per
partition. It selects the index among the sorted columns if any exist.
Parameters
----------
path : string, list or fastparquet.ParquetFile
Source directory for data, or path(s) to individual parquet files.
Prefix with a protocol like ``s3://`` to read from alternative
filesystems. To read from multiple files you can pass a globstring or a
list of paths, with the caveat that they must all have the same
protocol.
Alternatively, also accepts a previously opened
fastparquet.ParquetFile()
columns : string, list or None (default)
Field name(s) to read in as columns in the output. By default all
non-index fields will be read (as determined by the pandas parquet
metadata, if present). Provide a single field name instead of a list to
read in the data as a Series.
filters : list
List of filters to apply, like ``[('x', '>', 0), ...]``. This implements
row-group (partition) -level filtering only, i.e., to prevent the
loading of some chunks of the data, and only if relevant statistics
have been included in the metadata.
index : string, list, False or None (default)
Field name(s) to use as the output frame index. By default will be
inferred from the pandas parquet file metadata (if present). Use False
to read all fields as columns.
categories : list, dict or None
For any fields listed here, if the parquet encoding is Dictionary,
the column will be created with dtype category. Use only if it is
guaranteed that the column is encoded as dictionary in all row-groups.
If a list, assumes up to 2**16-1 labels; if a dict, specify the number
of labels expected; if None, will load categories automatically for
data written by dask/fastparquet, not otherwise.
storage_options : dict
Key/value pairs to be passed on to the file-system backend, if any.
engine : {'auto', 'fastparquet', 'pyarrow'}, default 'auto'
Parquet reader library to use. If only one library is installed, it
will use that one; if both, it will use 'fastparquet'
gather_statistics : bool or None (default).
Gather the statistics for each dataset partition. By default,
this will only be done if the _metadata file is available. Otherwise,
statistics will only be gathered if True, because the footer of
every file will be parsed (which is very slow on some systems).
Examples
--------
>>> df = dd.read_parquet('s3://bucket/my-parquet-data') # doctest: +SKIP
See Also
--------
to_parquet
"""
# First, check if this is a ParquetFile
try:
import fastparquet
if isinstance(path, fastparquet.api.ParquetFile):
if path.open != fastparquet.util.default_open:
assert re.match(".*://", path.fn), (
"ParquetFile: Path must contain protocol"
+ " (e.g., s3://...) when using other than the default"
+ " LocalFileSystem. Path given: "
+ path.fn
)
assert engine in ["auto", "fastparquet"], (
"'engine' should be set to 'auto' or 'fastparquet' "
+ "when reading from fastparquet.ParquetFile"
)
if path.fn.endswith("_metadata"):
# remove '_metadata' from path
urlpath = path.fn[: -len("_metadata")]
else:
urlpath = path.fn
fs, fs_token, paths = get_fs_token_paths(
urlpath, mode="rb", storage_options=storage_options
)
return read_parquet(
paths,
columns,
filters,
categories,
index,
storage_options,
engine,
gather_statistics,
)
except ImportError:
pass
if isinstance(columns, str):
df = read_parquet(
path,
[columns],
filters,
categories,
index,
storage_options,
engine,
gather_statistics,
)
return df[columns]
if columns is not None:
columns = list(columns)
name = "read-parquet-" + tokenize(
path,
columns,
filters,
categories,
index,
storage_options,
engine,
gather_statistics,
)
if isinstance(engine, str):
engine = get_engine(engine)
fs, _, paths = get_fs_token_paths(
path, mode="rb", storage_options=storage_options
)
paths = sorted(
paths, key=natural_sort_key
) # numeric rather than glob ordering
index, meta, statistics, parts = engine.read_metadata(
fs,
paths,
categories=categories,
index=index,
gather_statistics=gather_statistics,
)
sanitize_cols = False
if columns is None:
# User didn't specify columns, so remove intersection with
# user-specified index values (if necessary)
sanitize_cols = True
columns = meta.columns
if not set(columns).issubset(set(meta.columns)):
raise KeyError(
"The following columns were not found in the dataset %s\n"
"The following columns were found %s"
% (set(columns) - set(meta.columns), meta.columns)
)
if statistics:
result = list(
zip(
*[
(part, stats)
for part, stats in zip(parts, statistics)
if stats["num-rows"] > 0
]
)
)
if result:
parts, statistics = result
else:
parts, statistics = [], []
if filters:
parts, statistics = apply_filters(parts, statistics, filters)
out = sorted_columns(statistics)
if index and out:
out = [
o for o in out if o["name"] == index
] # only one valid column
if index is not False and len(out) == 1:
divisions = out[0]["divisions"]
index = out[0]["name"]
elif index is not False and len(out) > 1:
if any(o["name"] == "index" for o in out):
[o] = [o for o in out if o["name"] == "index"]
divisions = o["divisions"]
index = o["name"]
else:
# Multiple sorted columns found, cannot autodetect
index = False
divisions = [None] * (len(parts) + 1)
else:
divisions = [None] * (len(parts) + 1)
else:
divisions = [None] * (len(parts) + 1)
if index:
if isinstance(index, str):
index = [index]
if isinstance(columns, str):
columns = [columns]
if sanitize_cols:
columns = [col for col in columns if col not in index]
if set(index).intersection(columns):
raise ValueError(
"Specified index and column names must not " "intersect"
)
for ind in index:
if ind not in columns:
columns = columns + [ind]
meta = meta[list(columns)]
subgraph = {
(name, i): (
read_parquet_part,
engine.read_partition,
fs,
meta,
part["piece"],
columns,
index,
part["kwargs"],
)
for i, part in enumerate(parts)
}
if index:
meta = meta.set_index(index)
return new_dd_object(subgraph, name, meta, divisions)
def read_parquet_part(func, fs, meta, part, columns, index, kwargs):
""" Read a part of a parquet dataset """
df = func(fs, part, columns, index, **kwargs)
if not len(df):
df = meta
if index:
df = df.set_index(index)
return df
def to_parquet(
df,
path,
engine="auto",
compression="default",
write_index=True,
append=False,
ignore_divisions=False,
partition_on=None,
storage_options=None,
write_metadata_file=True,
compute=True,
**kwargs
):
"""Store Dask.dataframe to Parquet files
Notes
-----
Each partition will be written to a separate file.
Parameters
----------
df : dask.dataframe.DataFrame
path : string
Destination directory for data. Prepend with protocol like ``s3://``
or ``hdfs://`` for remote data.
engine : {'auto', 'fastparquet', 'pyarrow'}, default 'auto'
Parquet library to use. If only one library is installed, it will use
that one; if both, it will use 'fastparquet'.
compression : string or dict, optional
Either a string like ``"snappy"`` or a dictionary mapping column names
to compressors like ``{"name": "gzip", "values": "snappy"}``. The
default is ``"default"``, which uses the default compression for
whichever engine is selected.
write_index : boolean, optional
Whether or not to write the index. Defaults to True.
append : bool, optional
If False (default), construct data-set from scratch. If True, add new
row-group(s) to an existing data-set. In the latter case, the data-set
must exist, and the schema must match the input data.
ignore_divisions : bool, optional
If False (default) raises error when previous divisions overlap with
the new appended divisions. Ignored if append=False.
partition_on : list, optional
Construct directory-based partitioning by splitting on these fields'
values. Each dask partition will result in one or more datafiles,
there will be no global groupby.
storage_options : dict, optional
Key/value pairs to be passed on to the file-system backend, if any.
write_metadata_file : bool, optional
Whether to create the special "_metadata" file,
compute : bool, optional
If True (default) then the result is computed immediately. If False
then a ``dask.delayed`` object is returned for future computation.
**kwargs
Extra options to be passed on to the specific backend.
Examples
--------
>>> df = dd.read_csv(...) # doctest: +SKIP
>>> dd.to_parquet(df, '/path/to/output/', compression='snappy') # doctest: +SKIP
See Also
--------
read_parquet: Read parquet data to dask.dataframe
"""
from dask import delayed
partition_on = partition_on or []
if isinstance(partition_on, string_types):
partition_on = [partition_on]
if set(partition_on) - set(df.columns):
raise ValueError(
"Partitioning on non-existent column. "
"partition_on=%s ."
"columns=%s" % (str(partition_on), str(list(df.columns)))
)
if compression != "default":
kwargs["compression"] = compression
elif "snappy" in compress:
kwargs["compression"] = "snappy"
if isinstance(engine, str):
engine = get_engine(engine)
fs, _, _ = get_fs_token_paths(
path, mode="wb", storage_options=storage_options
)
# Trim any protocol information from the path before forwarding
# ideally, this should be done as a method of the file-system
path = infer_storage_options(path)["path"]
# Save divisions and corresponding index name
# TODO: What if the division are not along the index?
division_info = {"divisions": df.divisions, "name": df.index.name}
if division_info["name"] == None:
division_info["name"] = "index"
# If write_index==True (default), reset the index and record the
# name of the original index in `index_cols` (will be `index` if None).
# `fastparquet` will use `index_cols` to specify the index column(s)
# in the metadata. `pyarrow` will revert the `reset_index` call
# below if `index_cols` is populated (because pyarrow can handle
# index preservation itself). For both engines, the column index
# will be written to "pandas metadata" if write_index=True
index_cols = []
if write_index:
real_cols = set(df.columns)
df = df.reset_index()
index_cols = [c for c in set(df.columns).difference(real_cols)]
_to_parquet_kwargs = {
"engine",
"compression",
"write_index",
"append",
"ignore_divisions",
"partition_on",
"storage_options",
"write_metadata_file",
"compute"
}
kwargs_pass = {
k: v for k, v in kwargs.items() if k not in _to_parquet_kwargs
}
# Engine-specific initialization steps to write the dataset.
# Possibly create parquet metadata, and load existing stuff if appending
meta, i_offset = engine.initialize_write(
df,
fs,
path,
append=append,
ignore_divisions=ignore_divisions,
partition_on=partition_on,
division_info=division_info,
index_cols=index_cols,
**kwargs_pass
)
# Use i_offset and df.npartitions to define file-name list
filenames = [
"part.%i.parquet" % (i + i_offset) for i in range(df.npartitions)
]
# write parts
dwrite = delayed(engine.write_partition)
parts = [
dwrite(
d,
path,
fs,
filename,
partition_on,
with_metadata=write_metadata_file,
fmd=meta,
index_cols=index_cols,
**kwargs_pass
)
for d, filename in zip(df.to_delayed(), filenames)
]
# single task to complete
out = delayed(engine.write_metadata)(
parts, meta, fs, path, append=append, **kwargs_pass
)
if compute:
out = out.compute()
return out
_ENGINES = {}
def get_engine(engine):
"""Get the parquet engine backend implementation.
Parameters
----------
engine : {'auto', 'fastparquet', 'pyarrow'}, default 'auto'
Parquet reader library to use. Defaults to fastparquet if both are
installed
Returns
-------
A dict containing a ``'read'`` and ``'write'`` function.
"""
if engine in _ENGINES:
return _ENGINES[engine]
if engine == "auto":
for eng in ["fastparquet", "pyarrow"]:
try:
return get_engine(eng)
except RuntimeError:
pass
else:
raise RuntimeError("Please install either fastparquet or pyarrow")
elif engine == "fastparquet":
import_required("fastparquet", "`fastparquet` not installed")
from .fastparquet import FastParquetEngine
_ENGINES["fastparquet"] = eng = FastParquetEngine
return eng
elif engine == "pyarrow" or engine == "arrow":
pa = import_required("pyarrow", "`pyarrow` not installed")
from .arrow import ArrowEngine
if LooseVersion(pa.__version__) < "0.8.0":
raise RuntimeError("PyArrow version >= 0.8.0 required")
_ENGINES["pyarrow"] = eng = ArrowEngine
return eng
else:
raise ValueError(
'Unsupported engine: "{0}".'.format(engine)
+ ' Valid choices include "pyarrow" and "fastparquet".'
)
#####################
# Utility Functions #
#####################
def sorted_columns(statistics):
""" Find sorted columns given row-group statistics
This finds all columns that are sorted, along with appropriate divisions
values for those columns
Returns
-------
out: List of {'name': str, 'divisions': List[str]} dictionaries
"""
if not statistics:
return []
out = []
for i, c in enumerate(statistics[0]["columns"]):
if not all(
"min" in s["columns"][i] and "max" in s["columns"][i]
for s in statistics
):
continue
divisions = [c["min"]]
max = c["max"]
success = True
for stats in statistics[1:]:
c = stats["columns"][i]
if c["min"] >= max:
divisions.append(c["min"])
max = c["max"]
else:
success = False
break
if success:
divisions.append(max)
assert divisions == sorted(divisions)
out.append({"name": c["name"], "divisions": divisions})
return out
def apply_filters(parts, statistics, filters):
""" Apply filters onto parts/statistics pairs
Parameters
----------
parts: list
Tokens corresponding to row groups to read in the future
statistics: List[dict]
List of statistics for each part, including min and max values
filters: List[Tuple[str, str, Any]]
List like [('x', '>', 5), ('y', '==', 'Alice')]
Returns
-------
parts, statistics: the same as the input, but possibly a subset
"""
for column, operator, value in filters:
out_parts = []
out_statistics = []
for part, stats in zip(parts, statistics):
try:
c = toolz.groupby("name", stats["columns"])[column][0]
min = c["min"]
max = c["max"]
except KeyError:
out_parts.append(part)
out_statistics.append(stats)
else:
if (
operator == "=="
and min <= value <= max
or operator == "<"
and min < value
or operator == "<="
and min <= value
or operator == ">"
and max > value
or operator == ">="
and max >= value
):
out_parts.append(part)
out_statistics.append(stats)
parts, statistics = out_parts, out_statistics
return parts, statistics
if PY3:
DataFrame.to_parquet.__doc__ = to_parquet.__doc__