From 10d7faa3b3b6a29d10785f1503a8b007c495b7a0 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 20 Apr 2024 18:25:41 +0800 Subject: [PATCH 01/29] stash Signed-off-by: Runji Wang --- src/planner/explain.rs | 10 ++ src/planner/mod.rs | 10 ++ src/planner/partition.rs | 244 +++++++++++++++++++++++++++++++++++++++ 3 files changed, 264 insertions(+) create mode 100644 src/planner/partition.rs diff --git a/src/planner/explain.rs b/src/planner/explain.rs index 2cb2519f..b30f4436 100644 --- a/src/planner/explain.rs +++ b/src/planner/explain.rs @@ -339,6 +339,16 @@ impl<'a> Explain<'a> { vec![("windows", self.expr(windows).pretty())].with(cost, rows), vec![self.child(child).pretty()], ), + Exchange([dist, child]) => Pretty::simple_record( + "Exchange", + vec![("dist", self.expr(dist).pretty())].with(cost, rows), + vec![self.child(child).pretty()], + ), + Broadcast => Pretty::display(enode), + Hash(keys) => { + Pretty::childless_record("Hash", vec![("keys", self.expr(keys).pretty())]) + } + CreateTable(t) => { let fields = t.pretty_table().with(cost, rows); Pretty::childless_record("CreateTable", fields) diff --git a/src/planner/mod.rs b/src/planner/mod.rs index 23e5061d..a6f7855a 100644 --- a/src/planner/mod.rs +++ b/src/planner/mod.rs @@ -11,6 +11,7 @@ use crate::types::{ColumnIndex, DataType, DataValue, DateTimeField}; mod cost; mod explain; mod optimizer; +mod partition; mod rules; pub use explain::Explain; @@ -118,6 +119,15 @@ define_language! { // child must be ordered by keys "window" = Window([Id; 2]), // (window [over..] child) // output = child || exprs + + // parallelism + "exchange" = Exchange([Id; 2]), // (exchange dist child) + "to_dist" = ToDist(Id), // (to_dist child) + "single" = Single, // (single) merge all to one + "broadcast" = Broadcast, // (broadcast) broadcast to all + "random" = Random, // (random) random partition + "hash" = Hash(Id), // (hash key=[expr..]) partition by hash of key + CreateTable(Box), "create_view" = CreateView([Id; 2]), // (create_view create_table child) CreateFunction(CreateFunction), diff --git a/src/planner/partition.rs b/src/planner/partition.rs new file mode 100644 index 00000000..09eda8a8 --- /dev/null +++ b/src/planner/partition.rs @@ -0,0 +1,244 @@ +// Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. + +//! This module converts physical plans into distributed plans. +//! +//! `Exchange` nodes are inserted to shuffle data between partitions. + +use egg::{rewrite as rw, Analysis, EGraph, Language}; + +use super::*; +use crate::planner::RecExpr; + +/// Converts a physical plan into a distributed plan. +pub fn to_distributed(plan: RecExpr) -> RecExpr { + Builder::new(plan).build() +} + +/// This builder converts a physical plan into a distributed plan recursively. +struct Builder { + egraph: egg::EGraph, + root: Id, +} + +impl Builder { + /// Creates a new builder for the given physical plan. + fn new(plan: RecExpr) -> Self { + let mut egraph = egg::EGraph::new(PartitionAnalysis); + let root = egraph.add_expr(&plan); + Self { egraph, root } + } + + fn build(mut self) -> RecExpr { + let root = self.build_id(self.root, &[]); + self.recexpr(root) + } + + /// Get the node from id. + fn node(&self, id: Id) -> &Expr { + // each e-class has exactly one node since there is no rewrite or union. + &self.egraph[id].nodes[0] + } + + /// Extract a `RecExpr` from id. + fn recexpr(&self, id: Id) -> RecExpr { + self.node(id).build_recexpr(|id| self.node(id).clone()) + } + + /// Builds a distributed plan for the given physical `plan`, + /// requiring that it must be partitioned by `partition_key`. + /// Returns the node id in the egraph. + fn build_id(&mut self, plan: Id, partition_key: &[Id]) -> Id { + use Expr::*; + let new_plan = match self.node(plan).clone() { + Proj([proj, child]) => { + let child = self.build_id(child, partition_key); + self.egraph.add(Proj([proj, child])) + } + Filter([cond, child]) => { + let child = self.build_id(child, partition_key); + self.egraph.add(Filter([cond, child])) + } + Order([key, child]) => { + let child = self.build_id(child, partition_key); + self.egraph.add(Order([key, child])) + } + Limit([limit, offset, child]) => { + let child = self.build_id(child, partition_key); + self.egraph.add(Limit([limit, offset, child])) + } + HashJoin([t, cond, lkey, rkey, left, right]) => { + let lpartition = self.node(lkey).as_list().to_vec(); + let rpartition = self.node(rkey).as_list().to_vec(); + let left = self.build_id(left, &lpartition); + let right = self.build_id(right, &rpartition); + self.egraph + .add(HashJoin([t, cond, lkey, rkey, left, right])) + } + _ => todo!(), + }; + if !partition_key.is_empty() && &self.egraph[new_plan].data[..] != partition_key { + let list = self.egraph.add(List(partition_key.into())); + let dist = self.egraph.add(Hash(list)); + self.egraph.add(Exchange([dist, new_plan])) + } else { + new_plan + } + } +} + +fn exchange_rules() -> Vec { + vec![rw!("exchange-single"; + "(exchange ?dist (exchange ?any ?child))" => + "(exchange ?dist ?child)" + // FIXME: what if ?any = broadcast + )] +} + +fn to_dist_rules() -> Vec { + vec![ + rw!("scan-to-dist"; + "(to_dist (scan ?table ?columns ?filter))" => + "(exchange random (scan ?table ?columns ?filter))" + ), + rw!("proj-to-dist"; + "(to_dist (proj ?projs ?child))" => + "(proj ?projs (to_dist ?child))" + ), + rw!("filter-to-dist"; + "(to_dist (filter ?cond ?child))" => + "(filter ?cond (to_dist ?child))" + ), + rw!("order-to-dist"; + "(to_dist (order ?key ?child))" => + "(order ?key (exchange single (order ?key (to_dist ?child))))" + // TODO: merge sort in the second phase? + ), + rw!("limit-to-dist"; + "(to_dist (limit ?limit ?offset ?child))" => + "(limit ?limit ?offset (exchange single (to_dist ?child)))" + ), + rw!("topn-to-dist"; + "(to_dist (topn ?limit ?offset ?key ?child))" => + "(topn ?limit ?offset ?key (exchange single (to_dist ?child)))" + ), + // inner join can be partitioned by left key + rw!("inner-join-to-dist-left"; + "(to_dist (join inner ?cond ?left ?right))" => + "(join inner ?cond + (exchange random (to_dist ?left)) + (exchange broadcast (to_dist ?right)))" + ), + // ... or by right key + rw!("inner-join-to-dist-right"; + "(to_dist (join inner ?cond ?left ?right))" => + "(join inner ?cond + (exchange broadcast (to_dist ?left)) + (exchange random (to_dist ?right)))" + ), + // outer join can not be partitioned + rw!("join-to-dist-left"; + "(to_dist (join full_outer ?cond ?left ?right))" => + "(join full_outer ?cond + (exchange single (to_dist ?left)) + (exchange single (to_dist ?right)))" + ), + // hash join can be partitioned by join key + rw!("hashjoin-to-dist"; + "(to_dist (hashjoin ?type ?cond ?lkey ?rkey ?left ?right))" => + "(hashjoin ?type ?cond ?lkey ?rkey + (exchange (hash ?lkey) (to_dist ?left)) + (exchange (hash ?rkey) (to_dist ?right)))" + ), + // merge join can be partitioned by join key + rw!("mergejoin-to-dist"; + "(to_dist (mergejoin ?type ?cond ?lkey ?rkey ?left ?right))" => + "(mergejoin ?type ?cond ?lkey ?rkey + (exchange (hash ?lkey) (to_dist ?left)) + (exchange (hash ?rkey) (to_dist ?right)))" + ), + // 2-phase aggregation + rw!("agg-to-dist"; + "(to_dist (agg ?exprs ?child))" => + "(agg ?exprs (exchange single (agg ?exprs (exchange random (to_dist ?child)))))" + ), + // hash aggregation can be partitioned by group key + rw!("hashagg-to-dist"; + "(to_dist (hashagg ?keys ?aggs ?child))" => + "(hashagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" + ), + // sort aggregation can be partitioned by group key + rw!("sortagg-to-dist"; + "(to_dist (sortagg ?keys ?aggs ?child))" => + "(sortagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" + ), + // window function can not be partitioned for now + rw!("window-to-dist"; + "(to_dist (window ?exprs ?child))" => + "(window ?exprs (exchange single (to_dist ?child)))" + ), + ] +} + +struct PartitionAnalysis; + +impl Analysis for PartitionAnalysis { + type Data = Box<[Id]>; + + /// Analyze a node and give the result. + fn make(egraph: &EGraph, enode: &Expr) -> Self::Data { + use Expr::*; + let x = |c: &Id| egraph[*c].data.clone(); + match enode { + // list is the source for the following nodes + List(ids) => ids.clone(), + + // equal to child + Proj([_, c]) | Agg([_, c]) | Filter([_, c]) | Order([_, c]) | Limit([_, _, c]) + | TopN([_, _, _, c]) | Empty(c) | Window([_, c]) => x(c), + + Join(_) | Apply(_) => Box::new([]), + Scan(_) | Values(_) => Box::new([]), + + HashJoin([_, _, lkey, _, _, _]) | MergeJoin([_, _, lkey, _, _, _]) => x(lkey), + HashAgg([key, _, _]) | SortAgg([key, _, _]) => x(key), + + // not plan node + _ => Box::new([]), + } + } + + fn merge(&mut self, a: &mut Self::Data, b: Self::Data) -> egg::DidMerge { + unimplemented!("merge should not be called") + } +} + +// /// The data type of partition analysis. +// pub type PartitionKey = Box<[Id]>; + +// /// Returns all columns involved in the node. +// pub fn analyze_partition_key(enode: &Expr, x: impl Fn(&Id) -> PartitionKey) -> PartitionKey {} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_to_distributed() { + let input = " + (hashjoin inner true (list a) (list b) + (scan t1 (list a) true) + (scan t2 (list b) true) + ) + "; + let distributed = " + (hashjoin inner true (list a) (list b) + (exchange (hash (list a)) + (scan t1 (list a) true)) + (exchange (hash (list b)) + (scan t2 (list b) true)) + ) + "; + let output = to_distributed(input.parse().unwrap()); + println!("{}", output.pretty(60)); + } +} From f608270b5283a9b7a0f696db1eb811377e27397f Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 21 Apr 2024 00:02:51 +0800 Subject: [PATCH 02/29] basic support for converting to distributed plan Signed-off-by: Runji Wang --- src/planner/explain.rs | 5 +- src/planner/mod.rs | 1 - src/planner/partition.rs | 244 ------------------------------ src/planner/rules/mod.rs | 5 +- src/planner/rules/partition.rs | 261 +++++++++++++++++++++++++++++++++ 5 files changed, 269 insertions(+), 247 deletions(-) delete mode 100644 src/planner/partition.rs create mode 100644 src/planner/rules/partition.rs diff --git a/src/planner/explain.rs b/src/planner/explain.rs index b30f4436..2f57d106 100644 --- a/src/planner/explain.rs +++ b/src/planner/explain.rs @@ -344,7 +344,10 @@ impl<'a> Explain<'a> { vec![("dist", self.expr(dist).pretty())].with(cost, rows), vec![self.child(child).pretty()], ), - Broadcast => Pretty::display(enode), + ToDist(child) => { + Pretty::simple_record("ToDist", vec![], vec![self.child(child).pretty()]) + } + Single | Broadcast | Random => Pretty::display(enode), Hash(keys) => { Pretty::childless_record("Hash", vec![("keys", self.expr(keys).pretty())]) } diff --git a/src/planner/mod.rs b/src/planner/mod.rs index a6f7855a..a3fe848b 100644 --- a/src/planner/mod.rs +++ b/src/planner/mod.rs @@ -11,7 +11,6 @@ use crate::types::{ColumnIndex, DataType, DataValue, DateTimeField}; mod cost; mod explain; mod optimizer; -mod partition; mod rules; pub use explain::Explain; diff --git a/src/planner/partition.rs b/src/planner/partition.rs deleted file mode 100644 index 09eda8a8..00000000 --- a/src/planner/partition.rs +++ /dev/null @@ -1,244 +0,0 @@ -// Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. - -//! This module converts physical plans into distributed plans. -//! -//! `Exchange` nodes are inserted to shuffle data between partitions. - -use egg::{rewrite as rw, Analysis, EGraph, Language}; - -use super::*; -use crate::planner::RecExpr; - -/// Converts a physical plan into a distributed plan. -pub fn to_distributed(plan: RecExpr) -> RecExpr { - Builder::new(plan).build() -} - -/// This builder converts a physical plan into a distributed plan recursively. -struct Builder { - egraph: egg::EGraph, - root: Id, -} - -impl Builder { - /// Creates a new builder for the given physical plan. - fn new(plan: RecExpr) -> Self { - let mut egraph = egg::EGraph::new(PartitionAnalysis); - let root = egraph.add_expr(&plan); - Self { egraph, root } - } - - fn build(mut self) -> RecExpr { - let root = self.build_id(self.root, &[]); - self.recexpr(root) - } - - /// Get the node from id. - fn node(&self, id: Id) -> &Expr { - // each e-class has exactly one node since there is no rewrite or union. - &self.egraph[id].nodes[0] - } - - /// Extract a `RecExpr` from id. - fn recexpr(&self, id: Id) -> RecExpr { - self.node(id).build_recexpr(|id| self.node(id).clone()) - } - - /// Builds a distributed plan for the given physical `plan`, - /// requiring that it must be partitioned by `partition_key`. - /// Returns the node id in the egraph. - fn build_id(&mut self, plan: Id, partition_key: &[Id]) -> Id { - use Expr::*; - let new_plan = match self.node(plan).clone() { - Proj([proj, child]) => { - let child = self.build_id(child, partition_key); - self.egraph.add(Proj([proj, child])) - } - Filter([cond, child]) => { - let child = self.build_id(child, partition_key); - self.egraph.add(Filter([cond, child])) - } - Order([key, child]) => { - let child = self.build_id(child, partition_key); - self.egraph.add(Order([key, child])) - } - Limit([limit, offset, child]) => { - let child = self.build_id(child, partition_key); - self.egraph.add(Limit([limit, offset, child])) - } - HashJoin([t, cond, lkey, rkey, left, right]) => { - let lpartition = self.node(lkey).as_list().to_vec(); - let rpartition = self.node(rkey).as_list().to_vec(); - let left = self.build_id(left, &lpartition); - let right = self.build_id(right, &rpartition); - self.egraph - .add(HashJoin([t, cond, lkey, rkey, left, right])) - } - _ => todo!(), - }; - if !partition_key.is_empty() && &self.egraph[new_plan].data[..] != partition_key { - let list = self.egraph.add(List(partition_key.into())); - let dist = self.egraph.add(Hash(list)); - self.egraph.add(Exchange([dist, new_plan])) - } else { - new_plan - } - } -} - -fn exchange_rules() -> Vec { - vec![rw!("exchange-single"; - "(exchange ?dist (exchange ?any ?child))" => - "(exchange ?dist ?child)" - // FIXME: what if ?any = broadcast - )] -} - -fn to_dist_rules() -> Vec { - vec![ - rw!("scan-to-dist"; - "(to_dist (scan ?table ?columns ?filter))" => - "(exchange random (scan ?table ?columns ?filter))" - ), - rw!("proj-to-dist"; - "(to_dist (proj ?projs ?child))" => - "(proj ?projs (to_dist ?child))" - ), - rw!("filter-to-dist"; - "(to_dist (filter ?cond ?child))" => - "(filter ?cond (to_dist ?child))" - ), - rw!("order-to-dist"; - "(to_dist (order ?key ?child))" => - "(order ?key (exchange single (order ?key (to_dist ?child))))" - // TODO: merge sort in the second phase? - ), - rw!("limit-to-dist"; - "(to_dist (limit ?limit ?offset ?child))" => - "(limit ?limit ?offset (exchange single (to_dist ?child)))" - ), - rw!("topn-to-dist"; - "(to_dist (topn ?limit ?offset ?key ?child))" => - "(topn ?limit ?offset ?key (exchange single (to_dist ?child)))" - ), - // inner join can be partitioned by left key - rw!("inner-join-to-dist-left"; - "(to_dist (join inner ?cond ?left ?right))" => - "(join inner ?cond - (exchange random (to_dist ?left)) - (exchange broadcast (to_dist ?right)))" - ), - // ... or by right key - rw!("inner-join-to-dist-right"; - "(to_dist (join inner ?cond ?left ?right))" => - "(join inner ?cond - (exchange broadcast (to_dist ?left)) - (exchange random (to_dist ?right)))" - ), - // outer join can not be partitioned - rw!("join-to-dist-left"; - "(to_dist (join full_outer ?cond ?left ?right))" => - "(join full_outer ?cond - (exchange single (to_dist ?left)) - (exchange single (to_dist ?right)))" - ), - // hash join can be partitioned by join key - rw!("hashjoin-to-dist"; - "(to_dist (hashjoin ?type ?cond ?lkey ?rkey ?left ?right))" => - "(hashjoin ?type ?cond ?lkey ?rkey - (exchange (hash ?lkey) (to_dist ?left)) - (exchange (hash ?rkey) (to_dist ?right)))" - ), - // merge join can be partitioned by join key - rw!("mergejoin-to-dist"; - "(to_dist (mergejoin ?type ?cond ?lkey ?rkey ?left ?right))" => - "(mergejoin ?type ?cond ?lkey ?rkey - (exchange (hash ?lkey) (to_dist ?left)) - (exchange (hash ?rkey) (to_dist ?right)))" - ), - // 2-phase aggregation - rw!("agg-to-dist"; - "(to_dist (agg ?exprs ?child))" => - "(agg ?exprs (exchange single (agg ?exprs (exchange random (to_dist ?child)))))" - ), - // hash aggregation can be partitioned by group key - rw!("hashagg-to-dist"; - "(to_dist (hashagg ?keys ?aggs ?child))" => - "(hashagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" - ), - // sort aggregation can be partitioned by group key - rw!("sortagg-to-dist"; - "(to_dist (sortagg ?keys ?aggs ?child))" => - "(sortagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" - ), - // window function can not be partitioned for now - rw!("window-to-dist"; - "(to_dist (window ?exprs ?child))" => - "(window ?exprs (exchange single (to_dist ?child)))" - ), - ] -} - -struct PartitionAnalysis; - -impl Analysis for PartitionAnalysis { - type Data = Box<[Id]>; - - /// Analyze a node and give the result. - fn make(egraph: &EGraph, enode: &Expr) -> Self::Data { - use Expr::*; - let x = |c: &Id| egraph[*c].data.clone(); - match enode { - // list is the source for the following nodes - List(ids) => ids.clone(), - - // equal to child - Proj([_, c]) | Agg([_, c]) | Filter([_, c]) | Order([_, c]) | Limit([_, _, c]) - | TopN([_, _, _, c]) | Empty(c) | Window([_, c]) => x(c), - - Join(_) | Apply(_) => Box::new([]), - Scan(_) | Values(_) => Box::new([]), - - HashJoin([_, _, lkey, _, _, _]) | MergeJoin([_, _, lkey, _, _, _]) => x(lkey), - HashAgg([key, _, _]) | SortAgg([key, _, _]) => x(key), - - // not plan node - _ => Box::new([]), - } - } - - fn merge(&mut self, a: &mut Self::Data, b: Self::Data) -> egg::DidMerge { - unimplemented!("merge should not be called") - } -} - -// /// The data type of partition analysis. -// pub type PartitionKey = Box<[Id]>; - -// /// Returns all columns involved in the node. -// pub fn analyze_partition_key(enode: &Expr, x: impl Fn(&Id) -> PartitionKey) -> PartitionKey {} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_to_distributed() { - let input = " - (hashjoin inner true (list a) (list b) - (scan t1 (list a) true) - (scan t2 (list b) true) - ) - "; - let distributed = " - (hashjoin inner true (list a) (list b) - (exchange (hash (list a)) - (scan t1 (list a) true)) - (exchange (hash (list b)) - (scan t2 (list b) true)) - ) - "; - let output = to_distributed(input.parse().unwrap()); - println!("{}", output.pretty(60)); - } -} diff --git a/src/planner/rules/mod.rs b/src/planner/rules/mod.rs index 2d84f4b8..e5be74cf 100644 --- a/src/planner/rules/mod.rs +++ b/src/planner/rules/mod.rs @@ -14,7 +14,8 @@ //! | [`schema`] | column id to index | output schema of a plan | [`Schema`] | //! | [`type_`] | | data type | [`Type`] | //! | [`rows`] | | estimated rows | [`Rows`] | -//! | [`order`] | merge join | ordered keys | [`OrderKey`] | +//! | [`order`] | merge join | ordered keys | [`OrderKey`] | +//! | [`partition`] | to_distributed | data partition | [`Partition`] | //! //! It would be best if you have a background in program analysis. //! Here is a recommended course: . @@ -27,6 +28,7 @@ //! [`Type`]: type_::Type //! [`Rows`]: rows::Rows //! [`OrderKey`]: order::OrderKey +//! [`Partition`]: partition::Partition use std::collections::HashSet; use std::hash::Hash; @@ -40,6 +42,7 @@ use crate::types::F32; pub mod agg; pub mod expr; pub mod order; +pub mod partition; pub mod plan; pub mod range; pub mod rows; diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs new file mode 100644 index 00000000..e04f016c --- /dev/null +++ b/src/planner/rules/partition.rs @@ -0,0 +1,261 @@ +// Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. + +//! This module converts physical plans into distributed plans. +//! +//! In a distributed query plan, each node represents one or more physical operators, with each +//! operator processing data from one partition. Each node has a [`Distribution`] property that +//! describes how the data is partitioned. +//! +//! After the conversion, [`Exchange`](Expr::Exchange) nodes will be inserted when necessary to +//! redistribute data between partitions. + +use std::sync::LazyLock; + +use egg::{rewrite as rw, Analysis, EGraph, Language}; + +use super::*; +use crate::planner::RecExpr; + +/// Converts a physical plan into a distributed plan. +pub fn to_distributed(mut plan: RecExpr) -> RecExpr { + // add to_dist to the root node + let root_id = Id::from(plan.as_ref().len() - 1); + plan.add(Expr::ToDist(root_id)); + + let runner = egg::Runner::<_, _, ()>::new(PartitionAnalysis) + .with_expr(&plan) + .run(TO_DIST_RULES.iter()); + let extractor = egg::Extractor::new(&runner.egraph, ExtractDistributedPlan); + let (_, expr) = extractor.find_best(runner.roots[0]); + expr +} + +struct ExtractDistributedPlan; + +impl egg::CostFunction for ExtractDistributedPlan { + type Cost = usize; + fn cost(&mut self, enode: &Expr, mut costs: C) -> Self::Cost + where + C: FnMut(Id) -> Self::Cost, + { + if let Expr::ToDist(_) = enode { + return usize::MAX; + } + enode.fold(1, |sum, id| sum.saturating_add(costs(id))) + } +} + +type Rewrite = egg::Rewrite; + +static TO_DIST_RULES: LazyLock> = LazyLock::new(|| { + vec![ + // scan is not partitioned + rw!("scan-to-dist"; + "(to_dist (scan ?table ?columns ?filter))" => + "(exchange random (scan ?table ?columns ?filter))" + ), + // values is not partitioned + rw!("values-to-dist"; + "(to_dist (values ?values))" => + "(exchange random (values ?values))" + ), + // projection does not change distribution + rw!("proj-to-dist"; + "(to_dist (proj ?projs ?child))" => + "(proj ?projs (to_dist ?child))" + ), + // filter does not change distribution + rw!("filter-to-dist"; + "(to_dist (filter ?cond ?child))" => + "(filter ?cond (to_dist ?child))" + ), + // 2-phase ordering + rw!("order-to-dist"; + "(to_dist (order ?key ?child))" => + "(order ?key (exchange single (order ?key (to_dist ?child))))" + // TODO: merge sort in the second phase? + ), + // limit can not be partitioned + rw!("limit-to-dist"; + "(to_dist (limit ?limit ?offset ?child))" => + "(limit ?limit ?offset (exchange single (to_dist ?child)))" + ), + // topn can not be partitioned + rw!("topn-to-dist"; + "(to_dist (topn ?limit ?offset ?key ?child))" => + "(topn ?limit ?offset ?key (exchange single (to_dist ?child)))" + ), + // inner join is partitioned by left + // as the left side is materialized in memory + rw!("inner-join-to-dist"; + "(to_dist (join inner ?cond ?left ?right))" => + "(join inner ?cond + (exchange random (to_dist ?left)) + (exchange broadcast (to_dist ?right)))" + ), + // outer join can not be partitioned + rw!("join-to-dist"; + "(to_dist (join full_outer ?cond ?left ?right))" => + "(join full_outer ?cond + (exchange single (to_dist ?left)) + (exchange single (to_dist ?right)))" + ), + // hash join can be partitioned by join key + rw!("hashjoin-to-dist"; + "(to_dist (hashjoin ?type ?cond ?lkey ?rkey ?left ?right))" => + "(hashjoin ?type ?cond ?lkey ?rkey + (exchange (hash ?lkey) (to_dist ?left)) + (exchange (hash ?rkey) (to_dist ?right)))" + ), + // merge join can be partitioned by join key + rw!("mergejoin-to-dist"; + "(to_dist (mergejoin ?type ?cond ?lkey ?rkey ?left ?right))" => + "(mergejoin ?type ?cond ?lkey ?rkey + (exchange (hash ?lkey) (to_dist ?left)) + (exchange (hash ?rkey) (to_dist ?right)))" + ), + // 2-phase aggregation + rw!("agg-to-dist"; + "(to_dist (agg ?exprs ?child))" => + "(agg ?exprs (exchange single (agg ?exprs (exchange random (to_dist ?child)))))" + ), + // hash aggregation can be partitioned by group key + rw!("hashagg-to-dist"; + "(to_dist (hashagg ?keys ?aggs ?child))" => + "(hashagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" + ), + // sort aggregation can be partitioned by group key + rw!("sortagg-to-dist"; + "(to_dist (sortagg ?keys ?aggs ?child))" => + "(sortagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" + ), + // window function can not be partitioned for now + rw!("window-to-dist"; + "(to_dist (window ?exprs ?child))" => + "(window ?exprs (exchange single (to_dist ?child)))" + ), + // unnecessary exchange can be removed + rw!("remove-exchange"; + "(exchange ?dist ?child)" => "?child" + if partition_is_same("?child", "?dist") + ), + rw!("dedup-exchange"; + "(exchange ?dist (exchange ?dist2 ?child))" => + "(exchange ?dist ?child)" + ), + ] +}); + +/// Returns true if the distribution of the used columns is the same as the produced columns. +fn partition_is_same( + a: &str, + b: &str, +) -> impl Fn(&mut EGraph, Id, &Subst) -> bool { + let a = var(a); + let b = var(b); + move |egraph, _, subst| egraph[subst[a]].data == egraph[subst[b]].data +} + +/// Describes how data is partitioned. +#[derive(Debug, Default, Clone, PartialEq, Eq)] +pub enum Partition { + /// Distribution is unknown. + #[default] + Unknown, + /// Data is not partitioned. + Single, + /// Data is randomly partitioned. + Random, + /// Data is broadcasted to all partitions. + Broadcast, + /// Data is partitioned by hash of keys. + Hash(Box<[Id]>), +} + +struct PartitionAnalysis; + +impl Analysis for PartitionAnalysis { + type Data = Partition; + + fn make(egraph: &EGraph, enode: &Expr) -> Self::Data { + let x = |id: &Id| egraph[*id].data.clone(); + analyze_partition(enode, x) + } + + fn merge(&mut self, a: &mut Self::Data, b: Self::Data) -> egg::DidMerge { + merge_partition(a, b) + } +} + +/// Returns partition of the given plan node. +pub fn analyze_partition(enode: &Expr, x: impl Fn(&Id) -> Partition) -> Partition { + use Expr::*; + match enode { + // partition nodes + Single => Partition::Single, + Random => Partition::Random, + Broadcast => Partition::Broadcast, + Hash(list) => x(list), + List(ids) => Partition::Hash(ids.clone()), + + // exchange node changes distribution + Exchange([dist, _]) => x(dist), + + // leaf nodes + Scan(_) | Values(_) => Partition::Single, + + // equal to child or left child + Proj([_, c]) + | Filter([_, c]) + | Order([_, c]) + | Limit([_, _, c]) + | TopN([_, _, _, c]) + | Empty(c) + | Window([_, c]) + | Agg([_, c]) + | HashAgg([_, _, c]) + | SortAgg([_, _, c]) + | Join([_, _, c, _]) + | Apply([_, c, _]) + | HashJoin([_, _, _, _, c, _]) + | MergeJoin([_, _, _, _, c, _]) => x(c), + + // not a plan node + _ => Partition::Unknown, + } +} + +fn merge_partition(a: &mut Partition, b: Partition) -> egg::DidMerge { + if *a == Partition::Unknown && b != Partition::Unknown { + *a = b; + egg::DidMerge(true, false) + } else { + egg::DidMerge(false, true) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_to_distributed() { + let input = " + (hashjoin inner true (list a) (list b) + (scan t1 (list a) true) + (scan t2 (list b) true) + ) + "; + let distributed = " + (hashjoin inner true (list a) (list b) + (exchange (hash (list a)) + (scan t1 (list a) true)) + (exchange (hash (list b)) + (scan t2 (list b) true)) + ) + "; + let output = to_distributed(input.parse().unwrap()); + let expected: RecExpr = distributed.parse().unwrap(); + assert_eq!(output.to_string(), expected.to_string()); + } +} From 344a5f82dc2510df98a7b4a663af983858c92e73 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 21 Apr 2024 23:18:45 +0800 Subject: [PATCH 03/29] rename `distributed` to `parallel` Signed-off-by: Runji Wang --- src/db.rs | 2 + src/planner/cost.rs | 1 + src/planner/explain.rs | 4 +- src/planner/mod.rs | 2 +- src/planner/optimizer.rs | 6 ++ src/planner/rules/mod.rs | 2 +- src/planner/rules/order.rs | 1 + src/planner/rules/partition.rs | 103 ++++++++++++++++++--------------- src/planner/rules/rows.rs | 2 + src/planner/rules/schema.rs | 3 +- src/planner/rules/type_.rs | 3 +- 11 files changed, 75 insertions(+), 54 deletions(-) diff --git a/src/db.rs b/src/db.rs index 36c9202d..c8eb29af 100644 --- a/src/db.rs +++ b/src/db.rs @@ -93,6 +93,8 @@ impl Database { crate::planner::Config { enable_range_filter_scan: self.storage.support_range_filter_scan(), table_is_sorted_by_primary_key: self.storage.table_is_sorted_by_primary_key(), + generate_parallel_plan: tokio::runtime::Handle::current().metrics().num_workers() + > 1, }, ); diff --git a/src/planner/cost.rs b/src/planner/cost.rs index 019f450b..facd661b 100644 --- a/src/planner/cost.rs +++ b/src/planner/cost.rs @@ -69,6 +69,7 @@ impl egg::CostFunction for CostFn<'_> { Insert([_, _, c]) | CopyTo([_, c]) => rows(c) * cols(c) + costs(c), Empty(_) => 0.0, Max1Row(c) => costs(c), + Exchange([_, c]) => costs(c), // expressions Column(_) | Ref(_) => 0.01, // column reference is almost free List(_) => enode.fold(0.01, |sum, id| sum + costs(&id)), // list is almost free diff --git a/src/planner/explain.rs b/src/planner/explain.rs index 2f57d106..aab665b6 100644 --- a/src/planner/explain.rs +++ b/src/planner/explain.rs @@ -344,8 +344,8 @@ impl<'a> Explain<'a> { vec![("dist", self.expr(dist).pretty())].with(cost, rows), vec![self.child(child).pretty()], ), - ToDist(child) => { - Pretty::simple_record("ToDist", vec![], vec![self.child(child).pretty()]) + ToParallel(child) => { + Pretty::simple_record("ToParallel", vec![], vec![self.child(child).pretty()]) } Single | Broadcast | Random => Pretty::display(enode), Hash(keys) => { diff --git a/src/planner/mod.rs b/src/planner/mod.rs index a3fe848b..5abec85b 100644 --- a/src/planner/mod.rs +++ b/src/planner/mod.rs @@ -120,8 +120,8 @@ define_language! { // output = child || exprs // parallelism + "to_parallel" = ToParallel(Id), // (to_parallel child) "exchange" = Exchange([Id; 2]), // (exchange dist child) - "to_dist" = ToDist(Id), // (to_dist child) "single" = Single, // (single) merge all to one "broadcast" = Broadcast, // (broadcast) broadcast to all "random" = Random, // (random) random partition diff --git a/src/planner/optimizer.rs b/src/planner/optimizer.rs index 202509bc..48bf08e9 100644 --- a/src/planner/optimizer.rs +++ b/src/planner/optimizer.rs @@ -4,6 +4,7 @@ use std::sync::LazyLock; use egg::CostFunction; +use self::rules::partition::to_parallel_plan; use super::*; use crate::catalog::RootCatalogRef; @@ -18,6 +19,7 @@ pub struct Optimizer { pub struct Config { pub enable_range_filter_scan: bool, pub table_is_sorted_by_primary_key: bool, + pub generate_parallel_plan: bool, } impl Optimizer { @@ -49,6 +51,10 @@ impl Optimizer { self.optimize_stage(&mut expr, &mut cost, rules, 4, 6); // 3. join reorder and hashjoin self.optimize_stage(&mut expr, &mut cost, STAGE3_RULES.iter(), 3, 8); + + if self.analysis.config.generate_parallel_plan { + expr = to_parallel_plan(expr); + } expr } diff --git a/src/planner/rules/mod.rs b/src/planner/rules/mod.rs index e5be74cf..48011372 100644 --- a/src/planner/rules/mod.rs +++ b/src/planner/rules/mod.rs @@ -15,7 +15,7 @@ //! | [`type_`] | | data type | [`Type`] | //! | [`rows`] | | estimated rows | [`Rows`] | //! | [`order`] | merge join | ordered keys | [`OrderKey`] | -//! | [`partition`] | to_distributed | data partition | [`Partition`] | +//! | [`partition`] | to_parallel | data partition | [`Partition`] | //! //! It would be best if you have a background in program analysis. //! Here is a recommended course: . diff --git a/src/planner/rules/order.rs b/src/planner/rules/order.rs index d6b4e153..381bda9a 100644 --- a/src/planner/rules/order.rs +++ b/src/planner/rules/order.rs @@ -36,6 +36,7 @@ pub fn analyze_order(egraph: &EGraph, enode: &Expr) -> OrderKey { Proj([_, c]) | Filter([_, c]) | Window([_, c]) | Limit([_, _, c]) => x(c).clone(), MergeJoin([_, _, _, _, _, r]) => x(r).clone(), SortAgg([_, _, c]) => x(c).clone(), + Exchange([_, c]) => x(c).clone(), // unordered for other plans _ => Box::new([]), } diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index e04f016c..87a11b1e 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -1,9 +1,9 @@ // Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. -//! This module converts physical plans into distributed plans. +//! This module converts physical plans into parallel plans. //! -//! In a distributed query plan, each node represents one or more physical operators, with each -//! operator processing data from one partition. Each node has a [`Distribution`] property that +//! In a parallel query plan, each node represents one or more physical operators, with each +//! operator processing data from one partition. Each node has a [`Partition`] property that //! describes how the data is partitioned. //! //! After the conversion, [`Exchange`](Expr::Exchange) nodes will be inserted when necessary to @@ -16,29 +16,29 @@ use egg::{rewrite as rw, Analysis, EGraph, Language}; use super::*; use crate::planner::RecExpr; -/// Converts a physical plan into a distributed plan. -pub fn to_distributed(mut plan: RecExpr) -> RecExpr { - // add to_dist to the root node +/// Converts a physical plan into a parallel plan. +pub fn to_parallel_plan(mut plan: RecExpr) -> RecExpr { + // add to_parallel to the root node let root_id = Id::from(plan.as_ref().len() - 1); - plan.add(Expr::ToDist(root_id)); + plan.add(Expr::ToParallel(root_id)); let runner = egg::Runner::<_, _, ()>::new(PartitionAnalysis) .with_expr(&plan) - .run(TO_DIST_RULES.iter()); - let extractor = egg::Extractor::new(&runner.egraph, ExtractDistributedPlan); + .run(TO_PARALLEL_RULES.iter()); + let extractor = egg::Extractor::new(&runner.egraph, NoToParallel); let (_, expr) = extractor.find_best(runner.roots[0]); expr } -struct ExtractDistributedPlan; +struct NoToParallel; -impl egg::CostFunction for ExtractDistributedPlan { +impl egg::CostFunction for NoToParallel { type Cost = usize; fn cost(&mut self, enode: &Expr, mut costs: C) -> Self::Cost where C: FnMut(Id) -> Self::Cost, { - if let Expr::ToDist(_) = enode { + if let Expr::ToParallel(_) = enode { return usize::MAX; } enode.fold(1, |sum, id| sum.saturating_add(costs(id))) @@ -47,92 +47,99 @@ impl egg::CostFunction for ExtractDistributedPlan { type Rewrite = egg::Rewrite; -static TO_DIST_RULES: LazyLock> = LazyLock::new(|| { +static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { vec![ // scan is not partitioned rw!("scan-to-dist"; - "(to_dist (scan ?table ?columns ?filter))" => + "(to_parallel (scan ?table ?columns ?filter))" => "(exchange random (scan ?table ?columns ?filter))" ), // values is not partitioned rw!("values-to-dist"; - "(to_dist (values ?values))" => + "(to_parallel (values ?values))" => "(exchange random (values ?values))" ), // projection does not change distribution rw!("proj-to-dist"; - "(to_dist (proj ?projs ?child))" => - "(proj ?projs (to_dist ?child))" + "(to_parallel (proj ?projs ?child))" => + "(proj ?projs (to_parallel ?child))" ), // filter does not change distribution rw!("filter-to-dist"; - "(to_dist (filter ?cond ?child))" => - "(filter ?cond (to_dist ?child))" + "(to_parallel (filter ?cond ?child))" => + "(filter ?cond (to_parallel ?child))" ), - // 2-phase ordering + // order can not be partitioned rw!("order-to-dist"; - "(to_dist (order ?key ?child))" => - "(order ?key (exchange single (order ?key (to_dist ?child))))" + "(to_parallel (order ?key ?child))" => + "(order ?key (exchange single (to_parallel ?child)))" + // TODO: 2-phase ordering + // "(order ?key (exchange single (order ?key (to_parallel ?child))))" // TODO: merge sort in the second phase? ), // limit can not be partitioned rw!("limit-to-dist"; - "(to_dist (limit ?limit ?offset ?child))" => - "(limit ?limit ?offset (exchange single (to_dist ?child)))" + "(to_parallel (limit ?limit ?offset ?child))" => + "(limit ?limit ?offset (exchange single (to_parallel ?child)))" ), // topn can not be partitioned rw!("topn-to-dist"; - "(to_dist (topn ?limit ?offset ?key ?child))" => - "(topn ?limit ?offset ?key (exchange single (to_dist ?child)))" + "(to_parallel (topn ?limit ?offset ?key ?child))" => + "(topn ?limit ?offset ?key (exchange single (to_parallel ?child)))" ), // inner join is partitioned by left // as the left side is materialized in memory rw!("inner-join-to-dist"; - "(to_dist (join inner ?cond ?left ?right))" => + "(to_parallel (join inner ?cond ?left ?right))" => "(join inner ?cond - (exchange random (to_dist ?left)) - (exchange broadcast (to_dist ?right)))" + (exchange random (to_parallel ?left)) + (exchange broadcast (to_parallel ?right)))" ), // outer join can not be partitioned rw!("join-to-dist"; - "(to_dist (join full_outer ?cond ?left ?right))" => + "(to_parallel (join full_outer ?cond ?left ?right))" => "(join full_outer ?cond - (exchange single (to_dist ?left)) - (exchange single (to_dist ?right)))" + (exchange single (to_parallel ?left)) + (exchange single (to_parallel ?right)))" ), // hash join can be partitioned by join key rw!("hashjoin-to-dist"; - "(to_dist (hashjoin ?type ?cond ?lkey ?rkey ?left ?right))" => + "(to_parallel (hashjoin ?type ?cond ?lkey ?rkey ?left ?right))" => "(hashjoin ?type ?cond ?lkey ?rkey - (exchange (hash ?lkey) (to_dist ?left)) - (exchange (hash ?rkey) (to_dist ?right)))" + (exchange (hash ?lkey) (to_parallel ?left)) + (exchange (hash ?rkey) (to_parallel ?right)))" ), // merge join can be partitioned by join key rw!("mergejoin-to-dist"; - "(to_dist (mergejoin ?type ?cond ?lkey ?rkey ?left ?right))" => + "(to_parallel (mergejoin ?type ?cond ?lkey ?rkey ?left ?right))" => "(mergejoin ?type ?cond ?lkey ?rkey - (exchange (hash ?lkey) (to_dist ?left)) - (exchange (hash ?rkey) (to_dist ?right)))" + (exchange (hash ?lkey) (to_parallel ?left)) + (exchange (hash ?rkey) (to_parallel ?right)))" ), // 2-phase aggregation rw!("agg-to-dist"; - "(to_dist (agg ?exprs ?child))" => - "(agg ?exprs (exchange single (agg ?exprs (exchange random (to_dist ?child)))))" + "(to_parallel (agg ?exprs ?child))" => + "(agg ?exprs (exchange single (agg ?exprs (exchange random (to_parallel ?child)))))" ), // hash aggregation can be partitioned by group key rw!("hashagg-to-dist"; - "(to_dist (hashagg ?keys ?aggs ?child))" => - "(hashagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" + "(to_parallel (hashagg ?keys ?aggs ?child))" => + "(hashagg ?keys ?aggs (exchange (hash ?keys) (to_parallel ?child)))" ), // sort aggregation can be partitioned by group key rw!("sortagg-to-dist"; - "(to_dist (sortagg ?keys ?aggs ?child))" => - "(sortagg ?keys ?aggs (exchange (hash ?keys) (to_dist ?child)))" + "(to_parallel (sortagg ?keys ?aggs ?child))" => + "(sortagg ?keys ?aggs (exchange (hash ?keys) (to_parallel ?child)))" ), // window function can not be partitioned for now rw!("window-to-dist"; - "(to_dist (window ?exprs ?child))" => - "(window ?exprs (exchange single (to_dist ?child)))" + "(to_parallel (window ?exprs ?child))" => + "(window ?exprs (exchange single (to_parallel ?child)))" + ), + // explain + rw!("explain-to-dist"; + "(to_parallel (explain ?child))" => + "(explain (to_parallel ?child))" ), // unnecessary exchange can be removed rw!("remove-exchange"; @@ -239,7 +246,7 @@ mod tests { use super::*; #[test] - fn test_to_distributed() { + fn test_to_parallel() { let input = " (hashjoin inner true (list a) (list b) (scan t1 (list a) true) @@ -254,7 +261,7 @@ mod tests { (scan t2 (list b) true)) ) "; - let output = to_distributed(input.parse().unwrap()); + let output = to_parallel_plan(input.parse().unwrap()); let expected: RecExpr = distributed.parse().unwrap(); assert_eq!(output.to_string(), expected.to_string()); } diff --git a/src/planner/rules/rows.rs b/src/planner/rules/rows.rs index f483413d..afca6355 100644 --- a/src/planner/rules/rows.rs +++ b/src/planner/rules/rows.rs @@ -76,6 +76,8 @@ pub fn analyze_rows(egraph: &EGraph, enode: &Expr) -> Rows { }, Empty(_) => 0.0, Max1Row(_) => 1.0, + // FIXME: broadcast distribution should multiply the number of rows + Exchange([_, c]) => x(c), // for boolean expressions, the result represents selectivity Ref(a) => x(a), diff --git a/src/planner/rules/schema.rs b/src/planner/rules/schema.rs index 7806136b..1b6a89ed 100644 --- a/src/planner/rules/schema.rs +++ b/src/planner/rules/schema.rs @@ -17,7 +17,8 @@ pub fn analyze_schema( let concat = |v1: Vec, v2: Vec| v1.into_iter().chain(v2).collect(); match enode { // equal to child - Filter([_, c]) | Order([_, c]) | Limit([_, _, c]) | TopN([_, _, _, c]) | Empty(c) => x(c), + Filter([_, c]) | Order([_, c]) | Limit([_, _, c]) | TopN([_, _, _, c]) | Empty(c) + | Exchange([_, c]) => x(c), // concat 2 children Join([t, _, l, r]) diff --git a/src/planner/rules/type_.rs b/src/planner/rules/type_.rs index 21cd4c2f..294a0aa4 100644 --- a/src/planner/rules/type_.rs +++ b/src/planner/rules/type_.rs @@ -142,7 +142,8 @@ pub fn analyze_type( }), // equal to child - Filter([_, c]) | Order([_, c]) | Limit([_, _, c]) | TopN([_, _, _, c]) | Empty(c) => x(c), + Filter([_, c]) | Order([_, c]) | Limit([_, _, c]) | TopN([_, _, _, c]) | Empty(c) + | Exchange([_, c]) => x(c), // concat 2 children Join([t, _, l, r]) | HashJoin([t, _, _, _, l, r]) | MergeJoin([t, _, _, _, l, r]) => { From d51e7c42b94ed75a310bb17618322b1e7e5cba73 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 11 Nov 2024 00:09:56 +0800 Subject: [PATCH 04/29] hash partition executor Signed-off-by: Runji Wang --- src/array/ops.rs | 21 ++ src/executor/exchange.rs | 64 +++++ src/executor/mod.rs | 474 ++++++++++++++++++++++++--------- src/planner/rules/partition.rs | 32 +-- 4 files changed, 450 insertions(+), 141 deletions(-) create mode 100644 src/executor/exchange.rs diff --git a/src/array/ops.rs b/src/array/ops.rs index 581783d6..c8be4675 100644 --- a/src/array/ops.rs +++ b/src/array/ops.rs @@ -3,6 +3,7 @@ //! Array operations. use std::borrow::Borrow; +use std::hash::{Hash, Hasher}; use num_traits::ToPrimitive; use regex::Regex; @@ -204,6 +205,26 @@ impl ArrayImpl { Ok(A::new_bool(clear_null(unary_op(a.as_ref(), |b| !b)))) } + /// Hash the array into the given hasher. + pub fn hash(&self, hasher: &mut [impl Hasher]) { + assert_eq!(hasher.len(), self.len()); + match self { + A::Null(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Bool(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Int16(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Int32(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Int64(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Float64(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Decimal(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::String(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Date(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Timestamp(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::TimestampTz(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Interval(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + A::Blob(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), + } + } + pub fn like(&self, pattern: &str) -> Result { /// Converts a SQL LIKE pattern to a regex pattern. fn like_to_regex(pattern: &str) -> String { diff --git a/src/executor/exchange.rs b/src/executor/exchange.rs new file mode 100644 index 00000000..c2d72b49 --- /dev/null +++ b/src/executor/exchange.rs @@ -0,0 +1,64 @@ +// Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. + +use std::hash::{DefaultHasher, Hasher}; + +use super::*; + +/// Distribute the input data to multiple partitions by hash partitioning. +pub struct HashPartitionProducer { + /// The indices of the columns to hash. + pub hash_key: Vec, + /// The number of partitions. + pub num_partitions: usize, +} + +impl HashPartitionProducer { + #[try_stream(boxed, ok = (DataChunk, usize), error = ExecutorError)] + pub async fn execute(self, child: BoxedExecutor) { + // preallocate buffers for reuse + let mut hashers = vec![DefaultHasher::default(); PROCESSING_WINDOW_SIZE]; + let mut partition_indices = vec![0; PROCESSING_WINDOW_SIZE]; + let mut visibility = vec![false; PROCESSING_WINDOW_SIZE]; + + #[for_await] + for batch in child { + let batch = batch?; + + // reset buffers + hashers.clear(); + hashers.resize(batch.cardinality(), DefaultHasher::default()); + partition_indices.resize(batch.cardinality(), 0); + visibility.resize(batch.cardinality(), false); + + // calculate the hash + for index in &self.hash_key { + batch.array_at(*index).hash(&mut hashers); + } + for (hasher, target) in hashers.iter().zip(&mut partition_indices) { + *target = hasher.finish() as usize % self.num_partitions; + } + + // send the batch to the corresponding partition + for partition in 0..self.num_partitions { + for (row, p) in partition_indices.iter().enumerate() { + visibility[row] = *p == partition; + } + let chunk = batch.filter(&visibility); + yield (chunk, partition); + } + } + } +} + +/// Randomly distribute the input data to multiple partitions. +pub struct RandomPartitionProducer { + /// The number of partitions. + pub num_partitions: usize, +} + +impl RandomPartitionProducer { + #[try_stream(boxed, ok = (DataChunk, usize), error = ExecutorError)] + pub async fn execute(self, child: BoxedExecutor) { + todo!() + } +} diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 1ee23938..56327352 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -31,6 +31,7 @@ use self::drop::*; pub use self::error::Error as ExecutorError; use self::error::*; use self::evaluator::*; +use self::exchange::*; use self::explain::*; use self::filter::*; use self::hash_agg::*; @@ -64,6 +65,7 @@ mod create_view; mod delete; mod drop; mod evaluator; +mod exchange; mod explain; mod filter; mod hash_agg; @@ -94,6 +96,8 @@ const PROCESSING_WINDOW_SIZE: usize = 1024; /// It consumes one or more streams from its child executors, /// and produces a stream to its parent. pub type BoxedExecutor = BoxStream<'static, Result>; +/// A boxed dispatcher that distributes data to multiple partitions. +pub type BoxedDispatcher = BoxStream<'static, Result<(DataChunk, usize)>>; pub fn build(optimizer: Optimizer, storage: Arc, plan: &RecExpr) -> BoxedExecutor { Builder::new(optimizer, storage, plan).build() @@ -107,7 +111,7 @@ struct Builder { root: Id, /// For scans on views, we prebuild their executors and store them here. /// Multiple scans on the same view will share the same executor. - views: HashMap, + views: HashMap, } impl Builder { @@ -190,23 +194,18 @@ impl Builder { /// Builds the executor. fn build(self) -> BoxedExecutor { - self.build_id(self.root) + self.build_id(self.root).spawn_merge() } /// Builds the executor and returns its subscriber. - fn build_subscriber(self) -> StreamSubscriber { - self.build_id_subscriber(self.root) + fn build_subscriber(self) -> PartitionedStreamSubscriber { + self.build_id(self.root).spawn() } - /// Builds the executor for the given id. - fn build_id(&self, id: Id) -> BoxedExecutor { - self.build_id_subscriber(id).subscribe() - } - - /// Builds the executor for the given id and returns its subscriber. - fn build_id_subscriber(&self, id: Id) -> StreamSubscriber { + /// Builds stream for the given plan. + fn build_id(&self, id: Id) -> PartitionedStream { use Expr::*; - let stream = match self.node(id).clone() { + match self.node(id).clone() { Scan([table, list, filter]) => { let table_id = self.node(table).as_table(); let columns = (self.node(list).as_list().iter()) @@ -250,7 +249,12 @@ impl Builder { .collect(); projs.add(List(lists)); - ProjectionExecutor { projs }.execute(subscriber.subscribe()) + subscriber.subscribe().map(|c| { + ProjectionExecutor { + projs: projs.clone(), + } + .execute(c) + }) } else if table_id.schema_id == RootCatalog::SYSTEM_SCHEMA_ID { SystemTableScan { catalog: self.catalog().clone(), @@ -259,6 +263,7 @@ impl Builder { columns, } .execute() + .into() } else { TableScanExecutor { table_id, @@ -267,6 +272,7 @@ impl Builder { storage: self.storage.clone(), } .execute() + .into() } } @@ -282,121 +288,160 @@ impl Builder { .collect() }, } - .execute(), - - Proj([projs, child]) => ProjectionExecutor { - projs: self.resolve_column_index(projs, child), - } - .execute(self.build_id(child)), - - Filter([cond, child]) => FilterExecutor { - condition: self.resolve_column_index(cond, child), - } - .execute(self.build_id(child)), + .execute() + .into(), - Order([order_keys, child]) => OrderExecutor { - order_keys: self.resolve_column_index(order_keys, child), - types: self.plan_types(id).to_vec(), - } - .execute(self.build_id(child)), + Proj([projs, child]) => self.build_id(child).map(|c| { + ProjectionExecutor { + projs: self.resolve_column_index(projs, child), + } + .execute(c) + }), - Limit([limit, offset, child]) => LimitExecutor { - limit: (self.node(limit).as_const().as_usize().unwrap()).unwrap_or(usize::MAX / 2), - offset: self.node(offset).as_const().as_usize().unwrap().unwrap(), - } - .execute(self.build_id(child)), + Filter([cond, child]) => self.build_id(child).map(|c| { + FilterExecutor { + condition: self.resolve_column_index(cond, child), + } + .execute(c) + }), - TopN([limit, offset, order_keys, child]) => TopNExecutor { - limit: (self.node(limit).as_const().as_usize().unwrap()).unwrap_or(usize::MAX / 2), - offset: self.node(offset).as_const().as_usize().unwrap().unwrap(), - order_keys: self.resolve_column_index(order_keys, child), - types: self.plan_types(id).to_vec(), - } - .execute(self.build_id(child)), - - Join([op, on, left, right]) => match self.node(op) { - Inner | LeftOuter | RightOuter | FullOuter => NestedLoopJoinExecutor { - op: self.node(op).clone(), - condition: self.resolve_column_index2(on, left, right), - left_types: self.plan_types(left).to_vec(), - right_types: self.plan_types(right).to_vec(), + Order([order_keys, child]) => self.build_id(child).map(|c| { + OrderExecutor { + order_keys: self.resolve_column_index(order_keys, child), + types: self.plan_types(id).to_vec(), } - .execute(self.build_id(left), self.build_id(right)), - op @ Semi | op @ Anti => NestedLoopSemiJoinExecutor { - anti: matches!(op, Anti), - condition: self.resolve_column_index2(on, left, right), - left_types: self.plan_types(left).to_vec(), + .execute(c) + }), + + Limit([limit, offset, child]) => self.build_id(child).map(|c| { + LimitExecutor { + limit: (self.node(limit).as_const().as_usize().unwrap()) + .unwrap_or(usize::MAX / 2), + offset: self.node(offset).as_const().as_usize().unwrap().unwrap(), } - .execute(self.build_id(left), self.build_id(right)), - t => panic!("invalid join type: {t:?}"), - }, - - HashJoin(args @ [op, ..]) => match self.node(op) { - Inner => self.build_hashjoin::<{ JoinType::Inner }>(args), - LeftOuter => self.build_hashjoin::<{ JoinType::LeftOuter }>(args), - RightOuter => self.build_hashjoin::<{ JoinType::RightOuter }>(args), - FullOuter => self.build_hashjoin::<{ JoinType::FullOuter }>(args), - Semi => self.build_hashsemijoin(args, false), - Anti => self.build_hashsemijoin(args, true), - t => panic!("invalid join type: {t:?}"), - }, - - MergeJoin(args @ [op, ..]) => match self.node(op) { - Inner => self.build_mergejoin::<{ JoinType::Inner }>(args), - LeftOuter => self.build_mergejoin::<{ JoinType::LeftOuter }>(args), - RightOuter => self.build_mergejoin::<{ JoinType::RightOuter }>(args), - FullOuter => self.build_mergejoin::<{ JoinType::FullOuter }>(args), - t => panic!("invalid join type: {t:?}"), - }, + .execute(c) + }), + + TopN([limit, offset, order_keys, child]) => self.build_id(child).map(|c| { + TopNExecutor { + limit: (self.node(limit).as_const().as_usize().unwrap()) + .unwrap_or(usize::MAX / 2), + offset: self.node(offset).as_const().as_usize().unwrap().unwrap(), + order_keys: self.resolve_column_index(order_keys, child), + types: self.plan_types(id).to_vec(), + } + .execute(c) + }), + + Join([op, on, left, right]) => self + .build_id(left) + .spawn() // ends the pipeline of left side + .subscribe() + .zip(self.build_id(right)) + .map(|l, r| match self.node(op) { + Inner | LeftOuter | RightOuter | FullOuter => NestedLoopJoinExecutor { + op: self.node(op).clone(), + condition: self.resolve_column_index2(on, left, right), + left_types: self.plan_types(left).to_vec(), + right_types: self.plan_types(right).to_vec(), + } + .execute(l, r), + op @ Semi | op @ Anti => NestedLoopSemiJoinExecutor { + anti: matches!(op, Anti), + condition: self.resolve_column_index2(on, left, right), + left_types: self.plan_types(left).to_vec(), + } + .execute(l, r), + t => panic!("invalid join type: {t:?}"), + }), + + HashJoin(args @ [op, _, _, _, left, right]) => self + .build_id(left) + .spawn() // ends the pipeline of left side + .subscribe() + .zip(self.build_id(right)) + .map(|l, r| match self.node(op) { + Inner => self.build_hashjoin::<{ JoinType::Inner }>(args, l, r), + LeftOuter => self.build_hashjoin::<{ JoinType::LeftOuter }>(args, l, r), + RightOuter => self.build_hashjoin::<{ JoinType::RightOuter }>(args, l, r), + FullOuter => self.build_hashjoin::<{ JoinType::FullOuter }>(args, l, r), + Semi => self.build_hashsemijoin(args, false, l, r), + Anti => self.build_hashsemijoin(args, true, l, r), + t => panic!("invalid join type: {t:?}"), + }), + + MergeJoin(args @ [op, _, _, _, left, right]) => self + .build_id(left) + .spawn() // ends the pipeline of left side + .subscribe() + .zip(self.build_id(right)) + .map(|l, r| match self.node(op) { + Inner => self.build_mergejoin::<{ JoinType::Inner }>(args, l, r), + LeftOuter => self.build_mergejoin::<{ JoinType::LeftOuter }>(args, l, r), + RightOuter => self.build_mergejoin::<{ JoinType::RightOuter }>(args, l, r), + FullOuter => self.build_mergejoin::<{ JoinType::FullOuter }>(args, l, r), + t => panic!("invalid join type: {t:?}"), + }), Apply(_) => { panic!("Apply is not supported in executor. It should be rewritten to join by optimizer.") } - Agg([aggs, child]) => SimpleAggExecutor { - aggs: self.resolve_column_index(aggs, child), - types: self.plan_types(id).to_vec(), - } - .execute(self.build_id(child)), - - HashAgg([keys, aggs, child]) => HashAggExecutor { - keys: self.resolve_column_index(keys, child), - aggs: self.resolve_column_index(aggs, child), - types: self.plan_types(id).to_vec(), - } - .execute(self.build_id(child)), - - SortAgg([keys, aggs, child]) => SortAggExecutor { - keys: self.resolve_column_index(keys, child), - aggs: self.resolve_column_index(aggs, child), - types: self.plan_types(id).to_vec(), - } - .execute(self.build_id(child)), + Agg([aggs, child]) => self.build_id(child).map(|c| { + SimpleAggExecutor { + aggs: self.resolve_column_index(aggs, child), + types: self.plan_types(id).to_vec(), + } + .execute(c) + }), + + HashAgg([keys, aggs, child]) => self.build_id(child).map(|c| { + HashAggExecutor { + keys: self.resolve_column_index(keys, child), + aggs: self.resolve_column_index(aggs, child), + types: self.plan_types(id).to_vec(), + } + .execute(c) + }), + + SortAgg([keys, aggs, child]) => self.build_id(child).map(|c| { + SortAggExecutor { + keys: self.resolve_column_index(keys, child), + aggs: self.resolve_column_index(aggs, child), + types: self.plan_types(id).to_vec(), + } + .execute(c) + }), - Window([exprs, child]) => WindowExecutor { - exprs: self.resolve_column_index(exprs, child), - types: self.plan_types(exprs).to_vec(), - } - .execute(self.build_id(child)), + Window([exprs, child]) => self.build_id(child).map(|c| { + WindowExecutor { + exprs: self.resolve_column_index(exprs, child), + types: self.plan_types(exprs).to_vec(), + } + .execute(c) + }), CreateTable(table) => CreateTableExecutor { table, storage: self.storage.clone(), } - .execute(), + .execute() + .into(), CreateView([table, query]) => CreateViewExecutor { table: self.node(table).as_create_table(), query: self.recexpr(query), catalog: self.catalog().clone(), } - .execute(), + .execute() + .into(), CreateFunction(f) => CreateFunctionExecutor { f, catalog: self.optimizer.catalog().clone(), } - .execute(), + .execute() + .into(), Drop(tables) => DropExecutor { tables: (self.node(tables).as_list().iter()) @@ -405,7 +450,8 @@ impl Builder { catalog: self.catalog().clone(), storage: self.storage.clone(), } - .execute(), + .execute() + .into(), Insert([table, cols, child]) => InsertExecutor { table_id: self.node(table).as_table(), @@ -414,52 +460,94 @@ impl Builder { .collect(), storage: self.storage.clone(), } - .execute(self.build_id(child)), + .execute(self.build_id(child).assume_single()) + .into(), Delete([table, child]) => DeleteExecutor { table_id: self.node(table).as_table(), storage: self.storage.clone(), } - .execute(self.build_id(child)), + .execute(self.build_id(child).assume_single()) + .into(), CopyFrom([src, types]) => CopyFromFileExecutor { source: self.node(src).as_ext_source(), types: self.node(types).as_type().as_struct().to_vec(), } - .execute(), + .execute() + .into(), CopyTo([src, child]) => CopyToFileExecutor { source: self.node(src).as_ext_source(), } - .execute(self.build_id(child)), + .execute(self.build_id(child).assume_single()) + .into(), Explain(plan) => ExplainExecutor { plan: self.recexpr(plan), optimizer: self.optimizer.clone(), } - .execute(), - - Empty(_) => futures::stream::empty().boxed(), + .execute() + .into(), + + Empty(_) => futures::stream::empty().boxed().into(), + + Exchange([dist, child]) => match self.node(dist) { + Single => self.build_id(child).spawn_merge().into(), + Broadcast => { + let subscriber = self.build_id(child).spawn(); + let parallism = tokio::runtime::Handle::current().metrics().num_workers(); + PartitionedStream { + streams: (0..parallism) + .map(|_| subscriber.subscribe_merge()) + .collect(), + } + } + Hash(keys) => { + let parallism = tokio::runtime::Handle::current().metrics().num_workers(); + let hash_key = (self.node(keys).as_list().iter()) + .map(|id| self.node(*id).as_column().column_id as usize) + .collect_vec(); + self.build_id(child) + .spawn_dispatch(|c| { + HashPartitionProducer { + num_partitions: parallism, + hash_key: hash_key.clone(), + } + .execute(c) + }) + .subscribe() + } + node => panic!("invalid exchange type: {node:?}"), + }, node => panic!("not a plan: {node:?}"), - }; - spawn(&self.node(id).to_string(), stream) + } } - fn build_hashjoin(&self, args: [Id; 6]) -> BoxedExecutor { - let [_, cond, lkeys, rkeys, left, right] = args; + fn build_hashjoin( + &self, + [_, cond, lkey, rkey, left, right]: [Id; 6], + l: BoxedExecutor, + r: BoxedExecutor, + ) -> BoxedExecutor { assert_eq!(self.node(cond), &Expr::true_()); HashJoinExecutor:: { - left_keys: self.resolve_column_index(lkeys, left), - right_keys: self.resolve_column_index(rkeys, right), + left_keys: self.resolve_column_index(lkey, left), + right_keys: self.resolve_column_index(rkey, right), left_types: self.plan_types(left).to_vec(), right_types: self.plan_types(right).to_vec(), } - .execute(self.build_id(left), self.build_id(right)) + .execute(l, r) } - fn build_hashsemijoin(&self, args: [Id; 6], anti: bool) -> BoxedExecutor { - let [_, cond, lkeys, rkeys, left, right] = args; + fn build_hashsemijoin( + &self, + [_, cond, lkeys, rkeys, left, right]: [Id; 6], + anti: bool, + l: BoxedExecutor, + r: BoxedExecutor, + ) -> BoxedExecutor { if self.node(cond) == &Expr::true_() { HashSemiJoinExecutor { left_keys: self.resolve_column_index(lkeys, left), @@ -467,7 +555,7 @@ impl Builder { left_types: self.plan_types(left).to_vec(), anti, } - .execute(self.build_id(left), self.build_id(right)) + .execute(l, r) } else { HashSemiJoinExecutor2 { left_keys: self.resolve_column_index(lkeys, left), @@ -477,12 +565,16 @@ impl Builder { right_types: self.plan_types(right).to_vec(), anti, } - .execute(self.build_id(left), self.build_id(right)) + .execute(l, r) } } - fn build_mergejoin(&self, args: [Id; 6]) -> BoxedExecutor { - let [_, cond, lkeys, rkeys, left, right] = args; + fn build_mergejoin( + &self, + [_, cond, lkeys, rkeys, left, right]: [Id; 6], + l: BoxedExecutor, + r: BoxedExecutor, + ) -> BoxedExecutor { assert_eq!(self.node(cond), &Expr::true_()); MergeJoinExecutor:: { left_keys: self.resolve_column_index(lkeys, left), @@ -490,15 +582,14 @@ impl Builder { left_types: self.plan_types(left).to_vec(), right_types: self.plan_types(right).to_vec(), } - .execute(self.build_id(left), self.build_id(right)) + .execute(l, r) } } /// Spawn a new task to execute the given stream. -fn spawn(name: &str, mut stream: BoxedExecutor) -> StreamSubscriber { +fn spawn(mut stream: BoxedExecutor) -> StreamSubscriber { let (tx, rx) = async_broadcast::broadcast(16); let handle = tokio::task::Builder::default() - .name(name) .spawn(async move { while let Some(item) = stream.next().await { if tx.broadcast(item).await.is_err() { @@ -515,6 +606,139 @@ fn spawn(name: &str, mut stream: BoxedExecutor) -> StreamSubscriber { } } +/// Spawn new tasks to execute the given dispatchers. +/// Dispatch the output to multiple partitions by the associated partition index. +fn spawn_dispatchers(mut streams: Vec) -> PartitionedStreamSubscriber { + let (txs, rxs): (Vec<_>, Vec<_>) = streams + .into_iter() + .map(|_| async_broadcast::broadcast(16)) + .unzip(); + let mut handles = Vec::with_capacity(streams.len()); + for stream in streams { + let txs = txs.clone(); + let handle = tokio::task::Builder::default() + .spawn(async move { + while let Some(item) = stream.next().await { + match item { + // send the chunk to the corresponding partition (ignore error) + Ok((chunk, partition)) => _ = txs[partition].broadcast(Ok(chunk)).await, + // broadcast the error to all partitions + Err(e) => { + for tx in txs.iter() { + tx.broadcast(Err(e.clone())).await.unwrap(); + } + } + } + } + }) + .expect("failed to spawn task"); + handles.push(handle); + } + PartitionedStreamSubscriber { + subscribers: (rxs.into_iter().zip(handles)) + .map(|(rx, handle)| StreamSubscriber { + rx: rx.deactivate(), + handle: Arc::new(AbortOnDropHandle(handle)), + }) + .collect(), + } +} + +/// A set of partitioned output streams. +struct PartitionedStream { + streams: Vec, +} + +/// Creates from a single stream. +impl From for PartitionedStream { + fn from(stream: BoxedExecutor) -> Self { + PartitionedStream { + streams: vec![stream], + } + } +} + +impl PartitionedStream { + /// Assume that there is only one stream and returns it. + fn assume_single(self) -> BoxedExecutor { + assert_eq!(self.streams.len(), 1); + self.streams.into_iter().next().unwrap() + } + + /// Merges the partitioned streams into a single stream. + fn spawn_merge(self) -> BoxedExecutor { + futures::stream::select_all(self.spawn().subscribe().streams).boxed() + } + + /// Maps each stream with the given function. + fn map(self, f: impl Fn(BoxedExecutor) -> BoxedExecutor) -> PartitionedStream { + PartitionedStream { + streams: self.streams.into_iter().map(f).collect(), + } + } + + /// Zips up two sets of partitioned streams. + fn zip(self, other: PartitionedStream) -> ZippedPartitionedStream { + ZippedPartitionedStream { + left: self.streams, + right: other.streams, + } + } + + /// Spawns each stream with the given name. + fn spawn(self) -> PartitionedStreamSubscriber { + PartitionedStreamSubscriber { + subscribers: self.streams.into_iter().map(spawn).collect(), + } + } + + /// Spawns each stream and dispatches the output to multiple partitions. + fn spawn_dispatch( + self, + f: impl Fn(BoxedExecutor) -> BoxedDispatcher, + ) -> PartitionedStreamSubscriber { + spawn_dispatchers(self.streams.into_iter().map(f).collect()) + } +} + +/// The return type of `PartitionedStream::zip`. +struct ZippedPartitionedStream { + left: Vec, + right: Vec, +} + +impl ZippedPartitionedStream { + /// Maps each stream pair with the given function. + fn map(self, f: impl Fn(BoxedExecutor, BoxedExecutor) -> BoxedExecutor) -> PartitionedStream { + assert_eq!(self.left.len(), self.right.len()); + PartitionedStream { + streams: self + .left + .into_iter() + .zip(self.right.into_iter()) + .map(|(l, r)| f(l, r)) + .collect(), + } + } +} + +/// A set of partitioned stream subscribers. +struct PartitionedStreamSubscriber { + subscribers: Vec, +} + +impl PartitionedStreamSubscriber { + fn subscribe(&self) -> PartitionedStream { + PartitionedStream { + streams: self.subscribers.iter().map(|s| s.subscribe()).collect(), + } + } + + fn subscribe_merge(&self) -> BoxedExecutor { + futures::stream::select_all(self.subscribe().streams).boxed() + } +} + /// A subscriber of an executor's output stream. /// /// New streams can be created by calling `subscribe`. diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index 87a11b1e..c6acaa14 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -50,27 +50,27 @@ type Rewrite = egg::Rewrite; static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { vec![ // scan is not partitioned - rw!("scan-to-dist"; + rw!("scan-to-parallel"; "(to_parallel (scan ?table ?columns ?filter))" => "(exchange random (scan ?table ?columns ?filter))" ), // values is not partitioned - rw!("values-to-dist"; + rw!("values-to-parallel"; "(to_parallel (values ?values))" => "(exchange random (values ?values))" ), // projection does not change distribution - rw!("proj-to-dist"; + rw!("proj-to-parallel"; "(to_parallel (proj ?projs ?child))" => "(proj ?projs (to_parallel ?child))" ), // filter does not change distribution - rw!("filter-to-dist"; + rw!("filter-to-parallel"; "(to_parallel (filter ?cond ?child))" => "(filter ?cond (to_parallel ?child))" ), // order can not be partitioned - rw!("order-to-dist"; + rw!("order-to-parallel"; "(to_parallel (order ?key ?child))" => "(order ?key (exchange single (to_parallel ?child)))" // TODO: 2-phase ordering @@ -78,66 +78,66 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { // TODO: merge sort in the second phase? ), // limit can not be partitioned - rw!("limit-to-dist"; + rw!("limit-to-parallel"; "(to_parallel (limit ?limit ?offset ?child))" => "(limit ?limit ?offset (exchange single (to_parallel ?child)))" ), // topn can not be partitioned - rw!("topn-to-dist"; + rw!("topn-to-parallel"; "(to_parallel (topn ?limit ?offset ?key ?child))" => "(topn ?limit ?offset ?key (exchange single (to_parallel ?child)))" ), // inner join is partitioned by left // as the left side is materialized in memory - rw!("inner-join-to-dist"; + rw!("inner-join-to-parallel"; "(to_parallel (join inner ?cond ?left ?right))" => "(join inner ?cond (exchange random (to_parallel ?left)) (exchange broadcast (to_parallel ?right)))" ), // outer join can not be partitioned - rw!("join-to-dist"; + rw!("join-to-parallel"; "(to_parallel (join full_outer ?cond ?left ?right))" => "(join full_outer ?cond (exchange single (to_parallel ?left)) (exchange single (to_parallel ?right)))" ), // hash join can be partitioned by join key - rw!("hashjoin-to-dist"; + rw!("hashjoin-to-parallel"; "(to_parallel (hashjoin ?type ?cond ?lkey ?rkey ?left ?right))" => "(hashjoin ?type ?cond ?lkey ?rkey (exchange (hash ?lkey) (to_parallel ?left)) (exchange (hash ?rkey) (to_parallel ?right)))" ), // merge join can be partitioned by join key - rw!("mergejoin-to-dist"; + rw!("mergejoin-to-parallel"; "(to_parallel (mergejoin ?type ?cond ?lkey ?rkey ?left ?right))" => "(mergejoin ?type ?cond ?lkey ?rkey (exchange (hash ?lkey) (to_parallel ?left)) (exchange (hash ?rkey) (to_parallel ?right)))" ), // 2-phase aggregation - rw!("agg-to-dist"; + rw!("agg-to-parallel"; "(to_parallel (agg ?exprs ?child))" => "(agg ?exprs (exchange single (agg ?exprs (exchange random (to_parallel ?child)))))" ), // hash aggregation can be partitioned by group key - rw!("hashagg-to-dist"; + rw!("hashagg-to-parallel"; "(to_parallel (hashagg ?keys ?aggs ?child))" => "(hashagg ?keys ?aggs (exchange (hash ?keys) (to_parallel ?child)))" ), // sort aggregation can be partitioned by group key - rw!("sortagg-to-dist"; + rw!("sortagg-to-parallel"; "(to_parallel (sortagg ?keys ?aggs ?child))" => "(sortagg ?keys ?aggs (exchange (hash ?keys) (to_parallel ?child)))" ), // window function can not be partitioned for now - rw!("window-to-dist"; + rw!("window-to-parallel"; "(to_parallel (window ?exprs ?child))" => "(window ?exprs (exchange single (to_parallel ?child)))" ), // explain - rw!("explain-to-dist"; + rw!("explain-to-parallel"; "(to_parallel (explain ?child))" => "(explain (to_parallel ?child))" ), From 5a25a2ed08191ce3366b03bdb65d1a1d88da4f71 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 11 Nov 2024 20:07:12 +0800 Subject: [PATCH 05/29] fix Signed-off-by: Runji Wang --- Cargo.lock | 1 + Cargo.toml | 1 + src/executor/exchange.rs | 10 ++++- src/executor/mod.rs | 82 ++++++++++++++++++++++++++++++---------- 4 files changed, 73 insertions(+), 21 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e2aced3c..4b69612e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2564,6 +2564,7 @@ dependencies = [ "prost", "pyo3", "pyo3-build-config", + "rand", "ref-cast", "regex", "risinglight_proto", diff --git a/Cargo.toml b/Cargo.toml index 5534e992..1a907cdd 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -56,6 +56,7 @@ pgwire = "0.20" pretty-xmlish = "0.1" prost = "0.12" pyo3 = { version = "0.21", features = ["extension-module"], optional = true } +rand = "0.8" ref-cast = "1.0" regex = "1" risinglight_proto = "0.2" diff --git a/src/executor/exchange.rs b/src/executor/exchange.rs index c2d72b49..5e1a790d 100644 --- a/src/executor/exchange.rs +++ b/src/executor/exchange.rs @@ -2,6 +2,8 @@ use std::hash::{DefaultHasher, Hasher}; +use rand::{Rng, SeedableRng}; + use super::*; /// Distribute the input data to multiple partitions by hash partitioning. @@ -59,6 +61,12 @@ pub struct RandomPartitionProducer { impl RandomPartitionProducer { #[try_stream(boxed, ok = (DataChunk, usize), error = ExecutorError)] pub async fn execute(self, child: BoxedExecutor) { - todo!() + let mut rng = rand::rngs::SmallRng::from_entropy(); + #[for_await] + for batch in child { + let batch = batch?; + let partition = rng.gen_range(0..self.num_partitions); + yield (batch, partition); + } } } diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 56327352..ad117233 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -503,15 +503,29 @@ impl Builder { .collect(), } } + Random => { + let num_partitions = tokio::runtime::Handle::current().metrics().num_workers(); + self.build_id(child) + .spawn_dispatch(num_partitions, |c| { + RandomPartitionProducer { num_partitions }.execute(c) + }) + .subscribe() + } Hash(keys) => { - let parallism = tokio::runtime::Handle::current().metrics().num_workers(); - let hash_key = (self.node(keys).as_list().iter()) - .map(|id| self.node(*id).as_column().column_id as usize) + let num_partitions = tokio::runtime::Handle::current().metrics().num_workers(); + let child_schema = &self.egraph[child].data.schema; + let hash_key = (self.node(*keys).as_list().iter()) + .map(|id| { + child_schema + .iter() + .position(|&x| x == *id) + .expect("hash key not found in child's schema") + }) .collect_vec(); self.build_id(child) - .spawn_dispatch(|c| { + .spawn_dispatch(num_partitions, |c| { HashPartitionProducer { - num_partitions: parallism, + num_partitions, hash_key: hash_key.clone(), } .execute(c) @@ -602,19 +616,21 @@ fn spawn(mut stream: BoxedExecutor) -> StreamSubscriber { StreamSubscriber { rx: rx.deactivate(), - handle: Arc::new(AbortOnDropHandle(handle)), + task_handle: Arc::new(AbortOnDropHandle(handle)), } } /// Spawn new tasks to execute the given dispatchers. /// Dispatch the output to multiple partitions by the associated partition index. -fn spawn_dispatchers(mut streams: Vec) -> PartitionedStreamSubscriber { - let (txs, rxs): (Vec<_>, Vec<_>) = streams - .into_iter() +fn spawn_dispatchers( + streams: Vec, + num_partitions: usize, +) -> PartitionedStreamSubscriber { + let (txs, rxs): (Vec<_>, Vec<_>) = (0..num_partitions) .map(|_| async_broadcast::broadcast(16)) .unzip(); let mut handles = Vec::with_capacity(streams.len()); - for stream in streams { + for mut stream in streams { let txs = txs.clone(); let handle = tokio::task::Builder::default() .spawn(async move { @@ -634,11 +650,13 @@ fn spawn_dispatchers(mut streams: Vec) -> PartitionedStreamSubs .expect("failed to spawn task"); handles.push(handle); } + let handles = Arc::new(handles); PartitionedStreamSubscriber { - subscribers: (rxs.into_iter().zip(handles)) - .map(|(rx, handle)| StreamSubscriber { + subscribers: rxs + .into_iter() + .map(|rx| StreamSubscriber { rx: rx.deactivate(), - handle: Arc::new(AbortOnDropHandle(handle)), + task_handle: handles.clone(), // all task handles are shared by all subscribers }) .collect(), } @@ -666,11 +684,23 @@ impl PartitionedStream { } /// Merges the partitioned streams into a single stream. + /// + /// ```text + /// A0 -++-> A + /// A1 -+| + /// A2 --+ + /// ``` fn spawn_merge(self) -> BoxedExecutor { futures::stream::select_all(self.spawn().subscribe().streams).boxed() } /// Maps each stream with the given function. + /// + /// ```text + /// A0 --f-> B0 + /// A1 --f-> B1 + /// A2 --f-> B2 + /// ``` fn map(self, f: impl Fn(BoxedExecutor) -> BoxedExecutor) -> PartitionedStream { PartitionedStream { streams: self.streams.into_iter().map(f).collect(), @@ -678,6 +708,16 @@ impl PartitionedStream { } /// Zips up two sets of partitioned streams. + /// + /// ```text + /// A0 -+---> (A0,B0) + /// A1 -|+--> (A1,B1) + /// A2 -||+-> (A2,B2) + /// ||| + /// B0 -+|| + /// B1 --+| + /// B2 ---+ + /// ``` fn zip(self, other: PartitionedStream) -> ZippedPartitionedStream { ZippedPartitionedStream { left: self.streams, @@ -685,19 +725,21 @@ impl PartitionedStream { } } - /// Spawns each stream with the given name. + /// Spawns each partitioned stream as a tokio task. fn spawn(self) -> PartitionedStreamSubscriber { PartitionedStreamSubscriber { subscribers: self.streams.into_iter().map(spawn).collect(), } } - /// Spawns each stream and dispatches the output to multiple partitions. + /// Spawns each partitioned stream and dispatches the output to `num_partitions` partitions. + /// Returns a set of subscribers of `num_partitions` partitions. fn spawn_dispatch( self, + num_partitions: usize, f: impl Fn(BoxedExecutor) -> BoxedDispatcher, ) -> PartitionedStreamSubscriber { - spawn_dispatchers(self.streams.into_iter().map(f).collect()) + spawn_dispatchers(self.streams.into_iter().map(f).collect(), num_partitions) } } @@ -744,7 +786,7 @@ impl PartitionedStreamSubscriber { /// New streams can be created by calling `subscribe`. struct StreamSubscriber { rx: async_broadcast::InactiveReceiver>, - handle: Arc, + task_handle: Arc, } impl StreamSubscriber { @@ -753,15 +795,15 @@ impl StreamSubscriber { #[try_stream(boxed, ok = DataChunk, error = ExecutorError)] async fn to_stream( rx: async_broadcast::Receiver>, - handle: Arc, + task_handle: Arc, ) { #[for_await] for chunk in rx { yield chunk?; } - drop(handle); + drop(task_handle); } - to_stream(self.rx.activate_cloned(), self.handle.clone()) + to_stream(self.rx.activate_cloned(), self.task_handle.clone()) } } From 8fe33432edb2a05b37740b7c36123c0c1cc19830 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 12 Nov 2024 00:11:47 +0800 Subject: [PATCH 06/29] fix metrics and improve debug info Signed-off-by: Runji Wang --- src/executor/analyze.rs | 33 ++++++++++++++++++++++----------- src/executor/mod.rs | 20 ++++++++++++-------- src/utils/timed.rs | 12 +++++++++++- 3 files changed, 45 insertions(+), 20 deletions(-) diff --git a/src/executor/analyze.rs b/src/executor/analyze.rs index 8e521f09..4caf1bd0 100644 --- a/src/executor/analyze.rs +++ b/src/executor/analyze.rs @@ -1,5 +1,6 @@ // Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. +use std::fmt::{self, Debug}; use std::sync::atomic::{AtomicU64, Ordering}; use pretty_xmlish::PrettyConfig; @@ -26,10 +27,16 @@ impl AnalyzeExecutor { // explain the plan let get_metadata = |id| { - vec![ - ("rows", self.metrics.get_rows(id).to_string()), - ("time", format!("{:?}", self.metrics.get_time(id))), - ] + let mut metadata = Vec::new(); + if let Some(rows) = self.metrics.get_rows(id) { + let total = rows.iter().sum::(); + metadata.push(("rows", format!("{total} = {rows:?}"))); + } + if let Some(time) = self.metrics.get_time(id) { + let total = time.iter().sum::(); + metadata.push(("time", format!("{total:?} = {time:?}"))); + } + metadata }; let explain_obj = Explain::of(&self.plan) .with_catalog(&self.catalog) @@ -50,7 +57,7 @@ impl AnalyzeExecutor { } /// A collection of profiling information for a query. -#[derive(Default)] +#[derive(Default, Debug)] pub struct Metrics { spans: HashMap>, rows: HashMap>, @@ -64,19 +71,17 @@ impl Metrics { } /// Get the running time for a node. - pub fn get_time(&self, id: Id) -> Duration { + pub fn get_time(&self, id: Id) -> Option> { self.spans .get(&id) - .map(|spans| spans.iter().map(|span| span.busy_time()).sum()) - .unwrap() + .map(|spans| spans.iter().map(|span| span.busy_time()).collect()) } /// Get the number of rows produced by a node. - pub fn get_rows(&self, id: Id) -> u64 { + pub fn get_rows(&self, id: Id) -> Option> { self.rows .get(&id) - .map(|rows| rows.iter().map(|counter| counter.get()).sum()) - .unwrap() + .map(|rows| rows.iter().map(|counter| counter.get()).collect()) } } @@ -86,6 +91,12 @@ pub struct Counter { count: Arc, } +impl Debug for Counter { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.get()) + } +} + impl Counter { /// Increments the counter. pub fn inc(&self, value: u64) { diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 985f8c82..674545af 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -497,13 +497,17 @@ impl Builder { .execute() .into(), - Analyze(child) => AnalyzeExecutor { - plan: self.recexpr(child), - catalog: self.optimizer.catalog().clone(), - metrics: std::mem::take(&mut self.metrics), + Analyze(child) => { + let stream = self.build_id(child).spawn_merge(); + AnalyzeExecutor { + plan: self.recexpr(child), + catalog: self.optimizer.catalog().clone(), + // note: make sure to take the metrics after building the child stream + metrics: std::mem::take(&mut self.metrics), + } + .execute(stream) + .into() } - .execute(self.build_id(child).spawn_merge()) - .into(), Empty(_) => futures::stream::empty().boxed().into(), @@ -558,8 +562,8 @@ impl Builder { fn instrument(&mut self, id: Id, stream: PartitionedStream) -> PartitionedStream { // let name = self.node(id).to_string(); let partitions = stream.streams.len(); - let spans = vec![TimeSpan::default(); partitions]; - let output_row_counters = vec![Counter::default(); partitions]; + let spans = (0..partitions).map(|_| TimeSpan::default()).collect_vec(); + let output_row_counters = (0..partitions).map(|_| Counter::default()).collect_vec(); self.metrics .register(id, spans.clone(), output_row_counters.clone()); diff --git a/src/utils/timed.rs b/src/utils/timed.rs index ea65331e..7c0589c3 100644 --- a/src/utils/timed.rs +++ b/src/utils/timed.rs @@ -1,3 +1,4 @@ +use std::fmt::{self, Debug}; use std::sync::Arc; use std::task::Poll; use std::time::{Duration, Instant}; @@ -53,11 +54,20 @@ impl Stream for Timed { } } -#[derive(Debug, Default, Clone)] +#[derive(Default, Clone)] pub struct Span { inner: Arc>, } +impl Debug for Span { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Span") + .field("busy_time", &self.busy_time()) + .field("last_poll_time", &self.last_poll_time()) + .finish() + } +} + #[derive(Debug, Default)] struct SpanInner { busy_time: Duration, From cdea038004cfe0f9d8e53ab1a73ff030cef4ed57 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 17 Nov 2024 20:10:56 +0800 Subject: [PATCH 07/29] add a pragma to control parallel plan Signed-off-by: Runji Wang --- src/db.rs | 25 ++++++++++++------------- src/executor/mod.rs | 8 +++++++- src/planner/optimizer.rs | 9 ++++++++- 3 files changed, 27 insertions(+), 15 deletions(-) diff --git a/src/db.rs b/src/db.rs index c8eb29af..00242f84 100644 --- a/src/db.rs +++ b/src/db.rs @@ -27,8 +27,12 @@ pub struct Database { /// The configuration of the database. #[derive(Debug, Default)] struct Config { + /// If true, no optimization will be applied to the query. disable_optimizer: bool, mock_stat: Option, + /// If true, each operator will be parallelized and partitioned. + /// WARN: This feature is under development and may not work properly. + enable_parallel_execution: bool, } impl Database { @@ -93,8 +97,7 @@ impl Database { crate::planner::Config { enable_range_filter_scan: self.storage.support_range_filter_scan(), table_is_sorted_by_primary_key: self.storage.table_is_sorted_by_primary_key(), - generate_parallel_plan: tokio::runtime::Handle::current().metrics().num_workers() - > 1, + generate_parallel_plan: self.config.lock().unwrap().enable_parallel_execution, }, ); @@ -160,19 +163,15 @@ impl Database { /// Mock the row count of a table for planner test. fn handle_set(&self, stmt: &Statement) -> Result { if let Statement::Pragma { name, .. } = stmt { + let mut config = self.config.lock().unwrap(); match name.to_string().as_str() { - "enable_optimizer" => { - self.config.lock().unwrap().disable_optimizer = false; - return Ok(true); - } - "disable_optimizer" => { - self.config.lock().unwrap().disable_optimizer = true; - return Ok(true); - } - name => { - return Err(crate::binder::BindError::NoPragma(name.into()).into()); - } + "enable_optimizer" => config.disable_optimizer = false, + "disable_optimizer" => config.disable_optimizer = true, + "enable_parallel_execution" => config.enable_parallel_execution = true, + "disable_parallel_execution" => config.enable_parallel_execution = false, + name => return Err(crate::binder::BindError::NoPragma(name.into()).into()), } + return Ok(true); } let Statement::SetVariable { variable, value, .. diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 674545af..7e5a36a8 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -212,7 +212,7 @@ impl Builder { /// Builds stream for the given plan. fn build_id(&mut self, id: Id) -> PartitionedStream { use Expr::*; - let stream = match self.node(id).clone() { + let mut stream = match self.node(id).clone() { Scan([table, list, filter]) => { let table_id = self.node(table).as_table(); let columns = (self.node(list).as_list().iter()) @@ -556,6 +556,12 @@ impl Builder { node => panic!("not a plan: {node:?}"), }; + // if parallel plan is enabled, each executor will be partitioned and consecutive + // executors may be fused into a single task. otherwise, we spawn a new task for each + // executor so that executors can be executed in parallel. + if !self.optimizer.config().generate_parallel_plan { + stream = stream.spawn().subscribe(); + } self.instrument(id, stream) } diff --git a/src/planner/optimizer.rs b/src/planner/optimizer.rs index 48bf08e9..3e139187 100644 --- a/src/planner/optimizer.rs +++ b/src/planner/optimizer.rs @@ -8,7 +8,7 @@ use self::rules::partition::to_parallel_plan; use super::*; use crate::catalog::RootCatalogRef; -/// Plan optimizer. +/// Optimizer transforms the query plan into a more efficient one. #[derive(Clone)] pub struct Optimizer { analysis: ExprAnalysis, @@ -16,9 +16,11 @@ pub struct Optimizer { /// Optimizer configurations. #[derive(Debug, Clone, Default)] +#[non_exhaustive] pub struct Config { pub enable_range_filter_scan: bool, pub table_is_sorted_by_primary_key: bool, + /// If true, the optimizer will insert exchange operators to the plan. pub generate_parallel_plan: bool, } @@ -114,6 +116,11 @@ impl Optimizer { pub fn catalog(&self) -> &RootCatalogRef { &self.analysis.catalog } + + /// Returns the configurations. + pub fn config(&self) -> &Config { + &self.analysis.config + } } /// Stage1 rules in the optimizer. From fe2ee6c3c0d524a67ddd509fcba9a60270e8333a Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Fri, 22 Nov 2024 02:08:29 +0800 Subject: [PATCH 08/29] two-phase aggregation Signed-off-by: Runji Wang --- src/executor/mod.rs | 5 +- src/planner/explain.rs | 1 + src/planner/mod.rs | 3 ++ src/planner/rules/order.rs | 1 + src/planner/rules/partition.rs | 84 +++++++++++++++++++++++++++++++++- src/planner/rules/rows.rs | 1 + src/planner/rules/schema.rs | 1 + src/planner/rules/type_.rs | 2 +- 8 files changed, 94 insertions(+), 4 deletions(-) diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 7e5a36a8..8c31f970 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -511,6 +511,8 @@ impl Builder { Empty(_) => futures::stream::empty().boxed().into(), + Schema([_, child]) => self.build_id(child), // schema node is just pass-through + Exchange([dist, child]) => match self.node(dist) { Single => self.build_id(child).spawn_merge().into(), Broadcast => { @@ -556,13 +558,14 @@ impl Builder { node => panic!("not a plan: {node:?}"), }; + stream = self.instrument(id, stream); // if parallel plan is enabled, each executor will be partitioned and consecutive // executors may be fused into a single task. otherwise, we spawn a new task for each // executor so that executors can be executed in parallel. if !self.optimizer.config().generate_parallel_plan { stream = stream.spawn().subscribe(); } - self.instrument(id, stream) + stream } fn instrument(&mut self, id: Id, stream: PartitionedStream) -> PartitionedStream { diff --git a/src/planner/explain.rs b/src/planner/explain.rs index 6bf5a614..0feada96 100644 --- a/src/planner/explain.rs +++ b/src/planner/explain.rs @@ -386,6 +386,7 @@ impl<'a> Explain<'a> { ), Empty(_) => Pretty::childless_record("Empty", with_meta(vec![])), Max1Row(child) => Pretty::fieldless_record("Max1Row", vec![self.expr(child).pretty()]), + Schema([_, child]) => self.child(child).pretty(), } } } diff --git a/src/planner/mod.rs b/src/planner/mod.rs index cba38ac5..cec5f6c4 100644 --- a/src/planner/mod.rs +++ b/src/planner/mod.rs @@ -145,6 +145,9 @@ define_language! { // with the same schema as `child` "max1row" = Max1Row(Id), // (max1row child) // convert table to scalar + "schema" = Schema([Id; 2]), // (schema [expr..] child) + // reset schema of child to [expr..] + // this node is just pass-through in execution Symbol(Symbol), } diff --git a/src/planner/rules/order.rs b/src/planner/rules/order.rs index 381bda9a..7378e5c7 100644 --- a/src/planner/rules/order.rs +++ b/src/planner/rules/order.rs @@ -37,6 +37,7 @@ pub fn analyze_order(egraph: &EGraph, enode: &Expr) -> OrderKey { MergeJoin([_, _, _, _, _, r]) => x(r).clone(), SortAgg([_, _, c]) => x(c).clone(), Exchange([_, c]) => x(c).clone(), + Schema([_, c]) => x(c).clone(), // unordered for other plans _ => Box::new([]), } diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index de7005ed..ab2c798d 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -118,8 +118,13 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { ), // 2-phase aggregation rw!("agg-to-parallel"; - "(to_parallel (agg ?exprs ?child))" => - "(agg ?exprs (exchange single (agg ?exprs (exchange random (to_parallel ?child)))))" + "(to_parallel (agg ?aggs ?child))" => + { apply_global_aggs(" + (schema ?aggs (agg ?global_aggs (exchange single + (agg ?aggs (exchange random (to_parallel ?child)))))) + ") } + // to keep the schema unchanged, we add a `schema` node + // FIXME: check if all aggs are supported in 2-phase aggregation ), // hash aggregation can be partitioned by group key rw!("hashagg-to-parallel"; @@ -168,6 +173,62 @@ fn partition_is_same( move |egraph, _, subst| egraph[subst[a]].data == egraph[subst[b]].data } +/// Returns an applier that replaces `?global_aggs` with the nested `?aggs`. +/// +/// ```text +/// ?aggs = (list (sum a) (count b)) +/// ?global_aggs = (list (sum (ref (sum a))) (count (ref (count b)))) +/// ``` +fn apply_global_aggs(pattern_str: &str) -> impl Applier { + struct ApplyGlobalAggs { + pattern: Pattern, + aggs: Var, + global_aggs: Var, + } + impl Applier for ApplyGlobalAggs { + fn apply_one( + &self, + egraph: &mut EGraph, + eclass: Id, + subst: &Subst, + searcher_ast: Option<&PatternAst>, + rule_name: Symbol, + ) -> Vec { + let aggs = egraph[subst[self.aggs]].as_list().to_vec(); + let mut global_aggs = vec![]; + for agg in aggs { + use Expr::*; + let ref_id = egraph.add(Expr::Ref(agg)); + let global_agg = match &egraph[agg].nodes[0] { + RowCount => RowCount, + Max(_) => Max(ref_id), + Min(_) => Min(ref_id), + Sum(_) => Sum(ref_id), + Avg(_) => panic!("avg is not supported in 2-phase aggregation"), + Count(_) => Count(ref_id), + CountDistinct(_) => { + panic!("count distinct is not supported in 2-phase aggregation") + } + First(_) => First(ref_id), + Last(_) => Last(ref_id), + node => panic!("invalid agg: {}", node), + }; + global_aggs.push(egraph.add(global_agg)); + } + let id = egraph.add(Expr::List(global_aggs.into())); + let mut subst = subst.clone(); + subst.insert(self.global_aggs, id); + self.pattern + .apply_one(egraph, eclass, &subst, searcher_ast, rule_name) + } + } + ApplyGlobalAggs { + pattern: pattern(pattern_str), + aggs: var("?aggs"), + global_aggs: var("?global_aggs"), + } +} + /// Describes how data is partitioned. #[derive(Debug, Default, Clone, PartialEq, Eq)] pub enum Partition { @@ -270,4 +331,23 @@ mod tests { let expected: RecExpr = distributed.parse().unwrap(); assert_eq!(output.to_string(), expected.to_string()); } + + #[test] + fn test_two_phase_agg() { + let input = " + (agg (list (sum a)) + (scan t1 (list a) true)) + "; + let distributed = " + (schema (list (sum a)) + (agg (list (sum (ref (sum a)))) + (exchange single + (agg (list (sum a)) + (exchange random + (scan t1 (list a) true)))))) + "; + let output = to_parallel_plan(input.parse().unwrap()); + let expected: RecExpr = distributed.parse().unwrap(); + assert_eq!(output.to_string(), expected.to_string()); + } } diff --git a/src/planner/rules/rows.rs b/src/planner/rules/rows.rs index afca6355..c3dc2000 100644 --- a/src/planner/rules/rows.rs +++ b/src/planner/rules/rows.rs @@ -76,6 +76,7 @@ pub fn analyze_rows(egraph: &EGraph, enode: &Expr) -> Rows { }, Empty(_) => 0.0, Max1Row(_) => 1.0, + Schema([_, c]) => x(c), // FIXME: broadcast distribution should multiply the number of rows Exchange([_, c]) => x(c), diff --git a/src/planner/rules/schema.rs b/src/planner/rules/schema.rs index 1b6a89ed..4e8719ff 100644 --- a/src/planner/rules/schema.rs +++ b/src/planner/rules/schema.rs @@ -38,6 +38,7 @@ pub fn analyze_schema( Proj([exprs, _]) | Agg([exprs, _]) => x(exprs), Window([exprs, child]) => concat(x(child), x(exprs)), HashAgg([keys, aggs, _]) | SortAgg([keys, aggs, _]) => concat(x(keys), x(aggs)), + Schema([exprs, _]) => x(exprs), // not plan node _ => vec![], diff --git a/src/planner/rules/type_.rs b/src/planner/rules/type_.rs index 294a0aa4..59a9f0f3 100644 --- a/src/planner/rules/type_.rs +++ b/src/planner/rules/type_.rs @@ -143,7 +143,7 @@ pub fn analyze_type( // equal to child Filter([_, c]) | Order([_, c]) | Limit([_, _, c]) | TopN([_, _, _, c]) | Empty(c) - | Exchange([_, c]) => x(c), + | Exchange([_, c]) | Schema([_, c]) => x(c), // concat 2 children Join([t, _, l, r]) | HashJoin([t, _, _, _, l, r]) | MergeJoin([t, _, _, _, l, r]) => { From 67bf63bde8b0e0a4d83737859971c18990c716dd Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 16:28:37 +0800 Subject: [PATCH 09/29] update rust toolchain and dependencies Signed-off-by: Runji Wang --- Cargo.lock | 1106 ++++++++++++++++++++++++------------------------ rust-toolchain | 2 +- src/lib.rs | 1 - src/main.rs | 2 - 4 files changed, 551 insertions(+), 560 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 701be178..985fb101 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4,18 +4,18 @@ version = 3 [[package]] name = "addr2line" -version = "0.21.0" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a30b2e23b9e17a9f90641c7ab1549cd9b44f296d3ccbf309d2863cfe398a0cb" +checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" dependencies = [ "gimli", ] [[package]] -name = "adler" -version = "1.0.2" +name = "adler2" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" [[package]] name = "ahash" @@ -73,57 +73,58 @@ checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" [[package]] name = "anstream" -version = "0.6.13" +version = "0.6.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d96bd03f33fe50a863e394ee9718a706f988b9079b20c3784fb726e7678b62fb" +checksum = "8acc5369981196006228e28809f761875c0327210a891e941f4c683b3a99529b" dependencies = [ "anstyle", "anstyle-parse", "anstyle-query", "anstyle-wincon", "colorchoice", + "is_terminal_polyfill", "utf8parse", ] [[package]] name = "anstyle" -version = "1.0.6" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8901269c6307e8d93993578286ac0edf7f195079ffff5ebdeea6a59ffb7e36bc" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anstyle-parse" -version = "0.2.3" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c75ac65da39e5fe5ab759307499ddad880d724eed2f6ce5b5e8a26f4f387928c" +checksum = "3b2d16507662817a6a20a9ea92df6652ee4f94f914589377d69f3b21bc5798a9" dependencies = [ "utf8parse", ] [[package]] name = "anstyle-query" -version = "1.0.2" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e28923312444cdd728e4738b3f9c9cac739500909bb3d3c94b43551b16517648" +checksum = "79947af37f4177cfead1110013d678905c37501914fba0efea834c3fe9a8d60c" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "anstyle-wincon" -version = "3.0.2" +version = "3.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1cd54b81ec8d6180e24654d0b371ad22fc3dd083b6ff8ba325b72e00c87660a7" +checksum = "2109dbce0e72be3ec00bed26e6a7479ca384ad226efdd66db8fa2e3a38c83125" dependencies = [ "anstyle", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "anyhow" -version = "1.0.82" +version = "1.0.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f538837af36e6f6a9be0faa67f9a314f8119e4e4b5867c6ab40ed60360142519" +checksum = "4c95c10ba0b00a02636238b814946408b1322d5ac4760326e6fb8ec956d85775" [[package]] name = "array-init" @@ -133,49 +134,49 @@ checksum = "3d62b7694a562cdf5a74227903507c56ab2cc8bdd1f781ed5cb4cf9c9f810bfc" [[package]] name = "arrayvec" -version = "0.7.4" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "async-broadcast" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "258b52a1aa741b9f09783b2d86cf0aeeb617bbf847f6933340a39644227acbdb" +checksum = "20cd0e2e25ea8e5f7e9df04578dc6cf5c83577fd09b1a46aaf5c85e1c33f2a7e" dependencies = [ - "event-listener 5.3.0", - "event-listener-strategy 0.5.1", + "event-listener", + "event-listener-strategy", "futures-core", "pin-project-lite", ] [[package]] name = "async-lock" -version = "3.3.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d034b430882f8381900d3fe6f0aaa3ad94f2cb4ac519b429692a1bc2dda4ae7b" +checksum = "ff6e472cdea888a4bd64f342f09b3f50e1886d32afe8df3d663c01140b811b18" dependencies = [ - "event-listener 4.0.3", - "event-listener-strategy 0.4.0", + "event-listener", + "event-listener-strategy", "pin-project-lite", ] [[package]] name = "async-recursion" -version = "1.1.0" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30c5ef0ede93efbf733c1a727f3b6b5a1060bbedd5600183e66f6e4be4af0ec5" +checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "async-stream" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd56dd203fef61ac097dd65721a419ddccb106b2d2b70ba60a6b529f03961a51" +checksum = "0b5a71a6f37880a80d1d7f19efd781e4b5de42c88f0722cc13bcb6cc2cfe8476" dependencies = [ "async-stream-impl", "futures-core", @@ -184,31 +185,31 @@ dependencies = [ [[package]] name = "async-stream-impl" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" +checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "async-trait" -version = "0.1.80" +version = "0.1.83" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "autocfg" -version = "1.2.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1fdabc7756949593fe60f30ec81974b613357de856987752631dea1e3394c80" +checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "axum" @@ -257,17 +258,17 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.71" +version = "0.3.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b05800d2e817c8b3b4b54abd461726265fa9789ae34330622f2db9ee696f9d" +checksum = "8d82cb332cdfaed17ae235a638438ac4d4839913cc2af585c3c6746e8f8bee1a" dependencies = [ "addr2line", - "cc", "cfg-if", "libc", "miniz_oxide", "object", "rustc-demangle", + "windows-targets 0.52.6", ] [[package]] @@ -278,9 +279,9 @@ checksum = "9d297deb1925b89f2ccc13d7635fa0714f12c87adce1c75356b39ca9b7178567" [[package]] name = "base64" -version = "0.22.0" +version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9475866fec1451be56a3c2400fd081ff546538961565ccb5b7142cbd22bc7a51" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" [[package]] name = "base64ct" @@ -330,9 +331,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf4b9d6a944f767f8e5e0db018570623c85f3d925ac718db4e06d0187adb21c1" +checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de" [[package]] name = "bitvec" @@ -358,26 +359,25 @@ dependencies = [ [[package]] name = "borsh" -version = "1.4.0" +version = "1.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0901fc8eb0aca4c83be0106d6f2db17d86a08dfc2c25f0e84464bf381158add6" +checksum = "2506947f73ad44e344215ccd6403ac2ae18cd8e046e581a441bf8d199f257f03" dependencies = [ "borsh-derive", - "cfg_aliases", + "cfg_aliases 0.2.1", ] [[package]] name = "borsh-derive" -version = "1.4.0" +version = "1.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51670c3aa053938b0ee3bd67c3817e471e626151131b934038e83c5bf8de48f5" +checksum = "c2593a3b8b938bd68373196c9832f516be11fa487ef4ae745eb282e6a56a7244" dependencies = [ "once_cell", "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.60", - "syn_derive", + "syn 2.0.89", ] [[package]] @@ -425,9 +425,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.6.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "514de17de45fdb8dc022b1a7975556c53c86f9f0aa5f534b98977b171857c2c9" +checksum = "9ac0150caa2ae65ca5bd83f25c7de183dea78d4d366469f148435e2acfbad0da" [[package]] name = "cache-padded" @@ -443,9 +443,12 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.0.94" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f6e324229dc011159fcc089755d1e2e216a90d43a7dea6853ca740b84f35e7" +checksum = "fd9de9f2205d5ef3fd67e685b0df337994ddd4495e2a28d185500d0e1edfea47" +dependencies = [ + "shlex", +] [[package]] name = "cfg-if" @@ -459,6 +462,12 @@ version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" +[[package]] +name = "cfg_aliases" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" + [[package]] name = "chrono" version = "0.4.38" @@ -470,7 +479,7 @@ dependencies = [ "js-sys", "num-traits", "wasm-bindgen", - "windows-targets 0.52.5", + "windows-targets 0.52.6", ] [[package]] @@ -502,9 +511,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.4" +version = "4.5.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90bc066a67923782aa8515dbaea16946c5bcc5addbd668bb80af688e53e548a0" +checksum = "fb3b4b9e5a7c7514dfa52869339ee98b3156b0bfb4e8a77c4ff4babb64b1604f" dependencies = [ "clap_builder", "clap_derive", @@ -512,9 +521,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.2" +version = "4.5.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae129e2e766ae0ec03484e609954119f123cc1fe650337e155d03b022f24f7b4" +checksum = "b17a95aa67cc7b5ebd32aa5370189aa0d79069ef1c64ce893bd30fb24bff20ec" dependencies = [ "anstream", "anstyle", @@ -524,46 +533,46 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.4" +version = "4.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "528131438037fd55894f62d6e9f068b8f45ac57ffa77517819645d10aed04f64" +checksum = "4ac6a0c7b1a9e9a5186361f67dfa1b88213572f427fb9ab038efb2bd8c582dab" dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "clap_lex" -version = "0.7.0" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98cc8fbded0c607b7ba9dd60cd98df59af97e84d24e49c8557331cfc26d301ce" +checksum = "afb84c814227b90d6895e01398aee0d8033c00e7466aca416fb6a8e0eb19d8a7" [[package]] name = "clipboard-win" -version = "5.3.1" +version = "5.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79f4473f5144e20d9aceaf2972478f06ddf687831eafeeb434fbaf0acc4144ad" +checksum = "15efe7a882b08f34e38556b14f2fb3daa98769d06c7f0c1b076dfd0d983bc892" dependencies = [ "error-code", ] [[package]] name = "colorchoice" -version = "1.0.0" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acbf1af155f9b9ef647e42cdc158db4b64a1b61f743629225fde6f3e0be2a7c7" +checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" [[package]] name = "comfy-table" -version = "7.1.1" +version = "7.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b34115915337defe99b2aff5c2ce6771e5fbc4079f4b506301f5cf394c8452f7" +checksum = "24f165e7b643266ea80cb858aed492ad9280e3e05ce24d4a99d7d7b889b6a4d9" dependencies = [ "strum", "strum_macros", - "unicode-width", + "unicode-width 0.2.0", ] [[package]] @@ -574,9 +583,9 @@ checksum = "120133d4db2ec47efe2e26502ee984747630c67f51974fca0b6c1340cf2368d3" [[package]] name = "concurrent-queue" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d16048cd947b08fa32c24458a22f5dc5e835264f689f4f5653210c69fd107363" +checksum = "4ca0197aee26d1ae37445ee532fefce43251d24cc7c166799f4d46817f1d3973" dependencies = [ "crossbeam-utils", ] @@ -590,7 +599,7 @@ dependencies = [ "encode_unicode", "lazy_static", "libc", - "unicode-width", + "unicode-width 0.1.14", "windows-sys 0.52.0", ] @@ -639,24 +648,24 @@ checksum = "c2459377285ad874054d797f3ccebf984978aa39129f6eafde5cdc8315b612f8" [[package]] name = "core-foundation-sys" -version = "0.8.6" +version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06ea2b9bc92be3c2baa9334a323ebca2d6f074ff852cd1d7b11064035cd3868f" +checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" [[package]] name = "cpufeatures" -version = "0.2.12" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53fe5e26ff1b7aef8bca9c6080520cfb8d9333c7568e1829cef191a9723e5504" +checksum = "16b80225097f2e5ae4e7179dd2266824648f3e2f49d9134d584b76389d31c4c3" dependencies = [ "libc", ] [[package]] name = "crc32fast" -version = "1.4.0" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3855a8a784b474f333699ef2bbca9db2c4a1f6d9088a90a2d25b1eb53111eaa" +checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" dependencies = [ "cfg-if", ] @@ -701,9 +710,9 @@ dependencies = [ [[package]] name = "crossbeam-channel" -version = "0.5.12" +version = "0.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab3db02a9c5b5121e1e42fbdb1aeb65f5e02624cc58c43f2884c6ccac0b82f95" +checksum = "33480d6946193aa8033910124896ca395333cae7e2d1113d1fef6c3272217df2" dependencies = [ "crossbeam-utils", ] @@ -729,9 +738,9 @@ dependencies = [ [[package]] name = "crossbeam-utils" -version = "0.8.19" +version = "0.8.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "248e3bacc7dc6baa3b21e405ee045c3047101a49145e7e9eca583ab4c2ca5345" +checksum = "22ec99545bb0ed0ea7bb9b8e1e9122ea386ff8a48c0922e43f36d45ab09e0e80" [[package]] name = "crunchy" @@ -751,9 +760,9 @@ dependencies = [ [[package]] name = "csv" -version = "1.3.0" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac574ff4d437a7b5ad237ef331c17ccca63c46479e5b5453eb8e10bb99a759fe" +checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" dependencies = [ "csv-core", "itoa", @@ -807,7 +816,7 @@ checksum = "d150dea618e920167e5973d70ae6ece4385b7164e0d799fe7c122dd0a5d912ad" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -850,14 +859,14 @@ checksum = "75b325c5dbd37f80359721ad39aca5a29fb04c89279657cffdda8736d0c0b9d2" [[package]] name = "educe" -version = "0.4.23" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f0042ff8246a363dbe77d2ceedb073339e85a804b9a47636c6e016a9a32c05f" +checksum = "1d7bc049e1bd8cdeb31b68bbd586a9464ecf9f3944af3958a7a9d0f8b9799417" dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.89", ] [[package]] @@ -880,9 +889,9 @@ dependencies = [ [[package]] name = "either" -version = "1.11.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47c1c47d2f5964e29c61246e81db715514cd532db6b5116a25ea3c03d6780a2" +checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" [[package]] name = "encode_unicode" @@ -898,15 +907,22 @@ checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" [[package]] name = "enum-ordinalize" -version = "3.1.15" +version = "4.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bf1fa3f06bbff1ea5b1a9c7b14aa992a39657db60a2759457328d7e058f49ee" +checksum = "fea0dcfa4e54eeb516fe454635a95753ddd39acda650ce703031c6973e315dd5" +dependencies = [ + "enum-ordinalize-derive", +] + +[[package]] +name = "enum-ordinalize-derive" +version = "4.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" dependencies = [ - "num-bigint", - "num-traits", "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -918,7 +934,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -938,18 +954,19 @@ checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" [[package]] name = "erased-serde" -version = "0.4.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b73807008a3c7f171cc40312f37d95ef0396e048b5848d775f54b1a4dd4a0d3" +checksum = "24e2389d65ab4fab27dc2a5de7b191e1f6617d1f1c8855c0dc569c94a4cbb18d" dependencies = [ "serde", + "typeid", ] [[package]] name = "errno" -version = "0.3.8" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a258e46cdc063eb8519c00b9fc845fc47bcfca4130e2f08e88665ceda8474245" +checksum = "534c5cf6194dfab3db3242765c03bbe257cf92f22b38f6bc0c58d59108a820ba" dependencies = [ "libc", "windows-sys 0.52.0", @@ -957,35 +974,21 @@ dependencies = [ [[package]] name = "error-code" -version = "3.2.0" +version = "3.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0474425d51df81997e2f90a21591180b38eccf27292d755f3e30750225c175b" +checksum = "a5d9305ccc6942a704f4335694ecd3de2ea531b114ac2d51f5f843750787a92f" [[package]] name = "escape8259" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba4f4911e3666fcd7826997b4745c8224295a6f3072f1418c3067b97a67557ee" -dependencies = [ - "rustversion", -] - -[[package]] -name = "event-listener" -version = "4.0.3" +version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67b215c49b2b248c855fb73579eb1f4f26c38ffdc12973e20e07b91d78d5646e" -dependencies = [ - "concurrent-queue", - "parking", - "pin-project-lite", -] +checksum = "5692dd7b5a1978a5aeb0ce83b7655c58ca8efdcb79d21036ea249da95afec2c6" [[package]] name = "event-listener" -version = "5.3.0" +version = "5.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d9944b8ca13534cdfb2800775f8dd4902ff3fc75a50101466decadfdf322a24" +checksum = "6032be9bd27023a771701cc49f9f053c751055f71efb2e0ae5c15809093675ba" dependencies = [ "concurrent-queue", "parking", @@ -994,21 +997,11 @@ dependencies = [ [[package]] name = "event-listener-strategy" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "958e4d70b6d5e81971bebec42271ec641e7ff4e170a6fa605f2b8a8b65cb97d3" -dependencies = [ - "event-listener 4.0.3", - "pin-project-lite", -] - -[[package]] -name = "event-listener-strategy" -version = "0.5.1" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "332f51cb23d20b0de8458b86580878211da09bcd4503cb579c225b3d124cabb3" +checksum = "0f214dc438f977e6d4e3500aaa277f5ad94ca83fbbd9b1a15713ce2344ccc5a1" dependencies = [ - "event-listener 5.3.0", + "event-listener", "pin-project-lite", ] @@ -1020,9 +1013,9 @@ checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" [[package]] name = "fastrand" -version = "2.0.2" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "658bd65b1cf4c852a3cc96f18a8ce7b5640f6b703f905c7d74532294c2a63984" +checksum = "486f806e73c5707928240ddc295403b1b93c96a02038563881c4a2fd84b81ac4" [[package]] name = "fd-lock" @@ -1035,12 +1028,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "finl_unicode" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fcfdc7a0362c9f4444381a9e697c79d435fe65b52a37466fc2c1184cee9edc6" - [[package]] name = "fixedbitset" version = "0.4.2" @@ -1049,9 +1036,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flate2" -version = "1.0.28" +version = "1.0.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" +checksum = "c936bfdafb507ebbf50b8074c54fa31c5be9a1e7e5f467dd659697041407d07c" dependencies = [ "crc32fast", "miniz_oxide", @@ -1080,9 +1067,9 @@ checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" [[package]] name = "futures" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "645c6916888f6cb6350d2550b80fb63e734897a8498abe35cfb732b6487804b0" +checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" dependencies = [ "futures-channel", "futures-core", @@ -1095,9 +1082,9 @@ dependencies = [ [[package]] name = "futures-async-stream" -version = "0.2.10" +version = "0.2.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5774510e3d9ae616fd40a4232c723d46b74faa5b5c7484ffaa8a8a4a5f5e8a13" +checksum = "38c4ff593faf24530b9a6eeea6ea4b5e3c20591da3f6348069724cb261d6d643" dependencies = [ "futures-async-stream-macro", "futures-core", @@ -1106,20 +1093,20 @@ dependencies = [ [[package]] name = "futures-async-stream-macro" -version = "0.2.10" +version = "0.2.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7433cef2775d3bbd39a04f86b4bed2aa37ee3dbf75efaf8a783a36e7c91a33cc" +checksum = "f664c1c2186b81f798ac765d661fb8cefd74fdb398fd23c76c3fb3c1aec760e8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "futures-channel" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eac8f7d7865dcb88bd4373ab671c8cf4508703796caa2b1985a9ca867b3fcb78" +checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" dependencies = [ "futures-core", "futures-sink", @@ -1127,15 +1114,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfc6580bb841c5a68e9ef15c77ccc837b40a7504914d52e47b8b0e9bbda25a1d" +checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" [[package]] name = "futures-executor" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a576fc72ae164fca6b9db127eaa9a9dda0d61316034f33a0a0d4eda41f02b01d" +checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" dependencies = [ "futures-core", "futures-task", @@ -1144,38 +1131,38 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a44623e20b9681a318efdd71c299b6b222ed6f231972bfe2f224ebad6311f0c1" +checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" [[package]] name = "futures-macro" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" +checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "futures-sink" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fb8e00e87438d937621c1c6269e53f536c14d3fbd6a042bb24879e57d474fb5" +checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" [[package]] name = "futures-task" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38d84fa142264698cdce1a9f9172cf383a0c82de1bddcf3092901442c4097004" +checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" [[package]] name = "futures-util" -version = "0.3.30" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d6401deb83407ab3da39eba7e33987a73c3df0c82b4bb5813ee871c19c41d48" +checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" dependencies = [ "futures-channel", "futures-core", @@ -1210,9 +1197,9 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94b22e06ecb0110981051723910cbf0b5f5e09a2062dd7663334ee79a9d1286c" +checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" dependencies = [ "cfg-if", "libc", @@ -1221,9 +1208,9 @@ dependencies = [ [[package]] name = "gimli" -version = "0.28.1" +version = "0.31.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4271d37baee1b8c7e4b708028c57d816cf9d2434acb33a549475f78c181f6253" +checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" @@ -1243,7 +1230,7 @@ dependencies = [ "futures-sink", "futures-util", "http", - "indexmap 2.2.6", + "indexmap 2.6.0", "slab", "tokio", "tokio-util", @@ -1271,9 +1258,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.14.3" +version = "0.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "290f1a1d9242c78d09ce40a5e87e7554ee637af1351968159f4952f028f75604" +checksum = "3a9bfc1af68b1726ea47d3d5109de126281def866b33970e10fbab11b5dafab3" [[package]] name = "hdrhistogram" @@ -1306,6 +1293,12 @@ version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" +[[package]] +name = "hermit-abi" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbf6a919d6cf397374f7dfeeea91d974c7c0a7221d0d0f4f20d859d329e53fcc" + [[package]] name = "hex" version = "0.4.3" @@ -1354,9 +1347,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.8.0" +version = "1.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" +checksum = "7d71d3574edd2771538b901e6549113b4006ece66150fb69c0fb6d9a2adae946" [[package]] name = "httpdate" @@ -1372,9 +1365,9 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.28" +version = "0.14.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf96e135eb83a2a8ddf766e426a841d8ddd7449d5f00d34ea02b41d2f19eef80" +checksum = "8c08302e8fa335b151b788c775ff56e7a03ae64ff85c548ee820fecb70356e85" dependencies = [ "bytes", "futures-channel", @@ -1408,9 +1401,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.60" +version = "0.1.61" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7ffbb5a1b541ea2561f8c41c087286cc091e21e556a4f09a8f6cbf17b69b141" +checksum = "235e081f3925a06703c2d0117ea8b91f042756fd6e7a6e5d901e8ca1a996b220" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -1441,25 +1434,25 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.2.6" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "168fb715dda47215e360912c096649d23d58bf392ac62f73919e831745e40f26" +checksum = "707907fe3c25f5424cce2cb7e1cbcafee6bdbe735ca90ef77c29e84591e5b9da" dependencies = [ "equivalent", - "hashbrown 0.14.3", + "hashbrown 0.15.1", ] [[package]] name = "indicatif" -version = "0.17.8" +version = "0.17.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "763a5a8f45087d6bcea4222e7b72c291a054edf80e4ef6efd2a4979878c7bea3" +checksum = "cbf675b85ed934d3c67b5c5469701eec7db22689d0a2139d856e0925fa28b281" dependencies = [ "console", - "instant", "number_prefix", "portable-atomic", - "unicode-width", + "unicode-width 0.2.0", + "web-time", ] [[package]] @@ -1470,24 +1463,30 @@ checksum = "b248f5224d1d606005e02c97f5aa4e88eeb230488bcc03bc9ca4d7991399f2b5" [[package]] name = "instant" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a5bbe824c507c5da5956355e86a746d82e0e1464f65d862cc5e71da70e94b2c" +checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" dependencies = [ "cfg-if", ] [[package]] name = "is-terminal" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b" +checksum = "261f68e344040fbd0edea105bef17c66edf46f984ddb1115b775ce31be948f4b" dependencies = [ - "hermit-abi", + "hermit-abi 0.4.0", "libc", "windows-sys 0.52.0", ] +[[package]] +name = "is_terminal_polyfill" +version = "1.70.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" + [[package]] name = "iter-chunks" version = "0.2.2" @@ -1505,48 +1504,48 @@ dependencies = [ [[package]] name = "itertools" -version = "0.11.0" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" dependencies = [ "either", ] [[package]] name = "itertools" -version = "0.12.1" +version = "0.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" +checksum = "413ee7dfc52ee1a4949ceeb7dbc8a33f2d6c088194d9f922fb8318faf1f01186" dependencies = [ "either", ] [[package]] name = "itoa" -version = "1.0.11" +version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49f1f14873335454500d59611f1cf4a4b0f786f9ac11f4312a78e4cf2566695b" +checksum = "540654e97a3f4470a492cd30ff187bc95d89557a903a2bbf112e2fae98104ef2" [[package]] name = "js-sys" -version = "0.3.69" +version = "0.3.72" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "29c15563dc2726973df627357ce0c9ddddbea194836909d655df6a75d2cf296d" +checksum = "6a88f1bda2bd75b0452a14784937d796722fdebfe50df998aeb3f0b7603019a9" dependencies = [ "wasm-bindgen", ] [[package]] name = "lazy_static" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" [[package]] name = "libc" -version = "0.2.153" +version = "0.2.164" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c198f91728a82281a64e1f4f9eeb25d82cb32a5de251c6bd1b5154d63a8e7bd" +checksum = "433bfe06b8c75da9b2e3fbea6e5329ff87748f0b144ef75306e674c3f6f7c13f" [[package]] name = "libredox" @@ -1554,26 +1553,15 @@ version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "libc", ] [[package]] name = "libtest-mimic" -version = "0.6.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d8de370f98a6cb8a4606618e53e802f93b094ddec0f96988eaec2c27e6e9ce7" -dependencies = [ - "clap", - "termcolor", - "threadpool", -] - -[[package]] -name = "libtest-mimic" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fefdf21230d6143476a28adbee3d930e2b68a3d56443c777cae3fe9340eebff9" +checksum = "cc0bda45ed5b3a2904262c1bb91e526127aa70e7ef3758aba2ef93cf896b9b58" dependencies = [ "clap", "escape8259", @@ -1583,15 +1571,15 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.4.13" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01cda141df6706de531b6c46c3a33ecca755538219bd484262fa09410c13539c" +checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" [[package]] name = "lock_api" -version = "0.4.11" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" dependencies = [ "autocfg", "scopeguard", @@ -1599,9 +1587,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.21" +version = "0.4.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" [[package]] name = "matchers" @@ -1636,9 +1624,9 @@ checksum = "490cc448043f947bae3cbee9c203358d62dbee0db12107a74be5c30ccfd09771" [[package]] name = "memchr" -version = "2.7.2" +version = "2.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8640c5d730cb13ebd907d8d04b52f55ac9a2eec55b440c8892f40d56c76c1d" +checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" [[package]] name = "memoffset" @@ -1663,11 +1651,10 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "minitrace" -version = "0.6.4" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2df1d765f7ec35138abeefde2a023c3b26b8d9bb2e4a3b98ed132acf2d755a7" +checksum = "197d538cd69839d49a593c8c72df44291b0ea3296ecc0c85529002c53c8fbc6f" dependencies = [ - "futures", "minitrace-macro", "minstant", "once_cell", @@ -1679,9 +1666,9 @@ dependencies = [ [[package]] name = "minitrace-macro" -version = "0.6.4" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36aca96c5da5b6a8c7f75910fb52c8d5aecb70f27d821adeae06ba54d2cf74b0" +checksum = "14efd4b574325fcb981bce1ac700b9ccf071ec2eb94f7a6a6b583a84f228ba47" dependencies = [ "proc-macro-error", "proc-macro2", @@ -1691,11 +1678,11 @@ dependencies = [ [[package]] name = "miniz_oxide" -version = "0.7.2" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d811f3e15f28568be3407c8e7fdb6514c1cda3cb30683f15b6a1a1dc4ea14a7" +checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" dependencies = [ - "adler", + "adler2", ] [[package]] @@ -1710,27 +1697,28 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.11" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a650543ca06a924e8b371db273b2756685faae30f8487da1b56505a8f78b0c" +checksum = "80e04d1dcff3aae0704555fe5fee3bcfaf3d1fdf8a7e521d5b9d2b42acb52cec" dependencies = [ + "hermit-abi 0.3.9", "libc", "wasi", - "windows-sys 0.48.0", + "windows-sys 0.52.0", ] [[package]] name = "moka" -version = "0.12.7" +version = "0.12.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e0d88686dc561d743b40de8269b26eaf0dc58781bde087b0984646602021d08" +checksum = "32cf62eb4dd975d2dde76432fb1075c49e3ee2331cf36f1f8fd4b66550d32b6f" dependencies = [ "async-lock", "async-trait", "crossbeam-channel", "crossbeam-epoch", "crossbeam-utils", - "event-listener 5.3.0", + "event-listener", "futures-util", "once_cell", "parking_lot", @@ -1764,9 +1752,9 @@ version = "0.28.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab2156c4fce2f8df6c499cc1c763e4394b7482525bf2a9701c9d79d215f519e4" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "cfg-if", - "cfg_aliases", + "cfg_aliases 0.1.1", "libc", ] @@ -1790,37 +1778,17 @@ dependencies = [ "winapi", ] -[[package]] -name = "num-bigint" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "608e7659b5c3d7cba262d894801b9ec9d00de989e8a82bd4bef91d08da45cdc0" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - [[package]] name = "num-conv" version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "51d515d32fb182ee37cda2ccdcb92950d6a3c2893aa280e540671c2cd0f3b1d9" -[[package]] -name = "num-integer" -version = "0.1.46" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" -dependencies = [ - "num-traits", -] - [[package]] name = "num-traits" -version = "0.2.18" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da0df0e5185db44f69b44f26786fe401b6c293d1907744beaa7fa62b2e5a517a" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" dependencies = [ "autocfg", ] @@ -1831,7 +1799,7 @@ version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi", + "hermit-abi 0.3.9", "libc", ] @@ -1843,24 +1811,24 @@ checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" [[package]] name = "object" -version = "0.32.2" +version = "0.36.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" dependencies = [ "memchr", ] [[package]] name = "once_cell" -version = "1.19.0" +version = "1.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fdb12b2476b595f9358c5161aa467c2438859caa136dec86c26fdd2efe17b92" +checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" [[package]] name = "oorandom" -version = "11.1.3" +version = "11.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" +checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9" [[package]] name = "option-ext" @@ -1870,9 +1838,9 @@ checksum = "04744f49eae99ab78e0d5c0b603ab218f515ea8cfe5a456d7629ad883a3b6e7d" [[package]] name = "ordered-float" -version = "4.2.0" +version = "4.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a76df7075c7d4d01fdcb46c912dd17fba5b60c78ea480b475f2b6ab6f666584e" +checksum = "c65ee1f9701bf938026630b455d5315f490640234259037edb259798b3bcf85e" dependencies = [ "num-traits", "rand", @@ -1887,21 +1855,21 @@ checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" [[package]] name = "owo-colors" -version = "3.5.0" +version = "4.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1b04fb49957986fdce4d6ee7a65027d55d4b6d2265e5848bbb507b58ccfdb6f" +checksum = "fb37767f6569cd834a413442455e0f066d0d522de8630436e2a1761d9726ba56" [[package]] name = "parking" -version = "2.2.0" +version = "2.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb813b8af86854136c6922af0598d719255ecb2179515e6e7730d468f05c9cae" +checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" -version = "0.12.1" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3742b2c103b9f06bc9fff0a37ff4912935851bee6d36f3c02bcc755bcfec228f" +checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" dependencies = [ "lock_api", "parking_lot_core", @@ -1909,47 +1877,47 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.9" +version = "0.9.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" +checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" dependencies = [ "cfg-if", "libc", "redox_syscall", "smallvec", - "windows-targets 0.48.5", + "windows-targets 0.52.6", ] [[package]] name = "parse-display" -version = "0.9.0" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06af5f9333eb47bd9ba8462d612e37a8328a5cb80b13f0af4de4c3b89f52dee5" +checksum = "914a1c2265c98e2446911282c6ac86d8524f495792c38c5bd884f80499c7538a" dependencies = [ "parse-display-derive", "regex", - "regex-syntax 0.8.3", + "regex-syntax 0.8.5", ] [[package]] name = "parse-display-derive" -version = "0.9.0" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc9252f259500ee570c75adcc4e317fa6f57a1e47747d622e0bf838002a7b790" +checksum = "2ae7800a4c974efd12df917266338e79a7a74415173caf7e70aa0a0707345281" dependencies = [ "proc-macro2", "quote", "regex", - "regex-syntax 0.8.3", + "regex-syntax 0.8.5", "structmeta", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "paste" -version = "1.0.14" +version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de3145af08024dea9fa9914f381a17b8fc6034dfb00f3a84013f7ff43f29ed4c" +checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" [[package]] name = "pem" @@ -1957,7 +1925,7 @@ version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e459365e590736a54c3fa561947c84837534b8e9af6fc5bf781307e82658fae" dependencies = [ - "base64 0.22.0", + "base64 0.22.1", "serde", ] @@ -1969,22 +1937,22 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "petgraph" -version = "0.6.4" +version = "0.6.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1d3afd2628e69da2be385eb6f2fd57c8ac7977ceeff6dc166ff1657b0e386a9" +checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset", - "indexmap 2.2.6", + "indexmap 2.6.0", ] [[package]] name = "pgwire" -version = "0.20.0" +version = "0.20.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c00492c52bb65e0421211b7f4c5d9de7586e53786a3b244efb00f74851206bf6" +checksum = "41cb1fce3b46971c5572964b2c0e525c5877584fc9ef6c697f0ea87a56e136d8" dependencies = [ "async-trait", - "base64 0.22.0", + "base64 0.22.1", "bytes", "chrono", "derive-new", @@ -2006,29 +1974,29 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.5" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bf43b791c5b9e34c3d182969b4abb522f9343702850a2e57f460d00d09b4b3" +checksum = "be57f64e946e500c8ee36ef6331845d40a93055567ec57e8fae13efd33759b95" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.5" +version = "1.1.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" +checksum = "3c0f5fad0874fc7abcd4d750e76917eaebbecaa2c20bde22e1dbeeba8beb758c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "pin-project-lite" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bda66fc9667c18cb2758a2ac84d1167245054bcf85d5d1aaa6923f45801bdd02" +checksum = "915a1e146535de9163f3987b8944ed8cf49a18bb0056bcebcdcece385cece4ff" [[package]] name = "pin-utils" @@ -2038,9 +2006,9 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "plotters" -version = "0.3.5" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2c224ba00d7cadd4d5c660deaf2098e5e80e07846537c51f9cfa4be50c1fd45" +checksum = "5aeb6f403d7a4911efb1e33402027fc44f29b5bf6def3effcc22d7bb75f2b747" dependencies = [ "num-traits", "plotters-backend", @@ -2051,32 +2019,32 @@ dependencies = [ [[package]] name = "plotters-backend" -version = "0.3.5" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e76628b4d3a7581389a35d5b6e2139607ad7c75b17aed325f210aa91f4a9609" +checksum = "df42e13c12958a16b3f7f4386b9ab1f3e7933914ecea48da7139435263a4172a" [[package]] name = "plotters-svg" -version = "0.3.5" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38f6d39893cca0701371e3c27294f09797214b86f1fb951b89ade8ec04e2abab" +checksum = "51bae2ac328883f7acdfea3d66a7c35751187f870bc81f94563733a154d7a670" dependencies = [ "plotters-backend", ] [[package]] name = "portable-atomic" -version = "1.6.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7170ef9988bc169ba16dd36a7fa041e5c4cbeb6a35b76d4c03daded371eae7c0" +checksum = "cc9c68a3f6da06753e9335d63e27f6b9754dd1920d941135b7ea8224f141adb2" [[package]] name = "postgres-protocol" -version = "0.6.6" +version = "0.6.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49b6c5ef183cd3ab4ba005f1ca64c21e8bd97ce4699cfea9e8d9a2c4958ca520" +checksum = "acda0ebdebc28befa84bee35e651e4c5f09073d668c7aed4cf7e23c3cda84b23" dependencies = [ - "base64 0.21.7", + "base64 0.22.1", "byteorder", "bytes", "fallible-iterator", @@ -2090,9 +2058,9 @@ dependencies = [ [[package]] name = "postgres-types" -version = "0.2.6" +version = "0.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d2234cdee9408b523530a9b6d2d6b373d1db34f6a8e51dc03ded1828d7fb67c" +checksum = "f66ea23a2d0e5734297357705193335e0a957696f34bed2f2faefacb2fec336f" dependencies = [ "array-init", "bytes", @@ -2109,9 +2077,12 @@ checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" [[package]] name = "ppv-lite86" -version = "0.2.17" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b40af805b3121feab8a3c29f04d8ad262fa8e0561883e7653e024ae4479e6de" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +dependencies = [ + "zerocopy", +] [[package]] name = "pretty-xmlish" @@ -2121,19 +2092,19 @@ checksum = "96b8aab53732b7a9c5c39bb0e130f85671b48b188ef258c3b9f7f5da1877382a" [[package]] name = "prettyplease" -version = "0.2.19" +version = "0.2.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ac2cf0f2e4f42b49f5ffd07dae8d746508ef7526c13940e5f524012ae6c6550" +checksum = "64d1ec885c64d0457d564db4ec299b2dae3f9c02808b8ad9c3a089c591b18033" dependencies = [ "proc-macro2", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "proc-macro-crate" -version = "3.1.0" +version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d37c51ca738a55da99dc0c4a34860fd675453b8b36209178c2249bb13651284" +checksum = "8ecf48c7ca261d60b74ab1a7b20da18bede46776b2e55535cb958eb595c5fa7b" dependencies = [ "toml_edit", ] @@ -2164,18 +2135,18 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.81" +version = "1.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d1597b0c024618f09a9c3b8655b7e430397a36d23fdafec26d6965e9eec3eba" +checksum = "37d3544b3f2748c54e147655edb5025752e2303145b5aefb3c3ea2c78b973bb0" dependencies = [ "unicode-ident", ] [[package]] name = "prost" -version = "0.12.4" +version = "0.12.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0f5d036824e4761737860779c906171497f6d55681139d8312388f8fe398922" +checksum = "deb1435c188b76130da55f17a466d252ff7b1418b2ad3e037d127b94e3411f29" dependencies = [ "bytes", "prost-derive", @@ -2183,9 +2154,9 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.12.4" +version = "0.12.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80b776a1b2dc779f5ee0641f8ade0125bc1298dd41a9a0c16d8bd57b42d222b1" +checksum = "22505a5c94da8e3b7c2996394d1c933236c4d743e81a410bcca4e6989fc066a4" dependencies = [ "bytes", "heck 0.5.0", @@ -2198,28 +2169,28 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.60", + "syn 2.0.89", "tempfile", ] [[package]] name = "prost-derive" -version = "0.12.4" +version = "0.12.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19de2de2a00075bf566bee3bd4db014b11587e84184d3f7a791bc17f1a8e9e48" +checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" dependencies = [ "anyhow", "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "prost-types" -version = "0.12.4" +version = "0.12.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3235c33eb02c1f1e212abdbe34c78b264b038fb58ca612664343271e36e55ffe" +checksum = "9091c90b0a32608e984ff2fa4091273cbdd755d54935c51d520887f4a1dbd5b0" dependencies = [ "prost", ] @@ -2291,7 +2262,7 @@ dependencies = [ "proc-macro2", "pyo3-macros-backend", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -2304,7 +2275,7 @@ dependencies = [ "proc-macro2", "pyo3-build-config", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -2324,9 +2295,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.36" +version = "1.0.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" +checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" dependencies = [ "proc-macro2", ] @@ -2381,11 +2352,11 @@ dependencies = [ [[package]] name = "raw-cpuid" -version = "11.0.1" +version = "11.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d86a7c4638d42c44551f4791a20e687dbb4c3de1f33c43dd71e355cd429def1" +checksum = "1ab240315c661615f2ee9f0f2cd32d5a7343a84d5ebcccb99d46e6637565e7b0" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", ] [[package]] @@ -2410,18 +2381,18 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.4.1" +version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" +checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" dependencies = [ - "bitflags 1.3.2", + "bitflags 2.6.0", ] [[package]] name = "redox_users" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd283d9651eeda4b2a83a43c1c91b266c40fd76ecd39a50a8c630ae69dc72891" +checksum = "ba009ff324d1fc1b900bd1fdb31564febe58a8ccc8a6fdbb93b543d33b13ca43" dependencies = [ "getrandom", "libredox", @@ -2430,34 +2401,34 @@ dependencies = [ [[package]] name = "ref-cast" -version = "1.0.22" +version = "1.0.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4846d4c50d1721b1a3bef8af76924eef20d5e723647333798c1b519b3a9473f" +checksum = "ccf0a6f84d5f1d581da8b41b47ec8600871962f2a528115b542b362d4b744931" dependencies = [ "ref-cast-impl", ] [[package]] name = "ref-cast-impl" -version = "1.0.22" +version = "1.0.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fddb4f8d99b0a2ebafc65a87a69a7b9875e4b1ae1f00db265d300ef7f28bccc" +checksum = "bcc303e793d3734489387d205e9b186fac9c6cfacedd98cbb2e8a5943595f3e6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "regex" -version = "1.10.4" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c117dbdfde9c8308975b6a18d71f3f385c89461f7b3fb054288ecf2a2058ba4c" +checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" dependencies = [ "aho-corasick", "memchr", - "regex-automata 0.4.6", - "regex-syntax 0.8.3", + "regex-automata 0.4.9", + "regex-syntax 0.8.5", ] [[package]] @@ -2471,13 +2442,13 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.6" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b83b8b9847f9bf95ef68afb0b8e6cdb80f498442f5179a29fad448fcc1eaea" +checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" dependencies = [ "aho-corasick", "memchr", - "regex-syntax 0.8.3", + "regex-syntax 0.8.5", ] [[package]] @@ -2488,9 +2459,9 @@ checksum = "f162c6dd7b008981e4d40210aca20b4bd0f9b60ca9271061b07f78537722f2e1" [[package]] name = "regex-syntax" -version = "0.8.3" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adad44e29e4c806119491a7f06f03de4d1af22c3a680dd47f1e6e179439d1f56" +checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" [[package]] name = "rend" @@ -2551,7 +2522,7 @@ dependencies = [ "indoc", "iter-chunks", "itertools 0.12.1", - "libtest-mimic 0.7.2", + "libtest-mimic", "minitrace", "moka", "num-traits", @@ -2596,9 +2567,9 @@ dependencies = [ [[package]] name = "rkyv" -version = "0.7.44" +version = "0.7.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5cba464629b3394fc4dbc6f940ff8f5b4ff5c7aef40f29166fd4ad12acbc99c0" +checksum = "9008cd6385b9e161d8229e1f6549dd23c3d022f132a2ea37ac3a10ac4935779b" dependencies = [ "bitvec", "bytecheck", @@ -2614,9 +2585,9 @@ dependencies = [ [[package]] name = "rkyv_derive" -version = "0.7.44" +version = "0.7.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7dddfff8de25e6f62b9d64e6e432bf1c6736c57d20323e15ee10435fbda7c65" +checksum = "503d1d27590a2b0a3a4ca4c94755aa2875657196ecbf401a42eff41d7de532c0" dependencies = [ "proc-macro2", "quote", @@ -2634,9 +2605,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.35.0" +version = "1.36.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1790d1c4c0ca81211399e0e0af16333276f375209e71a37b67698a373db5b47a" +checksum = "b082d80e3e3cc52b2ed634388d436fe1f4de6af5786cc2de9ba9737527bdf555" dependencies = [ "arrayvec", "borsh", @@ -2650,26 +2621,26 @@ dependencies = [ [[package]] name = "rustc-demangle" -version = "0.1.23" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d626bb9dae77e28219937af045c257c28bfd3f69333c512553507f5f9798cb76" +checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" [[package]] name = "rustc_version" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" dependencies = [ "semver", ] [[package]] name = "rustix" -version = "0.38.32" +version = "0.38.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65e04861e65f21776e67888bfbea442b3642beaa0138fdb1dd7a84a52dffdb89" +checksum = "d7f649912bc1495e167a6edee79151c84b1bad49748cb4f1f1167f459f6224f6" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "errno", "libc", "linux-raw-sys", @@ -2678,9 +2649,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.22.3" +version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99008d7ad0bbbea527ec27bddbc0e432c5b87d8175178cee68d2eec9c4a1813c" +checksum = "bf4ef73721ac7bcd79b2b315da7779d8fc09718c6b3d2d1b2d94850eb8c18432" dependencies = [ "log", "ring", @@ -2692,15 +2663,15 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.4.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd36cc4259e3e4514335c4a138c6b43171a8d61d8f5c9348f9fc7529416f247" +checksum = "16f1201b3c9a7ee8039bcadc17b7e605e2945b27eee7631788c1bd2b0643674b" [[package]] name = "rustls-webpki" -version = "0.102.2" +version = "0.102.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faaa0a62740bedb9b2ef5afa303da42764c012f743917351dc9a237ea1663610" +checksum = "64ca1bc8749bd4cf37b5ce386cc146580777b4e8572c7b97baf22c83f444bee9" dependencies = [ "ring", "rustls-pki-types", @@ -2709,9 +2680,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.15" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80af6f9131f277a45a3fba6ce8e2258037bb0477a67e610d3c1fe046ab31de47" +checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" [[package]] name = "rustyline" @@ -2719,7 +2690,7 @@ version = "14.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7803e8936da37efd9b6d4478277f4b2b9bb5cdb37a113e8d63222e58da647e63" dependencies = [ - "bitflags 2.5.0", + "bitflags 2.6.0", "cfg-if", "clipboard-win", "fd-lock", @@ -2730,16 +2701,16 @@ dependencies = [ "nix", "radix_trie", "unicode-segmentation", - "unicode-width", + "unicode-width 0.1.14", "utf8parse", "windows-sys 0.52.0", ] [[package]] name = "ryu" -version = "1.0.17" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e86697c916019a8588c99b5fac3cead74ec0b4b819707a682fd4d23fa0ce1ba1" +checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" [[package]] name = "same-file" @@ -2764,37 +2735,38 @@ checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" [[package]] name = "semver" -version = "1.0.22" +version = "1.0.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92d43fe69e652f3df9bdc2b85b2854a0825b86e4fb76bc44d945137d053639ca" +checksum = "61697e0a1c7e512e84a621326239844a24d8207b4669b41bc18b32ea5cbf988b" [[package]] name = "serde" -version = "1.0.198" +version = "1.0.215" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9846a40c979031340571da2545a4e5b7c4163bdae79b301d5f86d03979451fcc" +checksum = "6513c1ad0b11a9376da888e3e0baa0077f1aed55c17f50e7b2397136129fb88f" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.198" +version = "1.0.215" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88edab869b01783ba905e7d0153f9fc1a6505a96e4ad3018011eedb838566d9" +checksum = "ad1e866f866923f252f05c889987993144fb74e722403468a4ebd70c3cd756c0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "serde_json" -version = "1.0.116" +version = "1.0.133" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e17db7126d17feb94eb3fad46bf1a96b034e8aacbc2e775fe81505f8b0b2813" +checksum = "c7fceb2473b9166b2294ef05efcb65a3db80803f0b03ef86a5fc88a2b85ee377" dependencies = [ "itoa", + "memchr", "ryu", "serde", ] @@ -2805,7 +2777,7 @@ version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 2.2.6", + "indexmap 2.6.0", "itoa", "ryu", "serde", @@ -2832,11 +2804,17 @@ dependencies = [ "lazy_static", ] +[[package]] +name = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + [[package]] name = "signal-hook-registry" -version = "1.4.1" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8229b473baa5980ac72ef434c4415e70c4b5e71b423043adb4ba059f89c99a1" +checksum = "a9e9e0b4211b72e7b8b6e85c807d36c212bdb33ea8587f7569562a84df5465b1" dependencies = [ "libc", ] @@ -2852,15 +2830,15 @@ dependencies = [ [[package]] name = "simdutf8" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f27f6278552951f1f2b8cf9da965d10969b2efdea95a6ec47987ab46edfe263a" +checksum = "e3a9fe34e3e7a50316060351f37187a3f546bce95496156754b601a5fa71b76e" [[package]] name = "similar" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa42c91313f1d05da9b26f267f931cf178d4aba455b4c4622dd7355eb80c6640" +checksum = "1de1d4f81173b03af4c0cbed3c898f6bff5b870e4a7f5d6f4057d62a7a4b686e" [[package]] name = "slab" @@ -2882,9 +2860,9 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.6" +version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05ffd9c0a93b7543e062e759284fcf5f5e3b098501104bfbdde4d404db792871" +checksum = "ce305eb0b4296696835b71df73eb912e0f1ffd2556a501fcede6e0c50349191c" dependencies = [ "libc", "windows-sys 0.52.0", @@ -2908,9 +2886,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.20.1" +version = "0.20.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e7c6a33098cd55e4fead1bd1f85c1d2064f02bafdb9fe004ca39fd94aee36e6" +checksum = "ff40a700928567c7303adc6f86cd17bfade5fa8c7dce1256f4691eca66e3ca42" dependencies = [ "async-trait", "educe", @@ -2918,8 +2896,8 @@ dependencies = [ "futures", "glob", "humantime", - "itertools 0.11.0", - "libtest-mimic 0.6.1", + "itertools 0.13.0", + "libtest-mimic", "md-5", "owo-colors", "regex", @@ -2951,7 +2929,7 @@ dependencies = [ "console", "futures-util", "glob", - "libtest-mimic 0.7.2", + "libtest-mimic", "serde", "serde_yaml", "similar", @@ -2960,13 +2938,13 @@ dependencies = [ [[package]] name = "stringprep" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb41d74e231a107a1b4ee36bd1214b11285b77768d2e3824aedafa988fd36ee6" +checksum = "7b4df3d392d81bd458a8a621b8bffbd2302a12ffe288a9d931670948749463b1" dependencies = [ - "finl_unicode", "unicode-bidi", "unicode-normalization", + "unicode-properties", ] [[package]] @@ -2984,7 +2962,7 @@ dependencies = [ "proc-macro2", "quote", "structmeta-derive", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -2995,43 +2973,43 @@ checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "strum" -version = "0.26.2" +version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" [[package]] name = "strum_macros" -version = "0.26.2" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6cf59daf282c0a494ba14fd21610a0325f9f90ec9d1231dea26bcb1d696c946" +checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" dependencies = [ - "heck 0.4.1", + "heck 0.5.0", "proc-macro2", "quote", "rustversion", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "subst" -version = "0.3.0" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca1318e5d6716d6541696727c88d9b8dfc8cfe6afd6908e186546fd4af7f5b98" +checksum = "7a3c1ba4fd019bc866333a61fe205fc9b686e3cf5971dd8dfc116657d933031c" dependencies = [ "memchr", - "unicode-width", + "unicode-width 0.1.14", ] [[package]] name = "subtle" -version = "2.5.0" +version = "2.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbol_table" @@ -3062,27 +3040,15 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.60" +version = "2.0.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "909518bc7b1c9b779f1bbf07f2929d35af9f0f37e47c6e9ef7f9dddc1e1821f3" +checksum = "44d46482f1c1c87acd84dea20c1bf5ebff4c757009ed6bf19cfd36fb10e92c4e" dependencies = [ "proc-macro2", "quote", "unicode-ident", ] -[[package]] -name = "syn_derive" -version = "0.1.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1329189c02ff984e9736652b1631330da25eaa6bc639089ed4915d25446cbe7b" -dependencies = [ - "proc-macro-error", - "proc-macro2", - "quote", - "syn 2.0.60", -] - [[package]] name = "sync_wrapper" version = "0.1.2" @@ -3103,20 +3069,21 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "target-lexicon" -version = "0.12.14" +version = "0.12.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1fc403891a21bcfb7c37834ba66a547a8f402146eba7265b5a6d88059c9ff2f" +checksum = "61c41af27dd6d1e27b1b16b489db798443478cef1f06a660c96db617ba5de3b1" [[package]] name = "tempfile" -version = "3.10.1" +version = "3.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85b77fafb263dd9d05cbeac119526425676db3784113aa9295c88498cbf8bff1" +checksum = "28cce251fcbc87fac86a866eeb0d6c2d536fc16d06f184bb61aeae11aa4cee0c" dependencies = [ "cfg-if", "fastrand", + "once_cell", "rustix", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -3146,7 +3113,7 @@ dependencies = [ "cfg-if", "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -3157,28 +3124,28 @@ checksum = "5c89e72a01ed4c579669add59014b9a524d609c0c88c6a585ce37485879f6ffb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", "test-case-core", ] [[package]] name = "thiserror" -version = "1.0.58" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03468839009160513471e86a034bb2c5c0e4baae3b43f79ffc55c4a5427b3297" +checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.58" +version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c61f3ba182994efc43764a46c018c347bc492c79f024e705f46567b418f6d4f7" +checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -3251,9 +3218,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.6.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87cc5ceb3875bb20c2890005a4e226a4651264a5c75edb2421b52861a0a0cb50" +checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" dependencies = [ "tinyvec_macros", ] @@ -3266,22 +3233,21 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.37.0" +version = "1.41.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1adbebffeca75fcfd058afa480fb6c0b81e165a0323f9c9d39c9697e37c46787" +checksum = "22cfb5bee7a6a52939ca9224d6ac897bb669134078daa8735560897f69de4d33" dependencies = [ "backtrace", "bytes", "libc", "mio", - "num_cpus", "parking_lot", "pin-project-lite", "signal-hook-registry", "socket2", "tokio-macros", "tracing", - "windows-sys 0.48.0", + "windows-sys 0.52.0", ] [[package]] @@ -3296,13 +3262,13 @@ dependencies = [ [[package]] name = "tokio-macros" -version = "2.2.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" +checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -3318,9 +3284,9 @@ dependencies = [ [[package]] name = "tokio-stream" -version = "0.1.15" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" +checksum = "4f4e6ce100d0eb49a2734f8c0812bcd324cf357d21810932c5df6b96ef2b86f1" dependencies = [ "futures-core", "pin-project-lite", @@ -3329,31 +3295,30 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.10" +version = "0.7.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" +checksum = "61e7c3654c13bcd040d4a03abee2c75b1d14a37b423cf5a813ceae1cc903ec6a" dependencies = [ "bytes", "futures-core", "futures-sink", "pin-project-lite", "tokio", - "tracing", ] [[package]] name = "toml_datetime" -version = "0.6.5" +version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3550f4e9685620ac18a50ed434eb3aec30db8ba93b0287467bca5826ea25baf1" +checksum = "0dd7358ecb8fc2f8d014bf86f6f638ce72ba252a2c3a2572f2a795f1d23efb41" [[package]] name = "toml_edit" -version = "0.21.1" +version = "0.22.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a8534fd7f78b5405e860340ad6575217ce99f38d4d5c8f2442cb5ecb50090e1" +checksum = "4ae48d6208a266e853d946088ed816055e556cc6028c5e8e2b84d9fa5dd7c7f5" dependencies = [ - "indexmap 2.2.6", + "indexmap 2.6.0", "toml_datetime", "winnow", ] @@ -3407,15 +3372,15 @@ dependencies = [ [[package]] name = "tower-layer" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c20c8dbed6283a09604c3e69b4b7eeb54e298b8a600d4d5ecb5ad39de609f1d0" +checksum = "121c2a6cda46980bb0fcd1647ffaf6cd3fc79a013de288782836f6df9c48780e" [[package]] name = "tower-service" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" +checksum = "8df9b6e13f2d32c91b9bd719c00d1958837bc7dec474d94952798cc8e69eeec3" [[package]] name = "tracing" @@ -3436,7 +3401,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] @@ -3491,6 +3456,12 @@ version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" +[[package]] +name = "typeid" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e13db2e0ccd5e14a544e8a246ba2312cd25223f616442d7f2cb0e3db614236e" + [[package]] name = "typenum" version = "1.17.0" @@ -3499,36 +3470,48 @@ checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "unicode-bidi" -version = "0.3.15" +version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08f95100a766bf4f8f28f90d77e0a5461bbdb219042e7679bebe79004fed8d75" +checksum = "5ab17db44d7388991a428b2ee655ce0c212e862eff1768a455c58f9aad6e7893" [[package]] name = "unicode-ident" -version = "1.0.12" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3354b9ac3fae1ff6755cb6db53683adb661634f67557942dea4facebec0fee4b" +checksum = "adb9e6ca4f869e1180728b7950e35922a7fc6397f7b641499e8f3ef06e50dc83" [[package]] name = "unicode-normalization" -version = "0.1.23" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a56d1686db2308d901306f92a263857ef59ea39678a5458e7cb17f01415101f5" +checksum = "5033c97c4262335cded6d6fc3e5c18ab755e1a3dc96376350f3d8e9f009ad956" dependencies = [ "tinyvec", ] +[[package]] +name = "unicode-properties" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e70f2a8b45122e719eb623c01822704c4e0907e7e426a05927e1a1cfff5b75d0" + [[package]] name = "unicode-segmentation" -version = "1.11.0" +version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4c87d22b6e3f4a18d4d40ef354e97c90fcb14dd91d7dc0aa9d8a1172ebf7202" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" [[package]] name = "unicode-width" -version = "0.1.11" +version = "0.1.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" + +[[package]] +name = "unicode-width" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51733f11c9c4f72aa0c160008246859e340b00807569a0da0e7a1079b27ba85" +checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" [[package]] name = "unindent" @@ -3550,15 +3533,15 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "utf8parse" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.8.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a183cf7feeba97b4dd1c0d46788634f6221d87fa961b305bed08c851829efcc0" +checksum = "f8c5f0a0af699448548ad1a2fbf920fb4bee257eae39953ba95cb84891a0446a" dependencies = [ "getrandom", ] @@ -3571,9 +3554,9 @@ checksum = "830b7e5d4d90034032940e4ace0d9a9a057e7a45cd94e6c007832e39edb82f6d" [[package]] name = "version_check" -version = "0.9.4" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49874b5167b65d7193b8aba1567f5c7d93d001cafc34600cee003eda787e483f" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" [[package]] name = "walkdir" @@ -3602,34 +3585,35 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.92" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4be2531df63900aeb2bca0daaaddec08491ee64ceecbee5076636a3b026795a8" +checksum = "128d1e363af62632b8eb57219c8fd7877144af57558fb2ef0368d0087bddeb2e" dependencies = [ "cfg-if", + "once_cell", "wasm-bindgen-macro", ] [[package]] name = "wasm-bindgen-backend" -version = "0.2.92" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "614d787b966d3989fa7bb98a654e369c762374fd3213d212cfc0251257e747da" +checksum = "cb6dd4d3ca0ddffd1dd1c9c04f94b868c37ff5fac97c30b97cff2d74fce3a358" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-macro" -version = "0.2.92" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1f8823de937b71b9460c0c34e25f3da88250760bec0ebac694b49997550d726" +checksum = "e79384be7f8f5a9dd5d7167216f022090cf1f9ec128e6e6a482a2cb5c5422c56" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3637,28 +3621,28 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.92" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" +checksum = "26c6ab57572f7a24a4985830b120de1594465e5d500f24afe89e16b4e833ef68" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.92" +version = "0.2.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af190c94f2773fdb3729c55b007a722abb5384da03bc0986df4c289bf5567e96" +checksum = "65fc09f10666a9f147042251e0dda9c18f166ff7de300607007e96bdebc1068d" [[package]] name = "web-sys" -version = "0.3.69" +version = "0.3.72" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77afa9a11836342370f4817622a2f0f418b134426d91a82dfb48f532d2ec13ef" +checksum = "f6488b90108c040df0fe62fa815cbdee25124641df01814dd7282749234c6112" dependencies = [ "js-sys", "wasm-bindgen", @@ -3692,11 +3676,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.6" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f29e6f9198ba0d26b4c9f07dbe6f9ed633e1f3d5b8b414090084349e46a52596" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "winapi", + "windows-sys 0.59.0", ] [[package]] @@ -3711,7 +3695,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets 0.52.5", + "windows-targets 0.52.6", ] [[package]] @@ -3729,7 +3713,16 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets 0.52.5", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", ] [[package]] @@ -3749,18 +3742,18 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f0713a46559409d202e70e28227288446bf7841d3211583a4b53e3f6d96e7eb" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" dependencies = [ - "windows_aarch64_gnullvm 0.52.5", - "windows_aarch64_msvc 0.52.5", - "windows_i686_gnu 0.52.5", + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", "windows_i686_gnullvm", - "windows_i686_msvc 0.52.5", - "windows_x86_64_gnu 0.52.5", - "windows_x86_64_gnullvm 0.52.5", - "windows_x86_64_msvc 0.52.5", + "windows_i686_msvc 0.52.6", + "windows_x86_64_gnu 0.52.6", + "windows_x86_64_gnullvm 0.52.6", + "windows_x86_64_msvc 0.52.6", ] [[package]] @@ -3771,9 +3764,9 @@ checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" [[package]] name = "windows_aarch64_gnullvm" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7088eed71e8b8dda258ecc8bac5fb1153c5cffaf2578fc8ff5d61e23578d3263" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" [[package]] name = "windows_aarch64_msvc" @@ -3783,9 +3776,9 @@ checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" [[package]] name = "windows_aarch64_msvc" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9985fd1504e250c615ca5f281c3f7a6da76213ebd5ccc9561496568a2752afb6" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" [[package]] name = "windows_i686_gnu" @@ -3795,15 +3788,15 @@ checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" [[package]] name = "windows_i686_gnu" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ba073cf16d5372720ec942a8ccbf61626074c6d4dd2e745299726ce8b89670" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" [[package]] name = "windows_i686_gnullvm" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87f4261229030a858f36b459e748ae97545d6f1ec60e5e0d6a3d32e0dc232ee9" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" [[package]] name = "windows_i686_msvc" @@ -3813,9 +3806,9 @@ checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" [[package]] name = "windows_i686_msvc" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db3c2bf3d13d5b658be73463284eaf12830ac9a26a90c717b7f771dfe97487bf" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" [[package]] name = "windows_x86_64_gnu" @@ -3825,9 +3818,9 @@ checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" [[package]] name = "windows_x86_64_gnu" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e4246f76bdeff09eb48875a0fd3e2af6aada79d409d33011886d3e1581517d9" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" [[package]] name = "windows_x86_64_gnullvm" @@ -3837,9 +3830,9 @@ checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" [[package]] name = "windows_x86_64_gnullvm" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "852298e482cd67c356ddd9570386e2862b5673c85bd5f88df9ab6802b334c596" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" [[package]] name = "windows_x86_64_msvc" @@ -3849,15 +3842,15 @@ checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" [[package]] name = "windows_x86_64_msvc" -version = "0.52.5" +version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec47e5bfd1bff0eeaf6d8b485cc1074891a197ab4225d504cb7a1ab88b02bf0" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" -version = "0.5.40" +version = "0.6.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f593a95398737aeed53e489c785df13f3618e41dbcd6718c6addbf1395aa6876" +checksum = "36c1fec1a2bb5866f07c25f68c26e565c4c200aebb96d7e55710c19d3e8ac49b" dependencies = [ "memchr", ] @@ -3892,29 +3885,30 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.7.32" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74d4d3961e53fa4c9a25a8637fc2bfaf2595b3d3ae34875568a5cf64787716be" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" dependencies = [ + "byteorder", "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.32" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce1b18ccd8e73a9321186f97e46f9f04b778851177567b1975109d26a08d2a6" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] [[package]] name = "zeroize" -version = "1.7.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "525b4ec142c6b68a2d10f01f7bbf6755599ca3f81ea53b8431b7dd348f5fdb2d" +checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" dependencies = [ "zeroize_derive", ] @@ -3927,5 +3921,5 @@ checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" dependencies = [ "proc-macro2", "quote", - "syn 2.0.60", + "syn 2.0.89", ] diff --git a/rust-toolchain b/rust-toolchain index 97764b39..c3f22bff 100644 --- a/rust-toolchain +++ b/rust-toolchain @@ -1 +1 @@ -nightly-2024-02-04 +nightly-2024-11-22 diff --git a/src/lib.rs b/src/lib.rs index f0cd1925..a9ab380f 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -15,7 +15,6 @@ #![feature(proc_macro_hygiene)] #![feature(trusted_len)] #![feature(adt_const_params)] -#![feature(lazy_cell)] #![feature(iterator_try_collect)] #![feature(let_chains)] #![feature(impl_trait_in_assoc_type)] diff --git a/src/main.rs b/src/main.rs index a3374485..c9212d69 100644 --- a/src/main.rs +++ b/src/main.rs @@ -2,8 +2,6 @@ //! A simple interactive shell of the database. -#![feature(div_duration)] - use std::fs::File; use std::path::PathBuf; use std::sync::Arc; From be64142047e91677da74078fa7906546005d19b1 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 16:32:57 +0800 Subject: [PATCH 10/29] upgrade dependencies Signed-off-by: Runji Wang --- Cargo.lock | 335 +++++++++++++++++++++++++++-------------------- Cargo.toml | 26 ++-- proto/Cargo.toml | 4 +- 3 files changed, 211 insertions(+), 154 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 985fb101..039f4d89 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1,6 +1,6 @@ # This file is automatically @generated by Cargo. # It is not intended for manual editing. -version = 3 +version = 4 [[package]] name = "addr2line" @@ -205,6 +205,12 @@ dependencies = [ "syn 2.0.89", ] +[[package]] +name = "atomic-waker" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" + [[package]] name = "autocfg" version = "1.4.0" @@ -213,18 +219,17 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "axum" -version = "0.6.20" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" +checksum = "edca88bc138befd0323b20752846e6587272d3b03b0343c8ea28a6f819e6e71f" dependencies = [ "async-trait", "axum-core", - "bitflags 1.3.2", "bytes", "futures-util", "http", "http-body", - "hyper", + "http-body-util", "itoa", "matchit", "memchr", @@ -233,25 +238,28 @@ dependencies = [ "pin-project-lite", "rustversion", "serde", - "sync_wrapper", - "tower", + "sync_wrapper 1.0.2", + "tower 0.5.1", "tower-layer", "tower-service", ] [[package]] name = "axum-core" -version = "0.3.4" +version = "0.4.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "759fa577a247914fd3f7f76d62972792636412fbfd634cd452f6a385a74d2d2c" +checksum = "09f2bd6146b97ae3359fa0cc6d6b376d9539582c7b4220f041a33ec24c226199" dependencies = [ "async-trait", "bytes", "futures-util", "http", "http-body", + "http-body-util", "mime", + "pin-project-lite", "rustversion", + "sync_wrapper 1.0.2", "tower-layer", "tower-service", ] @@ -310,24 +318,18 @@ dependencies = [ [[package]] name = "bit-set" -version = "0.5.3" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0700ddab506f33b20a03b13996eccd309a48e5ff77d0d95926aa0210fb4e95f1" +checksum = "08807e080ed7f9d5433fa9b275196cfc35414f66a0c79d864dc51a0d825231a3" dependencies = [ "bit-vec", ] [[package]] name = "bit-vec" -version = "0.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "349f9b6a179ed607305526ca489b34ad0a41aed5f7980fa90eb03160b69598fb" - -[[package]] -name = "bitflags" -version = "1.3.2" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" +checksum = "5e764a1d40d510daf35e07be9eb06e75770908c27d411ee6c92109c9840eaaf7" [[package]] name = "bitflags" @@ -364,7 +366,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2506947f73ad44e344215ccd6403ac2ae18cd8e046e581a441bf8d199f257f03" dependencies = [ "borsh-derive", - "cfg_aliases 0.2.1", + "cfg_aliases", ] [[package]] @@ -456,12 +458,6 @@ version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" -[[package]] -name = "cfg_aliases" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd16c4719339c4530435d38e511904438d07cce7950afa3718a84ac36c10e89e" - [[package]] name = "cfg_aliases" version = "0.2.1" @@ -537,7 +533,7 @@ version = "4.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4ac6a0c7b1a9e9a5186361f67dfa1b88213572f427fb9ab038efb2bd8c582dab" dependencies = [ - "heck 0.5.0", + "heck", "proc-macro2", "quote", "syn 2.0.89", @@ -605,9 +601,9 @@ dependencies = [ [[package]] name = "console-api" -version = "0.6.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd326812b3fd01da5bb1af7d340d0d555fd3d4b641e7f1dfcf5962a902952787" +checksum = "8030735ecb0d128428b64cd379809817e620a40e5001c54465b99ec5feec2857" dependencies = [ "futures-core", "prost", @@ -618,9 +614,9 @@ dependencies = [ [[package]] name = "console-subscriber" -version = "0.2.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7481d4c57092cd1c19dd541b92bdce883de840df30aa5d03fd48a3935c01842e" +checksum = "6539aa9c6a4cd31f4b1c040f860a1eac9aa80e7df6b05d506a6e7179936d6a01" dependencies = [ "console-api", "crossbeam-channel", @@ -628,6 +624,8 @@ dependencies = [ "futures-task", "hdrhistogram", "humantime", + "hyper-util", + "prost", "prost-types", "serde", "serde_json", @@ -884,7 +882,7 @@ dependencies = [ "smallvec", "symbol_table", "symbolic_expressions", - "thiserror", + "thiserror 1.0.69", ] [[package]] @@ -1052,9 +1050,9 @@ checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" [[package]] name = "fs-err" -version = "2.11.0" +version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88a41f105fe1d5b6b34b2055e3dc59bb79b46b48b2040b9e6c7b4b5de097aa41" +checksum = "8bb60e7409f34ef959985bc9d9c5ee8f5db24ee46ed9775850548021710f807f" dependencies = [ "autocfg", ] @@ -1220,15 +1218,15 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.26" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81fe527a889e1532da5c525686d96d4c2e74cdd345badf8dfef9f6b39dd5f5e8" +checksum = "ccae279728d634d083c00f6099cb58f01cc99c145b84b8be2f6c74618d79922e" dependencies = [ + "atomic-waker", "bytes", "fnv", "futures-core", "futures-sink", - "futures-util", "http", "indexmap 2.6.0", "slab", @@ -1275,12 +1273,6 @@ dependencies = [ "num-traits", ] -[[package]] -name = "heck" -version = "0.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95505c38b4572b2d910cecb0281560f54b440a19336cbbcb27bf6ce6adc6f5a8" - [[package]] name = "heck" version = "0.5.0" @@ -1325,9 +1317,9 @@ dependencies = [ [[package]] name = "http" -version = "0.2.12" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "601cbb57e577e2f5ef5be8e7b83f0f63994f25aa94d673e54a92d5c516d101f1" +checksum = "21b9ddb458710bc376481b842f5da65cdf31522de232c1ca8146abce2a358258" dependencies = [ "bytes", "fnv", @@ -1336,12 +1328,24 @@ dependencies = [ [[package]] name = "http-body" -version = "0.4.6" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ceab25649e9960c0311ea418d17bee82c0dcec1bd053b5f9a66e265a693bed2" +checksum = "1efedce1fb8e6913f23e0c92de8e62cd5b772a67e7b3946df930a62566c93184" dependencies = [ "bytes", "http", +] + +[[package]] +name = "http-body-util" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "793429d76616a256bcb62c2a2ec2bed781c8307e797e2598c50010f2bee2544f" +dependencies = [ + "bytes", + "futures-util", + "http", + "http-body", "pin-project-lite", ] @@ -1365,13 +1369,12 @@ checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" -version = "0.14.31" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c08302e8fa335b151b788c775ff56e7a03ae64ff85c548ee820fecb70356e85" +checksum = "97818827ef4f364230e16705d4706e2897df2bb60617d6ca15d598025a3c481f" dependencies = [ "bytes", "futures-channel", - "futures-core", "futures-util", "h2", "http", @@ -1380,23 +1383,41 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2", + "smallvec", "tokio", - "tower-service", - "tracing", "want", ] [[package]] name = "hyper-timeout" -version = "0.4.1" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b90d566bffbce6a75bd8b09a05aa8c2cb1fabb6cb348f8840c9e4c90a0d83b0" +dependencies = [ + "hyper", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", +] + +[[package]] +name = "hyper-util" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbb958482e8c7be4bc3cf272a766a2b0bf1a6755e7a6ae777f017a31d11b13b1" +checksum = "df2dcfbe0677734ab2f3ffa7fa7bfd4706bfdc1ef393f2ee30184aed67e631b4" dependencies = [ + "bytes", + "futures-channel", + "futures-util", + "http", + "http-body", "hyper", "pin-project-lite", + "socket2", "tokio", - "tokio-io-timeout", + "tower-service", + "tracing", ] [[package]] @@ -1502,15 +1523,6 @@ dependencies = [ "either", ] -[[package]] -name = "itertools" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" -dependencies = [ - "either", -] - [[package]] name = "itertools" version = "0.13.0" @@ -1553,7 +1565,7 @@ version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ - "bitflags 2.6.0", + "bitflags", "libc", ] @@ -1569,6 +1581,18 @@ dependencies = [ "threadpool", ] +[[package]] +name = "libtest-mimic" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5297962ef19edda4ce33aaa484386e0a5b3d7f2f4e037cbeee00503ef6b29d33" +dependencies = [ + "anstream", + "anstyle", + "clap", + "escape8259", +] + [[package]] name = "linux-raw-sys" version = "0.4.14" @@ -1726,7 +1750,7 @@ dependencies = [ "rustc_version", "smallvec", "tagptr", - "thiserror", + "thiserror 1.0.69", "triomphe", "uuid", ] @@ -1748,13 +1772,13 @@ dependencies = [ [[package]] name = "nix" -version = "0.28.0" +version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab2156c4fce2f8df6c499cc1c763e4394b7482525bf2a9701c9d79d215f519e4" +checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ - "bitflags 2.6.0", + "bitflags", "cfg-if", - "cfg_aliases 0.1.1", + "cfg_aliases", "libc", ] @@ -1890,9 +1914,9 @@ dependencies = [ [[package]] name = "parse-display" -version = "0.9.1" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "914a1c2265c98e2446911282c6ac86d8524f495792c38c5bd884f80499c7538a" +checksum = "287d8d3ebdce117b8539f59411e4ed9ec226e0a4153c7f55495c6070d68e6f72" dependencies = [ "parse-display-derive", "regex", @@ -1901,9 +1925,9 @@ dependencies = [ [[package]] name = "parse-display-derive" -version = "0.9.1" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ae7800a4c974efd12df917266338e79a7a74415173caf7e70aa0a0707345281" +checksum = "7fc048687be30d79502dea2f623d052f3a074012c6eac41726b7ab17213616b1" dependencies = [ "proc-macro2", "quote", @@ -1964,7 +1988,7 @@ dependencies = [ "rand", "ring", "stringprep", - "thiserror", + "thiserror 1.0.69", "time", "tokio", "tokio-rustls", @@ -2144,9 +2168,9 @@ dependencies = [ [[package]] name = "prost" -version = "0.12.6" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "deb1435c188b76130da55f17a466d252ff7b1418b2ad3e037d127b94e3411f29" +checksum = "7b0487d90e047de87f984913713b85c601c05609aad5b0df4b4573fbf69aa13f" dependencies = [ "bytes", "prost-derive", @@ -2154,13 +2178,13 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.12.6" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22505a5c94da8e3b7c2996394d1c933236c4d743e81a410bcca4e6989fc066a4" +checksum = "0c1318b19085f08681016926435853bbf7858f9c082d0999b80550ff5d9abe15" dependencies = [ "bytes", - "heck 0.5.0", - "itertools 0.12.1", + "heck", + "itertools 0.13.0", "log", "multimap", "once_cell", @@ -2175,12 +2199,12 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.12.6" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" +checksum = "e9552f850d5f0964a4e4d0bf306459ac29323ddfbae05e35a7c0d35cb0803cc5" dependencies = [ "anyhow", - "itertools 0.12.1", + "itertools 0.13.0", "proc-macro2", "quote", "syn 2.0.89", @@ -2188,9 +2212,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.12.6" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9091c90b0a32608e984ff2fa4091273cbdd755d54935c51d520887f4a1dbd5b0" +checksum = "4759aa0d3a6232fb8dbdb97b61de2c20047c68aca932c7ed76da9d788508d670" dependencies = [ "prost", ] @@ -2217,15 +2241,15 @@ dependencies = [ [[package]] name = "pyo3" -version = "0.21.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5e00b96a521718e08e03b1a622f01c8a8deb50719335de3f60b3b3950f069d8" +checksum = "7ebb0c0cc0de9678e53be9ccf8a2ab53045e6e3a8be03393ceccc5e7396ccb40" dependencies = [ "cfg-if", "indoc", "libc", "memoffset", - "parking_lot", + "once_cell", "portable-atomic", "pyo3-build-config", "pyo3-ffi", @@ -2235,9 +2259,9 @@ dependencies = [ [[package]] name = "pyo3-build-config" -version = "0.21.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7883df5835fafdad87c0d888b266c8ec0f4c9ca48a5bed6bbb592e8dedee1b50" +checksum = "80e3ce69c4ec34476534b490e412b871ba03a82e35604c3dfb95fcb6bfb60c09" dependencies = [ "once_cell", "target-lexicon", @@ -2245,9 +2269,9 @@ dependencies = [ [[package]] name = "pyo3-ffi" -version = "0.21.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01be5843dc60b916ab4dad1dca6d20b9b4e6ddc8e15f50c47fe6d85f1fb97403" +checksum = "3b09f311c76b36dfd6dd6f7fa6f9f18e7e46a1c937110d283e80b12ba2468a75" dependencies = [ "libc", "pyo3-build-config", @@ -2255,9 +2279,9 @@ dependencies = [ [[package]] name = "pyo3-macros" -version = "0.21.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77b34069fc0682e11b31dbd10321cbf94808394c56fd996796ce45217dfac53c" +checksum = "fd4f74086536d1e1deaff99ec0387481fb3325c82e4e48be0e75ab3d3fcb487a" dependencies = [ "proc-macro2", "pyo3-macros-backend", @@ -2267,11 +2291,11 @@ dependencies = [ [[package]] name = "pyo3-macros-backend" -version = "0.21.2" +version = "0.23.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08260721f32db5e1a5beae69a55553f56b99bd0e1c3e6e0a5e8851a9d0f5a85c" +checksum = "9e77dfeb76b32bbf069144a5ea0a36176ab59c8db9ce28732d0f06f096bbfbc8" dependencies = [ - "heck 0.4.1", + "heck", "proc-macro2", "pyo3-build-config", "quote", @@ -2356,7 +2380,7 @@ version = "11.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ab240315c661615f2ee9f0f2cd32d5a7343a84d5ebcccb99d46e6637565e7b0" dependencies = [ - "bitflags 2.6.0", + "bitflags", ] [[package]] @@ -2385,7 +2409,7 @@ version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" dependencies = [ - "bitflags 2.6.0", + "bitflags", ] [[package]] @@ -2396,7 +2420,7 @@ checksum = "ba009ff324d1fc1b900bd1fdb31564febe58a8ccc8a6fdbb93b543d33b13ca43" dependencies = [ "getrandom", "libredox", - "thiserror", + "thiserror 1.0.69", ] [[package]] @@ -2521,8 +2545,8 @@ dependencies = [ "indicatif", "indoc", "iter-chunks", - "itertools 0.12.1", - "libtest-mimic", + "itertools 0.13.0", + "libtest-mimic 0.8.1", "minitrace", "moka", "num-traits", @@ -2549,7 +2573,7 @@ dependencies = [ "sqlplannertest", "tempfile", "test-case", - "thiserror", + "thiserror 2.0.3", "tikv-jemallocator", "tokio", "tracing", @@ -2640,7 +2664,7 @@ version = "0.38.41" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d7f649912bc1495e167a6edee79151c84b1bad49748cb4f1f1167f459f6224f6" dependencies = [ - "bitflags 2.6.0", + "bitflags", "errno", "libc", "linux-raw-sys", @@ -2686,11 +2710,11 @@ checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" [[package]] name = "rustyline" -version = "14.0.0" +version = "15.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7803e8936da37efd9b6d4478277f4b2b9bb5cdb37a113e8d63222e58da647e63" +checksum = "2ee1e066dc922e513bda599c6ccb5f3bb2b0ea5870a579448f2622993f0a9a2f" dependencies = [ - "bitflags 2.6.0", + "bitflags", "cfg-if", "clipboard-win", "fd-lock", @@ -2701,9 +2725,9 @@ dependencies = [ "nix", "radix_trie", "unicode-segmentation", - "unicode-width 0.1.14", + "unicode-width 0.2.0", "utf8parse", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] @@ -2886,9 +2910,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.20.6" +version = "0.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff40a700928567c7303adc6f86cd17bfade5fa8c7dce1256f4691eca66e3ca42" +checksum = "ec31dce96f489e2247a165837f49bbce4912b0cbcf127b79b4fdd87503022ae9" dependencies = [ "async-trait", "educe", @@ -2897,14 +2921,14 @@ dependencies = [ "glob", "humantime", "itertools 0.13.0", - "libtest-mimic", + "libtest-mimic 0.8.1", "md-5", "owo-colors", "regex", "similar", "subst", "tempfile", - "thiserror", + "thiserror 2.0.3", "tracing", ] @@ -2920,16 +2944,16 @@ dependencies = [ [[package]] name = "sqlplannertest" -version = "0.1.0" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca53611e63d1376360583b1f0b2816b4e86537529558903c8361d11c2f9d1d46" +checksum = "40df613c24d7066362c37ef5047373532700efb87856088212a47f7b273c7424" dependencies = [ "anyhow", "async-trait", "console", "futures-util", "glob", - "libtest-mimic", + "libtest-mimic 0.7.3", "serde", "serde_yaml", "similar", @@ -2988,7 +3012,7 @@ version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" dependencies = [ - "heck 0.5.0", + "heck", "proc-macro2", "quote", "rustversion", @@ -3055,6 +3079,12 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" +[[package]] +name = "sync_wrapper" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bf256ce5efdfa370213c1dabab5935a12e49f2c58d15e9eac2870d3b4f27263" + [[package]] name = "tagptr" version = "0.2.0" @@ -3134,7 +3164,16 @@ version = "1.0.69" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" dependencies = [ - "thiserror-impl", + "thiserror-impl 1.0.69", +] + +[[package]] +name = "thiserror" +version = "2.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c006c85c7651b3cf2ada4584faa36773bd07bac24acfb39f3c431b36d7e667aa" +dependencies = [ + "thiserror-impl 2.0.3", ] [[package]] @@ -3148,6 +3187,17 @@ dependencies = [ "syn 2.0.89", ] +[[package]] +name = "thiserror-impl" +version = "2.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f077553d607adc1caf65430528a576c757a71ed73944b66ebb58ef2bbd243568" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.89", +] + [[package]] name = "thread_local" version = "1.1.8" @@ -3169,9 +3219,9 @@ dependencies = [ [[package]] name = "tikv-jemalloc-sys" -version = "0.5.4+5.3.0-patched" +version = "0.6.0+5.3.0-1-ge13ca993e8ccb9ba9847cc330696e02839f328f7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9402443cb8fd499b6f327e40565234ff34dbda27460c5b47db0db77443dd85d1" +checksum = "cd3c60906412afa9c2b5b5a48ca6a5abe5736aec9eb48ad05037a677e52e4e2d" dependencies = [ "cc", "libc", @@ -3179,9 +3229,9 @@ dependencies = [ [[package]] name = "tikv-jemallocator" -version = "0.5.4" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "965fe0c26be5c56c94e38ba547249074803efd52adfb66de62107d95aab3eaca" +checksum = "4cec5ff18518d81584f477e9bfdf957f5bb0979b0bac3af4ca30b5b3ae2d2865" dependencies = [ "libc", "tikv-jemalloc-sys", @@ -3250,16 +3300,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "tokio-io-timeout" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30b74022ada614a1b4834de765f9bb43877f910cc8ce4be40e89042c9223a8bf" -dependencies = [ - "pin-project-lite", - "tokio", -] - [[package]] name = "tokio-macros" version = "2.4.0" @@ -3325,26 +3365,29 @@ dependencies = [ [[package]] name = "tonic" -version = "0.10.2" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d560933a0de61cf715926b9cac824d4c883c2c43142f787595e48280c40a1d0e" +checksum = "877c5b330756d856ffcc4553ab34a5684481ade925ecc54bcd1bf02b1d0d4d52" dependencies = [ "async-stream", "async-trait", "axum", - "base64 0.21.7", + "base64 0.22.1", "bytes", "h2", "http", "http-body", + "http-body-util", "hyper", "hyper-timeout", + "hyper-util", "percent-encoding", "pin-project", "prost", + "socket2", "tokio", "tokio-stream", - "tower", + "tower 0.4.13", "tower-layer", "tower-service", "tracing", @@ -3370,6 +3413,20 @@ dependencies = [ "tracing", ] +[[package]] +name = "tower" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2873938d487c3cfb9aed7546dc9f2711d867c9f90c46b889989a2cb84eba6b4f" +dependencies = [ + "futures-core", + "futures-util", + "pin-project-lite", + "sync_wrapper 0.1.2", + "tower-layer", + "tower-service", +] + [[package]] name = "tower-layer" version = "0.3.3" @@ -3879,7 +3936,7 @@ dependencies = [ "ring", "signature", "spki", - "thiserror", + "thiserror 1.0.69", "zeroize", ] diff --git a/Cargo.toml b/Cargo.toml index fd226f2b..31cd0003 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -23,14 +23,14 @@ async-recursion = "1" async-stream = "0.3" async-trait = "0.1" binary-heap-plus = "0.5" -bit-set = "0.5" +bit-set = "0.8" bitvec = { version = "1", features = ["serde"] } btreemultimap = "0.1" bytes = "1" chrono = { version = "0.4", features = ["clock", "std", "wasmbind"] } clap = { version = "4", features = ["derive"] } comfy-table = { version = "7", default-features = false } -console-subscriber = "0.2" +console-subscriber = "0.4" crc32fast = "1" csv = "1" dirs = "5" @@ -44,31 +44,31 @@ humantime = "2" indicatif = "0.17" indoc = "2" iter-chunks = "0.2" -itertools = "0.12" +itertools = "0.13" minitrace = { version = "0.6", features = ["enable"] } moka = { version = "0.12", features = ["future"] } num-traits = "0.2" ordered-float = { version = "4", features = ["serde"] } parking_lot = "0.12" -parse-display = "0.9" +parse-display = "0.10" paste = "1" pgwire = "0.20" pin-project = "1" pretty-xmlish = "0.1" -prost = "0.12" -pyo3 = { version = "0.21", features = ["extension-module"], optional = true } +prost = "0.13" +pyo3 = { version = "0.23", features = ["extension-module"], optional = true } ref-cast = "1.0" regex = "1" risinglight_proto = "0.2" rust_decimal = "1" -rustyline = "14" +rustyline = "15" serde = { version = "1", features = ["derive", "rc"] } serde_json = "1" smallvec = { version = "1", features = ["serde"] } -sqllogictest = "0.20" +sqllogictest = "0.23" sqlparser = { version = "0.45", features = ["serde"] } -thiserror = "1" -tikv-jemallocator = { version = "0.5", optional = true, features = [ +thiserror = "2" +tikv-jemallocator = { version = "0.6", optional = true, features = [ "disable_initial_exec_tls", ] } tokio = { version = "1", features = ["full"] } @@ -81,13 +81,13 @@ tracing-subscriber = { version = "0.3", features = [ [dev-dependencies] criterion = { version = "0.5", features = ["async_tokio"] } glob = "0.3" -libtest-mimic = "0.7" -sqlplannertest = "0.1" +libtest-mimic = "0.8" +sqlplannertest = "0.2" tempfile = "3" test-case = "3" [build-dependencies] -pyo3-build-config = { version = "0.21", optional = true } +pyo3-build-config = { version = "0.23", optional = true } [[test]] name = "sqllogictest" diff --git a/proto/Cargo.toml b/proto/Cargo.toml index 0a853a7e..b25aa9f2 100644 --- a/proto/Cargo.toml +++ b/proto/Cargo.toml @@ -11,7 +11,7 @@ keywords = ["sql", "database", "embedded", "cli"] [dependencies] bytes = "1" -prost = "0.12" +prost = "0.13" [build-dependencies] -prost-build = "0.12" +prost-build = "0.13" From bff93fe90d5238da732d9527c1ae397b2b8e9f38 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 18:31:37 +0800 Subject: [PATCH 11/29] fix warnings Signed-off-by: Runji Wang --- src/executor/evaluator.rs | 39 +++++++++++-------- src/executor/hash_join.rs | 1 - src/executor/mod.rs | 1 - .../column/concrete_column_iterator.rs | 1 + src/storage/secondary/txn_iterator.rs | 1 + 5 files changed, 25 insertions(+), 18 deletions(-) diff --git a/src/executor/evaluator.rs b/src/executor/evaluator.rs index 0d22e431..f39690d0 100644 --- a/src/executor/evaluator.rs +++ b/src/executor/evaluator.rs @@ -211,22 +211,6 @@ impl<'a> Evaluator<'a> { /// Evaluate the aggregation. fn eval_agg(&self, state: AggState, chunk: &DataChunk) -> Result { - impl DataValue { - fn add(self, other: Self) -> Self { - if self.is_null() { - other - } else { - self + other - } - } - fn or(self, other: Self) -> Self { - if self.is_null() { - other - } else { - self - } - } - } use Expr::*; Ok(match state { AggState::Value(state) => AggState::Value(match self.node() { @@ -314,3 +298,26 @@ impl AggState { } } } + +trait Ext { + fn add(self, other: Self) -> Self; + fn or(self, other: Self) -> Self; +} + +impl Ext for DataValue { + fn add(self, other: Self) -> Self { + if self.is_null() { + other + } else { + self + other + } + } + + fn or(self, other: Self) -> Self { + if self.is_null() { + other + } else { + self + } + } +} diff --git a/src/executor/hash_join.rs b/src/executor/hash_join.rs index 313828c3..5ad4e86a 100644 --- a/src/executor/hash_join.rs +++ b/src/executor/hash_join.rs @@ -107,7 +107,6 @@ impl HashJoinExecutor { pub struct HashSemiJoinExecutor { pub left_keys: RecExpr, pub right_keys: RecExpr, - pub left_types: Vec, pub anti: bool, } diff --git a/src/executor/mod.rs b/src/executor/mod.rs index abe3823f..c32900d6 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -481,7 +481,6 @@ impl Builder { HashSemiJoinExecutor { left_keys: self.resolve_column_index(lkeys, left), right_keys: self.resolve_column_index(rkeys, right), - left_types: self.plan_types(left).to_vec(), anti, } .execute(self.build_id(left), self.build_id(right)) diff --git a/src/storage/secondary/column/concrete_column_iterator.rs b/src/storage/secondary/column/concrete_column_iterator.rs index 5388b945..aa5f35ae 100644 --- a/src/storage/secondary/column/concrete_column_iterator.rs +++ b/src/storage/secondary/column/concrete_column_iterator.rs @@ -25,6 +25,7 @@ pub trait BlockIteratorFactory: Send + Sync + 'static { /// Create a [`FakeBlockIterator`](super::super::block::FakeBlockIterator) from block index and /// seek to `start_pos`. + #[allow(dead_code)] fn get_fake_iterator(&self, index: &BlockIndex, start_pos: usize) -> Self::BlockIteratorImpl; } diff --git a/src/storage/secondary/txn_iterator.rs b/src/storage/secondary/txn_iterator.rs index 0ac29200..5d82e4e7 100644 --- a/src/storage/secondary/txn_iterator.rs +++ b/src/storage/secondary/txn_iterator.rs @@ -17,6 +17,7 @@ pub enum SecondaryIterator { } #[enum_dispatch(SecondaryIterator)] +#[allow(dead_code)] pub trait SecondaryIteratorImpl {} /// An iterator over all data in a transaction. From 68f0ec39ca07c3b573e337f9a0c940717d897165 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 21:30:25 +0800 Subject: [PATCH 12/29] support keyword completion Signed-off-by: Runji Wang --- src/db.rs | 102 ++++++++++++++++++++++++++++++++++++++++++++++++++++ src/main.rs | 8 +++-- 2 files changed, 107 insertions(+), 3 deletions(-) diff --git a/src/db.rs b/src/db.rs index 36c9202d..beb6a896 100644 --- a/src/db.rs +++ b/src/db.rs @@ -197,6 +197,11 @@ impl Database { .add_row_count(table_id, count); Ok(true) } + + /// Return all available pragma options. + fn pragma_options() -> &'static [&'static str] { + &["enable_optimizer", "disable_optimizer"] + } } /// The error type of database operations. @@ -230,3 +235,100 @@ pub enum Error { #[error("Internal error: {0}")] Internal(String), } + +impl rustyline::Helper for &Database {} +impl rustyline::validate::Validator for &Database {} +impl rustyline::highlight::Highlighter for &Database {} +impl rustyline::hint::Hinter for &Database { + type Hint = String; +} + +/// Implement SQL completion. +impl rustyline::completion::Completer for &Database { + type Candidate = rustyline::completion::Pair; + fn complete( + &self, + line: &str, + pos: usize, + _ctx: &rustyline::Context<'_>, + ) -> rustyline::Result<(usize, Vec)> { + // find the word before cursor + let (prefix, last_word) = line[..pos].rsplit_once(' ').unwrap_or(("", &line[..pos])); + + // completion for pragma options + if prefix.trim().eq_ignore_ascii_case("pragma") { + let candidates = Database::pragma_options() + .iter() + .filter(|option| option.starts_with(last_word)) + .map(|option| rustyline::completion::Pair { + display: option.to_string(), + replacement: option.to_string(), + }) + .collect(); + return Ok((pos - last_word.len(), candidates)); + } + + // TODO: complete table and column names + + // completion for keywords + + // for a given prefix, all keywords starting with the prefix are returned as candidates + // they should be ordered in principle that frequently used ones come first + const KEYWORDS: &[&str] = &[ + "AS", "ALL", "ANALYZE", "CREATE", "COPY", "DELETE", "DROP", "EXPLAIN", "FROM", + "FUNCTION", "INSERT", "JOIN", "ON", "PRAGMA", "SET", "SELECT", "TABLE", "UNION", + "VIEW", "WHERE", "WITH", + ]; + let last_word_upper = last_word.to_uppercase(); + let candidates = KEYWORDS + .iter() + .filter(|command| command.starts_with(&last_word_upper)) + .map(|command| rustyline::completion::Pair { + display: command.to_string(), + replacement: format!("{command} "), + }) + .collect(); + Ok((pos - last_word.len(), candidates)) + } +} + +#[cfg(test)] +mod tests { + use rustyline::history::DefaultHistory; + + use super::*; + + #[test] + fn test_completion() { + let db = Database::new_in_memory(); + assert_complete(&db, "sel", "SELECT "); + assert_complete(&db, "sel|ect", "SELECT ect"); + assert_complete(&db, "select a f", "select a FROM "); + assert_complete(&db, "pragma en", "pragma enable_optimizer"); + } + + /// Assert that if complete (e.g. press tab) the given `line_with_cursor`, the result will be + /// `completed_line`. + /// + /// `line_with_cursor` can optionally contain a `|` which indicates the cursor position. + /// If not provided, the cursor is assumed to be at the end of the line. + #[track_caller] + fn assert_complete(db: &Database, line_with_cursor: &str, completed_line: &str) { + // find cursor + let (before_cursor, after_cursor) = line_with_cursor + .split_once('|') + .unwrap_or((line_with_cursor, "")); + let pos = before_cursor.len(); + let mut line = format!("{before_cursor}{after_cursor}"); + + // complete + use rustyline::completion::Completer; + let (complete_pos, candidates) = db + .complete(&line, pos, &rustyline::Context::new(&DefaultHistory::new())) + .unwrap(); + + // assert + line.replace_range(complete_pos..pos, &candidates[0].replacement); + assert_eq!(line, completed_line); + } +} diff --git a/src/main.rs b/src/main.rs index a3374485..c002bcc3 100644 --- a/src/main.rs +++ b/src/main.rs @@ -20,7 +20,8 @@ use risinglight::storage::SecondaryStorageOptions; use risinglight::utils::time::RoundingDuration; use risinglight::Database; use rustyline::error::ReadlineError; -use rustyline::DefaultEditor; +use rustyline::history::DefaultHistory; +use rustyline::Editor; use sqllogictest::DefaultColumnType; use tokio::{select, signal}; use tracing::{info, warn, Level}; @@ -149,7 +150,7 @@ async fn run_query_in_background(db: Arc, sql: String, output_format: /// /// Note that `;` in string literals will also be treated as a terminator /// as long as it is at the end of a line. -fn read_sql(rl: &mut DefaultEditor) -> Result { +fn read_sql(rl: &mut Editor<&Database, DefaultHistory>) -> Result { let mut sql = String::new(); loop { let prompt = if sql.is_empty() { "> " } else { "? " }; @@ -174,7 +175,7 @@ fn read_sql(rl: &mut DefaultEditor) -> Result { /// Run RisingLight interactive mode async fn interactive(db: Database, output_format: Option) -> Result<()> { - let mut rl = DefaultEditor::new()?; + let mut rl = Editor::<&Database, DefaultHistory>::new()?; let history_path = dirs::cache_dir().map(|p| { let cache_dir = p.join("risinglight"); std::fs::create_dir_all(cache_dir.as_path()).ok(); @@ -192,6 +193,7 @@ async fn interactive(db: Database, output_format: Option) -> Result<()> } let db = Arc::new(db); + rl.set_helper(Some(&db)); loop { let read_sql = read_sql(&mut rl); From 9710023e7009ad49fba57c61b530d5e50cbf7dca Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 21:44:26 +0800 Subject: [PATCH 13/29] support cursor in completed line Signed-off-by: Runji Wang --- src/db.rs | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/src/db.rs b/src/db.rs index beb6a896..181cb4f8 100644 --- a/src/db.rs +++ b/src/db.rs @@ -302,33 +302,37 @@ mod tests { fn test_completion() { let db = Database::new_in_memory(); assert_complete(&db, "sel", "SELECT "); - assert_complete(&db, "sel|ect", "SELECT ect"); + assert_complete(&db, "sel|ect", "SELECT |ect"); assert_complete(&db, "select a f", "select a FROM "); assert_complete(&db, "pragma en", "pragma enable_optimizer"); } - /// Assert that if complete (e.g. press tab) the given `line_with_cursor`, the result will be + /// Assert that if complete (e.g. press tab) the given `line`, the result will be /// `completed_line`. /// - /// `line_with_cursor` can optionally contain a `|` which indicates the cursor position. - /// If not provided, the cursor is assumed to be at the end of the line. + /// Both `line` and `completed_line` can optionally contain a `|` which indicates the cursor + /// position. If not provided, the cursor is assumed to be at the end of the line. #[track_caller] - fn assert_complete(db: &Database, line_with_cursor: &str, completed_line: &str) { - // find cursor - let (before_cursor, after_cursor) = line_with_cursor - .split_once('|') - .unwrap_or((line_with_cursor, "")); - let pos = before_cursor.len(); - let mut line = format!("{before_cursor}{after_cursor}"); + fn assert_complete(db: &Database, line: &str, completed_line: &str) { + /// Find cursor position and remove it from the line. + fn get_line_and_cursor(line: &str) -> (String, usize) { + let (before_cursor, after_cursor) = line.split_once('|').unwrap_or((line, "")); + let pos = before_cursor.len(); + (format!("{before_cursor}{after_cursor}"), pos) + } + let (mut line, pos) = get_line_and_cursor(line); // complete use rustyline::completion::Completer; - let (complete_pos, candidates) = db + let (start_pos, candidates) = db .complete(&line, pos, &rustyline::Context::new(&DefaultHistory::new())) .unwrap(); + let replacement = &candidates[0].replacement; + line.replace_range(start_pos..pos, replacement); // assert - line.replace_range(complete_pos..pos, &candidates[0].replacement); + let (completed_line, completed_cursor_pos) = get_line_and_cursor(completed_line); assert_eq!(line, completed_line); + assert_eq!(start_pos + replacement.len(), completed_cursor_pos); } } From c77c0b04ec220814ae3538c93e409ce9f5d31ba6 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 21:53:34 +0800 Subject: [PATCH 14/29] fix clippy Signed-off-by: Runji Wang --- src/array/primitive_array.rs | 2 +- src/binder/mod.rs | 6 +++--- src/executor/copy_from_file.rs | 2 +- src/planner/explain.rs | 2 +- src/planner/rules/expr.rs | 1 - src/planner/rules/mod.rs | 2 +- src/storage/secondary/block/rle_block_iterator.rs | 6 +++--- src/storage/secondary/column/concrete_column_iterator.rs | 8 ++++---- src/storage/secondary/options.rs | 4 ++-- src/storage/secondary/version_manager.rs | 4 ++-- 10 files changed, 18 insertions(+), 19 deletions(-) diff --git a/src/array/primitive_array.rs b/src/array/primitive_array.rs index de48a00a..91002404 100644 --- a/src/array/primitive_array.rs +++ b/src/array/primitive_array.rs @@ -192,7 +192,7 @@ impl PrimitiveArray { impl PrimitiveArray { /// Rescale the decimals. pub fn rescale(&mut self, scale: u8) { - for v in self.data.iter_mut() { + for v in &mut self.data { v.rescale(scale as u32); } } diff --git a/src/binder/mod.rs b/src/binder/mod.rs index 45b3f2f6..68daf8d2 100644 --- a/src/binder/mod.rs +++ b/src/binder/mod.rs @@ -252,15 +252,15 @@ pub fn bind_header(mut chunk: array::Chunk, stmt: &Statement) -> array::Chunk { #[derive(Debug, Default)] struct Context { /// Defined CTEs. - /// cte_name -> (query_id, column_alias -> id) + /// `cte_name` -> (`query_id`, `column_alias` -> id) ctes: HashMap)>, /// Table aliases that can be accessed from the current query. table_aliases: HashSet, /// Column aliases that can be accessed from the current query. - /// column_alias -> (table_alias -> id) + /// `column_alias` -> (`table_alias` -> id) column_aliases: HashMap>, /// Column aliases that can be accessed from the outside query. - /// column_alias -> id + /// `column_alias` -> id output_aliases: HashMap, } diff --git a/src/executor/copy_from_file.rs b/src/executor/copy_from_file.rs index b77aa5a7..66e8290f 100644 --- a/src/executor/copy_from_file.rs +++ b/src/executor/copy_from_file.rs @@ -94,7 +94,7 @@ impl CopyFromFileExecutor { return Err(Error::length_mismatch(column_count, record.len())); } - size_count += record.as_slice().as_bytes().len(); + size_count += record.as_slice().len(); // push a raw str row and send it if necessary if let Some(chunk) = chunk_builder.push_str_row(record.iter())? { diff --git a/src/planner/explain.rs b/src/planner/explain.rs index 63c842cc..0cb2a03f 100644 --- a/src/planner/explain.rs +++ b/src/planner/explain.rs @@ -377,7 +377,7 @@ impl<'a> Explain<'a> { } } -impl<'a> fmt::Display for Explain<'a> { +impl fmt::Display for Explain<'_> { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { delegate_fmt(&self.pretty(), f, String::with_capacity(4096)) } diff --git a/src/planner/rules/expr.rs b/src/planner/rules/expr.rs index 817fb13f..6bab0d3d 100644 --- a/src/planner/rules/expr.rs +++ b/src/planner/rules/expr.rs @@ -241,7 +241,6 @@ mod tests { } egg::test_fn! { - #[cfg_attr(feature = "simd", ignore)] // FIXME: 'attempt to divide by zero' constant_folding, rules(), "(* (- (+ 1 2) 4) (/ 6 2))" => "-3", diff --git a/src/planner/rules/mod.rs b/src/planner/rules/mod.rs index 2d84f4b8..f683df77 100644 --- a/src/planner/rules/mod.rs +++ b/src/planner/rules/mod.rs @@ -118,7 +118,7 @@ impl Analysis for ExprAnalysis { let merge_columns = merge_small_set(&mut to.columns, from.columns); let merge_schema = egg::merge_max(&mut to.schema, from.schema); let merge_rows = egg::merge_min( - unsafe { std::mem::transmute(&mut to.rows) }, + unsafe { std::mem::transmute::<&mut f32, &mut F32>(&mut to.rows) }, F32::from(from.rows), ); let merge_order = egg::merge_max(&mut to.orderby, from.orderby); diff --git a/src/storage/secondary/block/rle_block_iterator.rs b/src/storage/secondary/block/rle_block_iterator.rs index fc838f75..70ebfdd6 100644 --- a/src/storage/secondary/block/rle_block_iterator.rs +++ b/src/storage/secondary/block/rle_block_iterator.rs @@ -33,13 +33,13 @@ where /// Indicates current position in the rle block cur_row: usize, - /// Indicates how many rows get scanned for cur_row + /// Indicates how many rows get scanned for `cur_row` cur_scanned_count: usize, /// Indicates the number of rows in the rle block rle_row_count: usize, - /// Indicates the element of current row get from block_iter + /// Indicates the element of current row get from `block_iter` cur_element: Option<::Owned>, /// Indicates how many rows get scanned for this iterator @@ -48,7 +48,7 @@ where /// Total count of elements in block row_count: usize, - /// If never_used is true, get an item from child iter in the beginning of next_batch() + /// If `never_used` is true, get an item from child iter in the beginning of `next_batch()` never_used: bool, } diff --git a/src/storage/secondary/column/concrete_column_iterator.rs b/src/storage/secondary/column/concrete_column_iterator.rs index aa5f35ae..a8c29726 100644 --- a/src/storage/secondary/column/concrete_column_iterator.rs +++ b/src/storage/secondary/column/concrete_column_iterator.rs @@ -32,10 +32,10 @@ pub trait BlockIteratorFactory: Send + Sync + 'static { /// `ConcreteColumnIterator` Statistics #[derive(Debug, Default)] pub struct Statistics { - /// next_batch call times + /// `next_batch` call times next_batch_count: u32, - /// get_block call times + /// `get_block` call times fetched_block_count: u32, } @@ -50,7 +50,7 @@ pub struct ConcreteColumnIterator> { /// Block iterator. block_iterator: F::BlockIteratorImpl, - /// RowID of the current column. + /// `RowID` of the current column. current_row_id: u32, /// Indicates whether this iterator has finished or not. @@ -59,7 +59,7 @@ pub struct ConcreteColumnIterator> { /// The factory for creating iterators. factory: F, - /// Indicate whether current_block_iter is fake. + /// Indicate whether `current_block_iter` is fake. is_fake_iter: bool, /// Statistics which used for reporting. diff --git a/src/storage/secondary/options.rs b/src/storage/secondary/options.rs index 0630e238..0a450b56 100644 --- a/src/storage/secondary/options.rs +++ b/src/storage/secondary/options.rs @@ -61,7 +61,7 @@ pub struct StorageOptions { /// Encode type pub encode_type: EncodeType, - /// Whether record first_key of each block into block_index + /// Whether record `first_key` of each block into `block_index` pub record_first_key: bool, /// Whether to disable all disk operations, only for test use @@ -117,7 +117,7 @@ pub struct ColumnBuilderOptions { /// Encode type pub encode_type: EncodeType, - /// Whether record first_key of each block + /// Whether record `first_key` of each block pub record_first_key: bool, } diff --git a/src/storage/secondary/version_manager.rs b/src/storage/secondary/version_manager.rs index ee80d65d..663072b0 100644 --- a/src/storage/secondary/version_manager.rs +++ b/src/storage/secondary/version_manager.rs @@ -106,10 +106,10 @@ pub struct VersionManagerInner { /// MVCC structure for this, and only record changes compared with last epoch. status: HashMap>, - /// (TableId, RowSetId) -> Object mapping + /// (`TableId`, `RowSetId`) -> Object mapping rowsets: HashMap<(u32, u32), Arc>, - /// (TableId, DVId) -> Object mapping + /// (`TableId`, `DVId`) -> Object mapping dvs: HashMap<(u32, u64), Arc>, /// Reference count of each epoch. From 685b148eb554ddb9eb376f5945bc401fab7a511f Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 22:46:54 +0800 Subject: [PATCH 15/29] fix to_parallel for left outer join and DDL statements Signed-off-by: Runji Wang --- src/db.rs | 7 ++++- src/executor/mod.rs | 14 +++------ src/planner/mod.rs | 8 ++++++ src/planner/rules/partition.rs | 52 ++++++++++++++++++++++++++++------ 4 files changed, 61 insertions(+), 20 deletions(-) diff --git a/src/db.rs b/src/db.rs index 2f2e5e92..9ec38fc6 100644 --- a/src/db.rs +++ b/src/db.rs @@ -201,7 +201,12 @@ impl Database { /// Return all available pragma options. fn pragma_options() -> &'static [&'static str] { - &["enable_optimizer", "disable_optimizer"] + &[ + "enable_optimizer", + "disable_optimizer", + "enable_parallel_execution", + "disable_parallel_execution", + ] } } diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 885989d7..ddbb9896 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -467,14 +467,14 @@ impl Builder { .collect(), storage: self.storage.clone(), } - .execute(self.build_id(child).assume_single()) + .execute(self.build_id(child).spawn_merge()) .into(), Delete([table, child]) => DeleteExecutor { table_id: self.node(table).as_table(), storage: self.storage.clone(), } - .execute(self.build_id(child).assume_single()) + .execute(self.build_id(child).spawn_merge()) .into(), CopyFrom([src, types]) => CopyFromFileExecutor { @@ -487,7 +487,7 @@ impl Builder { CopyTo([src, child]) => CopyToFileExecutor { source: self.node(src).as_ext_source(), } - .execute(self.build_id(child).assume_single()) + .execute(self.build_id(child).spawn_merge()) .into(), Explain(plan) => ExplainExecutor { @@ -556,7 +556,7 @@ impl Builder { node => panic!("invalid exchange type: {node:?}"), }, - node => panic!("not a plan: {node:?}"), + node => panic!("not a plan: {node:?}\n{:?}", self.egraph.dump()), }; stream = self.instrument(id, stream); // if parallel plan is enabled, each executor will be partitioned and consecutive @@ -738,12 +738,6 @@ impl From for PartitionedStream { } impl PartitionedStream { - /// Assume that there is only one stream and returns it. - fn assume_single(self) -> BoxedExecutor { - assert_eq!(self.streams.len(), 1); - self.streams.into_iter().next().unwrap() - } - /// Merges the partitioned streams into a single stream. /// /// ```text diff --git a/src/planner/mod.rs b/src/planner/mod.rs index cec5f6c4..8bd3a5e8 100644 --- a/src/planner/mod.rs +++ b/src/planner/mod.rs @@ -268,6 +268,14 @@ impl Expr { use Expr::*; matches!(self, RowNumber) || self.is_aggregate_function() } + + pub const fn is_ddl(&self) -> bool { + use Expr::*; + matches!( + self, + CreateTable(_) | CreateView(_) | CreateFunction(_) | Drop(_) + ) + } } trait ExprExt { diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index ab2c798d..81728463 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -18,6 +18,11 @@ use crate::planner::RecExpr; /// Converts a physical plan into a parallel plan. pub fn to_parallel_plan(mut plan: RecExpr) -> RecExpr { + // DDL statements are not parallelizable + if plan.as_ref()[plan.as_ref().len() - 1].is_ddl() { + return plan; + } + // add to_parallel to the root node let root_id = Id::from(plan.as_ref().len() - 1); plan.add(Expr::ToParallel(root_id)); @@ -27,6 +32,14 @@ pub fn to_parallel_plan(mut plan: RecExpr) -> RecExpr { .run(TO_PARALLEL_RULES.iter()); let extractor = egg::Extractor::new(&runner.egraph, NoToParallel); let (_, expr) = extractor.find_best(runner.roots[0]); + + assert!( + expr.as_ref() + .iter() + .all(|node| !matches!(node, Expr::ToParallel(_))), + "unexpected ToParallel in the parallel plan:\n{}", + expr.pretty(60) + ); expr } @@ -38,10 +51,12 @@ impl egg::CostFunction for NoToParallel { where C: FnMut(Id) -> Self::Cost, { + let cost = enode.fold(1usize, |sum, id| sum.saturating_add(costs(id))); + // if all candidates contain ToParallel, the one with the deepest ToParallel will be chosen. if let Expr::ToParallel(_) = enode { - return usize::MAX; + return cost * 1024; } - enode.fold(1, |sum, id| sum.saturating_add(costs(id))) + cost } } @@ -87,7 +102,7 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { "(to_parallel (topn ?limit ?offset ?key ?child))" => "(topn ?limit ?offset ?key (exchange single (to_parallel ?child)))" ), - // inner join is partitioned by left + // inner join and left outer join are partitioned by left // as the left side is materialized in memory rw!("inner-join-to-parallel"; "(to_parallel (join inner ?cond ?left ?right))" => @@ -95,12 +110,11 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { (exchange random (to_parallel ?left)) (exchange broadcast (to_parallel ?right)))" ), - // outer join can not be partitioned - rw!("join-to-parallel"; - "(to_parallel (join full_outer ?cond ?left ?right))" => - "(join full_outer ?cond - (exchange single (to_parallel ?left)) - (exchange single (to_parallel ?right)))" + rw!("left-outer-join-to-parallel"; + "(to_parallel (join left_outer ?cond ?left ?right))" => + "(join left_outer ?cond + (exchange random (to_parallel ?left)) + (exchange broadcast (to_parallel ?right)))" ), // hash join can be partitioned by join key rw!("hashjoin-to-parallel"; @@ -141,6 +155,26 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { "(to_parallel (window ?exprs ?child))" => "(window ?exprs (exchange single (to_parallel ?child)))" ), + // insert + rw!("insert-to-parallel"; + "(to_parallel (insert ?table ?columns ?child))" => + "(insert ?table ?columns (to_parallel ?child))" + ), + // delete + rw!("delete-to-parallel"; + "(to_parallel (delete ?table ?child))" => + "(delete ?table (to_parallel ?child))" + ), + // copy_from + rw!("copy_from-to-parallel"; + "(to_parallel (copy_from ?dest ?types))" => + "(copy_from ?dest ?types)" + ), + // copy_to + rw!("copy_to-to-parallel"; + "(to_parallel (copy_to ?dest ?child))" => + "(copy_to ?dest (to_parallel ?child))" + ), // explain rw!("explain-to-parallel"; "(to_parallel (explain ?child))" => From 322d8f10081a7a3c274eef357bb2f141b1cfe0cf Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 23:12:00 +0800 Subject: [PATCH 16/29] fix hash exchange Signed-off-by: Runji Wang --- src/executor/exchange.rs | 12 +++++++----- src/executor/mod.rs | 14 +++----------- 2 files changed, 10 insertions(+), 16 deletions(-) diff --git a/src/executor/exchange.rs b/src/executor/exchange.rs index 5e1a790d..fc94df73 100644 --- a/src/executor/exchange.rs +++ b/src/executor/exchange.rs @@ -8,9 +8,10 @@ use super::*; /// Distribute the input data to multiple partitions by hash partitioning. pub struct HashPartitionProducer { - /// The indices of the columns to hash. - pub hash_key: Vec, - /// The number of partitions. + /// The expression to extract the keys. + /// e.g. `(list #0 #1)` + pub keys: RecExpr, + /// The number of partitions to produce. pub num_partitions: usize, } @@ -33,8 +34,9 @@ impl HashPartitionProducer { visibility.resize(batch.cardinality(), false); // calculate the hash - for index in &self.hash_key { - batch.array_at(*index).hash(&mut hashers); + let keys_chunk = Evaluator::new(&self.keys).eval_list(&batch)?; + for column in keys_chunk.arrays() { + column.hash(&mut hashers); } for (hasher, target) in hashers.iter().zip(&mut partition_indices) { *target = hasher.finish() as usize % self.num_partitions; diff --git a/src/executor/mod.rs b/src/executor/mod.rs index ddbb9896..06b953df 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -513,7 +513,7 @@ impl Builder { Schema([_, child]) => self.build_id(child), // schema node is just pass-through - Exchange([dist, child]) => match self.node(dist) { + Exchange([dist, child]) => match self.node(dist).clone() { Single => self.build_id(child).spawn_merge().into(), Broadcast => { let subscriber = self.build_id(child).spawn(); @@ -534,20 +534,12 @@ impl Builder { } Hash(keys) => { let num_partitions = tokio::runtime::Handle::current().metrics().num_workers(); - let child_schema = &self.egraph[child].data.schema; - let hash_key = (self.node(*keys).as_list().iter()) - .map(|id| { - child_schema - .iter() - .position(|&x| x == *id) - .expect("hash key not found in child's schema") - }) - .collect_vec(); + let keys = self.resolve_column_index(keys, child); self.build_id(child) .spawn_dispatch(num_partitions, |c| { HashPartitionProducer { + keys: keys.clone(), num_partitions, - hash_key: hash_key.clone(), } .execute(c) }) From 0961aa1aa571f2e2c1ebba070200da4d25e48305 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sat, 23 Nov 2024 23:52:37 +0800 Subject: [PATCH 17/29] replace pragma `enable_parallel_execution` by set variable `parallelism` Signed-off-by: Runji Wang --- src/db.rs | 47 ++++++++++++++++++++++++++++++---------- src/executor/exchange.rs | 2 +- src/executor/mod.rs | 19 +++++----------- src/planner/optimizer.rs | 7 +++--- 4 files changed, 46 insertions(+), 29 deletions(-) diff --git a/src/db.rs b/src/db.rs index 9ec38fc6..665aca10 100644 --- a/src/db.rs +++ b/src/db.rs @@ -30,9 +30,9 @@ struct Config { /// If true, no optimization will be applied to the query. disable_optimizer: bool, mock_stat: Option, - /// If true, each operator will be parallelized and partitioned. - /// WARN: This feature is under development and may not work properly. - enable_parallel_execution: bool, + /// The number of partitions of each operator. + /// If set to 0, it will be automatically determined by the number of worker threads. + parallelism: usize, } impl Database { @@ -97,7 +97,11 @@ impl Database { crate::planner::Config { enable_range_filter_scan: self.storage.support_range_filter_scan(), table_is_sorted_by_primary_key: self.storage.table_is_sorted_by_primary_key(), - generate_parallel_plan: self.config.lock().unwrap().enable_parallel_execution, + parallelism: if self.config.lock().unwrap().parallelism > 0 { + self.config.lock().unwrap().parallelism + } else { + tokio::runtime::Handle::current().metrics().num_workers() + }, }, ); @@ -167,8 +171,6 @@ impl Database { match name.to_string().as_str() { "enable_optimizer" => config.disable_optimizer = false, "disable_optimizer" => config.disable_optimizer = true, - "enable_parallel_execution" => config.enable_parallel_execution = true, - "disable_parallel_execution" => config.enable_parallel_execution = false, name => return Err(crate::binder::BindError::NoPragma(name.into()).into()), } return Ok(true); @@ -179,6 +181,14 @@ impl Database { else { return Ok(false); }; + if variable.0[0].value == "parallelism" { + let mut config = self.config.lock().unwrap(); + config.parallelism = value[0] + .to_string() + .parse::() + .map_err(|_| Error::Internal("invalid parallelism".into()))?; + return Ok(true); + } let Some(table_name) = variable.0[0].value.strip_prefix("mock_rowcount_") else { return Ok(false); }; @@ -201,12 +211,12 @@ impl Database { /// Return all available pragma options. fn pragma_options() -> &'static [&'static str] { - &[ - "enable_optimizer", - "disable_optimizer", - "enable_parallel_execution", - "disable_parallel_execution", - ] + &["enable_optimizer", "disable_optimizer"] + } + + /// Return all available set variables. + fn set_variables() -> &'static [&'static str] { + &["parallelism"] } } @@ -274,6 +284,19 @@ impl rustyline::completion::Completer for &Database { return Ok((pos - last_word.len(), candidates)); } + // completion for set variable + if prefix.trim().eq_ignore_ascii_case("set") { + let candidates = Database::set_variables() + .iter() + .filter(|option| option.starts_with(last_word)) + .map(|option| rustyline::completion::Pair { + display: option.to_string(), + replacement: option.to_string(), + }) + .collect(); + return Ok((pos - last_word.len(), candidates)); + } + // TODO: complete table and column names // completion for keywords diff --git a/src/executor/exchange.rs b/src/executor/exchange.rs index fc94df73..50c53fda 100644 --- a/src/executor/exchange.rs +++ b/src/executor/exchange.rs @@ -63,7 +63,7 @@ pub struct RandomPartitionProducer { impl RandomPartitionProducer { #[try_stream(boxed, ok = (DataChunk, usize), error = ExecutorError)] pub async fn execute(self, child: BoxedExecutor) { - let mut rng = rand::rngs::SmallRng::from_entropy(); + let mut rng = rand::rngs::SmallRng::from_seed([0; 32]); #[for_await] for batch in child { let batch = batch?; diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 06b953df..d1df2f1d 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -212,7 +212,7 @@ impl Builder { /// Builds stream for the given plan. fn build_id(&mut self, id: Id) -> PartitionedStream { use Expr::*; - let mut stream = match self.node(id).clone() { + let stream = match self.node(id).clone() { Scan([table, list, filter]) => { let table_id = self.node(table).as_table(); let columns = (self.node(list).as_list().iter()) @@ -517,15 +517,15 @@ impl Builder { Single => self.build_id(child).spawn_merge().into(), Broadcast => { let subscriber = self.build_id(child).spawn(); - let parallism = tokio::runtime::Handle::current().metrics().num_workers(); + let num_partitions = self.optimizer.config().parallelism; PartitionedStream { - streams: (0..parallism) + streams: (0..num_partitions) .map(|_| subscriber.subscribe_merge()) .collect(), } } Random => { - let num_partitions = tokio::runtime::Handle::current().metrics().num_workers(); + let num_partitions = self.optimizer.config().parallelism; self.build_id(child) .spawn_dispatch(num_partitions, |c| { RandomPartitionProducer { num_partitions }.execute(c) @@ -533,7 +533,7 @@ impl Builder { .subscribe() } Hash(keys) => { - let num_partitions = tokio::runtime::Handle::current().metrics().num_workers(); + let num_partitions = self.optimizer.config().parallelism; let keys = self.resolve_column_index(keys, child); self.build_id(child) .spawn_dispatch(num_partitions, |c| { @@ -550,14 +550,7 @@ impl Builder { node => panic!("not a plan: {node:?}\n{:?}", self.egraph.dump()), }; - stream = self.instrument(id, stream); - // if parallel plan is enabled, each executor will be partitioned and consecutive - // executors may be fused into a single task. otherwise, we spawn a new task for each - // executor so that executors can be executed in parallel. - if !self.optimizer.config().generate_parallel_plan { - stream = stream.spawn().subscribe(); - } - stream + self.instrument(id, stream) } fn instrument(&mut self, id: Id, stream: PartitionedStream) -> PartitionedStream { diff --git a/src/planner/optimizer.rs b/src/planner/optimizer.rs index 3e139187..04bbd7d0 100644 --- a/src/planner/optimizer.rs +++ b/src/planner/optimizer.rs @@ -20,8 +20,9 @@ pub struct Optimizer { pub struct Config { pub enable_range_filter_scan: bool, pub table_is_sorted_by_primary_key: bool, - /// If true, the optimizer will insert exchange operators to the plan. - pub generate_parallel_plan: bool, + /// The number of partitions of each operator. + /// If set to >1, exchange operators will be inserted into the plan. + pub parallelism: usize, } impl Optimizer { @@ -54,7 +55,7 @@ impl Optimizer { // 3. join reorder and hashjoin self.optimize_stage(&mut expr, &mut cost, STAGE3_RULES.iter(), 3, 8); - if self.analysis.config.generate_parallel_plan { + if self.analysis.config.parallelism > 1 { expr = to_parallel_plan(expr); } expr From 0588dff3938c63827e7d630c23eea0653d7424de Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 00:17:27 +0800 Subject: [PATCH 18/29] fix 2-phase count agg Signed-off-by: Runji Wang --- src/planner/rules/partition.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index 81728463..057ea3d9 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -234,12 +234,12 @@ fn apply_global_aggs(pattern_str: &str) -> impl Applier use Expr::*; let ref_id = egraph.add(Expr::Ref(agg)); let global_agg = match &egraph[agg].nodes[0] { - RowCount => RowCount, Max(_) => Max(ref_id), Min(_) => Min(ref_id), Sum(_) => Sum(ref_id), Avg(_) => panic!("avg is not supported in 2-phase aggregation"), - Count(_) => Count(ref_id), + RowCount => Sum(ref_id), + Count(_) => Sum(ref_id), CountDistinct(_) => { panic!("count distinct is not supported in 2-phase aggregation") } From db3a019ff76e2018cc6822a628f9949671dec60b Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 01:27:52 +0800 Subject: [PATCH 19/29] enable partitioning in unit test. fix bugs Signed-off-by: Runji Wang --- src/array/internal_ext.rs | 5 ++++ src/array/ops.rs | 48 +++++++++++++++++++++++++++++----- src/executor/evaluator.rs | 4 +++ src/planner/rules/partition.rs | 32 +++++++++++++---------- tests/sql/join_left_inner.slt | 12 ++++----- tests/sqllogictest.rs | 1 + 6 files changed, 76 insertions(+), 26 deletions(-) diff --git a/src/array/internal_ext.rs b/src/array/internal_ext.rs index 8d3bf963..650c1c81 100644 --- a/src/array/internal_ext.rs +++ b/src/array/internal_ext.rs @@ -12,6 +12,11 @@ use crate::for_all_variants; pub trait ArrayValidExt: Array { fn get_valid_bitmap(&self) -> &BitVec; fn get_valid_bitmap_mut(&mut self) -> &mut BitVec; + + /// Returns the number of null values in this array. + fn null_count(&self) -> usize { + self.get_valid_bitmap().count_zeros() + } } pub trait ArrayImplValidExt { diff --git a/src/array/ops.rs b/src/array/ops.rs index c8be4675..de2f6146 100644 --- a/src/array/ops.rs +++ b/src/array/ops.rs @@ -621,12 +621,48 @@ impl ArrayImpl { /// Returns the sum of values. pub fn sum(&self) -> DataValue { match self { - Self::Int16(a) => DataValue::Int16(a.raw_iter().sum()), - Self::Int32(a) => DataValue::Int32(a.raw_iter().sum()), - Self::Int64(a) => DataValue::Int64(a.raw_iter().sum()), - Self::Float64(a) => DataValue::Float64(a.raw_iter().sum()), - Self::Decimal(a) => DataValue::Decimal(a.raw_iter().sum()), - Self::Interval(a) => DataValue::Interval(a.raw_iter().sum()), + Self::Int16(a) => { + if a.null_count() == a.len() { + DataValue::Null + } else { + DataValue::Int16(a.raw_iter().sum()) + } + } + Self::Int32(a) => { + if a.null_count() == a.len() { + DataValue::Null + } else { + DataValue::Int32(a.raw_iter().sum()) + } + } + Self::Int64(a) => { + if a.null_count() == a.len() { + DataValue::Null + } else { + DataValue::Int64(a.raw_iter().sum()) + } + } + Self::Float64(a) => { + if a.null_count() == a.len() { + DataValue::Null + } else { + DataValue::Float64(a.raw_iter().sum()) + } + } + Self::Decimal(a) => { + if a.null_count() == a.len() { + DataValue::Null + } else { + DataValue::Decimal(a.raw_iter().sum()) + } + } + Self::Interval(a) => { + if a.null_count() == a.len() { + DataValue::Null + } else { + DataValue::Interval(a.raw_iter().sum()) + } + } _ => panic!("can not sum array"), } } diff --git a/src/executor/evaluator.rs b/src/executor/evaluator.rs index f39690d0..5a278b72 100644 --- a/src/executor/evaluator.rs +++ b/src/executor/evaluator.rs @@ -305,14 +305,18 @@ trait Ext { } impl Ext for DataValue { + /// Add two values. The result is null only if both values are null. fn add(self, other: Self) -> Self { if self.is_null() { other + } else if other.is_null() { + self } else { self + other } } + /// Returns the first non-null value. fn or(self, other: Self) -> Self { if self.is_null() { other diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index 057ea3d9..c861dd5f 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -69,10 +69,11 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { "(to_parallel (scan ?table ?columns ?filter))" => "(exchange random (scan ?table ?columns ?filter))" ), - // values is not partitioned + // values and empty are not partitioned rw!("values-to-parallel"; - "(to_parallel (values ?values))" => - "(exchange random (values ?values))" + "(to_parallel ?child)" => + "(exchange random ?child)" + if node_is("?child", &["values", "empty"]) ), // projection does not change distribution rw!("proj-to-parallel"; @@ -102,19 +103,13 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { "(to_parallel (topn ?limit ?offset ?key ?child))" => "(topn ?limit ?offset ?key (exchange single (to_parallel ?child)))" ), - // inner join and left outer join are partitioned by left - // as the left side is materialized in memory - rw!("inner-join-to-parallel"; - "(to_parallel (join inner ?cond ?left ?right))" => - "(join inner ?cond - (exchange random (to_parallel ?left)) - (exchange broadcast (to_parallel ?right)))" - ), - rw!("left-outer-join-to-parallel"; - "(to_parallel (join left_outer ?cond ?left ?right))" => - "(join left_outer ?cond + // join is partitioned by left + rw!("join-to-parallel"; + "(to_parallel (join ?type ?cond ?left ?right))" => + "(join ?type ?cond (exchange random (to_parallel ?left)) (exchange broadcast (to_parallel ?right)))" + if node_is("?type", &["inner", "left_outer", "semi", "anti"]) ), // hash join can be partitioned by join key rw!("hashjoin-to-parallel"; @@ -207,6 +202,15 @@ fn partition_is_same( move |egraph, _, subst| egraph[subst[a]].data == egraph[subst[b]].data } +/// Returns true if the given node is one of the candidates. +fn node_is( + a: &str, + candidates: &'static [&'static str], +) -> impl Fn(&mut EGraph, Id, &Subst) -> bool { + let a = var(a); + move |egraph, _, subst| candidates.contains(&egraph[subst[a]].nodes[0].to_string().as_str()) +} + /// Returns an applier that replaces `?global_aggs` with the nested `?aggs`. /// /// ```text diff --git a/tests/sql/join_left_inner.slt b/tests/sql/join_left_inner.slt index ceeab1f5..d9600014 100644 --- a/tests/sql/join_left_inner.slt +++ b/tests/sql/join_left_inner.slt @@ -71,12 +71,12 @@ select v1, v2, v3, v4 from a, b; statement ok insert into b values (1, 100), (3, 300), (4, 400); -query IIII +query IIII rowsort select v1, v2, v3, v4 from a left join b on v1 = v3; ---- 1 1 1 100 -3 3 3 300 2 2 NULL NULL +3 3 3 300 statement ok drop table a; @@ -96,19 +96,19 @@ insert into a values (1, 1), (2, 2), (3, 3); statement ok insert into b values (1, 1, 1), (2, 2, 2), (3, 3, 4), (1, 1, 5); -query IIIII +query IIIII rowsort select v1, v2, v3, v4, v5 from a join b on v1 = v3 and v2 = v4; ---- 1 1 1 1 1 +1 1 1 1 5 2 2 2 2 2 3 3 3 3 4 -1 1 1 1 5 -query IIIII +query IIIII rowsort select v1, v2, v3, v4, v5 from a join b on v1 = v3 and v2 = v4 and v1 < v5; ---- -3 3 3 3 4 1 1 1 1 5 +3 3 3 3 4 statement ok drop table a; diff --git a/tests/sqllogictest.rs b/tests/sqllogictest.rs index 215bdd17..9295989a 100644 --- a/tests/sqllogictest.rs +++ b/tests/sqllogictest.rs @@ -79,6 +79,7 @@ async fn test(filename: impl AsRef, engine: Engine) -> Result<()> { Engine::Disk => Database::new_on_disk(SecondaryStorageOptions::default_for_test()).await, Engine::Mem => Database::new_in_memory(), }; + db.run("set parallelism = 2;").await?; // enable data partitioning let db = DatabaseWrapper(db); let mut tester = sqllogictest::Runner::new(|| async { Ok(&db) }); From 7f56f92cade2ebb8cae2edfa815c35dd36d03510 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 01:51:22 +0800 Subject: [PATCH 20/29] fix DDL to parallel Signed-off-by: Runji Wang --- src/binder/create_function.rs | 18 +++++++++--------- src/binder/create_table.rs | 17 +++++++++-------- src/binder/create_view.rs | 4 ++-- src/executor/create_function.rs | 8 ++++---- src/executor/create_table.rs | 4 ++-- src/executor/create_view.rs | 4 ++-- src/executor/mod.rs | 6 +++--- src/planner/explain.rs | 18 ++++++++++-------- src/planner/mod.rs | 29 +++++++++++++++-------------- src/planner/rules/partition.rs | 10 +++++----- 10 files changed, 61 insertions(+), 57 deletions(-) diff --git a/src/binder/create_function.rs b/src/binder/create_function.rs index 71cfd2fa..2bf10ffb 100644 --- a/src/binder/create_function.rs +++ b/src/binder/create_function.rs @@ -10,7 +10,7 @@ use serde::{Deserialize, Serialize}; use super::*; #[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Clone, Serialize, Deserialize)] -pub struct CreateFunction { +pub struct FunctionDef { pub schema_name: String, pub name: String, pub arg_types: Vec, @@ -20,14 +20,14 @@ pub struct CreateFunction { pub body: String, } -impl fmt::Display for CreateFunction { +impl fmt::Display for Box { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let explainer = Pretty::childless_record("CreateFunction", self.pretty_function()); + let explainer = Pretty::childless_record("FunctionDef", self.pretty_function()); delegate_fmt(&explainer, f, String::with_capacity(1000)) } } -impl FromStr for CreateFunction { +impl FromStr for Box { type Err = (); fn from_str(_s: &str) -> std::result::Result { @@ -35,7 +35,7 @@ impl FromStr for CreateFunction { } } -impl CreateFunction { +impl FunctionDef { pub fn pretty_function<'a>(&self) -> Vec<(&'a str, Pretty<'a>)> { vec![ ("name", Pretty::display(&self.name)), @@ -102,7 +102,7 @@ impl Binder { arg_names.push(arg.name.map_or("".to_string(), |n| n.to_string())); } - let f = self.egraph.add(Node::CreateFunction(CreateFunction { + let func_def = self.egraph.add(Node::FunctionDef(Box::new(FunctionDef { schema_name, name, arg_types, @@ -110,8 +110,8 @@ impl Binder { return_type, language, body, - })); - - Ok(f) + }))); + let id = self.egraph.add(Node::CreateFunction(func_def)); + Ok(id) } } diff --git a/src/binder/create_table.rs b/src/binder/create_table.rs index 6c86213c..2bf58fba 100644 --- a/src/binder/create_table.rs +++ b/src/binder/create_table.rs @@ -12,22 +12,22 @@ use super::*; use crate::catalog::{ColumnCatalog, ColumnDesc, ColumnId, SchemaId}; #[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Hash, Clone, Serialize, Deserialize)] -pub struct CreateTable { +pub struct TableDef { pub schema_id: SchemaId, pub table_name: String, pub columns: Vec, pub ordered_pk_ids: Vec, } -impl fmt::Display for CreateTable { +impl fmt::Display for TableDef { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let explainer = Pretty::childless_record("CreateTable", self.pretty_table()); + let explainer = Pretty::childless_record("TableDef", self.pretty_table()); delegate_fmt(&explainer, f, String::with_capacity(1000)) } } -impl CreateTable { - pub fn pretty_table<'a>(&self) -> Vec<(&'a str, Pretty<'a>)> { +impl TableDef { + pub fn pretty_table(&self) -> Vec<(&str, Pretty)> { let cols = Pretty::Array(self.columns.iter().map(|c| c.desc().pretty()).collect()); let ids = Pretty::Array(self.ordered_pk_ids.iter().map(Pretty::display).collect()); vec![ @@ -39,7 +39,7 @@ impl CreateTable { } } -impl FromStr for Box { +impl FromStr for Box { type Err = (); fn from_str(_s: &str) -> std::result::Result { @@ -119,13 +119,14 @@ impl Binder { columns[index as usize].set_nullable(false); } - let create = self.egraph.add(Node::CreateTable(Box::new(CreateTable { + let table_def = self.egraph.add(Node::TableDef(Box::new(TableDef { schema_id: schema.id(), table_name: table_name.into(), columns, ordered_pk_ids, }))); - Ok(create) + let id = self.egraph.add(Node::CreateTable(table_def)); + Ok(id) } /// get primary keys' id in declared order。 diff --git a/src/binder/create_view.rs b/src/binder/create_view.rs index 2b151307..30fd0278 100644 --- a/src/binder/create_view.rs +++ b/src/binder/create_view.rs @@ -49,13 +49,13 @@ impl Binder { }) .collect(); - let table = self.egraph.add(Node::CreateTable(Box::new(CreateTable { + let table_def = self.egraph.add(Node::TableDef(Box::new(TableDef { schema_id: schema.id(), table_name: table_name.into(), columns, ordered_pk_ids: vec![], }))); - let create_view = self.egraph.add(Node::CreateView([table, query])); + let create_view = self.egraph.add(Node::CreateView([table_def, query])); Ok(create_view) } } diff --git a/src/executor/create_function.rs b/src/executor/create_function.rs index cde51bcf..d0d8ba58 100644 --- a/src/executor/create_function.rs +++ b/src/executor/create_function.rs @@ -1,19 +1,19 @@ // Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. use super::*; -use crate::binder::CreateFunction; +use crate::binder::FunctionDef; use crate::catalog::RootCatalogRef; /// The executor of `create function` statement. pub struct CreateFunctionExecutor { - pub f: CreateFunction, + pub function: Box, pub catalog: RootCatalogRef, } impl CreateFunctionExecutor { #[try_stream(boxed, ok = DataChunk, error = ExecutorError)] pub async fn execute(self) { - let CreateFunction { + let FunctionDef { schema_name, name, arg_types, @@ -21,7 +21,7 @@ impl CreateFunctionExecutor { return_type, language, body, - } = self.f; + } = *self.function; self.catalog.create_function( schema_name.clone(), diff --git a/src/executor/create_table.rs b/src/executor/create_table.rs index e0848b4d..40064d67 100644 --- a/src/executor/create_table.rs +++ b/src/executor/create_table.rs @@ -3,12 +3,12 @@ use std::sync::Arc; use super::*; -use crate::binder::CreateTable; +use crate::binder::TableDef; use crate::storage::Storage; /// The executor of `create table` statement. pub struct CreateTableExecutor { - pub table: Box, + pub table: Box, pub storage: Arc, } diff --git a/src/executor/create_view.rs b/src/executor/create_view.rs index 60d5d794..9966d37b 100644 --- a/src/executor/create_view.rs +++ b/src/executor/create_view.rs @@ -1,12 +1,12 @@ // Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. use super::*; -use crate::binder::CreateTable; +use crate::binder::TableDef; use crate::catalog::RootCatalogRef; /// The executor of `create view` statement. pub struct CreateViewExecutor { - pub table: Box, + pub table: Box, pub query: RecExpr, pub catalog: RootCatalogRef, } diff --git a/src/executor/mod.rs b/src/executor/mod.rs index d1df2f1d..cc875c38 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -429,14 +429,14 @@ impl Builder { }), CreateTable(table) => CreateTableExecutor { - table, + table: self.node(table).as_table_def(), storage: self.storage.clone(), } .execute() .into(), CreateView([table, query]) => CreateViewExecutor { - table: self.node(table).as_create_table(), + table: self.node(table).as_table_def(), query: self.recexpr(query), catalog: self.catalog().clone(), } @@ -444,7 +444,7 @@ impl Builder { .into(), CreateFunction(f) => CreateFunctionExecutor { - f, + function: self.node(f).as_function_def(), catalog: self.optimizer.catalog().clone(), } .execute() diff --git a/src/planner/explain.rs b/src/planner/explain.rs index f941b247..c602d865 100644 --- a/src/planner/explain.rs +++ b/src/planner/explain.rs @@ -335,19 +335,21 @@ impl<'a> Explain<'a> { Pretty::childless_record("Hash", vec![("keys", self.expr(keys).pretty())]) } - CreateTable(t) => { - let fields = with_meta(t.pretty_table()); - Pretty::childless_record("CreateTable", fields) - } + CreateTable(table) => Pretty::childless_record( + "CreateTable", + with_meta(vec![("table", self.expr(table).pretty())]), + ), CreateView([table, query]) => Pretty::simple_record( "CreateView", with_meta(vec![("table", self.expr(table).pretty())]), vec![self.expr(query).pretty()], ), - CreateFunction(f) => { - let v = f.pretty_function(); - Pretty::childless_record("CreateFunction", v) - } + TableDef(t) => Pretty::childless_record("TableDef", t.pretty_table()), + CreateFunction(f) => Pretty::childless_record( + "CreateFunction", + with_meta(vec![("function", self.expr(f).pretty())]), + ), + FunctionDef(f) => Pretty::childless_record("FunctionDef", f.pretty_function()), Drop(tables) => { let fields = with_meta(vec![("objects", self.expr(tables).pretty())]); Pretty::childless_record("Drop", fields) diff --git a/src/planner/mod.rs b/src/planner/mod.rs index 8bd3a5e8..8f119f14 100644 --- a/src/planner/mod.rs +++ b/src/planner/mod.rs @@ -3,7 +3,7 @@ use egg::{define_language, Id, Symbol}; use crate::binder::copy::ExtSource; -use crate::binder::{CreateFunction, CreateTable}; +use crate::binder::{FunctionDef, TableDef}; use crate::catalog::{ColumnRefId, TableRefId}; use crate::parser::{BinaryOperator, UnaryOperator}; use crate::types::{ColumnIndex, DataType, DataValue, DateTimeField}; @@ -127,9 +127,11 @@ define_language! { "random" = Random, // (random) random partition "hash" = Hash(Id), // (hash key=[expr..]) partition by hash of key - CreateTable(Box), - "create_view" = CreateView([Id; 2]), // (create_view create_table child) - CreateFunction(CreateFunction), + "create_table" = CreateTable(Id), // (create_table table_def) + "create_view" = CreateView([Id; 2]), // (create_view table_def child) + TableDef(Box), + "create_function" = CreateFunction(Id), // (create_function func_def) + FunctionDef(Box), "drop" = Drop(Id), // (drop [table..]) "insert" = Insert([Id; 3]), // (insert table [column..] child) "delete" = Delete([Id; 2]), // (delete table child) @@ -201,13 +203,20 @@ impl Expr { t } - pub fn as_create_table(&self) -> Box { - let Self::CreateTable(v) = self else { + pub fn as_table_def(&self) -> Box { + let Self::TableDef(v) = self else { panic!("not a create table: {self}") }; v.clone() } + pub fn as_function_def(&self) -> Box { + let Self::FunctionDef(v) = self else { + panic!("not a function definition: {self}") + }; + v.clone() + } + pub fn as_ext_source(&self) -> ExtSource { let Self::ExtSource(v) = self else { panic!("not an external source: {self}") @@ -268,14 +277,6 @@ impl Expr { use Expr::*; matches!(self, RowNumber) || self.is_aggregate_function() } - - pub const fn is_ddl(&self) -> bool { - use Expr::*; - matches!( - self, - CreateTable(_) | CreateView(_) | CreateFunction(_) | Drop(_) - ) - } } trait ExprExt { diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index c861dd5f..3bdcfc60 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -18,11 +18,6 @@ use crate::planner::RecExpr; /// Converts a physical plan into a parallel plan. pub fn to_parallel_plan(mut plan: RecExpr) -> RecExpr { - // DDL statements are not parallelizable - if plan.as_ref()[plan.as_ref().len() - 1].is_ddl() { - return plan; - } - // add to_parallel to the root node let root_id = Id::from(plan.as_ref().len() - 1); plan.add(Expr::ToParallel(root_id)); @@ -180,6 +175,11 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { "(to_parallel (analyze ?child))" => "(analyze (to_parallel ?child))" ), + // no parallel for DDL + rw!("ddl-to-parallel"; + "(to_parallel ?child)" => "?child" + if node_is("?child", &["create_table", "create_view", "create_function", "drop"]) + ), // unnecessary exchange can be removed rw!("remove-exchange"; "(exchange ?dist ?child)" => "?child" From 87a7fb9f4942ccb0bb82e8f382ba1101f0652960 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 01:58:46 +0800 Subject: [PATCH 21/29] add unit test for Expr size Signed-off-by: Runji Wang --- src/binder/expr.rs | 2 +- src/planner/mod.rs | 13 ++++++++++++- src/types/date.rs | 8 ++++++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/src/binder/expr.rs b/src/binder/expr.rs index 82c984f1..74e56f54 100644 --- a/src/binder/expr.rs +++ b/src/binder/expr.rs @@ -238,7 +238,7 @@ impl Binder { fn bind_extract(&mut self, field: DateTimeField, expr: Expr) -> Result { let expr = self.bind_expr(expr)?; - let field = self.egraph.add(Node::Field(field.into())); + let field = self.egraph.add(Node::Field(Box::new(field.into()))); Ok(self.egraph.add(Node::Extract([field, expr]))) } diff --git a/src/planner/mod.rs b/src/planner/mod.rs index 8f119f14..0e56a5cd 100644 --- a/src/planner/mod.rs +++ b/src/planner/mod.rs @@ -65,7 +65,7 @@ define_language! { // functions "extract" = Extract([Id; 2]), // (extract field expr) - Field(DateTimeField), + Field(Box), "replace" = Replace([Id; 3]), // (replace expr pattern replacement) "substring" = Substring([Id; 3]), // (substring expr start length) @@ -303,3 +303,14 @@ impl ExprExt for egg::EClass { .expect("not a column") } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_expr_size() { + // the size of Expr should be as small as possible + assert_eq!(std::mem::size_of::(), 32); + } +} diff --git a/src/types/date.rs b/src/types/date.rs index 96819647..152bcbb3 100644 --- a/src/types/date.rs +++ b/src/types/date.rs @@ -193,3 +193,11 @@ impl FromStr for DateTimeField { })) } } + +impl FromStr for Box { + type Err = (); + + fn from_str(s: &str) -> Result { + DateTimeField::from_str(s).map(Box::new) + } +} From 85d131b2efa8eabd86dd4805d16e75c538b4c69b Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 13:52:04 +0800 Subject: [PATCH 22/29] fix timing Signed-off-by: Runji Wang --- src/executor/analyze.rs | 4 ++-- src/executor/nested_loop_join.rs | 5 +++++ src/utils/timed.rs | 23 +++++++++++------------ 3 files changed, 18 insertions(+), 14 deletions(-) diff --git a/src/executor/analyze.rs b/src/executor/analyze.rs index 4caf1bd0..c5a176d6 100644 --- a/src/executor/analyze.rs +++ b/src/executor/analyze.rs @@ -33,8 +33,8 @@ impl AnalyzeExecutor { metadata.push(("rows", format!("{total} = {rows:?}"))); } if let Some(time) = self.metrics.get_time(id) { - let total = time.iter().sum::(); - metadata.push(("time", format!("{total:?} = {time:?}"))); + let max = time.iter().max().unwrap(); + metadata.push(("time", format!("{max:?} = {time:?}"))); } metadata }; diff --git a/src/executor/nested_loop_join.rs b/src/executor/nested_loop_join.rs index 5f33c4da..a9ace38f 100644 --- a/src/executor/nested_loop_join.rs +++ b/src/executor/nested_loop_join.rs @@ -25,7 +25,12 @@ impl NestedLoopJoinExecutor { if !matches!(self.op, Expr::Inner | Expr::LeftOuter) { todo!("unsupported join type: {:?}", self.op); } + + // materialize left child let left_chunks = left_child.try_collect::>().await?; + if left_chunks.is_empty() { + return Ok(()); + } let left_rows = || left_chunks.iter().flat_map(|chunk| chunk.rows()); diff --git a/src/utils/timed.rs b/src/utils/timed.rs index 7c0589c3..b6790e0b 100644 --- a/src/utils/timed.rs +++ b/src/utils/timed.rs @@ -38,14 +38,13 @@ impl Stream for Timed { let this = self.project(); let _guard = this.span.as_ref().map(|s| s.enter()); - match this.inner.poll_next(cx) { - r @ Poll::Pending => r, - other => { - drop(_guard); - this.span.take(); - other - } + let result = this.inner.poll_next(cx); + if let Poll::Ready(None) = result { + // stream is finished + drop(_guard); + this.span.take(); } + result } #[inline] @@ -63,7 +62,7 @@ impl Debug for Span { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Span") .field("busy_time", &self.busy_time()) - .field("last_poll_time", &self.last_poll_time()) + .field("finish_time", &self.finish_time()) .finish() } } @@ -71,7 +70,7 @@ impl Debug for Span { #[derive(Debug, Default)] struct SpanInner { busy_time: Duration, - last_poll_time: Option, + finish_time: Option, } impl Span { @@ -86,8 +85,8 @@ impl Span { self.inner.lock().busy_time } - pub fn last_poll_time(&self) -> Option { - self.inner.lock().last_poll_time + pub fn finish_time(&self) -> Option { + self.inner.lock().finish_time } } @@ -101,6 +100,6 @@ impl Drop for Guard<'_> { let now = Instant::now(); let mut span = self.span.inner.lock(); span.busy_time += now - self.start_time; - span.last_poll_time = Some(now); + span.finish_time = Some(now); } } From 4ff94503e8f28266a7691fa2077a342d5489d071 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 19:00:43 +0800 Subject: [PATCH 23/29] add counted instrument Signed-off-by: Runji Wang --- src/executor/analyze.rs | 27 --------------- src/executor/mod.rs | 63 +++++++++++++--------------------- src/utils/counted.rs | 75 +++++++++++++++++++++++++++++++++++++++++ src/utils/mod.rs | 1 + 4 files changed, 99 insertions(+), 67 deletions(-) create mode 100644 src/utils/counted.rs diff --git a/src/executor/analyze.rs b/src/executor/analyze.rs index c5a176d6..f8a63768 100644 --- a/src/executor/analyze.rs +++ b/src/executor/analyze.rs @@ -1,8 +1,5 @@ // Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. -use std::fmt::{self, Debug}; -use std::sync::atomic::{AtomicU64, Ordering}; - use pretty_xmlish::PrettyConfig; use super::*; @@ -84,27 +81,3 @@ impl Metrics { .map(|rows| rows.iter().map(|counter| counter.get()).collect()) } } - -/// A counter. -#[derive(Default, Clone)] -pub struct Counter { - count: Arc, -} - -impl Debug for Counter { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}", self.get()) - } -} - -impl Counter { - /// Increments the counter. - pub fn inc(&self, value: u64) { - self.count.fetch_add(value, Ordering::Relaxed); - } - - /// Gets the current value of the counter. - pub fn get(&self) -> u64 { - self.count.load(Ordering::Relaxed) - } -} diff --git a/src/executor/mod.rs b/src/executor/mod.rs index cc875c38..b652520c 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -18,7 +18,6 @@ use std::time::Duration; use egg::{Id, Language}; use futures::stream::{BoxStream, StreamExt}; -use futures::Stream; use futures_async_stream::try_stream; use itertools::Itertools; @@ -59,6 +58,7 @@ use crate::catalog::{RootCatalog, RootCatalogRef, TableRefId}; use crate::planner::{Expr, ExprAnalysis, Optimizer, RecExpr, TypeSchemaAnalysis}; use crate::storage::Storage; use crate::types::{ColumnIndex, DataType}; +use crate::utils::counted::{Counter, StreamExt as _}; use crate::utils::timed::{Span as TimeSpan, StreamExt as _}; mod analyze; @@ -340,32 +340,29 @@ impl Builder { .execute(c) }), - Join([op, on, left, right]) => self - .build_id(left) - .spawn() // ends the pipeline of left side - .subscribe() - .zip(self.build_id(right)) - .map(|l, r| match self.node(op) { - Inner | LeftOuter | RightOuter | FullOuter => NestedLoopJoinExecutor { - op: self.node(op).clone(), - condition: self.resolve_column_index2(on, left, right), - left_types: self.plan_types(left).to_vec(), - right_types: self.plan_types(right).to_vec(), - } - .execute(l, r), - op @ Semi | op @ Anti => NestedLoopSemiJoinExecutor { - anti: matches!(op, Anti), - condition: self.resolve_column_index2(on, left, right), - left_types: self.plan_types(left).to_vec(), - } - .execute(l, r), - t => panic!("invalid join type: {t:?}"), - }), + Join([op, on, left, right]) => { + self.build_id(left) + .zip(self.build_id(right)) + .map(|l, r| match self.node(op) { + Inner | LeftOuter | RightOuter | FullOuter => NestedLoopJoinExecutor { + op: self.node(op).clone(), + condition: self.resolve_column_index2(on, left, right), + left_types: self.plan_types(left).to_vec(), + right_types: self.plan_types(right).to_vec(), + } + .execute(l, r), + op @ Semi | op @ Anti => NestedLoopSemiJoinExecutor { + anti: matches!(op, Anti), + condition: self.resolve_column_index2(on, left, right), + left_types: self.plan_types(left).to_vec(), + } + .execute(l, r), + t => panic!("invalid join type: {t:?}"), + }) + } HashJoin(args @ [op, _, _, _, left, right]) => self .build_id(left) - .spawn() // ends the pipeline of left side - .subscribe() .zip(self.build_id(right)) .map(|l, r| match self.node(op) { Inner => self.build_hashjoin::<{ JoinType::Inner }>(args, l, r), @@ -379,8 +376,6 @@ impl Builder { MergeJoin(args @ [op, _, _, _, left, right]) => self .build_id(left) - .spawn() // ends the pipeline of left side - .subscribe() .zip(self.build_id(right)) .map(|l, r| match self.node(op) { Inner => self.build_mergejoin::<{ JoinType::Inner }>(args, l, r), @@ -553,6 +548,7 @@ impl Builder { self.instrument(id, stream) } + /// Attaches metrics to the stream. fn instrument(&mut self, id: Id, stream: PartitionedStream) -> PartitionedStream { // let name = self.node(id).to_string(); let partitions = stream.streams.len(); @@ -562,26 +558,13 @@ impl Builder { self.metrics .register(id, spans.clone(), output_row_counters.clone()); - #[try_stream(boxed, ok = DataChunk, error = ExecutorError)] - async fn instrument( - stream: impl Stream> + Send + 'static, - output_row_counter: Counter, - ) { - #[for_await] - for chunk in stream { - let chunk = chunk?; - output_row_counter.inc(chunk.cardinality() as u64); - yield chunk; - } - } - PartitionedStream { streams: stream .streams .into_iter() .zip(spans) .zip(output_row_counters) - .map(|((stream, span), counter)| instrument(stream.timed(span), counter)) + .map(|((stream, span), counter)| stream.timed(span).counted(counter).boxed()) .collect(), } } diff --git a/src/utils/counted.rs b/src/utils/counted.rs new file mode 100644 index 00000000..f1a0b6a6 --- /dev/null +++ b/src/utils/counted.rs @@ -0,0 +1,75 @@ +use std::fmt::{self, Debug}; +use std::sync::atomic::{AtomicU64, Ordering}; +use std::sync::Arc; +use std::task::Poll; + +use futures::Stream; + +use crate::array::DataChunk; + +impl StreamExt for T {} + +/// An extension trait for `Streams` that provides counting instrument adapters. +pub trait StreamExt: Stream + Sized { + /// Binds a [`Counter`] to the [`Stream`] that counts the number of rows. + #[inline] + fn counted(self, counter: Counter) -> Counted { + Counted { + inner: self, + counter, + } + } +} + +/// Adapter for [`StreamExt::counted()`](StreamExt::counted). +#[pin_project::pin_project] +pub struct Counted { + #[pin] + inner: T, + counter: Counter, +} + +impl>> Stream for Counted { + type Item = T::Item; + + fn poll_next( + self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> Poll> { + let this = self.project(); + let result = this.inner.poll_next(cx); + if let Poll::Ready(Some(Ok(chunk))) = &result { + this.counter.inc(chunk.cardinality() as u64); + } + result + } + + #[inline] + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } +} + +/// A counter. +#[derive(Default, Clone)] +pub struct Counter { + count: Arc, +} + +impl Debug for Counter { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.get()) + } +} + +impl Counter { + /// Increments the counter. + pub fn inc(&self, value: u64) { + self.count.fetch_add(value, Ordering::Relaxed); + } + + /// Gets the current value of the counter. + pub fn get(&self) -> u64 { + self.count.load(Ordering::Relaxed) + } +} diff --git a/src/utils/mod.rs b/src/utils/mod.rs index 5c461a12..d327e9e0 100644 --- a/src/utils/mod.rs +++ b/src/utils/mod.rs @@ -1,4 +1,5 @@ // Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. +pub mod counted; pub mod time; pub mod timed; From a47bc49102f240bd45d04dbab0e7f875ec3c5f80 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 19:42:43 +0800 Subject: [PATCH 24/29] correctly show the time of exchange operator Signed-off-by: Runji Wang --- src/executor/analyze.rs | 11 +- src/executor/mod.rs | 219 ++++++++++++++++++++++++---------------- src/utils/counted.rs | 18 +++- 3 files changed, 154 insertions(+), 94 deletions(-) diff --git a/src/executor/analyze.rs b/src/executor/analyze.rs index f8a63768..6d462945 100644 --- a/src/executor/analyze.rs +++ b/src/executor/analyze.rs @@ -61,10 +61,13 @@ pub struct Metrics { } impl Metrics { - /// Register metrics for a node. - pub fn register(&mut self, id: Id, spans: Vec, rows: Vec) { - self.spans.insert(id, spans); - self.rows.insert(id, rows); + /// Create metrics for a node. + pub fn add(&mut self, id: Id, parallelism: usize) -> (Vec, Vec) { + let spans = (0..parallelism).map(|_| TimeSpan::default()).collect_vec(); + let counters = (0..parallelism).map(|_| Counter::default()).collect_vec(); + self.spans.insert(id, spans.clone()); + self.rows.insert(id, counters.clone()); + (spans, counters) } /// Get the running time for a node. diff --git a/src/executor/mod.rs b/src/executor/mod.rs index b652520c..6fa52a68 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -510,63 +510,45 @@ impl Builder { Exchange([dist, child]) => match self.node(dist).clone() { Single => self.build_id(child).spawn_merge().into(), - Broadcast => { - let subscriber = self.build_id(child).spawn(); - let num_partitions = self.optimizer.config().parallelism; - PartitionedStream { - streams: (0..num_partitions) - .map(|_| subscriber.subscribe_merge()) - .collect(), - } - } + Broadcast => self + .build_id(child) + .spawn_broadcast(self.optimizer.config().parallelism), Random => { + let stream = self.build_id(child); let num_partitions = self.optimizer.config().parallelism; - self.build_id(child) - .spawn_dispatch(num_partitions, |c| { + let (spans, counters) = self.metrics.add(id, stream.len()); + return stream + .dispatch(num_partitions, |c| { RandomPartitionProducer { num_partitions }.execute(c) }) - .subscribe() + .instrument(spans, counters) + .spawn() + .subscribe(); } Hash(keys) => { - let num_partitions = self.optimizer.config().parallelism; let keys = self.resolve_column_index(keys, child); - self.build_id(child) - .spawn_dispatch(num_partitions, |c| { + let num_partitions = self.optimizer.config().parallelism; + let stream = self.build_id(child); + let (spans, counters) = self.metrics.add(id, stream.len()); + return stream + .dispatch(num_partitions, |c| { HashPartitionProducer { keys: keys.clone(), num_partitions, } .execute(c) }) - .subscribe() + .instrument(spans, counters) + .spawn() + .subscribe(); } node => panic!("invalid exchange type: {node:?}"), }, node => panic!("not a plan: {node:?}\n{:?}", self.egraph.dump()), }; - self.instrument(id, stream) - } - - /// Attaches metrics to the stream. - fn instrument(&mut self, id: Id, stream: PartitionedStream) -> PartitionedStream { - // let name = self.node(id).to_string(); - let partitions = stream.streams.len(); - let spans = (0..partitions).map(|_| TimeSpan::default()).collect_vec(); - let output_row_counters = (0..partitions).map(|_| Counter::default()).collect_vec(); - - self.metrics - .register(id, spans.clone(), output_row_counters.clone()); - - PartitionedStream { - streams: stream - .streams - .into_iter() - .zip(spans) - .zip(output_row_counters) - .map(|((stream, span), counter)| stream.timed(span).counted(counter).boxed()) - .collect(), - } + let (spans, counters) = self.metrics.add(id, stream.len()); + stream.instrument(spans, counters) } fn build_hashjoin( @@ -649,48 +631,6 @@ fn spawn(mut stream: BoxedExecutor) -> StreamSubscriber { } } -/// Spawn new tasks to execute the given dispatchers. -/// Dispatch the output to multiple partitions by the associated partition index. -fn spawn_dispatchers( - streams: Vec, - num_partitions: usize, -) -> PartitionedStreamSubscriber { - let (txs, rxs): (Vec<_>, Vec<_>) = (0..num_partitions) - .map(|_| async_broadcast::broadcast(16)) - .unzip(); - let mut handles = Vec::with_capacity(streams.len()); - for mut stream in streams { - let txs = txs.clone(); - let handle = tokio::task::Builder::default() - .spawn(async move { - while let Some(item) = stream.next().await { - match item { - // send the chunk to the corresponding partition (ignore error) - Ok((chunk, partition)) => _ = txs[partition].broadcast(Ok(chunk)).await, - // broadcast the error to all partitions - Err(e) => { - for tx in txs.iter() { - tx.broadcast(Err(e.clone())).await.unwrap(); - } - } - } - } - }) - .expect("failed to spawn task"); - handles.push(handle); - } - let handles = Arc::new(handles); - PartitionedStreamSubscriber { - subscribers: rxs - .into_iter() - .map(|rx| StreamSubscriber { - rx: rx.deactivate(), - task_handle: handles.clone(), // all task handles are shared by all subscribers - }) - .collect(), - } -} - /// A set of partitioned output streams. struct PartitionedStream { streams: Vec, @@ -706,6 +646,11 @@ impl From for PartitionedStream { } impl PartitionedStream { + /// Returns the number of partitions. + fn len(&self) -> usize { + self.streams.len() + } + /// Merges the partitioned streams into a single stream. /// /// ```text @@ -717,6 +662,22 @@ impl PartitionedStream { futures::stream::select_all(self.spawn().subscribe().streams).boxed() } + /// Broadcasts each stream to `num_partitions` partitions. + /// + /// ```text + /// A0 -+-> A + /// A1 -+-> A + /// +-> A + /// ``` + fn spawn_broadcast(self, num_partitions: usize) -> PartitionedStream { + let subscriber = self.spawn(); + PartitionedStream { + streams: (0..num_partitions) + .map(|_| subscriber.subscribe_merge()) + .collect(), + } + } + /// Maps each stream with the given function. /// /// ```text @@ -730,6 +691,18 @@ impl PartitionedStream { } } + /// Dispatches each stream to `num_partitions` partitions with the given function. + fn dispatch( + self, + num_partitions: usize, + f: impl Fn(BoxedExecutor) -> BoxedDispatcher, + ) -> PartitionedDispatcher { + PartitionedDispatcher { + streams: self.streams.into_iter().map(f).collect(), + num_partitions, + } + } + /// Zips up two sets of partitioned streams. /// /// ```text @@ -755,14 +728,84 @@ impl PartitionedStream { } } - /// Spawns each partitioned stream and dispatches the output to `num_partitions` partitions. - /// Returns a set of subscribers of `num_partitions` partitions. - fn spawn_dispatch( - self, - num_partitions: usize, - f: impl Fn(BoxedExecutor) -> BoxedDispatcher, - ) -> PartitionedStreamSubscriber { - spawn_dispatchers(self.streams.into_iter().map(f).collect(), num_partitions) + /// Attaches metrics to the streams. + fn instrument(self, spans: Vec, counters: Vec) -> Self { + assert_eq!(self.streams.len(), spans.len()); + assert_eq!(self.streams.len(), counters.len()); + PartitionedStream { + streams: self + .streams + .into_iter() + .zip(spans) + .zip(counters) + .map(|((stream, span), counter)| stream.timed(span).counted(counter).boxed()) + .collect(), + } + } +} + +/// The return type of `PartitionedStream::dispatch`. +/// +/// This is the end of the pipeline. Call `spawn` to execute the streams and collect the results. +struct PartitionedDispatcher { + streams: Vec, + num_partitions: usize, +} + +impl PartitionedDispatcher { + /// Attaches metrics to the streams. + fn instrument(self, spans: Vec, counters: Vec) -> Self { + assert_eq!(self.streams.len(), spans.len()); + assert_eq!(self.streams.len(), counters.len()); + PartitionedDispatcher { + streams: self + .streams + .into_iter() + .zip(spans) + .zip(counters) + .map(|((stream, span), counter)| stream.timed(span).counted(counter).boxed()) + .collect(), + num_partitions: self.num_partitions, + } + } + + /// Spawn new tasks to execute the given dispatchers. + /// Dispatch the output to multiple partitions by the associated partition index. + fn spawn(self) -> PartitionedStreamSubscriber { + let (txs, rxs): (Vec<_>, Vec<_>) = (0..self.num_partitions) + .map(|_| async_broadcast::broadcast(16)) + .unzip(); + let mut handles = Vec::with_capacity(self.streams.len()); + for mut stream in self.streams { + let txs = txs.clone(); + let handle = tokio::task::Builder::default() + .spawn(async move { + while let Some(item) = stream.next().await { + match item { + // send the chunk to the corresponding partition (ignore error) + Ok((chunk, partition)) => _ = txs[partition].broadcast(Ok(chunk)).await, + // broadcast the error to all partitions + Err(e) => { + for tx in txs.iter() { + tx.broadcast(Err(e.clone())).await.unwrap(); + } + } + } + } + }) + .expect("failed to spawn task"); + handles.push(handle); + } + let handles = Arc::new(handles); + PartitionedStreamSubscriber { + subscribers: rxs + .into_iter() + .map(|rx| StreamSubscriber { + rx: rx.deactivate(), + task_handle: handles.clone(), // all task handles are shared by all subscribers + }) + .collect(), + } } } diff --git a/src/utils/counted.rs b/src/utils/counted.rs index f1a0b6a6..a97cd9d8 100644 --- a/src/utils/counted.rs +++ b/src/utils/counted.rs @@ -29,7 +29,7 @@ pub struct Counted { counter: Counter, } -impl>> Stream for Counted { +impl>> Stream for Counted { type Item = T::Item; fn poll_next( @@ -39,7 +39,7 @@ impl>> Stream for Counted { let this = self.project(); let result = this.inner.poll_next(cx); if let Poll::Ready(Some(Ok(chunk))) = &result { - this.counter.inc(chunk.cardinality() as u64); + this.counter.inc(chunk.as_data_chunk().cardinality() as u64); } result } @@ -50,6 +50,20 @@ impl>> Stream for Counted { } } +pub trait AsDataChunk { + fn as_data_chunk(&self) -> &DataChunk; +} +impl AsDataChunk for DataChunk { + fn as_data_chunk(&self) -> &DataChunk { + self + } +} +impl AsDataChunk for (DataChunk, usize) { + fn as_data_chunk(&self) -> &DataChunk { + &self.0 + } +} + /// A counter. #[derive(Default, Clone)] pub struct Counter { From cea7429fa7c44d76b6fe388e527e3baf805ebc42 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 20:08:10 +0800 Subject: [PATCH 25/29] use ahash to optimize hash Signed-off-by: Runji Wang --- src/executor/exchange.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/executor/exchange.rs b/src/executor/exchange.rs index 50c53fda..8ff22a67 100644 --- a/src/executor/exchange.rs +++ b/src/executor/exchange.rs @@ -1,7 +1,8 @@ // Copyright 2024 RisingLight Project Authors. Licensed under Apache-2.0. -use std::hash::{DefaultHasher, Hasher}; +use std::hash::Hasher; +use ahash::AHasher; use rand::{Rng, SeedableRng}; use super::*; @@ -19,7 +20,7 @@ impl HashPartitionProducer { #[try_stream(boxed, ok = (DataChunk, usize), error = ExecutorError)] pub async fn execute(self, child: BoxedExecutor) { // preallocate buffers for reuse - let mut hashers = vec![DefaultHasher::default(); PROCESSING_WINDOW_SIZE]; + let mut hashers = vec![AHasher::default(); PROCESSING_WINDOW_SIZE]; let mut partition_indices = vec![0; PROCESSING_WINDOW_SIZE]; let mut visibility = vec![false; PROCESSING_WINDOW_SIZE]; @@ -29,7 +30,7 @@ impl HashPartitionProducer { // reset buffers hashers.clear(); - hashers.resize(batch.cardinality(), DefaultHasher::default()); + hashers.resize(batch.cardinality(), AHasher::default()); partition_indices.resize(batch.cardinality(), 0); visibility.resize(batch.cardinality(), false); From 35c56fde0d0b47c4f6af5e969324d22d746c9897 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 20:21:43 +0800 Subject: [PATCH 26/29] decouple rows and time of exchange operator Signed-off-by: Runji Wang --- src/executor/analyze.rs | 11 ++++++++--- src/executor/mod.rs | 43 ++++++++++++++++++++++------------------- 2 files changed, 31 insertions(+), 23 deletions(-) diff --git a/src/executor/analyze.rs b/src/executor/analyze.rs index 6d462945..03a21095 100644 --- a/src/executor/analyze.rs +++ b/src/executor/analyze.rs @@ -62,9 +62,14 @@ pub struct Metrics { impl Metrics { /// Create metrics for a node. - pub fn add(&mut self, id: Id, parallelism: usize) -> (Vec, Vec) { - let spans = (0..parallelism).map(|_| TimeSpan::default()).collect_vec(); - let counters = (0..parallelism).map(|_| Counter::default()).collect_vec(); + pub fn add( + &mut self, + id: Id, + num_spans: usize, + num_counters: usize, + ) -> (Vec, Vec) { + let spans = (0..num_spans).map(|_| TimeSpan::default()).collect_vec(); + let counters = (0..num_counters).map(|_| Counter::default()).collect_vec(); self.spans.insert(id, spans.clone()); self.rows.insert(id, counters.clone()); (spans, counters) diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 6fa52a68..126665da 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -516,20 +516,21 @@ impl Builder { Random => { let stream = self.build_id(child); let num_partitions = self.optimizer.config().parallelism; - let (spans, counters) = self.metrics.add(id, stream.len()); + let (spans, counters) = self.metrics.add(id, stream.len(), num_partitions); return stream .dispatch(num_partitions, |c| { RandomPartitionProducer { num_partitions }.execute(c) }) - .instrument(spans, counters) + .instrument(spans) .spawn() - .subscribe(); + .subscribe() + .counted(counters); } Hash(keys) => { let keys = self.resolve_column_index(keys, child); let num_partitions = self.optimizer.config().parallelism; let stream = self.build_id(child); - let (spans, counters) = self.metrics.add(id, stream.len()); + let (spans, counters) = self.metrics.add(id, stream.len(), num_partitions); return stream .dispatch(num_partitions, |c| { HashPartitionProducer { @@ -538,16 +539,17 @@ impl Builder { } .execute(c) }) - .instrument(spans, counters) + .instrument(spans) .spawn() - .subscribe(); + .subscribe() + .counted(counters); } node => panic!("invalid exchange type: {node:?}"), }, node => panic!("not a plan: {node:?}\n{:?}", self.egraph.dump()), }; - let (spans, counters) = self.metrics.add(id, stream.len()); + let (spans, counters) = self.metrics.add(id, stream.len(), stream.len()); stream.instrument(spans, counters) } @@ -733,15 +735,21 @@ impl PartitionedStream { assert_eq!(self.streams.len(), spans.len()); assert_eq!(self.streams.len(), counters.len()); PartitionedStream { - streams: self - .streams - .into_iter() - .zip(spans) - .zip(counters) + streams: (self.streams.into_iter().zip(spans).zip(counters)) .map(|((stream, span), counter)| stream.timed(span).counted(counter).boxed()) .collect(), } } + + /// Attaches metrics to the streams. + fn counted(self, counters: Vec) -> Self { + assert_eq!(self.streams.len(), counters.len()); + PartitionedStream { + streams: (self.streams.into_iter().zip(counters)) + .map(|(stream, counter)| stream.counted(counter).boxed()) + .collect(), + } + } } /// The return type of `PartitionedStream::dispatch`. @@ -754,16 +762,11 @@ struct PartitionedDispatcher { impl PartitionedDispatcher { /// Attaches metrics to the streams. - fn instrument(self, spans: Vec, counters: Vec) -> Self { + fn instrument(self, spans: Vec) -> Self { assert_eq!(self.streams.len(), spans.len()); - assert_eq!(self.streams.len(), counters.len()); PartitionedDispatcher { - streams: self - .streams - .into_iter() - .zip(spans) - .zip(counters) - .map(|((stream, span), counter)| stream.timed(span).counted(counter).boxed()) + streams: (self.streams.into_iter().zip(spans)) + .map(|(stream, span)| stream.timed(span).boxed()) .collect(), num_partitions: self.num_partitions, } From 4a2d2aded4335d67e669010bf90db38921d5a63f Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 20:34:49 +0800 Subject: [PATCH 27/29] do not eliminate duplicate exchange Signed-off-by: Runji Wang --- src/planner/rules/partition.rs | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index 3bdcfc60..776b05d1 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -185,10 +185,6 @@ static TO_PARALLEL_RULES: LazyLock> = LazyLock::new(|| { "(exchange ?dist ?child)" => "?child" if partition_is_same("?child", "?dist") ), - rw!("dedup-exchange"; - "(exchange ?dist (exchange ?dist2 ?child))" => - "(exchange ?dist ?child)" - ), ] }); From 321e3305434501a2a535bb8aeb633018b0b24b0c Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 23:01:05 +0800 Subject: [PATCH 28/29] fix clippy Signed-off-by: Runji Wang --- src/array/ops.rs | 2 +- src/executor/exchange.rs | 2 +- src/executor/mod.rs | 4 ++-- src/planner/rules/mod.rs | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/array/ops.rs b/src/array/ops.rs index de2f6146..6c041bd2 100644 --- a/src/array/ops.rs +++ b/src/array/ops.rs @@ -206,7 +206,7 @@ impl ArrayImpl { } /// Hash the array into the given hasher. - pub fn hash(&self, hasher: &mut [impl Hasher]) { + pub fn hash_to(&self, hasher: &mut [impl Hasher]) { assert_eq!(hasher.len(), self.len()); match self { A::Null(a) => a.iter().zip(hasher).for_each(|(v, h)| v.hash(h)), diff --git a/src/executor/exchange.rs b/src/executor/exchange.rs index 8ff22a67..145c61db 100644 --- a/src/executor/exchange.rs +++ b/src/executor/exchange.rs @@ -37,7 +37,7 @@ impl HashPartitionProducer { // calculate the hash let keys_chunk = Evaluator::new(&self.keys).eval_list(&batch)?; for column in keys_chunk.arrays() { - column.hash(&mut hashers); + column.hash_to(&mut hashers); } for (hasher, target) in hashers.iter().zip(&mut partition_indices) { *target = hasher.finish() as usize % self.num_partitions; diff --git a/src/executor/mod.rs b/src/executor/mod.rs index 126665da..7d538203 100644 --- a/src/executor/mod.rs +++ b/src/executor/mod.rs @@ -789,7 +789,7 @@ impl PartitionedDispatcher { Ok((chunk, partition)) => _ = txs[partition].broadcast(Ok(chunk)).await, // broadcast the error to all partitions Err(e) => { - for tx in txs.iter() { + for tx in &txs { tx.broadcast(Err(e.clone())).await.unwrap(); } } @@ -826,7 +826,7 @@ impl ZippedPartitionedStream { streams: self .left .into_iter() - .zip(self.right.into_iter()) + .zip(self.right) .map(|(l, r)| f(l, r)) .collect(), } diff --git a/src/planner/rules/mod.rs b/src/planner/rules/mod.rs index 9e8497a3..ede94520 100644 --- a/src/planner/rules/mod.rs +++ b/src/planner/rules/mod.rs @@ -15,7 +15,7 @@ //! | [`type_`] | | data type | [`Type`] | //! | [`rows`] | | estimated rows | [`Rows`] | //! | [`order`] | merge join | ordered keys | [`OrderKey`] | -//! | [`partition`] | to_parallel | data partition | [`Partition`] | +//! | [`partition`] | to parallel plan | data partition | [`Partition`] | //! //! It would be best if you have a background in program analysis. //! Here is a recommended course: . From 1bc5611c05095d05116eec76808a5c7a976b5297 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 24 Nov 2024 23:04:59 +0800 Subject: [PATCH 29/29] fix unit test Signed-off-by: Runji Wang --- src/planner/rules/partition.rs | 12 +++++++----- tests/sql/merge_join.slt | 8 ++++---- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/planner/rules/partition.rs b/src/planner/rules/partition.rs index 776b05d1..aea2e622 100644 --- a/src/planner/rules/partition.rs +++ b/src/planner/rules/partition.rs @@ -346,7 +346,7 @@ mod tests { use super::*; #[test] - fn test_to_parallel() { + fn test_hash_join_to_parallel() { let input = " (hashjoin inner true (list a) (list b) (scan t1 (list a) true) @@ -355,10 +355,12 @@ mod tests { "; let distributed = " (hashjoin inner true (list a) (list b) - (exchange (hash (list a)) - (scan t1 (list a) true)) - (exchange (hash (list b)) - (scan t2 (list b) true)) + (exchange (hash (list a)) + (exchange random + (scan t1 (list a) true))) + (exchange (hash (list b)) + (exchange random + (scan t2 (list b) true))) ) "; let output = to_parallel_plan(input.parse().unwrap()); diff --git a/tests/sql/merge_join.slt b/tests/sql/merge_join.slt index 4eb543e1..b41df195 100644 --- a/tests/sql/merge_join.slt +++ b/tests/sql/merge_join.slt @@ -10,7 +10,7 @@ insert into t1 values (1, 10), (1, 11), (2, 20); statement ok insert into t2 values (1, -10), (1, -11), (3, -30); -query IIII +query IIII rowsort select * from (select a, b from t1 order by a, b) join (select c, d from t2 order by c, d desc) on a = c; @@ -20,7 +20,7 @@ join (select c, d from t2 order by c, d desc) on a = c; 1 11 1 -10 1 11 1 -11 -query IIII +query IIII rowsort select * from (select a, b from t1 order by a) left join (select c, d from t2 order by c) on a = c; @@ -31,7 +31,7 @@ left join (select c, d from t2 order by c) on a = c; 1 11 1 -11 2 20 NULL NULL -query IIII +query IIII rowsort select * from (select a, b from t1 order by a) right join (select c, d from t2 order by c) on a = c; @@ -42,7 +42,7 @@ right join (select c, d from t2 order by c) on a = c; 1 11 1 -11 NULL NULL 3 -30 -query IIII +query IIII rowsort select * from (select a, b from t1 order by a) full join (select c, d from t2 order by c) on a = c;