Skip to content
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

[CHORE] Refactor local hash joins + pipeline connections #2719

Merged
merged 6 commits into from
Aug 29, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 1 addition & 3 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

4 changes: 1 addition & 3 deletions src/daft-local-execution/Cargo.toml
Original file line number Diff line number Diff line change
@@ -1,6 +1,4 @@
[dependencies]
async-stream = {workspace = true}
async-trait = {workspace = true}
common-daft-config = {path = "../common/daft-config", default-features = false}
common-display = {path = "../common/display", default-features = false}
common-error = {path = "../common/error", default-features = false}
Expand All @@ -15,7 +13,6 @@ daft-parquet = {path = "../daft-parquet", default-features = false}
daft-physical-plan = {path = "../daft-physical-plan", default-features = false}
daft-plan = {path = "../daft-plan", default-features = false}
daft-scan = {path = "../daft-scan", default-features = false}
daft-stats = {path = "../daft-stats", default-features = false}
daft-table = {path = "../daft-table", default-features = false}
futures = {workspace = true}
lazy_static = {workspace = true}
Expand All @@ -24,6 +21,7 @@ num-format = "0.4.4"
pyo3 = {workspace = true, optional = true}
snafu = {workspace = true}
tokio = {workspace = true}
tokio-stream = {workspace = true}
tracing = {workspace = true}

[features]
Expand Down
148 changes: 65 additions & 83 deletions src/daft-local-execution/src/channel.rs
Original file line number Diff line number Diff line change
@@ -1,119 +1,118 @@
use std::sync::Arc;

use daft_micropartition::MicroPartition;
use crate::{
pipeline::PipelineResultType,
runtime_stats::{CountingReceiver, CountingSender, RuntimeStatsContext},
};

pub type SingleSender = tokio::sync::mpsc::Sender<Arc<MicroPartition>>;
pub type SingleReceiver = tokio::sync::mpsc::Receiver<Arc<MicroPartition>>;
pub type Sender<T> = tokio::sync::mpsc::Sender<T>;
pub type Receiver<T> = tokio::sync::mpsc::Receiver<T>;

pub fn create_single_channel(buffer_size: usize) -> (SingleSender, SingleReceiver) {
pub fn create_channel<T>(buffer_size: usize) -> (Sender<T>, Receiver<T>) {
tokio::sync::mpsc::channel(buffer_size)
}

pub fn create_channel(buffer_size: usize, in_order: bool) -> (MultiSender, MultiReceiver) {
if in_order {
let (senders, receivers) = (0..buffer_size).map(|_| create_single_channel(1)).unzip();
let sender = MultiSender::InOrder(InOrderSender::new(senders));
let receiver = MultiReceiver::InOrder(InOrderReceiver::new(receivers));
(sender, receiver)
} else {
let (sender, receiver) = create_single_channel(buffer_size);
let sender = MultiSender::OutOfOrder(OutOfOrderSender::new(sender));
let receiver = MultiReceiver::OutOfOrder(OutOfOrderReceiver::new(receiver));
(sender, receiver)
}
}

pub enum MultiSender {
InOrder(InOrderSender),
OutOfOrder(OutOfOrderSender),
pub struct PipelineChannel {
sender: PipelineSender,
receiver: PipelineReceiver,
}

impl MultiSender {
pub fn get_next_sender(&mut self) -> SingleSender {
match self {
Self::InOrder(sender) => sender.get_next_sender(),
Self::OutOfOrder(sender) => sender.get_sender(),
impl PipelineChannel {
pub fn new(buffer_size: usize, in_order: bool) -> Self {
match in_order {
true => {
let (senders, receivers) = (0..buffer_size).map(|_| create_channel(1)).unzip();
let sender = PipelineSender::InOrder(RoundRobinSender::new(senders));
let receiver = PipelineReceiver::InOrder(RoundRobinReceiver::new(receivers));
Self { sender, receiver }
}
false => {
let (sender, receiver) = create_channel(buffer_size);
let sender = PipelineSender::OutOfOrder(sender);
let receiver = PipelineReceiver::OutOfOrder(receiver);
Self { sender, receiver }
}
}
}

pub fn buffer_size(&self) -> usize {
match self {
Self::InOrder(sender) => sender.senders.len(),
Self::OutOfOrder(sender) => sender.sender.max_capacity(),
fn get_next_sender(&mut self) -> Sender<PipelineResultType> {
match &mut self.sender {
PipelineSender::InOrder(rr) => rr.get_next_sender(),
PipelineSender::OutOfOrder(sender) => sender.clone(),
}
}

pub fn in_order(&self) -> bool {
match self {
Self::InOrder(_) => true,
Self::OutOfOrder(_) => false,
}
pub(crate) fn get_next_sender_with_stats(
&mut self,
rt: &Arc<RuntimeStatsContext>,
) -> CountingSender {
CountingSender::new(self.get_next_sender(), rt.clone())
}

pub fn get_receiver(self) -> PipelineReceiver {
self.receiver
}

pub(crate) fn get_receiver_with_stats(self, rt: &Arc<RuntimeStatsContext>) -> CountingReceiver {
CountingReceiver::new(self.get_receiver(), rt.clone())
}
}
pub struct InOrderSender {
senders: Vec<SingleSender>,

pub enum PipelineSender {
InOrder(RoundRobinSender<PipelineResultType>),
OutOfOrder(Sender<PipelineResultType>),
}

pub struct RoundRobinSender<T> {
senders: Vec<Sender<T>>,
curr_sender_idx: usize,
}

impl InOrderSender {
pub fn new(senders: Vec<SingleSender>) -> Self {
impl<T> RoundRobinSender<T> {
pub fn new(senders: Vec<Sender<T>>) -> Self {
Self {
senders,
curr_sender_idx: 0,
}
}

pub fn get_next_sender(&mut self) -> SingleSender {
pub fn get_next_sender(&mut self) -> Sender<T> {
let next_idx = self.curr_sender_idx;
self.curr_sender_idx = (next_idx + 1) % self.senders.len();
self.senders[next_idx].clone()
}
}

pub struct OutOfOrderSender {
sender: SingleSender,
pub enum PipelineReceiver {
InOrder(RoundRobinReceiver<PipelineResultType>),
OutOfOrder(Receiver<PipelineResultType>),
}

impl OutOfOrderSender {
pub fn new(sender: SingleSender) -> Self {
Self { sender }
}

pub fn get_sender(&self) -> SingleSender {
self.sender.clone()
}
}

pub enum MultiReceiver {
InOrder(InOrderReceiver),
OutOfOrder(OutOfOrderReceiver),
}

impl MultiReceiver {
pub async fn recv(&mut self) -> Option<Arc<MicroPartition>> {
impl PipelineReceiver {
pub async fn recv(&mut self) -> Option<PipelineResultType> {
match self {
Self::InOrder(receiver) => receiver.recv().await,
Self::OutOfOrder(receiver) => receiver.recv().await,
PipelineReceiver::InOrder(rr) => rr.recv().await,
PipelineReceiver::OutOfOrder(r) => r.recv().await,
}
}
}

pub struct InOrderReceiver {
receivers: Vec<SingleReceiver>,
pub struct RoundRobinReceiver<T> {
receivers: Vec<Receiver<T>>,
curr_receiver_idx: usize,
is_done: bool,
}

impl InOrderReceiver {
pub fn new(receivers: Vec<SingleReceiver>) -> Self {
impl<T> RoundRobinReceiver<T> {
pub fn new(receivers: Vec<Receiver<T>>) -> Self {
Self {
receivers,
curr_receiver_idx: 0,
is_done: false,
}
}

pub async fn recv(&mut self) -> Option<Arc<MicroPartition>> {
pub async fn recv(&mut self) -> Option<T> {
if self.is_done {
return None;
}
Expand All @@ -128,20 +127,3 @@ impl InOrderReceiver {
None
}
}

pub struct OutOfOrderReceiver {
receiver: SingleReceiver,
}

impl OutOfOrderReceiver {
pub fn new(receiver: SingleReceiver) -> Self {
Self { receiver }
}

pub async fn recv(&mut self) -> Option<Arc<MicroPartition>> {
if let Some(val) = self.receiver.recv().await {
return Some(val);
}
None
}
}
20 changes: 15 additions & 5 deletions src/daft-local-execution/src/intermediate_ops/aggregate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,13 @@ use std::sync::Arc;

use common_error::DaftResult;
use daft_dsl::ExprRef;
use daft_micropartition::MicroPartition;
use tracing::instrument;

use super::intermediate_op::IntermediateOperator;
use crate::pipeline::PipelineResultType;

use super::intermediate_op::{
IntermediateOperator, IntermediateOperatorResult, IntermediateOperatorState,
};

pub struct AggregateOperator {
agg_exprs: Vec<ExprRef>,
Expand All @@ -23,9 +26,16 @@ impl AggregateOperator {

impl IntermediateOperator for AggregateOperator {
#[instrument(skip_all, name = "AggregateOperator::execute")]
fn execute(&self, input: &Arc<MicroPartition>) -> DaftResult<Arc<MicroPartition>> {
let out = input.agg(&self.agg_exprs, &self.group_by)?;
Ok(Arc::new(out))
fn execute(
&self,
_idx: usize,
input: &PipelineResultType,
_state: Option<&mut Box<dyn IntermediateOperatorState>>,
) -> DaftResult<IntermediateOperatorResult> {
let out = input.as_data().agg(&self.agg_exprs, &self.group_by)?;
Ok(IntermediateOperatorResult::NeedMoreInput(Some(Arc::new(
out,
))))
}

fn name(&self) -> &'static str {
Expand Down
20 changes: 15 additions & 5 deletions src/daft-local-execution/src/intermediate_ops/filter.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,13 @@ use std::sync::Arc;

use common_error::DaftResult;
use daft_dsl::ExprRef;
use daft_micropartition::MicroPartition;
use tracing::instrument;

use super::intermediate_op::IntermediateOperator;
use crate::pipeline::PipelineResultType;

use super::intermediate_op::{
IntermediateOperator, IntermediateOperatorResult, IntermediateOperatorState,
};

pub struct FilterOperator {
predicate: ExprRef,
Expand All @@ -19,9 +22,16 @@ impl FilterOperator {

impl IntermediateOperator for FilterOperator {
#[instrument(skip_all, name = "FilterOperator::execute")]
fn execute(&self, input: &Arc<MicroPartition>) -> DaftResult<Arc<MicroPartition>> {
let out = input.filter(&[self.predicate.clone()])?;
Ok(Arc::new(out))
fn execute(
&self,
_idx: usize,
input: &PipelineResultType,
_state: Option<&mut Box<dyn IntermediateOperatorState>>,
) -> DaftResult<IntermediateOperatorResult> {
let out = input.as_data().filter(&[self.predicate.clone()])?;
Ok(IntermediateOperatorResult::NeedMoreInput(Some(Arc::new(
out,
))))
}

fn name(&self) -> &'static str {
Expand Down
Loading
Loading