Skip to content

Commit

Permalink
refactor: fix error macros hygiene
Browse files Browse the repository at this point in the history
  • Loading branch information
crepererum committed Feb 27, 2024
1 parent c439bc7 commit 2f3e097
Show file tree
Hide file tree
Showing 132 changed files with 203 additions and 181 deletions.
2 changes: 1 addition & 1 deletion benchmarks/src/tpch/convert.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ use std::path::{Path, PathBuf};
use std::time::Instant;

use datafusion::common::not_impl_err;
use datafusion::error::DataFusionError;

use datafusion::error::Result;
use datafusion::prelude::*;
use parquet::basic::Compression;
Expand Down
2 changes: 1 addition & 1 deletion benchmarks/src/tpch/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ use arrow::datatypes::SchemaBuilder;
use datafusion::{
arrow::datatypes::{DataType, Field, Schema},
common::plan_err,
error::{DataFusionError, Result},
error::Result,
};
use std::fs;
mod run;
Expand Down
2 changes: 1 addition & 1 deletion datafusion-cli/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ use crate::object_storage::get_object_store;
use async_trait::async_trait;
use datafusion::catalog::schema::SchemaProvider;
use datafusion::catalog::{CatalogProvider, CatalogProviderList};
use datafusion::common::{plan_datafusion_err, DataFusionError};
use datafusion::common::plan_datafusion_err;
use datafusion::datasource::listing::{
ListingTable, ListingTableConfig, ListingTableUrl,
};
Expand Down
2 changes: 1 addition & 1 deletion datafusion-cli/src/functions.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use arrow::util::pretty::pretty_format_batches;
use async_trait::async_trait;
use datafusion::common::DataFusionError;

use datafusion::common::{plan_err, Column};
use datafusion::datasource::function::TableFunctionImpl;
use datafusion::datasource::TableProvider;
Expand Down
1 change: 0 additions & 1 deletion datafusion-examples/examples/advanced_udf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,6 @@ impl ScalarUDFImpl for PowUdf {
}
// if the types were not float, it is a bug in DataFusion
_ => {
use datafusion_common::DataFusionError;
internal_err!("Invalid argument types to pow function")
}
}
Expand Down
2 changes: 1 addition & 1 deletion datafusion-examples/examples/rewrite_expr.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use datafusion_common::config::ConfigOptions;
use datafusion_common::tree_node::{Transformed, TreeNode};
use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue};
use datafusion_common::{plan_err, Result, ScalarValue};
use datafusion_expr::{
AggregateUDF, Between, Expr, Filter, LogicalPlan, ScalarUDF, TableSource, WindowUDF,
};
Expand Down
2 changes: 1 addition & 1 deletion datafusion-examples/examples/simple_udtf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ use datafusion::execution::context::{ExecutionProps, SessionState};
use datafusion::physical_plan::memory::MemoryExec;
use datafusion::physical_plan::ExecutionPlan;
use datafusion::prelude::SessionContext;
use datafusion_common::{plan_err, DataFusionError, ScalarValue};
use datafusion_common::{plan_err, ScalarValue};
use datafusion_expr::{Expr, TableType};
use datafusion_optimizer::simplify_expressions::{ExprSimplifier, SimplifyContext};
use std::fs::File;
Expand Down
16 changes: 14 additions & 2 deletions datafusion/common/src/error.rs
Original file line number Diff line number Diff line change
Expand Up @@ -495,15 +495,27 @@ macro_rules! make_error {
#[macro_export]
macro_rules! $NAME_DF_ERR {
($d($d args:expr),*) => {
DataFusionError::$ERR(format!("{}{}", format!($d($d args),*), DataFusionError::get_back_trace()).into())
$crate::DataFusionError::$ERR(
format!(
"{}{}",
format!($d($d args),*),
$crate::DataFusionError::get_back_trace(),
).into()
)
}
}

/// Macro wraps Err(`$ERR`) to add backtrace feature
#[macro_export]
macro_rules! $NAME_ERR {
($d($d args:expr),*) => {
Err(DataFusionError::$ERR(format!("{}{}", format!($d($d args),*), DataFusionError::get_back_trace()).into()))
Err($crate::DataFusionError::$ERR(
format!(
"{}{}",
format!($d($d args),*),
$crate::DataFusionError::get_back_trace(),
).into()
))
}
}
}
Expand Down
2 changes: 1 addition & 1 deletion datafusion/common/src/hash_utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ use crate::cast::{
as_large_list_array, as_list_array, as_primitive_array, as_string_array,
as_struct_array,
};
use crate::error::{DataFusionError, Result, _internal_err};
use crate::error::{Result, _internal_err};

// Combines two hashes into one hash
#[inline]
Expand Down
2 changes: 1 addition & 1 deletion datafusion/common/src/scalar/struct_builder.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
//! [`ScalarStructBuilder`] for building [`ScalarValue::Struct`]
use crate::error::_internal_err;
use crate::{DataFusionError, Result, ScalarValue};
use crate::{Result, ScalarValue};
use arrow::array::{ArrayRef, StructArray};
use arrow::datatypes::{DataType, FieldRef, Fields};
use arrow_schema::Field;
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/catalog/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ pub use datafusion_sql::{ResolvedTableReference, TableReference};

use crate::catalog::schema::SchemaProvider;
use dashmap::DashMap;
use datafusion_common::{exec_err, not_impl_err, DataFusionError, Result};
use datafusion_common::{exec_err, not_impl_err, Result};
use std::any::Any;
use std::sync::Arc;

Expand Down
4 changes: 2 additions & 2 deletions datafusion/core/src/catalog/schema.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@
use async_trait::async_trait;
use dashmap::DashMap;
use datafusion_common::exec_err;
use datafusion_common::{exec_err, DataFusionError};
use std::any::Any;
use std::sync::Arc;

use crate::datasource::TableProvider;
use crate::error::{DataFusionError, Result};
use crate::error::Result;

/// Represents a schema, comprising a number of named tables.
///
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/default_table_source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ use std::sync::Arc;
use crate::datasource::TableProvider;

use arrow::datatypes::SchemaRef;
use datafusion_common::{internal_err, Constraints, DataFusionError};
use datafusion_common::{internal_err, Constraints};
use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource};

/// DataFusion default table source, wrapping TableProvider.
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/file_format/json.rs
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ use arrow::datatypes::SchemaRef;
use arrow::json;
use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter};
use arrow_array::RecordBatch;
use datafusion_common::{not_impl_err, DataFusionError, FileType};
use datafusion_common::{not_impl_err, FileType};
use datafusion_execution::TaskContext;
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement};
use datafusion_physical_plan::metrics::MetricsSet;
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/file_format/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ use crate::error::Result;
use crate::execution::context::SessionState;
use crate::physical_plan::{ExecutionPlan, Statistics};

use datafusion_common::{not_impl_err, DataFusionError, FileType};
use datafusion_common::{not_impl_err, FileType};
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement};

use async_trait::async_trait;
Expand Down
5 changes: 2 additions & 3 deletions datafusion/core/src/datasource/memory.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,7 @@ use std::sync::Arc;
use arrow::datatypes::SchemaRef;
use arrow::record_batch::RecordBatch;
use async_trait::async_trait;
use datafusion_common::{
not_impl_err, plan_err, Constraints, DFSchema, DataFusionError, SchemaExt,
};
use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt};
use datafusion_execution::TaskContext;
use parking_lot::Mutex;
use tokio::sync::RwLock;
Expand Down Expand Up @@ -370,6 +368,7 @@ mod tests {
use arrow::array::{AsArray, Int32Array};
use arrow::datatypes::{DataType, Field, Schema, UInt64Type};
use arrow::error::ArrowError;
use datafusion_common::DataFusionError;
use datafusion_expr::LogicalPlanBuilder;
use futures::StreamExt;
use std::collections::HashMap;
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ pub use crate::logical_expr::TableType;
pub use statistics::get_statistics_with_limit;

use arrow_schema::{Schema, SortOptions};
use datafusion_common::{plan_err, DataFusionError, Result};
use datafusion_common::{plan_err, Result};
use datafusion_expr::Expr;
use datafusion_physical_expr::{expressions, LexOrdering, PhysicalSortExpr};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,7 @@ use std::{

use super::{get_projected_output_ordering, FileGroupPartitioner};
use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl};
use crate::{
error::{DataFusionError, Result},
scalar::ScalarValue,
};
use crate::{error::Result, scalar::ScalarValue};

use arrow::array::{ArrayData, BufferBuilder};
use arrow::buffer::Buffer;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -531,7 +531,7 @@ mod tests {
};

use arrow_schema::Schema;
use datafusion_common::{internal_err, DataFusionError, Statistics};
use datafusion_common::{internal_err, Statistics};

use bytes::Bytes;
use futures::StreamExt;
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/physical_plan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ use std::{
};

use super::listing::ListingTableUrl;
use crate::error::{DataFusionError, Result};
use crate::error::Result;
use crate::physical_plan::{DisplayAs, DisplayFormatType};
use crate::{
datasource::{
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/provider.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ use std::any::Any;
use std::sync::Arc;

use async_trait::async_trait;
use datafusion_common::{not_impl_err, Constraints, DataFusionError, Statistics};
use datafusion_common::{not_impl_err, Constraints, Statistics};
use datafusion_expr::{CreateExternalTable, LogicalPlan};
pub use datafusion_expr::{TableProviderFilterPushDown, TableType};

Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/streaming.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ use std::sync::Arc;
use arrow::datatypes::SchemaRef;
use async_trait::async_trait;

use datafusion_common::{plan_err, DataFusionError, Result};
use datafusion_common::{plan_err, Result};
use datafusion_expr::{Expr, TableType};
use log::debug;

Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/physical_optimizer/enforce_sorting.rs
Original file line number Diff line number Diff line change
Expand Up @@ -59,8 +59,8 @@ use crate::physical_plan::windows::{
};
use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode};

use datafusion_common::plan_err;
use datafusion_common::tree_node::{Transformed, TreeNode};
use datafusion_common::{plan_err, DataFusionError};
use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement};
use datafusion_physical_plan::repartition::RepartitionExec;
use datafusion_physical_plan::sorts::partial_sort::PartialSortExec;
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/physical_optimizer/join_selection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -41,8 +41,8 @@ use crate::physical_plan::ExecutionPlan;

use arrow_schema::Schema;
use datafusion_common::tree_node::{Transformed, TreeNode};
use datafusion_common::JoinType;
use datafusion_common::{internal_err, JoinSide};
use datafusion_common::{DataFusionError, JoinType};
use datafusion_physical_expr::expressions::Column;
use datafusion_physical_expr::sort_properties::SortProperties;
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr};
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/physical_optimizer/pipeline_checker.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,8 +27,8 @@ use crate::physical_optimizer::PhysicalOptimizerRule;
use crate::physical_plan::ExecutionPlan;

use datafusion_common::config::OptimizerOptions;
use datafusion_common::plan_err;
use datafusion_common::tree_node::{Transformed, TreeNode};
use datafusion_common::{plan_err, DataFusionError};
use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported};
use datafusion_physical_plan::joins::SymmetricHashJoinExec;
use datafusion_physical_plan::tree_node::PlanContext;
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/physical_optimizer/sort_pushdown.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ use crate::physical_plan::tree_node::PlanContext;
use crate::physical_plan::ExecutionPlan;

use datafusion_common::tree_node::Transformed;
use datafusion_common::{plan_err, DataFusionError, JoinSide, Result};
use datafusion_common::{plan_err, JoinSide, Result};
use datafusion_expr::JoinType;
use datafusion_physical_expr::expressions::Column;
use datafusion_physical_expr::{
Expand Down
3 changes: 3 additions & 0 deletions datafusion/core/tests/core_integration.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,9 @@ mod sql;
/// Run all tests that are found in the `dataframe` directory
mod dataframe;

/// Run all tests that are found in the `macro_hygiene` directory
mod macro_hygiene;

#[cfg(test)]
#[ctor::ctor]
fn init() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ use datafusion::physical_plan::{
use datafusion::prelude::*;
use datafusion::scalar::ScalarValue;
use datafusion_common::cast::as_primitive_array;
use datafusion_common::{internal_err, not_impl_err, DataFusionError};
use datafusion_common::{internal_err, not_impl_err};
use datafusion_expr::expr::{BinaryExpr, Cast};

use async_trait::async_trait;
Expand Down
37 changes: 37 additions & 0 deletions datafusion/core/tests/macro_hygiene/mod.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
// 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.

mod plan_err {
// NO other imports!
use datafusion_common::plan_err;

#[test]
fn test_macro() {
// need type annotation for Ok variant
let _res: Result<(), _> = plan_err!("foo");
}
}

mod plan_datafusion_err {
// NO other imports!
use datafusion_common::plan_datafusion_err;

#[test]
fn test_macro() {
plan_datafusion_err!("foo");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,7 @@ use datafusion::{
prelude::SessionContext,
scalar::ScalarValue,
};
use datafusion_common::{
assert_contains, cast::as_primitive_array, exec_err, DataFusionError,
};
use datafusion_common::{assert_contains, cast::as_primitive_array, exec_err};
use datafusion_expr::{
create_udaf, AggregateUDFImpl, GroupsAccumulator, SimpleAggregateUDF,
};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ use datafusion::{execution::registry::FunctionRegistry, test_util};
use datafusion_common::cast::as_float64_array;
use datafusion_common::{
assert_batches_eq, assert_batches_sorted_eq, cast::as_int32_array, not_impl_err,
plan_err, DataFusionError, ExprSchema, Result, ScalarValue,
plan_err, ExprSchema, Result, ScalarValue,
};
use datafusion_expr::{
create_udaf, create_udf, Accumulator, ColumnarValue, ExprSchemable,
Expand Down
2 changes: 1 addition & 1 deletion datafusion/execution/src/registry.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

//! FunctionRegistry trait
use datafusion_common::{not_impl_err, plan_datafusion_err, DataFusionError, Result};
use datafusion_common::{not_impl_err, plan_datafusion_err, Result};
use datafusion_expr::{AggregateUDF, ScalarUDF, UserDefinedLogicalNode, WindowUDF};
use std::collections::HashMap;
use std::{collections::HashSet, sync::Arc};
Expand Down
2 changes: 1 addition & 1 deletion datafusion/expr/src/accumulator.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
//! Accumulator module contains the trait definition for aggregation function's accumulators.
use arrow::array::ArrayRef;
use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue};
use datafusion_common::{internal_err, Result, ScalarValue};
use std::fmt::Debug;

/// Tracks an aggregate function's state.
Expand Down
2 changes: 1 addition & 1 deletion datafusion/expr/src/columnar_value.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
use arrow::array::ArrayRef;
use arrow::array::NullArray;
use arrow::datatypes::DataType;
use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue};
use datafusion_common::{internal_err, Result, ScalarValue};
use std::sync::Arc;

/// Represents the result of evaluating an expression: either a single
Expand Down
2 changes: 1 addition & 1 deletion datafusion/expr/src/conditional_expressions.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
use crate::expr::Case;
use crate::{expr_schema::ExprSchemable, Expr};
use arrow::datatypes::DataType;
use datafusion_common::{plan_err, DFSchema, DataFusionError, Result};
use datafusion_common::{plan_err, DFSchema, Result};
use std::collections::HashSet;

/// Currently supported types by the coalesce function.
Expand Down
2 changes: 1 addition & 1 deletion datafusion/expr/src/expr.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ use crate::{built_in_window_function, udaf};
use arrow::datatypes::DataType;
use datafusion_common::tree_node::{Transformed, TreeNode};
use datafusion_common::{internal_err, DFSchema, OwnedTableReference};
use datafusion_common::{plan_err, Column, DataFusionError, Result, ScalarValue};
use datafusion_common::{plan_err, Column, Result, ScalarValue};
use sqlparser::ast::NullTreatment;
use std::collections::HashSet;
use std::fmt;
Expand Down
Loading

0 comments on commit 2f3e097

Please sign in to comment.