[Ballista] Streaming style push-based shuffle and All-at-once stage scheduling in Ballista#1842
[Ballista] Streaming style push-based shuffle and All-at-once stage scheduling in Ballista#1842mingmwang wants to merge 1 commit intoapache:masterfrom
Conversation
…cheduling in Ballista
| // re-plan the input execution plan and create All-at-once query stages. | ||
| // Now we just simply depends on the the stage count to decide whether to create All-at-once or normal stages. | ||
| // In future, we can have more sophisticated way to decide which way to go. | ||
| if stages.len() > 1 && stages.len() <= 4 { |
There was a problem hiding this comment.
If I understand the original design correctly, the "all-at-once" plan will only get scheduled when there are sufficient task slots available to run the entire plan. So should this be a function of the total number of partitions?
There was a problem hiding this comment.
If I understand the original design correctly, the "all-at-once" plan will only get scheduled when there are sufficient task slots available to run the entire plan. So should this be a function of the total number of partitions?
Yes, you are right. But currently the scheduler server doesn't have a clear view of how many task slots available. So here I just add simple check on the stage count. After @yahoNanJing refactor the scheduler state and keep more cpu/task info into the memory state, we can add more sophisticated check logic.
|
|
||
| // let schema = &self.schema; | ||
| // let rx = self.batch_receiver.lock().unwrap().pop().unwrap(); | ||
| // let join_handle = tokio::task::spawn(async move {}); | ||
| // Ok(RecordBatchReceiverStream::create(schema, rx, join_handle)) |
There was a problem hiding this comment.
| // let schema = &self.schema; | |
| // let rx = self.batch_receiver.lock().unwrap().pop().unwrap(); | |
| // let join_handle = tokio::task::spawn(async move {}); | |
| // Ok(RecordBatchReceiverStream::create(schema, rx, join_handle)) |
| info!("planning query stages for job {}", job_id); | ||
| let (modified_plan, mut stages) = self | ||
| .plan_query_stages_internal(job_id, execution_plan.clone()) |
There was a problem hiding this comment.
I think this block is only used in the else branch below when all at once mode is disabled?
|
marking as draft (so it is easer to see what PRs are waiting for review) |
|
Closing this PR since it has not been updated in a long time. Feel free to re-open if this is still being worked on. |
Which issue does this PR close?
Closes #1805.
Rationale for this change
Add a new streaming style push based shuffle implementation.
What changes are included in this PR?
Are there any user-facing changes?
No