-
Notifications
You must be signed in to change notification settings - Fork 1.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Refactor the event channel #1912
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,141 @@ | ||
// 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 std::sync::atomic::{AtomicBool, Ordering}; | ||
use std::sync::Arc; | ||
|
||
use async_trait::async_trait; | ||
use log::{error, info}; | ||
use tokio::sync::mpsc; | ||
|
||
use crate::error::{BallistaError, Result}; | ||
|
||
#[async_trait] | ||
pub trait EventAction<E>: Send + Sync { | ||
fn on_start(&self); | ||
|
||
fn on_stop(&self); | ||
|
||
async fn on_receive(&self, event: E) -> Result<Option<E>>; | ||
|
||
fn on_error(&self, error: BallistaError); | ||
} | ||
|
||
#[derive(Clone)] | ||
pub struct EventLoop<E> { | ||
name: String, | ||
stopped: Arc<AtomicBool>, | ||
buffer_size: usize, | ||
action: Arc<dyn EventAction<E>>, | ||
tx_event: Option<mpsc::Sender<E>>, | ||
} | ||
|
||
impl<E: Send + 'static> EventLoop<E> { | ||
pub fn new( | ||
name: String, | ||
buffer_size: usize, | ||
action: Arc<dyn EventAction<E>>, | ||
) -> Self { | ||
Self { | ||
name, | ||
stopped: Arc::new(AtomicBool::new(false)), | ||
buffer_size, | ||
action, | ||
tx_event: None, | ||
} | ||
} | ||
|
||
fn run(&self, mut rx_event: mpsc::Receiver<E>) { | ||
assert!( | ||
self.tx_event.is_some(), | ||
"The event sender should be initialized first!" | ||
); | ||
let tx_event = self.tx_event.as_ref().unwrap().clone(); | ||
let name = self.name.clone(); | ||
let stopped = self.stopped.clone(); | ||
let action = self.action.clone(); | ||
tokio::spawn(async move { | ||
info!("Starting the event loop {}", name); | ||
while !stopped.load(Ordering::SeqCst) { | ||
if let Some(event) = rx_event.recv().await { | ||
match action.on_receive(event).await { | ||
Ok(Some(event)) => { | ||
if let Err(e) = tx_event.send(event).await { | ||
let msg = format!("Fail to send event due to {}", e); | ||
error!("{}", msg); | ||
action.on_error(BallistaError::General(msg)); | ||
} | ||
} | ||
Err(e) => { | ||
error!("Fail to process event due to {}", e); | ||
action.on_error(e); | ||
} | ||
_ => {} | ||
} | ||
} else { | ||
info!("Event Channel closed, shutting down"); | ||
break; | ||
} | ||
} | ||
info!("The event loop {} has been stopped", name); | ||
}); | ||
} | ||
|
||
pub fn start(&mut self) -> Result<()> { | ||
if self.stopped.load(Ordering::SeqCst) { | ||
return Err(BallistaError::General(format!( | ||
"{} has already been stopped", | ||
self.name | ||
))); | ||
} | ||
self.action.on_start(); | ||
|
||
let (tx_event, rx_event) = mpsc::channel::<E>(self.buffer_size); | ||
self.tx_event = Some(tx_event); | ||
self.run(rx_event); | ||
|
||
Ok(()) | ||
} | ||
|
||
pub fn stop(&self) { | ||
if !self.stopped.swap(true, Ordering::SeqCst) { | ||
self.action.on_stop(); | ||
} else { | ||
// Keep quiet to allow calling `stop` multiple times. | ||
} | ||
} | ||
|
||
pub fn get_sender(&self) -> Result<EventSender<E>> { | ||
Ok(EventSender { | ||
tx_event: self.tx_event.as_ref().cloned().ok_or_else(|| { | ||
BallistaError::General("Event sender not exist!!!".to_string()) | ||
})?, | ||
}) | ||
} | ||
} | ||
|
||
pub struct EventSender<E> { | ||
tx_event: mpsc::Sender<E>, | ||
} | ||
|
||
impl<E> EventSender<E> { | ||
pub async fn post_event(&self, event: E) -> Result<()> { | ||
Ok(self.tx_event.send(event).await.map_err(|e| { | ||
BallistaError::General(format!("Fail to send event due to {}", e)) | ||
})?) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change | ||||||||
---|---|---|---|---|---|---|---|---|---|---|
@@ -0,0 +1,154 @@ | ||||||||||
// 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 std::sync::Arc; | ||||||||||
use std::time::Duration; | ||||||||||
|
||||||||||
use async_trait::async_trait; | ||||||||||
use log::{debug, warn}; | ||||||||||
|
||||||||||
use ballista_core::error::{BallistaError, Result}; | ||||||||||
use ballista_core::event_loop::EventAction; | ||||||||||
use ballista_core::serde::protobuf::{LaunchTaskParams, TaskDefinition}; | ||||||||||
use ballista_core::serde::scheduler::ExecutorData; | ||||||||||
use ballista_core::serde::{AsExecutionPlan, AsLogicalPlan}; | ||||||||||
|
||||||||||
use crate::scheduler_server::task_scheduler::TaskScheduler; | ||||||||||
use crate::scheduler_server::ExecutorsClient; | ||||||||||
use crate::state::SchedulerState; | ||||||||||
|
||||||||||
#[derive(Clone)] | ||||||||||
pub(crate) enum SchedulerServerEvent { | ||||||||||
JobSubmitted(String), | ||||||||||
} | ||||||||||
|
||||||||||
pub(crate) struct SchedulerServerEventAction< | ||||||||||
T: 'static + AsLogicalPlan, | ||||||||||
U: 'static + AsExecutionPlan, | ||||||||||
> { | ||||||||||
state: Arc<SchedulerState<T, U>>, | ||||||||||
executors_client: ExecutorsClient, | ||||||||||
} | ||||||||||
|
||||||||||
impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> | ||||||||||
SchedulerServerEventAction<T, U> | ||||||||||
{ | ||||||||||
pub fn new( | ||||||||||
state: Arc<SchedulerState<T, U>>, | ||||||||||
executors_client: ExecutorsClient, | ||||||||||
) -> Self { | ||||||||||
Self { | ||||||||||
state, | ||||||||||
executors_client, | ||||||||||
} | ||||||||||
} | ||||||||||
|
||||||||||
async fn offer_resources( | ||||||||||
&self, | ||||||||||
job_id: String, | ||||||||||
) -> Result<Option<SchedulerServerEvent>> { | ||||||||||
let mut available_executors = self.state.get_available_executors_data(); | ||||||||||
// In case of there's no enough resources, reschedule the tasks of the job | ||||||||||
if available_executors.is_empty() { | ||||||||||
// TODO Maybe it's better to use an exclusive runtime for this kind task scheduling | ||||||||||
warn!("Not enough available executors for task running"); | ||||||||||
tokio::time::sleep(Duration::from_millis(100)).await; | ||||||||||
return Ok(Some(SchedulerServerEvent::JobSubmitted(job_id))); | ||||||||||
} | ||||||||||
|
||||||||||
let (tasks_assigment, num_tasks) = self | ||||||||||
.state | ||||||||||
.fetch_tasks(&mut available_executors, &job_id) | ||||||||||
.await?; | ||||||||||
if num_tasks > 0 { | ||||||||||
self.launch_tasks(&available_executors, tasks_assigment) | ||||||||||
.await?; | ||||||||||
} | ||||||||||
|
||||||||||
Ok(None) | ||||||||||
} | ||||||||||
|
||||||||||
async fn launch_tasks( | ||||||||||
&self, | ||||||||||
executors: &[ExecutorData], | ||||||||||
tasks_assigment: Vec<Vec<TaskDefinition>>, | ||||||||||
) -> Result<()> { | ||||||||||
for (idx_executor, tasks) in tasks_assigment.into_iter().enumerate() { | ||||||||||
if !tasks.is_empty() { | ||||||||||
let executor_data = &executors[idx_executor]; | ||||||||||
debug!( | ||||||||||
"Start to launch tasks {:?} to executor {:?}", | ||||||||||
tasks | ||||||||||
.iter() | ||||||||||
.map(|task| { | ||||||||||
if let Some(task_id) = task.task_id.as_ref() { | ||||||||||
format!( | ||||||||||
"{}/{}/{}", | ||||||||||
task_id.job_id, | ||||||||||
task_id.stage_id, | ||||||||||
task_id.partition_id | ||||||||||
) | ||||||||||
} else { | ||||||||||
"".to_string() | ||||||||||
} | ||||||||||
}) | ||||||||||
.collect::<Vec<String>>(), | ||||||||||
executor_data.executor_id | ||||||||||
); | ||||||||||
let mut client = { | ||||||||||
let clients = self.executors_client.read().await; | ||||||||||
clients.get(&executor_data.executor_id).unwrap().clone() | ||||||||||
}; | ||||||||||
// Update the resources first | ||||||||||
self.state.save_executor_data(executor_data.clone()); | ||||||||||
// TODO check whether launching task is successful or not | ||||||||||
client.launch_task(LaunchTaskParams { task: tasks }).await?; | ||||||||||
} else { | ||||||||||
// Since the task assignment policy is round robin, | ||||||||||
// if find tasks for one executor is empty, just break fast | ||||||||||
break; | ||||||||||
} | ||||||||||
} | ||||||||||
|
||||||||||
Ok(()) | ||||||||||
} | ||||||||||
} | ||||||||||
|
||||||||||
#[async_trait] | ||||||||||
impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> | ||||||||||
EventAction<SchedulerServerEvent> for SchedulerServerEventAction<T, U> | ||||||||||
{ | ||||||||||
// TODO | ||||||||||
fn on_start(&self) {} | ||||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not sure what the plan is here, but sometimes the
Suggested change
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here, the reason not to use todo!() is to avoid panic. The reason to add TODO comments is for future error handling or other things to make the whole system more robust. If it makes confusing, we can just remove them currently. |
||||||||||
|
||||||||||
// TODO | ||||||||||
fn on_stop(&self) {} | ||||||||||
|
||||||||||
async fn on_receive( | ||||||||||
&self, | ||||||||||
event: SchedulerServerEvent, | ||||||||||
) -> Result<Option<SchedulerServerEvent>> { | ||||||||||
match event { | ||||||||||
SchedulerServerEvent::JobSubmitted(job_id) => { | ||||||||||
self.offer_resources(job_id).await | ||||||||||
} | ||||||||||
} | ||||||||||
} | ||||||||||
|
||||||||||
// TODO | ||||||||||
fn on_error(&self, _error: BallistaError) {} | ||||||||||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍