Skip to content

Commit

Permalink
rewrite more imports
Browse files Browse the repository at this point in the history
  • Loading branch information
andygrove committed Jun 14, 2022
1 parent 5fd6195 commit a231cd0
Show file tree
Hide file tree
Showing 20 changed files with 70 additions and 78 deletions.
5 changes: 2 additions & 3 deletions benchmarks/src/bin/tpch.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ use std::{

use datafusion::datasource::{MemTable, TableProvider};
use datafusion::error::{DataFusionError, Result};
use datafusion::logical_plan::LogicalPlan;
use datafusion::logical_expr::LogicalPlan;
use datafusion::parquet::basic::Compression;
use datafusion::parquet::file::properties::WriterProperties;
use datafusion::physical_plan::display::DisplayableExecutionPlan;
Expand Down Expand Up @@ -584,8 +584,7 @@ mod tests {

use datafusion::arrow::array::*;
use datafusion::arrow::util::display::array_value_to_string;
use datafusion::logical_plan::Expr;
use datafusion::logical_plan::Expr::Cast;
use datafusion::logical_expr::{Expr, Expr::Cast};

#[tokio::test]
async fn q1() -> Result<()> {
Expand Down
4 changes: 2 additions & 2 deletions datafusion-examples/examples/custom_datasource.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@ use datafusion::arrow::array::{UInt64Builder, UInt8Builder};
use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::dataframe::DataFrame;
use datafusion::datasource::{TableProvider, TableType};
use datafusion::datasource::{provider_as_source, TableProvider, TableType};
use datafusion::error::Result;
use datafusion::execution::context::{SessionState, TaskContext};
use datafusion::logical_plan::{provider_as_source, Expr, LogicalPlanBuilder};
use datafusion::logical_expr::{Expr, LogicalPlanBuilder};
use datafusion::physical_plan::expressions::PhysicalSortExpr;
use datafusion::physical_plan::memory::MemoryStream;
use datafusion::physical_plan::{
Expand Down
8 changes: 6 additions & 2 deletions datafusion-examples/examples/simple_udaf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,12 @@ use datafusion::arrow::{
};

use datafusion::from_slice::FromSlice;
use datafusion::{error::Result, logical_plan::create_udaf, physical_plan::Accumulator};
use datafusion::{logical_expr::Volatility, prelude::*, scalar::ScalarValue};
use datafusion::{error::Result, physical_plan::Accumulator};
use datafusion::{
logical_expr::{create_udaf, Volatility},
prelude::*,
scalar::ScalarValue,
};
use std::sync::Arc;

// create local session context with an in-memory table
Expand Down
19 changes: 10 additions & 9 deletions datafusion/core/tests/custom_sources.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,23 +20,24 @@ use arrow::compute::kernels::aggregate;
use arrow::datatypes::{DataType, Field, Int32Type, Schema, SchemaRef};
use arrow::error::Result as ArrowResult;
use arrow::record_batch::RecordBatch;
use datafusion::execution::context::{SessionContext, SessionState, TaskContext};
use datafusion::from_slice::FromSlice;
use datafusion::physical_plan::empty::EmptyExec;
use datafusion::physical_plan::expressions::PhysicalSortExpr;
use datafusion::physical_plan::{
project_schema, ColumnStatistics, ExecutionPlan, Partitioning, RecordBatchStream,
SendableRecordBatchStream, Statistics,
};
use datafusion::scalar::ScalarValue;
use datafusion::{
datasource::{TableProvider, TableType},
physical_plan::collect,
};
use datafusion::{error::Result, physical_plan::DisplayFormatType};

use datafusion::execution::context::{SessionContext, SessionState, TaskContext};
use datafusion::logical_plan::{
col, Expr, LogicalPlan, LogicalPlanBuilder, TableScan, UNNAMED_TABLE,
};
use datafusion::physical_plan::{
project_schema, ColumnStatistics, ExecutionPlan, Partitioning, RecordBatchStream,
SendableRecordBatchStream, Statistics,
use datafusion_expr::{
col,
logical_plan::{LogicalPlan, LogicalPlanBuilder, Projection, TableScan},
Expr,
};

use futures::stream::Stream;
Expand All @@ -46,7 +47,7 @@ use std::sync::Arc;
use std::task::{Context, Poll};

use async_trait::async_trait;
use datafusion::logical_plan::plan::Projection;
use datafusion_expr::logical_plan::builder::UNNAMED_TABLE;

//// Custom source dataframe tests ////

Expand Down
4 changes: 1 addition & 3 deletions datafusion/core/tests/dataframe.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,9 @@ use datafusion::assert_batches_eq;
use datafusion::dataframe::DataFrame;
use datafusion::error::Result;
use datafusion::execution::context::SessionContext;
use datafusion::logical_plan::{col, Expr};
use datafusion::prelude::CsvReadOptions;
use datafusion::{datasource::MemTable, prelude::JoinType};
use datafusion_expr::expr::GroupingSet;
use datafusion_expr::{avg, count, lit, sum};
use datafusion_expr::{avg, col, count, expr::GroupingSet, lit, sum, Expr};

#[tokio::test]
async fn join() -> Result<()> {
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/tests/join_fuzz.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,12 +24,12 @@ use arrow::util::pretty::pretty_format_batches;
use rand::rngs::StdRng;
use rand::{Rng, SeedableRng};

use datafusion::logical_plan::JoinType;
use datafusion::physical_plan::collect;
use datafusion::physical_plan::expressions::Column;
use datafusion::physical_plan::hash_join::{HashJoinExec, PartitionMode};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::physical_plan::sort_merge_join::SortMergeJoinExec;
use datafusion_expr::logical_plan::JoinType;

use datafusion::prelude::{SessionConfig, SessionContext};
use fuzz_utils::add_empty_batches;
Expand Down
4 changes: 2 additions & 2 deletions datafusion/core/tests/parquet_pruning.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,17 +30,17 @@ use arrow::{
util::pretty::pretty_format_batches,
};
use chrono::{Datelike, Duration};
use datafusion::logical_plan::provider_as_source;
use datafusion::datasource::provider_as_source;
use datafusion::{
datasource::TableProvider,
logical_plan::{col, lit, Expr, LogicalPlan, LogicalPlanBuilder},
physical_plan::{
accept, file_format::ParquetExec, metrics::MetricsSet, ExecutionPlan,
ExecutionPlanVisitor,
},
prelude::{ParquetReadOptions, SessionConfig, SessionContext},
scalar::ScalarValue,
};
use datafusion_expr::{col, lit, Expr, LogicalPlan, LogicalPlanBuilder};
use parquet::{arrow::ArrowWriter, file::properties::WriterProperties};
use tempfile::NamedTempFile;

Expand Down
12 changes: 4 additions & 8 deletions datafusion/core/tests/simplification.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,10 @@
//! This program demonstrates the DataFusion expression simplification API.
use arrow::datatypes::{DataType, Field, Schema};
use datafusion::logical_plan::ExprSchemable;
use datafusion::logical_plan::ExprSimplifiable;
use datafusion::{
error::Result,
execution::context::ExecutionProps,
logical_plan::{DFSchema, Expr, SimplifyInfo},
prelude::*,
};
use datafusion::{error::Result, execution::context::ExecutionProps, prelude::*};
use datafusion_common::DFSchema;
use datafusion_expr::{expr_schema::ExprSchemable, Expr};
use datafusion_optimizer::expr_simplifier::{ExprSimplifiable, SimplifyInfo};

/// In order to simplify expressions, DataFusion must have information
/// about the expressions.
Expand Down
6 changes: 2 additions & 4 deletions datafusion/core/tests/sql/explain.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,9 @@
// under the License.

use arrow::datatypes::{DataType, Field, Schema};
use datafusion::prelude::SessionContext;
use datafusion::test_util::scan_empty;
use datafusion::{
logical_plan::{LogicalPlan, PlanType},
prelude::SessionContext,
};
use datafusion_expr::logical_plan::{LogicalPlan, PlanType};

#[test]
fn optimize_explain() {
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/tests/sql/information_schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@ use datafusion::{
schema::{MemorySchemaProvider, SchemaProvider},
},
datasource::{TableProvider, TableType},
logical_plan::Expr,
};
use datafusion_expr::Expr;

use super::*;

Expand Down
4 changes: 1 addition & 3 deletions datafusion/core/tests/sql/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,6 @@ use datafusion::assert_contains;
use datafusion::assert_not_contains;
use datafusion::datasource::TableProvider;
use datafusion::from_slice::FromSlice;
use datafusion::logical_plan::plan::{Aggregate, Projection};
use datafusion::logical_plan::LogicalPlan;
use datafusion::logical_plan::TableScan;
use datafusion::physical_plan::metrics::MetricValue;
use datafusion::physical_plan::ExecutionPlan;
use datafusion::physical_plan::ExecutionPlanVisitor;
Expand All @@ -45,6 +42,7 @@ use datafusion::{
physical_plan::ColumnarValue,
};
use datafusion::{execution::context::SessionContext, physical_plan::displayable};
use datafusion_expr::logical_plan::{Aggregate, LogicalPlan, Projection, TableScan};
use datafusion_expr::Volatility;
use std::fs::File;
use std::io::Write;
Expand Down
3 changes: 2 additions & 1 deletion datafusion/core/tests/sql/projection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,9 @@
// specific language governing permissions and limitations
// under the License.

use datafusion::logical_plan::{provider_as_source, LogicalPlanBuilder, UNNAMED_TABLE};
use datafusion::datasource::provider_as_source;
use datafusion::test_util::scan_empty;
use datafusion_expr::logical_plan::builder::{LogicalPlanBuilder, UNNAMED_TABLE};
use tempfile::TempDir;

use super::*;
Expand Down
7 changes: 4 additions & 3 deletions datafusion/core/tests/sql/udf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,11 @@

use super::*;
use arrow::compute::add;
use datafusion::{
logical_plan::{create_udaf, FunctionRegistry, LogicalPlanBuilder},
physical_plan::{expressions::AvgAccumulator, functions::make_scalar_function},
use datafusion::execution::FunctionRegistry;
use datafusion::physical_plan::{
expressions::AvgAccumulator, functions::make_scalar_function,
};
use datafusion_expr::{create_udaf, logical_plan::builder::LogicalPlanBuilder};

/// test that casting happens on udfs.
/// c11 is f32, but `custom_sqrt` requires f64. Casting happens but the logical plan and
Expand Down
7 changes: 3 additions & 4 deletions datafusion/core/tests/statistics.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,11 @@
use std::{any::Any, sync::Arc};

use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use async_trait::async_trait;
use datafusion::execution::context::{SessionState, TaskContext};
use datafusion::{
datasource::{TableProvider, TableType},
error::Result,
logical_plan::Expr,
physical_plan::{
expressions::PhysicalSortExpr, project_schema, ColumnStatistics,
DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream,
Expand All @@ -32,9 +33,7 @@ use datafusion::{
prelude::SessionContext,
scalar::ScalarValue,
};

use async_trait::async_trait;
use datafusion::execution::context::{SessionState, TaskContext};
use datafusion_expr::Expr;

/// This is a testing structure for statistics
/// It will act both as a table provider and execution plan
Expand Down
8 changes: 5 additions & 3 deletions datafusion/core/tests/user_defined_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,6 @@ use datafusion::{
error::{DataFusionError, Result},
execution::context::QueryPlanner,
execution::context::SessionState,
logical_plan::{Expr, LogicalPlan, UserDefinedLogicalNode},
optimizer::{optimizer::OptimizerRule, utils::optimize_children},
physical_plan::{
expressions::PhysicalSortExpr,
Expand All @@ -88,9 +87,12 @@ use std::{any::Any, collections::BTreeMap, fmt, sync::Arc};
use async_trait::async_trait;
use datafusion::execution::context::TaskContext;
use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv};
use datafusion::logical_plan::plan::{Extension, Sort};
use datafusion::logical_plan::{DFSchemaRef, Limit};
use datafusion::optimizer::optimizer::OptimizerConfig;
use datafusion_common::DFSchemaRef;
use datafusion_expr::{
logical_plan::{Extension, Limit, LogicalPlan, Sort, UserDefinedLogicalNode},
Expr,
};

/// Execute the specified sql and return the resulting record batches
/// pretty printed as a String.
Expand Down
11 changes: 4 additions & 7 deletions datafusion/proto/src/bytes/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,16 +22,15 @@ use crate::{
protobuf,
};
use datafusion_common::{DataFusionError, Result};
use datafusion_expr::{Expr, LogicalPlan};
use datafusion_expr::{logical_plan::Extension, Expr, LogicalPlan};
use prost::{
bytes::{Bytes, BytesMut},
Message,
};

// Reexport Bytes which appears in the API
use datafusion::logical_plan::FunctionRegistry;
use datafusion::execution::FunctionRegistry;
use datafusion::prelude::SessionContext;
use datafusion_expr::logical_plan::Extension;

mod registry;

Expand Down Expand Up @@ -168,11 +167,9 @@ impl LogicalExtensionCodec for DefaultExtensionCodec {
mod test {
use super::*;
use arrow::{array::ArrayRef, datatypes::DataType};
use datafusion::physical_plan::functions::make_scalar_function;
use datafusion::prelude::SessionContext;
use datafusion::{
logical_plan::create_udf, physical_plan::functions::make_scalar_function,
};
use datafusion_expr::{lit, Volatility};
use datafusion_expr::{create_udf, lit, Volatility};
use std::sync::Arc;

#[test]
Expand Down
2 changes: 1 addition & 1 deletion datafusion/proto/src/bytes/registry.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

use std::{collections::HashSet, sync::Arc};

use datafusion::logical_plan::FunctionRegistry;
use datafusion::execution::FunctionRegistry;
use datafusion_common::{DataFusionError, Result};
use datafusion_expr::{AggregateUDF, ScalarUDF};

Expand Down
2 changes: 1 addition & 1 deletion datafusion/proto/src/from_proto.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ use crate::protobuf::{
RollupNode,
};
use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit, UnionMode};
use datafusion::logical_plan::FunctionRegistry;
use datafusion::execution::FunctionRegistry;
use datafusion_common::{
Column, DFField, DFSchema, DFSchemaRef, DataFusionError, ScalarValue,
};
Expand Down
12 changes: 7 additions & 5 deletions datafusion/proto/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -58,15 +58,17 @@ mod roundtrip_tests {
array::ArrayRef,
datatypes::{DataType, Field, IntervalUnit, TimeUnit, UnionMode},
};
use datafusion::logical_plan::create_udaf;
use datafusion::physical_plan::functions::make_scalar_function;
use datafusion::prelude::{create_udf, CsvReadOptions, SessionContext};
use datafusion_common::{DFSchemaRef, DataFusionError, ScalarValue};
use datafusion_expr::expr::GroupingSet;
use datafusion_expr::logical_plan::{Extension, UserDefinedLogicalNode};
use datafusion_expr::{
col, lit, Accumulator, AggregateFunction, BuiltinScalarFunction::Sqrt, Expr,
LogicalPlan, Volatility,
col, create_udaf,
expr::GroupingSet,
lit,
logical_plan::{Extension, UserDefinedLogicalNode},
Accumulator, AggregateFunction,
BuiltinScalarFunction::Sqrt,
Expr, LogicalPlan, Volatility,
};
use prost::Message;
use std::any::Any;
Expand Down
Loading

0 comments on commit a231cd0

Please sign in to comment.