@@ -24,6 +24,7 @@ use context::*;
2424mod task;
2525
2626use std:: sync:: Arc ;
27+ use std:: sync:: atomic:: { AtomicUsize , Ordering } ;
2728
2829use arrow_array:: RecordBatch ;
2930use futures:: channel:: mpsc:: { Sender , channel} ;
@@ -64,6 +65,8 @@ pub struct TableScanBuilder<'a> {
6465 // is still being worked on but will switch to a default of true
6566 // once this work is complete
6667 delete_file_processing_enabled : bool ,
68+
69+ limit : Option < usize > ,
6770}
6871
6972impl < ' a > TableScanBuilder < ' a > {
@@ -83,9 +86,16 @@ impl<'a> TableScanBuilder<'a> {
8386 row_group_filtering_enabled : true ,
8487 row_selection_enabled : false ,
8588 delete_file_processing_enabled : false ,
89+ limit : None ,
8690 }
8791 }
8892
93+ /// Sets the maximum number of records to return
94+ pub fn with_limit ( mut self , limit : Option < usize > ) -> Self {
95+ self . limit = limit;
96+ self
97+ }
98+
8999 /// Sets the desired size of batches in the response
90100 /// to something other than the default
91101 pub fn with_batch_size ( mut self , batch_size : Option < usize > ) -> Self {
@@ -299,6 +309,7 @@ impl<'a> TableScanBuilder<'a> {
299309 snapshot_schema : schema,
300310 case_sensitive : self . case_sensitive ,
301311 predicate : self . filter . map ( Arc :: new) ,
312+ limit : self . limit ,
302313 snapshot_bound_predicate : snapshot_bound_predicate. map ( Arc :: new) ,
303314 object_cache : self . table . object_cache ( ) ,
304315 field_ids : Arc :: new ( field_ids) ,
@@ -1441,6 +1452,83 @@ pub mod tests {
14411452 assert_eq ! ( int64_arr. value( 0 ) , 2 ) ;
14421453 }
14431454
1455+ #[ tokio:: test]
1456+ async fn test_limit ( ) {
1457+ let mut fixture = TableTestFixture :: new ( ) ;
1458+ fixture. setup_manifest_files ( ) . await ;
1459+
1460+ let mut builder = fixture. table . scan ( ) ;
1461+ builder = builder. with_limit ( Some ( 1 ) ) ;
1462+ let table_scan = builder. build ( ) . unwrap ( ) ;
1463+
1464+ let batch_stream = table_scan. to_arrow ( ) . await . unwrap ( ) ;
1465+
1466+ let batches: Vec < _ > = batch_stream. try_collect ( ) . await . unwrap ( ) ;
1467+
1468+ assert_eq ! ( batches. len( ) , 2 ) ;
1469+ assert_eq ! ( batches[ 0 ] . num_rows( ) , 1 ) ;
1470+ assert_eq ! ( batches[ 1 ] . num_rows( ) , 1 ) ;
1471+
1472+ let col = batches[ 0 ] . column_by_name ( "x" ) . unwrap ( ) ;
1473+ let int64_arr = col. as_any ( ) . downcast_ref :: < Int64Array > ( ) . unwrap ( ) ;
1474+ assert_eq ! ( int64_arr. value( 0 ) , 1 ) ;
1475+
1476+ let col = batches[ 0 ] . column_by_name ( "y" ) . unwrap ( ) ;
1477+ let int64_arr = col. as_any ( ) . downcast_ref :: < Int64Array > ( ) . unwrap ( ) ;
1478+ assert_eq ! ( int64_arr. value( 0 ) , 2 ) ;
1479+ }
1480+
1481+ #[ tokio:: test]
1482+ async fn test_limit_with_predicate ( ) {
1483+ let mut fixture = TableTestFixture :: new ( ) ;
1484+ fixture. setup_manifest_files ( ) . await ;
1485+
1486+ // Filter: y < 3
1487+ let mut builder = fixture. table . scan ( ) ;
1488+ let predicate = Reference :: new ( "y" ) . greater_than ( Datum :: long ( 3 ) ) ;
1489+ builder = builder. with_filter ( predicate) . with_limit ( Some ( 1 ) ) ;
1490+ let table_scan = builder. build ( ) . unwrap ( ) ;
1491+
1492+ let batch_stream = table_scan. to_arrow ( ) . await . unwrap ( ) ;
1493+
1494+ let batches: Vec < _ > = batch_stream. try_collect ( ) . await . unwrap ( ) ;
1495+
1496+ assert_eq ! ( batches. len( ) , 2 ) ;
1497+ assert_eq ! ( batches[ 0 ] . num_rows( ) , 1 ) ;
1498+ assert_eq ! ( batches[ 1 ] . num_rows( ) , 1 ) ;
1499+ }
1500+
1501+ #[ tokio:: test]
1502+ async fn test_limit_with_predicate_and_row_selection ( ) {
1503+ let mut fixture = TableTestFixture :: new ( ) ;
1504+ fixture. setup_manifest_files ( ) . await ;
1505+
1506+ // Filter: y < 3
1507+ let mut builder = fixture. table . scan ( ) ;
1508+ let predicate = Reference :: new ( "y" ) . greater_than ( Datum :: long ( 3 ) ) ;
1509+ builder = builder
1510+ . with_filter ( predicate)
1511+ . with_limit ( Some ( 1 ) )
1512+ . with_row_selection_enabled ( true ) ;
1513+ let table_scan = builder. build ( ) . unwrap ( ) ;
1514+
1515+ let batch_stream = table_scan. to_arrow ( ) . await . unwrap ( ) ;
1516+
1517+ let batches: Vec < _ > = batch_stream. try_collect ( ) . await . unwrap ( ) ;
1518+
1519+ assert_eq ! ( batches. len( ) , 2 ) ;
1520+ assert_eq ! ( batches[ 0 ] . num_rows( ) , 1 ) ;
1521+ assert_eq ! ( batches[ 1 ] . num_rows( ) , 1 ) ;
1522+
1523+ // let col = batches[0].column_by_name("x").unwrap();
1524+ // let int64_arr = col.as_any().downcast_ref::<Int64Array>().unwrap();
1525+ // assert_eq!(int64_arr.value(0), 1);
1526+ //
1527+ // let col = batches[0].column_by_name("y").unwrap();
1528+ // let int64_arr = col.as_any().downcast_ref::<Int64Array>().unwrap();
1529+ // assert_eq!(int64_arr.value(0), 2);
1530+ }
1531+
14441532 #[ tokio:: test]
14451533 async fn test_filter_on_arrow_gt_eq ( ) {
14461534 let mut fixture = TableTestFixture :: new ( ) ;
@@ -1816,6 +1904,7 @@ pub mod tests {
18161904 record_count : Some ( 100 ) ,
18171905 data_file_format : DataFileFormat :: Parquet ,
18181906 deletes : vec ! [ ] ,
1907+ limit : None ,
18191908 } ;
18201909 test_fn ( task) ;
18211910
@@ -1831,6 +1920,7 @@ pub mod tests {
18311920 record_count : None ,
18321921 data_file_format : DataFileFormat :: Avro ,
18331922 deletes : vec ! [ ] ,
1923+ limit : None ,
18341924 } ;
18351925 test_fn ( task) ;
18361926 }
0 commit comments