Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 0 additions & 1 deletion datafusion/catalog/src/catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,6 @@ use datafusion_common::Result;
/// [`UnityCatalogProvider`]: https://github.com/delta-io/delta-rs/blob/951436ecec476ce65b5ed3b58b50fb0846ca7b91/crates/deltalake-core/src/data_catalog/unity/datafusion.rs#L111-L123
///
/// [`TableProvider`]: crate::TableProvider

pub trait CatalogProvider: Debug + Sync + Send {
/// Returns the catalog provider as [`Any`]
/// so that it can be downcast to a specific implementation.
Expand Down
14 changes: 8 additions & 6 deletions datafusion/common/src/column.rs
Original file line number Diff line number Diff line change
Expand Up @@ -109,21 +109,23 @@ impl Column {
/// where `"foo.BAR"` would be parsed to a reference to column named `foo.BAR`
pub fn from_qualified_name(flat_name: impl Into<String>) -> Self {
let flat_name = flat_name.into();
Self::from_idents(&mut parse_identifiers_normalized(&flat_name, false))
.unwrap_or_else(|| Self {
Self::from_idents(&mut parse_identifiers_normalized(&flat_name, false)).unwrap_or(
Self {
relation: None,
name: flat_name,
})
},
)
}

/// Deserialize a fully qualified name string into a column preserving column text case
pub fn from_qualified_name_ignore_case(flat_name: impl Into<String>) -> Self {
let flat_name = flat_name.into();
Self::from_idents(&mut parse_identifiers_normalized(&flat_name, true))
.unwrap_or_else(|| Self {
Self::from_idents(&mut parse_identifiers_normalized(&flat_name, true)).unwrap_or(
Self {
relation: None,
name: flat_name,
})
},
)
}

/// return the column's name.
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 @@ -63,7 +63,7 @@ pub trait HashValue {
fn hash_one(&self, state: &RandomState) -> u64;
}

impl<'a, T: HashValue + ?Sized> HashValue for &'a T {
impl<T: HashValue + ?Sized> HashValue for &T {
fn hash_one(&self, state: &RandomState) -> u64 {
T::hash_one(self, state)
}
Expand Down
6 changes: 1 addition & 5 deletions datafusion/common/src/utils/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -319,17 +319,13 @@ pub fn longest_consecutive_prefix<T: Borrow<usize>>(
count
}

/// Array Utils

/// Wrap an array into a single element `ListArray`.
/// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]`
/// The field in the list array is nullable.
pub fn array_into_list_array_nullable(arr: ArrayRef) -> ListArray {
array_into_list_array(arr, true)
}

/// Array Utils

/// Wrap an array into a single element `ListArray`.
/// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]`
pub fn array_into_list_array(arr: ArrayRef, nullable: bool) -> ListArray {
Expand Down Expand Up @@ -569,7 +565,7 @@ pub mod datafusion_strsim {

struct StringWrapper<'a>(&'a str);

impl<'a, 'b> IntoIterator for &'a StringWrapper<'b> {
impl<'b> IntoIterator for &StringWrapper<'b> {
type Item = char;
type IntoIter = Chars<'b>;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ pub struct AvroArrowArrayReader<'a, R: Read> {
schema_lookup: BTreeMap<String, usize>,
}

impl<'a, R: Read> AvroArrowArrayReader<'a, R> {
impl<R: Read> AvroArrowArrayReader<'_, R> {
pub fn try_new(
reader: R,
schema: SchemaRef,
Expand Down
4 changes: 2 additions & 2 deletions datafusion/core/src/datasource/avro_to_arrow/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ pub struct Reader<'a, R: Read> {
batch_size: usize,
}

impl<'a, R: Read> Reader<'a, R> {
impl<R: Read> Reader<'_, R> {
/// Create a new Avro Reader from any value that implements the `Read` trait.
///
/// If reading a `File`, you can customise the Reader, such as to enable schema
Expand Down Expand Up @@ -157,7 +157,7 @@ impl<'a, R: Read> Reader<'a, R> {
}
}

impl<'a, R: Read> Iterator for Reader<'a, R> {
impl<R: Read> Iterator for Reader<'_, R> {
type Item = ArrowResult<RecordBatch>;

/// Returns the next batch of results (defined by `self.batch_size`), or `None` if there
Expand Down
10 changes: 5 additions & 5 deletions datafusion/core/src/datasource/file_format/options.rs
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ pub struct CsvReadOptions<'a> {
pub file_sort_order: Vec<Vec<SortExpr>>,
}

impl<'a> Default for CsvReadOptions<'a> {
impl Default for CsvReadOptions<'_> {
fn default() -> Self {
Self::new()
}
Expand Down Expand Up @@ -243,7 +243,7 @@ pub struct ParquetReadOptions<'a> {
pub file_sort_order: Vec<Vec<SortExpr>>,
}

impl<'a> Default for ParquetReadOptions<'a> {
impl Default for ParquetReadOptions<'_> {
fn default() -> Self {
Self {
file_extension: DEFAULT_PARQUET_EXTENSION,
Expand Down Expand Up @@ -323,7 +323,7 @@ pub struct ArrowReadOptions<'a> {
pub table_partition_cols: Vec<(String, DataType)>,
}

impl<'a> Default for ArrowReadOptions<'a> {
impl Default for ArrowReadOptions<'_> {
fn default() -> Self {
Self {
schema: None,
Expand Down Expand Up @@ -368,7 +368,7 @@ pub struct AvroReadOptions<'a> {
pub table_partition_cols: Vec<(String, DataType)>,
}

impl<'a> Default for AvroReadOptions<'a> {
impl Default for AvroReadOptions<'_> {
fn default() -> Self {
Self {
schema: None,
Expand Down Expand Up @@ -420,7 +420,7 @@ pub struct NdJsonReadOptions<'a> {
pub file_sort_order: Vec<Vec<SortExpr>>,
}

impl<'a> Default for NdJsonReadOptions<'a> {
impl Default for NdJsonReadOptions<'_> {
fn default() -> Self {
Self {
schema: None,
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/file_format/parquet.rs
Original file line number Diff line number Diff line change
Expand Up @@ -477,7 +477,7 @@ impl<'a> ObjectStoreFetch<'a> {
}
}

impl<'a> MetadataFetch for ObjectStoreFetch<'a> {
impl MetadataFetch for ObjectStoreFetch<'_> {
fn fetch(
&mut self,
range: Range<usize>,
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/datasource/listing/helpers.rs
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ pub fn split_files(
partitioned_files.sort_by(|a, b| a.path().cmp(b.path()));

// effectively this is div with rounding up instead of truncating
let chunk_size = (partitioned_files.len() + n - 1) / n;
let chunk_size = partitioned_files.len().div_ceil(n);
let mut chunks = Vec::with_capacity(n);
let mut current_chunk = Vec::with_capacity(chunk_size);
for file in partitioned_files.drain(..) {
Expand Down
3 changes: 1 addition & 2 deletions datafusion/core/src/datasource/physical_plan/file_groups.rs
Original file line number Diff line number Diff line change
Expand Up @@ -217,8 +217,7 @@ impl FileGroupPartitioner {
return None;
}

let target_partition_size =
(total_size as usize + (target_partitions) - 1) / (target_partitions);
let target_partition_size = (total_size as usize).div_ceil(target_partitions);

let current_partition_index: usize = 0;
let current_partition_size: usize = 0;
Expand Down
4 changes: 2 additions & 2 deletions datafusion/core/src/datasource/physical_plan/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -139,7 +139,7 @@ impl DisplayAs for FileScanConfig {
#[derive(Debug)]
struct FileGroupsDisplay<'a>(&'a [Vec<PartitionedFile>]);

impl<'a> DisplayAs for FileGroupsDisplay<'a> {
impl DisplayAs for FileGroupsDisplay<'_> {
fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult {
let n_groups = self.0.len();
let groups = if n_groups == 1 { "group" } else { "groups" };
Expand Down Expand Up @@ -171,7 +171,7 @@ impl<'a> DisplayAs for FileGroupsDisplay<'a> {
#[derive(Debug)]
pub(crate) struct FileGroupDisplay<'a>(pub &'a [PartitionedFile]);

impl<'a> DisplayAs for FileGroupDisplay<'a> {
impl DisplayAs for FileGroupDisplay<'_> {
fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult {
write!(f, "[")?;
match t {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -449,7 +449,7 @@ impl<'a> PagesPruningStatistics<'a> {
Some(vec)
}
}
impl<'a> PruningStatistics for PagesPruningStatistics<'a> {
impl PruningStatistics for PagesPruningStatistics<'_> {
fn min_values(&self, _column: &datafusion_common::Column) -> Option<ArrayRef> {
match self.converter.data_page_mins(
self.column_index,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ impl<'schema> PushdownChecker<'schema> {
}
}

impl<'schema> TreeNodeRewriter for PushdownChecker<'schema> {
impl TreeNodeRewriter for PushdownChecker<'_> {
type Node = Arc<dyn PhysicalExpr>;

fn f_down(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -374,7 +374,7 @@ impl<'a> RowGroupPruningStatistics<'a> {
}
}

impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> {
impl PruningStatistics for RowGroupPruningStatistics<'_> {
fn min_values(&self, column: &Column) -> Option<ArrayRef> {
self.statistics_converter(column)
.and_then(|c| Ok(c.row_group_mins(self.metadata_iter())?))
Expand Down
2 changes: 1 addition & 1 deletion datafusion/core/src/execution/context/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1764,7 +1764,7 @@ impl<'a> BadPlanVisitor<'a> {
}
}

impl<'n, 'a> TreeNodeVisitor<'n> for BadPlanVisitor<'a> {
impl<'n> TreeNodeVisitor<'n> for BadPlanVisitor<'_> {
type Node = LogicalPlan;

fn f_down(&mut self, node: &'n Self::Node) -> Result<TreeNodeRecursion> {
Expand Down
4 changes: 2 additions & 2 deletions datafusion/core/src/execution/session_state.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1636,7 +1636,7 @@ struct SessionContextProvider<'a> {
tables: HashMap<ResolvedTableReference, Arc<dyn TableSource>>,
}

impl<'a> ContextProvider for SessionContextProvider<'a> {
impl ContextProvider for SessionContextProvider<'_> {
fn get_expr_planners(&self) -> &[Arc<dyn ExprPlanner>] {
&self.state.expr_planners
}
Expand Down Expand Up @@ -1931,7 +1931,7 @@ impl<'a> SessionSimplifyProvider<'a> {
}
}

impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> {
impl SimplifyInfo for SessionSimplifyProvider<'_> {
fn is_boolean_type(&self, expr: &Expr) -> datafusion_common::Result<bool> {
Ok(expr.get_type(self.df_schema)? == DataType::Boolean)
}
Expand Down
2 changes: 2 additions & 0 deletions datafusion/core/src/physical_optimizer/test_utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

//! Collection of testing utility functions that are leveraged by the query optimizer rules

#![allow(missing_docs)]

use std::any::Any;
use std::fmt::Formatter;
use std::sync::Arc;
Expand Down
2 changes: 2 additions & 0 deletions datafusion/core/src/test/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

//! Common unit test utility methods

#![allow(missing_docs)]

use std::any::Any;
use std::fs::File;
use std::io::prelude::*;
Expand Down
1 change: 0 additions & 1 deletion datafusion/execution/src/cache/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ pub mod cache_unit;
/// This interface does not get `mut` references and thus has to handle its own
/// locking via internal mutability. It can be accessed via multiple concurrent queries
/// during planning and execution.

pub trait CacheAccessor<K, V>: Send + Sync {
// Extra info but not part of the cache key or cache value.
type Extra: Clone;
Expand Down
8 changes: 4 additions & 4 deletions datafusion/expr-common/src/type_coercion/aggregates.rs
Original file line number Diff line number Diff line change
Expand Up @@ -294,19 +294,19 @@ pub fn coerce_avg_type(func_name: &str, arg_types: &[DataType]) -> Result<Vec<Da
// Supported types smallint, int, bigint, real, double precision, decimal, or interval
// Refer to https://www.postgresql.org/docs/8.2/functions-aggregate.html doc
fn coerced_type(func_name: &str, data_type: &DataType) -> Result<DataType> {
return match &data_type {
match &data_type {
DataType::Decimal128(p, s) => Ok(DataType::Decimal128(*p, *s)),
DataType::Decimal256(p, s) => Ok(DataType::Decimal256(*p, *s)),
d if d.is_numeric() => Ok(DataType::Float64),
DataType::Dictionary(_, v) => return coerced_type(func_name, v.as_ref()),
DataType::Dictionary(_, v) => coerced_type(func_name, v.as_ref()),
_ => {
return plan_err!(
plan_err!(
"The function {:?} does not support inputs of type {:?}.",
func_name,
data_type
)
}
};
}
}
Ok(vec![coerced_type(func_name, &arg_types[0])?])
}
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 @@ -1852,7 +1852,7 @@ macro_rules! expr_vec_fmt {
}

struct SchemaDisplay<'a>(&'a Expr);
impl<'a> Display for SchemaDisplay<'a> {
impl Display for SchemaDisplay<'_> {
fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
match self.0 {
// The same as Display
Expand Down
2 changes: 1 addition & 1 deletion datafusion/expr/src/logical_plan/ddl.rs
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ impl DdlStatement {
/// See [crate::LogicalPlan::display] for an example
pub fn display(&self) -> impl Display + '_ {
struct Wrapper<'a>(&'a DdlStatement);
impl<'a> Display for Wrapper<'a> {
impl Display for Wrapper<'_> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self.0 {
DdlStatement::CreateExternalTable(CreateExternalTable {
Expand Down
8 changes: 4 additions & 4 deletions datafusion/expr/src/logical_plan/display.rs
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ impl<'a, 'b> IndentVisitor<'a, 'b> {
}
}

impl<'n, 'a, 'b> TreeNodeVisitor<'n> for IndentVisitor<'a, 'b> {
impl<'n> TreeNodeVisitor<'n> for IndentVisitor<'_, '_> {
type Node = LogicalPlan;

fn f_down(
Expand Down Expand Up @@ -113,7 +113,7 @@ impl<'n, 'a, 'b> TreeNodeVisitor<'n> for IndentVisitor<'a, 'b> {
pub fn display_schema(schema: &Schema) -> impl fmt::Display + '_ {
struct Wrapper<'a>(&'a Schema);

impl<'a> fmt::Display for Wrapper<'a> {
impl fmt::Display for Wrapper<'_> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
write!(f, "[")?;
for (idx, field) in self.0.fields().iter().enumerate() {
Expand Down Expand Up @@ -181,7 +181,7 @@ impl<'a, 'b> GraphvizVisitor<'a, 'b> {
}
}

impl<'n, 'a, 'b> TreeNodeVisitor<'n> for GraphvizVisitor<'a, 'b> {
impl<'n> TreeNodeVisitor<'n> for GraphvizVisitor<'_, '_> {
type Node = LogicalPlan;

fn f_down(
Expand Down Expand Up @@ -654,7 +654,7 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> {
}
}

impl<'n, 'a, 'b> TreeNodeVisitor<'n> for PgJsonVisitor<'a, 'b> {
impl<'n> TreeNodeVisitor<'n> for PgJsonVisitor<'_, '_> {
type Node = LogicalPlan;

fn f_down(
Expand Down
10 changes: 5 additions & 5 deletions datafusion/expr/src/logical_plan/plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1536,7 +1536,7 @@ impl LogicalPlan {
// Boilerplate structure to wrap LogicalPlan with something
// that that can be formatted
struct Wrapper<'a>(&'a LogicalPlan);
impl<'a> Display for Wrapper<'a> {
impl Display for Wrapper<'_> {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
let with_schema = false;
let mut visitor = IndentVisitor::new(f, with_schema);
Expand Down Expand Up @@ -1579,7 +1579,7 @@ impl LogicalPlan {
// Boilerplate structure to wrap LogicalPlan with something
// that that can be formatted
struct Wrapper<'a>(&'a LogicalPlan);
impl<'a> Display for Wrapper<'a> {
impl Display for Wrapper<'_> {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
let with_schema = true;
let mut visitor = IndentVisitor::new(f, with_schema);
Expand All @@ -1599,7 +1599,7 @@ impl LogicalPlan {
// Boilerplate structure to wrap LogicalPlan with something
// that that can be formatted
struct Wrapper<'a>(&'a LogicalPlan);
impl<'a> Display for Wrapper<'a> {
impl Display for Wrapper<'_> {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
let mut visitor = PgJsonVisitor::new(f);
visitor.with_schema(true);
Expand Down Expand Up @@ -1645,7 +1645,7 @@ impl LogicalPlan {
// Boilerplate structure to wrap LogicalPlan with something
// that that can be formatted
struct Wrapper<'a>(&'a LogicalPlan);
impl<'a> Display for Wrapper<'a> {
impl Display for Wrapper<'_> {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
let mut visitor = GraphvizVisitor::new(f);

Expand Down Expand Up @@ -1696,7 +1696,7 @@ impl LogicalPlan {
// Boilerplate structure to wrap LogicalPlan with something
// that that can be formatted
struct Wrapper<'a>(&'a LogicalPlan);
impl<'a> Display for Wrapper<'a> {
impl Display for Wrapper<'_> {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
match self.0 {
LogicalPlan::EmptyRelation(_) => write!(f, "EmptyRelation"),
Expand Down
Loading