Skip to content

Commit a7f1384

Browse files
Merge pull request #17 from coralogix/more-logging
Catch panics in task execution so jobs do not hang
2 parents d2199fb + 408f83c commit a7f1384

File tree

3 files changed

+33
-10
lines changed

3 files changed

+33
-10
lines changed

ballista/rust/executor/src/executor_server.rs

+26-9
Original file line numberDiff line numberDiff line change
@@ -16,9 +16,11 @@
1616
// under the License.
1717

1818
use ballista_core::BALLISTA_VERSION;
19+
use futures::FutureExt;
1920
use std::collections::HashMap;
2021
use std::convert::TryInto;
2122
use std::ops::Deref;
23+
use std::panic::AssertUnwindSafe;
2224
use std::sync::Arc;
2325
use std::time::{Duration, SystemTime, UNIX_EPOCH};
2426
use tokio::sync::{mpsc, RwLock};
@@ -52,6 +54,7 @@ use tokio::task::JoinHandle;
5254

5355
use crate::as_task_status;
5456
use crate::cpu_bound_executor::DedicatedExecutor;
57+
use crate::execution_loop::any_to_string;
5558
use crate::executor::Executor;
5659
use crate::shutdown::ShutdownNotifier;
5760

@@ -197,8 +200,6 @@ struct ExecutorEnv {
197200
tx_stop: mpsc::Sender<bool>,
198201
}
199202

200-
unsafe impl Sync for ExecutorEnv {}
201-
202203
impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> ExecutorServer<T, U> {
203204
fn new(
204205
scheduler_to_register: SchedulerGrpcClient<Channel>,
@@ -350,17 +351,26 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> ExecutorServer<T,
350351
plan,
351352
)?;
352353

353-
let execution_result = self
354-
.executor
355-
.execute_shuffle_write(
354+
let execution_result =
355+
match AssertUnwindSafe(self.executor.execute_shuffle_write(
356356
task_id.job_id.clone(),
357357
task_id.stage_id as usize,
358358
task_id.partition_id as usize,
359359
shuffle_writer_plan.clone(),
360360
task_context,
361361
shuffle_output_partitioning,
362-
)
363-
.await;
362+
))
363+
.catch_unwind()
364+
.await
365+
{
366+
Ok(Ok(r)) => Ok(r),
367+
Ok(Err(r)) => Err(r),
368+
Err(r) => {
369+
error!("Error executing task: {:?}", any_to_string(&r));
370+
Err(BallistaError::Internal(format!("{:#?}", any_to_string(&r))))
371+
}
372+
};
373+
364374
info!("Done with task {}", task_id_log);
365375
debug!("Statistics: {:?}", execution_result);
366376

@@ -606,13 +616,20 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> ExecutorGrpc
606616
} = request.into_inner();
607617
let task_sender = self.executor_env.tx_task.clone();
608618
for task in tasks {
609-
task_sender
619+
if let Err(e) = task_sender
610620
.send(CuratorTaskDefinition {
611621
scheduler_id: scheduler_id.clone(),
612622
task,
613623
})
614624
.await
615-
.unwrap();
625+
{
626+
let msg = format!(
627+
"Error launching tasks, failed to send to task channel: {:?}",
628+
e
629+
);
630+
error!("{}", msg);
631+
return Err(Status::internal(msg));
632+
}
616633
}
617634
Ok(Response::new(LaunchTaskResult { success: true }))
618635
}

ballista/rust/scheduler/src/state/execution_graph.rs

+6
Original file line numberDiff line numberDiff line change
@@ -647,24 +647,30 @@ impl ExecutionGraph {
647647
for event in events {
648648
match event {
649649
StageEvent::StageResolved(stage_id) => {
650+
info!("stage {}/{} resolved", self.job_id, stage_id);
650651
self.resolve_stage(stage_id)?;
651652
has_resolved = true;
652653
}
653654
StageEvent::StageCompleted(stage_id) => {
655+
info!("stage {}/{} completed", self.job_id, stage_id);
654656
self.complete_stage(stage_id);
655657
}
656658
StageEvent::StageFailed(stage_id, err_msg, failed_at) => {
659+
error!("stage {}/{} failed: {}", self.job_id, stage_id, err_msg);
657660
job_err_msg = format!("{}{}\n", job_err_msg, &err_msg);
658661
failed_at_time = failed_at;
659662
self.fail_stage(stage_id, err_msg);
660663
}
661664
StageEvent::RollBackRunningStage(stage_id) => {
665+
warn!("running stage {}/{} rolled back", self.job_id, stage_id);
662666
self.rollback_running_stage(stage_id)?;
663667
}
664668
StageEvent::RollBackResolvedStage(stage_id) => {
669+
warn!("running stage {}/{} rolled back", self.job_id, stage_id);
665670
self.rollback_resolved_stage(stage_id)?;
666671
}
667672
StageEvent::ReRunCompletedStage(stage_id) => {
673+
info!("re-running completed stage {}/{}", self.job_id, stage_id);
668674
self.rerun_completed_stage(stage_id);
669675
}
670676
}

ballista/rust/scheduler/src/state/task_manager.rs

+1-1
Original file line numberDiff line numberDiff line change
@@ -151,7 +151,7 @@ impl<T: 'static + AsLogicalPlan, U: 'static + AsExecutionPlan> TaskManager<T, U>
151151
let mut events: Vec<QueryStageSchedulerEvent> = vec![];
152152
for (job_id, statuses) in job_updates {
153153
let num_tasks = statuses.len();
154-
debug!("Updating {} tasks in job {}", num_tasks, job_id);
154+
info!("Updating {} tasks in job {}", num_tasks, job_id);
155155

156156
let graph = self.get_active_execution_graph(&job_id).await;
157157
let job_event = if let Some(graph) = graph {

0 commit comments

Comments
 (0)