@@ -26,14 +26,15 @@ use common_streams::DataBlockStream;
26
26
use common_streams:: SendableDataBlockStream ;
27
27
use futures:: TryStreamExt ;
28
28
use regex:: Regex ;
29
+ use common_base:: base:: { GlobalIORuntime , TrySpawn } ;
29
30
30
31
use super :: append2table;
31
- use super :: commit2table;
32
32
use crate :: interpreters:: Interpreter ;
33
+ use crate :: interpreters:: interpreter_common:: { execute_pipeline} ;
33
34
use crate :: interpreters:: SelectInterpreterV2 ;
34
35
use crate :: pipelines:: executor:: ExecutorSettings ;
35
36
use crate :: pipelines:: executor:: PipelineCompleteExecutor ;
36
- use crate :: pipelines:: Pipeline ;
37
+ use crate :: pipelines:: { Pipeline , PipelineBuildResult } ;
37
38
use crate :: sessions:: QueryContext ;
38
39
use crate :: sessions:: TableContext ;
39
40
use crate :: sql:: plans:: CopyPlanV2 ;
@@ -112,11 +113,11 @@ impl CopyInterpreterV2 {
112
113
}
113
114
}
114
115
115
- async fn purge_files ( & self , from : & ReadDataSourcePlan , files : & Vec < String > ) -> Result < ( ) > {
116
+ async fn purge_files ( ctx : Arc < QueryContext > , from : & ReadDataSourcePlan , files : & Vec < String > ) -> Result < ( ) > {
116
117
match & from. source_info {
117
118
SourceInfo :: StageSource ( table_info) => {
118
119
if table_info. stage_info . copy_options . purge {
119
- let rename_me: Arc < dyn TableContext > = self . ctx . clone ( ) ;
120
+ let rename_me: Arc < dyn TableContext > = ctx. clone ( ) ;
120
121
let op = StageSourceHelper :: get_op ( & rename_me, & table_info. stage_info ) . await ?;
121
122
for f in files {
122
123
if let Err ( e) = op. object ( f) . delete ( ) . await {
@@ -137,10 +138,10 @@ impl CopyInterpreterV2 {
137
138
/// Rewrite the ReadDataSourcePlan.S3StageSource.file_name to new file name.
138
139
fn rewrite_read_plan_file_name (
139
140
mut plan : ReadDataSourcePlan ,
140
- files : Vec < String > ,
141
+ files : & [ String ] ,
141
142
) -> ReadDataSourcePlan {
142
143
if let SourceInfo :: StageSource ( ref mut stage) = plan. source_info {
143
- stage. files = files
144
+ stage. files = files. to_vec ( )
144
145
}
145
146
plan
146
147
}
@@ -160,40 +161,64 @@ impl CopyInterpreterV2 {
160
161
tbl_name : & String ,
161
162
from : & ReadDataSourcePlan ,
162
163
files : Vec < String > ,
163
- ) -> Result < Vec < DataBlock > > {
164
- let ctx = self . ctx . clone ( ) ;
165
- let settings = self . ctx . get_settings ( ) ;
164
+ ) -> Result < PipelineBuildResult > {
165
+ let mut build_res = PipelineBuildResult :: create ( ) ;
166
166
167
- let mut pipeline = Pipeline :: create ( ) ;
168
- let read_source_plan = from. clone ( ) ;
169
- let read_source_plan = Self :: rewrite_read_plan_file_name ( read_source_plan, files) ;
167
+ let read_source_plan = Self :: rewrite_read_plan_file_name ( from. clone ( ) , & files) ;
170
168
tracing:: info!( "copy_files_to_table from source: {:?}" , read_source_plan) ;
171
- let table = ctx. build_table_from_source_plan ( & read_source_plan) ?;
172
- let res = table. read2 ( ctx. clone ( ) , & read_source_plan, & mut pipeline) ;
173
- if let Err ( e) = res {
174
- return Err ( e) ;
175
- }
176
169
177
- let table = ctx. get_table ( catalog_name, db_name, tbl_name) . await ?;
170
+ let from_table = self . ctx . build_table_from_source_plan ( & read_source_plan) ?;
171
+ from_table. read2 ( self . ctx . clone ( ) , & read_source_plan, & mut build_res. main_pipeline ) ?;
178
172
179
- table. append2 ( ctx. clone ( ) , & mut pipeline) ?;
180
- pipeline. set_max_threads ( settings. get_max_threads ( ) ? as usize ) ;
173
+ let to_table = self . ctx . get_table ( catalog_name, db_name, tbl_name) . await ?;
181
174
182
- let query_need_abort = ctx. query_need_abort ( ) ;
183
- let executor_settings = ExecutorSettings :: try_create ( & settings) ?;
184
- let executor =
185
- PipelineCompleteExecutor :: try_create ( query_need_abort, pipeline, executor_settings) ?;
186
- executor. execute ( ) ?;
175
+ to_table. append2 ( self . ctx . clone ( ) , & mut build_res. main_pipeline ) ?;
187
176
188
- Ok ( ctx. consume_precommit_blocks ( ) )
177
+ let ctx = self . ctx . clone ( ) ;
178
+ let catalog_name = catalog_name. clone ( ) ;
179
+ let files = files. clone ( ) ;
180
+ let from = from. clone ( ) ;
181
+
182
+ build_res. main_pipeline . set_on_finished ( move |may_error| {
183
+ if may_error. is_none ( ) {
184
+ // capture out variable
185
+ let ctx = ctx. clone ( ) ;
186
+ let files = files. clone ( ) ;
187
+ let from = from. clone ( ) ;
188
+ let catalog_name = catalog_name. clone ( ) ;
189
+ let to_table = to_table. clone ( ) ;
190
+
191
+ let task = GlobalIORuntime :: instance ( ) . spawn ( async move {
192
+ // Commit
193
+ let operations = ctx. consume_precommit_blocks ( ) ;
194
+ to_table. commit_insertion ( ctx. clone ( ) , & catalog_name, operations, false ) . await ?;
195
+
196
+ // Purge
197
+ CopyInterpreterV2 :: purge_files ( ctx, & from, & files) . await
198
+ } ) ;
199
+
200
+ return match futures:: executor:: block_on ( task) {
201
+ Ok ( Ok ( _) ) => Ok ( ( ) ) ,
202
+ Ok ( Err ( error) ) => Err ( error) ,
203
+ Err ( cause) => Err ( ErrorCode :: PanicError ( format ! (
204
+ "Maybe panic while in commit insert. {}" ,
205
+ cause
206
+ ) ) )
207
+ } ;
208
+ }
209
+
210
+ Err ( may_error. as_ref ( ) . unwrap ( ) . clone ( ) )
211
+ } ) ;
212
+
213
+ Ok ( build_res)
189
214
}
190
215
191
216
async fn execute_copy_into_stage (
192
217
& self ,
193
218
stage : & UserStageInfo ,
194
219
path : & str ,
195
220
query : & Plan ,
196
- ) -> Result < SendableDataBlockStream > {
221
+ ) -> Result < PipelineBuildResult > {
197
222
let ( s_expr, metadata, bind_context) = match query {
198
223
Plan :: Query {
199
224
s_expr,
@@ -231,22 +256,11 @@ impl CopyInterpreterV2 {
231
256
files : vec ! [ ] ,
232
257
} ;
233
258
234
- let build_res = select_interpreter. create_new_pipeline ( ) . await ?;
259
+ let mut build_res = select_interpreter. create_new_pipeline ( ) . await ?;
235
260
let table = StageTable :: try_create ( stage_table_info) ?;
236
261
237
- append2table (
238
- self . ctx . clone ( ) ,
239
- table. clone ( ) ,
240
- data_schema. clone ( ) ,
241
- build_res,
242
- ) ?;
243
- commit2table ( self . ctx . clone ( ) , table. clone ( ) , false ) . await ?;
244
-
245
- Ok ( Box :: pin ( DataBlockStream :: create (
246
- Arc :: new ( DataSchema :: empty ( ) ) ,
247
- None ,
248
- vec ! [ ] ,
249
- ) ) )
262
+ append2table ( self . ctx . clone ( ) , table. clone ( ) , data_schema. clone ( ) , & mut build_res, false , true ) ?;
263
+ Ok ( build_res)
250
264
}
251
265
}
252
266
@@ -258,6 +272,30 @@ impl Interpreter for CopyInterpreterV2 {
258
272
259
273
#[ tracing:: instrument( level = "debug" , name = "copy_interpreter_execute_v2" , skip( self ) , fields( ctx. id = self . ctx. get_id( ) . as_str( ) ) ) ]
260
274
async fn execute ( & self ) -> Result < SendableDataBlockStream > {
275
+ let build_res = self . create_new_pipeline ( ) . await ?;
276
+
277
+ let settings = self . ctx . get_settings ( ) ;
278
+ let query_need_abort = self . ctx . query_need_abort ( ) ;
279
+ let executor_settings = ExecutorSettings :: try_create ( & settings) ?;
280
+
281
+ let mut pipelines = build_res. sources_pipelines ;
282
+ pipelines. push ( build_res. main_pipeline ) ;
283
+
284
+ let executor = PipelineCompleteExecutor :: from_pipelines (
285
+ query_need_abort,
286
+ pipelines,
287
+ executor_settings,
288
+ ) ?;
289
+
290
+ executor. execute ( ) ?;
291
+ Ok ( Box :: pin ( DataBlockStream :: create (
292
+ Arc :: new ( DataSchema :: new ( vec ! [ ] ) ) ,
293
+ None ,
294
+ vec ! [ ] ,
295
+ ) ) )
296
+ }
297
+
298
+ async fn create_new_pipeline ( & self ) -> Result < PipelineBuildResult > {
261
299
match & self . plan {
262
300
// TODO(xuanwo): extract them as a separate function.
263
301
CopyPlanV2 :: IntoTable {
@@ -290,40 +328,9 @@ impl Interpreter for CopyInterpreterV2 {
290
328
}
291
329
292
330
tracing:: info!( "matched files: {:?}, pattern: {}" , & files, pattern) ;
293
-
294
- let write_results = self
295
- . copy_files_to_table (
296
- catalog_name,
297
- database_name,
298
- table_name,
299
- from,
300
- files. clone ( ) ,
301
- )
302
- . await ?;
303
-
304
- let table = self
305
- . ctx
306
- . get_table ( catalog_name, database_name, table_name)
307
- . await ?;
308
-
309
- // Commit.
310
- table
311
- . commit_insertion ( self . ctx . clone ( ) , catalog_name, write_results, false )
312
- . await ?;
313
-
314
- // Purge
315
- self . purge_files ( from, & files) . await ?;
316
-
317
- Ok ( Box :: pin ( DataBlockStream :: create (
318
- // TODO(xuanwo): Is this correct?
319
- Arc :: new ( DataSchema :: new ( vec ! [ ] ) ) ,
320
- None ,
321
- vec ! [ ] ,
322
- ) ) )
331
+ self . copy_files_to_table ( catalog_name, database_name, table_name, from, files. clone ( ) ) . await
323
332
}
324
- CopyPlanV2 :: IntoStage {
325
- stage, from, path, ..
326
- } => self . execute_copy_into_stage ( stage, path, from) . await ,
333
+ CopyPlanV2 :: IntoStage { stage, from, path, .. } => self . execute_copy_into_stage ( stage, path, from) . await ,
327
334
}
328
335
}
329
336
}
0 commit comments