@@ -25,15 +25,13 @@ use databend_common_expression::types::DataType;
2525use databend_common_expression:: types:: NumberDataType ;
2626use databend_common_expression:: BlockEntry ;
2727use databend_common_expression:: Column ;
28- use databend_common_expression:: ColumnBuilder ;
2928use databend_common_expression:: DataBlock ;
30- use databend_common_expression:: DataSchema ;
3129use databend_common_expression:: Value ;
3230use databend_common_pipeline_core:: processors:: InputPort ;
3331use databend_common_pipeline_core:: processors:: OutputPort ;
3432use databend_common_pipeline_core:: processors:: ProcessorPtr ;
35- use databend_common_pipeline_transforms:: processors :: AsyncTransform ;
36- use databend_common_pipeline_transforms:: processors :: AsyncTransformer ;
33+ use databend_common_pipeline_transforms:: AsyncAccumulatingTransform ;
34+ use databend_common_pipeline_transforms:: AsyncAccumulatingTransformer ;
3735use databend_storages_common_io:: ReadSettings ;
3836
3937use super :: native_rows_fetcher:: NativeRowsFetcher ;
@@ -132,17 +130,17 @@ pub fn row_fetch_processor(
132130pub trait RowsFetcher {
133131 async fn on_start ( & mut self ) -> Result < ( ) > ;
134132 async fn fetch ( & mut self , row_ids : & [ u64 ] ) -> Result < DataBlock > ;
135- fn schema ( & self ) -> DataSchema ;
136133}
137134
138135pub struct TransformRowsFetcher < F : RowsFetcher > {
139136 row_id_col_offset : usize ,
140137 fetcher : F ,
141138 need_wrap_nullable : bool ,
139+ blocks : Vec < DataBlock > ,
142140}
143141
144142#[ async_trait:: async_trait]
145- impl < F > AsyncTransform for TransformRowsFetcher < F >
143+ impl < F > AsyncAccumulatingTransform for TransformRowsFetcher < F >
146144where F : RowsFetcher + Send + Sync + ' static
147145{
148146 const NAME : & ' static str = "TransformRowsFetcher" ;
@@ -152,18 +150,25 @@ where F: RowsFetcher + Send + Sync + 'static
152150 self . fetcher . on_start ( ) . await
153151 }
154152
153+ async fn transform ( & mut self , data : DataBlock ) -> Result < Option < DataBlock > > {
154+ self . blocks . push ( data) ;
155+ Ok ( None )
156+ }
157+
155158 #[ async_backtrace:: framed]
156- async fn transform ( & mut self , mut data : DataBlock ) -> Result < DataBlock > {
159+ async fn on_finish ( & mut self , _output : bool ) -> Result < Option < DataBlock > > {
160+ if self . blocks . is_empty ( ) {
161+ return Ok ( None ) ;
162+ }
163+
164+ let start_time = std:: time:: Instant :: now ( ) ;
165+ let num_blocks = self . blocks . len ( ) ;
166+ let mut data = DataBlock :: concat ( & self . blocks ) ?;
167+ self . blocks . clear ( ) ;
168+
157169 let num_rows = data. num_rows ( ) ;
158170 if num_rows == 0 {
159- // Although the data block is empty, we need to add empty columns to align the schema.
160- let fetched_schema = self . fetcher . schema ( ) ;
161- for f in fetched_schema. fields ( ) . iter ( ) {
162- let builder = ColumnBuilder :: with_capacity ( f. data_type ( ) , 0 ) ;
163- let col = builder. build ( ) ;
164- data. add_column ( BlockEntry :: new ( f. data_type ( ) . clone ( ) , Value :: Column ( col) ) ) ;
165- }
166- return Ok ( data) ;
171+ return Ok ( None ) ;
167172 }
168173
169174 let entry = & data. columns ( ) [ self . row_id_col_offset ] ;
@@ -189,7 +194,14 @@ where F: RowsFetcher + Send + Sync + 'static
189194 }
190195 }
191196
192- Ok ( data)
197+ log:: info!(
198+ "TransformRowsFetcher on_finish: num_rows: {}, input blocks: {} in {} milliseconds" ,
199+ num_rows,
200+ num_blocks,
201+ start_time. elapsed( ) . as_millis( )
202+ ) ;
203+
204+ Ok ( Some ( data) )
193205 }
194206}
195207
@@ -203,10 +215,11 @@ where F: RowsFetcher + Send + Sync + 'static
203215 fetcher : F ,
204216 need_wrap_nullable : bool ,
205217 ) -> ProcessorPtr {
206- ProcessorPtr :: create ( AsyncTransformer :: create ( input, output, Self {
218+ ProcessorPtr :: create ( AsyncAccumulatingTransformer :: create ( input, output, Self {
207219 row_id_col_offset,
208220 fetcher,
209221 need_wrap_nullable,
222+ blocks : vec ! [ ] ,
210223 } ) )
211224 }
212225}
0 commit comments