From cef11e0e2b364bfdfb727bbedc3579466d308c98 Mon Sep 17 00:00:00 2001 From: Wang Date: Wed, 21 Sep 2022 23:53:20 +0800 Subject: [PATCH 1/4] Task level failure retry and Stage level failure retry --- ballista/rust/core/proto/ballista.proto | 168 +++- ballista/rust/core/src/client.rs | 28 +- ballista/rust/core/src/error.rs | 85 +- .../src/execution_plans/distributed_query.rs | 7 +- .../src/execution_plans/shuffle_reader.rs | 22 +- ballista/rust/core/src/utils.rs | 7 +- ballista/rust/executor/src/execution_loop.rs | 65 +- ballista/rust/executor/src/executor.rs | 60 +- ballista/rust/executor/src/executor_server.rs | 115 ++- ballista/rust/executor/src/lib.rs | 41 +- .../scheduler/src/scheduler_server/event.rs | 4 +- .../scheduler/src/scheduler_server/grpc.rs | 22 +- .../scheduler/src/scheduler_server/mod.rs | 68 +- .../scheduler_server/query_stage_scheduler.rs | 46 +- .../scheduler/src/state/execution_graph.rs | 789 ++++++++++++------ .../state/execution_graph/execution_stage.rs | 486 +++++++---- .../scheduler/src/state/executor_manager.rs | 46 +- ballista/rust/scheduler/src/state/mod.rs | 74 +- .../rust/scheduler/src/state/task_manager.rs | 217 +++-- 19 files changed, 1584 insertions(+), 766 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 7afe04e62..aff12c2bd 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -416,54 +416,85 @@ message ExecutionGraph { uint64 output_partitions = 5; repeated PartitionLocation output_locations = 6; string scheduler_id = 7; + uint32 tid_gen = 8; + repeated StageAttempts failed_attempts = 9; +} + +message StageAttempts { + uint32 stage_id = 1; + repeated uint32 stage_attempt_num = 2; } message ExecutionGraphStage { oneof StageType { UnResolvedStage unresolved_stage = 1; ResolvedStage resolved_stage = 2; - CompletedStage completed_stage = 3; + SuccessfulStage successful_stage = 3; FailedStage failed_stage = 4; } } message UnResolvedStage { - uint64 stage_id = 1; - PhysicalHashRepartition output_partitioning = 2; - repeated uint32 output_links = 3; - repeated GraphStageInput inputs = 4; - bytes plan = 5; -} - -message ResolvedStage { - uint64 stage_id = 1; - uint32 partitions = 2; + uint32 stage_id = 1; + uint32 stage_attempt_num = 2; PhysicalHashRepartition output_partitioning = 3; repeated uint32 output_links = 4; repeated GraphStageInput inputs = 5; bytes plan = 6; } -message CompletedStage { - uint64 stage_id = 1; - uint32 partitions = 2; - PhysicalHashRepartition output_partitioning = 3; - repeated uint32 output_links = 4; - repeated GraphStageInput inputs = 5; - bytes plan = 6; - repeated TaskStatus task_statuses = 7; - repeated OperatorMetricsSet stage_metrics = 8; +message ResolvedStage { + uint32 stage_id = 1; + uint32 stage_attempt_num = 2; + uint32 partitions = 3; + PhysicalHashRepartition output_partitioning = 4; + repeated uint32 output_links = 5; + repeated GraphStageInput inputs = 6; + bytes plan = 7; +} + +message SuccessfulStage { + uint32 stage_id = 1; + uint32 stage_attempt_num = 2; + uint32 partitions = 3; + PhysicalHashRepartition output_partitioning = 4; + repeated uint32 output_links = 5; + repeated GraphStageInput inputs = 6; + bytes plan = 7; + repeated TaskInfo task_infos = 8; + repeated OperatorMetricsSet stage_metrics = 9; } message FailedStage { - uint64 stage_id = 1; - uint32 partitions = 2; - PhysicalHashRepartition output_partitioning = 3; - repeated uint32 output_links = 4; - bytes plan = 5; - repeated TaskStatus task_statuses = 6; - repeated OperatorMetricsSet stage_metrics = 7; - string error_message = 8; + uint32 stage_id = 1; + uint32 stage_attempt_num = 2; + uint32 partitions = 3; + PhysicalHashRepartition output_partitioning = 4; + repeated uint32 output_links = 5; + bytes plan = 6; + repeated TaskInfo task_infos = 7; + repeated OperatorMetricsSet stage_metrics = 8; + string error_message = 9; +} + +message TaskInfo { + uint32 task_id = 1; + uint32 partition_id = 2; + // Scheduler schedule time + uint64 scheduled_time = 3; + // Scheduler launch time + uint64 launch_time = 4; + // The time the Executor start to run the task + uint64 start_exec_time = 5; + // The time the Executor finish the task + uint64 end_exec_time = 6; + // Scheduler side finish time + uint64 finish_time = 7; + oneof status { + RunningTask running = 8; + FailedTask failed = 9; + SuccessfulTask successful = 10; + } } message GraphStageInput { @@ -647,15 +678,48 @@ message RunningTask { message FailedTask { string error = 1; + bool retryable = 2; + // Whether this task failure should be counted to the maximum number of times the task is allowed to retry + bool count_to_failures = 3; + oneof failed_reason { + ExecutionError execution_error = 4; + FetchPartitionError fetch_partition_error = 5; + IOError io_error = 6; + ExecutorLost executor_lost = 7; + // A successful task's result is lost due to executor lost + ResultLost result_lost = 8; + TaskKilled task_killed = 9; + } } -message CompletedTask { +message SuccessfulTask { string executor_id = 1; // TODO tasks are currently always shuffle writes but this will not always be the case // so we might want to think about some refactoring of the task definitions repeated ShuffleWritePartition partitions = 2; } +message ExecutionError { +} + +message FetchPartitionError { + string executor_id = 1; + uint32 map_stage_id = 2; + uint32 map_partition_id = 3; +} + +message IOError { +} + +message ExecutorLost { +} + +message ResultLost { +} + +message TaskKilled { +} + message ShuffleWritePartition { uint64 partition_id = 1; string path = 2; @@ -665,13 +729,20 @@ message ShuffleWritePartition { } message TaskStatus { - PartitionId task_id = 1; + uint32 task_id = 1; + string job_id = 2; + uint32 stage_id = 3; + uint32 stage_attempt_num = 4; + uint32 partition_id = 5; + uint64 launch_time = 6; + uint64 start_exec_time = 7; + uint64 end_exec_time = 8; oneof status { - RunningTask running = 2; - FailedTask failed = 3; - CompletedTask completed = 4; + RunningTask running = 9; + FailedTask failed = 10; + SuccessfulTask successful = 11; } - repeated OperatorMetricsSet metrics = 5; + repeated OperatorMetricsSet metrics = 12; } message PollWorkParams { @@ -682,12 +753,18 @@ message PollWorkParams { } message TaskDefinition { - PartitionId task_id = 1; - bytes plan = 2; + uint32 task_id = 1; + uint32 task_attempt_num = 2; + string job_id = 3; + uint32 stage_id = 4; + uint32 stage_attempt_num = 5; + uint32 partition_id = 6; + bytes plan = 7; // Output partition for shuffle writer - PhysicalHashRepartition output_partitioning = 3; - string session_id = 4; - repeated KeyValuePair props = 5; + PhysicalHashRepartition output_partitioning = 8; + string session_id = 9; + uint64 launch_time = 10; + repeated KeyValuePair props = 11; } message SessionSettings { @@ -775,7 +852,7 @@ message GetJobStatusParams { string job_id = 1; } -message CompletedJob { +message SuccessfulJob { repeated PartitionLocation partition_location = 1; } @@ -793,7 +870,7 @@ message JobStatus { QueuedJob queued = 1; RunningJob running = 2; FailedJob failed = 3; - CompletedJob completed = 4; + SuccessfulJob successful = 4; } } @@ -834,13 +911,20 @@ message LaunchTaskResult { } message CancelTasksParams { - repeated PartitionId partition_id = 1; + repeated RunningTaskInfo task_infos = 1; } message CancelTasksResult { bool cancelled = 1; } +message RunningTaskInfo { + uint32 task_id = 1; + string job_id = 2; + uint32 stage_id = 3; + uint32 partition_id = 4;; +} + service SchedulerGrpc { // Executors must poll the scheduler for heartbeat and to receive tasks rpc PollWork (PollWorkParams) returns (PollWorkResult) {} diff --git a/ballista/rust/core/src/client.rs b/ballista/rust/core/src/client.rs index dfe2003fb..52bf2ae2b 100644 --- a/ballista/rust/core/src/client.rs +++ b/ballista/rust/core/src/client.rs @@ -25,7 +25,7 @@ use std::{ task::{Context, Poll}, }; -use crate::error::{ballista_error, BallistaError, Result}; +use crate::error::{BallistaError, Result}; use crate::serde::protobuf::{self}; use crate::serde::scheduler::Action; @@ -62,7 +62,7 @@ impl BallistaClient { create_grpc_client_connection(addr.clone()) .await .map_err(|e| { - BallistaError::General(format!( + BallistaError::GrpcConnectionError(format!( "Error connecting to Ballista scheduler or executor at {}: {:?}", addr, e )) @@ -76,6 +76,7 @@ impl BallistaClient { /// Fetch a partition from an executor pub async fn fetch_partition( &mut self, + executor_id: &str, job_id: &str, stage_id: usize, partition_id: usize, @@ -87,7 +88,18 @@ impl BallistaClient { partition_id, path: path.to_owned(), }; - self.execute_action(&action).await + self.execute_action(&action) + .await + .map_err(|error| match error { + // map grpc connection error to partition fetch error. + BallistaError::GrpcActionError(msg) => BallistaError::FetchFailed( + executor_id.to_owned(), + stage_id, + partition_id, + msg, + ), + other => other, + }) } /// Execute an action and retrieve the results @@ -101,7 +113,7 @@ impl BallistaClient { serialized_action .encode(&mut buf) - .map_err(|e| BallistaError::General(format!("{:?}", e)))?; + .map_err(|e| BallistaError::GrpcActionError(format!("{:?}", e)))?; let request = tonic::Request::new(Ticket { ticket: buf }); @@ -109,14 +121,14 @@ impl BallistaClient { .flight_client .do_get(request) .await - .map_err(|e| BallistaError::General(format!("{:?}", e)))? + .map_err(|e| BallistaError::GrpcActionError(format!("{:?}", e)))? .into_inner(); // the schema should be the first message returned, else client should error match stream .message() .await - .map_err(|e| BallistaError::General(format!("{:?}", e)))? + .map_err(|e| BallistaError::GrpcActionError(format!("{:?}", e)))? { Some(flight_data) => { // convert FlightData to a stream @@ -125,8 +137,8 @@ impl BallistaClient { // all the remaining stream messages should be dictionary and record batches Ok(Box::pin(FlightDataStream::new(stream, schema))) } - None => Err(ballista_error( - "Did not receive schema batch from flight server", + None => Err(BallistaError::GrpcActionError( + "Did not receive schema batch from flight server".to_string(), )), } } diff --git a/ballista/rust/core/src/error.rs b/ballista/rust/core/src/error.rs index dd4f9dda3..53d779e15 100644 --- a/ballista/rust/core/src/error.rs +++ b/ballista/rust/core/src/error.rs @@ -23,6 +23,8 @@ use std::{ io, result, }; +use crate::serde::protobuf::failed_task::FailedReason; +use crate::serde::protobuf::{ExecutionError, FailedTask, FetchPartitionError, IoError}; use datafusion::arrow::error::ArrowError; use datafusion::error::DataFusionError; use futures::future::Aborted; @@ -47,7 +49,11 @@ pub enum BallistaError { // KubeAPIResponseError(k8s_openapi::ResponseError), TonicError(tonic::transport::Error), GrpcError(tonic::Status), + GrpcConnectionError(String), TokioError(tokio::task::JoinError), + GrpcActionError(String), + // (executor_id, map_stage_id, map_partition_id, message) + FetchFailed(String, usize, usize, String), Cancelled, } @@ -70,7 +76,19 @@ impl From for BallistaError { impl From for BallistaError { fn from(e: ArrowError) -> Self { - BallistaError::ArrowError(e) + match e { + ArrowError::ExternalError(e) + if e.downcast_ref::().is_some() => + { + *e.downcast::().unwrap() + } + ArrowError::ExternalError(e) + if e.downcast_ref::().is_some() => + { + BallistaError::DataFusionError(*e.downcast::().unwrap()) + } + other => BallistaError::ArrowError(other), + } } } @@ -182,13 +200,78 @@ impl Display for BallistaError { // } BallistaError::TonicError(desc) => write!(f, "Tonic error: {}", desc), BallistaError::GrpcError(desc) => write!(f, "Grpc error: {}", desc), + BallistaError::GrpcConnectionError(desc) => { + write!(f, "Grpc connection error: {}", desc) + } BallistaError::Internal(desc) => { write!(f, "Internal Ballista error: {}", desc) } BallistaError::TokioError(desc) => write!(f, "Tokio join error: {}", desc), + BallistaError::GrpcActionError(desc) => { + write!(f, "Grpc Execute Action error: {}", desc) + } + BallistaError::FetchFailed(executor_id, map_stage, map_partition, desc) => { + write!( + f, + "Shuffle fetch partition error from Executor {}, map_stage {}, \ + map_partition {}, error desc: {}", + executor_id, map_stage, map_partition, desc + ) + } BallistaError::Cancelled => write!(f, "Task cancelled"), } } } +impl From for FailedTask { + fn from(e: BallistaError) -> Self { + match e { + BallistaError::FetchFailed( + executor_id, + map_stage_id, + map_partition_id, + desc, + ) => { + FailedTask { + error: desc, + // fetch partition error is considered to be non-retryable + retryable: false, + count_to_failures: false, + failed_reason: Some(FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id, + map_stage_id: map_stage_id as u32, + map_partition_id: map_partition_id as u32, + }, + )), + } + } + BallistaError::IoError(io) => { + FailedTask { + error: format!("Task failed due to Ballista IO error: {:?}", io), + // IO error is considered to be temporary and retryable + retryable: true, + count_to_failures: true, + failed_reason: Some(FailedReason::IoError(IoError {})), + } + } + BallistaError::DataFusionError(DataFusionError::IoError(io)) => { + FailedTask { + error: format!("Task failed due to DataFusion IO error: {:?}", io), + // IO error is considered to be temporary and retryable + retryable: true, + count_to_failures: true, + failed_reason: Some(FailedReason::IoError(IoError {})), + } + } + other => FailedTask { + error: format!("Task failed due to runtime execution error: {:?}", other), + retryable: false, + count_to_failures: false, + failed_reason: Some(FailedReason::ExecutionError(ExecutionError {})), + }, + } + } +} + impl Error for BallistaError {} diff --git a/ballista/rust/core/src/execution_plans/distributed_query.rs b/ballista/rust/core/src/execution_plans/distributed_query.rs index e9d852817..e76d1a3db 100644 --- a/ballista/rust/core/src/execution_plans/distributed_query.rs +++ b/ballista/rust/core/src/execution_plans/distributed_query.rs @@ -294,8 +294,8 @@ async fn execute_query( error!("{}", msg); break Err(DataFusionError::Execution(msg)); } - Some(job_status::Status::Completed(completed)) => { - let streams = completed.partition_location.into_iter().map(|p| { + Some(job_status::Status::Successful(successful)) => { + let streams = successful.partition_location.into_iter().map(|p| { let f = fetch_partition(p) .map_err(|e| ArrowError::ExternalError(Box::new(e))); @@ -323,11 +323,12 @@ async fn fetch_partition( .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; ballista_client .fetch_partition( + &metadata.id, &partition_id.job_id, partition_id.stage_id as usize, partition_id.partition_id as usize, &location.path, ) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e))) + .map_err(|e| DataFusionError::External(Box::new(e))) } diff --git a/ballista/rust/core/src/execution_plans/shuffle_reader.rs b/ballista/rust/core/src/execution_plans/shuffle_reader.rs index 5c0664ef2..45e9c2a45 100644 --- a/ballista/rust/core/src/execution_plans/shuffle_reader.rs +++ b/ballista/rust/core/src/execution_plans/shuffle_reader.rs @@ -16,6 +16,7 @@ // under the License. use std::any::Any; +use std::result; use std::sync::Arc; use crate::client::BallistaClient; @@ -33,6 +34,7 @@ use datafusion::physical_plan::{ }; use futures::{StreamExt, TryStreamExt}; +use crate::error::BallistaError; use datafusion::arrow::error::ArrowError; use datafusion::execution::context::TaskContext; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; @@ -202,7 +204,7 @@ fn stats_for_partitions( async fn fetch_partition( location: &PartitionLocation, -) -> Result { +) -> result::Result { let metadata = &location.executor_meta; let partition_id = &location.partition_id; // TODO for shuffle client connections, we should avoid creating new connections again and again. @@ -210,16 +212,26 @@ async fn fetch_partition( let mut ballista_client = BallistaClient::try_new(metadata.host.as_str(), metadata.port as u16) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; + .map_err(|error| match error { + // map grpc connection error to partition fetch error. + BallistaError::GrpcConnectionError(msg) => BallistaError::FetchFailed( + metadata.id.clone(), + partition_id.stage_id, + partition_id.partition_id, + msg, + ), + other => other, + })?; + ballista_client .fetch_partition( + &metadata.id, &partition_id.job_id, - partition_id.stage_id as usize, - partition_id.partition_id as usize, + partition_id.stage_id, + partition_id.partition_id, &location.path, ) .await - .map_err(|e| DataFusionError::Execution(format!("{:?}", e))) } #[cfg(test)] diff --git a/ballista/rust/core/src/utils.rs b/ballista/rust/core/src/utils.rs index ae8d57731..c00d3cbeb 100644 --- a/ballista/rust/core/src/utils.rs +++ b/ballista/rust/core/src/utils.rs @@ -46,6 +46,7 @@ use datafusion_proto::logical_plan::{ AsLogicalPlan, DefaultLogicalExtensionCodec, LogicalExtensionCodec, }; use futures::StreamExt; +use log::error; use std::io::{BufWriter, Write}; use std::marker::PhantomData; use std::sync::atomic::{AtomicUsize, Ordering}; @@ -63,10 +64,8 @@ pub async fn write_stream_to_disk( disk_write_metric: &metrics::Time, ) -> Result { let file = File::create(&path).map_err(|e| { - BallistaError::General(format!( - "Failed to create partition file at {}: {:?}", - path, e - )) + error!("Failed to create partition file at {}: {:?}", path, e); + BallistaError::IoError(e) })?; let mut num_rows = 0; diff --git a/ballista/rust/executor/src/execution_loop.rs b/ballista/rust/executor/src/execution_loop.rs index f7e029c20..3acdaa56a 100644 --- a/ballista/rust/executor/src/execution_loop.rs +++ b/ballista/rust/executor/src/execution_loop.rs @@ -22,11 +22,11 @@ use ballista_core::serde::protobuf::{ TaskDefinition, TaskStatus, }; -use crate::as_task_status; use crate::executor::Executor; +use crate::{as_task_status, TaskExecutionTimes}; use ballista_core::error::BallistaError; use ballista_core::serde::physical_plan::from_proto::parse_protobuf_hash_partitioning; -use ballista_core::serde::scheduler::ExecutorSpecification; +use ballista_core::serde::scheduler::{ExecutorSpecification, PartitionId}; use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; use ballista_core::utils::collect_plan_metrics; use datafusion::execution::context::TaskContext; @@ -40,6 +40,7 @@ use std::error::Error; use std::ops::Deref; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::mpsc::{Receiver, Sender, TryRecvError}; +use std::time::{SystemTime, UNIX_EPOCH}; use std::{sync::Arc, time::Duration}; use tonic::transport::Channel; @@ -136,16 +137,24 @@ async fn run_received_tasks, ) -> Result<(), BallistaError> { - let task_id = task.task_id.unwrap(); - let task_id_log = format!( - "{}/{}/{}", - task_id.job_id, task_id.stage_id, task_id.partition_id + let task_id = task.task_id; + let task_attempt_num = task.task_attempt_num; + let job_id = task.job_id; + let stage_id = task.stage_id; + let stage_attempt_num = task.stage_attempt_num; + let task_launch_time = task.launch_time; + let partition_id = task.partition_id; + let start_exec_time = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as u64; + let task_identity = format!( + "TID {} {}/{}.{}/{}.{}", + task_id, job_id, stage_id, stage_attempt_num, partition_id, task_attempt_num ); - info!("Received task {}", task_id_log); + info!("Received task {}", task_identity); available_tasks_slots.fetch_sub(1, Ordering::SeqCst); - let runtime = executor.runtime.clone(); - let session_id = task.session_id; let mut task_props = HashMap::new(); for kv_pair in task.props { task_props.insert(kv_pair.key, kv_pair.value); @@ -160,8 +169,10 @@ async fn run_received_tasks, BallistaError>>() .ok(); + 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_sender.send(as_task_status( execution_result, executor.metadata.id.clone(), task_id, + stage_attempt_num, + part, operator_metrics, + task_execution_times, )); }); diff --git a/ballista/rust/executor/src/executor.rs b/ballista/rust/executor/src/executor.rs index ff1cde808..47f5fe33c 100644 --- a/ballista/rust/executor/src/executor.rs +++ b/ballista/rust/executor/src/executor.rs @@ -37,7 +37,7 @@ use futures::future::AbortHandle; use ballista_core::serde::scheduler::PartitionId; use tokio::sync::Mutex; -type AbortHandles = Arc>>; +type AbortHandles = Arc>>; /// Ballista executor #[derive(Clone)] @@ -96,39 +96,34 @@ impl Executor { /// and statistics. pub async fn execute_shuffle_write( &self, - job_id: String, - stage_id: usize, - part: usize, + task_id: usize, + partition: PartitionId, shuffle_writer: Arc, task_ctx: Arc, _shuffle_output_partitioning: Option, ) -> Result, BallistaError> { let (task, abort_handle) = futures::future::abortable( - shuffle_writer.execute_shuffle_write(part, task_ctx), + shuffle_writer.execute_shuffle_write(partition.partition_id, task_ctx), ); { let mut abort_handles = self.abort_handles.lock().await; - abort_handles.insert( - PartitionId { - job_id: job_id.clone(), - stage_id, - partition_id: part, - }, - abort_handle, - ); + abort_handles.insert((task_id, partition.clone()), abort_handle); } let partitions = task.await??; - self.abort_handles.lock().await.remove(&PartitionId { - job_id: job_id.clone(), - stage_id, - partition_id: part, - }); + self.abort_handles + .lock() + .await + .remove(&(task_id, partition.clone())); - self.metrics_collector - .record_stage(&job_id, stage_id, part, shuffle_writer); + self.metrics_collector.record_stage( + &partition.job_id, + partition.stage_id, + partition.partition_id, + shuffle_writer, + ); Ok(partitions) } @@ -162,15 +157,19 @@ impl Executor { pub async fn cancel_task( &self, + task_id: usize, job_id: String, stage_id: usize, partition_id: usize, ) -> Result { - if let Some(handle) = self.abort_handles.lock().await.remove(&PartitionId { - job_id, - stage_id, - partition_id, - }) { + if let Some(handle) = self.abort_handles.lock().await.remove(&( + task_id, + PartitionId { + job_id, + stage_id, + partition_id, + }, + )) { handle.abort(); Ok(true) } else { @@ -194,6 +193,7 @@ mod test { use ballista_core::serde::protobuf::ExecutorRegistration; use datafusion::execution::context::TaskContext; + use ballista_core::serde::scheduler::PartitionId; use datafusion::physical_expr::PhysicalSortExpr; use datafusion::physical_plan::{ ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, @@ -314,11 +314,15 @@ mod test { // Spawn our non-terminating task on a separate fiber. let executor_clone = executor.clone(); tokio::task::spawn(async move { + let part = PartitionId { + job_id: "job-id".to_owned(), + stage_id: 1, + partition_id: 0, + }; let task_result = executor_clone .execute_shuffle_write( - "job-id".to_owned(), 1, - 0, + part, Arc::new(shuffle_write), ctx.task_ctx(), None, @@ -331,7 +335,7 @@ mod test { // poll until that happens. for _ in 0..20 { if executor - .cancel_task("job-id".to_owned(), 1, 0) + .cancel_task(1, "job-id".to_owned(), 1, 0) .await .expect("cancelling task") { diff --git a/ballista/rust/executor/src/executor_server.rs b/ballista/rust/executor/src/executor_server.rs index 6d0719ff0..33e9da47f 100644 --- a/ballista/rust/executor/src/executor_server.rs +++ b/ballista/rust/executor/src/executor_server.rs @@ -40,6 +40,7 @@ use ballista_core::serde::protobuf::{ RegisterExecutorParams, StopExecutorParams, StopExecutorResult, TaskDefinition, TaskStatus, UpdateTaskStatusParams, }; +use ballista_core::serde::scheduler::PartitionId; use ballista_core::serde::{AsExecutionPlan, BallistaCodec}; use ballista_core::utils::{ collect_plan_metrics, create_grpc_client_connection, create_grpc_server, @@ -50,10 +51,10 @@ use datafusion_proto::logical_plan::AsLogicalPlan; use tokio::sync::mpsc::error::TryRecvError; use tokio::task::JoinHandle; -use crate::as_task_status; use crate::cpu_bound_executor::DedicatedExecutor; use crate::executor::Executor; use crate::shutdown::ShutdownNotifier; +use crate::{as_task_status, TaskExecutionTimes}; type ServerHandle = JoinHandle>; type SchedulerClients = Arc>>>; @@ -291,19 +292,15 @@ impl ExecutorServer Result<(), BallistaError> { - let scheduler_id = curator_task.scheduler_id; + 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.unwrap(); - let task_id_log = format!( - "{}/{}/{}", - task_id.job_id, task_id.stage_id, task_id.partition_id - ); - info!("Start to run task {}", task_id_log); - - let runtime = self.executor.runtime.clone(); - let session_id = task.session_id; let mut task_props = HashMap::new(); for kv_pair in task.props { task_props.insert(kv_pair.key, kv_pair.value); @@ -318,8 +315,11 @@ impl ExecutorServer ExecutorServer ExecutorServer, BallistaError>>()?; 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, Some(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 { @@ -560,28 +583,29 @@ impl TaskRunnerPool ExecutorGrpc &self, request: Request, ) -> Result, Status> { - let partitions = request.into_inner().partition_id; - info!("Cancelling partition tasks for {:?}", partitions); + let task_infos = request.into_inner().task_infos; + info!("Cancelling tasks for {:?}", task_infos); let mut cancelled = true; - for partition in partitions { + for task in task_infos { if let Err(e) = self .executor .cancel_task( - partition.job_id, - partition.stage_id as usize, - partition.partition_id as usize, + task.task_id as usize, + task.job_id, + task.stage_id as usize, + task.partition_id as usize, ) .await { diff --git a/ballista/rust/executor/src/lib.rs b/ballista/rust/executor/src/lib.rs index 88578cbd5..9be576fff 100644 --- a/ballista/rust/executor/src/lib.rs +++ b/ballista/rust/executor/src/lib.rs @@ -33,15 +33,26 @@ pub use standalone::new_standalone_executor; use log::info; use ballista_core::serde::protobuf::{ - task_status, CompletedTask, FailedTask, OperatorMetricsSet, PartitionId, - ShuffleWritePartition, TaskStatus, + task_status, FailedTask, OperatorMetricsSet, ShuffleWritePartition, SuccessfulTask, + TaskStatus, }; +use ballista_core::serde::scheduler::PartitionId; + +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct TaskExecutionTimes { + launch_time: u64, + start_exec_time: u64, + end_exec_time: u64, +} pub fn as_task_status( execution_result: ballista_core::error::Result>, executor_id: String, - task_id: PartitionId, + task_id: u32, + stage_attempt_num: u32, + partition_id: PartitionId, operator_metrics: Option>, + execution_times: TaskExecutionTimes, ) -> TaskStatus { let metrics = operator_metrics.unwrap_or_default(); match execution_result { @@ -52,9 +63,16 @@ pub fn as_task_status( metrics.len() ); TaskStatus { - task_id: Some(task_id), + task_id, + job_id: partition_id.job_id, + stage_id: partition_id.stage_id as u32, + stage_attempt_num, + partition_id: partition_id.partition_id as u32, + launch_time: execution_times.launch_time, + start_exec_time: execution_times.start_exec_time, + end_exec_time: execution_times.end_exec_time, metrics, - status: Some(task_status::Status::Completed(CompletedTask { + status: Some(task_status::Status::Successful(SuccessfulTask { executor_id, partitions, })), @@ -65,11 +83,16 @@ pub fn as_task_status( info!("Task {:?} failed: {}", task_id, error_msg); TaskStatus { - task_id: Some(task_id), + task_id, + job_id: partition_id.job_id, + stage_id: partition_id.stage_id as u32, + stage_attempt_num, + partition_id: partition_id.partition_id as u32, + launch_time: execution_times.launch_time, + start_exec_time: execution_times.start_exec_time, + end_exec_time: execution_times.end_exec_time, metrics, - status: Some(task_status::Status::Failed(FailedTask { - error: format!("Task failed due to Tokio error: {}", error_msg), - })), + status: Some(task_status::Status::Failed(FailedTask::from(e))), } } } diff --git a/ballista/rust/scheduler/src/scheduler_server/event.rs b/ballista/rust/scheduler/src/scheduler_server/event.rs index ad462944a..c748bc756 100644 --- a/ballista/rust/scheduler/src/scheduler_server/event.rs +++ b/ballista/rust/scheduler/src/scheduler_server/event.rs @@ -19,6 +19,7 @@ use crate::state::executor_manager::ExecutorReservation; use datafusion::logical_plan::LogicalPlan; +use crate::state::execution_graph::RunningTaskInfo; use ballista_core::serde::protobuf::TaskStatus; use datafusion::prelude::SessionContext; use std::sync::Arc; @@ -35,9 +36,10 @@ pub enum QueryStageSchedulerEvent { JobPlanningFailed(String, String), JobFinished(String), // For a job fails with its execution graph setting failed - JobRunningFailed(String), + JobRunningFailed(String, String), JobUpdated(String), TaskUpdating(String, Vec), ReservationOffering(Vec), ExecutorLost(String, Option), + CancelTasks(Vec), } diff --git a/ballista/rust/scheduler/src/scheduler_server/grpc.rs b/ballista/rust/scheduler/src/scheduler_server/grpc.rs index 09f92cbd9..f4bc9fc00 100644 --- a/ballista/rust/scheduler/src/scheduler_server/grpc.rs +++ b/ballista/rust/scheduler/src/scheduler_server/grpc.rs @@ -531,17 +531,21 @@ impl SchedulerGrpc let job_id = request.into_inner().job_id; info!("Received cancellation request for job {}", job_id); - self.state - .task_manager - .cancel_job(&job_id, &self.state.executor_manager) - .await - .map_err(|e| { + match self.state.task_manager.cancel_job(&job_id).await { + Ok(tasks) => { + self.state.executor_manager.cancel_running_tasks(tasks).await.map_err(|e| { + let msg = format!("Error to cancel running task when cancel the job {} due to {:?}", job_id, e); + error!("{}", msg); + Status::internal(msg) + })?; + Ok(Response::new(CancelJobResult { cancelled: true })) + } + Err(e) => { let msg = format!("Error cancelling job {}: {:?}", job_id, e); - error!("{}", msg); - Status::internal(msg) - })?; - Ok(Response::new(CancelJobResult { cancelled: true })) + Ok(Response::new(CancelJobResult { cancelled: false })) + } + } } } diff --git a/ballista/rust/scheduler/src/scheduler_server/mod.rs b/ballista/rust/scheduler/src/scheduler_server/mod.rs index fed268cc9..48122c108 100644 --- a/ballista/rust/scheduler/src/scheduler_server/mod.rs +++ b/ballista/rust/scheduler/src/scheduler_server/mod.rs @@ -291,8 +291,8 @@ mod test { use ballista_core::error::Result; use ballista_core::serde::protobuf::{ - job_status, task_status, CompletedTask, FailedTask, JobStatus, PartitionId, - PhysicalPlanNode, ShuffleWritePartition, TaskStatus, + failed_task, job_status, task_status, ExecutionError, FailedTask, JobStatus, + PhysicalPlanNode, ShuffleWritePartition, SuccessfulTask, TaskStatus, }; use ballista_core::serde::scheduler::{ ExecutorData, ExecutorMetadata, ExecutorSpecification, @@ -370,16 +370,19 @@ mod test { // Complete the task let task_status = TaskStatus { - status: Some(task_status::Status::Completed(CompletedTask { + task_id: task.task_id as u32, + job_id: task.partition.job_id.clone(), + stage_id: task.partition.stage_id as u32, + stage_attempt_num: task.stage_attempt_num as u32, + partition_id: task.partition.partition_id as u32, + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + metrics: vec![], + status: Some(task_status::Status::Successful(SuccessfulTask { executor_id: "executor-1".to_owned(), partitions, })), - metrics: vec![], - task_id: Some(PartitionId { - job_id: job_id.to_owned(), - stage_id: task.partition.stage_id as u32, - partition_id: task.partition.partition_id as u32, - }), }; scheduler @@ -399,7 +402,7 @@ mod test { .expect("Fail to find graph in the cache"); let final_graph = final_graph.read().await; - assert!(final_graph.complete()); + assert!(final_graph.is_successful()); assert_eq!(final_graph.output_locations().len(), 4); for output_location in final_graph.output_locations() { @@ -450,7 +453,7 @@ mod test { .await .unwrap(); let graph = graph.read().await; - if graph.complete() { + if graph.is_successful() { break; } graph.available_tasks() @@ -504,18 +507,21 @@ mod test { // Complete the task let task_status = TaskStatus { - status: Some(task_status::Status::Completed( - CompletedTask { + task_id: task.task_id as u32, + job_id: task.partition.job_id.clone(), + stage_id: task.partition.stage_id as u32, + stage_attempt_num: task.stage_attempt_num as u32, + partition_id: task.partition.partition_id as u32, + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + metrics: vec![], + status: Some(task_status::Status::Successful( + SuccessfulTask { executor_id: executor.id.clone(), partitions, }, )), - metrics: vec![], - task_id: Some(PartitionId { - job_id: job_id.to_owned(), - stage_id: task.partition.stage_id as u32, - partition_id: task.partition.partition_id as u32, - }), }; scheduler @@ -550,7 +556,7 @@ mod test { .get_execution_graph(job_id) .await?; - assert!(final_graph.complete()); + assert!(final_graph.is_successful()); assert_eq!(final_graph.output_locations().len(), 4); Ok(()) @@ -635,15 +641,25 @@ mod test { // Complete the task let task_status = TaskStatus { + task_id: task.task_id as u32, + job_id: task.partition.job_id.clone(), + stage_id: task.partition.stage_id as u32, + stage_attempt_num: task.stage_attempt_num as u32, + partition_id: task.partition.partition_id as u32, + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + metrics: vec![], status: Some(task_status::Status::Failed(FailedTask { error: "".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some( + failed_task::FailedReason::ExecutionError( + ExecutionError {}, + ), + ), })), - metrics: vec![], - task_id: Some(PartitionId { - job_id: job_id.to_owned(), - stage_id: task.partition.stage_id as u32, - partition_id: task.partition.partition_id as u32, - }), }; scheduler diff --git a/ballista/rust/scheduler/src/scheduler_server/query_stage_scheduler.rs b/ballista/rust/scheduler/src/scheduler_server/query_stage_scheduler.rs index 7d186fcdf..02a22d5ab 100644 --- a/ballista/rust/scheduler/src/scheduler_server/query_stage_scheduler.rs +++ b/ballista/rust/scheduler/src/scheduler_server/query_stage_scheduler.rs @@ -125,20 +125,29 @@ impl .await?; } } - QueryStageSchedulerEvent::JobPlanningFailed(job_id, fail_message) => { - error!("Job {} failed: {}", job_id, fail_message); + QueryStageSchedulerEvent::JobPlanningFailed(job_id, failure_reason) => { + error!("Job {} failed: {}", job_id, failure_reason); self.state .task_manager - .fail_job(&job_id, fail_message) + .fail_unscheduled_job(&job_id, failure_reason) .await?; } QueryStageSchedulerEvent::JobFinished(job_id) => { - info!("Job {} complete", job_id); - self.state.task_manager.complete_job(&job_id).await?; + info!("Job {} success", job_id); + self.state.task_manager.succeed_job(&job_id).await?; } - QueryStageSchedulerEvent::JobRunningFailed(job_id) => { + QueryStageSchedulerEvent::JobRunningFailed(job_id, failure_reason) => { error!("Job {} running failed", job_id); - self.state.task_manager.fail_running_job(&job_id).await?; + let tasks = self + .state + .task_manager + .abort_job(&job_id, failure_reason) + .await?; + if !tasks.is_empty() { + tx_event + .post_event(QueryStageSchedulerEvent::CancelTasks(tasks)) + .await?; + } } QueryStageSchedulerEvent::JobUpdated(job_id) => { info!("Job {} Updated", job_id); @@ -184,17 +193,28 @@ impl } } QueryStageSchedulerEvent::ExecutorLost(executor_id, _) => { - self.state - .task_manager - .executor_lost(&executor_id) - .await - .unwrap_or_else(|e| { + match self.state.task_manager.executor_lost(&executor_id).await { + Ok(tasks) => { + if !tasks.is_empty() { + tx_event + .post_event(QueryStageSchedulerEvent::CancelTasks(tasks)) + .await?; + } + } + Err(e) => { let msg = format!( "TaskManager error to handle Executor {} lost: {}", executor_id, e ); error!("{}", msg); - }); + } + } + } + QueryStageSchedulerEvent::CancelTasks(tasks) => { + self.state + .executor_manager + .cancel_running_tasks(tasks) + .await? } } diff --git a/ballista/rust/scheduler/src/state/execution_graph.rs b/ballista/rust/scheduler/src/state/execution_graph.rs index 08e97d528..30d903bcd 100644 --- a/ballista/rust/scheduler/src/state/execution_graph.rs +++ b/ballista/rust/scheduler/src/state/execution_graph.rs @@ -18,7 +18,9 @@ use std::collections::{HashMap, HashSet}; use std::convert::TryInto; use std::fmt::{Debug, Formatter}; +use std::iter::FromIterator; use std::sync::Arc; +use std::time::{SystemTime, UNIX_EPOCH}; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{ @@ -30,9 +32,10 @@ use log::{error, info, warn}; use ballista_core::error::{BallistaError, Result}; use ballista_core::execution_plans::{ShuffleWriterExec, UnresolvedShuffleExec}; +use ballista_core::serde::protobuf::failed_task::FailedReason; use ballista_core::serde::protobuf::{ - self, execution_graph_stage::StageType, CompletedJob, JobStatus, QueuedJob, - TaskStatus, + self, execution_graph_stage::StageType, FailedTask, JobStatus, QueuedJob, ResultLost, + SuccessfulJob, TaskStatus, }; use ballista_core::serde::protobuf::{job_status, FailedJob, ShuffleWritePartition}; use ballista_core::serde::protobuf::{task_status, RunningTask}; @@ -45,9 +48,10 @@ use crate::display::print_stage_metrics; use crate::planner::DistributedPlanner; use crate::scheduler_server::event::QueryStageSchedulerEvent; use crate::state::execution_graph::execution_stage::{ - CompletedStage, ExecutionStage, FailedStage, ResolvedStage, StageOutput, + ExecutionStage, FailedStage, ResolvedStage, StageOutput, SuccessfulStage, TaskInfo, UnresolvedStage, }; +use crate::state::task_manager::UpdatedStages; mod execution_stage; @@ -110,6 +114,20 @@ pub struct ExecutionGraph { output_partitions: usize, /// Locations of this `ExecutionGraph` final output locations output_locations: Vec, + /// Task ID generator, generate unique TID in the execution graph + tid_generator: usize, + /// Failed stage attempts, record the failed stage attempts to limit the retry times. + /// Map from Stage ID -> Set + failed_stage_attempts: HashMap>, +} + +#[derive(Clone)] +pub struct RunningTaskInfo { + pub task_id: usize, + pub job_id: String, + pub stage_id: usize, + pub partition_id: usize, + pub executor_id: String, } impl ExecutionGraph { @@ -138,6 +156,8 @@ impl ExecutionGraph { stages, output_partitions, output_locations: vec![], + tid_generator: 0, + failed_stage_attempts: HashMap::new(), }) } @@ -157,11 +177,17 @@ impl ExecutionGraph { self.stages.len() } - /// An ExecutionGraph is complete if all its stages are complete - pub fn complete(&self) -> bool { + pub fn next_task_id(&mut self) -> usize { + let new_tid = self.tid_generator; + self.tid_generator += 1; + new_tid + } + + /// An ExecutionGraph is successful if all its stages are successful + pub fn is_successful(&self) -> bool { self.stages .values() - .all(|s| matches!(s, ExecutionStage::Completed(_))) + .all(|s| matches!(s, ExecutionStage::Successful(_))) } /// Revive the execution graph by converting the resolved stages to running stages @@ -198,68 +224,153 @@ impl ExecutionGraph { &mut self, executor: &ExecutorMetadata, task_statuses: Vec, - ) -> Result> { + max_task_failures: usize, + max_stage_failures: usize, + ) -> Result> { let job_id = self.job_id().to_owned(); // First of all, classify the statuses by stages let mut job_task_statuses: HashMap> = HashMap::new(); for task_status in task_statuses { - if let Some(task_id) = task_status.task_id.as_ref() { - if task_id.job_id != job_id { - return Err(BallistaError::Internal(format!( - "Error updating job {}: Invalid task status job ID {}", - job_id, task_id.job_id - ))); - } - let stage_task_statuses = job_task_statuses - .entry(task_id.stage_id as usize) - .or_insert_with(Vec::new); - stage_task_statuses.push(task_status); - } else { - error!("There's no task id when updating status"); - } + let stage_id = task_status.stage_id as usize; + let stage_task_statuses = + job_task_statuses.entry(stage_id).or_insert_with(Vec::new); + stage_task_statuses.push(task_status); } // Revive before updating due to some updates not saved // It will be refined later self.revive(); - let mut events = vec![]; + // Copy the failed stage attempts from self + let mut failed_stage_attempts: HashMap> = HashMap::new(); + for (stage_id, attempts) in self.failed_stage_attempts.iter() { + failed_stage_attempts + .insert(*stage_id, HashSet::from_iter(attempts.iter().copied())); + } + + let mut resolved_stages = vec![]; + let mut successful_stages = vec![]; + + let mut failed_stages = HashMap::new(); + + let mut rollback_running_stages = HashSet::new(); + let mut resubmit_successful_stages: HashMap> = + HashMap::new(); + for (stage_id, stage_task_statuses) in job_task_statuses { if let Some(stage) = self.stages.get_mut(&stage_id) { if let ExecutionStage::Running(running_stage) = stage { let mut locations = vec![]; for task_status in stage_task_statuses.into_iter() { - if let TaskStatus { - task_id: - Some(protobuf::PartitionId { - job_id, - stage_id, - partition_id, - }), - metrics: operator_metrics, - status: Some(status), - } = task_status { let stage_id = stage_id as usize; - let partition_id = partition_id as usize; + let task_stage_attempt_num = + task_status.stage_attempt_num as usize; + if task_stage_attempt_num < running_stage.stage_attempt_num { + warn!("Ignore TaskStatus update with TID {} as it's from Stage {}.{} and there is a more recent stage attempt {}.{} running", + task_status.task_id, stage_id, task_stage_attempt_num, stage_id, running_stage.stage_attempt_num); + continue; + } + let partition_id = task_status.clone().partition_id as usize; + let operator_metrics = task_status.metrics.clone(); - running_stage - .update_task_status(partition_id, status.clone()); + if !running_stage + .update_task_info(partition_id, task_status.clone()) + { + continue; + } - // TODO Should be able to reschedule this task. - if let task_status::Status::Failed(failed_task) = status { - events.push(StageEvent::StageFailed( - stage_id, - format!( - "Task {}/{}/{} failed: {}", - job_id, stage_id, partition_id, failed_task.error - ), - )); - break; - } else if let task_status::Status::Completed(completed_task) = - status + if let Some(task_status::Status::Failed(failed_task)) = + task_status.status + { + let failed_reason = failed_task.failed_reason; + + match failed_reason { + Some(FailedReason::FetchPartitionError( + fetch_partiton_error, + )) => { + let failed_attempts = failed_stage_attempts + .entry(stage_id) + .or_insert_with(HashSet::new); + failed_attempts.insert(task_stage_attempt_num); + if failed_attempts.len() < max_stage_failures { + let map_stage_id = + fetch_partiton_error.map_stage_id; + let map_partition_id = + fetch_partiton_error.map_partition_id; + + if failed_stages.contains_key(&stage_id) { + let error_msg = format!( + "Stage {} was marked failed, ignore FetchPartitionError from task with TID {}",stage_id, task_status.task_id); + warn!("{}", error_msg); + } else { + running_stage.remove_input_partition( + map_stage_id as usize, + map_partition_id as usize, + )?; + + rollback_running_stages.insert(stage_id); + let missing_inputs = + resubmit_successful_stages + .entry(map_stage_id as usize) + .or_insert_with(HashSet::new); + missing_inputs + .insert(map_partition_id as usize); + } + } else { + let error_msg = format!( + "Stage {} has failed {} times, \ + most recent failure reason: {:?}", + stage_id, + max_stage_failures, + failed_task.error + ); + error!("{}", error_msg); + failed_stages.insert(stage_id, error_msg); + } + } + Some(FailedReason::ExecutionError(_)) => { + failed_stages.insert(stage_id, failed_task.error); + } + Some(_) => { + if failed_task.retryable + && failed_task.count_to_failures + { + if running_stage + .task_failure_number(partition_id) + < max_task_failures + { + // TODO add new struct to track all the failed task infos + // The failure TaskInfo is ignored and set to None here + running_stage + .reset_task_info(partition_id); + } else { + let error_msg = format!( + "Task {} in Stage {} failed {} times, fail the stage, most recent failure reason: {:?}", + partition_id, stage_id, max_task_failures, failed_task.error + ); + error!("{}", error_msg); + failed_stages.insert(stage_id, error_msg); + } + } else if failed_task.retryable { + // TODO add new struct to track all the failed task infos + // The failure TaskInfo is ignored and set to None here + running_stage.reset_task_info(partition_id); + } + } + None => { + let error_msg = format!( + "Task {} in Stage {} failed with unknown failure reasons, fail the stage", + partition_id, stage_id); + error!("{}", error_msg); + failed_stages.insert(stage_id, error_msg); + } + } + } else if let Some(task_status::Status::Successful( + successful_task, + )) = task_status.status { - // update task metrics for completed task + // update task metrics for successfu task running_stage.update_task_metrics( partition_id, operator_metrics, @@ -269,17 +380,20 @@ impl ExecutionGraph { &job_id, stage_id, executor, - completed_task.partitions, + successful_task.partitions, )); } else { - warn!("The task {}/{}/{} with status {:?} is invalid for updating", job_id, stage_id, partition_id, status); + warn!( + "The task {}/{}/{}'s status is invalid for updating", + job_id, stage_id, partition_id + ); } } } - let is_completed = running_stage.is_completed(); - if is_completed { - events.push(StageEvent::StageCompleted(stage_id)); - // if this stage is completed, we want to combine the stage metrics to plan's metric set and print out the plan + let is_successful = running_stage.is_successful(); + if is_successful { + successful_stages.push(stage_id); + // if this stage is successful, we want to combine the stage metrics to plan's metric set and print out the plan if let Some(stage_metrics) = running_stage.stage_metrics.as_ref() { print_stage_metrics( @@ -292,9 +406,9 @@ impl ExecutionGraph { } let output_links = running_stage.output_links.clone(); - events.append(&mut self.update_stage_output_links( + resolved_stages.append(&mut self.update_stage_output_links( stage_id, - is_completed, + is_successful, locations, output_links, )?); @@ -303,7 +417,7 @@ impl ExecutionGraph { "Stage {}/{} is not in running when updating the status of tasks {:?}", job_id, stage_id, - stage_task_statuses.into_iter().map(|task_status| task_status.task_id.map(|task_id| task_id.partition_id)).collect::>(), + stage_task_statuses.into_iter().map(|task_status| task_status.partition_id).collect::>(), ); } } else { @@ -314,17 +428,122 @@ impl ExecutionGraph { } } - self.processing_stage_events(events) + // Update failed stage attempts back to self + for (stage_id, attempts) in failed_stage_attempts.iter() { + self.failed_stage_attempts + .insert(*stage_id, HashSet::from_iter(attempts.iter().copied())); + } + + for (stage_id, missing_parts) in &resubmit_successful_stages { + if let Some(stage) = self.stages.get_mut(stage_id) { + if let ExecutionStage::Successful(success_stage) = stage { + for partition in missing_parts { + let task_info = &mut success_stage.task_infos[*partition]; + // Update the task info to failed + task_info.task_status = task_status::Status::Failed(FailedTask { + error: "FetchPartitionError in parent stage".to_owned(), + retryable: true, + count_to_failures: false, + failed_reason: Some(FailedReason::ResultLost(ResultLost {})), + }); + } + } else { + warn!( + "Stage {}/{} is not in Successful state when try to resubmit this stage. ", + job_id, + stage_id); + } + } else { + return Err(BallistaError::Internal(format!( + "Invalid stage ID {} for job {}", + stage_id, job_id + ))); + } + } + + self.processing_stages_update(UpdatedStages { + resolved_stages, + successful_stages, + failed_stages, + rollback_running_stages, + resubmit_successful_stages: resubmit_successful_stages + .keys() + .cloned() + .collect(), + }) } + /// Processing stage status update after task status changing + fn processing_stages_update( + &mut self, + updated_stages: UpdatedStages, + ) -> Result> { + let job_id = self.job_id().to_owned(); + let mut has_resolved = false; + let mut job_err_msg = "".to_owned(); + + for stage_id in updated_stages.resolved_stages { + self.resolve_stage(stage_id)?; + has_resolved = true; + } + + for stage_id in updated_stages.successful_stages { + self.succeed_stage(stage_id); + } + + // Fail the stage and also abort the job + for (stage_id, err_msg) in &updated_stages.failed_stages { + job_err_msg = + format!("Job failed due to stage {} failed: {}\n", stage_id, err_msg); + } + + let mut events = vec![]; + // Only handle the rollback logic when there are no failed stages + if updated_stages.failed_stages.is_empty() { + let mut running_tasks_to_cancel = vec![]; + for stage_id in updated_stages.rollback_running_stages { + let tasks = self.rollback_running_stage(stage_id)?; + running_tasks_to_cancel.extend(tasks); + } + + for stage_id in updated_stages.resubmit_successful_stages { + self.rerun_successful_stage(stage_id); + } + + if !running_tasks_to_cancel.is_empty() { + events.push(QueryStageSchedulerEvent::CancelTasks( + running_tasks_to_cancel, + )); + } + } + + if !updated_stages.failed_stages.is_empty() { + info!("Job {} is failed", job_id); + self.fail_job(job_err_msg.clone()); + events.push(QueryStageSchedulerEvent::JobRunningFailed( + job_id, + job_err_msg, + )); + } else if self.is_successful() { + // If this ExecutionGraph is successful, finish it + info!("Job {} is success, finalizing output partitions", job_id); + self.succeed_job()?; + events.push(QueryStageSchedulerEvent::JobFinished(job_id)); + } else if has_resolved { + events.push(QueryStageSchedulerEvent::JobUpdated(job_id)) + } + Ok(events) + } + + /// Return a Vec of resolvable stage ids fn update_stage_output_links( &mut self, stage_id: usize, is_completed: bool, locations: Vec, output_links: Vec, - ) -> Result> { - let mut ret = vec![]; + ) -> Result> { + let mut resolved_stages = vec![]; let job_id = &self.job_id; if output_links.is_empty() { // If `output_links` is empty, then this is a final stage @@ -346,9 +565,7 @@ impl ExecutionGraph { // If all input partitions are ready, we can resolve any UnresolvedShuffleExec in the parent stage plan if linked_unresolved_stage.resolvable() { - ret.push(StageEvent::StageResolved( - linked_unresolved_stage.stage_id, - )); + resolved_stages.push(linked_unresolved_stage.stage_id); } } else { return Err(BallistaError::Internal(format!( @@ -364,12 +581,25 @@ impl ExecutionGraph { } } } + Ok(resolved_stages) + } - Ok(ret) + /// Return all the currently running stage ids + pub fn running_stages(&self) -> Vec { + self.stages + .iter() + .filter_map(|(stage_id, stage)| { + if let ExecutionStage::Running(_running) = stage { + Some(*stage_id) + } else { + None + } + }) + .collect::>() } /// Return all currently running tasks along with the executor ID on which they are assigned - pub fn running_tasks(&self) -> Vec<(PartitionId, String)> { + pub fn running_tasks(&self) -> Vec { self.stages .iter() .flat_map(|(_, stage)| { @@ -377,22 +607,21 @@ impl ExecutionGraph { stage .running_tasks() .into_iter() - .map(|(stage_id, partition_id, executor_id)| { - ( - PartitionId { - job_id: self.job_id.clone(), - stage_id, - partition_id, - }, + .map(|(task_id, stage_id, partition_id, executor_id)| { + RunningTaskInfo { + task_id, + job_id: self.job_id.clone(), + stage_id, + partition_id, executor_id, - ) + } }) - .collect::>() + .collect::>() } else { vec![] } }) - .collect::>() + .collect::>() } /// Total number of tasks in this plan that are ready for scheduling @@ -415,9 +644,23 @@ impl ExecutionGraph { /// available to the scheduler. /// If the task is not launched the status must be reset to allow the task to /// be scheduled elsewhere. - pub fn pop_next_task(&mut self, executor_id: &str) -> Result> { + pub fn pop_next_task(&mut self, executor_id: &str) -> Result> { let job_id = self.job_id.clone(); let session_id = self.session_id.clone(); + + let find_candidate = self.stages.iter().any(|(_stage_id, stage)| { + if let ExecutionStage::Running(stage) = stage { + stage.available_tasks() > 0 + } else { + false + } + }); + let next_task_id = if find_candidate { + Some(self.next_task_id()) + } else { + None + }; + let mut next_task = self.stages.iter_mut().find(|(_stage_id, stage)| { if let ExecutionStage::Running(stage) = stage { stage.available_tasks() > 0 @@ -427,10 +670,10 @@ impl ExecutionGraph { }).map(|(stage_id, stage)| { if let ExecutionStage::Running(stage) = stage { let (partition_id, _) = stage - .task_statuses + .task_infos .iter() .enumerate() - .find(|(_partition, status)| status.is_none()) + .find(|(_partition, info)| info.is_none()) .ok_or_else(|| { BallistaError::Internal(format!("Error getting next task for job {}: Stage {} is ready but has no pending tasks", job_id, stage_id)) })?; @@ -441,14 +684,33 @@ impl ExecutionGraph { partition_id, }; - // Set the status to Running - stage.task_statuses[partition_id] = Some(task_status::Status::Running(RunningTask { - executor_id: executor_id.to_owned() - })); + let task_id = next_task_id.unwrap(); + let task_attempt = stage.task_failure_numbers[partition_id]; + let task_info = TaskInfo { + task_id, + scheduled_time: SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis(), + // Those times will be updated when the task finish + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + finish_time: 0, + task_status: task_status::Status::Running(RunningTask { + executor_id: executor_id.to_owned() + }), + }; + + // Set the task info to Running for new task + stage.task_infos[partition_id] = Some(task_info); - Ok(Task { + Ok(TaskDefinition { session_id, partition, + stage_attempt_num: stage.stage_attempt_num, + task_id, + task_attempt, plan: stage.plan.clone(), output_partitioning: stage.output_partitioning.clone(), }) @@ -474,47 +736,47 @@ impl ExecutionGraph { self.status = status; } - /// Reset the status for the given task. This should be called is a task failed to - /// launch and it needs to be returned to the set of available tasks and be - /// re-scheduled. - pub fn reset_task_status(&mut self, task: Task) { - let stage_id = task.partition.stage_id; - let partition = task.partition.partition_id; - - if let Some(ExecutionStage::Running(stage)) = self.stages.get_mut(&stage_id) { - stage.task_statuses[partition] = None; - } - } - pub fn output_locations(&self) -> Vec { self.output_locations.clone() } - /// Reset running and completed stages on a given executor - /// This will first check the unresolved/resolved/running stages and reset the running tasks and completed tasks. - /// Then it will check the completed stage and whether there are running parent stages need to read shuffle from it. - /// If yes, reset the complete tasks and roll back the resolved shuffle recursively. + /// Reset running and successful stages on a given executor + /// This will first check the unresolved/resolved/running stages and reset the running tasks and successful tasks. + /// Then it will check the successful stage and whether there are running parent stages need to read shuffle from it. + /// If yes, reset the successful tasks and roll back the resolved shuffle recursively. /// - /// Returns the reset stage ids - pub fn reset_stages(&mut self, executor_id: &str) -> Result> { + /// Returns the reset stage ids and running tasks should be killed + pub fn reset_stages_on_lost_executor( + &mut self, + executor_id: &str, + ) -> Result<(HashSet, Vec)> { let mut reset = HashSet::new(); + let mut tasks_to_cancel = vec![]; loop { let reset_stage = self.reset_stages_internal(executor_id)?; - if !reset_stage.is_empty() { - reset.extend(reset_stage.iter()); + if !reset_stage.0.is_empty() { + reset.extend(reset_stage.0.iter()); + tasks_to_cancel.extend(reset_stage.1) } else { - return Ok(reset); + return Ok((reset, tasks_to_cancel)); } } } - fn reset_stages_internal(&mut self, executor_id: &str) -> Result> { - let mut reset_stage = HashSet::new(); + fn reset_stages_internal( + &mut self, + executor_id: &str, + ) -> Result<(HashSet, Vec)> { let job_id = self.job_id.clone(); - let mut stage_events = vec![]; + // collect the input stages that need to resubmit let mut resubmit_inputs: HashSet = HashSet::new(); - let mut empty_inputs: HashMap = HashMap::new(); + let mut reset_running_stage = HashSet::new(); + let mut rollback_resolved_stages = HashSet::new(); + let mut rollback_running_stages = HashSet::new(); + let mut resubmit_successful_stages = HashSet::new(); + + let mut empty_inputs: HashMap = HashMap::new(); // check the unresolved, resolved and running stages self.stages .iter_mut() @@ -533,7 +795,7 @@ impl ExecutionGraph { "Reset {} tasks for running job/stage {}/{} on lost Executor {}", reset, job_id, stage_id, executor_id ); - reset_stage.insert(*stage_id); + reset_running_stage.insert(*stage_id); } &mut stage.inputs } @@ -541,7 +803,7 @@ impl ExecutionGraph { }; // For each stage input, check whether there are input locations match that executor - // and calculate the resubmit input stages if the input stages are completed. + // and calculate the resubmit input stages if the input stages are successful. let mut rollback_stage = false; stage_inputs.iter_mut().for_each(|(input_stage_id, stage_output)| { let mut match_found = false; @@ -574,32 +836,31 @@ impl ExecutionGraph { if rollback_stage { match stage { ExecutionStage::Resolved(_) => { - stage_events.push(StageEvent::RollBackResolvedStage(*stage_id)); + rollback_resolved_stages.insert(*stage_id); warn!( "Roll back resolved job/stage {}/{} and change ShuffleReaderExec back to UnresolvedShuffleExec", job_id, stage_id); - reset_stage.insert(*stage_id); + }, ExecutionStage::Running(_) => { - stage_events.push(StageEvent::RollBackRunningStage(*stage_id)); + rollback_running_stages.insert(*stage_id); warn!( "Roll back running job/stage {}/{} and change ShuffleReaderExec back to UnresolvedShuffleExec", job_id, stage_id); - reset_stage.insert(*stage_id); }, _ => {}, } } }); - // check and reset the complete stages + // check and reset the successful stages if !resubmit_inputs.is_empty() { self.stages .iter_mut() .filter(|(stage_id, _stage)| resubmit_inputs.contains(stage_id)) .filter_map(|(_stage_id, stage)| { - if let ExecutionStage::Completed(completed) = stage { - Some(completed) + if let ExecutionStage::Successful(success) = stage { + Some(success) } else { None } @@ -607,79 +868,39 @@ impl ExecutionGraph { .for_each(|stage| { let reset = stage.reset_tasks(executor_id); if reset > 0 { - stage_events - .push(StageEvent::ReRunCompletedStage(stage.stage_id)); - reset_stage.insert(stage.stage_id); + resubmit_successful_stages.insert(stage.stage_id); warn!( - "Reset {} tasks for completed job/stage {}/{} on lost Executor {}", + "Reset {} tasks for successful job/stage {}/{} on lost Executor {}", reset, job_id, stage.stage_id, executor_id ) } }); } - self.processing_stage_events(stage_events)?; - Ok(reset_stage) - } - /// Processing stage events for stage state changing - pub fn processing_stage_events( - &mut self, - events: Vec, - ) -> Result> { - let mut has_resolved = false; - let mut job_err_msg = "".to_owned(); - for event in events { - match event { - StageEvent::StageResolved(stage_id) => { - self.resolve_stage(stage_id)?; - has_resolved = true; - } - StageEvent::StageCompleted(stage_id) => { - self.complete_stage(stage_id); - } - StageEvent::StageFailed(stage_id, err_msg) => { - job_err_msg = format!("{}{}\n", job_err_msg, &err_msg); - self.fail_stage(stage_id, err_msg); - } - StageEvent::RollBackRunningStage(stage_id) => { - self.rollback_running_stage(stage_id)?; - } - StageEvent::RollBackResolvedStage(stage_id) => { - self.rollback_resolved_stage(stage_id)?; - } - StageEvent::ReRunCompletedStage(stage_id) => { - self.rerun_completed_stage(stage_id); - } - } + for stage_id in rollback_resolved_stages.iter() { + self.rollback_resolved_stage(*stage_id)?; } - let event = if !job_err_msg.is_empty() { - // If this ExecutionGraph is complete, fail it - info!("Job {} is failed", self.job_id()); - self.fail_job(job_err_msg); + let mut all_running_tasks = vec![]; + for stage_id in rollback_running_stages.iter() { + let tasks = self.rollback_running_stage(*stage_id)?; + all_running_tasks.extend(tasks); + } - Some(QueryStageSchedulerEvent::JobRunningFailed( - self.job_id.clone(), - )) - } else if self.complete() { - // If this ExecutionGraph is complete, finalize it - info!( - "Job {} is complete, finalizing output partitions", - self.job_id() - ); - self.complete_job()?; - Some(QueryStageSchedulerEvent::JobFinished(self.job_id.clone())) - } else if has_resolved { - Some(QueryStageSchedulerEvent::JobUpdated(self.job_id.clone())) - } else { - None - }; + for stage_id in resubmit_successful_stages.iter() { + self.rerun_successful_stage(*stage_id); + } - Ok(event) + let mut reset_stage = HashSet::new(); + reset_stage.extend(reset_running_stage); + reset_stage.extend(rollback_resolved_stages); + reset_stage.extend(rollback_running_stages); + reset_stage.extend(resubmit_successful_stages); + Ok((reset_stage, all_running_tasks)) } /// Convert unresolved stage to be resolved - fn resolve_stage(&mut self, stage_id: usize) -> Result { + pub fn resolve_stage(&mut self, stage_id: usize) -> Result { if let Some(ExecutionStage::UnResolved(stage)) = self.stages.remove(&stage_id) { self.stages .insert(stage_id, ExecutionStage::Resolved(stage.to_resolved()?)); @@ -694,15 +915,16 @@ impl ExecutionGraph { } } - /// Convert running stage to be completed - fn complete_stage(&mut self, stage_id: usize) -> bool { + /// Convert running stage to be successful + pub fn succeed_stage(&mut self, stage_id: usize) -> bool { if let Some(ExecutionStage::Running(stage)) = self.stages.remove(&stage_id) { self.stages - .insert(stage_id, ExecutionStage::Completed(stage.to_completed())); + .insert(stage_id, ExecutionStage::Successful(stage.to_successful())); + self.clear_stage_failure(stage_id); true } else { warn!( - "Fail to find a running stage {}/{} to complete", + "Fail to find a running stage {}/{} to make it success", self.job_id(), stage_id ); @@ -711,13 +933,13 @@ impl ExecutionGraph { } /// Convert running stage to be failed - fn fail_stage(&mut self, stage_id: usize, err_msg: String) -> bool { + pub fn fail_stage(&mut self, stage_id: usize, err_msg: String) -> bool { if let Some(ExecutionStage::Running(stage)) = self.stages.remove(&stage_id) { self.stages .insert(stage_id, ExecutionStage::Failed(stage.to_failed(err_msg))); true } else { - warn!( + info!( "Fail to find a running stage {}/{} to fail", self.job_id(), stage_id @@ -726,24 +948,41 @@ impl ExecutionGraph { } } - /// Convert running stage to be unresolved - fn rollback_running_stage(&mut self, stage_id: usize) -> Result { + /// Convert running stage to be unresolved, + /// Returns a Vec of RunningTaskInfo for running tasks in this stage. + pub fn rollback_running_stage( + &mut self, + stage_id: usize, + ) -> Result> { if let Some(ExecutionStage::Running(stage)) = self.stages.remove(&stage_id) { + let running_tasks = stage + .running_tasks() + .into_iter() + .map( + |(task_id, stage_id, partition_id, executor_id)| RunningTaskInfo { + task_id, + job_id: self.job_id.clone(), + stage_id, + partition_id, + executor_id, + }, + ) + .collect(); self.stages .insert(stage_id, ExecutionStage::UnResolved(stage.to_unresolved()?)); - Ok(true) + Ok(running_tasks) } else { warn!( "Fail to find a running stage {}/{} to rollback", self.job_id(), stage_id ); - Ok(false) + Ok(vec![]) } } /// Convert resolved stage to be unresolved - fn rollback_resolved_stage(&mut self, stage_id: usize) -> Result { + pub fn rollback_resolved_stage(&mut self, stage_id: usize) -> Result { if let Some(ExecutionStage::Resolved(stage)) = self.stages.remove(&stage_id) { self.stages .insert(stage_id, ExecutionStage::UnResolved(stage.to_unresolved()?)); @@ -758,15 +997,15 @@ impl ExecutionGraph { } } - /// Convert completed stage to be running - fn rerun_completed_stage(&mut self, stage_id: usize) -> bool { - if let Some(ExecutionStage::Completed(stage)) = self.stages.remove(&stage_id) { + /// Convert successful stage to be running + pub fn rerun_successful_stage(&mut self, stage_id: usize) -> bool { + if let Some(ExecutionStage::Successful(stage)) = self.stages.remove(&stage_id) { self.stages .insert(stage_id, ExecutionStage::Running(stage.to_running())); true } else { warn!( - "Fail to find a completed stage {}/{} to rerun", + "Fail to find a successful stage {}/{} to rerun", self.job_id(), stage_id ); @@ -781,9 +1020,9 @@ impl ExecutionGraph { }; } - /// finalize job as completed - fn complete_job(&mut self) -> Result<()> { - if !self.complete() { + /// Mark the job success + pub fn succeed_job(&mut self) -> Result<()> { + if !self.is_successful() { return Err(BallistaError::Internal(format!( "Attempt to finalize an incomplete job {}", self.job_id() @@ -797,7 +1036,7 @@ impl ExecutionGraph { .collect::>>()?; self.status = JobStatus { - status: Some(job_status::Status::Completed(CompletedJob { + status: Some(job_status::Status::Successful(SuccessfulJob { partition_location, })), }; @@ -805,6 +1044,11 @@ impl ExecutionGraph { Ok(()) } + /// Clear the stage failure count for this stage if the stage is finally success + fn clear_stage_failure(&mut self, stage_id: usize) { + self.failed_stage_attempts.remove(&stage_id); + } + pub(crate) async fn decode_execution_graph< T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan, @@ -828,10 +1072,10 @@ impl ExecutionGraph { ResolvedStage::decode(stage, codec, session_ctx)?; (stage.stage_id, ExecutionStage::Resolved(stage)) } - StageType::CompletedStage(stage) => { - let stage: CompletedStage = - CompletedStage::decode(stage, codec, session_ctx)?; - (stage.stage_id, ExecutionStage::Completed(stage)) + StageType::SuccessfulStage(stage) => { + let stage: SuccessfulStage = + SuccessfulStage::decode(stage, codec, session_ctx)?; + (stage.stage_id, ExecutionStage::Successful(stage)) } StageType::FailedStage(stage) => { let stage: FailedStage = @@ -849,6 +1093,22 @@ impl ExecutionGraph { .map(|loc| loc.try_into()) .collect::>>()?; + let failed_stage_attempts = proto + .failed_attempts + .into_iter() + .map(|attempt| { + ( + attempt.stage_id as usize, + HashSet::from_iter( + attempt + .stage_attempt_num + .into_iter() + .map(|num| num as usize), + ), + ) + }) + .collect(); + Ok(ExecutionGraph { scheduler_id: proto.scheduler_id, job_id: proto.job_id, @@ -861,6 +1121,8 @@ impl ExecutionGraph { stages, output_partitions: proto.output_partitions as usize, output_locations, + tid_generator: proto.tid_gen as usize, + failed_stage_attempts, }) } @@ -889,8 +1151,8 @@ impl ExecutionGraph { ExecutionStage::Running(stage) => StageType::ResolvedStage( ResolvedStage::encode(stage.to_resolved(), codec)?, ), - ExecutionStage::Completed(stage) => StageType::CompletedStage( - CompletedStage::encode(job_id.clone(), stage, codec)?, + ExecutionStage::Successful(stage) => StageType::SuccessfulStage( + SuccessfulStage::encode(job_id.clone(), stage, codec)?, ), ExecutionStage::Failed(stage) => StageType::FailedStage( FailedStage::encode(job_id.clone(), stage, codec)?, @@ -908,6 +1170,21 @@ impl ExecutionGraph { .map(|loc| loc.try_into()) .collect::>>()?; + let failed_attempts: Vec = graph + .failed_stage_attempts + .into_iter() + .map(|(stage_id, attempts)| { + let stage_attempt_num = attempts + .into_iter() + .map(|num| num as u32) + .collect::>(); + protobuf::StageAttempts { + stage_id: stage_id as u32, + stage_attempt_num, + } + }) + .collect::>(); + Ok(protobuf::ExecutionGraph { job_id: graph.job_id, session_id: graph.session_id, @@ -916,6 +1193,8 @@ impl ExecutionGraph { output_partitions: graph.output_partitions as u64, output_locations, scheduler_id: graph.scheduler_id, + tid_gen: graph.tid_generator as u32, + failed_attempts, }) } } @@ -928,8 +1207,8 @@ impl Debug for ExecutionGraph { .map(|(_, stage)| format!("{:?}", stage)) .collect::>() .join(""); - write!(f, "ExecutionGraph[job_id={}, session_id={}, available_tasks={}, complete={}]\n{}", - self.job_id, self.session_id, self.available_tasks(), self.complete(), stages) + write!(f, "ExecutionGraph[job_id={}, session_id={}, available_tasks={}, is_successful={}]\n{}", + self.job_id, self.session_id, self.available_tasks(), self.is_successful(), stages) } } @@ -980,6 +1259,7 @@ impl ExecutionStageBuilder { let stage = if child_stages.is_empty() { ExecutionStage::Resolved(ResolvedStage::new( stage_id, + 0, stage, partitioning, output_links, @@ -1037,36 +1317,32 @@ impl ExecutionPlanVisitor for ExecutionStageBuilder { } } -#[derive(Clone)] -pub enum StageEvent { - StageResolved(usize), - StageCompleted(usize), - StageFailed(usize, String), - RollBackRunningStage(usize), - RollBackResolvedStage(usize), - ReRunCompletedStage(usize), -} - /// Represents the basic unit of work for the Ballista executor. Will execute /// one partition of one stage on one task slot. #[derive(Clone)] -pub struct Task { +pub struct TaskDefinition { pub session_id: String, pub partition: PartitionId, + pub stage_attempt_num: usize, + pub task_id: usize, + pub task_attempt: usize, pub plan: Arc, pub output_partitioning: Option, } -impl Debug for Task { +impl Debug for TaskDefinition { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { let plan = DisplayableExecutionPlan::new(self.plan.as_ref()).indent(); write!( f, - "Task[session_id: {}, job: {}, stage: {}, partition: {}]\n{}", + "TaskDefinition[session_id: {},job: {}, stage: {}.{}, partition: {} task_id {}, task attempt {}]\n{}", self.session_id, self.partition.job_id, self.partition.stage_id, + self.stage_attempt_num, self.partition.partition_id, + self.task_id, + self.task_attempt, plan ) } @@ -1112,7 +1388,7 @@ mod test { use ballista_core::serde::protobuf::{self, job_status, task_status, TaskStatus}; use ballista_core::serde::scheduler::{ExecutorMetadata, ExecutorSpecification}; - use crate::state::execution_graph::{ExecutionGraph, Task}; + use crate::state::execution_graph::{ExecutionGraph, TaskDefinition}; #[tokio::test] async fn test_drain_tasks() -> Result<()> { @@ -1122,14 +1398,17 @@ mod test { drain_tasks(&mut agg_graph)?; - assert!(agg_graph.complete(), "Failed to complete aggregation plan"); + assert!( + agg_graph.is_successful(), + "Failed to complete aggregation plan" + ); let mut coalesce_graph = test_coalesce_plan(4).await; drain_tasks(&mut coalesce_graph)?; assert!( - coalesce_graph.complete(), + coalesce_graph.is_successful(), "Failed to complete coalesce plan" ); @@ -1139,7 +1418,7 @@ mod test { println!("{:?}", join_graph); - assert!(join_graph.complete(), "Failed to complete join plan"); + assert!(join_graph.is_successful(), "Failed to complete join plan"); let mut union_all_graph = test_union_all_plan(4).await; @@ -1147,7 +1426,10 @@ mod test { println!("{:?}", union_all_graph); - assert!(union_all_graph.complete(), "Failed to complete union plan"); + assert!( + union_all_graph.is_successful(), + "Failed to complete union plan" + ); let mut union_graph = test_union_plan(4).await; @@ -1155,7 +1437,7 @@ mod test { println!("{:?}", union_graph); - assert!(union_graph.complete(), "Failed to complete union plan"); + assert!(union_graph.is_successful(), "Failed to complete union plan"); Ok(()) } @@ -1171,7 +1453,7 @@ mod test { assert!(matches!( status, protobuf::JobStatus { - status: Some(job_status::Status::Completed(_)) + status: Some(job_status::Status::Successful(_)) } )); @@ -1204,13 +1486,13 @@ mod test { // Complete the first stage if let Some(task) = join_graph.pop_next_task(&executor1.id)? { let task_status = mock_completed_task(task, &executor1.id); - join_graph.update_task_status(&executor1, vec![task_status])?; + join_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; } // Complete the second stage if let Some(task) = join_graph.pop_next_task(&executor2.id)? { let task_status = mock_completed_task(task, &executor2.id); - join_graph.update_task_status(&executor2, vec![task_status])?; + join_graph.update_task_status(&executor2, vec![task_status], 1, 1)?; } join_graph.revive(); @@ -1220,19 +1502,19 @@ mod test { // Complete 1 task if let Some(task) = join_graph.pop_next_task(&executor1.id)? { let task_status = mock_completed_task(task, &executor1.id); - join_graph.update_task_status(&executor1, vec![task_status])?; + join_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; } // Mock 1 running task let _task = join_graph.pop_next_task(&executor1.id)?; - let reset = join_graph.reset_stages(&executor1.id)?; + let reset = join_graph.reset_stages_on_lost_executor(&executor1.id)?; // Two stages were reset, 1 Running stage rollback to Unresolved and 1 Completed stage move to Running - assert_eq!(reset.len(), 2); + assert_eq!(reset.0.len(), 2); assert_eq!(join_graph.available_tasks(), 1); drain_tasks(&mut join_graph)?; - assert!(join_graph.complete(), "Failed to complete join plan"); + assert!(join_graph.is_successful(), "Failed to complete join plan"); Ok(()) } @@ -1255,26 +1537,26 @@ mod test { // Complete the first stage if let Some(task) = join_graph.pop_next_task(&executor1.id)? { let task_status = mock_completed_task(task, &executor1.id); - join_graph.update_task_status(&executor1, vec![task_status])?; + join_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; } // Complete the second stage if let Some(task) = join_graph.pop_next_task(&executor2.id)? { let task_status = mock_completed_task(task, &executor2.id); - join_graph.update_task_status(&executor2, vec![task_status])?; + join_graph.update_task_status(&executor2, vec![task_status], 1, 1)?; } // There are 0 tasks pending schedule now assert_eq!(join_graph.available_tasks(), 0); - let reset = join_graph.reset_stages(&executor1.id)?; + let reset = join_graph.reset_stages_on_lost_executor(&executor1.id)?; // Two stages were reset, 1 Resolved stage rollback to Unresolved and 1 Completed stage move to Running - assert_eq!(reset.len(), 2); + assert_eq!(reset.0.len(), 2); assert_eq!(join_graph.available_tasks(), 1); drain_tasks(&mut join_graph)?; - assert!(join_graph.complete(), "Failed to complete join plan"); + assert!(join_graph.is_successful(), "Failed to complete join plan"); Ok(()) } @@ -1297,19 +1579,19 @@ mod test { // Complete the first stage if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { let task_status = mock_completed_task(task, &executor1.id); - agg_graph.update_task_status(&executor1, vec![task_status])?; + agg_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; } // 1st task in the second stage if let Some(task) = agg_graph.pop_next_task(&executor2.id)? { let task_status = mock_completed_task(task, &executor2.id); - agg_graph.update_task_status(&executor2, vec![task_status])?; + agg_graph.update_task_status(&executor2, vec![task_status], 1, 1)?; } // 2rd task in the second stage if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { let task_status = mock_completed_task(task, &executor1.id); - agg_graph.update_task_status(&executor1, vec![task_status])?; + agg_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; } // 3rd task in the second stage, scheduled but not completed @@ -1318,23 +1600,23 @@ mod test { // There is 1 task pending schedule now assert_eq!(agg_graph.available_tasks(), 1); - let reset = agg_graph.reset_stages(&executor1.id)?; + let reset = agg_graph.reset_stages_on_lost_executor(&executor1.id)?; // 3rd task status update comes later. let task_status = mock_completed_task(task.unwrap(), &executor1.id); - agg_graph.update_task_status(&executor1, vec![task_status])?; + agg_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; // Two stages were reset, 1 Running stage rollback to Unresolved and 1 Completed stage move to Running - assert_eq!(reset.len(), 2); + assert_eq!(reset.0.len(), 2); assert_eq!(agg_graph.available_tasks(), 1); // Call the reset again - let reset = agg_graph.reset_stages(&executor1.id)?; - assert_eq!(reset.len(), 0); + let reset = agg_graph.reset_stages_on_lost_executor(&executor1.id)?; + assert_eq!(reset.0.len(), 0); assert_eq!(agg_graph.available_tasks(), 1); drain_tasks(&mut agg_graph)?; - assert!(agg_graph.complete(), "Failed to complete agg plan"); + assert!(agg_graph.is_successful(), "Failed to complete agg plan"); Ok(()) } @@ -1343,7 +1625,7 @@ mod test { let executor = mock_executor("executor-id1".to_string()); while let Some(task) = graph.pop_next_task(&executor.id)? { let task_status = mock_completed_task(task, &executor.id); - graph.update_task_status(&executor, vec![task_status])?; + graph.update_task_status(&executor, vec![task_status], 1, 1)?; } Ok(()) @@ -1503,7 +1785,7 @@ mod test { } } - fn mock_completed_task(task: Task, executor_id: &str) -> TaskStatus { + fn mock_completed_task(task: TaskDefinition, executor_id: &str) -> TaskStatus { let mut partitions: Vec = vec![]; let num_partitions = task @@ -1528,16 +1810,19 @@ mod test { // Complete the task protobuf::TaskStatus { - status: Some(task_status::Status::Completed(protobuf::CompletedTask { + task_id: task.task_id as u32, + job_id: task.partition.job_id.clone(), + stage_id: task.partition.stage_id as u32, + stage_attempt_num: task.stage_attempt_num as u32, + partition_id: task.partition.partition_id as u32, + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + metrics: vec![], + status: Some(task_status::Status::Successful(protobuf::SuccessfulTask { executor_id: executor_id.to_owned(), partitions, })), - metrics: vec![], - task_id: Some(protobuf::PartitionId { - job_id: task.partition.job_id.clone(), - stage_id: task.partition.stage_id as u32, - partition_id: task.partition.partition_id as u32, - }), } } } diff --git a/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs b/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs index b8d590cfa..f7e1b442b 100644 --- a/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs +++ b/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; use std::convert::TryInto; use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use std::time::{SystemTime, UNIX_EPOCH}; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::metrics::{MetricValue, MetricsSet}; @@ -29,8 +30,10 @@ use log::{debug, warn}; use ballista_core::error::{BallistaError, Result}; use ballista_core::serde::physical_plan::from_proto::parse_protobuf_hash_partitioning; +use ballista_core::serde::protobuf::failed_task::FailedReason; use ballista_core::serde::protobuf::{ - self, CompletedTask, FailedTask, GraphStageInput, OperatorMetricsSet, + self, task_info, FailedTask, GraphStageInput, OperatorMetricsSet, ResultLost, + SuccessfulTask, TaskStatus, }; use ballista_core::serde::protobuf::{task_status, RunningTask}; use ballista_core::serde::scheduler::to_proto::hash_partitioning_to_proto; @@ -47,13 +50,13 @@ use crate::display::DisplayableBallistaExecutionPlan; /// ↓ ↙ ↑ /// ResolvedStage → RunningStage /// ↓ -/// CompletedStage +/// SuccessfulStage #[derive(Clone)] pub(super) enum ExecutionStage { UnResolved(UnresolvedStage), Resolved(ResolvedStage), Running(RunningStage), - Completed(CompletedStage), + Successful(SuccessfulStage), Failed(FailedStage), } @@ -63,7 +66,7 @@ impl Debug for ExecutionStage { ExecutionStage::UnResolved(unresolved_stage) => unresolved_stage.fmt(f), ExecutionStage::Resolved(resolved_stage) => resolved_stage.fmt(f), ExecutionStage::Running(running_stage) => running_stage.fmt(f), - ExecutionStage::Completed(completed_stage) => completed_stage.fmt(f), + ExecutionStage::Successful(successful_stage) => successful_stage.fmt(f), ExecutionStage::Failed(failed_stage) => failed_stage.fmt(f), } } @@ -74,6 +77,8 @@ impl Debug for ExecutionStage { pub(super) struct UnresolvedStage { /// Stage ID pub(super) stage_id: usize, + /// Stage Attempt number + pub(super) stage_attempt_num: usize, /// Output partitioning for this stage. pub(super) output_partitioning: Option, /// Stage ID of the stage that will take this stages outputs as inputs. @@ -92,6 +97,8 @@ pub(super) struct UnresolvedStage { pub(super) struct ResolvedStage { /// Stage ID pub(super) stage_id: usize, + /// Stage Attempt number + pub(super) stage_attempt_num: usize, /// Total number of output partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, @@ -115,6 +122,8 @@ pub(super) struct ResolvedStage { pub(super) struct RunningStage { /// Stage ID pub(super) stage_id: usize, + /// Stage Attempt number + pub(super) stage_attempt_num: usize, /// Total number of output partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, @@ -127,17 +136,23 @@ pub(super) struct RunningStage { pub(super) inputs: HashMap, /// `ExecutionPlan` for this stage pub(super) plan: Arc, - /// Status of each already scheduled task. If status is None, the partition has not yet been scheduled - pub(super) task_statuses: Vec>, + /// TaskInfo of each already scheduled task. If info is None, the partition has not yet been scheduled. + /// The index of the Vec is the task's partition id + pub(super) task_infos: Vec>, + /// Track the number of failures for each partition's task attempts. + /// The index of the Vec is the task's partition id. + pub(super) task_failure_numbers: Vec, /// Combined metrics of the already finished tasks in the stage, If it is None, no task is finished yet. pub(super) stage_metrics: Option>, } /// If a stage finishes successfully, its task statuses and metrics will be finalized #[derive(Clone)] -pub(super) struct CompletedStage { +pub(super) struct SuccessfulStage { /// Stage ID pub(super) stage_id: usize, + /// Stage Attempt number + pub(super) stage_attempt_num: usize, /// Total number of output partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, @@ -150,8 +165,9 @@ pub(super) struct CompletedStage { pub(super) inputs: HashMap, /// `ExecutionPlan` for this stage pub(super) plan: Arc, - /// Status of each already scheduled task. - pub(super) task_statuses: Vec, + /// TaskInfo of each already successful task. + /// The index of the Vec is the task's partition id + pub(super) task_infos: Vec, /// Combined metrics of the already finished tasks in the stage. pub(super) stage_metrics: Vec, } @@ -161,6 +177,8 @@ pub(super) struct CompletedStage { pub(super) struct FailedStage { /// Stage ID pub(super) stage_id: usize, + /// Stage Attempt number + pub(super) stage_attempt_num: usize, /// Total number of output partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, @@ -171,14 +189,33 @@ pub(super) struct FailedStage { pub(super) output_links: Vec, /// `ExecutionPlan` for this stage pub(super) plan: Arc, - /// Status of each already scheduled task. If status is None, the partition has not yet been scheduled - pub(super) task_statuses: Vec>, + /// TaskInfo of each already scheduled tasks. If info is None, the partition has not yet been scheduled + /// The index of the Vec is the task's partition id + pub(super) task_infos: Vec>, /// Combined metrics of the already finished tasks in the stage, If it is None, no task is finished yet. pub(super) stage_metrics: Option>, /// Error message pub(super) error_message: String, } +#[derive(Clone)] +pub(super) struct TaskInfo { + /// Task ID + pub(super) task_id: usize, + /// Task scheduled time + pub(super) scheduled_time: u128, + /// Task launch time + pub(super) launch_time: u128, + /// Start execution time + pub(super) start_exec_time: u128, + /// Finish execution time + pub(super) end_exec_time: u128, + /// Task finish time + pub(super) finish_time: u128, + /// Task Status + pub(super) task_status: task_status::Status, +} + impl UnresolvedStage { pub(super) fn new( stage_id: usize, @@ -194,6 +231,7 @@ impl UnresolvedStage { Self { stage_id, + stage_attempt_num: 0, output_partitioning, output_links, inputs, @@ -203,6 +241,7 @@ impl UnresolvedStage { pub(super) fn new_with_inputs( stage_id: usize, + stage_attempt_num: usize, plan: Arc, output_partitioning: Option, output_links: Vec, @@ -210,6 +249,7 @@ impl UnresolvedStage { ) -> Self { Self { stage_id, + stage_attempt_num, output_partitioning, output_links, inputs, @@ -260,6 +300,7 @@ impl UnresolvedStage { )?; Ok(ResolvedStage::new( self.stage_id, + self.stage_attempt_num, plan, self.output_partitioning.clone(), self.output_links.clone(), @@ -289,6 +330,7 @@ impl UnresolvedStage { Ok(UnresolvedStage { stage_id: stage.stage_id as usize, + stage_attempt_num: stage.stage_attempt_num as usize, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as usize).collect(), plan, @@ -310,7 +352,8 @@ impl UnresolvedStage { hash_partitioning_to_proto(stage.output_partitioning.as_ref())?; Ok(protobuf::UnResolvedStage { - stage_id: stage.stage_id as u64, + stage_id: stage.stage_id as u32, + stage_attempt_num: stage.stage_attempt_num as u32, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as u32).collect(), inputs, @@ -325,8 +368,9 @@ impl Debug for UnresolvedStage { write!( f, - "=========UnResolvedStage[id={}, children={}]=========\nInputs{:?}\n{}", + "=========UnResolvedStage[stage_id={}.{}, children={}]=========\nInputs{:?}\n{}", self.stage_id, + self.stage_attempt_num, self.inputs.len(), self.inputs, plan @@ -337,6 +381,7 @@ impl Debug for UnresolvedStage { impl ResolvedStage { pub(super) fn new( stage_id: usize, + stage_attempt_num: usize, plan: Arc, output_partitioning: Option, output_links: Vec, @@ -346,6 +391,7 @@ impl ResolvedStage { Self { stage_id, + stage_attempt_num, partitions, output_partitioning, output_links, @@ -358,6 +404,7 @@ impl ResolvedStage { pub(super) fn to_running(&self) -> RunningStage { RunningStage::new( self.stage_id, + self.stage_attempt_num, self.plan.clone(), self.partitions, self.output_partitioning.clone(), @@ -372,6 +419,7 @@ impl ResolvedStage { let unresolved = UnresolvedStage::new_with_inputs( self.stage_id, + self.stage_attempt_num, new_plan, self.output_partitioning.clone(), self.output_links.clone(), @@ -402,6 +450,7 @@ impl ResolvedStage { Ok(ResolvedStage { stage_id: stage.stage_id as usize, + stage_attempt_num: stage.stage_attempt_num as usize, partitions: stage.partitions as usize, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as usize).collect(), @@ -424,7 +473,8 @@ impl ResolvedStage { let inputs = encode_inputs(stage.inputs)?; Ok(protobuf::ResolvedStage { - stage_id: stage.stage_id as u64, + stage_id: stage.stage_id as u32, + stage_attempt_num: stage.stage_attempt_num as u32, partitions: stage.partitions as u32, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as u32).collect(), @@ -440,8 +490,8 @@ impl Debug for ResolvedStage { write!( f, - "=========ResolvedStage[id={}, partitions={}]=========\n{}", - self.stage_id, self.partitions, plan + "=========ResolvedStage[stage_id={}.{}, partitions={}]=========\n{}", + self.stage_id, self.stage_attempt_num, self.partitions, plan ) } } @@ -449,6 +499,7 @@ impl Debug for ResolvedStage { impl RunningStage { pub(super) fn new( stage_id: usize, + stage_attempt_num: usize, plan: Arc, partitions: usize, output_partitioning: Option, @@ -457,26 +508,28 @@ impl RunningStage { ) -> Self { Self { stage_id, + stage_attempt_num, partitions, output_partitioning, output_links, inputs, plan, - task_statuses: vec![None; partitions], + task_infos: vec![None; partitions], + task_failure_numbers: vec![0; partitions], stage_metrics: None, } } - pub(super) fn to_completed(&self) -> CompletedStage { - let task_statuses = self - .task_statuses + pub(super) fn to_successful(&self) -> SuccessfulStage { + let task_infos = self + .task_infos .iter() .enumerate() - .map(|(task_id, status)| { - status.clone().unwrap_or_else(|| { + .map(|(partition_id, info)| { + info.clone().unwrap_or_else(|| { panic!( - "The status of task {}/{} should not be none", - self.stage_id, task_id + "TaskInfo for task {}.{}/{} should not be none", + self.stage_id, self.stage_attempt_num, partition_id ) }) }) @@ -485,14 +538,15 @@ impl RunningStage { warn!("The metrics for stage {} should not be none", self.stage_id); vec![] }); - CompletedStage { + SuccessfulStage { stage_id: self.stage_id, + stage_attempt_num: self.stage_attempt_num, partitions: self.partitions, output_partitioning: self.output_partitioning.clone(), output_links: self.output_links.clone(), inputs: self.inputs.clone(), plan: self.plan.clone(), - task_statuses, + task_infos, stage_metrics, } } @@ -500,19 +554,22 @@ impl RunningStage { pub(super) fn to_failed(&self, error_message: String) -> FailedStage { FailedStage { stage_id: self.stage_id, + stage_attempt_num: self.stage_attempt_num, partitions: self.partitions, output_partitioning: self.output_partitioning.clone(), output_links: self.output_links.clone(), plan: self.plan.clone(), - task_statuses: self.task_statuses.clone(), + task_infos: self.task_infos.clone(), stage_metrics: self.stage_metrics.clone(), error_message, } } + /// /// Change to the resolved state and bump the stage attempt number pub(super) fn to_resolved(&self) -> ResolvedStage { ResolvedStage::new( self.stage_id, + self.stage_attempt_num + 1, self.plan.clone(), self.output_partitioning.clone(), self.output_links.clone(), @@ -520,12 +577,13 @@ impl RunningStage { ) } - /// Change to the unresolved state + /// Change to the unresolved state and bump the stage attempt number pub(super) fn to_unresolved(&self) -> Result { let new_plan = crate::planner::rollback_resolved_shuffles(self.plan.clone())?; let unresolved = UnresolvedStage::new_with_inputs( self.stage_id, + self.stage_attempt_num + 1, new_plan, self.output_partitioning.clone(), self.output_links.clone(), @@ -534,34 +592,49 @@ impl RunningStage { Ok(unresolved) } - /// Returns `true` if all tasks for this stage are complete - pub(super) fn is_completed(&self) -> bool { - self.task_statuses - .iter() - .all(|status| matches!(status, Some(task_status::Status::Completed(_)))) + /// Returns `true` if all tasks for this stage are successful + pub(super) fn is_successful(&self) -> bool { + self.task_infos.iter().all(|info| { + matches!( + info, + Some(TaskInfo { + task_status: task_status::Status::Successful(_), + .. + }) + ) + }) } - /// Returns the number of completed tasks - pub(super) fn completed_tasks(&self) -> usize { - self.task_statuses + /// Returns the number of successful tasks + pub(super) fn successful_tasks(&self) -> usize { + self.task_infos .iter() - .filter(|status| matches!(status, Some(task_status::Status::Completed(_)))) + .filter(|info| { + matches!( + info, + Some(TaskInfo { + task_status: task_status::Status::Successful(_), + .. + }) + ) + }) .count() } /// Returns the number of scheduled tasks pub(super) fn scheduled_tasks(&self) -> usize { - self.task_statuses.iter().filter(|s| s.is_some()).count() + self.task_infos.iter().filter(|s| s.is_some()).count() } /// Returns a vector of currently running tasks in this stage - pub(super) fn running_tasks(&self) -> Vec<(usize, usize, String)> { - self.task_statuses + pub(super) fn running_tasks(&self) -> Vec<(usize, usize, usize, String)> { + self.task_infos .iter() .enumerate() - .filter_map(|(partition, status)| match status { - Some(task_status::Status::Running(RunningTask { executor_id })) => { - Some((self.stage_id, partition, executor_id.clone())) + .filter_map(|(partition, info)| match info { + Some(TaskInfo {task_id, + task_status: task_status::Status::Running(RunningTask { executor_id }), ..}) => { + Some((*task_id, self.stage_id, partition, executor_id.clone())) } _ => None, }) @@ -570,19 +643,50 @@ impl RunningStage { /// Returns the number of tasks in this stage which are available for scheduling. /// If the stage is not yet resolved, then this will return `0`, otherwise it will - /// return the number of tasks where the task status is not yet set. + /// return the number of tasks where the task info is not yet set. pub(super) fn available_tasks(&self) -> usize { - self.task_statuses.iter().filter(|s| s.is_none()).count() + self.task_infos.iter().filter(|s| s.is_none()).count() } - /// Update the status for task partition - pub(super) fn update_task_status( + /// Update the TaskInfo for task partition + pub(super) fn update_task_info( &mut self, partition_id: usize, - status: task_status::Status, - ) { - debug!("Updating task status for partition {}", partition_id); - self.task_statuses[partition_id] = Some(status); + status: TaskStatus, + ) -> bool { + debug!("Updating TaskInfo for partition {}", partition_id); + let task_info = self.task_infos[partition_id].as_ref().unwrap(); + let task_id = task_info.task_id; + if (status.task_id as usize) < task_id { + warn!("Ignore TaskStatus update with TID {} because there is more recent task attempt with TID {} running for partition {}", + status.task_id, task_id, partition_id); + return false; + } + let scheduled_time = task_info.scheduled_time; + let task_status = status.status.unwrap(); + let updated_task_info = TaskInfo { + task_id, + scheduled_time, + launch_time: status.launch_time as u128, + start_exec_time: status.start_exec_time as u128, + end_exec_time: status.end_exec_time as u128, + finish_time: SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis(), + task_status: task_status.clone(), + }; + self.task_infos[partition_id] = Some(updated_task_info); + + if let task_status::Status::Failed(failed_task) = task_status { + // if the failed task is retryable, increase the task failure count for this partition + if failed_task.retryable { + self.task_failure_numbers[partition_id] += 1; + } + } else { + self.task_failure_numbers[partition_id] = 0; + } + true } /// update and combine the task metrics to the stage metrics @@ -639,22 +743,37 @@ impl RunningStage { first.aggregate_by_partition() } + pub(super) fn task_failure_number(&self, partition_id: usize) -> usize { + self.task_failure_numbers[partition_id] + } + + /// Reset the task info for the given task partition. This should be called when a task failed and need to be + /// re-scheduled. + pub fn reset_task_info(&mut self, partition_id: usize) { + self.task_infos[partition_id] = None; + } + /// Reset the running and completed tasks on a given executor /// Returns the number of running tasks that were reset pub fn reset_tasks(&mut self, executor: &str) -> usize { let mut reset = 0; - for task in self.task_statuses.iter_mut() { + for task in self.task_infos.iter_mut() { match task { - Some(task_status::Status::Running(RunningTask { executor_id })) - if *executor == *executor_id => - { + Some(TaskInfo { + task_status: task_status::Status::Running(RunningTask { executor_id }), + .. + }) if *executor == *executor_id => { *task = None; reset += 1; } - Some(task_status::Status::Completed(CompletedTask { - executor_id, - partitions: _, - })) if *executor == *executor_id => { + Some(TaskInfo { + task_status: + task_status::Status::Successful(SuccessfulTask { + executor_id, + partitions: _, + }), + .. + }) if *executor == *executor_id => { *task = None; reset += 1; } @@ -663,6 +782,22 @@ impl RunningStage { } reset } + + /// Remove input partition from an input stage. + pub(super) fn remove_input_partition( + &mut self, + input_stage_id: usize, + input_partition_id: usize, + ) -> Result<()> { + if let Some(stage_inputs) = self.inputs.get_mut(&input_stage_id) { + stage_inputs.remove_partition(input_partition_id); + stage_inputs.complete = false; + } else { + return Err(BallistaError::Internal(format!("Error remove input partition to stage {}, {} is not a valid child stage ID", self.stage_id, input_stage_id))); + } + + Ok(()) + } } impl Debug for RunningStage { @@ -671,10 +806,11 @@ impl Debug for RunningStage { write!( f, - "=========RunningStage[id={}, partitions={}, completed_tasks={}, scheduled_tasks={}, available_tasks={}]=========\n{}", + "=========RunningStage[stage_id={}.{}, partitions={}, successful_tasks={}, scheduled_tasks={}, available_tasks={}]=========\n{}", self.stage_id, + self.stage_attempt_num, self.partitions, - self.completed_tasks(), + self.successful_tasks(), self.scheduled_tasks(), self.available_tasks(), plan @@ -682,41 +818,64 @@ impl Debug for RunningStage { } } -impl CompletedStage { +impl SuccessfulStage { + /// Change to the running state and bump the stage attempt number pub fn to_running(&self) -> RunningStage { - let mut task_status: Vec> = Vec::new(); - for task in self.task_statuses.iter() { + let mut task_infos: Vec> = Vec::new(); + for task in self.task_infos.iter() { match task { - task_status::Status::Completed(_) => task_status.push(Some(task.clone())), - _ => task_status.push(None), + TaskInfo { + task_status: task_status::Status::Successful(_), + .. + } => task_infos.push(Some(task.clone())), + _ => task_infos.push(None), } } RunningStage { stage_id: self.stage_id, + stage_attempt_num: self.stage_attempt_num + 1, partitions: self.partitions, output_partitioning: self.output_partitioning.clone(), output_links: self.output_links.clone(), inputs: self.inputs.clone(), plan: self.plan.clone(), - task_statuses: task_status, + task_infos, + // It is Ok to forget the previous task failure attempts + task_failure_numbers: vec![0; self.partitions], stage_metrics: Some(self.stage_metrics.clone()), } } - /// Reset the completed tasks on a given executor + /// Reset the successful tasks on a given executor /// Returns the number of running tasks that were reset pub fn reset_tasks(&mut self, executor: &str) -> usize { let mut reset = 0; let failure_reason = format!("Task failure due to Executor {} lost", executor); - for task in self.task_statuses.iter_mut() { + for task in self.task_infos.iter_mut() { match task { - task_status::Status::Completed(CompletedTask { - executor_id, - partitions: _, - }) if *executor == *executor_id => { - *task = task_status::Status::Failed(FailedTask { - error: failure_reason.clone(), - }); + TaskInfo { + task_id, + scheduled_time, + task_status: + task_status::Status::Successful(SuccessfulTask { + executor_id, .. + }), + .. + } if *executor == *executor_id => { + *task = TaskInfo { + task_id: *task_id, + scheduled_time: *scheduled_time, + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + finish_time: 0, + task_status: task_status::Status::Failed(FailedTask { + error: failure_reason.clone(), + retryable: true, + count_to_failures: false, + failed_reason: Some(FailedReason::ResultLost(ResultLost {})), + }), + }; reset += 1; } _ => {} @@ -726,10 +885,10 @@ impl CompletedStage { } pub(super) fn decode( - stage: protobuf::CompletedStage, + stage: protobuf::SuccessfulStage, codec: &BallistaCodec, session_ctx: &SessionContext, - ) -> Result { + ) -> Result { let plan_proto = U::try_decode(&stage.plan)?; let plan = plan_proto.try_into_physical_plan( session_ctx, @@ -744,41 +903,36 @@ impl CompletedStage { )?; let inputs = decode_inputs(stage.inputs)?; - - let task_statuses = stage - .task_statuses - .into_iter() - .enumerate() - .map(|(task_id, status)| { - status.status.unwrap_or_else(|| { - panic!("Status for task {} should not be none", task_id) - }) - }) - .collect(); - + assert_eq!( + stage.task_infos.len(), + stage.partitions as usize, + "protobuf::SuccessfulStage task_infos len not equal to partitions." + ); + let task_infos = stage.task_infos.into_iter().map(decode_taskinfo).collect(); let stage_metrics = stage .stage_metrics .into_iter() .map(|m| m.try_into()) .collect::>>()?; - Ok(CompletedStage { + Ok(SuccessfulStage { stage_id: stage.stage_id as usize, + stage_attempt_num: stage.stage_attempt_num as usize, partitions: stage.partitions as usize, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as usize).collect(), inputs, plan, - task_statuses, + task_infos, stage_metrics, }) } pub(super) fn encode( - job_id: String, - stage: CompletedStage, + _job_id: String, + stage: SuccessfulStage, codec: &BallistaCodec, - ) -> Result { + ) -> Result { let stage_id = stage.stage_id; let mut plan: Vec = vec![]; @@ -789,23 +943,11 @@ impl CompletedStage { hash_partitioning_to_proto(stage.output_partitioning.as_ref())?; let inputs = encode_inputs(stage.inputs)?; - - let task_statuses: Vec = stage - .task_statuses + let task_infos = stage + .task_infos .into_iter() .enumerate() - .map(|(partition, status)| { - protobuf::TaskStatus { - task_id: Some(protobuf::PartitionId { - job_id: job_id.clone(), - stage_id: stage_id as u32, - partition_id: partition as u32, - }), - // task metrics should not persist. - metrics: vec![], - status: Some(status), - } - }) + .map(|(partition, task_info)| encode_taskinfo(task_info, partition)) .collect(); let stage_metrics = stage @@ -814,20 +956,21 @@ impl CompletedStage { .map(|m| m.try_into()) .collect::>>()?; - Ok(protobuf::CompletedStage { - stage_id: stage_id as u64, + Ok(protobuf::SuccessfulStage { + stage_id: stage_id as u32, + stage_attempt_num: stage.stage_attempt_num as u32, partitions: stage.partitions as u32, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as u32).collect(), inputs, plan, - task_statuses, + task_infos, stage_metrics, }) } } -impl Debug for CompletedStage { +impl Debug for SuccessfulStage { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { let plan = DisplayableBallistaExecutionPlan::new( self.plan.as_ref(), @@ -837,31 +980,38 @@ impl Debug for CompletedStage { write!( f, - "=========CompletedStage[id={}, partitions={}]=========\n{}", - self.stage_id, self.partitions, plan + "=========SuccessfulStage[stage_id={}.{}, partitions={}]=========\n{}", + self.stage_id, self.stage_attempt_num, self.partitions, plan ) } } impl FailedStage { - /// Returns the number of completed tasks - pub(super) fn completed_tasks(&self) -> usize { - self.task_statuses + /// Returns the number of successful tasks + pub(super) fn successful_tasks(&self) -> usize { + self.task_infos .iter() - .filter(|status| matches!(status, Some(task_status::Status::Completed(_)))) + .filter(|info| { + matches!( + info, + Some(TaskInfo { + task_status: task_status::Status::Successful(_), + .. + }) + ) + }) .count() } - /// Returns the number of scheduled tasks pub(super) fn scheduled_tasks(&self) -> usize { - self.task_statuses.iter().filter(|s| s.is_some()).count() + self.task_infos.iter().filter(|s| s.is_some()).count() } /// Returns the number of tasks in this stage which are available for scheduling. /// If the stage is not yet resolved, then this will return `0`, otherwise it will /// return the number of tasks where the task status is not yet set. pub(super) fn available_tasks(&self) -> usize { - self.task_statuses.iter().filter(|s| s.is_none()).count() + self.task_infos.iter().filter(|s| s.is_none()).count() } pub(super) fn decode( @@ -882,12 +1032,9 @@ impl FailedStage { plan.schema().as_ref(), )?; - let mut task_statuses: Vec> = - vec![None; stage.partitions as usize]; - for status in stage.task_statuses { - if let Some(task_id) = status.task_id.as_ref() { - task_statuses[task_id.partition_id as usize] = status.status - } + let mut task_infos: Vec> = vec![None; stage.partitions as usize]; + for info in stage.task_infos { + task_infos[info.partition_id as usize] = Some(decode_taskinfo(info.clone())); } let stage_metrics = if stage.stage_metrics.is_empty() { @@ -903,18 +1050,19 @@ impl FailedStage { Ok(FailedStage { stage_id: stage.stage_id as usize, + stage_attempt_num: stage.stage_attempt_num as usize, partitions: stage.partitions as usize, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as usize).collect(), plan, - task_statuses, + task_infos, stage_metrics, error_message: stage.error_message, }) } pub(super) fn encode( - job_id: String, + _job_id: String, stage: FailedStage, codec: &BallistaCodec, ) -> Result { @@ -927,21 +1075,12 @@ impl FailedStage { let output_partitioning = hash_partitioning_to_proto(stage.output_partitioning.as_ref())?; - let task_statuses: Vec = stage - .task_statuses + let task_infos: Vec = stage + .task_infos .into_iter() .enumerate() - .filter_map(|(partition, status)| { - status.map(|status| protobuf::TaskStatus { - task_id: Some(protobuf::PartitionId { - job_id: job_id.clone(), - stage_id: stage_id as u32, - partition_id: partition as u32, - }), - // task metrics should not persist. - metrics: vec![], - status: Some(status), - }) + .filter_map(|(partition, task_info)| { + task_info.map(|info| encode_taskinfo(info, partition)) }) .collect(); @@ -953,12 +1092,13 @@ impl FailedStage { .collect::>>()?; Ok(protobuf::FailedStage { - stage_id: stage_id as u64, + stage_id: stage_id as u32, + stage_attempt_num: stage.stage_attempt_num as u32, partitions: stage.partitions as u32, output_partitioning, output_links: stage.output_links.into_iter().map(|l| l as u32).collect(), plan, - task_statuses, + task_infos, stage_metrics, error_message: stage.error_message, }) @@ -971,10 +1111,11 @@ impl Debug for FailedStage { write!( f, - "=========FailedStage[id={}, partitions={}, completed_tasks={}, scheduled_tasks={}, available_tasks={}, error_message={}]=========\n{}", + "=========FailedStage[stage_id={}.{}, partitions={}, successful_tasks={}, scheduled_tasks={}, available_tasks={}, error_message={}]=========\n{}", self.stage_id, + self.stage_attempt_num, self.partitions, - self.completed_tasks(), + self.successful_tasks(), self.scheduled_tasks(), self.available_tasks(), self.error_message, @@ -1017,6 +1158,10 @@ impl StageOutput { } } + pub(super) fn remove_partition(&mut self, partition_id: usize) { + self.partition_locations.remove(&partition_id); + } + pub(super) fn is_complete(&self) -> bool { self.complete } @@ -1079,3 +1224,48 @@ fn encode_inputs( } Ok(inputs) } + +fn decode_taskinfo(task_info: protobuf::TaskInfo) -> TaskInfo { + let task_info_status = match task_info.status { + Some(task_info::Status::Running(running)) => { + task_status::Status::Running(running) + } + Some(task_info::Status::Failed(failed)) => task_status::Status::Failed(failed), + Some(task_info::Status::Successful(success)) => { + task_status::Status::Successful(success) + } + _ => panic!( + "protobuf::TaskInfo status for task {} should not be none", + task_info.task_id + ), + }; + TaskInfo { + task_id: task_info.task_id as usize, + scheduled_time: task_info.scheduled_time as u128, + launch_time: task_info.launch_time as u128, + start_exec_time: task_info.start_exec_time as u128, + end_exec_time: task_info.end_exec_time as u128, + finish_time: task_info.finish_time as u128, + task_status: task_info_status, + } +} + +fn encode_taskinfo(task_info: TaskInfo, partition_id: usize) -> protobuf::TaskInfo { + let task_info_status = match task_info.task_status { + task_status::Status::Running(running) => task_info::Status::Running(running), + task_status::Status::Failed(failed) => task_info::Status::Failed(failed), + task_status::Status::Successful(success) => { + task_info::Status::Successful(success) + } + }; + protobuf::TaskInfo { + task_id: task_info.task_id as u32, + partition_id: partition_id as u32, + scheduled_time: task_info.scheduled_time as u64, + launch_time: task_info.launch_time as u64, + start_exec_time: task_info.start_exec_time as u64, + end_exec_time: task_info.end_exec_time as u64, + finish_time: task_info.finish_time as u64, + status: Some(task_info_status), + } +} diff --git a/ballista/rust/scheduler/src/state/executor_manager.rs b/ballista/rust/scheduler/src/state/executor_manager.rs index 9fc8df902..1d135ef84 100644 --- a/ballista/rust/scheduler/src/state/executor_manager.rs +++ b/ballista/rust/scheduler/src/state/executor_manager.rs @@ -23,14 +23,15 @@ use crate::state::{decode_into, decode_protobuf, encode_protobuf, with_lock}; use ballista_core::error::{BallistaError, Result}; use ballista_core::serde::protobuf; +use crate::state::execution_graph::RunningTaskInfo; use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient; use ballista_core::serde::protobuf::{ - executor_status, ExecutorHeartbeat, ExecutorStatus, + executor_status, CancelTasksParams, ExecutorHeartbeat, ExecutorStatus, }; use ballista_core::serde::scheduler::{ExecutorData, ExecutorMetadata}; use ballista_core::utils::create_grpc_client_connection; use futures::StreamExt; -use log::{debug, info}; +use log::{debug, error, info}; use parking_lot::RwLock; use std::collections::{HashMap, HashSet}; use std::sync::Arc; @@ -216,6 +217,47 @@ impl ExecutorManager { .await } + /// Send rpc to Executors to cancel the running tasks + pub async fn cancel_running_tasks(&self, tasks: Vec) -> Result<()> { + let mut tasks_to_cancel: HashMap<&str, Vec> = + Default::default(); + + for task_info in &tasks { + if let Some(infos) = tasks_to_cancel.get_mut(task_info.executor_id.as_str()) { + infos.push(protobuf::RunningTaskInfo { + task_id: task_info.task_id as u32, + job_id: task_info.job_id.clone(), + stage_id: task_info.stage_id as u32, + partition_id: task_info.partition_id as u32, + }) + } else { + tasks_to_cancel.insert( + task_info.executor_id.as_str(), + vec![protobuf::RunningTaskInfo { + task_id: task_info.task_id as u32, + job_id: task_info.job_id.clone(), + stage_id: task_info.stage_id as u32, + partition_id: task_info.partition_id as u32, + }], + ); + } + } + + for (executor_id, infos) in tasks_to_cancel { + if let Ok(mut client) = self.get_client(executor_id).await { + client + .cancel_tasks(CancelTasksParams { task_infos: infos }) + .await?; + } else { + error!( + "Failed to get client for executor ID {} to cancel tasks", + executor_id + ) + } + } + Ok(()) + } + pub async fn get_client( &self, executor_id: &str, diff --git a/ballista/rust/scheduler/src/state/mod.rs b/ballista/rust/scheduler/src/state/mod.rs index f45b8e253..638512045 100644 --- a/ballista/rust/scheduler/src/state/mod.rs +++ b/ballista/rust/scheduler/src/state/mod.rs @@ -287,8 +287,7 @@ mod test { use ballista_core::config::{BallistaConfig, BALLISTA_DEFAULT_SHUFFLE_PARTITIONS}; use ballista_core::error::Result; use ballista_core::serde::protobuf::{ - task_status, CompletedTask, PartitionId, PhysicalPlanNode, ShuffleWritePartition, - TaskStatus, + task_status, PhysicalPlanNode, ShuffleWritePartition, SuccessfulTask, TaskStatus, }; use ballista_core::serde::scheduler::{ ExecutorData, ExecutorMetadata, ExecutorSpecification, @@ -421,36 +420,49 @@ mod test { let (executor_metadata, executor_data) = executors[0].clone(); // Complete the first stage. So we should now have 4 pending tasks for this job stage 2 - let mut partitions: Vec = vec![]; - - for partition_id in 0..4 { - partitions.push(ShuffleWritePartition { - partition_id: partition_id as u64, - path: "some/path".to_string(), - num_batches: 1, - num_rows: 1, - num_bytes: 1, - }) - } - - state - .task_manager - .update_task_statuses( - &executor_metadata, - vec![TaskStatus { - task_id: Some(PartitionId { + { + let plan_graph = state + .task_manager + .get_active_execution_graph("job-1") + .await + .unwrap(); + let task_def = plan_graph + .write() + .await + .pop_next_task(&executor_data.executor_id)? + .unwrap(); + let mut partitions: Vec = vec![]; + for partition_id in 0..4 { + partitions.push(ShuffleWritePartition { + partition_id: partition_id as u64, + path: "some/path".to_string(), + num_batches: 1, + num_rows: 1, + num_bytes: 1, + }) + } + state + .task_manager + .update_task_statuses( + &executor_metadata, + vec![TaskStatus { + task_id: task_def.task_id as u32, job_id: "job-1".to_string(), - stage_id: 1, - partition_id: 0, - }), - metrics: vec![], - status: Some(task_status::Status::Completed(CompletedTask { - executor_id: "executor-1".to_string(), - partitions, - })), - }], - ) - .await?; + stage_id: task_def.partition.stage_id as u32, + stage_attempt_num: task_def.stage_attempt_num as u32, + partition_id: task_def.partition.partition_id as u32, + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + metrics: vec![], + status: Some(task_status::Status::Successful(SuccessfulTask { + executor_id: executor_data.executor_id.clone(), + partitions, + })), + }], + ) + .await?; + } state .executor_manager diff --git a/ballista/rust/scheduler/src/state/task_manager.rs b/ballista/rust/scheduler/src/state/task_manager.rs index e34c04c80..c3dbe4807 100644 --- a/ballista/rust/scheduler/src/state/task_manager.rs +++ b/ballista/rust/scheduler/src/state/task_manager.rs @@ -18,18 +18,19 @@ use crate::scheduler_server::event::QueryStageSchedulerEvent; use crate::scheduler_server::SessionBuilder; use crate::state::backend::{Keyspace, Lock, StateBackendClient}; -use crate::state::execution_graph::{ExecutionGraph, Task}; +use crate::state::execution_graph::{ + ExecutionGraph, RunningTaskInfo, TaskDefinition as TaskDef, +}; use crate::state::executor_manager::{ExecutorManager, ExecutorReservation}; use crate::state::{decode_protobuf, encode_protobuf, with_lock}; use ballista_core::config::BallistaConfig; #[cfg(not(test))] use ballista_core::error::BallistaError; use ballista_core::error::Result; -use ballista_core::serde::protobuf::executor_grpc_client::ExecutorGrpcClient; use crate::state::session_manager::create_datafusion_context; use ballista_core::serde::protobuf::{ - self, job_status, CancelTasksParams, FailedJob, JobStatus, TaskDefinition, TaskStatus, + self, job_status, FailedJob, JobStatus, TaskDefinition, TaskStatus, }; use ballista_core::serde::scheduler::to_proto::hash_partitioning_to_proto; use ballista_core::serde::scheduler::ExecutorMetadata; @@ -40,20 +41,22 @@ use datafusion_proto::logical_plan::AsLogicalPlan; use log::{debug, error, info, warn}; use rand::distributions::Alphanumeric; use rand::{thread_rng, Rng}; -use std::collections::HashMap; -use std::default::Default; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; +use std::time::{SystemTime, UNIX_EPOCH}; use tokio::sync::RwLock; -use tonic::transport::Channel; -type ExecutorClients = Arc>>>; type ExecutionGraphCache = Arc>>>>; +// TODO move to configuration file +/// Default max failure attempts for task level retry +pub const TASK_MAX_FAILURES: usize = 4; +/// Default max failure attempts for stage level retry +pub const STAGE_MAX_FAILURES: usize = 4; + #[derive(Clone)] pub struct TaskManager { state: Arc, - #[allow(dead_code)] - clients: ExecutorClients, session_builder: SessionBuilder, codec: BallistaCodec, scheduler_id: String, @@ -61,6 +64,15 @@ pub struct TaskManager active_job_cache: ExecutionGraphCache, } +#[derive(Clone)] +pub struct UpdatedStages { + pub resolved_stages: Vec, + pub successful_stages: Vec, + pub failed_stages: HashMap, + pub rollback_running_stages: HashSet, + pub resubmit_successful_stages: HashSet, +} + impl TaskManager { pub fn new( state: Arc, @@ -70,7 +82,6 @@ impl TaskManager ) -> Self { Self { state, - clients: Default::default(), session_builder, codec, scheduler_id, @@ -137,13 +148,9 @@ impl TaskManager let mut job_updates: HashMap> = HashMap::new(); for status in task_status { debug!("Task Update\n{:?}", status); - if let Some(job_id) = status.task_id.as_ref().map(|id| &id.job_id) { - let job_task_statuses = - job_updates.entry(job_id.clone()).or_insert_with(Vec::new); - job_task_statuses.push(status); - } else { - warn!("Received task with no job ID"); - } + let job_id = status.job_id.clone(); + let job_task_statuses = job_updates.entry(job_id).or_insert_with(Vec::new); + job_task_statuses.push(status); } let mut events: Vec = vec![]; @@ -152,16 +159,21 @@ impl TaskManager debug!("Updating {} tasks in job {}", num_tasks, job_id); let graph = self.get_active_execution_graph(&job_id).await; - let job_event = if let Some(graph) = graph { + let job_events = if let Some(graph) = graph { let mut graph = graph.write().await; - graph.update_task_status(executor, statuses)? + graph.update_task_status( + executor, + statuses, + TASK_MAX_FAILURES, + STAGE_MAX_FAILURES, + )? } else { // TODO Deal with curator changed case error!("Fail to find job {} in the active cache and it may not be curated by this scheduler", job_id); - None + vec![] }; - if let Some(event) = job_event { + for event in job_events { events.push(event); } } @@ -184,7 +196,7 @@ impl TaskManager pub async fn fill_reservations( &self, reservations: &[ExecutorReservation], - ) -> Result<(Vec<(String, Task)>, Vec, usize)> { + ) -> Result<(Vec<(String, TaskDef)>, Vec, usize)> { // Reinitialize the free reservations. let free_reservations: Vec = reservations .iter() @@ -193,7 +205,7 @@ impl TaskManager }) .collect(); - let mut assignments: Vec<(String, Task)> = vec![]; + let mut assignments: Vec<(String, TaskDef)> = vec![]; let mut pending_tasks = 0usize; let mut assign_tasks = 0usize; let job_cache = self.active_job_cache.read().await; @@ -220,16 +232,16 @@ impl TaskManager Ok((assignments, unassigned, pending_tasks)) } - /// Mark a job as completed. This will create a key under the CompletedJobs keyspace + /// Mark a job to success. This will create a key under the CompletedJobs keyspace /// and remove the job from ActiveJobs - pub async fn complete_job(&self, job_id: &str) -> Result<()> { - debug!("Moving job {} from Active to Completed", job_id); + pub async fn succeed_job(&self, job_id: &str) -> Result<()> { + debug!("Moving job {} from Active to Success", job_id); let lock = self.state.lock(Keyspace::ActiveJobs, "").await?; with_lock(lock, self.state.delete(Keyspace::ActiveJobs, job_id)).await?; if let Some(graph) = self.get_active_execution_graph(job_id).await { let graph = graph.read().await.clone(); - if graph.complete() { + if graph.is_successful() { let value = self.encode_execution_graph(graph)?; self.state .put(Keyspace::CompletedJobs, job_id.to_owned(), value) @@ -244,93 +256,70 @@ impl TaskManager Ok(()) } - pub(crate) async fn cancel_job( + /// Cancel the job and return a Vec of running tasks need to cancel + pub(crate) async fn cancel_job(&self, job_id: &str) -> Result> { + self.abort_job(job_id, "Cancelled".to_owned()).await + } + + /// Abort the job and return a Vec of running tasks need to cancel + pub(crate) async fn abort_job( &self, job_id: &str, - executor_manager: &ExecutorManager, - ) -> Result<()> { + failure_reason: String, + ) -> Result> { let lock = self.state.lock(Keyspace::ActiveJobs, "").await?; - - let running_tasks = self - .get_execution_graph(job_id) - .await - .map(|graph| graph.running_tasks()) - .unwrap_or_else(|_| vec![]); - - info!( - "Cancelling {} running tasks for job {}", - running_tasks.len(), - job_id - ); - - self.fail_job_inner(lock, job_id, "Cancelled".to_owned()) - .await?; - - let mut tasks: HashMap<&str, Vec> = Default::default(); - - for (partition, executor_id) in &running_tasks { - if let Some(parts) = tasks.get_mut(executor_id.as_str()) { - parts.push(protobuf::PartitionId { - job_id: job_id.to_owned(), - stage_id: partition.stage_id as u32, - partition_id: partition.partition_id as u32, - }) - } else { - tasks.insert( - executor_id.as_str(), - vec![protobuf::PartitionId { - job_id: job_id.to_owned(), - stage_id: partition.stage_id as u32, - partition_id: partition.partition_id as u32, - }], - ); - } - } - - for (executor_id, partitions) in tasks { - if let Ok(mut client) = executor_manager.get_client(executor_id).await { - client - .cancel_tasks(CancelTasksParams { - partition_id: partitions, - }) - .await?; - } else { - error!("Failed to get client for executor ID {}", executor_id) - } + if let Some(graph) = self.get_active_execution_graph(job_id).await { + let running_tasks = graph.read().await.running_tasks(); + info!( + "Cancelling {} running tasks for job {}", + running_tasks.len(), + job_id + ); + self.fail_job_state(lock, job_id, failure_reason).await?; + Ok(running_tasks) + } else { + // TODO listen the job state update event and fix task cancelling + warn!("Fail to find job {} in the cache, unable to cancel tasks for job, fail the job state only.", job_id); + self.fail_job_state(lock, job_id, failure_reason).await?; + Ok(vec![]) } - - Ok(()) } - /// Mark a job as failed. This will create a key under the FailedJobs keyspace + /// Mark a unscheduled job as failed. This will create a key under the FailedJobs keyspace /// and remove the job from ActiveJobs or QueuedJobs /// TODO this should be atomic - pub async fn fail_job(&self, job_id: &str, error_message: String) -> Result<()> { + pub async fn fail_unscheduled_job( + &self, + job_id: &str, + failure_reason: String, + ) -> Result<()> { debug!("Moving job {} from Active or Queue to Failed", job_id); let lock = self.state.lock(Keyspace::ActiveJobs, "").await?; - self.fail_job_inner(lock, job_id, error_message).await + self.fail_job_state(lock, job_id, failure_reason).await } - async fn fail_job_inner( + async fn fail_job_state( &self, lock: Box, job_id: &str, - error_message: String, + failure_reason: String, ) -> Result<()> { with_lock(lock, self.state.delete(Keyspace::ActiveJobs, job_id)).await?; let value = if let Some(graph) = self.get_active_execution_graph(job_id).await { let mut graph = graph.write().await; - graph.fail_job(error_message); + for stage_id in graph.running_stages() { + graph.fail_stage(stage_id, failure_reason.clone()); + } + graph.fail_job(failure_reason); let graph = graph.clone(); - self.encode_execution_graph(graph)? } else { warn!("Fail to find job {} in the cache", job_id); let status = JobStatus { status: Some(job_status::Status::Failed(FailedJob { - error: error_message.clone(), + error: failure_reason.clone(), })), }; encode_protobuf(&status)? @@ -343,27 +332,6 @@ impl TaskManager Ok(()) } - /// Mark a job as failed. This will create a key under the FailedJobs keyspace - /// and remove the job from ActiveJobs or QueuedJobs - /// TODO this should be atomic - pub async fn fail_running_job(&self, job_id: &str) -> Result<()> { - if let Some(graph) = self.get_active_execution_graph(job_id).await { - let graph = graph.read().await.clone(); - let value = self.encode_execution_graph(graph)?; - - debug!("Moving job {} from Active to Failed", job_id); - let lock = self.state.lock(Keyspace::ActiveJobs, "").await?; - with_lock(lock, self.state.delete(Keyspace::ActiveJobs, job_id)).await?; - self.state - .put(Keyspace::FailedJobs, job_id.to_owned(), value) - .await?; - } else { - warn!("Fail to find job {} in the cache", job_id); - } - - Ok(()) - } - pub async fn update_job(&self, job_id: &str) -> Result<()> { debug!("Update job {} in Active", job_id); if let Some(graph) = self.get_active_execution_graph(job_id).await { @@ -381,16 +349,20 @@ impl TaskManager Ok(()) } - pub async fn executor_lost(&self, executor_id: &str) -> Result<()> { + /// return a Vec of running tasks need to cancel + pub async fn executor_lost(&self, executor_id: &str) -> Result> { + // Collect all the running task need to cancel when there are running stages rolled back. + let mut running_tasks_to_cancel: Vec = vec![]; // Collect graphs we update so we can update them in storage let mut updated_graphs: HashMap = HashMap::new(); { let job_cache = self.active_job_cache.read().await; for (job_id, graph) in job_cache.iter() { let mut graph = graph.write().await; - let reset = graph.reset_stages(executor_id)?; - if !reset.is_empty() { + let reset = graph.reset_stages_on_lost_executor(executor_id)?; + if !reset.0.is_empty() { updated_graphs.insert(job_id.to_owned(), graph.clone()); + running_tasks_to_cancel.extend(reset.1); } } } @@ -406,7 +378,7 @@ impl TaskManager }) .collect::>>()?; self.state.put_txn(txn_ops).await?; - Ok(()) + Ok(running_tasks_to_cancel) }) .await } @@ -416,7 +388,7 @@ impl TaskManager pub(crate) async fn launch_task( &self, executor: &ExecutorMetadata, - task: Task, + task: TaskDef, executor_manager: &ExecutorManager, ) -> Result<()> { info!("Launching task {:?} on executor {:?}", task, executor.id); @@ -442,7 +414,7 @@ impl TaskManager pub(crate) async fn launch_task( &self, _executor: &ExecutorMetadata, - _task: Task, + _task: TaskDef, _executor_manager: &ExecutorManager, ) -> Result<()> { Ok(()) @@ -461,7 +433,7 @@ impl TaskManager } #[allow(dead_code)] - pub fn prepare_task_definition(&self, task: Task) -> Result { + pub fn prepare_task_definition(&self, task: TaskDef) -> Result { debug!("Preparing task definition for {:?}", task); let mut plan_buf: Vec = vec![]; let plan_proto = @@ -472,14 +444,19 @@ impl TaskManager hash_partitioning_to_proto(task.output_partitioning.as_ref())?; let task_definition = TaskDefinition { - task_id: Some(protobuf::PartitionId { - job_id: task.partition.job_id.clone(), - stage_id: task.partition.stage_id as u32, - partition_id: task.partition.partition_id as u32, - }), + task_id: task.task_id as u32, + task_attempt_num: task.task_attempt as u32, + job_id: task.partition.job_id.clone(), + stage_id: task.partition.stage_id as u32, + stage_attempt_num: task.stage_attempt_num as u32, + partition_id: task.partition.partition_id as u32, plan: plan_buf, output_partitioning, session_id: task.session_id, + launch_time: SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as u64, props: vec![], }; Ok(task_definition) From 48f6e3f5e8271db6e5cf75d2e097e08ce656ee64 Mon Sep 17 00:00:00 2001 From: Wang Date: Sat, 24 Sep 2022 23:56:24 +0800 Subject: [PATCH 2/4] Add UT --- ballista/rust/core/proto/ballista.proto | 12 +- .../src/execution_plans/shuffle_reader.rs | 53 + .../core/src/serde/scheduler/from_proto.rs | 1 + ballista/rust/core/src/serde/scheduler/mod.rs | 1 + .../rust/core/src/serde/scheduler/to_proto.rs | 1 + ballista/rust/scheduler/src/flight_sql.rs | 8 +- .../scheduler/src/state/execution_graph.rs | 1246 ++++++++++++++++- .../state/execution_graph/execution_stage.rs | 109 +- .../rust/scheduler/src/state/task_manager.rs | 6 +- 9 files changed, 1351 insertions(+), 86 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index 4d3190b66..e52c2937a 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -449,6 +449,7 @@ message UnResolvedStage { repeated uint32 output_links = 4; repeated GraphStageInput inputs = 5; bytes plan = 6; + repeated string last_attempt_failure_reasons = 7; } message ResolvedStage { @@ -459,6 +460,7 @@ message ResolvedStage { repeated uint32 output_links = 5; repeated GraphStageInput inputs = 6; bytes plan = 7; + repeated string last_attempt_failure_reasons = 8; } message SuccessfulStage { @@ -550,12 +552,12 @@ message FetchPartition { string path = 4; } -// Mapping from partition id to executor id message PartitionLocation { - PartitionId partition_id = 1; - ExecutorMetadata executor_meta = 2; - PartitionStats partition_stats = 3; - string path = 4; + uint32 map_partition_id = 1; + PartitionId partition_id = 2; + ExecutorMetadata executor_meta = 3; + PartitionStats partition_stats = 4; + string path = 5; } // Unique identifier for a materialized partition of data diff --git a/ballista/rust/core/src/execution_plans/shuffle_reader.rs b/ballista/rust/core/src/execution_plans/shuffle_reader.rs index a963216a2..9696ede8e 100644 --- a/ballista/rust/core/src/execution_plans/shuffle_reader.rs +++ b/ballista/rust/core/src/execution_plans/shuffle_reader.rs @@ -215,6 +215,10 @@ async fn fetch_partition( #[cfg(test)] mod tests { use super::*; + use crate::serde::scheduler::{ExecutorMetadata, ExecutorSpecification, PartitionId}; + use crate::utils; + use datafusion::arrow::datatypes::{DataType, Field, Schema}; + use datafusion::prelude::SessionContext; #[tokio::test] async fn test_stats_for_partitions_empty() { @@ -283,4 +287,53 @@ mod tests { assert_eq!(result, exptected); } + + #[tokio::test] + async fn test_fetch_partitions_error_mapping() -> Result<()> { + let session_ctx = SessionContext::new(); + let task_ctx = session_ctx.task_ctx(); + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + ]); + + let job_id = "test_job_1"; + let mut partitions: Vec = vec![]; + for partition_id in 0..4 { + partitions.push(PartitionLocation { + map_partition_id: 0, + partition_id: PartitionId { + job_id: job_id.to_string(), + stage_id: 2, + partition_id, + }, + executor_meta: ExecutorMetadata { + id: "executor_1".to_string(), + host: "executor_1".to_string(), + port: 7070, + grpc_port: 8080, + specification: ExecutorSpecification { task_slots: 1 }, + }, + partition_stats: Default::default(), + path: "test_path".to_string(), + }) + } + + let shuffle_reader_exec = + ShuffleReaderExec::try_new(vec![partitions], Arc::new(schema))?; + let mut stream = shuffle_reader_exec.execute(0, task_ctx)?; + let batches = utils::collect_stream(&mut stream).await; + + assert!(batches.is_err()); + + // BallistaError::FetchFailed -> ArrowError::ExternalError -> ballistaError::FetchFailed + let ballista_error = batches.unwrap_err(); + assert!(matches!( + ballista_error, + BallistaError::FetchFailed(_, _, _, _) + )); + + Ok(()) + } } diff --git a/ballista/rust/core/src/serde/scheduler/from_proto.rs b/ballista/rust/core/src/serde/scheduler/from_proto.rs index 728ad46be..2b0a147b3 100644 --- a/ballista/rust/core/src/serde/scheduler/from_proto.rs +++ b/ballista/rust/core/src/serde/scheduler/from_proto.rs @@ -87,6 +87,7 @@ impl TryInto for protobuf::PartitionLocation { fn try_into(self) -> Result { Ok(PartitionLocation { + map_partition_id: self.map_partition_id as usize, partition_id: self .partition_id .ok_or_else(|| { diff --git a/ballista/rust/core/src/serde/scheduler/mod.rs b/ballista/rust/core/src/serde/scheduler/mod.rs index 5afdb30e0..77d290516 100644 --- a/ballista/rust/core/src/serde/scheduler/mod.rs +++ b/ballista/rust/core/src/serde/scheduler/mod.rs @@ -63,6 +63,7 @@ impl PartitionId { #[derive(Debug, Clone)] pub struct PartitionLocation { + pub map_partition_id: usize, pub partition_id: PartitionId, pub executor_meta: ExecutorMetadata, pub partition_stats: PartitionStats, diff --git a/ballista/rust/core/src/serde/scheduler/to_proto.rs b/ballista/rust/core/src/serde/scheduler/to_proto.rs index 10c841e94..e46f8e896 100644 --- a/ballista/rust/core/src/serde/scheduler/to_proto.rs +++ b/ballista/rust/core/src/serde/scheduler/to_proto.rs @@ -68,6 +68,7 @@ impl TryInto for PartitionLocation { fn try_into(self) -> Result { Ok(protobuf::PartitionLocation { + map_partition_id: self.map_partition_id as u32, partition_id: Some(self.partition_id.into()), executor_meta: Some(self.executor_meta.into()), partition_stats: Some(self.partition_stats.into()), diff --git a/ballista/rust/scheduler/src/flight_sql.rs b/ballista/rust/scheduler/src/flight_sql.rs index 2b1c3fcda..921df8504 100644 --- a/ballista/rust/scheduler/src/flight_sql.rs +++ b/ballista/rust/scheduler/src/flight_sql.rs @@ -40,7 +40,7 @@ use arrow_flight::SchemaAsIpc; use ballista_core::config::BallistaConfig; use ballista_core::serde::protobuf; use ballista_core::serde::protobuf::job_status; -use ballista_core::serde::protobuf::CompletedJob; +use ballista_core::serde::protobuf::SuccessfulJob; use ballista_core::serde::protobuf::JobStatus; use ballista_core::serde::protobuf::PhysicalPlanNode; use datafusion::arrow; @@ -96,7 +96,7 @@ impl FlightSqlServiceImpl { Ok(plan) } - async fn check_job(&self, job_id: &String) -> Result, Status> { + async fn check_job(&self, job_id: &String) -> Result, Status> { let status = self .server .state @@ -134,13 +134,13 @@ impl FlightSqlServiceImpl { e.error )))? } - job_status::Status::Completed(comp) => Ok(Some(comp)), + job_status::Status::Successful(comp) => Ok(Some(comp)), } } async fn job_to_fetch_part( &self, - completed: CompletedJob, + completed: SuccessfulJob, num_rows: &mut i64, num_bytes: &mut i64, ) -> Result, Status> { diff --git a/ballista/rust/scheduler/src/state/execution_graph.rs b/ballista/rust/scheduler/src/state/execution_graph.rs index 5c83aff3c..c85cdd518 100644 --- a/ballista/rust/scheduler/src/state/execution_graph.rs +++ b/ballista/rust/scheduler/src/state/execution_graph.rs @@ -241,6 +241,9 @@ impl ExecutionGraph { // It will be refined later self.revive(); + let current_running_stages: HashSet = + HashSet::from_iter(self.running_stages()); + // Copy the failed stage attempts from self let mut failed_stage_attempts: HashMap> = HashMap::new(); for (stage_id, attempts) in self.failed_stage_attempts.iter() { @@ -248,14 +251,13 @@ impl ExecutionGraph { .insert(*stage_id, HashSet::from_iter(attempts.iter().copied())); } - let mut resolved_stages = vec![]; - let mut successful_stages = vec![]; - + let mut resolved_stages = HashSet::new(); + let mut successful_stages = HashSet::new(); let mut failed_stages = HashMap::new(); - - let mut rollback_running_stages = HashSet::new(); + let mut rollback_running_stages = HashMap::new(); let mut resubmit_successful_stages: HashMap> = HashMap::new(); + let mut reset_running_stages: HashMap> = HashMap::new(); for (stage_id, stage_task_statuses) in job_task_statuses { if let Some(stage) = self.stages.get_mut(&stage_id) { @@ -272,6 +274,14 @@ impl ExecutionGraph { continue; } let partition_id = task_status.clone().partition_id as usize; + let task_identity = format!( + "TID {} {}/{}.{}/{}", + task_status.task_id, + job_id, + stage_id, + task_stage_attempt_num, + partition_id + ); let operator_metrics = task_status.metrics.clone(); if !running_stage @@ -294,28 +304,48 @@ impl ExecutionGraph { .or_insert_with(HashSet::new); failed_attempts.insert(task_stage_attempt_num); if failed_attempts.len() < max_stage_failures { - let map_stage_id = - fetch_partiton_error.map_stage_id; - let map_partition_id = - fetch_partiton_error.map_partition_id; - - if failed_stages.contains_key(&stage_id) { + let map_stage_id = fetch_partiton_error + .map_stage_id + as usize; + let map_partition_id = fetch_partiton_error + .map_partition_id + as usize; + let executor_id = + fetch_partiton_error.executor_id; + + if !failed_stages.is_empty() { let error_msg = format!( - "Stage {} was marked failed, ignore FetchPartitionError from task with TID {}",stage_id, task_status.task_id); + "Stages was marked failed, ignore FetchPartitionError from task {}", task_identity); warn!("{}", error_msg); } else { - running_stage.remove_input_partition( - map_stage_id as usize, - map_partition_id as usize, - )?; + // There are different removal strategies here. + // We can choose just remove the map_partition_id in the FetchPartitionError, when resubmit the input stage, there are less tasks + // need to rerun, but this might miss many more bad input partitions, lead to more stage level retries in following. + // Here we choose remove all the bad input partitions which match the same executor id in this single input stage. + // There are other more aggressive approaches, like considering the executor is lost and check all the running stages in this graph. + // Or count the fetch failure number on executor and mark the executor lost globally. + let removed_map_partitions = + running_stage + .remove_input_partitions( + map_stage_id, + map_partition_id, + &executor_id, + )?; + + let failure_reasons = + rollback_running_stages + .entry(stage_id) + .or_insert_with(HashSet::new); + failure_reasons.insert(executor_id); - rollback_running_stages.insert(stage_id); let missing_inputs = resubmit_successful_stages - .entry(map_stage_id as usize) + .entry(map_stage_id) .or_insert_with(HashSet::new); missing_inputs - .insert(map_partition_id as usize); + .extend(removed_map_partitions); + warn!("Need to resubmit the current running Stage {} and its map Stage {} due to FetchPartitionError from task {}", + stage_id, map_stage_id, task_identity) } } else { let error_msg = format!( @@ -378,22 +408,24 @@ impl ExecutionGraph { locations.append(&mut partition_to_location( &job_id, + partition_id, stage_id, executor, successful_task.partitions, )); } else { warn!( - "The task {}/{}/{}'s status is invalid for updating", - job_id, stage_id, partition_id + "The task {}'s status is invalid for updating", + task_identity ); } } } - let is_successful = running_stage.is_successful(); - if is_successful { - successful_stages.push(stage_id); - // if this stage is successful, we want to combine the stage metrics to plan's metric set and print out the plan + let is_final_successful = running_stage.is_successful() + && !reset_running_stages.contains_key(&stage_id); + if is_final_successful { + successful_stages.insert(stage_id); + // if this stage is final successful, we want to combine the stage metrics to plan's metric set and print out the plan if let Some(stage_metrics) = running_stage.stage_metrics.as_ref() { print_stage_metrics( @@ -406,12 +438,99 @@ impl ExecutionGraph { } let output_links = running_stage.output_links.clone(); - resolved_stages.append(&mut self.update_stage_output_links( - stage_id, - is_successful, - locations, - output_links, - )?); + resolved_stages.extend( + &mut self + .update_stage_output_links( + stage_id, + is_final_successful, + locations, + output_links, + )? + .into_iter(), + ); + } else if let ExecutionStage::UnResolved(unsolved_stage) = stage { + for task_status in stage_task_statuses.into_iter() { + let stage_id = stage_id as usize; + let task_stage_attempt_num = + task_status.stage_attempt_num as usize; + let partition_id = task_status.clone().partition_id as usize; + let task_identity = format!( + "TID {} {}/{}.{}/{}", + task_status.task_id, + job_id, + stage_id, + task_stage_attempt_num, + partition_id + ); + let mut should_ignore = true; + // handle delayed failed tasks if the stage's next attempt is still in UnResolved status. + if let Some(task_status::Status::Failed(failed_task)) = + task_status.status + { + if unsolved_stage.stage_attempt_num - task_stage_attempt_num + == 1 + { + let failed_reason = failed_task.failed_reason; + match failed_reason { + Some(FailedReason::ExecutionError(_)) => { + should_ignore = false; + failed_stages.insert(stage_id, failed_task.error); + } + Some(FailedReason::FetchPartitionError( + fetch_partiton_error, + )) if failed_stages.is_empty() + && current_running_stages.contains( + &(fetch_partiton_error.map_stage_id as usize), + ) + && !unsolved_stage + .last_attempt_failure_reasons + .contains( + &fetch_partiton_error.executor_id, + ) => + { + should_ignore = false; + unsolved_stage + .last_attempt_failure_reasons + .insert( + fetch_partiton_error.executor_id.clone(), + ); + let map_stage_id = + fetch_partiton_error.map_stage_id as usize; + let map_partition_id = fetch_partiton_error + .map_partition_id + as usize; + let executor_id = + fetch_partiton_error.executor_id; + let removed_map_partitions = unsolved_stage + .remove_input_partitions( + map_stage_id, + map_partition_id, + &executor_id, + )?; + + let missing_inputs = reset_running_stages + .entry(map_stage_id) + .or_insert_with(HashSet::new); + missing_inputs.extend(removed_map_partitions); + warn!("Need to reset the current running Stage {} due to late come FetchPartitionError from its parent stage {} of task {}", + map_stage_id, stage_id, task_identity); + + // If the previous other task updates had already mark the map stage success, need to remove it. + if successful_stages.contains(&map_stage_id) { + successful_stages.remove(&map_stage_id); + } + if resolved_stages.contains(&stage_id) { + resolved_stages.remove(&stage_id); + } + } + _ => {} + } + } + } + if should_ignore { + warn!("Ignore TaskStatus update of task with TID {} as the Stage {}/{} is in UnResolved status", task_identity, job_id, stage_id); + } + } } else { warn!( "Stage {}/{} is not in running when updating the status of tasks {:?}", @@ -438,6 +557,12 @@ impl ExecutionGraph { if let Some(stage) = self.stages.get_mut(stage_id) { if let ExecutionStage::Successful(success_stage) = stage { for partition in missing_parts { + if *partition > success_stage.partitions { + return Err(BallistaError::Internal(format!( + "Invalid partition ID {} in map stage {}", + *partition, stage_id + ))); + } let task_info = &mut success_stage.task_infos[*partition]; // Update the task info to failed task_info.task_status = task_status::Status::Failed(FailedTask { @@ -461,6 +586,32 @@ impl ExecutionGraph { } } + for (stage_id, missing_parts) in &reset_running_stages { + if let Some(stage) = self.stages.get_mut(stage_id) { + if let ExecutionStage::Running(running_stage) = stage { + for partition in missing_parts { + if *partition > running_stage.partitions { + return Err(BallistaError::Internal(format!( + "Invalid partition ID {} in map stage {}", + *partition, stage_id + ))); + } + running_stage.reset_task_info(*partition); + } + } else { + warn!( + "Stage {}/{} is not in Running state when try to reset the running task. ", + job_id, + stage_id); + } + } else { + return Err(BallistaError::Internal(format!( + "Invalid stage ID {} for job {}", + stage_id, job_id + ))); + } + } + self.processing_stages_update(UpdatedStages { resolved_stages, successful_stages, @@ -501,8 +652,8 @@ impl ExecutionGraph { // Only handle the rollback logic when there are no failed stages if updated_stages.failed_stages.is_empty() { let mut running_tasks_to_cancel = vec![]; - for stage_id in updated_stages.rollback_running_stages { - let tasks = self.rollback_running_stage(stage_id)?; + for (stage_id, failure_reasons) in updated_stages.rollback_running_stages { + let tasks = self.rollback_running_stage(stage_id, failure_reasons)?; running_tasks_to_cancel.extend(tasks); } @@ -645,6 +796,16 @@ impl ExecutionGraph { /// If the task is not launched the status must be reset to allow the task to /// be scheduled elsewhere. pub fn pop_next_task(&mut self, executor_id: &str) -> Result> { + if matches!( + self.status, + JobStatus { + status: Some(job_status::Status::Failed(_)), + } + ) { + warn!("Call pop_next_task on failed Job"); + return Ok(None); + } + let job_id = self.job_id.clone(); let session_id = self.session_id.clone(); @@ -809,19 +970,9 @@ impl ExecutionGraph { let mut match_found = false; stage_output.partition_locations.iter_mut().for_each( |(_partition, locs)| { - let indexes = locs - .iter() - .enumerate() - .filter_map(|(idx, loc)| { - (loc.executor_meta.id == executor_id).then(|| idx) - }) - .collect::>(); - - // remove the matched partition locations - if !indexes.is_empty() { - for idx in &indexes { - locs.remove(*idx); - } + let before_len = locs.len(); + locs.retain(|loc| loc.executor_meta.id != executor_id); + if locs.len() < before_len { match_found = true; } }, @@ -883,7 +1034,10 @@ impl ExecutionGraph { let mut all_running_tasks = vec![]; for stage_id in rollback_running_stages.iter() { - let tasks = self.rollback_running_stage(*stage_id)?; + let tasks = self.rollback_running_stage( + *stage_id, + HashSet::from([executor_id.to_owned()]), + )?; all_running_tasks.extend(tasks); } @@ -953,6 +1107,7 @@ impl ExecutionGraph { pub fn rollback_running_stage( &mut self, stage_id: usize, + failure_reasons: HashSet, ) -> Result> { if let Some(ExecutionStage::Running(stage)) = self.stages.remove(&stage_id) { let running_tasks = stage @@ -968,8 +1123,10 @@ impl ExecutionGraph { }, ) .collect(); - self.stages - .insert(stage_id, ExecutionStage::UnResolved(stage.to_unresolved()?)); + self.stages.insert( + stage_id, + ExecutionStage::UnResolved(stage.to_unresolved(failure_reasons)?), + ); Ok(running_tasks) } else { warn!( @@ -1264,6 +1421,7 @@ impl ExecutionStageBuilder { partitioning, output_links, HashMap::new(), + HashSet::new(), )) } else { ExecutionStage::UnResolved(UnresolvedStage::new( @@ -1350,6 +1508,7 @@ impl Debug for TaskDefinition { fn partition_to_location( job_id: &str, + map_partition_id: usize, stage_id: usize, executor: &ExecutorMetadata, shuffles: Vec, @@ -1357,6 +1516,7 @@ fn partition_to_location( shuffles .into_iter() .map(|shuffle| PartitionLocation { + map_partition_id, partition_id: PartitionId { job_id: job_id.to_owned(), stage_id, @@ -1375,17 +1535,22 @@ fn partition_to_location( #[cfg(test)] mod test { + use std::collections::HashSet; use std::sync::Arc; use datafusion::arrow::datatypes::{DataType, Field, Schema}; - use datafusion::logical_expr::{col, sum, Expr}; + use datafusion::logical_expr::{col, count, sum, Expr}; use datafusion::logical_plan::JoinType; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion::test_util::scan_empty; + use crate::scheduler_server::event::QueryStageSchedulerEvent; use ballista_core::error::Result; - use ballista_core::serde::protobuf::{self, job_status, task_status, TaskStatus}; + use ballista_core::serde::protobuf::{ + self, failed_task, job_status, task_status, ExecutionError, FailedTask, + FetchPartitionError, IoError, JobStatus, TaskKilled, TaskStatus, + }; use ballista_core::serde::scheduler::{ExecutorMetadata, ExecutorSpecification}; use crate::state::execution_graph::{ExecutionGraph, TaskDefinition}; @@ -1469,7 +1634,7 @@ mod test { } #[tokio::test] - async fn test_reset_completed_stage() -> Result<()> { + async fn test_reset_completed_stage_executor_lost() -> Result<()> { let executor1 = mock_executor("executor-id1".to_string()); let executor2 = mock_executor("executor-id2".to_string()); let mut join_graph = test_join_plan(4).await; @@ -1520,7 +1685,7 @@ mod test { } #[tokio::test] - async fn test_reset_resolved_stage() -> Result<()> { + async fn test_reset_resolved_stage_executor_lost() -> Result<()> { let executor1 = mock_executor("executor-id1".to_string()); let executor2 = mock_executor("executor-id2".to_string()); let mut join_graph = test_join_plan(4).await; @@ -1621,6 +1786,915 @@ mod test { Ok(()) } + #[tokio::test] + async fn test_do_not_retry_killed_task() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let mut agg_graph = test_aggregation_plan(4).await; + // Call revive to move the leaf Resolved stages to Running + agg_graph.revive(); + + // Complete the first stage + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // 1st task in the second stage + let task1 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status1 = mock_completed_task(task1, &executor2.id); + + // 2rd task in the second stage + let task2 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status2 = mock_failed_task( + task2, + FailedTask { + error: "Killed".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::TaskKilled(TaskKilled {})), + }, + ); + + agg_graph.update_task_status( + &executor2, + vec![task_status1, task_status2], + 4, + 4, + )?; + + // TODO the JobStatus is not 'Running' here, no place to mark it to 'Running' in current code base. + assert!( + matches!( + agg_graph.status, + JobStatus { + status: Some(job_status::Status::Queued(_)) + } + ), + "Expected job status to be running" + ); + + assert_eq!(agg_graph.available_tasks(), 2); + drain_tasks(&mut agg_graph)?; + assert_eq!(agg_graph.available_tasks(), 0); + + assert!( + !agg_graph.is_successful(), + "Expected the agg graph can not complete" + ); + Ok(()) + } + + #[tokio::test] + async fn test_max_task_failed_count() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let mut agg_graph = test_aggregation_plan(2).await; + // Call revive to move the leaf Resolved stages to Running + agg_graph.revive(); + + // Complete the first stage + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // 1st task in the second stage + let task1 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status1 = mock_completed_task(task1, &executor2.id); + + // 2rd task in the second stage, failed due to IOError + let task2 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status2 = mock_failed_task( + task2.clone(), + FailedTask { + error: "IOError".to_string(), + retryable: true, + count_to_failures: true, + failed_reason: Some(failed_task::FailedReason::IoError(IoError {})), + }, + ); + + agg_graph.update_task_status( + &executor2, + vec![task_status1, task_status2], + 4, + 4, + )?; + + assert_eq!(agg_graph.available_tasks(), 1); + + let mut last_attempt = 0; + // 2rd task's attempts + for attempt in 1..5 { + if let Some(task2_attempt) = agg_graph.pop_next_task(&executor2.id)? { + assert_eq!( + task2_attempt.partition.partition_id, + task2.partition.partition_id + ); + assert_eq!(task2_attempt.task_attempt, attempt); + last_attempt = task2_attempt.task_attempt; + let task_status = mock_failed_task( + task2_attempt.clone(), + FailedTask { + error: "IOError".to_string(), + retryable: true, + count_to_failures: true, + failed_reason: Some(failed_task::FailedReason::IoError( + IoError {}, + )), + }, + ); + agg_graph.update_task_status(&executor2, vec![task_status], 4, 4)?; + } + } + + assert!( + matches!( + agg_graph.status, + JobStatus { + status: Some(job_status::Status::Failed(_)) + } + ), + "Expected job status to be Failed" + ); + + assert_eq!(last_attempt, 3); + + let failure_reason = format!("{:?}", agg_graph.status); + assert!(failure_reason.contains("Task 1 in Stage 2 failed 4 times, fail the stage, most recent failure reason")); + assert!(failure_reason.contains("IOError")); + assert!(!agg_graph.is_successful()); + + Ok(()) + } + + #[tokio::test] + async fn test_long_delayed_failed_task_after_executor_lost() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let mut agg_graph = test_aggregation_plan(4).await; + // Call revive to move the leaf Resolved stages to Running + agg_graph.revive(); + + // Complete the Stage 1 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; + } + + // 1st task in the Stage 2 + if let Some(task) = agg_graph.pop_next_task(&executor2.id)? { + let task_status = mock_completed_task(task, &executor2.id); + agg_graph.update_task_status(&executor2, vec![task_status], 1, 1)?; + } + + // 2rd task in the Stage 2 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; + } + + // 3rd task in the Stage 2, scheduled on executor 2 but not completed + let task = agg_graph.pop_next_task(&executor2.id)?; + + // There is 1 task pending schedule now + assert_eq!(agg_graph.available_tasks(), 1); + + // executor 1 lost + let reset = agg_graph.reset_stages_on_lost_executor(&executor1.id)?; + + // Two stages were reset, Stage 2 rollback to Unresolved and Stage 1 move to Running + assert_eq!(reset.0.len(), 2); + assert_eq!(agg_graph.available_tasks(), 1); + + // Complete the Stage 1 again + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 1, 1)?; + } + + // Stage 2 move to Running + agg_graph.revive(); + assert_eq!(agg_graph.available_tasks(), 4); + + // 3rd task in Stage 2 update comes very late due to runtime execution error. + let task_status = mock_failed_task( + task.unwrap(), + FailedTask { + error: "ExecutionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::ExecutionError( + ExecutionError {}, + )), + }, + ); + + // This long delayed failed task should not failure the stage/job and should not trigger any query stage events + let query_stage_events = + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + assert!(query_stage_events.is_empty()); + + drain_tasks(&mut agg_graph)?; + assert!(agg_graph.is_successful(), "Failed to complete agg plan"); + + Ok(()) + } + + #[tokio::test] + async fn test_normal_fetch_failure() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let mut agg_graph = test_aggregation_plan(4).await; + // Call revive to move the leaf Resolved stages to Running + agg_graph.revive(); + + // Complete the Stage 1 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // 1st task in the Stage 2 + let task1 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status1 = mock_completed_task(task1, &executor2.id); + + // 2nd task in the Stage 2, failed due to FetchPartitionError + let task2 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status2 = mock_failed_task( + task2, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 1, + map_partition_id: 0, + }, + )), + }, + ); + + let mut running_task_count = 0; + while let Some(_task) = agg_graph.pop_next_task(&executor2.id)? { + running_task_count += 1; + } + assert_eq!(running_task_count, 2); + + let stage_events = agg_graph.update_task_status( + &executor2, + vec![task_status1, task_status2], + 4, + 4, + )?; + + assert_eq!(stage_events.len(), 1); + assert!(matches!( + stage_events[0], + QueryStageSchedulerEvent::CancelTasks(_) + )); + + // Stage 1 is running + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 1); + assert_eq!(agg_graph.available_tasks(), 1); + + drain_tasks(&mut agg_graph)?; + assert!(agg_graph.is_successful(), "Failed to complete agg plan"); + Ok(()) + } + + #[tokio::test] + async fn test_many_fetch_failures_in_one_stage() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let executor3 = mock_executor("executor-id3".to_string()); + let mut agg_graph = test_two_aggregations_plan(8).await; + + agg_graph.revive(); + assert_eq!(agg_graph.stage_count(), 3); + + // Complete the Stage 1 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // Complete the Stage 2, 5 tasks run on executor_2 and 3 tasks run on executor_1 + for _i in 0..5 { + if let Some(task) = agg_graph.pop_next_task(&executor2.id)? { + let task_status = mock_completed_task(task, &executor2.id); + agg_graph.update_task_status(&executor2, vec![task_status], 4, 4)?; + } + } + assert_eq!(agg_graph.available_tasks(), 3); + for _i in 0..3 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + } + + // Run Stage 3, 6 tasks failed due to FetchPartitionError on different map partitions on executor_2 + let mut many_fetch_failure_status = vec![]; + for part in 2..8 { + if let Some(task) = agg_graph.pop_next_task(&executor3.id)? { + let task_status = mock_failed_task( + task, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some( + failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor2.id.clone(), + map_stage_id: 2, + map_partition_id: part, + }, + ), + ), + }, + ); + many_fetch_failure_status.push(task_status); + } + } + assert_eq!(many_fetch_failure_status.len(), 6); + agg_graph.update_task_status(&executor3, many_fetch_failure_status, 4, 4)?; + + // The Running stage should be Stage 2 now + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 5); + + drain_tasks(&mut agg_graph)?; + assert!(agg_graph.is_successful(), "Failed to complete agg plan"); + Ok(()) + } + + #[tokio::test] + async fn test_many_consecutive_stage_fetch_failures() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let mut agg_graph = test_aggregation_plan(4).await; + // Call revive to move the leaf Resolved stages to Running + agg_graph.revive(); + + for attempt in 0..6 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // 1rd task in the Stage 2, failed due to FetchPartitionError + if let Some(task1) = agg_graph.pop_next_task(&executor2.id)? { + let task_status1 = mock_failed_task( + task1.clone(), + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some( + failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 1, + map_partition_id: 0, + }, + ), + ), + }, + ); + + let stage_events = + agg_graph.update_task_status(&executor2, vec![task_status1], 4, 4)?; + + if attempt < 3 { + // No JobRunningFailed stage events + assert_eq!(stage_events.len(), 0); + // Stage 1 is running + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 1); + assert_eq!(agg_graph.available_tasks(), 1); + } else { + // Job is failed after exceeds the max_stage_failures + assert_eq!(stage_events.len(), 1); + assert!(matches!( + stage_events[0], + QueryStageSchedulerEvent::JobRunningFailed(_, _) + )); + // Stage 2 is still running + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + } + } + } + + drain_tasks(&mut agg_graph)?; + assert!(!agg_graph.is_successful(), "Expect to fail the agg plan"); + + let failure_reason = format!("{:?}", agg_graph.status); + assert!(failure_reason.contains("Job failed due to stage 2 failed: Stage 2 has failed 4 times, most recent failure reason")); + assert!(failure_reason.contains("FetchPartitionError")); + + Ok(()) + } + + #[tokio::test] + async fn test_long_delayed_fetch_failures() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let executor3 = mock_executor("executor-id3".to_string()); + let mut agg_graph = test_two_aggregations_plan(8).await; + + agg_graph.revive(); + assert_eq!(agg_graph.stage_count(), 3); + + // Complete the Stage 1 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // Complete the Stage 2, 5 tasks run on executor_2, 2 tasks run on executor_1, 1 task runs on executor_3 + for _i in 0..5 { + if let Some(task) = agg_graph.pop_next_task(&executor2.id)? { + let task_status = mock_completed_task(task, &executor2.id); + agg_graph.update_task_status(&executor2, vec![task_status], 4, 4)?; + } + } + assert_eq!(agg_graph.available_tasks(), 3); + + for _i in 0..2 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + } + + if let Some(task) = agg_graph.pop_next_task(&executor3.id)? { + let task_status = mock_completed_task(task, &executor3.id); + agg_graph.update_task_status(&executor3, vec![task_status], 4, 4)?; + } + assert_eq!(agg_graph.available_tasks(), 0); + + //Run Stage 3 + // 1st task scheduled + let task_1 = agg_graph.pop_next_task(&executor3.id)?.unwrap(); + // 2nd task scheduled + let task_2 = agg_graph.pop_next_task(&executor3.id)?.unwrap(); + // 3rd task scheduled + let task_3 = agg_graph.pop_next_task(&executor3.id)?.unwrap(); + // 4th task scheduled + let task_4 = agg_graph.pop_next_task(&executor3.id)?.unwrap(); + // 5th task scheduled + let task_5 = agg_graph.pop_next_task(&executor3.id)?.unwrap(); + + // Stage 3, 1st task failed due to FetchPartitionError(executor2) + let task_status_1 = mock_failed_task( + task_1, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor2.id.clone(), + map_stage_id: 2, + map_partition_id: 0, + }, + )), + }, + ); + agg_graph.update_task_status(&executor3, vec![task_status_1], 4, 4)?; + + // The Running stage is Stage 2 now + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 5); + + // Stage 3, 2nd task failed due to FetchPartitionError(executor2) + let task_status_2 = mock_failed_task( + task_2, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor2.id.clone(), + map_stage_id: 2, + map_partition_id: 1, + }, + )), + }, + ); + // This task update should be ignored + agg_graph.update_task_status(&executor3, vec![task_status_2], 4, 4)?; + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 5); + + // Stage 3, 3rd task failed due to FetchPartitionError(executor1) + let task_status_3 = mock_failed_task( + task_3, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 2, + map_partition_id: 1, + }, + )), + }, + ); + // This task update should be handled because it has a different failure reason + agg_graph.update_task_status(&executor3, vec![task_status_3], 4, 4)?; + // Running stage is still Stage 2, but available tasks changed to 7 + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 7); + + // Finish 4 tasks in Stage 2, to make some progress + for _i in 0..4 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + } + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 3); + + // Stage 3, 4th task failed due to FetchPartitionError(executor1) + let task_status_4 = mock_failed_task( + task_4, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 2, + map_partition_id: 1, + }, + )), + }, + ); + // This task update should be ignored because the same failure reason is already handled + agg_graph.update_task_status(&executor3, vec![task_status_4], 4, 4)?; + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 3); + + // Finish the other 3 tasks in Stage 2 + for _i in 0..3 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + } + assert_eq!(agg_graph.available_tasks(), 0); + + // Stage 3, the very long delayed 5th task failed due to FetchPartitionError(executor3) + // Although the failure reason is new, but this task should sbe ignored + // Because its map stage's new attempt is finished and this stage's new attempt is running + let task_status_5 = mock_failed_task( + task_5, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor3.id.clone(), + map_stage_id: 2, + map_partition_id: 1, + }, + )), + }, + ); + agg_graph.update_task_status(&executor3, vec![task_status_5], 4, 4)?; + // Stage 3's new attempt is running + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 3); + assert_eq!(agg_graph.available_tasks(), 8); + + // There is one failed stage attempts: Stage 3. Stage 2 does not count to failed attempts + assert_eq!(agg_graph.failed_stage_attempts.len(), 1); + assert_eq!( + agg_graph.failed_stage_attempts.get(&3).cloned(), + Some(HashSet::from([0])) + ); + drain_tasks(&mut agg_graph)?; + assert!(agg_graph.is_successful(), "Failed to complete agg plan"); + // Failed stage attempts are cleaned + assert_eq!(agg_graph.failed_stage_attempts.len(), 0); + + Ok(()) + } + + #[tokio::test] + // This test case covers a race condition in delayed fetch failure handling: + // TaskStatus of input stage's new attempt come together with the parent stage's delayed FetchFailure + async fn test_long_delayed_fetch_failures_race_condition() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let executor3 = mock_executor("executor-id3".to_string()); + let mut agg_graph = test_two_aggregations_plan(8).await; + + agg_graph.revive(); + assert_eq!(agg_graph.stage_count(), 3); + + // Complete the Stage 1 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // Complete the Stage 2, 5 tasks run on executor_2, 3 tasks run on executor_1 + for _i in 0..5 { + if let Some(task) = agg_graph.pop_next_task(&executor2.id)? { + let task_status = mock_completed_task(task, &executor2.id); + agg_graph.update_task_status(&executor2, vec![task_status], 4, 4)?; + } + } + assert_eq!(agg_graph.available_tasks(), 3); + + for _i in 0..3 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + } + assert_eq!(agg_graph.available_tasks(), 0); + + // Run Stage 3 + // 1st task scheduled + let task_1 = agg_graph.pop_next_task(&executor3.id)?.unwrap(); + // 2nd task scheduled + let task_2 = agg_graph.pop_next_task(&executor3.id)?.unwrap(); + + // Stage 3, 1st task failed due to FetchPartitionError(executor2) + let task_status_1 = mock_failed_task( + task_1, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor2.id.clone(), + map_stage_id: 2, + map_partition_id: 0, + }, + )), + }, + ); + agg_graph.update_task_status(&executor3, vec![task_status_1], 4, 4)?; + + // The Running stage is Stage 2 now + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 5); + + // Complete the 5 tasks in Stage 2's new attempts + let mut task_status_vec = vec![]; + for _i in 0..5 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + task_status_vec.push(mock_completed_task(task, &executor1.id)) + } + } + + // Stage 3, 2nd task failed due to FetchPartitionError(executor1) + let task_status_2 = mock_failed_task( + task_2, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 2, + map_partition_id: 1, + }, + )), + }, + ); + task_status_vec.push(task_status_2); + + // TaskStatus of Stage 2 come together with Stage 3 delayed FetchFailure update. + // The successful tasks from Stage 2 would try to succeed the Stage2 and the delayed fetch failure try to reset the TaskInfo + agg_graph.update_task_status(&executor3, task_status_vec, 4, 4)?; + //The Running stage is still Stage 2, 3 new pending tasks added due to FetchPartitionError(executor1) + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 3); + + drain_tasks(&mut agg_graph)?; + assert!(agg_graph.is_successful(), "Failed to complete agg plan"); + + Ok(()) + } + + #[tokio::test] + async fn test_fetch_failures_in_different_stages() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let executor3 = mock_executor("executor-id3".to_string()); + let mut agg_graph = test_two_aggregations_plan(8).await; + + agg_graph.revive(); + assert_eq!(agg_graph.stage_count(), 3); + + // Complete the Stage 1 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // Complete the Stage 2, 5 tasks run on executor_2, 3 tasks run on executor_1 + for _i in 0..5 { + if let Some(task) = agg_graph.pop_next_task(&executor2.id)? { + let task_status = mock_completed_task(task, &executor2.id); + agg_graph.update_task_status(&executor2, vec![task_status], 4, 4)?; + } + } + assert_eq!(agg_graph.available_tasks(), 3); + for _i in 0..3 { + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + } + assert_eq!(agg_graph.available_tasks(), 0); + + // Run Stage 3 + // 1rd task in the Stage 3, failed due to FetchPartitionError(executor1) + if let Some(task1) = agg_graph.pop_next_task(&executor3.id)? { + let task_status1 = mock_failed_task( + task1, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 2, + map_partition_id: 0, + }, + )), + }, + ); + + let _stage_events = + agg_graph.update_task_status(&executor3, vec![task_status1], 4, 4)?; + } + // The Running stage is Stage 2 now + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 2); + assert_eq!(agg_graph.available_tasks(), 3); + + // 1rd task in the Stage 2's new attempt, failed due to FetchPartitionError(executor1) + if let Some(task1) = agg_graph.pop_next_task(&executor3.id)? { + let task_status1 = mock_failed_task( + task1, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 1, + map_partition_id: 0, + }, + )), + }, + ); + let _stage_events = + agg_graph.update_task_status(&executor3, vec![task_status1], 4, 4)?; + } + // The Running stage is Stage 1 now + let running_stage = agg_graph.running_stages(); + assert_eq!(running_stage.len(), 1); + assert_eq!(running_stage[0], 1); + assert_eq!(agg_graph.available_tasks(), 1); + + // There are two failed stage attempts: Stage 2 and Stage 3 + assert_eq!(agg_graph.failed_stage_attempts.len(), 2); + assert_eq!( + agg_graph.failed_stage_attempts.get(&2).cloned(), + Some(HashSet::from([1])) + ); + assert_eq!( + agg_graph.failed_stage_attempts.get(&3).cloned(), + Some(HashSet::from([0])) + ); + + drain_tasks(&mut agg_graph)?; + assert!(agg_graph.is_successful(), "Failed to complete agg plan"); + assert_eq!(agg_graph.failed_stage_attempts.len(), 0); + Ok(()) + } + + #[tokio::test] + async fn test_fetch_failure_with_normal_task_failure() -> Result<()> { + let executor1 = mock_executor("executor-id1".to_string()); + let executor2 = mock_executor("executor-id2".to_string()); + let mut agg_graph = test_aggregation_plan(4).await; + // Call revive to move the leaf Resolved stages to Running + agg_graph.revive(); + + // Complete the Stage 1 + if let Some(task) = agg_graph.pop_next_task(&executor1.id)? { + let task_status = mock_completed_task(task, &executor1.id); + agg_graph.update_task_status(&executor1, vec![task_status], 4, 4)?; + } + + // 1st task in the Stage 2 + let task1 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status1 = mock_completed_task(task1, &executor2.id); + + // 2nd task in the Stage 2, failed due to FetchPartitionError + let task2 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status2 = mock_failed_task( + task2, + FailedTask { + error: "FetchPartitionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::FetchPartitionError( + FetchPartitionError { + executor_id: executor1.id.clone(), + map_stage_id: 1, + map_partition_id: 0, + }, + )), + }, + ); + + // 3rd task in the Stage 2, failed due to ExecutionError + let task3 = agg_graph.pop_next_task(&executor2.id)?.unwrap(); + let task_status3 = mock_failed_task( + task3, + FailedTask { + error: "ExecutionError".to_string(), + retryable: false, + count_to_failures: false, + failed_reason: Some(failed_task::FailedReason::ExecutionError( + ExecutionError {}, + )), + }, + ); + + let stage_events = agg_graph.update_task_status( + &executor2, + vec![task_status1, task_status2, task_status3], + 4, + 4, + )?; + + assert_eq!(stage_events.len(), 1); + assert!(matches!( + stage_events[0], + QueryStageSchedulerEvent::JobRunningFailed(_, _) + )); + + drain_tasks(&mut agg_graph)?; + assert!(!agg_graph.is_successful(), "Expect to fail the agg plan"); + + let failure_reason = format!("{:?}", agg_graph.status); + assert!(failure_reason.contains("Job failed due to stage 2 failed")); + assert!(failure_reason.contains("ExecutionError")); + + Ok(()) + } + + // #[tokio::test] + // async fn test_shuffle_files_should_cleaned_after_fetch_failure() -> Result<()> { + // todo!() + // } + fn drain_tasks(graph: &mut ExecutionGraph) -> Result<()> { let executor = mock_executor("executor-id1".to_string()); while let Some(task) = graph.pop_next_task(&executor.id)? { @@ -1656,6 +2730,34 @@ mod test { ExecutionGraph::new("localhost:50050", "job", "session", plan).unwrap() } + async fn test_two_aggregations_plan(partition: usize) -> ExecutionGraph { + let config = SessionConfig::new().with_target_partitions(partition); + let ctx = Arc::new(SessionContext::with_config(config)); + + let schema = Schema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("name", DataType::Utf8, false), + Field::new("gmv", DataType::UInt64, false), + ]); + + let logical_plan = scan_empty(None, &schema, Some(vec![0, 1, 2])) + .unwrap() + .aggregate(vec![col("id"), col("name")], vec![sum(col("gmv"))]) + .unwrap() + .aggregate(vec![col("id")], vec![count(col("id"))]) + .unwrap() + .build() + .unwrap(); + + let optimized_plan = ctx.optimize(&logical_plan).unwrap(); + + let plan = ctx.create_physical_plan(&optimized_plan).await.unwrap(); + + println!("{}", DisplayableExecutionPlan::new(plan.as_ref()).indent()); + + ExecutionGraph::new("localhost:50050", "job", "session", plan).unwrap() + } + async fn test_coalesce_plan(partition: usize) -> ExecutionGraph { let config = SessionConfig::new().with_target_partitions(partition); let ctx = Arc::new(SessionContext::with_config(config)); @@ -1825,4 +2927,42 @@ mod test { })), } } + + fn mock_failed_task(task: TaskDefinition, failed_task: FailedTask) -> TaskStatus { + let mut partitions: Vec = vec![]; + + let num_partitions = task + .output_partitioning + .map(|p| p.partition_count()) + .unwrap_or(1); + + for partition_id in 0..num_partitions { + partitions.push(protobuf::ShuffleWritePartition { + partition_id: partition_id as u64, + path: format!( + "/{}/{}/{}", + task.partition.job_id, + task.partition.stage_id, + task.partition.partition_id + ), + num_batches: 1, + num_rows: 1, + num_bytes: 1, + }) + } + + // Fail the task + protobuf::TaskStatus { + task_id: task.task_id as u32, + job_id: task.partition.job_id.clone(), + stage_id: task.partition.stage_id as u32, + stage_attempt_num: task.stage_attempt_num as u32, + partition_id: task.partition.partition_id as u32, + launch_time: 0, + start_exec_time: 0, + end_exec_time: 0, + metrics: vec![], + status: Some(task_status::Status::Failed(failed_task)), + } + } } diff --git a/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs b/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs index f7e1b442b..9e62f2a6c 100644 --- a/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs +++ b/ballista/rust/scheduler/src/state/execution_graph/execution_stage.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::convert::TryInto; use std::fmt::{Debug, Formatter}; +use std::iter::FromIterator; use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; @@ -89,6 +90,8 @@ pub(super) struct UnresolvedStage { pub(super) inputs: HashMap, /// `ExecutionPlan` for this stage pub(super) plan: Arc, + /// Record last attempt's failure reasons to avoid duplicate resubmits + pub(super) last_attempt_failure_reasons: HashSet, } /// For a stage, if it has no inputs or all of its input stages are completed, @@ -99,7 +102,7 @@ pub(super) struct ResolvedStage { pub(super) stage_id: usize, /// Stage Attempt number pub(super) stage_attempt_num: usize, - /// Total number of output partitions for this stage. + /// Total number of partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, /// Output partitioning for this stage. @@ -111,6 +114,8 @@ pub(super) struct ResolvedStage { pub(super) inputs: HashMap, /// `ExecutionPlan` for this stage pub(super) plan: Arc, + /// Record last attempt's failure reasons to avoid duplicate resubmits + pub(super) last_attempt_failure_reasons: HashSet, } /// Different from the resolved stage, a running stage will @@ -124,7 +129,7 @@ pub(super) struct RunningStage { pub(super) stage_id: usize, /// Stage Attempt number pub(super) stage_attempt_num: usize, - /// Total number of output partitions for this stage. + /// Total number of partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, /// Output partitioning for this stage. @@ -153,7 +158,7 @@ pub(super) struct SuccessfulStage { pub(super) stage_id: usize, /// Stage Attempt number pub(super) stage_attempt_num: usize, - /// Total number of output partitions for this stage. + /// Total number of partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, /// Output partitioning for this stage. @@ -179,7 +184,7 @@ pub(super) struct FailedStage { pub(super) stage_id: usize, /// Stage Attempt number pub(super) stage_attempt_num: usize, - /// Total number of output partitions for this stage. + /// Total number of partitions for this stage. /// This stage will produce on task for partition. pub(super) partitions: usize, /// Output partitioning for this stage. @@ -236,6 +241,7 @@ impl UnresolvedStage { output_links, inputs, plan, + last_attempt_failure_reasons: Default::default(), } } @@ -246,6 +252,7 @@ impl UnresolvedStage { output_partitioning: Option, output_links: Vec, inputs: HashMap, + last_attempt_failure_reasons: HashSet, ) -> Self { Self { stage_id, @@ -254,6 +261,7 @@ impl UnresolvedStage { output_links, inputs, plan, + last_attempt_failure_reasons, } } @@ -274,6 +282,35 @@ impl UnresolvedStage { Ok(()) } + /// Remove input partitions from an input stage on a given executor. + /// Return the HashSet of removed map partition ids + pub(super) fn remove_input_partitions( + &mut self, + input_stage_id: usize, + _input_partition_id: usize, + executor_id: &str, + ) -> Result> { + if let Some(stage_output) = self.inputs.get_mut(&input_stage_id) { + let mut bad_map_partitions = HashSet::new(); + stage_output + .partition_locations + .iter_mut() + .for_each(|(_partition, locs)| { + locs.iter().for_each(|loc| { + if loc.executor_meta.id == executor_id { + bad_map_partitions.insert(loc.map_partition_id); + } + }); + + locs.retain(|loc| loc.executor_meta.id != executor_id); + }); + stage_output.complete = false; + Ok(bad_map_partitions) + } else { + Err(BallistaError::Internal(format!("Error remove input partition for Stage {}, {} is not a valid child stage ID", self.stage_id, input_stage_id))) + } + } + /// Marks the input stage ID as complete. pub(super) fn complete_input(&mut self, stage_id: usize) { if let Some(input) = self.inputs.get_mut(&stage_id) { @@ -305,6 +342,7 @@ impl UnresolvedStage { self.output_partitioning.clone(), self.output_links.clone(), self.inputs.clone(), + self.last_attempt_failure_reasons.clone(), )) } @@ -335,6 +373,9 @@ impl UnresolvedStage { output_links: stage.output_links.into_iter().map(|l| l as usize).collect(), plan, inputs, + last_attempt_failure_reasons: HashSet::from_iter( + stage.last_attempt_failure_reasons, + ), }) } @@ -358,6 +399,9 @@ impl UnresolvedStage { output_links: stage.output_links.into_iter().map(|l| l as u32).collect(), inputs, plan, + last_attempt_failure_reasons: Vec::from_iter( + stage.last_attempt_failure_reasons, + ), }) } } @@ -386,6 +430,7 @@ impl ResolvedStage { output_partitioning: Option, output_links: Vec, inputs: HashMap, + last_attempt_failure_reasons: HashSet, ) -> Self { let partitions = plan.output_partitioning().partition_count(); @@ -397,6 +442,7 @@ impl ResolvedStage { output_links, inputs, plan, + last_attempt_failure_reasons, } } @@ -424,6 +470,7 @@ impl ResolvedStage { self.output_partitioning.clone(), self.output_links.clone(), self.inputs.clone(), + self.last_attempt_failure_reasons.clone(), ); Ok(unresolved) } @@ -456,6 +503,9 @@ impl ResolvedStage { output_links: stage.output_links.into_iter().map(|l| l as usize).collect(), inputs, plan, + last_attempt_failure_reasons: HashSet::from_iter( + stage.last_attempt_failure_reasons, + ), }) } @@ -480,6 +530,9 @@ impl ResolvedStage { output_links: stage.output_links.into_iter().map(|l| l as u32).collect(), inputs, plan, + last_attempt_failure_reasons: Vec::from_iter( + stage.last_attempt_failure_reasons, + ), }) } } @@ -565,7 +618,7 @@ impl RunningStage { } } - /// /// Change to the resolved state and bump the stage attempt number + /// Change to the resolved state and bump the stage attempt number pub(super) fn to_resolved(&self) -> ResolvedStage { ResolvedStage::new( self.stage_id, @@ -574,11 +627,15 @@ impl RunningStage { self.output_partitioning.clone(), self.output_links.clone(), self.inputs.clone(), + HashSet::new(), ) } /// Change to the unresolved state and bump the stage attempt number - pub(super) fn to_unresolved(&self) -> Result { + pub(super) fn to_unresolved( + &self, + failure_reasons: HashSet, + ) -> Result { let new_plan = crate::planner::rollback_resolved_shuffles(self.plan.clone())?; let unresolved = UnresolvedStage::new_with_inputs( @@ -588,6 +645,7 @@ impl RunningStage { self.output_partitioning.clone(), self.output_links.clone(), self.inputs.clone(), + failure_reasons, ); Ok(unresolved) } @@ -783,20 +841,33 @@ impl RunningStage { reset } - /// Remove input partition from an input stage. - pub(super) fn remove_input_partition( + /// Remove input partitions from an input stage on a given executor. + /// Return the HashSet of removed map partition ids + pub(super) fn remove_input_partitions( &mut self, input_stage_id: usize, - input_partition_id: usize, - ) -> Result<()> { - if let Some(stage_inputs) = self.inputs.get_mut(&input_stage_id) { - stage_inputs.remove_partition(input_partition_id); - stage_inputs.complete = false; + _input_partition_id: usize, + executor_id: &str, + ) -> Result> { + if let Some(stage_output) = self.inputs.get_mut(&input_stage_id) { + let mut bad_map_partitions = HashSet::new(); + stage_output + .partition_locations + .iter_mut() + .for_each(|(_partition, locs)| { + locs.iter().for_each(|loc| { + if loc.executor_meta.id == executor_id { + bad_map_partitions.insert(loc.map_partition_id); + } + }); + + locs.retain(|loc| loc.executor_meta.id != executor_id); + }); + stage_output.complete = false; + Ok(bad_map_partitions) } else { - return Err(BallistaError::Internal(format!("Error remove input partition to stage {}, {} is not a valid child stage ID", self.stage_id, input_stage_id))); + Err(BallistaError::Internal(format!("Error remove input partition for Stage {}, {} is not a valid child stage ID", self.stage_id, input_stage_id))) } - - Ok(()) } } @@ -1158,10 +1229,6 @@ impl StageOutput { } } - pub(super) fn remove_partition(&mut self, partition_id: usize) { - self.partition_locations.remove(&partition_id); - } - pub(super) fn is_complete(&self) -> bool { self.complete } diff --git a/ballista/rust/scheduler/src/state/task_manager.rs b/ballista/rust/scheduler/src/state/task_manager.rs index c3dbe4807..4a7968df4 100644 --- a/ballista/rust/scheduler/src/state/task_manager.rs +++ b/ballista/rust/scheduler/src/state/task_manager.rs @@ -66,10 +66,10 @@ pub struct TaskManager #[derive(Clone)] pub struct UpdatedStages { - pub resolved_stages: Vec, - pub successful_stages: Vec, + pub resolved_stages: HashSet, + pub successful_stages: HashSet, pub failed_stages: HashMap, - pub rollback_running_stages: HashSet, + pub rollback_running_stages: HashMap>, pub resubmit_successful_stages: HashSet, } From ddfb0144ddfb2ba7be81139b19d262fb0f1e0720 Mon Sep 17 00:00:00 2001 From: Wang Date: Sun, 25 Sep 2022 22:00:03 +0800 Subject: [PATCH 3/4] fix fmt --- ballista/rust/core/src/execution_plans/shuffle_reader.rs | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/ballista/rust/core/src/execution_plans/shuffle_reader.rs b/ballista/rust/core/src/execution_plans/shuffle_reader.rs index b75b0c3ca..866cac0a6 100644 --- a/ballista/rust/core/src/execution_plans/shuffle_reader.rs +++ b/ballista/rust/core/src/execution_plans/shuffle_reader.rs @@ -204,13 +204,7 @@ async fn fetch_partition( })?; ballista_client - .fetch_partition( - &metadata.id, - partition_id, - &location.path, - host, - port, - ) + .fetch_partition(&metadata.id, partition_id, &location.path, host, port) .await } From 0e0eecfd60016f2216f9f7db78d1795b9685c46d Mon Sep 17 00:00:00 2001 From: Wang Date: Fri, 30 Sep 2022 11:23:56 +0800 Subject: [PATCH 4/4] Resolve review comments --- ballista/rust/core/proto/ballista.proto | 58 ++++++++++--------- .../scheduler/src/state/execution_graph.rs | 12 ++-- 2 files changed, 36 insertions(+), 34 deletions(-) diff --git a/ballista/rust/core/proto/ballista.proto b/ballista/rust/core/proto/ballista.proto index aa4ec6dfe..e937e2597 100644 --- a/ballista/rust/core/proto/ballista.proto +++ b/ballista/rust/core/proto/ballista.proto @@ -424,7 +424,7 @@ message ExecutionGraph { uint64 output_partitions = 5; repeated PartitionLocation output_locations = 6; string scheduler_id = 7; - uint32 tid_gen = 8; + uint32 task_id_gen = 8; repeated StageAttempts failed_attempts = 9; } @@ -444,47 +444,47 @@ message ExecutionGraphStage { message UnResolvedStage { uint32 stage_id = 1; - uint32 stage_attempt_num = 2; - PhysicalHashRepartition output_partitioning = 3; - repeated uint32 output_links = 4; - repeated GraphStageInput inputs = 5; - bytes plan = 6; + PhysicalHashRepartition output_partitioning = 2; + repeated uint32 output_links = 3; + repeated GraphStageInput inputs = 4; + bytes plan = 5; + uint32 stage_attempt_num = 6; repeated string last_attempt_failure_reasons = 7; } message ResolvedStage { uint32 stage_id = 1; - uint32 stage_attempt_num = 2; - uint32 partitions = 3; - PhysicalHashRepartition output_partitioning = 4; - repeated uint32 output_links = 5; - repeated GraphStageInput inputs = 6; - bytes plan = 7; + uint32 partitions = 2; + PhysicalHashRepartition output_partitioning = 3; + repeated uint32 output_links = 4; + repeated GraphStageInput inputs = 5; + bytes plan = 6; + uint32 stage_attempt_num = 7; repeated string last_attempt_failure_reasons = 8; } message SuccessfulStage { uint32 stage_id = 1; - uint32 stage_attempt_num = 2; - uint32 partitions = 3; - PhysicalHashRepartition output_partitioning = 4; - repeated uint32 output_links = 5; - repeated GraphStageInput inputs = 6; - bytes plan = 7; - repeated TaskInfo task_infos = 8; - repeated OperatorMetricsSet stage_metrics = 9; + uint32 partitions = 2; + PhysicalHashRepartition output_partitioning = 3; + repeated uint32 output_links = 4; + repeated GraphStageInput inputs = 5; + bytes plan = 6; + repeated TaskInfo task_infos = 7; + repeated OperatorMetricsSet stage_metrics = 8; + uint32 stage_attempt_num = 9; } message FailedStage { uint32 stage_id = 1; - uint32 stage_attempt_num = 2; - uint32 partitions = 3; - PhysicalHashRepartition output_partitioning = 4; - repeated uint32 output_links = 5; - bytes plan = 6; - repeated TaskInfo task_infos = 7; - repeated OperatorMetricsSet stage_metrics = 8; - string error_message = 9; + uint32 partitions = 2; + PhysicalHashRepartition output_partitioning = 3; + repeated uint32 output_links = 4; + bytes plan = 5; + repeated TaskInfo task_infos = 6; + repeated OperatorMetricsSet stage_metrics = 7; + string error_message = 8; + uint32 stage_attempt_num = 9; } message TaskInfo { @@ -555,7 +555,9 @@ message FetchPartition { } message PartitionLocation { + // partition_id of the map stage who produces the shuffle. uint32 map_partition_id = 1; + // partition_id of the shuffle, a composition of(job_id + map_stage_id + partition_id). PartitionId partition_id = 2; ExecutorMetadata executor_meta = 3; PartitionStats partition_stats = 4; diff --git a/ballista/rust/scheduler/src/state/execution_graph.rs b/ballista/rust/scheduler/src/state/execution_graph.rs index e371cc8ac..a6c5c7dba 100644 --- a/ballista/rust/scheduler/src/state/execution_graph.rs +++ b/ballista/rust/scheduler/src/state/execution_graph.rs @@ -115,7 +115,7 @@ pub struct ExecutionGraph { /// Locations of this `ExecutionGraph` final output locations output_locations: Vec, /// Task ID generator, generate unique TID in the execution graph - tid_generator: usize, + task_id_gen: usize, /// Failed stage attempts, record the failed stage attempts to limit the retry times. /// Map from Stage ID -> Set failed_stage_attempts: HashMap>, @@ -156,7 +156,7 @@ impl ExecutionGraph { stages, output_partitions, output_locations: vec![], - tid_generator: 0, + task_id_gen: 0, failed_stage_attempts: HashMap::new(), }) } @@ -178,8 +178,8 @@ impl ExecutionGraph { } pub fn next_task_id(&mut self) -> usize { - let new_tid = self.tid_generator; - self.tid_generator += 1; + let new_tid = self.task_id_gen; + self.task_id_gen += 1; new_tid } @@ -1285,7 +1285,7 @@ impl ExecutionGraph { stages, output_partitions: proto.output_partitions as usize, output_locations, - tid_generator: proto.tid_gen as usize, + task_id_gen: proto.task_id_gen as usize, failed_stage_attempts, }) } @@ -1357,7 +1357,7 @@ impl ExecutionGraph { output_partitions: graph.output_partitions as u64, output_locations, scheduler_id: graph.scheduler_id, - tid_gen: graph.tid_generator as u32, + task_id_gen: graph.task_id_gen as u32, failed_attempts, }) }