Skip to content

Commit ed5746d

Browse files
authored
Ballista shuffle is finally working as intended, providing scalable distributed joins (#750)
1 parent 30693df commit ed5746d

File tree

17 files changed

+507
-141
lines changed

17 files changed

+507
-141
lines changed

ballista/rust/client/src/context.rs

+1
Original file line numberDiff line numberDiff line change
@@ -223,6 +223,7 @@ impl BallistaContext {
223223
&partition_id.job_id,
224224
partition_id.stage_id as usize,
225225
partition_id.partition_id as usize,
226+
&location.path,
226227
)
227228
.await
228229
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?)

ballista/rust/core/proto/ballista.proto

+14-2
Original file line numberDiff line numberDiff line change
@@ -544,7 +544,8 @@ message PhysicalNegativeNode {
544544
message UnresolvedShuffleExecNode {
545545
uint32 stage_id = 1;
546546
Schema schema = 2;
547-
uint32 partition_count = 3;
547+
uint32 input_partition_count = 3;
548+
uint32 output_partition_count = 4;
548549
}
549550

550551
message FilterExecNode {
@@ -700,7 +701,7 @@ message Action {
700701

701702
oneof ActionType {
702703
// Fetch a partition from an executor
703-
PartitionId fetch_partition = 3;
704+
FetchPartition fetch_partition = 3;
704705
}
705706

706707
// configuration settings
@@ -714,6 +715,15 @@ message ExecutePartition {
714715
PhysicalPlanNode plan = 4;
715716
// The task could need to read partitions from other executors
716717
repeated PartitionLocation partition_location = 5;
718+
// Output partition for shuffle writer
719+
PhysicalHashRepartition output_partitioning = 6;
720+
}
721+
722+
message FetchPartition {
723+
string job_id = 1;
724+
uint32 stage_id = 2;
725+
uint32 partition_id = 3;
726+
string path = 4;
717727
}
718728

719729
// Mapping from partition id to executor id
@@ -809,6 +819,8 @@ message PollWorkParams {
809819
message TaskDefinition {
810820
PartitionId task_id = 1;
811821
PhysicalPlanNode plan = 2;
822+
// Output partition for shuffle writer
823+
PhysicalHashRepartition output_partitioning = 3;
812824
}
813825

814826
message PollWorkResult {

ballista/rust/core/src/client.rs

+7-2
Original file line numberDiff line numberDiff line change
@@ -81,9 +81,14 @@ impl BallistaClient {
8181
job_id: &str,
8282
stage_id: usize,
8383
partition_id: usize,
84+
path: &str,
8485
) -> Result<SendableRecordBatchStream> {
85-
let action =
86-
Action::FetchPartition(PartitionId::new(job_id, stage_id, partition_id));
86+
let action = Action::FetchPartition {
87+
job_id: job_id.to_string(),
88+
stage_id,
89+
partition_id,
90+
path: path.to_owned(),
91+
};
8792
self.execute_action(&action).await
8893
}
8994

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

+20-15
Original file line numberDiff line numberDiff line change
@@ -76,6 +76,8 @@ impl ExecutionPlan for ShuffleReaderExec {
7676
}
7777

7878
fn output_partitioning(&self) -> Partitioning {
79+
// TODO partitioning may be known and could be populated here
80+
// see https://github.com/apache/arrow-datafusion/issues/758
7981
Partitioning::UnknownPartitioning(self.partition.len())
8082
}
8183

@@ -123,24 +125,26 @@ impl ExecutionPlan for ShuffleReaderExec {
123125
let loc_str = self
124126
.partition
125127
.iter()
126-
.map(|x| {
127-
x.iter()
128-
.map(|l| {
129-
format!(
130-
"[executor={} part={}:{}:{} stats={}]",
131-
l.executor_meta.id,
132-
l.partition_id.job_id,
133-
l.partition_id.stage_id,
134-
l.partition_id.partition_id,
135-
l.partition_stats
136-
)
137-
})
138-
.collect::<Vec<String>>()
139-
.join(",")
128+
.enumerate()
129+
.map(|(partition_id, locations)| {
130+
format!(
131+
"[partition={} paths={}]",
132+
partition_id,
133+
locations
134+
.iter()
135+
.map(|l| l.path.clone())
136+
.collect::<Vec<String>>()
137+
.join(",")
138+
)
140139
})
141140
.collect::<Vec<String>>()
142141
.join(", ");
143-
write!(f, "ShuffleReaderExec: partition_locations={}", loc_str)
142+
write!(
143+
f,
144+
"ShuffleReaderExec: partition_locations({})={}",
145+
self.partition.len(),
146+
loc_str
147+
)
144148
}
145149
}
146150
}
@@ -166,6 +170,7 @@ async fn fetch_partition(
166170
&partition_id.job_id,
167171
partition_id.stage_id as usize,
168172
partition_id.partition_id as usize,
173+
&location.path,
169174
)
170175
.await
171176
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?)

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

+11
Original file line numberDiff line numberDiff line change
@@ -141,6 +141,7 @@ impl ShuffleWriterExec {
141141

142142
match &self.shuffle_output_partitioning {
143143
None => {
144+
let start = Instant::now();
144145
path.push(&format!("{}", input_partition));
145146
std::fs::create_dir_all(&path)?;
146147
path.push("data.arrow");
@@ -156,6 +157,14 @@ impl ShuffleWriterExec {
156157
.await
157158
.map_err(|e| DataFusionError::Execution(format!("{:?}", e)))?;
158159

160+
self.metrics
161+
.input_rows
162+
.add(stats.num_rows.unwrap_or(0) as usize);
163+
self.metrics
164+
.output_rows
165+
.add(stats.num_rows.unwrap_or(0) as usize);
166+
self.metrics.write_time.add_elapsed(start);
167+
159168
info!(
160169
"Executed partition {} in {} seconds. Statistics: {}",
161170
input_partition,
@@ -227,6 +236,8 @@ impl ShuffleWriterExec {
227236
RecordBatch::try_new(input_batch.schema(), columns)?;
228237

229238
// write non-empty batch out
239+
240+
//TODO optimize so we don't write or fetch empty partitions
230241
//if output_batch.num_rows() > 0 {
231242
let start = Instant::now();
232243
match &mut writers[output_partition] {

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

+15-4
Original file line numberDiff line numberDiff line change
@@ -43,17 +43,26 @@ pub struct UnresolvedShuffleExec {
4343
// The schema this node will have once it is replaced with a ShuffleReaderExec
4444
pub schema: SchemaRef,
4545

46+
// The number of shuffle writer partition tasks that will produce the partitions
47+
pub input_partition_count: usize,
48+
4649
// The partition count this node will have once it is replaced with a ShuffleReaderExec
47-
pub partition_count: usize,
50+
pub output_partition_count: usize,
4851
}
4952

5053
impl UnresolvedShuffleExec {
5154
/// Create a new UnresolvedShuffleExec
52-
pub fn new(stage_id: usize, schema: SchemaRef, partition_count: usize) -> Self {
55+
pub fn new(
56+
stage_id: usize,
57+
schema: SchemaRef,
58+
input_partition_count: usize,
59+
output_partition_count: usize,
60+
) -> Self {
5361
Self {
5462
stage_id,
5563
schema,
56-
partition_count,
64+
input_partition_count,
65+
output_partition_count,
5766
}
5867
}
5968
}
@@ -69,7 +78,9 @@ impl ExecutionPlan for UnresolvedShuffleExec {
6978
}
7079

7180
fn output_partitioning(&self) -> Partitioning {
72-
Partitioning::UnknownPartitioning(self.partition_count)
81+
//TODO the output partition is known and should be populated here!
82+
// see https://github.com/apache/arrow-datafusion/issues/758
83+
Partitioning::UnknownPartitioning(self.output_partition_count)
7384
}
7485

7586
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {

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

+27-16
Original file line numberDiff line numberDiff line change
@@ -376,21 +376,9 @@ impl TryInto<Arc<dyn ExecutionPlan>> for &protobuf::PhysicalPlanNode {
376376
let input: Arc<dyn ExecutionPlan> =
377377
convert_box_required!(shuffle_writer.input)?;
378378

379-
let output_partitioning = match &shuffle_writer.output_partitioning {
380-
Some(hash_part) => {
381-
let expr = hash_part
382-
.hash_expr
383-
.iter()
384-
.map(|e| e.try_into())
385-
.collect::<Result<Vec<Arc<dyn PhysicalExpr>>, _>>()?;
386-
387-
Some(Partitioning::Hash(
388-
expr,
389-
hash_part.partition_count.try_into().unwrap(),
390-
))
391-
}
392-
None => None,
393-
};
379+
let output_partitioning = parse_protobuf_hash_partitioning(
380+
shuffle_writer.output_partitioning.as_ref(),
381+
)?;
394382

395383
Ok(Arc::new(ShuffleWriterExec::try_new(
396384
shuffle_writer.job_id.clone(),
@@ -466,7 +454,10 @@ impl TryInto<Arc<dyn ExecutionPlan>> for &protobuf::PhysicalPlanNode {
466454
Ok(Arc::new(UnresolvedShuffleExec {
467455
stage_id: unresolved_shuffle.stage_id as usize,
468456
schema,
469-
partition_count: unresolved_shuffle.partition_count as usize,
457+
input_partition_count: unresolved_shuffle.input_partition_count
458+
as usize,
459+
output_partition_count: unresolved_shuffle.output_partition_count
460+
as usize,
470461
}))
471462
}
472463
}
@@ -680,3 +671,23 @@ impl TryFrom<&protobuf::physical_window_expr_node::WindowFunction> for WindowFun
680671
}
681672
}
682673
}
674+
675+
pub fn parse_protobuf_hash_partitioning(
676+
partitioning: Option<&protobuf::PhysicalHashRepartition>,
677+
) -> Result<Option<Partitioning>, BallistaError> {
678+
match partitioning {
679+
Some(hash_part) => {
680+
let expr = hash_part
681+
.hash_expr
682+
.iter()
683+
.map(|e| e.try_into())
684+
.collect::<Result<Vec<Arc<dyn PhysicalExpr>>, _>>()?;
685+
686+
Ok(Some(Partitioning::Hash(
687+
expr,
688+
hash_part.partition_count.try_into().unwrap(),
689+
)))
690+
}
691+
None => Ok(None),
692+
}
693+
}

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

+2-1
Original file line numberDiff line numberDiff line change
@@ -397,7 +397,8 @@ impl TryInto<protobuf::PhysicalPlanNode> for Arc<dyn ExecutionPlan> {
397397
protobuf::UnresolvedShuffleExecNode {
398398
stage_id: exec.stage_id as u32,
399399
schema: Some(exec.schema().as_ref().into()),
400-
partition_count: exec.partition_count as u32,
400+
input_partition_count: exec.input_partition_count as u32,
401+
output_partition_count: exec.output_partition_count as u32,
401402
},
402403
)),
403404
})

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

+6-3
Original file line numberDiff line numberDiff line change
@@ -32,9 +32,12 @@ impl TryInto<Action> for protobuf::Action {
3232

3333
fn try_into(self) -> Result<Action, Self::Error> {
3434
match self.action_type {
35-
Some(ActionType::FetchPartition(partition)) => {
36-
Ok(Action::FetchPartition(partition.try_into()?))
37-
}
35+
Some(ActionType::FetchPartition(fetch)) => Ok(Action::FetchPartition {
36+
job_id: fetch.job_id,
37+
stage_id: fetch.stage_id as usize,
38+
partition_id: fetch.partition_id as usize,
39+
path: fetch.path,
40+
}),
3841
_ => Err(BallistaError::General(
3942
"scheduler::from_proto(Action) invalid or missing action".to_owned(),
4043
)),

ballista/rust/core/src/serde/scheduler/mod.rs

+11-1
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ use datafusion::arrow::array::{
2323
use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef};
2424
use datafusion::logical_plan::LogicalPlan;
2525
use datafusion::physical_plan::ExecutionPlan;
26+
use datafusion::physical_plan::Partitioning;
2627
use serde::Serialize;
2728
use uuid::Uuid;
2829

@@ -36,7 +37,12 @@ pub mod to_proto;
3637
#[derive(Debug, Clone)]
3738
pub enum Action {
3839
/// Collect a shuffle partition
39-
FetchPartition(PartitionId),
40+
FetchPartition {
41+
job_id: String,
42+
stage_id: usize,
43+
partition_id: usize,
44+
path: String,
45+
},
4046
}
4147

4248
/// Unique identifier for the output partition of an operator.
@@ -223,6 +229,8 @@ pub struct ExecutePartition {
223229
pub plan: Arc<dyn ExecutionPlan>,
224230
/// Location of shuffle partitions that this query stage may depend on
225231
pub shuffle_locations: HashMap<PartitionId, ExecutorMeta>,
232+
/// Output partitioning for shuffle writes
233+
pub output_partitioning: Option<Partitioning>,
226234
}
227235

228236
impl ExecutePartition {
@@ -232,13 +240,15 @@ impl ExecutePartition {
232240
partition_id: Vec<usize>,
233241
plan: Arc<dyn ExecutionPlan>,
234242
shuffle_locations: HashMap<PartitionId, ExecutorMeta>,
243+
output_partitioning: Option<Partitioning>,
235244
) -> Self {
236245
Self {
237246
job_id,
238247
stage_id,
239248
partition_id,
240249
plan,
241250
shuffle_locations,
251+
output_partitioning,
242252
}
243253
}
244254

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

+39-2
Original file line numberDiff line numberDiff line change
@@ -23,14 +23,25 @@ use crate::serde::protobuf::action::ActionType;
2323
use crate::serde::scheduler::{
2424
Action, ExecutePartition, PartitionId, PartitionLocation, PartitionStats,
2525
};
26+
use datafusion::physical_plan::Partitioning;
2627

2728
impl TryInto<protobuf::Action> for Action {
2829
type Error = BallistaError;
2930

3031
fn try_into(self) -> Result<protobuf::Action, Self::Error> {
3132
match self {
32-
Action::FetchPartition(partition_id) => Ok(protobuf::Action {
33-
action_type: Some(ActionType::FetchPartition(partition_id.into())),
33+
Action::FetchPartition {
34+
job_id,
35+
stage_id,
36+
partition_id,
37+
path,
38+
} => Ok(protobuf::Action {
39+
action_type: Some(ActionType::FetchPartition(protobuf::FetchPartition {
40+
job_id,
41+
stage_id: stage_id as u32,
42+
partition_id: partition_id as u32,
43+
path,
44+
})),
3445
settings: vec![],
3546
}),
3647
}
@@ -47,6 +58,9 @@ impl TryInto<protobuf::ExecutePartition> for ExecutePartition {
4758
partition_id: self.partition_id.iter().map(|n| *n as u32).collect(),
4859
plan: Some(self.plan.try_into()?),
4960
partition_location: vec![],
61+
output_partitioning: hash_partitioning_to_proto(
62+
self.output_partitioning.as_ref(),
63+
)?,
5064
})
5165
}
5266
}
@@ -87,3 +101,26 @@ impl Into<protobuf::PartitionStats> for PartitionStats {
87101
}
88102
}
89103
}
104+
105+
pub fn hash_partitioning_to_proto(
106+
output_partitioning: Option<&Partitioning>,
107+
) -> Result<Option<protobuf::PhysicalHashRepartition>, BallistaError> {
108+
match output_partitioning {
109+
Some(Partitioning::Hash(exprs, partition_count)) => {
110+
Ok(Some(protobuf::PhysicalHashRepartition {
111+
hash_expr: exprs
112+
.iter()
113+
.map(|expr| expr.clone().try_into())
114+
.collect::<Result<Vec<_>, BallistaError>>()?,
115+
partition_count: *partition_count as u64,
116+
}))
117+
}
118+
None => Ok(None),
119+
other => {
120+
return Err(BallistaError::General(format!(
121+
"scheduler::to_proto() invalid partitioning for ExecutePartition: {:?}",
122+
other
123+
)))
124+
}
125+
}
126+
}

0 commit comments

Comments
 (0)