Skip to content
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

Merged
merged 3 commits into from
Mar 9, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion ballista/rust/core/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ simd = ["datafusion/simd"]

[dependencies]
ahash = { version = "0.7", default-features = false }
async-trait = "0.1.36"
async-trait = "0.1.41"
futures = "0.3"
hashbrown = "0.12"
log = "0.4"
Expand Down
141 changes: 141 additions & 0 deletions ballista/rust/core/src/event_loop.rs
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");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

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))
})?)
}
}
1 change: 1 addition & 0 deletions ballista/rust/core/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ pub fn print_version() {
pub mod client;
pub mod config;
pub mod error;
pub mod event_loop;
pub mod execution_plans;
pub mod utils;

Expand Down
2 changes: 1 addition & 1 deletion ballista/rust/executor/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ snmalloc = ["snmalloc-rs"]
arrow = { version = "9.1" }
arrow-flight = { version = "9.1" }
anyhow = "1"
async-trait = "0.1.36"
async-trait = "0.1.41"
ballista-core = { path = "../core", version = "0.6.0" }
configure_me = "0.4.0"
datafusion = { path = "../../../datafusion", version = "7.0.0" }
Expand Down
1 change: 1 addition & 0 deletions ballista/rust/scheduler/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ tonic = "0.6"
tower = { version = "0.4" }
warp = "0.3"
parking_lot = "0.12"
async-trait = "0.1.41"

[dev-dependencies]
ballista-core = { path = "../core", version = "0.6.0" }
Expand Down
33 changes: 12 additions & 21 deletions ballista/rust/scheduler/src/main.rs
Original file line number Diff line number Diff line change
Expand Up @@ -40,15 +40,13 @@ use ballista_scheduler::state::EtcdClient;
#[cfg(feature = "sled")]
use ballista_scheduler::state::StandaloneClient;

use ballista_scheduler::scheduler_server::{
SchedulerEnv, SchedulerServer, TaskScheduler,
};
use ballista_scheduler::scheduler_server::SchedulerServer;
use ballista_scheduler::state::{ConfigBackend, ConfigBackendClient};

use ballista_core::config::TaskSchedulingPolicy;
use ballista_core::serde::BallistaCodec;
use log::info;
use tokio::sync::{mpsc, RwLock};
use tokio::sync::RwLock;

#[macro_use]
extern crate configure_me;
Expand Down Expand Up @@ -81,24 +79,15 @@ async fn start_server(
"Starting Scheduler grpc server with task scheduling policy of {:?}",
policy
);
let scheduler_server: SchedulerServer<LogicalPlanNode, PhysicalPlanNode> =
let mut scheduler_server: SchedulerServer<LogicalPlanNode, PhysicalPlanNode> =
match policy {
TaskSchedulingPolicy::PushStaged => {
// TODO make the buffer size configurable
let (tx_job, rx_job) = mpsc::channel::<String>(10000);
let scheduler_server = SchedulerServer::new_with_policy(
config_backend.clone(),
namespace.clone(),
policy,
Some(SchedulerEnv { tx_job }),
Arc::new(RwLock::new(ExecutionContext::new())),
BallistaCodec::default(),
);
let task_scheduler =
TaskScheduler::new(Arc::new(scheduler_server.clone()));
task_scheduler.start(rx_job);
scheduler_server
}
TaskSchedulingPolicy::PushStaged => SchedulerServer::new_with_policy(
config_backend.clone(),
namespace.clone(),
policy,
Arc::new(RwLock::new(ExecutionContext::new())),
BallistaCodec::default(),
),
_ => SchedulerServer::new(
config_backend.clone(),
namespace.clone(),
Expand All @@ -107,6 +96,8 @@ async fn start_server(
),
};

scheduler_server.init().await?;

Server::bind(&addr)
.serve(make_service_fn(move |request: &AddrStream| {
let scheduler_grpc_server =
Expand Down
154 changes: 154 additions & 0 deletions ballista/rust/scheduler/src/scheduler_server/event_loop.rs
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) {}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure what the plan is here, but sometimes the todo!() macro gets used in situations so the TODO isn't silently forgotten. However, since todo!() panic's this may not be possible if the functions are called

Suggested change
fn on_start(&self) {}
fn on_start(&self) {
todo!();
}

Copy link
Contributor Author

@yahoNanJing yahoNanJing Mar 7, 2022

Choose a reason for hiding this comment

The 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) {}
}
Loading