@@ -28,6 +28,7 @@ use futures::StreamExt;
28
28
use crate :: runtime:: spawn;
29
29
use crate :: scan:: { DeleteFileContext , FileScanTaskDeleteFile } ;
30
30
use crate :: spec:: { DataContentType , DataFile , Struct } ;
31
+ use crate :: { Error , ErrorKind , Result } ;
31
32
32
33
/// Index of delete files
33
34
#[ derive( Clone , Debug ) ]
@@ -47,7 +48,10 @@ struct PopulatedDeleteFileIndex {
47
48
global_deletes : Vec < Arc < DeleteFileContext > > ,
48
49
eq_deletes_by_partition : HashMap < Struct , Vec < Arc < DeleteFileContext > > > ,
49
50
pos_deletes_by_partition : HashMap < Struct , Vec < Arc < DeleteFileContext > > > ,
50
- pos_deletes_by_path : HashMap < String , Vec < Arc < DeleteFileContext > > > ,
51
+ // TODO: do we need this?
52
+ // pos_deletes_by_path: HashMap<String, Vec<Arc<DeleteFileContext>>>,
53
+
54
+ // TODO: Deletion Vector support
51
55
}
52
56
53
57
impl DeleteFileIndex {
@@ -75,7 +79,7 @@ impl DeleteFileIndex {
75
79
76
80
/// Gets all the delete files that apply to the specified data file.
77
81
///
78
- /// Returns a future that resolves to a Vec<FileScanTaskDeleteFile>
82
+ /// Returns a future that resolves to a Result< Vec<FileScanTaskDeleteFile> >
79
83
pub ( crate ) fn get_deletes_for_data_file < ' a > (
80
84
& self ,
81
85
data_file : & ' a DataFile ,
@@ -95,60 +99,41 @@ impl PopulatedDeleteFileIndex {
95
99
HashMap :: default ( ) ;
96
100
let mut pos_deletes_by_partition: HashMap < Struct , Vec < Arc < DeleteFileContext > > > =
97
101
HashMap :: default ( ) ;
98
- let mut pos_deletes_by_path: HashMap < String , Vec < Arc < DeleteFileContext > > > =
99
- HashMap :: default ( ) ;
100
102
101
- files. into_iter ( ) . for_each ( |del_file_ctx| {
102
- let arc_del_file_ctx = Arc :: new ( del_file_ctx) ;
103
- match arc_del_file_ctx. manifest_entry . content_type ( ) {
104
- DataContentType :: PositionDeletes => {
105
- // TODO: implement logic from ContentFileUtil.referencedDataFile
106
- // see https://github.com/apache/iceberg/blob/cdf748e8e5537f13d861aa4c617a51f3e11dc97c/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java#L54
107
- let referenced_data_file_path = "TODO" . to_string ( ) ;
108
-
109
- pos_deletes_by_path
110
- . entry ( referenced_data_file_path)
111
- . and_modify ( |entry| {
112
- entry. push ( arc_del_file_ctx. clone ( ) ) ;
113
- } )
114
- . or_insert ( vec ! [ arc_del_file_ctx. clone( ) ] ) ;
115
-
116
- pos_deletes_by_partition
117
- . entry (
118
- arc_del_file_ctx
119
- . manifest_entry
120
- . data_file ( )
121
- . partition ( )
122
- . clone ( ) ,
123
- )
124
- . and_modify ( |entry| {
125
- entry. push ( arc_del_file_ctx. clone ( ) ) ;
126
- } )
127
- . or_insert ( vec ! [ arc_del_file_ctx. clone( ) ] ) ;
128
- }
129
- DataContentType :: EqualityDeletes => {
130
- eq_deletes_by_partition
131
- . entry (
132
- arc_del_file_ctx
133
- . manifest_entry
134
- . data_file ( )
135
- . partition ( )
136
- . clone ( ) ,
137
- )
138
- . and_modify ( |entry| {
139
- entry. push ( arc_del_file_ctx. clone ( ) ) ;
140
- } )
141
- . or_insert ( vec ! [ arc_del_file_ctx. clone( ) ] ) ;
103
+ let mut global_deletes: Vec < Arc < DeleteFileContext > > = vec ! [ ] ;
104
+
105
+ files. into_iter ( ) . for_each ( |ctx| {
106
+ let arc_ctx = Arc :: new ( ctx) ;
107
+
108
+ let partition = arc_ctx. manifest_entry . data_file ( ) . partition ( ) ;
109
+
110
+ // The spec states that "Equality delete files stored with an unpartitioned spec are applied as global deletes".
111
+ if partition. fields ( ) . is_empty ( ) {
112
+ // TODO: confirm we're good to skip here if we encounter a pos del
113
+ if arc_ctx. manifest_entry . content_type ( ) != DataContentType :: PositionDeletes {
114
+ global_deletes. push ( arc_ctx) ;
115
+ return ;
142
116
}
143
- _ => unreachable ! ( ) ,
144
117
}
118
+
119
+ let destination_map = match arc_ctx. manifest_entry . content_type ( ) {
120
+ DataContentType :: PositionDeletes => & mut pos_deletes_by_partition,
121
+ DataContentType :: EqualityDeletes => & mut eq_deletes_by_partition,
122
+ _ => unreachable ! ( ) ,
123
+ } ;
124
+
125
+ destination_map
126
+ . entry ( partition. clone ( ) )
127
+ . and_modify ( |entry| {
128
+ entry. push ( arc_ctx. clone ( ) ) ;
129
+ } )
130
+ . or_insert ( vec ! [ arc_ctx. clone( ) ] ) ;
145
131
} ) ;
146
132
147
133
PopulatedDeleteFileIndex {
148
- global_deletes : vec ! [ ] ,
134
+ global_deletes,
149
135
eq_deletes_by_partition,
150
136
pos_deletes_by_partition,
151
- pos_deletes_by_path,
152
137
}
153
138
}
154
139
@@ -158,33 +143,47 @@ impl PopulatedDeleteFileIndex {
158
143
data_file : & DataFile ,
159
144
seq_num : Option < i64 > ,
160
145
) -> Vec < FileScanTaskDeleteFile > {
161
- let mut deletes_queue = vec ! [ ] ;
162
-
163
- if let Some ( deletes) = self . pos_deletes_by_path . get ( data_file. file_path ( ) ) {
164
- deletes_queue. extend ( deletes. iter ( ) ) ;
165
- }
166
-
167
- if let Some ( deletes) = self . pos_deletes_by_partition . get ( data_file. partition ( ) ) {
168
- deletes_queue. extend ( deletes. iter ( ) ) ;
169
- }
146
+ let mut results = vec ! [ ] ;
170
147
171
- if let Some ( deletes) = self . eq_deletes_by_partition . get ( data_file. partition ( ) ) {
172
- deletes_queue. extend ( deletes. iter ( ) ) ;
173
- }
174
-
175
- deletes_queue
148
+ self . global_deletes
176
149
. iter ( )
150
+ // filter that returns true if the provided delete file's sequence number is **greater than or equal to** `seq_num`
177
151
. filter ( |& delete| {
178
152
seq_num
179
- . map ( |seq_num| delete. manifest_entry . sequence_number ( ) > Some ( seq_num) )
153
+ . map ( |seq_num| delete. manifest_entry . sequence_number ( ) >= Some ( seq_num) )
180
154
. unwrap_or_else ( || true )
181
155
} )
182
- . map ( |delete| FileScanTaskDeleteFile {
183
- file_path : delete. manifest_entry . file_path ( ) . to_string ( ) ,
184
- file_type : delete. manifest_entry . content_type ( ) ,
185
- partition_spec_id : delete. partition_spec_id ,
186
- } )
187
- . collect ( )
156
+ . for_each ( |delete| results. push ( delete. as_ref ( ) . into ( ) ) ) ;
157
+
158
+ if let Some ( deletes) = self . eq_deletes_by_partition . get ( data_file. partition ( ) ) {
159
+ deletes
160
+ . iter ( )
161
+ // filter that returns true if the provided delete file's sequence number is **greater than or equal to** `seq_num`
162
+ . filter ( |& delete| {
163
+ seq_num
164
+ . map ( |seq_num| delete. manifest_entry . sequence_number ( ) >= Some ( seq_num) )
165
+ . unwrap_or_else ( || true )
166
+ } )
167
+ . for_each ( |delete| results. push ( delete. as_ref ( ) . into ( ) ) ) ;
168
+ }
169
+
170
+ // TODO: the spec states that:
171
+ // "The data file's file_path is equal to the delete file's referenced_data_file if it is non-null".
172
+ // we're not yet doing that here. The referenced data file's name will also be present in the positional
173
+ // delete file's file path column.
174
+ if let Some ( deletes) = self . pos_deletes_by_partition . get ( data_file. partition ( ) ) {
175
+ deletes
176
+ . iter ( )
177
+ // filter that returns true if the provided delete file's sequence number is **greater thano** `seq_num`
178
+ . filter ( |& delete| {
179
+ seq_num
180
+ . map ( |seq_num| delete. manifest_entry . sequence_number ( ) > Some ( seq_num) )
181
+ . unwrap_or_else ( || true )
182
+ } )
183
+ . for_each ( |delete| results. push ( delete. as_ref ( ) . into ( ) ) ) ;
184
+ }
185
+
186
+ results
188
187
}
189
188
}
190
189
@@ -196,18 +195,17 @@ pub(crate) struct DeletesForDataFile<'a> {
196
195
}
197
196
198
197
impl Future for DeletesForDataFile < ' _ > {
199
- type Output = Vec < FileScanTaskDeleteFile > ;
198
+ type Output = Result < Vec < FileScanTaskDeleteFile > > ;
200
199
201
200
fn poll ( self : Pin < & mut Self > , _cx : & mut Context < ' _ > ) -> Poll < Self :: Output > {
202
- let Ok ( guard) = self . state . try_read ( ) else {
203
- return Poll :: Pending ;
204
- } ;
205
-
206
- match guard. deref ( ) {
207
- DeleteFileIndexState :: Populated ( idx) => {
208
- Poll :: Ready ( idx. get_deletes_for_data_file ( self . data_file , self . seq_num ) )
209
- }
210
- _ => Poll :: Pending ,
201
+ match self . state . try_read ( ) {
202
+ Ok ( guard) => match guard. deref ( ) {
203
+ DeleteFileIndexState :: Populated ( idx) => Poll :: Ready ( Ok (
204
+ idx. get_deletes_for_data_file ( self . data_file , self . seq_num )
205
+ ) ) ,
206
+ _ => Poll :: Pending ,
207
+ } ,
208
+ Err ( err) => Poll :: Ready ( Err ( Error :: new ( ErrorKind :: Unexpected , err. to_string ( ) ) ) ) ,
211
209
}
212
210
}
213
211
}
0 commit comments