@@ -19,6 +19,7 @@ use std::sync::Arc;
19
19
20
20
use futures:: StreamExt ;
21
21
use futures:: stream:: BoxStream ;
22
+ use tracing:: field:: Empty ;
22
23
23
24
use crate :: delete_file_index:: DeleteFileIndex ;
24
25
use crate :: expr:: { Bind , BoundPredicate , Predicate } ;
@@ -53,14 +54,26 @@ pub(crate) struct ManifestEntryContext {
53
54
pub bound_predicates : Option < Arc < BoundPredicates > > ,
54
55
pub partition_spec_id : i32 ,
55
56
pub snapshot_schema : SchemaRef ,
57
+ pub ( crate ) span : tracing:: Span ,
56
58
}
57
59
58
60
impl ManifestFileContext {
59
61
/// Consumes this [`ManifestFileContext`], fetching its Manifest from FileIO and then
60
62
/// streaming its constituent [`ManifestEntries`]
61
63
pub ( crate ) async fn fetch_manifest_and_stream_entries (
62
64
self ,
65
+ parent_span : tracing:: Span ,
63
66
) -> Result < BoxStream < ' static , Result < ManifestEntryContext > > > {
67
+ let manifest_span = tracing:: debug_span!(
68
+ parent: parent_span,
69
+ "iceberg.scan.plan.process_manifest" ,
70
+ iceberg. scan. plan. manifest. file_path = self . manifest_file. manifest_path,
71
+ iceberg. scan. plan. manifest. entries_count = Empty ,
72
+ ) ;
73
+
74
+ let span = manifest_span. clone ( ) ;
75
+ let _guard = manifest_span. enter ( ) ;
76
+
64
77
let ManifestFileContext {
65
78
object_cache,
66
79
manifest_file,
@@ -73,15 +86,30 @@ impl ManifestFileContext {
73
86
74
87
let manifest = object_cache. get_manifest ( & manifest_file) . await ?;
75
88
89
+ span. record (
90
+ "iceberg.scan.plan.manifest.entries_count" ,
91
+ manifest. entries ( ) . len ( ) ,
92
+ ) ;
93
+
76
94
Ok ( async_stream:: stream! {
77
95
for manifest_entry in manifest. entries( ) {
96
+ let manifest_entry_span = tracing:: debug_span!(
97
+ parent: span. clone( ) ,
98
+ "iceberg.scan.plan.process_data_file" ,
99
+ iceberg. scam. plan. data_file. file_path = manifest_entry. file_path( ) ,
100
+ "iceberg.scan.plan_data_file.type" = Empty ,
101
+ iceberg. scan. plan. data_file. skipped = Empty ,
102
+ iceberg. scan. plan. data_file. skipped_reason = Empty ,
103
+ ) ;
104
+
78
105
yield Ok ( ManifestEntryContext {
79
106
manifest_entry: manifest_entry. clone( ) ,
80
107
expression_evaluator_cache: expression_evaluator_cache. clone( ) ,
81
108
field_ids: field_ids. clone( ) ,
82
109
partition_spec_id: manifest_file. partition_spec_id,
83
110
bound_predicates: bound_predicates. clone( ) ,
84
111
snapshot_schema: snapshot_schema. clone( ) ,
112
+ span: manifest_entry_span,
85
113
} ) ;
86
114
}
87
115
}
@@ -144,7 +172,11 @@ pub(crate) struct PlanContext {
144
172
}
145
173
146
174
impl PlanContext {
147
- #[ tracing:: instrument( skip_all) ]
175
+ #[ tracing:: instrument(
176
+ skip_all,
177
+ level = "debug" ,
178
+ fields( iceberg. scan. plan. manifest_list. file_path = ?self . snapshot. manifest_list( ) ) ,
179
+ ) ]
148
180
pub ( crate ) async fn get_manifest_list ( & self ) -> Result < Arc < ManifestList > > {
149
181
self . object_cache
150
182
. as_ref ( )
@@ -175,14 +207,19 @@ impl PlanContext {
175
207
176
208
#[ tracing:: instrument(
177
209
skip_all,
210
+ level = "debug" ,
211
+ name = "iceberg.scan.plan.process_manifest_list" ,
178
212
fields(
179
- manifest_list. len = manifest_list. entries( ) . len( ) ,
213
+ iceberg . scan . plan . manifest_list. entries_count = manifest_list. entries( ) . len( ) ,
180
214
)
181
215
) ]
182
- pub ( crate ) fn build_manifest_file_context_iter (
216
+ pub ( crate ) fn build_manifest_file_contexts (
183
217
& self ,
184
218
manifest_list : Arc < ManifestList > ,
185
- ) -> impl Iterator < Item = Result < ManifestFileContext > > {
219
+ ) -> (
220
+ Vec < Result < ManifestFileContext > > ,
221
+ Vec < Result < ManifestFileContext > > ,
222
+ ) {
186
223
let has_predicate = self . predicate . is_some ( ) ;
187
224
188
225
( 0 ..manifest_list. entries ( ) . len ( ) )
@@ -198,24 +235,28 @@ impl PlanContext {
198
235
. get ( manifest_file. partition_spec_id , predicate. clone ( ) )
199
236
. eval ( & manifest_file) ?
200
237
{
201
- tracing:: trace!( file_path = manifest_file. manifest_path, "iceberg.scan.manifest_file.skipped" ) ;
202
- metrics:: counter!( "iceberg.scan.manifest_file.skipped" , "reason" => "partition" ) . increment ( 1 ) ;
238
+ tracing:: debug!(
239
+ iceberg. scan. plan. manifest. file_path = manifest_file. manifest_path,
240
+ iceberg. scan. plan. manifest. skip_reason = "partition" ,
241
+ "iceberg.scan.plan.manifest_file.skipped"
242
+ ) ;
243
+ metrics:: counter!( "iceberg.scan.plan.manifest_file.skipped" , "reason" => "partition" ) . increment ( 1 ) ;
203
244
return Ok ( None ) ; // Skip this file.
204
245
}
205
246
Some ( predicate)
206
247
} else {
207
248
None
208
249
} ;
209
250
210
- tracing:: trace!( file_path = manifest_file. manifest_path, "iceberg.scan.manifest_file.included" ) ;
211
- metrics:: counter!( "iceberg.scan.manifest_file.included" ) . increment ( 1 ) ;
251
+ metrics:: counter!( "iceberg.scan.plan.manifest_file.included" ) . increment ( 1 ) ;
212
252
213
253
let context = self
214
254
. create_manifest_file_context ( manifest_file, partition_bound_predicate) ?;
215
255
Ok ( Some ( context) )
216
256
} ) ( )
217
257
. transpose ( )
218
258
} )
259
+ . partition ( |ctx| ctx. as_ref ( ) . map_or ( true , |ctx| ctx. is_delete ( ) ) )
219
260
}
220
261
221
262
fn create_manifest_file_context (
0 commit comments