@@ -35,7 +35,10 @@ use arrow::datatypes::SchemaRef;
3535use arrow:: error:: Result as ArrowResult ;
3636use arrow:: record_batch:: RecordBatch ;
3737
38- use super :: { RecordBatchStream , SendableRecordBatchStream , Statistics } ;
38+ use super :: {
39+ metrics:: { BaselineMetrics , ExecutionPlanMetricsSet , MetricsSet } ,
40+ RecordBatchStream , SendableRecordBatchStream , Statistics ,
41+ } ;
3942
4043use async_trait:: async_trait;
4144
@@ -46,12 +49,18 @@ pub struct GlobalLimitExec {
4649 input : Arc < dyn ExecutionPlan > ,
4750 /// Maximum number of rows to return
4851 limit : usize ,
52+ /// Execution metrics
53+ metrics : ExecutionPlanMetricsSet ,
4954}
5055
5156impl GlobalLimitExec {
5257 /// Create a new GlobalLimitExec
5358 pub fn new ( input : Arc < dyn ExecutionPlan > , limit : usize ) -> Self {
54- GlobalLimitExec { input, limit }
59+ GlobalLimitExec {
60+ input,
61+ limit,
62+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
63+ }
5564 }
5665
5766 /// Input execution plan
@@ -120,8 +129,13 @@ impl ExecutionPlan for GlobalLimitExec {
120129 ) ) ;
121130 }
122131
132+ let baseline_metrics = BaselineMetrics :: new ( & self . metrics , partition) ;
123133 let stream = self . input . execute ( 0 ) . await ?;
124- Ok ( Box :: pin ( LimitStream :: new ( stream, self . limit ) ) )
134+ Ok ( Box :: pin ( LimitStream :: new (
135+ stream,
136+ self . limit ,
137+ baseline_metrics,
138+ ) ) )
125139 }
126140
127141 fn fmt_as (
@@ -136,6 +150,10 @@ impl ExecutionPlan for GlobalLimitExec {
136150 }
137151 }
138152
153+ fn metrics ( & self ) -> Option < MetricsSet > {
154+ Some ( self . metrics . clone_inner ( ) )
155+ }
156+
139157 fn statistics ( & self ) -> Statistics {
140158 let input_stats = self . input . statistics ( ) ;
141159 match input_stats {
@@ -165,12 +183,18 @@ pub struct LocalLimitExec {
165183 input : Arc < dyn ExecutionPlan > ,
166184 /// Maximum number of rows to return
167185 limit : usize ,
186+ /// Execution metrics
187+ metrics : ExecutionPlanMetricsSet ,
168188}
169189
170190impl LocalLimitExec {
171191 /// Create a new LocalLimitExec partition
172192 pub fn new ( input : Arc < dyn ExecutionPlan > , limit : usize ) -> Self {
173- Self { input, limit }
193+ Self {
194+ input,
195+ limit,
196+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
197+ }
174198 }
175199
176200 /// Input execution plan
@@ -219,8 +243,13 @@ impl ExecutionPlan for LocalLimitExec {
219243 }
220244
221245 async fn execute ( & self , partition : usize ) -> Result < SendableRecordBatchStream > {
246+ let baseline_metrics = BaselineMetrics :: new ( & self . metrics , partition) ;
222247 let stream = self . input . execute ( partition) . await ?;
223- Ok ( Box :: pin ( LimitStream :: new ( stream, self . limit ) ) )
248+ Ok ( Box :: pin ( LimitStream :: new (
249+ stream,
250+ self . limit ,
251+ baseline_metrics,
252+ ) ) )
224253 }
225254
226255 fn fmt_as (
@@ -235,6 +264,10 @@ impl ExecutionPlan for LocalLimitExec {
235264 }
236265 }
237266
267+ fn metrics ( & self ) -> Option < MetricsSet > {
268+ Some ( self . metrics . clone_inner ( ) )
269+ }
270+
238271 fn statistics ( & self ) -> Statistics {
239272 let input_stats = self . input . statistics ( ) ;
240273 match input_stats {
@@ -280,20 +313,29 @@ struct LimitStream {
280313 schema : SchemaRef ,
281314 // the current number of rows which have been produced
282315 current_len : usize ,
316+ /// Execution time metrics
317+ baseline_metrics : BaselineMetrics ,
283318}
284319
285320impl LimitStream {
286- fn new ( input : SendableRecordBatchStream , limit : usize ) -> Self {
321+ fn new (
322+ input : SendableRecordBatchStream ,
323+ limit : usize ,
324+ baseline_metrics : BaselineMetrics ,
325+ ) -> Self {
287326 let schema = input. schema ( ) ;
288327 Self {
289328 limit,
290329 input : Some ( input) ,
291330 schema,
292331 current_len : 0 ,
332+ baseline_metrics,
293333 }
294334 }
295335
296336 fn stream_limit ( & mut self , batch : RecordBatch ) -> Option < RecordBatch > {
337+ // records time on drop
338+ let _timer = self . baseline_metrics . elapsed_compute ( ) . timer ( ) ;
297339 if self . current_len == self . limit {
298340 self . input = None ; // clear input so it can be dropped early
299341 None
@@ -316,14 +358,16 @@ impl Stream for LimitStream {
316358 mut self : Pin < & mut Self > ,
317359 cx : & mut Context < ' _ > ,
318360 ) -> Poll < Option < Self :: Item > > {
319- match & mut self . input {
361+ let poll = match & mut self . input {
320362 Some ( input) => input. poll_next_unpin ( cx) . map ( |x| match x {
321363 Some ( Ok ( batch) ) => Ok ( self . stream_limit ( batch) ) . transpose ( ) ,
322364 other => other,
323365 } ) ,
324366 // input has been cleared
325367 None => Poll :: Ready ( None ) ,
326- }
368+ } ;
369+
370+ self . baseline_metrics . record_poll ( poll)
327371 }
328372}
329373
@@ -394,7 +438,8 @@ mod tests {
394438
395439 // limit of six needs to consume the entire first record batch
396440 // (5 rows) and 1 row from the second (1 row)
397- let limit_stream = LimitStream :: new ( Box :: pin ( input) , 6 ) ;
441+ let baseline_metrics = BaselineMetrics :: new ( & ExecutionPlanMetricsSet :: new ( ) , 0 ) ;
442+ let limit_stream = LimitStream :: new ( Box :: pin ( input) , 6 , baseline_metrics) ;
398443 assert_eq ! ( index. value( ) , 0 ) ;
399444
400445 let results = collect ( Box :: pin ( limit_stream) ) . await . unwrap ( ) ;
0 commit comments