-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
Copy pathcore.py
2375 lines (2016 loc) · 88.3 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
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
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from collections import defaultdict
from contextlib import nullcontext
from functools import reduce
import inspect
import json
import os
import re
import operator
import warnings
import pyarrow as pa
try:
import pyarrow._parquet as _parquet
except ImportError as exc:
raise ImportError(
"The pyarrow installation is not built with support "
f"for the Parquet file format ({str(exc)})"
) from None
from pyarrow._parquet import (ParquetReader, Statistics, # noqa
FileMetaData, RowGroupMetaData,
ColumnChunkMetaData,
ParquetSchema, ColumnSchema,
ParquetLogicalType,
FileEncryptionProperties,
FileDecryptionProperties,
SortingColumn)
from pyarrow.fs import (LocalFileSystem, FileSystem, FileType,
_resolve_filesystem_and_path, _ensure_filesystem)
from pyarrow.util import guid, _is_path_like, _stringify_path, _deprecate_api
def _check_contains_null(val):
if isinstance(val, bytes):
for byte in val:
if isinstance(byte, bytes):
compare_to = chr(0)
else:
compare_to = 0
if byte == compare_to:
return True
elif isinstance(val, str):
return '\x00' in val
return False
def _check_filters(filters, check_null_strings=True):
"""
Check if filters are well-formed.
"""
if filters is not None:
if len(filters) == 0 or any(len(f) == 0 for f in filters):
raise ValueError("Malformed filters")
if isinstance(filters[0][0], str):
# We have encountered the situation where we have one nesting level
# too few:
# We have [(,,), ..] instead of [[(,,), ..]]
filters = [filters]
if check_null_strings:
for conjunction in filters:
for col, op, val in conjunction:
if (
isinstance(val, list) and
all(_check_contains_null(v) for v in val) or
_check_contains_null(val)
):
raise NotImplementedError(
"Null-terminated binary strings are not supported "
"as filter values."
)
return filters
_DNF_filter_doc = """Predicates are expressed using an ``Expression`` or using
the disjunctive normal form (DNF), like ``[[('x', '=', 0), ...], ...]``.
DNF allows arbitrary boolean logical combinations of single column predicates.
The innermost tuples each describe a single column predicate. The list of inner
predicates is interpreted as a conjunction (AND), forming a more selective and
multiple column predicate. Finally, the most outer list combines these filters
as a disjunction (OR).
Predicates may also be passed as List[Tuple]. This form is interpreted
as a single conjunction. To express OR in predicates, one must
use the (preferred) List[List[Tuple]] notation.
Each tuple has format: (``key``, ``op``, ``value``) and compares the
``key`` with the ``value``.
The supported ``op`` are: ``=`` or ``==``, ``!=``, ``<``, ``>``, ``<=``,
``>=``, ``in`` and ``not in``. If the ``op`` is ``in`` or ``not in``, the
``value`` must be a collection such as a ``list``, a ``set`` or a
``tuple``.
Examples:
Using the ``Expression`` API:
.. code-block:: python
import pyarrow.compute as pc
pc.field('x') = 0
pc.field('y').isin(['a', 'b', 'c'])
~pc.field('y').isin({'a', 'b'})
Using the DNF format:
.. code-block:: python
('x', '=', 0)
('y', 'in', ['a', 'b', 'c'])
('z', 'not in', {'a','b'})
"""
def filters_to_expression(filters):
"""
Check if filters are well-formed and convert to an ``Expression``.
Parameters
----------
filters : List[Tuple] or List[List[Tuple]]
Notes
-----
See internal ``pyarrow._DNF_filter_doc`` attribute for more details.
Examples
--------
>>> filters_to_expression([('foo', '==', 'bar')])
<pyarrow.compute.Expression (foo == "bar")>
Returns
-------
pyarrow.compute.Expression
An Expression representing the filters
"""
import pyarrow.dataset as ds
if isinstance(filters, ds.Expression):
return filters
filters = _check_filters(filters, check_null_strings=False)
def convert_single_predicate(col, op, val):
field = ds.field(col)
if op == "=" or op == "==":
return field == val
elif op == "!=":
return field != val
elif op == '<':
return field < val
elif op == '>':
return field > val
elif op == '<=':
return field <= val
elif op == '>=':
return field >= val
elif op == 'in':
return field.isin(val)
elif op == 'not in':
return ~field.isin(val)
else:
raise ValueError(
'"{0}" is not a valid operator in predicates.'.format(
(col, op, val)))
disjunction_members = []
for conjunction in filters:
conjunction_members = [
convert_single_predicate(col, op, val)
for col, op, val in conjunction
]
disjunction_members.append(reduce(operator.and_, conjunction_members))
return reduce(operator.or_, disjunction_members)
_filters_to_expression = _deprecate_api(
"_filters_to_expression", "filters_to_expression",
filters_to_expression, "10.0.0", DeprecationWarning)
# ----------------------------------------------------------------------
# Reading a single Parquet file
class ParquetFile:
"""
Reader interface for a single Parquet file.
Parameters
----------
source : str, pathlib.Path, pyarrow.NativeFile, or file-like object
Readable source. For passing bytes or buffer-like file containing a
Parquet file, use pyarrow.BufferReader.
metadata : FileMetaData, default None
Use existing metadata object, rather than reading from file.
common_metadata : FileMetaData, default None
Will be used in reads for pandas schema metadata if not found in the
main file's metadata, no other uses at the moment.
read_dictionary : list
List of column names to read directly as DictionaryArray.
memory_map : bool, default False
If the source is a file path, use a memory map to read file, which can
improve performance in some environments.
buffer_size : int, default 0
If positive, perform read buffering when deserializing individual
column chunks. Otherwise IO calls are unbuffered.
pre_buffer : bool, default False
Coalesce and issue file reads in parallel to improve performance on
high-latency filesystems (e.g. S3). If True, Arrow will use a
background I/O thread pool.
coerce_int96_timestamp_unit : str, default None
Cast timestamps that are stored in INT96 format to a particular
resolution (e.g. 'ms'). Setting to None is equivalent to 'ns'
and therefore INT96 timestamps will be inferred as timestamps
in nanoseconds.
decryption_properties : FileDecryptionProperties, default None
File decryption properties for Parquet Modular Encryption.
thrift_string_size_limit : int, default None
If not None, override the maximum total string size allocated
when decoding Thrift structures. The default limit should be
sufficient for most Parquet files.
thrift_container_size_limit : int, default None
If not None, override the maximum total size of containers allocated
when decoding Thrift structures. The default limit should be
sufficient for most Parquet files.
filesystem : FileSystem, default None
If nothing passed, will be inferred based on path.
Path will try to be found in the local on-disk filesystem otherwise
it will be parsed as an URI to determine the filesystem.
page_checksum_verification : bool, default False
If True, verify the checksum for each page read from the file.
Examples
--------
Generate an example PyArrow Table and write it to Parquet file:
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
Create a ``ParquetFile`` object from the Parquet file:
>>> parquet_file = pq.ParquetFile('example.parquet')
Read the data:
>>> parquet_file.read()
pyarrow.Table
n_legs: int64
animal: string
----
n_legs: [[2,2,4,4,5,100]]
animal: [["Flamingo","Parrot","Dog","Horse","Brittle stars","Centipede"]]
Create a ParquetFile object with "animal" column as DictionaryArray:
>>> parquet_file = pq.ParquetFile('example.parquet',
... read_dictionary=["animal"])
>>> parquet_file.read()
pyarrow.Table
n_legs: int64
animal: dictionary<values=string, indices=int32, ordered=0>
----
n_legs: [[2,2,4,4,5,100]]
animal: [ -- dictionary:
["Flamingo","Parrot",...,"Brittle stars","Centipede"] -- indices:
[0,1,2,3,4,5]]
"""
def __init__(self, source, *, metadata=None, common_metadata=None,
read_dictionary=None, memory_map=False, buffer_size=0,
pre_buffer=False, coerce_int96_timestamp_unit=None,
decryption_properties=None, thrift_string_size_limit=None,
thrift_container_size_limit=None, filesystem=None,
page_checksum_verification=False):
self._close_source = getattr(source, 'closed', True)
filesystem, source = _resolve_filesystem_and_path(
source, filesystem, memory_map=memory_map)
if filesystem is not None:
source = filesystem.open_input_file(source)
self._close_source = True # We opened it here, ensure we close it.
self.reader = ParquetReader()
self.reader.open(
source, use_memory_map=memory_map,
buffer_size=buffer_size, pre_buffer=pre_buffer,
read_dictionary=read_dictionary, metadata=metadata,
coerce_int96_timestamp_unit=coerce_int96_timestamp_unit,
decryption_properties=decryption_properties,
thrift_string_size_limit=thrift_string_size_limit,
thrift_container_size_limit=thrift_container_size_limit,
page_checksum_verification=page_checksum_verification,
)
self.common_metadata = common_metadata
self._nested_paths_by_prefix = self._build_nested_paths()
def __enter__(self):
return self
def __exit__(self, *args, **kwargs):
self.close()
def _build_nested_paths(self):
paths = self.reader.column_paths
result = defaultdict(list)
for i, path in enumerate(paths):
key = path[0]
rest = path[1:]
while True:
result[key].append(i)
if not rest:
break
key = '.'.join((key, rest[0]))
rest = rest[1:]
return result
@property
def metadata(self):
"""
Return the Parquet metadata.
"""
return self.reader.metadata
@property
def schema(self):
"""
Return the Parquet schema, unconverted to Arrow types
"""
return self.metadata.schema
@property
def schema_arrow(self):
"""
Return the inferred Arrow schema, converted from the whole Parquet
file's schema
Examples
--------
Generate an example Parquet file:
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
>>> parquet_file = pq.ParquetFile('example.parquet')
Read the Arrow schema:
>>> parquet_file.schema_arrow
n_legs: int64
animal: string
"""
return self.reader.schema_arrow
@property
def num_row_groups(self):
"""
Return the number of row groups of the Parquet file.
Examples
--------
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
>>> parquet_file = pq.ParquetFile('example.parquet')
>>> parquet_file.num_row_groups
1
"""
return self.reader.num_row_groups
def close(self, force: bool = False):
if self._close_source or force:
self.reader.close()
@property
def closed(self) -> bool:
return self.reader.closed
def read_row_group(self, i, columns=None, use_threads=True,
use_pandas_metadata=False):
"""
Read a single row group from a Parquet file.
Parameters
----------
i : int
Index of the individual row group that we want to read.
columns : list
If not None, only these columns will be read from the row group. A
column name may be a prefix of a nested field, e.g. 'a' will select
'a.b', 'a.c', and 'a.d.e'.
use_threads : bool, default True
Perform multi-threaded column reads.
use_pandas_metadata : bool, default False
If True and file has custom pandas schema metadata, ensure that
index columns are also loaded.
Returns
-------
pyarrow.table.Table
Content of the row group as a table (of columns)
Examples
--------
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
>>> parquet_file = pq.ParquetFile('example.parquet')
>>> parquet_file.read_row_group(0)
pyarrow.Table
n_legs: int64
animal: string
----
n_legs: [[2,2,4,4,5,100]]
animal: [["Flamingo","Parrot",...,"Brittle stars","Centipede"]]
"""
column_indices = self._get_column_indices(
columns, use_pandas_metadata=use_pandas_metadata)
return self.reader.read_row_group(i, column_indices=column_indices,
use_threads=use_threads)
def read_row_groups(self, row_groups, columns=None, use_threads=True,
use_pandas_metadata=False):
"""
Read a multiple row groups from a Parquet file.
Parameters
----------
row_groups : list
Only these row groups will be read from the file.
columns : list
If not None, only these columns will be read from the row group. A
column name may be a prefix of a nested field, e.g. 'a' will select
'a.b', 'a.c', and 'a.d.e'.
use_threads : bool, default True
Perform multi-threaded column reads.
use_pandas_metadata : bool, default False
If True and file has custom pandas schema metadata, ensure that
index columns are also loaded.
Returns
-------
pyarrow.table.Table
Content of the row groups as a table (of columns).
Examples
--------
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
>>> parquet_file = pq.ParquetFile('example.parquet')
>>> parquet_file.read_row_groups([0,0])
pyarrow.Table
n_legs: int64
animal: string
----
n_legs: [[2,2,4,4,5,...,2,4,4,5,100]]
animal: [["Flamingo","Parrot","Dog",...,"Brittle stars","Centipede"]]
"""
column_indices = self._get_column_indices(
columns, use_pandas_metadata=use_pandas_metadata)
return self.reader.read_row_groups(row_groups,
column_indices=column_indices,
use_threads=use_threads)
def iter_batches(self, batch_size=65536, row_groups=None, columns=None,
use_threads=True, use_pandas_metadata=False):
"""
Read streaming batches from a Parquet file.
Parameters
----------
batch_size : int, default 64K
Maximum number of records to yield per batch. Batches may be
smaller if there aren't enough rows in the file.
row_groups : list
Only these row groups will be read from the file.
columns : list
If not None, only these columns will be read from the file. A
column name may be a prefix of a nested field, e.g. 'a' will select
'a.b', 'a.c', and 'a.d.e'.
use_threads : boolean, default True
Perform multi-threaded column reads.
use_pandas_metadata : boolean, default False
If True and file has custom pandas schema metadata, ensure that
index columns are also loaded.
Yields
------
pyarrow.RecordBatch
Contents of each batch as a record batch
Examples
--------
Generate an example Parquet file:
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
>>> parquet_file = pq.ParquetFile('example.parquet')
>>> for i in parquet_file.iter_batches():
... print("RecordBatch")
... print(i.to_pandas())
...
RecordBatch
n_legs animal
0 2 Flamingo
1 2 Parrot
2 4 Dog
3 4 Horse
4 5 Brittle stars
5 100 Centipede
"""
if row_groups is None:
row_groups = range(0, self.metadata.num_row_groups)
column_indices = self._get_column_indices(
columns, use_pandas_metadata=use_pandas_metadata)
batches = self.reader.iter_batches(batch_size,
row_groups=row_groups,
column_indices=column_indices,
use_threads=use_threads)
return batches
def read(self, columns=None, use_threads=True, use_pandas_metadata=False):
"""
Read a Table from Parquet format.
Parameters
----------
columns : list
If not None, only these columns will be read from the file. A
column name may be a prefix of a nested field, e.g. 'a' will select
'a.b', 'a.c', and 'a.d.e'.
use_threads : bool, default True
Perform multi-threaded column reads.
use_pandas_metadata : bool, default False
If True and file has custom pandas schema metadata, ensure that
index columns are also loaded.
Returns
-------
pyarrow.table.Table
Content of the file as a table (of columns).
Examples
--------
Generate an example Parquet file:
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
>>> parquet_file = pq.ParquetFile('example.parquet')
Read a Table:
>>> parquet_file.read(columns=["animal"])
pyarrow.Table
animal: string
----
animal: [["Flamingo","Parrot",...,"Brittle stars","Centipede"]]
"""
column_indices = self._get_column_indices(
columns, use_pandas_metadata=use_pandas_metadata)
return self.reader.read_all(column_indices=column_indices,
use_threads=use_threads)
def scan_contents(self, columns=None, batch_size=65536):
"""
Read contents of file for the given columns and batch size.
Notes
-----
This function's primary purpose is benchmarking.
The scan is executed on a single thread.
Parameters
----------
columns : list of integers, default None
Select columns to read, if None scan all columns.
batch_size : int, default 64K
Number of rows to read at a time internally.
Returns
-------
num_rows : int
Number of rows in file
Examples
--------
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> import pyarrow.parquet as pq
>>> pq.write_table(table, 'example.parquet')
>>> parquet_file = pq.ParquetFile('example.parquet')
>>> parquet_file.scan_contents()
6
"""
column_indices = self._get_column_indices(columns)
return self.reader.scan_contents(column_indices,
batch_size=batch_size)
def _get_column_indices(self, column_names, use_pandas_metadata=False):
if column_names is None:
return None
indices = []
for name in column_names:
if name in self._nested_paths_by_prefix:
indices.extend(self._nested_paths_by_prefix[name])
if use_pandas_metadata:
file_keyvalues = self.metadata.metadata
common_keyvalues = (self.common_metadata.metadata
if self.common_metadata is not None
else None)
if file_keyvalues and b'pandas' in file_keyvalues:
index_columns = _get_pandas_index_columns(file_keyvalues)
elif common_keyvalues and b'pandas' in common_keyvalues:
index_columns = _get_pandas_index_columns(common_keyvalues)
else:
index_columns = []
if indices is not None and index_columns:
indices += [self.reader.column_name_idx(descr)
for descr in index_columns
if not isinstance(descr, dict)]
return indices
_SPARK_DISALLOWED_CHARS = re.compile('[ ,;{}()\n\t=]')
def _sanitized_spark_field_name(name):
return _SPARK_DISALLOWED_CHARS.sub('_', name)
def _sanitize_schema(schema, flavor):
if 'spark' in flavor:
sanitized_fields = []
schema_changed = False
for field in schema:
name = field.name
sanitized_name = _sanitized_spark_field_name(name)
if sanitized_name != name:
schema_changed = True
sanitized_field = pa.field(sanitized_name, field.type,
field.nullable, field.metadata)
sanitized_fields.append(sanitized_field)
else:
sanitized_fields.append(field)
new_schema = pa.schema(sanitized_fields, metadata=schema.metadata)
return new_schema, schema_changed
else:
return schema, False
def _sanitize_table(table, new_schema, flavor):
# TODO: This will not handle prohibited characters in nested field names
if 'spark' in flavor:
column_data = [table[i] for i in range(table.num_columns)]
return pa.Table.from_arrays(column_data, schema=new_schema)
else:
return table
_parquet_writer_arg_docs = """version : {"1.0", "2.4", "2.6"}, default "2.6"
Determine which Parquet logical types are available for use, whether the
reduced set from the Parquet 1.x.x format or the expanded logical types
added in later format versions.
Files written with version='2.4' or '2.6' may not be readable in all
Parquet implementations, so version='1.0' is likely the choice that
maximizes file compatibility.
UINT32 and some logical types are only available with version '2.4'.
Nanosecond timestamps are only available with version '2.6'.
Other features such as compression algorithms or the new serialized
data page format must be enabled separately (see 'compression' and
'data_page_version').
use_dictionary : bool or list, default True
Specify if we should use dictionary encoding in general or only for
some columns.
When encoding the column, if the dictionary size is too large, the
column will fallback to ``PLAIN`` encoding. Specially, ``BOOLEAN`` type
doesn't support dictionary encoding.
compression : str or dict, default 'snappy'
Specify the compression codec, either on a general basis or per-column.
Valid values: {'NONE', 'SNAPPY', 'GZIP', 'BROTLI', 'LZ4', 'ZSTD'}.
write_statistics : bool or list, default True
Specify if we should write statistics in general (default is True) or only
for some columns.
use_deprecated_int96_timestamps : bool, default None
Write timestamps to INT96 Parquet format. Defaults to False unless enabled
by flavor argument. This take priority over the coerce_timestamps option.
coerce_timestamps : str, default None
Cast timestamps to a particular resolution. If omitted, defaults are chosen
depending on `version`. For ``version='1.0'`` and ``version='2.4'``,
nanoseconds are cast to microseconds ('us'), while for
``version='2.6'`` (the default), they are written natively without loss
of resolution. Seconds are always cast to milliseconds ('ms') by default,
as Parquet does not have any temporal type with seconds resolution.
If the casting results in loss of data, it will raise an exception
unless ``allow_truncated_timestamps=True`` is given.
Valid values: {None, 'ms', 'us'}
allow_truncated_timestamps : bool, default False
Allow loss of data when coercing timestamps to a particular
resolution. E.g. if microsecond or nanosecond data is lost when coercing to
'ms', do not raise an exception. Passing ``allow_truncated_timestamp=True``
will NOT result in the truncation exception being ignored unless
``coerce_timestamps`` is not None.
data_page_size : int, default None
Set a target threshold for the approximate encoded size of data
pages within a column chunk (in bytes). If None, use the default data page
size of 1MByte.
flavor : {'spark'}, default None
Sanitize schema or set other compatibility options to work with
various target systems.
filesystem : FileSystem, default None
If nothing passed, will be inferred from `where` if path-like, else
`where` is already a file-like object so no filesystem is needed.
compression_level : int or dict, default None
Specify the compression level for a codec, either on a general basis or
per-column. If None is passed, arrow selects the compression level for
the compression codec in use. The compression level has a different
meaning for each codec, so you have to read the documentation of the
codec you are using.
An exception is thrown if the compression codec does not allow specifying
a compression level.
use_byte_stream_split : bool or list, default False
Specify if the byte_stream_split encoding should be used in general or
only for some columns. If both dictionary and byte_stream_stream are
enabled, then dictionary is preferred.
The byte_stream_split encoding is valid for integer, floating-point
and fixed-size binary data types (including decimals); it should be
combined with a compression codec so as to achieve size reduction.
column_encoding : string or dict, default None
Specify the encoding scheme on a per column basis.
Can only be used when ``use_dictionary`` is set to False, and
cannot be used in combination with ``use_byte_stream_split``.
Currently supported values: {'PLAIN', 'BYTE_STREAM_SPLIT',
'DELTA_BINARY_PACKED', 'DELTA_LENGTH_BYTE_ARRAY', 'DELTA_BYTE_ARRAY'}.
Certain encodings are only compatible with certain data types.
Please refer to the encodings section of `Reading and writing Parquet
files <https://arrow.apache.org/docs/cpp/parquet.html#encodings>`_.
data_page_version : {"1.0", "2.0"}, default "1.0"
The serialized Parquet data page format version to write, defaults to
1.0. This does not impact the file schema logical types and Arrow to
Parquet type casting behavior; for that use the "version" option.
use_compliant_nested_type : bool, default True
Whether to write compliant Parquet nested type (lists) as defined
`here <https://github.com/apache/parquet-format/blob/master/
LogicalTypes.md#nested-types>`_, defaults to ``True``.
For ``use_compliant_nested_type=True``, this will write into a list
with 3-level structure where the middle level, named ``list``,
is a repeated group with a single field named ``element``::
<list-repetition> group <name> (LIST) {
repeated group list {
<element-repetition> <element-type> element;
}
}
For ``use_compliant_nested_type=False``, this will also write into a list
with 3-level structure, where the name of the single field of the middle
level ``list`` is taken from the element name for nested columns in Arrow,
which defaults to ``item``::
<list-repetition> group <name> (LIST) {
repeated group list {
<element-repetition> <element-type> item;
}
}
encryption_properties : FileEncryptionProperties, default None
File encryption properties for Parquet Modular Encryption.
If None, no encryption will be done.
The encryption properties can be created using:
``CryptoFactory.file_encryption_properties()``.
write_batch_size : int, default None
Number of values to write to a page at a time. If None, use the default of
1024. ``write_batch_size`` is complementary to ``data_page_size``. If pages
are exceeding the ``data_page_size`` due to large column values, lowering
the batch size can help keep page sizes closer to the intended size.
dictionary_pagesize_limit : int, default None
Specify the dictionary page size limit per row group. If None, use the
default 1MB.
store_schema : bool, default True
By default, the Arrow schema is serialized and stored in the Parquet
file metadata (in the "ARROW:schema" key). When reading the file,
if this key is available, it will be used to more faithfully recreate
the original Arrow data. For example, for tz-aware timestamp columns
it will restore the timezone (Parquet only stores the UTC values without
timezone), or columns with duration type will be restored from the int64
Parquet column.
write_page_index : bool, default False
Whether to write a page index in general for all columns.
Writing statistics to the page index disables the old method of writing
statistics to each data page header. The page index makes statistics-based
filtering more efficient than the page header, as it gathers all the
statistics for a Parquet file in a single place, avoiding scattered I/O.
Note that the page index is not yet used on the read size by PyArrow.
write_page_checksum : bool, default False
Whether to write page checksums in general for all columns.
Page checksums enable detection of data corruption, which might occur during
transmission or in the storage.
sorting_columns : Sequence of SortingColumn, default None
Specify the sort order of the data being written. The writer does not sort
the data nor does it verify that the data is sorted. The sort order is
written to the row group metadata, which can then be used by readers.
store_decimal_as_integer : bool, default False
Allow decimals with 1 <= precision <= 18 to be stored as integers.
In Parquet, DECIMAL can be stored in any of the following physical types:
- int32: for 1 <= precision <= 9.
- int64: for 10 <= precision <= 18.
- fixed_len_byte_array: precision is limited by the array size.
Length n can store <= floor(log_10(2^(8*n - 1) - 1)) base-10 digits.
- binary: precision is unlimited. The minimum number of bytes to store the
unscaled value is used.
By default, this is DISABLED and all decimal types annotate fixed_len_byte_array.
When enabled, the writer will use the following physical types to store decimals:
- int32: for 1 <= precision <= 9.
- int64: for 10 <= precision <= 18.
- fixed_len_byte_array: for precision > 18.
As a consequence, decimal columns stored in integer types are more compact.
"""
_parquet_writer_example_doc = """\
Generate an example PyArrow Table and RecordBatch:
>>> import pyarrow as pa
>>> table = pa.table({'n_legs': [2, 2, 4, 4, 5, 100],
... 'animal': ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]})
>>> batch = pa.record_batch([[2, 2, 4, 4, 5, 100],
... ["Flamingo", "Parrot", "Dog", "Horse",
... "Brittle stars", "Centipede"]],
... names=['n_legs', 'animal'])
create a ParquetWriter object:
>>> import pyarrow.parquet as pq
>>> writer = pq.ParquetWriter('example.parquet', table.schema)
and write the Table into the Parquet file:
>>> writer.write_table(table)
>>> writer.close()
>>> pq.read_table('example.parquet').to_pandas()
n_legs animal
0 2 Flamingo
1 2 Parrot
2 4 Dog
3 4 Horse
4 5 Brittle stars
5 100 Centipede
create a ParquetWriter object for the RecordBatch:
>>> writer2 = pq.ParquetWriter('example2.parquet', batch.schema)
and write the RecordBatch into the Parquet file:
>>> writer2.write_batch(batch)
>>> writer2.close()
>>> pq.read_table('example2.parquet').to_pandas()
n_legs animal
0 2 Flamingo
1 2 Parrot
2 4 Dog
3 4 Horse
4 5 Brittle stars
5 100 Centipede
"""
class ParquetWriter:
__doc__ = """
Class for incrementally building a Parquet file for Arrow tables.
Parameters
----------
where : path or file-like object
schema : pyarrow.Schema
{}
writer_engine_version : unused
**options : dict
If options contains a key `metadata_collector` then the
corresponding value is assumed to be a list (or any object with
`.append` method) that will be filled with the file metadata instance
of the written file.
Examples
--------
{}
""".format(_parquet_writer_arg_docs, _parquet_writer_example_doc)
def __init__(self, where, schema, filesystem=None,
flavor=None,
version='2.6',
use_dictionary=True,
compression='snappy',
write_statistics=True,
use_deprecated_int96_timestamps=None,
compression_level=None,
use_byte_stream_split=False,
column_encoding=None,
writer_engine_version=None,
data_page_version='1.0',
use_compliant_nested_type=True,
encryption_properties=None,
write_batch_size=None,
dictionary_pagesize_limit=None,
store_schema=True,
write_page_index=False,
write_page_checksum=False,
sorting_columns=None,
store_decimal_as_integer=False,
**options):
if use_deprecated_int96_timestamps is None:
# Use int96 timestamps for Spark
if flavor is not None and 'spark' in flavor:
use_deprecated_int96_timestamps = True
else:
use_deprecated_int96_timestamps = False
self.flavor = flavor
if flavor is not None:
schema, self.schema_changed = _sanitize_schema(schema, flavor)
else: