1616// under the License.
1717
1818use std:: any:: Any ;
19+ use std:: pin:: Pin ;
1920use std:: sync:: Arc ;
2021
22+ use datafusion:: arrow:: array:: RecordBatch ;
2123use datafusion:: arrow:: datatypes:: SchemaRef as ArrowSchemaRef ;
2224use datafusion:: error:: Result as DFResult ;
2325use datafusion:: execution:: { SendableRecordBatchStream , TaskContext } ;
2426use datafusion:: physical_expr:: EquivalenceProperties ;
2527use datafusion:: physical_plan:: execution_plan:: { Boundedness , EmissionType } ;
2628use datafusion:: physical_plan:: stream:: RecordBatchStreamAdapter ;
2729use datafusion:: physical_plan:: { DisplayAs , ExecutionPlan , Partitioning , PlanProperties } ;
28- use futures:: { StreamExt , TryStreamExt } ;
30+ use futures:: { Stream , StreamExt , TryStreamExt } ;
2931use paimon:: table:: Table ;
3032use paimon:: DataSplit ;
3133
@@ -46,6 +48,8 @@ pub struct PaimonTableScan {
4648 /// Wrapped in `Arc` to avoid deep-cloning `DataSplit` metadata in `execute()`.
4749 planned_partitions : Vec < Arc < [ DataSplit ] > > ,
4850 plan_properties : PlanProperties ,
51+ /// Optional limit on the number of rows to return.
52+ limit : Option < usize > ,
4953}
5054
5155impl PaimonTableScan {
@@ -54,6 +58,7 @@ impl PaimonTableScan {
5458 table : Table ,
5559 projected_columns : Option < Vec < String > > ,
5660 planned_partitions : Vec < Arc < [ DataSplit ] > > ,
61+ limit : Option < usize > ,
5762 ) -> Self {
5863 let plan_properties = PlanProperties :: new (
5964 EquivalenceProperties :: new ( schema. clone ( ) ) ,
@@ -66,6 +71,7 @@ impl PaimonTableScan {
6671 projected_columns,
6772 planned_partitions,
6873 plan_properties,
74+ limit,
6975 }
7076 }
7177
@@ -77,6 +83,10 @@ impl PaimonTableScan {
7783 pub ( crate ) fn planned_partitions ( & self ) -> & [ Arc < [ DataSplit ] > ] {
7884 & self . planned_partitions
7985 }
86+
87+ pub fn limit ( & self ) -> Option < usize > {
88+ self . limit
89+ }
8090}
8191
8292impl ExecutionPlan for PaimonTableScan {
@@ -118,6 +128,7 @@ impl ExecutionPlan for PaimonTableScan {
118128 let table = self . table . clone ( ) ;
119129 let schema = self . schema ( ) ;
120130 let projected_columns = self . projected_columns . clone ( ) ;
131+ let limit = self . limit ;
121132
122133 let fut = async move {
123134 let mut read_builder = table. new_read_builder ( ) ;
@@ -138,9 +149,30 @@ impl ExecutionPlan for PaimonTableScan {
138149 } ;
139150
140151 let stream = futures:: stream:: once ( fut) . try_flatten ( ) ;
152+
153+ // Enforce the final LIMIT at the DataFusion execution layer.
154+ let limited_stream: Pin < Box < dyn Stream < Item = DFResult < RecordBatch > > + Send > > =
155+ if let Some ( limit) = limit {
156+ let mut remaining = limit;
157+ Box :: pin ( stream. try_filter_map ( move |batch| {
158+ futures:: future:: ready ( if remaining == 0 {
159+ Ok ( None )
160+ } else if batch. num_rows ( ) <= remaining {
161+ remaining -= batch. num_rows ( ) ;
162+ Ok ( Some ( batch) )
163+ } else {
164+ let limited_batch = batch. slice ( 0 , remaining) ;
165+ remaining = 0 ;
166+ Ok ( Some ( limited_batch) )
167+ } )
168+ } ) )
169+ } else {
170+ Box :: pin ( stream)
171+ } ;
172+
141173 Ok ( Box :: pin ( RecordBatchStreamAdapter :: new (
142174 self . schema ( ) ,
143- stream ,
175+ limited_stream ,
144176 ) ) )
145177 }
146178}
@@ -155,7 +187,11 @@ impl DisplayAs for PaimonTableScan {
155187 f,
156188 "PaimonTableScan: partitions={}" ,
157189 self . planned_partitions. len( )
158- )
190+ ) ?;
191+ if let Some ( limit) = self . limit {
192+ write ! ( f, ", limit={limit}" ) ?;
193+ }
194+ Ok ( ( ) )
159195 }
160196}
161197
@@ -176,7 +212,13 @@ mod tests {
176212 #[ test]
177213 fn test_partition_count_empty_plan ( ) {
178214 let schema = test_schema ( ) ;
179- let scan = PaimonTableScan :: new ( schema, dummy_table ( ) , None , vec ! [ Arc :: from( Vec :: new( ) ) ] ) ;
215+ let scan = PaimonTableScan :: new (
216+ schema,
217+ dummy_table ( ) ,
218+ None ,
219+ vec ! [ Arc :: from( Vec :: new( ) ) ] ,
220+ None ,
221+ ) ;
180222 assert_eq ! ( scan. properties( ) . output_partitioning( ) . partition_count( ) , 1 ) ;
181223 }
182224
@@ -188,7 +230,7 @@ mod tests {
188230 Arc :: from( Vec :: new( ) ) ,
189231 Arc :: from( Vec :: new( ) ) ,
190232 ] ;
191- let scan = PaimonTableScan :: new ( schema, dummy_table ( ) , None , planned_partitions) ;
233+ let scan = PaimonTableScan :: new ( schema, dummy_table ( ) , None , planned_partitions, None ) ;
192234 assert_eq ! ( scan. properties( ) . output_partitioning( ) . partition_count( ) , 3 ) ;
193235 }
194236
0 commit comments