Skip to content

Commit 8f4078d

Browse files
authored
ShuffleReaderExec now supports multiple locations per partition (#541)
* ShuffleReaderExec now supports multiple locations per partition * Remove TODO * avoid clone
1 parent ad70a1e commit 8f4078d

File tree

8 files changed

+130
-88
lines changed

8 files changed

+130
-88
lines changed

ballista/rust/client/src/context.rs

+5-34
Original file line numberDiff line numberDiff line change
@@ -29,21 +29,18 @@ use ballista_core::serde::protobuf::{
2929
execute_query_params::Query, job_status, ExecuteQueryParams, GetJobStatusParams,
3030
GetJobStatusResult,
3131
};
32+
use ballista_core::utils::WrappedStream;
3233
use ballista_core::{
3334
client::BallistaClient, datasource::DfTableAdapter, utils::create_datafusion_context,
3435
};
3536

3637
use datafusion::arrow::datatypes::Schema;
37-
use datafusion::arrow::datatypes::SchemaRef;
38-
use datafusion::arrow::error::Result as ArrowResult;
39-
use datafusion::arrow::record_batch::RecordBatch;
4038
use datafusion::catalog::TableReference;
4139
use datafusion::error::{DataFusionError, Result};
4240
use datafusion::logical_plan::LogicalPlan;
4341
use datafusion::physical_plan::csv::CsvReadOptions;
4442
use datafusion::{dataframe::DataFrame, physical_plan::RecordBatchStream};
4543
use futures::future;
46-
use futures::Stream;
4744
use futures::StreamExt;
4845
use log::{error, info};
4946

@@ -74,32 +71,6 @@ impl BallistaContextState {
7471
}
7572
}
7673

77-
struct WrappedStream {
78-
stream: Pin<Box<dyn Stream<Item = ArrowResult<RecordBatch>> + Send + Sync>>,
79-
schema: SchemaRef,
80-
}
81-
82-
impl RecordBatchStream for WrappedStream {
83-
fn schema(&self) -> SchemaRef {
84-
self.schema.clone()
85-
}
86-
}
87-
88-
impl Stream for WrappedStream {
89-
type Item = ArrowResult<RecordBatch>;
90-
91-
fn poll_next(
92-
mut self: Pin<&mut Self>,
93-
cx: &mut std::task::Context<'_>,
94-
) -> std::task::Poll<Option<Self::Item>> {
95-
self.stream.poll_next_unpin(cx)
96-
}
97-
98-
fn size_hint(&self) -> (usize, Option<usize>) {
99-
self.stream.size_hint()
100-
}
101-
}
102-
10374
#[allow(dead_code)]
10475

10576
pub struct BallistaContext {
@@ -287,10 +258,10 @@ impl BallistaContext {
287258
.into_iter()
288259
.collect::<Result<Vec<_>>>()?;
289260

290-
let result = WrappedStream {
291-
stream: Box::pin(futures::stream::iter(result).flatten()),
292-
schema: Arc::new(schema),
293-
};
261+
let result = WrappedStream::new(
262+
Box::pin(futures::stream::iter(result).flatten()),
263+
Arc::new(schema),
264+
);
294265
break Ok(Box::pin(result));
295266
}
296267
};

ballista/rust/core/proto/ballista.proto

+6-1
Original file line numberDiff line numberDiff line change
@@ -489,10 +489,15 @@ message HashAggregateExecNode {
489489
}
490490

491491
message ShuffleReaderExecNode {
492-
repeated PartitionLocation partition_location = 1;
492+
repeated ShuffleReaderPartition partition = 1;
493493
Schema schema = 2;
494494
}
495495

496+
message ShuffleReaderPartition {
497+
// each partition of a shuffle read can read data from multiple locations
498+
repeated PartitionLocation location = 1;
499+
}
500+
496501
message GlobalLimitExecNode {
497502
PhysicalPlanNode input = 1;
498503
uint32 limit = 2;

ballista/rust/core/src/execution_plans/shuffle_reader.rs

+57-37
Original file line numberDiff line numberDiff line change
@@ -15,42 +15,43 @@
1515
// specific language governing permissions and limitations
1616
// under the License.
1717

18+
use std::fmt::Formatter;
1819
use std::sync::Arc;
1920
use std::{any::Any, pin::Pin};
2021

2122
use crate::client::BallistaClient;
2223
use crate::memory_stream::MemoryStream;
2324
use crate::serde::scheduler::PartitionLocation;
2425

26+
use crate::utils::WrappedStream;
2527
use async_trait::async_trait;
2628
use datafusion::arrow::datatypes::SchemaRef;
29+
use datafusion::arrow::error::Result as ArrowResult;
30+
use datafusion::arrow::record_batch::RecordBatch;
2731
use datafusion::physical_plan::{DisplayFormatType, ExecutionPlan, Partitioning};
2832
use datafusion::{
2933
error::{DataFusionError, Result},
3034
physical_plan::RecordBatchStream,
3135
};
36+
use futures::{future, Stream, StreamExt};
3237
use log::info;
33-
use std::fmt::Formatter;
3438

35-
/// ShuffleReaderExec reads partitions that have already been materialized by an executor.
39+
/// ShuffleReaderExec reads partitions that have already been materialized by a query stage
40+
/// being executed by an executor
3641
#[derive(Debug, Clone)]
3742
pub struct ShuffleReaderExec {
38-
// The query stage that is responsible for producing the shuffle partitions that
39-
// this operator will read
40-
pub(crate) partition_location: Vec<PartitionLocation>,
43+
/// Each partition of a shuffle can read data from multiple locations
44+
pub(crate) partition: Vec<Vec<PartitionLocation>>,
4145
pub(crate) schema: SchemaRef,
4246
}
4347

4448
impl ShuffleReaderExec {
4549
/// Create a new ShuffleReaderExec
4650
pub fn try_new(
47-
partition_meta: Vec<PartitionLocation>,
51+
partition: Vec<Vec<PartitionLocation>>,
4852
schema: SchemaRef,
4953
) -> Result<Self> {
50-
Ok(Self {
51-
partition_location: partition_meta,
52-
schema,
53-
})
54+
Ok(Self { partition, schema })
5455
}
5556
}
5657

@@ -65,7 +66,7 @@ impl ExecutionPlan for ShuffleReaderExec {
6566
}
6667

6768
fn output_partitioning(&self) -> Partitioning {
68-
Partitioning::UnknownPartitioning(self.partition_location.len())
69+
Partitioning::UnknownPartitioning(self.partition.len())
6970
}
7071

7172
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
@@ -86,23 +87,18 @@ impl ExecutionPlan for ShuffleReaderExec {
8687
partition: usize,
8788
) -> Result<Pin<Box<dyn RecordBatchStream + Send + Sync>>> {
8889
info!("ShuffleReaderExec::execute({})", partition);
89-
let partition_location = &self.partition_location[partition];
90-
91-
let mut client = BallistaClient::try_new(
92-
&partition_location.executor_meta.host,
93-
partition_location.executor_meta.port,
94-
)
95-
.await
96-
.map_err(|e| DataFusionError::Execution(format!("Ballista Error: {:?}", e)))?;
9790

98-
client
99-
.fetch_partition(
100-
&partition_location.partition_id.job_id,
101-
partition_location.partition_id.stage_id,
102-
partition,
103-
)
91+
let partition_locations = &self.partition[partition];
92+
let result = future::join_all(partition_locations.iter().map(fetch_partition))
10493
.await
105-
.map_err(|e| DataFusionError::Execution(format!("Ballista Error: {:?}", e)))
94+
.into_iter()
95+
.collect::<Result<Vec<_>>>()?;
96+
97+
let result = WrappedStream::new(
98+
Box::pin(futures::stream::iter(result).flatten()),
99+
Arc::new(self.schema.as_ref().clone()),
100+
);
101+
Ok(Box::pin(result))
106102
}
107103

108104
fn fmt_as(
@@ -113,22 +109,46 @@ impl ExecutionPlan for ShuffleReaderExec {
113109
match t {
114110
DisplayFormatType::Default => {
115111
let loc_str = self
116-
.partition_location
112+
.partition
117113
.iter()
118-
.map(|l| {
119-
format!(
120-
"[executor={} part={}:{}:{} stats={:?}]",
121-
l.executor_meta.id,
122-
l.partition_id.job_id,
123-
l.partition_id.stage_id,
124-
l.partition_id.partition_id,
125-
l.partition_stats
126-
)
114+
.map(|x| {
115+
x.iter()
116+
.map(|l| {
117+
format!(
118+
"[executor={} part={}:{}:{} stats={:?}]",
119+
l.executor_meta.id,
120+
l.partition_id.job_id,
121+
l.partition_id.stage_id,
122+
l.partition_id.partition_id,
123+
l.partition_stats
124+
)
125+
})
126+
.collect::<Vec<String>>()
127+
.join(",")
127128
})
128129
.collect::<Vec<String>>()
129-
.join(",");
130+
.join("\n");
130131
write!(f, "ShuffleReaderExec: partition_locations={}", loc_str)
131132
}
132133
}
133134
}
134135
}
136+
137+
async fn fetch_partition(
138+
location: &PartitionLocation,
139+
) -> Result<Pin<Box<dyn RecordBatchStream + Send + Sync>>> {
140+
let metadata = &location.executor_meta;
141+
let partition_id = &location.partition_id;
142+
let mut ballista_client =
143+
BallistaClient::try_new(metadata.host.as_str(), metadata.port as u16)
144+
.await
145+
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?;
146+
Ok(ballista_client
147+
.fetch_partition(
148+
&partition_id.job_id,
149+
partition_id.stage_id as usize,
150+
partition_id.partition_id as usize,
151+
)
152+
.await
153+
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?)
154+
}

ballista/rust/core/src/serde/physical_plan/from_proto.rs

+9-3
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ use crate::error::BallistaError;
2525
use crate::execution_plans::{ShuffleReaderExec, UnresolvedShuffleExec};
2626
use crate::serde::protobuf::repartition_exec_node::PartitionMethod;
2727
use crate::serde::protobuf::LogicalExprNode;
28+
use crate::serde::protobuf::ShuffleReaderPartition;
2829
use crate::serde::scheduler::PartitionLocation;
2930
use crate::serde::{proto_error, protobuf};
3031
use crate::{convert_box_required, convert_required};
@@ -327,10 +328,15 @@ impl TryInto<Arc<dyn ExecutionPlan>> for &protobuf::PhysicalPlanNode {
327328
}
328329
PhysicalPlanType::ShuffleReader(shuffle_reader) => {
329330
let schema = Arc::new(convert_required!(shuffle_reader.schema)?);
330-
let partition_location: Vec<PartitionLocation> = shuffle_reader
331-
.partition_location
331+
let partition_location: Vec<Vec<PartitionLocation>> = shuffle_reader
332+
.partition
332333
.iter()
333-
.map(|p| p.clone().try_into())
334+
.map(|p| {
335+
p.location
336+
.iter()
337+
.map(|l| l.clone().try_into())
338+
.collect::<Result<Vec<_>, _>>()
339+
})
334340
.collect::<Result<Vec<_>, BallistaError>>()?;
335341
let shuffle_reader =
336342
ShuffleReaderExec::try_new(partition_location, schema)?;

ballista/rust/core/src/serde/physical_plan/to_proto.rs

+11-7
Original file line numberDiff line numberDiff line change
@@ -57,6 +57,7 @@ use protobuf::physical_plan_node::PhysicalPlanType;
5757

5858
use crate::execution_plans::{ShuffleReaderExec, UnresolvedShuffleExec};
5959
use crate::serde::protobuf::repartition_exec_node::PartitionMethod;
60+
use crate::serde::scheduler::PartitionLocation;
6061
use crate::serde::{protobuf, BallistaError};
6162
use datafusion::physical_plan::functions::{BuiltinScalarFunction, ScalarFunctionExpr};
6263
use datafusion::physical_plan::merge::MergeExec;
@@ -268,16 +269,19 @@ impl TryInto<protobuf::PhysicalPlanNode> for Arc<dyn ExecutionPlan> {
268269
)),
269270
})
270271
} else if let Some(exec) = plan.downcast_ref::<ShuffleReaderExec>() {
271-
let partition_location = exec
272-
.partition_location
273-
.iter()
274-
.map(|l| l.clone().try_into())
275-
.collect::<Result<_, _>>()?;
276-
272+
let mut partition = vec![];
273+
for location in &exec.partition {
274+
partition.push(protobuf::ShuffleReaderPartition {
275+
location: location
276+
.iter()
277+
.map(|l| l.clone().try_into())
278+
.collect::<Result<Vec<_>, _>>()?,
279+
});
280+
}
277281
Ok(protobuf::PhysicalPlanNode {
278282
physical_plan_type: Some(PhysicalPlanType::ShuffleReader(
279283
protobuf::ShuffleReaderExecNode {
280-
partition_location,
284+
partition,
281285
schema: Some(exec.schema().as_ref().into()),
282286
},
283287
)),

ballista/rust/core/src/utils.rs

+38-2
Original file line numberDiff line numberDiff line change
@@ -27,11 +27,12 @@ use crate::execution_plans::{QueryStageExec, UnresolvedShuffleExec};
2727
use crate::memory_stream::MemoryStream;
2828
use crate::serde::scheduler::PartitionStats;
2929

30+
use datafusion::arrow::error::Result as ArrowResult;
3031
use datafusion::arrow::{
3132
array::{
3233
ArrayBuilder, ArrayRef, StructArray, StructBuilder, UInt64Array, UInt64Builder,
3334
},
34-
datatypes::{DataType, Field},
35+
datatypes::{DataType, Field, SchemaRef},
3536
ipc::reader::FileReader,
3637
ipc::writer::FileWriter,
3738
record_batch::RecordBatch,
@@ -54,7 +55,7 @@ use datafusion::physical_plan::sort::SortExec;
5455
use datafusion::physical_plan::{
5556
AggregateExpr, ExecutionPlan, PhysicalExpr, RecordBatchStream,
5657
};
57-
use futures::StreamExt;
58+
use futures::{future, Stream, StreamExt};
5859

5960
/// Stream data to disk in Arrow IPC format
6061
@@ -234,3 +235,38 @@ pub fn create_datafusion_context() -> ExecutionContext {
234235
.with_physical_optimizer_rules(rules);
235236
ExecutionContext::with_config(config)
236237
}
238+
239+
pub struct WrappedStream {
240+
stream: Pin<Box<dyn Stream<Item = ArrowResult<RecordBatch>> + Send + Sync>>,
241+
schema: SchemaRef,
242+
}
243+
244+
impl WrappedStream {
245+
pub fn new(
246+
stream: Pin<Box<dyn Stream<Item = ArrowResult<RecordBatch>> + Send + Sync>>,
247+
schema: SchemaRef,
248+
) -> Self {
249+
Self { stream, schema }
250+
}
251+
}
252+
253+
impl RecordBatchStream for WrappedStream {
254+
fn schema(&self) -> SchemaRef {
255+
self.schema.clone()
256+
}
257+
}
258+
259+
impl Stream for WrappedStream {
260+
type Item = ArrowResult<RecordBatch>;
261+
262+
fn poll_next(
263+
mut self: Pin<&mut Self>,
264+
cx: &mut std::task::Context<'_>,
265+
) -> std::task::Poll<Option<Self::Item>> {
266+
self.stream.poll_next_unpin(cx)
267+
}
268+
269+
fn size_hint(&self) -> (usize, Option<usize>) {
270+
self.stream.size_hint()
271+
}
272+
}

ballista/rust/scheduler/src/planner.rs

+1-1
Original file line numberDiff line numberDiff line change
@@ -186,7 +186,7 @@ impl DistributedPlanner {
186186

187187
pub fn remove_unresolved_shuffles(
188188
stage: &dyn ExecutionPlan,
189-
partition_locations: &HashMap<usize, Vec<PartitionLocation>>,
189+
partition_locations: &HashMap<usize, Vec<Vec<PartitionLocation>>>,
190190
) -> Result<Arc<dyn ExecutionPlan>> {
191191
let mut new_children: Vec<Arc<dyn ExecutionPlan>> = vec![];
192192
for child in stage.children() {

0 commit comments

Comments
 (0)