forked from dask/dask
-
Notifications
You must be signed in to change notification settings - Fork 0
/
arrow.py
1615 lines (1441 loc) · 58.5 KB
/
arrow.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
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
import json
import textwrap
from collections import defaultdict
from datetime import datetime
import numpy as np
import pandas as pd
import pyarrow as pa
import pyarrow.parquet as pq
from packaging.version import parse as parse_version
from dask.base import tokenize
from dask.core import flatten
from dask.dataframe._compat import PANDAS_GT_120
from dask.dataframe.backends import pyarrow_schema_dispatch
from dask.dataframe.io.parquet.utils import (
Engine,
_get_aggregation_depth,
_normalize_index_columns,
_process_open_file_options,
_row_groups_to_parts,
_set_gather_statistics,
_set_metadata_task_size,
_sort_and_analyze_paths,
_split_user_options,
)
from dask.dataframe.io.utils import _get_pyarrow_dtypes, _is_local_fs, _open_input_files
from dask.dataframe.utils import clear_known_categories
from dask.delayed import Delayed
from dask.utils import getargspec, natural_sort_key
# Check PyArrow version for feature support
_pa_version = parse_version(pa.__version__)
from pyarrow import dataset as pa_ds
subset_stats_supported = _pa_version > parse_version("2.0.0")
pre_buffer_supported = _pa_version >= parse_version("5.0.0")
partitioning_supported = _pa_version >= parse_version("5.0.0")
del _pa_version
PYARROW_NULLABLE_DTYPE_MAPPING = {
pa.int8(): pd.Int8Dtype(),
pa.int16(): pd.Int16Dtype(),
pa.int32(): pd.Int32Dtype(),
pa.int64(): pd.Int64Dtype(),
pa.uint8(): pd.UInt8Dtype(),
pa.uint16(): pd.UInt16Dtype(),
pa.uint32(): pd.UInt32Dtype(),
pa.uint64(): pd.UInt64Dtype(),
pa.bool_(): pd.BooleanDtype(),
pa.string(): pd.StringDtype(),
}
if PANDAS_GT_120:
PYARROW_NULLABLE_DTYPE_MAPPING[pa.float32()] = pd.Float32Dtype()
PYARROW_NULLABLE_DTYPE_MAPPING[pa.float64()] = pd.Float64Dtype()
#
# Helper Utilities
#
def _append_row_groups(metadata, md):
"""Append row-group metadata and include a helpful
error message if an inconsistent schema is detected.
"""
try:
metadata.append_row_groups(md)
except RuntimeError as err:
if "requires equal schemas" in str(err):
raise RuntimeError(
"Schemas are inconsistent, try using "
'`to_parquet(..., schema="infer")`, or pass an explicit '
"pyarrow schema. Such as "
'`to_parquet(..., schema={"column1": pa.string()})`'
) from err
else:
raise err
def _write_partitioned(
table,
df,
root_path,
filename,
partition_cols,
fs,
pandas_to_arrow_table,
preserve_index,
index_cols=(),
return_metadata=True,
**kwargs,
):
"""Write table to a partitioned dataset with pyarrow.
Logic copied from pyarrow.parquet.
(arrow/python/pyarrow/parquet.py::write_to_dataset)
TODO: Remove this in favor of pyarrow's `write_to_dataset`
once ARROW-8244 is addressed.
"""
fs.mkdirs(root_path, exist_ok=True)
if preserve_index:
df.reset_index(inplace=True)
df = df[table.schema.names]
index_cols = list(index_cols) if index_cols else []
preserve_index = False
if index_cols:
df.set_index(index_cols, inplace=True)
preserve_index = True
partition_keys = [df[col] for col in partition_cols]
data_df = df.drop(partition_cols, axis="columns")
data_cols = df.columns.drop(partition_cols)
if len(data_cols) == 0 and not index_cols:
raise ValueError("No data left to save outside partition columns")
subschema = table.schema
for col in table.schema.names:
if col in partition_cols:
subschema = subschema.remove(subschema.get_field_index(col))
md_list = []
partition_keys = partition_keys[0] if len(partition_keys) == 1 else partition_keys
for keys, subgroup in data_df.groupby(partition_keys):
if not isinstance(keys, tuple):
keys = (keys,)
subdir = fs.sep.join(
[f"{name}={val}" for name, val in zip(partition_cols, keys)]
)
subtable = pandas_to_arrow_table(
subgroup, preserve_index=preserve_index, schema=subschema
)
prefix = fs.sep.join([root_path, subdir])
fs.mkdirs(prefix, exist_ok=True)
full_path = fs.sep.join([prefix, filename])
with fs.open(full_path, "wb") as f:
pq.write_table(
subtable,
f,
metadata_collector=md_list if return_metadata else None,
**kwargs,
)
if return_metadata:
md_list[-1].set_file_path(fs.sep.join([subdir, filename]))
return md_list
def _index_in_schema(index, schema):
"""Simple utility to check if all `index` columns are included
in the known `schema`.
"""
if index and schema is not None:
# Make sure all index columns are in user-defined schema
return len(set(index).intersection(schema.names)) == len(index)
elif index:
return True # Schema is not user-specified, all good
else:
return False # No index to check
class PartitionObj:
"""Simple class providing a `name` and `keys` attribute
for a single partition column.
This class was originally designed as a mechanism to build a
duck-typed version of pyarrow's deprecated `ParquetPartitions`
class. Now that `ArrowLegacyEngine` is deprecated, this class
can be modified/removed, but it is still used as a convenience.
"""
def __init__(self, name, keys):
self.name = name
self.keys = sorted(keys)
def _frag_subset(old_frag, row_groups):
"""Create new fragment with row-group subset."""
return old_frag.format.make_fragment(
old_frag.path,
old_frag.filesystem,
old_frag.partition_expression,
row_groups=row_groups,
)
def _get_pandas_metadata(schema):
"""Get pandas-specific metadata from schema."""
has_pandas_metadata = schema.metadata is not None and b"pandas" in schema.metadata
if has_pandas_metadata:
return json.loads(schema.metadata[b"pandas"].decode("utf8"))
else:
return {}
def _read_table_from_path(
path,
fs,
row_groups,
columns,
schema,
filters,
**kwargs,
):
"""Read arrow table from file path.
Used by `ArrowDatasetEngine._read_table` when no filters
are specified (otherwise fragments are converted directly
into tables).
"""
# Define file-opening options
read_kwargs = kwargs.get("read", {}).copy()
precache_options, open_file_options = _process_open_file_options(
read_kwargs.pop("open_file_options", {}),
**(
{
"allow_precache": False,
"default_cache": "none",
}
if _is_local_fs(fs)
else {
"columns": columns,
"row_groups": row_groups if row_groups == [None] else [row_groups],
"default_engine": "pyarrow",
"default_cache": "none",
}
),
)
# Use `pre_buffer=True` if the option is supported and an optimized
# "pre-caching" method isn't already specified in `precache_options`
# (The distinct fsspec and pyarrow optimizations will conflict)
pre_buffer_default = precache_options.get("method", None) is None
pre_buffer = (
{"pre_buffer": read_kwargs.pop("pre_buffer", pre_buffer_default)}
if pre_buffer_supported
else {}
)
with _open_input_files(
[path],
fs=fs,
precache_options=precache_options,
**open_file_options,
)[0] as fil:
if row_groups == [None]:
return pq.ParquetFile(fil, **pre_buffer).read(
columns=columns,
use_threads=False,
use_pandas_metadata=True,
**read_kwargs,
)
else:
return pq.ParquetFile(fil, **pre_buffer).read_row_groups(
row_groups,
columns=columns,
use_threads=False,
use_pandas_metadata=True,
**read_kwargs,
)
def _get_rg_statistics(row_group, col_indices):
"""Custom version of pyarrow's RowGroupInfo.statistics method
(https://github.com/apache/arrow/blob/master/python/pyarrow/_dataset.pyx)
We use col_indices to specify the specific subset of columns
that we need statistics for. This is more optimal than the
upstream `RowGroupInfo.statistics` method, which will return
statistics for all columns.
"""
if subset_stats_supported:
def name_stats(i):
col = row_group.metadata.column(i)
stats = col.statistics
if stats is None or not stats.has_min_max:
return None, None
name = col.path_in_schema
field_index = row_group.schema.get_field_index(name)
if field_index < 0:
return None, None
return col.path_in_schema, {
"min": stats.min,
"max": stats.max,
}
return {
name: stats
for name, stats in map(name_stats, col_indices.values())
if stats is not None
}
else:
return row_group.statistics
def _need_fragments(filters, partition_keys):
# Check if we need to generate a fragment for filtering.
# We only need to do this if we are applying filters to
# columns that were not already filtered by "partition".
partition_cols = (
{v[0] for v in flatten(partition_keys, container=list) if len(v)}
if partition_keys
else set()
)
filtered_cols = (
{v[0] for v in flatten(filters, container=list) if len(v)} if filters else set()
)
return bool(filtered_cols - partition_cols)
#
# ArrowDatasetEngine
#
class ArrowDatasetEngine(Engine):
#
# Public Class Methods
#
@classmethod
def read_metadata(
cls,
fs,
paths,
categories=None,
index=None,
use_nullable_dtypes=False,
gather_statistics=None,
filters=None,
split_row_groups=False,
chunksize=None,
aggregate_files=None,
ignore_metadata_file=False,
metadata_task_size=0,
parquet_file_extension=None,
**kwargs,
):
# Stage 1: Collect general dataset information
dataset_info = cls._collect_dataset_info(
paths,
fs,
categories,
index,
gather_statistics,
filters,
split_row_groups,
chunksize,
aggregate_files,
ignore_metadata_file,
metadata_task_size,
parquet_file_extension,
kwargs,
)
# Stage 2: Generate output `meta`
meta = cls._create_dd_meta(dataset_info, use_nullable_dtypes)
# Stage 3: Generate parts and stats
parts, stats, common_kwargs = cls._construct_collection_plan(dataset_info)
# Add `common_kwargs` and `aggregation_depth` to the first
# element of `parts`. We can return as a separate element
# in the future, but should avoid breaking the API for now.
if len(parts):
parts[0]["common_kwargs"] = common_kwargs
parts[0]["aggregation_depth"] = dataset_info["aggregation_depth"]
return (meta, stats, parts, dataset_info["index"])
@classmethod
def multi_support(cls):
return cls == ArrowDatasetEngine
@classmethod
def read_partition(
cls,
fs,
pieces,
columns,
index,
use_nullable_dtypes=False,
categories=(),
partitions=(),
filters=None,
schema=None,
**kwargs,
):
"""Read in a single output partition"""
if isinstance(index, list):
for level in index:
# unclear if we can use set ops here. I think the order matters.
# Need the membership test to avoid duplicating index when
# we slice with `columns` later on.
if level not in columns:
columns.append(level)
# Ensure `columns` and `partitions` do not overlap
columns_and_parts = columns.copy()
if not isinstance(partitions, (list, tuple)):
if columns_and_parts and partitions:
for part_name in partitions.partition_names:
if part_name in columns:
columns.remove(part_name)
else:
columns_and_parts.append(part_name)
columns = columns or None
# Always convert pieces to list
if not isinstance(pieces, list):
pieces = [pieces]
tables = []
multi_read = len(pieces) > 1
for piece in pieces:
if isinstance(piece, str):
# `piece` is a file-path string
path_or_frag = piece
row_group = None
partition_keys = None
else:
# `piece` contains (path, row_group, partition_keys)
(path_or_frag, row_group, partition_keys) = piece
# Convert row_group to a list and be sure to
# check if msgpack converted it to a tuple
if isinstance(row_group, tuple):
row_group = list(row_group)
if not isinstance(row_group, list):
row_group = [row_group]
# Read in arrow table and convert to pandas
arrow_table = cls._read_table(
path_or_frag,
fs,
row_group,
columns,
schema,
filters,
partitions,
partition_keys,
**kwargs,
)
if multi_read:
tables.append(arrow_table)
if multi_read:
arrow_table = pa.concat_tables(tables)
# Convert to pandas
df = cls._arrow_table_to_pandas(
arrow_table, categories, use_nullable_dtypes=use_nullable_dtypes, **kwargs
)
# For pyarrow.dataset api, need to convert partition columns
# to categorigal manually for integer types.
if partitions and isinstance(partitions, list):
for partition in partitions:
if df[partition.name].dtype.name != "category":
# We read directly from fragments, so the partition
# columns are already in our dataframe. We just
# need to convert non-categorical types.
df[partition.name] = pd.Series(
pd.Categorical(
categories=partition.keys,
values=df[partition.name].values,
),
index=df.index,
)
# Note that `to_pandas(ignore_metadata=False)` means
# pyarrow will use the pandas metadata to set the index.
index_in_columns_and_parts = set(df.index.names).issubset(
set(columns_and_parts)
)
if not index:
if index_in_columns_and_parts:
# User does not want to set index and a desired
# column/partition has been set to the index
df.reset_index(drop=False, inplace=True)
else:
# User does not want to set index and an
# "unwanted" column has been set to the index
df.reset_index(drop=True, inplace=True)
else:
if set(df.index.names) != set(index) and index_in_columns_and_parts:
# The wrong index has been set and it contains
# one or more desired columns/partitions
df.reset_index(drop=False, inplace=True)
elif index_in_columns_and_parts:
# The correct index has already been set
index = False
columns_and_parts = list(set(columns_and_parts) - set(df.index.names))
df = df[list(columns_and_parts)]
if index:
df = df.set_index(index)
return df
@classmethod
def initialize_write(
cls,
df,
fs,
path,
append=False,
partition_on=None,
ignore_divisions=False,
division_info=None,
schema="infer",
index_cols=None,
**kwargs,
):
if schema == "infer" or isinstance(schema, dict):
# Start with schema from _meta_nonempty
inferred_schema = pyarrow_schema_dispatch(
df._meta_nonempty.set_index(index_cols)
if index_cols
else df._meta_nonempty
).remove_metadata()
# Use dict to update our inferred schema
if isinstance(schema, dict):
schema = pa.schema(schema)
for name in schema.names:
i = inferred_schema.get_field_index(name)
j = schema.get_field_index(name)
inferred_schema = inferred_schema.set(i, schema.field(j))
schema = inferred_schema
# Check that target directory exists
fs.mkdirs(path, exist_ok=True)
if append and division_info is None:
ignore_divisions = True
full_metadata = None # metadata for the full dataset, from _metadata
tail_metadata = None # metadata for at least the last file in the dataset
i_offset = 0
metadata_file_exists = False
if append:
# Extract metadata and get file offset if appending
ds = pa_ds.dataset(path, filesystem=fs, format="parquet")
i_offset = len(ds.files)
if i_offset > 0:
try:
with fs.open(fs.sep.join([path, "_metadata"]), mode="rb") as fil:
full_metadata = pq.read_metadata(fil)
tail_metadata = full_metadata
metadata_file_exists = True
except OSError:
try:
with fs.open(
sorted(ds.files, key=natural_sort_key)[-1], mode="rb"
) as fil:
tail_metadata = pq.read_metadata(fil)
except OSError:
pass
else:
append = False # No existing files, can skip the append logic
# If appending, validate against the initial metadata file (if present)
if append and tail_metadata is not None:
arrow_schema = tail_metadata.schema.to_arrow_schema()
names = arrow_schema.names
has_pandas_metadata = (
arrow_schema.metadata is not None and b"pandas" in arrow_schema.metadata
)
if has_pandas_metadata:
pandas_metadata = json.loads(
arrow_schema.metadata[b"pandas"].decode("utf8")
)
categories = [
c["name"]
for c in pandas_metadata["columns"]
if c["pandas_type"] == "categorical"
]
else:
categories = None
dtypes = _get_pyarrow_dtypes(arrow_schema, categories)
if set(names) != set(df.columns) - set(partition_on):
raise ValueError(
"Appended columns not the same.\n"
"Previous: {} | New: {}".format(names, list(df.columns))
)
elif (pd.Series(dtypes).loc[names] != df[names].dtypes).any():
# TODO Coerce values for compatible but different dtypes
raise ValueError(
"Appended dtypes differ.\n{}".format(
set(dtypes.items()) ^ set(df.dtypes.items())
)
)
# Check divisions if necessary
if division_info["name"] not in names:
ignore_divisions = True
if not ignore_divisions:
old_end = None
row_groups = (
tail_metadata.row_group(i)
for i in range(tail_metadata.num_row_groups)
)
index_col_i = names.index(division_info["name"])
for row_group in row_groups:
column = row_group.column(index_col_i)
if column.statistics:
if old_end is None:
old_end = column.statistics.max
elif column.statistics.max > old_end:
old_end = column.statistics.max
else:
# Existing column on disk isn't sorted, set
# `old_end = None` to skip check below
old_end = None
break
divisions = division_info["divisions"]
if old_end is not None and divisions[0] <= old_end:
raise ValueError(
"The divisions of the appended dataframe overlap with "
"previously written divisions. If this is desired, set "
"``ignore_divisions=True`` to append anyway.\n"
"- End of last written partition: {old_end}\n"
"- Start of first new partition: {divisions[0]}"
)
extra_write_kwargs = {"schema": schema, "index_cols": index_cols}
return i_offset, full_metadata, metadata_file_exists, extra_write_kwargs
@classmethod
def _pandas_to_arrow_table(
cls, df: pd.DataFrame, preserve_index=False, schema=None
) -> pa.Table:
try:
return pa.Table.from_pandas(
df, nthreads=1, preserve_index=preserve_index, schema=schema
)
except pa.ArrowException as exc:
if schema is None:
raise
df_schema = pa.Schema.from_pandas(df)
expected = textwrap.indent(
schema.to_string(show_schema_metadata=False), " "
)
actual = textwrap.indent(
df_schema.to_string(show_schema_metadata=False), " "
)
raise ValueError(
f"Failed to convert partition to expected pyarrow schema:\n"
f" `{exc!r}`\n"
f"\n"
f"Expected partition schema:\n"
f"{expected}\n"
f"\n"
f"Received partition schema:\n"
f"{actual}\n"
f"\n"
f"This error *may* be resolved by passing in schema information for\n"
f"the mismatched column(s) using the `schema` keyword in `to_parquet`."
) from None
@classmethod
def write_partition(
cls,
df,
path,
fs,
filename,
partition_on,
return_metadata,
fmd=None,
compression=None,
index_cols=None,
schema=None,
head=False,
custom_metadata=None,
**kwargs,
):
_meta = None
preserve_index = False
if _index_in_schema(index_cols, schema):
df.set_index(index_cols, inplace=True)
preserve_index = True
else:
index_cols = []
t = cls._pandas_to_arrow_table(df, preserve_index=preserve_index, schema=schema)
if custom_metadata:
_md = t.schema.metadata
_md.update(custom_metadata)
t = t.replace_schema_metadata(metadata=_md)
if partition_on:
md_list = _write_partitioned(
t,
df,
path,
filename,
partition_on,
fs,
cls._pandas_to_arrow_table,
preserve_index,
index_cols=index_cols,
compression=compression,
return_metadata=return_metadata,
**kwargs,
)
if md_list:
_meta = md_list[0]
for i in range(1, len(md_list)):
_append_row_groups(_meta, md_list[i])
else:
md_list = []
with fs.open(fs.sep.join([path, filename]), "wb") as fil:
pq.write_table(
t,
fil,
compression=compression,
metadata_collector=md_list if return_metadata else None,
**kwargs,
)
if md_list:
_meta = md_list[0]
_meta.set_file_path(filename)
# Return the schema needed to write the metadata
if return_metadata:
d = {"meta": _meta}
if head:
# Only return schema if this is the "head" partition
d["schema"] = t.schema
return [d]
else:
return []
@classmethod
def write_metadata(cls, parts, meta, fs, path, append=False, **kwargs):
schema = parts[0][0].get("schema", None)
parts = [p for p in parts if p[0]["meta"] is not None]
if parts:
if not append:
# Get only arguments specified in the function
common_metadata_path = fs.sep.join([path, "_common_metadata"])
keywords = getargspec(pq.write_metadata).args
kwargs_meta = {k: v for k, v in kwargs.items() if k in keywords}
with fs.open(common_metadata_path, "wb") as fil:
pq.write_metadata(schema, fil, **kwargs_meta)
# Aggregate metadata and write to _metadata file
metadata_path = fs.sep.join([path, "_metadata"])
if append and meta is not None:
_meta = meta
i_start = 0
else:
_meta = parts[0][0]["meta"]
i_start = 1
for i in range(i_start, len(parts)):
_append_row_groups(_meta, parts[i][0]["meta"])
with fs.open(metadata_path, "wb") as fil:
_meta.write_metadata_file(fil)
#
# Private Class Methods
#
@classmethod
def _collect_dataset_info(
cls,
paths,
fs,
categories,
index,
gather_statistics,
filters,
split_row_groups,
chunksize,
aggregate_files,
ignore_metadata_file,
metadata_task_size,
parquet_file_extension,
kwargs,
):
"""pyarrow.dataset version of _collect_dataset_info
Use pyarrow.dataset API to construct a dictionary of all
general information needed to read the dataset.
"""
# Use pyarrow.dataset API
ds = None
valid_paths = None # Only used if `paths` is a list containing _metadata
# Extract "supported" key-word arguments from `kwargs`
(
_dataset_kwargs,
read_kwargs,
user_kwargs,
) = _split_user_options(**kwargs)
if "partitioning" not in _dataset_kwargs:
_dataset_kwargs["partitioning"] = "hive"
if "format" not in _dataset_kwargs:
_dataset_kwargs["format"] = pa_ds.ParquetFileFormat()
# Case-dependent pyarrow.dataset creation
has_metadata_file = False
if len(paths) == 1 and fs.isdir(paths[0]):
# Use _analyze_paths to avoid relative-path
# problems (see GH#5608)
paths, base, fns = _sort_and_analyze_paths(paths, fs)
paths = fs.sep.join([base, fns[0]])
meta_path = fs.sep.join([paths, "_metadata"])
if not ignore_metadata_file and fs.exists(meta_path):
# Use _metadata file
ds = pa_ds.parquet_dataset(
meta_path,
filesystem=fs,
**_dataset_kwargs,
)
has_metadata_file = True
elif parquet_file_extension:
# Need to materialize all paths if we are missing the _metadata file
# Raise error if all files have been filtered by extension
len0 = len(paths)
paths = [
path
for path in fs.find(paths)
if path.endswith(parquet_file_extension)
]
if len0 and paths == []:
raise ValueError(
"No files satisfy the `parquet_file_extension` criteria "
f"(files must end with {parquet_file_extension})."
)
elif len(paths) > 1:
paths, base, fns = _sort_and_analyze_paths(paths, fs)
meta_path = fs.sep.join([base, "_metadata"])
if "_metadata" in fns:
# Pyarrow cannot handle "_metadata" when `paths` is a list
# Use _metadata file
if not ignore_metadata_file:
ds = pa_ds.parquet_dataset(
meta_path,
filesystem=fs,
**_dataset_kwargs,
)
has_metadata_file = True
# Populate valid_paths, since the original path list
# must be used to filter the _metadata-based dataset
fns.remove("_metadata")
valid_paths = fns
# Final "catch-all" pyarrow.dataset call
if ds is None:
ds = pa_ds.dataset(
paths,
filesystem=fs,
**_dataset_kwargs,
)
# Deal with directory partitioning
# Get all partition keys (without filters) to populate partition_obj
partition_obj = [] # See `partition_info` description below
hive_categories = defaultdict(list)
file_frag = None
for file_frag in ds.get_fragments():
if partitioning_supported:
# Can avoid manual category discovery for pyarrow>=5.0.0
break
keys = pa_ds._get_partition_keys(file_frag.partition_expression)
if not (keys or hive_categories):
break # Bail - This is not a hive-partitioned dataset
for k, v in keys.items():
if v not in hive_categories[k]:
hive_categories[k].append(v)
physical_schema = ds.schema
if file_frag is not None:
physical_schema = file_frag.physical_schema
# Check/correct order of `categories` using last file_frag
# TODO: Remove this after pyarrow>=5.0.0 is required
#
# Note that `_get_partition_keys` does NOT preserve the
# partition-hierarchy order of the keys. Therefore, we
# use custom logic to determine the "correct" oredering
# of the `categories` output.
#
# Example (why we need to "reorder" `categories`):
#
# # Fragment path has "hive" structure
# file_frag.path
#
# '/data/path/b=x/c=x/part.1.parquet'
#
# # `categories` may NOT preserve the hierachy order
# categories.keys()
#
# dict_keys(['c', 'b'])
#
cat_keys = [
part.split("=")[0]
for part in file_frag.path.split(fs.sep)
if "=" in part
]
if set(hive_categories) == set(cat_keys):
hive_categories = {
k: hive_categories[k] for k in cat_keys if k in hive_categories
}
if (
partitioning_supported
and ds.partitioning.dictionaries
and all(arr is not None for arr in ds.partitioning.dictionaries)
):
# Use ds.partitioning for pyarrow>=5.0.0
partition_names = list(ds.partitioning.schema.names)
for i, name in enumerate(partition_names):
partition_obj.append(
PartitionObj(name, ds.partitioning.dictionaries[i].to_pandas())
)
else:
partition_names = list(hive_categories)
for name in partition_names:
partition_obj.append(PartitionObj(name, hive_categories[name]))
# Check the `aggregate_files` setting
aggregation_depth = _get_aggregation_depth(aggregate_files, partition_names)
# Note on (hive) partitioning information:
#
# - "partitions" : (list of PartitionObj) This is a list of
# simple objects providing `name` and `keys` attributes
# for each partition column.
# - "partition_names" : (list) This is a list containing the
# names of partitioned columns.
#
return {
"ds": ds,
"physical_schema": physical_schema,
"has_metadata_file": has_metadata_file,
"schema": ds.schema,
"fs": fs,
"valid_paths": valid_paths,
"gather_statistics": gather_statistics,
"categories": categories,
"index": index,
"filters": filters,
"split_row_groups": split_row_groups,
"chunksize": chunksize,
"aggregate_files": aggregate_files,
"aggregation_depth": aggregation_depth,
"partitions": partition_obj,
"partition_names": partition_names,
"metadata_task_size": metadata_task_size,
"kwargs": {
"dataset": _dataset_kwargs,
"read": read_kwargs,
**user_kwargs,
},
}
@classmethod
def _create_dd_meta(cls, dataset_info, use_nullable_dtypes=False):
"""Use parquet schema and hive-partition information
(stored in dataset_info) to construct DataFrame metadata.
"""
# Collect necessary information from dataset_info
schema = dataset_info["schema"]
index = dataset_info["index"]
categories = dataset_info["categories"]
partition_obj = dataset_info["partitions"]
partitions = dataset_info["partition_names"]
physical_column_names = dataset_info.get("physical_schema", schema).names
columns = None
# Use pandas metadata to update categories
pandas_metadata = _get_pandas_metadata(schema)
if pandas_metadata:
if categories is None: