From a9fb58ca3aac8936d058b79fab4dc9ae4bd7edc2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 7 Feb 2025 13:48:28 -0500 Subject: [PATCH 01/12] Minor: Update changelog for `45.0.0` and tweak `CHANGELOG` docs (#14545) * Update release README instruction for publish ordering * Update changelog (#14460) --- dev/changelog/45.0.0.md | 10 ++++++++-- dev/release/README.md | 4 ++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/dev/changelog/45.0.0.md b/dev/changelog/45.0.0.md index 2303eee92a1d..ca905c0a1a56 100644 --- a/dev/changelog/45.0.0.md +++ b/dev/changelog/45.0.0.md @@ -19,7 +19,7 @@ under the License. # Apache DataFusion 45.0.0 Changelog -This release consists of 252 commits from 83 contributors. See credits at the end of this changelog for more information. +This release consists of 258 commits from 83 contributors. See credits at the end of this changelog for more information. **Breaking changes:** @@ -94,6 +94,7 @@ This release consists of 252 commits from 83 contributors. See credits at the en - Support arrays_overlap function (alias of `array_has_any`) [#14217](https://github.com/apache/datafusion/pull/14217) (erenavsarogullari) - chore: Adding commit activity badge [#14386](https://github.com/apache/datafusion/pull/14386) (comphead) - docs: Clarify join behavior in `DataFrame::join` [#14393](https://github.com/apache/datafusion/pull/14393) (rkrishn7) +- Prepare for `45.0.0` release: Version and Changelog [#14397](https://github.com/apache/datafusion/pull/14397) (alamb) **Other:** @@ -290,13 +291,18 @@ This release consists of 252 commits from 83 contributors. See credits at the en - FFI support for versions and alternate tokio runtimes [#13937](https://github.com/apache/datafusion/pull/13937) (timsaucer) - Do not rename struct fields when coercing types in `CASE` [#14384](https://github.com/apache/datafusion/pull/14384) (alamb) - Add `TableProvider::insert_into` into FFI Bindings [#14391](https://github.com/apache/datafusion/pull/14391) (davisp) +- [branch-45]: Backport chore: Upgrade to `arrow`/`parquet` `54.1.0` and fix clippy/ci (#14415) [#14453](https://github.com/apache/datafusion/pull/14453) (alamb) +- [release-45] Fix join type coercion (#14387) [#14454](https://github.com/apache/datafusion/pull/14454) (alamb) +- [branch-45] Support `Utf8View` to `numeric` coercion (#14377) [#14455](https://github.com/apache/datafusion/pull/14455) (alamb) +- [branch-45] Update REGEXP_MATCH scalar function to support Utf8View (#14449) [#14457](https://github.com/apache/datafusion/pull/14457) (alamb) +- [branch-45] Fix regression list Type Coercion List with inner type struct which has large/view types (#14385) [#14456](https://github.com/apache/datafusion/pull/14456) (alamb) ## Credits Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. ``` - 46 Andrew Lamb + 52 Andrew Lamb 22 Ian Lai 20 dependabot[bot] 8 Bruce Ritchie diff --git a/dev/release/README.md b/dev/release/README.md index a692708f5d2f..1672129c0ac1 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -275,21 +275,21 @@ Verify that the Cargo.toml in the tarball contains the correct version (cd datafusion/functions-aggregate-common && cargo publish) (cd datafusion/functions-window-common && cargo publish) (cd datafusion/doc && cargo publish) -(cd datafusion/macros && cargo publish) (cd datafusion/expr && cargo publish) +(cd datafusion/macros && cargo publish) (cd datafusion/execution && cargo publish) (cd datafusion/functions && cargo publish) (cd datafusion/physical-expr && cargo publish) (cd datafusion/functions-aggregate && cargo publish) (cd datafusion/functions-window && cargo publish) (cd datafusion/functions-nested && cargo publish) -(cd datafusion/functions-table && cargo publish) (cd datafusion/sql && cargo publish) (cd datafusion/optimizer && cargo publish) (cd datafusion/common-runtime && cargo publish) (cd datafusion/physical-plan && cargo publish) (cd datafusion/physical-optimizer && cargo publish) (cd datafusion/catalog && cargo publish) +(cd datafusion/functions-table && cargo publish) (cd datafusion/core && cargo publish) (cd datafusion/proto-common && cargo publish) (cd datafusion/proto && cargo publish) From c0d4ae7fcc654ec5625ee9b1cd7ce1312f294afc Mon Sep 17 00:00:00 2001 From: zjregee Date: Sat, 8 Feb 2025 02:49:05 +0800 Subject: [PATCH 02/12] polish MemoryStream related code (#14537) --- datafusion/physical-plan/src/memory.rs | 8 +++----- .../sqllogictest/test_files/string/dictionary_utf8.slt | 2 +- .../sqllogictest/test_files/string/large_string.slt | 2 +- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 5607ad9e2a9f..9d5d65d4e18a 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -726,7 +726,7 @@ pub struct MemoryStream { projection: Option>, /// Index into the data index: usize, - /// The remaining number of rows to return + /// The remaining number of rows to return. If None, all rows are returned fetch: Option, } @@ -778,11 +778,9 @@ impl Stream for MemoryStream { None => batch.clone(), }; - if self.fetch.is_none() { + let Some(&fetch) = self.fetch.as_ref() else { return Poll::Ready(Some(Ok(batch))); - } - - let fetch = self.fetch.unwrap(); + }; if fetch == 0 { return Poll::Ready(None); } diff --git a/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt b/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt index 2f12e9c7a39b..99271a28f950 100644 --- a/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt +++ b/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt @@ -67,4 +67,4 @@ statement ok drop table test_substr_base; statement ok -drop table test_datetime_base; \ No newline at end of file +drop table test_datetime_base; diff --git a/datafusion/sqllogictest/test_files/string/large_string.slt b/datafusion/sqllogictest/test_files/string/large_string.slt index 93ec796ec6f0..9126a80383ef 100644 --- a/datafusion/sqllogictest/test_files/string/large_string.slt +++ b/datafusion/sqllogictest/test_files/string/large_string.slt @@ -75,4 +75,4 @@ statement ok drop table test_substr_base; statement ok -drop table test_datetime_base; \ No newline at end of file +drop table test_datetime_base; From fc1835dbd1606cd7f49acbeac1b6e8fe330b6504 Mon Sep 17 00:00:00 2001 From: Ian Lai <108986288+Chen-Yuan-Lai@users.noreply.github.com> Date: Sat, 8 Feb 2025 02:53:13 +0800 Subject: [PATCH 03/12] refactor: switch BooleanBufferBuilder to NullBufferBuilder in MaybeNullBufferBuilder (#14504) * refactor: switch BooleanBufferBuilder to NullBufferBuilder in MaybeNullBufferBuilder * fix: cargo test failed * fix: typo --------- Co-authored-by: Ian Lai Co-authored-by: Andrew Lamb --- .../aggregates/group_values/null_builder.rs | 102 ++++++------------ 1 file changed, 30 insertions(+), 72 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs b/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs index 369d921d2fc8..23ffc69f218b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs @@ -15,120 +15,78 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::BooleanBufferBuilder; +use arrow::array::NullBufferBuilder; use arrow::buffer::NullBuffer; /// Builder for an (optional) null mask /// /// Optimized for avoid creating the bitmask when all values are non-null #[derive(Debug)] -pub(crate) enum MaybeNullBufferBuilder { - /// seen `row_count` rows but no nulls yet - NoNulls { row_count: usize }, - /// have at least one null value - /// +pub(crate) struct MaybeNullBufferBuilder { /// Note this is an Arrow *VALIDITY* buffer (so it is false for nulls, true /// for non-nulls) - Nulls(BooleanBufferBuilder), + nulls: NullBufferBuilder, } impl MaybeNullBufferBuilder { /// Create a new builder pub fn new() -> Self { - Self::NoNulls { row_count: 0 } + Self { + nulls: NullBufferBuilder::new(0), + } } /// Return true if the row at index `row` is null pub fn is_null(&self, row: usize) -> bool { - match self { - Self::NoNulls { .. } => false, + match self.nulls.as_slice() { // validity mask means a unset bit is NULL - Self::Nulls(builder) => !builder.get_bit(row), + Some(_) => !self.nulls.is_valid(row), + None => false, } } /// Set the nullness of the next row to `is_null` /// - /// num_values is the current length of the rows being tracked - /// /// If `value` is true, the row is null. /// If `value` is false, the row is non null pub fn append(&mut self, is_null: bool) { - match self { - Self::NoNulls { row_count } if is_null => { - // have seen no nulls so far, this is the first null, - // need to create the nulls buffer for all currently valid values - // alloc 2x the need given we push a new but immediately - let mut nulls = BooleanBufferBuilder::new(*row_count * 2); - nulls.append_n(*row_count, true); - nulls.append(false); - *self = Self::Nulls(nulls); - } - Self::NoNulls { row_count } => { - *row_count += 1; - } - Self::Nulls(builder) => builder.append(!is_null), - } + self.nulls.append(!is_null) } pub fn append_n(&mut self, n: usize, is_null: bool) { - match self { - Self::NoNulls { row_count } if is_null => { - // have seen no nulls so far, this is the first null, - // need to create the nulls buffer for all currently valid values - // alloc 2x the need given we push a new but immediately - let mut nulls = BooleanBufferBuilder::new(*row_count * 2); - nulls.append_n(*row_count, true); - nulls.append_n(n, false); - *self = Self::Nulls(nulls); - } - Self::NoNulls { row_count } => { - *row_count += n; - } - Self::Nulls(builder) => builder.append_n(n, !is_null), + if is_null { + self.nulls.append_n_nulls(n); + } else { + self.nulls.append_n_non_nulls(n); } } /// return the number of heap allocated bytes used by this structure to store boolean values pub fn allocated_size(&self) -> usize { - match self { - Self::NoNulls { .. } => 0, - // BooleanBufferBuilder builder::capacity returns capacity in bits (not bytes) - Self::Nulls(builder) => builder.capacity() / 8, - } + // NullBufferBuilder builder::allocated_size returns capacity in bits + self.nulls.allocated_size() / 8 } /// Return a NullBuffer representing the accumulated nulls so far - pub fn build(self) -> Option { - match self { - Self::NoNulls { .. } => None, - Self::Nulls(mut builder) => Some(NullBuffer::from(builder.finish())), - } + pub fn build(mut self) -> Option { + self.nulls.finish() } /// Returns a NullBuffer representing the first `n` rows accumulated so far /// shifting any remaining down by `n` pub fn take_n(&mut self, n: usize) -> Option { - match self { - Self::NoNulls { row_count } => { - *row_count -= n; - None - } - Self::Nulls(builder) => { - // Copy over the values at n..len-1 values to the start of a - // new builder and leave it in self - // - // TODO: it would be great to use something like `set_bits` from arrow here. - let mut new_builder = BooleanBufferBuilder::new(builder.len()); - for i in n..builder.len() { - new_builder.append(builder.get_bit(i)); - } - std::mem::swap(&mut new_builder, builder); - - // take only first n values from the original builder - new_builder.truncate(n); - Some(NullBuffer::from(new_builder.finish())) - } + // Copy over the values at n..len-1 values to the start of a + // new builder and leave it in self + // + // TODO: it would be great to use something like `set_bits` from arrow here. + let mut new_builder = NullBufferBuilder::new(self.nulls.len()); + for i in n..self.nulls.len() { + new_builder.append(self.nulls.is_valid(i)); } + std::mem::swap(&mut new_builder, &mut self.nulls); + + // take only first n values from the original builder + new_builder.truncate(n); + new_builder.finish() } } From 22fb5f7aaf7bbd0a1fca699a9066101df96b65d7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 7 Feb 2025 22:09:40 +0100 Subject: [PATCH 04/12] Allow constructing ScalarUDF from shared implementation (#14541) When composing or decorating functions, it's useful to operate on Impl trait objects (dyn ScalarUDFImpl) instead of the Expr-level adapter (ScalarUDF). However, before the change, it was impossible to construct `ScalarUDF` from `Arc`, even though the use of `Arc` is already part of the type API. This commit allows constructing the scalar from a shared impl object. For consistency, same change is applied for window functions and aggregations. --- datafusion/expr/src/udaf.rs | 9 ++++++--- datafusion/expr/src/udf.rs | 9 ++++++--- datafusion/expr/src/udwf.rs | 9 ++++++--- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 84a513b21b47..7ffc6623ea92 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -119,9 +119,12 @@ impl AggregateUDF { where F: AggregateUDFImpl + 'static, { - Self { - inner: Arc::new(fun), - } + Self::new_from_shared_impl(Arc::new(fun)) + } + + /// Create a new `AggregateUDF` from a `[AggregateUDFImpl]` trait object + pub fn new_from_shared_impl(fun: Arc) -> AggregateUDF { + Self { inner: fun } } /// Return the underlying [`AggregateUDFImpl`] trait object for this function diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index aa6a5cddad95..b41d97520362 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -91,9 +91,12 @@ impl ScalarUDF { where F: ScalarUDFImpl + 'static, { - Self { - inner: Arc::new(fun), - } + Self::new_from_shared_impl(Arc::new(fun)) + } + + /// Create a new `ScalarUDF` from a `[ScalarUDFImpl]` trait object + pub fn new_from_shared_impl(fun: Arc) -> ScalarUDF { + Self { inner: fun } } /// Return the underlying [`ScalarUDFImpl`] trait object for this function diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index 39e1e8f261a2..96929ffeb0ed 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -100,9 +100,12 @@ impl WindowUDF { where F: WindowUDFImpl + 'static, { - Self { - inner: Arc::new(fun), - } + Self::new_from_shared_impl(Arc::new(fun)) + } + + /// Create a new `WindowUDF` from a `[WindowUDFImpl]` trait object + pub fn new_from_shared_impl(fun: Arc) -> WindowUDF { + Self { inner: fun } } /// Return the underlying [`WindowUDFImpl`] trait object for this function From d5f19f3b67a53319152233b681b6f7308c195d28 Mon Sep 17 00:00:00 2001 From: Elia Perantoni Date: Fri, 7 Feb 2025 23:16:22 +0100 Subject: [PATCH 05/12] Add `DataFusionError::Collection` to return multiple `DataFusionError`s (#14439) * feat: collect multiple errors when possible * chore: format * doc: explain implementation of `DataFusionError::{message,source}` * feat: update cli lockfile * doc: explain `DataFusionError::iter` * chore: cargo fmt --all * fix: test * chore: update slt files * fix: tests * fix: strip backtrace in slt tests * feat: pr feedback * chore: fix clippy * fix: test * update expected * more update * revert test in errors * try again --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/error.rs | 111 +++++++++++++++++- .../expr/src/type_coercion/functions.rs | 15 ++- datafusion/sql/src/select.rs | 14 ++- datafusion/sql/src/set_expr.rs | 20 +++- datafusion/sql/tests/cases/collection.rs | 59 ++++++++++ datafusion/sql/tests/cases/mod.rs | 1 + datafusion/sql/tests/sql_integration.rs | 15 ++- .../src/engines/datafusion_engine/error.rs | 2 +- datafusion/sqllogictest/test_files/errors.slt | 2 +- .../sqllogictest/test_files/functions.slt | 12 +- .../sqllogictest/test_files/identifiers.slt | 8 +- datafusion/sqllogictest/test_files/select.slt | 6 +- 12 files changed, 234 insertions(+), 31 deletions(-) create mode 100644 datafusion/sql/tests/cases/collection.rs diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index af3a774e06bd..013b1d5a2cab 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -20,6 +20,7 @@ use std::backtrace::{Backtrace, BacktraceStatus}; use std::borrow::Cow; +use std::collections::VecDeque; use std::error::Error; use std::fmt::{Display, Formatter}; use std::io; @@ -136,6 +137,13 @@ pub enum DataFusionError { /// human-readable messages, and locations in the source query that relate /// to the error in some way. Diagnostic(Box, Box), + /// A collection of one or more [`DataFusionError`]. Useful in cases where + /// DataFusion can recover from an erroneous state, and produce more errors + /// before terminating. e.g. when planning a SELECT clause, DataFusion can + /// synchronize to the next `SelectItem` if the previous one had errors. The + /// end result is that the user can see errors about all `SelectItem`, + /// instead of just the first one. + Collection(Vec), /// A [`DataFusionError`] which shares an underlying [`DataFusionError`]. /// /// This is useful when the same underlying [`DataFusionError`] is passed @@ -360,6 +368,14 @@ impl Error for DataFusionError { DataFusionError::Context(_, e) => Some(e.as_ref()), DataFusionError::Substrait(_) => None, DataFusionError::Diagnostic(_, e) => Some(e.as_ref()), + // Can't really make a Collection fit into the mold of "an error has + // at most one source", but returning the first one is probably good + // idea. Especially since `DataFusionError::Collection` is mostly + // meant for consumption by the end user, so shouldn't interfere + // with programmatic usage too much. Plus, having 1 or 5 errors + // doesn't really change the fact that the query is invalid and + // can't be executed. + DataFusionError::Collection(errs) => errs.first().map(|e| e as &dyn Error), DataFusionError::Shared(e) => Some(e.as_ref()), } } @@ -463,18 +479,27 @@ impl DataFusionError { DataFusionError::ObjectStore(_) => "Object Store error: ", DataFusionError::IoError(_) => "IO error: ", DataFusionError::SQL(_, _) => "SQL error: ", - DataFusionError::NotImplemented(_) => "This feature is not implemented: ", + DataFusionError::NotImplemented(_) => { + "This feature is not implemented: " + } DataFusionError::Internal(_) => "Internal error: ", DataFusionError::Plan(_) => "Error during planning: ", - DataFusionError::Configuration(_) => "Invalid or Unsupported Configuration: ", + DataFusionError::Configuration(_) => { + "Invalid or Unsupported Configuration: " + } DataFusionError::SchemaError(_, _) => "Schema error: ", DataFusionError::Execution(_) => "Execution error: ", DataFusionError::ExecutionJoin(_) => "ExecutionJoin error: ", - DataFusionError::ResourcesExhausted(_) => "Resources exhausted: ", + DataFusionError::ResourcesExhausted(_) => { + "Resources exhausted: " + } DataFusionError::External(_) => "External error: ", DataFusionError::Context(_, _) => "", DataFusionError::Substrait(_) => "Substrait error: ", DataFusionError::Diagnostic(_, _) => "", + DataFusionError::Collection(errs) => { + errs.first().expect("cannot construct DataFusionError::Collection with 0 errors, but got one such case").error_prefix() + } DataFusionError::Shared(_) => "", } } @@ -517,6 +542,13 @@ impl DataFusionError { } DataFusionError::Substrait(ref desc) => Cow::Owned(desc.to_string()), DataFusionError::Diagnostic(_, ref err) => Cow::Owned(err.to_string()), + // Returning the message of the first error is probably fine enough, + // and makes `DataFusionError::Collection` a transparent wrapped, + // unless the end user explicitly calls `DataFusionError::iter`. + DataFusionError::Collection(ref errs) => errs + .first() + .expect("cannot construct DataFusionError::Collection with 0 errors") + .message(), DataFusionError::Shared(ref desc) => Cow::Owned(desc.to_string()), } } @@ -569,6 +601,63 @@ impl DataFusionError { DiagnosticsIterator { head: self }.next() } + + /// Sometimes DataFusion is able to collect multiple errors in a SQL query + /// before terminating, e.g. across different expressions in a SELECT + /// statements or different sides of a UNION. This method returns an + /// iterator over all the errors in the collection. + /// + /// For this to work, the top-level error must be a + /// `DataFusionError::Collection`, not something that contains it. + pub fn iter(&self) -> impl Iterator { + struct ErrorIterator<'a> { + queue: VecDeque<&'a DataFusionError>, + } + + impl<'a> Iterator for ErrorIterator<'a> { + type Item = &'a DataFusionError; + + fn next(&mut self) -> Option { + loop { + let popped = self.queue.pop_front()?; + match popped { + DataFusionError::Collection(errs) => self.queue.extend(errs), + _ => return Some(popped), + } + } + } + } + + let mut queue = VecDeque::new(); + queue.push_back(self); + ErrorIterator { queue } + } +} + +pub struct DataFusionErrorBuilder(Vec); + +impl DataFusionErrorBuilder { + pub fn new() -> Self { + Self(Vec::new()) + } + + pub fn add_error(&mut self, error: DataFusionError) { + self.0.push(error); + } + + pub fn error_or(self, ok: T) -> Result { + match self.0.len() { + 0 => Ok(ok), + 1 => Err(self.0.into_iter().next().expect("length matched 1")), + _ => Err(DataFusionError::Collection(self.0)), + } + } +} + +impl Default for DataFusionErrorBuilder { + fn default() -> Self { + Self::new() + } } /// Unwrap an `Option` if possible. Otherwise return an `DataFusionError::Internal`. @@ -954,4 +1043,20 @@ mod test { assert_eq!(e.strip_backtrace(), exp.strip_backtrace()); assert_eq!(std::mem::discriminant(e), std::mem::discriminant(&exp),) } + + #[test] + fn test_iter() { + let err = DataFusionError::Collection(vec![ + DataFusionError::Plan("a".to_string()), + DataFusionError::Collection(vec![ + DataFusionError::Plan("b".to_string()), + DataFusionError::Plan("c".to_string()), + ]), + ]); + let errs = err.iter().collect::>(); + assert_eq!(errs.len(), 3); + assert_eq!(errs[0].strip_backtrace(), "Error during planning: a"); + assert_eq!(errs[1].strip_backtrace(), "Error during planning: b"); + assert_eq!(errs[2].strip_backtrace(), "Error during planning: c"); + } } diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 0f9dbec722c2..7ac836ef3aeb 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -261,8 +261,9 @@ fn get_valid_types_with_scalar_udf( TypeSignature::UserDefined => match func.coerce_types(current_types) { Ok(coerced_types) => Ok(vec![coerced_types]), Err(e) => exec_err!( - "Function '{}' user-defined coercion failed with {e:?}", - func.name() + "Function '{}' user-defined coercion failed with {:?}", + func.name(), + e.strip_backtrace() ), }, TypeSignature::OneOf(signatures) => { @@ -304,8 +305,9 @@ fn get_valid_types_with_aggregate_udf( Ok(coerced_types) => vec![coerced_types], Err(e) => { return exec_err!( - "Function '{}' user-defined coercion failed with {e:?}", - func.name() + "Function '{}' user-defined coercion failed with {:?}", + func.name(), + e.strip_backtrace() ) } }, @@ -332,8 +334,9 @@ fn get_valid_types_with_window_udf( Ok(coerced_types) => vec![coerced_types], Err(e) => { return exec_err!( - "Function '{}' user-defined coercion failed with {e:?}", - func.name() + "Function '{}' user-defined coercion failed with {:?}", + func.name(), + e.strip_backtrace() ) } }, diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 50d89dea6763..05782e6ecd75 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -25,6 +25,7 @@ use crate::utils::{ CheckColumnsSatisfyExprsPurpose, }; +use datafusion_common::error::DataFusionErrorBuilder; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion_common::{not_impl_err, plan_err, Result}; use datafusion_common::{RecursionUnnestOption, UnnestOptions}; @@ -574,10 +575,15 @@ impl SqlToRel<'_, S> { empty_from: bool, planner_context: &mut PlannerContext, ) -> Result> { - projection - .into_iter() - .map(|expr| self.sql_select_to_rex(expr, plan, empty_from, planner_context)) - .collect::>>() + let mut prepared_select_exprs = vec![]; + let mut error_builder = DataFusionErrorBuilder::new(); + for expr in projection { + match self.sql_select_to_rex(expr, plan, empty_from, planner_context) { + Ok(expr) => prepared_select_exprs.push(expr), + Err(err) => error_builder.add_error(err), + } + } + error_builder.error_or(prepared_select_exprs) } /// Generate a relational expression from a select SQL expression diff --git a/datafusion/sql/src/set_expr.rs b/datafusion/sql/src/set_expr.rs index 3ddbe373ecd3..2579f2397228 100644 --- a/datafusion/sql/src/set_expr.rs +++ b/datafusion/sql/src/set_expr.rs @@ -16,7 +16,9 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::{not_impl_err, plan_err, Diagnostic, Result, Span}; +use datafusion_common::{ + not_impl_err, plan_err, DataFusionError, Diagnostic, Result, Span, +}; use datafusion_expr::{LogicalPlan, LogicalPlanBuilder}; use sqlparser::ast::{SetExpr, SetOperator, SetQuantifier, Spanned}; @@ -39,8 +41,19 @@ impl SqlToRel<'_, S> { } => { let left_span = Span::try_from_sqlparser_span(left.span()); let right_span = Span::try_from_sqlparser_span(right.span()); - let left_plan = self.set_expr_to_plan(*left, planner_context)?; - let right_plan = self.set_expr_to_plan(*right, planner_context)?; + let left_plan = self.set_expr_to_plan(*left, planner_context); + let right_plan = self.set_expr_to_plan(*right, planner_context); + let (left_plan, right_plan) = match (left_plan, right_plan) { + (Ok(left_plan), Ok(right_plan)) => (left_plan, right_plan), + (Err(left_err), Err(right_err)) => { + return Err(DataFusionError::Collection(vec![ + left_err, right_err, + ])); + } + (Err(err), _) | (_, Err(err)) => { + return Err(err); + } + }; self.validate_set_expr_num_of_columns( op, left_span, @@ -49,6 +62,7 @@ impl SqlToRel<'_, S> { &right_plan, set_expr_span, )?; + self.set_operation_to_plan(op, left_plan, right_plan, set_quantifier) } SetExpr::Query(q) => self.query_to_plan(*q, planner_context), diff --git a/datafusion/sql/tests/cases/collection.rs b/datafusion/sql/tests/cases/collection.rs new file mode 100644 index 000000000000..59704d6445b3 --- /dev/null +++ b/datafusion/sql/tests/cases/collection.rs @@ -0,0 +1,59 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use datafusion_common::{assert_contains, DataFusionError}; +use datafusion_sql::planner::SqlToRel; +use sqlparser::{dialect::GenericDialect, parser::Parser}; + +use crate::{MockContextProvider, MockSessionState}; + +fn do_query(sql: &'static str) -> DataFusionError { + let dialect = GenericDialect {}; + let statement = Parser::new(&dialect) + .try_with_sql(sql) + .expect("unable to create parser") + .parse_statement() + .expect("unable to parse query"); + let state = MockSessionState::default(); + let context = MockContextProvider { state }; + let sql_to_rel = SqlToRel::new(&context); + sql_to_rel + .sql_statement_to_plan(statement) + .expect_err("expected error") +} + +#[test] +fn test_collect_select_items() { + let query = "SELECT first_namex, last_namex FROM person"; + let error = do_query(query); + let errors = error.iter().collect::>(); + assert_eq!(errors.len(), 2); + assert!(errors[0] + .to_string() + .contains("No field named first_namex.")); + assert_contains!(errors[1].to_string(), "No field named last_namex."); +} + +#[test] +fn test_collect_set_exprs() { + let query = "SELECT first_namex FROM person UNION ALL SELECT last_namex FROM person"; + let error = do_query(query); + let errors = error.iter().collect::>(); + assert_eq!(errors.len(), 2); + assert_contains!(errors[0].to_string(), "No field named first_namex."); + assert_contains!(errors[1].to_string(), "No field named last_namex."); +} diff --git a/datafusion/sql/tests/cases/mod.rs b/datafusion/sql/tests/cases/mod.rs index 48574984c0ca..b3eedcdc41e3 100644 --- a/datafusion/sql/tests/cases/mod.rs +++ b/datafusion/sql/tests/cases/mod.rs @@ -15,5 +15,6 @@ // specific language governing permissions and limitations // under the License. +mod collection; mod diagnostic; mod plan_to_sql; diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 22af160f723f..c514458d4a27 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -4422,10 +4422,17 @@ fn plan_create_index() { } } -fn assert_field_not_found(err: DataFusionError, name: &str) { - let err = match err { - DataFusionError::Diagnostic(_, err) => *err, - err => err, +fn assert_field_not_found(mut err: DataFusionError, name: &str) { + let err = loop { + match err { + DataFusionError::Diagnostic(_, wrapped_err) => { + err = *wrapped_err; + } + DataFusionError::Collection(errs) => { + err = errs.into_iter().next().unwrap(); + } + err => break err, + } }; match err { DataFusionError::SchemaError { .. } => { diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/error.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/error.rs index 5bb40aca2ab8..ae56c0260564 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/error.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/error.rs @@ -30,7 +30,7 @@ pub enum DFSqlLogicTestError { #[error("SqlLogicTest error(from sqllogictest-rs crate): {0}")] SqlLogicTest(#[from] TestError), /// Error from datafusion - #[error("DataFusion error: {0}")] + #[error("DataFusion error: {}", .0.strip_backtrace())] DataFusion(#[from] DataFusionError), /// Error returned when SQL is syntactically incorrect. #[error("SQL Parser error: {0}")] diff --git a/datafusion/sqllogictest/test_files/errors.slt b/datafusion/sqllogictest/test_files/errors.slt index 7e2af5b9cbc9..5a94ba9c0583 100644 --- a/datafusion/sqllogictest/test_files/errors.slt +++ b/datafusion/sqllogictest/test_files/errors.slt @@ -148,7 +148,7 @@ SELECT query error DataFusion error: Arrow error: Cast error: Cannot cast string 'foo' to value of Int64 type create table foo as values (1), ('foo'); -query error No function matches +query error user-defined coercion failed select 1 group by substr(''); # Error in filter should be reported diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index 838387965e6a..b9699dfd5c06 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -267,16 +267,24 @@ statement error Did you mean 'to_timestamp_seconds'? SELECT to_TIMESTAMPS_second(v2) from test; # Aggregate function -query error DataFusion error: Error during planning: Invalid function 'counter' +query error SELECT counter(*) from test; +---- +DataFusion error: Error during planning: Invalid function 'counter'. +Did you mean 'count'? + # Aggregate function statement error Did you mean 'stddev'? SELECT STDEV(v1) from test; # Aggregate function -statement error DataFusion error: Error during planning: Invalid function 'covaria'.\nDid you mean 'covar'? +statement error SELECT COVARIA(1,1); +---- +DataFusion error: Error during planning: Invalid function 'covaria'. +Did you mean 'covar'? + # Window function statement error diff --git a/datafusion/sqllogictest/test_files/identifiers.slt b/datafusion/sqllogictest/test_files/identifiers.slt index f1b2777bece0..755d617e7a2a 100644 --- a/datafusion/sqllogictest/test_files/identifiers.slt +++ b/datafusion/sqllogictest/test_files/identifiers.slt @@ -90,16 +90,16 @@ drop table case_insensitive_test statement ok CREATE TABLE test("Column1" string) AS VALUES ('content1'); -statement error DataFusion error: Schema error: No field named column1. Valid fields are test\."Column1"\. +statement error DataFusion error: Schema error: No field named column1\. Valid fields are test\."Column1"\. SELECT COLumn1 from test -statement error DataFusion error: Schema error: No field named column1. Valid fields are test\."Column1"\. +statement error DataFusion error: Schema error: No field named column1\. Valid fields are test\."Column1"\. SELECT Column1 from test -statement error DataFusion error: Schema error: No field named column1. Valid fields are test\."Column1"\. +statement error DataFusion error: Schema error: No field named column1\. Valid fields are test\."Column1"\. SELECT column1 from test -statement error DataFusion error: Schema error: No field named column1. Valid fields are test\."Column1"\. +statement error DataFusion error: Schema error: No field named column1\. Valid fields are test\."Column1"\. SELECT "column1" from test statement ok diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 0650f978d3a8..e12bdca37e6f 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1156,7 +1156,7 @@ SELECT * FROM empty_table statement ok CREATE TABLE case_sensitive_table("INT32" int) AS VALUES (1), (2), (3), (4), (5); -statement error DataFusion error: Schema error: No field named int32. Valid fields are case_sensitive_table."INT32". +statement error DataFusion error: Schema error: No field named int32\. Valid fields are case_sensitive_table\."INT32"\. select "int32" from case_sensitive_table query I @@ -1799,7 +1799,7 @@ select a + b from (select 1 as a, 2 as b, 1 as "a + b"); 3 # Can't reference an output column by expression over projection. -query error DataFusion error: Schema error: No field named a. Valid fields are "a \+ Int64\(1\)"\. +query error DataFusion error: Schema error: No field named a\. Valid fields are "a \+ Int64\(1\)"\. select a + 1 from (select a+1 from (select 1 as a)); query I @@ -1834,7 +1834,7 @@ statement ok DROP TABLE test; # Can't reference an unqualified column by a qualified name -query error DataFusion error: Schema error: No field named t1.v1. Column names are case sensitive. You can use double quotes to refer to the "t1.v1" column or set the datafusion.sql_parser.enable_ident_normalization configuration. Valid fields are "t1.v1". +query error DataFusion error: Schema error: No field named t1\.v1\. Column names are case sensitive\. You can use double quotes to refer to the "t1\.v1" column or set the datafusion\.sql_parser\.enable_ident_normalization configuration\. Valid fields are "t1\.v1"\. SELECT t1.v1 FROM (SELECT 1 AS "t1.v1"); # Test issue: https://github.com/apache/datafusion/issues/14124 From 7ccc6d7c55ae9dbcb7dee031f394bf11a03000ba Mon Sep 17 00:00:00 2001 From: logan-keede <68557630+logan-keede@users.noreply.github.com> Date: Sat, 8 Feb 2025 03:58:44 +0530 Subject: [PATCH 06/12] some dependency removals and setup for refactor of `FileScanConfig` (#14543) * some dependency removals and setup for refactor * fix: CI for linux build * move FileGroupsPartitioner * remove old * Fix supports_repartition * fix --------- Co-authored-by: Andrew Lamb --- .../examples/custom_file_format.rs | 16 ++++++++------- .../src}/file_groups.rs | 2 +- datafusion/catalog-listing/src/mod.rs | 2 +- datafusion/core/src/datasource/data_source.rs | 5 +++++ .../core/src/datasource/file_format/arrow.rs | 12 +++++------ .../core/src/datasource/file_format/avro.rs | 12 +++++------ .../core/src/datasource/file_format/csv.rs | 16 ++++++++------- .../core/src/datasource/file_format/json.rs | 16 ++++++++------- .../core/src/datasource/file_format/mod.rs | 14 ++++++------- .../src/datasource/file_format/parquet.rs | 19 ++++++++++-------- .../core/src/datasource/listing/table.rs | 20 +++++++++---------- .../datasource/physical_plan/arrow_file.rs | 4 ++++ .../core/src/datasource/physical_plan/avro.rs | 5 +++++ .../core/src/datasource/physical_plan/csv.rs | 3 +++ .../physical_plan/file_scan_config.rs | 12 +++-------- .../core/src/datasource/physical_plan/json.rs | 4 ++++ .../core/src/datasource/physical_plan/mod.rs | 5 +---- .../physical_plan/parquet/source.rs | 3 +++ 18 files changed, 97 insertions(+), 73 deletions(-) rename datafusion/{core/src/datasource/physical_plan => catalog-listing/src}/file_groups.rs (99%) diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index 4d85ce882923..c44210e55318 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -21,11 +21,14 @@ use arrow::{ array::{AsArray, RecordBatch, StringArray, UInt8Array}, datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type}, }; -use datafusion::common::{GetExt, Statistics}; use datafusion::datasource::data_source::FileSource; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::physical_expr::LexRequirement; use datafusion::physical_expr::PhysicalExpr; +use datafusion::{ + catalog::Session, + common::{GetExt, Statistics}, +}; use datafusion::{ datasource::{ file_format::{ @@ -36,7 +39,6 @@ use datafusion::{ MemTable, }, error::Result, - execution::context::SessionState, physical_plan::ExecutionPlan, prelude::SessionContext, }; @@ -84,7 +86,7 @@ impl FileFormat for TSVFileFormat { async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -95,7 +97,7 @@ impl FileFormat for TSVFileFormat { async fn infer_stats( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, table_schema: SchemaRef, object: &ObjectMeta, @@ -107,7 +109,7 @@ impl FileFormat for TSVFileFormat { async fn create_physical_plan( &self, - state: &SessionState, + state: &dyn Session, conf: FileScanConfig, filters: Option<&Arc>, ) -> Result> { @@ -119,7 +121,7 @@ impl FileFormat for TSVFileFormat { async fn create_writer_physical_plan( &self, input: Arc, - state: &SessionState, + state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { @@ -153,7 +155,7 @@ impl TSVFileFactory { impl FileFormatFactory for TSVFileFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &std::collections::HashMap, ) -> Result> { let mut new_options = format_options.clone(); diff --git a/datafusion/core/src/datasource/physical_plan/file_groups.rs b/datafusion/catalog-listing/src/file_groups.rs similarity index 99% rename from datafusion/core/src/datasource/physical_plan/file_groups.rs rename to datafusion/catalog-listing/src/file_groups.rs index f681dfe219b5..2c2b791f2365 100644 --- a/datafusion/core/src/datasource/physical_plan/file_groups.rs +++ b/datafusion/catalog-listing/src/file_groups.rs @@ -17,7 +17,7 @@ //! Logic for managing groups of [`PartitionedFile`]s in DataFusion -use crate::datasource::listing::{FileRange, PartitionedFile}; +use crate::{FileRange, PartitionedFile}; use itertools::Itertools; use std::cmp::min; use std::collections::BinaryHeap; diff --git a/datafusion/catalog-listing/src/mod.rs b/datafusion/catalog-listing/src/mod.rs index e952e39fd479..709fa88b5867 100644 --- a/datafusion/catalog-listing/src/mod.rs +++ b/datafusion/catalog-listing/src/mod.rs @@ -18,9 +18,9 @@ //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. +pub mod file_groups; pub mod helpers; pub mod url; - use chrono::TimeZone; use datafusion_common::Result; use datafusion_common::{ScalarValue, Statistics}; diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 3984ffa64c76..03bfb4175022 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -62,4 +62,9 @@ pub trait FileSource: Send + Sync { fn fmt_extra(&self, _t: DisplayFormatType, _f: &mut Formatter) -> fmt::Result { Ok(()) } + /// Return true if the file format supports repartition + /// + /// If this returns true, the DataSourceExec may repartition the data + /// by breaking up the input files into multiple smaller groups. + fn supports_repartition(&self, config: &FileScanConfig) -> bool; } diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index e88144c2ea16..5a4bf103e7ce 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -35,7 +35,6 @@ use crate::datasource::physical_plan::{ ArrowSource, FileGroupDisplay, FileScanConfig, FileSink, FileSinkConfig, }; use crate::error::Result; -use crate::execution::context::SessionState; use crate::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use arrow::ipc::convert::fb_to_schema; @@ -43,6 +42,7 @@ use arrow::ipc::reader::FileReader; use arrow::ipc::writer::IpcWriteOptions; use arrow::ipc::{root_as_message, CompressionType}; use arrow_schema::{ArrowError, Schema, SchemaRef}; +use datafusion_catalog::Session; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::{ not_impl_err, DataFusionError, GetExt, Statistics, DEFAULT_ARROW_EXTENSION, @@ -84,7 +84,7 @@ impl ArrowFormatFactory { impl FileFormatFactory for ArrowFormatFactory { fn create( &self, - _state: &SessionState, + _state: &dyn Session, _format_options: &HashMap, ) -> Result> { Ok(Arc::new(ArrowFormat)) @@ -135,7 +135,7 @@ impl FileFormat for ArrowFormat { async fn infer_schema( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -159,7 +159,7 @@ impl FileFormat for ArrowFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -169,7 +169,7 @@ impl FileFormat for ArrowFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -180,7 +180,7 @@ impl FileFormat for ArrowFormat { async fn create_writer_physical_plan( &self, input: Arc, - _state: &SessionState, + _state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index 8e5192d12f5d..100aa4fd51e2 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -28,7 +28,6 @@ use super::FileFormatFactory; use crate::datasource::avro_to_arrow::read_avro_schema_from_reader; use crate::datasource::physical_plan::{AvroSource, FileScanConfig}; use crate::error::Result; -use crate::execution::context::SessionState; use crate::physical_plan::ExecutionPlan; use crate::physical_plan::Statistics; @@ -36,6 +35,7 @@ use crate::datasource::data_source::FileSource; use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use async_trait::async_trait; +use datafusion_catalog::Session; use datafusion_common::internal_err; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::GetExt; @@ -57,7 +57,7 @@ impl AvroFormatFactory { impl FileFormatFactory for AvroFormatFactory { fn create( &self, - _state: &SessionState, + _state: &dyn Session, _format_options: &HashMap, ) -> Result> { Ok(Arc::new(AvroFormat)) @@ -112,7 +112,7 @@ impl FileFormat for AvroFormat { async fn infer_schema( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -137,7 +137,7 @@ impl FileFormat for AvroFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -147,7 +147,7 @@ impl FileFormat for AvroFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -505,7 +505,7 @@ mod tests { } async fn get_exec( - state: &SessionState, + state: &dyn Session, file_name: &str, projection: Option>, limit: Option, diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 89df3a70963f..eab3494be026 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -44,6 +44,7 @@ use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow_schema::ArrowError; +use datafusion_catalog::Session; use datafusion_common::config::{ConfigField, ConfigFileType, CsvOptions}; use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::{ @@ -95,9 +96,10 @@ impl Debug for CsvFormatFactory { impl FileFormatFactory for CsvFormatFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &HashMap, ) -> Result> { + let state = state.as_any().downcast_ref::().unwrap(); let csv_options = match &self.options { None => { let mut table_options = state.default_table_options(); @@ -365,7 +367,7 @@ impl FileFormat for CsvFormat { async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -400,7 +402,7 @@ impl FileFormat for CsvFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -410,7 +412,7 @@ impl FileFormat for CsvFormat { async fn create_physical_plan( &self, - state: &SessionState, + state: &dyn Session, mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -440,7 +442,7 @@ impl FileFormat for CsvFormat { async fn create_writer_physical_plan( &self, input: Arc, - state: &SessionState, + state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { @@ -485,7 +487,7 @@ impl CsvFormat { /// number of lines that were read async fn infer_schema_from_stream( &self, - state: &SessionState, + state: &dyn Session, mut records_to_read: usize, stream: impl Stream>, ) -> Result<(Schema, usize)> { @@ -1147,7 +1149,7 @@ mod tests { } async fn get_exec( - state: &SessionState, + state: &dyn Session, file_name: &str, projection: Option>, limit: Option, diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index d96efeec3895..60609e467163 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -36,7 +36,7 @@ use crate::datasource::physical_plan::{ FileGroupDisplay, FileSink, FileSinkConfig, JsonSource, }; use crate::error::Result; -use crate::execution::context::SessionState; +use crate::execution::SessionState; use crate::physical_plan::insert::{DataSink, DataSinkExec}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, SendableRecordBatchStream, Statistics, @@ -48,6 +48,7 @@ use arrow::json; use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; use arrow_array::RecordBatch; use arrow_schema::ArrowError; +use datafusion_catalog::Session; use datafusion_common::config::{ConfigField, ConfigFileType, JsonOptions}; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::{not_impl_err, GetExt, DEFAULT_JSON_EXTENSION}; @@ -87,9 +88,10 @@ impl JsonFormatFactory { impl FileFormatFactory for JsonFormatFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &HashMap, ) -> Result> { + let state = state.as_any().downcast_ref::().unwrap(); let json_options = match &self.options { None => { let mut table_options = state.default_table_options(); @@ -189,7 +191,7 @@ impl FileFormat for JsonFormat { async fn infer_schema( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -237,7 +239,7 @@ impl FileFormat for JsonFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, _store: &Arc, table_schema: SchemaRef, _object: &ObjectMeta, @@ -247,7 +249,7 @@ impl FileFormat for JsonFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, mut conf: FileScanConfig, _filters: Option<&Arc>, ) -> Result> { @@ -261,7 +263,7 @@ impl FileFormat for JsonFormat { async fn create_writer_physical_plan( &self, input: Arc, - _state: &SessionState, + _state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { @@ -538,7 +540,7 @@ mod tests { } async fn get_exec( - state: &SessionState, + state: &dyn Session, projection: Option>, limit: Option, ) -> Result> { diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index bb7e42d3f854..ab5ce91ec57d 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -40,11 +40,11 @@ use std::task::Poll; use crate::arrow::datatypes::SchemaRef; use crate::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; use crate::error::Result; -use crate::execution::context::SessionState; use crate::physical_plan::{ExecutionPlan, Statistics}; use arrow_array::RecordBatch; use arrow_schema::{ArrowError, DataType, Field, FieldRef, Schema}; +use datafusion_catalog::Session; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{internal_err, not_impl_err, GetExt}; use datafusion_expr::Expr; @@ -66,7 +66,7 @@ pub trait FileFormatFactory: Sync + Send + GetExt + Debug { /// Initialize a [FileFormat] and configure based on session and command level options fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &HashMap, ) -> Result>; @@ -104,7 +104,7 @@ pub trait FileFormat: Send + Sync + Debug { /// the files have schemas that cannot be merged. async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result; @@ -118,7 +118,7 @@ pub trait FileFormat: Send + Sync + Debug { /// TODO: should the file source return statistics for only columns referred to in the table schema? async fn infer_stats( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, table_schema: SchemaRef, object: &ObjectMeta, @@ -128,7 +128,7 @@ pub trait FileFormat: Send + Sync + Debug { /// according to this file format. async fn create_physical_plan( &self, - state: &SessionState, + state: &dyn Session, conf: FileScanConfig, filters: Option<&Arc>, ) -> Result>; @@ -138,7 +138,7 @@ pub trait FileFormat: Send + Sync + Debug { async fn create_writer_physical_plan( &self, _input: Arc, - _state: &SessionState, + _state: &dyn Session, _conf: FileSinkConfig, _order_requirements: Option, ) -> Result> { @@ -569,7 +569,7 @@ pub(crate) mod test_util { }; pub async fn scan_format( - state: &SessionState, + state: &dyn Session, format: &dyn FileFormat, store_root: &str, file_name: &str, diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 70121c96ae16..8b91bc2cfdbe 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -39,7 +39,7 @@ use crate::datasource::physical_plan::parquet::source::ParquetSource; use crate::datasource::physical_plan::{FileGroupDisplay, FileSink, FileSinkConfig}; use crate::datasource::statistics::{create_max_min_accs, get_col_stats}; use crate::error::Result; -use crate::execution::context::SessionState; +use crate::execution::SessionState; use crate::physical_plan::insert::{DataSink, DataSinkExec}; use crate::physical_plan::{ Accumulator, DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, @@ -47,6 +47,7 @@ use crate::physical_plan::{ }; use arrow::compute::sum; +use datafusion_catalog::Session; use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; @@ -121,9 +122,10 @@ impl ParquetFormatFactory { impl FileFormatFactory for ParquetFormatFactory { fn create( &self, - state: &SessionState, + state: &dyn Session, format_options: &std::collections::HashMap, ) -> Result> { + let state = state.as_any().downcast_ref::().unwrap(); let parquet_options = match &self.options { None => { let mut table_options = state.default_table_options(); @@ -325,7 +327,7 @@ impl FileFormat for ParquetFormat { async fn infer_schema( &self, - state: &SessionState, + state: &dyn Session, store: &Arc, objects: &[ObjectMeta], ) -> Result { @@ -378,7 +380,7 @@ impl FileFormat for ParquetFormat { async fn infer_stats( &self, - _state: &SessionState, + _state: &dyn Session, store: &Arc, table_schema: SchemaRef, object: &ObjectMeta, @@ -395,7 +397,7 @@ impl FileFormat for ParquetFormat { async fn create_physical_plan( &self, - _state: &SessionState, + _state: &dyn Session, mut conf: FileScanConfig, filters: Option<&Arc>, ) -> Result> { @@ -429,7 +431,7 @@ impl FileFormat for ParquetFormat { async fn create_writer_physical_plan( &self, input: Arc, - _state: &SessionState, + _state: &dyn Session, conf: FileSinkConfig, order_requirements: Option, ) -> Result> { @@ -1295,6 +1297,7 @@ pub(crate) mod test_util { mod tests { use super::super::test_util::scan_format; use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; + use crate::execution::SessionState; use crate::physical_plan::collect; use crate::test_util::bounded_stream; use std::fmt::{Display, Formatter}; @@ -2229,13 +2232,13 @@ mod tests { } async fn get_exec( - state: &SessionState, + state: &dyn Session, file_name: &str, projection: Option>, limit: Option, ) -> Result> { let testdata = crate::test_util::parquet_test_data(); - + let state = state.as_any().downcast_ref::().unwrap(); let format = state .get_file_format_factory("parquet") .map(|factory| factory.create(state, &Default::default()).unwrap()) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 7d95a40186b2..f1d00ba9a37a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -137,7 +137,7 @@ impl ListingTableConfig { } /// Infer `ListingOptions` based on `table_path` suffix. - pub async fn infer_options(self, state: &SessionState) -> Result { + pub async fn infer_options(self, state: &dyn Session) -> Result { let store = if let Some(url) = self.table_paths.first() { state.runtime_env().object_store(url)? } else { @@ -164,7 +164,7 @@ impl ListingTableConfig { format_options .insert("format.compression".to_string(), compression_type.clone()); } - + let state = state.as_any().downcast_ref::().unwrap(); let file_format = state .get_file_format_factory(&file_extension) .ok_or(config_datafusion_err!( @@ -191,7 +191,7 @@ impl ListingTableConfig { } /// Infer the [`SchemaRef`] based on `table_path` suffix. Requires `self.options` to be set prior to using. - pub async fn infer_schema(self, state: &SessionState) -> Result { + pub async fn infer_schema(self, state: &dyn Session) -> Result { match self.options { Some(options) => { let schema = if let Some(url) = self.table_paths.first() { @@ -211,12 +211,12 @@ impl ListingTableConfig { } /// Convenience wrapper for calling `infer_options` and `infer_schema` - pub async fn infer(self, state: &SessionState) -> Result { + pub async fn infer(self, state: &dyn Session) -> Result { self.infer_options(state).await?.infer_schema(state).await } /// Infer the partition columns from the path. Requires `self.options` to be set prior to using. - pub async fn infer_partitions_from_path(self, state: &SessionState) -> Result { + pub async fn infer_partitions_from_path(self, state: &dyn Session) -> Result { match self.options { Some(options) => { let Some(url) = self.table_paths.first() else { @@ -484,7 +484,7 @@ impl ListingOptions { /// locally or ask a remote service to do it (e.g a scheduler). pub async fn infer_schema<'a>( &'a self, - state: &SessionState, + state: &dyn Session, table_path: &'a ListingTableUrl, ) -> Result { let store = state.runtime_env().object_store(table_path)?; @@ -509,7 +509,7 @@ impl ListingOptions { /// Allows specifying partial partitions. pub async fn validate_partitions( &self, - state: &SessionState, + state: &dyn Session, table_path: &ListingTableUrl, ) -> Result<()> { if self.table_partition_cols.is_empty() { @@ -563,7 +563,7 @@ impl ListingOptions { /// and therefore may fail to detect invalid partitioning. pub(crate) async fn infer_partitions( &self, - state: &SessionState, + state: &dyn Session, table_path: &ListingTableUrl, ) -> Result> { let store = state.runtime_env().object_store(table_path)?; @@ -1095,7 +1095,7 @@ impl ListingTable { /// be distributed to different threads / executors. async fn list_files_for_scan<'a>( &'a self, - ctx: &'a SessionState, + ctx: &'a dyn Session, filters: &'a [Expr], limit: Option, ) -> Result<(Vec>, Statistics)> { @@ -1156,7 +1156,7 @@ impl ListingTable { /// If they are not, it infers the statistics from the file and stores them in the cache. async fn do_collect_statistics( &self, - ctx: &SessionState, + ctx: &dyn Session, store: &Arc, part_file: &PartitionedFile, ) -> Result> { diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 5c5dce46e936..1a486a54ca39 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -256,6 +256,10 @@ impl FileSource for ArrowSource { fn file_type(&self) -> &str { "arrow" } + + fn supports_repartition(&self, config: &FileScanConfig) -> bool { + !(config.file_compression_type.is_compressed() || config.new_lines_in_values) + } } /// The struct arrow that implements `[FileOpener]` trait diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index ee3f7071c11f..b148c412c48e 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -255,6 +255,11 @@ impl FileSource for AvroSource { fn file_type(&self) -> &str { "avro" } + fn supports_repartition(&self, config: &FileScanConfig) -> bool { + !(config.file_compression_type.is_compressed() + || config.new_lines_in_values + || self.as_any().downcast_ref::().is_some()) + } } #[cfg(feature = "avro")] diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 269af4df903a..bfc2c1df8eab 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -618,6 +618,9 @@ impl FileSource for CsvSource { fn fmt_extra(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { write!(f, ", has_header={}", self.has_header) } + fn supports_repartition(&self, config: &FileScanConfig) -> bool { + !(config.file_compression_type.is_compressed() || config.new_lines_in_values) + } } impl FileOpener for CsvOpener { diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 652632c31554..e31c7dfc247a 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -19,8 +19,8 @@ //! file sources. use super::{ - get_projected_output_ordering, statistics::MinMaxStatistics, AvroSource, - FileGroupPartitioner, FileGroupsDisplay, FileStream, + get_projected_output_ordering, statistics::MinMaxStatistics, FileGroupPartitioner, + FileGroupsDisplay, FileStream, }; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; @@ -209,7 +209,7 @@ impl DataSource for FileScanConfig { repartition_file_min_size: usize, output_ordering: Option, ) -> Result>> { - if !self.supports_repartition() { + if !self.source.supports_repartition(self) { return Ok(None); } @@ -598,12 +598,6 @@ impl FileScanConfig { pub fn file_source(&self) -> &Arc { &self.source } - - fn supports_repartition(&self) -> bool { - !(self.file_compression_type.is_compressed() - || self.new_lines_in_values - || self.source.as_any().downcast_ref::().is_some()) - } } /// A helper that projects partition columns into the file record batches. diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index f581f5cd0b4a..76cb657b0c5f 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -313,6 +313,10 @@ impl FileSource for JsonSource { fn file_type(&self) -> &str { "json" } + + fn supports_repartition(&self, config: &FileScanConfig) -> bool { + !(config.file_compression_type.is_compressed() || config.new_lines_in_values) + } } impl FileOpener for JsonOpener { diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 28ac73cef521..4cc65714d53e 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -20,14 +20,12 @@ mod arrow_file; mod avro; mod csv; -mod file_groups; mod file_scan_config; mod file_stream; mod json; #[cfg(feature = "parquet")] pub mod parquet; mod statistics; - pub(crate) use self::csv::plan_to_csv; pub(crate) use self::json::plan_to_json; #[cfg(feature = "parquet")] @@ -37,7 +35,6 @@ pub use self::parquet::source::ParquetSource; pub use self::parquet::{ ParquetExec, ParquetExecBuilder, ParquetFileMetrics, ParquetFileReaderFactory, }; - #[allow(deprecated)] pub use arrow_file::ArrowExec; pub use arrow_file::ArrowSource; @@ -47,8 +44,8 @@ pub use avro::AvroSource; #[allow(deprecated)] pub use csv::{CsvExec, CsvExecBuilder}; pub use csv::{CsvOpener, CsvSource}; +pub use datafusion_catalog_listing::file_groups::FileGroupPartitioner; use datafusion_expr::dml::InsertOp; -pub use file_groups::FileGroupPartitioner; pub use file_scan_config::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, }; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/core/src/datasource/physical_plan/parquet/source.rs index c00fe91b859f..0705a398f4fb 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/source.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/source.rs @@ -586,4 +586,7 @@ impl FileSource for ParquetSource { } } } + fn supports_repartition(&self, _config: &FileScanConfig) -> bool { + true + } } From 91c0975c8a1382dff97474bfc45e504bc62ef016 Mon Sep 17 00:00:00 2001 From: Kaifeng Zheng <100595273+Kev1n8@users.noreply.github.com> Date: Sat, 8 Feb 2025 18:15:19 +0800 Subject: [PATCH 07/12] Always use `StringViewArray` as output of `substr` (#14498) * change substr return type to utf8view (bin/sqllogicaltests.rs to fix) * fix sqllogictest/string_view * fix clippy * fix tcph benchmark result --- datafusion/functions/src/unicode/substr.rs | 118 +++++++++--------- .../test_files/string/string_view.slt | 2 +- .../test_files/tpch/plans/q22.slt.part | 12 +- 3 files changed, 64 insertions(+), 68 deletions(-) diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index 3767166cab51..20d5f6e3abeb 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -19,10 +19,10 @@ use std::any::Any; use std::sync::Arc; use crate::strings::make_and_append_view; -use crate::utils::{make_scalar_function, utf8_to_str_type}; +use crate::utils::make_scalar_function; use arrow::array::{ - Array, ArrayIter, ArrayRef, AsArray, GenericStringBuilder, Int64Array, - NullBufferBuilder, OffsetSizeTrait, StringArrayType, StringViewArray, + Array, ArrayIter, ArrayRef, AsArray, Int64Array, NullBufferBuilder, StringArrayType, + StringViewArray, StringViewBuilder, }; use arrow::buffer::ScalarBuffer; use arrow::datatypes::DataType; @@ -90,12 +90,9 @@ impl ScalarUDFImpl for SubstrFunc { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { - if arg_types[0] == DataType::Utf8View { - Ok(DataType::Utf8View) - } else { - utf8_to_str_type(&arg_types[0], "substr") - } + // `SubstrFunc` always generates `Utf8View` output for its efficiency. + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Utf8View) } fn invoke_batch( @@ -189,11 +186,11 @@ pub fn substr(args: &[ArrayRef]) -> Result { match args[0].data_type() { DataType::Utf8 => { let string_array = args[0].as_string::(); - string_substr::<_, i32>(string_array, &args[1..]) + string_substr::<_>(string_array, &args[1..]) } DataType::LargeUtf8 => { let string_array = args[0].as_string::(); - string_substr::<_, i64>(string_array, &args[1..]) + string_substr::<_>(string_array, &args[1..]) } DataType::Utf8View => { let string_array = args[0].as_string_view(); @@ -429,10 +426,9 @@ fn string_view_substr( } } -fn string_substr<'a, V, T>(string_array: V, args: &[ArrayRef]) -> Result +fn string_substr<'a, V>(string_array: V, args: &[ArrayRef]) -> Result where V: StringArrayType<'a>, - T: OffsetSizeTrait, { let start_array = as_int64_array(&args[0])?; let count_array_opt = if args.len() == 2 { @@ -447,7 +443,7 @@ where match args.len() { 1 => { let iter = ArrayIter::new(string_array); - let mut result_builder = GenericStringBuilder::::new(); + let mut result_builder = StringViewBuilder::new(); for (string, start) in iter.zip(start_array.iter()) { match (string, start) { (Some(string), Some(start)) => { @@ -470,7 +466,7 @@ where 2 => { let iter = ArrayIter::new(string_array); let count_array = count_array_opt.unwrap(); - let mut result_builder = GenericStringBuilder::::new(); + let mut result_builder = StringViewBuilder::new(); for ((string, start), count) in iter.zip(start_array.iter()).zip(count_array.iter()) @@ -512,8 +508,8 @@ where #[cfg(test)] mod tests { - use arrow::array::{Array, StringArray, StringViewArray}; - use arrow::datatypes::DataType::{Utf8, Utf8View}; + use arrow::array::{Array, StringViewArray}; + use arrow::datatypes::DataType::Utf8View; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl}; @@ -623,8 +619,8 @@ mod tests { ], Ok(Some("alphabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -634,8 +630,8 @@ mod tests { ], Ok(Some("ésoj")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -645,8 +641,8 @@ mod tests { ], Ok(Some("joséésoj")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -656,8 +652,8 @@ mod tests { ], Ok(Some("alphabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -667,8 +663,8 @@ mod tests { ], Ok(Some("lphabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -678,8 +674,8 @@ mod tests { ], Ok(Some("phabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -689,8 +685,8 @@ mod tests { ], Ok(Some("alphabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -700,8 +696,8 @@ mod tests { ], Ok(Some("")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -711,8 +707,8 @@ mod tests { ], Ok(None), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -723,8 +719,8 @@ mod tests { ], Ok(Some("ph")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -735,8 +731,8 @@ mod tests { ], Ok(Some("phabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -747,8 +743,8 @@ mod tests { ], Ok(Some("alph")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); // starting from 5 (10 + -5) test_function!( @@ -760,8 +756,8 @@ mod tests { ], Ok(Some("alph")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); // starting from -1 (4 + -5) test_function!( @@ -773,8 +769,8 @@ mod tests { ], Ok(Some("")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); // starting from 0 (5 + -5) test_function!( @@ -786,8 +782,8 @@ mod tests { ], Ok(Some("")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -798,8 +794,8 @@ mod tests { ], Ok(None), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -810,8 +806,8 @@ mod tests { ], Ok(None), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -822,8 +818,8 @@ mod tests { ], exec_err!("negative substring length not allowed: substr(, 1, -1)"), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -834,8 +830,8 @@ mod tests { ], Ok(Some("és")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); #[cfg(not(feature = "unicode_expressions"))] test_function!( @@ -848,8 +844,8 @@ mod tests { "function substr requires compilation with feature flag: unicode_expressions." ), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -859,8 +855,8 @@ mod tests { ], Ok(Some("abc")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -871,8 +867,8 @@ mod tests { ], exec_err!("negative overflow when calculating skip value"), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); Ok(()) diff --git a/datafusion/sqllogictest/test_files/string/string_view.slt b/datafusion/sqllogictest/test_files/string/string_view.slt index c54e2aa7002c..a04cea7cd800 100644 --- a/datafusion/sqllogictest/test_files/string/string_view.slt +++ b/datafusion/sqllogictest/test_files/string/string_view.slt @@ -382,7 +382,7 @@ EXPLAIN SELECT FROM test; ---- logical_plan -01)Projection: starts_with(test.column1_utf8, substr(test.column1_utf8, Int64(1), Int64(2))) AS c1, starts_with(test.column1_large_utf8, substr(test.column1_large_utf8, Int64(1), Int64(2))) AS c2, starts_with(test.column1_utf8view, substr(test.column1_utf8view, Int64(1), Int64(2))) AS c3 +01)Projection: starts_with(CAST(test.column1_utf8 AS Utf8View), substr(test.column1_utf8, Int64(1), Int64(2))) AS c1, starts_with(CAST(test.column1_large_utf8 AS Utf8View), substr(test.column1_large_utf8, Int64(1), Int64(2))) AS c2, starts_with(test.column1_utf8view, substr(test.column1_utf8view, Int64(1), Int64(2))) AS c3 02)--TableScan: test projection=[column1_utf8, column1_large_utf8, column1_utf8view] query BBB diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index e7b1f0a5989e..9ad99361256c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -64,15 +64,15 @@ logical_plan 06)----------Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_2.avg(customer.c_acctbal) 07)------------Projection: customer.c_phone, customer.c_acctbal 08)--------------LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey -09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) -10)------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")])] +09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]) +10)------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")])] 11)----------------SubqueryAlias: __correlated_sq_1 12)------------------TableScan: orders projection=[o_custkey] 13)------------SubqueryAlias: __scalar_sq_2 14)--------------Aggregate: groupBy=[[]], aggr=[[avg(customer.c_acctbal)]] 15)----------------Projection: customer.c_acctbal -16)------------------Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) -17)--------------------TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")])] +16)------------------Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")]) +17)--------------------TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8View("13"), Utf8View("31"), Utf8View("23"), Utf8View("29"), Utf8View("30"), Utf8View("18"), Utf8View("17")])] physical_plan 01)SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] 02)--SortExec: expr=[cntrycode@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -90,7 +90,7 @@ physical_plan 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------FilterExec: Use substr(c_phone@1, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) +17)--------------------------------FilterExec: substr(c_phone@1, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]) 18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 19)------------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_phone, c_acctbal], file_type=csv, has_header=false 20)--------------------------CoalesceBatchesExec: target_batch_size=8192 @@ -100,6 +100,6 @@ physical_plan 24)----------------------CoalescePartitionsExec 25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] 26)--------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]), projection=[c_acctbal@1] +27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND substr(c_phone@0, 1, 2) IN ([Literal { value: Utf8View("13") }, Literal { value: Utf8View("31") }, Literal { value: Utf8View("23") }, Literal { value: Utf8View("29") }, Literal { value: Utf8View("30") }, Literal { value: Utf8View("18") }, Literal { value: Utf8View("17") }]), projection=[c_acctbal@1] 28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 29)--------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], file_type=csv, has_header=false From faa8c1bcc794b1fb92eeb06a88b1db64bc76b7c8 Mon Sep 17 00:00:00 2001 From: Ian Lai <108986288+Chen-Yuan-Lai@users.noreply.github.com> Date: Sat, 8 Feb 2025 21:27:22 +0800 Subject: [PATCH 08/12] refactor: remove remaining uses of `arrow_array` and use reexport in `arrow` instead (#14528) * refactor: remove remaining uses of arrow_array and use reexport in arrow instead * fix: remove blank * fix: run carg fmt * fix: import record_batch * fix: update cargo.lock * fix: add chrono-tz feature in arrow * fix: remove remaining arrow-array & update cargo.lock --------- Co-authored-by: Cheng-Yuan-Lai Co-authored-by: Ian Lai Co-authored-by: Andrew Lamb --- Cargo.toml | 2 -- datafusion-cli/Cargo.lock | 21 +++++------- .../ffi/ffi_example_table_provider/Cargo.toml | 1 - .../ffi/ffi_example_table_provider/src/lib.rs | 9 ++--- datafusion/common/Cargo.toml | 1 - datafusion/common/src/cast.rs | 8 ++--- datafusion/common/src/pyarrow.rs | 3 +- datafusion/common/src/scalar/mod.rs | 34 ++++++++++--------- datafusion/common/src/test_util.rs | 6 ++-- datafusion/common/src/utils/mod.rs | 13 ++++--- datafusion/common/src/utils/string_utils.rs | 4 +-- datafusion/core/Cargo.toml | 1 - datafusion/core/benches/data_utils/mod.rs | 12 +++---- datafusion/core/benches/map_query_sql.rs | 2 +- datafusion/core/benches/sql_planner.rs | 2 +- datafusion/core/benches/sql_query_with_io.rs | 3 +- .../core/src/datasource/file_format/csv.rs | 2 +- .../core/src/datasource/file_format/json.rs | 2 +- .../core/src/datasource/file_format/mod.rs | 2 +- .../src/datasource/file_format/parquet.rs | 7 ++-- .../src/datasource/file_format/write/demux.rs | 7 ++-- .../src/datasource/file_format/write/mod.rs | 2 +- .../file_format/write/orchestration.rs | 2 +- .../core/src/datasource/listing/table.rs | 2 +- .../physical_plan/file_scan_config.rs | 7 ++-- .../core/src/datasource/physical_plan/mod.rs | 6 ++-- .../physical_plan/parquet/row_group_filter.rs | 4 +-- .../datasource/physical_plan/statistics.rs | 2 +- .../core/src/datasource/schema_adapter.rs | 4 +-- datafusion/core/src/datasource/stream.rs | 2 +- .../core/src/execution/session_state.rs | 2 +- datafusion/core/src/lib.rs | 6 ++-- datafusion/core/src/physical_planner.rs | 3 +- .../tests/dataframe/dataframe_functions.rs | 3 +- datafusion/core/tests/dataframe/mod.rs | 18 ++++------ .../core/tests/execution/logical_plan.rs | 2 +- datafusion/core/tests/expr_api/mod.rs | 6 ++-- .../core/tests/expr_api/simplification.rs | 2 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 4 +-- .../aggregation_fuzzer/context_generator.rs | 2 +- .../aggregation_fuzzer/data_generator.rs | 4 +-- .../fuzz_cases/aggregation_fuzzer/fuzzer.rs | 2 +- .../fuzz_cases/aggregation_fuzzer/mod.rs | 2 +- .../fuzz_cases/distinct_count_string_fuzz.rs | 4 +-- .../tests/fuzz_cases/equivalence/utils.rs | 2 +- .../core/tests/fuzz_cases/limit_fuzz.rs | 3 +- datafusion/core/tests/fuzz_cases/pruning.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 2 +- datafusion/core/tests/memory_limit/mod.rs | 3 +- .../core/tests/parquet/schema_coercion.rs | 7 ++-- .../core/tests/user_defined/expr_planner.rs | 2 +- .../user_defined/user_defined_aggregates.rs | 10 +++--- .../user_defined_scalar_functions.rs | 9 +++-- .../user_defined_window_functions.rs | 20 +++++------ datafusion/ffi/Cargo.toml | 1 - datafusion/ffi/src/tests/async_provider.rs | 2 +- datafusion/ffi/src/tests/mod.rs | 2 +- datafusion/functions-nested/Cargo.toml | 1 - datafusion/functions-nested/benches/map.rs | 2 +- datafusion/functions-nested/src/array_has.rs | 5 +-- .../functions-nested/src/cardinality.rs | 2 +- datafusion/functions-nested/src/concat.rs | 2 +- datafusion/functions-nested/src/distance.rs | 2 +- datafusion/functions-nested/src/empty.rs | 2 +- datafusion/functions-nested/src/except.rs | 3 +- datafusion/functions-nested/src/extract.rs | 2 +- datafusion/functions-nested/src/flatten.rs | 2 +- datafusion/functions-nested/src/length.rs | 2 +- datafusion/functions-nested/src/make_array.rs | 10 +++--- datafusion/functions-nested/src/map.rs | 3 +- .../functions-nested/src/map_extract.rs | 7 ++-- datafusion/functions-nested/src/map_keys.rs | 2 +- datafusion/functions-nested/src/map_values.rs | 2 +- datafusion/functions-nested/src/position.rs | 6 ++-- datafusion/functions-nested/src/remove.rs | 10 +++--- datafusion/functions-nested/src/repeat.rs | 13 ++++--- datafusion/functions-nested/src/replace.rs | 2 +- datafusion/functions-nested/src/resize.rs | 2 +- datafusion/functions-nested/src/reverse.rs | 7 ++-- datafusion/functions-nested/src/string.rs | 8 +++-- datafusion/functions-nested/src/utils.rs | 8 ++--- datafusion/functions/src/datetime/mod.rs | 2 +- datafusion/physical-expr/Cargo.toml | 1 - datafusion/physical-expr/benches/case_when.rs | 2 +- datafusion/physical-expr/benches/is_null.rs | 3 +- .../src/expressions/is_not_null.rs | 9 +++-- .../physical-expr/src/expressions/is_null.rs | 9 +++-- datafusion/physical-expr/src/planner.rs | 2 +- .../physical-expr/src/scalar_function.rs | 3 +- datafusion/physical-expr/src/utils/mod.rs | 2 +- .../physical-expr/src/window/window_expr.rs | 2 +- datafusion/physical-plan/Cargo.toml | 1 - datafusion/physical-plan/benches/spm.rs | 3 +- .../src/aggregates/group_values/mod.rs | 5 ++- .../group_values/multi_group_by/bytes_view.rs | 3 +- .../group_values/multi_group_by/mod.rs | 5 ++- .../src/aggregates/group_values/row.rs | 3 +- .../group_values/single_group_by/bytes.rs | 2 +- .../single_group_by/bytes_view.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 11 +++--- .../physical-plan/src/aggregates/order/mod.rs | 2 +- .../src/aggregates/order/partial.rs | 2 +- .../src/aggregates/topk/hash_table.rs | 9 +++-- .../physical-plan/src/aggregates/topk/heap.rs | 8 +++-- .../src/aggregates/topk/priority_map.rs | 4 +-- .../src/aggregates/topk_stream.rs | 2 +- datafusion/physical-plan/src/coalesce/mod.rs | 10 +++--- datafusion/physical-plan/src/common.rs | 2 +- .../physical-plan/src/execution_plan.rs | 11 +++--- datafusion/physical-plan/src/insert.rs | 3 +- .../physical-plan/src/joins/cross_join.rs | 3 +- .../physical-plan/src/joins/hash_join.rs | 7 ++-- .../src/joins/nested_loop_join.rs | 2 +- .../src/joins/sort_merge_join.rs | 11 +++--- .../physical-plan/src/joins/test_utils.rs | 9 +++-- datafusion/physical-plan/src/joins/utils.rs | 13 +++---- datafusion/physical-plan/src/limit.rs | 2 +- datafusion/physical-plan/src/memory.rs | 3 +- .../physical-plan/src/placeholder_row.rs | 3 +- .../physical-plan/src/repartition/mod.rs | 3 +- datafusion/physical-plan/src/sorts/cursor.rs | 11 +++--- datafusion/physical-plan/src/sorts/sort.rs | 3 +- .../src/sorts/sort_preserving_merge.rs | 7 ++-- .../src/sorts/streaming_merge.rs | 2 +- datafusion/physical-plan/src/spill.rs | 3 +- datafusion/physical-plan/src/test.rs | 2 +- datafusion/physical-plan/src/topk/mod.rs | 6 ++-- datafusion/physical-plan/src/unnest.rs | 5 ++- .../src/windows/bounded_window_agg_exec.rs | 8 +++-- datafusion/physical-plan/src/work_table.rs | 2 +- datafusion/sql/Cargo.toml | 1 - datafusion/sql/src/unparser/expr.rs | 19 ++++++----- 132 files changed, 297 insertions(+), 354 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 8d25478ab549..cbde1b46591c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -80,8 +80,6 @@ ahash = { version = "0.8", default-features = false, features = [ ] } arrow = { version = "54.1.0", features = [ "prettyprint", -] } -arrow-array = { version = "54.1.0", default-features = false, features = [ "chrono-tz", ] } arrow-buffer = { version = "54.1.0", default-features = false } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index e320b2ffc835..00f59f9d78ad 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1210,7 +1210,6 @@ version = "45.0.0" dependencies = [ "apache-avro", "arrow", - "arrow-array", "arrow-ipc", "arrow-schema", "async-compression", @@ -1331,7 +1330,6 @@ dependencies = [ "ahash", "apache-avro", "arrow", - "arrow-array", "arrow-ipc", "arrow-schema", "base64 0.22.1", @@ -1471,7 +1469,6 @@ name = "datafusion-functions-nested" version = "45.0.0" dependencies = [ "arrow", - "arrow-array", "arrow-ord", "arrow-schema", "datafusion-common", @@ -1556,7 +1553,6 @@ version = "45.0.0" dependencies = [ "ahash", "arrow", - "arrow-array", "arrow-schema", "datafusion-common", "datafusion-expr", @@ -1610,7 +1606,6 @@ version = "45.0.0" dependencies = [ "ahash", "arrow", - "arrow-array", "arrow-ord", "arrow-schema", "async-trait", @@ -1638,7 +1633,6 @@ name = "datafusion-sql" version = "45.0.0" dependencies = [ "arrow", - "arrow-array", "arrow-schema", "bigdecimal", "datafusion-common", @@ -2854,9 +2848,9 @@ dependencies = [ [[package]] name = "once_cell" -version = "1.20.2" +version = "1.20.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" +checksum = "945462a4b81e43c4e3ba96bd7b49d834c6f61198356aa858733bc4acf3cbe62e" [[package]] name = "openssl-probe" @@ -3407,9 +3401,9 @@ checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" [[package]] name = "rustc-hash" -version = "2.1.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7fb8039b3032c191086b10f11f319a6e99e1e82889c5cc6046f515c9db1d497" +checksum = "357703d41365b4b27c590e3ed91eabb1b663f07c4c084095e60cbed4362dff0d" [[package]] name = "rustc_version" @@ -4280,11 +4274,12 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.12.1" +version = "1.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3758f5e68192bb96cc8f9b7e2c2cfdabb435499a28499a42f8f984092adad4b" +checksum = "ced87ca4be083373936a67f8de945faa23b6b42384bd5b64434850802c6dccd0" dependencies = [ - "getrandom 0.2.15", + "getrandom 0.3.1", + "js-sys", "serde", "wasm-bindgen", ] diff --git a/datafusion-examples/examples/ffi/ffi_example_table_provider/Cargo.toml b/datafusion-examples/examples/ffi/ffi_example_table_provider/Cargo.toml index 52efdb7461ab..2d91ea2329e4 100644 --- a/datafusion-examples/examples/ffi/ffi_example_table_provider/Cargo.toml +++ b/datafusion-examples/examples/ffi/ffi_example_table_provider/Cargo.toml @@ -24,7 +24,6 @@ publish = false [dependencies] abi_stable = "0.11.3" arrow = { workspace = true } -arrow-array = { workspace = true } arrow-schema = { workspace = true } datafusion = { workspace = true } datafusion-ffi = { workspace = true } diff --git a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs index c37d8f835ce8..a83f15926f05 100644 --- a/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs +++ b/datafusion-examples/examples/ffi/ffi_example_table_provider/src/lib.rs @@ -18,12 +18,9 @@ use std::sync::Arc; use abi_stable::{export_root_module, prefix_type::PrefixTypeTrait}; -use arrow_array::RecordBatch; -use datafusion::{ - arrow::datatypes::{DataType, Field, Schema}, - common::record_batch, - datasource::MemTable, -}; +use arrow::array::RecordBatch; +use arrow::datatypes::{DataType, Field, Schema}; +use datafusion::{common::record_batch, datasource::MemTable}; use datafusion_ffi::table_provider::FFI_TableProvider; use ffi_module_interface::{TableProviderModule, TableProviderModuleRef}; diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 1050b376be8a..215a06e81c3d 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -51,7 +51,6 @@ apache-avro = { version = "0.17", default-features = false, features = [ "zstandard", ], optional = true } arrow = { workspace = true } -arrow-array = { workspace = true } arrow-ipc = { workspace = true } arrow-schema = { workspace = true } base64 = "0.22.1" diff --git a/datafusion/common/src/cast.rs b/datafusion/common/src/cast.rs index bb9d80996571..bc37e59c9b92 100644 --- a/datafusion/common/src/cast.rs +++ b/datafusion/common/src/cast.rs @@ -21,6 +21,10 @@ //! kernels in arrow-rs such as `as_boolean_array` do. use crate::{downcast_value, DataFusionError, Result}; +use arrow::array::{ + BinaryViewArray, Float16Array, Int16Array, Int8Array, LargeBinaryArray, + LargeStringArray, StringViewArray, UInt16Array, +}; use arrow::{ array::{ Array, BinaryArray, BooleanArray, Date32Array, Date64Array, Decimal128Array, @@ -36,10 +40,6 @@ use arrow::{ }, datatypes::{ArrowDictionaryKeyType, ArrowPrimitiveType}, }; -use arrow_array::{ - BinaryViewArray, Float16Array, Int16Array, Int8Array, LargeBinaryArray, - LargeStringArray, StringViewArray, UInt16Array, -}; // Downcast ArrayRef to Date32Array pub fn as_date32_array(array: &dyn Array) -> Result<&Date32Array> { diff --git a/datafusion/common/src/pyarrow.rs b/datafusion/common/src/pyarrow.rs index 60dde7861104..ff413e08ab07 100644 --- a/datafusion/common/src/pyarrow.rs +++ b/datafusion/common/src/pyarrow.rs @@ -17,9 +17,8 @@ //! Conversions between PyArrow and DataFusion types -use arrow::array::ArrayData; +use arrow::array::{Array, ArrayData}; use arrow::pyarrow::{FromPyArrow, ToPyArrow}; -use arrow_array::Array; use pyo3::exceptions::PyException; use pyo3::prelude::PyErr; use pyo3::types::{PyAnyMethods, PyList}; diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 6cd6a43941c8..5db0f5ed5cc0 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -40,22 +40,24 @@ use crate::cast::{ use crate::error::{DataFusionError, Result, _exec_err, _internal_err, _not_impl_err}; use crate::hash_utils::create_hashes; use crate::utils::SingleRowListArrayBuilder; -use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::array::{ + types::{IntervalDayTime, IntervalMonthDayNano}, + *, +}; use arrow::buffer::ScalarBuffer; -use arrow::compute::kernels::numeric::*; -use arrow::util::display::{array_value_to_string, ArrayFormatter, FormatOptions}; -use arrow::{ - array::*, - compute::kernels::cast::{cast_with_options, CastOptions}, - datatypes::{ - i256, ArrowDictionaryKeyType, ArrowNativeType, ArrowTimestampType, DataType, - Date32Type, Date64Type, Field, Float32Type, Int16Type, Int32Type, Int64Type, - Int8Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, - IntervalYearMonthType, TimeUnit, TimestampMicrosecondType, - TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, - UInt16Type, UInt32Type, UInt64Type, UInt8Type, DECIMAL128_MAX_PRECISION, - }, +use arrow::compute::kernels::{ + cast::{cast_with_options, CastOptions}, + numeric::*, }; +use arrow::datatypes::{ + i256, ArrowDictionaryKeyType, ArrowNativeType, ArrowTimestampType, DataType, + Date32Type, Date64Type, Field, Float32Type, Int16Type, Int32Type, Int64Type, + Int8Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalUnit, + IntervalYearMonthType, TimeUnit, TimestampMicrosecondType, TimestampMillisecondType, + TimestampNanosecondType, TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, + UInt8Type, DECIMAL128_MAX_PRECISION, +}; +use arrow::util::display::{array_value_to_string, ArrayFormatter, FormatOptions}; use arrow_schema::{UnionFields, UnionMode}; use crate::format::DEFAULT_CAST_OPTIONS; @@ -165,7 +167,7 @@ pub use struct_builder::ScalarStructBuilder; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{DataType, Field, Fields}; -/// # use arrow_array::{ArrayRef, Int32Array, StructArray, StringArray}; +/// # use arrow::array::{ArrayRef, Int32Array, StructArray, StringArray}; /// # use datafusion_common::ScalarValue; /// // Build a struct like: {a: 1, b: "foo"} /// // Field description @@ -1674,7 +1676,7 @@ impl ScalarValue { /// /// assert_eq!(&result, &expected); /// ``` - /// [`Datum`]: arrow_array::Datum + /// [`Datum`]: arrow::array::Datum pub fn to_scalar(&self) -> Result> { Ok(Scalar::new(self.to_array_of_size(1)?)) } diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index d3b8c8451258..22a7d87a8949 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -34,7 +34,7 @@ use std::{error::Error, path::PathBuf}; /// ``` /// # use std::sync::Arc; /// # use arrow::record_batch::RecordBatch; -/// # use arrow_array::{ArrayRef, Int32Array}; +/// # use arrow::array::{ArrayRef, Int32Array}; /// # use datafusion_common::assert_batches_eq; /// let col: ArrayRef = Arc::new(Int32Array::from(vec![1, 2])); /// let batch = RecordBatch::try_from_iter([("column", col)]).unwrap(); @@ -344,7 +344,7 @@ macro_rules! record_batch { )* ])); - let batch = arrow_array::RecordBatch::try_new( + let batch = arrow::array::RecordBatch::try_new( schema, vec![$( $crate::create_array!($type, $values), @@ -416,7 +416,7 @@ mod tests { #[test] fn test_create_record_batch() -> Result<()> { - use arrow_array::Array; + use arrow::array::Array; let batch = record_batch!( ("a", Int32, vec![1, 2, 3, 4]), diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index 068bfe96103d..f2377cc5410a 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -24,14 +24,13 @@ pub mod string_utils; use crate::error::{_internal_datafusion_err, _internal_err}; use crate::{DataFusionError, Result, ScalarValue}; -use arrow::array::ArrayRef; +use arrow::array::{ + cast::AsArray, Array, ArrayRef, FixedSizeListArray, LargeListArray, ListArray, + OffsetSizeTrait, +}; use arrow::buffer::OffsetBuffer; use arrow::compute::{partition, SortColumn, SortOptions}; use arrow::datatypes::{Field, SchemaRef}; -use arrow_array::cast::AsArray; -use arrow_array::{ - Array, FixedSizeListArray, LargeListArray, ListArray, OffsetSizeTrait, -}; use arrow_schema::DataType; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; @@ -329,8 +328,8 @@ pub fn longest_consecutive_prefix>( /// # Example /// ``` /// # use std::sync::Arc; -/// # use arrow_array::{Array, ListArray}; -/// # use arrow_array::types::Int64Type; +/// # use arrow::array::{Array, ListArray}; +/// # use arrow::array::types::Int64Type; /// # use datafusion_common::utils::SingleRowListArrayBuilder; /// // Array is [1, 2, 3] /// let arr = ListArray::from_iter_primitive::(vec![ diff --git a/datafusion/common/src/utils/string_utils.rs b/datafusion/common/src/utils/string_utils.rs index a2231e6786a7..64c7892a5376 100644 --- a/datafusion/common/src/utils/string_utils.rs +++ b/datafusion/common/src/utils/string_utils.rs @@ -17,8 +17,8 @@ //! Utilities for working with strings -use arrow::{array::AsArray, datatypes::DataType}; -use arrow_array::Array; +use arrow::array::{Array, AsArray}; +use arrow::datatypes::DataType; /// Convenient function to convert an Arrow string array to a vector of strings pub fn string_array_to_vec(array: &dyn Array) -> Vec> { diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index c90e5ce0b5b6..bbd999ffe98b 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -85,7 +85,6 @@ extended_tests = [] [dependencies] apache-avro = { version = "0.17", optional = true } arrow = { workspace = true } -arrow-array = { workspace = true } arrow-ipc = { workspace = true } arrow-schema = { workspace = true } async-compression = { version = "0.4.0", features = [ diff --git a/datafusion/core/benches/data_utils/mod.rs b/datafusion/core/benches/data_utils/mod.rs index 9d2864919225..38f6a2c76df6 100644 --- a/datafusion/core/benches/data_utils/mod.rs +++ b/datafusion/core/benches/data_utils/mod.rs @@ -17,15 +17,11 @@ //! This module provides the in-memory table for more realistic benchmarking. -use arrow::{ - array::Float32Array, - array::Float64Array, - array::StringArray, - array::UInt64Array, - datatypes::{DataType, Field, Schema, SchemaRef}, - record_batch::RecordBatch, +use arrow::array::{ + builder::{Int64Builder, StringBuilder}, + Float32Array, Float64Array, RecordBatch, StringArray, UInt64Array, }; -use arrow_array::builder::{Int64Builder, StringBuilder}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::datasource::MemTable; use datafusion::error::Result; use datafusion_common::DataFusionError; diff --git a/datafusion/core/benches/map_query_sql.rs b/datafusion/core/benches/map_query_sql.rs index e4c5f7c5deb3..79229dfc2fbd 100644 --- a/datafusion/core/benches/map_query_sql.rs +++ b/datafusion/core/benches/map_query_sql.rs @@ -17,7 +17,7 @@ use std::sync::Arc; -use arrow_array::{ArrayRef, Int32Array, RecordBatch}; +use arrow::array::{ArrayRef, Int32Array, RecordBatch}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use parking_lot::Mutex; use rand::prelude::ThreadRng; diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index a1b339eea355..2d79778d4d42 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -23,8 +23,8 @@ extern crate datafusion; mod data_utils; use crate::criterion::Criterion; +use arrow::array::{ArrayRef, RecordBatch}; use arrow::datatypes::{DataType, Field, Fields, Schema}; -use arrow_array::{ArrayRef, RecordBatch}; use criterion::Bencher; use datafusion::datasource::MemTable; use datafusion::execution::context::SessionContext; diff --git a/datafusion/core/benches/sql_query_with_io.rs b/datafusion/core/benches/sql_query_with_io.rs index aef39a04e47e..58f8409313aa 100644 --- a/datafusion/core/benches/sql_query_with_io.rs +++ b/datafusion/core/benches/sql_query_with_io.rs @@ -17,8 +17,7 @@ use std::{fmt::Write, sync::Arc, time::Duration}; -use arrow::array::{Int64Builder, UInt64Builder}; -use arrow_array::RecordBatch; +use arrow::array::{Int64Builder, RecordBatch, UInt64Builder}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use bytes::Bytes; use criterion::{criterion_group, criterion_main, Criterion, SamplingMode}; diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index eab3494be026..cd8a4dedc169 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -760,10 +760,10 @@ mod tests { use crate::prelude::{CsvReadOptions, SessionConfig, SessionContext}; use crate::test_util::arrow_test_data; + use arrow::array::{BooleanArray, Float64Array, Int32Array, StringArray}; use arrow::compute::concat_batches; use arrow::csv::ReaderBuilder; use arrow::util::pretty::pretty_format_batches; - use arrow_array::{BooleanArray, Float64Array, Int32Array, StringArray}; use datafusion_common::cast::as_string_array; use datafusion_common::internal_err; use datafusion_common::stats::Precision; diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 60609e467163..3aae692151f6 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -42,11 +42,11 @@ use crate::physical_plan::{ DisplayAs, DisplayFormatType, SendableRecordBatchStream, Statistics, }; +use arrow::array::RecordBatch; use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use arrow::json; use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; -use arrow_array::RecordBatch; use arrow_schema::ArrowError; use datafusion_catalog::Session; use datafusion_common::config::{ConfigField, ConfigFileType, JsonOptions}; diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index ab5ce91ec57d..383d2b14b31c 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -37,12 +37,12 @@ use std::fmt::{self, Debug, Display}; use std::sync::Arc; use std::task::Poll; +use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::SchemaRef; use crate::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; use crate::error::Result; use crate::physical_plan::{ExecutionPlan, Statistics}; -use arrow_array::RecordBatch; use arrow_schema::{ArrowError, DataType, Field, FieldRef, Schema}; use datafusion_catalog::Session; use datafusion_common::file_options::file_type::FileType; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 8b91bc2cfdbe..0b23fd8da612 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -1309,9 +1309,10 @@ mod tests { use crate::datasource::file_format::parquet::test_util::store_parquet; use crate::physical_plan::metrics::MetricValue; use crate::prelude::{ParquetReadOptions, SessionConfig, SessionContext}; - use arrow::array::{Array, ArrayRef, StringArray}; - use arrow_array::types::Int32Type; - use arrow_array::{DictionaryArray, Int32Array, Int64Array}; + use arrow::array::{ + types::Int32Type, Array, ArrayRef, DictionaryArray, Int32Array, Int64Array, + StringArray, + }; use arrow_schema::{DataType, Field}; use async_trait::async_trait; use datafusion_common::cast::{ diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index 48db2c080255..454666003254 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -27,9 +27,10 @@ use crate::datasource::physical_plan::FileSinkConfig; use crate::error::Result; use crate::physical_plan::SendableRecordBatchStream; -use arrow_array::builder::UInt64Builder; -use arrow_array::cast::AsArray; -use arrow_array::{downcast_dictionary_array, RecordBatch, StringArray, StructArray}; +use arrow::array::{ + builder::UInt64Builder, cast::AsArray, downcast_dictionary_array, RecordBatch, + StringArray, StructArray, +}; use arrow_schema::{DataType, Schema}; use datafusion_common::cast::{ as_boolean_array, as_date32_array, as_date64_array, as_int32_array, as_int64_array, diff --git a/datafusion/core/src/datasource/file_format/write/mod.rs b/datafusion/core/src/datasource/file_format/write/mod.rs index c064999c1e5b..81ecf3f0f88c 100644 --- a/datafusion/core/src/datasource/file_format/write/mod.rs +++ b/datafusion/core/src/datasource/file_format/write/mod.rs @@ -25,7 +25,7 @@ use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::physical_plan::FileSinkConfig; use crate::error::Result; -use arrow_array::RecordBatch; +use arrow::array::RecordBatch; use arrow_schema::Schema; use bytes::Bytes; use object_store::buffered::BufWriter; diff --git a/datafusion/core/src/datasource/file_format/write/orchestration.rs b/datafusion/core/src/datasource/file_format/write/orchestration.rs index 7a271def7dc6..75836d1b48b0 100644 --- a/datafusion/core/src/datasource/file_format/write/orchestration.rs +++ b/datafusion/core/src/datasource/file_format/write/orchestration.rs @@ -26,7 +26,7 @@ use super::{create_writer, BatchSerializer}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::error::Result; -use arrow_array::RecordBatch; +use arrow::array::RecordBatch; use datafusion_common::{internal_datafusion_err, internal_err, DataFusionError}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index f1d00ba9a37a..5d3a0f886fe4 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1998,7 +1998,7 @@ mod tests { // Create a new batch of data to insert into the table let batch = RecordBatch::try_new( schema.clone(), - vec![Arc::new(arrow_array::Int32Array::from(vec![ + vec![Arc::new(arrow::array::Int32Array::from(vec![ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, ]))], )?; diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index e31c7dfc247a..dc9207da51cb 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -32,10 +32,11 @@ use std::{ mem::size_of, sync::Arc, vec, }; -use arrow::array::{ArrayData, BufferBuilder}; +use arrow::array::{ + ArrayData, ArrayRef, BufferBuilder, DictionaryArray, RecordBatch, RecordBatchOptions, +}; use arrow::buffer::Buffer; use arrow::datatypes::{ArrowNativeType, UInt16Type}; -use arrow_array::{ArrayRef, DictionaryArray, RecordBatch, RecordBatchOptions}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_common::stats::Precision; use datafusion_common::{ @@ -850,7 +851,7 @@ fn create_output_array( #[cfg(test)] mod tests { - use arrow_array::Int32Array; + use arrow::array::Int32Array; use super::*; use crate::datasource::physical_plan::ArrowSource; diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 4cc65714d53e..873df859702a 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -580,9 +580,9 @@ mod tests { use super::*; use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; - use arrow_array::cast::AsArray; - use arrow_array::types::{Float32Type, Float64Type, UInt32Type}; - use arrow_array::{ + use arrow::array::{ + cast::AsArray, + types::{Float32Type, Float64Type, UInt32Type}, BinaryArray, BooleanArray, Float32Array, Int32Array, Int64Array, RecordBatch, StringArray, UInt64Array, }; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs index b008157a8324..27bfb26902e5 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs @@ -21,8 +21,8 @@ use std::sync::Arc; use super::{ParquetAccessPlan, ParquetFileMetrics}; use crate::datasource::listing::FileRange; -use arrow::{array::ArrayRef, datatypes::Schema}; -use arrow_array::BooleanArray; +use arrow::array::{ArrayRef, BooleanArray}; +use arrow::datatypes::Schema; use datafusion_common::{Column, Result, ScalarValue}; use datafusion_physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs index b4a8f377d256..64eb2b00de94 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/statistics.rs @@ -28,11 +28,11 @@ use std::sync::Arc; use crate::datasource::listing::PartitionedFile; +use arrow::array::RecordBatch; use arrow::{ compute::SortColumn, row::{Row, Rows}, }; -use arrow_array::RecordBatch; use arrow_schema::SchemaRef; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index 7e41e450ce23..efaae403b415 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -21,8 +21,8 @@ //! physical format into how they should be used by DataFusion. For instance, a schema //! can be stored external to a parquet file that maps parquet logical types to arrow types. +use arrow::array::{new_null_array, RecordBatch, RecordBatchOptions}; use arrow::compute::{can_cast_types, cast}; -use arrow_array::{new_null_array, RecordBatch, RecordBatchOptions}; use arrow_schema::{Schema, SchemaRef}; use datafusion_common::plan_err; use std::fmt::Debug; @@ -434,9 +434,9 @@ mod tests { use std::sync::Arc; use crate::assert_batches_sorted_eq; + use arrow::array::{Int32Array, StringArray}; use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; - use arrow_array::{Int32Array, StringArray}; use arrow_schema::{DataType, SchemaRef}; use object_store::path::Path; use object_store::ObjectMeta; diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index 56cbb126d02c..ff2e4436e94d 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -28,7 +28,7 @@ use std::sync::Arc; use crate::catalog::{TableProvider, TableProviderFactory}; use crate::datasource::create_ordering; -use arrow_array::{RecordBatch, RecordBatchReader, RecordBatchWriter}; +use arrow::array::{RecordBatch, RecordBatchReader, RecordBatchWriter}; use arrow_schema::SchemaRef; use datafusion_common::{config_err, plan_err, Constraints, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 4f752dc69be5..85c2b2a0fd78 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1990,7 +1990,7 @@ mod tests { use super::{SessionContextProvider, SessionStateBuilder}; use crate::datasource::MemTable; use crate::execution::context::SessionState; - use arrow_array::{ArrayRef, Int32Array, RecordBatch, StringArray}; + use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion_catalog::MemoryCatalogProviderList; use datafusion_common::DFSchema; diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 70b302a55c22..b256ed38039a 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -60,7 +60,7 @@ //! # use datafusion::prelude::*; //! # use datafusion::error::Result; //! # use datafusion::functions_aggregate::expr_fn::min; -//! # use datafusion::arrow::record_batch::RecordBatch; +//! # use datafusion::arrow::array::RecordBatch; //! //! # #[tokio::main] //! # async fn main() -> Result<()> { @@ -101,7 +101,7 @@ //! ``` //! # use datafusion::prelude::*; //! # use datafusion::error::Result; -//! # use datafusion::arrow::record_batch::RecordBatch; +//! # use datafusion::arrow::array::RecordBatch; //! //! # #[tokio::main] //! # async fn main() -> Result<()> { @@ -684,7 +684,7 @@ //! [`PhysicalOptimizerRule`]: datafusion_physical_optimizer::PhysicalOptimizerRule //! [`Schema`]: arrow::datatypes::Schema //! [`PhysicalExpr`]: physical_plan::PhysicalExpr -//! [`RecordBatch`]: arrow::record_batch::RecordBatch +//! [`RecordBatch`]: arrow::array::RecordBatch //! [`RecordBatchReader`]: arrow::record_batch::RecordBatchReader //! [`Array`]: arrow::array::Array diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index d96e60c25f40..edac0fb16381 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -59,10 +59,9 @@ use crate::physical_plan::{ Partitioning, PhysicalExpr, WindowExpr, }; +use arrow::array::{builder::StringBuilder, RecordBatch}; use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; -use arrow_array::builder::StringBuilder; -use arrow_array::RecordBatch; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; use datafusion_common::{ diff --git a/datafusion/core/tests/dataframe/dataframe_functions.rs b/datafusion/core/tests/dataframe/dataframe_functions.rs index 1bd90fce839d..89ec5a5908de 100644 --- a/datafusion/core/tests/dataframe/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe/dataframe_functions.rs @@ -15,13 +15,12 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::{types::Int32Type, ListArray}; use arrow::datatypes::{DataType, Field, Schema}; use arrow::{ array::{Int32Array, StringArray}, record_batch::RecordBatch, }; -use arrow_array::types::Int32Type; -use arrow_array::ListArray; use arrow_schema::SchemaRef; use std::sync::Arc; diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index acd5ee6d5ef0..772d9dbc8f46 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -19,21 +19,15 @@ mod dataframe_functions; mod describe; +use arrow::array::{ + record_batch, Array, ArrayRef, BooleanArray, DictionaryArray, FixedSizeListArray, + FixedSizeListBuilder, Float32Array, Float64Array, Int32Array, Int32Builder, + Int8Array, LargeListArray, ListArray, ListBuilder, RecordBatch, StringArray, + StringBuilder, StructBuilder, UInt32Array, UInt32Builder, UnionArray, +}; use arrow::buffer::ScalarBuffer; use arrow::datatypes::{DataType, Field, Float32Type, Int32Type, Schema, UInt64Type}; use arrow::util::pretty::pretty_format_batches; -use arrow::{ - array::{ - ArrayRef, FixedSizeListArray, FixedSizeListBuilder, Int32Array, Int32Builder, - LargeListArray, ListArray, ListBuilder, StringArray, StringBuilder, - StructBuilder, UInt32Array, UInt32Builder, - }, - record_batch::RecordBatch, -}; -use arrow_array::{ - record_batch, Array, BooleanArray, DictionaryArray, Float32Array, Float64Array, - Int8Array, UnionArray, -}; use arrow_schema::{ArrowError, SchemaRef, UnionFields, UnionMode}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::expr_fn::{ diff --git a/datafusion/core/tests/execution/logical_plan.rs b/datafusion/core/tests/execution/logical_plan.rs index 9f093ccf24b6..86acbe16474c 100644 --- a/datafusion/core/tests/execution/logical_plan.rs +++ b/datafusion/core/tests/execution/logical_plan.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use arrow_array::Int64Array; +use arrow::array::Int64Array; use arrow_schema::{DataType, Field}; use datafusion::execution::session_state::SessionStateBuilder; use datafusion_common::{Column, DFSchema, Result, ScalarValue, Spans}; diff --git a/datafusion/core/tests/expr_api/mod.rs b/datafusion/core/tests/expr_api/mod.rs index b9f1632ea695..8f8ca21c206d 100644 --- a/datafusion/core/tests/expr_api/mod.rs +++ b/datafusion/core/tests/expr_api/mod.rs @@ -15,9 +15,11 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::{ + builder::{ListBuilder, StringBuilder}, + ArrayRef, Int64Array, RecordBatch, StringArray, StructArray, +}; use arrow::util::pretty::{pretty_format_batches, pretty_format_columns}; -use arrow_array::builder::{ListBuilder, StringBuilder}; -use arrow_array::{ArrayRef, Int64Array, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Field}; use datafusion::prelude::*; use datafusion_common::{DFSchema, ScalarValue}; diff --git a/datafusion/core/tests/expr_api/simplification.rs b/datafusion/core/tests/expr_api/simplification.rs index 76df4a1f1105..83e96bffdf48 100644 --- a/datafusion/core/tests/expr_api/simplification.rs +++ b/datafusion/core/tests/expr_api/simplification.rs @@ -18,8 +18,8 @@ //! This program demonstrates the DataFusion expression simplification API. use arrow::array::types::IntervalDayTime; +use arrow::array::{ArrayRef, Int32Array}; use arrow::datatypes::{DataType, Field, Schema}; -use arrow_array::{ArrayRef, Int32Array}; use chrono::{DateTime, TimeZone, Utc}; use datafusion::{error::Result, execution::context::ExecutionProps, prelude::*}; use datafusion_common::cast::as_int32_array; diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index a58855438475..0257850ffc50 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -22,12 +22,10 @@ use crate::fuzz_cases::aggregation_fuzzer::{ AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, }; -use arrow::array::{Array, ArrayRef, AsArray, Int64Array}; +use arrow::array::{types::Int64Type, Array, ArrayRef, AsArray, Int64Array, RecordBatch}; use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::DataType; -use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; -use arrow_array::types::Int64Type; use arrow_schema::{ IntervalUnit, TimeUnit, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION, DECIMAL256_MAX_SCALE, diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs index 2aeecd8ff2ea..9c8f83f75ccb 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs @@ -253,7 +253,7 @@ impl SkipPartialParams { #[cfg(test)] mod test { - use arrow_array::{RecordBatch, StringArray, UInt32Array}; + use arrow::array::{RecordBatch, StringArray, UInt32Array}; use arrow_schema::{DataType, Field, Schema}; use crate::fuzz_cases::aggregation_fuzzer::check_equality_of_batches; diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs index e4c0cb6fe77f..3ebd899f4e15 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -17,6 +17,7 @@ use std::sync::Arc; +use arrow::array::{ArrayRef, RecordBatch}; use arrow::datatypes::{ BinaryType, BinaryViewType, BooleanType, ByteArrayType, ByteViewType, Date32Type, Date64Type, Decimal128Type, Decimal256Type, Float32Type, Float64Type, Int16Type, @@ -26,7 +27,6 @@ use arrow::datatypes::{ TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, Utf8Type, }; -use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::{DataType, Field, IntervalUnit, Schema, TimeUnit}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; @@ -728,7 +728,7 @@ impl RecordBatchGenerator { #[cfg(test)] mod test { - use arrow_array::UInt32Array; + use arrow::array::UInt32Array; use crate::fuzz_cases::aggregation_fuzzer::check_equality_of_batches; diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs index d021e73f35b2..db61eaef25c9 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs @@ -18,8 +18,8 @@ use std::collections::HashSet; use std::sync::Arc; +use arrow::array::RecordBatch; use arrow::util::pretty::pretty_format_batches; -use arrow_array::RecordBatch; use datafusion_common::{DataFusionError, Result}; use rand::{thread_rng, Rng}; use tokio::task::JoinSet; diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs index d93a5b7b9360..7c5b25e4a0e0 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::RecordBatch; use arrow::util::pretty::pretty_format_batches; -use arrow_array::RecordBatch; use datafusion::prelude::SessionContext; use datafusion_common::error::Result; diff --git a/datafusion/core/tests/fuzz_cases/distinct_count_string_fuzz.rs b/datafusion/core/tests/fuzz_cases/distinct_count_string_fuzz.rs index 64b858cebc84..d817e4c7a3b4 100644 --- a/datafusion/core/tests/fuzz_cases/distinct_count_string_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/distinct_count_string_fuzz.rs @@ -19,10 +19,8 @@ use std::sync::Arc; -use arrow::record_batch::RecordBatch; -use arrow_array::{Array, OffsetSizeTrait}; +use arrow::array::{cast::AsArray, Array, OffsetSizeTrait, RecordBatch}; -use arrow_array::cast::AsArray; use datafusion::datasource::MemTable; use std::collections::HashSet; use tokio::task::JoinSet; diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 5bf42ea6889f..b66b7b2aca43 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -22,9 +22,9 @@ use std::any::Any; use std::cmp::Ordering; use std::sync::Arc; +use arrow::array::{ArrayRef, Float32Array, Float64Array, RecordBatch, UInt32Array}; use arrow::compute::{lexsort_to_indices, take_record_batch, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; -use arrow_array::{ArrayRef, Float32Array, Float64Array, RecordBatch, UInt32Array}; use arrow_schema::{SchemaRef, SortOptions}; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, plan_datafusion_err, DataFusionError, Result}; diff --git a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs index a82849f4ea92..a73845c56a0f 100644 --- a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs @@ -17,10 +17,9 @@ //! Fuzz Test for Sort + Fetch/Limit (TopK!) +use arrow::array::{Float64Array, Int32Array, Int64Array, RecordBatch, StringArray}; use arrow::compute::concat_batches; use arrow::util::pretty::pretty_format_batches; -use arrow::{array::Int32Array, record_batch::RecordBatch}; -use arrow_array::{Float64Array, Int64Array, StringArray}; use arrow_schema::SchemaRef; use datafusion::datasource::MemTable; use datafusion::prelude::SessionContext; diff --git a/datafusion/core/tests/fuzz_cases/pruning.rs b/datafusion/core/tests/fuzz_cases/pruning.rs index a43886b6df21..fef009fa911c 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -17,7 +17,7 @@ use std::sync::{Arc, OnceLock}; -use arrow_array::{Array, RecordBatch, StringArray}; +use arrow::array::{Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; use bytes::{BufMut, Bytes, BytesMut}; use datafusion::{ diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 602205beadcc..8ffc78a9f59d 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -19,8 +19,8 @@ mod sp_repartition_fuzz_tests { use std::sync::Arc; + use arrow::array::{ArrayRef, Int64Array, RecordBatch, UInt64Array}; use arrow::compute::{concat_batches, lexsort, SortColumn}; - use arrow_array::{ArrayRef, Int64Array, RecordBatch, UInt64Array}; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion::physical_plan::{ diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 733d6cdee0ea..b7c656627187 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -23,9 +23,8 @@ use std::sync::{Arc, LazyLock}; #[cfg(feature = "extended_tests")] mod memory_limit_validation; +use arrow::array::{ArrayRef, DictionaryArray, RecordBatch}; use arrow::datatypes::{Int32Type, SchemaRef}; -use arrow::record_batch::RecordBatch; -use arrow_array::{ArrayRef, DictionaryArray}; use arrow_schema::SortOptions; use datafusion::assert_batches_eq; use datafusion::datasource::{MemTable, TableProvider}; diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index 3b9c43685deb..9175a6e91e91 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -17,10 +17,11 @@ use std::sync::Arc; +use arrow::array::{ + types::Int32Type, ArrayRef, DictionaryArray, Float32Array, Int64Array, RecordBatch, + StringArray, +}; use arrow::datatypes::{Field, Schema}; -use arrow::record_batch::RecordBatch; -use arrow_array::types::Int32Type; -use arrow_array::{ArrayRef, DictionaryArray, Float32Array, Int64Array, StringArray}; use arrow_schema::DataType; use datafusion::assert_batches_sorted_eq; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; diff --git a/datafusion/core/tests/user_defined/expr_planner.rs b/datafusion/core/tests/user_defined/expr_planner.rs index ad9c1280d6b1..75d890359ba8 100644 --- a/datafusion/core/tests/user_defined/expr_planner.rs +++ b/datafusion/core/tests/user_defined/expr_planner.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use arrow_array::RecordBatch; +use arrow::array::RecordBatch; use std::sync::Arc; use datafusion::common::{assert_batches_eq, DFSchema}; diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index bf32eef3b011..aa0f6c8fed8d 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -25,10 +25,10 @@ use std::sync::{ Arc, }; -use arrow::{array::AsArray, datatypes::Fields}; -use arrow_array::{ - types::UInt64Type, Int32Array, PrimitiveArray, StringArray, StructArray, +use arrow::array::{ + types::UInt64Type, AsArray, Int32Array, PrimitiveArray, StringArray, StructArray, }; +use arrow::datatypes::Fields; use arrow_schema::Schema; use datafusion::dataframe::DataFrame; @@ -834,7 +834,7 @@ impl GroupsAccumulator for TestGroupsAccumulator { &mut self, _values: &[ArrayRef], _group_indices: &[usize], - _opt_filter: Option<&arrow_array::BooleanArray>, + _opt_filter: Option<&arrow::array::BooleanArray>, _total_num_groups: usize, ) -> Result<()> { Ok(()) @@ -858,7 +858,7 @@ impl GroupsAccumulator for TestGroupsAccumulator { &mut self, _values: &[ArrayRef], _group_indices: &[usize], - _opt_filter: Option<&arrow_array::BooleanArray>, + _opt_filter: Option<&arrow::array::BooleanArray>, _total_num_groups: usize, ) -> Result<()> { Ok(()) diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index a228eb0286aa..ea83bd16b468 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -20,12 +20,11 @@ use std::hash::{DefaultHasher, Hash, Hasher}; use std::sync::Arc; use arrow::array::as_string_array; -use arrow::compute::kernels::numeric::add; -use arrow_array::builder::BooleanBuilder; -use arrow_array::cast::AsArray; -use arrow_array::{ - Array, ArrayRef, Float32Array, Float64Array, Int32Array, RecordBatch, StringArray, +use arrow::array::{ + builder::BooleanBuilder, cast::AsArray, Array, ArrayRef, Float32Array, Float64Array, + Int32Array, RecordBatch, StringArray, }; +use arrow::compute::kernels::numeric::add; use arrow_schema::{DataType, Field, Schema}; use datafusion::execution::context::{FunctionFactory, RegisterFunction, SessionState}; use datafusion::prelude::*; diff --git a/datafusion/core/tests/user_defined/user_defined_window_functions.rs b/datafusion/core/tests/user_defined/user_defined_window_functions.rs index 10ee0c5cd2dc..204d786994f8 100644 --- a/datafusion/core/tests/user_defined/user_defined_window_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_window_functions.rs @@ -18,17 +18,7 @@ //! This module contains end to end tests of creating //! user defined window functions -use std::{ - any::Any, - ops::Range, - sync::{ - atomic::{AtomicUsize, Ordering}, - Arc, - }, -}; - -use arrow::array::AsArray; -use arrow_array::{ArrayRef, Int64Array, RecordBatch, StringArray}; +use arrow::array::{ArrayRef, AsArray, Int64Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::{assert_batches_eq, prelude::SessionContext}; use datafusion_common::{Result, ScalarValue}; @@ -43,6 +33,14 @@ use datafusion_physical_expr::{ expressions::{col, lit}, PhysicalExpr, }; +use std::{ + any::Any, + ops::Range, + sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, + }, +}; /// A query with a window function evaluated over the entire partition const UNBOUNDED_WINDOW_QUERY: &str = "SELECT x, y, val, \ diff --git a/datafusion/ffi/Cargo.toml b/datafusion/ffi/Cargo.toml index 1a6248322d1c..c33c87786de8 100644 --- a/datafusion/ffi/Cargo.toml +++ b/datafusion/ffi/Cargo.toml @@ -38,7 +38,6 @@ crate-type = ["cdylib", "rlib"] [dependencies] abi_stable = "0.11.3" arrow = { workspace = true, features = ["ffi"] } -arrow-array = { workspace = true } arrow-schema = { workspace = true } async-ffi = { version = "0.5.0", features = ["abi_stable"] } async-trait = { workspace = true } diff --git a/datafusion/ffi/src/tests/async_provider.rs b/datafusion/ffi/src/tests/async_provider.rs index 0e56a318cd87..eff3ed61d739 100644 --- a/datafusion/ffi/src/tests/async_provider.rs +++ b/datafusion/ffi/src/tests/async_provider.rs @@ -28,7 +28,7 @@ use std::{any::Any, fmt::Debug, sync::Arc}; use crate::table_provider::FFI_TableProvider; -use arrow_array::RecordBatch; +use arrow::array::RecordBatch; use arrow_schema::Schema; use async_trait::async_trait; use datafusion::{ diff --git a/datafusion/ffi/src/tests/mod.rs b/datafusion/ffi/src/tests/mod.rs index d2e865d6c2a1..a5fc74b840d1 100644 --- a/datafusion/ffi/src/tests/mod.rs +++ b/datafusion/ffi/src/tests/mod.rs @@ -27,7 +27,7 @@ use abi_stable::{ }; use super::table_provider::FFI_TableProvider; -use arrow_array::RecordBatch; +use arrow::array::RecordBatch; use async_provider::create_async_table_provider; use datafusion::{ arrow::datatypes::{DataType, Field, Schema}, diff --git a/datafusion/functions-nested/Cargo.toml b/datafusion/functions-nested/Cargo.toml index 01fbc73cba12..7835985b297f 100644 --- a/datafusion/functions-nested/Cargo.toml +++ b/datafusion/functions-nested/Cargo.toml @@ -41,7 +41,6 @@ path = "src/lib.rs" [dependencies] arrow = { workspace = true } -arrow-array = { workspace = true } arrow-ord = { workspace = true } arrow-schema = { workspace = true } datafusion-common = { workspace = true } diff --git a/datafusion/functions-nested/benches/map.rs b/datafusion/functions-nested/benches/map.rs index 22bef99becef..e60f7f388ac1 100644 --- a/datafusion/functions-nested/benches/map.rs +++ b/datafusion/functions-nested/benches/map.rs @@ -17,8 +17,8 @@ extern crate criterion; +use arrow::array::{Int32Array, ListArray, StringArray}; use arrow::buffer::{OffsetBuffer, ScalarBuffer}; -use arrow_array::{Int32Array, ListArray, StringArray}; use arrow_schema::{DataType, Field}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use rand::prelude::ThreadRng; diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index e56f5633b2a5..5c694600b822 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -17,11 +17,12 @@ //! [`ScalarUDFImpl`] definitions for array_has, array_has_all and array_has_any functions. -use arrow::array::{Array, ArrayRef, BooleanArray, OffsetSizeTrait}; +use arrow::array::{ + Array, ArrayRef, BooleanArray, Datum, GenericListArray, OffsetSizeTrait, Scalar, +}; use arrow::buffer::BooleanBuffer; use arrow::datatypes::DataType; use arrow::row::{RowConverter, Rows, SortField}; -use arrow_array::{Datum, GenericListArray, Scalar}; use datafusion_common::cast::as_generic_list_array; use datafusion_common::utils::string_utils::string_array_to_vec; use datafusion_common::{exec_err, Result, ScalarValue}; diff --git a/datafusion/functions-nested/src/cardinality.rs b/datafusion/functions-nested/src/cardinality.rs index a46c5348d123..21ab9fb35982 100644 --- a/datafusion/functions-nested/src/cardinality.rs +++ b/datafusion/functions-nested/src/cardinality.rs @@ -18,7 +18,7 @@ //! [`ScalarUDFImpl`] definitions for cardinality function. use crate::utils::make_scalar_function; -use arrow_array::{ +use arrow::array::{ Array, ArrayRef, GenericListArray, MapArray, OffsetSizeTrait, UInt64Array, }; use arrow_schema::DataType; diff --git a/datafusion/functions-nested/src/concat.rs b/datafusion/functions-nested/src/concat.rs index 0e98c31ba663..723dab9a76b7 100644 --- a/datafusion/functions-nested/src/concat.rs +++ b/datafusion/functions-nested/src/concat.rs @@ -533,7 +533,7 @@ where Ok(Arc::new(GenericListArray::::try_new( Arc::new(Field::new_list_field(data_type.to_owned(), true)), OffsetBuffer::new(offsets.into()), - arrow_array::make_array(data), + arrow::array::make_array(data), None, )?)) } diff --git a/datafusion/functions-nested/src/distance.rs b/datafusion/functions-nested/src/distance.rs index 8559b1096bc5..6a5d6b4fa0ff 100644 --- a/datafusion/functions-nested/src/distance.rs +++ b/datafusion/functions-nested/src/distance.rs @@ -18,7 +18,7 @@ //! [ScalarUDFImpl] definitions for array_distance function. use crate::utils::make_scalar_function; -use arrow_array::{ +use arrow::array::{ Array, ArrayRef, Float64Array, LargeListArray, ListArray, OffsetSizeTrait, }; use arrow_schema::DataType; diff --git a/datafusion/functions-nested/src/empty.rs b/datafusion/functions-nested/src/empty.rs index 9739ffb15f6b..b5e2df6f8952 100644 --- a/datafusion/functions-nested/src/empty.rs +++ b/datafusion/functions-nested/src/empty.rs @@ -18,7 +18,7 @@ //! [`ScalarUDFImpl`] definitions for array_empty function. use crate::utils::make_scalar_function; -use arrow_array::{ArrayRef, BooleanArray, OffsetSizeTrait}; +use arrow::array::{ArrayRef, BooleanArray, OffsetSizeTrait}; use arrow_schema::DataType; use arrow_schema::DataType::{Boolean, FixedSizeList, LargeList, List}; use datafusion_common::cast::as_generic_list_array; diff --git a/datafusion/functions-nested/src/except.rs b/datafusion/functions-nested/src/except.rs index 8cb870dba058..79e2c0f23ce3 100644 --- a/datafusion/functions-nested/src/except.rs +++ b/datafusion/functions-nested/src/except.rs @@ -18,10 +18,9 @@ //! [`ScalarUDFImpl`] definitions for array_except function. use crate::utils::{check_datatypes, make_scalar_function}; +use arrow::array::{cast::AsArray, Array, ArrayRef, GenericListArray, OffsetSizeTrait}; use arrow::buffer::OffsetBuffer; use arrow::row::{RowConverter, SortField}; -use arrow_array::cast::AsArray; -use arrow_array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait}; use arrow_schema::{DataType, FieldRef}; use datafusion_common::{exec_err, internal_err, HashSet, Result}; use datafusion_expr::{ diff --git a/datafusion/functions-nested/src/extract.rs b/datafusion/functions-nested/src/extract.rs index 33f7904c1687..2f59dccad94a 100644 --- a/datafusion/functions-nested/src/extract.rs +++ b/datafusion/functions-nested/src/extract.rs @@ -633,7 +633,7 @@ where Ok(Arc::new(GenericListArray::::try_new( Arc::new(Field::new_list_field(array.value_type(), true)), OffsetBuffer::::new(offsets.into()), - arrow_array::make_array(data), + arrow::array::make_array(data), null_builder.finish(), )?)) } diff --git a/datafusion/functions-nested/src/flatten.rs b/datafusion/functions-nested/src/flatten.rs index b97b9e3c68a9..4fd14c79644b 100644 --- a/datafusion/functions-nested/src/flatten.rs +++ b/datafusion/functions-nested/src/flatten.rs @@ -18,8 +18,8 @@ //! [`ScalarUDFImpl`] definitions for flatten function. use crate::utils::make_scalar_function; +use arrow::array::{ArrayRef, GenericListArray, OffsetSizeTrait}; use arrow::buffer::OffsetBuffer; -use arrow_array::{ArrayRef, GenericListArray, OffsetSizeTrait}; use arrow_schema::DataType; use arrow_schema::DataType::{FixedSizeList, LargeList, List, Null}; use datafusion_common::cast::{ diff --git a/datafusion/functions-nested/src/length.rs b/datafusion/functions-nested/src/length.rs index 70a9188a2c3d..1081a682897f 100644 --- a/datafusion/functions-nested/src/length.rs +++ b/datafusion/functions-nested/src/length.rs @@ -18,7 +18,7 @@ //! [`ScalarUDFImpl`] definitions for array_length function. use crate::utils::make_scalar_function; -use arrow_array::{ +use arrow::array::{ Array, ArrayRef, Int64Array, LargeListArray, ListArray, OffsetSizeTrait, UInt64Array, }; use arrow_schema::DataType; diff --git a/datafusion/functions-nested/src/make_array.rs b/datafusion/functions-nested/src/make_array.rs index 6bc1f6f2c2d1..8bf5f37b8add 100644 --- a/datafusion/functions-nested/src/make_array.rs +++ b/datafusion/functions-nested/src/make_array.rs @@ -22,11 +22,11 @@ use std::sync::Arc; use std::vec; use crate::utils::make_scalar_function; -use arrow::array::{ArrayData, Capacities, MutableArrayData}; -use arrow::buffer::OffsetBuffer; -use arrow_array::{ - new_null_array, Array, ArrayRef, GenericListArray, NullArray, OffsetSizeTrait, +use arrow::array::{ + new_null_array, Array, ArrayData, ArrayRef, Capacities, GenericListArray, + MutableArrayData, NullArray, OffsetSizeTrait, }; +use arrow::buffer::OffsetBuffer; use arrow_schema::DataType::{List, Null}; use arrow_schema::{DataType, Field}; use datafusion_common::utils::SingleRowListArrayBuilder; @@ -275,7 +275,7 @@ fn array_array( Ok(Arc::new(GenericListArray::::try_new( Arc::new(Field::new_list_field(data_type, true)), OffsetBuffer::new(offsets.into()), - arrow_array::make_array(data), + arrow::array::make_array(data), None, )?)) } diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index 8c78de68f86e..77e06b28a8d6 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -19,10 +19,9 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; -use arrow::array::ArrayData; +use arrow::array::{Array, ArrayData, ArrayRef, MapArray, OffsetSizeTrait, StructArray}; use arrow::buffer::Buffer; use arrow::datatypes::ToByteSlice; -use arrow_array::{Array, ArrayRef, MapArray, OffsetSizeTrait, StructArray}; use arrow_schema::{DataType, Field, SchemaBuilder}; use datafusion_common::utils::{fixed_size_list_to_arrays, list_to_arrays}; diff --git a/datafusion/functions-nested/src/map_extract.rs b/datafusion/functions-nested/src/map_extract.rs index 268c3235cc49..47d977a8c01c 100644 --- a/datafusion/functions-nested/src/map_extract.rs +++ b/datafusion/functions-nested/src/map_extract.rs @@ -17,12 +17,11 @@ //! [`ScalarUDFImpl`] definitions for map_extract functions. -use arrow::array::{ArrayRef, Capacities, MutableArrayData}; -use arrow_array::{make_array, ListArray}; - +use arrow::array::{ + make_array, Array, ArrayRef, Capacities, ListArray, MapArray, MutableArrayData, +}; use arrow::buffer::OffsetBuffer; use arrow::datatypes::DataType; -use arrow_array::{Array, MapArray}; use arrow_schema::Field; use datafusion_common::{cast::as_map_array, exec_err, Result}; diff --git a/datafusion/functions-nested/src/map_keys.rs b/datafusion/functions-nested/src/map_keys.rs index 0dfd34b39089..60039865daae 100644 --- a/datafusion/functions-nested/src/map_keys.rs +++ b/datafusion/functions-nested/src/map_keys.rs @@ -18,7 +18,7 @@ //! [`ScalarUDFImpl`] definitions for map_keys function. use crate::utils::{get_map_entry_field, make_scalar_function}; -use arrow_array::{Array, ArrayRef, ListArray}; +use arrow::array::{Array, ArrayRef, ListArray}; use arrow_schema::{DataType, Field}; use datafusion_common::{cast::as_map_array, exec_err, Result}; use datafusion_expr::{ diff --git a/datafusion/functions-nested/src/map_values.rs b/datafusion/functions-nested/src/map_values.rs index 009d5c861093..c6d31f3d9067 100644 --- a/datafusion/functions-nested/src/map_values.rs +++ b/datafusion/functions-nested/src/map_values.rs @@ -18,7 +18,7 @@ //! [`ScalarUDFImpl`] definitions for map_values function. use crate::utils::{get_map_entry_field, make_scalar_function}; -use arrow_array::{Array, ArrayRef, ListArray}; +use arrow::array::{Array, ArrayRef, ListArray}; use arrow_schema::{DataType, Field}; use datafusion_common::{cast::as_map_array, exec_err, Result}; use datafusion_expr::{ diff --git a/datafusion/functions-nested/src/position.rs b/datafusion/functions-nested/src/position.rs index eec2a32fa2a2..d5c9944709b3 100644 --- a/datafusion/functions-nested/src/position.rs +++ b/datafusion/functions-nested/src/position.rs @@ -27,9 +27,9 @@ use datafusion_macros::user_doc; use std::any::Any; use std::sync::Arc; -use arrow_array::types::UInt64Type; -use arrow_array::{ - Array, ArrayRef, GenericListArray, ListArray, OffsetSizeTrait, UInt64Array, +use arrow::array::{ + types::UInt64Type, Array, ArrayRef, GenericListArray, ListArray, OffsetSizeTrait, + UInt64Array, }; use datafusion_common::cast::{ as_generic_list_array, as_int64_array, as_large_list_array, as_list_array, diff --git a/datafusion/functions-nested/src/remove.rs b/datafusion/functions-nested/src/remove.rs index 64b6405176a3..099cc7e1131d 100644 --- a/datafusion/functions-nested/src/remove.rs +++ b/datafusion/functions-nested/src/remove.rs @@ -19,11 +19,11 @@ use crate::utils; use crate::utils::make_scalar_function; -use arrow::buffer::OffsetBuffer; -use arrow_array::cast::AsArray; -use arrow_array::{ - new_empty_array, Array, ArrayRef, BooleanArray, GenericListArray, OffsetSizeTrait, +use arrow::array::{ + cast::AsArray, new_empty_array, Array, ArrayRef, BooleanArray, GenericListArray, + OffsetSizeTrait, }; +use arrow::buffer::OffsetBuffer; use arrow_schema::{DataType, Field}; use datafusion_common::cast::as_int64_array; use datafusion_common::{exec_err, Result}; @@ -330,7 +330,7 @@ fn array_remove_internal( /// /// The type of each **element** in `list_array` must be the same as the type of /// `element_array`. This function also handles nested arrays -/// ([`arrow_array::ListArray`] of [`arrow_array::ListArray`]s) +/// ([`arrow::array::ListArray`] of [`arrow::array::ListArray`]s) /// /// For example, when called to remove a list array (where each element is a /// list of int32s, the second argument are int32 arrays, and the diff --git a/datafusion/functions-nested/src/repeat.rs b/datafusion/functions-nested/src/repeat.rs index 455fb3dd3023..4772da9a4bf4 100644 --- a/datafusion/functions-nested/src/repeat.rs +++ b/datafusion/functions-nested/src/repeat.rs @@ -18,14 +18,13 @@ //! [`ScalarUDFImpl`] definitions for array_repeat function. use crate::utils::make_scalar_function; -use arrow::array::{Capacities, MutableArrayData}; +use arrow::array::{ + new_null_array, Array, ArrayRef, Capacities, GenericListArray, ListArray, + MutableArrayData, OffsetSizeTrait, UInt64Array, +}; use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::compute::cast; -use arrow_array::{ - new_null_array, Array, ArrayRef, GenericListArray, ListArray, OffsetSizeTrait, - UInt64Array, -}; use arrow_schema::DataType::{LargeList, List}; use arrow_schema::{DataType, Field}; use datafusion_common::cast::{as_large_list_array, as_list_array, as_uint64_array}; @@ -220,7 +219,7 @@ fn general_repeat( } let data = mutable.freeze(); - arrow_array::make_array(data) + arrow::array::make_array(data) }; new_values.push(repeated_array); } @@ -277,7 +276,7 @@ fn general_list_repeat( } let data = mutable.freeze(); - let repeated_array = arrow_array::make_array(data); + let repeated_array = arrow::array::make_array(data); let list_arr = GenericListArray::::try_new( Arc::new(Field::new_list_field(value_type.clone(), true)), diff --git a/datafusion/functions-nested/src/replace.rs b/datafusion/functions-nested/src/replace.rs index 1f12625a52b8..939fce6fdf3f 100644 --- a/datafusion/functions-nested/src/replace.rs +++ b/datafusion/functions-nested/src/replace.rs @@ -375,7 +375,7 @@ fn general_replace( Ok(Arc::new(GenericListArray::::try_new( Arc::new(Field::new_list_field(list_array.value_type(), true)), OffsetBuffer::::new(offsets.into()), - arrow_array::make_array(data), + arrow::array::make_array(data), valid.finish(), )?)) } diff --git a/datafusion/functions-nested/src/resize.rs b/datafusion/functions-nested/src/resize.rs index f167134f9b22..3cd7bb5dac81 100644 --- a/datafusion/functions-nested/src/resize.rs +++ b/datafusion/functions-nested/src/resize.rs @@ -239,7 +239,7 @@ fn general_list_resize>( Ok(Arc::new(GenericListArray::::try_new( Arc::clone(field), OffsetBuffer::::new(offsets.into()), - arrow_array::make_array(data), + arrow::array::make_array(data), null_builder.finish(), )?)) } diff --git a/datafusion/functions-nested/src/reverse.rs b/datafusion/functions-nested/src/reverse.rs index 9fd955094ae6..a60f84cb0320 100644 --- a/datafusion/functions-nested/src/reverse.rs +++ b/datafusion/functions-nested/src/reverse.rs @@ -18,9 +18,10 @@ //! [`ScalarUDFImpl`] definitions for array_reverse function. use crate::utils::make_scalar_function; -use arrow::array::{Capacities, MutableArrayData}; +use arrow::array::{ + Array, ArrayRef, Capacities, GenericListArray, MutableArrayData, OffsetSizeTrait, +}; use arrow::buffer::OffsetBuffer; -use arrow_array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait}; use arrow_schema::DataType::{LargeList, List, Null}; use arrow_schema::{DataType, FieldRef}; use datafusion_common::cast::{as_large_list_array, as_list_array}; @@ -173,7 +174,7 @@ fn general_array_reverse>( Ok(Arc::new(GenericListArray::::try_new( Arc::clone(field), OffsetBuffer::::new(offsets.into()), - arrow_array::make_array(data), + arrow::array::make_array(data), Some(nulls.into()), )?)) } diff --git a/datafusion/functions-nested/src/string.rs b/datafusion/functions-nested/src/string.rs index bbe1dc2a01e9..1a0676aa39d5 100644 --- a/datafusion/functions-nested/src/string.rs +++ b/datafusion/functions-nested/src/string.rs @@ -33,10 +33,12 @@ use datafusion_common::{ use std::any::Any; use crate::utils::make_scalar_function; +use arrow::array::{ + builder::{ArrayBuilder, LargeStringBuilder, StringViewBuilder}, + cast::AsArray, + GenericStringArray, StringArrayType, StringViewArray, +}; use arrow::compute::cast; -use arrow_array::builder::{ArrayBuilder, LargeStringBuilder, StringViewBuilder}; -use arrow_array::cast::AsArray; -use arrow_array::{GenericStringArray, StringArrayType, StringViewArray}; use arrow_schema::DataType::{ Dictionary, FixedSizeList, LargeList, LargeUtf8, List, Null, Utf8, Utf8View, }; diff --git a/datafusion/functions-nested/src/utils.rs b/datafusion/functions-nested/src/utils.rs index e1961dccf54a..5dd812a23b9a 100644 --- a/datafusion/functions-nested/src/utils.rs +++ b/datafusion/functions-nested/src/utils.rs @@ -19,13 +19,13 @@ use std::sync::Arc; -use arrow::{array::ArrayRef, datatypes::DataType}; +use arrow::datatypes::DataType; -use arrow::buffer::OffsetBuffer; -use arrow_array::{ - Array, BooleanArray, GenericListArray, ListArray, OffsetSizeTrait, Scalar, +use arrow::array::{ + Array, ArrayRef, BooleanArray, GenericListArray, ListArray, OffsetSizeTrait, Scalar, UInt32Array, }; +use arrow::buffer::OffsetBuffer; use arrow_schema::{Field, Fields}; use datafusion_common::cast::{as_large_list_array, as_list_array}; use datafusion_common::{ diff --git a/datafusion/functions/src/datetime/mod.rs b/datafusion/functions/src/datetime/mod.rs index 96ca63010ee4..eec5e3cef624 100644 --- a/datafusion/functions/src/datetime/mod.rs +++ b/datafusion/functions/src/datetime/mod.rs @@ -136,7 +136,7 @@ pub mod expr_fn { /// # use datafusion::error::Result; /// # use datafusion_common::ScalarValue::TimestampNanosecond; /// # use std::sync::Arc; - /// # use arrow_array::{Date32Array, RecordBatch, StringArray}; + /// # use arrow::array::{Date32Array, RecordBatch, StringArray}; /// # use arrow_schema::{DataType, Field, Schema}; /// # #[tokio::main] /// # async fn main() -> Result<()> { diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index d93a402db318..33983676472b 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -38,7 +38,6 @@ path = "src/lib.rs" [dependencies] ahash = { workspace = true } arrow = { workspace = true } -arrow-array = { workspace = true } arrow-schema = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } diff --git a/datafusion/physical-expr/benches/case_when.rs b/datafusion/physical-expr/benches/case_when.rs index 9eda1277c263..480b1043fbf5 100644 --- a/datafusion/physical-expr/benches/case_when.rs +++ b/datafusion/physical-expr/benches/case_when.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::builder::{Int32Builder, StringBuilder}; use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; -use arrow_array::builder::{Int32Builder, StringBuilder}; use arrow_schema::DataType; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-expr/benches/is_null.rs b/datafusion/physical-expr/benches/is_null.rs index 7d26557afb1b..ed393b8900f1 100644 --- a/datafusion/physical-expr/benches/is_null.rs +++ b/datafusion/physical-expr/benches/is_null.rs @@ -15,9 +15,8 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::{builder::Int32Builder, RecordBatch}; use arrow::datatypes::{Field, Schema}; -use arrow::record_batch::RecordBatch; -use arrow_array::builder::Int32Builder; use arrow_schema::DataType; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_physical_expr::expressions::{Column, IsNotNullExpr, IsNullExpr}; diff --git a/datafusion/physical-expr/src/expressions/is_not_null.rs b/datafusion/physical-expr/src/expressions/is_not_null.rs index 8e3544622b80..47dc53d12555 100644 --- a/datafusion/physical-expr/src/expressions/is_not_null.rs +++ b/datafusion/physical-expr/src/expressions/is_not_null.rs @@ -115,12 +115,11 @@ pub fn is_not_null(arg: Arc) -> Result> mod tests { use super::*; use crate::expressions::col; - use arrow::buffer::ScalarBuffer; - use arrow::{ - array::{BooleanArray, StringArray}, - datatypes::*, + use arrow::array::{ + Array, BooleanArray, Float64Array, Int32Array, StringArray, UnionArray, }; - use arrow_array::{Array, Float64Array, Int32Array, UnionArray}; + use arrow::buffer::ScalarBuffer; + use arrow::datatypes::*; use datafusion_common::cast::as_boolean_array; #[test] diff --git a/datafusion/physical-expr/src/expressions/is_null.rs b/datafusion/physical-expr/src/expressions/is_null.rs index ca8d67230557..5e883dff997a 100644 --- a/datafusion/physical-expr/src/expressions/is_null.rs +++ b/datafusion/physical-expr/src/expressions/is_null.rs @@ -114,12 +114,11 @@ pub fn is_null(arg: Arc) -> Result> { mod tests { use super::*; use crate::expressions::col; - use arrow::buffer::ScalarBuffer; - use arrow::{ - array::{BooleanArray, StringArray}, - datatypes::*, + use arrow::array::{ + Array, BooleanArray, Float64Array, Int32Array, StringArray, UnionArray, }; - use arrow_array::{Array, Float64Array, Int32Array, UnionArray}; + use arrow::buffer::ScalarBuffer; + use arrow::datatypes::*; use datafusion_common::cast::as_boolean_array; #[test] diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index e05de362bf14..8504705f2a09 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -392,7 +392,7 @@ pub fn logical2physical(expr: &Expr, schema: &Schema) -> Arc { #[cfg(test)] mod tests { - use arrow_array::{ArrayRef, BooleanArray, RecordBatch, StringArray}; + use arrow::array::{ArrayRef, BooleanArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Field}; use datafusion_expr::{col, lit}; diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 936adbc098d6..bd38fb22ccbc 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -37,9 +37,8 @@ use std::sync::Arc; use crate::expressions::Literal; use crate::PhysicalExpr; +use arrow::array::{Array, RecordBatch}; use arrow::datatypes::{DataType, Schema}; -use arrow::record_batch::RecordBatch; -use arrow_array::Array; use datafusion_common::{internal_err, DFSchema, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::ExprProperties; diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index c06efd554098..25769bef7200 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -258,7 +258,7 @@ pub(crate) mod tests { use super::*; use crate::expressions::{binary, cast, col, in_list, lit, Literal}; - use arrow_array::{ArrayRef, Float32Array, Float64Array}; + use arrow::array::{ArrayRef, Float32Array, Float64Array}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{exec_err, DataFusionError, ScalarValue}; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 8b130506cdea..be7d080b683c 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -552,7 +552,7 @@ mod tests { use crate::window::window_expr::is_row_ahead; - use arrow_array::{ArrayRef, Float64Array}; + use arrow::array::{ArrayRef, Float64Array}; use arrow_schema::SortOptions; use datafusion_common::Result; diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index a002e3861f11..b84243b1b56b 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -41,7 +41,6 @@ path = "src/lib.rs" [dependencies] ahash = { workspace = true } arrow = { workspace = true } -arrow-array = { workspace = true } arrow-ord = { workspace = true } arrow-schema = { workspace = true } async-trait = { workspace = true } diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 08feb9bbe04f..3a2ecb57394b 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -17,8 +17,7 @@ use std::sync::Arc; -use arrow::record_batch::RecordBatch; -use arrow_array::{ArrayRef, Int32Array, Int64Array, StringArray}; +use arrow::array::{ArrayRef, Int32Array, Int64Array, RecordBatch, StringArray}; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index e4a7eb049e9e..4cbeed9951f9 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -17,13 +17,12 @@ //! [`GroupValues`] trait for storing and interning group keys -use arrow::record_batch::RecordBatch; -use arrow_array::types::{ +use arrow::array::types::{ Date32Type, Date64Type, Decimal128Type, Time32MillisecondType, Time32SecondType, Time64MicrosecondType, Time64NanosecondType, TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, }; -use arrow_array::{downcast_primitive, ArrayRef}; +use arrow::array::{downcast_primitive, ArrayRef, RecordBatch}; use arrow_schema::TimeUnit; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs index d170411b833c..b6d97b5d788d 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs @@ -17,10 +17,9 @@ use crate::aggregates::group_values::multi_group_by::{nulls_equal_to, GroupColumn}; use crate::aggregates::group_values::null_builder::MaybeNullBufferBuilder; -use arrow::array::{make_view, AsArray, ByteView}; +use arrow::array::{make_view, Array, ArrayRef, AsArray, ByteView, GenericByteViewArray}; use arrow::buffer::{Buffer, ScalarBuffer}; use arrow::datatypes::ByteViewType; -use arrow_array::{Array, ArrayRef, GenericByteViewArray}; use itertools::izip; use std::marker::PhantomData; use std::mem::{replace, size_of}; diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs index 540f9c3c6480..96885f03146c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs @@ -29,6 +29,7 @@ use crate::aggregates::group_values::multi_group_by::{ }; use crate::aggregates::group_values::GroupValues; use ahash::RandomState; +use arrow::array::{Array, ArrayRef, RecordBatch}; use arrow::compute::cast; use arrow::datatypes::{ BinaryViewType, Date32Type, Date64Type, Decimal128Type, Float32Type, Float64Type, @@ -37,8 +38,6 @@ use arrow::datatypes::{ TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; -use arrow::record_batch::RecordBatch; -use arrow_array::{Array, ArrayRef}; use arrow_schema::{DataType, Schema, SchemaRef, TimeUnit}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -1236,8 +1235,8 @@ fn supported_type(data_type: &DataType) -> bool { mod tests { use std::{collections::HashMap, sync::Arc}; + use arrow::array::{ArrayRef, Int64Array, RecordBatch, StringArray, StringViewArray}; use arrow::{compute::concat_batches, util::pretty::pretty_format_batches}; - use arrow_array::{ArrayRef, Int64Array, RecordBatch, StringArray, StringViewArray}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_common::utils::proxy::HashTableAllocExt; use datafusion_expr::EmitTo; diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index edc3f909bbd6..a0331bf3fa3d 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -17,10 +17,9 @@ use crate::aggregates::group_values::GroupValues; use ahash::RandomState; +use arrow::array::{Array, ArrayRef, ListArray, RecordBatch, StructArray}; use arrow::compute::cast; -use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, Rows, SortField}; -use arrow_array::{Array, ArrayRef, ListArray, StructArray}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs index 013c027e7306..9686b8c3521d 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs @@ -16,7 +16,7 @@ // under the License. use crate::aggregates::group_values::GroupValues; -use arrow_array::{Array, ArrayRef, OffsetSizeTrait, RecordBatch}; +use arrow::array::{Array, ArrayRef, OffsetSizeTrait, RecordBatch}; use datafusion_expr::EmitTo; use datafusion_physical_expr_common::binary_map::{ArrowBytesMap, OutputType}; use std::mem::size_of; diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs index 7379b7a538b4..be9a0334e3ee 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs @@ -16,7 +16,7 @@ // under the License. use crate::aggregates::group_values::GroupValues; -use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow::array::{Array, ArrayRef, RecordBatch}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; use datafusion_physical_expr_common::binary_view_map::ArrowBytesViewMap; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b4f54b0d943a..85b41da85742 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -34,10 +34,9 @@ use crate::{ SendableRecordBatchStream, Statistics, }; -use arrow::array::ArrayRef; +use arrow::array::{ArrayRef, UInt16Array, UInt32Array, UInt64Array, UInt8Array}; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use arrow_array::{UInt16Array, UInt32Array, UInt64Array, UInt8Array}; use datafusion_common::stats::Precision; use datafusion_common::{internal_err, not_impl_err, Constraint, Constraints, Result}; use datafusion_execution::TaskContext; @@ -1354,12 +1353,12 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::RecordBatchStream; - use arrow::array::{Float64Array, UInt32Array}; + use arrow::array::{ + DictionaryArray, Float32Array, Float64Array, Int32Array, StructArray, + UInt32Array, UInt64Array, + }; use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::{DataType, Int32Type}; - use arrow_array::{ - DictionaryArray, Float32Array, Int32Array, StructArray, UInt64Array, - }; use datafusion_common::{ assert_batches_eq, assert_batches_sorted_eq, internal_err, DataFusionError, ScalarValue, diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 7d9a50e20ae0..61a0ab8b247d 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use arrow_array::ArrayRef; +use arrow::array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; use datafusion_expr::EmitTo; diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index 5a05b88798ef..30655cd0ad59 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::ArrayRef; use arrow::row::{OwnedRow, RowConverter, Rows, SortField}; -use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; diff --git a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs index 514214858fa1..8c7ba21b37c0 100644 --- a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs +++ b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs @@ -21,12 +21,11 @@ use crate::aggregates::group_values::HashValue; use crate::aggregates::topk::heap::Comparable; use ahash::RandomState; use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; -use arrow::datatypes::i256; -use arrow_array::builder::PrimitiveBuilder; -use arrow_array::cast::AsArray; -use arrow_array::{ - downcast_primitive, Array, ArrayRef, ArrowPrimitiveType, PrimitiveArray, StringArray, +use arrow::array::{ + builder::PrimitiveBuilder, cast::AsArray, downcast_primitive, Array, ArrayRef, + ArrowPrimitiveType, PrimitiveArray, StringArray, }; +use arrow::datatypes::i256; use arrow_schema::DataType; use datafusion_common::DataFusionError; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/aggregates/topk/heap.rs b/datafusion/physical-plan/src/aggregates/topk/heap.rs index fc68df9b82ed..09dae3df0a96 100644 --- a/datafusion/physical-plan/src/aggregates/topk/heap.rs +++ b/datafusion/physical-plan/src/aggregates/topk/heap.rs @@ -17,11 +17,13 @@ //! A custom binary heap implementation for performant top K aggregation -use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::array::{ + cast::AsArray, + types::{IntervalDayTime, IntervalMonthDayNano}, +}; +use arrow::array::{downcast_primitive, ArrayRef, ArrowPrimitiveType, PrimitiveArray}; use arrow::buffer::ScalarBuffer; use arrow::datatypes::i256; -use arrow_array::cast::AsArray; -use arrow_array::{downcast_primitive, ArrayRef, ArrowPrimitiveType, PrimitiveArray}; use arrow_schema::DataType; use datafusion_common::DataFusionError; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/aggregates/topk/priority_map.rs b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs index ed41d22e935b..3cb12f0af089 100644 --- a/datafusion/physical-plan/src/aggregates/topk/priority_map.rs +++ b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs @@ -19,7 +19,7 @@ use crate::aggregates::topk::hash_table::{new_hash_table, ArrowHashTable}; use crate::aggregates::topk::heap::{new_heap, ArrowHeap}; -use arrow_array::ArrayRef; +use arrow::array::ArrayRef; use arrow_schema::DataType; use datafusion_common::Result; @@ -108,8 +108,8 @@ impl PriorityMap { #[cfg(test)] mod tests { use super::*; + use arrow::array::{Int64Array, RecordBatch, StringArray}; use arrow::util::pretty::pretty_format_batches; - use arrow_array::{Int64Array, RecordBatch, StringArray}; use arrow_schema::Field; use arrow_schema::Schema; use arrow_schema::SchemaRef; diff --git a/datafusion/physical-plan/src/aggregates/topk_stream.rs b/datafusion/physical-plan/src/aggregates/topk_stream.rs index 5d18f40d13bc..8a984fc0d27b 100644 --- a/datafusion/physical-plan/src/aggregates/topk_stream.rs +++ b/datafusion/physical-plan/src/aggregates/topk_stream.rs @@ -23,8 +23,8 @@ use crate::aggregates::{ PhysicalGroupBy, }; use crate::{RecordBatchStream, SendableRecordBatchStream}; +use arrow::array::{Array, ArrayRef, RecordBatch}; use arrow::util::pretty::print_batches; -use arrow_array::{Array, ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; use datafusion_common::DataFusionError; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/coalesce/mod.rs b/datafusion/physical-plan/src/coalesce/mod.rs index f38876d93ec1..ed60a350e300 100644 --- a/datafusion/physical-plan/src/coalesce/mod.rs +++ b/datafusion/physical-plan/src/coalesce/mod.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::{ + builder::StringViewBuilder, cast::AsArray, Array, ArrayRef, RecordBatch, + RecordBatchOptions, +}; use arrow::compute::concat_batches; -use arrow_array::builder::StringViewBuilder; -use arrow_array::cast::AsArray; -use arrow_array::{Array, ArrayRef, RecordBatch, RecordBatchOptions}; use arrow_schema::SchemaRef; use std::sync::Arc; @@ -277,9 +278,8 @@ mod tests { use super::*; + use arrow::array::{builder::ArrayBuilder, StringViewArray, UInt32Array}; use arrow::datatypes::{DataType, Field, Schema}; - use arrow_array::builder::ArrayBuilder; - use arrow_array::{StringViewArray, UInt32Array}; #[test] fn test_coalesce() { diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 20a4e89dba94..541f8bcae122 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -26,10 +26,10 @@ use super::SendableRecordBatchStream; use crate::stream::RecordBatchReceiverStream; use crate::{ColumnStatistics, Statistics}; +use arrow::array::Array; use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; -use arrow_array::Array; use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index da3d33cd0e21..0104c91eb5d5 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -42,9 +42,8 @@ use crate::repartition::RepartitionExec; use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::stream::RecordBatchStreamAdapter; +use arrow::array::{Array, RecordBatch}; use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use arrow_array::Array; use datafusion_common::config::ConfigOptions; use datafusion_common::{exec_err, Constraints, Result}; use datafusion_execution::TaskContext; @@ -283,7 +282,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// /// ``` /// # use std::sync::Arc; - /// # use arrow_array::RecordBatch; + /// # use arrow::array::RecordBatch; /// # use arrow_schema::SchemaRef; /// # use datafusion_common::Result; /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -313,7 +312,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// /// ``` /// # use std::sync::Arc; - /// # use arrow_array::RecordBatch; + /// # use arrow::array::RecordBatch; /// # use arrow_schema::SchemaRef; /// # use datafusion_common::Result; /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -348,7 +347,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// /// ``` /// # use std::sync::Arc; - /// # use arrow_array::RecordBatch; + /// # use arrow::array::RecordBatch; /// # use arrow_schema::SchemaRef; /// # use futures::TryStreamExt; /// # use datafusion_common::Result; @@ -1055,7 +1054,7 @@ pub enum CardinalityEffect { #[cfg(test)] mod tests { use super::*; - use arrow_array::{DictionaryArray, Int32Array, NullArray, RunArray}; + use arrow::array::{DictionaryArray, Int32Array, NullArray, RunArray}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use std::any::Any; use std::sync::Arc; diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index bfb1e9d53df5..0b8c1eede36c 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -30,9 +30,8 @@ use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; use crate::ExecutionPlanProperties; +use arrow::array::{ArrayRef, RecordBatch, UInt64Array}; use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use arrow_array::{ArrayRef, UInt64Array}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index cec717a25cf0..ca4c26251de0 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -38,10 +38,9 @@ use crate::{ SendableRecordBatchStream, Statistics, }; +use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::compute::concat_batches; use arrow::datatypes::{Fields, Schema, SchemaRef}; -use arrow::record_batch::RecordBatch; -use arrow_array::RecordBatchOptions; use datafusion_common::stats::Precision; use datafusion_common::{internal_err, JoinType, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index b4e03b57e87f..6cdd91bb1721 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -58,14 +58,14 @@ use crate::{ }; use arrow::array::{ - Array, ArrayRef, BooleanArray, BooleanBufferBuilder, UInt32Array, UInt64Array, + cast::downcast_array, Array, ArrayRef, BooleanArray, BooleanBufferBuilder, + UInt32Array, UInt64Array, }; use arrow::compute::kernels::cmp::{eq, not_distinct}; use arrow::compute::{and, concat_batches, take, FilterBuilder}; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; -use arrow_array::cast::downcast_array; use arrow_schema::ArrowError; use datafusion_common::utils::memory::estimate_memory_size; use datafusion_common::{ @@ -1644,10 +1644,9 @@ mod tests { test::exec::MockExec, }; - use arrow::array::{Date32Array, Int32Array}; + use arrow::array::{Date32Array, Int32Array, StructArray}; use arrow::buffer::NullBuffer; use arrow::datatypes::{DataType, Field}; - use arrow_array::StructArray; use datafusion_common::{ assert_batches_eq, assert_batches_sorted_eq, assert_contains, exec_err, ScalarValue, diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 5bd2658dc719..07289d861bcf 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1036,8 +1036,8 @@ pub(crate) mod tests { common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, }; + use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field}; - use arrow_array::Int32Array; use arrow_schema::SortOptions; use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_execution::runtime_env::RuntimeEnvBuilder; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 5570ceb12ae0..a3e835c64131 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -53,14 +53,13 @@ use crate::{ SendableRecordBatchStream, Statistics, }; -use arrow::array::*; +use arrow::array::{types::UInt64Type, *}; use arrow::compute::{ self, concat_batches, filter_record_batch, is_not_null, take, SortOptions, }; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; use arrow::error::ArrowError; use arrow::ipc::reader::FileReader; -use arrow_array::types::UInt64Type; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DataFusionError, HashSet, JoinSide, JoinType, Result, @@ -2525,12 +2524,12 @@ fn is_join_arrays_equal( mod tests { use std::sync::Arc; - use arrow::array::{Date32Array, Date64Array, Int32Array}; + use arrow::array::{ + builder::{BooleanBuilder, UInt64Builder}, + BooleanArray, Date32Array, Date64Array, Int32Array, RecordBatch, UInt64Array, + }; use arrow::compute::{concat_batches, filter_record_batch, SortOptions}; use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use arrow_array::builder::{BooleanBuilder, UInt64Builder}; - use arrow_array::{BooleanArray, UInt64Array}; use datafusion_common::JoinSide; use datafusion_common::JoinType::*; diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index a866f7291bbd..9e34c27ee7f4 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -28,12 +28,11 @@ use crate::repartition::RepartitionExec; use crate::source::DataSourceExec; use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; -use arrow::array::types::IntervalDayTime; -use arrow::util::pretty::pretty_format_batches; -use arrow_array::{ - ArrayRef, Float64Array, Int32Array, IntervalDayTimeArray, RecordBatch, - TimestampMillisecondArray, +use arrow::array::{ + types::IntervalDayTime, ArrayRef, Float64Array, Int32Array, IntervalDayTimeArray, + RecordBatch, TimestampMillisecondArray, }; +use arrow::util::pretty::pretty_format_batches; use arrow_schema::{DataType, Schema}; use datafusion_common::{Result, ScalarValue}; use datafusion_execution::TaskContext; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 61b7d2a06c5f..bccfd2a69383 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -33,16 +33,14 @@ use crate::{ pub use super::join_filter::JoinFilter; use arrow::array::{ - downcast_array, new_null_array, Array, BooleanBufferBuilder, UInt32Array, - UInt32Builder, UInt64Array, + builder::UInt64Builder, downcast_array, new_null_array, Array, ArrowPrimitiveType, + BooleanBufferBuilder, NativeAdapter, PrimitiveArray, RecordBatch, RecordBatchOptions, + UInt32Array, UInt32Builder, UInt64Array, }; use arrow::compute; use arrow::datatypes::{ ArrowNativeType, Field, Schema, SchemaBuilder, UInt32Type, UInt64Type, }; -use arrow::record_batch::{RecordBatch, RecordBatchOptions}; -use arrow_array::builder::UInt64Builder; -use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray}; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; @@ -1820,13 +1818,12 @@ pub(super) fn swap_join_projection( #[cfg(test)] mod tests { - use std::pin::Pin; - use super::*; + use std::pin::Pin; + use arrow::array::Int32Array; use arrow::datatypes::{DataType, Fields}; use arrow::error::{ArrowError, Result as ArrowResult}; - use arrow_array::Int32Array; use arrow_schema::SortOptions; use datafusion_common::stats::Precision::{Absent, Exact, Inexact}; use datafusion_common::{arrow_datafusion_err, arrow_err, ScalarValue}; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 9665a09e42c9..15f19f6456a5 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -480,7 +480,7 @@ mod tests { use crate::test; use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; - use arrow_array::RecordBatchOptions; + use arrow::array::RecordBatchOptions; use arrow_schema::Schema; use datafusion_common::stats::Precision; use datafusion_physical_expr::expressions::col; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 9d5d65d4e18a..3d161a047853 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -33,9 +33,8 @@ use crate::projection::{ }; use crate::source::{DataSource, DataSourceExec}; +use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use arrow_array::RecordBatchOptions; use arrow_schema::Schema; use datafusion_common::{ internal_err, plan_err, project_schema, Constraints, Result, ScalarValue, diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 355e51070f1f..6a8f247ec0e6 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -25,9 +25,8 @@ use crate::execution_plan::{Boundedness, EmissionType}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::array::{ArrayRef, NullArray}; +use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; -use arrow::record_batch::RecordBatch; -use arrow_array::RecordBatchOptions; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c01c0a9564e0..ffd1a5b520fa 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -40,10 +40,9 @@ use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; +use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; -use arrow::record_batch::RecordBatch; -use arrow_array::{PrimitiveArray, RecordBatchOptions}; use datafusion_common::utils::transpose; use datafusion_common::HashMap; use datafusion_common::{not_impl_err, DataFusionError, Result}; diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index 2c298c9327f4..e6986b86046c 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -17,15 +17,14 @@ use std::cmp::Ordering; -use arrow::buffer::ScalarBuffer; -use arrow::buffer::{Buffer, OffsetBuffer}; +use arrow::array::{ + types::ByteArrayType, Array, ArrowPrimitiveType, GenericByteArray, OffsetSizeTrait, + PrimitiveArray, +}; +use arrow::buffer::{Buffer, OffsetBuffer, ScalarBuffer}; use arrow::compute::SortOptions; use arrow::datatypes::ArrowNativeTypeOp; use arrow::row::Rows; -use arrow_array::types::ByteArrayType; -use arrow_array::{ - Array, ArrowPrimitiveType, GenericByteArray, OffsetSizeTrait, PrimitiveArray, -}; use datafusion_execution::memory_pool::MemoryReservation; /// A comparable collection of values for use with [`Cursor`] diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 2db6e90a1e03..6c538801d71a 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -44,11 +44,10 @@ use crate::{ Statistics, }; +use arrow::array::{Array, RecordBatch, RecordBatchOptions, UInt32Array}; use arrow::compute::{concat_batches, lexsort_to_indices, take_arrays, SortColumn}; use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; -use arrow_array::{Array, RecordBatchOptions, UInt32Array}; use arrow_schema::DataType; use datafusion_common::{internal_err, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 25572fa60a3d..c2cd9d7db3c6 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -393,11 +393,12 @@ mod tests { use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; - use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampNanosecondArray}; + use arrow::array::{ + ArrayRef, Int32Array, Int64Array, RecordBatch, StringArray, + TimestampNanosecondArray, + }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use arrow_array::Int64Array; use arrow_schema::SchemaRef; use datafusion_common::{assert_batches_eq, assert_contains, DataFusionError}; use datafusion_common_runtime::SpawnedTask; diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 448d70760de1..909b5875c8c5 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -24,8 +24,8 @@ use crate::sorts::{ stream::{FieldCursorStream, RowCursorStream}, }; use crate::SendableRecordBatchStream; +use arrow::array::*; use arrow::datatypes::{DataType, SchemaRef}; -use arrow_array::*; use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_physical_expr_common::sort_expr::LexOrdering; diff --git a/datafusion/physical-plan/src/spill.rs b/datafusion/physical-plan/src/spill.rs index f20adb0d2fab..dbcc46baf8ca 100644 --- a/datafusion/physical-plan/src/spill.rs +++ b/datafusion/physical-plan/src/spill.rs @@ -184,10 +184,9 @@ mod tests { use super::*; use crate::spill::{spill_record_batch_by_size, spill_record_batches}; use crate::test::build_table_i32; - use arrow::array::{Float64Array, Int32Array}; + use arrow::array::{Float64Array, Int32Array, ListArray}; use arrow::datatypes::{DataType, Field, Int32Type, Schema}; use arrow::record_batch::RecordBatch; - use arrow_array::ListArray; use datafusion_common::Result; use datafusion_execution::disk_manager::DiskManagerConfig; use datafusion_execution::DiskManager; diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 2d493e27c06a..757e2df65831 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -21,7 +21,7 @@ use std::collections::HashMap; use std::pin::Pin; use std::sync::Arc; -use arrow_array::{ArrayRef, Int32Array, RecordBatch}; +use arrow::array::{ArrayRef, Int32Array, RecordBatch}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use futures::{Future, FutureExt}; diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index ed1df6b1b8ff..4cc8fc8711de 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -27,7 +27,7 @@ use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; use super::metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder}; use crate::spill::get_record_batch_memory_size; use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; -use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow::array::{Array, ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; use datafusion_common::HashMap; use datafusion_common::Result; @@ -647,10 +647,8 @@ impl RecordBatchStore { #[cfg(test)] mod tests { use super::*; - use arrow::array::Int32Array; + use arrow::array::{Float64Array, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use arrow_array::Float64Array; /// This test ensures the size calculation is correct for RecordBatches with multiple columns. #[test] diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 942dd7881052..430391de5922 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -29,15 +29,14 @@ use crate::{ }; use arrow::array::{ - Array, ArrayRef, AsArray, FixedSizeListArray, LargeListArray, ListArray, - PrimitiveArray, + new_null_array, Array, ArrayRef, AsArray, FixedSizeListArray, Int64Array, + LargeListArray, ListArray, PrimitiveArray, Scalar, StructArray, }; use arrow::compute::kernels::length::length; use arrow::compute::kernels::zip::zip; use arrow::compute::{cast, is_not_null, kernels, sum}; use arrow::datatypes::{DataType, Int64Type, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use arrow_array::{new_null_array, Int64Array, Scalar, StructArray}; use arrow_ord::cmp::lt; use datafusion_common::{ exec_datafusion_err, exec_err, internal_err, HashMap, HashSet, Result, UnnestOptions, diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 41de16289cf5..79eaf4447434 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1181,8 +1181,10 @@ mod tests { }; use crate::{execute_stream, get_plan_string, ExecutionPlan}; - use arrow_array::builder::{Int64Builder, UInt64Builder}; - use arrow_array::RecordBatch; + use arrow::array::{ + builder::{Int64Builder, UInt64Builder}, + RecordBatch, + }; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion_common::{ assert_batches_eq, exec_datafusion_err, Result, ScalarValue, @@ -1528,7 +1530,7 @@ mod tests { // Create a new batch of data to insert into the table let batch = RecordBatch::try_new( Arc::clone(&schema), - vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], + vec![Arc::new(arrow::array::Int32Array::from(vec![1, 2, 3]))], )?; let memory_exec = MemorySourceConfig::try_new_exec( diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index b1dd4d9308f4..a6ced527cbb2 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -229,7 +229,7 @@ impl ExecutionPlan for WorkTableExec { #[cfg(test)] mod tests { use super::*; - use arrow_array::{ArrayRef, Int32Array}; + use arrow::array::{ArrayRef, Int32Array}; use datafusion_execution::memory_pool::{MemoryConsumer, UnboundedMemoryPool}; #[test] diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index 224c7cb191a3..b53f3674d13a 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -43,7 +43,6 @@ recursive_protection = ["dep:recursive"] [dependencies] arrow = { workspace = true } -arrow-array = { workspace = true } arrow-schema = { workspace = true } bigdecimal = { workspace = true } datafusion-common = { workspace = true, default-features = true } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 51ae73e03e29..909533773435 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -26,14 +26,16 @@ use std::vec; use super::dialect::IntervalStyle; use super::Unparser; +use arrow::array::{ + types::{ + ArrowTemporalType, Time32MillisecondType, Time32SecondType, + Time64MicrosecondType, Time64NanosecondType, TimestampMicrosecondType, + TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, + }, + ArrayRef, Date32Array, Date64Array, PrimitiveArray, +}; use arrow::datatypes::{Decimal128Type, Decimal256Type, DecimalType}; use arrow::util::display::array_value_to_string; -use arrow_array::types::{ - ArrowTemporalType, Time32MillisecondType, Time32SecondType, Time64MicrosecondType, - Time64NanosecondType, TimestampMicrosecondType, TimestampMillisecondType, - TimestampNanosecondType, TimestampSecondType, -}; -use arrow_array::{ArrayRef, Date32Array, Date64Array, PrimitiveArray}; use arrow_schema::DataType; use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, Column, Result, @@ -1646,9 +1648,8 @@ mod tests { use std::ops::{Add, Sub}; use std::{any::Any, sync::Arc, vec}; - use arrow::datatypes::{Field, Schema}; - use arrow::datatypes::{Int32Type, TimeUnit}; - use arrow_array::{LargeListArray, ListArray}; + use arrow::array::{LargeListArray, ListArray}; + use arrow::datatypes::{Field, Int32Type, Schema, TimeUnit}; use arrow_schema::DataType::Int8; use ast::ObjectName; use datafusion_common::{Spans, TableReference}; From bd7daffe0c8643f008689f097eb887f09d063e7e Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 8 Feb 2025 08:28:17 -0500 Subject: [PATCH 09/12] Improve documentation about extended tests (#14320) * Improve documentation about extended tests * Improvements * clarify extent of overage * Update docs/source/contributor-guide/testing.md --------- Co-authored-by: Oleks V --- .github/workflows/extended.yml | 8 +++++--- docs/source/contributor-guide/testing.md | 12 ++++++++++++ 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/.github/workflows/extended.yml b/.github/workflows/extended.yml index 839c0d135c4e..7fa89ea773f8 100644 --- a/.github/workflows/extended.yml +++ b/.github/workflows/extended.yml @@ -23,9 +23,11 @@ concurrency: # https://docs.github.com/en/actions/writing-workflows/choosing-when-your-workflow-runs/events-that-trigger-workflows#running-your-pull_request-workflow-when-a-pull-request-merges # -# this job is intended to only run only on the main branch as it is time consuming -# and should not fail often. However, it is important coverage to ensure correctness -# in the (very rare) event of a hash failure or sqlite query failure. +# These jobs only run on the `main` branch as they are time consuming +# and should not fail often. +# +# We still run them as they provide important coverage to ensure correctness +# in the (very rare) event of a hash failure or sqlite library query failure. on: # Run on all commits to main push: diff --git a/docs/source/contributor-guide/testing.md b/docs/source/contributor-guide/testing.md index 3da0e0fe594d..2a9f22d22d66 100644 --- a/docs/source/contributor-guide/testing.md +++ b/docs/source/contributor-guide/testing.md @@ -58,6 +58,18 @@ Like similar systems such as [DuckDB](https://duckdb.org/dev/testing), DataFusio DataFusion has integrated [sqlite's test suite](https://sqlite.org/sqllogictest/doc/trunk/about.wiki) as a supplemental test suite that is run whenever a PR is merged into DataFusion. To run it manually please refer to the [README](https://github.com/apache/datafusion/blob/main/datafusion/sqllogictest/README.md#running-tests-sqlite) file for instructions. +## Extended Tests + +In addition to the standard CI test suite that is run on all PRs prior to merge, +DataFusion has "extended" tests (defined in [extended.yml]) that are run on each +commit to `main`. These tests rarely fail but take significantly longer to run +than the standard test suite and add important test coverage such as that the +code works when there are hash collisions as well as running the relevant +portions of the entire [sqlite test suite]. + +[extended.yml]: https://github.com/apache/datafusion/blob/main/.github/workflows/extended.yml +[sqlite test suite]: https://www.sqlite.org/sqllogictest/dir?ci=tip + ## Rust Integration Tests There are several tests of the public interface of the DataFusion library in the [tests](https://github.com/apache/datafusion/tree/main/datafusion/core/tests) directory. From 56a30acbde6bf5ad1357c98ff9e6ee9fcc9d7f60 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 8 Feb 2025 11:05:02 -0500 Subject: [PATCH 10/12] chore: update datafusion-testing pin to fix extended tests (#14556) --- datafusion-testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-testing b/datafusion-testing index 5b424aefd7f6..a169d8ded675 160000 --- a/datafusion-testing +++ b/datafusion-testing @@ -1 +1 @@ -Subproject commit 5b424aefd7f6bf198220c37f59d39dbb25b47695 +Subproject commit a169d8ded67529174f08e22fe88c014397a4d967 From 94d2baf318328a3fd692bbe0b2482a90793a93b9 Mon Sep 17 00:00:00 2001 From: Matthijs Brobbel Date: Sat, 8 Feb 2025 17:05:25 +0100 Subject: [PATCH 11/12] Add `Cargo.lock` (#14483) * Add `Cargo.lock` * Style fix * Update `Cargo.lock` * Remove unused path * Remove cli-specific ci job and dependabot config * Remove home pin * Make cli test work with backtrace feature * More changes resulting from moving the cli crate in the workspace * Exclude `depcheck` `Cargo.lock` * Remove `--locked` from `depcheck` run * Refer to guidance instead of updated guidance * Remove `--locked` from benchmark script * Only run with `--locked` in `linux-build-lib` job of `Rust` workflow * Remove unrelated formatting changes * Add a note about using `--locked` * Update cargo.lock * Update .github/workflows/rust.yml * Update Cargo.lock * fix yaml --------- Co-authored-by: Andrew Lamb --- .github/dependabot.yml | 15 - .github/workflows/dependencies.yml | 4 +- .github/workflows/rust.yml | 130 +- .gitignore | 2 - datafusion-cli/Cargo.lock => Cargo.lock | 2769 +++++++++++++++++++++-- Cargo.toml | 4 +- README.md | 27 +- ci/scripts/rust_clippy.sh | 2 - ci/scripts/rust_docs.sh | 2 - ci/scripts/rust_fmt.sh | 2 - datafusion-cli/Cargo.toml | 53 +- datafusion-cli/Dockerfile | 10 +- datafusion-cli/README.md | 16 - datafusion-cli/src/object_storage.rs | 2 +- dev/depcheck/.gitignore | 1 + dev/release/README.md | 9 - 16 files changed, 2699 insertions(+), 349 deletions(-) rename datafusion-cli/Cargo.lock => Cargo.lock (65%) create mode 100644 dev/depcheck/.gitignore diff --git a/.github/dependabot.yml b/.github/dependabot.yml index afdbaaf72b6c..42134c3edfd2 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -28,24 +28,9 @@ updates: # arrow is bumped manually - dependency-name: "arrow*" update-types: ["version-update:semver-major"] - - package-ecosystem: cargo - directory: "datafusion-cli/" - schedule: - interval: daily - open-pull-requests-limit: 10 - target-branch: main - labels: [auto-dependencies] - ignore: - # arrow is bumped manually - - dependency-name: "arrow*" - update-types: ["version-update:semver-major"] - # datafusion is bumped manually - - dependency-name: "datafusion*" - update-types: ["version-update:semver-major"] - package-ecosystem: "github-actions" directory: "/" schedule: interval: "daily" open-pull-requests-limit: 10 labels: [auto-dependencies] - diff --git a/.github/workflows/dependencies.yml b/.github/workflows/dependencies.yml index f87215565bb5..a577725fed4b 100644 --- a/.github/workflows/dependencies.yml +++ b/.github/workflows/dependencies.yml @@ -25,9 +25,11 @@ on: push: paths: - "**/Cargo.toml" + - "**/Cargo.lock" pull_request: paths: - "**/Cargo.toml" + - "**/Cargo.lock" # manual trigger # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow workflow_dispatch: @@ -50,4 +52,4 @@ jobs: - name: Check dependencies run: | cd dev/depcheck - cargo run \ No newline at end of file + cargo run diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index 0be709a4a785..c023faa9b168 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -60,7 +60,11 @@ jobs: with: rust-version: stable - name: Prepare cargo build - run: cargo check --profile ci --all-targets --features integration-tests + run: | + # Adding `--locked` here to assert that the `Cargo.lock` file is up to + # date with the manifest. When this fails, please make sure to commit + # the changes to `Cargo.lock` after building with the updated manifest. + cargo check --profile ci --workspace --all-targets --features integration-tests --locked # cargo check common, functions and substrait with no default features linux-cargo-check-no-default-features: @@ -95,12 +99,6 @@ jobs: - name: Check workspace with additional features run: cargo check --profile ci --workspace --benches --features avro,json,integration-tests - - name: Check Cargo.lock for datafusion-cli - run: | - # If this test fails, try running `cargo update` in the `datafusion-cli` directory - # and check in the updated Cargo.lock file. - cargo check --profile ci --manifest-path datafusion-cli/Cargo.toml --locked - # cargo check datafusion to ensure that the datafusion crate can be built with only a # subset of the function packages enabled. linux-cargo-check-datafusion: @@ -189,28 +187,6 @@ jobs: - name: Verify Working Directory Clean run: git diff --exit-code - linux-test-datafusion-cli: - name: cargo test datafusion-cli (amd64) - needs: linux-build-lib - runs-on: ubuntu-latest - container: - image: amd64/rust - steps: - - uses: actions/checkout@v4 - with: - submodules: true - fetch-depth: 1 - - name: Setup Rust toolchain - uses: ./.github/actions/setup-builder - with: - rust-version: stable - - name: Run tests (excluding doctests) - run: | - cd datafusion-cli - cargo test --profile ci --lib --tests --bins --all-features - - name: Verify Working Directory Clean - run: git diff --exit-code - linux-test-example: name: cargo examples (amd64) needs: linux-build-lib @@ -252,10 +228,7 @@ jobs: with: rust-version: stable - name: Run doctests - run: | - cargo test --profile ci --doc --features avro,json - cd datafusion-cli - cargo test --profile ci --doc --all-features + run: cargo test --profile ci --doc --features avro,json - name: Verify Working Directory Clean run: git diff --exit-code @@ -364,45 +337,40 @@ jobs: POSTGRES_HOST: postgres POSTGRES_PORT: ${{ job.services.postgres.ports[5432] }} -# Temporarily commenting out the Windows flow, the reason is enormously slow running build -# Waiting for new Windows 2025 github runner -# Details: https://github.com/apache/datafusion/issues/13726 -# -# windows: -# name: cargo test (win64) -# runs-on: windows-latest -# steps: -# - uses: actions/checkout@v4 -# with: -# submodules: true -# - name: Setup Rust toolchain -# uses: ./.github/actions/setup-windows-builder -# - name: Run tests (excluding doctests) -# shell: bash -# run: | -# export PATH=$PATH:$HOME/d/protoc/bin -# cargo test --lib --tests --bins --features avro,json,backtrace -# cd datafusion-cli -# cargo test --lib --tests --bins --all-features - -# Commenting out intel mac build as so few users would ever use it -# Details: https://github.com/apache/datafusion/issues/13846 -# macos: -# name: cargo test (macos) -# runs-on: macos-latest -# steps: -# - uses: actions/checkout@v4 -# with: -# submodules: true -# fetch-depth: 1 -# - name: Setup Rust toolchain -# uses: ./.github/actions/setup-macos-builder -# - name: Run tests (excluding doctests) -# shell: bash -# run: | -# cargo test run --profile ci --exclude datafusion-examples --exclude datafusion-benchmarks --workspace --lib --tests --bins --features avro,json,backtrace -# cd datafusion-cli -# cargo test run --profile ci --lib --tests --bins --all-features + # Temporarily commenting out the Windows flow, the reason is enormously slow running build + # Waiting for new Windows 2025 github runner + # Details: https://github.com/apache/datafusion/issues/13726 + # + # windows: + # name: cargo test (win64) + # runs-on: windows-latest + # steps: + # - uses: actions/checkout@v4 + # with: + # submodules: true + # - name: Setup Rust toolchain + # uses: ./.github/actions/setup-windows-builder + # - name: Run tests (excluding doctests) + # shell: bash + # run: | + # export PATH=$PATH:$HOME/d/protoc/bin + # cargo test --lib --tests --bins --features avro,json,backtrace + + # Commenting out intel mac build as so few users would ever use it + # Details: https://github.com/apache/datafusion/issues/13846 + # macos: + # name: cargo test (macos) + # runs-on: macos-latest + # steps: + # - uses: actions/checkout@v4 + # with: + # submodules: true + # fetch-depth: 1 + # - name: Setup Rust toolchain + # uses: ./.github/actions/setup-macos-builder + # - name: Run tests (excluding doctests) + # shell: bash + # run: cargo test run --profile ci --exclude datafusion-examples --exclude datafusion-benchmarks --workspace --lib --tests --bins --features avro,json,backtrace macos-aarch64: name: cargo test (macos-aarch64) @@ -416,10 +384,7 @@ jobs: uses: ./.github/actions/setup-macos-aarch64-builder - name: Run tests (excluding doctests) shell: bash - run: | - cargo test --profile ci --lib --tests --bins --features avro,json,backtrace,integration-tests - cd datafusion-cli - cargo test --profile ci --lib --tests --bins --all-features + run: cargo test --profile ci --lib --tests --bins --features avro,json,backtrace,integration-tests test-datafusion-pyarrow: name: cargo test pyarrow (amd64) @@ -615,19 +580,19 @@ jobs: # (Min Supported Rust Version) than the one specified in the # `rust-version` key of `Cargo.toml`. # - # To reproduce: - # 1. Install the version of Rust that is failing. Example: + # To reproduce: + # 1. Install the version of Rust that is failing. Example: # rustup install 1.80.1 # 2. Run the command that failed with that version. Example: # cargo +1.80.1 check -p datafusion - # + # # To resolve, either: - # 1. Change your code to use older Rust features, + # 1. Change your code to use older Rust features, # 2. Revert dependency update # 3. Update the MSRV version in `Cargo.toml` # # Please see the DataFusion Rust Version Compatibility Policy before - # updating Cargo.toml. You may have to update the code instead. + # updating Cargo.toml. You may have to update the code instead. # https://github.com/apache/datafusion/blob/main/README.md#rust-version-compatibility-policy cargo msrv --output-format json --log-target stdout verify - name: Check datafusion-substrait @@ -636,6 +601,3 @@ jobs: - name: Check datafusion-proto working-directory: datafusion/proto run: cargo msrv --output-format json --log-target stdout verify - - name: Check datafusion-cli - working-directory: datafusion-cli - run: cargo msrv --output-format json --log-target stdout verify diff --git a/.gitignore b/.gitignore index 1fa79249ff8e..6196385144ab 100644 --- a/.gitignore +++ b/.gitignore @@ -42,8 +42,6 @@ venv/* # Rust target -Cargo.lock -!datafusion-cli/Cargo.lock rusty-tags.vi .history diff --git a/datafusion-cli/Cargo.lock b/Cargo.lock similarity index 65% rename from datafusion-cli/Cargo.lock rename to Cargo.lock index 00f59f9d78ad..d22e95b1a736 100644 --- a/datafusion-cli/Cargo.lock +++ b/Cargo.lock @@ -2,6 +2,54 @@ # It is not intended for manual editing. version = 3 +[[package]] +name = "abi_stable" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69d6512d3eb05ffe5004c59c206de7f99c34951504056ce23fc953842f12c445" +dependencies = [ + "abi_stable_derive", + "abi_stable_shared", + "const_panic", + "core_extensions", + "crossbeam-channel", + "generational-arena", + "libloading", + "lock_api", + "parking_lot", + "paste", + "repr_offset", + "rustc_version", + "serde", + "serde_derive", + "serde_json", +] + +[[package]] +name = "abi_stable_derive" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7178468b407a4ee10e881bc7a328a65e739f0863615cca4429d43916b05e898" +dependencies = [ + "abi_stable_shared", + "as_derive_utils", + "core_extensions", + "proc-macro2", + "quote", + "rustc_version", + "syn 1.0.109", + "typed-arena", +] + +[[package]] +name = "abi_stable_shared" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b2b5df7688c123e63f4d4d649cba63f2967ba7f7861b1664fca3f77d3dad2b63" +dependencies = [ + "core_extensions", +] + [[package]] name = "addr2line" version = "0.24.2" @@ -23,6 +71,17 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" +[[package]] +name = "ahash" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "891477e0c6a8957309ee5c45a6368af3ae14bb510732d2684ffa19af310920f9" +dependencies = [ + "getrandom 0.2.15", + "once_cell", + "version_check", +] + [[package]] name = "ahash" version = "0.8.11" @@ -34,7 +93,7 @@ dependencies = [ "getrandom 0.2.15", "once_cell", "version_check", - "zerocopy", + "zerocopy 0.7.35", ] [[package]] @@ -82,6 +141,12 @@ dependencies = [ "libc", ] +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + [[package]] name = "anstream" version = "0.6.18" @@ -132,6 +197,12 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "anyhow" +version = "1.0.95" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" + [[package]] name = "apache-avro" version = "0.17.0" @@ -146,7 +217,7 @@ dependencies = [ "log", "num-bigint", "quad-rand", - "rand", + "rand 0.8.5", "regex-lite", "serde", "serde_bytes", @@ -192,6 +263,9 @@ dependencies = [ "arrow-schema", "arrow-select", "arrow-string", + "half", + "pyo3", + "rand 0.8.5", ] [[package]] @@ -214,7 +288,7 @@ version = "54.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fb4a06d507f54b70a277be22a127c8ffe0cec6cd98c0ad8a48e77779bbda8223" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow-buffer", "arrow-data", "arrow-schema", @@ -285,6 +359,33 @@ dependencies = [ "num", ] +[[package]] +name = "arrow-flight" +version = "54.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a9b3aaba47ed4b6146563c8b79ad0f7aa283f794cde0c057c656291b81196746" +dependencies = [ + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", + "base64 0.22.1", + "bytes", + "futures", + "once_cell", + "paste", + "prost", + "prost-types", + "tonic", +] + [[package]] name = "arrow-ipc" version = "54.1.0" @@ -312,7 +413,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap", + "indexmap 2.7.1", "lexical-core", "num", "serde", @@ -350,6 +451,10 @@ name = "arrow-schema" version = "54.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "105f01ec0090259e9a33a9263ec18ff223ab91a0ea9fbc18042f7e38005142f6" +dependencies = [ + "bitflags 2.8.0", + "serde", +] [[package]] name = "arrow-select" @@ -357,7 +462,7 @@ version = "54.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f690752fdbd2dee278b5f1636fefad8f2f7134c85e20fd59c4199e15a39a6807" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow-array", "arrow-buffer", "arrow-data", @@ -382,6 +487,18 @@ dependencies = [ "regex-syntax", ] +[[package]] +name = "as_derive_utils" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ff3c96645900a44cf11941c111bd08a6573b0e2f9f69bc9264b179d8fae753c4" +dependencies = [ + "core_extensions", + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "assert_cmd" version = "2.0.16" @@ -415,6 +532,48 @@ dependencies = [ "zstd-safe", ] +[[package]] +name = "async-ffi" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f4de21c0feef7e5a556e51af767c953f0501f7f300ba785cc99c47bdc8081a50" +dependencies = [ + "abi_stable", +] + +[[package]] +name = "async-recursion" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "async-stream" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b5a71a6f37880a80d1d7f19efd781e4b5de42c88f0722cc13bcb6cc2cfe8476" +dependencies = [ + "async-stream-impl", + "futures-core", + "pin-project-lite", +] + +[[package]] +name = "async-stream-impl" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + [[package]] name = "async-trait" version = "0.1.86" @@ -423,7 +582,7 @@ checksum = "644dd749086bf3771a2fbc5f256fdb982d53f011c7d5d560304eafeecebce79d" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -744,6 +903,53 @@ dependencies = [ "tracing", ] +[[package]] +name = "axum" +version = "0.7.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "edca88bc138befd0323b20752846e6587272d3b03b0343c8ea28a6f819e6e71f" +dependencies = [ + "async-trait", + "axum-core", + "bytes", + "futures-util", + "http 1.2.0", + "http-body 1.0.1", + "http-body-util", + "itoa", + "matchit", + "memchr", + "mime", + "percent-encoding", + "pin-project-lite", + "rustversion", + "serde", + "sync_wrapper", + "tower 0.5.2", + "tower-layer", + "tower-service", +] + +[[package]] +name = "axum-core" +version = "0.4.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09f2bd6146b97ae3359fa0cc6d6b376d9539582c7b4220f041a33ec24c226199" +dependencies = [ + "async-trait", + "bytes", + "futures-util", + "http 1.2.0", + "http-body 1.0.1", + "http-body-util", + "mime", + "pin-project-lite", + "rustversion", + "sync_wrapper", + "tower-layer", + "tower-service", +] + [[package]] name = "backtrace" version = "0.3.74" @@ -756,7 +962,7 @@ dependencies = [ "miniz_oxide", "object", "rustc-demangle", - "windows-targets", + "windows-targets 0.52.6", ] [[package]] @@ -807,6 +1013,18 @@ version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f68f53c83ab957f72c32642f3868eec03eb974d1fb82e453128456482613d36" +[[package]] +name = "bitvec" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc2832c24239b0141d5674bb9174f9d68a8b5b3f2753311927c172ca46f7e9c" +dependencies = [ + "funty", + "radium", + "tap", + "wyz", +] + [[package]] name = "blake2" version = "0.10.6" @@ -838,6 +1056,79 @@ dependencies = [ "generic-array", ] +[[package]] +name = "bollard" +version = "0.18.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97ccca1260af6a459d75994ad5acc1651bcabcbdbc41467cc9786519ab854c30" +dependencies = [ + "base64 0.22.1", + "bollard-stubs", + "bytes", + "futures-core", + "futures-util", + "hex", + "home", + "http 1.2.0", + "http-body-util", + "hyper 1.6.0", + "hyper-named-pipe", + "hyper-rustls 0.27.5", + "hyper-util", + "hyperlocal", + "log", + "pin-project-lite", + "rustls 0.23.22", + "rustls-native-certs 0.8.1", + "rustls-pemfile 2.2.0", + "rustls-pki-types", + "serde", + "serde_derive", + "serde_json", + "serde_repr", + "serde_urlencoded", + "thiserror 2.0.11", + "tokio", + "tokio-util", + "tower-service", + "url", + "winapi", +] + +[[package]] +name = "bollard-stubs" +version = "1.47.1-rc.27.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f179cfbddb6e77a5472703d4b30436bff32929c0aa8a9008ecf23d1d3cdd0da" +dependencies = [ + "serde", + "serde_repr", + "serde_with", +] + +[[package]] +name = "borsh" +version = "1.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5430e3be710b68d984d1391c854eb431a9d548640711faa54eecb1df93db91cc" +dependencies = [ + "borsh-derive", + "cfg_aliases 0.2.1", +] + +[[package]] +name = "borsh-derive" +version = "1.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8b668d39970baad5356d7c83a86fee3a539e6f93bf6764c97368243e17a0487" +dependencies = [ + "once_cell", + "proc-macro-crate", + "proc-macro2", + "quote", + "syn 2.0.98", +] + [[package]] name = "brotli" version = "7.0.0" @@ -876,6 +1167,28 @@ version = "3.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1628fb46dfa0b37568d12e5edd512553eccf6a22a78e8bde00bb4aed84d5bdbf" +[[package]] +name = "bytecheck" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23cdc57ce23ac53c931e88a43d06d070a6fd142f2617be5855eb75efc9beb1c2" +dependencies = [ + "bytecheck_derive", + "ptr_meta", + "simdutf8", +] + +[[package]] +name = "bytecheck_derive" +version = "0.6.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3db406d29fbcd95542e92559bed4d8ad92636d1ca8b3b72ede10b4bcc010e659" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "byteorder" version = "1.5.0" @@ -929,11 +1242,17 @@ dependencies = [ "pkg-config", ] +[[package]] +name = "cast" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" + [[package]] name = "cc" -version = "1.2.12" +version = "1.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "755717a7de9ec452bf7f3f1a3099085deabd7f2962b861dae91ecd7a365903d2" +checksum = "c7777341816418c02e033934a09f20dc0ccaf65a5201ef8a450ae0105a573fda" dependencies = [ "jobserver", "libc", @@ -946,6 +1265,12 @@ 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" @@ -960,9 +1285,11 @@ checksum = "7e36cc9d416881d2e24f9a963be5fb1cd90966419ac844274161d10488b3e825" dependencies = [ "android-tzdata", "iana-time-zone", + "js-sys", "num-traits", "serde", - "windows-targets", + "wasm-bindgen", + "windows-targets 0.52.6", ] [[package]] @@ -986,6 +1313,44 @@ dependencies = [ "phf_codegen", ] +[[package]] +name = "ciborium" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" + +[[package]] +name = "ciborium-ll" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" +dependencies = [ + "ciborium-io", + "half", +] + +[[package]] +name = "clap" +version = "2.34.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0610544180c38b88101fecf2dd634b174a62eef6946f84dfc6a7127512b381c" +dependencies = [ + "bitflags 1.3.2", + "textwrap", + "unicode-width 0.1.14", +] + [[package]] name = "clap" version = "4.5.28" @@ -1014,10 +1379,10 @@ version = "4.5.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bf4ced95c6f4a675af3da73304b9ac4ed991640c36374e4b46795c49e17cf1ed" dependencies = [ - "heck", + "heck 0.5.0", "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -1036,20 +1401,51 @@ dependencies = [ ] [[package]] -name = "colorchoice" -version = "1.0.3" +name = "cmake" +version = "0.1.53" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" - +checksum = "e24a03c8b52922d68a1589ad61032f2c1aa5a8158d2aa0d93c6e9534944bbad6" +dependencies = [ + "cc", +] + +[[package]] +name = "colorchoice" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" + [[package]] name = "comfy-table" -version = "7.1.3" +version = "7.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24f165e7b643266ea80cb858aed492ad9280e3e05ce24d4a99d7d7b889b6a4d9" +checksum = "4a65ebfec4fb190b6f90e944a817d60499ee0744e582530e2c9900a22e591d9a" dependencies = [ - "strum", - "strum_macros", - "unicode-width", + "unicode-segmentation", + "unicode-width 0.2.0", +] + +[[package]] +name = "console" +version = "0.15.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea3c6ecd8059b57859df5c69830340ed3c41d30e3da0c1cbed90a96ac853041b" +dependencies = [ + "encode_unicode", + "libc", + "once_cell", + "unicode-width 0.2.0", + "windows-sys 0.59.0", +] + +[[package]] +name = "console_error_panic_hook" +version = "0.1.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a06aeb73f470f66dcdbf7223caeebb85984942f22f1adb2a088cf9668146bbbc" +dependencies = [ + "cfg-if", + "wasm-bindgen", ] [[package]] @@ -1072,6 +1468,12 @@ dependencies = [ "tiny-keccak", ] +[[package]] +name = "const_panic" +version = "0.2.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2459fc9262a1aa204eb4b5764ad4f189caec88aea9634389c0a25f8be7f6265e" + [[package]] name = "constant_time_eq" version = "0.3.1" @@ -1113,6 +1515,21 @@ dependencies = [ "memchr", ] +[[package]] +name = "core_extensions" +version = "1.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92c71dc07c9721607e7a16108336048ee978c3a8b129294534272e8bac96c0ee" +dependencies = [ + "core_extensions_proc_macros", +] + +[[package]] +name = "core_extensions_proc_macros" +version = "1.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69f3b219d28b6e3b4ac87bc1fc522e0803ab22e055da177bff0068c4150c61a6" + [[package]] name = "cpufeatures" version = "0.2.17" @@ -1131,6 +1548,72 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "criterion" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +dependencies = [ + "anes", + "cast", + "ciborium", + "clap 4.5.28", + "criterion-plot", + "futures", + "is-terminal", + "itertools 0.10.5", + "num-traits", + "once_cell", + "oorandom", + "plotters", + "rayon", + "regex", + "serde", + "serde_derive", + "serde_json", + "tinytemplate", + "tokio", + "walkdir", +] + +[[package]] +name = "criterion-plot" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +dependencies = [ + "cast", + "itertools 0.10.5", +] + +[[package]] +name = "crossbeam-channel" +version = "0.5.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06ba6d68e24814cb8de6bb986db8222d3a027d15872cabc0d18817bc3c0e4471" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-utils" version = "0.8.21" @@ -1181,7 +1664,42 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a2785755761f3ddc1492979ce1e48d2c00d09311c39e4466429188f3dd6501" dependencies = [ "quote", - "syn", + "syn 2.0.98", +] + +[[package]] +name = "darling" +version = "0.20.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f63b86c8a8826a49b8c21f08a2d07338eec8d900540f8630dc76284be802989" +dependencies = [ + "darling_core", + "darling_macro", +] + +[[package]] +name = "darling_core" +version = "0.20.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "95133861a8032aaea082871032f5815eb9e98cef03fa916ab4500513994df9e5" +dependencies = [ + "fnv", + "ident_case", + "proc-macro2", + "quote", + "strsim", + "syn 2.0.98", +] + +[[package]] +name = "darling_macro" +version = "0.20.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" +dependencies = [ + "darling_core", + "quote", + "syn 2.0.98", ] [[package]] @@ -1217,6 +1735,8 @@ dependencies = [ "bytes", "bzip2 0.5.0", "chrono", + "criterion", + "ctor", "datafusion-catalog", "datafusion-catalog-listing", "datafusion-common", @@ -1228,25 +1748,35 @@ dependencies = [ "datafusion-functions-nested", "datafusion-functions-table", "datafusion-functions-window", + "datafusion-functions-window-common", "datafusion-optimizer", "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-sql", + "doc-comment", + "env_logger", "flate2", "futures", "glob", "itertools 0.14.0", "log", + "nix 0.29.0", "num-traits", "object_store", "parking_lot", "parquet", - "rand", + "paste", + "rand 0.8.5", + "rand_distr", "regex", + "rstest", + "serde_json", "sqlparser", + "sysinfo", "tempfile", + "test-utils", "tokio", "tokio-util", "url", @@ -1255,6 +1785,27 @@ dependencies = [ "zstd", ] +[[package]] +name = "datafusion-benchmarks" +version = "45.0.0" +dependencies = [ + "arrow", + "datafusion", + "datafusion-common", + "datafusion-proto", + "env_logger", + "futures", + "log", + "mimalloc", + "parquet", + "serde", + "serde_json", + "snmalloc-rs", + "structopt", + "test-utils", + "tokio", +] + [[package]] name = "datafusion-catalog" version = "45.0.0" @@ -1272,6 +1823,7 @@ dependencies = [ "log", "parking_lot", "sqlparser", + "tokio", ] [[package]] @@ -1280,6 +1832,8 @@ version = "45.0.0" dependencies = [ "arrow", "arrow-schema", + "async-compression", + "async-trait", "chrono", "datafusion-catalog", "datafusion-common", @@ -1293,6 +1847,8 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", + "tempfile", + "tokio", "url", ] @@ -1305,7 +1861,7 @@ dependencies = [ "async-trait", "aws-config", "aws-credential-types", - "clap", + "clap 4.5.28", "ctor", "datafusion", "dirs", @@ -1327,20 +1883,23 @@ dependencies = [ name = "datafusion-common" version = "45.0.0" dependencies = [ - "ahash", + "ahash 0.8.11", "apache-avro", "arrow", "arrow-ipc", "arrow-schema", "base64 0.22.1", + "chrono", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.7.1", "libc", "log", "object_store", "parquet", "paste", + "pyo3", + "rand 0.8.5", "recursive", "sqlparser", "tokio", @@ -1359,11 +1918,39 @@ dependencies = [ name = "datafusion-doc" version = "45.0.0" +[[package]] +name = "datafusion-examples" +version = "45.0.0" +dependencies = [ + "arrow", + "arrow-flight", + "arrow-schema", + "async-trait", + "bytes", + "dashmap", + "datafusion", + "datafusion-proto", + "env_logger", + "futures", + "log", + "mimalloc", + "nix 0.28.0", + "object_store", + "prost", + "tempfile", + "test-utils", + "tokio", + "tonic", + "url", + "uuid", +] + [[package]] name = "datafusion-execution" version = "45.0.0" dependencies = [ "arrow", + "chrono", "dashmap", "datafusion-common", "datafusion-expr", @@ -1371,7 +1958,7 @@ dependencies = [ "log", "object_store", "parking_lot", - "rand", + "rand 0.8.5", "tempfile", "url", ] @@ -1382,13 +1969,15 @@ version = "45.0.0" dependencies = [ "arrow", "chrono", + "ctor", "datafusion-common", "datafusion-doc", "datafusion-expr-common", "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap", + "env_logger", + "indexmap 2.7.1", "paste", "recursive", "serde_json", @@ -1405,6 +1994,25 @@ dependencies = [ "paste", ] +[[package]] +name = "datafusion-ffi" +version = "45.0.0" +dependencies = [ + "abi_stable", + "arrow", + "arrow-schema", + "async-ffi", + "async-trait", + "datafusion", + "datafusion-proto", + "doc-comment", + "futures", + "log", + "prost", + "semver", + "tokio", +] + [[package]] name = "datafusion-functions" version = "45.0.0" @@ -1415,6 +2023,7 @@ dependencies = [ "blake2", "blake3", "chrono", + "criterion", "datafusion-common", "datafusion-doc", "datafusion-execution", @@ -1426,9 +2035,10 @@ dependencies = [ "itertools 0.14.0", "log", "md-5", - "rand", + "rand 0.8.5", "regex", "sha2", + "tokio", "unicode-segmentation", "uuid", ] @@ -1437,9 +2047,10 @@ dependencies = [ name = "datafusion-functions-aggregate" version = "45.0.0" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow", "arrow-schema", + "criterion", "datafusion-common", "datafusion-doc", "datafusion-execution", @@ -1451,17 +2062,20 @@ dependencies = [ "half", "log", "paste", + "rand 0.8.5", ] [[package]] name = "datafusion-functions-aggregate-common" version = "45.0.0" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow", + "criterion", "datafusion-common", "datafusion-expr-common", "datafusion-physical-expr-common", + "rand 0.8.5", ] [[package]] @@ -1471,6 +2085,7 @@ dependencies = [ "arrow", "arrow-ord", "arrow-schema", + "criterion", "datafusion-common", "datafusion-doc", "datafusion-execution", @@ -1482,6 +2097,7 @@ dependencies = [ "itertools 0.14.0", "log", "paste", + "rand 0.8.5", ] [[package]] @@ -1502,6 +2118,7 @@ dependencies = [ name = "datafusion-functions-window" version = "45.0.0" dependencies = [ + "arrow", "datafusion-common", "datafusion-doc", "datafusion-expr", @@ -1527,7 +2144,7 @@ version = "45.0.0" dependencies = [ "datafusion-expr", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -1535,11 +2152,18 @@ name = "datafusion-optimizer" version = "45.0.0" dependencies = [ "arrow", + "async-trait", "chrono", + "ctor", "datafusion-common", "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-functions-window", + "datafusion-functions-window-common", "datafusion-physical-expr", - "indexmap", + "datafusion-sql", + "env_logger", + "indexmap 2.7.1", "itertools 0.14.0", "log", "recursive", @@ -1551,28 +2175,32 @@ dependencies = [ name = "datafusion-physical-expr" version = "45.0.0" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow", "arrow-schema", + "criterion", "datafusion-common", "datafusion-expr", "datafusion-expr-common", + "datafusion-functions", "datafusion-functions-aggregate-common", "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.7.1", "itertools 0.14.0", "log", "paste", - "petgraph", + "petgraph 0.7.1", + "rand 0.8.5", + "rstest", ] [[package]] name = "datafusion-physical-expr-common" version = "45.0.0" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow", "datafusion-common", "datafusion-expr-common", @@ -1590,6 +2218,7 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-expr-common", + "datafusion-functions-nested", "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", @@ -1597,6 +2226,8 @@ dependencies = [ "itertools 0.14.0", "log", "recursive", + "rstest", + "tokio", "url", ] @@ -1604,30 +2235,72 @@ dependencies = [ name = "datafusion-physical-plan" version = "45.0.0" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow", "arrow-ord", "arrow-schema", "async-trait", "chrono", + "criterion", "datafusion-common", "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-functions-window", "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", "futures", "half", "hashbrown 0.14.5", - "indexmap", + "indexmap 2.7.1", "itertools 0.14.0", "log", "parking_lot", "pin-project-lite", + "rand 0.8.5", + "rstest", + "rstest_reuse", + "tokio", +] + +[[package]] +name = "datafusion-proto" +version = "45.0.0" +dependencies = [ + "arrow", + "chrono", + "datafusion", + "datafusion-common", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-window-common", + "datafusion-proto-common", + "doc-comment", + "object_store", + "pbjson", + "prost", + "serde", + "serde_json", + "strum", "tokio", ] +[[package]] +name = "datafusion-proto-common" +version = "45.0.0" +dependencies = [ + "arrow", + "datafusion-common", + "doc-comment", + "pbjson", + "prost", + "serde", + "serde_json", +] + [[package]] name = "datafusion-sql" version = "45.0.0" @@ -1635,35 +2308,127 @@ dependencies = [ "arrow", "arrow-schema", "bigdecimal", + "ctor", "datafusion-common", "datafusion-expr", - "indexmap", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-nested", + "datafusion-functions-window", + "env_logger", + "indexmap 2.7.1", "log", + "paste", "recursive", "regex", + "rstest", "sqlparser", ] [[package]] -name = "deranged" -version = "0.3.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b42b6fa04a440b495c8b04d0e71b707c585f83cb9cb28cf8cd0d976c315e31b4" +name = "datafusion-sqllogictest" +version = "45.0.0" dependencies = [ - "powerfmt", + "arrow", + "async-trait", + "bigdecimal", + "bytes", + "chrono", + "clap 4.5.28", + "datafusion", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "env_logger", + "futures", + "half", + "indicatif", + "itertools 0.14.0", + "log", + "object_store", + "postgres-protocol", + "postgres-types", + "rust_decimal", + "sqllogictest", + "sqlparser", + "tempfile", + "testcontainers", + "testcontainers-modules", + "thiserror 2.0.11", + "tokio", + "tokio-postgres", ] [[package]] -name = "difflib" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6184e33543162437515c2e2b48714794e37845ec9851711914eec9d308f6ebe8" - -[[package]] -name = "digest" -version = "0.10.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +name = "datafusion-substrait" +version = "45.0.0" +dependencies = [ + "async-recursion", + "async-trait", + "chrono", + "datafusion", + "datafusion-functions-aggregate", + "itertools 0.14.0", + "object_store", + "pbjson-types", + "prost", + "serde_json", + "substrait", + "tokio", + "url", +] + +[[package]] +name = "datafusion-wasmtest" +version = "45.0.0" +dependencies = [ + "chrono", + "console_error_panic_hook", + "datafusion", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", + "datafusion-functions-table", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-sql", + "getrandom 0.2.15", + "parquet", + "tokio", + "wasm-bindgen", + "wasm-bindgen-futures", + "wasm-bindgen-test", +] + +[[package]] +name = "deranged" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b42b6fa04a440b495c8b04d0e71b707c585f83cb9cb28cf8cd0d976c315e31b4" +dependencies = [ + "powerfmt", + "serde", +] + +[[package]] +name = "difflib" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6184e33543162437515c2e2b48714794e37845ec9851711914eec9d308f6ebe8" + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", "crypto-common", @@ -1699,7 +2464,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -1708,18 +2473,73 @@ version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fea41bba32d969b513997752735605054bc0dfa92b4c56bf1189f2e174be7a10" +[[package]] +name = "docker_credential" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "31951f49556e34d90ed28342e1df7e1cb7a229c4cab0aecc627b5d91edd41d07" +dependencies = [ + "base64 0.21.7", + "serde", + "serde_json", +] + +[[package]] +name = "dyn-clone" +version = "1.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "feeef44e73baff3a26d371801df019877a9866a8c493d315ab00177843314f35" + +[[package]] +name = "educe" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d7bc049e1bd8cdeb31b68bbd586a9464ecf9f3944af3958a7a9d0f8b9799417" +dependencies = [ + "enum-ordinalize", + "proc-macro2", + "quote", + "syn 2.0.98", +] + [[package]] name = "either" version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" +[[package]] +name = "encode_unicode" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34aa73646ffb006b8f5147f3dc182bd4bcb190227ce861fc4a4844bf8e3cb2c0" + [[package]] name = "endian-type" version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" +[[package]] +name = "enum-ordinalize" +version = "4.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +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 = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + [[package]] name = "env_filter" version = "0.1.3" @@ -1765,6 +2585,29 @@ version = "3.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a5d9305ccc6942a704f4335694ecd3de2ea531b114ac2d51f5f843750787a92f" +[[package]] +name = "escape8259" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5692dd7b5a1978a5aeb0ce83b7655c58ca8efdcb79d21036ea249da95afec2c6" + +[[package]] +name = "etcetera" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "136d1b5283a1ab77bd9257427ffd09d8667ced0570b6f938942bc7568ed5b943" +dependencies = [ + "cfg-if", + "home", + "windows-sys 0.48.0", +] + +[[package]] +name = "fallible-iterator" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4443176a9f2c162692bd3d352d745ef9413eec5782a80d8fd6f8a1ac692a07f7" + [[package]] name = "fastrand" version = "2.3.0" @@ -1782,6 +2625,55 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "ffi_example_table_provider" +version = "0.1.0" +dependencies = [ + "abi_stable", + "arrow", + "arrow-schema", + "datafusion", + "datafusion-ffi", + "ffi_module_interface", +] + +[[package]] +name = "ffi_module_interface" +version = "0.1.0" +dependencies = [ + "abi_stable", + "datafusion-ffi", +] + +[[package]] +name = "ffi_module_loader" +version = "0.1.0" +dependencies = [ + "abi_stable", + "datafusion", + "datafusion-ffi", + "ffi_module_interface", + "tokio", +] + +[[package]] +name = "filetime" +version = "0.2.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35c0522e981e68cbfa8c3f978441a5f34b30b96e146b33cd3359176b50fe8586" +dependencies = [ + "cfg-if", + "libc", + "libredox", + "windows-sys 0.59.0", +] + +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + [[package]] name = "fixedbitset" version = "0.5.7" @@ -1823,6 +2715,12 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" +[[package]] +name = "foldhash" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0d2fde1f7b3d48b8395d5f2de76c18a528bd6a9cdde438df747bfcba3e05d6f" + [[package]] name = "form_urlencoded" version = "1.2.1" @@ -1832,6 +2730,21 @@ dependencies = [ "percent-encoding", ] +[[package]] +name = "fs-err" +version = "3.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f89bda4c2a21204059a977ed3bfe746677dfd137b83c339e702b0ac91d482aa" +dependencies = [ + "autocfg", +] + +[[package]] +name = "funty" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" + [[package]] name = "futures" version = "0.3.31" @@ -1888,7 +2801,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -1927,6 +2840,31 @@ dependencies = [ "slab", ] +[[package]] +name = "gen" +version = "0.1.0" +dependencies = [ + "pbjson-build", + "prost-build", +] + +[[package]] +name = "gen-common" +version = "0.1.0" +dependencies = [ + "pbjson-build", + "prost-build", +] + +[[package]] +name = "generational-arena" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "877e94aff08e743b651baaea359664321055749b398adff8740a7399af7796e7" +dependencies = [ + "cfg-if", +] + [[package]] name = "generic-array" version = "0.14.7" @@ -1959,7 +2897,7 @@ dependencies = [ "cfg-if", "libc", "wasi 0.13.3+wasi-0.2.2", - "windows-targets", + "windows-targets 0.52.6", ] [[package]] @@ -1986,7 +2924,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap", + "indexmap 2.7.1", "slab", "tokio", "tokio-util", @@ -2005,7 +2943,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.2.0", - "indexmap", + "indexmap 2.7.1", "slab", "tokio", "tokio-util", @@ -2023,13 +2961,22 @@ dependencies = [ "num-traits", ] +[[package]] +name = "hashbrown" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8a9ee70c43aaf417c914396645a0fa852624801b24ebb7ae78fe8272889ac888" +dependencies = [ + "ahash 0.7.8", +] + [[package]] name = "hashbrown" version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" dependencies = [ - "ahash", + "ahash 0.8.11", "allocator-api2", ] @@ -2038,6 +2985,20 @@ name = "hashbrown" version = "0.15.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" +dependencies = [ + "allocator-api2", + "equivalent", + "foldhash", +] + +[[package]] +name = "heck" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" +dependencies = [ + "unicode-segmentation", +] [[package]] name = "heck" @@ -2045,6 +3006,12 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" +[[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" @@ -2180,6 +3147,7 @@ dependencies = [ "http 1.2.0", "http-body 1.0.1", "httparse", + "httpdate", "itoa", "pin-project-lite", "smallvec", @@ -2187,6 +3155,21 @@ dependencies = [ "want", ] +[[package]] +name = "hyper-named-pipe" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73b7d8abf35697b81a825e386fc151e0d503e8cb5fcb93cc8669c376dfd6f278" +dependencies = [ + "hex", + "hyper 1.6.0", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", + "winapi", +] + [[package]] name = "hyper-rustls" version = "0.24.2" @@ -2221,6 +3204,19 @@ dependencies = [ "tower-service", ] +[[package]] +name = "hyper-timeout" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b90d566bffbce6a75bd8b09a05aa8c2cb1fabb6cb348f8840c9e4c90a0d83b0" +dependencies = [ + "hyper 1.6.0", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", +] + [[package]] name = "hyper-util" version = "0.1.10" @@ -2240,6 +3236,21 @@ dependencies = [ "tracing", ] +[[package]] +name = "hyperlocal" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "986c5ce3b994526b3cd75578e62554abd09f0899d6206de48b3e96ab34ccc8c7" +dependencies = [ + "hex", + "http-body-util", + "hyper 1.6.0", + "hyper-util", + "pin-project-lite", + "tokio", + "tower-service", +] + [[package]] name = "iana-time-zone" version = "0.1.61" @@ -2251,7 +3262,7 @@ dependencies = [ "iana-time-zone-haiku", "js-sys", "wasm-bindgen", - "windows-core", + "windows-core 0.52.0", ] [[package]] @@ -2378,9 +3389,15 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] +[[package]] +name = "ident_case" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" + [[package]] name = "idna" version = "1.0.3" @@ -2402,6 +3419,17 @@ dependencies = [ "icu_properties", ] +[[package]] +name = "indexmap" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd070e393353796e801d209ad339e89596eb4c8d430d18ede6a1cced8fafbd99" +dependencies = [ + "autocfg", + "hashbrown 0.12.3", + "serde", +] + [[package]] name = "indexmap" version = "2.7.1" @@ -2410,26 +3438,66 @@ checksum = "8c9c992b02b5b4c94ea26e32fe5bccb7aa7d9f390ab5c1221ff895bc7ea8b652" dependencies = [ "equivalent", "hashbrown 0.15.2", + "serde", ] [[package]] -name = "integer-encoding" -version = "3.0.4" +name = "indicatif" +version = "0.17.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" +checksum = "183b3088984b400f4cfac3620d5e076c84da5364016b4f49473de574b2586235" +dependencies = [ + "console", + "number_prefix", + "portable-atomic", + "unicode-width 0.2.0", + "web-time", +] [[package]] -name = "ipnet" -version = "2.11.0" +name = "indoc" +version = "2.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b248f5224d1d606005e02c97f5aa4e88eeb230488bcc03bc9ca4d7991399f2b5" + +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "ipnet" +version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" +[[package]] +name = "is-terminal" +version = "0.4.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e19b23d53f35ce9f56aebc7d1bb4e6ac1e9c0db7ac85c8d1760c04379edced37" +dependencies = [ + "hermit-abi", + "libc", + "windows-sys 0.59.0", +] + [[package]] name = "is_terminal_polyfill" version = "1.70.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + [[package]] name = "itertools" version = "0.13.0" @@ -2573,6 +3641,16 @@ dependencies = [ "rle-decode-fast", ] +[[package]] +name = "libloading" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b67380fd3b2fbe7527a606e18729d21c6f3951633d0500574c4dc22d2d638b9f" +dependencies = [ + "cfg-if", + "winapi", +] + [[package]] name = "libm" version = "0.2.11" @@ -2597,6 +3675,19 @@ checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ "bitflags 2.8.0", "libc", + "redox_syscall 0.5.8", +] + +[[package]] +name = "libtest-mimic" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5297962ef19edda4ce33aaa484386e0a5b3d7f2f4e037cbeee00503ef6b29d33" +dependencies = [ + "anstream", + "anstyle", + "clap 4.5.28", + "escape8259", ] [[package]] @@ -2647,6 +3738,12 @@ dependencies = [ "pkg-config", ] +[[package]] +name = "matchit" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" + [[package]] name = "md-5" version = "0.10.6" @@ -2663,6 +3760,15 @@ version = "2.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" +[[package]] +name = "memoffset" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "488016bfae457b036d996092f6cb448677611ce4449e970ceaf42695203f218a" +dependencies = [ + "autocfg", +] + [[package]] name = "mimalloc" version = "0.1.43" @@ -2678,6 +3784,16 @@ version = "0.3.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" +[[package]] +name = "minicov" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f27fe9f1cc3c22e1687f9446c2083c4c5fc7f0bcf1c7a86bdbded14985895b4b" +dependencies = [ + "cc", + "walkdir", +] + [[package]] name = "miniz_oxide" version = "0.8.3" @@ -2698,6 +3814,12 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "multimap" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "defc4c55412d89136f966bbb339008b474350e5e6e78d2714439c386b3137a03" + [[package]] name = "nibble_vec" version = "0.1.0" @@ -2707,6 +3829,18 @@ dependencies = [ "smallvec", ] +[[package]] +name = "nix" +version = "0.28.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab2156c4fce2f8df6c499cc1c763e4394b7482525bf2a9701c9d79d215f519e4" +dependencies = [ + "bitflags 2.8.0", + "cfg-if", + "cfg_aliases 0.1.1", + "libc", +] + [[package]] name = "nix" version = "0.29.0" @@ -2715,7 +3849,7 @@ checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ "bitflags 2.8.0", "cfg-if", - "cfg_aliases", + "cfg_aliases 0.2.1", "libc", ] @@ -2725,6 +3859,15 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "61807f77802ff30975e01f4f071c8ba10c022052f98b3294119f3e615d13e5be" +[[package]] +name = "ntapi" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8a3895c6391c39d7fe7ebc444a87eb2991b2a0bc718fdabd071eec617fc68e4" +dependencies = [ + "winapi", +] + [[package]] name = "num" version = "0.4.3" @@ -2806,6 +3949,12 @@ dependencies = [ "libm", ] +[[package]] +name = "number_prefix" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" + [[package]] name = "object" version = "0.36.7" @@ -2833,7 +3982,7 @@ dependencies = [ "parking_lot", "percent-encoding", "quick-xml", - "rand", + "rand 0.8.5", "reqwest", "ring", "rustls-pemfile 2.2.0", @@ -2852,6 +4001,12 @@ version = "1.20.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "945462a4b81e43c4e3ba96bd7b49d834c6f61198356aa858733bc4acf3cbe62e" +[[package]] +name = "oorandom" +version = "11.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9" + [[package]] name = "openssl-probe" version = "0.1.6" @@ -2879,6 +4034,12 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1a80800c0488c3a21695ea981a54918fbb37abf04f4d0720c453632255e2ff0e" +[[package]] +name = "owo-colors" +version = "4.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb37767f6569cd834a413442455e0f066d0d522de8630436e2a1761d9726ba56" + [[package]] name = "parking_lot" version = "0.12.3" @@ -2897,9 +4058,9 @@ checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" dependencies = [ "cfg-if", "libc", - "redox_syscall", + "redox_syscall 0.5.8", "smallvec", - "windows-targets", + "windows-targets 0.52.6", ] [[package]] @@ -2908,7 +4069,7 @@ version = "54.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8a01a0efa30bbd601ae85b375c728efdb211ade54390281628a7b16708beb235" dependencies = [ - "ahash", + "ahash 0.8.11", "arrow-array", "arrow-buffer", "arrow-cast", @@ -2939,6 +4100,31 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "parse-display" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "914a1c2265c98e2446911282c6ac86d8524f495792c38c5bd884f80499c7538a" +dependencies = [ + "parse-display-derive", + "regex", + "regex-syntax", +] + +[[package]] +name = "parse-display-derive" +version = "0.9.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2ae7800a4c974efd12df917266338e79a7a74415173caf7e70aa0a0707345281" +dependencies = [ + "proc-macro2", + "quote", + "regex", + "regex-syntax", + "structmeta", + "syn 2.0.98", +] + [[package]] name = "parse-zoneinfo" version = "0.3.1" @@ -2954,20 +4140,67 @@ version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" +[[package]] +name = "pbjson" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7e6349fa080353f4a597daffd05cb81572a9c031a6d4fff7e504947496fcc68" +dependencies = [ + "base64 0.21.7", + "serde", +] + +[[package]] +name = "pbjson-build" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6eea3058763d6e656105d1403cb04e0a41b7bbac6362d413e7c33be0c32279c9" +dependencies = [ + "heck 0.5.0", + "itertools 0.13.0", + "prost", + "prost-types", +] + +[[package]] +name = "pbjson-types" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e54e5e7bfb1652f95bc361d76f3c780d8e526b134b85417e774166ee941f0887" +dependencies = [ + "bytes", + "chrono", + "pbjson", + "pbjson-build", + "prost", + "prost-build", + "serde", +] + [[package]] name = "percent-encoding" version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" +[[package]] +name = "petgraph" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" +dependencies = [ + "fixedbitset 0.4.2", + "indexmap 2.7.1", +] + [[package]] name = "petgraph" version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ - "fixedbitset", - "indexmap", + "fixedbitset 0.5.7", + "indexmap 2.7.1", ] [[package]] @@ -2996,7 +4229,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c80231409c20246a13fddb31776fb942c38553c51e871f8cbd687a4cfb5843d" dependencies = [ "phf_shared", - "rand", + "rand 0.8.5", ] [[package]] @@ -3008,6 +4241,26 @@ dependencies = [ "siphasher", ] +[[package]] +name = "pin-project" +version = "1.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dfe2e71e1471fe07709406bf725f710b02927c9c54b2b5b2ec0e8087d97c327d" +dependencies = [ + "pin-project-internal", +] + +[[package]] +name = "pin-project-internal" +version = "1.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6e859e6e5bd50440ab63c47e3ebabc90f26251f7c73c3d3e837b74a1cc3fa67" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + [[package]] name = "pin-project-lite" version = "0.2.16" @@ -3026,6 +4279,83 @@ version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "953ec861398dccce10c670dfeaf3ec4911ca479e9c02154b3a215178c5f566f2" +[[package]] +name = "plotters" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5aeb6f403d7a4911efb1e33402027fc44f29b5bf6def3effcc22d7bb75f2b747" +dependencies = [ + "num-traits", + "plotters-backend", + "plotters-svg", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "plotters-backend" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df42e13c12958a16b3f7f4386b9ab1f3e7933914ecea48da7139435263a4172a" + +[[package]] +name = "plotters-svg" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51bae2ac328883f7acdfea3d66a7c35751187f870bc81f94563733a154d7a670" +dependencies = [ + "plotters-backend", +] + +[[package]] +name = "portable-atomic" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "280dc24453071f1b63954171985a0b0d30058d287960968b9b2aca264c8d4ee6" + +[[package]] +name = "postgres-derive" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69700ea4603c5ef32d447708e6a19cd3e8ac197a000842e97f527daea5e4175f" +dependencies = [ + "heck 0.5.0", + "proc-macro2", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "postgres-protocol" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76ff0abab4a9b844b93ef7b81f1efc0a366062aaef2cd702c76256b5dc075c54" +dependencies = [ + "base64 0.22.1", + "byteorder", + "bytes", + "fallible-iterator", + "hmac", + "md-5", + "memchr", + "rand 0.9.0", + "sha2", + "stringprep", +] + +[[package]] +name = "postgres-types" +version = "0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "613283563cd90e1dfc3518d548caee47e0e725455ed619881f5cf21f36de4b48" +dependencies = [ + "bytes", + "chrono", + "fallible-iterator", + "postgres-derive", + "postgres-protocol", +] + [[package]] name = "powerfmt" version = "0.2.0" @@ -3038,7 +4368,7 @@ version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" dependencies = [ - "zerocopy", + "zerocopy 0.7.35", ] [[package]] @@ -3071,6 +4401,16 @@ dependencies = [ "termtree", ] +[[package]] +name = "prettyplease" +version = "0.2.29" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6924ced06e1f7dfe3fa48d57b9f74f55d8915f5036121bef647ef4b204895fac" +dependencies = [ + "proc-macro2", + "syn 2.0.98", +] + [[package]] name = "proc-macro-crate" version = "3.2.0" @@ -3080,6 +4420,30 @@ dependencies = [ "toml_edit", ] +[[package]] +name = "proc-macro-error" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "da25490ff9892aab3fcf7c36f08cfb902dd3e71ca0f9f9517bea02a73a5ce38c" +dependencies = [ + "proc-macro-error-attr", + "proc-macro2", + "quote", + "syn 1.0.109", + "version_check", +] + +[[package]] +name = "proc-macro-error-attr" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a1be40180e52ecc98ad80b184934baf3d0d29f979574e439af5a55274b35f869" +dependencies = [ + "proc-macro2", + "quote", + "version_check", +] + [[package]] name = "proc-macro2" version = "1.0.93" @@ -3090,57 +4454,201 @@ dependencies = [ ] [[package]] -name = "psm" -version = "0.1.24" +name = "prost" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200b9ff220857e53e184257720a14553b2f4aa02577d2ed9842d45d4b9654810" +checksum = "2c0fef6c4230e4ccf618a35c59d7ede15dea37de8427500f50aff708806e42ec" dependencies = [ - "cc", + "bytes", + "prost-derive", ] [[package]] -name = "quad-rand" -version = "0.2.3" +name = "prost-build" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" +checksum = "d0f3e5beed80eb580c68e2c600937ac2c4eedabdfd5ef1e5b7ea4f3fba84497b" +dependencies = [ + "heck 0.5.0", + "itertools 0.13.0", + "log", + "multimap", + "once_cell", + "petgraph 0.6.5", + "prettyplease", + "prost", + "prost-types", + "regex", + "syn 2.0.98", + "tempfile", +] [[package]] -name = "quick-xml" -version = "0.37.2" +name = "prost-derive" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "165859e9e55f79d67b96c5d96f4e88b6f2695a1972849c15a6a3f5c59fc2c003" +checksum = "157c5a9d7ea5c2ed2d9fb8f495b64759f7816c7eaea54ba3978f0d63000162e3" dependencies = [ - "memchr", - "serde", + "anyhow", + "itertools 0.13.0", + "proc-macro2", + "quote", + "syn 2.0.98", ] [[package]] -name = "quinn" -version = "0.11.6" +name = "prost-types" +version = "0.13.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62e96808277ec6f97351a2380e6c25114bc9e67037775464979f3037c92d05ef" +checksum = "cc2f1e56baa61e93533aebc21af4d2134b70f66275e0fcdf3cbe43d77ff7e8fc" dependencies = [ - "bytes", - "pin-project-lite", - "quinn-proto", - "quinn-udp", - "rustc-hash", - "rustls 0.23.22", - "socket2", - "thiserror 2.0.11", - "tokio", - "tracing", + "prost", ] [[package]] -name = "quinn-proto" -version = "0.11.9" +name = "protobuf-src" +version = "2.1.1+27.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6217c3504da19b85a3a4b2e9a5183d635822d83507ba0986624b5c05b83bfc40" +dependencies = [ + "cmake", +] + +[[package]] +name = "psm" +version = "0.1.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "200b9ff220857e53e184257720a14553b2f4aa02577d2ed9842d45d4b9654810" +dependencies = [ + "cc", +] + +[[package]] +name = "ptr_meta" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0738ccf7ea06b608c10564b31debd4f5bc5e197fc8bfe088f68ae5ce81e7a4f1" +dependencies = [ + "ptr_meta_derive", +] + +[[package]] +name = "ptr_meta_derive" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b845dbfca988fa33db069c0e230574d15a3088f147a87b64c7589eb662c9ac" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "pyo3" +version = "0.23.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57fe09249128b3173d092de9523eaa75136bf7ba85e0d69eca241c7939c933cc" +dependencies = [ + "cfg-if", + "indoc", + "libc", + "memoffset", + "once_cell", + "portable-atomic", + "pyo3-build-config", + "pyo3-ffi", + "pyo3-macros", + "unindent", +] + +[[package]] +name = "pyo3-build-config" +version = "0.23.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1cd3927b5a78757a0d71aa9dff669f903b1eb64b54142a9bd9f757f8fde65fd7" +dependencies = [ + "once_cell", + "target-lexicon", +] + +[[package]] +name = "pyo3-ffi" +version = "0.23.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dab6bb2102bd8f991e7749f130a70d05dd557613e39ed2deeee8e9ca0c4d548d" +dependencies = [ + "libc", + "pyo3-build-config", +] + +[[package]] +name = "pyo3-macros" +version = "0.23.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "91871864b353fd5ffcb3f91f2f703a22a9797c91b9ab497b1acac7b07ae509c7" +dependencies = [ + "proc-macro2", + "pyo3-macros-backend", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "pyo3-macros-backend" +version = "0.23.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43abc3b80bc20f3facd86cd3c60beed58c3e2aa26213f3cda368de39c60a27e4" +dependencies = [ + "heck 0.5.0", + "proc-macro2", + "pyo3-build-config", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "quad-rand" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" + +[[package]] +name = "quick-xml" +version = "0.37.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "165859e9e55f79d67b96c5d96f4e88b6f2695a1972849c15a6a3f5c59fc2c003" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quinn" +version = "0.11.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62e96808277ec6f97351a2380e6c25114bc9e67037775464979f3037c92d05ef" +dependencies = [ + "bytes", + "pin-project-lite", + "quinn-proto", + "quinn-udp", + "rustc-hash", + "rustls 0.23.22", + "socket2", + "thiserror 2.0.11", + "tokio", + "tracing", +] + +[[package]] +name = "quinn-proto" +version = "0.11.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2fe5ef3495d7d2e377ff17b1a8ce2ee2ec2a18cde8b6ad6619d65d0701c135d" dependencies = [ "bytes", "getrandom 0.2.15", - "rand", + "rand 0.8.5", "ring", "rustc-hash", "rustls 0.23.22", @@ -3158,7 +4666,7 @@ version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1c40286217b4ba3a71d644d752e6a0b71f13f1b6a2c5311acfcbe0c2418ed904" dependencies = [ - "cfg_aliases", + "cfg_aliases 0.2.1", "libc", "once_cell", "socket2", @@ -3175,6 +4683,12 @@ dependencies = [ "proc-macro2", ] +[[package]] +name = "radium" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc33ff2d4973d518d823d61aa239014831e521c75da58e3df4840d3f47749d09" + [[package]] name = "radix_trie" version = "0.2.1" @@ -3192,8 +4706,19 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" dependencies = [ "libc", - "rand_chacha", - "rand_core", + "rand_chacha 0.3.1", + "rand_core 0.6.4", +] + +[[package]] +name = "rand" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3779b94aeb87e8bd4e834cee3650289ee9e0d5677f976ecdb6d219e5f4f6cd94" +dependencies = [ + "rand_chacha 0.9.0", + "rand_core 0.9.0", + "zerocopy 0.8.17", ] [[package]] @@ -3203,7 +4728,17 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" dependencies = [ "ppv-lite86", - "rand_core", + "rand_core 0.6.4", +] + +[[package]] +name = "rand_chacha" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" +dependencies = [ + "ppv-lite86", + "rand_core 0.9.0", ] [[package]] @@ -3215,6 +4750,46 @@ dependencies = [ "getrandom 0.2.15", ] +[[package]] +name = "rand_core" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b08f3c9802962f7e1b25113931d94f43ed9725bebc59db9d0c3e9a23b67e15ff" +dependencies = [ + "getrandom 0.3.1", + "zerocopy 0.8.17", +] + +[[package]] +name = "rand_distr" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32cb0b9bc82b0a0876c2dd994a7e7a2683d3e7390ca40e6886785ef0c7e3ee31" +dependencies = [ + "num-traits", + "rand 0.8.5", +] + +[[package]] +name = "rayon" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b418a60154510ca1a002a752ca9714984e21e4241e804d32555251faf8b78ffa" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + [[package]] name = "recursive" version = "0.1.1" @@ -3232,7 +4807,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn", + "syn 2.0.98", +] + +[[package]] +name = "redox_syscall" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" +dependencies = [ + "bitflags 1.3.2", ] [[package]] @@ -3290,12 +4874,40 @@ version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" +[[package]] +name = "regress" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78ef7fa9ed0256d64a688a3747d0fef7a88851c18a5e1d57f115f38ec2e09366" +dependencies = [ + "hashbrown 0.15.2", + "memchr", +] + [[package]] name = "relative-path" version = "1.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ba39f3699c378cd8970968dcbff9c43159ea4cfbd88d43c00b22f2ef10a435d2" +[[package]] +name = "rend" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "71fe3824f5629716b1589be05dacd749f6aa084c87e00e016714a8cdfccc997c" +dependencies = [ + "bytecheck", +] + +[[package]] +name = "repr_offset" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb1070755bd29dffc19d0971cab794e607839ba2ef4b69a9e6fbc8733c1b72ea" +dependencies = [ + "tstr", +] + [[package]] name = "reqwest" version = "0.12.12" @@ -3332,7 +4944,7 @@ dependencies = [ "tokio", "tokio-rustls 0.26.1", "tokio-util", - "tower", + "tower 0.5.2", "tower-service", "url", "wasm-bindgen", @@ -3357,6 +4969,35 @@ dependencies = [ "windows-sys 0.52.0", ] +[[package]] +name = "rkyv" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9008cd6385b9e161d8229e1f6549dd23c3d022f132a2ea37ac3a10ac4935779b" +dependencies = [ + "bitvec", + "bytecheck", + "bytes", + "hashbrown 0.12.3", + "ptr_meta", + "rend", + "rkyv_derive", + "seahash", + "tinyvec", + "uuid", +] + +[[package]] +name = "rkyv_derive" +version = "0.7.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "503d1d27590a2b0a3a4ca4c94755aa2875657196ecbf401a42eff41d7de532c0" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "rle-decode-fast" version = "1.0.3" @@ -3389,10 +5030,38 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn", + "syn 2.0.98", "unicode-ident", ] +[[package]] +name = "rstest_reuse" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b3a8fb4672e840a587a66fc577a5491375df51ddb88f2a2c2a792598c326fe14" +dependencies = [ + "quote", + "rand 0.8.5", + "syn 2.0.98", +] + +[[package]] +name = "rust_decimal" +version = "1.36.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b082d80e3e3cc52b2ed634388d436fe1f4de6af5786cc2de9ba9737527bdf555" +dependencies = [ + "arrayvec", + "borsh", + "bytes", + "num-traits", + "postgres-types", + "rand 0.8.5", + "rkyv", + "serde", + "serde_json", +] + [[package]] name = "rustc-demangle" version = "0.1.24" @@ -3545,10 +5214,10 @@ dependencies = [ "libc", "log", "memchr", - "nix", + "nix 0.29.0", "radix_trie", "unicode-segmentation", - "unicode-width", + "unicode-width 0.2.0", "utf8parse", "windows-sys 0.59.0", ] @@ -3577,6 +5246,30 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "schemars" +version = "0.8.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09c024468a378b7e36765cd36702b7a90cc3cba11654f6685c8f233408e89e92" +dependencies = [ + "dyn-clone", + "schemars_derive", + "serde", + "serde_json", +] + +[[package]] +name = "schemars_derive" +version = "0.8.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1eee588578aff73f856ab961cd2f79e36bc45d7ded33a7562adba4667aecc0e" +dependencies = [ + "proc-macro2", + "quote", + "serde_derive_internals", + "syn 2.0.98", +] + [[package]] name = "scopeguard" version = "1.2.0" @@ -3593,6 +5286,12 @@ dependencies = [ "untrusted", ] +[[package]] +name = "seahash" +version = "4.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" + [[package]] name = "security-framework" version = "2.11.1" @@ -3634,6 +5333,9 @@ name = "semver" version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f79dfe2d285b0488816f30e700a7438c5a73d816b5b7d3ac72fbc48b0d185e03" +dependencies = [ + "serde", +] [[package]] name = "seq-macro" @@ -3667,7 +5369,18 @@ checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", +] + +[[package]] +name = "serde_derive_internals" +version = "0.29.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", ] [[package]] @@ -3682,6 +5395,29 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_repr" +version = "0.1.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "serde_tokenstream" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "64060d864397305347a78851c51588fd283767e7e7589829e8121d65512340f1" +dependencies = [ + "proc-macro2", + "quote", + "serde", + "syn 2.0.98", +] + [[package]] name = "serde_urlencoded" version = "0.7.1" @@ -3694,6 +5430,49 @@ dependencies = [ "serde", ] +[[package]] +name = "serde_with" +version = "3.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6b6f7f2fcb69f747921f79f3926bd1e203fce4fef62c268dd3abfb6d86029aa" +dependencies = [ + "base64 0.22.1", + "chrono", + "hex", + "indexmap 1.9.3", + "indexmap 2.7.1", + "serde", + "serde_derive", + "serde_json", + "serde_with_macros", + "time", +] + +[[package]] +name = "serde_with_macros" +version = "3.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8d00caa5193a3c8362ac2b73be6b9e768aa5a4b2f721d8f4b339600c3cb51f8e" +dependencies = [ + "darling", + "proc-macro2", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "serde_yaml" +version = "0.9.34+deprecated" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" +dependencies = [ + "indexmap 2.7.1", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] + [[package]] name = "sha2" version = "0.10.8" @@ -3726,6 +5505,12 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3a9fe34e3e7a50316060351f37187a3f546bce95496156754b601a5fa71b76e" +[[package]] +name = "similar" +version = "2.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbbb5d9659141646ae647b42fe094daf6c6192d1620870b449d9557f748b2daa" + [[package]] name = "siphasher" version = "1.0.1" @@ -3762,10 +5547,10 @@ version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "03c3c6b7927ffe7ecaa769ee0e3994da3b8cafc8f444578982c83ecb161af917" dependencies = [ - "heck", + "heck 0.5.0", "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -3774,6 +5559,24 @@ version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" +[[package]] +name = "snmalloc-rs" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eb317153089fdfa4d8a2eec059d40a5a23c3bde43995ea23b19121c3f621e74a" +dependencies = [ + "snmalloc-sys", +] + +[[package]] +name = "snmalloc-sys" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "065fea53d32bb77bc36cca466cb191f2e5216ebfd0ed360b1d64889ee6e559ea" +dependencies = [ + "cmake", +] + [[package]] name = "socket2" version = "0.5.8" @@ -3790,6 +5593,31 @@ version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" +[[package]] +name = "sqllogictest" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3bc65b5525b4674a844eb6e39a5d4ef2385a3a2b96c13ef82bbe73220f24bcad" +dependencies = [ + "async-trait", + "educe", + "fs-err", + "futures", + "glob", + "humantime", + "itertools 0.13.0", + "libtest-mimic", + "md-5", + "owo-colors", + "rand 0.8.5", + "regex", + "similar", + "subst", + "tempfile", + "thiserror 2.0.11", + "tracing", +] + [[package]] name = "sqlparser" version = "0.54.0" @@ -3809,7 +5637,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -3837,17 +5665,78 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" +[[package]] +name = "stringprep" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7b4df3d392d81bd458a8a621b8bffbd2302a12ffe288a9d931670948749463b1" +dependencies = [ + "unicode-bidi", + "unicode-normalization", + "unicode-properties", +] + [[package]] name = "strsim" version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" +[[package]] +name = "structmeta" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e1575d8d40908d70f6fd05537266b90ae71b15dbbe7a8b7dffa2b759306d329" +dependencies = [ + "proc-macro2", + "quote", + "structmeta-derive", + "syn 2.0.98", +] + +[[package]] +name = "structmeta-derive" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "152a0b65a590ff6c3da95cabe2353ee04e6167c896b28e3b14478c2636c922fc" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "structopt" +version = "0.3.26" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c6b5c64445ba8094a6ab0c3cd2ad323e07171012d9c98b0b15651daf1787a10" +dependencies = [ + "clap 2.34.0", + "lazy_static", + "structopt-derive", +] + +[[package]] +name = "structopt-derive" +version = "0.4.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dcb5ae327f9cc13b68763b5749770cb9e048a99bd9dfdfa58d0cf05d5f64afe0" +dependencies = [ + "heck 0.3.3", + "proc-macro-error", + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" +dependencies = [ + "strum_macros", +] [[package]] name = "strum_macros" @@ -3855,11 +5744,47 @@ version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" dependencies = [ - "heck", + "heck 0.5.0", "proc-macro2", "quote", "rustversion", - "syn", + "syn 2.0.98", +] + +[[package]] +name = "subst" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33e7942675ea19db01ef8cf15a1e6443007208e6c74568bd64162da26d40160d" +dependencies = [ + "memchr", + "unicode-width 0.1.14", +] + +[[package]] +name = "substrait" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef201e3234acdb66865840012c0b9c3d04269b74416fb6285cd480b01718c2f9" +dependencies = [ + "heck 0.5.0", + "pbjson", + "pbjson-build", + "pbjson-types", + "prettyplease", + "prost", + "prost-build", + "prost-types", + "protobuf-src", + "regress", + "schemars", + "semver", + "serde", + "serde_json", + "serde_yaml", + "syn 2.0.98", + "typify", + "walkdir", ] [[package]] @@ -3868,6 +5793,17 @@ version = "2.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + [[package]] name = "syn" version = "2.0.98" @@ -3896,9 +5832,35 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", +] + +[[package]] +name = "sysinfo" +version = "0.33.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fc858248ea01b66f19d8e8a6d55f41deaf91e9d495246fd01368d99935c6c01" +dependencies = [ + "core-foundation-sys", + "libc", + "memchr", + "ntapi", + "rayon", + "windows", ] +[[package]] +name = "tap" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" + +[[package]] +name = "target-lexicon" +version = "0.12.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "61c41af27dd6d1e27b1b16b489db798443478cef1f06a660c96db617ba5de3b1" + [[package]] name = "tempfile" version = "3.16.0" @@ -3919,6 +5881,64 @@ version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8f50febec83f5ee1df3015341d8bd429f2d1cc62bcba7ea2076759d315084683" +[[package]] +name = "test-utils" +version = "0.1.0" +dependencies = [ + "arrow", + "chrono-tz", + "datafusion-common", + "env_logger", + "rand 0.8.5", +] + +[[package]] +name = "testcontainers" +version = "0.23.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "042009c52a4204476bff461ca8ef17bab6f1a91628504a8a36c6fd2c1cde2d5e" +dependencies = [ + "async-trait", + "bollard", + "bollard-stubs", + "bytes", + "docker_credential", + "either", + "etcetera", + "futures", + "log", + "memchr", + "parse-display", + "pin-project-lite", + "serde", + "serde_json", + "serde_with", + "thiserror 2.0.11", + "tokio", + "tokio-stream", + "tokio-tar", + "tokio-util", + "url", +] + +[[package]] +name = "testcontainers-modules" +version = "0.11.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d43ed4e8f58424c3a2c6c56dbea6643c3c23e8666a34df13c54f0a184e6c707" +dependencies = [ + "testcontainers", +] + +[[package]] +name = "textwrap" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d326610f408c7a4eb6f51c37c330e496b08506c9457c9d34287ecc38809fb060" +dependencies = [ + "unicode-width 0.1.14", +] + [[package]] name = "thiserror" version = "1.0.69" @@ -3945,7 +5965,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -3956,7 +5976,7 @@ checksum = "26afc1baea8a989337eeb52b6e72a039780ce45c3edfcc9c5b9d112feeb173c2" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -3977,6 +5997,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "35e7868883861bd0e56d9ac6efcaaca0d6d5d82a2a7ec8209ff492c07cf37b21" dependencies = [ "deranged", + "itoa", "num-conv", "powerfmt", "serde", @@ -4019,6 +6040,16 @@ dependencies = [ "zerovec", ] +[[package]] +name = "tinytemplate" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +dependencies = [ + "serde", + "serde_json", +] + [[package]] name = "tinyvec" version = "1.8.1" @@ -4060,7 +6091,33 @@ checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", +] + +[[package]] +name = "tokio-postgres" +version = "0.7.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6c95d533c83082bb6490e0189acaa0bbeef9084e60471b696ca6988cd0541fb0" +dependencies = [ + "async-trait", + "byteorder", + "bytes", + "fallible-iterator", + "futures-channel", + "futures-util", + "log", + "parking_lot", + "percent-encoding", + "phf", + "pin-project-lite", + "postgres-protocol", + "postgres-types", + "rand 0.9.0", + "socket2", + "tokio", + "tokio-util", + "whoami", ] [[package]] @@ -4074,13 +6131,39 @@ dependencies = [ ] [[package]] -name = "tokio-rustls" -version = "0.26.1" +name = "tokio-rustls" +version = "0.26.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" +dependencies = [ + "rustls 0.23.22", + "tokio", +] + +[[package]] +name = "tokio-stream" +version = "0.1.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eca58d7bba4a75707817a2c44174253f9236b2d5fbd055602e9d5c07c139a047" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "tokio-tar" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" +checksum = "9d5714c010ca3e5c27114c1cdeb9d14641ace49874aa5626d7149e47aedace75" dependencies = [ - "rustls 0.23.22", + "filetime", + "futures-core", + "libc", + "redox_syscall 0.3.5", "tokio", + "tokio-stream", + "xattr", ] [[package]] @@ -4108,11 +6191,61 @@ version = "0.22.23" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "02a8b472d1a3d7c18e2d61a489aee3453fd9031c33e4f55bd533f4a7adca1bee" dependencies = [ - "indexmap", + "indexmap 2.7.1", "toml_datetime", "winnow", ] +[[package]] +name = "tonic" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "877c5b330756d856ffcc4553ab34a5684481ade925ecc54bcd1bf02b1d0d4d52" +dependencies = [ + "async-stream", + "async-trait", + "axum", + "base64 0.22.1", + "bytes", + "h2 0.4.7", + "http 1.2.0", + "http-body 1.0.1", + "http-body-util", + "hyper 1.6.0", + "hyper-timeout", + "hyper-util", + "percent-encoding", + "pin-project", + "prost", + "socket2", + "tokio", + "tokio-stream", + "tower 0.4.13", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tower" +version = "0.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" +dependencies = [ + "futures-core", + "futures-util", + "indexmap 1.9.3", + "pin-project", + "pin-project-lite", + "rand 0.8.5", + "slab", + "tokio", + "tokio-util", + "tower-layer", + "tower-service", + "tracing", +] + [[package]] name = "tower" version = "0.5.2" @@ -4159,7 +6292,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -4177,6 +6310,21 @@ version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" +[[package]] +name = "tstr" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f8e0294f14baae476d0dd0a2d780b2e24d66e349a9de876f5126777a37bdba7" +dependencies = [ + "tstr_proc_macros", +] + +[[package]] +name = "tstr_proc_macros" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e78122066b0cb818b8afd08f7ed22f7fdbc3e90815035726f0840d0d26c0747a" + [[package]] name = "twox-hash" version = "1.6.3" @@ -4187,6 +6335,12 @@ dependencies = [ "static_assertions", ] +[[package]] +name = "typed-arena" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6af6ae20167a9ece4bcb41af5b80f8a1f1df981f6391189ce00fd257af04126a" + [[package]] name = "typed-builder" version = "0.19.1" @@ -4204,7 +6358,7 @@ checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] @@ -4213,24 +6367,110 @@ version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" +[[package]] +name = "typify" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e03ba3643450cfd95a1aca2e1938fef63c1c1994489337998aff4ad771f21ef8" +dependencies = [ + "typify-impl", + "typify-macro", +] + +[[package]] +name = "typify-impl" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bce48219a2f3154aaa2c56cbf027728b24a3c8fe0a47ed6399781de2b3f3eeaf" +dependencies = [ + "heck 0.5.0", + "log", + "proc-macro2", + "quote", + "regress", + "schemars", + "semver", + "serde", + "serde_json", + "syn 2.0.98", + "thiserror 2.0.11", + "unicode-ident", +] + +[[package]] +name = "typify-macro" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68b5780d745920ed73c5b7447496a9b5c42ed2681a9b70859377aec423ecf02b" +dependencies = [ + "proc-macro2", + "quote", + "schemars", + "semver", + "serde", + "serde_json", + "serde_tokenstream", + "syn 2.0.98", + "typify-impl", +] + +[[package]] +name = "unicode-bidi" +version = "0.3.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" + [[package]] name = "unicode-ident" version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a210d160f08b701c8721ba1c726c11662f877ea6b7094007e1ca9a1041945034" +[[package]] +name = "unicode-normalization" +version = "0.1.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +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.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" +[[package]] +name = "unicode-width" +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 = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" +[[package]] +name = "unindent" +version = "0.2.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7de7d73e1754487cb58364ee906a499937a0dfabd86bcb980fa99ec8c8fa2ce" + +[[package]] +name = "unsafe-libyaml" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" + [[package]] name = "untrusted" version = "0.9.0" @@ -4246,6 +6486,7 @@ dependencies = [ "form_urlencoded", "idna", "percent-encoding", + "serde", ] [[package]] @@ -4339,6 +6580,12 @@ dependencies = [ "wit-bindgen-rt", ] +[[package]] +name = "wasite" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" + [[package]] name = "wasm-bindgen" version = "0.2.100" @@ -4361,7 +6608,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn", + "syn 2.0.98", "wasm-bindgen-shared", ] @@ -4396,7 +6643,7 @@ checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4410,6 +6657,30 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "wasm-bindgen-test" +version = "0.3.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66c8d5e33ca3b6d9fa3b4676d774c5778031d27a578c2b007f905acf816152c3" +dependencies = [ + "js-sys", + "minicov", + "wasm-bindgen", + "wasm-bindgen-futures", + "wasm-bindgen-test-macro", +] + +[[package]] +name = "wasm-bindgen-test-macro" +version = "0.3.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "17d5042cc5fa009658f9a7333ef24291b1291a25b6382dd68862a7f3b969f69b" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + [[package]] name = "wasm-streams" version = "0.4.2" @@ -4443,6 +6714,33 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "whoami" +version = "1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "372d5b87f58ec45c384ba03563b03544dc5fadc3983e434b286913f5b4a9bb6d" +dependencies = [ + "redox_syscall 0.5.8", + "wasite", + "web-sys", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + [[package]] name = "winapi-util" version = "0.1.9" @@ -4452,13 +6750,63 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows" +version = "0.57.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "12342cb4d8e3b046f3d80effd474a7a02447231330ef77d71daa6fbc40681143" +dependencies = [ + "windows-core 0.57.0", + "windows-targets 0.52.6", +] + [[package]] name = "windows-core" version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-targets", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-core" +version = "0.57.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2ed2439a290666cd67ecce2b0ffaad89c2a56b976b736e6ece670297897832d" +dependencies = [ + "windows-implement", + "windows-interface", + "windows-result 0.1.2", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-implement" +version = "0.57.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9107ddc059d5b6fbfbffdfa7a7fe3e22a226def0b2608f72e9d552763d3e1ad7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", +] + +[[package]] +name = "windows-interface" +version = "0.57.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29bee4b38ea3cde66011baa44dba677c432a78593e202392d1e9070cf2a7fca7" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", ] [[package]] @@ -4467,9 +6815,18 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e400001bb720a623c1c69032f8e3e4cf09984deec740f007dd2b03ec864804b0" dependencies = [ - "windows-result", + "windows-result 0.2.0", "windows-strings", - "windows-targets", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-result" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5e383302e8ec8515204254685643de10811af0ed97ea37210dc26fb0032647f8" +dependencies = [ + "windows-targets 0.52.6", ] [[package]] @@ -4478,7 +6835,7 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" dependencies = [ - "windows-targets", + "windows-targets 0.52.6", ] [[package]] @@ -4487,8 +6844,17 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" dependencies = [ - "windows-result", - "windows-targets", + "windows-result 0.2.0", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.48.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "677d2418bec65e3338edb076e806bc1ec15693c5d0104683f2efe857f61056a9" +dependencies = [ + "windows-targets 0.48.5", ] [[package]] @@ -4497,7 +6863,7 @@ version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" dependencies = [ - "windows-targets", + "windows-targets 0.52.6", ] [[package]] @@ -4506,7 +6872,22 @@ version = "0.59.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" dependencies = [ - "windows-targets", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-targets" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a2fa6e2155d7247be68c096456083145c183cbbbc2764150dda45a87197940c" +dependencies = [ + "windows_aarch64_gnullvm 0.48.5", + "windows_aarch64_msvc 0.48.5", + "windows_i686_gnu 0.48.5", + "windows_i686_msvc 0.48.5", + "windows_x86_64_gnu 0.48.5", + "windows_x86_64_gnullvm 0.48.5", + "windows_x86_64_msvc 0.48.5", ] [[package]] @@ -4515,28 +6896,46 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" dependencies = [ - "windows_aarch64_gnullvm", - "windows_aarch64_msvc", - "windows_i686_gnu", + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", "windows_i686_gnullvm", - "windows_i686_msvc", - "windows_x86_64_gnu", - "windows_x86_64_gnullvm", - "windows_x86_64_msvc", + "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]] +name = "windows_aarch64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b38e32f0abccf9987a4e3079dfb67dcd799fb61361e53e2882c3cbaf0d905d8" + [[package]] name = "windows_aarch64_gnullvm" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" +[[package]] +name = "windows_aarch64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dc35310971f3b2dbbf3f0690a219f40e2d9afcf64f9ab7cc1be722937c26b4bc" + [[package]] name = "windows_aarch64_msvc" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" +[[package]] +name = "windows_i686_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a75915e7def60c94dcef72200b9a8e58e5091744960da64ec734a6c6e9b3743e" + [[package]] name = "windows_i686_gnu" version = "0.52.6" @@ -4549,24 +6948,48 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" +[[package]] +name = "windows_i686_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f55c233f70c4b27f66c523580f78f1004e8b5a8b659e05a4eb49d4166cca406" + [[package]] name = "windows_i686_msvc" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" +[[package]] +name = "windows_x86_64_gnu" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "53d40abd2583d23e4718fddf1ebec84dbff8381c07cae67ff7768bbf19c6718e" + [[package]] name = "windows_x86_64_gnu" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b7b52767868a23d5bab768e390dc5f5c55825b6d30b86c844ff2dc7414044cc" + [[package]] name = "windows_x86_64_gnullvm" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" +[[package]] +name = "windows_x86_64_msvc" +version = "0.48.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed94fce61571a4006852b7389a063ab983c02eb1bb37b47f8272ce92d06d9538" + [[package]] name = "windows_x86_64_msvc" version = "0.52.6" @@ -4603,6 +7026,26 @@ version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" +[[package]] +name = "wyz" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f360fc0b24296329c78fda852a1e9ae82de9cf7b27dae4b7f62f118f77b9ed" +dependencies = [ + "tap", +] + +[[package]] +name = "xattr" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e105d177a3871454f754b33bb0ee637ecaaac997446375fd3e5d43a2ed00c909" +dependencies = [ + "libc", + "linux-raw-sys", + "rustix", +] + [[package]] name = "xmlparser" version = "0.13.6" @@ -4638,7 +7081,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", "synstructure", ] @@ -4649,7 +7092,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" dependencies = [ "byteorder", - "zerocopy-derive", + "zerocopy-derive 0.7.35", +] + +[[package]] +name = "zerocopy" +version = "0.8.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa91407dacce3a68c56de03abe2760159582b846c6a4acd2f456618087f12713" +dependencies = [ + "zerocopy-derive 0.8.17", ] [[package]] @@ -4660,7 +7112,18 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", +] + +[[package]] +name = "zerocopy-derive" +version = "0.8.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06718a168365cad3d5ff0bb133aad346959a2074bd4a85c121255a11304a8626" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.98", ] [[package]] @@ -4680,7 +7143,7 @@ checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", "synstructure", ] @@ -4709,7 +7172,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn", + "syn 2.0.98", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index cbde1b46591c..56bc218f2706 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,8 +16,7 @@ # under the License. [workspace] -# datafusion-cli is excluded because of its Cargo.lock. See datafusion-cli/README.md. -exclude = ["datafusion-cli", "dev/depcheck"] +exclude = ["dev/depcheck"] members = [ "datafusion/common", "datafusion/common-runtime", @@ -48,6 +47,7 @@ members = [ "datafusion/sqllogictest", "datafusion/substrait", "datafusion/wasmtest", + "datafusion-cli", "datafusion-examples", "datafusion-examples/examples/ffi/ffi_example_table_provider", "datafusion-examples/examples/ffi/ffi_module_interface", diff --git a/README.md b/README.md index 0377306abb68..2c2febab09cc 100644 --- a/README.md +++ b/README.md @@ -152,26 +152,13 @@ deprecate methods before removing them, according to the [deprecation guidelines [deprecation guidelines]: https://datafusion.apache.org/library-user-guide/api-health.html -## Dependencies and a `Cargo.lock` +## Dependencies and `Cargo.lock` -`datafusion` is intended for use as a library and thus purposely does not have a -`Cargo.lock` file checked in. You can read more about the distinction in the -[Cargo book]. +Following the [guidance] on committing `Cargo.lock` files, this project commits +its `Cargo.lock` file. -CI tests always run against the latest compatible versions of all dependencies -(the equivalent of doing `cargo update`), as suggested in the [Cargo CI guide] -and we rely on Dependabot for other upgrades. This strategy has two problems -that occasionally arise: +CI uses the committed `Cargo.lock` file, and dependencies are updated regularly +using [Dependabot] PRs. -1. CI failures when downstream libraries upgrade in some non compatible way -2. Local development builds that fail when DataFusion inadvertently relies on - a feature in a newer version of a dependency than declared in `Cargo.toml` - (e.g. a new method is added to a trait that we use). - -However, we think the current strategy is the best tradeoff between maintenance -overhead and user experience and ensures DataFusion always works with the latest -compatible versions of all dependencies. If you encounter either of these -problems, please open an issue or PR. - -[cargo book]: https://doc.rust-lang.org/cargo/guide/cargo-toml-vs-cargo-lock.html -[cargo ci guide]: https://doc.rust-lang.org/cargo/guide/continuous-integration.html#verifying-latest-dependencies +[guidance]: https://blog.rust-lang.org/2023/08/29/committing-lockfiles.html +[dependabot]: https://docs.github.com/en/code-security/dependabot/working-with-dependabot diff --git a/ci/scripts/rust_clippy.sh b/ci/scripts/rust_clippy.sh index 01eb6e710a2d..8118ecc57700 100755 --- a/ci/scripts/rust_clippy.sh +++ b/ci/scripts/rust_clippy.sh @@ -19,5 +19,3 @@ set -ex cargo clippy --all-targets --workspace --features avro,pyarrow,integration-tests -- -D warnings -cd datafusion-cli -cargo clippy --all-targets --all-features -- -D warnings diff --git a/ci/scripts/rust_docs.sh b/ci/scripts/rust_docs.sh index 5c93711b6fb6..e90bfdf8bc27 100755 --- a/ci/scripts/rust_docs.sh +++ b/ci/scripts/rust_docs.sh @@ -20,5 +20,3 @@ set -ex export RUSTDOCFLAGS="-D warnings" cargo doc --document-private-items --no-deps --workspace -cd datafusion-cli -cargo doc --document-private-items --no-deps diff --git a/ci/scripts/rust_fmt.sh b/ci/scripts/rust_fmt.sh index cb9bb5e877e7..9d8325877aad 100755 --- a/ci/scripts/rust_fmt.sh +++ b/ci/scripts/rust_fmt.sh @@ -19,5 +19,3 @@ set -ex cargo fmt --all -- --check -cd datafusion-cli -cargo fmt --all -- --check diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 3dc2f0098f82..987ac97452a9 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -18,23 +18,22 @@ [package] name = "datafusion-cli" description = "Command Line Client for DataFusion query engine." -version = "45.0.0" -authors = ["Apache DataFusion "] -edition = "2021" -keywords = ["arrow", "datafusion", "query", "sql"] -license = "Apache-2.0" -homepage = "https://datafusion.apache.org" -repository = "https://github.com/apache/datafusion" -rust-version = "1.81.0" readme = "README.md" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } [dependencies] -arrow = { version = "54.1.0" } -async-trait = "0.1.0" +arrow = { workspace = true } +async-trait = { workspace = true } aws-config = "1.5.16" aws-credential-types = "1.2.0" clap = { version = "4.5.28", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "45.0.0", features = [ +datafusion = { workspace = true, features = [ "avro", "crypto_expressions", "datetime_expressions", @@ -46,31 +45,19 @@ datafusion = { path = "../datafusion/core", version = "45.0.0", features = [ "compression", ] } dirs = "6.0.0" -env_logger = "0.11" -futures = "0.3" -# pin as home 0.5.11 has MSRV 1.81. Can remove this once we bump MSRV to 1.81 +env_logger = { workspace = true } +futures = { workspace = true } mimalloc = { version = "0.1", default-features = false } -object_store = { version = "0.11.0", features = ["aws", "gcp", "http"] } -parking_lot = { version = "0.12" } -parquet = { version = "54.1.0", default-features = false } -regex = "1.8" +object_store = { workspace = true, features = ["aws", "gcp", "http"] } +parking_lot = { workspace = true } +parquet = { workspace = true, default-features = false } +regex = { workspace = true } rustyline = "15.0" -tokio = { version = "1.43", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot", "signal"] } -url = "2.5.4" +tokio = { workspace = true, features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot", "signal"] } +url = { workspace = true } [dev-dependencies] assert_cmd = "2.0" -ctor = "0.2.9" +ctor = { workspace = true } predicates = "3.0" -rstest = "0.24" - -[profile.ci] -inherits = "dev" -incremental = false - -# ci turns off debug info, etc for dependencies to allow for smaller binaries making caching more effective -[profile.ci.package."*"] -debug = false -debug-assertions = false -strip = "debuginfo" -incremental = false +rstest = { workspace = true } diff --git a/datafusion-cli/Dockerfile b/datafusion-cli/Dockerfile index f73b76b5dbdf..4da9390a2d7a 100644 --- a/datafusion-cli/Dockerfile +++ b/datafusion-cli/Dockerfile @@ -18,18 +18,14 @@ FROM rust:bookworm AS builder COPY . /usr/src/datafusion -COPY ./datafusion /usr/src/datafusion/datafusion -COPY ./datafusion-cli /usr/src/datafusion/datafusion-cli -WORKDIR /usr/src/datafusion/datafusion-cli +WORKDIR /usr/src/datafusion -RUN rustup component add rustfmt - -RUN cargo build --release +RUN cargo build -p datafusion-cli --release FROM debian:bookworm-slim -COPY --from=builder /usr/src/datafusion/datafusion-cli/target/release/datafusion-cli /usr/local/bin +COPY --from=builder /usr/src/datafusion/target/release/datafusion-cli /usr/local/bin RUN mkdir /data diff --git a/datafusion-cli/README.md b/datafusion-cli/README.md index ce09c3b345b9..ca796b525fa1 100644 --- a/datafusion-cli/README.md +++ b/datafusion-cli/README.md @@ -30,19 +30,3 @@ DataFusion CLI (`datafusion-cli`) is a small command line utility that runs SQL ## Where can I find more information? See the [`datafusion-cli` documentation](https://datafusion.apache.org/user-guide/cli/index.html) for further information. - -## How do I make my IDE work with `datafusion-cli`? - -"open" the `datafusion/datafusion-cli` project as its own top level -project in my IDE (rather than opening `datafusion`) - -The reason `datafusion-cli` is not part of the main workspace in -[`datafusion Cargo.toml`] file is that `datafusion-cli` is a binary and has a -checked in `Cargo.lock` file to ensure reproducible builds. - -However, the `datafusion` and sub crates are intended for use as libraries and -thus do not have a `Cargo.lock` file checked in, as described in the [main -README] file. - -[`datafusion cargo.toml`]: https://github.com/apache/datafusion/blob/main/Cargo.toml -[main readme]: ../README.md diff --git a/datafusion-cli/src/object_storage.rs b/datafusion-cli/src/object_storage.rs index 045c924e5037..c31310093ac6 100644 --- a/datafusion-cli/src/object_storage.rs +++ b/datafusion-cli/src/object_storage.rs @@ -549,7 +549,7 @@ mod tests { .await .unwrap_err(); - assert_eq!(err.to_string(), "Invalid or Unsupported Configuration: Invalid endpoint: http://endpoint33. HTTP is not allowed for S3 endpoints. To allow HTTP, set 'aws.allow_http' to true"); + assert_eq!(err.to_string().lines().next().unwrap_or_default(), "Invalid or Unsupported Configuration: Invalid endpoint: http://endpoint33. HTTP is not allowed for S3 endpoints. To allow HTTP, set 'aws.allow_http' to true"); } else { return plan_err!("LogicalPlan is not a CreateExternalTable"); } diff --git a/dev/depcheck/.gitignore b/dev/depcheck/.gitignore new file mode 100644 index 000000000000..03314f77b5aa --- /dev/null +++ b/dev/depcheck/.gitignore @@ -0,0 +1 @@ +Cargo.lock diff --git a/dev/release/README.md b/dev/release/README.md index 1672129c0ac1..e74d84096792 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -141,15 +141,6 @@ git checkout apache/main Manually update the datafusion version in the root `Cargo.toml` to `38.0.0`. -Run `cargo update` in the root directory and also in `datafusion-cli`: - -```shell -cargo update -cd datafustion-cli -cargo update -cd .. -``` - Run `cargo test` to re-generate some example files: ```shell From 3550758d43c02a8675faa19acd1299bf45404aa3 Mon Sep 17 00:00:00 2001 From: irenjj Date: Sun, 9 Feb 2025 02:08:52 +0800 Subject: [PATCH 12/12] chore: remove partition_keys from (Bounded)WindowAggExec (#14526) * chore: remove partition_keys from (Bounded)WindowAggExec * support bounded_window_agg_exec * fix * fix * fix --- datafusion/core/src/physical_planner.rs | 31 ++++------------ .../core/tests/fuzz_cases/window_fuzz.rs | 6 ++-- .../tests/physical_optimizer/test_utils.rs | 4 +-- .../src/enforce_distribution.rs | 4 +-- .../src/enforce_sorting/mod.rs | 8 ++--- .../src/windows/bounded_window_agg_exec.rs | 35 ++++++++++++++----- datafusion/physical-plan/src/windows/mod.rs | 6 ++-- .../src/windows/window_agg_exec.rs | 31 ++++++++++++---- datafusion/proto/src/physical_plan/mod.rs | 8 ++--- .../tests/cases/roundtrip_physical_plan.rs | 10 +++--- 10 files changed, 79 insertions(+), 64 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index edac0fb16381..9fcb9562a485 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -86,7 +86,6 @@ use datafusion_physical_plan::execution_plan::InvariantLevel; use datafusion_physical_plan::memory::MemorySourceConfig; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::unnest::ListUnnest; -use datafusion_sql::utils::window_expr_common_partition_keys; use crate::schema_equivalence::schema_satisfied_by; use async_trait::async_trait; @@ -557,34 +556,13 @@ impl DefaultPhysicalPlanner { return exec_err!("Table '{table_name}' does not exist"); } } - LogicalPlan::Window(Window { - input, window_expr, .. - }) => { + LogicalPlan::Window(Window { window_expr, .. }) => { if window_expr.is_empty() { return internal_err!("Impossibly got empty window expression"); } let input_exec = children.one()?; - // at this moment we are guaranteed by the logical planner - // to have all the window_expr to have equal sort key - let partition_keys = window_expr_common_partition_keys(window_expr)?; - - let can_repartition = !partition_keys.is_empty() - && session_state.config().target_partitions() > 1 - && session_state.config().repartition_window_functions(); - - let physical_partition_keys = if can_repartition { - partition_keys - .iter() - .map(|e| { - self.create_physical_expr(e, input.schema(), session_state) - }) - .collect::>>>()? - } else { - vec![] - }; - let get_sort_keys = |expr: &Expr| match expr { Expr::WindowFunction(WindowFunction { ref partition_by, @@ -626,6 +604,9 @@ impl DefaultPhysicalPlanner { }) .collect::>>()?; + let can_repartition = session_state.config().target_partitions() > 1 + && session_state.config().repartition_window_functions(); + let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); // If all window expressions can run with bounded memory, @@ -634,14 +615,14 @@ impl DefaultPhysicalPlanner { Arc::new(BoundedWindowAggExec::try_new( window_expr, input_exec, - physical_partition_keys, InputOrderMode::Sorted, + can_repartition, )?) } else { Arc::new(WindowAggExec::try_new( window_expr, input_exec, - physical_partition_keys, + can_repartition, )?) } } diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 9c66bf2d78f2..4a484221a88a 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -293,8 +293,8 @@ async fn bounded_window_causal_non_causal() -> Result<()> { let running_window_exec = Arc::new(BoundedWindowAggExec::try_new( vec![window_expr], memory_exec.clone(), - vec![], Linear, + false, )?); let task_ctx = ctx.task_ctx(); let collected_results = collect(running_window_exec, task_ctx).await?; @@ -660,7 +660,7 @@ async fn run_window_test( false, )?], exec1, - vec![], + false, )?) as _; let exec2 = Arc::new(DataSourceExec::new(Arc::new( MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None)? @@ -678,8 +678,8 @@ async fn run_window_test( false, )?], exec2, - vec![], search_mode.clone(), + false, )?) as _; let task_ctx = ctx.task_ctx(); let collected_usual = collect(usual_window_exec, task_ctx.clone()).await?; diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index fb36be3da1b4..721dfca029b0 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -237,8 +237,8 @@ pub fn bounded_window_exec_with_partition( BoundedWindowAggExec::try_new( vec![window_expr], Arc::clone(&input), - vec![], InputOrderMode::Sorted, + false, ) .unwrap(), ) @@ -266,8 +266,8 @@ pub fn bounded_window_exec_non_set_monotonic( ) .unwrap()], Arc::clone(&input), - vec![], InputOrderMode::Sorted, + false, ) .unwrap(), ) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index d0f7b3671e60..5e76edad1f56 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1196,7 +1196,7 @@ pub fn ensure_distribution( if let Some(updated_window) = get_best_fitting_window( exec.window_expr(), exec.input(), - &exec.partition_keys, + &exec.partition_keys(), )? { plan = updated_window; } @@ -1204,7 +1204,7 @@ pub fn ensure_distribution( if let Some(updated_window) = get_best_fitting_window( exec.window_expr(), exec.input(), - &exec.partition_keys, + &exec.partition_keys(), )? { plan = updated_window; } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 2d23894d6b5e..a25e6c6f17ac 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -460,12 +460,12 @@ fn adjust_window_sort_removal( if let Some(exec) = plan.downcast_ref::() { let window_expr = exec.window_expr(); let new_window = - get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?; + get_best_fitting_window(window_expr, child_plan, &exec.partition_keys())?; (window_expr, new_window) } else if let Some(exec) = plan.downcast_ref::() { let window_expr = exec.window_expr(); let new_window = - get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?; + get_best_fitting_window(window_expr, child_plan, &exec.partition_keys())?; (window_expr, new_window) } else { return plan_err!("Expected WindowAggExec or BoundedWindowAggExec"); @@ -493,14 +493,14 @@ fn adjust_window_sort_removal( Arc::new(BoundedWindowAggExec::try_new( window_expr.to_vec(), child_plan, - window_expr[0].partition_by().to_vec(), InputOrderMode::Sorted, + !window_expr[0].partition_by().is_empty(), )?) as _ } else { Arc::new(WindowAggExec::try_new( window_expr.to_vec(), child_plan, - window_expr[0].partition_by().to_vec(), + !window_expr[0].partition_by().is_empty(), )?) as _ } }; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 79eaf4447434..a734feae5533 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -78,8 +78,6 @@ pub struct BoundedWindowAggExec { window_expr: Vec>, /// Schema after the window is run schema: SchemaRef, - /// Partition Keys - pub partition_keys: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Describes how the input is ordered relative to the partition keys @@ -93,6 +91,8 @@ pub struct BoundedWindowAggExec { ordered_partition_by_indices: Vec, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// If `can_rerepartition` is false, partition_keys is always empty. + can_repartition: bool, } impl BoundedWindowAggExec { @@ -100,8 +100,8 @@ impl BoundedWindowAggExec { pub fn try_new( window_expr: Vec>, input: Arc, - partition_keys: Vec>, input_order_mode: InputOrderMode, + can_repartition: bool, ) -> Result { let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); @@ -128,11 +128,11 @@ impl BoundedWindowAggExec { input, window_expr, schema, - partition_keys, metrics: ExecutionPlanMetricsSet::new(), input_order_mode, ordered_partition_by_indices, cache, + can_repartition, }) } @@ -209,6 +209,23 @@ impl BoundedWindowAggExec { input.boundedness(), ) } + + pub fn partition_keys(&self) -> Vec> { + if !self.can_repartition { + vec![] + } else { + let all_partition_keys = self + .window_expr() + .iter() + .map(|expr| expr.partition_by().to_vec()) + .collect::>(); + + all_partition_keys + .into_iter() + .min_by_key(|s| s.len()) + .unwrap_or_else(Vec::new) + } + } } impl DisplayAs for BoundedWindowAggExec { @@ -269,11 +286,11 @@ impl ExecutionPlan for BoundedWindowAggExec { } fn required_input_distribution(&self) -> Vec { - if self.partition_keys.is_empty() { + if self.partition_keys().is_empty() { debug!("No partition defined for BoundedWindowAggExec!!!"); vec![Distribution::SinglePartition] } else { - vec![Distribution::HashPartitioned(self.partition_keys.clone())] + vec![Distribution::HashPartitioned(self.partition_keys().clone())] } } @@ -288,8 +305,8 @@ impl ExecutionPlan for BoundedWindowAggExec { Ok(Arc::new(BoundedWindowAggExec::try_new( self.window_expr.clone(), Arc::clone(&children[0]), - self.partition_keys.clone(), self.input_order_mode.clone(), + self.can_repartition, )?)) } @@ -1329,8 +1346,8 @@ mod tests { false, )?], input, - partitionby_exprs, input_order_mode, + true, )?)) } @@ -1610,8 +1627,8 @@ mod tests { let physical_plan = BoundedWindowAggExec::try_new( window_exprs, memory_exec, - vec![], InputOrderMode::Sorted, + true, ) .map(|e| Arc::new(e) as Arc)?; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 3785230c0e79..cdab1fa5929d 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -425,8 +425,8 @@ pub fn get_best_fitting_window( Ok(Some(Arc::new(BoundedWindowAggExec::try_new( window_expr, Arc::clone(input), - physical_partition_keys.to_vec(), input_order_mode, + !physical_partition_keys.is_empty(), )?) as _)) } else if input_order_mode != InputOrderMode::Sorted { // For `WindowAggExec` to work correctly PARTITION BY columns should be sorted. @@ -438,7 +438,7 @@ pub fn get_best_fitting_window( Ok(Some(Arc::new(WindowAggExec::try_new( window_expr, Arc::clone(input), - physical_partition_keys.to_vec(), + !physical_partition_keys.is_empty(), )?) as _)) } } @@ -663,7 +663,7 @@ mod tests { false, )?], blocking_exec, - vec![], + false, )?); let fut = collect(window_agg_exec, task_ctx); diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index f0c258a02576..d31fd66ca1f1 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -57,8 +57,6 @@ pub struct WindowAggExec { window_expr: Vec>, /// Schema after the window is run schema: SchemaRef, - /// Partition Keys - pub partition_keys: Vec>, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Partition by indices that defines preset for existing ordering @@ -66,6 +64,8 @@ pub struct WindowAggExec { ordered_partition_by_indices: Vec, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, + /// If `can_partition` is false, partition_keys is always empty. + can_repartition: bool, } impl WindowAggExec { @@ -73,7 +73,7 @@ impl WindowAggExec { pub fn try_new( window_expr: Vec>, input: Arc, - partition_keys: Vec>, + can_repartition: bool, ) -> Result { let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); @@ -85,10 +85,10 @@ impl WindowAggExec { input, window_expr, schema, - partition_keys, metrics: ExecutionPlanMetricsSet::new(), ordered_partition_by_indices, cache, + can_repartition, }) } @@ -139,6 +139,23 @@ impl WindowAggExec { input.boundedness(), ) } + + pub fn partition_keys(&self) -> Vec> { + if !self.can_repartition { + vec![] + } else { + let all_partition_keys = self + .window_expr() + .iter() + .map(|expr| expr.partition_by().to_vec()) + .collect::>(); + + all_partition_keys + .into_iter() + .min_by_key(|s| s.len()) + .unwrap_or_else(Vec::new) + } + } } impl DisplayAs for WindowAggExec { @@ -206,10 +223,10 @@ impl ExecutionPlan for WindowAggExec { } fn required_input_distribution(&self) -> Vec { - if self.partition_keys.is_empty() { + if self.partition_keys().is_empty() { vec![Distribution::SinglePartition] } else { - vec![Distribution::HashPartitioned(self.partition_keys.clone())] + vec![Distribution::HashPartitioned(self.partition_keys())] } } @@ -220,7 +237,7 @@ impl ExecutionPlan for WindowAggExec { Ok(Arc::new(WindowAggExec::try_new( self.window_expr.clone(), Arc::clone(&children[0]), - self.partition_keys.clone(), + true, )?)) } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 6fa28e882ed6..84b952965958 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -404,14 +404,14 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { Ok(Arc::new(BoundedWindowAggExec::try_new( physical_window_expr, input, - partition_keys, input_order_mode, + !partition_keys.is_empty(), )?)) } else { Ok(Arc::new(WindowAggExec::try_new( physical_window_expr, input, - partition_keys, + !partition_keys.is_empty(), )?)) } } @@ -1921,7 +1921,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .collect::>>()?; let partition_keys = exec - .partition_keys + .partition_keys() .iter() .map(|e| serialize_physical_expr(e, extension_codec)) .collect::>>()?; @@ -1951,7 +1951,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .collect::>>()?; let partition_keys = exec - .partition_keys + .partition_keys() .iter() .map(|e| serialize_physical_expr(e, extension_codec)) .collect::>>()?; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index fdd529cfd1b9..7418184fcac1 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -305,8 +305,8 @@ fn roundtrip_udwf() -> Result<()> { roundtrip_test(Arc::new(BoundedWindowAggExec::try_new( vec![udwf_expr], input, - vec![col("a", &schema)?], InputOrderMode::Sorted, + true, )?)) } @@ -382,7 +382,7 @@ fn roundtrip_window() -> Result<()> { roundtrip_test(Arc::new(WindowAggExec::try_new( vec![plain_aggr_window_expr, sliding_aggr_window_expr, udwf_expr], input, - vec![col("b", &schema)?], + false, )?)) } @@ -1108,7 +1108,7 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { Arc::new(WindowFrame::new(None)), ))], filter, - vec![col("author", &schema)?], + true, )?); let aggregate = Arc::new(AggregateExec::try_new( @@ -1163,8 +1163,8 @@ fn roundtrip_udwf_extension_codec() -> Result<()> { let window = Arc::new(BoundedWindowAggExec::try_new( vec![udwf_expr], input, - vec![col("b", &schema)?], InputOrderMode::Sorted, + true, )?); let ctx = SessionContext::new(); @@ -1216,7 +1216,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { Arc::new(WindowFrame::new(None)), ))], filter, - vec![col("author", &schema)?], + true, )?); let aggr_expr = AggregateExprBuilder::new(udaf, aggr_args.clone())