-
Notifications
You must be signed in to change notification settings - Fork 166
/
pyarrow.py
1897 lines (1519 loc) · 73.3 KB
/
pyarrow.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.
# pylint: disable=redefined-outer-name,arguments-renamed,fixme
"""FileIO implementation for reading and writing table files that uses pyarrow.fs.
This file contains a FileIO implementation that relies on the filesystem interface provided
by PyArrow. It relies on PyArrow's `from_uri` method that infers the correct filesystem
type to use. Theoretically, this allows the supported storage types to grow naturally
with the pyarrow library.
"""
from __future__ import annotations
import concurrent.futures
import fnmatch
import itertools
import logging
import os
import re
from abc import ABC, abstractmethod
from concurrent.futures import Future
from dataclasses import dataclass
from enum import Enum
from functools import lru_cache, singledispatch
from typing import (
TYPE_CHECKING,
Any,
Callable,
Dict,
Generic,
Iterable,
Iterator,
List,
Optional,
Set,
Tuple,
TypeVar,
Union,
cast,
)
from urllib.parse import urlparse
import numpy as np
import pyarrow as pa
import pyarrow.compute as pc
import pyarrow.dataset as ds
import pyarrow.lib
import pyarrow.parquet as pq
from pyarrow import ChunkedArray
from pyarrow.fs import (
FileInfo,
FileSystem,
FileType,
FSSpecHandler,
)
from sortedcontainers import SortedList
from pyiceberg.conversions import to_bytes
from pyiceberg.exceptions import ResolveError
from pyiceberg.expressions import (
AlwaysTrue,
BooleanExpression,
BoundTerm,
)
from pyiceberg.expressions.literals import Literal
from pyiceberg.expressions.visitors import (
BoundBooleanExpressionVisitor,
bind,
extract_field_ids,
translate_column_names,
)
from pyiceberg.expressions.visitors import visit as boolean_expression_visit
from pyiceberg.io import (
GCS_DEFAULT_LOCATION,
GCS_ENDPOINT,
GCS_TOKEN,
GCS_TOKEN_EXPIRES_AT_MS,
HDFS_HOST,
HDFS_KERB_TICKET,
HDFS_PORT,
HDFS_USER,
S3_ACCESS_KEY_ID,
S3_CONNECT_TIMEOUT,
S3_ENDPOINT,
S3_PROXY_URI,
S3_REGION,
S3_SECRET_ACCESS_KEY,
S3_SESSION_TOKEN,
FileIO,
InputFile,
InputStream,
OutputFile,
OutputStream,
)
from pyiceberg.manifest import (
DataFile,
DataFileContent,
FileFormat,
)
from pyiceberg.partitioning import PartitionField, PartitionSpec, partition_record_value
from pyiceberg.schema import (
PartnerAccessor,
PreOrderSchemaVisitor,
Schema,
SchemaVisitorPerPrimitiveType,
SchemaWithPartnerVisitor,
pre_order_visit,
promote,
prune_columns,
sanitize_column_names,
visit,
visit_with_partner,
)
from pyiceberg.table import PropertyUtil, TableProperties, WriteTask
from pyiceberg.table.metadata import TableMetadata
from pyiceberg.table.name_mapping import NameMapping
from pyiceberg.transforms import TruncateTransform
from pyiceberg.typedef import EMPTY_DICT, Properties, Record
from pyiceberg.types import (
BinaryType,
BooleanType,
DateType,
DecimalType,
DoubleType,
FixedType,
FloatType,
IcebergType,
IntegerType,
ListType,
LongType,
MapType,
NestedField,
PrimitiveType,
StringType,
StructType,
TimestampType,
TimestamptzType,
TimeType,
UUIDType,
)
from pyiceberg.utils.concurrent import ExecutorFactory
from pyiceberg.utils.datetime import millis_to_datetime
from pyiceberg.utils.singleton import Singleton
from pyiceberg.utils.truncate import truncate_upper_bound_binary_string, truncate_upper_bound_text_string
if TYPE_CHECKING:
from pyiceberg.table import FileScanTask, Table
logger = logging.getLogger(__name__)
ONE_MEGABYTE = 1024 * 1024
BUFFER_SIZE = "buffer-size"
ICEBERG_SCHEMA = b"iceberg.schema"
# The PARQUET: in front means that it is Parquet specific, in this case the field_id
PYARROW_PARQUET_FIELD_ID_KEY = b"PARQUET:field_id"
PYARROW_FIELD_DOC_KEY = b"doc"
LIST_ELEMENT_NAME = "element"
MAP_KEY_NAME = "key"
MAP_VALUE_NAME = "value"
DOC = "doc"
T = TypeVar("T")
class PyArrowLocalFileSystem(pyarrow.fs.LocalFileSystem):
def open_output_stream(self, path: str, *args: Any, **kwargs: Any) -> pyarrow.NativeFile:
# In LocalFileSystem, parent directories must be first created before opening an output stream
self.create_dir(os.path.dirname(path), recursive=True)
return super().open_output_stream(path, *args, **kwargs)
class PyArrowFile(InputFile, OutputFile):
"""A combined InputFile and OutputFile implementation that uses a pyarrow filesystem to generate pyarrow.lib.NativeFile instances.
Args:
location (str): A URI or a path to a local file.
Attributes:
location(str): The URI or path to a local file for a PyArrowFile instance.
Examples:
>>> from pyiceberg.io.pyarrow import PyArrowFile
>>> # input_file = PyArrowFile("s3://foo/bar.txt")
>>> # Read the contents of the PyArrowFile instance
>>> # Make sure that you have permissions to read/write
>>> # file_content = input_file.open().read()
>>> # output_file = PyArrowFile("s3://baz/qux.txt")
>>> # Write bytes to a file
>>> # Make sure that you have permissions to read/write
>>> # output_file.create().write(b'foobytes')
"""
_fs: FileSystem
_path: str
_buffer_size: int
def __init__(self, location: str, path: str, fs: FileSystem, buffer_size: int = ONE_MEGABYTE):
self._filesystem = fs
self._path = path
self._buffer_size = buffer_size
super().__init__(location=location)
def _file_info(self) -> FileInfo:
"""Retrieve a pyarrow.fs.FileInfo object for the location.
Raises:
PermissionError: If the file at self.location cannot be accessed due to a permission error such as
an AWS error code 15.
"""
try:
file_info = self._filesystem.get_file_info(self._path)
except OSError as e:
if e.errno == 13 or "AWS Error [code 15]" in str(e):
raise PermissionError(f"Cannot get file info, access denied: {self.location}") from e
raise # pragma: no cover - If some other kind of OSError, raise the raw error
if file_info.type == FileType.NotFound:
raise FileNotFoundError(f"Cannot get file info, file not found: {self.location}")
return file_info
def __len__(self) -> int:
"""Return the total length of the file, in bytes."""
file_info = self._file_info()
return file_info.size
def exists(self) -> bool:
"""Check whether the location exists."""
try:
self._file_info() # raises FileNotFoundError if it does not exist
return True
except FileNotFoundError:
return False
def open(self, seekable: bool = True) -> InputStream:
"""Open the location using a PyArrow FileSystem inferred from the location.
Args:
seekable: If the stream should support seek, or if it is consumed sequential.
Returns:
pyarrow.lib.NativeFile: A NativeFile instance for the file located at `self.location`.
Raises:
FileNotFoundError: If the file at self.location does not exist.
PermissionError: If the file at self.location cannot be accessed due to a permission error such as
an AWS error code 15.
"""
try:
if seekable:
input_file = self._filesystem.open_input_file(self._path)
else:
input_file = self._filesystem.open_input_stream(self._path, buffer_size=self._buffer_size)
except FileNotFoundError:
raise
except PermissionError:
raise
except OSError as e:
if e.errno == 2 or "Path does not exist" in str(e):
raise FileNotFoundError(f"Cannot open file, does not exist: {self.location}") from e
elif e.errno == 13 or "AWS Error [code 15]" in str(e):
raise PermissionError(f"Cannot open file, access denied: {self.location}") from e
raise # pragma: no cover - If some other kind of OSError, raise the raw error
return input_file
def create(self, overwrite: bool = False) -> OutputStream:
"""Create a writable pyarrow.lib.NativeFile for this PyArrowFile's location.
Args:
overwrite (bool): Whether to overwrite the file if it already exists.
Returns:
pyarrow.lib.NativeFile: A NativeFile instance for the file located at self.location.
Raises:
FileExistsError: If the file already exists at `self.location` and `overwrite` is False.
Note:
This retrieves a pyarrow NativeFile by opening an output stream. If overwrite is set to False,
a check is first performed to verify that the file does not exist. This is not thread-safe and
a possibility does exist that the file can be created by a concurrent process after the existence
check yet before the output stream is created. In such a case, the default pyarrow behavior will
truncate the contents of the existing file when opening the output stream.
"""
try:
if not overwrite and self.exists() is True:
raise FileExistsError(f"Cannot create file, already exists: {self.location}")
output_file = self._filesystem.open_output_stream(self._path, buffer_size=self._buffer_size)
except PermissionError:
raise
except OSError as e:
if e.errno == 13 or "AWS Error [code 15]" in str(e):
raise PermissionError(f"Cannot create file, access denied: {self.location}") from e
raise # pragma: no cover - If some other kind of OSError, raise the raw error
return output_file
def to_input_file(self) -> PyArrowFile:
"""Return a new PyArrowFile for the location of an existing PyArrowFile instance.
This method is included to abide by the OutputFile abstract base class. Since this implementation uses a single
PyArrowFile class (as opposed to separate InputFile and OutputFile implementations), this method effectively returns
a copy of the same instance.
"""
return self
class PyArrowFileIO(FileIO):
fs_by_scheme: Callable[[str, Optional[str]], FileSystem]
def __init__(self, properties: Properties = EMPTY_DICT):
self.fs_by_scheme: Callable[[str, Optional[str]], FileSystem] = lru_cache(self._initialize_fs)
super().__init__(properties=properties)
@staticmethod
def parse_location(location: str) -> Tuple[str, str, str]:
"""Return the path without the scheme."""
uri = urlparse(location)
if not uri.scheme:
return "file", uri.netloc, os.path.abspath(location)
elif uri.scheme == "hdfs":
return uri.scheme, uri.netloc, location
else:
return uri.scheme, uri.netloc, f"{uri.netloc}{uri.path}"
def _initialize_fs(self, scheme: str, netloc: Optional[str] = None) -> FileSystem:
if scheme in {"s3", "s3a", "s3n"}:
from pyarrow.fs import S3FileSystem
client_kwargs: Dict[str, Any] = {
"endpoint_override": self.properties.get(S3_ENDPOINT),
"access_key": self.properties.get(S3_ACCESS_KEY_ID),
"secret_key": self.properties.get(S3_SECRET_ACCESS_KEY),
"session_token": self.properties.get(S3_SESSION_TOKEN),
"region": self.properties.get(S3_REGION),
}
if proxy_uri := self.properties.get(S3_PROXY_URI):
client_kwargs["proxy_options"] = proxy_uri
if connect_timeout := self.properties.get(S3_CONNECT_TIMEOUT):
client_kwargs["connect_timeout"] = float(connect_timeout)
return S3FileSystem(**client_kwargs)
elif scheme == "hdfs":
from pyarrow.fs import HadoopFileSystem
hdfs_kwargs: Dict[str, Any] = {}
if netloc:
return HadoopFileSystem.from_uri(f"hdfs://{netloc}")
if host := self.properties.get(HDFS_HOST):
hdfs_kwargs["host"] = host
if port := self.properties.get(HDFS_PORT):
# port should be an integer type
hdfs_kwargs["port"] = int(port)
if user := self.properties.get(HDFS_USER):
hdfs_kwargs["user"] = user
if kerb_ticket := self.properties.get(HDFS_KERB_TICKET):
hdfs_kwargs["kerb_ticket"] = kerb_ticket
return HadoopFileSystem(**hdfs_kwargs)
elif scheme in {"gs", "gcs"}:
from pyarrow.fs import GcsFileSystem
gcs_kwargs: Dict[str, Any] = {}
if access_token := self.properties.get(GCS_TOKEN):
gcs_kwargs["access_token"] = access_token
if expiration := self.properties.get(GCS_TOKEN_EXPIRES_AT_MS):
gcs_kwargs["credential_token_expiration"] = millis_to_datetime(int(expiration))
if bucket_location := self.properties.get(GCS_DEFAULT_LOCATION):
gcs_kwargs["default_bucket_location"] = bucket_location
if endpoint := self.properties.get(GCS_ENDPOINT):
url_parts = urlparse(endpoint)
gcs_kwargs["scheme"] = url_parts.scheme
gcs_kwargs["endpoint_override"] = url_parts.netloc
return GcsFileSystem(**gcs_kwargs)
elif scheme == "file":
return PyArrowLocalFileSystem()
else:
raise ValueError(f"Unrecognized filesystem type in URI: {scheme}")
def new_input(self, location: str) -> PyArrowFile:
"""Get a PyArrowFile instance to read bytes from the file at the given location.
Args:
location (str): A URI or a path to a local file.
Returns:
PyArrowFile: A PyArrowFile instance for the given location.
"""
scheme, netloc, path = self.parse_location(location)
return PyArrowFile(
fs=self.fs_by_scheme(scheme, netloc),
location=location,
path=path,
buffer_size=int(self.properties.get(BUFFER_SIZE, ONE_MEGABYTE)),
)
def new_output(self, location: str) -> PyArrowFile:
"""Get a PyArrowFile instance to write bytes to the file at the given location.
Args:
location (str): A URI or a path to a local file.
Returns:
PyArrowFile: A PyArrowFile instance for the given location.
"""
scheme, netloc, path = self.parse_location(location)
return PyArrowFile(
fs=self.fs_by_scheme(scheme, netloc),
location=location,
path=path,
buffer_size=int(self.properties.get(BUFFER_SIZE, ONE_MEGABYTE)),
)
def delete(self, location: Union[str, InputFile, OutputFile]) -> None:
"""Delete the file at the given location.
Args:
location (Union[str, InputFile, OutputFile]): The URI to the file--if an InputFile instance or an OutputFile instance is provided,
the location attribute for that instance is used as the location to delete.
Raises:
FileNotFoundError: When the file at the provided location does not exist.
PermissionError: If the file at the provided location cannot be accessed due to a permission error such as
an AWS error code 15.
"""
str_location = location.location if isinstance(location, (InputFile, OutputFile)) else location
scheme, netloc, path = self.parse_location(str_location)
fs = self.fs_by_scheme(scheme, netloc)
try:
fs.delete_file(path)
except FileNotFoundError:
raise
except PermissionError:
raise
except OSError as e:
if e.errno == 2 or "Path does not exist" in str(e):
raise FileNotFoundError(f"Cannot delete file, does not exist: {location}") from e
elif e.errno == 13 or "AWS Error [code 15]" in str(e):
raise PermissionError(f"Cannot delete file, access denied: {location}") from e
raise # pragma: no cover - If some other kind of OSError, raise the raw error
def schema_to_pyarrow(schema: Union[Schema, IcebergType], metadata: Dict[bytes, bytes] = EMPTY_DICT) -> pa.schema:
return visit(schema, _ConvertToArrowSchema(metadata))
class _ConvertToArrowSchema(SchemaVisitorPerPrimitiveType[pa.DataType]):
_metadata: Dict[bytes, bytes]
def __init__(self, metadata: Dict[bytes, bytes] = EMPTY_DICT) -> None:
self._metadata = metadata
def schema(self, _: Schema, struct_result: pa.StructType) -> pa.schema:
return pa.schema(list(struct_result), metadata=self._metadata)
def struct(self, _: StructType, field_results: List[pa.DataType]) -> pa.DataType:
return pa.struct(field_results)
def field(self, field: NestedField, field_result: pa.DataType) -> pa.Field:
return pa.field(
name=field.name,
type=field_result,
nullable=field.optional,
metadata={PYARROW_FIELD_DOC_KEY: field.doc, PYARROW_PARQUET_FIELD_ID_KEY: str(field.field_id)}
if field.doc
else {PYARROW_PARQUET_FIELD_ID_KEY: str(field.field_id)},
)
def list(self, list_type: ListType, element_result: pa.DataType) -> pa.DataType:
element_field = self.field(list_type.element_field, element_result)
return pa.list_(value_type=element_field)
def map(self, map_type: MapType, key_result: pa.DataType, value_result: pa.DataType) -> pa.DataType:
key_field = self.field(map_type.key_field, key_result)
value_field = self.field(map_type.value_field, value_result)
return pa.map_(key_type=key_field, item_type=value_field)
def visit_fixed(self, fixed_type: FixedType) -> pa.DataType:
return pa.binary(len(fixed_type))
def visit_decimal(self, decimal_type: DecimalType) -> pa.DataType:
return pa.decimal128(decimal_type.precision, decimal_type.scale)
def visit_boolean(self, _: BooleanType) -> pa.DataType:
return pa.bool_()
def visit_integer(self, _: IntegerType) -> pa.DataType:
return pa.int32()
def visit_long(self, _: LongType) -> pa.DataType:
return pa.int64()
def visit_float(self, _: FloatType) -> pa.DataType:
# 32-bit IEEE 754 floating point
return pa.float32()
def visit_double(self, _: DoubleType) -> pa.DataType:
# 64-bit IEEE 754 floating point
return pa.float64()
def visit_date(self, _: DateType) -> pa.DataType:
# Date encoded as an int
return pa.date32()
def visit_time(self, _: TimeType) -> pa.DataType:
return pa.time64("us")
def visit_timestamp(self, _: TimestampType) -> pa.DataType:
return pa.timestamp(unit="us")
def visit_timestamptz(self, _: TimestamptzType) -> pa.DataType:
return pa.timestamp(unit="us", tz="UTC")
def visit_string(self, _: StringType) -> pa.DataType:
return pa.string()
def visit_uuid(self, _: UUIDType) -> pa.DataType:
return pa.binary(16)
def visit_binary(self, _: BinaryType) -> pa.DataType:
return pa.large_binary()
def _convert_scalar(value: Any, iceberg_type: IcebergType) -> pa.scalar:
if not isinstance(iceberg_type, PrimitiveType):
raise ValueError(f"Expected primitive type, got: {iceberg_type}")
return pa.scalar(value=value, type=schema_to_pyarrow(iceberg_type))
class _ConvertToArrowExpression(BoundBooleanExpressionVisitor[pc.Expression]):
def visit_in(self, term: BoundTerm[pc.Expression], literals: Set[Any]) -> pc.Expression:
pyarrow_literals = pa.array(literals, type=schema_to_pyarrow(term.ref().field.field_type))
return pc.field(term.ref().field.name).isin(pyarrow_literals)
def visit_not_in(self, term: BoundTerm[pc.Expression], literals: Set[Any]) -> pc.Expression:
pyarrow_literals = pa.array(literals, type=schema_to_pyarrow(term.ref().field.field_type))
return ~pc.field(term.ref().field.name).isin(pyarrow_literals)
def visit_is_nan(self, term: BoundTerm[Any]) -> pc.Expression:
ref = pc.field(term.ref().field.name)
return pc.is_nan(ref)
def visit_not_nan(self, term: BoundTerm[Any]) -> pc.Expression:
ref = pc.field(term.ref().field.name)
return ~pc.is_nan(ref)
def visit_is_null(self, term: BoundTerm[Any]) -> pc.Expression:
return pc.field(term.ref().field.name).is_null(nan_is_null=False)
def visit_not_null(self, term: BoundTerm[Any]) -> pc.Expression:
return pc.field(term.ref().field.name).is_valid()
def visit_equal(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return pc.field(term.ref().field.name) == _convert_scalar(literal.value, term.ref().field.field_type)
def visit_not_equal(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return pc.field(term.ref().field.name) != _convert_scalar(literal.value, term.ref().field.field_type)
def visit_greater_than_or_equal(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return pc.field(term.ref().field.name) >= _convert_scalar(literal.value, term.ref().field.field_type)
def visit_greater_than(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return pc.field(term.ref().field.name) > _convert_scalar(literal.value, term.ref().field.field_type)
def visit_less_than(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return pc.field(term.ref().field.name) < _convert_scalar(literal.value, term.ref().field.field_type)
def visit_less_than_or_equal(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return pc.field(term.ref().field.name) <= _convert_scalar(literal.value, term.ref().field.field_type)
def visit_starts_with(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return pc.starts_with(pc.field(term.ref().field.name), literal.value)
def visit_not_starts_with(self, term: BoundTerm[Any], literal: Literal[Any]) -> pc.Expression:
return ~pc.starts_with(pc.field(term.ref().field.name), literal.value)
def visit_true(self) -> pc.Expression:
return pc.scalar(True)
def visit_false(self) -> pc.Expression:
return pc.scalar(False)
def visit_not(self, child_result: pc.Expression) -> pc.Expression:
return ~child_result
def visit_and(self, left_result: pc.Expression, right_result: pc.Expression) -> pc.Expression:
return left_result & right_result
def visit_or(self, left_result: pc.Expression, right_result: pc.Expression) -> pc.Expression:
return left_result | right_result
def expression_to_pyarrow(expr: BooleanExpression) -> pc.Expression:
return boolean_expression_visit(expr, _ConvertToArrowExpression())
@lru_cache
def _get_file_format(file_format: FileFormat, **kwargs: Dict[str, Any]) -> ds.FileFormat:
if file_format == FileFormat.PARQUET:
return ds.ParquetFileFormat(**kwargs)
else:
raise ValueError(f"Unsupported file format: {file_format}")
def _construct_fragment(fs: FileSystem, data_file: DataFile, file_format_kwargs: Dict[str, Any] = EMPTY_DICT) -> ds.Fragment:
_, _, path = PyArrowFileIO.parse_location(data_file.file_path)
return _get_file_format(data_file.file_format, **file_format_kwargs).make_fragment(path, fs)
def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedArray]:
delete_fragment = _construct_fragment(
fs, data_file, file_format_kwargs={"dictionary_columns": ("file_path",), "pre_buffer": True, "buffer_size": ONE_MEGABYTE}
)
table = ds.Scanner.from_fragment(fragment=delete_fragment).to_table()
table = table.unify_dictionaries()
return {
file.as_py(): table.filter(pc.field("file_path") == file).column("pos")
for file in table.column("file_path").chunks[0].dictionary
}
def _combine_positional_deletes(positional_deletes: List[pa.ChunkedArray], rows: int) -> pa.Array:
if len(positional_deletes) == 1:
all_chunks = positional_deletes[0]
else:
all_chunks = pa.chunked_array(itertools.chain(*[arr.chunks for arr in positional_deletes]))
return np.setdiff1d(np.arange(rows), all_chunks, assume_unique=False)
def pyarrow_to_schema(schema: pa.Schema, name_mapping: Optional[NameMapping] = None) -> Schema:
has_ids = visit_pyarrow(schema, _HasIds())
if has_ids:
visitor = _ConvertToIceberg()
elif name_mapping is not None:
visitor = _ConvertToIceberg(name_mapping=name_mapping)
else:
raise ValueError(
"Parquet file does not have field-ids and the Iceberg table does not have 'schema.name-mapping.default' defined"
)
return visit_pyarrow(schema, visitor)
def _pyarrow_to_schema_without_ids(schema: pa.Schema) -> Schema:
return visit_pyarrow(schema, _ConvertToIcebergWithoutIDs())
@singledispatch
def visit_pyarrow(obj: Union[pa.DataType, pa.Schema], visitor: PyArrowSchemaVisitor[T]) -> T:
"""Apply a pyarrow schema visitor to any point within a schema.
The function traverses the schema in post-order fashion.
Args:
obj (Union[pa.DataType, pa.Schema]): An instance of a Schema or an IcebergType.
visitor (PyArrowSchemaVisitor[T]): An instance of an implementation of the generic PyarrowSchemaVisitor base class.
Raises:
NotImplementedError: If attempting to visit an unrecognized object type.
"""
raise NotImplementedError(f"Cannot visit non-type: {obj}")
@visit_pyarrow.register(pa.Schema)
def _(obj: pa.Schema, visitor: PyArrowSchemaVisitor[T]) -> T:
return visitor.schema(obj, visit_pyarrow(pa.struct(obj), visitor))
@visit_pyarrow.register(pa.StructType)
def _(obj: pa.StructType, visitor: PyArrowSchemaVisitor[T]) -> T:
results = []
for field in obj:
visitor.before_field(field)
result = visit_pyarrow(field.type, visitor)
results.append(visitor.field(field, result))
visitor.after_field(field)
return visitor.struct(obj, results)
@visit_pyarrow.register(pa.ListType)
@visit_pyarrow.register(pa.FixedSizeListType)
@visit_pyarrow.register(pa.LargeListType)
def _(obj: Union[pa.ListType, pa.LargeListType, pa.FixedSizeListType], visitor: PyArrowSchemaVisitor[T]) -> T:
visitor.before_list_element(obj.value_field)
result = visit_pyarrow(obj.value_type, visitor)
visitor.after_list_element(obj.value_field)
return visitor.list(obj, result)
@visit_pyarrow.register(pa.MapType)
def _(obj: pa.MapType, visitor: PyArrowSchemaVisitor[T]) -> T:
visitor.before_map_key(obj.key_field)
key_result = visit_pyarrow(obj.key_type, visitor)
visitor.after_map_key(obj.key_field)
visitor.before_map_value(obj.item_field)
value_result = visit_pyarrow(obj.item_type, visitor)
visitor.after_map_value(obj.item_field)
return visitor.map(obj, key_result, value_result)
@visit_pyarrow.register(pa.DataType)
def _(obj: pa.DataType, visitor: PyArrowSchemaVisitor[T]) -> T:
if pa.types.is_nested(obj):
raise TypeError(f"Expected primitive type, got: {type(obj)}")
return visitor.primitive(obj)
class PyArrowSchemaVisitor(Generic[T], ABC):
def before_field(self, field: pa.Field) -> None:
"""Override this method to perform an action immediately before visiting a field."""
def after_field(self, field: pa.Field) -> None:
"""Override this method to perform an action immediately after visiting a field."""
def before_list_element(self, element: pa.Field) -> None:
"""Override this method to perform an action immediately before visiting an element within a ListType."""
def after_list_element(self, element: pa.Field) -> None:
"""Override this method to perform an action immediately after visiting an element within a ListType."""
def before_map_key(self, key: pa.Field) -> None:
"""Override this method to perform an action immediately before visiting a key within a MapType."""
def after_map_key(self, key: pa.Field) -> None:
"""Override this method to perform an action immediately after visiting a key within a MapType."""
def before_map_value(self, value: pa.Field) -> None:
"""Override this method to perform an action immediately before visiting a value within a MapType."""
def after_map_value(self, value: pa.Field) -> None:
"""Override this method to perform an action immediately after visiting a value within a MapType."""
@abstractmethod
def schema(self, schema: pa.Schema, struct_result: T) -> T:
"""Visit a schema."""
@abstractmethod
def struct(self, struct: pa.StructType, field_results: List[T]) -> T:
"""Visit a struct."""
@abstractmethod
def field(self, field: pa.Field, field_result: T) -> T:
"""Visit a field."""
@abstractmethod
def list(self, list_type: pa.ListType, element_result: T) -> T:
"""Visit a list."""
@abstractmethod
def map(self, map_type: pa.MapType, key_result: T, value_result: T) -> T:
"""Visit a map."""
@abstractmethod
def primitive(self, primitive: pa.DataType) -> T:
"""Visit a primitive type."""
def _get_field_id(field: pa.Field) -> Optional[int]:
return (
int(field_id_str.decode())
if (field.metadata and (field_id_str := field.metadata.get(PYARROW_PARQUET_FIELD_ID_KEY)))
else None
)
class _HasIds(PyArrowSchemaVisitor[bool]):
def schema(self, schema: pa.Schema, struct_result: bool) -> bool:
return struct_result
def struct(self, struct: pa.StructType, field_results: List[bool]) -> bool:
return all(field_results)
def field(self, field: pa.Field, field_result: bool) -> bool:
return all([_get_field_id(field) is not None, field_result])
def list(self, list_type: pa.ListType, element_result: bool) -> bool:
element_field = list_type.value_field
element_id = _get_field_id(element_field)
return element_result and element_id is not None
def map(self, map_type: pa.MapType, key_result: bool, value_result: bool) -> bool:
key_field = map_type.key_field
key_id = _get_field_id(key_field)
value_field = map_type.item_field
value_id = _get_field_id(value_field)
return all([key_id is not None, value_id is not None, key_result, value_result])
def primitive(self, primitive: pa.DataType) -> bool:
return True
class _ConvertToIceberg(PyArrowSchemaVisitor[Union[IcebergType, Schema]]):
"""Converts PyArrowSchema to Iceberg Schema. Applies the IDs from name_mapping if provided."""
_field_names: List[str]
_name_mapping: Optional[NameMapping]
def __init__(self, name_mapping: Optional[NameMapping] = None) -> None:
self._field_names = []
self._name_mapping = name_mapping
def _field_id(self, field: pa.Field) -> int:
if self._name_mapping:
return self._name_mapping.find(*self._field_names).field_id
elif (field_id := _get_field_id(field)) is not None:
return field_id
else:
raise ValueError(f"Cannot convert {field} to Iceberg Field as field_id is empty.")
def schema(self, schema: pa.Schema, struct_result: StructType) -> Schema:
return Schema(*struct_result.fields)
def struct(self, struct: pa.StructType, field_results: List[NestedField]) -> StructType:
return StructType(*field_results)
def field(self, field: pa.Field, field_result: IcebergType) -> NestedField:
field_id = self._field_id(field)
field_doc = doc_str.decode() if (field.metadata and (doc_str := field.metadata.get(PYARROW_FIELD_DOC_KEY))) else None
field_type = field_result
return NestedField(field_id, field.name, field_type, required=not field.nullable, doc=field_doc)
def list(self, list_type: pa.ListType, element_result: IcebergType) -> ListType:
element_field = list_type.value_field
self._field_names.append(LIST_ELEMENT_NAME)
element_id = self._field_id(element_field)
self._field_names.pop()
return ListType(element_id, element_result, element_required=not element_field.nullable)
def map(self, map_type: pa.MapType, key_result: IcebergType, value_result: IcebergType) -> MapType:
key_field = map_type.key_field
self._field_names.append(MAP_KEY_NAME)
key_id = self._field_id(key_field)
self._field_names.pop()
value_field = map_type.item_field
self._field_names.append(MAP_VALUE_NAME)
value_id = self._field_id(value_field)
self._field_names.pop()
return MapType(key_id, key_result, value_id, value_result, value_required=not value_field.nullable)
def primitive(self, primitive: pa.DataType) -> PrimitiveType:
if pa.types.is_boolean(primitive):
return BooleanType()
elif pa.types.is_integer(primitive):
width = primitive.bit_width
if width <= 32:
return IntegerType()
elif width <= 64:
return LongType()
else:
# Does not exist (yet)
raise TypeError(f"Unsupported integer type: {primitive}")
elif pa.types.is_float32(primitive):
return FloatType()
elif pa.types.is_float64(primitive):
return DoubleType()
elif isinstance(primitive, pa.Decimal128Type):
primitive = cast(pa.Decimal128Type, primitive)
return DecimalType(primitive.precision, primitive.scale)
elif pa.types.is_string(primitive) or pa.types.is_large_string(primitive):
return StringType()
elif pa.types.is_date32(primitive):
return DateType()
elif isinstance(primitive, pa.Time64Type) and primitive.unit == "us":
return TimeType()
elif pa.types.is_timestamp(primitive):
primitive = cast(pa.TimestampType, primitive)
if primitive.unit == "us":
if primitive.tz == "UTC" or primitive.tz == "+00:00":
return TimestamptzType()
elif primitive.tz is None:
return TimestampType()
elif pa.types.is_binary(primitive) or pa.types.is_large_binary(primitive):
return BinaryType()
elif pa.types.is_fixed_size_binary(primitive):
primitive = cast(pa.FixedSizeBinaryType, primitive)
return FixedType(primitive.byte_width)
raise TypeError(f"Unsupported type: {primitive}")
def before_field(self, field: pa.Field) -> None:
self._field_names.append(field.name)
def after_field(self, field: pa.Field) -> None:
self._field_names.pop()
def before_list_element(self, element: pa.Field) -> None:
self._field_names.append(LIST_ELEMENT_NAME)
def after_list_element(self, element: pa.Field) -> None:
self._field_names.pop()
def before_map_key(self, key: pa.Field) -> None:
self._field_names.append(MAP_KEY_NAME)
def after_map_key(self, element: pa.Field) -> None:
self._field_names.pop()
def before_map_value(self, value: pa.Field) -> None:
self._field_names.append(MAP_VALUE_NAME)
def after_map_value(self, element: pa.Field) -> None:
self._field_names.pop()
class _ConvertToIcebergWithoutIDs(_ConvertToIceberg):
"""
Converts PyArrowSchema to Iceberg Schema with all -1 ids.
The schema generated through this visitor should always be
used in conjunction with `new_table_metadata` function to
assign new field ids in order. This is currently used only
when creating an Iceberg Schema from a PyArrow schema when
creating a new Iceberg table.
"""
def _field_id(self, field: pa.Field) -> int:
return -1
def _task_to_table(
fs: FileSystem,
task: FileScanTask,
bound_row_filter: BooleanExpression,
projected_schema: Schema,
projected_field_ids: Set[int],
positional_deletes: Optional[List[ChunkedArray]],
case_sensitive: bool,
row_counts: List[int],
limit: Optional[int] = None,
name_mapping: Optional[NameMapping] = None,
) -> Optional[pa.Table]:
if limit and sum(row_counts) >= limit:
return None
_, _, path = PyArrowFileIO.parse_location(task.file.file_path)
arrow_format = ds.ParquetFileFormat(pre_buffer=True, buffer_size=(ONE_MEGABYTE * 8))
with fs.open_input_file(path) as fin:
fragment = arrow_format.make_fragment(fin)
physical_schema = fragment.physical_schema
schema_raw = None
if metadata := physical_schema.metadata:
schema_raw = metadata.get(ICEBERG_SCHEMA)
file_schema = (
Schema.model_validate_json(schema_raw) if schema_raw is not None else pyarrow_to_schema(physical_schema, name_mapping)
)
pyarrow_filter = None
if bound_row_filter is not AlwaysTrue():
translated_row_filter = translate_column_names(bound_row_filter, file_schema, case_sensitive=case_sensitive)
bound_file_filter = bind(file_schema, translated_row_filter, case_sensitive=case_sensitive)
pyarrow_filter = expression_to_pyarrow(bound_file_filter)
file_project_schema = sanitize_column_names(prune_columns(file_schema, projected_field_ids, select_full_types=False))
if file_schema is None:
raise ValueError(f"Missing Iceberg schema in Metadata for file: {path}")
fragment_scanner = ds.Scanner.from_fragment(
fragment=fragment,
schema=physical_schema,
# This will push down the query to Arrow.
# But in case there are positional deletes, we have to apply them first
filter=pyarrow_filter if not positional_deletes else None,
columns=[col.name for col in file_project_schema.columns],
)
if positional_deletes:
# Create the mask of indices that we're interested in
indices = _combine_positional_deletes(positional_deletes, fragment.count_rows())
if limit:
if pyarrow_filter is not None:
# In case of the filter, we don't exactly know how many rows
# we need to fetch upfront, can be optimized in the future:
# https://github.com/apache/arrow/issues/35301
arrow_table = fragment_scanner.take(indices)
arrow_table = arrow_table.filter(pyarrow_filter)
arrow_table = arrow_table.slice(0, limit)
else: