-
Notifications
You must be signed in to change notification settings - Fork 1.5k
Fix Ballista executing during plan #2428
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -30,9 +30,8 @@ use crate::serde::protobuf::{ | |
ExecuteQueryParams, GetJobStatusParams, GetJobStatusResult, KeyValuePair, | ||
PartitionLocation, | ||
}; | ||
use crate::utils::WrappedStream; | ||
|
||
use datafusion::arrow::datatypes::{Schema, SchemaRef}; | ||
use datafusion::arrow::datatypes::SchemaRef; | ||
use datafusion::error::{DataFusionError, Result}; | ||
use datafusion::logical_plan::LogicalPlan; | ||
use datafusion::physical_plan::expressions::PhysicalSortExpr; | ||
|
@@ -43,12 +42,14 @@ use datafusion::physical_plan::{ | |
use crate::serde::protobuf::execute_query_params::OptionalSessionId; | ||
use crate::serde::{AsLogicalPlan, DefaultLogicalExtensionCodec, LogicalExtensionCodec}; | ||
use async_trait::async_trait; | ||
use datafusion::arrow::error::{ArrowError, Result as ArrowResult}; | ||
use datafusion::arrow::record_batch::RecordBatch; | ||
use datafusion::execution::context::TaskContext; | ||
use futures::future; | ||
use futures::StreamExt; | ||
use datafusion::physical_plan::stream::RecordBatchStreamAdapter; | ||
use futures::{Stream, StreamExt, TryFutureExt, TryStreamExt}; | ||
use log::{error, info}; | ||
|
||
/// This operator sends a logial plan to a Ballista scheduler for execution and | ||
/// This operator sends a logical plan to a Ballista scheduler for execution and | ||
/// polls the scheduler until the query is complete and then fetches the resulting | ||
/// batches directly from the executors that hold the results from the final | ||
/// query stage. | ||
|
@@ -168,15 +169,6 @@ impl<T: 'static + AsLogicalPlan> ExecutionPlan for DistributedQueryExec<T> { | |
) -> Result<SendableRecordBatchStream> { | ||
assert_eq!(0, partition); | ||
|
||
info!("Connecting to Ballista scheduler at {}", self.scheduler_url); | ||
// TODO reuse the scheduler to avoid connecting to the Ballista scheduler again and again | ||
|
||
let mut scheduler = SchedulerGrpcClient::connect(self.scheduler_url.clone()) | ||
.await | ||
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; | ||
|
||
let schema: Schema = self.plan.schema().as_ref().clone().into(); | ||
|
||
let mut buf: Vec<u8> = vec![]; | ||
let plan_message = | ||
T::try_from_logical_plan(&self.plan, self.extension_codec.as_ref()).map_err( | ||
|
@@ -191,88 +183,30 @@ impl<T: 'static + AsLogicalPlan> ExecutionPlan for DistributedQueryExec<T> { | |
DataFusionError::Execution(format!("failed to encode logical plan: {:?}", e)) | ||
})?; | ||
|
||
let query_result = scheduler | ||
.execute_query(ExecuteQueryParams { | ||
query: Some(Query::LogicalPlan(buf)), | ||
settings: self | ||
.config | ||
.settings() | ||
.iter() | ||
.map(|(k, v)| KeyValuePair { | ||
key: k.to_owned(), | ||
value: v.to_owned(), | ||
}) | ||
.collect::<Vec<_>>(), | ||
optional_session_id: Some(OptionalSessionId::SessionId( | ||
self.session_id.clone(), | ||
)), | ||
}) | ||
.await | ||
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? | ||
.into_inner(); | ||
|
||
let response_session_id = query_result.session_id; | ||
assert_eq!( | ||
self.session_id.clone(), | ||
response_session_id, | ||
"Session id inconsistent between Client and Server side in DistributedQueryExec." | ||
); | ||
let query = ExecuteQueryParams { | ||
query: Some(Query::LogicalPlan(buf)), | ||
settings: self | ||
.config | ||
.settings() | ||
.iter() | ||
.map(|(k, v)| KeyValuePair { | ||
key: k.to_owned(), | ||
value: v.to_owned(), | ||
}) | ||
.collect::<Vec<_>>(), | ||
optional_session_id: Some(OptionalSessionId::SessionId( | ||
self.session_id.clone(), | ||
)), | ||
}; | ||
|
||
let job_id = query_result.job_id; | ||
let mut prev_status: Option<job_status::Status> = None; | ||
let stream = futures::stream::once( | ||
execute_query(self.scheduler_url.clone(), self.session_id.clone(), query) | ||
.map_err(|e| ArrowError::ExternalError(Box::new(e))), | ||
) | ||
.try_flatten(); | ||
|
||
loop { | ||
let GetJobStatusResult { status } = scheduler | ||
.get_job_status(GetJobStatusParams { | ||
job_id: job_id.clone(), | ||
}) | ||
.await | ||
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? | ||
.into_inner(); | ||
let status = status.and_then(|s| s.status).ok_or_else(|| { | ||
DataFusionError::Internal("Received empty status message".to_owned()) | ||
})?; | ||
let wait_future = tokio::time::sleep(Duration::from_millis(100)); | ||
let has_status_change = prev_status.map(|x| x != status).unwrap_or(true); | ||
match status { | ||
job_status::Status::Queued(_) => { | ||
if has_status_change { | ||
info!("Job {} still queued...", job_id); | ||
} | ||
wait_future.await; | ||
prev_status = Some(status); | ||
} | ||
job_status::Status::Running(_) => { | ||
if has_status_change { | ||
info!("Job {} is running...", job_id); | ||
} | ||
wait_future.await; | ||
prev_status = Some(status); | ||
} | ||
job_status::Status::Failed(err) => { | ||
let msg = format!("Job {} failed: {}", job_id, err.error); | ||
error!("{}", msg); | ||
break Err(DataFusionError::Execution(msg)); | ||
} | ||
job_status::Status::Completed(completed) => { | ||
let result = future::join_all( | ||
completed | ||
.partition_location | ||
.into_iter() | ||
.map(fetch_partition), | ||
) | ||
.await | ||
.into_iter() | ||
.collect::<Result<Vec<_>>>()?; | ||
|
||
let result = WrappedStream::new( | ||
Box::pin(futures::stream::iter(result).flatten()), | ||
Arc::new(schema), | ||
); | ||
break Ok(Box::pin(result)); | ||
} | ||
}; | ||
} | ||
let schema = self.schema(); | ||
Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream))) | ||
} | ||
|
||
fn fmt_as( | ||
|
@@ -299,6 +233,79 @@ impl<T: 'static + AsLogicalPlan> ExecutionPlan for DistributedQueryExec<T> { | |
} | ||
} | ||
|
||
async fn execute_query( | ||
scheduler_url: String, | ||
session_id: String, | ||
query: ExecuteQueryParams, | ||
) -> Result<impl Stream<Item = ArrowResult<RecordBatch>> + Send> { | ||
info!("Connecting to Ballista scheduler at {}", scheduler_url); | ||
// TODO reuse the scheduler to avoid connecting to the Ballista scheduler again and again | ||
|
||
let mut scheduler = SchedulerGrpcClient::connect(scheduler_url.clone()) | ||
.await | ||
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?; | ||
|
||
let query_result = scheduler | ||
.execute_query(query) | ||
.await | ||
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? | ||
.into_inner(); | ||
|
||
assert_eq!( | ||
session_id, query_result.session_id, | ||
"Session id inconsistent between Client and Server side in DistributedQueryExec." | ||
); | ||
|
||
let job_id = query_result.job_id; | ||
let mut prev_status: Option<job_status::Status> = None; | ||
|
||
loop { | ||
let GetJobStatusResult { status } = scheduler | ||
.get_job_status(GetJobStatusParams { | ||
job_id: job_id.clone(), | ||
}) | ||
.await | ||
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))? | ||
.into_inner(); | ||
let status = status.and_then(|s| s.status).ok_or_else(|| { | ||
DataFusionError::Internal("Received empty status message".to_owned()) | ||
})?; | ||
let wait_future = tokio::time::sleep(Duration::from_millis(100)); | ||
let has_status_change = prev_status.map(|x| x != status).unwrap_or(true); | ||
match status { | ||
job_status::Status::Queued(_) => { | ||
if has_status_change { | ||
info!("Job {} still queued...", job_id); | ||
} | ||
wait_future.await; | ||
prev_status = Some(status); | ||
} | ||
job_status::Status::Running(_) => { | ||
if has_status_change { | ||
info!("Job {} is running...", job_id); | ||
} | ||
wait_future.await; | ||
prev_status = Some(status); | ||
} | ||
job_status::Status::Failed(err) => { | ||
let msg = format!("Job {} failed: {}", job_id, err.error); | ||
error!("{}", msg); | ||
break Err(DataFusionError::Execution(msg)); | ||
} | ||
job_status::Status::Completed(completed) => { | ||
let streams = completed.partition_location.into_iter().map(|p| { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is a breaking change, previously it would buffer up all records from all partitions and then yield them in partition order, now it will stream them potentially interleaving results from different partitions. I'm fairly certain this is fine, but want to draw attention to this |
||
let f = fetch_partition(p) | ||
.map_err(|e| ArrowError::ExternalError(Box::new(e))); | ||
|
||
futures::stream::once(f).try_flatten() | ||
}); | ||
|
||
break Ok(futures::stream::iter(streams).flatten()); | ||
} | ||
}; | ||
} | ||
} | ||
|
||
async fn fetch_partition( | ||
location: PartitionLocation, | ||
) -> Result<SendableRecordBatchStream> { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I recommend reviewing this without whitespace
https://github.com/apache/arrow-datafusion/pull/2428/files?w=1