-
Notifications
You must be signed in to change notification settings - Fork 209
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
Add executor terminating status for graceful shutdown #667
Changes from 2 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 |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
//! Ballista Executor Process | ||
|
||
use std::net::SocketAddr; | ||
use std::sync::atomic::Ordering; | ||
use std::sync::Arc; | ||
use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; | ||
use std::{env, io}; | ||
|
@@ -41,18 +42,21 @@ use datafusion_proto::protobuf::{LogicalPlanNode, PhysicalPlanNode}; | |
|
||
use ballista_core::config::{LogRotationPolicy, TaskSchedulingPolicy}; | ||
use ballista_core::error::BallistaError; | ||
use ballista_core::serde::protobuf::executor_resource::Resource; | ||
use ballista_core::serde::protobuf::executor_status::Status; | ||
use ballista_core::serde::protobuf::{ | ||
executor_registration, scheduler_grpc_client::SchedulerGrpcClient, | ||
ExecutorRegistration, ExecutorStoppedParams, | ||
ExecutorRegistration, ExecutorResource, ExecutorSpecification, ExecutorStatus, | ||
ExecutorStoppedParams, HeartBeatParams, | ||
}; | ||
use ballista_core::serde::scheduler::ExecutorSpecification; | ||
use ballista_core::serde::BallistaCodec; | ||
use ballista_core::utils::{ | ||
create_grpc_client_connection, create_grpc_server, with_object_store_provider, | ||
}; | ||
use ballista_core::BALLISTA_VERSION; | ||
|
||
use crate::executor::Executor; | ||
use crate::executor::{Executor, TasksDrainedFuture}; | ||
use crate::executor_server::TERMINATING; | ||
use crate::flight_service::BallistaFlightService; | ||
use crate::metrics::LoggingMetricsCollector; | ||
use crate::shutdown::Shutdown; | ||
|
@@ -155,12 +159,11 @@ pub async fn start_executor_process(opt: ExecutorProcessConfig) -> Result<()> { | |
.map(executor_registration::OptionalHost::Host), | ||
port: opt.port as u32, | ||
grpc_port: opt.grpc_port as u32, | ||
specification: Some( | ||
ExecutorSpecification { | ||
task_slots: concurrent_tasks as u32, | ||
} | ||
.into(), | ||
), | ||
specification: Some(ExecutorSpecification { | ||
resources: vec![ExecutorResource { | ||
resource: Some(Resource::TaskSlots(concurrent_tasks as u32)), | ||
}], | ||
}), | ||
}; | ||
|
||
let config = with_object_store_provider( | ||
|
@@ -295,6 +298,8 @@ pub async fn start_executor_process(opt: ExecutorProcessConfig) -> Result<()> { | |
shutdown_noti.subscribe_for_shutdown(), | ||
))); | ||
|
||
let tasks_drained = TasksDrainedFuture(executor); | ||
|
||
// Concurrently run the service checking and listen for the `shutdown` signal and wait for the stop request coming. | ||
// The check_services runs until an error is encountered, so under normal circumstances, this `select!` statement runs | ||
// until the `shutdown` signal is received or a stop request is coming. | ||
|
@@ -319,7 +324,41 @@ pub async fn start_executor_process(opt: ExecutorProcessConfig) -> Result<()> { | |
}, | ||
}; | ||
|
||
// Set status to fenced | ||
info!("setting executor to TERMINATING status"); | ||
TERMINATING.store(true, Ordering::Release); | ||
|
||
if notify_scheduler { | ||
// Send a heartbeat to update status of executor to `Fenced`. This should signal to the | ||
// scheduler to no longer scheduler tasks on this executor | ||
thinkharderdev marked this conversation as resolved.
Show resolved
Hide resolved
|
||
if let Err(error) = scheduler | ||
.heart_beat_from_executor(HeartBeatParams { | ||
executor_id: executor_id.clone(), | ||
metrics: vec![], | ||
status: Some(ExecutorStatus { | ||
status: Some(Status::Terminating(String::default())), | ||
}), | ||
metadata: Some(ExecutorRegistration { | ||
id: executor_id.clone(), | ||
optional_host: opt | ||
.external_host | ||
.clone() | ||
.map(executor_registration::OptionalHost::Host), | ||
port: opt.port as u32, | ||
grpc_port: opt.grpc_port as u32, | ||
specification: Some(ExecutorSpecification { | ||
resources: vec![ExecutorResource { | ||
resource: Some(Resource::TaskSlots(concurrent_tasks as u32)), | ||
}], | ||
}), | ||
}), | ||
}) | ||
.await | ||
{ | ||
error!("error sending heartbeat with fenced status: {:?}", error); | ||
} | ||
|
||
// TODO we probably don't need a separate rpc call for this.... | ||
if let Err(error) = scheduler | ||
.executor_stopped(ExecutorStoppedParams { | ||
executor_id, | ||
Comment on lines
+361
to
364
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 think you can remove this RPC call (in the following PR), since this RPC is only used here. |
||
|
@@ -329,6 +368,9 @@ pub async fn start_executor_process(opt: ExecutorProcessConfig) -> Result<()> { | |
{ | ||
error!("ExecutorStopped grpc failed: {:?}", error); | ||
} | ||
|
||
// Wait for tasks to drain | ||
tasks_drained.await; | ||
} | ||
|
||
// Extract the `shutdown_complete` receiver and transmitter | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -258,6 +258,7 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerGrpc | |
metrics, | ||
status, | ||
}; | ||
|
||
self.state | ||
.executor_manager | ||
.save_executor_heartbeat(executor_heartbeat) | ||
|
@@ -521,13 +522,14 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> SchedulerGrpc | |
error!("{}", msg); | ||
Status::internal(msg) | ||
})?; | ||
Self::remove_executor(executor_manager, event_sender, &executor_id, Some(reason)) | ||
.await | ||
.map_err(|e| { | ||
let msg = format!("Error to remove executor in Scheduler due to {e:?}"); | ||
error!("{}", msg); | ||
Status::internal(msg) | ||
})?; | ||
|
||
Self::remove_executor( | ||
executor_manager, | ||
event_sender, | ||
&executor_id, | ||
Some(reason), | ||
self.executor_termination_grace_period, | ||
); | ||
|
||
Ok(Response::new(ExecutorStoppedResult {})) | ||
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. Maybe we should rename the method name to |
||
} | ||
|
@@ -603,6 +605,7 @@ mod test { | |
|
||
use crate::state::executor_manager::DEFAULT_EXECUTOR_TIMEOUT_SECONDS; | ||
use crate::state::SchedulerState; | ||
use crate::test_utils::await_condition; | ||
use crate::test_utils::test_cluster_context; | ||
|
||
use super::{SchedulerGrpc, SchedulerServer}; | ||
|
@@ -702,7 +705,7 @@ mod test { | |
"localhost:50050".to_owned(), | ||
cluster.clone(), | ||
BallistaCodec::default(), | ||
SchedulerConfig::default(), | ||
SchedulerConfig::default().with_remove_executor_wait_secs(0), | ||
default_metrics_collector().unwrap(), | ||
); | ||
scheduler.init().await?; | ||
|
@@ -760,15 +763,22 @@ mod test { | |
.await | ||
.expect("getting executor"); | ||
|
||
let is_stopped = await_condition(Duration::from_millis(10), 5, || { | ||
futures::future::ready(Ok(state.executor_manager.is_dead_executor("abc"))) | ||
}) | ||
.await?; | ||
|
||
// executor should be marked to dead | ||
assert!(state.executor_manager.is_dead_executor("abc")); | ||
assert!(is_stopped, "Executor not marked dead after 50ms"); | ||
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. Nice test |
||
|
||
let active_executors = state | ||
.executor_manager | ||
.get_alive_executors_within_one_minute(); | ||
assert!(active_executors.is_empty()); | ||
|
||
let expired_executors = state.executor_manager.get_expired_executors(); | ||
let expired_executors = state | ||
.executor_manager | ||
.get_expired_executors(scheduler.executor_termination_grace_period); | ||
assert!(expired_executors.is_empty()); | ||
|
||
Ok(()) | ||
|
@@ -895,7 +905,9 @@ mod test { | |
.get_alive_executors_within_one_minute(); | ||
assert_eq!(active_executors.len(), 1); | ||
|
||
let expired_executors = state.executor_manager.get_expired_executors(); | ||
let expired_executors = state | ||
.executor_manager | ||
.get_expired_executors(scheduler.executor_termination_grace_period); | ||
assert!(expired_executors.is_empty()); | ||
|
||
// simulate the heartbeat timeout | ||
|
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.
In the
StopExecutorParams
struct, there is aforce
flag to tell theExecutor
whether it should quit immediately or waiting for draining all the running tasks. I'm not sure whether it is correctly implemented or not. Maybe you can take a look.