-
Notifications
You must be signed in to change notification settings - Fork 464
Expand file tree
/
Copy path__init__.py
More file actions
2319 lines (1873 loc) · 91.4 KB
/
__init__.py
File metadata and controls
2319 lines (1873 loc) · 91.4 KB
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 __future__ import annotations
import itertools
import os
import uuid
import warnings
from abc import ABC, abstractmethod
from collections.abc import Callable, Iterable, Iterator
from dataclasses import dataclass
from functools import cached_property
from itertools import chain
from types import TracebackType
from typing import TYPE_CHECKING, Any, TypeVar
from pydantic import Field
import pyiceberg.expressions.parser as parser
from pyiceberg.expressions import AlwaysFalse, AlwaysTrue, And, BooleanExpression, EqualTo, IsNull, Or, Reference
from pyiceberg.expressions.visitors import (
ResidualEvaluator,
_InclusiveMetricsEvaluator,
bind,
expression_evaluator,
inclusive_projection,
manifest_evaluator,
)
from pyiceberg.io import FileIO, load_file_io
from pyiceberg.manifest import DataFile, DataFileContent, ManifestContent, ManifestEntry, ManifestFile
from pyiceberg.partitioning import PARTITION_FIELD_ID_START, UNPARTITIONED_PARTITION_SPEC, PartitionKey, PartitionSpec
from pyiceberg.schema import Schema
from pyiceberg.table.delete_file_index import DeleteFileIndex
from pyiceberg.table.inspect import InspectTable
from pyiceberg.table.locations import LocationProvider, load_location_provider
from pyiceberg.table.maintenance import MaintenanceTable
from pyiceberg.table.metadata import INITIAL_SEQUENCE_NUMBER, TableMetadata
from pyiceberg.table.name_mapping import NameMapping
from pyiceberg.table.refs import MAIN_BRANCH, SnapshotRef
from pyiceberg.table.snapshots import Snapshot, SnapshotLogEntry
from pyiceberg.table.sorting import UNSORTED_SORT_ORDER, SortOrder
from pyiceberg.table.update import (
AddPartitionSpecUpdate,
AddSchemaUpdate,
AddSortOrderUpdate,
AssertCreate,
AssertRefSnapshotId,
AssertTableUUID,
AssignUUIDUpdate,
RemovePropertiesUpdate,
SetCurrentSchemaUpdate,
SetDefaultSortOrderUpdate,
SetDefaultSpecUpdate,
SetLocationUpdate,
SetPropertiesUpdate,
SetSnapshotRefUpdate,
TableRequirement,
TableUpdate,
UpdatesAndRequirements,
UpgradeFormatVersionUpdate,
update_table_metadata,
)
from pyiceberg.table.update.schema import UpdateSchema
from pyiceberg.table.update.snapshot import ManageSnapshots, UpdateSnapshot, _FastAppendFiles
from pyiceberg.table.update.sorting import UpdateSortOrder
from pyiceberg.table.update.spec import UpdateSpec
from pyiceberg.table.update.statistics import UpdateStatistics
from pyiceberg.transforms import IdentityTransform
from pyiceberg.typedef import (
EMPTY_DICT,
IcebergBaseModel,
IcebergRootModel,
Identifier,
KeyDefaultDict,
Properties,
Record,
TableVersion,
)
from pyiceberg.types import strtobool
from pyiceberg.utils.concurrent import ExecutorFactory
from pyiceberg.utils.config import Config
from pyiceberg.utils.properties import property_as_bool
if TYPE_CHECKING:
import bodo.pandas as bd
import daft
import pandas as pd
import polars as pl
import pyarrow as pa
import ray
from duckdb import DuckDBPyConnection
from pyiceberg_core.datafusion import IcebergDataFusionTable
from pyiceberg.catalog import Catalog
from pyiceberg.catalog.rest.scan_planning import RESTContentFile, RESTDeleteFile, RESTFileScanTask
ALWAYS_TRUE = AlwaysTrue()
DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE = "downcast-ns-timestamp-to-us-on-write"
@dataclass()
class UpsertResult:
"""Summary the upsert operation."""
rows_updated: int = 0
rows_inserted: int = 0
class TableProperties:
PARQUET_ROW_GROUP_SIZE_BYTES = "write.parquet.row-group-size-bytes"
PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT = 128 * 1024 * 1024 # 128 MB
PARQUET_ROW_GROUP_LIMIT = "write.parquet.row-group-limit"
PARQUET_ROW_GROUP_LIMIT_DEFAULT = 1048576
PARQUET_PAGE_SIZE_BYTES = "write.parquet.page-size-bytes"
PARQUET_PAGE_SIZE_BYTES_DEFAULT = 1024 * 1024 # 1 MB
PARQUET_PAGE_ROW_LIMIT = "write.parquet.page-row-limit"
PARQUET_PAGE_ROW_LIMIT_DEFAULT = 20000
PARQUET_DICT_SIZE_BYTES = "write.parquet.dict-size-bytes"
PARQUET_DICT_SIZE_BYTES_DEFAULT = 2 * 1024 * 1024 # 2 MB
PARQUET_COMPRESSION = "write.parquet.compression-codec"
PARQUET_COMPRESSION_DEFAULT = "zstd"
PARQUET_COMPRESSION_LEVEL = "write.parquet.compression-level"
PARQUET_COMPRESSION_LEVEL_DEFAULT = None
PARQUET_BLOOM_FILTER_MAX_BYTES = "write.parquet.bloom-filter-max-bytes"
PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT = 1024 * 1024
PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX = "write.parquet.bloom-filter-enabled.column"
WRITE_TARGET_FILE_SIZE_BYTES = "write.target-file-size-bytes"
WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT = 512 * 1024 * 1024 # 512 MB
WRITE_AVRO_COMPRESSION = "write.avro.compression-codec"
WRITE_AVRO_COMPRESSION_DEFAULT = "gzip"
DEFAULT_WRITE_METRICS_MODE = "write.metadata.metrics.default"
DEFAULT_WRITE_METRICS_MODE_DEFAULT = "truncate(16)"
METRICS_MODE_COLUMN_CONF_PREFIX = "write.metadata.metrics.column"
WRITE_PARTITION_SUMMARY_LIMIT = "write.summary.partition-limit"
WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT = 0
WRITE_PY_LOCATION_PROVIDER_IMPL = "write.py-location-provider.impl"
OBJECT_STORE_ENABLED = "write.object-storage.enabled"
OBJECT_STORE_ENABLED_DEFAULT = False
WRITE_OBJECT_STORE_PARTITIONED_PATHS = "write.object-storage.partitioned-paths"
WRITE_OBJECT_STORE_PARTITIONED_PATHS_DEFAULT = True
WRITE_DATA_PATH = "write.data.path"
WRITE_FILE_FORMAT = "write.format.default"
WRITE_FILE_FORMAT_DEFAULT = "parquet"
WRITE_METADATA_PATH = "write.metadata.path"
DELETE_MODE = "write.delete.mode"
DELETE_MODE_COPY_ON_WRITE = "copy-on-write"
DELETE_MODE_MERGE_ON_READ = "merge-on-read"
DELETE_MODE_DEFAULT = DELETE_MODE_COPY_ON_WRITE
DEFAULT_NAME_MAPPING = "schema.name-mapping.default"
FORMAT_VERSION = "format-version"
DEFAULT_FORMAT_VERSION: TableVersion = 2
MANIFEST_TARGET_SIZE_BYTES = "commit.manifest.target-size-bytes"
MANIFEST_TARGET_SIZE_BYTES_DEFAULT = 8 * 1024 * 1024 # 8 MB
MANIFEST_MIN_MERGE_COUNT = "commit.manifest.min-count-to-merge"
MANIFEST_MIN_MERGE_COUNT_DEFAULT = 100
MANIFEST_MERGE_ENABLED = "commit.manifest-merge.enabled"
MANIFEST_MERGE_ENABLED_DEFAULT = False
METADATA_PREVIOUS_VERSIONS_MAX = "write.metadata.previous-versions-max"
METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT = 100
METADATA_DELETE_AFTER_COMMIT_ENABLED = "write.metadata.delete-after-commit.enabled"
METADATA_DELETE_AFTER_COMMIT_ENABLED_DEFAULT = False
MAX_SNAPSHOT_AGE_MS = "history.expire.max-snapshot-age-ms"
MAX_SNAPSHOT_AGE_MS_DEFAULT = 5 * 24 * 60 * 60 * 1000 # 5 days
MIN_SNAPSHOTS_TO_KEEP = "history.expire.min-snapshots-to-keep"
MIN_SNAPSHOTS_TO_KEEP_DEFAULT = 1
class Transaction:
_table: Table
_autocommit: bool
_updates: tuple[TableUpdate, ...]
_requirements: tuple[TableRequirement, ...]
def __init__(self, table: Table, autocommit: bool = False):
"""Open a transaction to stage and commit changes to a table.
Args:
table: The table that will be altered.
autocommit: Option to automatically commit the changes when they are staged.
"""
self._table = table
self._autocommit = autocommit
self._updates = ()
self._requirements = ()
@property
def table_metadata(self) -> TableMetadata:
return update_table_metadata(self._table.metadata, self._updates)
def __enter__(self) -> Transaction:
"""Start a transaction to update the table."""
return self
def __exit__(self, exctype: type[BaseException] | None, excinst: BaseException | None, exctb: TracebackType | None) -> None:
"""Close and commit the transaction if no exceptions have been raised."""
if exctype is None and excinst is None and exctb is None:
self.commit_transaction()
def _stage(
self,
updates: tuple[TableUpdate, ...],
requirements: tuple[TableRequirement, ...] = (),
) -> Transaction:
"""Stage updates to the transaction state without committing to the catalog.
Args:
updates: The updates to stage.
requirements: The requirements that must be met.
Returns:
This transaction for method chaining.
"""
for requirement in requirements:
requirement.validate(self.table_metadata)
self._updates += updates
# For the requirements, it does not make sense to add a requirement more than once
# For example, you cannot assert that the current schema has two different IDs
existing_requirements = {type(requirement) for requirement in self._requirements}
for new_requirement in requirements:
if type(new_requirement) not in existing_requirements:
self._requirements = self._requirements + (new_requirement,)
return self
def _apply(
self,
updates: tuple[TableUpdate, ...],
requirements: tuple[TableRequirement, ...] = (),
) -> Transaction:
"""Check if the requirements are met, and applies the updates to the metadata."""
self._stage(updates, requirements)
if self._autocommit:
self.commit_transaction()
return self
def _scan(self, row_filter: str | BooleanExpression = ALWAYS_TRUE, case_sensitive: bool = True) -> DataScan:
"""Minimal data scan of the table with the current state of the transaction."""
return DataScan(
table_metadata=self.table_metadata, io=self._table.io, row_filter=row_filter, case_sensitive=case_sensitive
)
def upgrade_table_version(self, format_version: TableVersion) -> Transaction:
"""Set the table to a certain version.
Args:
format_version: The newly set version.
Returns:
The alter table builder.
"""
if format_version not in {1, 2}:
raise ValueError(f"Unsupported table format version: {format_version}")
if format_version < self.table_metadata.format_version:
raise ValueError(f"Cannot downgrade v{self.table_metadata.format_version} table to v{format_version}")
if format_version > self.table_metadata.format_version:
return self._apply((UpgradeFormatVersionUpdate(format_version=format_version),))
return self
def set_properties(self, properties: Properties = EMPTY_DICT, **kwargs: Any) -> Transaction:
"""Set properties.
When a property is already set, it will be overwritten.
Args:
properties: The properties set on the table.
kwargs: properties can also be pass as kwargs.
Returns:
The alter table builder.
"""
if properties and kwargs:
raise ValueError("Cannot pass both properties and kwargs")
updates = properties or kwargs
return self._apply((SetPropertiesUpdate(updates=updates),))
def _set_ref_snapshot(
self,
snapshot_id: int,
ref_name: str,
type: str,
max_ref_age_ms: int | None = None,
max_snapshot_age_ms: int | None = None,
min_snapshots_to_keep: int | None = None,
) -> UpdatesAndRequirements:
"""Update a ref to a snapshot.
Returns:
The updates and requirements for the set-snapshot-ref staged
"""
updates = (
SetSnapshotRefUpdate(
snapshot_id=snapshot_id,
ref_name=ref_name,
type=type,
max_ref_age_ms=max_ref_age_ms,
max_snapshot_age_ms=max_snapshot_age_ms,
min_snapshots_to_keep=min_snapshots_to_keep,
),
)
requirements = (
AssertRefSnapshotId(
snapshot_id=self.table_metadata.refs[ref_name].snapshot_id if ref_name in self.table_metadata.refs else None,
ref=ref_name,
),
)
return updates, requirements
def _build_partition_predicate(
self, partition_records: set[Record], spec: PartitionSpec, schema: Schema
) -> BooleanExpression:
"""Build a filter predicate matching any of the input partition records.
Args:
partition_records: A set of partition records to match
spec: An optional partition spec, if none then defaults to current
schema: An optional schema, if none then defaults to current
Returns:
A predicate matching any of the input partition records.
"""
partition_fields = [schema.find_field(field.source_id).name for field in spec.fields]
expr: BooleanExpression = AlwaysFalse()
for partition_record in partition_records:
match_partition_expression: BooleanExpression = AlwaysTrue()
for pos, partition_field in enumerate(partition_fields):
predicate = (
EqualTo(Reference(partition_field), partition_record[pos])
if partition_record[pos] is not None
else IsNull(Reference(partition_field))
)
match_partition_expression = And(match_partition_expression, predicate)
expr = Or(expr, match_partition_expression)
return expr
def _append_snapshot_producer(
self, snapshot_properties: dict[str, str], branch: str | None = MAIN_BRANCH
) -> _FastAppendFiles:
"""Determine the append type based on table properties.
Args:
snapshot_properties: Custom properties to be added to the snapshot summary
Returns:
Either a fast-append or a merge-append snapshot producer.
"""
manifest_merge_enabled = property_as_bool(
self.table_metadata.properties,
TableProperties.MANIFEST_MERGE_ENABLED,
TableProperties.MANIFEST_MERGE_ENABLED_DEFAULT,
)
update_snapshot = self.update_snapshot(snapshot_properties=snapshot_properties, branch=branch)
return update_snapshot.merge_append() if manifest_merge_enabled else update_snapshot.fast_append()
def update_schema(self, allow_incompatible_changes: bool = False, case_sensitive: bool = True) -> UpdateSchema:
"""Create a new UpdateSchema to alter the columns of this table.
Args:
allow_incompatible_changes: If changes are allowed that might break downstream consumers.
case_sensitive: If field names are case-sensitive.
Returns:
A new UpdateSchema.
"""
return UpdateSchema(
self,
allow_incompatible_changes=allow_incompatible_changes,
case_sensitive=case_sensitive,
name_mapping=self.table_metadata.name_mapping(),
)
def update_sort_order(self, case_sensitive: bool = True) -> UpdateSortOrder:
"""Create a new UpdateSortOrder to update the sort order of this table.
Args:
case_sensitive: If field names are case-sensitive.
Returns:
A new UpdateSortOrder.
"""
return UpdateSortOrder(
self,
case_sensitive=case_sensitive,
)
def update_snapshot(
self, snapshot_properties: dict[str, str] = EMPTY_DICT, branch: str | None = MAIN_BRANCH
) -> UpdateSnapshot:
"""Create a new UpdateSnapshot to produce a new snapshot for the table.
Returns:
A new UpdateSnapshot
"""
return UpdateSnapshot(self, io=self._table.io, branch=branch, snapshot_properties=snapshot_properties)
def update_statistics(self) -> UpdateStatistics:
"""
Create a new UpdateStatistics to update the statistics of the table.
Returns:
A new UpdateStatistics
"""
return UpdateStatistics(transaction=self)
def append(self, df: pa.Table, snapshot_properties: dict[str, str] = EMPTY_DICT, branch: str | None = MAIN_BRANCH) -> None:
"""
Shorthand API for appending a PyArrow table to a table transaction.
Args:
df: The Arrow dataframe that will be appended to overwrite the table
snapshot_properties: Custom properties to be added to the snapshot summary
branch: Branch Reference to run the append operation
"""
try:
import pyarrow as pa
except ModuleNotFoundError as e:
raise ModuleNotFoundError("For writes PyArrow needs to be installed") from e
from pyiceberg.io.pyarrow import _check_pyarrow_schema_compatible, _dataframe_to_data_files
if not isinstance(df, pa.Table):
raise ValueError(f"Expected PyArrow table, got: {df}")
downcast_ns_timestamp_to_us = Config().get_bool(DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE) or False
_check_pyarrow_schema_compatible(
self.table_metadata.schema(),
provided_schema=df.schema,
downcast_ns_timestamp_to_us=downcast_ns_timestamp_to_us,
format_version=self.table_metadata.format_version,
)
with self._append_snapshot_producer(snapshot_properties, branch=branch) as append_files:
# skip writing data files if the dataframe is empty
if df.shape[0] > 0:
data_files = list(
_dataframe_to_data_files(
table_metadata=self.table_metadata, write_uuid=append_files.commit_uuid, df=df, io=self._table.io
)
)
for data_file in data_files:
append_files.append_data_file(data_file)
def dynamic_partition_overwrite(
self, df: pa.Table, snapshot_properties: dict[str, str] = EMPTY_DICT, branch: str | None = MAIN_BRANCH
) -> None:
"""
Shorthand for overwriting existing partitions with a PyArrow table.
The function detects partition values in the provided arrow table using the current
partition spec, and deletes existing partitions matching these values. Finally, the
data in the table is appended to the table.
Args:
df: The Arrow dataframe that will be used to overwrite the table
snapshot_properties: Custom properties to be added to the snapshot summary
branch: Branch Reference to run the dynamic partition overwrite operation
"""
try:
import pyarrow as pa
except ModuleNotFoundError as e:
raise ModuleNotFoundError("For writes PyArrow needs to be installed") from e
from pyiceberg.io.pyarrow import _check_pyarrow_schema_compatible, _dataframe_to_data_files
if not isinstance(df, pa.Table):
raise ValueError(f"Expected PyArrow table, got: {df}")
if self.table_metadata.spec().is_unpartitioned():
raise ValueError("Cannot apply dynamic overwrite on an unpartitioned table.")
for field in self.table_metadata.spec().fields:
if not isinstance(field.transform, IdentityTransform):
raise ValueError(
f"For now dynamic overwrite does not support a table with non-identity-transform field "
f"in the latest partition spec: {field}"
)
downcast_ns_timestamp_to_us = Config().get_bool(DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE) or False
_check_pyarrow_schema_compatible(
self.table_metadata.schema(),
provided_schema=df.schema,
downcast_ns_timestamp_to_us=downcast_ns_timestamp_to_us,
format_version=self.table_metadata.format_version,
)
# If dataframe does not have data, there is no need to overwrite
if df.shape[0] == 0:
return
append_snapshot_commit_uuid = uuid.uuid4()
data_files: list[DataFile] = list(
_dataframe_to_data_files(
table_metadata=self._table.metadata, write_uuid=append_snapshot_commit_uuid, df=df, io=self._table.io
)
)
partitions_to_overwrite = {data_file.partition for data_file in data_files}
delete_filter = self._build_partition_predicate(
partition_records=partitions_to_overwrite, spec=self.table_metadata.spec(), schema=self.table_metadata.schema()
)
self.delete(delete_filter=delete_filter, snapshot_properties=snapshot_properties, branch=branch)
with self._append_snapshot_producer(snapshot_properties, branch=branch) as append_files:
append_files.commit_uuid = append_snapshot_commit_uuid
for data_file in data_files:
append_files.append_data_file(data_file)
def overwrite(
self,
df: pa.Table,
overwrite_filter: BooleanExpression | str = ALWAYS_TRUE,
snapshot_properties: dict[str, str] = EMPTY_DICT,
case_sensitive: bool = True,
branch: str | None = MAIN_BRANCH,
) -> None:
"""
Shorthand for adding a table overwrite with a PyArrow table to the transaction.
An overwrite may produce zero or more snapshots based on the operation:
- DELETE: In case existing Parquet files can be dropped completely.
- OVERWRITE: In case existing Parquet files need to be rewritten to drop rows that match the overwrite filter.
- APPEND: In case new data is being inserted into the table.
Args:
df: The Arrow dataframe that will be used to overwrite the table
overwrite_filter: ALWAYS_TRUE when you overwrite all the data,
or a boolean expression in case of a partial overwrite
snapshot_properties: Custom properties to be added to the snapshot summary
case_sensitive: A bool determine if the provided `overwrite_filter` is case-sensitive
branch: Branch Reference to run the overwrite operation
"""
try:
import pyarrow as pa
except ModuleNotFoundError as e:
raise ModuleNotFoundError("For writes PyArrow needs to be installed") from e
from pyiceberg.io.pyarrow import _check_pyarrow_schema_compatible, _dataframe_to_data_files
if not isinstance(df, pa.Table):
raise ValueError(f"Expected PyArrow table, got: {df}")
downcast_ns_timestamp_to_us = Config().get_bool(DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE) or False
_check_pyarrow_schema_compatible(
self.table_metadata.schema(),
provided_schema=df.schema,
downcast_ns_timestamp_to_us=downcast_ns_timestamp_to_us,
format_version=self.table_metadata.format_version,
)
if overwrite_filter != AlwaysFalse():
# Only delete when the filter is != AlwaysFalse
self.delete(
delete_filter=overwrite_filter,
case_sensitive=case_sensitive,
snapshot_properties=snapshot_properties,
branch=branch,
)
with self._append_snapshot_producer(snapshot_properties, branch=branch) as append_files:
# skip writing data files if the dataframe is empty
if df.shape[0] > 0:
data_files = _dataframe_to_data_files(
table_metadata=self.table_metadata, write_uuid=append_files.commit_uuid, df=df, io=self._table.io
)
for data_file in data_files:
append_files.append_data_file(data_file)
def replace(
self,
df: pa.Table,
files_to_delete: Iterable[DataFile],
snapshot_properties: dict[str, str] = EMPTY_DICT,
branch: str | None = MAIN_BRANCH,
) -> None:
"""
Shorthand for replacing existing files.
A replace will produce a REPLACE snapshot that will ignore existing
files and replace them with the new files.
Args:
df: The Arrow dataframe that will be used to generate the new data files
files_to_delete: The files to delete
snapshot_properties: Custom properties to be added to the snapshot summary
branch: Branch Reference to run the replace operation
"""
try:
import pyarrow as pa
except ModuleNotFoundError as e:
raise ModuleNotFoundError("For writes PyArrow needs to be installed") from e
from pyiceberg.io.pyarrow import _check_pyarrow_schema_compatible, _dataframe_to_data_files
if not isinstance(df, pa.Table):
raise ValueError(f"Expected PyArrow table, got: {df}")
downcast_ns_timestamp_to_us = Config().get_bool(DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE) or False
_check_pyarrow_schema_compatible(
self.table_metadata.schema(),
provided_schema=df.schema,
downcast_ns_timestamp_to_us=downcast_ns_timestamp_to_us,
format_version=self.table_metadata.format_version,
)
with self.update_snapshot(snapshot_properties=snapshot_properties, branch=branch).replace() as replace_snapshot:
for file_to_delete in files_to_delete:
replace_snapshot.delete_data_file(file_to_delete)
if df.shape[0] > 0:
data_files = _dataframe_to_data_files(
table_metadata=self.table_metadata, write_uuid=replace_snapshot.commit_uuid, df=df, io=self._table.io
)
for data_file in data_files:
replace_snapshot.append_data_file(data_file)
def delete(
self,
delete_filter: str | BooleanExpression,
snapshot_properties: dict[str, str] = EMPTY_DICT,
case_sensitive: bool = True,
branch: str | None = MAIN_BRANCH,
) -> None:
"""
Shorthand for deleting record from a table.
A delete may produce zero or more snapshots based on the operation:
- DELETE: In case existing Parquet files can be dropped completely.
- OVERWRITE: In case existing Parquet files need to be rewritten to drop rows that match the delete filter.
Args:
delete_filter: A boolean expression to delete rows from a table
snapshot_properties: Custom properties to be added to the snapshot summary
case_sensitive: A bool determine if the provided `delete_filter` is case-sensitive
branch: Branch Reference to run the delete operation
"""
from pyiceberg.io.pyarrow import ArrowScan, _dataframe_to_data_files, _expression_to_complementary_pyarrow
if (
self.table_metadata.properties.get(TableProperties.DELETE_MODE, TableProperties.DELETE_MODE_DEFAULT)
== TableProperties.DELETE_MODE_MERGE_ON_READ
):
warnings.warn("Merge on read is not yet supported, falling back to copy-on-write", stacklevel=2)
if isinstance(delete_filter, str):
delete_filter = _parse_row_filter(delete_filter)
with self.update_snapshot(snapshot_properties=snapshot_properties, branch=branch).delete() as delete_snapshot:
delete_snapshot.delete_by_predicate(delete_filter, case_sensitive)
# Check if there are any files that require an actual rewrite of a data file
if delete_snapshot.rewrites_needed is True:
bound_delete_filter = bind(self.table_metadata.schema(), delete_filter, case_sensitive)
preserve_row_filter = _expression_to_complementary_pyarrow(bound_delete_filter, self.table_metadata.schema())
file_scan = self._scan(row_filter=delete_filter, case_sensitive=case_sensitive)
if branch is not None:
file_scan = file_scan.use_ref(branch)
files = file_scan.plan_files()
commit_uuid = uuid.uuid4()
counter = itertools.count(0)
replaced_files: list[tuple[DataFile, list[DataFile]]] = []
# This will load the Parquet file into memory, including:
# - Filter out the rows based on the delete filter
# - Projecting it to the current schema
# - Applying the positional deletes if they are there
# When writing
# - Apply the latest partition-spec
# - And sort order when added
for original_file in files:
df = ArrowScan(
table_metadata=self.table_metadata,
io=self._table.io,
projected_schema=self.table_metadata.schema(),
row_filter=AlwaysTrue(),
).to_table(tasks=[original_file])
filtered_df = df.filter(preserve_row_filter)
# Only rewrite if there are records being deleted
if len(filtered_df) == 0:
replaced_files.append((original_file.file, []))
elif len(df) != len(filtered_df):
replaced_files.append(
(
original_file.file,
list(
_dataframe_to_data_files(
io=self._table.io,
df=filtered_df,
table_metadata=self.table_metadata,
write_uuid=commit_uuid,
counter=counter,
)
),
)
)
if len(replaced_files) > 0:
with self.update_snapshot(
snapshot_properties=snapshot_properties, branch=branch
).overwrite() as overwrite_snapshot:
overwrite_snapshot.commit_uuid = commit_uuid
for original_data_file, replaced_data_files in replaced_files:
overwrite_snapshot.delete_data_file(original_data_file)
for replaced_data_file in replaced_data_files:
overwrite_snapshot.append_data_file(replaced_data_file)
if not delete_snapshot.files_affected and not delete_snapshot.rewrites_needed:
warnings.warn("Delete operation did not match any records", stacklevel=2)
def upsert(
self,
df: pa.Table,
join_cols: list[str] | None = None,
when_matched_update_all: bool = True,
when_not_matched_insert_all: bool = True,
case_sensitive: bool = True,
branch: str | None = MAIN_BRANCH,
snapshot_properties: dict[str, str] = EMPTY_DICT,
) -> UpsertResult:
"""Shorthand API for performing an upsert to an iceberg table.
Args:
df: The input dataframe to upsert with the table's data.
join_cols: Columns to join on, if not provided, it will use the identifier-field-ids.
when_matched_update_all: Bool indicating to update rows that are matched but require an update
due to a value in a non-key column changing
when_not_matched_insert_all: Bool indicating new rows to be inserted that do not match any
existing rows in the table
case_sensitive: Bool indicating if the match should be case-sensitive
branch: Branch Reference to run the upsert operation
snapshot_properties: Custom properties to be added to the snapshot summary
To learn more about the identifier-field-ids: https://iceberg.apache.org/spec/#identifier-field-ids
Example Use Cases:
Case 1: Both Parameters = True (Full Upsert)
Existing row found → Update it
New row found → Insert it
Case 2: when_matched_update_all = False, when_not_matched_insert_all = True
Existing row found → Do nothing (no updates)
New row found → Insert it
Case 3: when_matched_update_all = True, when_not_matched_insert_all = False
Existing row found → Update it
New row found → Do nothing (no inserts)
Case 4: Both Parameters = False (No Merge Effect)
Existing row found → Do nothing
New row found → Do nothing
(Function effectively does nothing)
Returns:
An UpsertResult class (contains details of rows updated and inserted)
"""
try:
import pyarrow as pa # noqa: F401
except ModuleNotFoundError as e:
raise ModuleNotFoundError("For writes PyArrow needs to be installed") from e
from pyiceberg.io.pyarrow import expression_to_pyarrow
from pyiceberg.table import upsert_util
if join_cols is None:
join_cols = []
for field_id in self.table_metadata.schema().identifier_field_ids:
col = self.table_metadata.schema().find_column_name(field_id)
if col is not None:
join_cols.append(col)
else:
raise ValueError(f"Field-ID could not be found: {join_cols}")
if len(join_cols) == 0:
raise ValueError("Join columns could not be found, please set identifier-field-ids or pass in explicitly.")
if not when_matched_update_all and not when_not_matched_insert_all:
raise ValueError("no upsert options selected...exiting")
if upsert_util.has_duplicate_rows(df, join_cols):
raise ValueError("Duplicate rows found in source dataset based on the key columns. No upsert executed")
from pyiceberg.io.pyarrow import _check_pyarrow_schema_compatible
downcast_ns_timestamp_to_us = Config().get_bool(DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE) or False
_check_pyarrow_schema_compatible(
self.table_metadata.schema(),
provided_schema=df.schema,
downcast_ns_timestamp_to_us=downcast_ns_timestamp_to_us,
format_version=self.table_metadata.format_version,
)
# get list of rows that exist so we don't have to load the entire target table
matched_predicate = upsert_util.create_match_filter(df, join_cols)
# We must use Transaction.table_metadata for the scan. This includes all uncommitted - but relevant - changes.
matched_iceberg_record_batches_scan = DataScan(
table_metadata=self.table_metadata,
io=self._table.io,
row_filter=matched_predicate,
case_sensitive=case_sensitive,
)
if branch in self.table_metadata.refs:
matched_iceberg_record_batches_scan = matched_iceberg_record_batches_scan.use_ref(branch)
matched_iceberg_record_batches = matched_iceberg_record_batches_scan.to_arrow_batch_reader()
batches_to_overwrite = []
overwrite_predicates = []
rows_to_insert = df
for batch in matched_iceberg_record_batches:
rows = pa.Table.from_batches([batch])
if when_matched_update_all:
# function get_rows_to_update is doing a check on non-key columns to see if any of the
# values have actually changed. We don't want to do just a blanket overwrite for matched
# rows if the actual non-key column data hasn't changed.
# this extra step avoids unnecessary IO and writes
rows_to_update = upsert_util.get_rows_to_update(df, rows, join_cols)
if len(rows_to_update) > 0:
# build the match predicate filter
overwrite_mask_predicate = upsert_util.create_match_filter(rows_to_update, join_cols)
batches_to_overwrite.append(rows_to_update)
overwrite_predicates.append(overwrite_mask_predicate)
if when_not_matched_insert_all:
expr_match = upsert_util.create_match_filter(rows, join_cols)
expr_match_bound = bind(self.table_metadata.schema(), expr_match, case_sensitive=case_sensitive)
expr_match_arrow = expression_to_pyarrow(expr_match_bound)
# Filter rows per batch.
rows_to_insert = rows_to_insert.filter(~expr_match_arrow)
update_row_cnt = 0
insert_row_cnt = 0
if batches_to_overwrite:
rows_to_update = pa.concat_tables(batches_to_overwrite)
update_row_cnt = len(rows_to_update)
self.overwrite(
rows_to_update,
overwrite_filter=Or(*overwrite_predicates) if len(overwrite_predicates) > 1 else overwrite_predicates[0],
branch=branch,
snapshot_properties=snapshot_properties,
)
if when_not_matched_insert_all:
insert_row_cnt = len(rows_to_insert)
if rows_to_insert:
self.append(rows_to_insert, branch=branch, snapshot_properties=snapshot_properties)
return UpsertResult(rows_updated=update_row_cnt, rows_inserted=insert_row_cnt)
def add_files(
self,
file_paths: list[str],
snapshot_properties: dict[str, str] = EMPTY_DICT,
check_duplicate_files: bool = True,
branch: str | None = MAIN_BRANCH,
) -> None:
"""
Shorthand API for adding files as data files to the table transaction.
Args:
file_paths: The list of full file paths to be added as data files to the table
Raises:
FileNotFoundError: If the file does not exist.
ValueError: Raises a ValueError given file_paths contains duplicate files
ValueError: Raises a ValueError given file_paths already referenced by table
"""
if len(file_paths) != len(set(file_paths)):
raise ValueError("File paths must be unique")
if check_duplicate_files:
import pyarrow.compute as pc
expr = pc.field("file_path").isin(file_paths)
referenced_files = [file["file_path"] for file in self._table.inspect.data_files().filter(expr).to_pylist()]
if referenced_files:
raise ValueError(f"Cannot add files that are already referenced by table, files: {', '.join(referenced_files)}")
if self.table_metadata.name_mapping() is None:
self.set_properties(
**{TableProperties.DEFAULT_NAME_MAPPING: self.table_metadata.schema().name_mapping.model_dump_json()}
)
with self._append_snapshot_producer(snapshot_properties, branch=branch) as append_files:
data_files = _parquet_files_to_data_files(
table_metadata=self.table_metadata, file_paths=file_paths, io=self._table.io
)
for data_file in data_files:
append_files.append_data_file(data_file)
def update_spec(self) -> UpdateSpec:
"""Create a new UpdateSpec to update the partitioning of the table.
Returns:
A new UpdateSpec.
"""
return UpdateSpec(self)
def remove_properties(self, *removals: str) -> Transaction:
"""Remove properties.
Args:
removals: Properties to be removed.
Returns:
The alter table builder.
"""
return self._apply((RemovePropertiesUpdate(removals=removals),))
def update_location(self, location: str) -> Transaction:
"""Set the new table location.
Args:
location: The new location of the table.
Returns:
The alter table builder.
"""
raise NotImplementedError("Not yet implemented")
def commit_transaction(self) -> Table:
"""Commit the changes to the catalog.
Returns:
The table with the updates applied.
"""
if len(self._updates) > 0:
self._requirements += (AssertTableUUID(uuid=self.table_metadata.table_uuid),)
self._table._do_commit( # pylint: disable=W0212
updates=self._updates,
requirements=self._requirements,
)
self._updates = ()
self._requirements = ()
return self._table