kapot_executor/
executor_server.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

use kapot_core::KAPOT_VERSION;
use std::collections::HashMap;
use std::convert::TryInto;
use std::path::{Path, PathBuf};
use std::sync::atomic::{AtomicBool, Ordering};
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use tokio::sync::mpsc;

use log::{debug, error, info, warn};
use tonic::transport::Channel;
use tonic::{Request, Response, Status};

use kapot_core::config::KAPOT_DATA_CACHE_ENABLED;
use kapot_core::error::KapotError;
use kapot_core::serde::protobuf::{
    executor_grpc_server::{ExecutorGrpc, ExecutorGrpcServer},
    executor_metric, executor_status,
    scheduler_grpc_client::SchedulerGrpcClient,
    CancelTasksParams, CancelTasksResult, ExecutorMetric, ExecutorStatus,
    HeartBeatParams, LaunchMultiTaskParams, LaunchMultiTaskResult, LaunchTaskParams,
    LaunchTaskResult, RegisterExecutorParams, RemoveJobDataParams, RemoveJobDataResult,
    StopExecutorParams, StopExecutorResult, TaskStatus, UpdateTaskStatusParams,
};
use kapot_core::serde::scheduler::from_proto::{
    get_task_definition, get_task_definition_vec,
};
use kapot_core::serde::scheduler::PartitionId;
use kapot_core::serde::scheduler::TaskDefinition;
use kapot_core::serde::KapotCodec;
use kapot_core::utils::{create_grpc_client_connection, create_grpc_server};
use dashmap::DashMap;
use datafusion::config::ConfigOptions;
use datafusion::execution::TaskContext;
use datafusion::prelude::SessionConfig;
use datafusion_proto::{logical_plan::AsLogicalPlan, physical_plan::AsExecutionPlan};
use tokio::sync::mpsc::error::TryRecvError;
use tokio::task::JoinHandle;

use crate::cpu_bound_executor::DedicatedExecutor;
use crate::executor::Executor;
use crate::executor_process::ExecutorProcessConfig;
use crate::shutdown::ShutdownNotifier;
use crate::{as_task_status, TaskExecutionTimes};

type ServerHandle = JoinHandle<Result<(), KapotError>>;
type SchedulerClients = Arc<DashMap<String, SchedulerGrpcClient<Channel>>>;

/// Wrap TaskDefinition with its curator scheduler id for task update to its specific curator scheduler later
#[derive(Debug)]
struct CuratorTaskDefinition {
    scheduler_id: String,
    task: TaskDefinition,
}

/// Wrap TaskStatus with its curator scheduler id for task update to its specific curator scheduler later
#[derive(Debug)]
struct CuratorTaskStatus {
    scheduler_id: String,
    task_status: TaskStatus,
}

pub async fn startup<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan>(
    mut scheduler: SchedulerGrpcClient<Channel>,
    config: Arc<ExecutorProcessConfig>,
    executor: Arc<Executor>,
    codec: KapotCodec<T, U>,
    stop_send: mpsc::Sender<bool>,
    shutdown_noti: &ShutdownNotifier,
) -> Result<ServerHandle, KapotError> {
    let channel_buf_size = executor.concurrent_tasks * 50;
    let (tx_task, rx_task) = mpsc::channel::<CuratorTaskDefinition>(channel_buf_size);
    let (tx_task_status, rx_task_status) =
        mpsc::channel::<CuratorTaskStatus>(channel_buf_size);

    let executor_server = ExecutorServer::new(
        scheduler.clone(),
        executor.clone(),
        ExecutorEnv {
            tx_task,
            tx_task_status,
            tx_stop: stop_send,
        },
        codec,
        config.grpc_max_encoding_message_size as usize,
        config.grpc_max_decoding_message_size as usize,
    );

    // 1. Start executor grpc service
    let server = {
        let executor_meta = executor.metadata.clone();
        let addr = format!("{}:{}", config.bind_host, executor_meta.grpc_port);
        let addr = addr.parse().unwrap();

        info!(
            "kapot v{} Rust Executor Grpc Server listening on {:?}",
            KAPOT_VERSION, addr
        );
        let server = ExecutorGrpcServer::new(executor_server.clone())
            .max_encoding_message_size(config.grpc_max_encoding_message_size as usize)
            .max_decoding_message_size(config.grpc_max_decoding_message_size as usize);
        let mut grpc_shutdown = shutdown_noti.subscribe_for_shutdown();
        tokio::spawn(async move {
            let shutdown_signal = grpc_shutdown.recv();
            let grpc_server_future = create_grpc_server()
                .add_service(server)
                .serve_with_shutdown(addr, shutdown_signal);
            grpc_server_future.await.map_err(|e| {
                error!("Tonic error, Could not start Executor Grpc Server.");
                KapotError::TonicError(e)
            })
        })
    };

    // 2. Do executor registration
    // TODO the executor registration should happen only after the executor grpc server started.
    let executor_server = Arc::new(executor_server);
    match register_executor(&mut scheduler, executor.clone()).await {
        Ok(_) => {
            info!("Executor registration succeed");
        }
        Err(error) => {
            error!("Executor registration failed due to: {}", error);
            // abort the Executor Grpc Future
            server.abort();
            return Err(error);
        }
    };

    // 3. Start Heartbeater loop
    {
        let heartbeater = Heartbeater::new(executor_server.clone());
        heartbeater.start(shutdown_noti, config.executor_heartbeat_interval_seconds);
    }

    // 4. Start TaskRunnerPool loop
    {
        let task_runner_pool = TaskRunnerPool::new(executor_server.clone());
        task_runner_pool.start(rx_task, rx_task_status, shutdown_noti);
    }

    Ok(server)
}

#[allow(clippy::clone_on_copy)]
async fn register_executor(
    scheduler: &mut SchedulerGrpcClient<Channel>,
    executor: Arc<Executor>,
) -> Result<(), KapotError> {
    let result = scheduler
        .register_executor(RegisterExecutorParams {
            metadata: Some(executor.metadata.clone()),
        })
        .await?;
    if result.into_inner().success {
        Ok(())
    } else {
        Err(KapotError::General(
            "Executor registration failed!!!".to_owned(),
        ))
    }
}

#[derive(Clone)]
pub struct ExecutorServer<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
    _start_time: u128,
    executor: Arc<Executor>,
    executor_env: ExecutorEnv,
    codec: KapotCodec<T, U>,
    scheduler_to_register: SchedulerGrpcClient<Channel>,
    schedulers: SchedulerClients,
    grpc_max_encoding_message_size: usize,
    grpc_max_decoding_message_size: usize,
}

#[derive(Clone)]
struct ExecutorEnv {
    /// Receive `TaskDefinition` from rpc then send to CPU bound tasks pool `dedicated_executor`.
    tx_task: mpsc::Sender<CuratorTaskDefinition>,
    /// Receive `TaskStatus` from CPU bound tasks pool `dedicated_executor` then use rpc send back to scheduler.
    tx_task_status: mpsc::Sender<CuratorTaskStatus>,
    /// Receive stop executor request from rpc.
    tx_stop: mpsc::Sender<bool>,
}

unsafe impl Sync for ExecutorEnv {}

/// Global flag indicating whether the executor is terminating. This should be
/// set to `true` when the executor receives a shutdown signal
pub static TERMINATING: AtomicBool = AtomicBool::new(false);

impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> ExecutorServer<T, U> {
    fn new(
        scheduler_to_register: SchedulerGrpcClient<Channel>,
        executor: Arc<Executor>,
        executor_env: ExecutorEnv,
        codec: KapotCodec<T, U>,
        grpc_max_encoding_message_size: usize,
        grpc_max_decoding_message_size: usize,
    ) -> Self {
        Self {
            _start_time: SystemTime::now()
                .duration_since(UNIX_EPOCH)
                .unwrap()
                .as_millis(),
            executor,
            executor_env,
            codec,
            scheduler_to_register,
            schedulers: Default::default(),
            grpc_max_encoding_message_size,
            grpc_max_decoding_message_size,
        }
    }

    async fn get_scheduler_client(
        &self,
        scheduler_id: &str,
    ) -> Result<SchedulerGrpcClient<Channel>, KapotError> {
        let scheduler = self.schedulers.get(scheduler_id).map(|value| value.clone());
        // If channel does not exist, create a new one
        if let Some(scheduler) = scheduler {
            Ok(scheduler)
        } else {
            let scheduler_url = format!("http://{scheduler_id}");
            let connection = create_grpc_client_connection(scheduler_url).await?;
            let scheduler = SchedulerGrpcClient::new(connection)
                .max_encoding_message_size(self.grpc_max_encoding_message_size)
                .max_decoding_message_size(self.grpc_max_decoding_message_size);

            {
                self.schedulers
                    .insert(scheduler_id.to_owned(), scheduler.clone());
            }

            Ok(scheduler)
        }
    }

    /// 1. First Heartbeat to its registration scheduler, if successful then return; else go next.
    /// 2. Heartbeat to schedulers which has launching tasks to this executor until one succeeds
    async fn heartbeat(&self) {
        let status = if TERMINATING.load(Ordering::Acquire) {
            executor_status::Status::Terminating(String::default())
        } else {
            executor_status::Status::Active(String::default())
        };

        let heartbeat_params = HeartBeatParams {
            executor_id: self.executor.metadata.id.clone(),
            metrics: self.get_executor_metrics(),
            status: Some(ExecutorStatus {
                status: Some(status),
            }),
            metadata: Some(self.executor.metadata.clone()),
        };
        let mut scheduler = self.scheduler_to_register.clone();
        match scheduler
            .heart_beat_from_executor(heartbeat_params.clone())
            .await
        {
            Ok(_) => {
                return;
            }
            Err(e) => {
                warn!(
                    "Fail to update heartbeat to its registration scheduler due to {:?}",
                    e
                );
            }
        };

        for mut item in self.schedulers.iter_mut() {
            let scheduler_id = item.key().clone();
            let scheduler = item.value_mut();

            match scheduler
                .heart_beat_from_executor(heartbeat_params.clone())
                .await
            {
                Ok(_) => {
                    break;
                }
                Err(e) => {
                    warn!(
                        "Fail to update heartbeat to scheduler {} due to {:?}",
                        scheduler_id, e
                    );
                }
            }
        }
    }

    /// This method should not return Err. If task fails, a failure task status should be sent
    /// to the channel to notify the scheduler.
    async fn run_task(&self, task_identity: String, curator_task: CuratorTaskDefinition) {
        let start_exec_time = SystemTime::now()
            .duration_since(UNIX_EPOCH)
            .unwrap()
            .as_millis() as u64;
        info!("Start to run task {}", task_identity);
        let task = curator_task.task;

        let task_id = task.task_id;
        let job_id = task.job_id;
        let stage_id = task.stage_id;
        let stage_attempt_num = task.stage_attempt_num;
        let partition_id = task.partition_id;
        let plan = task.plan;

        let part = PartitionId {
            job_id: job_id.clone(),
            stage_id,
            partition_id,
        };

        let query_stage_exec = self
            .executor
            .execution_engine
            .create_query_stage_exec(
                job_id.clone(),
                stage_id,
                plan,
                &self.executor.work_dir,
            )
            .unwrap();

        let task_context = {
            let task_props = task.props;
            let data_cache = task_props
                .get(KAPOT_DATA_CACHE_ENABLED)
                .map(|data_cache| data_cache.parse().unwrap_or(false))
                .unwrap_or(false);
            let mut config = ConfigOptions::new();
            for (k, v) in task_props.iter() {
                if let Err(e) = config.set(k, v) {
                    debug!("Fail to set session config for ({},{}): {:?}", k, v, e);
                }
            }
            let session_config = SessionConfig::from(config);

            let function_registry = task.function_registry;
            if data_cache {
                info!("Data cache will be enabled for {}", task_identity);
            }
            let runtime = self.executor.get_runtime(data_cache);

            Arc::new(TaskContext::new(
                Some(task_identity.clone()),
                task.session_id,
                session_config,
                function_registry.scalar_functions.clone(),
                function_registry.aggregate_functions.clone(),
                function_registry.window_functions.clone(),
                runtime,
            ))
        };

        info!("Start to execute shuffle write for task {}", task_identity);

        let execution_result = self
            .executor
            .execute_query_stage(
                task_id,
                part.clone(),
                query_stage_exec.clone(),
                task_context,
            )
            .await;
        info!("Done with task {}", task_identity);
        debug!("Statistics: {:?}", execution_result);

        let plan_metrics = query_stage_exec.collect_plan_metrics();
        let operator_metrics = match plan_metrics
            .into_iter()
            .map(|m| m.try_into())
            .collect::<Result<Vec<_>, KapotError>>()
        {
            Ok(metrics) => Some(metrics),
            Err(_) => None,
        };
        let executor_id = &self.executor.metadata.id;

        let end_exec_time = SystemTime::now()
            .duration_since(UNIX_EPOCH)
            .unwrap()
            .as_millis() as u64;
        let task_execution_times = TaskExecutionTimes {
            launch_time: task.launch_time,
            start_exec_time,
            end_exec_time,
        };

        let task_status = as_task_status(
            execution_result,
            executor_id.clone(),
            task_id,
            stage_attempt_num,
            part,
            operator_metrics,
            task_execution_times,
        );

        let scheduler_id = curator_task.scheduler_id;
        let task_status_sender = self.executor_env.tx_task_status.clone();
        task_status_sender
            .send(CuratorTaskStatus {
                scheduler_id,
                task_status,
            })
            .await
            .unwrap();
    }

    // TODO populate with real metrics
    fn get_executor_metrics(&self) -> Vec<ExecutorMetric> {
        let available_memory = ExecutorMetric {
            metric: Some(executor_metric::Metric::AvailableMemory(u64::MAX)),
        };
        let executor_metrics = vec![available_memory];
        executor_metrics
    }
}

/// Heartbeater will run forever until a shutdown notification received.
struct Heartbeater<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
    executor_server: Arc<ExecutorServer<T, U>>,
}

impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> Heartbeater<T, U> {
    fn new(executor_server: Arc<ExecutorServer<T, U>>) -> Self {
        Self { executor_server }
    }

    fn start(
        &self,
        shutdown_noti: &ShutdownNotifier,
        executor_heartbeat_interval_seconds: u64,
    ) {
        let executor_server = self.executor_server.clone();
        let mut heartbeat_shutdown = shutdown_noti.subscribe_for_shutdown();
        let heartbeat_complete = shutdown_noti.shutdown_complete_tx.clone();
        tokio::spawn(async move {
            info!("Starting heartbeater to send heartbeat the scheduler periodically");
            // As long as the shutdown notification has not been received
            while !heartbeat_shutdown.is_shutdown() {
                executor_server.heartbeat().await;
                tokio::select! {
                    _ = tokio::time::sleep(Duration::from_secs(executor_heartbeat_interval_seconds)) => {},
                    _ = heartbeat_shutdown.recv() => {
                        info!("Stop heartbeater");
                        drop(heartbeat_complete);
                        return;
                    }
                };
            }
        });
    }
}

/// There are two loop(future) running separately in tokio runtime.
/// First is for sending back task status to scheduler
/// Second is for receiving task from scheduler and run.
/// The two loops will run forever until a shutdown notification received.
struct TaskRunnerPool<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> {
    executor_server: Arc<ExecutorServer<T, U>>,
}

impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> TaskRunnerPool<T, U> {
    fn new(executor_server: Arc<ExecutorServer<T, U>>) -> Self {
        Self { executor_server }
    }

    fn start(
        &self,
        mut rx_task: mpsc::Receiver<CuratorTaskDefinition>,
        mut rx_task_status: mpsc::Receiver<CuratorTaskStatus>,
        shutdown_noti: &ShutdownNotifier,
    ) {
        //1. loop for task status reporting
        let executor_server = self.executor_server.clone();
        let mut tasks_status_shutdown = shutdown_noti.subscribe_for_shutdown();
        let tasks_status_complete = shutdown_noti.shutdown_complete_tx.clone();
        tokio::spawn(async move {
            info!("Starting the task status reporter");
            // As long as the shutdown notification has not been received
            while !tasks_status_shutdown.is_shutdown() {
                let mut curator_task_status_map: HashMap<String, Vec<TaskStatus>> =
                    HashMap::new();
                // First try to fetch task status from the channel in *blocking* mode
                let maybe_task_status: Option<CuratorTaskStatus> = tokio::select! {
                     task_status = rx_task_status.recv() => task_status,
                    _ = tasks_status_shutdown.recv() => {
                        info!("Stop task status reporting loop");
                        drop(tasks_status_complete);
                        return;
                    }
                };

                let mut fetched_task_num = 0usize;
                if let Some(task_status) = maybe_task_status {
                    let task_status_vec = curator_task_status_map
                        .entry(task_status.scheduler_id)
                        .or_default();
                    task_status_vec.push(task_status.task_status);
                    fetched_task_num += 1;
                } else {
                    info!("Channel is closed and will exit the task status report loop.");
                    drop(tasks_status_complete);
                    return;
                }

                // Then try to fetch by non-blocking mode to fetch as much finished tasks as possible
                loop {
                    match rx_task_status.try_recv() {
                        Ok(task_status) => {
                            let task_status_vec = curator_task_status_map
                                .entry(task_status.scheduler_id)
                                .or_default();
                            task_status_vec.push(task_status.task_status);
                            fetched_task_num += 1;
                        }
                        Err(TryRecvError::Empty) => {
                            info!("Fetched {} tasks status to report", fetched_task_num);
                            break;
                        }
                        Err(TryRecvError::Disconnected) => {
                            info!("Channel is closed and will exit the task status report loop");
                            drop(tasks_status_complete);
                            return;
                        }
                    }
                }

                for (scheduler_id, tasks_status) in curator_task_status_map.into_iter() {
                    match executor_server.get_scheduler_client(&scheduler_id).await {
                        Ok(mut scheduler) => {
                            if let Err(e) = scheduler
                                .update_task_status(UpdateTaskStatusParams {
                                    executor_id: executor_server
                                        .executor
                                        .metadata
                                        .id
                                        .clone(),
                                    task_status: tasks_status.clone(),
                                })
                                .await
                            {
                                error!(
                                    "Fail to update tasks {:?} due to {:?}",
                                    tasks_status, e
                                );
                            }
                        }
                        Err(e) => {
                            error!(
                                "Fail to connect to scheduler {} due to {:?}",
                                scheduler_id, e
                            );
                        }
                    }
                }
            }
        });

        //2. loop for task fetching and running
        let executor_server = self.executor_server.clone();
        let mut task_runner_shutdown = shutdown_noti.subscribe_for_shutdown();
        let task_runner_complete = shutdown_noti.shutdown_complete_tx.clone();
        tokio::spawn(async move {
            info!("Starting the task runner pool");

            // Use a dedicated executor for CPU bound tasks so that the main tokio
            // executor can still answer requests even when under load
            let dedicated_executor = DedicatedExecutor::new(
                "task_runner",
                executor_server.executor.concurrent_tasks,
            );

            // As long as the shutdown notification has not been received
            while !task_runner_shutdown.is_shutdown() {
                let maybe_task: Option<CuratorTaskDefinition> = tokio::select! {
                     task = rx_task.recv() => task,
                    _ = task_runner_shutdown.recv() => {
                        info!("Stop the task runner pool");
                        drop(task_runner_complete);
                        return;
                    }
                };
                if let Some(curator_task) = maybe_task {
                    let task_identity = format!(
                        "TID {} {}/{}.{}/{}.{}",
                        &curator_task.task.task_id,
                        &curator_task.task.job_id,
                        &curator_task.task.stage_id,
                        &curator_task.task.stage_attempt_num,
                        &curator_task.task.partition_id,
                        &curator_task.task.task_attempt_num,
                    );
                    info!("Received task {:?}", &task_identity);

                    let server = executor_server.clone();
                    dedicated_executor.spawn(async move {
                        server.run_task(task_identity.clone(), curator_task).await;
                    });
                } else {
                    info!("Channel is closed and will exit the task receive loop");
                    drop(task_runner_complete);
                    return;
                }
            }
        });
    }
}

#[tonic::async_trait]
impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> ExecutorGrpc
    for ExecutorServer<T, U>
{
    async fn launch_task(
        &self,
        request: Request<LaunchTaskParams>,
    ) -> Result<Response<LaunchTaskResult>, Status> {
        let LaunchTaskParams {
            tasks,
            scheduler_id,
        } = request.into_inner();
        let task_sender = self.executor_env.tx_task.clone();
        for task in tasks {
            task_sender
                .send(CuratorTaskDefinition {
                    scheduler_id: scheduler_id.clone(),
                    task: get_task_definition(
                        task,
                        self.executor.get_runtime(false),
                        self.executor.scalar_functions.clone(),
                        self.executor.aggregate_functions.clone(),
                        self.executor.window_functions.clone(),
                        self.codec.clone(),
                    )
                    .map_err(|e| Status::invalid_argument(format!("{e}")))?,
                })
                .await
                .unwrap();
        }
        Ok(Response::new(LaunchTaskResult { success: true }))
    }

    /// by this interface, it can reduce the deserialization cost for multiple tasks
    /// belong to the same job stage running on the same one executor
    async fn launch_multi_task(
        &self,
        request: Request<LaunchMultiTaskParams>,
    ) -> Result<Response<LaunchMultiTaskResult>, Status> {
        let LaunchMultiTaskParams {
            multi_tasks,
            scheduler_id,
        } = request.into_inner();
        let task_sender = self.executor_env.tx_task.clone();
        for multi_task in multi_tasks {
            let multi_task: Vec<TaskDefinition> = get_task_definition_vec(
                multi_task,
                self.executor.get_runtime(false),
                self.executor.scalar_functions.clone(),
                self.executor.aggregate_functions.clone(),
                self.executor.window_functions.clone(),
                self.codec.clone(),
            )
            .map_err(|e| Status::invalid_argument(format!("{e}")))?;
            for task in multi_task {
                task_sender
                    .send(CuratorTaskDefinition {
                        scheduler_id: scheduler_id.clone(),
                        task,
                    })
                    .await
                    .unwrap();
            }
        }
        Ok(Response::new(LaunchMultiTaskResult { success: true }))
    }

    async fn stop_executor(
        &self,
        request: Request<StopExecutorParams>,
    ) -> Result<Response<StopExecutorResult>, Status> {
        let stop_request = request.into_inner();
        if stop_request.executor_id != self.executor.metadata.id {
            warn!(
                "The executor id {} in request is different from {}. The stop request will be ignored",
                stop_request.executor_id, self.executor.metadata.id
            );
            return Ok(Response::new(StopExecutorResult {}));
        }
        let stop_reason = stop_request.reason;
        let force = stop_request.force;
        info!(
            "Receive stop executor request, reason: {:?}, force {:?}",
            stop_reason, force
        );
        let stop_sender = self.executor_env.tx_stop.clone();
        stop_sender.send(force).await.unwrap();
        Ok(Response::new(StopExecutorResult {}))
    }

    async fn cancel_tasks(
        &self,
        request: Request<CancelTasksParams>,
    ) -> Result<Response<CancelTasksResult>, Status> {
        let task_infos = request.into_inner().task_infos;
        info!("Cancelling tasks for {:?}", task_infos);

        let mut cancelled = true;

        for task in task_infos {
            if let Err(e) = self
                .executor
                .cancel_task(
                    task.task_id as usize,
                    task.job_id,
                    task.stage_id as usize,
                    task.partition_id as usize,
                )
                .await
            {
                error!("Error cancelling task: {:?}", e);
                cancelled = false;
            }
        }

        Ok(Response::new(CancelTasksResult { cancelled }))
    }

    async fn remove_job_data(
        &self,
        request: Request<RemoveJobDataParams>,
    ) -> Result<Response<RemoveJobDataResult>, Status> {
        let job_id = request.into_inner().job_id;

        let work_dir = PathBuf::from(&self.executor.work_dir);
        let mut path = work_dir.clone();
        path.push(&job_id);

        // Verify it's an existing directory
        if !path.is_dir() {
            return if !path.exists() {
                Ok(Response::new(RemoveJobDataResult {}))
            } else {
                Err(Status::invalid_argument(format!(
                    "Path {path:?} is not for a directory!!!"
                )))
            };
        }

        if !is_subdirectory(path.as_path(), work_dir.as_path()) {
            return Err(Status::invalid_argument(format!(
                "Path {path:?} is not a subdirectory of {work_dir:?}!!!"
            )));
        }

        info!("Remove data for job {:?}", job_id);

        std::fs::remove_dir_all(&path)?;

        Ok(Response::new(RemoveJobDataResult {}))
    }
}

// Check whether the path is the subdirectory of the base directory
fn is_subdirectory(path: &Path, base_path: &Path) -> bool {
    if let (Ok(path), Ok(base_path)) = (path.canonicalize(), base_path.canonicalize()) {
        if let Some(parent_path) = path.parent() {
            parent_path.starts_with(base_path)
        } else {
            false
        }
    } else {
        false
    }
}

#[cfg(test)]
mod test {
    use crate::executor_server::is_subdirectory;
    use std::fs;
    use std::path::{Path, PathBuf};
    use tempfile::TempDir;

    #[tokio::test]
    async fn test_is_subdirectory() {
        let base_dir = TempDir::new().unwrap().into_path();

        // Normal correct one
        {
            let job_path = prepare_testing_job_directory(&base_dir, "job_a");
            assert!(is_subdirectory(&job_path, base_dir.as_path()));
        }

        // Empty job id
        {
            let job_path = prepare_testing_job_directory(&base_dir, "");
            assert!(!is_subdirectory(&job_path, base_dir.as_path()));

            let job_path = prepare_testing_job_directory(&base_dir, ".");
            assert!(!is_subdirectory(&job_path, base_dir.as_path()));
        }

        // Malicious job id
        {
            let job_path = prepare_testing_job_directory(&base_dir, "..");
            assert!(!is_subdirectory(&job_path, base_dir.as_path()));
        }
    }

    fn prepare_testing_job_directory(base_dir: &Path, job_id: &str) -> PathBuf {
        let mut path = base_dir.to_path_buf();
        path.push(job_id);
        if !path.exists() {
            fs::create_dir(&path).unwrap();
        }
        path
    }
}