diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml index ed5cae627..99457b872 100644 --- a/.github/workflows/test.yaml +++ b/.github/workflows/test.yaml @@ -79,10 +79,6 @@ jobs: path: ~/.cargo key: cargo-cache-${{ steps.rust-toolchain.outputs.cachekey }}-${{ hashFiles('Cargo.lock') }} - - name: Check Formatting - if: ${{ matrix.python-version == '3.10' && matrix.toolchain == 'stable' }} - run: cargo fmt -- --check - - name: Run Clippy if: ${{ matrix.python-version == '3.10' && matrix.toolchain == 'stable' }} run: cargo clippy --all-targets --all-features -- -D clippy::all -D warnings -A clippy::redundant_closure @@ -125,3 +121,19 @@ jobs: cd examples/tpch uv run --no-project python convert_data_to_parquet.py uv run --no-project pytest _tests.py + + nightly-fmt: + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v5 + + - name: Setup Rust Toolchain + uses: dtolnay/rust-toolchain@stable + id: rust-toolchain + with: + toolchain: "nightly" + components: clippy,rustfmt + + - name: Check Formatting + run: cargo +nightly fmt -- --check diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index e8c451262..bcefa405d 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -33,7 +33,7 @@ repos: - id: rust-fmt name: Rust fmt description: Run cargo fmt on files included in the commit. rustfmt should be installed before-hand. - entry: cargo fmt --all -- + entry: cargo +nightly fmt --all -- pass_filenames: true types: [file, rust] language: system diff --git a/ci/scripts/rust_fmt.sh b/ci/scripts/rust_fmt.sh index 9d8325877..05cb6b208 100755 --- a/ci/scripts/rust_fmt.sh +++ b/ci/scripts/rust_fmt.sh @@ -18,4 +18,4 @@ # under the License. set -ex -cargo fmt --all -- --check +cargo +nightly fmt --all -- --check diff --git a/rustfmt.toml b/rustfmt.toml new file mode 100644 index 000000000..d04700d69 --- /dev/null +++ b/rustfmt.toml @@ -0,0 +1,19 @@ +# 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. + +group_imports = "StdExternalCrate" +imports_granularity = "Module" diff --git a/src/catalog.rs b/src/catalog.rs index 398c5881f..6f949f8ca 100644 --- a/src/catalog.rs +++ b/src/catalog.rs @@ -15,25 +15,26 @@ // specific language governing permissions and limitations // under the License. -use crate::dataset::Dataset; -use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; -use crate::table::PyTable; -use crate::utils::{validate_pycapsule, wait_for_future}; +use std::any::Any; +use std::collections::HashSet; +use std::sync::Arc; + use async_trait::async_trait; -use datafusion::catalog::{MemoryCatalogProvider, MemorySchemaProvider}; -use datafusion::common::DataFusionError; -use datafusion::{ - catalog::{CatalogProvider, SchemaProvider}, - datasource::TableProvider, +use datafusion::catalog::{ + CatalogProvider, MemoryCatalogProvider, MemorySchemaProvider, SchemaProvider, }; +use datafusion::common::DataFusionError; +use datafusion::datasource::TableProvider; use datafusion_ffi::schema_provider::{FFI_SchemaProvider, ForeignSchemaProvider}; use pyo3::exceptions::PyKeyError; use pyo3::prelude::*; use pyo3::types::PyCapsule; use pyo3::IntoPyObjectExt; -use std::any::Any; -use std::collections::HashSet; -use std::sync::Arc; + +use crate::dataset::Dataset; +use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; +use crate::table::PyTable; +use crate::utils::{validate_pycapsule, wait_for_future}; #[pyclass(frozen, name = "RawCatalog", module = "datafusion.catalog", subclass)] #[derive(Clone)] diff --git a/src/common/data_type.rs b/src/common/data_type.rs index 3cbe31332..69f686f78 100644 --- a/src/common/data_type.rs +++ b/src/common/data_type.rs @@ -19,8 +19,8 @@ use datafusion::arrow::array::Array; use datafusion::arrow::datatypes::{DataType, IntervalUnit, TimeUnit}; use datafusion::common::ScalarValue; use datafusion::logical_expr::sqlparser::ast::NullTreatment as DFNullTreatment; -use pyo3::exceptions::PyNotImplementedError; -use pyo3::{exceptions::PyValueError, prelude::*}; +use pyo3::exceptions::{PyNotImplementedError, PyValueError}; +use pyo3::prelude::*; #[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd)] pub struct PyScalarValue(pub ScalarValue); diff --git a/src/common/schema.rs b/src/common/schema.rs index 14ab630d3..4e46592aa 100644 --- a/src/common/schema.rs +++ b/src/common/schema.rs @@ -15,26 +15,25 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::borrow::Cow; use std::fmt::{self, Display, Formatter}; use std::sync::Arc; -use std::{any::Any, borrow::Cow}; use arrow::datatypes::Schema; use arrow::pyarrow::PyArrowType; use datafusion::arrow::datatypes::SchemaRef; use datafusion::common::Constraints; use datafusion::datasource::TableType; +use datafusion::logical_expr::utils::split_conjunction; use datafusion::logical_expr::{Expr, TableProviderFilterPushDown, TableSource}; +use parking_lot::RwLock; use pyo3::prelude::*; -use datafusion::logical_expr::utils::split_conjunction; - +use super::data_type::DataTypeMap; +use super::function::SqlFunction; use crate::sql::logical::PyLogicalPlan; -use super::{data_type::DataTypeMap, function::SqlFunction}; - -use parking_lot::RwLock; - #[pyclass(name = "SqlSchema", module = "datafusion.common", subclass, frozen)] #[derive(Debug, Clone)] pub struct SqlSchema { diff --git a/src/config.rs b/src/config.rs index 1726e5d9b..a25a4072e 100644 --- a/src/config.rs +++ b/src/config.rs @@ -17,14 +17,13 @@ use std::sync::Arc; +use datafusion::config::ConfigOptions; +use parking_lot::RwLock; use pyo3::prelude::*; use pyo3::types::*; -use datafusion::config::ConfigOptions; - use crate::errors::PyDataFusionResult; use crate::utils::py_obj_to_scalar_value; -use parking_lot::RwLock; #[pyclass(name = "Config", module = "datafusion", subclass, frozen)] #[derive(Clone)] pub(crate) struct PyConfig { diff --git a/src/context.rs b/src/context.rs index e8d875800..f64cc1683 100644 --- a/src/context.rs +++ b/src/context.rs @@ -23,43 +23,17 @@ use std::sync::Arc; use arrow::array::RecordBatchReader; use arrow::ffi_stream::ArrowArrayStreamReader; use arrow::pyarrow::FromPyArrow; -use datafusion::execution::session_state::SessionStateBuilder; -use object_store::ObjectStore; -use url::Url; -use uuid::Uuid; - -use pyo3::exceptions::{PyKeyError, PyValueError}; -use pyo3::prelude::*; - -use crate::catalog::{PyCatalog, RustWrappedPyCatalogProvider}; -use crate::dataframe::PyDataFrame; -use crate::dataset::Dataset; -use crate::errors::{py_datafusion_err, PyDataFusionResult}; -use crate::expr::sort_expr::PySortExpr; -use crate::physical_plan::PyExecutionPlan; -use crate::record_batch::PyRecordBatchStream; -use crate::sql::exceptions::py_value_err; -use crate::sql::logical::PyLogicalPlan; -use crate::store::StorageContexts; -use crate::table::PyTable; -use crate::udaf::PyAggregateUDF; -use crate::udf::PyScalarUDF; -use crate::udtf::PyTableFunction; -use crate::udwf::PyWindowUDF; -use crate::utils::{get_global_ctx, spawn_future, validate_pycapsule, wait_for_future}; use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::arrow::pyarrow::PyArrowType; use datafusion::arrow::record_batch::RecordBatch; use datafusion::catalog::CatalogProvider; -use datafusion::common::TableReference; -use datafusion::common::{exec_err, ScalarValue}; +use datafusion::common::{exec_err, ScalarValue, TableReference}; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, }; -use datafusion::datasource::MemTable; -use datafusion::datasource::TableProvider; +use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::context::{ DataFilePaths, SQLOptions, SessionConfig, SessionContext, TaskContext, }; @@ -67,12 +41,35 @@ use datafusion::execution::disk_manager::DiskManagerMode; use datafusion::execution::memory_pool::{FairSpillPool, GreedyMemoryPool, UnboundedMemoryPool}; use datafusion::execution::options::ReadOptions; use datafusion::execution::runtime_env::RuntimeEnvBuilder; +use datafusion::execution::session_state::SessionStateBuilder; use datafusion::prelude::{ AvroReadOptions, CsvReadOptions, DataFrame, NdJsonReadOptions, ParquetReadOptions, }; use datafusion_ffi::catalog_provider::{FFI_CatalogProvider, ForeignCatalogProvider}; +use object_store::ObjectStore; +use pyo3::exceptions::{PyKeyError, PyValueError}; +use pyo3::prelude::*; use pyo3::types::{PyCapsule, PyDict, PyList, PyTuple, PyType}; use pyo3::IntoPyObjectExt; +use url::Url; +use uuid::Uuid; + +use crate::catalog::{PyCatalog, RustWrappedPyCatalogProvider}; +use crate::dataframe::PyDataFrame; +use crate::dataset::Dataset; +use crate::errors::{py_datafusion_err, PyDataFusionResult}; +use crate::expr::sort_expr::PySortExpr; +use crate::physical_plan::PyExecutionPlan; +use crate::record_batch::PyRecordBatchStream; +use crate::sql::exceptions::py_value_err; +use crate::sql::logical::PyLogicalPlan; +use crate::store::StorageContexts; +use crate::table::PyTable; +use crate::udaf::PyAggregateUDF; +use crate::udf::PyScalarUDF; +use crate::udtf::PyTableFunction; +use crate::udwf::PyWindowUDF; +use crate::utils::{get_global_ctx, spawn_future, validate_pycapsule, wait_for_future}; /// Configuration options for a SessionContext #[pyclass(frozen, name = "SessionConfig", module = "datafusion", subclass)] diff --git a/src/dataframe.rs b/src/dataframe.rs index 187bb0acf..98c7584ee 100644 --- a/src/dataframe.rs +++ b/src/dataframe.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use cstr::cstr; use std::collections::HashMap; use std::ffi::{CStr, CString}; use std::sync::Arc; @@ -26,6 +25,7 @@ use arrow::error::ArrowError; use arrow::ffi::FFI_ArrowSchema; use arrow::ffi_stream::FFI_ArrowArrayStream; use arrow::pyarrow::FromPyArrow; +use cstr::cstr; use datafusion::arrow::datatypes::{Schema, SchemaRef}; use datafusion::arrow::pyarrow::{PyArrowType, ToPyArrow}; use datafusion::arrow::util::pretty; @@ -40,14 +40,16 @@ use datafusion::logical_expr::SortExpr; use datafusion::parquet::basic::{BrotliLevel, Compression, GzipLevel, ZstdLevel}; use datafusion::prelude::*; use futures::{StreamExt, TryStreamExt}; +use parking_lot::Mutex; use pyo3::exceptions::PyValueError; use pyo3::prelude::*; use pyo3::pybacked::PyBackedStr; use pyo3::types::{PyCapsule, PyList, PyTuple, PyTupleMethods}; use pyo3::PyErr; -use crate::errors::{py_datafusion_err, PyDataFusionError}; -use crate::expr::sort_expr::to_sort_expressions; +use crate::errors::{py_datafusion_err, PyDataFusionError, PyDataFusionResult}; +use crate::expr::sort_expr::{to_sort_expressions, PySortExpr}; +use crate::expr::PyExpr; use crate::physical_plan::PyExecutionPlan; use crate::record_batch::{poll_next_batch, PyRecordBatchStream}; use crate::sql::logical::PyLogicalPlan; @@ -55,12 +57,6 @@ use crate::table::{PyTable, TempViewTable}; use crate::utils::{ is_ipython_env, py_obj_to_scalar_value, spawn_future, validate_pycapsule, wait_for_future, }; -use crate::{ - errors::PyDataFusionResult, - expr::{sort_expr::PySortExpr, PyExpr}, -}; - -use parking_lot::Mutex; /// File-level static CStr for the Arrow array stream capsule name. static ARROW_ARRAY_STREAM_NAME: &CStr = cstr!("arrow_array_stream"); diff --git a/src/dataset.rs b/src/dataset.rs index 0baf4da2a..6abe8ae3b 100644 --- a/src/dataset.rs +++ b/src/dataset.rs @@ -15,25 +15,22 @@ // specific language governing permissions and limitations // under the License. -use datafusion::catalog::Session; -use pyo3::exceptions::PyValueError; -/// Implements a Datafusion TableProvider that delegates to a PyArrow Dataset -/// This allows us to use PyArrow Datasets as Datafusion tables while pushing down projections and filters -use pyo3::prelude::*; -use pyo3::types::PyType; - use std::any::Any; use std::sync::Arc; use async_trait::async_trait; - use datafusion::arrow::datatypes::SchemaRef; use datafusion::arrow::pyarrow::PyArrowType; +use datafusion::catalog::Session; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::{DataFusionError, Result as DFResult}; -use datafusion::logical_expr::Expr; -use datafusion::logical_expr::TableProviderFilterPushDown; +use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; use datafusion::physical_plan::ExecutionPlan; +use pyo3::exceptions::PyValueError; +/// Implements a Datafusion TableProvider that delegates to a PyArrow Dataset +/// This allows us to use PyArrow Datasets as Datafusion tables while pushing down projections and filters +use pyo3::prelude::*; +use pyo3::types::PyType; use crate::dataset_exec::DatasetExec; use crate::pyarrow_filter_expression::PyArrowFilterExpression; diff --git a/src/dataset_exec.rs b/src/dataset_exec.rs index aab8d7566..a35951394 100644 --- a/src/dataset_exec.rs +++ b/src/dataset_exec.rs @@ -15,20 +15,11 @@ // specific language governing permissions and limitations // under the License. -use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; -/// Implements a Datafusion physical ExecutionPlan that delegates to a PyArrow Dataset -/// This actually performs the projection, filtering and scanning of a Dataset -use pyo3::prelude::*; -use pyo3::types::{PyDict, PyIterator, PyList}; - use std::any::Any; use std::sync::Arc; -use futures::{stream, TryStreamExt}; - use datafusion::arrow::datatypes::SchemaRef; -use datafusion::arrow::error::ArrowError; -use datafusion::arrow::error::Result as ArrowResult; +use datafusion::arrow::error::{ArrowError, Result as ArrowResult}; use datafusion::arrow::pyarrow::PyArrowType; use datafusion::arrow::record_batch::RecordBatch; use datafusion::error::{DataFusionError as InnerDataFusionError, Result as DFResult}; @@ -36,11 +27,17 @@ use datafusion::execution::context::TaskContext; use datafusion::logical_expr::utils::conjunction; use datafusion::logical_expr::Expr; use datafusion::physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, Partitioning, SendableRecordBatchStream, Statistics, }; +use futures::{stream, TryStreamExt}; +/// Implements a Datafusion physical ExecutionPlan that delegates to a PyArrow Dataset +/// This actually performs the projection, filtering and scanning of a Dataset +use pyo3::prelude::*; +use pyo3::types::{PyDict, PyIterator, PyList}; use crate::errors::PyDataFusionResult; use crate::pyarrow_filter_expression::PyArrowFilterExpression; diff --git a/src/errors.rs b/src/errors.rs index d4f4f221d..fc079eb6c 100644 --- a/src/errors.rs +++ b/src/errors.rs @@ -22,7 +22,8 @@ use std::fmt::Debug; use datafusion::arrow::error::ArrowError; use datafusion::error::DataFusionError as InnerDataFusionError; use prost::EncodeError; -use pyo3::{exceptions::PyException, PyErr}; +use pyo3::exceptions::PyException; +use pyo3::PyErr; pub type PyDataFusionResult = std::result::Result; diff --git a/src/expr.rs b/src/expr.rs index c9eddaa2d..78bee3d4a 100644 --- a/src/expr.rs +++ b/src/expr.rs @@ -15,27 +15,34 @@ // specific language governing permissions and limitations // under the License. -use datafusion::logical_expr::expr::{AggregateFunctionParams, FieldMetadata}; -use datafusion::logical_expr::utils::exprlist_to_fields; -use datafusion::logical_expr::{ - lit_with_metadata, ExprFuncBuilder, ExprFunctionExt, LogicalPlan, WindowFunctionDefinition, -}; -use pyo3::IntoPyObjectExt; -use pyo3::{basic::CompareOp, prelude::*}; use std::collections::HashMap; use std::convert::{From, Into}; use std::sync::Arc; -use window::PyWindowFrame; use datafusion::arrow::datatypes::{DataType, Field}; use datafusion::arrow::pyarrow::PyArrowType; use datafusion::functions::core::expr_ext::FieldAccessor; +use datafusion::logical_expr::expr::{ + AggregateFunction, AggregateFunctionParams, FieldMetadata, InList, InSubquery, ScalarFunction, + WindowFunction, +}; +use datafusion::logical_expr::utils::exprlist_to_fields; use datafusion::logical_expr::{ - col, - expr::{AggregateFunction, InList, InSubquery, ScalarFunction, WindowFunction}, - lit, Between, BinaryExpr, Case, Cast, Expr, Like, Operator, TryCast, + col, lit, lit_with_metadata, Between, BinaryExpr, Case, Cast, Expr, ExprFuncBuilder, + ExprFunctionExt, Like, LogicalPlan, Operator, TryCast, WindowFunctionDefinition, }; +use pyo3::basic::CompareOp; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; +use window::PyWindowFrame; +use self::alias::PyAlias; +use self::bool_expr::{ + PyIsFalse, PyIsNotFalse, PyIsNotNull, PyIsNotTrue, PyIsNotUnknown, PyIsNull, PyIsTrue, + PyIsUnknown, PyNegative, PyNot, +}; +use self::like::{PyILike, PyLike, PySimilarTo}; +use self::scalar_variable::PyScalarVariable; use crate::common::data_type::{DataTypeMap, NullTreatment, PyScalarValue, RexType}; use crate::errors::{py_runtime_err, py_type_err, py_unsupported_variant_err, PyDataFusionResult}; use crate::expr::aggregate_expr::PyAggregateFunction; @@ -46,14 +53,6 @@ use crate::functions::add_builder_fns_to_window; use crate::pyarrow_util::scalar_to_pyarrow; use crate::sql::logical::PyLogicalPlan; -use self::alias::PyAlias; -use self::bool_expr::{ - PyIsFalse, PyIsNotFalse, PyIsNotNull, PyIsNotTrue, PyIsNotUnknown, PyIsNull, PyIsTrue, - PyIsUnknown, PyNegative, PyNot, -}; -use self::like::{PyILike, PyLike, PySimilarTo}; -use self::scalar_variable::PyScalarVariable; - pub mod aggregate; pub mod aggregate_expr; pub mod alias; diff --git a/src/expr/aggregate.rs b/src/expr/aggregate.rs index 4af7c755a..4cb41b26a 100644 --- a/src/expr/aggregate.rs +++ b/src/expr/aggregate.rs @@ -15,12 +15,14 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::common::DataFusionError; use datafusion::logical_expr::expr::{AggregateFunction, AggregateFunctionParams, Alias}; use datafusion::logical_expr::logical_plan::Aggregate; use datafusion::logical_expr::Expr; -use pyo3::{prelude::*, IntoPyObjectExt}; -use std::fmt::{self, Display, Formatter}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; use crate::common::df_schema::PyDFSchema; diff --git a/src/expr/aggregate_expr.rs b/src/expr/aggregate_expr.rs index 72ba0638f..d3b695a27 100644 --- a/src/expr/aggregate_expr.rs +++ b/src/expr/aggregate_expr.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::expr::PyExpr; +use std::fmt::{Display, Formatter}; + use datafusion::logical_expr::expr::AggregateFunction; use pyo3::prelude::*; -use std::fmt::{Display, Formatter}; + +use crate::expr::PyExpr; #[pyclass( frozen, diff --git a/src/expr/alias.rs b/src/expr/alias.rs index 588c00fdf..c6d486284 100644 --- a/src/expr/alias.rs +++ b/src/expr/alias.rs @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::expr::PyExpr; -use pyo3::prelude::*; use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::expr::Alias; +use pyo3::prelude::*; + +use crate::expr::PyExpr; #[pyclass(frozen, name = "Alias", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/analyze.rs b/src/expr/analyze.rs index c7caeebc8..05ec8dc22 100644 --- a/src/expr/analyze.rs +++ b/src/expr/analyze.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use datafusion::logical_expr::logical_plan::Analyze; -use pyo3::{prelude::*, IntoPyObjectExt}; use std::fmt::{self, Display, Formatter}; +use datafusion::logical_expr::logical_plan::Analyze; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + use super::logical_node::LogicalNode; use crate::common::df_schema::PyDFSchema; use crate::sql::logical::PyLogicalPlan; diff --git a/src/expr/between.rs b/src/expr/between.rs index 1f61599a3..4f0b34add 100644 --- a/src/expr/between.rs +++ b/src/expr/between.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::expr::PyExpr; +use std::fmt::{self, Display, Formatter}; + use datafusion::logical_expr::expr::Between; use pyo3::prelude::*; -use std::fmt::{self, Display, Formatter}; + +use crate::expr::PyExpr; #[pyclass(frozen, name = "Between", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/binary_expr.rs b/src/expr/binary_expr.rs index 94379583c..f67a08c7c 100644 --- a/src/expr/binary_expr.rs +++ b/src/expr/binary_expr.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::expr::PyExpr; use datafusion::logical_expr::BinaryExpr; use pyo3::prelude::*; +use crate::expr::PyExpr; + #[pyclass(frozen, name = "BinaryExpr", module = "datafusion.expr", subclass)] #[derive(Clone)] pub struct PyBinaryExpr { diff --git a/src/expr/bool_expr.rs b/src/expr/bool_expr.rs index 0d2b051e6..abd259409 100644 --- a/src/expr/bool_expr.rs +++ b/src/expr/bool_expr.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::logical_expr::Expr; use pyo3::prelude::*; -use std::fmt::{self, Display, Formatter}; use super::PyExpr; diff --git a/src/expr/case.rs b/src/expr/case.rs index 1a7369826..b49c19081 100644 --- a/src/expr/case.rs +++ b/src/expr/case.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::expr::PyExpr; use datafusion::logical_expr::Case; use pyo3::prelude::*; +use crate::expr::PyExpr; + #[pyclass(frozen, name = "Case", module = "datafusion.expr", subclass)] #[derive(Clone)] pub struct PyCase { diff --git a/src/expr/cast.rs b/src/expr/cast.rs index 03e2b8476..1aca9ea95 100644 --- a/src/expr/cast.rs +++ b/src/expr/cast.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::{common::data_type::PyDataType, expr::PyExpr}; use datafusion::logical_expr::{Cast, TryCast}; use pyo3::prelude::*; +use crate::common::data_type::PyDataType; +use crate::expr::PyExpr; + #[pyclass(frozen, name = "Cast", module = "datafusion.expr", subclass)] #[derive(Clone)] pub struct PyCast { diff --git a/src/expr/conditional_expr.rs b/src/expr/conditional_expr.rs index 21f538ba0..da6102dbf 100644 --- a/src/expr/conditional_expr.rs +++ b/src/expr/conditional_expr.rs @@ -15,11 +15,13 @@ // specific language governing permissions and limitations // under the License. -use crate::{errors::PyDataFusionResult, expr::PyExpr}; use datafusion::logical_expr::conditional_expressions::CaseBuilder; use datafusion::prelude::Expr; use pyo3::prelude::*; +use crate::errors::PyDataFusionResult; +use crate::expr::PyExpr; + // TODO(tsaucer) replace this all with CaseBuilder after it implements Clone #[derive(Clone, Debug)] #[pyclass(name = "CaseBuilder", module = "datafusion.expr", subclass, frozen)] diff --git a/src/expr/copy_to.rs b/src/expr/copy_to.rs index 422ab77f4..0b874e37d 100644 --- a/src/expr/copy_to.rs +++ b/src/expr/copy_to.rs @@ -15,18 +15,17 @@ // specific language governing permissions and limitations // under the License. -use std::{ - collections::HashMap, - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::collections::HashMap; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; -use datafusion::{common::file_options::file_type::FileType, logical_expr::dml::CopyTo}; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::sql::logical::PyLogicalPlan; +use datafusion::common::file_options::file_type::FileType; +use datafusion::logical_expr::dml::CopyTo; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "CopyTo", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/create_catalog.rs b/src/expr/create_catalog.rs index 361387894..400246a82 100644 --- a/src/expr/create_catalog.rs +++ b/src/expr/create_catalog.rs @@ -15,17 +15,16 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; use datafusion::logical_expr::CreateCatalog; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "CreateCatalog", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/create_catalog_schema.rs b/src/expr/create_catalog_schema.rs index cb3be2d30..641e2116d 100644 --- a/src/expr/create_catalog_schema.rs +++ b/src/expr/create_catalog_schema.rs @@ -15,17 +15,16 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; use datafusion::logical_expr::CreateCatalogSchema; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; +use crate::sql::logical::PyLogicalPlan; #[pyclass( frozen, diff --git a/src/expr/create_external_table.rs b/src/expr/create_external_table.rs index 920d0d613..1bd37b376 100644 --- a/src/expr/create_external_table.rs +++ b/src/expr/create_external_table.rs @@ -15,19 +15,20 @@ // specific language governing permissions and limitations // under the License. -use crate::{common::schema::PyConstraints, expr::PyExpr, sql::logical::PyLogicalPlan}; -use std::{ - collections::HashMap, - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::collections::HashMap; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; use datafusion::logical_expr::CreateExternalTable; -use pyo3::{prelude::*, IntoPyObjectExt}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; +use super::logical_node::LogicalNode; +use super::sort_expr::PySortExpr; use crate::common::df_schema::PyDFSchema; - -use super::{logical_node::LogicalNode, sort_expr::PySortExpr}; +use crate::common::schema::PyConstraints; +use crate::expr::PyExpr; +use crate::sql::logical::PyLogicalPlan; #[pyclass( frozen, diff --git a/src/expr/create_function.rs b/src/expr/create_function.rs index 1b663b466..2a35635c2 100644 --- a/src/expr/create_function.rs +++ b/src/expr/create_function.rs @@ -15,19 +15,19 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; use datafusion::logical_expr::{ CreateFunction, CreateFunctionBody, OperateFunctionArg, Volatility, }; -use pyo3::{prelude::*, IntoPyObjectExt}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; use super::PyExpr; -use crate::common::{data_type::PyDataType, df_schema::PyDFSchema}; +use crate::common::data_type::PyDataType; +use crate::common::df_schema::PyDFSchema; use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "CreateFunction", module = "datafusion.expr", subclass)] diff --git a/src/expr/create_index.rs b/src/expr/create_index.rs index 7b68df305..5c378332c 100644 --- a/src/expr/create_index.rs +++ b/src/expr/create_index.rs @@ -15,17 +15,17 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; use datafusion::logical_expr::CreateIndex; -use pyo3::{prelude::*, IntoPyObjectExt}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; - -use super::{logical_node::LogicalNode, sort_expr::PySortExpr}; +use super::logical_node::LogicalNode; +use super::sort_expr::PySortExpr; +use crate::common::df_schema::PyDFSchema; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "CreateIndex", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/create_memory_table.rs b/src/expr/create_memory_table.rs index 15aaa810b..7759eb420 100644 --- a/src/expr/create_memory_table.rs +++ b/src/expr/create_memory_table.rs @@ -18,11 +18,11 @@ use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::CreateMemoryTable; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::sql::logical::PyLogicalPlan; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::sql::logical::PyLogicalPlan; #[pyclass( frozen, diff --git a/src/expr/create_view.rs b/src/expr/create_view.rs index 49b3b6199..16faaf9d5 100644 --- a/src/expr/create_view.rs +++ b/src/expr/create_view.rs @@ -18,11 +18,12 @@ use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::{CreateView, DdlStatement, LogicalPlan}; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::{errors::py_type_err, sql::logical::PyLogicalPlan}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::errors::py_type_err; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "CreateView", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/describe_table.rs b/src/expr/describe_table.rs index 315026fef..9b139ed3b 100644 --- a/src/expr/describe_table.rs +++ b/src/expr/describe_table.rs @@ -15,18 +15,18 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; -use arrow::{datatypes::Schema, pyarrow::PyArrowType}; +use arrow::datatypes::Schema; +use arrow::pyarrow::PyArrowType; use datafusion::logical_expr::DescribeTable; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "DescribeTable", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/distinct.rs b/src/expr/distinct.rs index 5770b849d..1505ec3e6 100644 --- a/src/expr/distinct.rs +++ b/src/expr/distinct.rs @@ -18,11 +18,11 @@ use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::Distinct; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::sql::logical::PyLogicalPlan; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "Distinct", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/dml.rs b/src/expr/dml.rs index 4437a9de9..091dcbc18 100644 --- a/src/expr/dml.rs +++ b/src/expr/dml.rs @@ -17,12 +17,13 @@ use datafusion::logical_expr::dml::InsertOp; use datafusion::logical_expr::{DmlStatement, WriteOp}; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::common::schema::PyTableSource; -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; +use crate::common::schema::PyTableSource; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "DmlStatement", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/drop_catalog_schema.rs b/src/expr/drop_catalog_schema.rs index 7008bcd24..db6041a1b 100644 --- a/src/expr/drop_catalog_schema.rs +++ b/src/expr/drop_catalog_schema.rs @@ -15,17 +15,18 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; -use datafusion::{common::SchemaReference, logical_expr::DropCatalogSchema, sql::TableReference}; -use pyo3::{exceptions::PyValueError, prelude::*, IntoPyObjectExt}; - -use crate::common::df_schema::PyDFSchema; +use datafusion::common::SchemaReference; +use datafusion::logical_expr::DropCatalogSchema; +use datafusion::sql::TableReference; +use pyo3::exceptions::PyValueError; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; use crate::sql::logical::PyLogicalPlan; #[pyclass( diff --git a/src/expr/drop_function.rs b/src/expr/drop_function.rs index 42ad3e1fe..070d15783 100644 --- a/src/expr/drop_function.rs +++ b/src/expr/drop_function.rs @@ -15,13 +15,12 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; use datafusion::logical_expr::DropFunction; -use pyo3::{prelude::*, IntoPyObjectExt}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; use crate::common::df_schema::PyDFSchema; diff --git a/src/expr/drop_table.rs b/src/expr/drop_table.rs index 6ff4f01c4..ffb56e4ed 100644 --- a/src/expr/drop_table.rs +++ b/src/expr/drop_table.rs @@ -18,11 +18,11 @@ use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::logical_plan::DropTable; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::sql::logical::PyLogicalPlan; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "DropTable", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/drop_view.rs b/src/expr/drop_view.rs index b2aff4e9b..9d72f2077 100644 --- a/src/expr/drop_view.rs +++ b/src/expr/drop_view.rs @@ -15,17 +15,15 @@ // specific language governing permissions and limitations // under the License. -use std::{ - fmt::{self, Display, Formatter}, - sync::Arc, -}; +use std::fmt::{self, Display, Formatter}; +use std::sync::Arc; use datafusion::logical_expr::DropView; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::common::df_schema::PyDFSchema; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "DropView", module = "datafusion.expr", subclass)] diff --git a/src/expr/empty_relation.rs b/src/expr/empty_relation.rs index 797a8c02d..35c3fa79b 100644 --- a/src/expr/empty_relation.rs +++ b/src/expr/empty_relation.rs @@ -15,12 +15,15 @@ // specific language governing permissions and limitations // under the License. -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; -use datafusion::logical_expr::EmptyRelation; -use pyo3::{prelude::*, IntoPyObjectExt}; use std::fmt::{self, Display, Formatter}; +use datafusion::logical_expr::EmptyRelation; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "EmptyRelation", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/explain.rs b/src/expr/explain.rs index 71b7b2c13..c6884e98a 100644 --- a/src/expr/explain.rs +++ b/src/expr/explain.rs @@ -17,12 +17,15 @@ use std::fmt::{self, Display, Formatter}; -use datafusion::logical_expr::{logical_plan::Explain, LogicalPlan}; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::{common::df_schema::PyDFSchema, errors::py_type_err, sql::logical::PyLogicalPlan}; +use datafusion::logical_expr::logical_plan::Explain; +use datafusion::logical_expr::LogicalPlan; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; +use crate::errors::py_type_err; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "Explain", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/extension.rs b/src/expr/extension.rs index 7d913ff8c..b4c688bd0 100644 --- a/src/expr/extension.rs +++ b/src/expr/extension.rs @@ -16,11 +16,11 @@ // under the License. use datafusion::logical_expr::Extension; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::sql::logical::PyLogicalPlan; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "Extension", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/filter.rs b/src/expr/filter.rs index 76338d139..25a1e76b3 100644 --- a/src/expr/filter.rs +++ b/src/expr/filter.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use datafusion::logical_expr::logical_plan::Filter; -use pyo3::{prelude::*, IntoPyObjectExt}; use std::fmt::{self, Display, Formatter}; +use datafusion::logical_expr::logical_plan::Filter; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + use crate::common::df_schema::PyDFSchema; use crate::expr::logical_node::LogicalNode; use crate::expr::PyExpr; diff --git a/src/expr/in_list.rs b/src/expr/in_list.rs index e2e6d7832..128c3f4c2 100644 --- a/src/expr/in_list.rs +++ b/src/expr/in_list.rs @@ -15,10 +15,11 @@ // specific language governing permissions and limitations // under the License. -use crate::expr::PyExpr; use datafusion::logical_expr::expr::InList; use pyo3::prelude::*; +use crate::expr::PyExpr; + #[pyclass(frozen, name = "InList", module = "datafusion.expr", subclass)] #[derive(Clone)] pub struct PyInList { diff --git a/src/expr/in_subquery.rs b/src/expr/in_subquery.rs index 6d4a38e49..5cff86c06 100644 --- a/src/expr/in_subquery.rs +++ b/src/expr/in_subquery.rs @@ -18,7 +18,8 @@ use datafusion::logical_expr::expr::InSubquery; use pyo3::prelude::*; -use super::{subquery::PySubquery, PyExpr}; +use super::subquery::PySubquery; +use super::PyExpr; #[pyclass(frozen, name = "InSubquery", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/join.rs b/src/expr/join.rs index 3fde874d5..82cc2a607 100644 --- a/src/expr/join.rs +++ b/src/expr/join.rs @@ -15,13 +15,16 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::common::NullEquality; use datafusion::logical_expr::logical_plan::{Join, JoinConstraint, JoinType}; -use pyo3::{prelude::*, IntoPyObjectExt}; -use std::fmt::{self, Display, Formatter}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use crate::common::df_schema::PyDFSchema; -use crate::expr::{logical_node::LogicalNode, PyExpr}; +use crate::expr::logical_node::LogicalNode; +use crate::expr::PyExpr; use crate::sql::logical::PyLogicalPlan; #[derive(Debug, Clone, PartialEq, Eq, Hash)] diff --git a/src/expr/like.rs b/src/expr/like.rs index 0a36dcd92..94860bd6c 100644 --- a/src/expr/like.rs +++ b/src/expr/like.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::logical_expr::expr::Like; use pyo3::prelude::*; -use std::fmt::{self, Display, Formatter}; use crate::expr::PyExpr; diff --git a/src/expr/limit.rs b/src/expr/limit.rs index cf6971fb3..9318eff97 100644 --- a/src/expr/limit.rs +++ b/src/expr/limit.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use datafusion::logical_expr::logical_plan::Limit; -use pyo3::{prelude::*, IntoPyObjectExt}; use std::fmt::{self, Display, Formatter}; +use datafusion::logical_expr::logical_plan::Limit; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + use crate::common::df_schema::PyDFSchema; use crate::expr::logical_node::LogicalNode; use crate::sql::logical::PyLogicalPlan; diff --git a/src/expr/literal.rs b/src/expr/literal.rs index 8a589b55a..3e8e229f9 100644 --- a/src/expr/literal.rs +++ b/src/expr/literal.rs @@ -15,9 +15,12 @@ // specific language governing permissions and limitations // under the License. +use datafusion::common::ScalarValue; +use datafusion::logical_expr::expr::FieldMetadata; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + use crate::errors::PyDataFusionError; -use datafusion::{common::ScalarValue, logical_expr::expr::FieldMetadata}; -use pyo3::{prelude::*, IntoPyObjectExt}; #[pyclass(name = "Literal", module = "datafusion.expr", subclass, frozen)] #[derive(Clone)] diff --git a/src/expr/projection.rs b/src/expr/projection.rs index b2d5db79b..bd21418a2 100644 --- a/src/expr/projection.rs +++ b/src/expr/projection.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::logical_expr::logical_plan::Projection; use datafusion::logical_expr::Expr; -use pyo3::{prelude::*, IntoPyObjectExt}; -use std::fmt::{self, Display, Formatter}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use crate::common::df_schema::PyDFSchema; use crate::expr::logical_node::LogicalNode; diff --git a/src/expr/recursive_query.rs b/src/expr/recursive_query.rs index fe047315e..0e1171ea9 100644 --- a/src/expr/recursive_query.rs +++ b/src/expr/recursive_query.rs @@ -18,11 +18,11 @@ use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::RecursiveQuery; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::sql::logical::PyLogicalPlan; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "RecursiveQuery", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/repartition.rs b/src/expr/repartition.rs index ee6d1dc45..0b3cc4b2b 100644 --- a/src/expr/repartition.rs +++ b/src/expr/repartition.rs @@ -17,12 +17,15 @@ use std::fmt::{self, Display, Formatter}; -use datafusion::logical_expr::{logical_plan::Repartition, Expr, Partitioning}; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::{errors::py_type_err, sql::logical::PyLogicalPlan}; - -use super::{logical_node::LogicalNode, PyExpr}; +use datafusion::logical_expr::logical_plan::Repartition; +use datafusion::logical_expr::{Expr, Partitioning}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + +use super::logical_node::LogicalNode; +use super::PyExpr; +use crate::errors::py_type_err; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "Repartition", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/sort.rs b/src/expr/sort.rs index d5ea07fdd..8914c8f93 100644 --- a/src/expr/sort.rs +++ b/src/expr/sort.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::common::DataFusionError; use datafusion::logical_expr::logical_plan::Sort; -use pyo3::{prelude::*, IntoPyObjectExt}; -use std::fmt::{self, Display, Formatter}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use crate::common::df_schema::PyDFSchema; use crate::expr::logical_node::LogicalNode; diff --git a/src/expr/sort_expr.rs b/src/expr/sort_expr.rs index 3f279027e..23c066156 100644 --- a/src/expr/sort_expr.rs +++ b/src/expr/sort_expr.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::expr::PyExpr; +use std::fmt::{self, Display, Formatter}; + use datafusion::logical_expr::SortExpr; use pyo3::prelude::*; -use std::fmt::{self, Display, Formatter}; + +use crate::expr::PyExpr; #[pyclass(frozen, name = "SortExpr", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/statement.rs b/src/expr/statement.rs index 1ea4f9f7f..47f5c4337 100644 --- a/src/expr/statement.rs +++ b/src/expr/statement.rs @@ -19,11 +19,13 @@ use datafusion::logical_expr::{ Deallocate, Execute, Prepare, SetVariable, TransactionAccessMode, TransactionConclusion, TransactionEnd, TransactionIsolationLevel, TransactionStart, }; -use pyo3::{prelude::*, IntoPyObjectExt}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; -use crate::{common::data_type::PyDataType, sql::logical::PyLogicalPlan}; - -use super::{logical_node::LogicalNode, PyExpr}; +use super::logical_node::LogicalNode; +use super::PyExpr; +use crate::common::data_type::PyDataType; +use crate::sql::logical::PyLogicalPlan; #[pyclass( frozen, diff --git a/src/expr/subquery.rs b/src/expr/subquery.rs index 785cf7d1a..94c2583ba 100644 --- a/src/expr/subquery.rs +++ b/src/expr/subquery.rs @@ -18,11 +18,11 @@ use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::Subquery; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::sql::logical::PyLogicalPlan; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "Subquery", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/subquery_alias.rs b/src/expr/subquery_alias.rs index ab1229bfe..9bf1c9c51 100644 --- a/src/expr/subquery_alias.rs +++ b/src/expr/subquery_alias.rs @@ -18,11 +18,12 @@ use std::fmt::{self, Display, Formatter}; use datafusion::logical_expr::SubqueryAlias; -use pyo3::{prelude::*, IntoPyObjectExt}; - -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; use super::logical_node::LogicalNode; +use crate::common::df_schema::PyDFSchema; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "SubqueryAlias", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/table_scan.rs b/src/expr/table_scan.rs index 34a140df3..bbf225f4c 100644 --- a/src/expr/table_scan.rs +++ b/src/expr/table_scan.rs @@ -15,14 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::common::TableReference; use datafusion::logical_expr::logical_plan::TableScan; -use pyo3::{prelude::*, IntoPyObjectExt}; -use std::fmt::{self, Display, Formatter}; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; +use crate::common::df_schema::PyDFSchema; use crate::expr::logical_node::LogicalNode; +use crate::expr::PyExpr; use crate::sql::logical::PyLogicalPlan; -use crate::{common::df_schema::PyDFSchema, expr::PyExpr}; #[pyclass(frozen, name = "TableScan", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/union.rs b/src/expr/union.rs index b7b589650..c74d170aa 100644 --- a/src/expr/union.rs +++ b/src/expr/union.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use datafusion::logical_expr::logical_plan::Union; -use pyo3::{prelude::*, IntoPyObjectExt}; use std::fmt::{self, Display, Formatter}; +use datafusion::logical_expr::logical_plan::Union; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + use crate::common::df_schema::PyDFSchema; use crate::expr::logical_node::LogicalNode; use crate::sql::logical::PyLogicalPlan; diff --git a/src/expr/unnest.rs b/src/expr/unnest.rs index 7ed7919b1..7e68c15f4 100644 --- a/src/expr/unnest.rs +++ b/src/expr/unnest.rs @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. -use datafusion::logical_expr::logical_plan::Unnest; -use pyo3::{prelude::*, IntoPyObjectExt}; use std::fmt::{self, Display, Formatter}; +use datafusion::logical_expr::logical_plan::Unnest; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + use crate::common::df_schema::PyDFSchema; use crate::expr::logical_node::LogicalNode; use crate::sql::logical::PyLogicalPlan; diff --git a/src/expr/unnest_expr.rs b/src/expr/unnest_expr.rs index 2cdf46a59..dc6c4cb50 100644 --- a/src/expr/unnest_expr.rs +++ b/src/expr/unnest_expr.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + use datafusion::logical_expr::expr::Unnest; use pyo3::prelude::*; -use std::fmt::{self, Display, Formatter}; use super::PyExpr; diff --git a/src/expr/values.rs b/src/expr/values.rs index 63d94ce00..7ae7350fc 100644 --- a/src/expr/values.rs +++ b/src/expr/values.rs @@ -18,12 +18,13 @@ use std::sync::Arc; use datafusion::logical_expr::Values; -use pyo3::{prelude::*, IntoPyObjectExt}; -use pyo3::{pyclass, PyErr, PyResult, Python}; +use pyo3::prelude::*; +use pyo3::{pyclass, IntoPyObjectExt, PyErr, PyResult, Python}; -use crate::{common::df_schema::PyDFSchema, sql::logical::PyLogicalPlan}; - -use super::{logical_node::LogicalNode, PyExpr}; +use super::logical_node::LogicalNode; +use super::PyExpr; +use crate::common::df_schema::PyDFSchema; +use crate::sql::logical::PyLogicalPlan; #[pyclass(frozen, name = "Values", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/expr/window.rs b/src/expr/window.rs index 2723007ec..b93e813c4 100644 --- a/src/expr/window.rs +++ b/src/expr/window.rs @@ -15,6 +15,15 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{self, Display, Formatter}; + +use datafusion::common::{DataFusionError, ScalarValue}; +use datafusion::logical_expr::{Expr, Window, WindowFrame, WindowFrameBound, WindowFrameUnits}; +use pyo3::exceptions::PyNotImplementedError; +use pyo3::prelude::*; +use pyo3::IntoPyObjectExt; + +use super::py_expr_list; use crate::common::data_type::PyScalarValue; use crate::common::df_schema::PyDFSchema; use crate::errors::{py_type_err, PyDataFusionResult}; @@ -22,13 +31,6 @@ use crate::expr::logical_node::LogicalNode; use crate::expr::sort_expr::{py_sort_expr_list, PySortExpr}; use crate::expr::PyExpr; use crate::sql::logical::PyLogicalPlan; -use datafusion::common::{DataFusionError, ScalarValue}; -use datafusion::logical_expr::{Expr, Window, WindowFrame, WindowFrameBound, WindowFrameUnits}; -use pyo3::exceptions::PyNotImplementedError; -use pyo3::{prelude::*, IntoPyObjectExt}; -use std::fmt::{self, Display, Formatter}; - -use super::py_expr_list; #[pyclass(frozen, name = "WindowExpr", module = "datafusion.expr", subclass)] #[derive(Clone)] diff --git a/src/functions.rs b/src/functions.rs index 5956b67cf..38f54d1ff 100644 --- a/src/functions.rs +++ b/src/functions.rs @@ -17,32 +17,24 @@ use std::collections::HashMap; +use datafusion::common::{Column, ScalarValue, TableReference}; +use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::all_default_aggregate_functions; use datafusion::functions_window::all_default_window_functions; -use datafusion::logical_expr::expr::FieldMetadata; -use datafusion::logical_expr::expr::WindowFunctionParams; -use datafusion::logical_expr::ExprFunctionExt; -use datafusion::logical_expr::WindowFrame; -use pyo3::{prelude::*, wrap_pyfunction}; - -use crate::common::data_type::NullTreatment; -use crate::common::data_type::PyScalarValue; +use datafusion::logical_expr::expr::{Alias, FieldMetadata, WindowFunction, WindowFunctionParams}; +use datafusion::logical_expr::sqlparser::ast::NullTreatment as DFNullTreatment; +use datafusion::logical_expr::{lit, Expr, ExprFunctionExt, WindowFrame, WindowFunctionDefinition}; +use datafusion::{functions, functions_aggregate, functions_window}; +use pyo3::prelude::*; +use pyo3::wrap_pyfunction; + +use crate::common::data_type::{NullTreatment, PyScalarValue}; use crate::context::PySessionContext; -use crate::errors::PyDataFusionError; -use crate::errors::PyDataFusionResult; +use crate::errors::{PyDataFusionError, PyDataFusionResult}; use crate::expr::conditional_expr::PyCaseBuilder; -use crate::expr::sort_expr::to_sort_expressions; -use crate::expr::sort_expr::PySortExpr; +use crate::expr::sort_expr::{to_sort_expressions, PySortExpr}; use crate::expr::window::PyWindowFrame; use crate::expr::PyExpr; -use datafusion::common::{Column, ScalarValue, TableReference}; -use datafusion::execution::FunctionRegistry; -use datafusion::functions; -use datafusion::functions_aggregate; -use datafusion::functions_window; -use datafusion::logical_expr::expr::Alias; -use datafusion::logical_expr::sqlparser::ast::NullTreatment as DFNullTreatment; -use datafusion::logical_expr::{expr::WindowFunction, lit, Expr, WindowFunctionDefinition}; fn add_builder_fns_to_aggregate( agg_fn: Expr, diff --git a/src/lib.rs b/src/lib.rs index 4f816d887..9483a5252 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -15,19 +15,16 @@ // specific language governing permissions and limitations // under the License. -#[cfg(feature = "mimalloc")] -use mimalloc::MiMalloc; -use pyo3::prelude::*; - // Re-export Apache Arrow DataFusion dependencies pub use datafusion; -pub use datafusion::common as datafusion_common; -pub use datafusion::logical_expr as datafusion_expr; -pub use datafusion::optimizer; -pub use datafusion::sql as datafusion_sql; - +pub use datafusion::{ + common as datafusion_common, logical_expr as datafusion_expr, optimizer, sql as datafusion_sql, +}; #[cfg(feature = "substrait")] pub use datafusion_substrait; +#[cfg(feature = "mimalloc")] +use mimalloc::MiMalloc; +use pyo3::prelude::*; #[allow(clippy::borrow_deref_ref)] pub mod catalog; diff --git a/src/physical_plan.rs b/src/physical_plan.rs index 4994b0114..62ec5e881 100644 --- a/src/physical_plan.rs +++ b/src/physical_plan.rs @@ -15,14 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use datafusion::physical_plan::{displayable, ExecutionPlan, ExecutionPlanProperties}; use datafusion_proto::physical_plan::{AsExecutionPlan, DefaultPhysicalExtensionCodec}; use prost::Message; -use std::sync::Arc; - -use pyo3::{exceptions::PyRuntimeError, prelude::*, types::PyBytes}; +use pyo3::exceptions::PyRuntimeError; +use pyo3::prelude::*; +use pyo3::types::PyBytes; -use crate::{context::PySessionContext, errors::PyDataFusionResult}; +use crate::context::PySessionContext; +use crate::errors::PyDataFusionResult; #[pyclass(frozen, name = "ExecutionPlan", module = "datafusion", subclass)] #[derive(Debug, Clone)] diff --git a/src/pyarrow_filter_expression.rs b/src/pyarrow_filter_expression.rs index 7fbb1dc2a..db6146955 100644 --- a/src/pyarrow_filter_expression.rs +++ b/src/pyarrow_filter_expression.rs @@ -15,14 +15,14 @@ // specific language governing permissions and limitations // under the License. -/// Converts a Datafusion logical plan expression (Expr) into a PyArrow compute expression -use pyo3::{prelude::*, IntoPyObjectExt}; - use std::convert::TryFrom; use std::result::Result; use datafusion::common::{Column, ScalarValue}; -use datafusion::logical_expr::{expr::InList, Between, BinaryExpr, Expr, Operator}; +use datafusion::logical_expr::expr::InList; +use datafusion::logical_expr::{Between, BinaryExpr, Expr, Operator}; +/// Converts a Datafusion logical plan expression (Expr) into a PyArrow compute expression +use pyo3::{prelude::*, IntoPyObjectExt}; use crate::errors::{PyDataFusionError, PyDataFusionResult}; use crate::pyarrow_util::scalar_to_pyarrow; diff --git a/src/record_batch.rs b/src/record_batch.rs index 00d023b73..8c1933a1c 100644 --- a/src/record_batch.rs +++ b/src/record_batch.rs @@ -17,8 +17,6 @@ use std::sync::Arc; -use crate::errors::PyDataFusionError; -use crate::utils::wait_for_future; use datafusion::arrow::pyarrow::ToPyArrow; use datafusion::arrow::record_batch::RecordBatch; use datafusion::physical_plan::SendableRecordBatchStream; @@ -28,6 +26,9 @@ use pyo3::prelude::*; use pyo3::{pyclass, pymethods, PyObject, PyResult, Python}; use tokio::sync::Mutex; +use crate::errors::PyDataFusionError; +use crate::utils::wait_for_future; + #[pyclass(name = "RecordBatch", module = "datafusion", subclass, frozen)] pub struct PyRecordBatch { batch: RecordBatch, diff --git a/src/sql/logical.rs b/src/sql/logical.rs index 47ea39fdc..86373fc7c 100644 --- a/src/sql/logical.rs +++ b/src/sql/logical.rs @@ -17,6 +17,13 @@ use std::sync::Arc; +use datafusion::logical_expr::{DdlStatement, LogicalPlan, Statement}; +use datafusion_proto::logical_plan::{AsLogicalPlan, DefaultLogicalExtensionCodec}; +use prost::Message; +use pyo3::exceptions::PyRuntimeError; +use pyo3::prelude::*; +use pyo3::types::PyBytes; + use crate::context::PySessionContext; use crate::errors::PyDataFusionResult; use crate::expr::aggregate::PyAggregate; @@ -42,6 +49,7 @@ use crate::expr::extension::PyExtension; use crate::expr::filter::PyFilter; use crate::expr::join::PyJoin; use crate::expr::limit::PyLimit; +use crate::expr::logical_node::LogicalNode; use crate::expr::projection::PyProjection; use crate::expr::recursive_query::PyRecursiveQuery; use crate::expr::repartition::PyRepartition; @@ -56,12 +64,6 @@ use crate::expr::union::PyUnion; use crate::expr::unnest::PyUnnest; use crate::expr::values::PyValues; use crate::expr::window::PyWindowExpr; -use datafusion::logical_expr::{DdlStatement, LogicalPlan, Statement}; -use datafusion_proto::logical_plan::{AsLogicalPlan, DefaultLogicalExtensionCodec}; -use prost::Message; -use pyo3::{exceptions::PyRuntimeError, prelude::*, types::PyBytes}; - -use crate::expr::logical_node::LogicalNode; #[pyclass(frozen, name = "LogicalPlan", module = "datafusion", subclass)] #[derive(Debug, Clone)] diff --git a/src/store.rs b/src/store.rs index 08702e903..dcbcbd325 100644 --- a/src/store.rs +++ b/src/store.rs @@ -17,14 +17,13 @@ use std::sync::Arc; -use pyo3::prelude::*; - use object_store::aws::{AmazonS3, AmazonS3Builder}; use object_store::azure::{MicrosoftAzure, MicrosoftAzureBuilder}; use object_store::gcp::{GoogleCloudStorage, GoogleCloudStorageBuilder}; use object_store::http::{HttpBuilder, HttpStore}; use object_store::local::LocalFileSystem; use pyo3::exceptions::PyValueError; +use pyo3::prelude::*; use url::Url; #[derive(FromPyObject)] diff --git a/src/substrait.rs b/src/substrait.rs index 291892cf8..0e0305cfe 100644 --- a/src/substrait.rs +++ b/src/substrait.rs @@ -15,18 +15,18 @@ // specific language governing permissions and limitations // under the License. -use pyo3::{prelude::*, types::PyBytes}; +use datafusion_substrait::logical_plan::{consumer, producer}; +use datafusion_substrait::serializer; +use datafusion_substrait::substrait::proto::Plan; +use prost::Message; +use pyo3::prelude::*; +use pyo3::types::PyBytes; use crate::context::PySessionContext; use crate::errors::{py_datafusion_err, PyDataFusionError, PyDataFusionResult}; use crate::sql::logical::PyLogicalPlan; use crate::utils::wait_for_future; -use datafusion_substrait::logical_plan::{consumer, producer}; -use datafusion_substrait::serializer; -use datafusion_substrait::substrait::proto::Plan; -use prost::Message; - #[pyclass(frozen, name = "Plan", module = "datafusion.substrait", subclass)] #[derive(Debug, Clone)] pub struct PyPlan { diff --git a/src/table.rs b/src/table.rs index fdca4d3e6..1e8542134 100644 --- a/src/table.rs +++ b/src/table.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use crate::dataframe::PyDataFrame; -use crate::dataset::Dataset; -use crate::utils::table_provider_from_pycapsule; +use std::any::Any; +use std::sync::Arc; + use arrow::datatypes::SchemaRef; use arrow::pyarrow::ToPyArrow; use async_trait::async_trait; @@ -28,8 +28,10 @@ use datafusion::logical_expr::{Expr, LogicalPlanBuilder, TableProviderFilterPush use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::DataFrame; use pyo3::prelude::*; -use std::any::Any; -use std::sync::Arc; + +use crate::dataframe::PyDataFrame; +use crate::dataset::Dataset; +use crate::utils::table_provider_from_pycapsule; /// This struct is used as a common method for all TableProviders, /// whether they refer to an FFI provider, an internally known diff --git a/src/udaf.rs b/src/udaf.rs index e48e35f8d..b3d497557 100644 --- a/src/udaf.rs +++ b/src/udaf.rs @@ -17,12 +17,6 @@ use std::sync::Arc; -use pyo3::{prelude::*, types::PyTuple}; - -use crate::common::data_type::PyScalarValue; -use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionResult}; -use crate::expr::PyExpr; -use crate::utils::{parse_volatility, validate_pycapsule}; use datafusion::arrow::array::{Array, ArrayRef}; use datafusion::arrow::datatypes::DataType; use datafusion::arrow::pyarrow::{PyArrowType, ToPyArrow}; @@ -32,7 +26,13 @@ use datafusion::logical_expr::{ create_udaf, Accumulator, AccumulatorFactoryFunction, AggregateUDF, }; use datafusion_ffi::udaf::{FFI_AggregateUDF, ForeignAggregateUDF}; -use pyo3::types::PyCapsule; +use pyo3::prelude::*; +use pyo3::types::{PyCapsule, PyTuple}; + +use crate::common::data_type::PyScalarValue; +use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionResult}; +use crate::expr::PyExpr; +use crate::utils::{parse_volatility, validate_pycapsule}; #[derive(Debug)] struct RustAccumulator { diff --git a/src/udf.rs b/src/udf.rs index a9249d6c8..25043d2ea 100644 --- a/src/udf.rs +++ b/src/udf.rs @@ -17,21 +17,17 @@ use std::sync::Arc; -use datafusion_ffi::udf::{FFI_ScalarUDF, ForeignScalarUDF}; -use pyo3::types::PyCapsule; -use pyo3::{prelude::*, types::PyTuple}; - use datafusion::arrow::array::{make_array, Array, ArrayData, ArrayRef}; use datafusion::arrow::datatypes::DataType; -use datafusion::arrow::pyarrow::FromPyArrow; -use datafusion::arrow::pyarrow::{PyArrowType, ToPyArrow}; +use datafusion::arrow::pyarrow::{FromPyArrow, PyArrowType, ToPyArrow}; use datafusion::error::DataFusionError; use datafusion::logical_expr::function::ScalarFunctionImplementation; -use datafusion::logical_expr::ScalarUDF; -use datafusion::logical_expr::{create_udf, ColumnarValue}; +use datafusion::logical_expr::{create_udf, ColumnarValue, ScalarUDF}; +use datafusion_ffi::udf::{FFI_ScalarUDF, ForeignScalarUDF}; +use pyo3::prelude::*; +use pyo3::types::{PyCapsule, PyTuple}; -use crate::errors::to_datafusion_err; -use crate::errors::{py_datafusion_err, PyDataFusionResult}; +use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionResult}; use crate::expr::PyExpr; use crate::utils::{parse_volatility, validate_pycapsule}; diff --git a/src/udtf.rs b/src/udtf.rs index f6604e5bc..15006edf4 100644 --- a/src/udtf.rs +++ b/src/udtf.rs @@ -15,19 +15,20 @@ // specific language governing permissions and limitations // under the License. -use pyo3::prelude::*; use std::sync::Arc; -use crate::errors::{py_datafusion_err, to_datafusion_err}; -use crate::expr::PyExpr; -use crate::table::PyTable; -use crate::utils::validate_pycapsule; use datafusion::catalog::{TableFunctionImpl, TableProvider}; use datafusion::error::Result as DataFusionResult; use datafusion::logical_expr::Expr; use datafusion_ffi::udtf::{FFI_TableFunction, ForeignTableFunction}; +use pyo3::prelude::*; use pyo3::types::{PyCapsule, PyTuple}; +use crate::errors::{py_datafusion_err, to_datafusion_err}; +use crate::expr::PyExpr; +use crate::table::PyTable; +use crate::utils::validate_pycapsule; + /// Represents a user defined table function #[pyclass(frozen, name = "TableFunction", module = "datafusion")] #[derive(Debug, Clone)] diff --git a/src/udwf.rs b/src/udwf.rs index ceeaa0ef1..aae9f4c4c 100644 --- a/src/udwf.rs +++ b/src/udwf.rs @@ -20,25 +20,25 @@ use std::ops::Range; use std::sync::Arc; use arrow::array::{make_array, Array, ArrayData, ArrayRef}; +use datafusion::arrow::datatypes::DataType; +use datafusion::arrow::pyarrow::{FromPyArrow, PyArrowType, ToPyArrow}; +use datafusion::error::{DataFusionError, Result}; use datafusion::logical_expr::function::{PartitionEvaluatorArgs, WindowUDFFieldArgs}; +use datafusion::logical_expr::ptr_eq::PtrEq; use datafusion::logical_expr::window_state::WindowAggState; +use datafusion::logical_expr::{ + PartitionEvaluator, PartitionEvaluatorFactory, Signature, Volatility, WindowUDF, WindowUDFImpl, +}; use datafusion::scalar::ScalarValue; +use datafusion_ffi::udwf::{FFI_WindowUDF, ForeignWindowUDF}; use pyo3::exceptions::PyValueError; use pyo3::prelude::*; +use pyo3::types::{PyCapsule, PyList, PyTuple}; use crate::common::data_type::PyScalarValue; use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionResult}; use crate::expr::PyExpr; use crate::utils::{parse_volatility, validate_pycapsule}; -use datafusion::arrow::datatypes::DataType; -use datafusion::arrow::pyarrow::{FromPyArrow, PyArrowType, ToPyArrow}; -use datafusion::error::{DataFusionError, Result}; -use datafusion::logical_expr::ptr_eq::PtrEq; -use datafusion::logical_expr::{ - PartitionEvaluator, PartitionEvaluatorFactory, Signature, Volatility, WindowUDF, WindowUDFImpl, -}; -use datafusion_ffi::udwf::{FFI_WindowUDF, ForeignWindowUDF}; -use pyo3::types::{PyCapsule, PyList, PyTuple}; #[derive(Debug)] struct RustPartitionEvaluator { diff --git a/src/unparser/mod.rs b/src/unparser/mod.rs index f234345a7..908b59d3b 100644 --- a/src/unparser/mod.rs +++ b/src/unparser/mod.rs @@ -19,9 +19,11 @@ mod dialect; use std::sync::Arc; -use datafusion::sql::unparser::{dialect::Dialect, Unparser}; +use datafusion::sql::unparser::dialect::Dialect; +use datafusion::sql::unparser::Unparser; use dialect::PyDialect; -use pyo3::{exceptions::PyValueError, prelude::*}; +use pyo3::exceptions::PyValueError; +use pyo3::prelude::*; use crate::sql::logical::PyLogicalPlan; diff --git a/src/utils.rs b/src/utils.rs index 9624f7d77..ce7395e95 100644 --- a/src/utils.rs +++ b/src/utils.rs @@ -15,25 +15,25 @@ // specific language governing permissions and limitations // under the License. -use crate::errors::py_datafusion_err; -use crate::{ - common::data_type::PyScalarValue, - errors::{to_datafusion_err, PyDataFusionError, PyDataFusionResult}, - TokioRuntime, -}; -use datafusion::{ - common::ScalarValue, datasource::TableProvider, execution::context::SessionContext, - logical_expr::Volatility, -}; +use std::future::Future; +use std::sync::{Arc, OnceLock}; +use std::time::Duration; + +use datafusion::common::ScalarValue; +use datafusion::datasource::TableProvider; +use datafusion::execution::context::SessionContext; +use datafusion::logical_expr::Volatility; use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider}; +use pyo3::exceptions::PyValueError; use pyo3::prelude::*; -use pyo3::{exceptions::PyValueError, types::PyCapsule}; -use std::{ - future::Future, - sync::{Arc, OnceLock}, - time::Duration, -}; -use tokio::{runtime::Runtime, task::JoinHandle, time::sleep}; +use pyo3::types::PyCapsule; +use tokio::runtime::Runtime; +use tokio::task::JoinHandle; +use tokio::time::sleep; + +use crate::common::data_type::PyScalarValue; +use crate::errors::{py_datafusion_err, to_datafusion_err, PyDataFusionError, PyDataFusionResult}; +use crate::TokioRuntime; /// Utility to get the Tokio Runtime from Python #[inline]