@@ -31,6 +31,8 @@ use datafusion_common::{internal_err, plan_err, Result};
3131use datafusion_execution:: TaskContext ;
3232use datafusion_physical_expr:: { EquivalenceProperties , LexOrdering } ;
3333
34+ use crate :: limit:: LimitStream ;
35+ use crate :: metrics:: { BaselineMetrics , ExecutionPlanMetricsSet , MetricsSet } ;
3436use async_trait:: async_trait;
3537use futures:: stream:: StreamExt ;
3638use log:: debug;
@@ -58,7 +60,9 @@ pub struct StreamingTableExec {
5860 projected_schema : SchemaRef ,
5961 projected_output_ordering : Vec < LexOrdering > ,
6062 infinite : bool ,
63+ limit : Option < usize > ,
6164 cache : PlanProperties ,
65+ metrics : ExecutionPlanMetricsSet ,
6266}
6367
6468impl StreamingTableExec {
@@ -69,6 +73,7 @@ impl StreamingTableExec {
6973 projection : Option < & Vec < usize > > ,
7074 projected_output_ordering : impl IntoIterator < Item = LexOrdering > ,
7175 infinite : bool ,
76+ limit : Option < usize > ,
7277 ) -> Result < Self > {
7378 for x in partitions. iter ( ) {
7479 let partition_schema = x. schema ( ) ;
@@ -99,7 +104,9 @@ impl StreamingTableExec {
99104 projection : projection. cloned ( ) . map ( Into :: into) ,
100105 projected_output_ordering,
101106 infinite,
107+ limit,
102108 cache,
109+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
103110 } )
104111 }
105112
@@ -127,6 +134,10 @@ impl StreamingTableExec {
127134 self . infinite
128135 }
129136
137+ pub fn limit ( & self ) -> Option < usize > {
138+ self . limit
139+ }
140+
130141 /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
131142 fn compute_properties (
132143 schema : SchemaRef ,
@@ -180,6 +191,9 @@ impl DisplayAs for StreamingTableExec {
180191 if self . infinite {
181192 write ! ( f, ", infinite_source=true" ) ?;
182193 }
194+ if let Some ( fetch) = self . limit {
195+ write ! ( f, ", fetch={fetch}" ) ?;
196+ }
183197
184198 display_orderings ( f, & self . projected_output_ordering ) ?;
185199
@@ -224,14 +238,116 @@ impl ExecutionPlan for StreamingTableExec {
224238 ctx : Arc < TaskContext > ,
225239 ) -> Result < SendableRecordBatchStream > {
226240 let stream = self . partitions [ partition] . execute ( ctx) ;
227- Ok ( match self . projection . clone ( ) {
241+ let projected_stream = match self . projection . clone ( ) {
228242 Some ( projection) => Box :: pin ( RecordBatchStreamAdapter :: new (
229243 self . projected_schema . clone ( ) ,
230244 stream. map ( move |x| {
231245 x. and_then ( |b| b. project ( projection. as_ref ( ) ) . map_err ( Into :: into) )
232246 } ) ,
233247 ) ) ,
234248 None => stream,
249+ } ;
250+ Ok ( match self . limit {
251+ None => projected_stream,
252+ Some ( fetch) => {
253+ let baseline_metrics = BaselineMetrics :: new ( & self . metrics , partition) ;
254+ Box :: pin ( LimitStream :: new (
255+ projected_stream,
256+ 0 ,
257+ Some ( fetch) ,
258+ baseline_metrics,
259+ ) )
260+ }
235261 } )
236262 }
263+
264+ fn metrics ( & self ) -> Option < MetricsSet > {
265+ Some ( self . metrics . clone_inner ( ) )
266+ }
267+ }
268+
269+ #[ cfg( test) ]
270+ mod test {
271+ use super :: * ;
272+ use crate :: collect_partitioned;
273+ use crate :: streaming:: PartitionStream ;
274+ use crate :: test:: { make_partition, TestPartitionStream } ;
275+ use arrow:: record_batch:: RecordBatch ;
276+
277+ #[ tokio:: test]
278+ async fn test_no_limit ( ) {
279+ let exec = TestBuilder :: new ( )
280+ // make 2 batches, each with 100 rows
281+ . with_batches ( vec ! [ make_partition( 100 ) , make_partition( 100 ) ] )
282+ . build ( ) ;
283+
284+ let counts = collect_num_rows ( Arc :: new ( exec) ) . await ;
285+ assert_eq ! ( counts, vec![ 200 ] ) ;
286+ }
287+
288+ #[ tokio:: test]
289+ async fn test_limit ( ) {
290+ let exec = TestBuilder :: new ( )
291+ // make 2 batches, each with 100 rows
292+ . with_batches ( vec ! [ make_partition( 100 ) , make_partition( 100 ) ] )
293+ // limit to only the first 75 rows back
294+ . with_limit ( Some ( 75 ) )
295+ . build ( ) ;
296+
297+ let counts = collect_num_rows ( Arc :: new ( exec) ) . await ;
298+ assert_eq ! ( counts, vec![ 75 ] ) ;
299+ }
300+
301+ /// Runs the provided execution plan and returns a vector of the number of
302+ /// rows in each partition
303+ async fn collect_num_rows ( exec : Arc < dyn ExecutionPlan > ) -> Vec < usize > {
304+ let ctx = Arc :: new ( TaskContext :: default ( ) ) ;
305+ let partition_batches = collect_partitioned ( exec, ctx) . await . unwrap ( ) ;
306+ partition_batches
307+ . into_iter ( )
308+ . map ( |batches| batches. iter ( ) . map ( |b| b. num_rows ( ) ) . sum :: < usize > ( ) )
309+ . collect ( )
310+ }
311+
312+ #[ derive( Default ) ]
313+ struct TestBuilder {
314+ schema : Option < SchemaRef > ,
315+ partitions : Vec < Arc < dyn PartitionStream > > ,
316+ projection : Option < Vec < usize > > ,
317+ projected_output_ordering : Vec < LexOrdering > ,
318+ infinite : bool ,
319+ limit : Option < usize > ,
320+ }
321+
322+ impl TestBuilder {
323+ fn new ( ) -> Self {
324+ Self :: default ( )
325+ }
326+
327+ /// Set the batches for the stream
328+ fn with_batches ( mut self , batches : Vec < RecordBatch > ) -> Self {
329+ let stream = TestPartitionStream :: new_with_batches ( batches) ;
330+ self . schema = Some ( stream. schema ( ) . clone ( ) ) ;
331+ self . partitions = vec ! [ Arc :: new( stream) ] ;
332+ self
333+ }
334+
335+ /// Set the limit for the stream
336+ fn with_limit ( mut self , limit : Option < usize > ) -> Self {
337+ self . limit = limit;
338+ self
339+ }
340+
341+ fn build ( self ) -> StreamingTableExec {
342+ StreamingTableExec :: try_new (
343+ self . schema . unwrap ( ) ,
344+ self . partitions ,
345+ self . projection . as_ref ( ) ,
346+ self . projected_output_ordering ,
347+ self . infinite ,
348+ self . limit ,
349+ )
350+ . unwrap ( )
351+ }
352+ }
237353}
0 commit comments