-
Notifications
You must be signed in to change notification settings - Fork 174
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FEAT] [New Executor] [2/N] daft-execution crate + proof-of-concept c…
…ompute ops and partition reference + metadata model for new executor. (#2340) This PR adds the `daft-execution` subcrate containing a set of proof-of-concept local compute ops and the partition reference + metadata model for the new executor. Partial metadata machinery isn't yet implemented since no task scheduler or exchange op has required it yet. ## TODOs - [ ] Add unit tests for compute ops. - [ ] Add doc strings for abstractions.
- Loading branch information
1 parent
021b103
commit 22ef02b
Showing
21 changed files
with
1,743 additions
and
15 deletions.
There are no files selected for viewing
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,30 @@ | ||
[dependencies] | ||
async-trait = {workspace = true} | ||
common-error = {path = "../common/error", default-features = false} | ||
daft-core = {path = "../daft-core", default-features = false} | ||
daft-dsl = {path = "../daft-dsl", default-features = false} | ||
daft-io = {path = "../daft-io", default-features = false} | ||
daft-micropartition = {path = "../daft-micropartition", default-features = false} | ||
daft-plan = {path = "../daft-plan", default-features = false} | ||
daft-scan = {path = "../daft-scan", default-features = false} | ||
futures = {workspace = true} | ||
itertools = {workspace = true} | ||
log = {workspace = true} | ||
pyo3 = {workspace = true, optional = true} | ||
rand = {workspace = true} | ||
rayon = {workspace = true} | ||
snafu = {workspace = true} | ||
sysinfo = {workspace = true} | ||
tokio = {workspace = true} | ||
|
||
[dev-dependencies] | ||
rstest = {workspace = true} | ||
|
||
[features] | ||
default = ["python"] | ||
python = ["dep:pyo3", "common-error/python", "daft-core/python", "daft-dsl/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python", "daft-scan/python"] | ||
|
||
[package] | ||
edition = {workspace = true} | ||
name = "daft-execution" | ||
version = {workspace = true} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,39 @@ | ||
#![feature(impl_trait_in_assoc_type)] | ||
#![feature(let_chains)] | ||
#![feature(assert_matches)] | ||
// TODO(Clark): Remove this once stage planner, partial metadata, etc. are implemented. | ||
#![allow(dead_code)] | ||
#![allow(unused)] | ||
|
||
mod ops; | ||
mod partition; | ||
|
||
use common_error::DaftError; | ||
use snafu::Snafu; | ||
|
||
#[cfg(feature = "python")] | ||
use pyo3::prelude::*; | ||
|
||
#[derive(Debug, Snafu)] | ||
pub enum Error { | ||
#[snafu(display("Error joining spawned task: {}", source))] | ||
JoinError { source: tokio::task::JoinError }, | ||
#[snafu(display( | ||
"Sender of OneShot Channel Dropped before sending data over: {}", | ||
source | ||
))] | ||
OneShotRecvError { | ||
source: tokio::sync::oneshot::error::RecvError, | ||
}, | ||
} | ||
|
||
impl From<Error> for DaftError { | ||
fn from(err: Error) -> DaftError { | ||
DaftError::External(err.into()) | ||
} | ||
} | ||
|
||
#[cfg(feature = "python")] | ||
pub fn register_modules(_py: Python, parent: &PyModule) -> PyResult<()> { | ||
Ok(()) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,49 @@ | ||
use std::sync::Arc; | ||
|
||
use common_error::DaftResult; | ||
use daft_dsl::ExprRef; | ||
use daft_micropartition::MicroPartition; | ||
use daft_plan::ResourceRequest; | ||
|
||
use crate::partition::partition_ref::PartitionMetadata; | ||
|
||
use super::PartitionTaskOp; | ||
|
||
/// Filter task op, applying a predicate to its input. | ||
#[derive(Debug)] | ||
pub struct FilterOp { | ||
predicate: Vec<ExprRef>, | ||
resource_request: ResourceRequest, | ||
} | ||
|
||
impl FilterOp { | ||
pub fn new(predicate: Vec<ExprRef>) -> Self { | ||
Self { | ||
predicate, | ||
resource_request: ResourceRequest::default_cpu(), | ||
} | ||
} | ||
} | ||
|
||
impl PartitionTaskOp for FilterOp { | ||
type Input = MicroPartition; | ||
|
||
fn execute(&self, inputs: &[Arc<MicroPartition>]) -> DaftResult<Vec<Arc<MicroPartition>>> { | ||
assert_eq!(inputs.len(), 1); | ||
let input = inputs.iter().next().unwrap(); | ||
let out = input.filter(self.predicate.as_slice())?; | ||
Ok(vec![Arc::new(out)]) | ||
} | ||
|
||
fn resource_request(&self) -> &ResourceRequest { | ||
&self.resource_request | ||
} | ||
|
||
fn partial_metadata_from_input_metadata(&self, _: &[PartitionMetadata]) -> PartitionMetadata { | ||
todo!() | ||
} | ||
|
||
fn name(&self) -> &str { | ||
"FilterOp" | ||
} | ||
} |
Oops, something went wrong.