From d2d887c44b5bc12f5cd727134714697c11277131 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Mon, 9 Oct 2023 16:50:35 -0700 Subject: [PATCH 01/14] draft: feat: compile and execute partitioned query The main change here is having the `sparrow-execution` tests compile from a logical plan, created within a session, and execute things. Changes Include - Fix naming `field_ref` vs. `fieldref` (matches Python) - Tweak pipeline scheduling so that only transforms are grouped together. - Tweak pipelines so that each may output to multiple consumers - Introduces a `SourceTasks` so we can poll all inputs on a single tokio task. - Fixes to how the scheduler detects "completion" (empty queue isn't sufficient). TODO - [x] Actually execute the compiled query - [x] Connect to sources - [x] Move compilation / execution into session. - [x] Look at compiling out `tracing::trace` level statements. - [x] Look at running the query in the sessions tokio runtime - [ ] Move logical expr building to session (out of sparrow-logical) - [ ] Replace `ScalarValue` with arrows scalar? Literal too? - [ ] Look at suspending workers when queue is empty / waking on push - [ ] Look at a trait for logical functions (type-checking, compilation) --- Cargo.lock | 32 +- Cargo.toml | 2 +- crates/sparrow-arrow/Cargo.toml | 7 +- crates/sparrow-backend/Cargo.toml | 2 +- crates/sparrow-backend/src/compile.rs | 7 +- .../src/logical_to_physical.rs | 174 ++++++--- .../sparrow-backend/src/pipeline_schedule.rs | 69 ++-- ...sts__logical_to_physical_arithmetic-2.snap | 15 +- ...sts__logical_to_physical_arithmetic-3.snap | 2 +- ...tests__logical_to_physical_arithmetic.snap | 15 +- crates/sparrow-batch/src/batch.rs | 17 + crates/sparrow-compiler/Cargo.toml | 1 - crates/sparrow-execution/Cargo.toml | 14 +- crates/sparrow-execution/src/error.rs | 21 ++ crates/sparrow-execution/src/lib.rs | 225 +++++++++++- crates/sparrow-execution/src/source_tasks.rs | 101 ++++++ crates/sparrow-execution/src/tests.rs | 342 +++++------------- .../src/write_channel_pipeline.rs | 82 +++++ crates/sparrow-expressions/src/error.rs | 7 +- crates/sparrow-expressions/src/evaluators.rs | 7 +- .../evaluators/{field_ref.rs => fieldref.rs} | 5 +- crates/sparrow-expressions/src/lib.rs | 2 +- crates/sparrow-interfaces/Cargo.toml | 1 - crates/sparrow-interfaces/src/source.rs | 30 +- crates/sparrow-logical/src/error.rs | 10 +- crates/sparrow-logical/src/expr.rs | 40 +- crates/sparrow-logical/src/typecheck.rs | 34 +- crates/sparrow-merge/Cargo.toml | 6 - crates/sparrow-physical/Cargo.toml | 3 +- crates/sparrow-physical/src/plan.rs | 21 ++ crates/sparrow-physical/src/step.rs | 51 ++- crates/sparrow-scheduler/Cargo.toml | 4 +- crates/sparrow-scheduler/src/input_handles.rs | 87 +++++ crates/sparrow-scheduler/src/lib.rs | 5 +- crates/sparrow-scheduler/src/pending.rs | 122 +++++++ crates/sparrow-scheduler/src/queue.rs | 4 +- crates/sparrow-scheduler/src/sink.rs | 37 -- crates/sparrow-scheduler/src/task.rs | 36 +- crates/sparrow-scheduler/src/worker.rs | 36 +- crates/sparrow-scheduler/src/worker_pool.rs | 154 +++++--- crates/sparrow-session/Cargo.toml | 6 +- crates/sparrow-session/src/execution.rs | 27 +- crates/sparrow-session/src/lib.rs | 1 + crates/sparrow-session/src/partitioned.rs | 2 + .../src/partitioned/session.rs | 124 +++++++ crates/sparrow-session/src/session.rs | 17 +- crates/sparrow-sources/Cargo.toml | 4 +- crates/sparrow-sources/src/in_memory.rs | 8 +- crates/sparrow-transforms/src/project.rs | 4 +- crates/sparrow-transforms/src/select.rs | 4 +- .../src/transform_pipeline.rs | 38 +- python/Cargo.lock | 46 +++ 52 files changed, 1555 insertions(+), 556 deletions(-) create mode 100644 crates/sparrow-execution/src/error.rs create mode 100644 crates/sparrow-execution/src/source_tasks.rs create mode 100644 crates/sparrow-execution/src/write_channel_pipeline.rs rename crates/sparrow-expressions/src/evaluators/{field_ref.rs => fieldref.rs} (94%) create mode 100644 crates/sparrow-scheduler/src/input_handles.rs create mode 100644 crates/sparrow-scheduler/src/pending.rs delete mode 100644 crates/sparrow-scheduler/src/sink.rs create mode 100644 crates/sparrow-session/src/partitioned.rs create mode 100644 crates/sparrow-session/src/partitioned/session.rs diff --git a/Cargo.lock b/Cargo.lock index 9bc0cfa18..6932cd87d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4252,7 +4252,6 @@ dependencies = [ "num", "proptest", "serde", - "static_init", "tracing", ] @@ -4271,11 +4270,11 @@ dependencies = [ "itertools 0.11.0", "smallvec", "sparrow-arrow", - "sparrow-core", "sparrow-expressions", "sparrow-logical", "sparrow-physical", "static_init", + "tracing", "uuid 1.4.1", ] @@ -4358,7 +4357,6 @@ dependencies = [ "sparrow-arrow", "sparrow-core", "sparrow-instructions", - "sparrow-merge", "sparrow-sources", "sparrow-syntax", "static_init", @@ -4393,18 +4391,22 @@ dependencies = [ "derive_more", "error-stack", "futures", + "hashbrown 0.14.0", "index_vec", + "itertools 0.11.0", "parking_lot 0.12.1", - "sparrow-arrow", + "sparrow-backend", "sparrow-batch", - "sparrow-core", "sparrow-interfaces", + "sparrow-logical", "sparrow-physical", "sparrow-scheduler", + "sparrow-session", "sparrow-sources", "sparrow-testing", "sparrow-transforms", "tokio", + "tracing", "uuid 1.4.1", ] @@ -4486,7 +4488,6 @@ dependencies = [ "error-stack", "futures", "sparrow-batch", - "sparrow-core", ] [[package]] @@ -4596,18 +4597,12 @@ dependencies = [ "arrow-ord", "arrow-schema", "arrow-select", - "async-broadcast", - "async-stream", "bit-set", - "derive_more", - "error-stack", - "futures", "itertools 0.11.0", "proptest", "smallvec", "sparrow-arrow", "sparrow-core", - "tracing", ] [[package]] @@ -4621,6 +4616,7 @@ dependencies = [ "serde_yaml", "sparrow-arrow", "strum_macros 0.25.2", + "tracing", "uuid 1.4.1", ] @@ -4735,11 +4731,13 @@ dependencies = [ "core_affinity", "derive_more", "error-stack", + "hashbrown 0.14.0", "index_vec", "itertools 0.11.0", "loom", + "parking_lot 0.12.1", "serde", - "sparrow-arrow", + "smallvec", "sparrow-batch", "tracing", "work-queue", @@ -4758,9 +4756,13 @@ dependencies = [ "itertools 0.11.0", "smallvec", "sparrow-api", + "sparrow-backend", "sparrow-compiler", + "sparrow-execution", + "sparrow-expressions", "sparrow-instructions", - "sparrow-merge", + "sparrow-interfaces", + "sparrow-logical", "sparrow-runtime", "sparrow-sources", "sparrow-syntax", @@ -4779,11 +4781,9 @@ dependencies = [ "arrow-schema", "async-broadcast", "async-stream", - "derive_more", "error-stack", "futures", "sparrow-batch", - "sparrow-core", "sparrow-interfaces", "sparrow-merge", "tracing", diff --git a/Cargo.toml b/Cargo.toml index baed629a3..92a749fef 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -129,7 +129,7 @@ tonic = "0.9.2" tonic-build = { version = "0.9.2", features = ["prost"] } tonic-health = "0.9.2" tonic-reflection = "0.9.2" -tracing = "0.1.37" +tracing = { version = "0.1.37", features = ["release_max_level_debug"] } tracing-error = "0.2.0" tracing-opentelemetry = "0.19.0" tracing-serde = "0.1.3" diff --git a/crates/sparrow-arrow/Cargo.toml b/crates/sparrow-arrow/Cargo.toml index 1b213d6a1..157ceb298 100644 --- a/crates/sparrow-arrow/Cargo.toml +++ b/crates/sparrow-arrow/Cargo.toml @@ -34,7 +34,6 @@ itertools.workspace = true num.workspace = true proptest = { workspace = true, optional = true } serde.workspace = true -static_init.workspace = true tracing.workspace = true [dev-dependencies] @@ -43,4 +42,8 @@ insta.workspace = true proptest.workspace = true [lib] -doctest = false \ No newline at end of file +doctest = false + +[package.metadata.cargo-machete] +# Optional package enabled by `testing` +ignored = ["proptest"] \ No newline at end of file diff --git a/crates/sparrow-backend/Cargo.toml b/crates/sparrow-backend/Cargo.toml index 2ea14773d..77d59af5c 100644 --- a/crates/sparrow-backend/Cargo.toml +++ b/crates/sparrow-backend/Cargo.toml @@ -20,12 +20,12 @@ index_vec.workspace = true itertools.workspace = true smallvec.workspace = true sparrow-arrow = { path = "../sparrow-arrow" } -sparrow-core = { path = "../sparrow-core" } sparrow-expressions = { path = "../sparrow-expressions" } sparrow-logical = { path = "../sparrow-logical" } sparrow-physical = { path = "../sparrow-physical" } uuid.workspace = true static_init.workspace = true +tracing.workspace = true [dev-dependencies] insta.workspace = true diff --git a/crates/sparrow-backend/src/compile.rs b/crates/sparrow-backend/src/compile.rs index 957133e2d..6379e4c6b 100644 --- a/crates/sparrow-backend/src/compile.rs +++ b/crates/sparrow-backend/src/compile.rs @@ -18,6 +18,11 @@ pub fn compile( Cow::Owned(CompileOptions::default()) }; - let physical = LogicalToPhysical::new().apply(root)?; + // Convert the logical expression tree to a physical plan. + let mut physical = LogicalToPhysical::new().apply(root)?; + + // Schedule the steps in the physical plan. + physical.pipelines = crate::pipeline_schedule(&physical.steps); + Ok(physical) } diff --git a/crates/sparrow-backend/src/logical_to_physical.rs b/crates/sparrow-backend/src/logical_to_physical.rs index e96647de4..ea89a5118 100644 --- a/crates/sparrow-backend/src/logical_to_physical.rs +++ b/crates/sparrow-backend/src/logical_to_physical.rs @@ -50,7 +50,12 @@ pub(super) struct LogicalToPhysical { #[derive(Debug)] struct Reference { /// Reference to a specific step. - step_id: sparrow_physical::StepId, + /// + /// This should only be `None` if the expression is a literal. It should + /// not be `None` if the expression is a literal arising within a specific + /// step (domain) -- only for cases where the value originally has no + /// domain. + step_id: Option, /// Expression corresponding to the result of compilation. /// /// Will be the "identity" pattern containing only `?input` if this @@ -62,10 +67,20 @@ impl Reference { /// Create a reference to the input to the given step. fn step_input(step_id: sparrow_physical::StepId) -> error_stack::Result { Ok(Self { - step_id, + step_id: Some(step_id), expr: ExprPattern::new_input()?, }) } + + fn literal(literal: ScalarValue) -> error_stack::Result { + let mut expr = ExprPattern::default(); + let data_type = literal.data_type(); + expr.add_instruction("literal", smallvec![literal], smallvec![], data_type)?; + Ok(Self { + step_id: None, + expr, + }) + } } impl LogicalToPhysical { @@ -85,7 +100,7 @@ impl LogicalToPhysical { ) -> error_stack::Result<(sparrow_physical::StepId, Vec), Error> { match args .iter() - .map(|reference| reference.step_id) + .flat_map(|reference| reference.step_id) .unique() .sorted() .at_most_one() @@ -100,7 +115,10 @@ impl LogicalToPhysical { .map(|arg| { // It shouldn't be possible to have an argument refer to the entire result of the // projection step at this point (while we're building the physical plan). - debug_assert_eq!(arg.step_id, step); + debug_assert!( + arg.step_id.is_none() + || arg.step_id.is_some_and(|arg_step| arg_step == step) + ); arg.expr }) .collect(); @@ -132,27 +150,32 @@ impl LogicalToPhysical { let mut exprs = ExprPattern::new_input()?; let data_type = self.reference_type(&arg)?.clone(); - exprs.add_instruction( - "fieldref", - // Note: that `merge` should produce a record with a - // field for each merged step, identified by the - // step ID being merged. This may change depending - // on (a) optimizations that change steps and - // whether it is difficult to update these and (b) - // how we choose to implement merge. - // - // It may be more practical to use the "index of the - // step ID in the inputs" which would be more stable - // as we change step IDs. - smallvec![ScalarValue::Utf8(Some(format!("step_{}", arg.step_id)))], - smallvec![egg::Id::from(0)], - data_type, - )?; - // Then add the actual expression, replacing the `?input` with the fieldref. - let mut subst = egg::Subst::with_capacity(1); - subst.insert(*crate::exprs::INPUT_VAR, egg::Id::from(1)); - exprs.add_pattern(&arg.expr, &subst)?; + if let Some(input_step) = arg.step_id { + exprs.add_instruction( + "fieldref", + // Note: that `merge` should produce a record with a + // field for each merged step, identified by the + // step ID being merged. This may change depending + // on (a) optimizations that change steps and + // whether it is difficult to update these and (b) + // how we choose to implement merge. + // + // It may be more practical to use the "index of the + // step ID in the inputs" which would be more stable + // as we change step IDs. + smallvec![ScalarValue::Utf8(Some(format!("step_{}", input_step)))], + smallvec![egg::Id::from(0)], + data_type, + )?; + + // Then add the actual expression, replacing the `?input` with the fieldref. + let mut subst = egg::Subst::with_capacity(1); + subst.insert(*crate::exprs::INPUT_VAR, egg::Id::from(1)); + exprs.add_pattern(&arg.expr, &subst)?; + } else { + exprs.add_pattern(&arg.expr, &egg::Subst::default())?; + } Ok(exprs) }) .collect::>()?; @@ -167,7 +190,16 @@ impl LogicalToPhysical { /// the result as well as expressions to apply to that steps output in order /// to prdouce the result. fn visit(&mut self, node: &sparrow_logical::ExprRef) -> error_stack::Result { - match node.name.as_ref() { + match node.name { + "literal" => { + let Some(literal) = node.literal_opt() else { + error_stack::bail!(Error::invalid_logical_plan( + "expected one literal argument to 'literal'" + )) + }; + let literal = logical_to_physical_literal(literal)?; + Ok(Reference::literal(literal)?) + } "read" => { // A logical scan instruction should have a single literal argument // containing the UUID of the table to scan. @@ -180,7 +212,7 @@ impl LogicalToPhysical { let step_id = self.plan.get_or_create_step_id( sparrow_physical::StepKind::Read { - source_id: table_id, + source_uuid: table_id, }, vec![], ExprVec::empty(), @@ -216,11 +248,31 @@ impl LogicalToPhysical { ))); }; - let args = node - .args - .iter() - .map(|arg| self.visit(arg)) - .collect::>()?; + let args = if instruction != "record" { + node.args + .iter() + .map(|arg| self.visit(arg)) + .collect::>()? + } else { + // The arguments to the record are 2N -- (name, value) for each field. + // However, we've already captured the names in the type. We could + // have the creation of logical expressions drop the names, but then + // we should move similar handling of literals there as well (eg., fieldrefs). + // + // TODO: Consider introducing a trait that allows (a) typechecking and (b) + // configuring how arguments convert to logical nodes. This would benefit UDFs + // as well as provide a place to encapsulate the various special handling. + // + // If such a trait had both `create_logical` and `logical_to_physical`, then + // it would give us a place to see and test that the result of creating the + // logical node can be turned into physical nodes, etc. + node.args + .iter() + .skip(1) + .step_by(2) + .map(|arg| self.visit(arg)) + .collect::>()? + }; let (step_id, args) = self.resolve_args(args)?; assert_eq!( @@ -236,7 +288,10 @@ impl LogicalToPhysical { args, node.result_type.clone(), )?; - Ok(Reference { step_id, expr }) + Ok(Reference { + step_id: Some(step_id), + expr, + }) } } } @@ -248,7 +303,10 @@ impl LogicalToPhysical { let result = self.visit(root)?; // Make sure the resulting step is the last step. - assert!(result.step_id == self.plan.last_step_id()); + let result_step_id = result.step_id.ok_or_else(|| { + Error::internal("result must be associated with a step (non-literal)") + })?; + assert!(result_step_id == self.plan.last_step_id()); debug_assert_eq!(self.reference_type(&result)?, root.result_type); @@ -260,7 +318,7 @@ impl LogicalToPhysical { // trivially instantiated within a projection, but we do this check here // for the case of the expression being the result of a non-projection // step, since it lets us avoid adding the projection step. - result.step_id + result_step_id } else { // Add a projection for the remaining expressions. // @@ -268,12 +326,13 @@ impl LogicalToPhysical { // need to deal with mutating the expressions in the projection. For now, we think it // is easier to treat steps as immutable once created, and then combine steps in a // later optimization pass of the physical plan. + let input_type = self.plan.step_result_type(result_step_id).clone(); let result_type = root.result_type.clone(); self.plan.get_or_create_step_id( sparrow_physical::StepKind::Project, - vec![result.step_id], - result.expr.instantiate(result_type)?, - root.result_type.clone(), + vec![result_step_id], + result.expr.instantiate(input_type)?, + result_type, ) }; @@ -284,7 +343,10 @@ impl LogicalToPhysical { fn reference_type(&self, reference: &Reference) -> error_stack::Result { match reference.expr.last() { ENodeOrVar::Var(var) if *var == *crate::exprs::INPUT_VAR => { - Ok(self.plan.step_result_type(reference.step_id).clone()) + let reference_step_id = reference + .step_id + .ok_or_else(|| Error::internal("literal expressions don't have `?input`"))?; + Ok(self.plan.step_result_type(reference_step_id).clone()) } ENodeOrVar::Var(other) => { error_stack::bail!(Error::internal(format!( @@ -296,6 +358,24 @@ impl LogicalToPhysical { } } +fn logical_to_physical_literal( + literal: &sparrow_logical::Literal, +) -> error_stack::Result { + match literal { + sparrow_logical::Literal::Null => todo!(), + sparrow_logical::Literal::Bool(_) => todo!(), + sparrow_logical::Literal::String(s) => Ok(ScalarValue::Utf8(Some(s.clone()))), + sparrow_logical::Literal::Int64(_) => todo!(), + sparrow_logical::Literal::UInt64(_) => todo!(), + sparrow_logical::Literal::Float64(_) => todo!(), + sparrow_logical::Literal::Timedelta { + seconds: _, + nanos: _, + } => todo!(), + sparrow_logical::Literal::Uuid(_) => todo!(), + } +} + #[cfg(test)] mod tests { use std::sync::Arc; @@ -324,17 +404,15 @@ mod tests { )); let x = Arc::new(sparrow_logical::Expr::new_literal_str("x")); let y = Arc::new(sparrow_logical::Expr::new_literal_str("y")); - let x1 = Arc::new( - sparrow_logical::Expr::try_new("fieldref".into(), vec![source1.clone(), x]).unwrap(), - ); + let x1 = + Arc::new(sparrow_logical::Expr::try_new("fieldref", vec![source1.clone(), x]).unwrap()); let physical_x1 = LogicalToPhysical::new().apply(&x1).unwrap(); insta::assert_yaml_snapshot!(physical_x1); - let y1 = Arc::new( - sparrow_logical::Expr::try_new("fieldref".into(), vec![source1, y.clone()]).unwrap(), - ); + let y1 = + Arc::new(sparrow_logical::Expr::try_new("fieldref", vec![source1, y.clone()]).unwrap()); let add_x1_y1 = - Arc::new(sparrow_logical::Expr::try_new("add".into(), vec![x1.clone(), y1]).unwrap()); + Arc::new(sparrow_logical::Expr::try_new("add", vec![x1.clone(), y1]).unwrap()); let physical_add_x1_y1 = LogicalToPhysical::new().apply(&add_x1_y1).unwrap(); insta::assert_yaml_snapshot!(physical_add_x1_y1); @@ -345,10 +423,8 @@ mod tests { struct_type.clone(), group, )); - let y2 = - Arc::new(sparrow_logical::Expr::try_new("fieldref".into(), vec![source2, y]).unwrap()); - let add_x1_y2 = - Arc::new(sparrow_logical::Expr::try_new("add".into(), vec![x1, y2]).unwrap()); + let y2 = Arc::new(sparrow_logical::Expr::try_new("fieldref", vec![source2, y]).unwrap()); + let add_x1_y2 = Arc::new(sparrow_logical::Expr::try_new("add", vec![x1, y2]).unwrap()); let physical_add_x1_y2 = LogicalToPhysical::new().apply(&add_x1_y2).unwrap(); insta::assert_yaml_snapshot!(physical_add_x1_y2); } diff --git a/crates/sparrow-backend/src/pipeline_schedule.rs b/crates/sparrow-backend/src/pipeline_schedule.rs index ef7dfdcf1..066467aeb 100644 --- a/crates/sparrow-backend/src/pipeline_schedule.rs +++ b/crates/sparrow-backend/src/pipeline_schedule.rs @@ -1,8 +1,5 @@ use index_vec::{IndexSlice, IndexVec}; -use sparrow_core::debug_println; -use sparrow_physical::{Pipeline, Step, StepId, StepKind}; - -const DEBUG_SCHEDULING: bool = false; +use sparrow_physical::{Pipeline, Step, StepId}; /// Determine the pipeline each step should be part of. /// @@ -29,7 +26,7 @@ pub fn pipeline_schedule(steps: &IndexSlice) -> Vec { let mut pipelines = Vec::new(); for (index, step) in steps.iter_enumerated() { - let assignment = if is_pipeline_breaker(index, step, &references) { + let assignment = if is_pipeline_breaker(index, step, &references, steps) { // A step with no input (such as a scan) starts a new pipeline. // A step with multiple inputs (such as a merge) is a separate pipeline. let index = pipelines.len(); @@ -49,32 +46,35 @@ pub fn pipeline_schedule(steps: &IndexSlice) -> Vec { } /// Return true if the step is "pipeline breaking". -fn is_pipeline_breaker(index: StepId, step: &Step, references: &IndexVec) -> bool { +/// +/// A "pipeline breaker" must start a new pipeline. +fn is_pipeline_breaker( + index: StepId, + step: &Step, + references: &IndexVec, + steps: &IndexSlice, +) -> bool { match &step.kind { + _ if !step.kind.is_transform() => { + tracing::trace!("Step {index} is new pipeline since it not a transform"); + true + } _ if step.inputs.len() != 1 => { - debug_println!( - DEBUG_SCHEDULING, - "Step {index} is new pipeline since it has multiple inputs" - ); + tracing::trace!("Step {index} is new pipeline since it has multiple inputs"); + true + } + _ if !steps[step.inputs[0]].kind.is_transform() => { + tracing::trace!("Step {index} is a new pipeline since it's input is not a trasnform"); true } _ if references[step.inputs[0]] > 1 => { - debug_println!( - DEBUG_SCHEDULING, + tracing::trace!( "Step {index} is new pipeline since it's only input ({}) is referenced {} times", step.inputs[0], references[step.inputs[0]] ); true } - StepKind::Read { .. } | StepKind::Merge | StepKind::Repartition { .. } => { - debug_println!( - DEBUG_SCHEDULING, - "Step {index} is new pipeline based on kind {:?}", - step.kind - ); - true - } _ => false, } } @@ -96,7 +96,9 @@ mod tests { // 0: scan table1 Step { id: 0.into(), - kind: StepKind::Read { source_id: table1 }, + kind: StepKind::Read { + source_uuid: table1 + }, inputs: vec![], result_type: result_type.clone(), exprs: Exprs::new(), @@ -104,7 +106,9 @@ mod tests { // 1: scan table2 Step { id: 1.into(), - kind: StepKind::Read { source_id: table2 }, + kind: StepKind::Read { + source_uuid: table2 + }, inputs: vec![], result_type: result_type.clone(), exprs: Exprs::new(), @@ -125,7 +129,7 @@ mod tests { result_type: result_type.clone(), exprs: Exprs::new(), }, - // 4: project 2 -> same pipeline since only consumer + // 4: project 2 -> new pipeline since input (2) is a merge, not a transform Step { id: 4.into(), kind: StepKind::Project, @@ -133,11 +137,19 @@ mod tests { result_type: result_type.clone(), exprs: Exprs::new(), }, - // 5: merge 3 and 4 -> new pipeline since merge + // 5: filter 4 -> same pipeline since only consumer Step { id: 5.into(), + kind: StepKind::Filter, + inputs: vec![4.into()], + result_type: result_type.clone(), + exprs: Exprs::new(), + }, + // 6: merge 3 and 5 -> new pipeline since merge + Step { + id: 6.into(), kind: StepKind::Merge, - inputs: vec![3.into(), 4.into()], + inputs: vec![3.into(), 5.into()], result_type, exprs: Exprs::new(), }, @@ -153,13 +165,16 @@ mod tests { steps: vec![1.into()] }, Pipeline { - steps: vec![2.into(), 4.into()] + steps: vec![2.into()] }, Pipeline { steps: vec![3.into()] }, Pipeline { - steps: vec![5.into()] + steps: vec![4.into(), 5.into()] + }, + Pipeline { + steps: vec![6.into()] } ] ); diff --git a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap index 71ed0c14a..a8977864f 100644 --- a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap +++ b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap @@ -32,7 +32,20 @@ steps: - name: input literal_args: [] args: [] - result_type: Float64 + result_type: + Struct: + - name: x + data_type: Int64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: y + data_type: Float64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} - name: fieldref literal_args: - Utf8: y diff --git a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap index 5199870d1..fc1205942 100644 --- a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap +++ b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap @@ -59,7 +59,7 @@ steps: - name: input literal_args: [] args: [] - result_type: Float64 + result_type: "Null" - name: fieldref literal_args: - Utf8: step_1 diff --git a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap index 9ac219a84..d64e7d4ae 100644 --- a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap +++ b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap @@ -32,7 +32,20 @@ steps: - name: input literal_args: [] args: [] - result_type: Int64 + result_type: + Struct: + - name: x + data_type: Int64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: y + data_type: Float64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} - name: fieldref literal_args: - Utf8: x diff --git a/crates/sparrow-batch/src/batch.rs b/crates/sparrow-batch/src/batch.rs index 8fc8dbdba..a94800f48 100644 --- a/crates/sparrow-batch/src/batch.rs +++ b/crates/sparrow-batch/src/batch.rs @@ -77,6 +77,23 @@ impl Batch { }) } + pub fn into_record_batch(self, schema: Arc) -> Option { + self.data.map(|data| { + if let Some(fields) = data.data.as_struct_opt() { + let mut columns = Vec::with_capacity(3 + fields.num_columns()); + columns.extend_from_slice(&[data.time, data.subsort, data.key_hash]); + columns.extend_from_slice(fields.columns()); + RecordBatch::try_new(schema, columns).expect("create_batch") + } else { + RecordBatch::try_new( + schema, + vec![data.time, data.subsort, data.key_hash, data.data], + ) + .expect("create_batch") + } + }) + } + pub fn is_empty(&self) -> bool { self.data.is_none() } diff --git a/crates/sparrow-compiler/Cargo.toml b/crates/sparrow-compiler/Cargo.toml index 5bfbd76b1..5b5cae275 100644 --- a/crates/sparrow-compiler/Cargo.toml +++ b/crates/sparrow-compiler/Cargo.toml @@ -32,7 +32,6 @@ sparrow-api = { path = "../sparrow-api" } sparrow-arrow = { path = "../sparrow-arrow" } sparrow-core = { path = "../sparrow-core" } sparrow-instructions = { path = "../sparrow-instructions" } -sparrow-merge = { path = "../sparrow-merge" } sparrow-sources = { path = "../sparrow-sources" } sparrow-syntax = { path = "../sparrow-syntax" } static_init.workspace = true diff --git a/crates/sparrow-execution/Cargo.toml b/crates/sparrow-execution/Cargo.toml index 3911cf2ac..7987a7d62 100644 --- a/crates/sparrow-execution/Cargo.toml +++ b/crates/sparrow-execution/Cargo.toml @@ -10,25 +10,31 @@ Implementations of the pipelines to be executed. """ [dependencies] +arrow-array.workspace = true +arrow-schema.workspace = true derive_more.workspace = true error-stack.workspace = true -parking_lot.workspace = true futures.workspace = true -sparrow-arrow = { path = "../sparrow-arrow" } +hashbrown.workspace = true +itertools.workspace = true +parking_lot.workspace = true sparrow-batch = { path = "../sparrow-batch" } -sparrow-core = { path = "../sparrow-core" } sparrow-interfaces = { path = "../sparrow-interfaces" } sparrow-physical = { path = "../sparrow-physical" } sparrow-transforms = { path = "../sparrow-transforms" } sparrow-scheduler = { path = "../sparrow-scheduler" } -sparrow-sources = { path = "../sparrow-sources" } tokio.workspace = true +tracing.workspace = true uuid.workspace = true [dev-dependencies] arrow-array.workspace = true arrow-schema.workspace = true index_vec.workspace = true +sparrow-backend = { path = "../sparrow-backend" } +sparrow-logical = { path = "../sparrow-logical" } +sparrow-session = { path = "../sparrow-session" } +sparrow-sources = { path = "../sparrow-sources" } sparrow-testing = { path = "../sparrow-testing" } [lib] diff --git a/crates/sparrow-execution/src/error.rs b/crates/sparrow-execution/src/error.rs new file mode 100644 index 000000000..2e76f3216 --- /dev/null +++ b/crates/sparrow-execution/src/error.rs @@ -0,0 +1,21 @@ +use uuid::Uuid; + +#[derive(derive_more::Display, Debug)] +pub enum Error { + #[display(fmt = "error creating executor")] + Creating, + #[display(fmt = "no source with id: {source_id}")] + NoSuchSource { source_id: Uuid }, + #[display(fmt = "error receiving input")] + SourceError, + #[display(fmt = "panic while running source")] + SourcePanic, + #[display(fmt = "error closing input")] + ClosingInput, + #[display(fmt = "error starting workers")] + Starting, + #[display(fmt = "error stopping workers")] + Stopping, +} + +impl error_stack::Context for Error {} diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index 1b1c435fb..e8cd7380e 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -8,5 +8,228 @@ clippy::undocumented_unsafe_blocks )] -//! Implementations of the pipelines to be executed. +//! Execution of physical plans. + +use arrow_array::RecordBatch; +use arrow_schema::{DataType, Field, Schema, SchemaRef, TimeUnit}; +use hashbrown::HashMap; +use sparrow_interfaces::{ReadConfig, Source}; +use sparrow_physical::StepId; +use sparrow_transforms::TransformPipeline; +use std::sync::Arc; + +use error_stack::ResultExt; +use itertools::Itertools; +use sparrow_scheduler::{InputHandles, WorkerPool}; +use uuid::Uuid; + +mod error; +mod source_tasks; +mod write_channel_pipeline; + +#[cfg(test)] mod tests; + +pub use error::*; + +use crate::source_tasks::SourceTasks; +use crate::write_channel_pipeline::WriteChannelPipeline; + +pub struct PlanExecutor { + worker_pool: WorkerPool, + source_tasks: SourceTasks, + read_config: Arc, +} + +fn result_type_to_output_schema(result_type: &DataType) -> SchemaRef { + let mut output_fields = vec![ + Arc::new(Field::new( + "_time", + DataType::Timestamp(TimeUnit::Nanosecond, None), + false, + )), + // TODO: Convert key hash to key. + Arc::new(Field::new("_key_hash", DataType::UInt64, false)), + ]; + match result_type { + DataType::Struct(fields) => { + output_fields.extend(fields.iter().cloned()); + } + other => output_fields.push(Arc::new(Field::new("output", other.clone(), true))), + }; + let output_schema = Schema::new(output_fields); + Arc::new(output_schema) +} + +impl PlanExecutor { + /// Create the plan executor for the given plan. + /// + /// This creates the worker threads but does not inject inputs, meaning + /// the threads won't have anything to do until `run` is called. + pub fn try_new( + query_id: String, + plan: sparrow_physical::Plan, + sources: &HashMap>, + output: tokio::sync::mpsc::Sender, + ) -> error_stack::Result { + let mut executor = PlanExecutor { + worker_pool: WorkerPool::new(query_id).change_context(Error::Creating)?, + source_tasks: SourceTasks::default(), + read_config: Arc::new(ReadConfig { + keep_open: false, + start_time: None, + end_time: None, + }), + }; + + let last_step = plan.steps.last().expect("at least one step"); + let output_schema = result_type_to_output_schema(&last_step.result_type); + + let sink_pipeline = executor + .worker_pool + .add_pipeline(1, WriteChannelPipeline::new(output, output_schema)); + + // Map from the producing step ID to the consumers. + let mut step_consumers: HashMap = HashMap::new(); + step_consumers + .entry(plan.steps.last_idx()) + .or_default() + .add_consumer(sink_pipeline, 0); + + // Iterate in reverse so the receivers (consumers) are created before the senders (producers). + for pipeline in plan.pipelines.iter().rev() { + let first_step_id = *pipeline.steps.first().expect("at least one step"); + let last_step_id = *pipeline.steps.last().expect("at least one step"); + + // Get the consumer(s) for the last step in the pipeline. + // While we schedule and execute pipelines, the structure of the individual + // steps identify which *step* they receive from. So, to determine where + // this pipeline "pushes output" we need to determine where the last step + // in this pipeline "pushes output". + let consumers = step_consumers + .remove(&last_step_id) + .expect("at least one consumer for step"); + + let first_step_inputs = &plan.steps[first_step_id].inputs; + + // Create a transform pipeline (if possible), othrewise create the + // appropriate non-transform pipeline. + let pipeline = if pipeline + .steps + .iter() + .all(|id| plan.steps[*id].kind.is_transform()) + { + debug_assert_eq!( + first_step_inputs.len(), + 1, + "Transforms should have a single input" + ); + let first_step_input_id = first_step_inputs[0]; + + // If all of the steps are transforms, then we use the transform pipeline. + executor.add_transform_pipeline( + first_step_input_id, + &plan, + &pipeline.steps, + consumers, + )? + } else { + assert_eq!( + pipeline.steps.len(), + 1, + "Non-transforms should be in separate pipelines" + ); + + let step = &plan.steps[pipeline.steps[0]]; + + // `add_non_transform_pipeline` returns `Some(pipeline)` if it adds a + // pipeline for the step. It returns `None` if the step is a source and + // thus has no corresponding pipeline. + if let Some(pipeline) = + executor.add_non_transform_pipeline(step, consumers, sources)? + { + pipeline + } else { + debug_assert_eq!(step.inputs.len(), 0); + continue; + } + }; + + // Add the pipeline as a consumer for it's inputs. + for (input_index, input_step_id) in first_step_inputs.iter().enumerate() { + step_consumers + .entry(*input_step_id) + .or_default() + .add_consumer(pipeline.clone(), input_index); + } + } + + Ok(executor) + } + + pub async fn execute( + self, + _stop_signal_rx: tokio::sync::watch::Receiver, + ) -> error_stack::Result<(), Error> { + let Self { + worker_pool, + source_tasks, + .. + } = self; + + let injector = worker_pool.injector().clone(); + + let workers = worker_pool.start().change_context(Error::Starting)?; + source_tasks.run_sources(injector).await?; + + workers.join().change_context(Error::Stopping)?; + + Ok(()) + } + + fn add_non_transform_pipeline( + &mut self, + step: &sparrow_physical::Step, + consumers: InputHandles, + sources: &HashMap>, + ) -> error_stack::Result>, Error> { + match step.kind { + sparrow_physical::StepKind::Read { source_uuid } => { + let channel = sources.get(&source_uuid).ok_or(Error::NoSuchSource { + source_id: source_uuid, + })?; + let stream = channel.read(&step.result_type, self.read_config.clone()); + self.source_tasks.add_read(source_uuid, stream, consumers); + Ok(None) + } + other if other.is_transform() => { + unreachable!("Transforms should use add_transform_pipeline") + } + other => { + todo!("Unsupported step kind: {other}") + } + } + } + + /// Convert a physical plan Pipeline into the executable scheduler Pipeline. + fn add_transform_pipeline( + &mut self, + first_step_input_id: sparrow_physical::StepId, + plan: &sparrow_physical::Plan, + steps: &[sparrow_physical::StepId], + consumers: InputHandles, + ) -> error_stack::Result, Error> { + tracing::trace!( + "Creating transform pipeline: {}", + steps.iter().format_with(",", |step_id, f| { + let step = &plan.steps[*step_id]; + f(&format_args!("{step}")) + }) + ); + + let steps = steps.iter().map(|id| &plan.steps[*id]); + let pipeline = TransformPipeline::try_new(first_step_input_id, steps, consumers) + .change_context(Error::Creating)?; + Ok(self.worker_pool.add_pipeline(1, pipeline)) + } +} diff --git a/crates/sparrow-execution/src/source_tasks.rs b/crates/sparrow-execution/src/source_tasks.rs new file mode 100644 index 000000000..40c158108 --- /dev/null +++ b/crates/sparrow-execution/src/source_tasks.rs @@ -0,0 +1,101 @@ +use error_stack::ResultExt; +use futures::stream::{BoxStream, FuturesUnordered}; +use futures::{StreamExt, TryStreamExt}; +use sparrow_batch::Batch; +use sparrow_interfaces::SourceError; +use sparrow_scheduler::{Injector, InputHandles}; +use tracing::Instrument; +use uuid::Uuid; + +use crate::Error; + +/// Manages the async streams that must be read for input. +#[derive(Default)] +pub(super) struct SourceTasks { + tasks: Vec, + // /// Streams to be read. Each stream should produce a `(batch, index)` pair, + // /// with indices corresponding to the input in `read_inputs` to send the batch + // /// to. + // /// + // // TODO: We may want to encapsulate this differently / better. Could we use + // // `futures::sink(...)` to do this in some way? We may also want to allow + // // the polling of these sources to be weighted towards the inputs that need + // // a new batch to progress (eg., back-pressure). + // reads: Vec>>, + // /// For each read operation, the `PipelineInputs` to send the batches to. + // read_inputs: Vec, +} + +struct SourceTask { + source_uuid: Uuid, + stream: BoxStream<'static, error_stack::Result>, + consumers: InputHandles, +} + +impl SourceTasks { + pub fn add_read( + &mut self, + source_uuid: Uuid, + stream: BoxStream<'static, error_stack::Result>, + consumers: InputHandles, + ) { + self.tasks.push(SourceTask { + source_uuid, + stream, + consumers, + }) + } + + /// Execute all of the sources. + pub async fn run_sources(self, injector: Injector) -> error_stack::Result<(), Error> { + let mut tasks: FuturesUnordered>> = + self.tasks + .into_iter() + .map(|task| { + let SourceTask { + mut stream, + consumers, + source_uuid, + } = task; + let span = tracing::info_span!("Source Task", source=?source_uuid); + let mut injector = injector.clone(); + tokio::spawn( + async move { + tracing::info!("Reading batches for source"); + while let Some(batch) = + stream.try_next().await.change_context(Error::SourceError)? + { + consumers + .add_input(0.into(), batch, &mut injector) + .change_context(Error::SourceError)?; + } + + tracing::info!("Completed stream for source. Closing."); + consumers + .close_input(0.into(), &mut injector) + .change_context(Error::SourceError)?; + + tracing::info!("Closed source"); + Ok(()) + } + .instrument(span), + ) + }) + .collect(); + + while let Some(task_result) = tasks.next().await { + match task_result { + Ok(Ok(())) => tracing::trace!("Source task completed. {} remaining.", tasks.len()), + Ok(Err(e)) => return Err(e), + Err(e) => { + tracing::trace!("Source task panicked: {e}"); + error_stack::bail!(Error::SourcePanic); + } + } + tracing::info!("Source completed. {} remaining.", tasks.len()); + } + tracing::info!("All sources completed."); + + Ok(()) + } +} diff --git a/crates/sparrow-execution/src/tests.rs b/crates/sparrow-execution/src/tests.rs index cb239bc66..d44da850c 100644 --- a/crates/sparrow-execution/src/tests.rs +++ b/crates/sparrow-execution/src/tests.rs @@ -1,263 +1,97 @@ -#[cfg(test)] -mod tests { - - use std::sync::Arc; - - use arrow_array::cast::AsArray; - use arrow_array::{ArrayRef, Int64Array, RecordBatch, TimestampNanosecondArray, UInt64Array}; - use arrow_schema::{DataType, Field, Fields, Schema, SchemaRef, TimeUnit}; - use error_stack::{IntoReport, ResultExt}; - use futures::stream::BoxStream; - use futures::TryStreamExt; - use index_vec::index_vec; - use parking_lot::Mutex; - use sparrow_arrow::scalar_value::ScalarValue; - use sparrow_batch::Batch; - use sparrow_interfaces::ReadConfig; - use sparrow_interfaces::Source; - use sparrow_scheduler::{Pipeline, PipelineError, PipelineInput, WorkerPool}; - use sparrow_transforms::TransformPipeline; - - #[derive(derive_more::Display, Debug)] - pub enum Error { - #[display(fmt = "error creating executor")] - Creating, - #[display(fmt = "error executing")] - Executing, - } - - impl error_stack::Context for Error {} - - fn in_memory_source(schema: SchemaRef) -> sparrow_sources::InMemory { - let source = sparrow_sources::InMemory::new(true, schema.clone()).unwrap(); - source - } - - #[tokio::test] - #[ignore] - async fn test_query() { - sparrow_testing::init_test_logging(); +use arrow_array::cast::AsArray; +use arrow_array::{Int64Array, RecordBatch, TimestampNanosecondArray, UInt64Array}; +use sparrow_interfaces::{Source, SourceExt}; +use sparrow_logical::ExprRef; +use sparrow_session::partitioned::Session; +use sparrow_sources::InMemory; +use std::sync::Arc; + +use arrow_schema::{DataType, Field, Schema}; + +fn query( + session: &Session, + source: ExprRef, +) -> error_stack::Result { + let a_str = session.add_literal(sparrow_logical::Literal::new_str("a"))?; + let a = session.add_expr("fieldref", vec![source.clone(), a_str])?; + + let b_str = session.add_literal(sparrow_logical::Literal::new_str("b"))?; + let b = session.add_expr("fieldref", vec![source.clone(), b_str])?; + + let c_str = session.add_literal(sparrow_logical::Literal::new_str("c"))?; + let c = session.add_expr("fieldref", vec![source.clone(), c_str])?; + + let a_plus_b = session.add_expr("add", vec![a, b])?; + let a_plus_b_plus_c = session.add_expr("add", vec![a_plus_b.clone(), c])?; + + let ab_str = session.add_literal(sparrow_logical::Literal::new_str("ab"))?; + let abc_str = session.add_literal(sparrow_logical::Literal::new_str("abc"))?; + session.add_expr("record", vec![ab_str, a_plus_b, abc_str, a_plus_b_plus_c]) +} - let (output_tx, mut output_rx) = tokio::sync::mpsc::channel(10); +fn add_input_batch(session: &Session, source: &Arc) { + let source_prepared_schema = Arc::new(Schema::new(vec![ + Field::new( + "_time", + DataType::Timestamp(arrow_schema::TimeUnit::Nanosecond, None), + false, + ), + Field::new("_subsort", DataType::UInt64, false), + Field::new("_key_hash", DataType::UInt64, false), + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + ])); + + let time = Arc::new(TimestampNanosecondArray::from(vec![0, 1, 2, 3])); + let subsort = Arc::new(UInt64Array::from(vec![0, 1, 2, 3])); + let key_hash = Arc::new(UInt64Array::from(vec![0, 1, 2, 3])); + let a = Arc::new(Int64Array::from(vec![0, 1, 2, 3])); + let b = Arc::new(Int64Array::from(vec![4, 7, 10, 11])); + let c = Arc::new(Int64Array::from(vec![Some(21), None, Some(387), Some(87)])); + let batch = RecordBatch::try_new( + source_prepared_schema.clone(), + vec![time, subsort, key_hash, a, b, c], + ) + .unwrap(); + + let source: &InMemory = source.downcast_source(); + session.block_on(source.add_batch(batch)).unwrap(); +} - let input_fields = Fields::from(vec![ - Field::new( - "_time", - DataType::Timestamp(TimeUnit::Nanosecond, None), - false, - ), - Field::new("_subsort", DataType::UInt64, true), - Field::new("_key_hash", DataType::UInt64, false), - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - ]); - let projected_datatype = DataType::Struct(input_fields.clone()); - let schema = Arc::new(Schema::new(input_fields.clone())); - let read_config = ReadConfig { - keep_open: true, - start_time: None, - end_time: None, - }; +#[test] +fn test_logical_query() { + sparrow_testing::init_test_logging(); + let mut session = Session::default(); - let input_source = in_memory_source(schema.clone()); - let input_rx = input_source.read(&projected_datatype, read_config); + let source_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + ])); - let output_fields = Fields::from(vec![ - Field::new("ab", DataType::Int64, true), - Field::new("abc", DataType::Int64, true), - ]); + let source: Arc = Arc::new(InMemory::new(true, source_schema.clone()).unwrap()); + let source_expr = session.add_source(source.clone()).unwrap(); - let input_columns: Vec = vec![ - Arc::new(TimestampNanosecondArray::from(vec![0, 1, 2, 3])), // time - Arc::new(UInt64Array::from(vec![0, 1, 2, 3])), // subsort - Arc::new(UInt64Array::from(vec![0, 1, 2, 3])), // key_hash - Arc::new(Int64Array::from(vec![0, 1, 2, 3])), // a - Arc::new(Int64Array::from(vec![4, 7, 10, 11])), // b - Arc::new(Int64Array::from(vec![Some(21), None, Some(387), Some(87)])), // c - ]; - let input_batch = RecordBatch::try_new(schema.clone(), input_columns).unwrap(); - input_source.add_batch(input_batch).await.unwrap(); + // Add some data before running the query. + // Note this uses an odd downcast because we've "erased" the type to `dyn Expr`. + {} - execute( - "hello".to_owned(), - DataType::Struct(input_fields), - input_rx, - DataType::Struct(output_fields), - output_tx, - ) - .await + let query = query(&session, source_expr).unwrap(); + let execution = session + .execute(&query, sparrow_session::ExecutionOptions::default()) .unwrap(); - let output = output_rx.recv().await.unwrap(); - let output = output.data().unwrap().as_struct(); - let ab = output.column_by_name("ab").unwrap(); - let abc = output.column_by_name("abc").unwrap(); - assert_eq!(ab.as_primitive(), &Int64Array::from(vec![4, 8, 12, 14])); - assert_eq!( - abc.as_primitive(), - &Int64Array::from(vec![Some(25), None, Some(399), Some(101)]) - ); - } - - /// Execute a physical plan. - pub async fn execute( - query_id: String, - input_type: DataType, - mut input: BoxStream<'_, error_stack::Result>, - output_type: DataType, - output: tokio::sync::mpsc::Sender, - ) -> error_stack::Result<(), Error> { - let mut worker_pool = WorkerPool::start(query_id).change_context(Error::Creating)?; - - // This sets up some fake stuff: - // - We don't have sinks yet, so we use tokio channels. - // - We create a "hypothetical" scan step (0) - // - We create a hard-coded "project" step (1) - // - We output the results to the channel. - - let table_id = uuid::uuid!("67e55044-10b1-426f-9247-bb680e5fe0c8"); - - let scan = sparrow_physical::Step { - id: 0.into(), - kind: sparrow_physical::StepKind::Read { - source_id: table_id, - }, - inputs: vec![], - result_type: input_type, - exprs: sparrow_physical::Exprs::new(), - }; - - let project = sparrow_physical::Step { - id: 1.into(), - kind: sparrow_physical::StepKind::Project, - inputs: vec![0.into()], - result_type: output_type.clone(), - exprs: index_vec![ - sparrow_physical::Expr { - name: "column".into(), - literal_args: vec![ScalarValue::Utf8(Some("a".to_owned()))], - args: vec![], - result_type: DataType::Int64 - }, - sparrow_physical::Expr { - name: "column".into(), - literal_args: vec![ScalarValue::Utf8(Some("b".to_owned()))], - args: vec![], - result_type: DataType::Int64 - }, - sparrow_physical::Expr { - name: "add".into(), - literal_args: vec![], - args: vec![0.into(), 1.into()], - result_type: DataType::Int64 - }, - sparrow_physical::Expr { - name: "column".into(), - literal_args: vec![ScalarValue::Utf8(Some("c".to_owned()))], - args: vec![], - result_type: DataType::Int64 - }, - sparrow_physical::Expr { - name: "add".into(), - literal_args: vec![], - args: vec![2.into(), 3.into()], - result_type: DataType::Int64 - }, - sparrow_physical::Expr { - name: "record".into(), - literal_args: vec![], - args: vec![2.into(), 4.into()], - result_type: output_type - } - ], - }; - - let sink_pipeline = worker_pool.add_pipeline(1, WriteChannelPipeline::new(output)); - let transform_pipeline = worker_pool.add_pipeline( - 1, - TransformPipeline::try_new( - &scan, - [project].iter(), - PipelineInput::new(sink_pipeline, 0), - ) - .change_context(Error::Creating)?, - ); - let transform_pipeline_input = PipelineInput::new(transform_pipeline, 0); - - let mut injector = worker_pool.injector().clone(); - while let Some(batch) = input.try_next().await.unwrap() { - transform_pipeline_input - .add_input(0.into(), batch, &mut injector) - .change_context(Error::Executing)?; - } - transform_pipeline_input - .close_input(0.into(), &mut injector) - .change_context(Error::Executing)?; - worker_pool.stop().change_context(Error::Executing)?; - - Ok(()) - } - - #[derive(Debug)] - struct WriteChannelPipeline(Mutex>>); - - impl WriteChannelPipeline { - fn new(channel: tokio::sync::mpsc::Sender) -> Self { - Self(Mutex::new(Some(channel))) - } - } - - impl Pipeline for WriteChannelPipeline { - fn initialize( - &mut self, - _tasks: sparrow_scheduler::Partitioned, - ) { - } - - fn add_input( - &self, - input_partition: sparrow_scheduler::Partition, - input: usize, - batch: Batch, - _scheduler: &mut dyn sparrow_scheduler::Scheduler, - ) -> error_stack::Result<(), PipelineError> { - let channel = self.0.lock(); - channel - .as_ref() - .ok_or(PipelineError::InputClosed { - input, - input_partition, - })? - .blocking_send(batch) - .into_report() - .change_context(PipelineError::Execution) - } - - fn close_input( - &self, - input_partition: sparrow_scheduler::Partition, - input: usize, - _scheduler: &mut dyn sparrow_scheduler::Scheduler, - ) -> error_stack::Result<(), PipelineError> { - let mut channel = self.0.lock(); - error_stack::ensure!( - channel.is_some(), - PipelineError::InputClosed { - input, - input_partition, - }, - ); - *channel = None; - Ok(()) - } - - fn do_work( - &self, - _partition: sparrow_scheduler::Partition, - _scheduler: &mut dyn sparrow_scheduler::Scheduler, - ) -> error_stack::Result<(), PipelineError> { - Ok(()) - } - } + add_input_batch(&session, &source); + + let output = execution.collect_all_blocking().unwrap(); + assert_eq!(output.len(), 1); + let output = output.into_iter().next().unwrap(); + let ab = output.column_by_name("ab").unwrap(); + let abc = output.column_by_name("abc").unwrap(); + assert_eq!(ab.as_primitive(), &Int64Array::from(vec![4, 8, 12, 14])); + assert_eq!( + abc.as_primitive(), + &Int64Array::from(vec![Some(25), None, Some(399), Some(101)]) + ); } diff --git a/crates/sparrow-execution/src/write_channel_pipeline.rs b/crates/sparrow-execution/src/write_channel_pipeline.rs new file mode 100644 index 000000000..e10744226 --- /dev/null +++ b/crates/sparrow-execution/src/write_channel_pipeline.rs @@ -0,0 +1,82 @@ +use arrow_array::RecordBatch; +use arrow_schema::SchemaRef; +use error_stack::{IntoReport, ResultExt}; +use parking_lot::Mutex; +use sparrow_batch::Batch; +use sparrow_scheduler::{Pipeline, PipelineError}; + +/// Pipeline for writing to a Tokio channel. +/// +/// This is currently a place holder for a more complete write solution. +#[derive(Debug)] +pub(super) struct WriteChannelPipeline { + channel: Mutex>>, + tasks: sparrow_scheduler::Partitioned, + schema: SchemaRef, +} + +impl WriteChannelPipeline { + pub fn new(channel: tokio::sync::mpsc::Sender, schema: SchemaRef) -> Self { + Self { + channel: Mutex::new(Some(channel)), + tasks: sparrow_scheduler::Partitioned::new(), + schema, + } + } +} + +impl Pipeline for WriteChannelPipeline { + fn initialize(&mut self, tasks: sparrow_scheduler::Partitioned) { + assert_eq!(tasks.len(), 1); + self.tasks = tasks; + } + + fn add_input( + &self, + input_partition: sparrow_scheduler::Partition, + input: usize, + batch: Batch, + _scheduler: &mut dyn sparrow_scheduler::Scheduler, + ) -> error_stack::Result<(), PipelineError> { + if let Some(batch) = batch.into_record_batch(self.schema.clone()) { + let channel = self.channel.lock(); + channel + .as_ref() + .ok_or(PipelineError::InputClosed { + input, + input_partition, + })? + .blocking_send(batch) + .into_report() + .change_context(PipelineError::Execution)? + } + Ok(()) + } + + fn close_input( + &self, + input_partition: sparrow_scheduler::Partition, + input: usize, + _scheduler: &mut dyn sparrow_scheduler::Scheduler, + ) -> error_stack::Result<(), PipelineError> { + let mut channel = self.channel.lock(); + error_stack::ensure!( + channel.is_some(), + PipelineError::InputClosed { + input, + input_partition, + }, + ); + *channel = None; + self.tasks[0].complete(); + Ok(()) + } + + fn do_work( + &self, + _partition: sparrow_scheduler::Partition, + _scheduler: &mut dyn sparrow_scheduler::Scheduler, + ) -> error_stack::Result<(), PipelineError> { + Ok(()) + } +} diff --git a/crates/sparrow-expressions/src/error.rs b/crates/sparrow-expressions/src/error.rs index 69bd39151..73868bced 100644 --- a/crates/sparrow-expressions/src/error.rs +++ b/crates/sparrow-expressions/src/error.rs @@ -73,8 +73,11 @@ pub enum Error { b_label: &'static str, b_len: usize, }, - #[display(fmt = "no field named '{field_name}' in struct")] - NoSuchField { field_name: String }, + #[display(fmt = "no field named '{field_name}' in struct '{fields:?}")] + NoSuchField { + field_name: String, + fields: arrow_schema::Fields, + }, #[display(fmt = "unsupported: {_0}")] Unsupported(Cow<'static, str>), } diff --git a/crates/sparrow-expressions/src/evaluators.rs b/crates/sparrow-expressions/src/evaluators.rs index 8972e499b..e14e16c03 100644 --- a/crates/sparrow-expressions/src/evaluators.rs +++ b/crates/sparrow-expressions/src/evaluators.rs @@ -15,7 +15,7 @@ use crate::Error; mod cast; mod coalesce; mod comparison; -mod field_ref; +mod fieldref; mod hash; mod input; mod is_valid; @@ -219,6 +219,11 @@ pub fn intern_name(name: &str) -> Option<&'static str> { EVALUATORS.get_key_value(name).map(|(k, _)| *k) } +// Exposed so we can report "nearest" names. +pub fn names() -> impl Iterator { + EVALUATORS.keys().copied() +} + #[cfg(test)] mod tests { #[test] diff --git a/crates/sparrow-expressions/src/evaluators/field_ref.rs b/crates/sparrow-expressions/src/evaluators/fieldref.rs similarity index 94% rename from crates/sparrow-expressions/src/evaluators/field_ref.rs rename to crates/sparrow-expressions/src/evaluators/fieldref.rs index d34802607..beefa711e 100644 --- a/crates/sparrow-expressions/src/evaluators/field_ref.rs +++ b/crates/sparrow-expressions/src/evaluators/fieldref.rs @@ -5,11 +5,11 @@ use crate::values::StructValue; use crate::Error; inventory::submit!(crate::evaluators::EvaluatorFactory { - name: "field_ref", + name: "fieldref", create: &create }); -/// Evaluator for `field_ref` expressions. +/// Evaluator for `fieldref` expressions. struct FieldRefEvaluator { input: StructValue, field: usize, @@ -37,6 +37,7 @@ fn create(info: super::StaticInfo<'_>) -> error_stack::Result DataType::Struct(fields) => { let (index, field) = fields.find(field).ok_or_else(|| Error::NoSuchField { field_name: field.to_owned(), + fields: fields.clone(), })?; error_stack::ensure!( result_type == field.data_type(), diff --git a/crates/sparrow-expressions/src/lib.rs b/crates/sparrow-expressions/src/lib.rs index e54cf1de0..76ac3bf85 100644 --- a/crates/sparrow-expressions/src/lib.rs +++ b/crates/sparrow-expressions/src/lib.rs @@ -93,5 +93,5 @@ mod values; mod work_area; pub use error::*; -pub use evaluators::intern_name; +pub use evaluators::{intern_name, names}; pub use executor::*; diff --git a/crates/sparrow-interfaces/Cargo.toml b/crates/sparrow-interfaces/Cargo.toml index 48d225263..dda71821d 100644 --- a/crates/sparrow-interfaces/Cargo.toml +++ b/crates/sparrow-interfaces/Cargo.toml @@ -15,7 +15,6 @@ derive_more.workspace = true error-stack.workspace = true futures.workspace = true sparrow-batch = { path = "../sparrow-batch" } -sparrow-core = { path = "../sparrow-core" } [dev-dependencies] diff --git a/crates/sparrow-interfaces/src/source.rs b/crates/sparrow-interfaces/src/source.rs index 4b9b74127..46d682d96 100644 --- a/crates/sparrow-interfaces/src/source.rs +++ b/crates/sparrow-interfaces/src/source.rs @@ -1,3 +1,5 @@ +use std::sync::Arc; + use arrow_schema::{DataType, SchemaRef}; use futures::stream::BoxStream; use sparrow_batch::Batch; @@ -21,12 +23,34 @@ pub trait Source: Send + Sync { fn read( &self, projected_datatype: &DataType, - read_config: ReadConfig, - ) -> BoxStream<'_, error_stack::Result>; + read_config: Arc, + ) -> BoxStream<'static, error_stack::Result>; + + /// Allow downcasting the source. + fn as_any(&self) -> &dyn std::any::Any; +} + +pub trait SourceExt { + fn downcast_source_opt(&self) -> Option<&T>; + fn downcast_source(&self) -> &T { + self.downcast_source_opt().expect("unexpected type") + } +} + +impl SourceExt for Arc { + fn downcast_source_opt(&self) -> Option<&T> { + self.as_any().downcast_ref() + } +} + +impl SourceExt for &Arc { + fn downcast_source_opt(&self) -> Option<&T> { + self.as_any().downcast_ref() + } } /// Defines the configuration for a read from a source. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct ReadConfig { /// If true, the read will act as an unbounded source and continue reading /// as new data is added. It is on the consumer to close the channel. diff --git a/crates/sparrow-logical/src/error.rs b/crates/sparrow-logical/src/error.rs index 08f54f18e..a650b0c31 100644 --- a/crates/sparrow-logical/src/error.rs +++ b/crates/sparrow-logical/src/error.rs @@ -1,3 +1,5 @@ +use std::borrow::Cow; + use crate::{ExprRef, Grouping}; use arrow_schema::DataType; @@ -6,7 +8,7 @@ use sparrow_types::DisplayFenlType; #[derive(derive_more::Display, Debug)] pub enum Error { #[display(fmt = "internal error: {_0}")] - Internal(&'static str), + Internal(Cow<'static, str>), #[display(fmt = "invalid non-struct type: {}", "_0.display()")] InvalidNonStructType(DataType), #[display(fmt = "invalid non-string literal: {_0:?}")] @@ -22,4 +24,10 @@ pub enum Error { InvalidFunction(String), } +impl Error { + pub(crate) fn internal(message: impl Into>) -> Self { + Self::Internal(message.into()) + } +} + impl error_stack::Context for Error {} diff --git a/crates/sparrow-logical/src/expr.rs b/crates/sparrow-logical/src/expr.rs index 0006d1651..3eaab0098 100644 --- a/crates/sparrow-logical/src/expr.rs +++ b/crates/sparrow-logical/src/expr.rs @@ -1,7 +1,6 @@ use crate::{Error, Grouping}; use arrow_schema::{DataType, TimeUnit}; use sparrow_types::Types; -use std::borrow::Cow; use std::sync::Arc; use uuid::Uuid; @@ -9,8 +8,18 @@ use uuid::Uuid; #[derive(Debug)] pub struct Expr { /// The instruction being applied by this expression. - pub name: Cow<'static, str>, + pub name: &'static str, /// Zero or more literal-valued arguments. + /// + /// In logical plans, very few things should use literal_args. Currently, these are: + /// + /// 1. The expression corresponding to a literal has name `literal`, `literal_args` + /// corresponding to the value, and empty `args`. + /// 2. Sources have name `read`, `literal_args` containing the UUID of the source + /// being read, and empty `args`. + /// + /// Other things that use `literal_args` (such as field refs) in the physical + /// plan are handled by the conversion of logical plans to physical plans. pub literal_args: Vec, /// Arguments to the expression. pub args: Vec, @@ -34,15 +43,18 @@ pub enum Literal { Uuid(Uuid), } +impl Literal { + pub fn new_str(str: impl Into) -> Self { + Self::String(str.into()) + } +} + impl Expr { - pub fn try_new( - name: Cow<'static, str>, - args: Vec, - ) -> error_stack::Result { + pub fn try_new(name: &'static str, args: Vec) -> error_stack::Result { let Types { arguments: arg_types, result: result_type, - } = crate::typecheck::typecheck(name.as_ref(), &args)?; + } = crate::typecheck::typecheck(name, &args)?; // If any of the types are different, we'll need to create new arguments. let args = args @@ -75,7 +87,7 @@ impl Expr { grouping: Grouping, ) -> Self { Self { - name: Cow::Borrowed(name), + name, literal_args: vec![Literal::Uuid(uuid)], args: vec![], result_type, @@ -95,7 +107,7 @@ impl Expr { Literal::Uuid(_) => DataType::FixedSizeBinary(BYTES_IN_UUID), }; Self { - name: Cow::Borrowed("literal"), + name: "literal", literal_args: vec![literal], args: vec![], result_type, @@ -114,7 +126,7 @@ impl Expr { } else { let grouping = self.grouping; Ok(Arc::new(Expr { - name: Cow::Borrowed("cast"), + name: "cast", literal_args: vec![], args: vec![self], result_type: data_type, @@ -123,7 +135,7 @@ impl Expr { } } - /// If this expression is a literal, return the corresponding scalar value. + /// If this expression is a literal, return the corresponding value. pub fn literal_opt(&self) -> Option<&Literal> { if self.name == "literal" { debug_assert_eq!(self.literal_args.len(), 1); @@ -179,7 +191,7 @@ mod tests { Grouping::Literal, )); let field = Expr::try_new( - "fieldref".into(), + "fieldref", vec![ source, Arc::new(Expr::new_literal(Literal::String("a".to_owned()))), @@ -206,7 +218,7 @@ mod tests { )); let a_i32 = Arc::new( Expr::try_new( - "fieldref".into(), + "fieldref", vec![ source, Arc::new(Expr::new_literal(Literal::String("a".to_owned()))), @@ -217,7 +229,7 @@ mod tests { // i32 + f64 literal => f64 let a_i32_plus_1 = Expr::try_new( - "add".into(), + "add", vec![ a_i32.clone(), Arc::new(Expr::new_literal(Literal::Float64(1.0.into()))), diff --git a/crates/sparrow-logical/src/typecheck.rs b/crates/sparrow-logical/src/typecheck.rs index bfaf1d119..e26b58178 100644 --- a/crates/sparrow-logical/src/typecheck.rs +++ b/crates/sparrow-logical/src/typecheck.rs @@ -2,6 +2,7 @@ use crate::{Error, ExprRef}; use arrow_schema::DataType; use error_stack::ResultExt; use hashbrown::HashMap; +use itertools::Itertools; use sparrow_types::{Signature, Types}; /// Type-check the given function name. @@ -10,7 +11,7 @@ pub(crate) fn typecheck(name: &str, args: &[ExprRef]) -> error_stack::Result { error_stack::ensure!( args.len() == 2, - Error::Internal("invalid arguments for fieldref") + Error::internal(format!("invalid arguments ({}) for fieldref", args.len())) ); let DataType::Struct(fields) = &args[0].result_type else { @@ -31,6 +32,37 @@ pub(crate) fn typecheck(name: &str, args: &[ExprRef]) -> error_stack::Result { + error_stack::ensure!( + !args.is_empty() && args.len() % 2 == 0, + Error::internal(format!( + "expected non-zero, even number of arguments, was {}", + args.len() + )) + ); + + let mut arguments = Vec::with_capacity(args.len()); + let mut fields = Vec::with_capacity(args.len() / 2); + for (name, field) in args.iter().tuples() { + let Some(name) = name.literal_str_opt() else { + error_stack::bail!(Error::InvalidNonStringLiteral(args[0].clone())) + }; + + arguments.push(DataType::Utf8); + arguments.push(field.result_type.clone()); + fields.push(arrow_schema::Field::new( + name, + field.result_type.clone(), + true, + )); + } + + let types = Types { + arguments, + result: DataType::Struct(fields.into()), + }; + Ok(types) + } _ => { let signature = get_signature(name)?; // TODO: Ideally, instantiate would accept references so we didn't need to clone. diff --git a/crates/sparrow-merge/Cargo.toml b/crates/sparrow-merge/Cargo.toml index 6cd261b72..cf9becbb1 100644 --- a/crates/sparrow-merge/Cargo.toml +++ b/crates/sparrow-merge/Cargo.toml @@ -19,18 +19,12 @@ arrow-array.workspace = true arrow-csv = { workspace = true, optional = true } arrow-schema.workspace = true arrow-select.workspace = true -async-broadcast.workspace = true -async-stream.workspace = true bit-set.workspace = true -derive_more.workspace = true -error-stack.workspace = true -futures.workspace = true itertools.workspace = true proptest = { workspace = true, optional = true } smallvec.workspace = true sparrow-arrow = { path = "../sparrow-arrow" } sparrow-core = { path = "../sparrow-core" } -tracing.workspace = true [dev-dependencies] arrow-csv.workspace = true diff --git a/crates/sparrow-physical/Cargo.toml b/crates/sparrow-physical/Cargo.toml index dc904ec02..8249dc01b 100644 --- a/crates/sparrow-physical/Cargo.toml +++ b/crates/sparrow-physical/Cargo.toml @@ -12,13 +12,14 @@ Physical execution plans for Kaskada queries. [dependencies] arrow-schema.workspace = true index_vec.workspace = true +serde_yaml.workspace = true serde.workspace = true sparrow-arrow = { path = "../sparrow-arrow" } strum_macros.workspace = true +tracing.workspace = true uuid.workspace = true [dev-dependencies] -serde_yaml.workspace = true insta.workspace = true [lib] diff --git a/crates/sparrow-physical/src/plan.rs b/crates/sparrow-physical/src/plan.rs index 1197a3118..7c6bcefbb 100644 --- a/crates/sparrow-physical/src/plan.rs +++ b/crates/sparrow-physical/src/plan.rs @@ -15,6 +15,27 @@ pub struct Plan { pub pipelines: Vec, } +impl Plan { + pub fn yaml(&self) -> impl std::fmt::Display + '_ { + PlanYaml(self) + } +} + +/// Prints a plan as Yaml. +struct PlanYaml<'a>(&'a Plan); + +impl<'a> std::fmt::Display for PlanYaml<'a> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // If these YAML strings are large, we could potentially implement `std::io::Write` + // to the `std::fmt::Formatter`, but for now we go via a string. + let yaml = serde_yaml::to_string(self.0).map_err(|e| { + tracing::error!("Failed to write plan YAML: {e:?}"); + std::fmt::Error + })?; + write!(f, "{yaml}") + } +} + /// Information about a specific "pipeline" within the plan. /// /// Pipelines take a single input through a linear sequence of diff --git a/crates/sparrow-physical/src/step.rs b/crates/sparrow-physical/src/step.rs index 559d00111..64d4221b1 100644 --- a/crates/sparrow-physical/src/step.rs +++ b/crates/sparrow-physical/src/step.rs @@ -47,6 +47,15 @@ pub struct Step { pub exprs: IndexVec, } +impl std::fmt::Display for Step { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let Self { + id, kind, inputs, .. + } = self; + write!(f, "Step {id}: kind {kind} reads {:?}", inputs) + } +} + /// The kinds of steps that can occur in the physical plan. #[derive( Clone, @@ -60,11 +69,12 @@ pub struct Step { Hash, )] #[serde(rename_all = "snake_case")] +#[strum(serialize_all = "snake_case")] pub enum StepKind { /// Read the given source. Read { - source_id: uuid::Uuid, + source_uuid: uuid::Uuid, }, /// Merge the given relations. Merge, @@ -90,3 +100,42 @@ pub enum StepKind { }, Error, } + +impl StepKind { + /// Return true if the step is implemented as a transform (rather than an operation). + /// + /// This should generally be true for steps that take a single input and + /// effectively perform a "flat map" over the batches. Specifically, they + /// may change the number of rows in a batch (or omit the batch entirely) + /// and they may change the columns in the batch but they should not accept + /// multiple inputs or need to interact with scheduling. + /// + /// Examples: + /// + /// - `project` is a transform because it computes new columns for each row in + /// every batch + /// - `filter` is a transform because it removes rows from each batch, and omits + /// empty batches + /// - `merge` is not a transform because it accepts multiple inputs + /// - `shift` may be a transform if it uses the time in the input batch to determine + /// which rows to output or it may be an operation if it interacts with scheduling + /// in a more sophisticated way. + pub fn is_transform(&self) -> bool { + matches!(self, StepKind::Project | StepKind::Filter) + } +} + +impl std::fmt::Display for StepKind { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + StepKind::Read { + source_uuid: source_id, + } => write!(f, "read({source_id})"), + StepKind::Repartition { num_partitions } => write!(f, "repartition({num_partitions})"), + _ => { + let name: &'static str = self.into(); + write!(f, "{name}") + } + } + } +} diff --git a/crates/sparrow-scheduler/Cargo.toml b/crates/sparrow-scheduler/Cargo.toml index dab7fab27..98575ba65 100644 --- a/crates/sparrow-scheduler/Cargo.toml +++ b/crates/sparrow-scheduler/Cargo.toml @@ -13,10 +13,12 @@ Scheduler for managing local, multi-threaded execution. core_affinity.workspace = true derive_more.workspace = true error-stack.workspace = true +hashbrown.workspace = true index_vec.workspace = true itertools.workspace = true +parking_lot.workspace = true serde.workspace = true -sparrow-arrow = { path = "../sparrow-arrow" } +smallvec.workspace = true sparrow-batch = { path = "../sparrow-batch" } tracing.workspace = true work-queue = "0.1.4" diff --git a/crates/sparrow-scheduler/src/input_handles.rs b/crates/sparrow-scheduler/src/input_handles.rs new file mode 100644 index 000000000..0389d0627 --- /dev/null +++ b/crates/sparrow-scheduler/src/input_handles.rs @@ -0,0 +1,87 @@ +use std::sync::Arc; + +use sparrow_batch::Batch; + +use crate::{Partition, Pipeline, PipelineError, Scheduler}; + +/// A struct used for sending batches to a specific input port of a down-stream pipeline. +#[derive(Debug)] +struct InputHandle { + pipeline: Arc, + input: usize, +} + +#[derive(Debug, Default)] +pub struct InputHandles(smallvec::SmallVec<[InputHandle; 1]>); + +impl InputHandles { + pub fn is_empty(&self) -> bool { + self.0.is_empty() + } + + pub fn len(&self) -> usize { + self.0.len() + } + + pub fn add_consumer(&mut self, pipeline: Arc, input: usize) { + self.0.push(InputHandle::new(pipeline, input)); + } + + pub fn add_input( + &self, + input_partition: Partition, + batch: Batch, + scheduler: &mut dyn Scheduler, + ) -> error_stack::Result<(), PipelineError> { + debug_assert!(!self.0.is_empty(), "Inputs should be non-empty when used."); + + // TODO: Currently, there is a chance that one pipeline outputting to multiple + // pipelines schedules all of the consumers on the same task pool. This should be + // OK due to task-stealing, but we may be able to do better. If we place the + // first consumer that needs to be woken on the local queue, we could place the + // others on the global queue. This would cause the batch to move to another + // core, but would let both consumers run in parallel. + for input in self.0.iter() { + input.add_input(input_partition, batch.clone(), scheduler)?; + } + Ok(()) + } + + pub fn close_input( + &self, + input_partition: Partition, + scheduler: &mut dyn Scheduler, + ) -> error_stack::Result<(), PipelineError> { + debug_assert!(!self.0.is_empty(), "Inputs should be non-empty when used."); + + for input in self.0.iter() { + input.close_input(input_partition, scheduler)?; + } + Ok(()) + } +} + +impl InputHandle { + pub fn new(pipeline: Arc, input: usize) -> Self { + Self { pipeline, input } + } + + pub fn add_input( + &self, + input_partition: Partition, + batch: Batch, + scheduler: &mut dyn Scheduler, + ) -> error_stack::Result<(), PipelineError> { + self.pipeline + .add_input(input_partition, self.input, batch, scheduler) + } + + pub fn close_input( + &self, + input_partition: Partition, + scheduler: &mut dyn Scheduler, + ) -> error_stack::Result<(), PipelineError> { + self.pipeline + .close_input(input_partition, self.input, scheduler) + } +} diff --git a/crates/sparrow-scheduler/src/lib.rs b/crates/sparrow-scheduler/src/lib.rs index 0a14aa506..ab2cc437c 100644 --- a/crates/sparrow-scheduler/src/lib.rs +++ b/crates/sparrow-scheduler/src/lib.rs @@ -11,19 +11,20 @@ //! Scheduler for local, multi-threaded execution of Sparrow plans. mod error; +mod input_handles; mod partition; +mod pending; mod pipeline; mod queue; mod schedule_count; -mod sink; mod task; mod worker; mod worker_pool; pub use error::*; +pub use input_handles::*; pub use partition::*; pub use pipeline::*; -pub use sink::*; pub use task::*; pub use worker::*; pub use worker_pool::*; diff --git a/crates/sparrow-scheduler/src/pending.rs b/crates/sparrow-scheduler/src/pending.rs new file mode 100644 index 000000000..8c9b815ee --- /dev/null +++ b/crates/sparrow-scheduler/src/pending.rs @@ -0,0 +1,122 @@ +use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; +use std::sync::Arc; + +use hashbrown::HashMap; +use parking_lot::Mutex; + +use crate::Partition; + +/// Track the number of pending pipeline partitions. +#[derive(Default)] +pub(crate) struct PendingSet { + /// Map from pending partition to name of the transform. + /// + /// Used to report the currently pending partitions. Should not + /// be used for checking whether a specific partition is pending. + pending_partitions: Mutex>, + /// Count of the total pending partitions. + /// + /// Keeping this outside the mutex allows for fast checking of the current + /// count. + pending_partition_count: AtomicUsize, +} + +impl std::fmt::Debug for PendingSet { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let pending_partitions = self.pending_partitions.lock(); + f.debug_struct("PendingSet") + .field("pending_partitions", &pending_partitions) + .field("pending_partition_count", &self.pending_partition_count) + .finish() + } +} + +pub(crate) struct PendingPartition { + pending_set: Arc, + pub(crate) pipeline_index: usize, + pub(crate) partition: Partition, + is_complete: AtomicBool, +} + +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +struct PendingPartitionKey { + pipeline_index: usize, + partition: Partition, +} + +impl PendingSet { + pub fn add_pending( + self: &Arc, + pipeline_index: usize, + partition: Partition, + name: &'static str, + ) -> PendingPartition { + let key = PendingPartitionKey { + pipeline_index, + partition, + }; + let pending_partition = PendingPartition { + pending_set: self.clone(), + pipeline_index, + partition, + is_complete: AtomicBool::new(false), + }; + + let previous = self.pending_partitions.lock().insert(key, name); + debug_assert_eq!( + previous, None, + "Duplicate pipeline partition added to pending set" + ); + self.pending_partition_count.fetch_add(1, Ordering::Release); + + pending_partition + } + + pub fn pending_partition_count(&self) -> usize { + self.pending_partition_count.load(Ordering::Acquire) + } +} + +impl std::fmt::Debug for PendingPartition { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("PendingPartition") + .field("pipeline_index", &self.pipeline_index) + .field("partition", &self.partition) + .field("pending", &self.is_complete) + .finish_non_exhaustive() + } +} + +impl PendingPartition { + /// Record this pending partition completed. + pub fn complete(&self) { + let previous = self.is_complete.fetch_or(true, Ordering::AcqRel); + debug_assert!(!previous, "Task already completed"); + + let previous = self + .pending_set + .pending_partitions + .lock() + .remove(&PendingPartitionKey { + pipeline_index: self.pipeline_index, + partition: self.partition, + }); + + let previous = previous.expect("task should be in pending set"); + + let remaining = self + .pending_set + .pending_partition_count + .fetch_sub(1, Ordering::AcqRel); + tracing::info!( + "Completed partition {} of pipeline {previous} {}. {remaining} remaining.", + self.partition, + self.pipeline_index, + ); + } + + /// Return true if this partition is completed. + pub fn is_complete(&self) -> bool { + self.is_complete.load(Ordering::Acquire) + } +} diff --git a/crates/sparrow-scheduler/src/queue.rs b/crates/sparrow-scheduler/src/queue.rs index d1ce3e2c6..4eb7184f7 100644 --- a/crates/sparrow-scheduler/src/queue.rs +++ b/crates/sparrow-scheduler/src/queue.rs @@ -32,7 +32,7 @@ impl Clone for GlobalQueue { } } -impl GlobalQueue { +impl GlobalQueue { pub(crate) fn new(local_queues: usize, local_queue_size: u16) -> Self { Self { queue: work_queue::Queue::new(local_queues, local_queue_size), @@ -53,7 +53,7 @@ impl GlobalQueue { } } -impl LocalQueue { +impl LocalQueue { /// Pop an item from the local queue, or steal from the global and sibling queues if it is empty. pub fn pop(&mut self) -> Option { self.queue.pop() diff --git a/crates/sparrow-scheduler/src/sink.rs b/crates/sparrow-scheduler/src/sink.rs deleted file mode 100644 index b99533eca..000000000 --- a/crates/sparrow-scheduler/src/sink.rs +++ /dev/null @@ -1,37 +0,0 @@ -use std::sync::Arc; - -use sparrow_batch::Batch; - -use crate::{Partition, Pipeline, PipelineError, Scheduler}; - -/// A struct used for sending batches to a specific input port of a down-stream pipeline. -#[derive(Debug)] -pub struct PipelineInput { - pipeline: Arc, - input: usize, -} - -impl PipelineInput { - pub fn new(pipeline: Arc, input: usize) -> Self { - Self { pipeline, input } - } - - pub fn add_input( - &self, - partition: Partition, - batch: Batch, - scheduler: &mut dyn Scheduler, - ) -> error_stack::Result<(), PipelineError> { - self.pipeline - .add_input(partition, self.input, batch, scheduler) - } - - pub fn close_input( - &self, - input_partition: Partition, - scheduler: &mut dyn Scheduler, - ) -> error_stack::Result<(), PipelineError> { - self.pipeline - .close_input(input_partition, self.input, scheduler) - } -} diff --git a/crates/sparrow-scheduler/src/task.rs b/crates/sparrow-scheduler/src/task.rs index 23fbac6a4..c19501e00 100644 --- a/crates/sparrow-scheduler/src/task.rs +++ b/crates/sparrow-scheduler/src/task.rs @@ -2,8 +2,9 @@ use std::sync::Arc; use error_stack::ResultExt; +use crate::pending::PendingPartition; use crate::schedule_count::ScheduleCount; -use crate::{Error, Partition, Pipeline, Scheduler}; +use crate::{Error, Pipeline, Scheduler}; /// The unit of work executed by the scheduler. /// @@ -11,16 +12,14 @@ use crate::{Error, Partition, Pipeline, Scheduler}; /// single [Pipeline] and produces a single unit of output (typically a batch). #[derive(Debug)] pub struct Task { - /// Index of this pipeline. Used for debugging. - index: usize, + /// Entry recording the status (pending or not) of this task. + pending: PendingPartition, /// Name of the pipeline implementation. name: &'static str, /// The pipeline to execute. /// /// This is a weak reference to avoid cycles. pipeline: std::sync::Weak, - /// The partition of the pipeline to execute. - partition: Partition, /// An atomic counter tracking how many times the task has been submitted. /// /// This is reset after the task is executed. @@ -30,16 +29,14 @@ pub struct Task { impl Task { /// Create a new task executing the given pipeline and partition. pub(crate) fn new( - index: usize, + pending: PendingPartition, name: &'static str, pipeline: std::sync::Weak, - partition: Partition, ) -> Self { Self { - index, + pending, name, pipeline, - partition, schedule_count: ScheduleCount::default(), } } @@ -59,18 +56,18 @@ impl Task { fn pipeline(&self) -> error_stack::Result, Error> { Ok(self.pipeline.upgrade().ok_or(Error::PipelineDropped { - index: self.index, + index: self.pending.pipeline_index, name: self.name, - partition: self.partition, + partition: self.pending.partition, })?) } fn error(&self, method: &'static str) -> Error { Error::Pipeline { method, - index: self.index, + index: self.pending.pipeline_index, name: self.name, - partition: self.partition, + partition: self.pending.partition, } } @@ -81,10 +78,21 @@ impl Task { ) -> error_stack::Result { let guard = self.schedule_count.guard(); self.pipeline()? - .do_work(self.partition, scheduler) + .do_work(self.pending.partition, scheduler) .change_context_lazy(|| self.error("do_work"))?; Ok(guard.finish()) } + + /// Mark this task as completed. + /// + /// After this, it should not be scheduled nor should work be done. + pub fn complete(&self) { + self.pending.complete() + } + + pub fn is_complete(&self) -> bool { + self.pending.is_complete() + } } pub type TaskRef = Arc; diff --git a/crates/sparrow-scheduler/src/worker.rs b/crates/sparrow-scheduler/src/worker.rs index c6064cd24..315339378 100644 --- a/crates/sparrow-scheduler/src/worker.rs +++ b/crates/sparrow-scheduler/src/worker.rs @@ -1,3 +1,6 @@ +use std::sync::Arc; + +use crate::pending::PendingSet; use crate::{queue::*, Error, TaskRef}; pub trait Scheduler { @@ -71,15 +74,34 @@ pub struct Worker { impl Worker { /// Run the work loop to completion. - pub(crate) fn work_loop(mut self) -> error_stack::Result<(), Error> { - while let Some(task) = self.queue.pop() { - if task.do_work(&mut self)? { - // This means that the task was schedule while we were executing. - // As a result, we didn't add it to any queue yet, so we need to - // do so now. - self.queue.push_global(task); + pub(crate) fn work_loop( + mut self, + index: usize, + pending_set: Arc, + ) -> error_stack::Result<(), Error> { + loop { + while let Some(task) = self.queue.pop() { + tracing::info!("Running task: {task:?} on worker {index}"); + if task.do_work(&mut self)? { + // This means that the task was scheduled while we were executing. + // As a result, we didn't add it to any queue yet, so we need to + // do so now. + self.queue.push_global(task); + } + } + + let pending_count = pending_set.pending_partition_count(); + if pending_count == 0 { + break; + } else { + // Right now, this "busy-waits" by immediately trying to pull more work. + // This potentially leads to thread thrashing. We should instead call + // `thread::park` to park this thread, and call thread::unpark` when + // work is added to the global queue / back of the local queues. } } + + tracing::info!("All partitions completed. Shutting down worker {index}"); Ok(()) } } diff --git a/crates/sparrow-scheduler/src/worker_pool.rs b/crates/sparrow-scheduler/src/worker_pool.rs index 724eba7c4..b6d3076e6 100644 --- a/crates/sparrow-scheduler/src/worker_pool.rs +++ b/crates/sparrow-scheduler/src/worker_pool.rs @@ -1,7 +1,8 @@ use std::sync::Arc; +use crate::pending::PendingSet; use crate::worker::Injector; -use crate::{Error, Pipeline, Task, TaskRef}; +use crate::{Error, Partition, Pipeline, Task, TaskRef, Worker}; use error_stack::{IntoReport, ResultExt}; use itertools::Itertools; @@ -12,71 +13,39 @@ const DEFAULT_THREAD_COUNT: usize = 8; /// Number of slots each thread should have in it's local task queue. const LOCAL_QUEUE_SIZE: u16 = 32; -#[derive(Debug)] pub struct WorkerPool { query_id: String, injector: Injector, - handles: Vec>>, + workers: Vec, /// A vector of the pipelines we created. pipelines: Vec>, + /// Track which pipelines / partitions are still running. + pending: Arc, } impl WorkerPool { - pub fn start(query_id: String) -> error_stack::Result { + /// Create a worker pool. + /// + /// Args: + /// query_id: The query ID associated with this worker pool. Used as a + /// prefix in the tracing spans for each worker thread. + pub fn new(query_id: String) -> error_stack::Result { let core_ids = core_affinity::get_core_ids(); - let threads = core_ids - .as_ref() - .map(Vec::len) - .unwrap_or(DEFAULT_THREAD_COUNT); + let threads = core_ids.as_ref().map(Vec::len).unwrap_or_else(|| { + tracing::info!( + "No cores retrieved. Assuming default ({DEFAULT_THREAD_COUNT}) thread count" + ); + DEFAULT_THREAD_COUNT + }); let (injector, workers) = Injector::create(threads, LOCAL_QUEUE_SIZE); - let core_ids = core_ids - .into_iter() - .flatten() - .map(Some) - .chain(std::iter::repeat(None)); - let handles = workers - .into_iter() - .zip(core_ids) - .enumerate() - .map(|(index, (worker, core_id))| { - // Spawn the worker thread. - let span = tracing::info_span!("compute", query_id, index); - std::thread::Builder::new() - .name(format!("compute-{index}")) - .spawn(move || { - let _enter = span.enter(); - - // Set the core affinity, if possible, so this thread always - // executes on the same core. - if let Some(core_id) = core_id { - if core_affinity::set_for_current(core_id) { - tracing::info!( - "Set core affinity for thread {index} to {core_id:?}" - ); - } else { - tracing::info!( - "Failed to set core affinity for thread {index} to {core_id:?}" - ); - } - } else { - tracing::info!("Setting core affinity not supported"); - }; - - // Run the worker - worker.work_loop() - }) - .into_report() - .change_context(Error::SpawnWorker) - }) - .try_collect()?; - let scheduler = Self { query_id, injector, - handles, + workers, pipelines: vec![], + pending: Arc::new(PendingSet::default()), }; Ok(scheduler) } @@ -99,11 +68,15 @@ impl WorkerPool { // `new_cyclic` provides a `Weak` reference to the pipeline before it is // created. This allows us to create tasks that reference the pipeline // (via weak references) and pass those tasks to the pipeline. + let pending = self.pending.clone(); let pipeline: Arc = Arc::new_cyclic(move |weak| { let tasks = (0..partitions) .map(|partition| -> TaskRef { - let weak: std::sync::Weak = weak.clone(); - let task = Task::new(index, name, weak, partition.into()); + let pipeline: std::sync::Weak = weak.clone(); + let partition: Partition = partition.into(); + + let pending = pending.add_pending(index, partition, name); + let task = Task::new(pending, name, pipeline); Arc::new(task) }) .collect(); @@ -117,10 +90,85 @@ impl WorkerPool { }); let pipeline: Arc = pipeline; self.pipelines.push(pipeline.clone()); + + tracing::info!("Added {partitions} partitions for pipeline {index} {name}"); + pipeline } - pub fn stop(self) -> error_stack::Result<(), Error> { + /// Start executing the pipelines. + /// + /// Returns a `RunningWorkers` used for completing the workers. + pub fn start(self) -> error_stack::Result { + let Self { + pending, + workers, + query_id, + pipelines, + .. + } = self; + + let core_ids = core_affinity::get_core_ids(); + let core_ids = core_ids + .into_iter() + .flatten() + .map(Some) + .chain(std::iter::repeat(None)); + + let handles = workers + .into_iter() + .zip(core_ids) + .enumerate() + .map(|(index, (worker, core_id))| { + // Spawn the worker thread. + let span = tracing::info_span!("compute", query_id, index); + let pending = pending.clone(); + std::thread::Builder::new() + .name(format!("compute-{index}")) + .spawn(move || { + let _enter = span.enter(); + + // Set the core affinity, if possible, so this thread always + // executes on the same core. + if let Some(core_id) = core_id { + if core_affinity::set_for_current(core_id) { + tracing::info!( + "Set core affinity for thread {index} to {core_id:?}" + ); + } else { + tracing::info!( + "Failed to set core affinity for thread {index} to {core_id:?}" + ); + } + } else { + tracing::info!("Setting core affinity not supported"); + }; + + // Run the worker + worker.work_loop(index, pending.clone()) + }) + .into_report() + .change_context(Error::SpawnWorker) + }) + .try_collect()?; + + Ok(RunningWorkers { + query_id, + _pipelines: pipelines, + handles, + }) + } +} + +pub struct RunningWorkers { + query_id: String, + /// Hold the Arcs for the pipelines so they aren't dropped. + _pipelines: Vec>, + handles: Vec>>, +} + +impl RunningWorkers { + pub fn join(self) -> error_stack::Result<(), Error> { tracing::info!(self.query_id, "Waiting for completion of query"); for handle in self.handles { match handle.join() { diff --git a/crates/sparrow-session/Cargo.toml b/crates/sparrow-session/Cargo.toml index 42a51a779..e4daa2a6f 100644 --- a/crates/sparrow-session/Cargo.toml +++ b/crates/sparrow-session/Cargo.toml @@ -19,8 +19,12 @@ futures.workspace = true itertools.workspace = true smallvec.workspace = true sparrow-api = { path = "../sparrow-api" } +sparrow-backend = { path = "../sparrow-backend" } sparrow-compiler = { path = "../sparrow-compiler" } -sparrow-merge = { path = "../sparrow-merge" } +sparrow-execution = { path = "../sparrow-execution" } +sparrow-expressions = { path = "../sparrow-expressions" } +sparrow-interfaces = { path = "../sparrow-interfaces" } +sparrow-logical = { path = "../sparrow-logical" } sparrow-runtime = { path = "../sparrow-runtime" } sparrow-sources = { path = "../sparrow-sources" } sparrow-syntax = { path = "../sparrow-syntax" } diff --git a/crates/sparrow-session/src/execution.rs b/crates/sparrow-session/src/execution.rs index 74210df0d..594e7c9a4 100644 --- a/crates/sparrow-session/src/execution.rs +++ b/crates/sparrow-session/src/execution.rs @@ -1,9 +1,7 @@ use arrow_array::RecordBatch; use arrow_schema::SchemaRef; use futures::future::BoxFuture; -use futures::stream::BoxStream; use futures::StreamExt; -use sparrow_api::kaskada::v1alpha::ExecuteResponse; use crate::Error; @@ -15,7 +13,7 @@ pub struct Execution { // Future that resolves to the first error, if one occurred. status: Status, /// Stop signal. Send `true` to stop execution. - stop_signal_rx: tokio::sync::watch::Sender, + stop_signal_tx: tokio::sync::watch::Sender, pub schema: SchemaRef, } @@ -29,32 +27,19 @@ impl Execution { pub(super) fn new( handle: tokio::runtime::Handle, output_rx: tokio::sync::mpsc::Receiver, - progress: BoxStream<'static, error_stack::Result>, - stop_signal_rx: tokio::sync::watch::Sender, + future: BoxFuture<'static, error_stack::Result<(), Error>>, + stop_signal_tx: tokio::sync::watch::Sender, schema: SchemaRef, ) -> Self { let output = tokio_stream::wrappers::ReceiverStream::new(output_rx); // Constructs a futures that resolves to the first error, if one occurred. - let status = Status::Running(Box::pin(async move { - let mut errors = progress - .filter_map(|result| { - futures::future::ready(if let Err(e) = result { Some(e) } else { None }) - }) - .boxed(); - let first_error = errors.next().await; - if let Some(first_error) = first_error { - Err(first_error) - } else { - Ok(()) - } - })); - + let status = Status::Running(future); Self { handle, output, status, - stop_signal_rx, + stop_signal_tx, schema, } } @@ -96,7 +81,7 @@ impl Execution { /// /// This method does *not* wait for all batches to be processed. pub fn stop(&mut self) { - self.stop_signal_rx.send_if_modified(|stop| { + self.stop_signal_tx.send_if_modified(|stop| { *stop = true; true }); diff --git a/crates/sparrow-session/src/lib.rs b/crates/sparrow-session/src/lib.rs index ec84580f4..4678c0e0a 100644 --- a/crates/sparrow-session/src/lib.rs +++ b/crates/sparrow-session/src/lib.rs @@ -10,6 +10,7 @@ mod error; mod execution; mod expr; +pub mod partitioned; mod session; mod table; diff --git a/crates/sparrow-session/src/partitioned.rs b/crates/sparrow-session/src/partitioned.rs new file mode 100644 index 000000000..bfd66dd63 --- /dev/null +++ b/crates/sparrow-session/src/partitioned.rs @@ -0,0 +1,2 @@ +mod session; +pub use session::*; diff --git a/crates/sparrow-session/src/partitioned/session.rs b/crates/sparrow-session/src/partitioned/session.rs new file mode 100644 index 000000000..a3351397d --- /dev/null +++ b/crates/sparrow-session/src/partitioned/session.rs @@ -0,0 +1,124 @@ +use std::sync::Arc; + +use arrow_schema::{DataType, Schema}; +use error_stack::ResultExt; +use futures::FutureExt; +use hashbrown::HashMap; +use sparrow_compiler::NearestMatches; +use sparrow_interfaces::Source; +use sparrow_logical::{ExprRef, Literal}; +use uuid::Uuid; + +use crate::{Error, Execution, ExecutionOptions}; + +/// Session for creating and executing partitioned queries. +pub struct Session { + rt: tokio::runtime::Runtime, + sources: HashMap>, +} + +impl Default for Session { + fn default() -> Self { + Self { + rt: tokio::runtime::Runtime::new().expect("session"), + sources: HashMap::default(), + } + } +} + +impl Session { + pub fn add_source(&mut self, source: Arc) -> error_stack::Result { + let result_type = DataType::Struct(source.prepared_schema().fields.clone()); + + let mut uuid = uuid::Uuid::new_v4(); + while self.sources.contains_key(&uuid) { + uuid = uuid::Uuid::new_v4(); + } + self.sources.insert(uuid, source); + + Ok(Arc::new(sparrow_logical::Expr::new_uuid( + "read", + uuid, + result_type, + // TODO: Managing groupings of sources. + sparrow_logical::Grouping::new(0), + ))) + } + + pub fn add_literal(&self, literal: Literal) -> error_stack::Result { + Ok(Arc::new(sparrow_logical::Expr::new_literal(literal))) + } + + pub fn add_expr( + &self, + function: &str, + args: Vec, + ) -> error_stack::Result { + // TODO: Would be good to intern the names early (here) to be `&'static str`. + + let function = match function { + "record" => "record", + "fieldref" => "fieldref", + other => { + // TODO: This *should* use a list of available _logical_ functions. + // For now, we approximate that with the set of available _physical_ functions. + sparrow_expressions::intern_name(other).ok_or_else(|| Error::NoSuchFunction { + name: other.to_owned(), + nearest: NearestMatches::new_nearest_strings( + other, + sparrow_expressions::names().map(|s| s.to_owned()), + ), + })? + } + }; + let expr = sparrow_logical::Expr::try_new(function, args).change_context(Error::Invalid)?; + Ok(Arc::new(expr)) + } + + pub fn execute( + &self, + query: &ExprRef, + _options: ExecutionOptions, + ) -> error_stack::Result { + let plan = sparrow_backend::compile(query, None).change_context(Error::Compile)?; + + let (output_tx, output_rx) = tokio::sync::mpsc::channel(10); + let executor = sparrow_execution::PlanExecutor::try_new( + "query_id".to_owned(), + plan, + &self.sources, + output_tx, + ) + .change_context(Error::Execute)?; + + let (stop_signal_tx, stop_signal_rx) = tokio::sync::watch::channel(false); + + let handle = self.rt.handle().clone(); + + // TODO: Change `schema` to a `DataType` so we can output single columns. + // This was a legacy constraint from old compilation / execution. + let DataType::Struct(fields) = &query.result_type else { + panic!("Change `schema` to `data_type` in execution and support primitive output") + }; + let schema = Arc::new(Schema::new(fields.clone())); + + Ok(Execution::new( + handle, + output_rx, + executor + .execute(stop_signal_rx) + .map(|result| result.change_context(Error::ExecutionFailed)) + .boxed(), + stop_signal_tx, + schema, + )) + } + + /// Allow running a future on this sessions runtime. + /// + /// This is currently only exposed for some tests which need to run + /// futures adding data to sources without having a separate runtime. + pub fn block_on(&self, future: impl futures::Future) -> T { + self.rt.block_on(future) + } +} diff --git a/crates/sparrow-session/src/session.rs b/crates/sparrow-session/src/session.rs index 4231209cb..f95f1a3c2 100644 --- a/crates/sparrow-session/src/session.rs +++ b/crates/sparrow-session/src/session.rs @@ -563,11 +563,26 @@ impl Session { .map_err(|e| e.change_context(Error::Execute)) .boxed(); + // Create a future that resolves to either `Err(first_error)` or `Ok`. + let future = Box::pin(async move { + let mut errors = progress + .filter_map(|result| { + futures::future::ready(if let Err(e) = result { Some(e) } else { None }) + }) + .boxed(); + let first_error = errors.next().await; + if let Some(first_error) = first_error { + Err(first_error) + } else { + Ok(()) + } + }); + let handle = self.rt.handle().clone(); Ok(Execution::new( handle, output_rx, - progress, + future, stop_signal_tx, schema, )) diff --git a/crates/sparrow-sources/Cargo.toml b/crates/sparrow-sources/Cargo.toml index 75414dbe3..65b6283ec 100644 --- a/crates/sparrow-sources/Cargo.toml +++ b/crates/sparrow-sources/Cargo.toml @@ -6,7 +6,7 @@ edition.workspace = true license.workspace = true publish = false description = """ -Defines the input source implementations. +Defines the input source implementations. """ [dependencies] @@ -14,10 +14,8 @@ arrow-array.workspace = true async-broadcast.workspace = true async-stream.workspace = true arrow-schema.workspace = true -derive_more.workspace = true error-stack.workspace = true futures.workspace = true -sparrow-core = { path = "../sparrow-core" } sparrow-merge = { path = "../sparrow-merge" } sparrow-batch = { path = "../sparrow-batch" } sparrow-interfaces = { path = "../sparrow-interfaces" } diff --git a/crates/sparrow-sources/src/in_memory.rs b/crates/sparrow-sources/src/in_memory.rs index e6b951683..c6d405a17 100644 --- a/crates/sparrow-sources/src/in_memory.rs +++ b/crates/sparrow-sources/src/in_memory.rs @@ -39,6 +39,10 @@ impl InMemory { } impl Source for InMemory { + fn as_any(&self) -> &dyn std::any::Any { + self + } + fn prepared_schema(&self) -> SchemaRef { self.prepared_schema.clone() } @@ -46,8 +50,8 @@ impl Source for InMemory { fn read( &self, projected_datatype: &DataType, - read_config: ReadConfig, - ) -> futures::stream::BoxStream<'_, error_stack::Result> { + read_config: Arc, + ) -> futures::stream::BoxStream<'static, error_stack::Result> { assert_eq!( &DataType::Struct(self.prepared_schema().fields().clone()), projected_datatype, diff --git a/crates/sparrow-transforms/src/project.rs b/crates/sparrow-transforms/src/project.rs index e1a358cad..96311e0da 100644 --- a/crates/sparrow-transforms/src/project.rs +++ b/crates/sparrow-transforms/src/project.rs @@ -97,13 +97,13 @@ mod tests { result_type: input_type.clone(), }, sparrow_physical::Expr { - name: "field_ref".into(), + name: "fieldref".into(), literal_args: vec![ScalarValue::Utf8(Some("a".to_owned()))], args: vec![0.into()], result_type: DataType::Int64, }, sparrow_physical::Expr { - name: "field_ref".into(), + name: "fieldref".into(), literal_args: vec![ScalarValue::Utf8(Some("b".to_owned()))], args: vec![0.into()], result_type: DataType::Int64, diff --git a/crates/sparrow-transforms/src/select.rs b/crates/sparrow-transforms/src/select.rs index 59f4c4c2f..d0fd1a827 100644 --- a/crates/sparrow-transforms/src/select.rs +++ b/crates/sparrow-transforms/src/select.rs @@ -141,13 +141,13 @@ mod tests { result_type: input_type.clone(), }, sparrow_physical::Expr { - name: "field_ref".into(), + name: "fieldref".into(), literal_args: vec![ScalarValue::Utf8(Some("a".to_owned()))], args: vec![0.into()], result_type: DataType::Int64, }, sparrow_physical::Expr { - name: "field_ref".into(), + name: "fieldref".into(), literal_args: vec![ScalarValue::Utf8(Some("b".to_owned()))], args: vec![0.into()], result_type: DataType::Int64, diff --git a/crates/sparrow-transforms/src/transform_pipeline.rs b/crates/sparrow-transforms/src/transform_pipeline.rs index 7332fbf60..72659a5c2 100644 --- a/crates/sparrow-transforms/src/transform_pipeline.rs +++ b/crates/sparrow-transforms/src/transform_pipeline.rs @@ -7,7 +7,7 @@ use parking_lot::Mutex; use sparrow_batch::Batch; use sparrow_physical::{StepId, StepKind}; use sparrow_scheduler::{ - Partition, Partitioned, Pipeline, PipelineError, PipelineInput, Scheduler, TaskRef, + InputHandles, Partition, Partitioned, Pipeline, PipelineError, Scheduler, TaskRef, }; use crate::transform::Transform; @@ -17,8 +17,8 @@ pub struct TransformPipeline { /// The state for each partition. partitions: Partitioned, transforms: Vec>, - /// Sink for the down-stream computation. - sink: PipelineInput, + /// Input handles for the consuming (receiving) computations. + consumers: InputHandles, } impl std::fmt::Debug for TransformPipeline { @@ -82,12 +82,21 @@ pub enum Error { impl error_stack::Context for Error {} impl TransformPipeline { + /// Create a new transform pipeline. + /// + /// Args: + /// first_step_input_id: The `StepId` of the step that produces input to + /// this pipeline. It should be the only input step to the first step in + /// `steps`. + /// steps: Iterator over the steps (in order) comprising the pipeline. + /// They should all be transforms. + /// consumers: The `InputHandles` to output the result of the transform to. pub fn try_new<'a>( - input_step: &sparrow_physical::Step, + first_step_input_id: StepId, steps: impl Iterator + ExactSizeIterator, - sink: PipelineInput, + consumers: InputHandles, ) -> error_stack::Result { - let mut input_step = input_step; + let mut input_step_id = first_step_input_id; let mut transforms = Vec::with_capacity(steps.len()); for step in steps { error_stack::ensure!( @@ -98,9 +107,9 @@ impl TransformPipeline { } ); error_stack::ensure!( - step.inputs[0] == input_step.id, + step.inputs[0] == input_step_id, Error::UnexpectedInput { - expected: input_step.id, + expected: input_step_id, actual: step.inputs[0] } ); @@ -125,12 +134,12 @@ impl TransformPipeline { } }; transforms.push(transform); - input_step = step; + input_step_id = step.id; } Ok(Self { partitions: Partitioned::default(), transforms, - sink, + consumers, }) } } @@ -169,7 +178,6 @@ impl Pipeline for TransformPipeline { input_partition } ); - partition.add_input(batch); scheduler.schedule(partition.task.clone()); Ok(()) @@ -218,7 +226,7 @@ impl Pipeline for TransformPipeline { partition.is_input_closed(), PipelineError::illegal_state("scheduled without work") ); - return self.sink.close_input(input_partition, scheduler); + return self.consumers.close_input(input_partition, scheduler); }; tracing::trace!( @@ -244,7 +252,7 @@ impl Pipeline for TransformPipeline { // If the result is non-empty, output it. if !batch.is_empty() { - self.sink + self.consumers .add_input(input_partition, batch, scheduler) .change_context(PipelineError::Execution)?; } @@ -252,9 +260,11 @@ impl Pipeline for TransformPipeline { // If the input is closed and empty, then we should close the sink. if partition.is_input_closed() && partition.is_input_empty() { - self.sink + tracing::info!("Input is closed and empty. Closing consumers"); + self.consumers .close_input(input_partition, scheduler) .change_context(PipelineError::Execution)?; + partition.task.complete(); } // Note: We don't re-schedule the transform if there is input. diff --git a/python/Cargo.lock b/python/Cargo.lock index 4ca113d45..fa67a0130 100644 --- a/python/Cargo.lock +++ b/python/Cargo.lock @@ -3695,6 +3695,20 @@ dependencies = [ "tracing", ] +[[package]] +name = "sparrow-batch" +version = "0.11.0" +dependencies = [ + "arrow", + "arrow-array", + "arrow-schema", + "arrow-select", + "derive_more", + "error-stack", + "itertools 0.11.0", + "static_init", +] + [[package]] name = "sparrow-compiler" version = "0.11.0" @@ -3724,6 +3738,7 @@ dependencies = [ "sparrow-core", "sparrow-instructions", "sparrow-merge", + "sparrow-sources", "sparrow-syntax", "static_init", "strum 0.25.0", @@ -3784,6 +3799,18 @@ dependencies = [ "uuid 1.4.1", ] +[[package]] +name = "sparrow-interfaces" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "derive_more", + "error-stack", + "futures", + "sparrow-batch", + "sparrow-core", +] + [[package]] name = "sparrow-kernels" version = "0.11.0" @@ -3912,6 +3939,7 @@ dependencies = [ "sparrow-instructions", "sparrow-merge", "sparrow-runtime", + "sparrow-sources", "sparrow-syntax", "static_init", "tempfile", @@ -3920,6 +3948,24 @@ dependencies = [ "uuid 1.4.1", ] +[[package]] +name = "sparrow-sources" +version = "0.11.0" +dependencies = [ + "arrow-array", + "arrow-schema", + "async-broadcast", + "async-stream", + "derive_more", + "error-stack", + "futures", + "sparrow-batch", + "sparrow-core", + "sparrow-interfaces", + "sparrow-merge", + "tracing", +] + [[package]] name = "sparrow-syntax" version = "0.11.0" From 2faf0ccdd70e6f2bf3f65e6b256689f469fd9f50 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Fri, 13 Oct 2023 09:24:46 -0700 Subject: [PATCH 02/14] comments --- crates/sparrow-backend/src/logical_to_physical.rs | 9 ++++++--- crates/sparrow-execution/src/write_channel_pipeline.rs | 3 +++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/crates/sparrow-backend/src/logical_to_physical.rs b/crates/sparrow-backend/src/logical_to_physical.rs index ea89a5118..21c0ef93c 100644 --- a/crates/sparrow-backend/src/logical_to_physical.rs +++ b/crates/sparrow-backend/src/logical_to_physical.rs @@ -149,10 +149,13 @@ impl LogicalToPhysical { // Start with replacement containing `?input => (fieldref ?input "step_)") let mut exprs = ExprPattern::new_input()?; + // `new_input` adds `?input` as the first expression in the pattern. + let input_id = egg::Id::from(0); + let data_type = self.reference_type(&arg)?.clone(); if let Some(input_step) = arg.step_id { - exprs.add_instruction( + let merged_input_id = exprs.add_instruction( "fieldref", // Note: that `merge` should produce a record with a // field for each merged step, identified by the @@ -165,13 +168,13 @@ impl LogicalToPhysical { // step ID in the inputs" which would be more stable // as we change step IDs. smallvec![ScalarValue::Utf8(Some(format!("step_{}", input_step)))], - smallvec![egg::Id::from(0)], + smallvec![input_id], data_type, )?; // Then add the actual expression, replacing the `?input` with the fieldref. let mut subst = egg::Subst::with_capacity(1); - subst.insert(*crate::exprs::INPUT_VAR, egg::Id::from(1)); + subst.insert(*crate::exprs::INPUT_VAR, merged_input_id); exprs.add_pattern(&arg.expr, &subst)?; } else { exprs.add_pattern(&arg.expr, &egg::Subst::default())?; diff --git a/crates/sparrow-execution/src/write_channel_pipeline.rs b/crates/sparrow-execution/src/write_channel_pipeline.rs index e10744226..04754e4ff 100644 --- a/crates/sparrow-execution/src/write_channel_pipeline.rs +++ b/crates/sparrow-execution/src/write_channel_pipeline.rs @@ -38,6 +38,9 @@ impl Pipeline for WriteChannelPipeline { batch: Batch, _scheduler: &mut dyn sparrow_scheduler::Scheduler, ) -> error_stack::Result<(), PipelineError> { + // HACK: This converts `Batch` to `RecordBatch` because the current execution logic + // expects `RecordBatch` outputs. This should be changed to standardize on `Batch` + // which makes it easier to carry a primitive value out. if let Some(batch) = batch.into_record_batch(self.schema.clone()) { let channel = self.channel.lock(); channel From b9797e2c671edfc85ce763b580e264c2512ac4a4 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Fri, 13 Oct 2023 11:21:16 -0700 Subject: [PATCH 03/14] monitor thread panics --- Cargo.lock | 2 + crates/sparrow-execution/src/lib.rs | 2 +- crates/sparrow-execution/src/tests.rs | 8 +- crates/sparrow-scheduler/Cargo.toml | 2 + crates/sparrow-scheduler/src/error.rs | 4 +- crates/sparrow-scheduler/src/lib.rs | 1 + crates/sparrow-scheduler/src/monitor.rs | 35 ++++++ crates/sparrow-scheduler/src/pending.rs | 16 +++ crates/sparrow-scheduler/src/task.rs | 10 ++ crates/sparrow-scheduler/src/worker_pool.rs | 112 +++++++++++++++++- .../src/transform_pipeline.rs | 36 ++++-- 11 files changed, 201 insertions(+), 27 deletions(-) create mode 100644 crates/sparrow-scheduler/src/monitor.rs diff --git a/Cargo.lock b/Cargo.lock index 6932cd87d..1c6749579 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4739,6 +4739,8 @@ dependencies = [ "serde", "smallvec", "sparrow-batch", + "sparrow-testing", + "tokio", "tracing", "work-queue", ] diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index e8cd7380e..eb5feec16 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -182,7 +182,7 @@ impl PlanExecutor { let workers = worker_pool.start().change_context(Error::Starting)?; source_tasks.run_sources(injector).await?; - workers.join().change_context(Error::Stopping)?; + workers.join().await.change_context(Error::Stopping)?; Ok(()) } diff --git a/crates/sparrow-execution/src/tests.rs b/crates/sparrow-execution/src/tests.rs index d44da850c..44d70ff19 100644 --- a/crates/sparrow-execution/src/tests.rs +++ b/crates/sparrow-execution/src/tests.rs @@ -60,7 +60,7 @@ fn add_input_batch(session: &Session, source: &Arc) { } #[test] -fn test_logical_query() { +fn test_logical_query_data_before_execute() { sparrow_testing::init_test_logging(); let mut session = Session::default(); @@ -73,17 +73,13 @@ fn test_logical_query() { let source: Arc = Arc::new(InMemory::new(true, source_schema.clone()).unwrap()); let source_expr = session.add_source(source.clone()).unwrap(); - // Add some data before running the query. - // Note this uses an odd downcast because we've "erased" the type to `dyn Expr`. - {} + add_input_batch(&session, &source); let query = query(&session, source_expr).unwrap(); let execution = session .execute(&query, sparrow_session::ExecutionOptions::default()) .unwrap(); - add_input_batch(&session, &source); - let output = execution.collect_all_blocking().unwrap(); assert_eq!(output.len(), 1); let output = output.into_iter().next().unwrap(); diff --git a/crates/sparrow-scheduler/Cargo.toml b/crates/sparrow-scheduler/Cargo.toml index 98575ba65..9aabf253a 100644 --- a/crates/sparrow-scheduler/Cargo.toml +++ b/crates/sparrow-scheduler/Cargo.toml @@ -21,6 +21,7 @@ serde.workspace = true smallvec.workspace = true sparrow-batch = { path = "../sparrow-batch" } tracing.workspace = true +tokio.workspace = true work-queue = "0.1.4" [package.metadata.cargo-machete] @@ -28,6 +29,7 @@ work-queue = "0.1.4" ignored = ["serde"] [dev-dependencies] +sparrow-testing = { path = "../sparrow-testing" } [target.'cfg(loom)'.dependencies] loom = "0.6.0" diff --git a/crates/sparrow-scheduler/src/error.rs b/crates/sparrow-scheduler/src/error.rs index 210610d30..1ee9e707d 100644 --- a/crates/sparrow-scheduler/src/error.rs +++ b/crates/sparrow-scheduler/src/error.rs @@ -3,8 +3,8 @@ use crate::Partition; /// Top level errors reported during partitioned pipeline execution. #[derive(derive_more::Display, Debug)] pub enum Error { - #[display(fmt = "worker panicked")] - WorkerPanicked, + #[display(fmt = "pipeline panicked")] + PipelinePanic, #[display(fmt = "spawning worker")] SpawnWorker, #[display( diff --git a/crates/sparrow-scheduler/src/lib.rs b/crates/sparrow-scheduler/src/lib.rs index ab2cc437c..aee9ddf29 100644 --- a/crates/sparrow-scheduler/src/lib.rs +++ b/crates/sparrow-scheduler/src/lib.rs @@ -12,6 +12,7 @@ mod error; mod input_handles; +mod monitor; mod partition; mod pending; mod pipeline; diff --git a/crates/sparrow-scheduler/src/monitor.rs b/crates/sparrow-scheduler/src/monitor.rs new file mode 100644 index 000000000..6b5c48fcd --- /dev/null +++ b/crates/sparrow-scheduler/src/monitor.rs @@ -0,0 +1,35 @@ +use std::thread::ThreadId; + +/// Create a monitor that watches for thread completions. +pub(crate) struct Monitor { + tx: tokio::sync::mpsc::Sender, + pub(crate) rx: tokio::sync::mpsc::Receiver, +} + +pub(crate) struct MonitorGuard { + tx: tokio::sync::mpsc::Sender, +} + +impl Drop for MonitorGuard { + fn drop(&mut self) { + match self.tx.blocking_send(std::thread::current().id()) { + Ok(_) => (), + Err(thread_id) => { + tracing::error!("Failed to send thread completion for {thread_id:?}"); + } + } + } +} + +impl Monitor { + pub fn with_capacity(capacity: usize) -> Self { + let (tx, rx) = tokio::sync::mpsc::channel(capacity); + Monitor { tx, rx } + } + + pub fn guard(&self) -> MonitorGuard { + MonitorGuard { + tx: self.tx.clone(), + } + } +} diff --git a/crates/sparrow-scheduler/src/pending.rs b/crates/sparrow-scheduler/src/pending.rs index 8c9b815ee..9faf389c2 100644 --- a/crates/sparrow-scheduler/src/pending.rs +++ b/crates/sparrow-scheduler/src/pending.rs @@ -2,6 +2,7 @@ use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use std::sync::Arc; use hashbrown::HashMap; +use itertools::Itertools; use parking_lot::Mutex; use crate::Partition; @@ -31,6 +32,20 @@ impl std::fmt::Debug for PendingSet { } } +impl std::fmt::Display for PendingSet { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let pending = self.pending_partitions.lock(); + write!( + f, + "[{}]", + pending.iter().format_with(", ", |elt, f| f(&format_args!( + "{}({}) partition {}", + elt.1, elt.0.pipeline_index, elt.0.partition + ))) + ) + } +} + pub(crate) struct PendingPartition { pending_set: Arc, pub(crate) pipeline_index: usize, @@ -113,6 +128,7 @@ impl PendingPartition { self.partition, self.pipeline_index, ); + tracing::trace!("Remaining pipelines: {}", self.pending_set); } /// Return true if this partition is completed. diff --git a/crates/sparrow-scheduler/src/task.rs b/crates/sparrow-scheduler/src/task.rs index c19501e00..67902ea3b 100644 --- a/crates/sparrow-scheduler/src/task.rs +++ b/crates/sparrow-scheduler/src/task.rs @@ -26,6 +26,16 @@ pub struct Task { schedule_count: ScheduleCount, } +impl std::fmt::Display for Task { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "{}({}) partition {}", + self.name, self.pending.pipeline_index, self.pending.partition + ) + } +} + impl Task { /// Create a new task executing the given pipeline and partition. pub(crate) fn new( diff --git a/crates/sparrow-scheduler/src/worker_pool.rs b/crates/sparrow-scheduler/src/worker_pool.rs index b6d3076e6..9500c6e97 100644 --- a/crates/sparrow-scheduler/src/worker_pool.rs +++ b/crates/sparrow-scheduler/src/worker_pool.rs @@ -1,9 +1,12 @@ use std::sync::Arc; +use std::thread::ThreadId; +use crate::monitor::Monitor; use crate::pending::PendingSet; use crate::worker::Injector; use crate::{Error, Partition, Pipeline, Task, TaskRef, Worker}; use error_stack::{IntoReport, ResultExt}; +use hashbrown::HashMap; use itertools::Itertools; /// Default thread count to use if we aren't able to determine @@ -115,6 +118,7 @@ impl WorkerPool { .map(Some) .chain(std::iter::repeat(None)); + let monitor = Monitor::with_capacity(workers.len()); let handles = workers .into_iter() .zip(core_ids) @@ -123,6 +127,7 @@ impl WorkerPool { // Spawn the worker thread. let span = tracing::info_span!("compute", query_id, index); let pending = pending.clone(); + let guard = monitor.guard(); std::thread::Builder::new() .name(format!("compute-{index}")) .spawn(move || { @@ -145,17 +150,24 @@ impl WorkerPool { }; // Run the worker - worker.work_loop(index, pending.clone()) + let result = worker.work_loop(index, pending.clone()); + + // Make sure the monitor guard is moved into this thread + // and dropped before returning. + std::mem::drop(guard); + result }) .into_report() .change_context(Error::SpawnWorker) }) + .map_ok(|handle| (handle.thread().id(), handle)) .try_collect()?; Ok(RunningWorkers { query_id, _pipelines: pipelines, handles, + finishing_threads: monitor.rx, }) } } @@ -164,21 +176,36 @@ pub struct RunningWorkers { query_id: String, /// Hold the Arcs for the pipelines so they aren't dropped. _pipelines: Vec>, - handles: Vec>>, + handles: HashMap>>, + finishing_threads: tokio::sync::mpsc::Receiver, } impl RunningWorkers { - pub fn join(self) -> error_stack::Result<(), Error> { + pub async fn join(mut self) -> error_stack::Result<(), Error> { tracing::info!(self.query_id, "Waiting for completion of query"); - for handle in self.handles { + while let Some(finished) = self.finishing_threads.recv().await { + let handle = self + .handles + .remove(&finished) + // This should only happen if a spawned thread had a `MonitorGuard` + // but was not added to `handles`. This should not happen. + .expect("Finished unregistered handle"); + match handle.join() { Ok(worker_result) => worker_result?, Err(_) => { - error_stack::bail!(Error::WorkerPanicked) + error_stack::bail!(Error::PipelinePanic) } } } + // This should only happen if a spawned thread was added to the handles + // but did not register a `MonitorGuard`. This should not happen. + assert!( + self.handles.is_empty(), + "Not all handles reported completion via monitor" + ); + Ok(()) } } @@ -188,3 +215,78 @@ impl RunningWorkers { pub struct CreateError(&'static str); impl error_stack::Context for CreateError {} + +#[cfg(test)] +mod tests { + use sparrow_batch::{Batch, RowTime}; + + use crate::{ + Error, Partition, Partitioned, Pipeline, PipelineError, Scheduler, TaskRef, WorkerPool, + }; + + #[derive(Debug, Default)] + struct PanicPipeline { + tasks: Partitioned, + } + + impl Pipeline for PanicPipeline { + fn initialize(&mut self, tasks: Partitioned) { + self.tasks = tasks; + } + + fn add_input( + &self, + input_partition: Partition, + _input: usize, + _batch: Batch, + scheduler: &mut dyn Scheduler, + ) -> error_stack::Result<(), PipelineError> { + scheduler.schedule(self.tasks[input_partition].clone()); + Ok(()) + } + + fn close_input( + &self, + _input_partition: Partition, + _input: usize, + _scheduler: &mut dyn Scheduler, + ) -> error_stack::Result<(), PipelineError> { + unreachable!("Should panic before closing"); + } + + fn do_work( + &self, + _partition: Partition, + _scheduler: &mut dyn Scheduler, + ) -> error_stack::Result<(), PipelineError> { + panic!("PanicPipeline is meant to panic"); + } + } + + #[tokio::test] + async fn test_pipeline_panic() { + sparrow_testing::init_test_logging(); + + let mut workers = WorkerPool::new("query".to_owned()).unwrap(); + let pipeline = workers.add_pipeline(1, PanicPipeline::default()); + let mut injector = workers.injector().clone(); + let workers = workers.start().unwrap(); + + pipeline + .add_input( + 0.into(), + 0, + Batch::new_empty(RowTime::from_timestamp_ns(73)), + &mut injector, + ) + .unwrap(); + let result = workers.join().await; + assert!(result.is_err(), "Expected {result:?} to be an error."); + let error = result.unwrap_err(); + let error = error.current_context(); + assert!( + matches!(error, Error::PipelinePanic), + "Expected {error:?} to be pipeline panic error" + ); + } +} diff --git a/crates/sparrow-transforms/src/transform_pipeline.rs b/crates/sparrow-transforms/src/transform_pipeline.rs index 72659a5c2..31830b8a4 100644 --- a/crates/sparrow-transforms/src/transform_pipeline.rs +++ b/crates/sparrow-transforms/src/transform_pipeline.rs @@ -45,12 +45,12 @@ struct TransformPartition { impl TransformPartition { /// Close the input. Returns true if the input buffer is empty. - fn close_input(&self) -> bool { + fn close(&self) -> bool { self.is_closed.store(true, Ordering::Release); self.inputs.lock().is_empty() } - fn is_input_closed(&self) -> bool { + fn is_closed(&self) -> bool { self.is_closed.load(Ordering::Acquire) } @@ -172,7 +172,7 @@ impl Pipeline for TransformPipeline { ); let partition = &self.partitions[input_partition]; error_stack::ensure!( - !partition.is_input_closed(), + !partition.is_closed(), PipelineError::InputClosed { input, input_partition @@ -198,17 +198,18 @@ impl Pipeline for TransformPipeline { ); let partition = &self.partitions[input_partition]; error_stack::ensure!( - !partition.is_input_closed(), + !partition.is_closed(), PipelineError::InputClosed { input, input_partition } ); + tracing::trace!("Closing input for {}", partition.task); // Don't close the sink here. We may be currently executing a `do_work` // loop, in which case we need to allow it to output to the sink before // we close it. - partition.close_input(); + partition.close(); scheduler.schedule(partition.task.clone()); Ok(()) @@ -223,10 +224,13 @@ impl Pipeline for TransformPipeline { let Some(batch) = partition.pop_input() else { error_stack::ensure!( - partition.is_input_closed(), + partition.is_closed(), PipelineError::illegal_state("scheduled without work") ); - return self.consumers.close_input(input_partition, scheduler); + tracing::info!("Input is closed and empty. Closing consumers and finishing pipeline."); + self.consumers.close_input(input_partition, scheduler)?; + partition.task.complete(); + return Ok(()); }; tracing::trace!( @@ -259,12 +263,18 @@ impl Pipeline for TransformPipeline { } // If the input is closed and empty, then we should close the sink. - if partition.is_input_closed() && partition.is_input_empty() { - tracing::info!("Input is closed and empty. Closing consumers"); - self.consumers - .close_input(input_partition, scheduler) - .change_context(PipelineError::Execution)?; - partition.task.complete(); + if partition.is_closed() { + if partition.is_input_empty() { + tracing::info!( + "Input is closed and empty. Closing consumers and finishing pipeline." + ); + self.consumers + .close_input(input_partition, scheduler) + .change_context(PipelineError::Execution)?; + partition.task.complete(); + } else { + tracing::trace!("Input is closed but not empty.") + } } // Note: We don't re-schedule the transform if there is input. From 5e65b9bb786787572e5ae92349449e718282e42d Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Fri, 13 Oct 2023 14:43:57 -0700 Subject: [PATCH 04/14] line up output schemas --- crates/sparrow-batch/src/batch.rs | 2 ++ crates/sparrow-execution/src/lib.rs | 1 + 2 files changed, 3 insertions(+) diff --git a/crates/sparrow-batch/src/batch.rs b/crates/sparrow-batch/src/batch.rs index a94800f48..4322216f5 100644 --- a/crates/sparrow-batch/src/batch.rs +++ b/crates/sparrow-batch/src/batch.rs @@ -79,6 +79,8 @@ impl Batch { pub fn into_record_batch(self, schema: Arc) -> Option { self.data.map(|data| { + println!("Schema: {schema:?}"); + println!("Data: {:?}", data.data.data_type()); if let Some(fields) = data.data.as_struct_opt() { let mut columns = Vec::with_capacity(3 + fields.num_columns()); columns.extend_from_slice(&[data.time, data.subsort, data.key_hash]); diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index eb5feec16..95068f7e5 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -48,6 +48,7 @@ fn result_type_to_output_schema(result_type: &DataType) -> SchemaRef { DataType::Timestamp(TimeUnit::Nanosecond, None), false, )), + Arc::new(Field::new("_subsort", DataType::UInt64, false)), // TODO: Convert key hash to key. Arc::new(Field::new("_key_hash", DataType::UInt64, false)), ]; From ced7840d0d0ba0afd47c64ffb880aa7695e44411 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Fri, 13 Oct 2023 14:51:22 -0700 Subject: [PATCH 05/14] some comments --- .../sparrow-backend/src/pipeline_schedule.rs | 2 +- crates/sparrow-execution/src/lib.rs | 16 +++---- crates/sparrow-scheduler/src/input_handles.rs | 45 +++++-------------- crates/sparrow-transforms/src/lib.rs | 2 +- .../src/transform_pipeline.rs | 8 ++-- 5 files changed, 25 insertions(+), 48 deletions(-) diff --git a/crates/sparrow-backend/src/pipeline_schedule.rs b/crates/sparrow-backend/src/pipeline_schedule.rs index 066467aeb..ff0578f0b 100644 --- a/crates/sparrow-backend/src/pipeline_schedule.rs +++ b/crates/sparrow-backend/src/pipeline_schedule.rs @@ -64,7 +64,7 @@ fn is_pipeline_breaker( true } _ if !steps[step.inputs[0]].kind.is_transform() => { - tracing::trace!("Step {index} is a new pipeline since it's input is not a trasnform"); + tracing::trace!("Step {index} is a new pipeline since it's input is not a transform"); true } _ if references[step.inputs[0]] > 1 => { diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index 95068f7e5..e310d1d10 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -125,15 +125,9 @@ impl PlanExecutor { 1, "Transforms should have a single input" ); - let first_step_input_id = first_step_inputs[0]; // If all of the steps are transforms, then we use the transform pipeline. - executor.add_transform_pipeline( - first_step_input_id, - &plan, - &pipeline.steps, - consumers, - )? + executor.add_transform_pipeline(&plan, &pipeline.steps, consumers)? } else { assert_eq!( pipeline.steps.len(), @@ -215,7 +209,6 @@ impl PlanExecutor { /// Convert a physical plan Pipeline into the executable scheduler Pipeline. fn add_transform_pipeline( &mut self, - first_step_input_id: sparrow_physical::StepId, plan: &sparrow_physical::Plan, steps: &[sparrow_physical::StepId], consumers: InputHandles, @@ -228,8 +221,13 @@ impl PlanExecutor { }) ); + // Determine the producer of the + let first_step_inputs = &plan.steps[*steps.first().expect("at least one step")].inputs; + debug_assert_eq!(first_step_inputs.len(), 1); + let producer_id = first_step_inputs[0]; + let steps = steps.iter().map(|id| &plan.steps[*id]); - let pipeline = TransformPipeline::try_new(first_step_input_id, steps, consumers) + let pipeline = TransformPipeline::try_new(producer_id, steps, consumers) .change_context(Error::Creating)?; Ok(self.worker_pool.add_pipeline(1, pipeline)) } diff --git a/crates/sparrow-scheduler/src/input_handles.rs b/crates/sparrow-scheduler/src/input_handles.rs index 0389d0627..095c551c4 100644 --- a/crates/sparrow-scheduler/src/input_handles.rs +++ b/crates/sparrow-scheduler/src/input_handles.rs @@ -4,6 +4,9 @@ use sparrow_batch::Batch; use crate::{Partition, Pipeline, PipelineError, Scheduler}; +#[derive(Debug, Default)] +pub struct InputHandles(smallvec::SmallVec<[InputHandle; 1]>); + /// A struct used for sending batches to a specific input port of a down-stream pipeline. #[derive(Debug)] struct InputHandle { @@ -11,9 +14,6 @@ struct InputHandle { input: usize, } -#[derive(Debug, Default)] -pub struct InputHandles(smallvec::SmallVec<[InputHandle; 1]>); - impl InputHandles { pub fn is_empty(&self) -> bool { self.0.is_empty() @@ -24,7 +24,7 @@ impl InputHandles { } pub fn add_consumer(&mut self, pipeline: Arc, input: usize) { - self.0.push(InputHandle::new(pipeline, input)); + self.0.push(InputHandle { pipeline, input }); } pub fn add_input( @@ -41,8 +41,10 @@ impl InputHandles { // first consumer that needs to be woken on the local queue, we could place the // others on the global queue. This would cause the batch to move to another // core, but would let both consumers run in parallel. - for input in self.0.iter() { - input.add_input(input_partition, batch.clone(), scheduler)?; + for handle in self.0.iter() { + handle + .pipeline + .add_input(input_partition, handle.input, batch.clone(), scheduler)?; } Ok(()) } @@ -54,34 +56,11 @@ impl InputHandles { ) -> error_stack::Result<(), PipelineError> { debug_assert!(!self.0.is_empty(), "Inputs should be non-empty when used."); - for input in self.0.iter() { - input.close_input(input_partition, scheduler)?; + for handle in self.0.iter() { + handle + .pipeline + .close_input(input_partition, handle.input, scheduler)?; } Ok(()) } } - -impl InputHandle { - pub fn new(pipeline: Arc, input: usize) -> Self { - Self { pipeline, input } - } - - pub fn add_input( - &self, - input_partition: Partition, - batch: Batch, - scheduler: &mut dyn Scheduler, - ) -> error_stack::Result<(), PipelineError> { - self.pipeline - .add_input(input_partition, self.input, batch, scheduler) - } - - pub fn close_input( - &self, - input_partition: Partition, - scheduler: &mut dyn Scheduler, - ) -> error_stack::Result<(), PipelineError> { - self.pipeline - .close_input(input_partition, self.input, scheduler) - } -} diff --git a/crates/sparrow-transforms/src/lib.rs b/crates/sparrow-transforms/src/lib.rs index 48641e74b..5b061fac3 100644 --- a/crates/sparrow-transforms/src/lib.rs +++ b/crates/sparrow-transforms/src/lib.rs @@ -11,7 +11,7 @@ //! Pipeline for executing 1 or more transform. //! //! Transforms are simpler than pipelines -- they apply processing logic to an -//! input batch to produce an output batch. Only the last trasnform in a pipeline +//! input batch to produce an output batch. Only the last transform in a pipeline //! may affect the keys associated with rows -- after that a repartition pipeline //! must be executed to move data to the appropriate partitions. diff --git a/crates/sparrow-transforms/src/transform_pipeline.rs b/crates/sparrow-transforms/src/transform_pipeline.rs index 31830b8a4..ffd49b246 100644 --- a/crates/sparrow-transforms/src/transform_pipeline.rs +++ b/crates/sparrow-transforms/src/transform_pipeline.rs @@ -85,18 +85,18 @@ impl TransformPipeline { /// Create a new transform pipeline. /// /// Args: - /// first_step_input_id: The `StepId` of the step that produces input to - /// this pipeline. It should be the only input step to the first step in + /// producer_id: The `StepId` of the step that produces input to this + /// pipeline. It should be the only input step to the first step in /// `steps`. /// steps: Iterator over the steps (in order) comprising the pipeline. /// They should all be transforms. /// consumers: The `InputHandles` to output the result of the transform to. pub fn try_new<'a>( - first_step_input_id: StepId, + producer_id: StepId, steps: impl Iterator + ExactSizeIterator, consumers: InputHandles, ) -> error_stack::Result { - let mut input_step_id = first_step_input_id; + let mut input_step_id = producer_id; let mut transforms = Vec::with_capacity(steps.len()); for step in steps { error_stack::ensure!( From 9eb4afa88c8e39a33d317c3148d50310d567f88a Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Fri, 13 Oct 2023 15:56:53 -0700 Subject: [PATCH 06/14] ref: cleanup thread handling issues --- ...sts__logical_to_physical_arithmetic-2.snap | 2 +- ...sts__logical_to_physical_arithmetic-3.snap | 4 +- ...tests__logical_to_physical_arithmetic.snap | 2 +- crates/sparrow-batch/src/batch.rs | 2 - .../src/write_channel_pipeline.rs | 4 + crates/sparrow-scheduler/src/error.rs | 4 +- crates/sparrow-scheduler/src/idle_workers.rs | 100 ++++++ crates/sparrow-scheduler/src/lib.rs | 2 +- crates/sparrow-scheduler/src/monitor.rs | 102 +++++- crates/sparrow-scheduler/src/pending.rs | 138 -------- .../sparrow-scheduler/src/schedule_count.rs | 9 +- crates/sparrow-scheduler/src/task.rs | 52 ++- crates/sparrow-scheduler/src/worker.rs | 87 +++-- crates/sparrow-scheduler/src/worker_pool.rs | 126 +++---- .../src/transform_pipeline.rs | 37 +- python/Cargo.lock | 328 +++++++++++++++++- 16 files changed, 664 insertions(+), 335 deletions(-) create mode 100644 crates/sparrow-scheduler/src/idle_workers.rs delete mode 100644 crates/sparrow-scheduler/src/pending.rs diff --git a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap index a8977864f..e1f338b24 100644 --- a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap +++ b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-2.snap @@ -6,7 +6,7 @@ steps: - id: 0 kind: read: - source_id: 00000000-0000-0000-0000-000000000001 + source_uuid: 00000000-0000-0000-0000-000000000001 inputs: [] result_type: Struct: diff --git a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap index fc1205942..7abb6b36e 100644 --- a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap +++ b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap @@ -6,7 +6,7 @@ steps: - id: 0 kind: read: - source_id: 00000000-0000-0000-0000-000000000001 + source_uuid: 00000000-0000-0000-0000-000000000001 inputs: [] result_type: Struct: @@ -26,7 +26,7 @@ steps: - id: 1 kind: read: - source_id: 00000000-0000-0000-0000-000000000002 + source_uuid: 00000000-0000-0000-0000-000000000002 inputs: [] result_type: Struct: diff --git a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap index d64e7d4ae..4ef96af50 100644 --- a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap +++ b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic.snap @@ -6,7 +6,7 @@ steps: - id: 0 kind: read: - source_id: 00000000-0000-0000-0000-000000000001 + source_uuid: 00000000-0000-0000-0000-000000000001 inputs: [] result_type: Struct: diff --git a/crates/sparrow-batch/src/batch.rs b/crates/sparrow-batch/src/batch.rs index 4322216f5..a94800f48 100644 --- a/crates/sparrow-batch/src/batch.rs +++ b/crates/sparrow-batch/src/batch.rs @@ -79,8 +79,6 @@ impl Batch { pub fn into_record_batch(self, schema: Arc) -> Option { self.data.map(|data| { - println!("Schema: {schema:?}"); - println!("Data: {:?}", data.data.data_type()); if let Some(fields) = data.data.as_struct_opt() { let mut columns = Vec::with_capacity(3 + fields.num_columns()); columns.extend_from_slice(&[data.time, data.subsort, data.key_hash]); diff --git a/crates/sparrow-execution/src/write_channel_pipeline.rs b/crates/sparrow-execution/src/write_channel_pipeline.rs index 04754e4ff..3a78e3c36 100644 --- a/crates/sparrow-execution/src/write_channel_pipeline.rs +++ b/crates/sparrow-execution/src/write_channel_pipeline.rs @@ -63,6 +63,10 @@ impl Pipeline for WriteChannelPipeline { _scheduler: &mut dyn sparrow_scheduler::Scheduler, ) -> error_stack::Result<(), PipelineError> { let mut channel = self.channel.lock(); + tracing::info!("Closing input of write_channel"); + assert_eq!(input, 0); + assert_eq!(input_partition, 0); + error_stack::ensure!( channel.is_some(), PipelineError::InputClosed { diff --git a/crates/sparrow-scheduler/src/error.rs b/crates/sparrow-scheduler/src/error.rs index 1ee9e707d..7a40e9031 100644 --- a/crates/sparrow-scheduler/src/error.rs +++ b/crates/sparrow-scheduler/src/error.rs @@ -1,4 +1,4 @@ -use crate::Partition; +use crate::{Partition, TaskRef}; /// Top level errors reported during partitioned pipeline execution. #[derive(derive_more::Display, Debug)] @@ -22,6 +22,8 @@ pub enum Error { name: &'static str, partition: Partition, }, + #[display(fmt = "Saw task {_0} after all workers idled")] + TaskAfterAllIdle(TaskRef), } impl error_stack::Context for Error {} diff --git a/crates/sparrow-scheduler/src/idle_workers.rs b/crates/sparrow-scheduler/src/idle_workers.rs new file mode 100644 index 000000000..371b79e2b --- /dev/null +++ b/crates/sparrow-scheduler/src/idle_workers.rs @@ -0,0 +1,100 @@ +use std::sync::Arc; + +use parking_lot::{Condvar, Mutex}; + +/// Track the idle workers. +#[derive(Debug)] +pub(crate) struct IdleWorkers { + state: Mutex, + condition: Condvar, +} + +#[derive(Debug)] +struct State { + /// Number of workers. + num_workers: usize, + + /// Number of (curretly) active workers. + num_idle: usize, + + /// Whether sources have been completed. + sources_done: bool, + + // Whether all workers have been idle at once after sources finished. + all_idle: bool, +} + +/// Reason that the thread was woken. +#[must_use] +pub(crate) enum WakeReason { + /// The thread was woken by an explicit call to `wake_one` or `wake_all`. + Woken, + /// The thread was woken when all threads were idle. + AllIdle, +} + +impl IdleWorkers { + /// Create a new `WorkerIdle` for the given number of workers. + pub fn new(num_workers: usize) -> Arc { + Arc::new(IdleWorkers { + state: Mutex::new(State { + num_workers, + num_idle: 0, + sources_done: false, + all_idle: false, + }), + condition: Condvar::default(), + }) + } + + /// Idle the current thread until woken or all threads are idle. + /// + /// Returns `true` if all threads were idle + pub fn idle(&self) -> WakeReason { + let mut state = self.state.lock(); + state.num_idle += 1; + + if state.sources_done && state.num_idle == state.num_workers { + tracing::trace!( + "All {} threads are idle. Waking workers to shutdown.", + state.num_workers + ); + + state.all_idle = true; + state.num_idle -= 1; + std::mem::drop(state); + + self.condition.notify_all(); + WakeReason::AllIdle + } else { + let thread_id = std::thread::current().id(); + tracing::trace!( + "Idling thread {thread_id:?} ({}/{} idle)", + state.num_idle, + state.num_workers + ); + self.condition.wait(&mut state); + state.num_idle -= 1; + if state.all_idle { + tracing::trace!("Thread {thread_id:?} woken to shutdown (all workers idle)."); + WakeReason::AllIdle + } else { + tracing::trace!( + "Thread {thread_id:?} woken ({}/{} idle).", + state.num_idle, + state.num_workers + ); + WakeReason::Woken + } + } + } + + pub fn finish_sources(&self) { + self.state.lock().sources_done = true; + } + + pub fn wake_one(&self) { + tracing::trace!("Waking one worker"); + self.condition.notify_one(); + } +} diff --git a/crates/sparrow-scheduler/src/lib.rs b/crates/sparrow-scheduler/src/lib.rs index aee9ddf29..b6d53912d 100644 --- a/crates/sparrow-scheduler/src/lib.rs +++ b/crates/sparrow-scheduler/src/lib.rs @@ -11,10 +11,10 @@ //! Scheduler for local, multi-threaded execution of Sparrow plans. mod error; +mod idle_workers; mod input_handles; mod monitor; mod partition; -mod pending; mod pipeline; mod queue; mod schedule_count; diff --git a/crates/sparrow-scheduler/src/monitor.rs b/crates/sparrow-scheduler/src/monitor.rs index 6b5c48fcd..0854724d4 100644 --- a/crates/sparrow-scheduler/src/monitor.rs +++ b/crates/sparrow-scheduler/src/monitor.rs @@ -1,18 +1,30 @@ use std::thread::ThreadId; +use error_stack::{IntoReport, ResultExt}; +use hashbrown::HashMap; + +use crate::Error; + /// Create a monitor that watches for thread completions. +/// +/// Allows spawning "guarded" threads, which report their thread ID upon +/// completion. In turn, this allows joining on the thread results in the order +/// they complete which allows surfacing errors that would otherwise block other +/// threads. pub(crate) struct Monitor { - tx: tokio::sync::mpsc::Sender, - pub(crate) rx: tokio::sync::mpsc::Receiver, + tx: tokio::sync::mpsc::UnboundedSender, + rx: tokio::sync::mpsc::UnboundedReceiver, + handles: HashMap>>, } +/// Guard used within a thread to watch for thread completion. pub(crate) struct MonitorGuard { - tx: tokio::sync::mpsc::Sender, + tx: tokio::sync::mpsc::UnboundedSender, } impl Drop for MonitorGuard { fn drop(&mut self) { - match self.tx.blocking_send(std::thread::current().id()) { + match self.tx.send(std::thread::current().id()) { Ok(_) => (), Err(thread_id) => { tracing::error!("Failed to send thread completion for {thread_id:?}"); @@ -21,15 +33,85 @@ impl Drop for MonitorGuard { } } -impl Monitor { - pub fn with_capacity(capacity: usize) -> Self { - let (tx, rx) = tokio::sync::mpsc::channel(capacity); - Monitor { tx, rx } +impl std::default::Default for Monitor { + fn default() -> Self { + let (tx, rx) = tokio::sync::mpsc::unbounded_channel(); + Self { + tx, + rx, + handles: Default::default(), + } } +} - pub fn guard(&self) -> MonitorGuard { - MonitorGuard { +impl Monitor { + pub fn spawn_guarded error_stack::Result<(), Error> + Send + 'static>( + &mut self, + name: String, + f: F, + ) -> error_stack::Result<(), Error> { + let guard = MonitorGuard { tx: self.tx.clone(), + }; + + let handle = std::thread::Builder::new() + .name(name) + .spawn(move || { + let result = f(); + + std::mem::drop(guard); + result + }) + .into_report() + .change_context(Error::SpawnWorker)?; + + // We can't insert the thread until after we spawn it. While + // it seems that we could have a race condition if the the + // thread finishes (and sends out its ID) before we insert the + // handle, this sholudn't happen. We have `&mut self` here and + // observing the thread IDs shutting down requires `self`, so + // this must finish before `wait_all` is called. + self.handles.insert(handle.thread().id(), handle); + + Ok(()) + } + + pub async fn join_all(self) -> error_stack::Result<(), Error> { + // Take the `rx` out of self (and drop the `tx` so the stream will end). + let Self { + mut rx, + mut handles, + tx, + } = self; + std::mem::drop(tx); + + while let Some(finished) = rx.recv().await { + let handle = handles + .remove(&finished) + // This should only happen if a spawned thread had a `MonitorGuard` + // but was not added to `handles`. This should not happen. + .expect("Finished unregistered handle"); + + match handle.join() { + Ok(worker_result) => worker_result?, + Err(_) => { + error_stack::bail!(Error::PipelinePanic) + } + } + + tracing::trace!( + "Thread {finished:?} completed. Waiting for {} threads.", + handles.len() + ); } + + // This should only happen if a spawned thread was added to the handles + // but did not register a `MonitorGuard`. This should not happen. + assert!( + handles.is_empty(), + "Not all handles reported completion via monitor" + ); + + Ok(()) } } diff --git a/crates/sparrow-scheduler/src/pending.rs b/crates/sparrow-scheduler/src/pending.rs deleted file mode 100644 index 9faf389c2..000000000 --- a/crates/sparrow-scheduler/src/pending.rs +++ /dev/null @@ -1,138 +0,0 @@ -use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; -use std::sync::Arc; - -use hashbrown::HashMap; -use itertools::Itertools; -use parking_lot::Mutex; - -use crate::Partition; - -/// Track the number of pending pipeline partitions. -#[derive(Default)] -pub(crate) struct PendingSet { - /// Map from pending partition to name of the transform. - /// - /// Used to report the currently pending partitions. Should not - /// be used for checking whether a specific partition is pending. - pending_partitions: Mutex>, - /// Count of the total pending partitions. - /// - /// Keeping this outside the mutex allows for fast checking of the current - /// count. - pending_partition_count: AtomicUsize, -} - -impl std::fmt::Debug for PendingSet { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let pending_partitions = self.pending_partitions.lock(); - f.debug_struct("PendingSet") - .field("pending_partitions", &pending_partitions) - .field("pending_partition_count", &self.pending_partition_count) - .finish() - } -} - -impl std::fmt::Display for PendingSet { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let pending = self.pending_partitions.lock(); - write!( - f, - "[{}]", - pending.iter().format_with(", ", |elt, f| f(&format_args!( - "{}({}) partition {}", - elt.1, elt.0.pipeline_index, elt.0.partition - ))) - ) - } -} - -pub(crate) struct PendingPartition { - pending_set: Arc, - pub(crate) pipeline_index: usize, - pub(crate) partition: Partition, - is_complete: AtomicBool, -} - -#[derive(Debug, Clone, Eq, PartialEq, Hash)] -struct PendingPartitionKey { - pipeline_index: usize, - partition: Partition, -} - -impl PendingSet { - pub fn add_pending( - self: &Arc, - pipeline_index: usize, - partition: Partition, - name: &'static str, - ) -> PendingPartition { - let key = PendingPartitionKey { - pipeline_index, - partition, - }; - let pending_partition = PendingPartition { - pending_set: self.clone(), - pipeline_index, - partition, - is_complete: AtomicBool::new(false), - }; - - let previous = self.pending_partitions.lock().insert(key, name); - debug_assert_eq!( - previous, None, - "Duplicate pipeline partition added to pending set" - ); - self.pending_partition_count.fetch_add(1, Ordering::Release); - - pending_partition - } - - pub fn pending_partition_count(&self) -> usize { - self.pending_partition_count.load(Ordering::Acquire) - } -} - -impl std::fmt::Debug for PendingPartition { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("PendingPartition") - .field("pipeline_index", &self.pipeline_index) - .field("partition", &self.partition) - .field("pending", &self.is_complete) - .finish_non_exhaustive() - } -} - -impl PendingPartition { - /// Record this pending partition completed. - pub fn complete(&self) { - let previous = self.is_complete.fetch_or(true, Ordering::AcqRel); - debug_assert!(!previous, "Task already completed"); - - let previous = self - .pending_set - .pending_partitions - .lock() - .remove(&PendingPartitionKey { - pipeline_index: self.pipeline_index, - partition: self.partition, - }); - - let previous = previous.expect("task should be in pending set"); - - let remaining = self - .pending_set - .pending_partition_count - .fetch_sub(1, Ordering::AcqRel); - tracing::info!( - "Completed partition {} of pipeline {previous} {}. {remaining} remaining.", - self.partition, - self.pipeline_index, - ); - tracing::trace!("Remaining pipelines: {}", self.pending_set); - } - - /// Return true if this partition is completed. - pub fn is_complete(&self) -> bool { - self.is_complete.load(Ordering::Acquire) - } -} diff --git a/crates/sparrow-scheduler/src/schedule_count.rs b/crates/sparrow-scheduler/src/schedule_count.rs index d96ff0e79..854c156a8 100644 --- a/crates/sparrow-scheduler/src/schedule_count.rs +++ b/crates/sparrow-scheduler/src/schedule_count.rs @@ -13,12 +13,15 @@ impl ScheduleCount { /// /// Returns true if this task wasn't previously scheduled. pub fn schedule(&self) -> bool { - self.0.fetch_add(1, Ordering::SeqCst) == 0 + let count = self.0.fetch_add(1, Ordering::SeqCst); + tracing::trace!("Schedule count {count}"); + count == 0 } /// Returns a `TaskGuard` which will return the count pub fn guard(&self) -> ScheduleGuard<'_> { let entry_count = self.0.load(Ordering::SeqCst); + tracing::trace!("Schedule count on start: {entry_count}"); debug_assert!(entry_count > 0, "Running task with entry count 0"); ScheduleGuard { count: self, @@ -40,6 +43,10 @@ impl<'a> ScheduleGuard<'a> { /// this will return `true` to indicate the task should be re-scheduled. pub fn finish(self) -> bool { let schedule_count = self.count.0.fetch_sub(self.entry_count, Ordering::SeqCst); + tracing::trace!( + "Count on entry {}, count on finish {schedule_count}", + self.entry_count + ); schedule_count != self.entry_count } } diff --git a/crates/sparrow-scheduler/src/task.rs b/crates/sparrow-scheduler/src/task.rs index 67902ea3b..bb92f4704 100644 --- a/crates/sparrow-scheduler/src/task.rs +++ b/crates/sparrow-scheduler/src/task.rs @@ -1,10 +1,10 @@ +use std::sync::atomic::AtomicBool; use std::sync::Arc; use error_stack::ResultExt; -use crate::pending::PendingPartition; use crate::schedule_count::ScheduleCount; -use crate::{Error, Pipeline, Scheduler}; +use crate::{Error, Partition, Pipeline, Scheduler}; /// The unit of work executed by the scheduler. /// @@ -12,10 +12,10 @@ use crate::{Error, Pipeline, Scheduler}; /// single [Pipeline] and produces a single unit of output (typically a batch). #[derive(Debug)] pub struct Task { - /// Entry recording the status (pending or not) of this task. - pending: PendingPartition, /// Name of the pipeline implementation. name: &'static str, + partition: Partition, + pipeline_index: usize, /// The pipeline to execute. /// /// This is a weak reference to avoid cycles. @@ -24,6 +24,8 @@ pub struct Task { /// /// This is reset after the task is executed. schedule_count: ScheduleCount, + /// Whether this task has been completed. + complete: AtomicBool, } impl std::fmt::Display for Task { @@ -31,7 +33,7 @@ impl std::fmt::Display for Task { write!( f, "{}({}) partition {}", - self.name, self.pending.pipeline_index, self.pending.partition + self.name, self.pipeline_index, self.partition ) } } @@ -39,15 +41,18 @@ impl std::fmt::Display for Task { impl Task { /// Create a new task executing the given pipeline and partition. pub(crate) fn new( - pending: PendingPartition, name: &'static str, + partition: Partition, + pipeline_index: usize, pipeline: std::sync::Weak, ) -> Self { Self { - pending, name, + partition, + pipeline_index, pipeline, schedule_count: ScheduleCount::default(), + complete: AtomicBool::new(false), } } @@ -61,47 +66,62 @@ impl Task { /// If this is called while it is being executed (eg., during `do_work`) then /// the `guard` will return `true` to indicate the task should be re-executed. pub(crate) fn schedule(&self) -> bool { + debug_assert!(!self.is_complete(), "Scheduling completed task"); self.schedule_count.schedule() } fn pipeline(&self) -> error_stack::Result, Error> { Ok(self.pipeline.upgrade().ok_or(Error::PipelineDropped { - index: self.pending.pipeline_index, + index: self.pipeline_index, name: self.name, - partition: self.pending.partition, + partition: self.partition, })?) } fn error(&self, method: &'static str) -> Error { Error::Pipeline { method, - index: self.pending.pipeline_index, + index: self.pipeline_index, name: self.name, - partition: self.pending.partition, + partition: self.partition, } } - #[inline] + /// Execute one iteration of the given task. + /// + /// Returns `true` if the task was scheduled during execution, indicating + /// it should be added to the queue again. pub(crate) fn do_work( &self, scheduler: &mut dyn Scheduler, ) -> error_stack::Result { let guard = self.schedule_count.guard(); + let span = tracing::debug_span!( + "Running task", + name = self.name, + pipeline = self.pipeline_index, + partition = ?self.partition + ); + let _enter = span.enter(); + + tracing::info!("Start"); self.pipeline()? - .do_work(self.pending.partition, scheduler) + .do_work(self.partition, scheduler) .change_context_lazy(|| self.error("do_work"))?; + tracing::info!("Done"); Ok(guard.finish()) } /// Mark this task as completed. /// /// After this, it should not be scheduled nor should work be done. - pub fn complete(&self) { - self.pending.complete() + pub fn complete(&self) -> bool { + self.complete + .fetch_or(true, std::sync::atomic::Ordering::AcqRel) } pub fn is_complete(&self) -> bool { - self.pending.is_complete() + self.complete.load(std::sync::atomic::Ordering::Acquire) } } diff --git a/crates/sparrow-scheduler/src/worker.rs b/crates/sparrow-scheduler/src/worker.rs index 315339378..be2728a8f 100644 --- a/crates/sparrow-scheduler/src/worker.rs +++ b/crates/sparrow-scheduler/src/worker.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use crate::pending::PendingSet; +use crate::idle_workers::{IdleWorkers, WakeReason}; use crate::{queue::*, Error, TaskRef}; pub trait Scheduler { @@ -38,70 +38,91 @@ pub trait Scheduler { #[derive(Debug, Clone)] pub struct Injector { queue: GlobalQueue, + pub(crate) idle_workers: Arc, } impl Injector { pub fn create(workers: usize, local_queue_size: u16) -> (Self, Vec) { + let idle_workers = IdleWorkers::new(workers); let queue = GlobalQueue::new(workers, local_queue_size); let workers = queue .take_local_queues() - .map(|queue| Worker { queue }) + .map(|queue| Worker { + queue, + idle_workers: idle_workers.clone(), + }) .collect(); - (Injector { queue }, workers) + ( + Injector { + queue, + idle_workers, + }, + workers, + ) } } impl Scheduler for Injector { fn schedule_global(&self, task: TaskRef) { if task.schedule() { - self.queue.push(task) + tracing::trace!("1 Added {task:?} to queue"); + self.queue.push(task); + } else { + tracing::trace!("1 {task:?} already executing"); } + self.idle_workers.wake_one(); } fn schedule(&mut self, task: TaskRef) { - self.schedule_global(task) + self.schedule_global(task); } fn schedule_yield(&mut self, task: TaskRef) { - self.schedule_global(task) + self.schedule_global(task); } } /// An individual worker that allows adding work to the local or global queue. pub struct Worker { queue: LocalQueue, + idle_workers: Arc, } impl Worker { /// Run the work loop to completion. - pub(crate) fn work_loop( - mut self, - index: usize, - pending_set: Arc, - ) -> error_stack::Result<(), Error> { + pub(crate) fn work_loop(mut self, index: usize) -> error_stack::Result<(), Error> { + let thread_id = std::thread::current().id(); + let _span = tracing::info_span!("Worker", ?thread_id, index).entered(); + tracing::info!("Starting work loop on thread {thread_id:?}"); loop { while let Some(task) = self.queue.pop() { - tracing::info!("Running task: {task:?} on worker {index}"); if task.do_work(&mut self)? { + if task.is_complete() { + tracing::warn!("Completed task scheduled during execution."); + } + // This means that the task was scheduled while we were executing. // As a result, we didn't add it to any queue yet, so we need to - // do so now. - self.queue.push_global(task); + // do so now. We use the global queue because generlaly it won't + // be processing data just produced. + tracing::trace!("Task {task} scheduled during execution. Re-adding."); + self.queue.push_yield(task); + } else { + tracing::trace!("Task {task} not scheduled during execution."); } } - let pending_count = pending_set.pending_partition_count(); - if pending_count == 0 { - break; - } else { - // Right now, this "busy-waits" by immediately trying to pull more work. - // This potentially leads to thread thrashing. We should instead call - // `thread::park` to park this thread, and call thread::unpark` when - // work is added to the global queue / back of the local queues. + match self.idle_workers.idle() { + WakeReason::Woken => continue, + WakeReason::AllIdle => { + if let Some(next) = self.queue.pop() { + error_stack::bail!(Error::TaskAfterAllIdle(next)); + } + + break; + } } } - - tracing::info!("All partitions completed. Shutting down worker {index}"); Ok(()) } } @@ -109,19 +130,31 @@ impl Worker { impl Scheduler for Worker { fn schedule(&mut self, task: TaskRef) { if task.schedule() { - self.queue.push(task) + tracing::trace!("2 Added {task:?} to queue"); + self.queue.push(task); + } else { + tracing::trace!("2 {task:?} already executing"); } + self.idle_workers.wake_one(); } fn schedule_yield(&mut self, task: TaskRef) { if task.schedule() { - self.queue.push_yield(task) + tracing::trace!("3 Added {task:?} to queue"); + self.queue.push_yield(task); + } else { + tracing::trace!("3 {task:?} already executing"); } + self.idle_workers.wake_one(); } fn schedule_global(&self, task: TaskRef) { if task.schedule() { - self.queue.push_global(task) + tracing::trace!("4 Added {task:?} to queue"); + self.queue.push_global(task); + } else { + tracing::trace!("4 {task:?} already executing"); } + self.idle_workers.wake_one(); } } diff --git a/crates/sparrow-scheduler/src/worker_pool.rs b/crates/sparrow-scheduler/src/worker_pool.rs index 9500c6e97..eeb6553ca 100644 --- a/crates/sparrow-scheduler/src/worker_pool.rs +++ b/crates/sparrow-scheduler/src/worker_pool.rs @@ -1,13 +1,8 @@ use std::sync::Arc; -use std::thread::ThreadId; use crate::monitor::Monitor; -use crate::pending::PendingSet; use crate::worker::Injector; use crate::{Error, Partition, Pipeline, Task, TaskRef, Worker}; -use error_stack::{IntoReport, ResultExt}; -use hashbrown::HashMap; -use itertools::Itertools; /// Default thread count to use if we aren't able to determine /// the number of cores. @@ -22,8 +17,6 @@ pub struct WorkerPool { workers: Vec, /// A vector of the pipelines we created. pipelines: Vec>, - /// Track which pipelines / partitions are still running. - pending: Arc, } impl WorkerPool { @@ -41,6 +34,7 @@ impl WorkerPool { DEFAULT_THREAD_COUNT }); + tracing::info!("Creating workers to execute query {query_id} with {threads} threads"); let (injector, workers) = Injector::create(threads, LOCAL_QUEUE_SIZE); let scheduler = Self { @@ -48,7 +42,6 @@ impl WorkerPool { injector, workers, pipelines: vec![], - pending: Arc::new(PendingSet::default()), }; Ok(scheduler) } @@ -65,21 +58,19 @@ impl WorkerPool { where T: Pipeline + 'static, { - let index = self.pipelines.len(); + let pipeline_index = self.pipelines.len(); let name = std::any::type_name::(); // `new_cyclic` provides a `Weak` reference to the pipeline before it is // created. This allows us to create tasks that reference the pipeline // (via weak references) and pass those tasks to the pipeline. - let pending = self.pending.clone(); let pipeline: Arc = Arc::new_cyclic(move |weak| { let tasks = (0..partitions) .map(|partition| -> TaskRef { let pipeline: std::sync::Weak = weak.clone(); let partition: Partition = partition.into(); - let pending = pending.add_pending(index, partition, name); - let task = Task::new(pending, name, pipeline); + let task = Task::new(name, partition, pipeline_index, pipeline); Arc::new(task) }) .collect(); @@ -94,7 +85,7 @@ impl WorkerPool { let pipeline: Arc = pipeline; self.pipelines.push(pipeline.clone()); - tracing::info!("Added {partitions} partitions for pipeline {index} {name}"); + tracing::trace!("Added {partitions} partitions for pipeline {pipeline_index} {name}"); pipeline } @@ -104,11 +95,10 @@ impl WorkerPool { /// Returns a `RunningWorkers` used for completing the workers. pub fn start(self) -> error_stack::Result { let Self { - pending, workers, query_id, pipelines, - .. + injector, } = self; let core_ids = core_affinity::get_core_ids(); @@ -118,56 +108,34 @@ impl WorkerPool { .map(Some) .chain(std::iter::repeat(None)); - let monitor = Monitor::with_capacity(workers.len()); - let handles = workers - .into_iter() - .zip(core_ids) - .enumerate() - .map(|(index, (worker, core_id))| { - // Spawn the worker thread. - let span = tracing::info_span!("compute", query_id, index); - let pending = pending.clone(); - let guard = monitor.guard(); - std::thread::Builder::new() - .name(format!("compute-{index}")) - .spawn(move || { - let _enter = span.enter(); - - // Set the core affinity, if possible, so this thread always - // executes on the same core. - if let Some(core_id) = core_id { - if core_affinity::set_for_current(core_id) { - tracing::info!( - "Set core affinity for thread {index} to {core_id:?}" - ); - } else { - tracing::info!( - "Failed to set core affinity for thread {index} to {core_id:?}" - ); - } - } else { - tracing::info!("Setting core affinity not supported"); - }; - - // Run the worker - let result = worker.work_loop(index, pending.clone()); - - // Make sure the monitor guard is moved into this thread - // and dropped before returning. - std::mem::drop(guard); - result - }) - .into_report() - .change_context(Error::SpawnWorker) - }) - .map_ok(|handle| (handle.thread().id(), handle)) - .try_collect()?; + let mut monitor = Monitor::default(); + + for (index, (worker, core_id)) in workers.into_iter().zip(core_ids).enumerate() { + // Spawn the worker thread. + let span = tracing::info_span!("compute", query_id, index); + monitor.spawn_guarded(format!("compute-{index}"), move || { + let _enter = span.enter(); + + // Set the core affinity, if possible, so this thread always + // executes on the same core. + if let Some(core_id) = core_id { + if core_affinity::set_for_current(core_id) { + tracing::info!("Set core affinity for thread {index} to {core_id:?}"); + } + } else { + tracing::info!("Setting core affinity not supported"); + }; + + // Run the worker + worker.work_loop(index) + })?; + } Ok(RunningWorkers { query_id, _pipelines: pipelines, - handles, - finishing_threads: monitor.rx, + monitor, + injector, }) } } @@ -176,36 +144,20 @@ pub struct RunningWorkers { query_id: String, /// Hold the Arcs for the pipelines so they aren't dropped. _pipelines: Vec>, - handles: HashMap>>, - finishing_threads: tokio::sync::mpsc::Receiver, + monitor: Monitor, + injector: Injector, } impl RunningWorkers { - pub async fn join(mut self) -> error_stack::Result<(), Error> { + /// Mark the sources as complete and wait for workres to finish. + /// + /// This should not be called until after all source tasks have been + /// added for processing. + pub async fn join(self) -> error_stack::Result<(), Error> { tracing::info!(self.query_id, "Waiting for completion of query"); - while let Some(finished) = self.finishing_threads.recv().await { - let handle = self - .handles - .remove(&finished) - // This should only happen if a spawned thread had a `MonitorGuard` - // but was not added to `handles`. This should not happen. - .expect("Finished unregistered handle"); - - match handle.join() { - Ok(worker_result) => worker_result?, - Err(_) => { - error_stack::bail!(Error::PipelinePanic) - } - } - } - - // This should only happen if a spawned thread was added to the handles - // but did not register a `MonitorGuard`. This should not happen. - assert!( - self.handles.is_empty(), - "Not all handles reported completion via monitor" - ); - + self.injector.idle_workers.finish_sources(); + self.monitor.join_all().await?; + tracing::info!(self.query_id, "Complete"); Ok(()) } } diff --git a/crates/sparrow-transforms/src/transform_pipeline.rs b/crates/sparrow-transforms/src/transform_pipeline.rs index ffd49b246..de87260b9 100644 --- a/crates/sparrow-transforms/src/transform_pipeline.rs +++ b/crates/sparrow-transforms/src/transform_pipeline.rs @@ -54,10 +54,6 @@ impl TransformPartition { self.is_closed.load(Ordering::Acquire) } - fn is_input_empty(&self) -> bool { - self.inputs.lock().is_empty() - } - fn add_input(&self, batch: Batch) { self.inputs.lock().push_back(batch); } @@ -197,6 +193,7 @@ impl Pipeline for TransformPipeline { } ); let partition = &self.partitions[input_partition]; + tracing::trace!("Closing input for transform {}", partition.task); error_stack::ensure!( !partition.is_closed(), PipelineError::InputClosed { @@ -204,7 +201,6 @@ impl Pipeline for TransformPipeline { input_partition } ); - tracing::trace!("Closing input for {}", partition.task); // Don't close the sink here. We may be currently executing a `do_work` // loop, in which case we need to allow it to output to the sink before @@ -221,23 +217,20 @@ impl Pipeline for TransformPipeline { scheduler: &mut dyn Scheduler, ) -> error_stack::Result<(), PipelineError> { let partition = &self.partitions[input_partition]; + let _enter = tracing::trace_span!("TransformPipeline::do_work", %partition.task).entered(); let Some(batch) = partition.pop_input() else { error_stack::ensure!( partition.is_closed(), PipelineError::illegal_state("scheduled without work") ); + tracing::info!("Input is closed and empty. Closing consumers and finishing pipeline."); self.consumers.close_input(input_partition, scheduler)?; partition.task.complete(); return Ok(()); }; - tracing::trace!( - "Performing work for partition {input_partition} on {} rows", - batch.num_rows() - ); - // If the batch is non empty, process it. // TODO: Propagate empty batches to further the watermark. if !batch.is_empty() { @@ -262,25 +255,11 @@ impl Pipeline for TransformPipeline { } } - // If the input is closed and empty, then we should close the sink. - if partition.is_closed() { - if partition.is_input_empty() { - tracing::info!( - "Input is closed and empty. Closing consumers and finishing pipeline." - ); - self.consumers - .close_input(input_partition, scheduler) - .change_context(PipelineError::Execution)?; - partition.task.complete(); - } else { - tracing::trace!("Input is closed but not empty.") - } - } - - // Note: We don't re-schedule the transform if there is input. - // This should be handled by the fact that we scheduled the transform - // when we added the batch, which should trigger the "scheduled during - // execution" -> "re-schedule" logic (see ScheduleCount). + // Note: We don't re-schedule the transform if there is input or it's + // closed. This should be handled by the fact that we scheduled the + // transform when we added the batch (or closed it), which should + // trigger the "scheduled during execution" -> "re-schedule" logic (see + // ScheduleCount). Ok(()) } diff --git a/python/Cargo.lock b/python/Cargo.lock index fa67a0130..c8623d205 100644 --- a/python/Cargo.lock +++ b/python/Cargo.lock @@ -1046,6 +1046,17 @@ version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" +[[package]] +name = "core_affinity" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "622892f5635ce1fc38c8f16dfc938553ed64af482edb5e150bf4caedbfcb2304" +dependencies = [ + "libc", + "num_cpus", + "winapi", +] + [[package]] name = "cpu-time" version = "1.0.0" @@ -1273,6 +1284,18 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "enum-as-inner" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ffccbb6966c05b32ef8fbac435df276c4ae4d3dc55a8cd0eb9745e6c12f546a" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "syn 2.0.27", +] + [[package]] name = "enum-map" version = "2.6.0" @@ -1563,6 +1586,19 @@ dependencies = [ "byteorder", ] +[[package]] +name = "generator" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cc16584ff22b460a382b7feec54b23d2908d858152e5739a120b949293bd74e" +dependencies = [ + "cc", + "libc", + "log", + "rustversion", + "windows", +] + [[package]] name = "generic-array" version = "0.14.7" @@ -1827,6 +1863,15 @@ dependencies = [ "unicode-normalization", ] +[[package]] +name = "index_vec" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74086667896a940438f2118212f313abba4aff3831fef6f4b17d02add5c8bb60" +dependencies = [ + "serde", +] + [[package]] name = "indexmap" version = "1.9.3" @@ -2215,6 +2260,19 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "loom" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce9394216e2be01e607cf9e9e2b64c387506df1e768b14cbd2854a3650c3c03e" +dependencies = [ + "cfg-if", + "generator", + "scoped-tls", + "tracing", + "tracing-subscriber", +] + [[package]] name = "lz4" version = "1.24.0" @@ -2235,6 +2293,15 @@ dependencies = [ "libc", ] +[[package]] +name = "matchers" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" +dependencies = [ + "regex-automata 0.1.10", +] + [[package]] name = "matchit" version = "0.7.0" @@ -2243,9 +2310,9 @@ checksum = "b87248edafb776e59e6ee64a79086f65890d3510f2c656c000bf2a7e8a0aea40" [[package]] name = "memchr" -version = "2.5.0" +version = "2.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" +checksum = "f665ee40bc4a3c5590afb1e9677db74a508659dfd71e126420da8274909a0167" [[package]] name = "memoffset" @@ -2337,6 +2404,16 @@ dependencies = [ "minimal-lexical", ] +[[package]] +name = "nu-ansi-term" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +dependencies = [ + "overload", + "winapi", +] + [[package]] name = "num" version = "0.4.1" @@ -2543,6 +2620,12 @@ dependencies = [ "num-traits", ] +[[package]] +name = "overload" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" + [[package]] name = "owning_ref" version = "0.4.1" @@ -3197,25 +3280,34 @@ dependencies = [ [[package]] name = "regex" -version = "1.9.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2eae68fc220f7cf2532e4494aded17545fce192d59cd996e0fe7887f4ceb575" +checksum = "d119d7c7ca818f8a53c300863d4f87566aac09943aef5b355bb83969dae75d87" dependencies = [ "aho-corasick", "memchr", - "regex-automata", - "regex-syntax 0.7.4", + "regex-automata 0.4.1", + "regex-syntax 0.8.1", ] [[package]] name = "regex-automata" -version = "0.3.3" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39354c10dd07468c2e73926b23bb9c2caca74c5501e38a35da70406f1d923310" +checksum = "465c6fc0621e4abc4187a2bda0937bfd4f722c2730b29562e19689ea796c9a4b" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.7.4", + "regex-syntax 0.8.1", ] [[package]] @@ -3230,6 +3322,12 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" +[[package]] +name = "regex-syntax" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "56d84fdd47036b038fc80dd333d10b6aab10d5d31f4a366e20014def75328d33" + [[package]] name = "reqwest" version = "0.11.18" @@ -3426,6 +3524,12 @@ dependencies = [ "windows-sys", ] +[[package]] +name = "scoped-tls" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" + [[package]] name = "scopeguard" version = "1.2.0" @@ -3691,8 +3795,29 @@ dependencies = [ "itertools 0.11.0", "num", "serde", + "tracing", +] + +[[package]] +name = "sparrow-backend" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "bitvec", + "derive_more", + "egg", + "error-stack", + "hashbrown 0.14.0", + "index_vec", + "itertools 0.11.0", + "smallvec", + "sparrow-arrow", + "sparrow-expressions", + "sparrow-logical", + "sparrow-physical", "static_init", "tracing", + "uuid 1.4.1", ] [[package]] @@ -3737,7 +3862,6 @@ dependencies = [ "sparrow-arrow", "sparrow-core", "sparrow-instructions", - "sparrow-merge", "sparrow-sources", "sparrow-syntax", "static_init", @@ -3762,6 +3886,56 @@ dependencies = [ "tonic", ] +[[package]] +name = "sparrow-execution" +version = "0.11.0" +dependencies = [ + "arrow-array", + "arrow-schema", + "derive_more", + "error-stack", + "futures", + "hashbrown 0.14.0", + "itertools 0.11.0", + "parking_lot 0.12.1", + "sparrow-batch", + "sparrow-interfaces", + "sparrow-physical", + "sparrow-scheduler", + "sparrow-transforms", + "tokio", + "tracing", + "uuid 1.4.1", +] + +[[package]] +name = "sparrow-expressions" +version = "0.11.0" +dependencies = [ + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ord", + "arrow-schema", + "arrow-select", + "arrow-string", + "derive_more", + "error-stack", + "hashbrown 0.14.0", + "index_vec", + "inventory", + "itertools 0.11.0", + "num", + "serde_json", + "sparrow-arrow", + "sparrow-batch", + "sparrow-physical", + "static_init", + "substring", +] + [[package]] name = "sparrow-instructions" version = "0.11.0" @@ -3808,7 +3982,6 @@ dependencies = [ "error-stack", "futures", "sparrow-batch", - "sparrow-core", ] [[package]] @@ -3824,6 +3997,24 @@ dependencies = [ "substring", ] +[[package]] +name = "sparrow-logical" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "decorum", + "derive_more", + "enum-as-inner", + "error-stack", + "hashbrown 0.14.0", + "itertools 0.11.0", + "serde", + "serde_yaml", + "sparrow-types", + "static_init", + "uuid 1.4.1", +] + [[package]] name = "sparrow-merge" version = "0.11.0" @@ -3834,18 +4025,26 @@ dependencies = [ "arrow-csv", "arrow-schema", "arrow-select", - "async-broadcast", - "async-stream", "bit-set", - "derive_more", - "error-stack", - "futures", "itertools 0.11.0", "proptest", "smallvec", "sparrow-arrow", "sparrow-core", +] + +[[package]] +name = "sparrow-physical" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "index_vec", + "serde", + "serde_yaml", + "sparrow-arrow", + "strum_macros 0.25.1", "tracing", + "uuid 1.4.1", ] [[package]] @@ -3922,6 +4121,26 @@ dependencies = [ "uuid 1.4.1", ] +[[package]] +name = "sparrow-scheduler" +version = "0.11.0" +dependencies = [ + "core_affinity", + "derive_more", + "error-stack", + "hashbrown 0.14.0", + "index_vec", + "itertools 0.11.0", + "loom", + "parking_lot 0.12.1", + "serde", + "smallvec", + "sparrow-batch", + "tokio", + "tracing", + "work-queue", +] + [[package]] name = "sparrow-session" version = "0.11.0" @@ -3935,9 +4154,13 @@ dependencies = [ "itertools 0.11.0", "smallvec", "sparrow-api", + "sparrow-backend", "sparrow-compiler", + "sparrow-execution", + "sparrow-expressions", "sparrow-instructions", - "sparrow-merge", + "sparrow-interfaces", + "sparrow-logical", "sparrow-runtime", "sparrow-sources", "sparrow-syntax", @@ -3956,11 +4179,9 @@ dependencies = [ "arrow-schema", "async-broadcast", "async-stream", - "derive_more", "error-stack", "futures", "sparrow-batch", - "sparrow-core", "sparrow-interfaces", "sparrow-merge", "tracing", @@ -3986,6 +4207,40 @@ dependencies = [ "static_init", ] +[[package]] +name = "sparrow-transforms" +version = "0.11.0" +dependencies = [ + "arrow-array", + "arrow-schema", + "arrow-select", + "derive_more", + "error-stack", + "itertools 0.11.0", + "parking_lot 0.12.1", + "sparrow-arrow", + "sparrow-batch", + "sparrow-expressions", + "sparrow-physical", + "sparrow-scheduler", + "tracing", +] + +[[package]] +name = "sparrow-types" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "derive_more", + "enum-map", + "error-stack", + "hashbrown 0.14.0", + "itertools 0.11.0", + "regex", + "serde", + "static_init", +] + [[package]] name = "spin" version = "0.5.2" @@ -4473,6 +4728,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" dependencies = [ "once_cell", + "valuable", ] [[package]] @@ -4485,15 +4741,33 @@ dependencies = [ "tracing-subscriber", ] +[[package]] +name = "tracing-log" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78ddad33d2d10b1ed7eb9d1f518a5674713876e97e5bb9b7345a7984fbb4f922" +dependencies = [ + "lazy_static", + "log", + "tracing-core", +] + [[package]] name = "tracing-subscriber" version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "30a651bc37f915e81f087d86e62a18eec5f79550c7faff886f7090b4ea757c77" dependencies = [ + "matchers", + "nu-ansi-term", + "once_cell", + "regex", "sharded-slab", + "smallvec", "thread_local", + "tracing", "tracing-core", + "tracing-log", ] [[package]] @@ -4655,6 +4929,12 @@ dependencies = [ "serde", ] +[[package]] +name = "valuable" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" + [[package]] name = "value-bag" version = "1.4.1" @@ -4953,6 +5233,16 @@ dependencies = [ "winapi", ] +[[package]] +name = "work-queue" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1c8d5eab11a669da42347cdd5244563592edabfe06c7778553185ffeedb623d" +dependencies = [ + "concurrent-queue", + "loom", +] + [[package]] name = "wyz" version = "0.5.1" From 76acd0ed0228bc1c6c7bb1d0133c6de7538fe662 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Mon, 16 Oct 2023 15:18:55 -0700 Subject: [PATCH 07/14] lock changes --- python/Cargo.lock | 328 +++++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 309 insertions(+), 19 deletions(-) diff --git a/python/Cargo.lock b/python/Cargo.lock index fa67a0130..e386b63b1 100644 --- a/python/Cargo.lock +++ b/python/Cargo.lock @@ -1046,6 +1046,17 @@ version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" +[[package]] +name = "core_affinity" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "622892f5635ce1fc38c8f16dfc938553ed64af482edb5e150bf4caedbfcb2304" +dependencies = [ + "libc", + "num_cpus", + "winapi", +] + [[package]] name = "cpu-time" version = "1.0.0" @@ -1273,6 +1284,18 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "enum-as-inner" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ffccbb6966c05b32ef8fbac435df276c4ae4d3dc55a8cd0eb9745e6c12f546a" +dependencies = [ + "heck 0.4.1", + "proc-macro2", + "quote", + "syn 2.0.27", +] + [[package]] name = "enum-map" version = "2.6.0" @@ -1563,6 +1586,19 @@ dependencies = [ "byteorder", ] +[[package]] +name = "generator" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cc16584ff22b460a382b7feec54b23d2908d858152e5739a120b949293bd74e" +dependencies = [ + "cc", + "libc", + "log", + "rustversion", + "windows", +] + [[package]] name = "generic-array" version = "0.14.7" @@ -1827,6 +1863,15 @@ dependencies = [ "unicode-normalization", ] +[[package]] +name = "index_vec" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "74086667896a940438f2118212f313abba4aff3831fef6f4b17d02add5c8bb60" +dependencies = [ + "serde", +] + [[package]] name = "indexmap" version = "1.9.3" @@ -2215,6 +2260,19 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "loom" +version = "0.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce9394216e2be01e607cf9e9e2b64c387506df1e768b14cbd2854a3650c3c03e" +dependencies = [ + "cfg-if", + "generator", + "scoped-tls", + "tracing", + "tracing-subscriber", +] + [[package]] name = "lz4" version = "1.24.0" @@ -2235,6 +2293,15 @@ dependencies = [ "libc", ] +[[package]] +name = "matchers" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8263075bb86c5a1b1427b5ae862e8889656f126e9f77c484496e8b47cf5c5558" +dependencies = [ + "regex-automata 0.1.10", +] + [[package]] name = "matchit" version = "0.7.0" @@ -2243,9 +2310,9 @@ checksum = "b87248edafb776e59e6ee64a79086f65890d3510f2c656c000bf2a7e8a0aea40" [[package]] name = "memchr" -version = "2.5.0" +version = "2.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dffe52ecf27772e601905b7522cb4ef790d2cc203488bbd0e2fe85fcb74566d" +checksum = "f665ee40bc4a3c5590afb1e9677db74a508659dfd71e126420da8274909a0167" [[package]] name = "memoffset" @@ -2337,6 +2404,16 @@ dependencies = [ "minimal-lexical", ] +[[package]] +name = "nu-ansi-term" +version = "0.46.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +dependencies = [ + "overload", + "winapi", +] + [[package]] name = "num" version = "0.4.1" @@ -2543,6 +2620,12 @@ dependencies = [ "num-traits", ] +[[package]] +name = "overload" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" + [[package]] name = "owning_ref" version = "0.4.1" @@ -3197,25 +3280,34 @@ dependencies = [ [[package]] name = "regex" -version = "1.9.1" +version = "1.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2eae68fc220f7cf2532e4494aded17545fce192d59cd996e0fe7887f4ceb575" +checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" dependencies = [ "aho-corasick", "memchr", - "regex-automata", - "regex-syntax 0.7.4", + "regex-automata 0.4.3", + "regex-syntax 0.8.2", ] [[package]] name = "regex-automata" -version = "0.3.3" +version = "0.1.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c230d73fb8d8c1b9c0b3135c5142a8acee3a0558fb8db5cf1cb65f8d7862132" +dependencies = [ + "regex-syntax 0.6.29", +] + +[[package]] +name = "regex-automata" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "39354c10dd07468c2e73926b23bb9c2caca74c5501e38a35da70406f1d923310" +checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.7.4", + "regex-syntax 0.8.2", ] [[package]] @@ -3230,6 +3322,12 @@ version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5ea92a5b6195c6ef2a0295ea818b312502c6fc94dde986c5553242e18fd4ce2" +[[package]] +name = "regex-syntax" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" + [[package]] name = "reqwest" version = "0.11.18" @@ -3426,6 +3524,12 @@ dependencies = [ "windows-sys", ] +[[package]] +name = "scoped-tls" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" + [[package]] name = "scopeguard" version = "1.2.0" @@ -3691,8 +3795,29 @@ dependencies = [ "itertools 0.11.0", "num", "serde", + "tracing", +] + +[[package]] +name = "sparrow-backend" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "bitvec", + "derive_more", + "egg", + "error-stack", + "hashbrown 0.14.0", + "index_vec", + "itertools 0.11.0", + "smallvec", + "sparrow-arrow", + "sparrow-expressions", + "sparrow-logical", + "sparrow-physical", "static_init", "tracing", + "uuid 1.4.1", ] [[package]] @@ -3737,7 +3862,6 @@ dependencies = [ "sparrow-arrow", "sparrow-core", "sparrow-instructions", - "sparrow-merge", "sparrow-sources", "sparrow-syntax", "static_init", @@ -3762,6 +3886,56 @@ dependencies = [ "tonic", ] +[[package]] +name = "sparrow-execution" +version = "0.11.0" +dependencies = [ + "arrow-array", + "arrow-schema", + "derive_more", + "error-stack", + "futures", + "hashbrown 0.14.0", + "itertools 0.11.0", + "parking_lot 0.12.1", + "sparrow-batch", + "sparrow-interfaces", + "sparrow-physical", + "sparrow-scheduler", + "sparrow-transforms", + "tokio", + "tracing", + "uuid 1.4.1", +] + +[[package]] +name = "sparrow-expressions" +version = "0.11.0" +dependencies = [ + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ord", + "arrow-schema", + "arrow-select", + "arrow-string", + "derive_more", + "error-stack", + "hashbrown 0.14.0", + "index_vec", + "inventory", + "itertools 0.11.0", + "num", + "serde_json", + "sparrow-arrow", + "sparrow-batch", + "sparrow-physical", + "static_init", + "substring", +] + [[package]] name = "sparrow-instructions" version = "0.11.0" @@ -3808,7 +3982,6 @@ dependencies = [ "error-stack", "futures", "sparrow-batch", - "sparrow-core", ] [[package]] @@ -3824,6 +3997,24 @@ dependencies = [ "substring", ] +[[package]] +name = "sparrow-logical" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "decorum", + "derive_more", + "enum-as-inner", + "error-stack", + "hashbrown 0.14.0", + "itertools 0.11.0", + "serde", + "serde_yaml", + "sparrow-types", + "static_init", + "uuid 1.4.1", +] + [[package]] name = "sparrow-merge" version = "0.11.0" @@ -3834,18 +4025,26 @@ dependencies = [ "arrow-csv", "arrow-schema", "arrow-select", - "async-broadcast", - "async-stream", "bit-set", - "derive_more", - "error-stack", - "futures", "itertools 0.11.0", "proptest", "smallvec", "sparrow-arrow", "sparrow-core", +] + +[[package]] +name = "sparrow-physical" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "index_vec", + "serde", + "serde_yaml", + "sparrow-arrow", + "strum_macros 0.25.1", "tracing", + "uuid 1.4.1", ] [[package]] @@ -3922,6 +4121,26 @@ dependencies = [ "uuid 1.4.1", ] +[[package]] +name = "sparrow-scheduler" +version = "0.11.0" +dependencies = [ + "core_affinity", + "derive_more", + "error-stack", + "hashbrown 0.14.0", + "index_vec", + "itertools 0.11.0", + "loom", + "parking_lot 0.12.1", + "serde", + "smallvec", + "sparrow-batch", + "tokio", + "tracing", + "work-queue", +] + [[package]] name = "sparrow-session" version = "0.11.0" @@ -3935,9 +4154,13 @@ dependencies = [ "itertools 0.11.0", "smallvec", "sparrow-api", + "sparrow-backend", "sparrow-compiler", + "sparrow-execution", + "sparrow-expressions", "sparrow-instructions", - "sparrow-merge", + "sparrow-interfaces", + "sparrow-logical", "sparrow-runtime", "sparrow-sources", "sparrow-syntax", @@ -3956,11 +4179,9 @@ dependencies = [ "arrow-schema", "async-broadcast", "async-stream", - "derive_more", "error-stack", "futures", "sparrow-batch", - "sparrow-core", "sparrow-interfaces", "sparrow-merge", "tracing", @@ -3986,6 +4207,40 @@ dependencies = [ "static_init", ] +[[package]] +name = "sparrow-transforms" +version = "0.11.0" +dependencies = [ + "arrow-array", + "arrow-schema", + "arrow-select", + "derive_more", + "error-stack", + "itertools 0.11.0", + "parking_lot 0.12.1", + "sparrow-arrow", + "sparrow-batch", + "sparrow-expressions", + "sparrow-physical", + "sparrow-scheduler", + "tracing", +] + +[[package]] +name = "sparrow-types" +version = "0.11.0" +dependencies = [ + "arrow-schema", + "derive_more", + "enum-map", + "error-stack", + "hashbrown 0.14.0", + "itertools 0.11.0", + "regex", + "serde", + "static_init", +] + [[package]] name = "spin" version = "0.5.2" @@ -4473,6 +4728,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" dependencies = [ "once_cell", + "valuable", ] [[package]] @@ -4485,15 +4741,33 @@ dependencies = [ "tracing-subscriber", ] +[[package]] +name = "tracing-log" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78ddad33d2d10b1ed7eb9d1f518a5674713876e97e5bb9b7345a7984fbb4f922" +dependencies = [ + "lazy_static", + "log", + "tracing-core", +] + [[package]] name = "tracing-subscriber" version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "30a651bc37f915e81f087d86e62a18eec5f79550c7faff886f7090b4ea757c77" dependencies = [ + "matchers", + "nu-ansi-term", + "once_cell", + "regex", "sharded-slab", + "smallvec", "thread_local", + "tracing", "tracing-core", + "tracing-log", ] [[package]] @@ -4655,6 +4929,12 @@ dependencies = [ "serde", ] +[[package]] +name = "valuable" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" + [[package]] name = "value-bag" version = "1.4.1" @@ -4953,6 +5233,16 @@ dependencies = [ "winapi", ] +[[package]] +name = "work-queue" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1c8d5eab11a669da42347cdd5244563592edabfe06c7778553185ffeedb623d" +dependencies = [ + "concurrent-queue", + "loom", +] + [[package]] name = "wyz" version = "0.5.1" From c31dc1bfc76eeb3902180365de6c4333acb4d3c1 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Mon, 16 Oct 2023 20:49:07 -0700 Subject: [PATCH 08/14] comment --- crates/sparrow-execution/src/lib.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index e310d1d10..ffa30f488 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -123,7 +123,7 @@ impl PlanExecutor { debug_assert_eq!( first_step_inputs.len(), 1, - "Transforms should have a single input" + "Transform steps should have a single input" ); // If all of the steps are transforms, then we use the transform pipeline. @@ -132,7 +132,7 @@ impl PlanExecutor { assert_eq!( pipeline.steps.len(), 1, - "Non-transforms should be in separate pipelines" + "Non-transform steps should be in separate pipelines" ); let step = &plan.steps[pipeline.steps[0]]; From e6a85ddfcbc5ead5369b22842ab9f1b8e9712a94 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Tue, 17 Oct 2023 10:53:15 -0700 Subject: [PATCH 09/14] eliminate readconfig in favor of execution options --- crates/sparrow-execution/src/lib.rs | 13 ++--- crates/sparrow-execution/src/tests.rs | 2 +- .../src/execution_options.rs | 30 +++++++++++ crates/sparrow-interfaces/src/lib.rs | 2 + crates/sparrow-interfaces/src/source.rs | 25 ++------- .../src/partitioned/session.rs | 7 +-- crates/sparrow-session/src/session.rs | 53 ++++++------------- crates/sparrow-sources/src/in_memory.rs | 6 +-- 8 files changed, 63 insertions(+), 75 deletions(-) create mode 100644 crates/sparrow-interfaces/src/execution_options.rs diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index ffa30f488..af0a3f0f7 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -13,7 +13,7 @@ use arrow_array::RecordBatch; use arrow_schema::{DataType, Field, Schema, SchemaRef, TimeUnit}; use hashbrown::HashMap; -use sparrow_interfaces::{ReadConfig, Source}; +use sparrow_interfaces::{ExecutionOptions, Source}; use sparrow_physical::StepId; use sparrow_transforms::TransformPipeline; use std::sync::Arc; @@ -38,7 +38,7 @@ use crate::write_channel_pipeline::WriteChannelPipeline; pub struct PlanExecutor { worker_pool: WorkerPool, source_tasks: SourceTasks, - read_config: Arc, + execution_options: Arc, } fn result_type_to_output_schema(result_type: &DataType) -> SchemaRef { @@ -72,15 +72,12 @@ impl PlanExecutor { plan: sparrow_physical::Plan, sources: &HashMap>, output: tokio::sync::mpsc::Sender, + execution_options: Arc, ) -> error_stack::Result { let mut executor = PlanExecutor { worker_pool: WorkerPool::new(query_id).change_context(Error::Creating)?, source_tasks: SourceTasks::default(), - read_config: Arc::new(ReadConfig { - keep_open: false, - start_time: None, - end_time: None, - }), + execution_options: execution_options.clone(), }; let last_step = plan.steps.last().expect("at least one step"); @@ -193,7 +190,7 @@ impl PlanExecutor { let channel = sources.get(&source_uuid).ok_or(Error::NoSuchSource { source_id: source_uuid, })?; - let stream = channel.read(&step.result_type, self.read_config.clone()); + let stream = channel.read(&step.result_type, self.execution_options.clone()); self.source_tasks.add_read(source_uuid, stream, consumers); Ok(None) } diff --git a/crates/sparrow-execution/src/tests.rs b/crates/sparrow-execution/src/tests.rs index 44d70ff19..853c58e32 100644 --- a/crates/sparrow-execution/src/tests.rs +++ b/crates/sparrow-execution/src/tests.rs @@ -77,7 +77,7 @@ fn test_logical_query_data_before_execute() { let query = query(&session, source_expr).unwrap(); let execution = session - .execute(&query, sparrow_session::ExecutionOptions::default()) + .execute(&query, sparrow_interfaces::ExecutionOptions::default()) .unwrap(); let output = execution.collect_all_blocking().unwrap(); diff --git a/crates/sparrow-interfaces/src/execution_options.rs b/crates/sparrow-interfaces/src/execution_options.rs new file mode 100644 index 000000000..6c70ce6a5 --- /dev/null +++ b/crates/sparrow-interfaces/src/execution_options.rs @@ -0,0 +1,30 @@ +/// The kind of results the execution should produce. +#[derive(Default)] +pub enum Results { + /// Produce the entire history of changes. + #[default] + History, + /// Produce only a snapshot at a specific point in time (for each entity). + Snapshot, +} + +/// Options affecting the execution of a query. +#[derive(Default)] +pub struct ExecutionOptions { + /// The maximum number of rows to return. + pub row_limit: Option, + /// The maximum number of rows to return in a single batch. + pub max_batch_size: Option, + /// Whether to run execute as a materialization or not. + pub materialize: bool, + /// History or Snapshot results. + pub results: Results, + /// The changed since time. This is the minimum timestamp of changes to events. + /// For historic queries, this limits the output points. + /// For snapshot queries, this limits the set of entities that are considered changed. + pub changed_since_time_s: Option, + /// The final at time. This is the maximum timestamp output. + /// For historic queries, this limits the output points. + /// For snapshot queries, this determines the time at which the snapshot is produced. + pub final_at_time_s: Option, +} diff --git a/crates/sparrow-interfaces/src/lib.rs b/crates/sparrow-interfaces/src/lib.rs index 6aab43cca..a2afbd79f 100644 --- a/crates/sparrow-interfaces/src/lib.rs +++ b/crates/sparrow-interfaces/src/lib.rs @@ -7,8 +7,10 @@ clippy::print_stderr )] +mod execution_options; mod source; mod source_error; +pub use execution_options::*; pub use source::*; pub use source_error::SourceError; diff --git a/crates/sparrow-interfaces/src/source.rs b/crates/sparrow-interfaces/src/source.rs index 46d682d96..bbb3a2bb1 100644 --- a/crates/sparrow-interfaces/src/source.rs +++ b/crates/sparrow-interfaces/src/source.rs @@ -4,7 +4,7 @@ use arrow_schema::{DataType, SchemaRef}; use futures::stream::BoxStream; use sparrow_batch::Batch; -use crate::SourceError; +use crate::{ExecutionOptions, SourceError}; /// Trait implemented by sources. pub trait Source: Send + Sync { @@ -19,11 +19,11 @@ pub trait Source: Send + Sync { /// Parameters: /// * `projected_datatype`: The datatype of the data to produce. Note that /// this may differ from the prepared type. - /// * `read_config`: Configuration for the read. + /// * `execution_options`: Options for the entire execution. fn read( &self, projected_datatype: &DataType, - read_config: Arc, + execution_options: Arc, ) -> BoxStream<'static, error_stack::Result>; /// Allow downcasting the source. @@ -48,22 +48,3 @@ impl SourceExt for &Arc { self.as_any().downcast_ref() } } - -/// Defines the configuration for a read from a source. -#[derive(Clone, Debug)] -pub struct ReadConfig { - /// If true, the read will act as an unbounded source and continue reading - /// as new data is added. It is on the consumer to close the channel. - /// - /// If false, the read will act as a bounded source, and stop once the set - /// of data available at the time of the read has been processed. - pub keep_open: bool, - /// Optional timestamp in nanos at which to start reading. - /// - /// Defaults to the earliest available timestamp. - pub start_time: Option, - /// Optional timestamp in nanos at which to end reading. - /// - /// Defaults to reading until the source is closed. - pub end_time: Option, -} diff --git a/crates/sparrow-session/src/partitioned/session.rs b/crates/sparrow-session/src/partitioned/session.rs index a3351397d..ee77f44e4 100644 --- a/crates/sparrow-session/src/partitioned/session.rs +++ b/crates/sparrow-session/src/partitioned/session.rs @@ -5,11 +5,11 @@ use error_stack::ResultExt; use futures::FutureExt; use hashbrown::HashMap; use sparrow_compiler::NearestMatches; -use sparrow_interfaces::Source; +use sparrow_interfaces::{ExecutionOptions, Source}; use sparrow_logical::{ExprRef, Literal}; use uuid::Uuid; -use crate::{Error, Execution, ExecutionOptions}; +use crate::{Error, Execution}; /// Session for creating and executing partitioned queries. pub struct Session { @@ -78,7 +78,7 @@ impl Session { pub fn execute( &self, query: &ExprRef, - _options: ExecutionOptions, + execution_options: ExecutionOptions, ) -> error_stack::Result { let plan = sparrow_backend::compile(query, None).change_context(Error::Compile)?; @@ -88,6 +88,7 @@ impl Session { plan, &self.sources, output_tx, + Arc::new(execution_options), ) .change_context(Error::Execute)?; diff --git a/crates/sparrow-session/src/session.rs b/crates/sparrow-session/src/session.rs index f95f1a3c2..bdccd664a 100644 --- a/crates/sparrow-session/src/session.rs +++ b/crates/sparrow-session/src/session.rs @@ -1,4 +1,5 @@ use hashbrown::HashMap; +use sparrow_interfaces::{ExecutionOptions, Results}; use sparrow_runtime::stores::{ObjectStoreRegistry, ObjectStoreUrl}; use std::borrow::Cow; use std::sync::Arc; @@ -68,13 +69,6 @@ impl Default for Session { } } -#[derive(Default)] -pub enum Results { - #[default] - History, - Snapshot, -} - #[derive(Debug, PartialEq, Eq)] pub enum ExplanationKind { InitialDfg, @@ -82,36 +76,6 @@ pub enum ExplanationKind { FinalPlan, } -#[derive(Default)] -pub struct ExecutionOptions { - /// The maximum number of rows to return. - pub row_limit: Option, - /// The maximum number of rows to return in a single batch. - pub max_batch_size: Option, - /// Whether to run execute as a materialization or not. - pub materialize: bool, - /// History or Snapshot results. - pub results: Results, - /// The changed since time. This is the minimum timestamp of changes to events. - /// For historic queries, this limits the output points. - /// For snapshot queries, this limits the set of entities that are considered changed. - pub changed_since_time_s: Option, - /// The final at time. This is the maximum timestamp output. - /// For historic queries, this limits the output points. - /// For snapshot queries, this determines the time at which the snapshot is produced. - pub final_at_time_s: Option, -} - -impl ExecutionOptions { - pub fn per_entity_behavior(&self) -> PerEntityBehavior { - match self.results { - Results::History => PerEntityBehavior::All, - Results::Snapshot if self.final_at_time_s.is_some() => PerEntityBehavior::FinalAtTime, - Results::Snapshot => PerEntityBehavior::Final, - } - } -} - /// Adds a table to the session. impl Session { pub fn add_literal(&mut self, literal: Literal) -> error_stack::Result { @@ -650,7 +614,20 @@ static RECORD_EXTENSION_ARGUMENTS: [Located; 2] = [ #[static_init::dynamic] static CAST_ARGUMENTS: [Located; 1] = [Located::internal_string("input")]; -impl ExecutionOptions { +trait ExecutionOptionsExt { + fn per_entity_behavior(&self) -> PerEntityBehavior; + fn to_sparrow_options(&self) -> sparrow_runtime::execute::ExecutionOptions; +} + +impl ExecutionOptionsExt for ExecutionOptions { + fn per_entity_behavior(&self) -> PerEntityBehavior { + match self.results { + Results::History => PerEntityBehavior::All, + Results::Snapshot if self.final_at_time_s.is_some() => PerEntityBehavior::FinalAtTime, + Results::Snapshot => PerEntityBehavior::Final, + } + } + fn to_sparrow_options(&self) -> sparrow_runtime::execute::ExecutionOptions { let mut options = sparrow_runtime::execute::ExecutionOptions { max_batch_size: self.max_batch_size, diff --git a/crates/sparrow-sources/src/in_memory.rs b/crates/sparrow-sources/src/in_memory.rs index c6d405a17..6c6d40d14 100644 --- a/crates/sparrow-sources/src/in_memory.rs +++ b/crates/sparrow-sources/src/in_memory.rs @@ -6,7 +6,7 @@ use error_stack::{IntoReportCompat, ResultExt}; use futures::{Stream, StreamExt, TryStreamExt}; use sparrow_batch::Batch; -use sparrow_interfaces::{ReadConfig, Source}; +use sparrow_interfaces::{ExecutionOptions, Source}; use sparrow_merge::old::homogeneous_merge; use sparrow_interfaces::SourceError; @@ -50,7 +50,7 @@ impl Source for InMemory { fn read( &self, projected_datatype: &DataType, - read_config: Arc, + execution_options: Arc, ) -> futures::stream::BoxStream<'static, error_stack::Result> { assert_eq!( &DataType::Struct(self.prepared_schema().fields().clone()), @@ -58,7 +58,7 @@ impl Source for InMemory { "Projection not yet supported" ); - let input_stream = if read_config.keep_open { + let input_stream = if execution_options.materialize { self.data .subscribe() .map_err(|e| e.change_context(SourceError::internal_msg("invalid input"))) From 9a86aa4100f95541ba1fd9f03a81d2bbcb3f2e15 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Tue, 17 Oct 2023 11:26:38 -0700 Subject: [PATCH 10/14] more fixes / renamings --- crates/sparrow-execution/src/lib.rs | 6 +++--- crates/sparrow-scheduler/src/worker_pool.rs | 17 +++++++++-------- python/Cargo.lock | 1 + python/Cargo.toml | 1 + python/src/expr.rs | 10 +++++----- 5 files changed, 19 insertions(+), 16 deletions(-) diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index af0a3f0f7..107432aab 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use error_stack::ResultExt; use itertools::Itertools; -use sparrow_scheduler::{InputHandles, WorkerPool}; +use sparrow_scheduler::{InputHandles, WorkerPoolBuilder}; use uuid::Uuid; mod error; @@ -36,7 +36,7 @@ use crate::source_tasks::SourceTasks; use crate::write_channel_pipeline::WriteChannelPipeline; pub struct PlanExecutor { - worker_pool: WorkerPool, + worker_pool: WorkerPoolBuilder, source_tasks: SourceTasks, execution_options: Arc, } @@ -75,7 +75,7 @@ impl PlanExecutor { execution_options: Arc, ) -> error_stack::Result { let mut executor = PlanExecutor { - worker_pool: WorkerPool::new(query_id).change_context(Error::Creating)?, + worker_pool: WorkerPoolBuilder::new(query_id).change_context(Error::Creating)?, source_tasks: SourceTasks::default(), execution_options: execution_options.clone(), }; diff --git a/crates/sparrow-scheduler/src/worker_pool.rs b/crates/sparrow-scheduler/src/worker_pool.rs index eeb6553ca..0d7d119bd 100644 --- a/crates/sparrow-scheduler/src/worker_pool.rs +++ b/crates/sparrow-scheduler/src/worker_pool.rs @@ -11,7 +11,7 @@ const DEFAULT_THREAD_COUNT: usize = 8; /// Number of slots each thread should have in it's local task queue. const LOCAL_QUEUE_SIZE: u16 = 32; -pub struct WorkerPool { +pub struct WorkerPoolBuilder { query_id: String, injector: Injector, workers: Vec, @@ -19,7 +19,7 @@ pub struct WorkerPool { pipelines: Vec>, } -impl WorkerPool { +impl WorkerPoolBuilder { /// Create a worker pool. /// /// Args: @@ -93,7 +93,7 @@ impl WorkerPool { /// Start executing the pipelines. /// /// Returns a `RunningWorkers` used for completing the workers. - pub fn start(self) -> error_stack::Result { + pub fn start(self) -> error_stack::Result { let Self { workers, query_id, @@ -131,7 +131,7 @@ impl WorkerPool { })?; } - Ok(RunningWorkers { + Ok(WorkerPool { query_id, _pipelines: pipelines, monitor, @@ -140,7 +140,7 @@ impl WorkerPool { } } -pub struct RunningWorkers { +pub struct WorkerPool { query_id: String, /// Hold the Arcs for the pipelines so they aren't dropped. _pipelines: Vec>, @@ -148,7 +148,7 @@ pub struct RunningWorkers { injector: Injector, } -impl RunningWorkers { +impl WorkerPool { /// Mark the sources as complete and wait for workres to finish. /// /// This should not be called until after all source tasks have been @@ -173,7 +173,8 @@ mod tests { use sparrow_batch::{Batch, RowTime}; use crate::{ - Error, Partition, Partitioned, Pipeline, PipelineError, Scheduler, TaskRef, WorkerPool, + Error, Partition, Partitioned, Pipeline, PipelineError, Scheduler, TaskRef, + WorkerPoolBuilder, }; #[derive(Debug, Default)] @@ -219,7 +220,7 @@ mod tests { async fn test_pipeline_panic() { sparrow_testing::init_test_logging(); - let mut workers = WorkerPool::new("query".to_owned()).unwrap(); + let mut workers = WorkerPoolBuilder::new("query".to_owned()).unwrap(); let pipeline = workers.add_pipeline(1, PanicPipeline::default()); let mut injector = workers.injector().clone(); let workers = workers.start().unwrap(); diff --git a/python/Cargo.lock b/python/Cargo.lock index 2f442586f..eec422523 100644 --- a/python/Cargo.lock +++ b/python/Cargo.lock @@ -2029,6 +2029,7 @@ dependencies = [ "pyo3", "pyo3-asyncio", "sparrow-instructions", + "sparrow-interfaces", "sparrow-runtime", "sparrow-session", "sparrow-syntax", diff --git a/python/Cargo.toml b/python/Cargo.toml index e18057770..6a2ffd16b 100644 --- a/python/Cargo.toml +++ b/python/Cargo.toml @@ -25,6 +25,7 @@ pyo3 = {version = "0.19.1", features = ["abi3-py38", "extension-module", "genera pyo3-asyncio = { version = "0.19.0", features = ["tokio-runtime"] } sparrow-session = { path = "../crates/sparrow-session" } sparrow-instructions = { path = "../crates/sparrow-instructions" } +sparrow-interfaces = { path = "../crates/sparrow-interfaces" } sparrow-runtime = { path = "../crates/sparrow-runtime" } sparrow-syntax = { path = "../crates/sparrow-syntax" } tokio = { version = "1.27.0", features = ["sync"] } diff --git a/python/src/expr.rs b/python/src/expr.rs index 35602ce6c..c4ee43098 100644 --- a/python/src/expr.rs +++ b/python/src/expr.rs @@ -189,9 +189,9 @@ impl Literal { } } -fn extract_options(options: Option<&PyAny>) -> Result { +fn extract_options(options: Option<&PyAny>) -> Result { match options { - None => Ok(sparrow_session::ExecutionOptions::default()), + None => Ok(sparrow_interfaces::ExecutionOptions::default()), Some(options) => { let py = options.py(); let row_limit = pyo3::intern!(py, "row_limit"); @@ -203,8 +203,8 @@ fn extract_options(options: Option<&PyAny>) -> Result sparrow_session::Results::History, - "snapshot" => sparrow_session::Results::Snapshot, + "history" => sparrow_interfaces::Results::History, + "snapshot" => sparrow_interfaces::Results::Snapshot, invalid => { return Err( PyValueError::new_err(format!("invalid results '{invalid}'")).into(), @@ -212,7 +212,7 @@ fn extract_options(options: Option<&PyAny>) -> Result Date: Tue, 17 Oct 2023 11:39:40 -0700 Subject: [PATCH 11/14] assign types to merge inputs --- .../src/logical_to_physical.rs | 20 ++++- ...sts__logical_to_physical_arithmetic-3.snap | 82 ++++++++++++++++++- 2 files changed, 97 insertions(+), 5 deletions(-) diff --git a/crates/sparrow-backend/src/logical_to_physical.rs b/crates/sparrow-backend/src/logical_to_physical.rs index 21c0ef93c..46bc9393d 100644 --- a/crates/sparrow-backend/src/logical_to_physical.rs +++ b/crates/sparrow-backend/src/logical_to_physical.rs @@ -28,7 +28,7 @@ //! For example, if the sequence of expressions exceeded a certain //! threshold, or contained a UDF, we could introduce a projection. -use arrow_schema::DataType; +use arrow_schema::{DataType, Field}; use egg::ENodeOrVar; use itertools::Itertools; use smallvec::smallvec; @@ -136,11 +136,21 @@ impl LogicalToPhysical { // of binary merges by choosing the order to perform the merges based on the number // of rows. let inputs: Vec<_> = must_merge.into_iter().collect(); + let result_fields: arrow_schema::Fields = inputs + .iter() + .map(|input_step| { + self.step_type(*input_step).map(|data_type| { + Field::new(format!("step_{}", input_step), data_type, true) + }) + }) + .try_collect()?; + let result_type = DataType::Struct(result_fields); + let merged_step = self.plan.get_or_create_step_id( sparrow_physical::StepKind::Merge, inputs.clone(), ExprVec::empty(), - DataType::Null, + result_type, ); let exprs = args .into_iter() @@ -343,13 +353,17 @@ impl LogicalToPhysical { Ok(plan) } + fn step_type(&self, step_id: sparrow_physical::StepId) -> error_stack::Result { + Ok(self.plan.step_result_type(step_id).clone()) + } + fn reference_type(&self, reference: &Reference) -> error_stack::Result { match reference.expr.last() { ENodeOrVar::Var(var) if *var == *crate::exprs::INPUT_VAR => { let reference_step_id = reference .step_id .ok_or_else(|| Error::internal("literal expressions don't have `?input`"))?; - Ok(self.plan.step_result_type(reference_step_id).clone()) + self.step_type(reference_step_id) } ENodeOrVar::Var(other) => { error_stack::bail!(Error::internal(format!( diff --git a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap index 7abb6b36e..9e2b3d8ef 100644 --- a/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap +++ b/crates/sparrow-backend/src/snapshots/sparrow_backend__logical_to_physical__tests__logical_to_physical_arithmetic-3.snap @@ -48,7 +48,46 @@ steps: inputs: - 0 - 1 - result_type: "Null" + result_type: + Struct: + - name: step_0 + data_type: + Struct: + - name: x + data_type: Int64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: y + data_type: Float64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + nullable: true + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: step_1 + data_type: + Struct: + - name: x + data_type: Int64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: y + data_type: Float64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + nullable: true + dict_id: 0 + dict_is_ordered: false + metadata: {} exprs: [] - id: 3 kind: project @@ -59,7 +98,46 @@ steps: - name: input literal_args: [] args: [] - result_type: "Null" + result_type: + Struct: + - name: step_0 + data_type: + Struct: + - name: x + data_type: Int64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: y + data_type: Float64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + nullable: true + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: step_1 + data_type: + Struct: + - name: x + data_type: Int64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + - name: y + data_type: Float64 + nullable: false + dict_id: 0 + dict_is_ordered: false + metadata: {} + nullable: true + dict_id: 0 + dict_is_ordered: false + metadata: {} - name: fieldref literal_args: - Utf8: step_1 From 648e7d83bb0610c29c9c0747b2c0b0a6f4bf3f62 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Tue, 17 Oct 2023 13:15:03 -0700 Subject: [PATCH 12/14] comments --- crates/sparrow-execution/src/source_tasks.rs | 11 ----------- crates/sparrow-physical/src/step.rs | 8 +++++--- crates/sparrow-scheduler/src/worker.rs | 18 +++++++++--------- crates/sparrow-scheduler/src/worker_pool.rs | 2 +- 4 files changed, 15 insertions(+), 24 deletions(-) diff --git a/crates/sparrow-execution/src/source_tasks.rs b/crates/sparrow-execution/src/source_tasks.rs index 40c158108..1bc8fdb87 100644 --- a/crates/sparrow-execution/src/source_tasks.rs +++ b/crates/sparrow-execution/src/source_tasks.rs @@ -13,17 +13,6 @@ use crate::Error; #[derive(Default)] pub(super) struct SourceTasks { tasks: Vec, - // /// Streams to be read. Each stream should produce a `(batch, index)` pair, - // /// with indices corresponding to the input in `read_inputs` to send the batch - // /// to. - // /// - // // TODO: We may want to encapsulate this differently / better. Could we use - // // `futures::sink(...)` to do this in some way? We may also want to allow - // // the polling of these sources to be weighted towards the inputs that need - // // a new batch to progress (eg., back-pressure). - // reads: Vec>>, - // /// For each read operation, the `PipelineInputs` to send the batches to. - // read_inputs: Vec, } struct SourceTask { diff --git a/crates/sparrow-physical/src/step.rs b/crates/sparrow-physical/src/step.rs index 64d4221b1..7cb449600 100644 --- a/crates/sparrow-physical/src/step.rs +++ b/crates/sparrow-physical/src/step.rs @@ -117,9 +117,11 @@ impl StepKind { /// - `filter` is a transform because it removes rows from each batch, and omits /// empty batches /// - `merge` is not a transform because it accepts multiple inputs - /// - `shift` may be a transform if it uses the time in the input batch to determine - /// which rows to output or it may be an operation if it interacts with scheduling - /// in a more sophisticated way. + /// - `shift` depends on how we choose to implement it. If it is implemented as + /// a stateful transform that just buffers and emits as processing proceeds + /// through time, then it would be a transform. If we find ways to implement + /// it more efficiently by implementing the pipeline interface, then it may + /// not be a transform. pub fn is_transform(&self) -> bool { matches!(self, StepKind::Project | StepKind::Filter) } diff --git a/crates/sparrow-scheduler/src/worker.rs b/crates/sparrow-scheduler/src/worker.rs index be2728a8f..839977e8b 100644 --- a/crates/sparrow-scheduler/src/worker.rs +++ b/crates/sparrow-scheduler/src/worker.rs @@ -65,10 +65,10 @@ impl Injector { impl Scheduler for Injector { fn schedule_global(&self, task: TaskRef) { if task.schedule() { - tracing::trace!("1 Added {task:?} to queue"); + tracing::trace!("Added {task:?} to queue"); self.queue.push(task); } else { - tracing::trace!("1 {task:?} already executing"); + tracing::trace!("{task:?} already executing"); } self.idle_workers.wake_one(); } @@ -93,7 +93,7 @@ impl Worker { pub(crate) fn work_loop(mut self, index: usize) -> error_stack::Result<(), Error> { let thread_id = std::thread::current().id(); let _span = tracing::info_span!("Worker", ?thread_id, index).entered(); - tracing::info!("Starting work loop on thread {thread_id:?}"); + tracing::info!("Starting work loop"); loop { while let Some(task) = self.queue.pop() { if task.do_work(&mut self)? { @@ -130,30 +130,30 @@ impl Worker { impl Scheduler for Worker { fn schedule(&mut self, task: TaskRef) { if task.schedule() { - tracing::trace!("2 Added {task:?} to queue"); + tracing::trace!("Added {task:?} to queue"); self.queue.push(task); } else { - tracing::trace!("2 {task:?} already executing"); + tracing::trace!("{task:?} already executing"); } self.idle_workers.wake_one(); } fn schedule_yield(&mut self, task: TaskRef) { if task.schedule() { - tracing::trace!("3 Added {task:?} to queue"); + tracing::trace!("Added {task:?} to queue"); self.queue.push_yield(task); } else { - tracing::trace!("3 {task:?} already executing"); + tracing::trace!("{task:?} already executing"); } self.idle_workers.wake_one(); } fn schedule_global(&self, task: TaskRef) { if task.schedule() { - tracing::trace!("4 Added {task:?} to queue"); + tracing::trace!("Added {task:?} to queue"); self.queue.push_global(task); } else { - tracing::trace!("4 {task:?} already executing"); + tracing::trace!("{task:?} already executing"); } self.idle_workers.wake_one(); } diff --git a/crates/sparrow-scheduler/src/worker_pool.rs b/crates/sparrow-scheduler/src/worker_pool.rs index 0d7d119bd..713bb39f1 100644 --- a/crates/sparrow-scheduler/src/worker_pool.rs +++ b/crates/sparrow-scheduler/src/worker_pool.rs @@ -20,7 +20,7 @@ pub struct WorkerPoolBuilder { } impl WorkerPoolBuilder { - /// Create a worker pool. + /// Create a worker pool builder. /// /// Args: /// query_id: The query ID associated with this worker pool. Used as a From 297594fc427facee6659c95fc9e5e3a9c741552e Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Tue, 17 Oct 2023 13:15:34 -0700 Subject: [PATCH 13/14] comments --- crates/sparrow-scheduler/src/worker_pool.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/sparrow-scheduler/src/worker_pool.rs b/crates/sparrow-scheduler/src/worker_pool.rs index 713bb39f1..2551a5754 100644 --- a/crates/sparrow-scheduler/src/worker_pool.rs +++ b/crates/sparrow-scheduler/src/worker_pool.rs @@ -92,7 +92,7 @@ impl WorkerPoolBuilder { /// Start executing the pipelines. /// - /// Returns a `RunningWorkers` used for completing the workers. + /// Returns a `WorkerPool` used for completing the workers. pub fn start(self) -> error_stack::Result { let Self { workers, From 378bfe2a8031e34250e78467780cfc1aac358d10 Mon Sep 17 00:00:00 2001 From: Ben Chambers <35960+bjchambers@users.noreply.github.com> Date: Tue, 17 Oct 2023 13:35:07 -0700 Subject: [PATCH 14/14] fix comment --- crates/sparrow-execution/src/lib.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/crates/sparrow-execution/src/lib.rs b/crates/sparrow-execution/src/lib.rs index 107432aab..1c1ac8c63 100644 --- a/crates/sparrow-execution/src/lib.rs +++ b/crates/sparrow-execution/src/lib.rs @@ -218,7 +218,8 @@ impl PlanExecutor { }) ); - // Determine the producer of the + // Determine the producer of the input to the transform pipeline. + // This is the input of the first step in the pipeline. let first_step_inputs = &plan.steps[*steps.first().expect("at least one step")].inputs; debug_assert_eq!(first_step_inputs.len(), 1); let producer_id = first_step_inputs[0];