@@ -36,6 +36,8 @@ use crate::pipelines::executor::executor_worker_context::ExecutorWorkerContext;
36
36
use crate :: pipelines:: executor:: ExecutorSettings ;
37
37
use crate :: pipelines:: pipeline:: Pipeline ;
38
38
39
+ pub type InitCallback = Arc < Box < dyn Fn ( ) -> Result < ( ) > + Send + Sync + ' static > > ;
40
+
39
41
pub type FinishedCallback =
40
42
Arc < Box < dyn Fn ( & Option < ErrorCode > ) -> Result < ( ) > + Send + Sync + ' static > > ;
41
43
@@ -45,6 +47,7 @@ pub struct PipelineExecutor {
45
47
workers_condvar : Arc < WorkersCondvar > ,
46
48
pub async_runtime : Arc < Runtime > ,
47
49
pub global_tasks_queue : Arc < ExecutorTasksQueue > ,
50
+ on_init_callback : InitCallback ,
48
51
on_finished_callback : FinishedCallback ,
49
52
settings : ExecutorSettings ,
50
53
finished_notify : Notify ,
@@ -57,12 +60,14 @@ impl PipelineExecutor {
57
60
settings : ExecutorSettings ,
58
61
) -> Result < Arc < PipelineExecutor > > {
59
62
let threads_num = pipeline. get_max_threads ( ) ;
63
+ let on_init_callback = pipeline. take_on_init ( ) ;
60
64
let on_finished_callback = pipeline. take_on_finished ( ) ;
61
65
62
66
assert_ne ! ( threads_num, 0 , "Pipeline max threads cannot equals zero." ) ;
63
67
Self :: try_create (
64
68
RunningGraph :: create ( pipeline) ?,
65
69
threads_num,
70
+ on_init_callback,
66
71
on_finished_callback,
67
72
settings,
68
73
)
@@ -82,6 +87,11 @@ impl PipelineExecutor {
82
87
. max ( )
83
88
. unwrap_or ( 0 ) ;
84
89
90
+ let on_init_callbacks = pipelines
91
+ . iter_mut ( )
92
+ . map ( |x| x. take_on_init ( ) )
93
+ . collect :: < Vec < _ > > ( ) ;
94
+
85
95
let on_finished_callbacks = pipelines
86
96
. iter_mut ( )
87
97
. map ( |x| x. take_on_finished ( ) )
@@ -91,6 +101,13 @@ impl PipelineExecutor {
91
101
Self :: try_create (
92
102
RunningGraph :: from_pipelines ( pipelines) ?,
93
103
threads_num,
104
+ Arc :: new ( Box :: new ( move || {
105
+ for on_init_callback in & on_init_callbacks {
106
+ on_init_callback ( ) ?;
107
+ }
108
+
109
+ Ok ( ( ) )
110
+ } ) ) ,
94
111
Arc :: new ( Box :: new ( move |may_error| {
95
112
for on_finished_callback in & on_finished_callbacks {
96
113
on_finished_callback ( may_error) ?;
@@ -105,33 +122,25 @@ impl PipelineExecutor {
105
122
fn try_create (
106
123
graph : RunningGraph ,
107
124
threads_num : usize ,
125
+ on_init_callback : InitCallback ,
108
126
on_finished_callback : FinishedCallback ,
109
127
settings : ExecutorSettings ,
110
128
) -> Result < Arc < PipelineExecutor > > {
111
- unsafe {
112
- let workers_condvar = WorkersCondvar :: create ( threads_num) ;
113
- let global_tasks_queue = ExecutorTasksQueue :: create ( threads_num) ;
114
-
115
- let mut init_schedule_queue = graph. init_schedule_queue ( ) ?;
129
+ let workers_condvar = WorkersCondvar :: create ( threads_num) ;
130
+ let global_tasks_queue = ExecutorTasksQueue :: create ( threads_num) ;
116
131
117
- let mut tasks = VecDeque :: new ( ) ;
118
- while let Some ( task) = init_schedule_queue. pop_task ( ) {
119
- tasks. push_back ( task) ;
120
- }
121
- global_tasks_queue. init_tasks ( tasks) ;
122
-
123
- Ok ( Arc :: new ( PipelineExecutor {
124
- graph,
125
- threads_num,
126
- workers_condvar,
127
- global_tasks_queue,
128
- on_finished_callback,
129
- async_runtime : GlobalIORuntime :: instance ( ) ,
130
- settings,
131
- finished_notify : Notify :: new ( ) ,
132
- finished_error : Mutex :: new ( None ) ,
133
- } ) )
134
- }
132
+ Ok ( Arc :: new ( PipelineExecutor {
133
+ graph,
134
+ threads_num,
135
+ workers_condvar,
136
+ global_tasks_queue,
137
+ on_init_callback,
138
+ on_finished_callback,
139
+ async_runtime : GlobalIORuntime :: instance ( ) ,
140
+ settings,
141
+ finished_notify : Notify :: new ( ) ,
142
+ finished_error : Mutex :: new ( None ) ,
143
+ } ) )
135
144
}
136
145
137
146
pub fn finish ( & self , cause : Option < ErrorCode > ) {
@@ -145,6 +154,8 @@ impl PipelineExecutor {
145
154
}
146
155
147
156
pub fn execute ( self : & Arc < Self > ) -> Result < ( ) > {
157
+ self . init ( ) ?;
158
+
148
159
self . start_executor_daemon ( ) ?;
149
160
150
161
let mut thread_join_handles = self . execute_threads ( self . threads_num ) ;
@@ -171,6 +182,24 @@ impl PipelineExecutor {
171
182
Ok ( ( ) )
172
183
}
173
184
185
+ fn init ( self : & Arc < Self > ) -> Result < ( ) > {
186
+ unsafe {
187
+ // TODO: the on init callback cannot be killed.
188
+ ( self . on_init_callback ) ( ) ?;
189
+
190
+ let mut init_schedule_queue = self . graph . init_schedule_queue ( ) ?;
191
+
192
+ let mut tasks = VecDeque :: new ( ) ;
193
+ while let Some ( task) = init_schedule_queue. pop_task ( ) {
194
+ tasks. push_back ( task) ;
195
+ }
196
+
197
+ self . global_tasks_queue . init_tasks ( tasks) ;
198
+
199
+ Ok ( ( ) )
200
+ }
201
+ }
202
+
174
203
fn start_executor_daemon ( self : & Arc < Self > ) -> Result < ( ) > {
175
204
if !self . settings . max_execute_time . is_zero ( ) {
176
205
let this = self . clone ( ) ;
0 commit comments