@@ -10,7 +10,7 @@ use datafusion::common::Statistics;
10
10
use datafusion:: error:: { DataFusionError , Result as DataFusionResult } ;
11
11
use datafusion:: execution:: SendableRecordBatchStream ;
12
12
use datafusion:: physical_plan:: execution_plan:: { Boundedness , EmissionType } ;
13
- use datafusion:: physical_plan:: stream :: RecordBatchStreamAdapter ;
13
+ use datafusion:: physical_plan:: metrics :: { BaselineMetrics , ExecutionPlanMetricsSet , MetricsSet } ;
14
14
use datafusion:: physical_plan:: { DisplayAs , DisplayFormatType , ExecutionPlan , PlanProperties } ;
15
15
use datafusion_physical_expr:: { EquivalenceProperties , Partitioning } ;
16
16
use futures:: stream:: { self } ;
@@ -24,7 +24,9 @@ use tracing::instrument;
24
24
use crate :: index:: prefilter:: DatasetPreFilter ;
25
25
use crate :: { index:: DatasetIndexInternalExt , Dataset } ;
26
26
27
- use super :: utils:: { FilteredRowIdsToPrefilter , SelectionVectorToPrefilter } ;
27
+ use super :: utils:: {
28
+ FilteredRowIdsToPrefilter , InstrumentedRecordBatchStreamAdapter , SelectionVectorToPrefilter ,
29
+ } ;
28
30
use super :: PreFilterSource ;
29
31
30
32
/// An execution node that performs full text search
@@ -41,6 +43,8 @@ pub struct FtsExec {
41
43
/// Prefiltering input
42
44
prefilter_source : PreFilterSource ,
43
45
properties : PlanProperties ,
46
+
47
+ metrics : ExecutionPlanMetricsSet ,
44
48
}
45
49
46
50
impl DisplayAs for FtsExec {
@@ -72,6 +76,7 @@ impl FtsExec {
72
76
query,
73
77
prefilter_source,
74
78
properties,
79
+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
75
80
}
76
81
}
77
82
}
@@ -108,6 +113,7 @@ impl ExecutionPlan for FtsExec {
108
113
query : self . query . clone ( ) ,
109
114
prefilter_source : PreFilterSource :: None ,
110
115
properties : self . properties . clone ( ) ,
116
+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
111
117
} ,
112
118
1 => {
113
119
let src = children. pop ( ) . unwrap ( ) ;
@@ -130,6 +136,7 @@ impl ExecutionPlan for FtsExec {
130
136
query : self . query . clone ( ) ,
131
137
prefilter_source,
132
138
properties : self . properties . clone ( ) ,
139
+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
133
140
}
134
141
}
135
142
_ => {
@@ -150,6 +157,7 @@ impl ExecutionPlan for FtsExec {
150
157
let query = self . query . clone ( ) ;
151
158
let ds = self . dataset . clone ( ) ;
152
159
let prefilter_source = self . prefilter_source . clone ( ) ;
160
+ let baseline_metrics = BaselineMetrics :: new ( & self . metrics , partition) ;
153
161
154
162
let indices = self . indices . clone ( ) ;
155
163
let stream = stream:: iter ( indices)
@@ -208,16 +216,21 @@ impl ExecutionPlan for FtsExec {
208
216
} )
209
217
. buffered ( self . indices . len ( ) ) ;
210
218
let schema = self . schema ( ) ;
211
- Ok (
212
- Box :: pin ( RecordBatchStreamAdapter :: new ( schema, stream. boxed ( ) ) )
213
- as SendableRecordBatchStream ,
214
- )
219
+ Ok ( Box :: pin ( InstrumentedRecordBatchStreamAdapter :: new (
220
+ schema,
221
+ stream. boxed ( ) ,
222
+ baseline_metrics,
223
+ ) ) as SendableRecordBatchStream )
215
224
}
216
225
217
226
fn statistics ( & self ) -> DataFusionResult < datafusion:: physical_plan:: Statistics > {
218
227
Ok ( Statistics :: new_unknown ( & FTS_SCHEMA ) )
219
228
}
220
229
230
+ fn metrics ( & self ) -> Option < MetricsSet > {
231
+ Some ( self . metrics . clone_inner ( ) )
232
+ }
233
+
221
234
fn properties ( & self ) -> & PlanProperties {
222
235
& self . properties
223
236
}
@@ -235,6 +248,7 @@ pub struct FlatFtsExec {
235
248
column_inputs : Vec < ( String , Vec < Index > , Arc < dyn ExecutionPlan > ) > ,
236
249
query : FullTextSearchQuery ,
237
250
properties : PlanProperties ,
251
+ metrics : ExecutionPlanMetricsSet ,
238
252
}
239
253
240
254
impl DisplayAs for FlatFtsExec {
@@ -264,6 +278,7 @@ impl FlatFtsExec {
264
278
column_inputs,
265
279
query,
266
280
properties,
281
+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
267
282
}
268
283
}
269
284
}
@@ -309,6 +324,7 @@ impl ExecutionPlan for FlatFtsExec {
309
324
column_inputs,
310
325
query : self . query . clone ( ) ,
311
326
properties : self . properties . clone ( ) ,
327
+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
312
328
} ) )
313
329
}
314
330
@@ -321,6 +337,7 @@ impl ExecutionPlan for FlatFtsExec {
321
337
let query = self . query . clone ( ) ;
322
338
let ds = self . dataset . clone ( ) ;
323
339
let column_inputs = self . column_inputs . clone ( ) ;
340
+ let baseline_metrics = BaselineMetrics :: new ( & self . metrics , partition) ;
324
341
325
342
let stream = stream:: iter ( column_inputs)
326
343
. map ( move |( column, indices, input) | {
@@ -353,16 +370,21 @@ impl ExecutionPlan for FlatFtsExec {
353
370
. buffered ( self . column_inputs . len ( ) )
354
371
. try_flatten ( ) ;
355
372
let schema = self . schema ( ) ;
356
- Ok (
357
- Box :: pin ( RecordBatchStreamAdapter :: new ( schema, stream. boxed ( ) ) )
358
- as SendableRecordBatchStream ,
359
- )
373
+ Ok ( Box :: pin ( InstrumentedRecordBatchStreamAdapter :: new (
374
+ schema,
375
+ stream. boxed ( ) ,
376
+ baseline_metrics,
377
+ ) ) as SendableRecordBatchStream )
360
378
}
361
379
362
380
fn statistics ( & self ) -> DataFusionResult < datafusion:: physical_plan:: Statistics > {
363
381
Ok ( Statistics :: new_unknown ( & FTS_SCHEMA ) )
364
382
}
365
383
384
+ fn metrics ( & self ) -> Option < MetricsSet > {
385
+ Some ( self . metrics . clone_inner ( ) )
386
+ }
387
+
366
388
fn properties ( & self ) -> & PlanProperties {
367
389
& self . properties
368
390
}
0 commit comments