|
| 1 | +// Licensed to the Apache Software Foundation (ASF) under one |
| 2 | +// or more contributor license agreements. See the NOTICE file |
| 3 | +// distributed with this work for additional information |
| 4 | +// regarding copyright ownership. The ASF licenses this file |
| 5 | +// to you under the Apache License, Version 2.0 (the |
| 6 | +// "License"); you may not use this file except in compliance |
| 7 | +// with the License. You may obtain a copy of the License at |
| 8 | +// |
| 9 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | +// |
| 11 | +// Unless required by applicable law or agreed to in writing, |
| 12 | +// software distributed under the License is distributed on an |
| 13 | +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 14 | +// KIND, either express or implied. See the License for the |
| 15 | +// specific language governing permissions and limitations |
| 16 | +// under the License. |
| 17 | + |
| 18 | +use std::sync::Arc; |
| 19 | +use std::time::Duration; |
| 20 | + |
| 21 | +use async_trait::async_trait; |
| 22 | +use log::{debug, warn}; |
| 23 | + |
| 24 | +use ballista_core::error::{BallistaError, Result}; |
| 25 | +use ballista_core::event_loop::EventAction; |
| 26 | +use ballista_core::serde::protobuf::{LaunchTaskParams, TaskDefinition}; |
| 27 | +use ballista_core::serde::scheduler::ExecutorData; |
| 28 | +use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan}; |
| 29 | + |
| 30 | +use crate::scheduler_server::task_scheduler::TaskScheduler; |
| 31 | +use crate::scheduler_server::ExecutorsClient; |
| 32 | +use crate::state::SchedulerState; |
| 33 | + |
| 34 | +#[derive(Clone)] |
| 35 | +pub(crate) enum SchedulerServerEvent { |
| 36 | + JobSubmitted(String), |
| 37 | +} |
| 38 | + |
| 39 | +pub(crate) struct SchedulerServerEventAction< |
| 40 | + T: 'static + AsLogicalPlan, |
| 41 | + U: 'static + AsExecutionPlan, |
| 42 | +> { |
| 43 | + state: Arc<SchedulerState<T, U>>, |
| 44 | + executors_client: ExecutorsClient, |
| 45 | +} |
| 46 | + |
| 47 | +impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> |
| 48 | + SchedulerServerEventAction<T, U> |
| 49 | +{ |
| 50 | + pub fn new( |
| 51 | + state: Arc<SchedulerState<T, U>>, |
| 52 | + executors_client: ExecutorsClient, |
| 53 | + ) -> Self { |
| 54 | + Self { |
| 55 | + state, |
| 56 | + executors_client, |
| 57 | + } |
| 58 | + } |
| 59 | + |
| 60 | + async fn offer_resources( |
| 61 | + &self, |
| 62 | + job_id: String, |
| 63 | + ) -> Result<Option<SchedulerServerEvent>> { |
| 64 | + let mut available_executors = self.state.get_available_executors_data(); |
| 65 | + // In case of there's no enough resources, reschedule the tasks of the job |
| 66 | + if available_executors.is_empty() { |
| 67 | + // TODO Maybe it's better to use an exclusive runtime for this kind task scheduling |
| 68 | + warn!("Not enough available executors for task running"); |
| 69 | + tokio::time::sleep(Duration::from_millis(100)).await; |
| 70 | + return Ok(Some(SchedulerServerEvent::JobSubmitted(job_id))); |
| 71 | + } |
| 72 | + |
| 73 | + let (tasks_assigment, num_tasks) = self |
| 74 | + .state |
| 75 | + .fetch_tasks(&mut available_executors, &job_id) |
| 76 | + .await?; |
| 77 | + if num_tasks > 0 { |
| 78 | + self.launch_tasks(&available_executors, tasks_assigment) |
| 79 | + .await?; |
| 80 | + } |
| 81 | + |
| 82 | + Ok(None) |
| 83 | + } |
| 84 | + |
| 85 | + async fn launch_tasks( |
| 86 | + &self, |
| 87 | + executors: &[ExecutorData], |
| 88 | + tasks_assigment: Vec<Vec<TaskDefinition>>, |
| 89 | + ) -> Result<()> { |
| 90 | + for (idx_executor, tasks) in tasks_assigment.into_iter().enumerate() { |
| 91 | + if !tasks.is_empty() { |
| 92 | + let executor_data = &executors[idx_executor]; |
| 93 | + debug!( |
| 94 | + "Start to launch tasks {:?} to executor {:?}", |
| 95 | + tasks |
| 96 | + .iter() |
| 97 | + .map(|task| { |
| 98 | + if let Some(task_id) = task.task_id.as_ref() { |
| 99 | + format!( |
| 100 | + "{}/{}/{}", |
| 101 | + task_id.job_id, |
| 102 | + task_id.stage_id, |
| 103 | + task_id.partition_id |
| 104 | + ) |
| 105 | + } else { |
| 106 | + "".to_string() |
| 107 | + } |
| 108 | + }) |
| 109 | + .collect::<Vec<String>>(), |
| 110 | + executor_data.executor_id |
| 111 | + ); |
| 112 | + let mut client = { |
| 113 | + let clients = self.executors_client.read().await; |
| 114 | + clients.get(&executor_data.executor_id).unwrap().clone() |
| 115 | + }; |
| 116 | + // Update the resources first |
| 117 | + self.state.save_executor_data(executor_data.clone()); |
| 118 | + // TODO check whether launching task is successful or not |
| 119 | + client.launch_task(LaunchTaskParams { task: tasks }).await?; |
| 120 | + } else { |
| 121 | + // Since the task assignment policy is round robin, |
| 122 | + // if find tasks for one executor is empty, just break fast |
| 123 | + break; |
| 124 | + } |
| 125 | + } |
| 126 | + |
| 127 | + Ok(()) |
| 128 | + } |
| 129 | +} |
| 130 | + |
| 131 | +#[async_trait] |
| 132 | +impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> |
| 133 | + EventAction<SchedulerServerEvent> for SchedulerServerEventAction<T, U> |
| 134 | +{ |
| 135 | + // TODO |
| 136 | + fn on_start(&self) {} |
| 137 | + |
| 138 | + // TODO |
| 139 | + fn on_stop(&self) {} |
| 140 | + |
| 141 | + async fn on_receive( |
| 142 | + &self, |
| 143 | + event: SchedulerServerEvent, |
| 144 | + ) -> Result<Option<SchedulerServerEvent>> { |
| 145 | + match event { |
| 146 | + SchedulerServerEvent::JobSubmitted(job_id) => { |
| 147 | + self.offer_resources(job_id).await |
| 148 | + } |
| 149 | + } |
| 150 | + } |
| 151 | + |
| 152 | + // TODO |
| 153 | + fn on_error(&self, _error: BallistaError) {} |
| 154 | +} |
0 commit comments