@@ -45,7 +45,7 @@ pub(crate) struct ManifestFileContext {
45
45
object_cache : Arc < ObjectCache > ,
46
46
snapshot_schema : SchemaRef ,
47
47
expression_evaluator_cache : Arc < ExpressionEvaluatorCache > ,
48
- delete_file_index : Option < DeleteFileIndex > ,
48
+ delete_file_index : DeleteFileIndex ,
49
49
}
50
50
51
51
/// Wraps a [`ManifestEntryRef`] alongside the objects that are needed
@@ -58,7 +58,7 @@ pub(crate) struct ManifestEntryContext {
58
58
pub bound_predicates : Option < Arc < BoundPredicates > > ,
59
59
pub partition_spec_id : i32 ,
60
60
pub snapshot_schema : SchemaRef ,
61
- pub delete_file_index : Option < DeleteFileIndex > ,
61
+ pub delete_file_index : DeleteFileIndex ,
62
62
}
63
63
64
64
impl ManifestFileContext {
@@ -105,16 +105,13 @@ impl ManifestEntryContext {
105
105
/// consume this `ManifestEntryContext`, returning a `FileScanTask`
106
106
/// created from it
107
107
pub ( crate ) async fn into_file_scan_task ( self ) -> Result < FileScanTask > {
108
- let deletes = if let Some ( delete_file_index) = self . delete_file_index {
109
- delete_file_index
110
- . get_deletes_for_data_file (
111
- self . manifest_entry . data_file ( ) ,
112
- self . manifest_entry . sequence_number ( ) ,
113
- )
114
- . await ?
115
- } else {
116
- vec ! [ ]
117
- } ;
108
+ let deletes = self
109
+ . delete_file_index
110
+ . get_deletes_for_data_file (
111
+ self . manifest_entry . data_file ( ) ,
112
+ self . manifest_entry . sequence_number ( ) ,
113
+ )
114
+ . await ?;
118
115
119
116
Ok ( FileScanTask {
120
117
start : 0 ,
@@ -188,24 +185,19 @@ impl PlanContext {
188
185
& self ,
189
186
manifest_list : Arc < ManifestList > ,
190
187
tx_data : Sender < ManifestEntryContext > ,
191
- delete_file_idx_and_tx : Option < ( DeleteFileIndex , Sender < ManifestEntryContext > ) > ,
188
+ delete_file_idx : DeleteFileIndex ,
189
+ tx_delete : Sender < ManifestEntryContext > ,
192
190
) -> Result < Box < impl Iterator < Item = Result < ManifestFileContext > > + ' static > > {
193
191
let manifest_files = manifest_list. entries ( ) . iter ( ) ;
194
192
195
193
// TODO: Ideally we could ditch this intermediate Vec as we return an iterator.
196
194
let mut filtered_mfcs = vec ! [ ] ;
197
195
198
196
for manifest_file in manifest_files {
199
- let ( delete_file_idx, tx) = if manifest_file. content == ManifestContentType :: Deletes {
200
- let Some ( ( delete_file_idx, tx) ) = delete_file_idx_and_tx. as_ref ( ) else {
201
- continue ;
202
- } ;
203
- ( Some ( delete_file_idx. clone ( ) ) , tx. clone ( ) )
197
+ let tx = if manifest_file. content == ManifestContentType :: Deletes {
198
+ tx_delete. clone ( )
204
199
} else {
205
- (
206
- delete_file_idx_and_tx. as_ref ( ) . map ( |x| x. 0 . clone ( ) ) ,
207
- tx_data. clone ( ) ,
208
- )
200
+ tx_data. clone ( )
209
201
} ;
210
202
211
203
let partition_bound_predicate = if self . predicate . is_some ( ) {
@@ -233,7 +225,7 @@ impl PlanContext {
233
225
manifest_file,
234
226
partition_bound_predicate,
235
227
tx,
236
- delete_file_idx,
228
+ delete_file_idx. clone ( ) ,
237
229
) ;
238
230
239
231
filtered_mfcs. push ( Ok ( mfc) ) ;
@@ -247,7 +239,7 @@ impl PlanContext {
247
239
manifest_file : & ManifestFile ,
248
240
partition_filter : Option < Arc < BoundPredicate > > ,
249
241
sender : Sender < ManifestEntryContext > ,
250
- delete_file_index : Option < DeleteFileIndex > ,
242
+ delete_file_index : DeleteFileIndex ,
251
243
) -> ManifestFileContext {
252
244
let bound_predicates =
253
245
if let ( Some ( ref partition_bound_predicate) , Some ( snapshot_bound_predicate) ) =
0 commit comments