13
13
// limitations under the License.
14
14
15
15
use std:: any:: Any ;
16
- use std:: collections:: VecDeque ;
17
16
use std:: str:: FromStr ;
18
17
use std:: sync:: Arc ;
19
18
@@ -30,15 +29,14 @@ use common_legacy_planners::TruncateTablePlan;
30
29
use common_meta_app:: schema:: TableInfo ;
31
30
use common_pipeline_core:: processors:: port:: InputPort ;
32
31
use common_pipeline_core:: SinkPipeBuilder ;
32
+ use common_pipeline_sources:: processors:: sources:: input_formats:: InputContext ;
33
33
use parking_lot:: Mutex ;
34
34
use tracing:: info;
35
35
36
36
use super :: StageSourceHelper ;
37
- use crate :: pipelines:: processors:: port:: OutputPort ;
38
37
use crate :: pipelines:: processors:: ContextSink ;
39
38
use crate :: pipelines:: processors:: TransformLimit ;
40
39
use crate :: pipelines:: Pipeline ;
41
- use crate :: pipelines:: SourcePipeBuilder ;
42
40
use crate :: sessions:: TableContext ;
43
41
use crate :: storages:: Table ;
44
42
@@ -48,6 +46,7 @@ pub struct StageTable {
48
46
// But the Table trait need it:
49
47
// fn get_table_info(&self) -> &TableInfo).
50
48
table_info_placeholder : TableInfo ,
49
+ input_context : Mutex < Option < Arc < InputContext > > > ,
51
50
}
52
51
53
52
impl StageTable {
@@ -57,8 +56,14 @@ impl StageTable {
57
56
Ok ( Arc :: new ( Self {
58
57
table_info,
59
58
table_info_placeholder,
59
+ input_context : Default :: default ( ) ,
60
60
} ) )
61
61
}
62
+
63
+ fn get_input_context ( & self ) -> Option < Arc < InputContext > > {
64
+ let guard = self . input_context . lock ( ) ;
65
+ guard. clone ( )
66
+ }
62
67
}
63
68
64
69
#[ async_trait:: async_trait]
@@ -74,39 +79,35 @@ impl Table for StageTable {
74
79
75
80
async fn read_partitions (
76
81
& self ,
77
- _ctx : Arc < dyn TableContext > ,
82
+ ctx : Arc < dyn TableContext > ,
78
83
_push_downs : Option < Extras > ,
79
84
) -> Result < ( Statistics , Partitions ) > {
85
+ let operator = StageSourceHelper :: get_op ( & ctx, & self . table_info . stage_info ) . await ?;
86
+ let input_ctx = Arc :: new (
87
+ InputContext :: try_create_from_copy (
88
+ operator,
89
+ ctx. get_settings ( ) . clone ( ) ,
90
+ ctx. get_format_settings ( ) ?,
91
+ self . table_info . schema . clone ( ) ,
92
+ self . table_info . stage_info . clone ( ) ,
93
+ self . table_info . files . clone ( ) ,
94
+ ctx. get_scan_progress ( )
95
+ )
96
+ . await ?,
97
+ ) ;
98
+ let mut guard = self . input_context . lock ( ) ;
99
+ * guard = Some ( input_ctx) ;
80
100
Ok ( ( Statistics :: default ( ) , vec ! [ ] ) )
81
101
}
82
102
83
103
fn read2 (
84
104
& self ,
85
- ctx : Arc < dyn TableContext > ,
105
+ _ctx : Arc < dyn TableContext > ,
86
106
_plan : & ReadDataSourcePlan ,
87
107
pipeline : & mut Pipeline ,
88
108
) -> Result < ( ) > {
89
- let settings = ctx. get_settings ( ) ;
90
- let mut builder = SourcePipeBuilder :: create ( ) ;
91
- let table_info = & self . table_info ;
92
- let schema = table_info. schema . clone ( ) ;
93
- let mut files_deque = VecDeque :: with_capacity ( table_info. files . len ( ) ) ;
94
- for f in & table_info. files {
95
- files_deque. push_back ( f. to_string ( ) ) ;
96
- }
97
- let files = Arc :: new ( Mutex :: new ( files_deque) ) ;
98
-
99
- let stage_source = StageSourceHelper :: try_create ( ctx, schema, table_info. clone ( ) , files) ?;
100
-
101
- for _index in 0 ..settings. get_max_threads ( ) ? {
102
- let output = OutputPort :: create ( ) ;
103
- builder. add_source ( output. clone ( ) , stage_source. get_splitter ( output) ?) ;
104
- }
105
- pipeline. add_pipe ( builder. finalize ( ) ) ;
106
-
107
- pipeline. add_transform ( |transform_input_port, transform_output_port| {
108
- stage_source. get_deserializer ( transform_input_port, transform_output_port)
109
- } ) ?;
109
+ let input_ctx = self . get_input_context ( ) . unwrap ( ) ;
110
+ input_ctx. format . exec_copy ( input_ctx. clone ( ) , pipeline) ?;
110
111
111
112
let limit = self . table_info . stage_info . copy_options . size_limit ;
112
113
if limit > 0 {
0 commit comments