50
50
import pyiceberg .expressions .visitors as visitors
51
51
from pyiceberg .exceptions import CommitFailedException , ResolveError , ValidationError
52
52
from pyiceberg .expressions import (
53
+ AlwaysFalse ,
53
54
AlwaysTrue ,
54
55
And ,
55
56
BooleanExpression ,
56
57
EqualTo ,
58
+ Or ,
57
59
Reference ,
58
60
)
59
61
from pyiceberg .io import FileIO , load_file_io
@@ -2710,6 +2712,114 @@ def _commit(self) -> UpdatesAndRequirements:
2710
2712
)
2711
2713
2712
2714
2715
+ class DeleteFiles (_MergingSnapshotProducer ):
2716
+ _predicate : BooleanExpression
2717
+
2718
+ def __init__ (
2719
+ self ,
2720
+ operation : Operation ,
2721
+ transaction : Transaction ,
2722
+ io : FileIO ,
2723
+ commit_uuid : Optional [uuid .UUID ] = None ,
2724
+ snapshot_properties : Dict [str , str ] = EMPTY_DICT ,
2725
+ ):
2726
+ super ().__init__ (operation , transaction , io , commit_uuid , snapshot_properties )
2727
+ self ._predicate = AlwaysFalse ()
2728
+
2729
+ def _build_partition_projection (self , spec_id : int ) -> BooleanExpression :
2730
+ schema = self ._transaction .table_metadata .schema ()
2731
+ spec = self ._transaction .table_metadata .specs ()[spec_id ]
2732
+ project = visitors .inclusive_projection (schema , spec )
2733
+ return project (self ._predicate )
2734
+
2735
+ @cached_property
2736
+ def partition_filters (self ) -> KeyDefaultDict [int , BooleanExpression ]:
2737
+ return KeyDefaultDict (self ._build_partition_projection )
2738
+
2739
+ def _build_manifest_evaluator (self , spec_id : int ) -> Callable [[ManifestFile ], bool ]:
2740
+ schema = self ._transaction .table_metadata .schema ()
2741
+ spec = self ._transaction .table_metadata .specs ()[spec_id ]
2742
+ return visitors .manifest_evaluator (spec , schema , self .partition_filters [spec_id ], case_sensitive = True )
2743
+
2744
+ def _build_partition_evaluator (self , spec_id : int ) -> Callable [[DataFile ], bool ]:
2745
+ schema = self ._transaction .table_metadata .schema ()
2746
+ spec = self ._transaction .table_metadata .specs ()[spec_id ]
2747
+ partition_type = spec .partition_type (schema )
2748
+ partition_schema = Schema (* partition_type .fields )
2749
+ partition_expr = self .partition_filters [spec_id ]
2750
+
2751
+ return lambda data_file : visitors .expression_evaluator (partition_schema , partition_expr , case_sensitive = True )(
2752
+ data_file .partition
2753
+ )
2754
+
2755
+ def delete (self , predicate : BooleanExpression ) -> None :
2756
+ self ._predicate = Or (self ._predicate , predicate )
2757
+
2758
+ @cached_property
2759
+ def _compute_deletes (self ) -> Tuple [List [ManifestFile ], List [ManifestEntry ]]:
2760
+ schema = self ._transaction .table_metadata .schema ()
2761
+
2762
+ def _copy_with_new_status (entry : ManifestEntry , status : ManifestEntryStatus ) -> ManifestEntry :
2763
+ return ManifestEntry (
2764
+ status = status ,
2765
+ snapshot_id = entry .snapshot_id ,
2766
+ data_sequence_number = entry .data_sequence_number ,
2767
+ file_sequence_number = entry .file_sequence_number ,
2768
+ data_file = entry .data_file ,
2769
+ )
2770
+
2771
+ manifest_evaluators : Dict [int , Callable [[ManifestFile ], bool ]] = KeyDefaultDict (self ._build_manifest_evaluator )
2772
+ strict_metrics_evaluator = visitors ._StrictMetricsEvaluator (schema , self ._predicate , case_sensitive = True ).eval
2773
+ inclusive_metrics_evaluator = visitors ._InclusiveMetricsEvaluator (schema , self ._predicate , case_sensitive = True ).eval
2774
+
2775
+ existing_manifests = []
2776
+ total_deleted_entries = []
2777
+ if snapshot := self ._transaction .table_metadata .current_snapshot ():
2778
+ for num , manifest_file in enumerate (snapshot .manifests (io = self ._io )):
2779
+ if not manifest_evaluators [manifest_file .partition_spec_id ](manifest_file ):
2780
+ # If the manifest isn't relevant, we can just keep it in the manifest-list
2781
+ existing_manifests .append (manifest_file )
2782
+ else :
2783
+ # It is relevant, let's check out the content
2784
+ deleted_entries = []
2785
+ existing_entries = []
2786
+ for entry in manifest_file .fetch_manifest_entry (io = self ._io ):
2787
+ if strict_metrics_evaluator (entry .data_file ) == visitors .ROWS_MUST_MATCH :
2788
+ deleted_entries .append (_copy_with_new_status (entry , ManifestEntryStatus .DELETED ))
2789
+ elif inclusive_metrics_evaluator (entry .data_file ) == visitors .ROWS_CANNOT_MATCH :
2790
+ existing_entries .append (_copy_with_new_status (entry , ManifestEntryStatus .EXISTING ))
2791
+ else :
2792
+ raise ValueError ("Deletes do not support rewrites of data files" )
2793
+
2794
+ if len (deleted_entries ) > 0 :
2795
+ total_deleted_entries += deleted_entries
2796
+
2797
+ # Rewrite the manifest
2798
+ if len (existing_entries ) > 0 :
2799
+ output_file_location = _new_manifest_path (
2800
+ location = self ._transaction .table_metadata .location , num = num , commit_uuid = self .commit_uuid
2801
+ )
2802
+ with write_manifest (
2803
+ format_version = self ._transaction .table_metadata .format_version ,
2804
+ spec = self ._transaction .table_metadata .specs ()[manifest_file .partition_spec_id ],
2805
+ schema = self ._transaction .table_metadata .schema (),
2806
+ output_file = self ._io .new_output (output_file_location ),
2807
+ snapshot_id = self ._snapshot_id ,
2808
+ ) as writer :
2809
+ for existing_entry in existing_entries :
2810
+ writer .add_entry (existing_entry )
2811
+ else :
2812
+ existing_manifests .append (manifest_file )
2813
+
2814
+ return existing_manifests , total_deleted_entries
2815
+
2816
+ def _existing_manifests (self ) -> List [ManifestFile ]:
2817
+ return self ._compute_deletes [0 ]
2818
+
2819
+ def _deleted_entries (self ) -> List [ManifestEntry ]:
2820
+ return self ._compute_deletes [1 ]
2821
+
2822
+
2713
2823
class FastAppendFiles (_MergingSnapshotProducer ):
2714
2824
def _existing_manifests (self ) -> List [ManifestFile ]:
2715
2825
"""To determine if there are any existing manifest files.
@@ -2787,7 +2897,7 @@ class UpdateSnapshot:
2787
2897
_io : FileIO
2788
2898
_snapshot_properties : Dict [str , str ]
2789
2899
2790
- def __init__ (self , transaction : Transaction , io : FileIO , snapshot_properties : Dict [str , str ]) -> None :
2900
+ def __init__ (self , transaction : Transaction , io : FileIO , snapshot_properties : Dict [str , str ] = EMPTY_DICT ) -> None :
2791
2901
self ._transaction = transaction
2792
2902
self ._io = io
2793
2903
self ._snapshot_properties = snapshot_properties
@@ -2807,6 +2917,14 @@ def overwrite(self) -> OverwriteFiles:
2807
2917
snapshot_properties = self ._snapshot_properties ,
2808
2918
)
2809
2919
2920
+ def delete (self ) -> DeleteFiles :
2921
+ return DeleteFiles (
2922
+ operation = Operation .DELETE ,
2923
+ transaction = self ._transaction ,
2924
+ io = self ._io ,
2925
+ snapshot_properties = self ._snapshot_properties ,
2926
+ )
2927
+
2810
2928
2811
2929
class UpdateSpec (UpdateTableMetadata ["UpdateSpec" ]):
2812
2930
_transaction : Transaction
0 commit comments